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 2017/11/03 17:11:24 UTC

[01/21] flink git commit: [FLINK-7420] [avro] Abstract all Avro interaction behind AvroUtils

Repository: flink
Updated Branches:
  refs/heads/master 16b088218 -> 87bf57816


[FLINK-7420] [avro] Abstract all Avro interaction behind AvroUtils

Before, we would try and dynamicall load Avro-related classes in several
places. Now, we only reflectively instantiate the right AvroUtils and
all other operations are methods on this.

The default AvroUtils throw exceptions with a helpful message for most
operations.


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

Branch: refs/heads/master
Commit: 65e87045c48ce3200ea6690d945ed87b061808af
Parents: 29249b2
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Mon Oct 30 15:02:18 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Nov 3 16:40:34 2017 +0100

----------------------------------------------------------------------
 .../flink/api/java/typeutils/AvroUtils.java     | 120 +++++++++++++++++++
 .../flink/api/java/typeutils/PojoTypeInfo.java  |  23 +---
 .../flink/api/java/typeutils/TypeExtractor.java |  33 +----
 .../typeutils/runtime/kryo/KryoSerializer.java  |   3 +-
 .../typeutils/runtime/kryo/Serializers.java     |  63 ++--------
 .../avro/utils/AvroKryoSerializerUtils.java     |  54 +++++++--
 6 files changed, 182 insertions(+), 114 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/65e87045/flink-core/src/main/java/org/apache/flink/api/java/typeutils/AvroUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/AvroUtils.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/AvroUtils.java
new file mode 100644
index 0000000..e19f5fb
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/AvroUtils.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.typeutils;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.typeutils.runtime.KryoRegistration;
+import org.apache.flink.api.java.typeutils.runtime.kryo.Serializers;
+
+import java.util.LinkedHashMap;
+
+import static org.apache.flink.api.java.typeutils.TypeExtractionUtils.hasSuperclass;
+
+/**
+ * Utility methods for dealing with Avro types. This has a default implementation for the case that
+ * Avro is not present on the classpath and an actual implementation in flink-avro that is
+ * dynamically loaded when present.
+ */
+public abstract class AvroUtils {
+
+	protected static final String AVRO_SPECIFIC_RECORD_BASE = "org.apache.avro.specific.SpecificRecordBase";
+
+	protected static final String AVRO_GENERIC_RECORD = "org.apache.avro.generic.GenericData$Record";
+
+	private static final String AVRO_KRYO_UTILS = "org.apache.flink.formats.avro.utils.AvroKryoSerializerUtils";
+
+	private static final String AVRO_GENERIC_DATA_ARRAY = "org.apache.avro.generic.GenericData$Array";
+
+	/**
+	 * Loads the utility class from <code>flink-avro</code> and adds Avro-specific serializers. This
+	 * method will throw an exception if we see an Avro type but flink-avro is not in the classpath.
+	 */
+	public abstract void addAvroSerializersIfRequired(ExecutionConfig reg, Class<?> type);
+
+	/**
+	 * Registers a special Serializer for GenericData.Array.
+	 */
+	public abstract void addAvroGenericDataArrayRegistration(
+			LinkedHashMap<String,
+			KryoRegistration> kryoRegistrations);
+
+	/**
+	 * Creates an {@code AvroSerializer} if flink-avro is present, otherwise throws an exception.
+	 */
+	public abstract <T> TypeSerializer<T> createAvroSerializer(Class<T> type);
+
+	/**
+	 * Creates an {@code AvroTypeInfo} if flink-avro is present, otherwise throws an exception.
+	 */
+	public abstract <T> TypeInformation<T> createAvroTypeInfo(Class<T> type);
+
+	/**
+	 * Returns either the default {@link AvroUtils} which throw an exception in cases where Avro
+	 * would be needed or loads the specific utils for Avro from flink-avro.
+	 */
+	public static AvroUtils getAvroUtils() {
+
+		// try and load the special AvroUtils from the flink-avro package
+		Class<?> clazz;
+		try {
+			clazz = Class.forName(AVRO_KRYO_UTILS, false, AvroUtils.class.getClassLoader());
+		} catch (ClassNotFoundException e) {
+			// cannot find the utils, return the default implementation
+			return new DefaultAvroUtils();
+		}
+
+		try {
+			return (AvroUtils) clazz.getConstructor().newInstance();
+		} catch (Exception e) {
+			throw new RuntimeException("Could not instantiate " + AVRO_KRYO_UTILS + ".");
+		}
+	}
+
+	private static class DefaultAvroUtils extends AvroUtils {
+		@Override
+		public void addAvroSerializersIfRequired(ExecutionConfig reg, Class<?> type) {
+			if (hasSuperclass(type, AVRO_SPECIFIC_RECORD_BASE) || hasSuperclass(
+				type,
+				AVRO_GENERIC_RECORD)) {
+				throw new RuntimeException("Could not load class '" + AVRO_KRYO_UTILS + "'. " +
+					"You may be missing the 'flink-avro' dependency.");
+			}
+		}
+
+		@Override
+		public void addAvroGenericDataArrayRegistration(LinkedHashMap<String, KryoRegistration> kryoRegistrations) {
+			kryoRegistrations.put(AVRO_GENERIC_DATA_ARRAY,
+				new KryoRegistration(Serializers.DummyAvroRegisteredClass.class, (Class) Serializers.DummyAvroKryoSerializerClass.class));
+		}
+
+		@Override
+		public <T> TypeSerializer<T> createAvroSerializer(Class<T> type) {
+			throw new RuntimeException("Could not load the AvroSerializer class. " +
+				"You may be missing the 'flink-avro' dependency.");
+		}
+
+		@Override
+		public <T> TypeInformation<T> createAvroTypeInfo(Class<T> type) {
+			throw new RuntimeException("Could not load the AvroTypeInfo class. " +
+				"You may be missing the 'flink-avro' dependency.");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/65e87045/flink-core/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java
index b24f425..2e893bb 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java
@@ -31,9 +31,7 @@ import org.apache.flink.api.java.typeutils.runtime.PojoComparator;
 import org.apache.flink.api.java.typeutils.runtime.PojoSerializer;
 import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer;
 
-import java.lang.reflect.Constructor;
 import java.lang.reflect.Field;
-import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Modifier;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -303,27 +301,12 @@ public class PojoTypeInfo<T> extends CompositeType<T> {
 	@PublicEvolving
 	@SuppressWarnings("unchecked")
 	public TypeSerializer<T> createSerializer(ExecutionConfig config) {
-		if(config.isForceKryoEnabled()) {
+		if (config.isForceKryoEnabled()) {
 			return new KryoSerializer<>(getTypeClass(), config);
 		}
 
-		if(config.isForceAvroEnabled()) {
-			Class<?> clazz;
-			try {
-				clazz = Class.forName("org.apache.flink.formats.avro.typeutils.AvroSerializer");
-			} catch (ClassNotFoundException e) {
-				throw new RuntimeException("Could not load the AvroSerializer class. " +
-					"You may be missing the 'flink-avro' dependency.");
-			}
-
-			try {
-				Constructor<?> constructor = clazz.getConstructor(Class.class);
-				return (TypeSerializer<T>) constructor.newInstance(getTypeClass());
-			} catch (NoSuchMethodException | IllegalAccessException | InstantiationException e) {
-				throw new RuntimeException("Incompatible versions of the Avro classes found.");
-			} catch (InvocationTargetException e) {
-				throw new RuntimeException("Cannot create AvroSerializer.", e.getTargetException());
-			}
+		if (config.isForceAvroEnabled()) {
+			return AvroUtils.getAvroUtils().createAvroSerializer(getTypeClass());
 		}
 
 		TypeSerializer<?>[] fieldSerializers = new TypeSerializer<?>[fields.length];

http://git-wip-us.apache.org/repos/asf/flink/blob/65e87045/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
index 1a9cecb..8ea2e1a 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
@@ -117,8 +117,6 @@ public class TypeExtractor {
 
 	private static final String AVRO_SPECIFIC_RECORD_BASE_CLASS = "org.apache.avro.specific.SpecificRecordBase";
 
-	private static final String AVRO_TYPEINFO_CLASS = "org.apache.flink.formats.avro.typeutils.AvroTypeInfo";
-
 	private static final Logger LOG = LoggerFactory.getLogger(TypeExtractor.class);
 
 	public static final int[] NO_INDEX = new int[] {};
@@ -1794,7 +1792,7 @@ public class TypeExtractor {
 
 		// special case for POJOs generated by Avro.
 		if (hasSuperclass(clazz, AVRO_SPECIFIC_RECORD_BASE_CLASS)) {
-			return createAvroTypeInfo(clazz);
+			return AvroUtils.getAvroUtils().createAvroTypeInfo(clazz);
 		}
 
 		if (Modifier.isInterface(clazz.getModifiers())) {
@@ -2175,33 +2173,4 @@ public class TypeExtractor {
 			// ignore
 		}
 	}
-
-	// ------------------------------------------------------------------------
-	//  Utilities to handle Avro's 'SpecificRecord' type via reflection
-	// ------------------------------------------------------------------------
-
-	private static <T> TypeInformation<T> createAvroTypeInfo(Class<T> clazz) {
-		Class<?> typeInfoClass;
-		try {
-			typeInfoClass = Class.forName(AVRO_TYPEINFO_CLASS, false, TypeExtractor.class.getClassLoader());
-		}
-		catch (ClassNotFoundException e) {
-			throw new RuntimeException("Could not load the TypeInformation for the class '"
-					+ AVRO_TYPEINFO_CLASS + "'. You may be missing the 'flink-avro' dependency.");
-		}
-
-		try {
-			Constructor<?> constr = typeInfoClass.getConstructor(Class.class);
-
-			@SuppressWarnings("unchecked")
-			TypeInformation<T> typeInfo = (TypeInformation<T>) constr.newInstance(clazz);
-			return typeInfo;
-		}
-		catch (NoSuchMethodException | IllegalAccessException | InstantiationException e) {
-			throw new RuntimeException("Incompatible versions of the Avro classes found.");
-		}
-		catch (InvocationTargetException e) {
-			throw new RuntimeException("Cannot create AvroTypeInfo.", e.getTargetException());
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/65e87045/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
index 269cf35..560e5b1 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
@@ -29,6 +29,7 @@ import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeutils.CompatibilityResult;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.java.typeutils.AvroUtils;
 import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream;
 import org.apache.flink.api.java.typeutils.runtime.DataOutputViewStream;
 import org.apache.flink.api.java.typeutils.runtime.KryoRegistration;
@@ -477,7 +478,7 @@ public class KryoSerializer<T> extends TypeSerializer<T> {
 		}
 
 		// add Avro support if flink-avro is available; a dummy otherwise
-		Serializers.addAvroGenericDataArrayRegistration(kryoRegistrations);
+		AvroUtils.getAvroUtils().addAvroGenericDataArrayRegistration(kryoRegistrations);
 
 		return kryoRegistrations;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/65e87045/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/Serializers.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/Serializers.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/Serializers.java
index de7b2fc..9659dc6 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/Serializers.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/Serializers.java
@@ -22,10 +22,10 @@ import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.CompositeType;
+import org.apache.flink.api.java.typeutils.AvroUtils;
 import org.apache.flink.api.java.typeutils.GenericTypeInfo;
 import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo;
 import org.apache.flink.api.java.typeutils.TypeExtractionUtils;
-import org.apache.flink.api.java.typeutils.runtime.KryoRegistration;
 
 import com.esotericsoftware.kryo.Kryo;
 import com.esotericsoftware.kryo.Serializer;
@@ -36,19 +36,14 @@ import com.esotericsoftware.kryo.serializers.CollectionSerializer;
 import java.io.Serializable;
 import java.lang.reflect.Field;
 import java.lang.reflect.GenericArrayType;
-import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Modifier;
 import java.lang.reflect.ParameterizedType;
 import java.lang.reflect.Type;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Set;
 
-import static org.apache.flink.api.java.typeutils.TypeExtractionUtils.hasSuperclass;
-
-
 /**
  * Class containing utilities for the serializers of the Flink Runtime.
  *
@@ -61,14 +56,6 @@ import static org.apache.flink.api.java.typeutils.TypeExtractionUtils.hasSupercl
 @Internal
 public class Serializers {
 
-	private static final String AVRO_SPECIFIC_RECORD_BASE = "org.apache.avro.specific.SpecificRecordBase";
-
-	private static final String AVRO_GENERIC_RECORD = "org.apache.avro.generic.GenericData$Record";
-
-	private static final String AVRO_KRYO_UTILS = "org.apache.flink.formats.avro.utils.AvroKryoSerializerUtils";
-
-	private static final String AVRO_GENERIC_DATA_ARRAY = "org.apache.avro.generic.GenericData$Array";
-
 	public static void recursivelyRegisterType(TypeInformation<?> typeInfo, ExecutionConfig config, Set<Class<?>> alreadySeen) {
 		if (typeInfo instanceof GenericTypeInfo) {
 			GenericTypeInfo<?> genericTypeInfo = (GenericTypeInfo<?>) typeInfo;
@@ -104,9 +91,7 @@ public class Serializers {
 		else {
 			config.registerKryoType(type);
 			// add serializers for Avro type if necessary
-			if (hasSuperclass(type, AVRO_SPECIFIC_RECORD_BASE) || hasSuperclass(type, AVRO_GENERIC_RECORD)) {
-				addAvroSerializers(config, type);
-			}
+			AvroUtils.getAvroUtils().addAvroSerializersIfRequired(config, type);
 
 			Field[] fields = type.getDeclaredFields();
 			for (Field field : fields) {
@@ -161,43 +146,19 @@ public class Serializers {
 	}
 
 	/**
-	 * Loads the utility class from <code>flink-avro</code> and adds Avro-specific serializers.
+	 * This is used in case we don't have Avro on the classpath. Flink versions before 1.4
+	 * always registered special Serializers for Kryo but starting with Flink 1.4 we don't have
+	 * Avro on the classpath by default anymore. We still have to retain the same registered
+	 * Serializers for backwards compatibility of savepoints.
 	 */
-	private static void addAvroSerializers(ExecutionConfig reg, Class<?> type) {
-		Class<?> clazz;
-		try {
-			clazz = Class.forName(AVRO_KRYO_UTILS, false, Serializers.class.getClassLoader());
-		}
-		catch (ClassNotFoundException e) {
-			throw new RuntimeException("Could not load class '" + AVRO_KRYO_UTILS + "'. " +
-				"You may be missing the 'flink-avro' dependency.");
-		}
-		try {
-			clazz.getDeclaredMethod("addAvroSerializers", ExecutionConfig.class, Class.class).invoke(null, reg, type);
-		} catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) {
-			throw new RuntimeException("Could not access method in 'flink-avro' dependency.", e);
-		}
-	}
-
-	@SuppressWarnings("unchecked")
-	public static void addAvroGenericDataArrayRegistration(LinkedHashMap<String, KryoRegistration> kryoRegistrations) {
-		try {
-			Class<?> clazz = Class.forName(AVRO_GENERIC_DATA_ARRAY, false, Serializers.class.getClassLoader());
-
-			kryoRegistrations.put(
-				AVRO_GENERIC_DATA_ARRAY,
-				new KryoRegistration(
-						clazz,
-						new ExecutionConfig.SerializableSerializer<>(new Serializers.SpecificInstanceCollectionSerializerForArrayList())));
-		}
-		catch (ClassNotFoundException e) {
-			kryoRegistrations.put(AVRO_GENERIC_DATA_ARRAY,
-				new KryoRegistration(DummyAvroRegisteredClass.class, (Class) DummyAvroKryoSerializerClass.class));
-		}
-	}
-
 	public static class DummyAvroRegisteredClass {}
 
+	/**
+	 * This is used in case we don't have Avro on the classpath. Flink versions before 1.4
+	 * always registered special Serializers for Kryo but starting with Flink 1.4 we don't have
+	 * Avro on the classpath by default anymore. We still have to retain the same registered
+	 * Serializers for backwards compatibility of savepoints.
+	 */
 	public static class DummyAvroKryoSerializerClass<T> extends Serializer<T> {
 		@Override
 		public void write(Kryo kryo, Output output, Object o) {

http://git-wip-us.apache.org/repos/asf/flink/blob/65e87045/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/AvroKryoSerializerUtils.java
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/AvroKryoSerializerUtils.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/AvroKryoSerializerUtils.java
index 7305f23..c28f6cf 100644
--- a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/AvroKryoSerializerUtils.java
+++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/AvroKryoSerializerUtils.java
@@ -19,7 +19,13 @@
 package org.apache.flink.formats.avro.utils;
 
 import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.typeutils.AvroUtils;
+import org.apache.flink.api.java.typeutils.runtime.KryoRegistration;
 import org.apache.flink.api.java.typeutils.runtime.kryo.Serializers;
+import org.apache.flink.formats.avro.typeutils.AvroSerializer;
+import org.apache.flink.formats.avro.typeutils.AvroTypeInfo;
 
 import com.esotericsoftware.kryo.Kryo;
 import com.esotericsoftware.kryo.Serializer;
@@ -29,22 +35,50 @@ import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericData;
 
 import java.io.Serializable;
+import java.util.LinkedHashMap;
+
+import static org.apache.flink.api.java.typeutils.TypeExtractionUtils.hasSuperclass;
 
 /**
  * Utilities for integrating Avro serializers in Kryo.
  */
-public class AvroKryoSerializerUtils {
+public class AvroKryoSerializerUtils extends AvroUtils {
+
+	@Override
+	public void addAvroSerializersIfRequired(ExecutionConfig reg, Class<?> type) {
+		if (hasSuperclass(type, AVRO_SPECIFIC_RECORD_BASE) || hasSuperclass(type, AVRO_GENERIC_RECORD)) {
+			// Avro POJOs contain java.util.List which have GenericData.Array as their runtime type
+			// because Kryo is not able to serialize them properly, we use this serializer for them
+			reg.registerTypeWithKryoSerializer(GenericData.Array.class, Serializers.SpecificInstanceCollectionSerializerForArrayList.class);
+
+			// We register this serializer for users who want to use untyped Avro records (GenericData.Record).
+			// Kryo is able to serialize everything in there, except for the Schema.
+			// This serializer is very slow, but using the GenericData.Records of Kryo is in general a bad idea.
+			// we add the serializer as a default serializer because Avro is using a private sub-type at runtime.
+			reg.addDefaultKryoSerializer(Schema.class, AvroSchemaSerializer.class);
+		}
+	}
+
+	@Override
+	public void addAvroGenericDataArrayRegistration(LinkedHashMap<String, KryoRegistration> kryoRegistrations) {
+		kryoRegistrations.put(
+			GenericData.Array.class.getName(),
+			new KryoRegistration(
+				GenericData.Array.class,
+				new ExecutionConfig.SerializableSerializer<>(new Serializers.SpecificInstanceCollectionSerializerForArrayList())));
+	}
 
-	public static void addAvroSerializers(ExecutionConfig reg, Class<?> type) {
-		// Avro POJOs contain java.util.List which have GenericData.Array as their runtime type
-		// because Kryo is not able to serialize them properly, we use this serializer for them
-		reg.registerTypeWithKryoSerializer(GenericData.Array.class, Serializers.SpecificInstanceCollectionSerializerForArrayList.class);
+	@Override
+	public <T> TypeSerializer<T> createAvroSerializer(Class<T> type) {
+		return new AvroSerializer<>(type);
+	}
 
-		// We register this serializer for users who want to use untyped Avro records (GenericData.Record).
-		// Kryo is able to serialize everything in there, except for the Schema.
-		// This serializer is very slow, but using the GenericData.Records of Kryo is in general a bad idea.
-		// we add the serializer as a default serializer because Avro is using a private sub-type at runtime.
-		reg.addDefaultKryoSerializer(Schema.class, AvroSchemaSerializer.class);
+	@Override
+	@SuppressWarnings({"rawtypes", "unchecked"})
+	public <T> TypeInformation<T> createAvroTypeInfo(Class<T> type) {
+		// we have to be raw here because we cannot have "<T extends SpecificRecordBase>" in
+		// the interface of AvroUtils
+		return new AvroTypeInfo(type);
 	}
 
 	/**


[18/21] flink git commit: [hotfix] [avro] Improve AvroUtils to perform reflection lookups only once.

Posted by se...@apache.org.
[hotfix] [avro] Improve AvroUtils to perform reflection lookups only once.

This also fixes minor warnings (unchecked casts) and moves the constants
into scopes that avoid bridge methods for access outside of the nested classes.


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

Branch: refs/heads/master
Commit: c438e293adc541c56cbaec222aa2942969e49c71
Parents: 6dd5e2b
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Nov 2 21:12:08 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Nov 3 16:40:35 2017 +0100

----------------------------------------------------------------------
 .../flink/api/java/typeutils/AvroUtils.java     | 60 ++++++++++++--------
 1 file changed, 35 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c438e293/flink-core/src/main/java/org/apache/flink/api/java/typeutils/AvroUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/AvroUtils.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/AvroUtils.java
index e19f5fb..2983be0 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/AvroUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/AvroUtils.java
@@ -35,13 +35,32 @@ import static org.apache.flink.api.java.typeutils.TypeExtractionUtils.hasSupercl
  */
 public abstract class AvroUtils {
 
-	protected static final String AVRO_SPECIFIC_RECORD_BASE = "org.apache.avro.specific.SpecificRecordBase";
+	private static final String AVRO_KRYO_UTILS = "org.apache.flink.formats.avro.utils.AvroKryoSerializerUtils";
 
-	protected static final String AVRO_GENERIC_RECORD = "org.apache.avro.generic.GenericData$Record";
+	private static final AvroUtils INSTANCE = loadAvroKryoUtils();
 
-	private static final String AVRO_KRYO_UTILS = "org.apache.flink.formats.avro.utils.AvroKryoSerializerUtils";
+	private static AvroUtils loadAvroKryoUtils() {
+		// try and load the special AvroUtils from the flink-avro package
+		try {
+			Class<?> clazz = Class.forName(AVRO_KRYO_UTILS, false, AvroUtils.class.getClassLoader());
+			return clazz.asSubclass(AvroUtils.class).getConstructor().newInstance();
+		} catch (ClassNotFoundException e) {
+			// cannot find the utils, return the default implementation
+			return new DefaultAvroUtils();
+		} catch (Exception e) {
+			throw new RuntimeException("Could not instantiate " + AVRO_KRYO_UTILS + ".", e);
+		}
+	}
 
-	private static final String AVRO_GENERIC_DATA_ARRAY = "org.apache.avro.generic.GenericData$Array";
+	/**
+	 * Returns either the default {@link AvroUtils} which throw an exception in cases where Avro
+	 * would be needed or loads the specific utils for Avro from flink-avro.
+	 */
+	public static AvroUtils getAvroUtils() {
+		return INSTANCE;
+	}
+
+	// ------------------------------------------------------------------------
 
 	/**
 	 * Loads the utility class from <code>flink-avro</code> and adds Avro-specific serializers. This
@@ -66,39 +85,30 @@ public abstract class AvroUtils {
 	 */
 	public abstract <T> TypeInformation<T> createAvroTypeInfo(Class<T> type);
 
+	// ------------------------------------------------------------------------
+
 	/**
-	 * Returns either the default {@link AvroUtils} which throw an exception in cases where Avro
-	 * would be needed or loads the specific utils for Avro from flink-avro.
+	 * A default implementation of the AvroUtils used in the absence of Avro.
 	 */
-	public static AvroUtils getAvroUtils() {
+	private static class DefaultAvroUtils extends AvroUtils {
 
-		// try and load the special AvroUtils from the flink-avro package
-		Class<?> clazz;
-		try {
-			clazz = Class.forName(AVRO_KRYO_UTILS, false, AvroUtils.class.getClassLoader());
-		} catch (ClassNotFoundException e) {
-			// cannot find the utils, return the default implementation
-			return new DefaultAvroUtils();
-		}
+		private static final String AVRO_SPECIFIC_RECORD_BASE = "org.apache.avro.specific.SpecificRecordBase";
 
-		try {
-			return (AvroUtils) clazz.getConstructor().newInstance();
-		} catch (Exception e) {
-			throw new RuntimeException("Could not instantiate " + AVRO_KRYO_UTILS + ".");
-		}
-	}
+		private static final String AVRO_GENERIC_RECORD = "org.apache.avro.generic.GenericData$Record";
+
+		private static final String AVRO_GENERIC_DATA_ARRAY = "org.apache.avro.generic.GenericData$Array";
 
-	private static class DefaultAvroUtils extends AvroUtils {
 		@Override
 		public void addAvroSerializersIfRequired(ExecutionConfig reg, Class<?> type) {
-			if (hasSuperclass(type, AVRO_SPECIFIC_RECORD_BASE) || hasSuperclass(
-				type,
-				AVRO_GENERIC_RECORD)) {
+			if (hasSuperclass(type, AVRO_SPECIFIC_RECORD_BASE) ||
+				hasSuperclass(type, AVRO_GENERIC_RECORD)) {
+
 				throw new RuntimeException("Could not load class '" + AVRO_KRYO_UTILS + "'. " +
 					"You may be missing the 'flink-avro' dependency.");
 			}
 		}
 
+		@SuppressWarnings({"rawtypes", "unchecked"})
 		@Override
 		public void addAvroGenericDataArrayRegistration(LinkedHashMap<String, KryoRegistration> kryoRegistrations) {
 			kryoRegistrations.put(AVRO_GENERIC_DATA_ARRAY,


[11/21] flink git commit: [FLINK-7420] [avro] Move all Avro code to flink-avro

Posted by se...@apache.org.
[FLINK-7420] [avro] Move all Avro code to flink-avro


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

Branch: refs/heads/master
Commit: 537a10ea2ff6a2d8507483c66f413f77884e77c4
Parents: 2c0fa24
Author: twalthr <tw...@apache.org>
Authored: Wed Aug 16 12:17:00 2017 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Nov 3 16:40:34 2017 +0100

----------------------------------------------------------------------
 .gitignore                                      |   2 +-
 flink-connectors/flink-avro/pom.xml             | 265 ---------
 .../apache/flink/api/avro/DataInputDecoder.java | 212 --------
 .../flink/api/avro/DataOutputEncoder.java       | 180 -------
 .../api/avro/FSDataInputStreamWrapper.java      |  67 ---
 .../flink/api/java/io/AvroInputFormat.java      | 207 --------
 .../flink/api/java/io/AvroOutputFormat.java     | 201 -------
 .../src/test/assembly/test-assembly.xml         |  36 --
 .../api/avro/AvroExternalJarProgramITCase.java  |  92 ----
 .../flink/api/avro/AvroOutputFormatITCase.java  | 177 -------
 .../flink/api/avro/EncoderDecoderTest.java      | 529 ------------------
 .../avro/testjar/AvroExternalJarProgram.java    | 211 --------
 .../apache/flink/api/io/avro/AvroPojoTest.java  | 255 ---------
 .../api/io/avro/AvroRecordInputFormatTest.java  | 460 ----------------
 .../io/avro/AvroSplittableInputFormatTest.java  | 325 ------------
 .../api/io/avro/example/AvroTypeExample.java    | 106 ----
 .../apache/flink/api/io/avro/example/User.java  | 269 ----------
 .../io/AvroInputFormatTypeExtractionTest.java   |  86 ---
 .../flink/api/java/io/AvroOutputFormatTest.java | 197 -------
 .../api/java/typeutils/AvroTypeInfoTest.java    |  37 --
 .../src/test/resources/avro/user.avsc           |  35 --
 .../src/test/resources/log4j-test.properties    |  27 -
 .../flink-avro/src/test/resources/testdata.avro | Bin 4572 -> 0 bytes
 .../flink-connector-filesystem/pom.xml          |   9 +
 .../connectors/fs/AvroKeyValueSinkWriter.java   |   2 +-
 .../flink-connector-kafka-0.10/pom.xml          |  12 +-
 .../flink-connector-kafka-0.11/pom.xml          |  12 +-
 .../flink-connector-kafka-0.8/pom.xml           |  12 +-
 .../flink-connector-kafka-0.9/pom.xml           |  12 +-
 .../flink-connector-kafka-base/pom.xml          |  31 +-
 .../AvroRowDeserializationSchema.java           | 179 -------
 .../AvroRowSerializationSchema.java             | 149 ------
 .../kafka/AvroRowDeSerializationSchemaTest.java | 148 ------
 .../kafka/KafkaAvroTableSourceTestBase.java     |   2 +-
 .../kafka/testutils/AvroTestUtils.java          | 152 ------
 flink-connectors/pom.xml                        |   1 -
 flink-core/pom.xml                              |   9 +-
 .../flink/api/common/ExecutionConfig.java       |  10 +-
 .../flink/api/java/typeutils/AvroTypeInfo.java  |  84 ---
 .../flink/api/java/typeutils/PojoTypeInfo.java  |  26 +-
 .../api/java/typeutils/TypeExtractionUtils.java |  35 ++
 .../flink/api/java/typeutils/TypeExtractor.java |  64 ++-
 .../java/typeutils/runtime/AvroSerializer.java  | 332 ------------
 .../typeutils/runtime/DataInputDecoder.java     | 230 --------
 .../typeutils/runtime/DataOutputEncoder.java    | 191 -------
 .../typeutils/runtime/kryo/KryoSerializer.java  |  11 +-
 .../typeutils/runtime/kryo/Serializers.java     | 134 +++--
 .../runtime/AvroGenericArraySerializerTest.java |  28 -
 .../runtime/AvroGenericTypeComparatorTest.java  |  28 -
 .../runtime/AvroGenericTypeSerializerTest.java  |  29 -
 .../runtime/AvroSerializerEmptyArrayTest.java   | 189 -------
 .../kryo/KryoSerializerCompatibilityTest.java   |  28 +-
 .../resources/kryo-serializer-flink1.3-snapshot | Bin 0 -> 1305 bytes
 .../scala/examples/twitter/TwitterExample.scala |   3 +-
 flink-formats/flink-avro/pom.xml                | 280 ++++++++++
 .../flink/api/java/typeutils/AvroTypeInfo.java  |  38 ++
 .../flink/formats/avro/AvroInputFormat.java     | 207 ++++++++
 .../flink/formats/avro/AvroOutputFormat.java    | 201 +++++++
 .../avro/AvroRowDeserializationSchema.java      | 179 +++++++
 .../avro/AvroRowSerializationSchema.java        | 149 ++++++
 .../formats/avro/typeutils/AvroSerializer.java  | 338 ++++++++++++
 .../formats/avro/typeutils/AvroTypeInfo.java    | 100 ++++
 .../avro/utils/AvroKryoSerializerUtils.java     |  72 +++
 .../formats/avro/utils/DataInputDecoder.java    | 212 ++++++++
 .../formats/avro/utils/DataOutputEncoder.java   | 180 +++++++
 .../avro/utils/FSDataInputStreamWrapper.java    |  67 +++
 .../src/test/assembly/test-assembly.xml         |  36 ++
 .../avro/AvroExternalJarProgramITCase.java      |  92 ++++
 .../avro/AvroInputFormatTypeExtractionTest.java |  86 +++
 .../formats/avro/AvroOutputFormatITCase.java    | 188 +++++++
 .../formats/avro/AvroOutputFormatTest.java      | 207 ++++++++
 .../formats/avro/AvroRecordInputFormatTest.java | 459 ++++++++++++++++
 .../avro/AvroRowDeSerializationSchemaTest.java  | 146 +++++
 .../avro/AvroSplittableInputFormatTest.java     | 324 +++++++++++
 .../flink/formats/avro/EncoderDecoderTest.java  | 531 +++++++++++++++++++
 .../avro/testjar/AvroExternalJarProgram.java    | 211 ++++++++
 .../AvroGenericArraySerializerTest.java         |  33 ++
 .../AvroGenericTypeComparatorTest.java          |  33 ++
 .../AvroGenericTypeSerializerTest.java          |  33 ++
 .../typeutils/AvroSerializerEmptyArrayTest.java | 217 ++++++++
 .../avro/typeutils/AvroTypeExtractionTest.java  | 257 +++++++++
 .../avro/typeutils/AvroTypeInfoTest.java        |  37 ++
 .../flink/formats/avro/utils/AvroTestUtils.java | 152 ++++++
 .../src/test/resources/avro/user.avsc           |  35 ++
 .../src/test/resources/log4j-test.properties    |  27 +
 .../flink-avro/src/test/resources/testdata.avro | Bin 0 -> 4576 bytes
 flink-formats/pom.xml                           |  42 ++
 .../org/apache/flink/hdfstests/HDFSTest.java    |  35 +-
 flink-libraries/flink-cep/pom.xml               |   8 +-
 .../flink-shaded-hadoop2/pom.xml                |  12 +
 flink-shaded-hadoop/pom.xml                     |   5 -
 flink-tests/pom.xml                             |  15 -
 pom.xml                                         |  21 +-
 tools/maven/suppressions.xml                    |   3 +-
 tools/travis_mvn_watchdog.sh                    |   2 +-
 95 files changed, 5488 insertions(+), 5910 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 3c9e4e8..8fc9fce 100644
--- a/.gitignore
+++ b/.gitignore
@@ -17,7 +17,7 @@ tmp
 *.log
 .DS_Store
 build-target
-flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/generated/
+flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/generated/
 flink-runtime-web/web-dashboard/assets/fonts/
 flink-runtime-web/web-dashboard/node_modules/
 flink-runtime-web/web-dashboard/bower_components/

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-connectors/flink-avro/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/pom.xml b/flink-connectors/flink-avro/pom.xml
deleted file mode 100644
index f8d9293..0000000
--- a/flink-connectors/flink-avro/pom.xml
+++ /dev/null
@@ -1,265 +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-connectors</artifactId>
-		<version>1.4-SNAPSHOT</version>
-		<relativePath>..</relativePath>
-	</parent>
-
-	<artifactId>flink-avro_${scala.binary.version}</artifactId>
-	<name>flink-avro</name>
-
-	<packaging>jar</packaging>
-
-	<dependencies>
-
-		<!-- core dependencies -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-core</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.avro</groupId>
-			<artifactId>avro</artifactId>
-			<!-- version is derived from base module -->
-		</dependency>
-
-		<!-- test dependencies -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils-junit</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_${scala.binary.version}</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-core</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-		</dependency>
-	</dependencies>
-
-	<build>
-		<plugins>
-			<plugin>
-				<artifactId>maven-assembly-plugin</artifactId>
-				<executions>
-					<execution>
-						<id>create-test-dependency</id>
-						<phase>process-test-classes</phase>
-						<goals>
-							<goal>single</goal>
-						</goals>
-						<configuration>
-							<archive>
-								<manifest>
-									<mainClass>org.apache.flink.api.avro.testjar.AvroExternalJarProgram</mainClass>
-								</manifest>
-							</archive>
-							<finalName>maven</finalName>
-							<attach>false</attach>
-							<descriptors>
-								<descriptor>src/test/assembly/test-assembly.xml</descriptor>
-							</descriptors>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-			<!--Remove the AvroExternalJarProgram code from the test-classes directory since it musn't be in the
-			classpath when running the tests to actually test whether the user code class loader
-			is properly used.-->
-			<plugin>
-				<artifactId>maven-clean-plugin</artifactId>
-				<version>2.5</version><!--$NO-MVN-MAN-VER$-->
-				<executions>
-					<execution>
-						<id>remove-avroexternalprogram</id>
-						<phase>process-test-classes</phase>
-						<goals>
-							<goal>clean</goal>
-						</goals>
-						<configuration>
-							<excludeDefaultDirectories>true</excludeDefaultDirectories>
-							<filesets>
-								<fileset>
-									<directory>${project.build.testOutputDirectory}</directory>
-									<includes>
-										<include>**/testjar/*.class</include>
-									</includes>
-								</fileset>
-							</filesets>
-						</configuration>
-					</execution>
-				</executions>
-				<configuration>
-					<filesets>
-						<fileset>
-							<directory>${project.basedir}/src/test/java/org/apache/flink/api/io/avro/generated</directory>
-						</fileset>
-					</filesets>
-				</configuration>
-			</plugin>
-			<!-- Generate Test class from avro schema -->
-			<plugin>
-				<groupId>org.apache.avro</groupId>
-				<artifactId>avro-maven-plugin</artifactId>
-				<version>1.8.2</version>
-				<executions>
-					<execution>
-						<phase>generate-sources</phase>
-						<goals>
-							<goal>schema</goal>
-						</goals>
-						<configuration>
-							<testSourceDirectory>${project.basedir}/src/test/resources/avro</testSourceDirectory>
-							<testOutputDirectory>${project.basedir}/src/test/java/</testOutputDirectory>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-
-			<!-- Add Avro test classes to test jar in order to test AvroTypeInfo. -->
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-jar-plugin</artifactId>
-				<executions>
-					<execution>
-						<goals>
-							<goal>test-jar</goal>
-						</goals>
-					</execution>
-				</executions>
-			</plugin>
-
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-shade-plugin</artifactId>
-				<executions>
-					<execution>
-						<id>shade-flink</id>
-						<phase>package</phase>
-						<goals>
-							<goal>shade</goal>
-						</goals>
-						<configuration combine.self="override">
-							<dependencyReducedPomLocation>${project.basedir}/target/dependency-reduced-pom.xml</dependencyReducedPomLocation>
-							<promoteTransitiveDependencies>true</promoteTransitiveDependencies>
-							<artifactSet>
-								<includes>
-									<include>org.codehaus.jackson:*</include>
-								</includes>
-							</artifactSet>
-							<relocations>
-								<relocation>
-									<pattern>org.codehaus.jackson</pattern>
-									<shadedPattern>org.apache.flink.avro.shaded.org.codehouse.jackson</shadedPattern>
-								</relocation>
-							</relocations>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-
-		<pluginManagement>
-			<plugins>
-				<!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
-				<plugin>
-					<groupId>org.eclipse.m2e</groupId>
-					<artifactId>lifecycle-mapping</artifactId>
-					<version>1.0.0</version>
-					<configuration>
-						<lifecycleMappingMetadata>
-							<pluginExecutions>
-								<pluginExecution>
-									<pluginExecutionFilter>
-										<groupId>org.apache.maven.plugins</groupId>
-										<artifactId>maven-assembly-plugin</artifactId>
-										<versionRange>[2.4,)</versionRange>
-										<goals>
-											<goal>single</goal>
-										</goals>
-									</pluginExecutionFilter>
-									<action>
-										<ignore/>
-									</action>
-								</pluginExecution>
-								<pluginExecution>
-									<pluginExecutionFilter>
-										<groupId>org.apache.maven.plugins</groupId>
-										<artifactId>maven-clean-plugin</artifactId>
-										<versionRange>[1,)</versionRange>
-										<goals>
-											<goal>clean</goal>
-										</goals>
-									</pluginExecutionFilter>
-									<action>
-										<ignore/>
-									</action>
-								</pluginExecution>
-								<pluginExecution>
-									<pluginExecutionFilter>
-										<groupId>org.apache.avro</groupId>
-										<artifactId>avro-maven-plugin</artifactId>
-										<versionRange>[1.7.7,)</versionRange>
-										<goals>
-											<goal>schema</goal>
-										</goals>
-									</pluginExecutionFilter>
-									<action>
-										<ignore/>
-									</action>
-								</pluginExecution>
-							</pluginExecutions>
-						</lifecycleMappingMetadata>
-					</configuration>
-				</plugin>
-			</plugins>
-		</pluginManagement>
-	</build>
-
-</project>

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/FSDataInputStreamWrapper.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/FSDataInputStreamWrapper.java b/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/FSDataInputStreamWrapper.java
deleted file mode 100644
index 19e4a89..0000000
--- a/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/avro/FSDataInputStreamWrapper.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.api.avro;
-
-import org.apache.flink.core.fs.FSDataInputStream;
-
-import org.apache.avro.file.SeekableInput;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-/**
- * Code copy pasted from org.apache.avro.mapred.FSInput (which is Apache licensed as well).
- *
- * <p>The wrapper keeps track of the position in the data stream.
- */
-public class FSDataInputStreamWrapper implements Closeable, SeekableInput {
-	private final FSDataInputStream stream;
-	private long pos;
-	private long len;
-
-	public FSDataInputStreamWrapper(FSDataInputStream stream, long len) {
-		this.stream = stream;
-		this.pos = 0;
-		this.len = len;
-	}
-
-	public long length() throws IOException {
-		return this.len;
-	}
-
-	public int read(byte[] b, int off, int len) throws IOException {
-		int read;
-		read = stream.read(b, off, len);
-		pos += read;
-		return read;
-	}
-
-	public void seek(long p) throws IOException {
-		stream.seek(p);
-		pos = p;
-	}
-
-	public long tell() throws IOException {
-		return pos;
-	}
-
-	public void close() throws IOException {
-		stream.close();
-	}
-}

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

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

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroOutputFormatITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroOutputFormatITCase.java b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroOutputFormatITCase.java
deleted file mode 100644
index f630f41..0000000
--- a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroOutputFormatITCase.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.flink.api.avro;
-
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.io.avro.example.User;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.AvroOutputFormat;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-import org.apache.avro.file.DataFileReader;
-import org.apache.avro.io.DatumReader;
-import org.apache.avro.reflect.ReflectDatumReader;
-import org.apache.avro.specific.SpecificDatumReader;
-import org.junit.Assert;
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * IT cases for the {@link AvroOutputFormat}.
- */
-@SuppressWarnings("serial")
-public class AvroOutputFormatITCase extends JavaProgramTestBase {
-
-	public static String outputPath1;
-
-	public static String outputPath2;
-
-	public static String inputPath;
-
-	public static String userData = "alice|1|blue\n" +
-		"bob|2|red\n" +
-		"john|3|yellow\n" +
-		"walt|4|black\n";
-
-	@Override
-	protected void preSubmit() throws Exception {
-		inputPath = createTempFile("user", userData);
-		outputPath1 = getTempDirPath("avro_output1");
-		outputPath2 = getTempDirPath("avro_output2");
-	}
-
-	@Override
-	protected void testProgram() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<String, Integer, String>> input = env.readCsvFile(inputPath)
-			.fieldDelimiter("|")
-			.types(String.class, Integer.class, String.class);
-
-		//output the data with AvroOutputFormat for specific user type
-		DataSet<User> specificUser = input.map(new ConvertToUser());
-		AvroOutputFormat<User> avroOutputFormat = new AvroOutputFormat<User>(User.class);
-		avroOutputFormat.setCodec(AvroOutputFormat.Codec.SNAPPY); // FLINK-4771: use a codec
-		avroOutputFormat.setSchema(User.SCHEMA$); //FLINK-3304: Ensure the OF is properly serializing the schema
-		specificUser.write(avroOutputFormat, outputPath1);
-
-		//output the data with AvroOutputFormat for reflect user type
-		DataSet<ReflectiveUser> reflectiveUser = specificUser.map(new ConvertToReflective());
-		reflectiveUser.write(new AvroOutputFormat<ReflectiveUser>(ReflectiveUser.class), outputPath2);
-
-		env.execute();
-	}
-
-	@Override
-	protected void postSubmit() throws Exception {
-		//compare result for specific user type
-		File [] output1;
-		File file1 = asFile(outputPath1);
-		if (file1.isDirectory()) {
-			output1 = file1.listFiles();
-			// check for avro ext in dir.
-			for (File avroOutput : output1) {
-				Assert.assertTrue("Expect extension '.avro'", avroOutput.toString().endsWith(".avro"));
-			}
-		} else {
-			output1 = new File[] {file1};
-		}
-		List<String> result1 = new ArrayList<String>();
-		DatumReader<User> userDatumReader1 = new SpecificDatumReader<User>(User.class);
-		for (File avroOutput : output1) {
-
-			DataFileReader<User> dataFileReader1 = new DataFileReader<User>(avroOutput, userDatumReader1);
-			while (dataFileReader1.hasNext()) {
-				User user = dataFileReader1.next();
-				result1.add(user.getName() + "|" + user.getFavoriteNumber() + "|" + user.getFavoriteColor());
-			}
-		}
-		for (String expectedResult : userData.split("\n")) {
-			Assert.assertTrue("expected user " + expectedResult + " not found.", result1.contains(expectedResult));
-		}
-
-		//compare result for reflect user type
-		File [] output2;
-		File file2 = asFile(outputPath2);
-		if (file2.isDirectory()) {
-			output2 = file2.listFiles();
-		} else {
-			output2 = new File[] {file2};
-		}
-		List<String> result2 = new ArrayList<String>();
-		DatumReader<ReflectiveUser> userDatumReader2 = new ReflectDatumReader<ReflectiveUser>(ReflectiveUser.class);
-		for (File avroOutput : output2) {
-			DataFileReader<ReflectiveUser> dataFileReader2 = new DataFileReader<ReflectiveUser>(avroOutput, userDatumReader2);
-			while (dataFileReader2.hasNext()) {
-				ReflectiveUser user = dataFileReader2.next();
-				result2.add(user.getName() + "|" + user.getFavoriteNumber() + "|" + user.getFavoriteColor());
-			}
-		}
-		for (String expectedResult : userData.split("\n")) {
-			Assert.assertTrue("expected user " + expectedResult + " not found.", result2.contains(expectedResult));
-		}
-
-	}
-
-	private static final class ConvertToUser extends RichMapFunction<Tuple3<String, Integer, String>, User> {
-
-		@Override
-		public User map(Tuple3<String, Integer, String> value) throws Exception {
-			return new User(value.f0, value.f1, value.f2);
-		}
-	}
-
-	private static final class ConvertToReflective extends RichMapFunction<User, ReflectiveUser> {
-
-		@Override
-		public ReflectiveUser map(User value) throws Exception {
-			return new ReflectiveUser(value.getName().toString(), value.getFavoriteNumber(), value.getFavoriteColor().toString());
-		}
-	}
-
-	private static class ReflectiveUser {
-		private String name;
-		private int favoriteNumber;
-		private String favoriteColor;
-
-		public ReflectiveUser() {}
-
-		public ReflectiveUser(String name, int favoriteNumber, String favoriteColor) {
-			this.name = name;
-			this.favoriteNumber = favoriteNumber;
-			this.favoriteColor = favoriteColor;
-		}
-
-		public String getName() {
-			return this.name;
-		}
-
-		public String getFavoriteColor() {
-			return this.favoriteColor;
-		}
-
-		public int getFavoriteNumber() {
-			return this.favoriteNumber;
-		}
-	}
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/testjar/AvroExternalJarProgram.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/testjar/AvroExternalJarProgram.java b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/testjar/AvroExternalJarProgram.java
deleted file mode 100644
index a8541b6..0000000
--- a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/testjar/AvroExternalJarProgram.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.api.avro.testjar;
-
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.functions.RichReduceFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.AvroInputFormat;
-import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.core.fs.Path;
-
-import org.apache.avro.file.DataFileWriter;
-import org.apache.avro.io.DatumWriter;
-import org.apache.avro.reflect.ReflectData;
-import org.apache.avro.reflect.ReflectDatumWriter;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
-/**
- * This file defines the classes for the AvroExternalJarProgramITCase.
- */
-public class AvroExternalJarProgram  {
-
-	private static final class Color {
-
-		private String name;
-		private double saturation;
-
-		public Color() {
-			name = "";
-			saturation = 1.0;
-		}
-
-		public Color(String name, double saturation) {
-			this.name = name;
-			this.saturation = saturation;
-		}
-
-		public String getName() {
-			return name;
-		}
-
-		public void setName(String name) {
-			this.name = name;
-		}
-
-		public double getSaturation() {
-			return saturation;
-		}
-
-		public void setSaturation(double saturation) {
-			this.saturation = saturation;
-		}
-
-		@Override
-		public String toString() {
-			return name + '(' + saturation + ')';
-		}
-	}
-
-	private static final class MyUser {
-
-		private String name;
-		private List<Color> colors;
-
-		public MyUser() {
-			name = "unknown";
-			colors = new ArrayList<Color>();
-		}
-
-		public MyUser(String name, List<Color> colors) {
-			this.name = name;
-			this.colors = colors;
-		}
-
-		public String getName() {
-			return name;
-		}
-
-		public List<Color> getColors() {
-			return colors;
-		}
-
-		public void setName(String name) {
-			this.name = name;
-		}
-
-		public void setColors(List<Color> colors) {
-			this.colors = colors;
-		}
-
-		@Override
-		public String toString() {
-			return name + " : " + colors;
-		}
-	}
-
-	// --------------------------------------------------------------------------------------------
-
-	// --------------------------------------------------------------------------------------------
-
-	private static final class NameExtractor extends RichMapFunction<MyUser, Tuple2<String, MyUser>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<String, MyUser> map(MyUser u) {
-			String namePrefix = u.getName().substring(0, 1);
-			return new Tuple2<String, MyUser>(namePrefix, u);
-		}
-	}
-
-	private static final class NameGrouper extends RichReduceFunction<Tuple2<String, MyUser>> {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public Tuple2<String, MyUser> reduce(Tuple2<String, MyUser> val1, Tuple2<String, MyUser> val2) {
-			return val1;
-		}
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Test Data
-	// --------------------------------------------------------------------------------------------
-
-	private static final class Generator {
-
-		private final Random rnd = new Random(2389756789345689276L);
-
-		public MyUser nextUser() {
-			return randomUser();
-		}
-
-		private MyUser randomUser() {
-
-			int numColors = rnd.nextInt(5);
-			ArrayList<Color> colors = new ArrayList<Color>(numColors);
-			for (int i = 0; i < numColors; i++) {
-				colors.add(new Color(randomString(), rnd.nextDouble()));
-			}
-
-			return new MyUser(randomString(), colors);
-		}
-
-		private String randomString() {
-			char[] c = new char[this.rnd.nextInt(20) + 5];
-
-			for (int i = 0; i < c.length; i++) {
-				c[i] = (char) (this.rnd.nextInt(150) + 40);
-			}
-
-			return new String(c);
-		}
-	}
-
-	public static void writeTestData(File testFile, int numRecords) throws IOException {
-
-		DatumWriter<MyUser> userDatumWriter = new ReflectDatumWriter<MyUser>(MyUser.class);
-		DataFileWriter<MyUser> dataFileWriter = new DataFileWriter<MyUser>(userDatumWriter);
-
-		dataFileWriter.create(ReflectData.get().getSchema(MyUser.class), testFile);
-
-		Generator generator = new Generator();
-
-		for (int i = 0; i < numRecords; i++) {
-			MyUser user = generator.nextUser();
-			dataFileWriter.append(user);
-		}
-
-		dataFileWriter.close();
-	}
-
-//	public static void main(String[] args) throws Exception {
-//		String testDataFile = new File("src/test/resources/testdata.avro").getAbsolutePath();
-//		writeTestData(new File(testDataFile), 50);
-//	}
-
-	public static void main(String[] args) throws Exception {
-		String inputPath = args[0];
-
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<MyUser> input = env.createInput(new AvroInputFormat<MyUser>(new Path(inputPath), MyUser.class));
-
-		DataSet<Tuple2<String, MyUser>> result = input.map(new NameExtractor()).groupBy(0).reduce(new NameGrouper());
-
-		result.output(new DiscardingOutputFormat<Tuple2<String, MyUser>>());
-		env.execute();
-	}
-}


[03/21] flink git commit: [hotfix] [tests] Remove console poluting output in tests of flink-streaming-java

Posted by se...@apache.org.
[hotfix] [tests] Remove console poluting output in tests of flink-streaming-java


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

Branch: refs/heads/master
Commit: 2c0fa24032502c2ea375d5a9628efc9748bcf9ec
Parents: fe931d0
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Nov 3 11:28:18 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Nov 3 16:40:34 2017 +0100

----------------------------------------------------------------------
 .../api/functions/sink/TwoPhaseCommitSinkFunctionTest.java         | 1 +
 .../flink/streaming/api/operators/AbstractStreamOperatorTest.java  | 2 ++
 .../flink/streaming/api/operators/KeyedProcessOperatorTest.java    | 2 --
 3 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/2c0fa240/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunctionTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunctionTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunctionTest.java
index c004423..63147a0 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunctionTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunctionTest.java
@@ -109,6 +109,7 @@ public class TwoPhaseCommitSinkFunctionTest {
 	 * @see #testLogTimeoutAlmostReachedWarningDuringRecovery
 	 */
 	private void setupLogger() {
+		Logger.getRootLogger().removeAllAppenders();
 		logger = Logger.getLogger(TwoPhaseCommitSinkFunction.class);
 		testAppender = new AppenderSkeleton() {
 			@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/2c0fa240/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java
index 847ff13..7262560 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractStreamOperatorTest.java
@@ -45,6 +45,7 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.mockito.internal.util.reflection.Whitebox;
+import org.powermock.core.classloader.annotations.PowerMockIgnore;
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
@@ -77,6 +78,7 @@ import static org.powermock.api.mockito.PowerMockito.whenNew;
  */
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(AbstractStreamOperator.class)
+@PowerMockIgnore({"java.*", "javax.*", "org.slf4j.*", "org.apache.log4j.*"})
 public class AbstractStreamOperatorTest {
 
 	@Test

http://git-wip-us.apache.org/repos/asf/flink/blob/2c0fa240/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/KeyedProcessOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/KeyedProcessOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/KeyedProcessOperatorTest.java
index 3ad8d34..f043fa8 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/KeyedProcessOperatorTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/KeyedProcessOperatorTest.java
@@ -276,8 +276,6 @@ public class KeyedProcessOperatorTest extends TestLogger {
 		expectedOutput.add(new StreamRecord<>("EVENT:1777", 6L));
 		expectedOutput.add(new Watermark(6));
 
-		System.out.println("GOT: " + testHarness.getOutput());
-
 		TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput());
 
 		testHarness.close();


[21/21] flink git commit: [hotfix] [avro] Add test that validates deserialization of Kryo Serializer in the absence of Avro

Posted by se...@apache.org.
[hotfix] [avro] Add test that validates deserialization of Kryo Serializer in the absence of Avro


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

Branch: refs/heads/master
Commit: 87bf578163facc272138141fbcfa35c3bedbe74d
Parents: c438e29
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Nov 2 22:38:54 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Nov 3 16:40:36 2017 +0100

----------------------------------------------------------------------
 .../TypeSerializerSerializationUtil.java        |   1 +
 .../kryo/KryoSerializerCompatibilityTest.java   |  21 ++++++++++++++++++-
 .../resources/serialized-kryo-serializer-1.3    | Bin 0 -> 1710 bytes
 3 files changed, 21 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/87bf5781/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationUtil.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationUtil.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationUtil.java
index d03498a..c6291ad 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationUtil.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationUtil.java
@@ -30,6 +30,7 @@ import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 import org.apache.flink.util.InstantiationUtil;
 import org.apache.flink.util.Preconditions;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/87bf5781/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerCompatibilityTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerCompatibilityTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerCompatibilityTest.java
index 11c95f1..89e9ec3 100644
--- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerCompatibilityTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerCompatibilityTest.java
@@ -20,6 +20,7 @@ package org.apache.flink.api.java.typeutils.runtime.kryo;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
 import org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil;
 import org.apache.flink.core.memory.DataInputViewStreamWrapper;
@@ -29,6 +30,7 @@ import com.esotericsoftware.kryo.Kryo;
 import com.esotericsoftware.kryo.Serializer;
 import com.esotericsoftware.kryo.io.Input;
 import com.esotericsoftware.kryo.io.Output;
+
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
@@ -42,6 +44,7 @@ import java.util.List;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
@@ -67,6 +70,22 @@ public class KryoSerializerCompatibilityTest {
 		assertFalse(compatResult.isRequiresMigration());
 	}
 
+	@Test
+	public void testDeserializingKryoSerializerWithoutAvro() throws Exception {
+		final String resource = "serialized-kryo-serializer-1.3";
+
+		TypeSerializer<?> serializer;
+
+		try (InputStream in = getClass().getClassLoader().getResourceAsStream(resource)) {
+			DataInputViewStreamWrapper inView = new DataInputViewStreamWrapper(in);
+
+			serializer = TypeSerializerSerializationUtil.tryReadSerializer(inView, getClass().getClassLoader());
+		}
+
+		assertNotNull(serializer);
+		assertTrue(serializer instanceof KryoSerializer);
+	}
+
 	/**
 	 * Verifies that reconfiguration result is INCOMPATIBLE if data type has changed.
 	 */
@@ -150,7 +169,7 @@ public class KryoSerializerCompatibilityTest {
 				DataInputViewStreamWrapper inputView = new DataInputViewStreamWrapper(f)) {
 
 				thrown.expectMessage("Could not find required Avro dependency");
-				FakeAvroClass myTestClass = kryoSerializer.deserialize(inputView);
+				kryoSerializer.deserialize(inputView);
 			}
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/87bf5781/flink-core/src/test/resources/serialized-kryo-serializer-1.3
----------------------------------------------------------------------
diff --git a/flink-core/src/test/resources/serialized-kryo-serializer-1.3 b/flink-core/src/test/resources/serialized-kryo-serializer-1.3
new file mode 100644
index 0000000..d977c50
Binary files /dev/null and b/flink-core/src/test/resources/serialized-kryo-serializer-1.3 differ


[10/21] flink git commit: [FLINK-7420] [avro] Move all Avro code to flink-avro

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroOutputFormatTest.java b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroOutputFormatTest.java
deleted file mode 100644
index 71ebd78..0000000
--- a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroOutputFormatTest.java
+++ /dev/null
@@ -1,197 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.io;
-
-import org.apache.flink.api.io.avro.example.User;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
-
-import org.apache.avro.Schema;
-import org.apache.avro.file.DataFileReader;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.generic.GenericDatumReader;
-import org.apache.avro.generic.GenericRecord;
-import org.junit.Test;
-import org.mockito.internal.util.reflection.Whitebox;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-
-import static org.apache.flink.api.java.io.AvroOutputFormat.Codec;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * Tests for {@link AvroOutputFormat}.
- */
-public class AvroOutputFormatTest {
-
-	@Test
-	public void testSetCodec() throws Exception {
-		// given
-		final AvroOutputFormat<User> outputFormat = new AvroOutputFormat<>(User.class);
-
-		// when
-		try {
-			outputFormat.setCodec(Codec.SNAPPY);
-		} catch (Exception ex) {
-			// then
-			fail("unexpected exception");
-		}
-	}
-
-	@Test
-	public void testSetCodecError() throws Exception {
-		// given
-		boolean error = false;
-		final AvroOutputFormat<User> outputFormat = new AvroOutputFormat<>(User.class);
-
-		// when
-		try {
-			outputFormat.setCodec(null);
-		} catch (Exception ex) {
-			error = true;
-		}
-
-		// then
-		assertTrue(error);
-	}
-
-	@Test
-	public void testSerialization() throws Exception {
-
-		serializeAndDeserialize(null, null);
-		serializeAndDeserialize(null, User.SCHEMA$);
-		for (final Codec codec : Codec.values()) {
-			serializeAndDeserialize(codec, null);
-			serializeAndDeserialize(codec, User.SCHEMA$);
-		}
-	}
-
-	private void serializeAndDeserialize(final Codec codec, final Schema schema) throws IOException, ClassNotFoundException {
-		// given
-		final AvroOutputFormat<User> outputFormat = new AvroOutputFormat<>(User.class);
-		if (codec != null) {
-			outputFormat.setCodec(codec);
-		}
-		if (schema != null) {
-			outputFormat.setSchema(schema);
-		}
-
-		final ByteArrayOutputStream bos = new ByteArrayOutputStream();
-
-		// when
-		try (final ObjectOutputStream oos = new ObjectOutputStream(bos)) {
-			oos.writeObject(outputFormat);
-		}
-		try (final ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(bos.toByteArray()))) {
-			// then
-			Object o = ois.readObject();
-			assertTrue(o instanceof AvroOutputFormat);
-			final AvroOutputFormat<User> restored = (AvroOutputFormat<User>) o;
-			final Codec restoredCodec = (Codec) Whitebox.getInternalState(restored, "codec");
-			final Schema restoredSchema = (Schema) Whitebox.getInternalState(restored, "userDefinedSchema");
-
-			assertTrue(codec != null ? restoredCodec == codec : restoredCodec == null);
-			assertTrue(schema != null ? restoredSchema.equals(schema) : restoredSchema == null);
-		}
-	}
-
-	@Test
-	public void testCompression() throws Exception {
-		// given
-		final Path outputPath = new Path(File.createTempFile("avro-output-file", "avro").getAbsolutePath());
-		final AvroOutputFormat<User> outputFormat = new AvroOutputFormat<>(outputPath, User.class);
-		outputFormat.setWriteMode(FileSystem.WriteMode.OVERWRITE);
-
-		final Path compressedOutputPath = new Path(File.createTempFile("avro-output-file", "compressed.avro").getAbsolutePath());
-		final AvroOutputFormat<User> compressedOutputFormat = new AvroOutputFormat<>(compressedOutputPath, User.class);
-		compressedOutputFormat.setWriteMode(FileSystem.WriteMode.OVERWRITE);
-		compressedOutputFormat.setCodec(Codec.SNAPPY);
-
-		// when
-		output(outputFormat);
-		output(compressedOutputFormat);
-
-		// then
-		assertTrue(fileSize(outputPath) > fileSize(compressedOutputPath));
-
-		// cleanup
-		FileSystem fs = FileSystem.getLocalFileSystem();
-		fs.delete(outputPath, false);
-		fs.delete(compressedOutputPath, false);
-	}
-
-	private long fileSize(Path path) throws IOException {
-		return path.getFileSystem().getFileStatus(path).getLen();
-	}
-
-	private void output(final AvroOutputFormat<User> outputFormat) throws IOException {
-		outputFormat.configure(new Configuration());
-		outputFormat.open(1, 1);
-		for (int i = 0; i < 100; i++) {
-			outputFormat.writeRecord(new User("testUser", 1, "blue"));
-		}
-		outputFormat.close();
-	}
-
-	@Test
-	public void testGenericRecord() throws IOException {
-		final Path outputPath = new Path(File.createTempFile("avro-output-file", "generic.avro").getAbsolutePath());
-		final AvroOutputFormat<GenericRecord> outputFormat = new AvroOutputFormat<>(outputPath, GenericRecord.class);
-		Schema schema = new Schema.Parser().parse("{\"type\":\"record\", \"name\":\"user\", \"fields\": [{\"name\":\"user_name\", \"type\":\"string\"}, {\"name\":\"favorite_number\", \"type\":\"int\"}, {\"name\":\"favorite_color\", \"type\":\"string\"}]}");
-		outputFormat.setWriteMode(FileSystem.WriteMode.OVERWRITE);
-		outputFormat.setSchema(schema);
-		output(outputFormat, schema);
-
-		GenericDatumReader<GenericRecord> reader = new GenericDatumReader<>(schema);
-		DataFileReader<GenericRecord> dataFileReader = new DataFileReader<>(new File(outputPath.getPath()), reader);
-
-		while (dataFileReader.hasNext()) {
-			GenericRecord record = dataFileReader.next();
-			assertEquals(record.get("user_name").toString(), "testUser");
-			assertEquals(record.get("favorite_number"), 1);
-			assertEquals(record.get("favorite_color").toString(), "blue");
-		}
-
-		//cleanup
-		FileSystem fs = FileSystem.getLocalFileSystem();
-		fs.delete(outputPath, false);
-
-	}
-
-	private void output(final AvroOutputFormat<GenericRecord> outputFormat, Schema schema) throws IOException {
-		outputFormat.configure(new Configuration());
-		outputFormat.open(1, 1);
-		for (int i = 0; i < 100; i++) {
-			GenericRecord record = new GenericData.Record(schema);
-			record.put("user_name", "testUser");
-			record.put("favorite_number", 1);
-			record.put("favorite_color", "blue");
-			outputFormat.writeRecord(record);
-		}
-		outputFormat.close();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/java/typeutils/AvroTypeInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/java/typeutils/AvroTypeInfoTest.java b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/java/typeutils/AvroTypeInfoTest.java
deleted file mode 100644
index e0bb1a1..0000000
--- a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/java/typeutils/AvroTypeInfoTest.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.api.java.typeutils;
-
-import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
-import org.apache.flink.api.io.avro.generated.Address;
-import org.apache.flink.api.io.avro.generated.User;
-
-/**
- * Test for {@link AvroTypeInfo}.
- */
-public class AvroTypeInfoTest extends TypeInformationTestBase<AvroTypeInfo<?>> {
-
-	@Override
-	protected AvroTypeInfo<?>[] getTestData() {
-		return new AvroTypeInfo<?>[] {
-			new AvroTypeInfo<>(Address.class),
-			new AvroTypeInfo<>(User.class),
-		};
-	}
-}

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

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-connectors/flink-connector-filesystem/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/pom.xml b/flink-connectors/flink-connector-filesystem/pom.xml
index f39758b..12a151e 100644
--- a/flink-connectors/flink-connector-filesystem/pom.xml
+++ b/flink-connectors/flink-connector-filesystem/pom.xml
@@ -57,6 +57,15 @@ under the License.
 			<scope>provided</scope>
 		</dependency>
 
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-avro_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+			<!-- Projects depending on this project, won't depend on flink-avro. -->
+			<optional>true</optional>
+		</dependency>
+
 		<!-- test dependencies -->
 
 		<dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/AvroKeyValueSinkWriter.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/AvroKeyValueSinkWriter.java b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/AvroKeyValueSinkWriter.java
index 317ee55..e931633 100644
--- a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/AvroKeyValueSinkWriter.java
+++ b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/AvroKeyValueSinkWriter.java
@@ -65,7 +65,7 @@ Usage:
 }
 </pre>
 */
-public class AvroKeyValueSinkWriter<K, V> extends StreamWriterBase<Tuple2<K, V>>  implements Writer<Tuple2<K, V>>, InputTypeConfigurable {
+public class AvroKeyValueSinkWriter<K, V> extends StreamWriterBase<Tuple2<K, V>> implements Writer<Tuple2<K, V>>, InputTypeConfigurable {
 	private static final long serialVersionUID = 1L;
 	public static final String CONF_OUTPUT_KEY_SCHEMA = "avro.schema.output.key";
 	public static final String CONF_OUTPUT_VALUE_SCHEMA = "avro.schema.output.value";

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-connectors/flink-connector-kafka-0.10/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/pom.xml b/flink-connectors/flink-connector-kafka-0.10/pom.xml
index 581640d..2b6660d 100644
--- a/flink-connectors/flink-connector-kafka-0.10/pom.xml
+++ b/flink-connectors/flink-connector-kafka-0.10/pom.xml
@@ -78,8 +78,16 @@ under the License.
 			<artifactId>flink-table_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
-			<!-- Projects depending on this project,
-			won't depend on flink-table. -->
+			<!-- Projects depending on this project, won't depend on flink-table. -->
+			<optional>true</optional>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-avro_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+			<!-- Projects depending on this project, won't depend on flink-avro. -->
 			<optional>true</optional>
 		</dependency>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-connectors/flink-connector-kafka-0.11/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.11/pom.xml b/flink-connectors/flink-connector-kafka-0.11/pom.xml
index 475c842..162d5d0 100644
--- a/flink-connectors/flink-connector-kafka-0.11/pom.xml
+++ b/flink-connectors/flink-connector-kafka-0.11/pom.xml
@@ -78,8 +78,16 @@ under the License.
 			<artifactId>flink-table_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
-			<!-- Projects depending on this project,
-			won't depend on flink-table. -->
+			<!-- Projects depending on this project, won't depend on flink-table. -->
+			<optional>true</optional>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-avro_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+			<!-- Projects depending on this project, won't depend on flink-avro. -->
 			<optional>true</optional>
 		</dependency>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-connectors/flink-connector-kafka-0.8/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/pom.xml b/flink-connectors/flink-connector-kafka-0.8/pom.xml
index dd7a542..c990188 100644
--- a/flink-connectors/flink-connector-kafka-0.8/pom.xml
+++ b/flink-connectors/flink-connector-kafka-0.8/pom.xml
@@ -69,8 +69,16 @@ under the License.
 			<artifactId>flink-table_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
-			<!-- Projects depending on this project,
-			won't depend on flink-table. -->
+			<!-- Projects depending on this project, won't depend on flink-table. -->
+			<optional>true</optional>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-avro_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+			<!-- Projects depending on this project, won't depend on flink-avro. -->
 			<optional>true</optional>
 		</dependency>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-connectors/flink-connector-kafka-0.9/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/pom.xml b/flink-connectors/flink-connector-kafka-0.9/pom.xml
index fef070d..819d590 100644
--- a/flink-connectors/flink-connector-kafka-0.9/pom.xml
+++ b/flink-connectors/flink-connector-kafka-0.9/pom.xml
@@ -68,8 +68,16 @@ under the License.
 			<artifactId>flink-table_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
-			<!-- Projects depending on this project,
-			won't depend on flink-table. -->
+			<!-- Projects depending on this project, won't depend on flink-table. -->
+			<optional>true</optional>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-avro_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+			<!-- Projects depending on this project, won't depend on flink-avro. -->
 			<optional>true</optional>
 		</dependency>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-connectors/flink-connector-kafka-base/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/pom.xml b/flink-connectors/flink-connector-kafka-base/pom.xml
index c9f7de2..4f2fb45 100644
--- a/flink-connectors/flink-connector-kafka-base/pom.xml
+++ b/flink-connectors/flink-connector-kafka-base/pom.xml
@@ -62,8 +62,16 @@ under the License.
 			<artifactId>flink-table_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
-			<!-- Projects depending on this project,
-			won't depend on flink-table. -->
+			<!-- Projects depending on this project, won't depend on flink-table. -->
+			<optional>true</optional>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-avro_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+			<!-- Projects depending on this project, won't depend on flink-avro. -->
 			<optional>true</optional>
 		</dependency>
 
@@ -218,25 +226,6 @@ under the License.
 				<inherited>true</inherited>
 				<extensions>true</extensions>
 			</plugin>
-			<!-- Add Avro generated classes for testing. -->
-			<plugin>
-				<groupId>org.codehaus.mojo</groupId>
-				<artifactId>build-helper-maven-plugin</artifactId>
-				<version>1.7</version>
-				<executions>
-					<execution>
-						<phase>generate-test-sources</phase>
-						<goals>
-							<goal>add-test-source</goal>
-						</goals>
-						<configuration>
-							<sources>
-								<source>${project.basedir}/../flink-avro/src/test/java/org/apache/flink/api/io/avro/generated</source>
-							</sources>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
 		</plugins>
 	</build>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowDeserializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowDeserializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowDeserializationSchema.java
deleted file mode 100644
index 0d36f4c..0000000
--- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowDeserializationSchema.java
+++ /dev/null
@@ -1,179 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.util.serialization;
-
-import org.apache.flink.api.common.serialization.AbstractDeserializationSchema;
-import org.apache.flink.types.Row;
-import org.apache.flink.util.Preconditions;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.io.DatumReader;
-import org.apache.avro.io.Decoder;
-import org.apache.avro.io.DecoderFactory;
-import org.apache.avro.specific.SpecificData;
-import org.apache.avro.specific.SpecificDatumReader;
-import org.apache.avro.specific.SpecificRecord;
-import org.apache.avro.util.Utf8;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.util.List;
-
-/**
- * Deserialization schema from Avro bytes over {@link SpecificRecord} to {@link Row}.
- *
- * <p>Deserializes the <code>byte[]</code> messages into (nested) Flink Rows.
- *
- * {@link Utf8} is converted to regular Java Strings.
- */
-public class AvroRowDeserializationSchema extends AbstractDeserializationSchema<Row> {
-
-	/**
-	 * Avro record class.
-	 */
-	private Class<? extends SpecificRecord> recordClazz;
-
-	/**
-	 * Schema for deterministic field order.
-	 */
-	private transient Schema schema;
-
-	/**
-	 * Reader that deserializes byte array into a record.
-	 */
-	private transient DatumReader<SpecificRecord> datumReader;
-
-	/**
-	 * Input stream to read message from.
-	 */
-	private transient MutableByteArrayInputStream inputStream;
-
-	/**
-	 * Avro decoder that decodes binary data.
-	 */
-	private transient Decoder decoder;
-
-	/**
-	 * Record to deserialize byte array to.
-	 */
-	private SpecificRecord record;
-
-	/**
-	 * Creates a Avro deserialization schema for the given record.
-	 *
-	 * @param recordClazz Avro record class used to deserialize Avro's record to Flink's row
-	 */
-	public AvroRowDeserializationSchema(Class<? extends SpecificRecord> recordClazz) {
-		Preconditions.checkNotNull(recordClazz, "Avro record class must not be null.");
-		this.recordClazz = recordClazz;
-		this.schema = SpecificData.get().getSchema(recordClazz);
-		this.datumReader = new SpecificDatumReader<>(schema);
-		this.record = (SpecificRecord) SpecificData.newInstance(recordClazz, schema);
-		this.inputStream = new MutableByteArrayInputStream();
-		this.decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
-	}
-
-	@Override
-	public Row deserialize(byte[] message) throws IOException {
-		// read record
-		try {
-			inputStream.setBuffer(message);
-			this.record = datumReader.read(record, decoder);
-		} catch (IOException e) {
-			throw new RuntimeException("Failed to deserialize Row.", e);
-		}
-
-		// convert to row
-		final Object row = convertToRow(schema, record);
-		return (Row) row;
-	}
-
-	private void writeObject(ObjectOutputStream oos) throws IOException {
-		oos.writeObject(recordClazz);
-	}
-
-	@SuppressWarnings("unchecked")
-	private void readObject(ObjectInputStream ois) throws ClassNotFoundException, IOException {
-		this.recordClazz = (Class<? extends SpecificRecord>) ois.readObject();
-		this.schema = SpecificData.get().getSchema(recordClazz);
-		this.datumReader = new SpecificDatumReader<>(schema);
-		this.record = (SpecificRecord) SpecificData.newInstance(recordClazz, schema);
-		this.inputStream = new MutableByteArrayInputStream();
-		this.decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
-	}
-
-	/**
-	 * Converts a (nested) Avro {@link SpecificRecord} into Flink's Row type.
-	 * Avro's {@link Utf8} fields are converted into regular Java strings.
-	 */
-	private static Object convertToRow(Schema schema, Object recordObj) {
-		if (recordObj instanceof GenericRecord) {
-			// records can be wrapped in a union
-			if (schema.getType() == Schema.Type.UNION) {
-				final List<Schema> types = schema.getTypes();
-				if (types.size() == 2 && types.get(0).getType() == Schema.Type.NULL && types.get(1).getType() == Schema.Type.RECORD) {
-					schema = types.get(1);
-				}
-				else {
-					throw new RuntimeException("Currently we only support schemas of the following form: UNION[null, RECORD]. Given: " + schema);
-				}
-			} else if (schema.getType() != Schema.Type.RECORD) {
-				throw new RuntimeException("Record type for row type expected. But is: " + schema);
-			}
-			final List<Schema.Field> fields = schema.getFields();
-			final Row row = new Row(fields.size());
-			final GenericRecord record = (GenericRecord) recordObj;
-			for (int i = 0; i < fields.size(); i++) {
-				final Schema.Field field = fields.get(i);
-				row.setField(i, convertToRow(field.schema(), record.get(field.pos())));
-			}
-			return row;
-		} else if (recordObj instanceof Utf8) {
-			return recordObj.toString();
-		} else {
-			return recordObj;
-		}
-	}
-
-	/**
-	 * An extension of the ByteArrayInputStream that allows to change a buffer that should be
-	 * read without creating a new ByteArrayInputStream instance. This allows to re-use the same
-	 * InputStream instance, copying message to process, and creation of Decoder on every new message.
-	 */
-	private static final class MutableByteArrayInputStream extends ByteArrayInputStream {
-
-		public MutableByteArrayInputStream() {
-			super(new byte[0]);
-		}
-
-		/**
-		 * Set buffer that can be read via the InputStream interface and reset the input stream.
-		 * This has the same effect as creating a new ByteArrayInputStream with a new buffer.
-		 *
-		 * @param buf the new buffer to read.
-		 */
-		public void setBuffer(byte[] buf) {
-			this.buf = buf;
-			this.pos = 0;
-			this.count = buf.length;
-		}
-	}
-}


[12/21] flink git commit: [FLINK-7420] [avro] Replace GenericData.Array by dummy when reading TypeSerializers

Posted by se...@apache.org.
[FLINK-7420] [avro] Replace GenericData.Array by dummy when reading TypeSerializers

This also adds a new test that verifies that we correctly register
Avro Serializers when they are present and modifies an existing test to
verify that we correctly register dummy classes.


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

Branch: refs/heads/master
Commit: 29249b2eeb9cb9910a5a55ae6c3a0b648d67d2b5
Parents: db7c70f
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Wed Oct 25 17:38:24 2017 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Nov 3 16:40:34 2017 +0100

----------------------------------------------------------------------
 .../flink-connector-kafka-0.10/pom.xml          |   8 ++
 .../flink-connector-kafka-0.11/pom.xml          |   8 ++
 .../flink-connector-kafka-0.8/pom.xml           |   8 ++
 .../flink-connector-kafka-0.9/pom.xml           |   8 ++
 .../TypeSerializerSerializationUtil.java        |  23 +++-
 ...ryoRegistrationSerializerConfigSnapshot.java |   2 +-
 .../kryo/KryoSerializerCompatibilityTest.java   | 125 +++++++++++++++++++
 .../type-with-avro-serialized-using-kryo        |   1 +
 .../type-without-avro-serialized-using-kryo     | Bin 0 -> 31 bytes
 .../AvroKryoSerializerRegistrationsTest.java    | 117 +++++++++++++++++
 .../test/resources/flink_11-kryo_registrations  |  86 +++++++++++++
 flink-libraries/flink-cep/pom.xml               |   8 --
 ...ckendStateMetaInfoSnapshotReaderWriters.java |   4 +-
 .../misc/KryoSerializerRegistrationsTest.java   |  11 ++
 pom.xml                                         |  21 ++--
 15 files changed, 404 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/29249b2e/flink-connectors/flink-connector-kafka-0.10/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/pom.xml b/flink-connectors/flink-connector-kafka-0.10/pom.xml
index 2b6660d..3357591 100644
--- a/flink-connectors/flink-connector-kafka-0.10/pom.xml
+++ b/flink-connectors/flink-connector-kafka-0.10/pom.xml
@@ -95,6 +95,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-avro_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>

http://git-wip-us.apache.org/repos/asf/flink/blob/29249b2e/flink-connectors/flink-connector-kafka-0.11/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.11/pom.xml b/flink-connectors/flink-connector-kafka-0.11/pom.xml
index 162d5d0..4f6be1d 100644
--- a/flink-connectors/flink-connector-kafka-0.11/pom.xml
+++ b/flink-connectors/flink-connector-kafka-0.11/pom.xml
@@ -104,6 +104,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-avro_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>

http://git-wip-us.apache.org/repos/asf/flink/blob/29249b2e/flink-connectors/flink-connector-kafka-0.8/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/pom.xml b/flink-connectors/flink-connector-kafka-0.8/pom.xml
index c990188..b96274a 100644
--- a/flink-connectors/flink-connector-kafka-0.8/pom.xml
+++ b/flink-connectors/flink-connector-kafka-0.8/pom.xml
@@ -83,6 +83,14 @@ under the License.
 		</dependency>
 
 		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-avro_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+
+		<dependency>
 			<groupId>org.apache.kafka</groupId>
 			<artifactId>kafka_${scala.binary.version}</artifactId>
 			<version>${kafka.version}</version>

http://git-wip-us.apache.org/repos/asf/flink/blob/29249b2e/flink-connectors/flink-connector-kafka-0.9/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/pom.xml b/flink-connectors/flink-connector-kafka-0.9/pom.xml
index 819d590..c711c5f 100644
--- a/flink-connectors/flink-connector-kafka-0.9/pom.xml
+++ b/flink-connectors/flink-connector-kafka-0.9/pom.xml
@@ -91,6 +91,14 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-avro_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>

http://git-wip-us.apache.org/repos/asf/flink/blob/29249b2e/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationUtil.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationUtil.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationUtil.java
index 058ef46..d03498a 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationUtil.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationUtil.java
@@ -20,6 +20,7 @@ package org.apache.flink.api.common.typeutils;
 
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.runtime.KryoRegistrationSerializerConfigSnapshot;
 import org.apache.flink.core.io.VersionedIOReadableWritable;
 import org.apache.flink.core.memory.ByteArrayInputStreamWithPos;
 import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos;
@@ -74,7 +75,9 @@ public class TypeSerializerSerializationUtil {
 
 	/**
 	 * An {@link ObjectInputStream} that ignores serialVersionUID mismatches when deserializing objects of
-	 * anonymous classes or our Scala serializer classes.
+	 * anonymous classes or our Scala serializer classes and also replaces occurences of GenericData.Array
+	 * (from Avro) by a dummy class so that the KryoSerializer can still be deserialized without
+	 * Avro being on the classpath.
 	 *
 	 * <p>The {@link TypeSerializerSerializationProxy} uses this specific object input stream to read serializers,
 	 * so that mismatching serialVersionUIDs of anonymous classes / Scala serializers are ignored.
@@ -83,9 +86,9 @@ public class TypeSerializerSerializationUtil {
 	 *
 	 * @see <a href="https://issues.apache.org/jira/browse/FLINK-6869">FLINK-6869</a>
 	 */
-	public static class SerialUIDMismatchTolerantInputStream extends InstantiationUtil.ClassLoaderObjectInputStream {
+	public static class FailureTolerantObjectInputStream extends InstantiationUtil.ClassLoaderObjectInputStream {
 
-		public SerialUIDMismatchTolerantInputStream(InputStream in, ClassLoader cl) throws IOException {
+		public FailureTolerantObjectInputStream(InputStream in, ClassLoader cl) throws IOException {
 			super(in, cl);
 		}
 
@@ -93,6 +96,16 @@ public class TypeSerializerSerializationUtil {
 		protected ObjectStreamClass readClassDescriptor() throws IOException, ClassNotFoundException {
 			ObjectStreamClass streamClassDescriptor = super.readClassDescriptor();
 
+			try {
+				Class.forName(streamClassDescriptor.getName(), false, classLoader);
+			} catch (ClassNotFoundException e) {
+				if (streamClassDescriptor.getName().equals("org.apache.avro.generic.GenericData$Array")) {
+					ObjectStreamClass result = ObjectStreamClass.lookup(
+						KryoRegistrationSerializerConfigSnapshot.DummyRegisteredClass.class);
+					return result;
+				}
+			}
+
 			Class localClass = resolveClass(streamClassDescriptor);
 			if (scalaSerializerClassnames.contains(localClass.getName()) || localClass.isAnonymousClass()
 				// isAnonymousClass does not work for anonymous Scala classes; additionally check by classname
@@ -433,8 +446,8 @@ public class TypeSerializerSerializationUtil {
 
 			ClassLoader previousClassLoader = Thread.currentThread().getContextClassLoader();
 			try (
-				SerialUIDMismatchTolerantInputStream ois =
-					new SerialUIDMismatchTolerantInputStream(new ByteArrayInputStream(buffer), userClassLoader)) {
+				FailureTolerantObjectInputStream ois =
+					new FailureTolerantObjectInputStream(new ByteArrayInputStream(buffer), userClassLoader)) {
 
 				Thread.currentThread().setContextClassLoader(userClassLoader);
 				typeSerializer = (TypeSerializer<T>) ois.readObject();

http://git-wip-us.apache.org/repos/asf/flink/blob/29249b2e/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoRegistrationSerializerConfigSnapshot.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoRegistrationSerializerConfigSnapshot.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoRegistrationSerializerConfigSnapshot.java
index 14287ca..cdf6b23 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoRegistrationSerializerConfigSnapshot.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoRegistrationSerializerConfigSnapshot.java
@@ -217,7 +217,7 @@ public abstract class KryoRegistrationSerializerConfigSnapshot<T> extends Generi
 	/**
 	 * Placeholder dummy for a previously registered class that can no longer be found in classpath on restore.
 	 */
-	public static class DummyRegisteredClass {}
+	public static class DummyRegisteredClass implements Serializable {}
 
 	/**
 	 * Placeholder dummmy for a previously registered Kryo serializer that is no longer valid or in classpath on restore.

http://git-wip-us.apache.org/repos/asf/flink/blob/29249b2e/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerCompatibilityTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerCompatibilityTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerCompatibilityTest.java
index 1cacc9e..11c95f1 100644
--- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerCompatibilityTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerCompatibilityTest.java
@@ -29,14 +29,20 @@ import com.esotericsoftware.kryo.Kryo;
 import com.esotericsoftware.kryo.Serializer;
 import com.esotericsoftware.kryo.io.Input;
 import com.esotericsoftware.kryo.io.Output;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
+import java.io.FileInputStream;
 import java.io.InputStream;
+import java.util.List;
 
+import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 /**
@@ -44,6 +50,9 @@ import static org.junit.Assert.assertTrue;
  */
 public class KryoSerializerCompatibilityTest {
 
+	@Rule
+	public ExpectedException thrown = ExpectedException.none();
+
 	@Test
 	public void testMigrationStrategyForRemovedAvroDependency() throws Exception {
 		KryoSerializer<TestClass> kryoSerializerForA = new KryoSerializer<>(TestClass.class, new ExecutionConfig());
@@ -85,6 +94,122 @@ public class KryoSerializerCompatibilityTest {
 		assertTrue(compatResult.isRequiresMigration());
 	}
 
+	@Test
+	public void testMigrationOfTypeWithAvroType() throws Exception {
+
+		/*
+		 When Avro sees the schema "{"type" : "array", "items" : "boolean"}" it will create a field
+		 of type List<Integer> but the actual type will be GenericData.Array<Integer>. The
+		 KryoSerializer registers a special Serializer for this type that simply deserializes
+		 as ArrayList because Kryo cannot handle GenericData.Array well. Before Flink 1.4 Avro
+		 was always in the classpath but after 1.4 it's only present if the flink-avro jar is
+		 included. This test verifies that we can still deserialize data written pre-1.4.
+		 */
+		class FakeAvroClass {
+			public List<Integer> array;
+
+			FakeAvroClass(List<Integer> array) {
+				this.array = array;
+			}
+		}
+
+		/*
+		// This has to be executed on a pre-1.4 branch to generate the binary blob
+		{
+			ExecutionConfig executionConfig = new ExecutionConfig();
+			KryoSerializer<FakeAvroClass> kryoSerializer =
+				new KryoSerializer<>(FakeAvroClass.class, executionConfig);
+
+			try (
+				FileOutputStream f = new FileOutputStream(
+					"src/test/resources/type-with-avro-serialized-using-kryo");
+				DataOutputViewStreamWrapper outputView = new DataOutputViewStreamWrapper(f)) {
+
+
+				GenericData.Array<Integer> array =
+					new GenericData.Array<>(10, Schema.createArray(Schema.create(Schema.Type.INT)));
+
+				array.add(10);
+				array.add(20);
+				array.add(30);
+
+				FakeAvroClass myTestClass = new FakeAvroClass(array);
+
+				kryoSerializer.serialize(myTestClass, outputView);
+			}
+		}
+		*/
+
+		{
+			ExecutionConfig executionConfig = new ExecutionConfig();
+			KryoSerializer<FakeAvroClass> kryoSerializer =
+				new KryoSerializer<>(FakeAvroClass.class, executionConfig);
+
+			try (
+				FileInputStream f = new FileInputStream("src/test/resources/type-with-avro-serialized-using-kryo");
+				DataInputViewStreamWrapper inputView = new DataInputViewStreamWrapper(f)) {
+
+				thrown.expectMessage("Could not find required Avro dependency");
+				FakeAvroClass myTestClass = kryoSerializer.deserialize(inputView);
+			}
+		}
+	}
+
+	@Test
+	public void testMigrationWithTypeDevoidOfAvroTypes() throws Exception {
+
+		class FakeClass {
+			public List<Integer> array;
+
+			FakeClass(List<Integer> array) {
+				this.array = array;
+			}
+		}
+
+		/*
+		// This has to be executed on a pre-1.4 branch to generate the binary blob
+		{
+			ExecutionConfig executionConfig = new ExecutionConfig();
+			KryoSerializer<FakeClass> kryoSerializer =
+				new KryoSerializer<>(FakeClass.class, executionConfig);
+
+			try (
+				FileOutputStream f = new FileOutputStream(
+					"src/test/resources/type-without-avro-serialized-using-kryo");
+				DataOutputViewStreamWrapper outputView = new DataOutputViewStreamWrapper(f)) {
+
+
+				List<Integer> array = new ArrayList<>(10);
+
+				array.add(10);
+				array.add(20);
+				array.add(30);
+
+				FakeClass myTestClass = new FakeClass(array);
+
+				kryoSerializer.serialize(myTestClass, outputView);
+			}
+		}
+		*/
+
+		{
+			ExecutionConfig executionConfig = new ExecutionConfig();
+			KryoSerializer<FakeClass> kryoSerializer =
+				new KryoSerializer<>(FakeClass.class, executionConfig);
+
+			try (
+				FileInputStream f = new FileInputStream("src/test/resources/type-without-avro-serialized-using-kryo");
+				DataInputViewStreamWrapper inputView = new DataInputViewStreamWrapper(f)) {
+
+				FakeClass myTestClass = kryoSerializer.deserialize(inputView);
+
+				assertThat(myTestClass.array.get(0), is(10));
+				assertThat(myTestClass.array.get(1), is(20));
+				assertThat(myTestClass.array.get(2), is(30));
+			}
+		}
+	}
+
 	/**
 	 * Tests that after reconfiguration, registration ids are reconfigured to
 	 * remain the same as the preceding KryoSerializer.

http://git-wip-us.apache.org/repos/asf/flink/blob/29249b2e/flink-core/src/test/resources/type-with-avro-serialized-using-kryo
----------------------------------------------------------------------
diff --git a/flink-core/src/test/resources/type-with-avro-serialized-using-kryo b/flink-core/src/test/resources/type-with-avro-serialized-using-kryo
new file mode 100644
index 0000000..3901024
--- /dev/null
+++ b/flink-core/src/test/resources/type-with-avro-serialized-using-kryo
@@ -0,0 +1 @@
+
(<
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/29249b2e/flink-core/src/test/resources/type-without-avro-serialized-using-kryo
----------------------------------------------------------------------
diff --git a/flink-core/src/test/resources/type-without-avro-serialized-using-kryo b/flink-core/src/test/resources/type-without-avro-serialized-using-kryo
new file mode 100644
index 0000000..d95094c
Binary files /dev/null and b/flink-core/src/test/resources/type-without-avro-serialized-using-kryo differ

http://git-wip-us.apache.org/repos/asf/flink/blob/29249b2e/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroKryoSerializerRegistrationsTest.java
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroKryoSerializerRegistrationsTest.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroKryoSerializerRegistrationsTest.java
new file mode 100644
index 0000000..060cfdd
--- /dev/null
+++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroKryoSerializerRegistrationsTest.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.formats.avro;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Registration;
+import org.junit.Test;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests that the set of Kryo registrations is the same across compatible
+ * Flink versions.
+ *
+ * <p>Special version of {@code KryoSerializerRegistrationsTest} that sits in the Avro module
+ * and verifies that we correctly register Avro types at the {@link KryoSerializer} when
+ * Avro is present.
+ */
+public class AvroKryoSerializerRegistrationsTest {
+
+	/**
+	 * Tests that the registered classes in Kryo did not change.
+	 *
+	 * <p>Once we have proper serializer versioning this test will become obsolete.
+	 * But currently a change in the serializers can break savepoint backwards
+	 * compatibility between Flink versions.
+	 */
+	@Test
+	public void testDefaultKryoRegisteredClassesDidNotChange() throws Exception {
+		final Kryo kryo = new KryoSerializer<>(Integer.class, new ExecutionConfig()).getKryo();
+
+		try (BufferedReader reader = new BufferedReader(new InputStreamReader(
+				getClass().getClassLoader().getResourceAsStream("flink_11-kryo_registrations")))) {
+
+			String line;
+			while ((line = reader.readLine()) != null) {
+				String[] split = line.split(",");
+				final int tag = Integer.parseInt(split[0]);
+				final String registeredClass = split[1];
+
+				Registration registration = kryo.getRegistration(tag);
+
+				if (registration == null) {
+					fail(String.format("Registration for %d = %s got lost", tag, registeredClass));
+				}
+				else if (!registeredClass.equals(registration.getType().getName())) {
+					fail(String.format("Registration for %d = %s changed to %s",
+							tag, registeredClass, registration.getType().getName()));
+				}
+			}
+		}
+	}
+
+	/**
+	 * Creates a Kryo serializer and writes the default registrations out to a
+	 * comma separated file with one entry per line:
+	 *
+	 * <pre>
+	 * id,class
+	 * </pre>
+	 *
+	 * <p>The produced file is used to check that the registered IDs don't change
+	 * in future Flink versions.
+	 *
+	 * <p>This method is not used in the tests, but documents how the test file
+	 * has been created and can be used to re-create it if needed.
+	 *
+	 * @param filePath File path to write registrations to
+	 */
+	private void writeDefaultKryoRegistrations(String filePath) throws IOException {
+		final File file = new File(filePath);
+		if (file.exists()) {
+			assertTrue(file.delete());
+		}
+
+		final Kryo kryo = new KryoSerializer<>(Integer.class, new ExecutionConfig()).getKryo();
+		final int nextId = kryo.getNextRegistrationId();
+
+		try (BufferedWriter writer = new BufferedWriter(new FileWriter(file))) {
+			for (int i = 0; i < nextId; i++) {
+				Registration registration = kryo.getRegistration(i);
+				String str = registration.getId() + "," + registration.getType().getName();
+				writer.write(str, 0, str.length());
+				writer.newLine();
+			}
+
+			System.out.println("Created file with registrations at " + file.getAbsolutePath());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/29249b2e/flink-formats/flink-avro/src/test/resources/flink_11-kryo_registrations
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/test/resources/flink_11-kryo_registrations b/flink-formats/flink-avro/src/test/resources/flink_11-kryo_registrations
new file mode 100644
index 0000000..7000e62
--- /dev/null
+++ b/flink-formats/flink-avro/src/test/resources/flink_11-kryo_registrations
@@ -0,0 +1,86 @@
+0,int
+1,java.lang.String
+2,float
+3,boolean
+4,byte
+5,char
+6,short
+7,long
+8,double
+9,void
+10,scala.collection.convert.Wrappers$SeqWrapper
+11,scala.collection.convert.Wrappers$IteratorWrapper
+12,scala.collection.convert.Wrappers$MapWrapper
+13,scala.collection.convert.Wrappers$JListWrapper
+14,scala.collection.convert.Wrappers$JMapWrapper
+15,scala.Some
+16,scala.util.Left
+17,scala.util.Right
+18,scala.collection.immutable.Vector
+19,scala.collection.immutable.Set$Set1
+20,scala.collection.immutable.Set$Set2
+21,scala.collection.immutable.Set$Set3
+22,scala.collection.immutable.Set$Set4
+23,scala.collection.immutable.HashSet$HashTrieSet
+24,scala.collection.immutable.Map$Map1
+25,scala.collection.immutable.Map$Map2
+26,scala.collection.immutable.Map$Map3
+27,scala.collection.immutable.Map$Map4
+28,scala.collection.immutable.HashMap$HashTrieMap
+29,scala.collection.immutable.Range$Inclusive
+30,scala.collection.immutable.NumericRange$Inclusive
+31,scala.collection.immutable.NumericRange$Exclusive
+32,scala.collection.mutable.BitSet
+33,scala.collection.mutable.HashMap
+34,scala.collection.mutable.HashSet
+35,scala.collection.convert.Wrappers$IterableWrapper
+36,scala.Tuple1
+37,scala.Tuple2
+38,scala.Tuple3
+39,scala.Tuple4
+40,scala.Tuple5
+41,scala.Tuple6
+42,scala.Tuple7
+43,scala.Tuple8
+44,scala.Tuple9
+45,scala.Tuple10
+46,scala.Tuple11
+47,scala.Tuple12
+48,scala.Tuple13
+49,scala.Tuple14
+50,scala.Tuple15
+51,scala.Tuple16
+52,scala.Tuple17
+53,scala.Tuple18
+54,scala.Tuple19
+55,scala.Tuple20
+56,scala.Tuple21
+57,scala.Tuple22
+58,scala.Tuple1$mcJ$sp
+59,scala.Tuple1$mcI$sp
+60,scala.Tuple1$mcD$sp
+61,scala.Tuple2$mcJJ$sp
+62,scala.Tuple2$mcJI$sp
+63,scala.Tuple2$mcJD$sp
+64,scala.Tuple2$mcIJ$sp
+65,scala.Tuple2$mcII$sp
+66,scala.Tuple2$mcID$sp
+67,scala.Tuple2$mcDJ$sp
+68,scala.Tuple2$mcDI$sp
+69,scala.Tuple2$mcDD$sp
+70,scala.Symbol
+71,scala.reflect.ClassTag
+72,scala.runtime.BoxedUnit
+73,java.util.Arrays$ArrayList
+74,java.util.BitSet
+75,java.util.PriorityQueue
+76,java.util.regex.Pattern
+77,java.sql.Date
+78,java.sql.Time
+79,java.sql.Timestamp
+80,java.net.URI
+81,java.net.InetSocketAddress
+82,java.util.UUID
+83,java.util.Locale
+84,java.text.SimpleDateFormat
+85,org.apache.avro.generic.GenericData$Array

http://git-wip-us.apache.org/repos/asf/flink/blob/29249b2e/flink-libraries/flink-cep/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/pom.xml b/flink-libraries/flink-cep/pom.xml
index bd57d17..a561cca 100644
--- a/flink-libraries/flink-cep/pom.xml
+++ b/flink-libraries/flink-cep/pom.xml
@@ -88,14 +88,6 @@ under the License.
             <version>${project.version}</version>
             <scope>test</scope>
         </dependency>
-
-        <!-- we include Avro to make the CEPMigrationTest work, it uses a Kryo-serialized savepoint (see FLINK-7420) -->
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-avro_${scala.binary.version}</artifactId>
-            <version>${project.version}</version>
-            <scope>test</scope>
-        </dependency>
     </dependencies>
     
     <build>

http://git-wip-us.apache.org/repos/asf/flink/blob/29249b2e/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorBackendStateMetaInfoSnapshotReaderWriters.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorBackendStateMetaInfoSnapshotReaderWriters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorBackendStateMetaInfoSnapshotReaderWriters.java
index dc322c3..c333397 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorBackendStateMetaInfoSnapshotReaderWriters.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorBackendStateMetaInfoSnapshotReaderWriters.java
@@ -168,8 +168,8 @@ public class OperatorBackendStateMetaInfoSnapshotReaderWriters {
 			DataInputViewStream dis = new DataInputViewStream(in);
 			ClassLoader previousClassLoader = Thread.currentThread().getContextClassLoader();
 			try (
-				TypeSerializerSerializationUtil.SerialUIDMismatchTolerantInputStream ois =
-					new TypeSerializerSerializationUtil.SerialUIDMismatchTolerantInputStream(dis, userCodeClassLoader)) {
+				TypeSerializerSerializationUtil.FailureTolerantObjectInputStream ois =
+					new TypeSerializerSerializationUtil.FailureTolerantObjectInputStream(dis, userCodeClassLoader)) {
 
 				Thread.currentThread().setContextClassLoader(userCodeClassLoader);
 				TypeSerializer<S> stateSerializer = (TypeSerializer<S>) ois.readObject();

http://git-wip-us.apache.org/repos/asf/flink/blob/29249b2e/flink-runtime/src/test/java/org/apache/flink/runtime/misc/KryoSerializerRegistrationsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/misc/KryoSerializerRegistrationsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/misc/KryoSerializerRegistrationsTest.java
index 77d2a1a..cbe9394 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/misc/KryoSerializerRegistrationsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/misc/KryoSerializerRegistrationsTest.java
@@ -33,6 +33,8 @@ import java.io.FileWriter;
 import java.io.IOException;
 import java.io.InputStreamReader;
 
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -72,6 +74,15 @@ public class KryoSerializerRegistrationsTest {
 				if (registration == null) {
 					fail(String.format("Registration for %d = %s got lost", tag, registeredClass));
 				}
+				else if (registeredClass.equals("org.apache.avro.generic.GenericData$Array")) {
+					// starting with Flink 1.4 Avro is no longer a dependency of core. Avro is
+					// only available if flink-avro is present. There is a special version of
+					// this test in AvroKryoSerializerRegistrationsTest that verifies correct
+					// registration of Avro types if present
+					assertThat(
+						registration.getType().getName(),
+						is("org.apache.flink.api.java.typeutils.runtime.kryo.Serializers$DummyAvroRegisteredClass"));
+				}
 				else if (!registeredClass.equals(registration.getType().getName())) {
 					fail(String.format("Registration for %d = %s changed to %s",
 							tag, registeredClass, registration.getType().getName()));

http://git-wip-us.apache.org/repos/asf/flink/blob/29249b2e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 773dc34..b93251b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -215,11 +215,11 @@ under the License.
 	</dependencies>
 
 	<!-- this section defines the module versions that are used if nothing else is specified. -->
-	
+
 	<dependencyManagement>
-		<!-- WARN: 
-			DO NOT put 	guava, 
-						protobuf, 
+		<!-- WARN:
+			DO NOT put 	guava,
+						protobuf,
 						asm,
 						netty
 					here. It will overwrite Hadoop's guava dependency (even though we handle it
@@ -367,7 +367,7 @@ under the License.
 				<artifactId>joda-convert</artifactId>
 				<version>1.7</version>
 			</dependency>
-			
+
 			<!-- kryo used in different versions by Flink an chill -->
 			<dependency>
 				<groupId>com.esotericsoftware.kryo</groupId>
@@ -579,7 +579,7 @@ under the License.
 									<outputDir>${project.build.directory}/spotbugs</outputDir>
 									<!-- A list of available stylesheets can be found here: https://github.com/findbugsproject/findbugs/tree/master/findbugs/src/xsl -->
 									<stylesheet>plain.xsl</stylesheet>
-									
+
 									<fileMappers>
 										<fileMapper
 											implementation="org.codehaus.plexus.components.io.filemappers.FileExtensionMapper">
@@ -772,7 +772,7 @@ under the License.
 				</plugins>
 			</build>
 		</profile>
-		
+
 		<profile>
 			<!--japicmp 0.7 does not support deactivation from the command
 				line, so we have to use a workaround with profiles instead.
@@ -842,7 +842,7 @@ under the License.
 				</dependency>
 			</dependencies>
 		</profile>
-		
+
 		<profile>
 			<id>release</id>
 			<properties>
@@ -1027,6 +1027,7 @@ under the License.
 
 						<!-- Test Data. -->
 						<exclude>flink-tests/src/test/resources/testdata/terainput.txt</exclude>
+                        <exclude>flink-formats/flink-avro/src/test/resources/flink_11-kryo_registrations</exclude>
 						<exclude>flink-runtime/src/test/resources/flink_11-kryo_registrations</exclude>
 						<exclude>flink-core/src/test/resources/kryo-serializer-config-snapshot-v1</exclude>
 						<exclude>flink-formats/flink-avro/src/test/resources/avro/*.avsc</exclude>
@@ -1287,7 +1288,7 @@ under the License.
 			</plugin>
 		</plugins>
 
-		<!-- Plugin configurations for plugins activated in sub-projects --> 
+		<!-- Plugin configurations for plugins activated in sub-projects -->
 
 		<pluginManagement>
 			<plugins>
@@ -1310,7 +1311,7 @@ under the License.
 					<artifactId>maven-shade-plugin</artifactId>
 					<version>2.4.1</version>
 				</plugin>
-				
+
 				<!-- Disable certain plugins in Eclipse -->
 				<plugin>
 					<groupId>org.eclipse.m2e</groupId>


[19/21] flink git commit: [hotfix] [avro] Minor XML formatting cleanup

Posted by se...@apache.org.
[hotfix] [avro] Minor XML formatting cleanup


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

Branch: refs/heads/master
Commit: ad1c6075d29af1a3ded945efac5e252ab3aae32d
Parents: b4dead9
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Nov 2 15:44:04 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Nov 3 16:40:35 2017 +0100

----------------------------------------------------------------------
 flink-formats/flink-avro/src/test/assembly/test-assembly.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/ad1c6075/flink-formats/flink-avro/src/test/assembly/test-assembly.xml
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/test/assembly/test-assembly.xml b/flink-formats/flink-avro/src/test/assembly/test-assembly.xml
index 8361693..089acc4 100644
--- a/flink-formats/flink-avro/src/test/assembly/test-assembly.xml
+++ b/flink-formats/flink-avro/src/test/assembly/test-assembly.xml
@@ -26,7 +26,7 @@ under the License.
 	<fileSets>
 		<fileSet>
 			<directory>${project.build.testOutputDirectory}</directory>
-			<outputDirectory></outputDirectory>
+			<outputDirectory/>
 			<!--modify/add include to match your package(s) -->
 			<includes>
 				<include>org/apache/flink/formats/avro/testjar/**</include>


[09/21] flink git commit: [FLINK-7420] [avro] Move all Avro code to flink-avro

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowSerializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowSerializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowSerializationSchema.java
deleted file mode 100644
index 6f03b12..0000000
--- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowSerializationSchema.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.util.serialization;
-
-import org.apache.flink.api.common.serialization.SerializationSchema;
-import org.apache.flink.types.Row;
-import org.apache.flink.util.Preconditions;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.io.DatumWriter;
-import org.apache.avro.io.Encoder;
-import org.apache.avro.io.EncoderFactory;
-import org.apache.avro.specific.SpecificData;
-import org.apache.avro.specific.SpecificDatumWriter;
-import org.apache.avro.specific.SpecificRecord;
-import org.apache.avro.util.Utf8;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.util.List;
-
-/**
- * Serialization schema that serializes {@link Row} over {@link SpecificRecord} into a Avro bytes.
- */
-public class AvroRowSerializationSchema implements SerializationSchema<Row> {
-
-	/**
-	 * Avro record class.
-	 */
-	private Class<? extends SpecificRecord> recordClazz;
-
-	/**
-	 * Avro serialization schema.
-	 */
-	private transient Schema schema;
-
-	/**
-	 * Writer to serialize Avro record into a byte array.
-	 */
-	private transient DatumWriter<GenericRecord> datumWriter;
-
-	/**
-	 * Output stream to serialize records into byte array.
-	 */
-	private transient ByteArrayOutputStream arrayOutputStream = new ByteArrayOutputStream();
-
-	/**
-	 * Low-level class for serialization of Avro values.
-	 */
-	private transient Encoder encoder = EncoderFactory.get().binaryEncoder(arrayOutputStream, null);
-
-	/**
-	 * Creates a Avro serialization schema for the given schema.
-	 *
-	 * @param recordClazz Avro record class used to deserialize Avro's record to Flink's row
-	 */
-	public AvroRowSerializationSchema(Class<? extends SpecificRecord> recordClazz) {
-		Preconditions.checkNotNull(recordClazz, "Avro record class must not be null.");
-		this.recordClazz = recordClazz;
-		this.schema = SpecificData.get().getSchema(recordClazz);
-		this.datumWriter = new SpecificDatumWriter<>(schema);
-	}
-
-	@Override
-	@SuppressWarnings("unchecked")
-	public byte[] serialize(Row row) {
-		// convert to record
-		final Object record = convertToRecord(schema, row);
-
-		// write
-		try {
-			arrayOutputStream.reset();
-			datumWriter.write((GenericRecord) record, encoder);
-			encoder.flush();
-			return arrayOutputStream.toByteArray();
-		} catch (IOException e) {
-			throw new RuntimeException("Failed to serialize Row.", e);
-		}
-	}
-
-	private void writeObject(ObjectOutputStream oos) throws IOException {
-		oos.writeObject(recordClazz);
-	}
-
-	@SuppressWarnings("unchecked")
-	private void readObject(ObjectInputStream ois) throws ClassNotFoundException, IOException {
-		this.recordClazz = (Class<? extends SpecificRecord>) ois.readObject();
-		this.schema = SpecificData.get().getSchema(recordClazz);
-		this.datumWriter = new SpecificDatumWriter<>(schema);
-		this.arrayOutputStream = new ByteArrayOutputStream();
-		this.encoder = EncoderFactory.get().binaryEncoder(arrayOutputStream, null);
-	}
-
-	/**
-	 * Converts a (nested) Flink Row into Avro's {@link GenericRecord}.
-	 * Strings are converted into Avro's {@link Utf8} fields.
-	 */
-	private static Object convertToRecord(Schema schema, Object rowObj) {
-		if (rowObj instanceof Row) {
-			// records can be wrapped in a union
-			if (schema.getType() == Schema.Type.UNION) {
-				final List<Schema> types = schema.getTypes();
-				if (types.size() == 2 && types.get(0).getType() == Schema.Type.NULL && types.get(1).getType() == Schema.Type.RECORD) {
-					schema = types.get(1);
-				}
-				else if (types.size() == 2 && types.get(0).getType() == Schema.Type.RECORD && types.get(1).getType() == Schema.Type.NULL) {
-					schema = types.get(0);
-				}
-				else {
-					throw new RuntimeException("Currently we only support schemas of the following form: UNION[null, RECORD] or UNION[RECORD, NULL] Given: " + schema);
-				}
-			} else if (schema.getType() != Schema.Type.RECORD) {
-				throw new RuntimeException("Record type for row type expected. But is: " + schema);
-			}
-			final List<Schema.Field> fields = schema.getFields();
-			final GenericRecord record = new GenericData.Record(schema);
-			final Row row = (Row) rowObj;
-			for (int i = 0; i < fields.size(); i++) {
-				final Schema.Field field = fields.get(i);
-				record.put(field.pos(), convertToRecord(field.schema(), row.getField(i)));
-			}
-			return record;
-		} else if (rowObj instanceof String) {
-			return new Utf8((String) rowObj);
-		} else {
-			return rowObj;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/AvroRowDeSerializationSchemaTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/AvroRowDeSerializationSchemaTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/AvroRowDeSerializationSchemaTest.java
deleted file mode 100644
index 28f2ed3..0000000
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/AvroRowDeSerializationSchemaTest.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka;
-
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.streaming.connectors.kafka.testutils.AvroTestUtils;
-import org.apache.flink.streaming.util.serialization.AvroRowDeserializationSchema;
-import org.apache.flink.streaming.util.serialization.AvroRowSerializationSchema;
-import org.apache.flink.types.Row;
-import org.apache.flink.util.InstantiationUtil;
-
-import org.apache.avro.specific.SpecificRecord;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Test for the Avro serialization and deserialization schema.
- */
-public class AvroRowDeSerializationSchemaTest {
-
-	@Test
-	public void testSerializeDeserializeSimpleRow() throws IOException {
-		final Tuple3<Class<? extends SpecificRecord>, SpecificRecord, Row> testData = AvroTestUtils.getSimpleTestData();
-
-		final AvroRowSerializationSchema serializationSchema = new AvroRowSerializationSchema(testData.f0);
-		final AvroRowDeserializationSchema deserializationSchema = new AvroRowDeserializationSchema(testData.f0);
-
-		final byte[] bytes = serializationSchema.serialize(testData.f2);
-		final Row actual = deserializationSchema.deserialize(bytes);
-
-		assertEquals(testData.f2, actual);
-	}
-
-	@Test
-	public void testSerializeSimpleRowSeveralTimes() throws IOException {
-		final Tuple3<Class<? extends SpecificRecord>, SpecificRecord, Row> testData = AvroTestUtils.getSimpleTestData();
-
-		final AvroRowSerializationSchema serializationSchema = new AvroRowSerializationSchema(testData.f0);
-		final AvroRowDeserializationSchema deserializationSchema = new AvroRowDeserializationSchema(testData.f0);
-
-		serializationSchema.serialize(testData.f2);
-		serializationSchema.serialize(testData.f2);
-		final byte[] bytes = serializationSchema.serialize(testData.f2);
-		final Row actual = deserializationSchema.deserialize(bytes);
-
-		assertEquals(testData.f2, actual);
-	}
-
-	@Test
-	public void testDeserializeRowSeveralTimes() throws IOException {
-		final Tuple3<Class<? extends SpecificRecord>, SpecificRecord, Row> testData = AvroTestUtils.getSimpleTestData();
-
-		final AvroRowSerializationSchema serializationSchema = new AvroRowSerializationSchema(testData.f0);
-		final AvroRowDeserializationSchema deserializationSchema = new AvroRowDeserializationSchema(testData.f0);
-
-		final byte[] bytes = serializationSchema.serialize(testData.f2);
-		deserializationSchema.deserialize(bytes);
-		deserializationSchema.deserialize(bytes);
-		final Row actual = deserializationSchema.deserialize(bytes);
-
-		assertEquals(testData.f2, actual);
-	}
-
-	@Test
-	public void testSerializeDeserializeComplexRow() throws IOException {
-		final Tuple3<Class<? extends SpecificRecord>, SpecificRecord, Row> testData = AvroTestUtils.getComplexTestData();
-
-		final AvroRowSerializationSchema serializationSchema = new AvroRowSerializationSchema(testData.f0);
-		final AvroRowDeserializationSchema deserializationSchema = new AvroRowDeserializationSchema(testData.f0);
-
-		final byte[] bytes = serializationSchema.serialize(testData.f2);
-		final Row actual = deserializationSchema.deserialize(bytes);
-
-		assertEquals(testData.f2, actual);
-	}
-
-	@Test
-	public void testSerializeComplexRowSeveralTimes() throws IOException {
-		final Tuple3<Class<? extends SpecificRecord>, SpecificRecord, Row> testData = AvroTestUtils.getComplexTestData();
-
-		final AvroRowSerializationSchema serializationSchema = new AvroRowSerializationSchema(testData.f0);
-		final AvroRowDeserializationSchema deserializationSchema = new AvroRowDeserializationSchema(testData.f0);
-
-		serializationSchema.serialize(testData.f2);
-		serializationSchema.serialize(testData.f2);
-		final byte[] bytes = serializationSchema.serialize(testData.f2);
-		final Row actual = deserializationSchema.deserialize(bytes);
-
-		assertEquals(testData.f2, actual);
-	}
-
-	@Test
-	public void testDeserializeComplexRowSeveralTimes() throws IOException {
-		final Tuple3<Class<? extends SpecificRecord>, SpecificRecord, Row> testData = AvroTestUtils.getComplexTestData();
-
-		final AvroRowSerializationSchema serializationSchema = new AvroRowSerializationSchema(testData.f0);
-		final AvroRowDeserializationSchema deserializationSchema = new AvroRowDeserializationSchema(testData.f0);
-
-		final byte[] bytes = serializationSchema.serialize(testData.f2);
-		deserializationSchema.deserialize(bytes);
-		deserializationSchema.deserialize(bytes);
-		final Row actual = deserializationSchema.deserialize(bytes);
-
-		assertEquals(testData.f2, actual);
-	}
-
-	@Test
-	public void testSerializability() throws IOException, ClassNotFoundException {
-		final Tuple3<Class<? extends SpecificRecord>, SpecificRecord, Row> testData = AvroTestUtils.getComplexTestData();
-
-		final AvroRowSerializationSchema serOrig = new AvroRowSerializationSchema(testData.f0);
-		final AvroRowDeserializationSchema deserOrig = new AvroRowDeserializationSchema(testData.f0);
-
-		byte[] serBytes = InstantiationUtil.serializeObject(serOrig);
-		byte[] deserBytes = InstantiationUtil.serializeObject(deserOrig);
-
-		AvroRowSerializationSchema serCopy =
-			InstantiationUtil.deserializeObject(serBytes, Thread.currentThread().getContextClassLoader());
-		AvroRowDeserializationSchema deserCopy =
-			InstantiationUtil.deserializeObject(deserBytes, Thread.currentThread().getContextClassLoader());
-
-		final byte[] bytes = serCopy.serialize(testData.f2);
-		deserCopy.deserialize(bytes);
-		deserCopy.deserialize(bytes);
-		final Row actual = deserCopy.deserialize(bytes);
-
-		assertEquals(testData.f2, actual);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSourceTestBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSourceTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSourceTestBase.java
index def16b2..871a6f6 100644
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSourceTestBase.java
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSourceTestBase.java
@@ -20,7 +20,7 @@ package org.apache.flink.streaming.connectors.kafka;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.typeutils.RowTypeInfo;
-import org.apache.flink.streaming.connectors.kafka.testutils.AvroTestUtils;
+import org.apache.flink.formats.avro.utils.AvroTestUtils;
 import org.apache.flink.table.api.Types;
 
 import org.apache.avro.Schema;

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/AvroTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/AvroTestUtils.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/AvroTestUtils.java
deleted file mode 100644
index a41125a..0000000
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/AvroTestUtils.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.connectors.kafka.testutils;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.io.avro.generated.Address;
-import org.apache.flink.api.io.avro.generated.Colors;
-import org.apache.flink.api.io.avro.generated.User;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.types.Row;
-
-import org.apache.avro.Schema;
-import org.apache.avro.SchemaBuilder;
-import org.apache.avro.reflect.ReflectData;
-import org.apache.avro.specific.SpecificRecord;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-
-/**
- * Utilities for creating Avro Schemas.
- */
-public final class AvroTestUtils {
-
-	private static final String NAMESPACE = "org.apache.flink.streaming.connectors.kafka";
-
-	/**
-	 * Creates a flat Avro Schema for testing.
-	 */
-	public static Schema createFlatAvroSchema(String[] fieldNames, TypeInformation[] fieldTypes) {
-		final SchemaBuilder.FieldAssembler<Schema> fieldAssembler = SchemaBuilder
-			.record("BasicAvroRecord")
-			.namespace(NAMESPACE)
-			.fields();
-
-		final Schema nullSchema = Schema.create(Schema.Type.NULL);
-
-		for (int i = 0; i < fieldNames.length; i++) {
-			Schema schema = ReflectData.get().getSchema(fieldTypes[i].getTypeClass());
-			Schema unionSchema = Schema.createUnion(Arrays.asList(nullSchema, schema));
-			fieldAssembler.name(fieldNames[i]).type(unionSchema).noDefault();
-		}
-
-		return fieldAssembler.endRecord();
-	}
-
-	/**
-	 * Tests a simple Avro data types without nesting.
-	 */
-	public static Tuple3<Class<? extends SpecificRecord>, SpecificRecord, Row> getSimpleTestData() {
-		final Address addr = Address.newBuilder()
-			.setNum(42)
-			.setStreet("Main Street 42")
-			.setCity("Test City")
-			.setState("Test State")
-			.setZip("12345")
-			.build();
-
-		final Row rowAddr = new Row(5);
-		rowAddr.setField(0, 42);
-		rowAddr.setField(1, "Main Street 42");
-		rowAddr.setField(2, "Test City");
-		rowAddr.setField(3, "Test State");
-		rowAddr.setField(4, "12345");
-
-		final Tuple3<Class<? extends SpecificRecord>, SpecificRecord, Row> t = new Tuple3<>();
-		t.f0 = Address.class;
-		t.f1 = addr;
-		t.f2 = rowAddr;
-
-		return t;
-	}
-
-	/**
-	 * Tests all Avro data types as well as nested types.
-	 */
-	public static Tuple3<Class<? extends SpecificRecord>, SpecificRecord, Row> getComplexTestData() {
-		final Address addr = Address.newBuilder()
-			.setNum(42)
-			.setStreet("Main Street 42")
-			.setCity("Test City")
-			.setState("Test State")
-			.setZip("12345")
-			.build();
-
-		final Row rowAddr = new Row(5);
-		rowAddr.setField(0, 42);
-		rowAddr.setField(1, "Main Street 42");
-		rowAddr.setField(2, "Test City");
-		rowAddr.setField(3, "Test State");
-		rowAddr.setField(4, "12345");
-
-		final User user = User.newBuilder()
-			.setName("Charlie")
-			.setFavoriteNumber(null)
-			.setFavoriteColor("blue")
-			.setTypeLongTest(1337L)
-			.setTypeDoubleTest(1.337d)
-			.setTypeNullTest(null)
-			.setTypeBoolTest(false)
-			.setTypeArrayString(new ArrayList<CharSequence>())
-			.setTypeArrayBoolean(new ArrayList<Boolean>())
-			.setTypeNullableArray(null)
-			.setTypeEnum(Colors.RED)
-			.setTypeMap(new HashMap<CharSequence, Long>())
-			.setTypeFixed(null)
-			.setTypeUnion(null)
-			.setTypeNested(addr)
-			.build();
-
-		final Row rowUser = new Row(15);
-		rowUser.setField(0, "Charlie");
-		rowUser.setField(1, null);
-		rowUser.setField(2, "blue");
-		rowUser.setField(3, 1337L);
-		rowUser.setField(4, 1.337d);
-		rowUser.setField(5, null);
-		rowUser.setField(6, false);
-		rowUser.setField(7, new ArrayList<CharSequence>());
-		rowUser.setField(8, new ArrayList<Boolean>());
-		rowUser.setField(9, null);
-		rowUser.setField(10, Colors.RED);
-		rowUser.setField(11, new HashMap<CharSequence, Long>());
-		rowUser.setField(12, null);
-		rowUser.setField(13, null);
-		rowUser.setField(14, rowAddr);
-
-		final Tuple3<Class<? extends SpecificRecord>, SpecificRecord, Row> t = new Tuple3<>();
-		t.f0 = User.class;
-		t.f1 = user;
-		t.f2 = rowUser;
-
-		return t;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-connectors/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/pom.xml b/flink-connectors/pom.xml
index 97c9f20..7468b67 100644
--- a/flink-connectors/pom.xml
+++ b/flink-connectors/pom.xml
@@ -36,7 +36,6 @@ under the License.
 	<packaging>pom</packaging>
 
 	<modules>
-		<module>flink-avro</module>
 		<module>flink-jdbc</module>
 		<module>flink-hadoop-compatibility</module>
 		<module>flink-hbase</module>

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-core/pom.xml
----------------------------------------------------------------------
diff --git a/flink-core/pom.xml b/flink-core/pom.xml
index ae3f56e..0ca742c 100644
--- a/flink-core/pom.xml
+++ b/flink-core/pom.xml
@@ -80,12 +80,6 @@ under the License.
 			<!-- managed version -->
 		</dependency>
 
-		<!-- Avro is needed for the interoperability with Avro types for serialization -->
-		<dependency>
-			<groupId>org.apache.avro</groupId>
-			<artifactId>avro</artifactId>
-		</dependency>
-
 		<!-- We explicitly depend on snappy since connectors that require it load it through the system class loader -->
 		<dependency>
 			<groupId>org.xerial.snappy</groupId>
@@ -128,7 +122,7 @@ under the License.
 			<scope>test</scope>
 		</dependency>
 
-    </dependencies>
+	</dependencies>
 
 	<profiles>
 		<profile>
@@ -209,6 +203,7 @@ under the License.
 							<exclude>org.apache.flink.core.fs.FileSystem#isFlinkSupportedScheme(java.lang.String)</exclude>
 							<exclude>org.apache.flink.core.fs.FileSystem#setDefaultScheme(org.apache.flink.configuration.Configuration)</exclude>
 							<exclude>org.apache.flink.api.java.typeutils.WritableTypeInfo</exclude>
+							<exclude>org.apache.flink.api.java.typeutils.AvroTypeInfo</exclude>
 							<!-- Breaking changes between 1.1 and 1.2.
 							We ignore these changes because these are low-level, internal runtime configuration parameters -->
 							<exclude>org.apache.flink.configuration.ConfigConstants#DEFAULT_JOB_MANAGER_MAX_ATTEMPTS_HISTORY_SIZE</exclude>

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
index fc66ccd..88d524e 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
@@ -571,16 +571,24 @@ public class ExecutionConfig implements Serializable, Archiveable<ArchivedExecut
 	}
 
 	/**
-	 * Force Flink to use the AvroSerializer for POJOs.
+	 * Forces Flink to use the Apache Avro serializer for POJOs.
+	 *
+	 * <b>Important:</b> Make sure to include the <i>flink-avro</i> module.
 	 */
 	public void enableForceAvro() {
 		forceAvro = true;
 	}
 
+	/**
+	 * Disables the Apache Avro serializer as the forced serializer for POJOs.
+	 */
 	public void disableForceAvro() {
 		forceAvro = false;
 	}
 
+	/**
+	 * Returns whether the Apache Avro is the default serializer for POJOs.
+	 */
 	public boolean isForceAvroEnabled() {
 		return forceAvro;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-core/src/main/java/org/apache/flink/api/java/typeutils/AvroTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/AvroTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/AvroTypeInfo.java
deleted file mode 100644
index 1356e53..0000000
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/AvroTypeInfo.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.flink.api.java.typeutils;
-
-import org.apache.avro.specific.SpecificRecordBase;
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.annotation.Public;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-
-import java.lang.reflect.Type;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Special type information to generate a special AvroTypeInfo for Avro POJOs (implementing SpecificRecordBase, the typed Avro POJOs)
- *
- * Proceeding: It uses a regular pojo type analysis and replaces all {@code GenericType<CharSequence>}
- *     with a {@code GenericType<avro.Utf8>}.
- * All other types used by Avro are standard Java types.
- * Only strings are represented as CharSequence fields and represented as Utf8 classes at runtime.
- * CharSequence is not comparable. To make them nicely usable with field expressions, we replace them here
- * by generic type infos containing Utf8 classes (which are comparable),
- *
- * This class is checked by the AvroPojoTest.
- * @param <T>
- */
-@Public
-public class AvroTypeInfo<T extends SpecificRecordBase> extends PojoTypeInfo<T> {
-	@PublicEvolving
-	public AvroTypeInfo(Class<T> typeClass) {
-		super(typeClass, generateFieldsFromAvroSchema(typeClass));
-	}
-
-	private static <T extends SpecificRecordBase> List<PojoField> generateFieldsFromAvroSchema(Class<T> typeClass) {
-		PojoTypeExtractor pte = new PojoTypeExtractor();
-		ArrayList<Type> typeHierarchy = new ArrayList<>();
-		typeHierarchy.add(typeClass);
-		TypeInformation ti = pte.analyzePojo(typeClass, typeHierarchy, null, null, null);
-
-		if(!(ti instanceof PojoTypeInfo)) {
-			throw new IllegalStateException("Expecting type to be a PojoTypeInfo");
-		}
-		PojoTypeInfo pti =  (PojoTypeInfo) ti;
-		List<PojoField> newFields = new ArrayList<>(pti.getTotalFields());
-
-		for(int i = 0; i < pti.getArity(); i++) {
-			PojoField f = pti.getPojoFieldAt(i);
-			TypeInformation newType = f.getTypeInformation();
-			// check if type is a CharSequence
-			if(newType instanceof GenericTypeInfo) {
-				if((newType).getTypeClass().equals(CharSequence.class)) {
-					// replace the type by a org.apache.avro.util.Utf8
-					newType = new GenericTypeInfo(org.apache.avro.util.Utf8.class);
-				}
-			}
-			PojoField newField = new PojoField(f.getField(), newType);
-			newFields.add(newField);
-		}
-		return newFields;
-	}
-
-	private static class PojoTypeExtractor extends TypeExtractor {
-		private PojoTypeExtractor() {
-			super();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-core/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java
index 8a4fbbe..b24f425 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java
@@ -27,12 +27,13 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.CompositeType;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.typeutils.runtime.AvroSerializer;
 import org.apache.flink.api.java.typeutils.runtime.PojoComparator;
 import org.apache.flink.api.java.typeutils.runtime.PojoSerializer;
 import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer;
 
+import java.lang.reflect.Constructor;
 import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Modifier;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -300,15 +301,32 @@ public class PojoTypeInfo<T> extends CompositeType<T> {
 
 	@Override
 	@PublicEvolving
+	@SuppressWarnings("unchecked")
 	public TypeSerializer<T> createSerializer(ExecutionConfig config) {
 		if(config.isForceKryoEnabled()) {
-			return new KryoSerializer<T>(getTypeClass(), config);
+			return new KryoSerializer<>(getTypeClass(), config);
 		}
+
 		if(config.isForceAvroEnabled()) {
-			return new AvroSerializer<T>(getTypeClass());
+			Class<?> clazz;
+			try {
+				clazz = Class.forName("org.apache.flink.formats.avro.typeutils.AvroSerializer");
+			} catch (ClassNotFoundException e) {
+				throw new RuntimeException("Could not load the AvroSerializer class. " +
+					"You may be missing the 'flink-avro' dependency.");
+			}
+
+			try {
+				Constructor<?> constructor = clazz.getConstructor(Class.class);
+				return (TypeSerializer<T>) constructor.newInstance(getTypeClass());
+			} catch (NoSuchMethodException | IllegalAccessException | InstantiationException e) {
+				throw new RuntimeException("Incompatible versions of the Avro classes found.");
+			} catch (InvocationTargetException e) {
+				throw new RuntimeException("Cannot create AvroSerializer.", e.getTargetException());
+			}
 		}
 
-		TypeSerializer<?>[] fieldSerializers = new TypeSerializer<?>[fields.length ];
+		TypeSerializer<?>[] fieldSerializers = new TypeSerializer<?>[fields.length];
 		Field[] reflectiveFields = new Field[fields.length];
 
 		for (int i = 0; i < fields.length; i++) {

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractionUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractionUtils.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractionUtils.java
index 41d260d..c5c2565 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractionUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractionUtils.java
@@ -287,4 +287,39 @@ public class TypeExtractionUtils {
 			((TypeVariable<?>) t1).getName().equals(((TypeVariable<?>) t2).getName()) &&
 			((TypeVariable<?>) t1).getGenericDeclaration().equals(((TypeVariable<?>) t2).getGenericDeclaration());
 	}
+
+	/**
+	 * Traverses the type hierarchy of a type up until a certain stop class is found.
+	 *
+	 * @param t type for which a hierarchy need to be created
+	 * @return type of the immediate child of the stop class
+	 */
+	public static Type getTypeHierarchy(List<Type> typeHierarchy, Type t, Class<?> stopAtClass) {
+		while (!(isClassType(t) && typeToClass(t).equals(stopAtClass))) {
+			typeHierarchy.add(t);
+			t = typeToClass(t).getGenericSuperclass();
+
+			if (t == null) {
+				break;
+			}
+		}
+		return t;
+	}
+
+	/**
+	 * Returns true if the given class has a superclass of given name.
+	 *
+	 * @param clazz class to be analyzed
+	 * @param superClassName class name of the super class
+	 */
+	public static boolean hasSuperclass(Class<?> clazz, String superClassName) {
+		List<Type> hierarchy = new ArrayList<>();
+		getTypeHierarchy(hierarchy, clazz, Object.class);
+		for (Type t : hierarchy) {
+			if (isClassType(t) && typeToClass(t).getName().equals(superClassName)) {
+				return true;
+			}
+		}
+		return false;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
index c50dfc9..1a9cecb 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.api.java.typeutils;
 
-import org.apache.avro.specific.SpecificRecordBase;
 import org.apache.commons.lang3.ClassUtils;
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.annotation.Public;
@@ -73,6 +72,8 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.flink.api.java.typeutils.TypeExtractionUtils.getTypeHierarchy;
+import static org.apache.flink.api.java.typeutils.TypeExtractionUtils.hasSuperclass;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 import static org.apache.flink.api.java.typeutils.TypeExtractionUtils.checkAndExtractLambda;
 import static org.apache.flink.api.java.typeutils.TypeExtractionUtils.getAllDeclaredMethods;
@@ -114,6 +115,10 @@ public class TypeExtractor {
 
 	private static final String HADOOP_WRITABLE_TYPEINFO_CLASS = "org.apache.flink.api.java.typeutils.WritableTypeInfo";
 
+	private static final String AVRO_SPECIFIC_RECORD_BASE_CLASS = "org.apache.avro.specific.SpecificRecordBase";
+
+	private static final String AVRO_TYPEINFO_CLASS = "org.apache.flink.formats.avro.typeutils.AvroTypeInfo";
+
 	private static final Logger LOG = LoggerFactory.getLogger(TypeExtractor.class);
 
 	public static final int[] NO_INDEX = new int[] {};
@@ -1583,24 +1588,6 @@ public class TypeExtractor {
 	}
 
 	/**
-	 * Traverses the type hierarchy of a type up until a certain stop class is found.
-	 *
-	 * @param t type for which a hierarchy need to be created
-	 * @return type of the immediate child of the stop class
-	 */
-	private static Type getTypeHierarchy(ArrayList<Type> typeHierarchy, Type t, Class<?> stopAtClass) {
-		while (!(isClassType(t) && typeToClass(t).equals(stopAtClass))) {
-			typeHierarchy.add(t);
-			t = typeToClass(t).getGenericSuperclass();
-
-			if (t == null) {
-				break;
-			}
-		}
-		return t;
-	}
-
-	/**
 	 * Traverses the type hierarchy up until a type information factory can be found.
 	 *
 	 * @param typeHierarchy hierarchy to be filled while traversing up
@@ -1806,8 +1793,8 @@ public class TypeExtractor {
 		}
 
 		// special case for POJOs generated by Avro.
-		if(SpecificRecordBase.class.isAssignableFrom(clazz)) {
-			return new AvroTypeInfo(clazz);
+		if (hasSuperclass(clazz, AVRO_SPECIFIC_RECORD_BASE_CLASS)) {
+			return createAvroTypeInfo(clazz);
 		}
 
 		if (Modifier.isInterface(clazz.getModifiers())) {
@@ -2119,7 +2106,7 @@ public class TypeExtractor {
 	private static boolean hasHadoopWritableInterface(Class<?> clazz,  HashSet<Class<?>> alreadySeen) {
 		Class<?>[] interfaces = clazz.getInterfaces();
 		for (Class<?> c : interfaces) {
-			if (c.getName().equals("org.apache.hadoop.io.Writable")) {
+			if (c.getName().equals(HADOOP_WRITABLE_CLASS)) {
 				return true;
 			}
 			else if (alreadySeen.add(c) && hasHadoopWritableInterface(c, alreadySeen)) {
@@ -2155,7 +2142,7 @@ public class TypeExtractor {
 			throw new RuntimeException("Incompatible versions of the Hadoop Compatibility classes found.");
 		}
 		catch (InvocationTargetException e) {
-			throw new RuntimeException("Cannot create Hadoop Writable Type info", e.getTargetException());
+			throw new RuntimeException("Cannot create Hadoop WritableTypeInfo.", e.getTargetException());
 		}
 	}
 
@@ -2171,7 +2158,7 @@ public class TypeExtractor {
 				// this is actually a writable type info
 				// check if the type is a writable
 				if (!(type instanceof Class && isHadoopWritable((Class<?>) type))) {
-					throw new InvalidTypesException(HADOOP_WRITABLE_CLASS + " type expected");
+					throw new InvalidTypesException(HADOOP_WRITABLE_CLASS + " type expected.");
 				}
 
 				// check writable type contents
@@ -2188,4 +2175,33 @@ public class TypeExtractor {
 			// ignore
 		}
 	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities to handle Avro's 'SpecificRecord' type via reflection
+	// ------------------------------------------------------------------------
+
+	private static <T> TypeInformation<T> createAvroTypeInfo(Class<T> clazz) {
+		Class<?> typeInfoClass;
+		try {
+			typeInfoClass = Class.forName(AVRO_TYPEINFO_CLASS, false, TypeExtractor.class.getClassLoader());
+		}
+		catch (ClassNotFoundException e) {
+			throw new RuntimeException("Could not load the TypeInformation for the class '"
+					+ AVRO_TYPEINFO_CLASS + "'. You may be missing the 'flink-avro' dependency.");
+		}
+
+		try {
+			Constructor<?> constr = typeInfoClass.getConstructor(Class.class);
+
+			@SuppressWarnings("unchecked")
+			TypeInformation<T> typeInfo = (TypeInformation<T>) constr.newInstance(clazz);
+			return typeInfo;
+		}
+		catch (NoSuchMethodException | IllegalAccessException | InstantiationException e) {
+			throw new RuntimeException("Incompatible versions of the Avro classes found.");
+		}
+		catch (InvocationTargetException e) {
+			throw new RuntimeException("Cannot create AvroTypeInfo.", e.getTargetException());
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java
deleted file mode 100644
index 565bd4d..0000000
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java
+++ /dev/null
@@ -1,332 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.typeutils.runtime;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.util.LinkedHashMap;
-import java.util.Map;
-
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.reflect.ReflectDatumReader;
-import org.apache.avro.reflect.ReflectDatumWriter;
-import org.apache.avro.util.Utf8;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.java.typeutils.runtime.kryo.Serializers;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.util.InstantiationUtil;
-
-import com.esotericsoftware.kryo.Kryo;
-import org.apache.flink.util.Preconditions;
-import org.objenesis.strategy.StdInstantiatorStrategy;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * General purpose serialization. Currently using Apache Avro's Reflect-serializers for serialization and
- * Kryo for deep object copies. We want to change this to Kryo-only.
- *
- * @param <T> The type serialized.
- */
-@Internal
-public final class AvroSerializer<T> extends TypeSerializer<T> {
-
-	private static final long serialVersionUID = 1L;
-	
-	private final Class<T> type;
-	
-	private final Class<? extends T> typeToInstantiate;
-
-	/**
-	 * Map of class tag (using classname as tag) to their Kryo registration.
-	 *
-	 * <p>This map serves as a preview of the final registration result of
-	 * the Kryo instance, taking into account registration overwrites.
-	 */
-	private LinkedHashMap<String, KryoRegistration> kryoRegistrations;
-	
-	private transient ReflectDatumWriter<T> writer;
-	private transient ReflectDatumReader<T> reader;
-	
-	private transient DataOutputEncoder encoder;
-	private transient DataInputDecoder decoder;
-	
-	private transient Kryo kryo;
-	
-	private transient T deepCopyInstance;
-	
-	// --------------------------------------------------------------------------------------------
-	
-	public AvroSerializer(Class<T> type) {
-		this(type, type);
-	}
-	
-	public AvroSerializer(Class<T> type, Class<? extends T> typeToInstantiate) {
-		this.type = checkNotNull(type);
-		this.typeToInstantiate = checkNotNull(typeToInstantiate);
-		
-		InstantiationUtil.checkForInstantiation(typeToInstantiate);
-
-		this.kryoRegistrations = buildKryoRegistrations(type);
-	}
-
-	// --------------------------------------------------------------------------------------------
-	
-	@Override
-	public boolean isImmutableType() {
-		return false;
-	}
-
-	@Override
-	public AvroSerializer<T> duplicate() {
-		return new AvroSerializer<T>(type, typeToInstantiate);
-	}
-	
-	@Override
-	public T createInstance() {
-		return InstantiationUtil.instantiate(this.typeToInstantiate);
-	}
-
-	@Override
-	public T copy(T from) {
-		checkKryoInitialized();
-
-		return KryoUtils.copy(from, kryo, this);
-	}
-	
-	@Override
-	public T copy(T from, T reuse) {
-		checkKryoInitialized();
-
-		return KryoUtils.copy(from, reuse, kryo, this);
-	}
-
-	@Override
-	public int getLength() {
-		return -1;
-	}
-
-	@Override
-	public void serialize(T value, DataOutputView target) throws IOException {
-		checkAvroInitialized();
-		this.encoder.setOut(target);
-		this.writer.write(value, this.encoder);
-	}
-	
-	@Override
-	public T deserialize(DataInputView source) throws IOException {
-		checkAvroInitialized();
-		this.decoder.setIn(source);
-		return this.reader.read(null, this.decoder);
-	}
-
-	@Override
-	public T deserialize(T reuse, DataInputView source) throws IOException {
-		checkAvroInitialized();
-		this.decoder.setIn(source);
-		return this.reader.read(reuse, this.decoder);
-	}
-
-	@Override
-	public void copy(DataInputView source, DataOutputView target) throws IOException {
-		checkAvroInitialized();
-		
-		if (this.deepCopyInstance == null) {
-			this.deepCopyInstance = InstantiationUtil.instantiate(type, Object.class);
-		}
-		
-		this.decoder.setIn(source);
-		this.encoder.setOut(target);
-		
-		T tmp = this.reader.read(this.deepCopyInstance, this.decoder);
-		this.writer.write(tmp, this.encoder);
-	}
-	
-	
-	private void checkAvroInitialized() {
-		if (this.reader == null) {
-			this.reader = new ReflectDatumReader<T>(type);
-			this.writer = new ReflectDatumWriter<T>(type);
-			this.encoder = new DataOutputEncoder();
-			this.decoder = new DataInputDecoder();
-		}
-	}
-	
-	private void checkKryoInitialized() {
-		if (this.kryo == null) {
-			this.kryo = new Kryo();
-
-			Kryo.DefaultInstantiatorStrategy instantiatorStrategy = new Kryo.DefaultInstantiatorStrategy();
-			instantiatorStrategy.setFallbackInstantiatorStrategy(new StdInstantiatorStrategy());
-			kryo.setInstantiatorStrategy(instantiatorStrategy);
-
-			kryo.setAsmEnabled(true);
-
-			KryoUtils.applyRegistrations(kryo, kryoRegistrations.values());
-		}
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	@Override
-	public int hashCode() {
-		return 31 * this.type.hashCode() + this.typeToInstantiate.hashCode();
-	}
-	
-	@Override
-	public boolean equals(Object obj) {
-		if (obj instanceof AvroSerializer) {
-			@SuppressWarnings("unchecked")
-			AvroSerializer<T> avroSerializer = (AvroSerializer<T>) obj;
-
-			return avroSerializer.canEqual(this) &&
-				type == avroSerializer.type &&
-				typeToInstantiate == avroSerializer.typeToInstantiate;
-		} else {
-			return false;
-		}
-	}
-
-	@Override
-	public boolean canEqual(Object obj) {
-		return obj instanceof AvroSerializer;
-	}
-
-	// --------------------------------------------------------------------------------------------
-	// Serializer configuration snapshotting & compatibility
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public AvroSerializerConfigSnapshot<T> snapshotConfiguration() {
-		return new AvroSerializerConfigSnapshot<>(type, typeToInstantiate, kryoRegistrations);
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public CompatibilityResult<T> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
-		if (configSnapshot instanceof AvroSerializerConfigSnapshot) {
-			final AvroSerializerConfigSnapshot<T> config = (AvroSerializerConfigSnapshot<T>) configSnapshot;
-
-			if (type.equals(config.getTypeClass()) && typeToInstantiate.equals(config.getTypeToInstantiate())) {
-				// resolve Kryo registrations; currently, since the Kryo registrations in Avro
-				// are fixed, there shouldn't be a problem with the resolution here.
-
-				LinkedHashMap<String, KryoRegistration> oldRegistrations = config.getKryoRegistrations();
-				oldRegistrations.putAll(kryoRegistrations);
-
-				for (Map.Entry<String, KryoRegistration> reconfiguredRegistrationEntry : kryoRegistrations.entrySet()) {
-					if (reconfiguredRegistrationEntry.getValue().isDummy()) {
-						return CompatibilityResult.requiresMigration();
-					}
-				}
-
-				this.kryoRegistrations = oldRegistrations;
-				return CompatibilityResult.compatible();
-			}
-		}
-
-		// ends up here if the preceding serializer is not
-		// the ValueSerializer, or serialized data type has changed
-		return CompatibilityResult.requiresMigration();
-	}
-
-	public static class AvroSerializerConfigSnapshot<T> extends KryoRegistrationSerializerConfigSnapshot<T> {
-
-		private static final int VERSION = 1;
-
-		private Class<? extends T> typeToInstantiate;
-
-		public AvroSerializerConfigSnapshot() {}
-
-		public AvroSerializerConfigSnapshot(
-				Class<T> baseType,
-				Class<? extends T> typeToInstantiate,
-				LinkedHashMap<String, KryoRegistration> kryoRegistrations) {
-
-			super(baseType, kryoRegistrations);
-			this.typeToInstantiate = Preconditions.checkNotNull(typeToInstantiate);
-		}
-
-		@Override
-		public void write(DataOutputView out) throws IOException {
-			super.write(out);
-
-			out.writeUTF(typeToInstantiate.getName());
-		}
-
-		@SuppressWarnings("unchecked")
-		@Override
-		public void read(DataInputView in) throws IOException {
-			super.read(in);
-
-			String classname = in.readUTF();
-			try {
-				typeToInstantiate = (Class<? extends T>) Class.forName(classname, true, getUserCodeClassLoader());
-			} catch (ClassNotFoundException e) {
-				throw new IOException("Cannot find requested class " + classname + " in classpath.", e);
-			}
-		}
-
-		@Override
-		public int getVersion() {
-			return VERSION;
-		}
-
-		public Class<? extends T> getTypeToInstantiate() {
-			return typeToInstantiate;
-		}
-	}
-
-	// --------------------------------------------------------------------------------------------
-
-	private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
-		in.defaultReadObject();
-
-		// kryoRegistrations may be null if this Avro serializer is deserialized from an old version
-		if (kryoRegistrations == null) {
-			this.kryoRegistrations = buildKryoRegistrations(type);
-		}
-	}
-
-	private static <T> LinkedHashMap<String, KryoRegistration> buildKryoRegistrations(Class<T> serializedDataType) {
-		final LinkedHashMap<String, KryoRegistration> registrations = new LinkedHashMap<>();
-
-		// register Avro types.
-		registrations.put(
-				GenericData.Array.class.getName(),
-				new KryoRegistration(
-						GenericData.Array.class,
-						new ExecutionConfig.SerializableSerializer<>(new Serializers.SpecificInstanceCollectionSerializerForArrayList())));
-		registrations.put(Utf8.class.getName(), new KryoRegistration(Utf8.class));
-		registrations.put(GenericData.EnumSymbol.class.getName(), new KryoRegistration(GenericData.EnumSymbol.class));
-		registrations.put(GenericData.Fixed.class.getName(), new KryoRegistration(GenericData.Fixed.class));
-		registrations.put(GenericData.StringType.class.getName(), new KryoRegistration(GenericData.StringType.class));
-
-		// register the serialized data type
-		registrations.put(serializedDataType.getName(), new KryoRegistration(serializedDataType));
-
-		return registrations;
-	}
-}

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
index 6730136..269cf35 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
@@ -24,8 +24,6 @@ import com.esotericsoftware.kryo.Serializer;
 import com.esotericsoftware.kryo.io.Input;
 import com.esotericsoftware.kryo.io.Output;
 
-import org.apache.avro.generic.GenericData;
-
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeutils.CompatibilityResult;
@@ -406,7 +404,7 @@ public class KryoSerializer<T> extends TypeSerializer<T> {
 				}
 
 				// there's actually no way to tell if new Kryo serializers are compatible with
-				// the previous ones they overwrite; we can only signal compatibly and hope for the best
+				// the previous ones they overwrite; we can only signal compatibility and hope for the best
 				this.kryoRegistrations = reconfiguredRegistrations;
 				return CompatibilityResult.compatible();
 			}
@@ -478,11 +476,8 @@ public class KryoSerializer<T> extends TypeSerializer<T> {
 							registeredTypeWithSerializerEntry.getValue()));
 		}
 
-		kryoRegistrations.put(
-				GenericData.Array.class.getName(),
-				new KryoRegistration(
-						GenericData.Array.class,
-						new ExecutionConfig.SerializableSerializer<>(new Serializers.SpecificInstanceCollectionSerializerForArrayList())));
+		// add Avro support if flink-avro is available; a dummy otherwise
+		Serializers.addAvroGenericDataArrayRegistration(kryoRegistrations);
 
 		return kryoRegistrations;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/Serializers.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/Serializers.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/Serializers.java
index 4976d6a..de7b2fc 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/Serializers.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/Serializers.java
@@ -18,16 +18,6 @@
 
 package org.apache.flink.api.java.typeutils.runtime.kryo;
 
-import com.esotericsoftware.kryo.Kryo;
-import com.esotericsoftware.kryo.Serializer;
-import com.esotericsoftware.kryo.io.Input;
-import com.esotericsoftware.kryo.io.Output;
-import com.esotericsoftware.kryo.serializers.CollectionSerializer;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.specific.SpecificRecordBase;
-
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -35,18 +25,29 @@ import org.apache.flink.api.common.typeutils.CompositeType;
 import org.apache.flink.api.java.typeutils.GenericTypeInfo;
 import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo;
 import org.apache.flink.api.java.typeutils.TypeExtractionUtils;
+import org.apache.flink.api.java.typeutils.runtime.KryoRegistration;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import com.esotericsoftware.kryo.serializers.CollectionSerializer;
 
 import java.io.Serializable;
 import java.lang.reflect.Field;
 import java.lang.reflect.GenericArrayType;
+import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Modifier;
 import java.lang.reflect.ParameterizedType;
 import java.lang.reflect.Type;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Set;
 
+import static org.apache.flink.api.java.typeutils.TypeExtractionUtils.hasSuperclass;
+
 
 /**
  * Class containing utilities for the serializers of the Flink Runtime.
@@ -60,6 +61,14 @@ import java.util.Set;
 @Internal
 public class Serializers {
 
+	private static final String AVRO_SPECIFIC_RECORD_BASE = "org.apache.avro.specific.SpecificRecordBase";
+
+	private static final String AVRO_GENERIC_RECORD = "org.apache.avro.generic.GenericData$Record";
+
+	private static final String AVRO_KRYO_UTILS = "org.apache.flink.formats.avro.utils.AvroKryoSerializerUtils";
+
+	private static final String AVRO_GENERIC_DATA_ARRAY = "org.apache.avro.generic.GenericData$Array";
+
 	public static void recursivelyRegisterType(TypeInformation<?> typeInfo, ExecutionConfig config, Set<Class<?>> alreadySeen) {
 		if (typeInfo instanceof GenericTypeInfo) {
 			GenericTypeInfo<?> genericTypeInfo = (GenericTypeInfo<?>) typeInfo;
@@ -94,8 +103,11 @@ public class Serializers {
 		}
 		else {
 			config.registerKryoType(type);
-			checkAndAddSerializerForTypeAvro(config, type);
-	
+			// add serializers for Avro type if necessary
+			if (hasSuperclass(type, AVRO_SPECIFIC_RECORD_BASE) || hasSuperclass(type, AVRO_GENERIC_RECORD)) {
+				addAvroSerializers(config, type);
+			}
+
 			Field[] fields = type.getDeclaredFields();
 			for (Field field : fields) {
 				if (Modifier.isStatic(field.getModifiers()) || Modifier.isTransient(field.getModifiers())) {
@@ -147,20 +159,54 @@ public class Serializers {
 			}
 		}
 	}
-	
-	// ------------------------------------------------------------------------
-	
-	private static void checkAndAddSerializerForTypeAvro(ExecutionConfig reg, Class<?> type) {
-		if (GenericData.Record.class.isAssignableFrom(type) || SpecificRecordBase.class.isAssignableFrom(type)) {
-			// Avro POJOs contain java.util.List which have GenericData.Array as their runtime type
-			// because Kryo is not able to serialize them properly, we use this serializer for them
-			reg.registerTypeWithKryoSerializer(GenericData.Array.class, SpecificInstanceCollectionSerializerForArrayList.class);
-
-			// We register this serializer for users who want to use untyped Avro records (GenericData.Record).
-			// Kryo is able to serialize everything in there, except for the Schema.
-			// This serializer is very slow, but using the GenericData.Records of Kryo is in general a bad idea.
-			// we add the serializer as a default serializer because Avro is using a private sub-type at runtime.
-			reg.addDefaultKryoSerializer(Schema.class, AvroSchemaSerializer.class);
+
+	/**
+	 * Loads the utility class from <code>flink-avro</code> and adds Avro-specific serializers.
+	 */
+	private static void addAvroSerializers(ExecutionConfig reg, Class<?> type) {
+		Class<?> clazz;
+		try {
+			clazz = Class.forName(AVRO_KRYO_UTILS, false, Serializers.class.getClassLoader());
+		}
+		catch (ClassNotFoundException e) {
+			throw new RuntimeException("Could not load class '" + AVRO_KRYO_UTILS + "'. " +
+				"You may be missing the 'flink-avro' dependency.");
+		}
+		try {
+			clazz.getDeclaredMethod("addAvroSerializers", ExecutionConfig.class, Class.class).invoke(null, reg, type);
+		} catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) {
+			throw new RuntimeException("Could not access method in 'flink-avro' dependency.", e);
+		}
+	}
+
+	@SuppressWarnings("unchecked")
+	public static void addAvroGenericDataArrayRegistration(LinkedHashMap<String, KryoRegistration> kryoRegistrations) {
+		try {
+			Class<?> clazz = Class.forName(AVRO_GENERIC_DATA_ARRAY, false, Serializers.class.getClassLoader());
+
+			kryoRegistrations.put(
+				AVRO_GENERIC_DATA_ARRAY,
+				new KryoRegistration(
+						clazz,
+						new ExecutionConfig.SerializableSerializer<>(new Serializers.SpecificInstanceCollectionSerializerForArrayList())));
+		}
+		catch (ClassNotFoundException e) {
+			kryoRegistrations.put(AVRO_GENERIC_DATA_ARRAY,
+				new KryoRegistration(DummyAvroRegisteredClass.class, (Class) DummyAvroKryoSerializerClass.class));
+		}
+	}
+
+	public static class DummyAvroRegisteredClass {}
+
+	public static class DummyAvroKryoSerializerClass<T> extends Serializer<T> {
+		@Override
+		public void write(Kryo kryo, Output output, Object o) {
+			throw new UnsupportedOperationException("Could not find required Avro dependency.");
+		}
+
+		@Override
+		public T read(Kryo kryo, Input input, Class<T> aClass) {
+			throw new UnsupportedOperationException("Could not find required Avro dependency.");
 		}
 	}
 
@@ -168,6 +214,9 @@ public class Serializers {
 	// Custom Serializers
 	// --------------------------------------------------------------------------------------------
 
+	/**
+	 * Special serializer for Java's {@link ArrayList} used for Avro's GenericData.Array.
+	 */
 	@SuppressWarnings("rawtypes")
 	public static class SpecificInstanceCollectionSerializerForArrayList extends SpecificInstanceCollectionSerializer<ArrayList> {
 		private static final long serialVersionUID = 1L;
@@ -176,19 +225,19 @@ public class Serializers {
 			super(ArrayList.class);
 		}
 	}
+
 	/**
 	 * Special serializer for Java collections enforcing certain instance types.
 	 * Avro is serializing collections with an "GenericData.Array" type. Kryo is not able to handle
 	 * this type, so we use ArrayLists.
 	 */
 	@SuppressWarnings("rawtypes")
-	public static class SpecificInstanceCollectionSerializer<T extends Collection> 
-			extends CollectionSerializer implements Serializable
-	{
+	public static class SpecificInstanceCollectionSerializer<T extends Collection>
+			extends CollectionSerializer implements Serializable {
 		private static final long serialVersionUID = 1L;
-		
+
 		private Class<T> type;
-		
+
 		public SpecificInstanceCollectionSerializer(Class<T> type) {
 			this.type = type;
 		}
@@ -203,27 +252,4 @@ public class Serializers {
 			return kryo.newInstance(this.type);
 		}
 	}
-
-	/**
-	 * Slow serialization approach for Avro schemas.
-	 * This is only used with {{@link org.apache.avro.generic.GenericData.Record}} types.
-	 * Having this serializer, we are able to handle avro Records.
-	 */
-	public static class AvroSchemaSerializer extends Serializer<Schema> implements Serializable {
-		private static final long serialVersionUID = 1L;
-
-		@Override
-		public void write(Kryo kryo, Output output, Schema object) {
-			String schemaAsString = object.toString(false);
-			output.writeString(schemaAsString);
-		}
-
-		@Override
-		public Schema read(Kryo kryo, Input input, Class<Schema> type) {
-			String schemaAsString = input.readString();
-			// the parser seems to be stateful, to we need a new one for every type.
-			Schema.Parser sParser = new Schema.Parser();
-			return sParser.parse(schemaAsString);
-		}
-	}
 }

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericTypeSerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericTypeSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericTypeSerializerTest.java
deleted file mode 100644
index df1ff60..0000000
--- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericTypeSerializerTest.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.api.java.typeutils.runtime;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-
-public class AvroGenericTypeSerializerTest extends AbstractGenericTypeSerializerTest {
-
-	@Override
-	protected <T> TypeSerializer<T> createSerializer(Class<T> type) {
-		return new AvroSerializer<T>(type);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializerEmptyArrayTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializerEmptyArrayTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializerEmptyArrayTest.java
deleted file mode 100644
index 8a89410..0000000
--- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializerEmptyArrayTest.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.java.typeutils.runtime;
-
-import static org.junit.Assert.*;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.avro.reflect.Nullable;
-import org.apache.flink.api.common.typeutils.SerializerTestInstance;
-import org.junit.Test;
-
-public class AvroSerializerEmptyArrayTest {
-
-	@Test
-	public void testBookSerialization() {
-		try {
-			Book b = new Book(123, "This is a test book", 26382648);
-			AvroSerializer<Book> serializer = new AvroSerializer<Book>(Book.class);
-			SerializerTestInstance<Book> test = new SerializerTestInstance<Book>(serializer, Book.class, -1, b);
-			test.testAll();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testSerialization() {
-		try {
-			List<String> titles = new ArrayList<String>();
-
-			List<Book> books = new ArrayList<Book>();
-			books.add(new Book(123, "This is a test book", 1));
-			books.add(new Book(24234234, "This is a test book", 1));
-			books.add(new Book(1234324, "This is a test book", 3));
-
-			BookAuthor a = new BookAuthor(1, titles, "Test Author");
-			a.books = books;
-			a.bookType = BookAuthor.BookType.journal;
-			
-			AvroSerializer<BookAuthor> serializer = new AvroSerializer<BookAuthor>(BookAuthor.class);
-			
-			SerializerTestInstance<BookAuthor> test = new SerializerTestInstance<BookAuthor>(serializer, BookAuthor.class, -1, a);
-			test.testAll();
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-
-	public static class Book {
-
-		long bookId;
-		@Nullable
-		String title;
-		long authorId;
-
-		public Book() {}
-
-		public Book(long bookId, String title, long authorId) {
-			this.bookId = bookId;
-			this.title = title;
-			this.authorId = authorId;
-		}
-
-		@Override
-		public int hashCode() {
-			final int prime = 31;
-			int result = 1;
-			result = prime * result + (int) (authorId ^ (authorId >>> 32));
-			result = prime * result + (int) (bookId ^ (bookId >>> 32));
-			result = prime * result + ((title == null) ? 0 : title.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;
-			Book other = (Book) obj;
-			if (authorId != other.authorId)
-				return false;
-			if (bookId != other.bookId)
-				return false;
-			if (title == null) {
-				if (other.title != null)
-					return false;
-			} else if (!title.equals(other.title))
-				return false;
-			return true;
-		}
-	}
-
-	public static class BookAuthor {
-
-		enum BookType {
-			book,
-			article,
-			journal
-		}
-
-		long authorId;
-
-		@Nullable
-		List<String> bookTitles;
-
-		@Nullable
-		List<Book> books;
-
-		String authorName;
-
-		BookType bookType;
-
-		public BookAuthor() {}
-
-		public BookAuthor(long authorId, List<String> bookTitles, String authorName) {
-			this.authorId = authorId;
-			this.bookTitles = bookTitles;
-			this.authorName = authorName;
-		}
-
-		@Override
-		public int hashCode() {
-			final int prime = 31;
-			int result = 1;
-			result = prime * result + (int) (authorId ^ (authorId >>> 32));
-			result = prime * result + ((authorName == null) ? 0 : authorName.hashCode());
-			result = prime * result + ((bookTitles == null) ? 0 : bookTitles.hashCode());
-			result = prime * result + ((bookType == null) ? 0 : bookType.hashCode());
-			result = prime * result + ((books == null) ? 0 : books.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;
-			BookAuthor other = (BookAuthor) obj;
-			if (authorId != other.authorId)
-				return false;
-			if (authorName == null) {
-				if (other.authorName != null)
-					return false;
-			} else if (!authorName.equals(other.authorName))
-				return false;
-			if (bookTitles == null) {
-				if (other.bookTitles != null)
-					return false;
-			} else if (!bookTitles.equals(other.bookTitles))
-				return false;
-			if (bookType != other.bookType)
-				return false;
-			if (books == null) {
-				if (other.books != null)
-					return false;
-			} else if (!books.equals(other.books))
-				return false;
-			return true;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerCompatibilityTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerCompatibilityTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerCompatibilityTest.java
index 5a404bd..1cacc9e 100644
--- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerCompatibilityTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerCompatibilityTest.java
@@ -18,20 +18,22 @@
 
 package org.apache.flink.api.java.typeutils.runtime.kryo;
 
-import com.esotericsoftware.kryo.Kryo;
-import com.esotericsoftware.kryo.Serializer;
-import com.esotericsoftware.kryo.io.Input;
-import com.esotericsoftware.kryo.io.Output;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeutils.CompatibilityResult;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
 import org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil;
 import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
 import org.junit.Test;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
+import java.io.InputStream;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -42,6 +44,20 @@ import static org.junit.Assert.assertTrue;
  */
 public class KryoSerializerCompatibilityTest {
 
+	@Test
+	public void testMigrationStrategyForRemovedAvroDependency() throws Exception {
+		KryoSerializer<TestClass> kryoSerializerForA = new KryoSerializer<>(TestClass.class, new ExecutionConfig());
+
+		// read configuration again from bytes
+		TypeSerializerConfigSnapshot kryoSerializerConfigSnapshot;
+		try (InputStream in = getClass().getResourceAsStream("/kryo-serializer-flink1.3-snapshot")) {
+			kryoSerializerConfigSnapshot = TypeSerializerSerializationUtil.readSerializerConfigSnapshot(
+				new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader());
+		}
+		CompatibilityResult<TestClass> compatResult = kryoSerializerForA.ensureCompatibility(kryoSerializerConfigSnapshot);
+		assertFalse(compatResult.isRequiresMigration());
+	}
+
 	/**
 	 * Verifies that reconfiguration result is INCOMPATIBLE if data type has changed.
 	 */
@@ -60,7 +76,7 @@ public class KryoSerializerCompatibilityTest {
 		KryoSerializer<TestClassB> kryoSerializerForB = new KryoSerializer<>(TestClassB.class, new ExecutionConfig());
 
 		// read configuration again from bytes
-		try(ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) {
+		try (ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) {
 			kryoSerializerConfigSnapshot = TypeSerializerSerializationUtil.readSerializerConfigSnapshot(
 				new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader());
 		}
@@ -103,7 +119,7 @@ public class KryoSerializerCompatibilityTest {
 		kryoSerializer = new KryoSerializer<>(TestClass.class, executionConfig);
 
 		// read configuration from bytes
-		try(ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) {
+		try (ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) {
 			kryoSerializerConfigSnapshot = TypeSerializerSerializationUtil.readSerializerConfigSnapshot(
 				new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader());
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-core/src/test/resources/kryo-serializer-flink1.3-snapshot
----------------------------------------------------------------------
diff --git a/flink-core/src/test/resources/kryo-serializer-flink1.3-snapshot b/flink-core/src/test/resources/kryo-serializer-flink1.3-snapshot
new file mode 100644
index 0000000..0123a9c
Binary files /dev/null and b/flink-core/src/test/resources/kryo-serializer-flink1.3-snapshot differ


[17/21] flink git commit: [hotfix] [avro] Fix some serializability warnings and problems

Posted by se...@apache.org.
[hotfix] [avro] Fix some serializability warnings and problems


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

Branch: refs/heads/master
Commit: 4cd7a802cfbfca01708ef75063cecc6e2ded5461
Parents: eb99181
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Nov 2 19:48:32 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Nov 3 16:40:35 2017 +0100

----------------------------------------------------------------------
 .../java/org/apache/flink/formats/avro/typeutils/AvroTypeInfo.java | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/4cd7a802/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroTypeInfo.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroTypeInfo.java
index ddc89a8..ad6b06e 100644
--- a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroTypeInfo.java
+++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroTypeInfo.java
@@ -47,6 +47,8 @@ import java.util.List;
  */
 public class AvroTypeInfo<T extends SpecificRecordBase> extends PojoTypeInfo<T> {
 
+	private static final long serialVersionUID = 1L;
+
 	public AvroTypeInfo(Class<T> typeClass) {
 		super(typeClass, generateFieldsFromAvroSchema(typeClass));
 	}


[02/21] flink git commit: [FLINK-7420] [avro] (follow-up) Do not shade or exclude avro in Hadoop

Posted by se...@apache.org.
[FLINK-7420] [avro] (follow-up) Do not shade or exclude avro in Hadoop

Excluding Avro in Hadoop will make basic classes like sequence file writers fail
to load, even if Avro is not used.

Shading Avro is not possible because it is part of the API for users that
use Avro types.


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

Branch: refs/heads/master
Commit: db7c70faa0b5aed652b525e8091cdc0e265362bc
Parents: 537a10e
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Nov 3 12:42:21 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Nov 3 16:40:34 2017 +0100

----------------------------------------------------------------------
 flink-formats/flink-avro/pom.xml                 |  2 +-
 flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml | 19 +++++++------------
 pom.xml                                          |  7 +++++++
 3 files changed, 15 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/db7c70fa/flink-formats/flink-avro/pom.xml
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/pom.xml b/flink-formats/flink-avro/pom.xml
index 19d9129..a0f63be 100644
--- a/flink-formats/flink-avro/pom.xml
+++ b/flink-formats/flink-avro/pom.xml
@@ -55,7 +55,7 @@ under the License.
 		<dependency>
 			<groupId>org.apache.avro</groupId>
 			<artifactId>avro</artifactId>
-			<version>1.8.2</version>
+			<!-- managed version -->
 		</dependency>
 
 		<!-- test dependencies -->

http://git-wip-us.apache.org/repos/asf/flink/blob/db7c70fa/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
----------------------------------------------------------------------
diff --git a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml b/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
index 864c257..66d8549 100644
--- a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
+++ b/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
@@ -43,6 +43,13 @@ under the License.
 			<!-- managed version -->
 		</dependency>
 
+		<!-- explicitly include avro to force Flink's managed version -->
+		<dependency>
+			<groupId>org.apache.avro</groupId>
+			<artifactId>avro</artifactId>
+			<!-- managed version -->
+		</dependency>
+
 		<dependency>
 			<groupId>org.apache.hadoop</groupId>
 			<artifactId>hadoop-common</artifactId>
@@ -161,12 +168,6 @@ under the License.
 					<groupId>commons-beanutils</groupId>
 					<artifactId>commons-beanutils</artifactId>
 				</exclusion>
-
-				<!-- we don't want Hadoop's Avro dependency, since Flink adds its own Avro support --> 
-				<exclusion>
-					<groupId>org.apache.avro</groupId>
-					<artifactId>avro</artifactId>
-				</exclusion>
 			</exclusions>
 		</dependency>
 
@@ -418,12 +419,6 @@ under the License.
 					<groupId>com.sun.jersey.contribs</groupId>
 					<artifactId>jersey-guice</artifactId>
 				</exclusion>
-
-				<!-- we don't want Hadoop's Avro dependency, since Flink adds its own Avro support -->
-				<exclusion>
-					<groupId>org.apache.avro</groupId>
-					<artifactId>avro</artifactId>
-				</exclusion>
 			</exclusions>
 		</dependency>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/db7c70fa/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5302c5c..773dc34 100644
--- a/pom.xml
+++ b/pom.xml
@@ -282,6 +282,13 @@ under the License.
 				<version>1.1.1.3</version>
 			</dependency>
 
+			<!-- Make sure we use a consistent avro version between Flink and Hadoop -->		
+ 			<dependency>
+				<groupId>org.apache.avro</groupId>
+				<artifactId>avro</artifactId>
+				<version>1.8.2</version>
+			</dependency>
+
 			<!-- Make sure we use a consistent commons-cli version throughout the project -->
 			<dependency>
 				<groupId>commons-cli</groupId>


[16/21] flink git commit: [FLINK-7847] [avro] Fix typo in jackson shading pattern

Posted by se...@apache.org.
[FLINK-7847] [avro] Fix typo in jackson shading pattern

This closes #4931


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

Branch: refs/heads/master
Commit: b4dead96a16c8772ccf86b533f5de6feb0f3d1f6
Parents: 65e8704
Author: zentol <ch...@apache.org>
Authored: Wed Nov 1 12:43:00 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Nov 3 16:40:35 2017 +0100

----------------------------------------------------------------------
 flink-formats/flink-avro/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/b4dead96/flink-formats/flink-avro/pom.xml
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/pom.xml b/flink-formats/flink-avro/pom.xml
index a0f63be..5080b82 100644
--- a/flink-formats/flink-avro/pom.xml
+++ b/flink-formats/flink-avro/pom.xml
@@ -211,7 +211,7 @@ under the License.
 							<relocations>
 								<relocation>
 									<pattern>org.codehaus.jackson</pattern>
-									<shadedPattern>org.apache.flink.avro.shaded.org.codehouse.jackson</shadedPattern>
+									<shadedPattern>org.apache.flink.avro.shaded.org.codehaus.jackson</shadedPattern>
 								</relocation>
 							</relocations>
 						</configuration>


[06/21] flink git commit: [FLINK-7420] [avro] Move all Avro code to flink-avro

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

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroTypeInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroTypeInfoTest.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroTypeInfoTest.java
new file mode 100644
index 0000000..79a4a45
--- /dev/null
+++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroTypeInfoTest.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.formats.avro.typeutils;
+
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
+import org.apache.flink.formats.avro.generated.Address;
+import org.apache.flink.formats.avro.generated.User;
+
+/**
+ * Test for {@link AvroTypeInfo}.
+ */
+public class AvroTypeInfoTest extends TypeInformationTestBase<AvroTypeInfo<?>> {
+
+	@Override
+	protected AvroTypeInfo<?>[] getTestData() {
+		return new AvroTypeInfo<?>[] {
+			new AvroTypeInfo<>(Address.class),
+			new AvroTypeInfo<>(User.class),
+		};
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/utils/AvroTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/utils/AvroTestUtils.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/utils/AvroTestUtils.java
new file mode 100644
index 0000000..90ac040
--- /dev/null
+++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/utils/AvroTestUtils.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.formats.avro.utils;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.formats.avro.generated.Address;
+import org.apache.flink.formats.avro.generated.Colors;
+import org.apache.flink.formats.avro.generated.User;
+import org.apache.flink.types.Row;
+
+import org.apache.avro.Schema;
+import org.apache.avro.SchemaBuilder;
+import org.apache.avro.reflect.ReflectData;
+import org.apache.avro.specific.SpecificRecord;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+
+/**
+ * Utilities for creating Avro Schemas.
+ */
+public final class AvroTestUtils {
+
+	private static final String NAMESPACE = "org.apache.flink.streaming.connectors.kafka";
+
+	/**
+	 * Creates a flat Avro Schema for testing.
+	 */
+	public static Schema createFlatAvroSchema(String[] fieldNames, TypeInformation[] fieldTypes) {
+		final SchemaBuilder.FieldAssembler<Schema> fieldAssembler = SchemaBuilder
+			.record("BasicAvroRecord")
+			.namespace(NAMESPACE)
+			.fields();
+
+		final Schema nullSchema = Schema.create(Schema.Type.NULL);
+
+		for (int i = 0; i < fieldNames.length; i++) {
+			Schema schema = ReflectData.get().getSchema(fieldTypes[i].getTypeClass());
+			Schema unionSchema = Schema.createUnion(Arrays.asList(nullSchema, schema));
+			fieldAssembler.name(fieldNames[i]).type(unionSchema).noDefault();
+		}
+
+		return fieldAssembler.endRecord();
+	}
+
+	/**
+	 * Tests a simple Avro data types without nesting.
+	 */
+	public static Tuple3<Class<? extends SpecificRecord>, SpecificRecord, Row> getSimpleTestData() {
+		final Address addr = Address.newBuilder()
+			.setNum(42)
+			.setStreet("Main Street 42")
+			.setCity("Test City")
+			.setState("Test State")
+			.setZip("12345")
+			.build();
+
+		final Row rowAddr = new Row(5);
+		rowAddr.setField(0, 42);
+		rowAddr.setField(1, "Main Street 42");
+		rowAddr.setField(2, "Test City");
+		rowAddr.setField(3, "Test State");
+		rowAddr.setField(4, "12345");
+
+		final Tuple3<Class<? extends SpecificRecord>, SpecificRecord, Row> t = new Tuple3<>();
+		t.f0 = Address.class;
+		t.f1 = addr;
+		t.f2 = rowAddr;
+
+		return t;
+	}
+
+	/**
+	 * Tests all Avro data types as well as nested types.
+	 */
+	public static Tuple3<Class<? extends SpecificRecord>, SpecificRecord, Row> getComplexTestData() {
+		final Address addr = Address.newBuilder()
+			.setNum(42)
+			.setStreet("Main Street 42")
+			.setCity("Test City")
+			.setState("Test State")
+			.setZip("12345")
+			.build();
+
+		final Row rowAddr = new Row(5);
+		rowAddr.setField(0, 42);
+		rowAddr.setField(1, "Main Street 42");
+		rowAddr.setField(2, "Test City");
+		rowAddr.setField(3, "Test State");
+		rowAddr.setField(4, "12345");
+
+		final User user = User.newBuilder()
+			.setName("Charlie")
+			.setFavoriteNumber(null)
+			.setFavoriteColor("blue")
+			.setTypeLongTest(1337L)
+			.setTypeDoubleTest(1.337d)
+			.setTypeNullTest(null)
+			.setTypeBoolTest(false)
+			.setTypeArrayString(new ArrayList<CharSequence>())
+			.setTypeArrayBoolean(new ArrayList<Boolean>())
+			.setTypeNullableArray(null)
+			.setTypeEnum(Colors.RED)
+			.setTypeMap(new HashMap<CharSequence, Long>())
+			.setTypeFixed(null)
+			.setTypeUnion(null)
+			.setTypeNested(addr)
+			.build();
+
+		final Row rowUser = new Row(15);
+		rowUser.setField(0, "Charlie");
+		rowUser.setField(1, null);
+		rowUser.setField(2, "blue");
+		rowUser.setField(3, 1337L);
+		rowUser.setField(4, 1.337d);
+		rowUser.setField(5, null);
+		rowUser.setField(6, false);
+		rowUser.setField(7, new ArrayList<CharSequence>());
+		rowUser.setField(8, new ArrayList<Boolean>());
+		rowUser.setField(9, null);
+		rowUser.setField(10, Colors.RED);
+		rowUser.setField(11, new HashMap<CharSequence, Long>());
+		rowUser.setField(12, null);
+		rowUser.setField(13, null);
+		rowUser.setField(14, rowAddr);
+
+		final Tuple3<Class<? extends SpecificRecord>, SpecificRecord, Row> t = new Tuple3<>();
+		t.f0 = User.class;
+		t.f1 = user;
+		t.f2 = rowUser;
+
+		return t;
+	}
+}

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-formats/flink-avro/src/test/resources/testdata.avro
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/test/resources/testdata.avro b/flink-formats/flink-avro/src/test/resources/testdata.avro
new file mode 100644
index 0000000..3102d03
Binary files /dev/null and b/flink-formats/flink-avro/src/test/resources/testdata.avro differ

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

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/HDFSTest.java
----------------------------------------------------------------------
diff --git a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/HDFSTest.java b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/HDFSTest.java
index e4b907a..2906eb8 100644
--- a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/HDFSTest.java
+++ b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/HDFSTest.java
@@ -23,7 +23,7 @@ import org.apache.flink.api.common.io.FileOutputFormat;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.ExecutionEnvironmentFactory;
 import org.apache.flink.api.java.LocalEnvironment;
-import org.apache.flink.api.java.io.AvroOutputFormat;
+import org.apache.flink.api.java.io.TextOutputFormat;
 import org.apache.flink.configuration.BlobServerOptions;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.HighAvailabilityOptions;
@@ -168,31 +168,30 @@ public class HDFSTest {
 	}
 
 	@Test
-	public void testAvroOut() {
-		String type = "one";
-		AvroOutputFormat<String> avroOut =
-				new AvroOutputFormat<String>(String.class);
+	public void testChangingFileNames() {
+		org.apache.hadoop.fs.Path hdfsPath = new org.apache.hadoop.fs.Path(hdfsURI + "/hdfsTest");
+		Path path = new Path(hdfsPath.toString());
 
-		org.apache.hadoop.fs.Path result = new org.apache.hadoop.fs.Path(hdfsURI + "/avroTest");
+		String type = "one";
+		TextOutputFormat<String> outputFormat = new TextOutputFormat<>(path);
 
-		avroOut.setOutputFilePath(new Path(result.toString()));
-		avroOut.setWriteMode(FileSystem.WriteMode.NO_OVERWRITE);
-		avroOut.setOutputDirectoryMode(FileOutputFormat.OutputDirectoryMode.ALWAYS);
+		outputFormat.setWriteMode(FileSystem.WriteMode.NO_OVERWRITE);
+		outputFormat.setOutputDirectoryMode(FileOutputFormat.OutputDirectoryMode.ALWAYS);
 
 		try {
-			avroOut.open(0, 2);
-			avroOut.writeRecord(type);
-			avroOut.close();
+			outputFormat.open(0, 2);
+			outputFormat.writeRecord(type);
+			outputFormat.close();
 
-			avroOut.open(1, 2);
-			avroOut.writeRecord(type);
-			avroOut.close();
+			outputFormat.open(1, 2);
+			outputFormat.writeRecord(type);
+			outputFormat.close();
 
-			assertTrue("No result file present", hdfs.exists(result));
-			FileStatus[] files = hdfs.listStatus(result);
+			assertTrue("No result file present", hdfs.exists(hdfsPath));
+			FileStatus[] files = hdfs.listStatus(hdfsPath);
 			Assert.assertEquals(2, files.length);
 			for (FileStatus file : files) {
-				assertTrue("1.avro".equals(file.getPath().getName()) || "2.avro".equals(file.getPath().getName()));
+				assertTrue("1".equals(file.getPath().getName()) || "2".equals(file.getPath().getName()));
 			}
 
 		} catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-libraries/flink-cep/pom.xml
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-cep/pom.xml b/flink-libraries/flink-cep/pom.xml
index 23978b2..bd57d17 100644
--- a/flink-libraries/flink-cep/pom.xml
+++ b/flink-libraries/flink-cep/pom.xml
@@ -89,7 +89,13 @@ under the License.
             <scope>test</scope>
         </dependency>
 
-
+        <!-- we include Avro to make the CEPMigrationTest work, it uses a Kryo-serialized savepoint (see FLINK-7420) -->
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-avro_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
     
     <build>

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
----------------------------------------------------------------------
diff --git a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml b/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
index ff6f84d..864c257 100644
--- a/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
+++ b/flink-shaded-hadoop/flink-shaded-hadoop2/pom.xml
@@ -161,6 +161,12 @@ under the License.
 					<groupId>commons-beanutils</groupId>
 					<artifactId>commons-beanutils</artifactId>
 				</exclusion>
+
+				<!-- we don't want Hadoop's Avro dependency, since Flink adds its own Avro support --> 
+				<exclusion>
+					<groupId>org.apache.avro</groupId>
+					<artifactId>avro</artifactId>
+				</exclusion>
 			</exclusions>
 		</dependency>
 
@@ -412,6 +418,12 @@ under the License.
 					<groupId>com.sun.jersey.contribs</groupId>
 					<artifactId>jersey-guice</artifactId>
 				</exclusion>
+
+				<!-- we don't want Hadoop's Avro dependency, since Flink adds its own Avro support -->
+				<exclusion>
+					<groupId>org.apache.avro</groupId>
+					<artifactId>avro</artifactId>
+				</exclusion>
 			</exclusions>
 		</dependency>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-shaded-hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/flink-shaded-hadoop/pom.xml b/flink-shaded-hadoop/pom.xml
index ba90fc9..3e7cb41 100644
--- a/flink-shaded-hadoop/pom.xml
+++ b/flink-shaded-hadoop/pom.xml
@@ -52,11 +52,6 @@ under the License.
 			<artifactId>slf4j-api</artifactId>
 			<scope>provided</scope>
 		</dependency>
-		<dependency>
-			<groupId>org.apache.avro</groupId>
-			<artifactId>avro</artifactId>
-			<scope>provided</scope>
-		</dependency>
 	</dependencies>
 
 	<build>

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml
index 8e10a2e..2217199 100644
--- a/flink-tests/pom.xml
+++ b/flink-tests/pom.xml
@@ -170,21 +170,6 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-avro_${scala.binary.version}</artifactId>
-			<version>${project.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-avro_${scala.binary.version}</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-optimizer_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 9cf603a..5302c5c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -66,6 +66,7 @@ under the License.
 		<module>flink-streaming-java</module>
 		<module>flink-streaming-scala</module>
 		<module>flink-connectors</module>
+		<module>flink-formats</module>
 		<module>flink-examples</module>
 		<module>flink-clients</module>
 		<module>flink-queryable-state</module>
@@ -275,19 +276,6 @@ under the License.
 				<version>5.0.4-1.0</version>
 			</dependency>
 
-			<!-- Make sure we use a consistent avro version throughout the project -->
-			<dependency>
-				<groupId>org.apache.avro</groupId>
-				<artifactId>avro</artifactId>
-				<version>1.8.2</version>
-			</dependency>
-			
-			<dependency>
-				<groupId>org.apache.avro</groupId>
-				<artifactId>avro-ipc</artifactId>
-				<version>1.8.2</version>
-			</dependency>
-
 			<dependency>
 				<groupId>org.xerial.snappy</groupId>
 				<artifactId>snappy-java</artifactId>
@@ -1033,7 +1021,8 @@ under the License.
 						<!-- Test Data. -->
 						<exclude>flink-tests/src/test/resources/testdata/terainput.txt</exclude>
 						<exclude>flink-runtime/src/test/resources/flink_11-kryo_registrations</exclude>
-						<exclude>flink-connectors/flink-avro/src/test/resources/avro/*.avsc</exclude>
+						<exclude>flink-core/src/test/resources/kryo-serializer-config-snapshot-v1</exclude>
+						<exclude>flink-formats/flink-avro/src/test/resources/avro/*.avsc</exclude>
 						<exclude>out/test/flink-avro/avro/user.avsc</exclude>
 						<exclude>flink-libraries/flink-table/src/test/scala/resources/*.out</exclude>
 						<exclude>test-infra/end-to-end-test/test-data/*</exclude>
@@ -1042,8 +1031,8 @@ under the License.
 						<exclude>**/src/test/resources/*-snapshot</exclude>
 						<exclude>**/src/test/resources/*-savepoint</exclude>
 
-						<exclude>flink-connectors/flink-avro/src/test/resources/testdata.avro</exclude>
-						<exclude>flink-connectors/flink-avro/src/test/java/org/apache/flink/api/io/avro/generated/*.java</exclude>
+						<exclude>flink-formats/flink-avro/src/test/resources/testdata.avro</exclude>
+						<exclude>flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/generated/*.java</exclude>
 						<exclude>flink-libraries/flink-python/src/test/python/org/apache/flink/python/api/data_csv</exclude>
 						<exclude>flink-libraries/flink-python/src/test/python/org/apache/flink/python/api/data_text</exclude>
 						<!-- Configuration Files. -->

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/tools/maven/suppressions.xml
----------------------------------------------------------------------
diff --git a/tools/maven/suppressions.xml b/tools/maven/suppressions.xml
index 5d5c455..a58e17c 100644
--- a/tools/maven/suppressions.xml
+++ b/tools/maven/suppressions.xml
@@ -23,8 +23,7 @@ under the License.
 		"http://www.puppycrawl.com/dtds/suppressions_1_1.dtd">
 
 <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[\\/]formats[\\/]avro[\\/]generated[\\/].*.java" checks="[a-zA-Z0-9]*"/>
 		<!-- Sometimes we have to temporarily fix very long, different formatted Calcite files. -->
 		<suppress files="org[\\/]apache[\\/]calcite.*" checks="[a-zA-Z0-9]*"/>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/tools/travis_mvn_watchdog.sh
----------------------------------------------------------------------
diff --git a/tools/travis_mvn_watchdog.sh b/tools/travis_mvn_watchdog.sh
index 978bc9f..fda6023 100755
--- a/tools/travis_mvn_watchdog.sh
+++ b/tools/travis_mvn_watchdog.sh
@@ -77,7 +77,7 @@ flink-filesystems/flink-hadoop-fs,\
 flink-filesystems/flink-mapr-fs,\
 flink-filesystems/flink-s3-fs-hadoop,\
 flink-filesystems/flink-s3-fs-presto,\
-flink-connectors/flink-avro,\
+flink-formats/flink-avro,\
 flink-connectors/flink-hbase,\
 flink-connectors/flink-hcatalog,\
 flink-connectors/flink-hadoop-compatibility,\


[13/21] flink git commit: [FLINK-7420] [avro] Make flink-avro Scala independent

Posted by se...@apache.org.
[FLINK-7420] [avro] Make flink-avro Scala independent

This removes all dependencies on Scala-dependent projects.

This commit introduces a hard wired test dependency to
'flink-test-utils_2.11' to avoid introducing a Scala version dependency
due to a non-exported test utility.


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

Branch: refs/heads/master
Commit: eb99181ddd4851d2f4a64377ebd4fe0ac11e2581
Parents: ad1c607
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Nov 2 16:18:21 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Nov 3 16:40:35 2017 +0100

----------------------------------------------------------------------
 docs/dev/batch/connectors.md                    |  2 +-
 .../flink-connector-filesystem/pom.xml          |  2 +-
 .../flink-connector-kafka-0.10/pom.xml          |  4 +--
 .../flink-connector-kafka-0.11/pom.xml          |  4 +--
 .../flink-connector-kafka-0.8/pom.xml           | 18 ++++++-------
 .../flink-connector-kafka-0.9/pom.xml           |  4 +--
 .../flink-connector-kafka-base/pom.xml          |  4 +--
 flink-dist/pom.xml                              |  2 +-
 flink-formats/flink-avro/pom.xml                | 28 ++++----------------
 flink-fs-tests/pom.xml                          |  2 +-
 .../main/resources/archetype-resources/pom.xml  |  2 +-
 .../main/resources/archetype-resources/pom.xml  |  2 +-
 12 files changed, 28 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/eb99181d/docs/dev/batch/connectors.md
----------------------------------------------------------------------
diff --git a/docs/dev/batch/connectors.md b/docs/dev/batch/connectors.md
index 388b599..2391744 100644
--- a/docs/dev/batch/connectors.md
+++ b/docs/dev/batch/connectors.md
@@ -92,7 +92,7 @@ Also, the serialization framework of Flink is able to handle classes generated f
 ~~~xml
 <dependency>
   <groupId>org.apache.flink</groupId>
-  <artifactId>flink-avro{{ site.scala_version_suffix }}</artifactId>
+  <artifactId>flink-avro</artifactId>
   <version>{{site.version }}</version>
 </dependency>
 ~~~

http://git-wip-us.apache.org/repos/asf/flink/blob/eb99181d/flink-connectors/flink-connector-filesystem/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-filesystem/pom.xml b/flink-connectors/flink-connector-filesystem/pom.xml
index 12a151e..7efe68e 100644
--- a/flink-connectors/flink-connector-filesystem/pom.xml
+++ b/flink-connectors/flink-connector-filesystem/pom.xml
@@ -59,7 +59,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-avro_${scala.binary.version}</artifactId>
+			<artifactId>flink-avro</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 			<!-- Projects depending on this project, won't depend on flink-avro. -->

http://git-wip-us.apache.org/repos/asf/flink/blob/eb99181d/flink-connectors/flink-connector-kafka-0.10/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/pom.xml b/flink-connectors/flink-connector-kafka-0.10/pom.xml
index 3357591..a01431a 100644
--- a/flink-connectors/flink-connector-kafka-0.10/pom.xml
+++ b/flink-connectors/flink-connector-kafka-0.10/pom.xml
@@ -84,7 +84,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-avro_${scala.binary.version}</artifactId>
+			<artifactId>flink-avro</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 			<!-- Projects depending on this project, won't depend on flink-avro. -->
@@ -95,7 +95,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-avro_${scala.binary.version}</artifactId>
+			<artifactId>flink-avro</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/flink/blob/eb99181d/flink-connectors/flink-connector-kafka-0.11/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.11/pom.xml b/flink-connectors/flink-connector-kafka-0.11/pom.xml
index 4f6be1d..f66a31d 100644
--- a/flink-connectors/flink-connector-kafka-0.11/pom.xml
+++ b/flink-connectors/flink-connector-kafka-0.11/pom.xml
@@ -84,7 +84,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-avro_${scala.binary.version}</artifactId>
+			<artifactId>flink-avro</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 			<!-- Projects depending on this project, won't depend on flink-avro. -->
@@ -104,7 +104,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-avro_${scala.binary.version}</artifactId>
+			<artifactId>flink-avro</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/flink/blob/eb99181d/flink-connectors/flink-connector-kafka-0.8/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/pom.xml b/flink-connectors/flink-connector-kafka-0.8/pom.xml
index b96274a..c9a1770 100644
--- a/flink-connectors/flink-connector-kafka-0.8/pom.xml
+++ b/flink-connectors/flink-connector-kafka-0.8/pom.xml
@@ -75,7 +75,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-avro_${scala.binary.version}</artifactId>
+			<artifactId>flink-avro</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 			<!-- Projects depending on this project, won't depend on flink-avro. -->
@@ -83,14 +83,6 @@ under the License.
 		</dependency>
 
 		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-avro_${scala.binary.version}</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-		</dependency>
-
-		<dependency>
 			<groupId>org.apache.kafka</groupId>
 			<artifactId>kafka_${scala.binary.version}</artifactId>
 			<version>${kafka.version}</version>
@@ -145,6 +137,14 @@ under the License.
 		</dependency>
 
 		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-avro</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+
+		<dependency>
 			<groupId>org.apache.curator</groupId>
 			<artifactId>curator-test</artifactId>
 			<version>${curator.version}</version>

http://git-wip-us.apache.org/repos/asf/flink/blob/eb99181d/flink-connectors/flink-connector-kafka-0.9/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/pom.xml b/flink-connectors/flink-connector-kafka-0.9/pom.xml
index c711c5f..ed322d3 100644
--- a/flink-connectors/flink-connector-kafka-0.9/pom.xml
+++ b/flink-connectors/flink-connector-kafka-0.9/pom.xml
@@ -74,7 +74,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-avro_${scala.binary.version}</artifactId>
+			<artifactId>flink-avro</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 			<!-- Projects depending on this project, won't depend on flink-avro. -->
@@ -91,7 +91,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-avro_${scala.binary.version}</artifactId>
+			<artifactId>flink-avro</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 			<type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/flink/blob/eb99181d/flink-connectors/flink-connector-kafka-base/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/pom.xml b/flink-connectors/flink-connector-kafka-base/pom.xml
index 4f2fb45..9743a01 100644
--- a/flink-connectors/flink-connector-kafka-base/pom.xml
+++ b/flink-connectors/flink-connector-kafka-base/pom.xml
@@ -68,7 +68,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-avro_${scala.binary.version}</artifactId>
+			<artifactId>flink-avro</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
 			<!-- Projects depending on this project, won't depend on flink-avro. -->
@@ -177,7 +177,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-avro_${scala.binary.version}</artifactId>
+			<artifactId>flink-avro</artifactId>
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>

http://git-wip-us.apache.org/repos/asf/flink/blob/eb99181d/flink-dist/pom.xml
----------------------------------------------------------------------
diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml
index 29f28ff..b2af9e8 100644
--- a/flink-dist/pom.xml
+++ b/flink-dist/pom.xml
@@ -81,7 +81,7 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-avro_${scala.binary.version}</artifactId>
+			<artifactId>flink-avro</artifactId>
 			<version>${project.version}</version>
 		</dependency>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/eb99181d/flink-formats/flink-avro/pom.xml
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/pom.xml b/flink-formats/flink-avro/pom.xml
index 5080b82..8732a21 100644
--- a/flink-formats/flink-avro/pom.xml
+++ b/flink-formats/flink-avro/pom.xml
@@ -29,7 +29,7 @@ under the License.
 		<relativePath>..</relativePath>
 	</parent>
 
-	<artifactId>flink-avro_${scala.binary.version}</artifactId>
+	<artifactId>flink-avro</artifactId>
 	<name>flink-avro</name>
 
 	<packaging>jar</packaging>
@@ -46,13 +46,6 @@ under the License.
 		</dependency>
 
 		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<dependency>
 			<groupId>org.apache.avro</groupId>
 			<artifactId>avro</artifactId>
 			<!-- managed version -->
@@ -67,30 +60,18 @@ under the License.
 			<scope>test</scope>
 		</dependency>
 
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
+		<!-- To avoid having to have the 'flink-avro' project dependent on a particular
+			Scala version, we hard-refer the flink-test-utils_2.11 here -->
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients_${scala.binary.version}</artifactId>
+			<artifactId>flink-test-utils_2.11</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
-			<version>${project.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-core</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
@@ -158,6 +139,7 @@ under the License.
 					</filesets>
 				</configuration>
 			</plugin>
+
 			<!-- Generate Test class from avro schema -->
 			<plugin>
 				<groupId>org.apache.avro</groupId>

http://git-wip-us.apache.org/repos/asf/flink/blob/eb99181d/flink-fs-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-fs-tests/pom.xml b/flink-fs-tests/pom.xml
index 446d167..3ac62df 100644
--- a/flink-fs-tests/pom.xml
+++ b/flink-fs-tests/pom.xml
@@ -60,7 +60,7 @@ under the License.
 		
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-avro_${scala.binary.version}</artifactId>
+			<artifactId>flink-avro</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
 		</dependency>

http://git-wip-us.apache.org/repos/asf/flink/blob/eb99181d/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
index 740e64a..34e2abb 100644
--- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
+++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml
@@ -205,7 +205,7 @@ under the License.
 									<exclude>org.apache.flink:flink-runtime_${scala.binary.version}</exclude>
 									<exclude>org.apache.flink:flink-optimizer_${scala.binary.version}</exclude>
 									<exclude>org.apache.flink:flink-clients_${scala.binary.version}</exclude>
-									<exclude>org.apache.flink:flink-avro_${scala.binary.version}</exclude>
+									<exclude>org.apache.flink:flink-avro</exclude>
 									<exclude>org.apache.flink:flink-examples-batch_${scala.binary.version}</exclude>
 									<exclude>org.apache.flink:flink-examples-streaming_${scala.binary.version}</exclude>
 									<exclude>org.apache.flink:flink-streaming-java_${scala.binary.version}</exclude>

http://git-wip-us.apache.org/repos/asf/flink/blob/eb99181d/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
----------------------------------------------------------------------
diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
index e5f6821..d705e8c 100644
--- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
+++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml
@@ -207,7 +207,7 @@ under the License.
 									<exclude>org.apache.flink:flink-runtime_${scala.binary.version}</exclude>
 									<exclude>org.apache.flink:flink-optimizer_${scala.binary.version}</exclude>
 									<exclude>org.apache.flink:flink-clients_${scala.binary.version}</exclude>
-									<exclude>org.apache.flink:flink-avro_${scala.binary.version}</exclude>
+									<exclude>org.apache.flink:flink-avro</exclude>
 									<exclude>org.apache.flink:flink-examples-batch_${scala.binary.version}</exclude>
 									<exclude>org.apache.flink:flink-examples-streaming_${scala.binary.version}</exclude>
 									<exclude>org.apache.flink:flink-streaming-java_${scala.binary.version}</exclude>


[15/21] flink git commit: [hotfix] [avro] Remove incorrect serializability from DataOutputEncoder

Posted by se...@apache.org.
[hotfix] [avro] Remove incorrect serializability from DataOutputEncoder


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

Branch: refs/heads/master
Commit: a042ba9fdbb9251185cf4ac756601c9de56a92a6
Parents: 25dcdea
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Nov 2 19:55:33 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Nov 3 16:40:35 2017 +0100

----------------------------------------------------------------------
 .../org/apache/flink/formats/avro/utils/DataOutputEncoder.java   | 4 +---
 1 file changed, 1 insertion(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/a042ba9f/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/DataOutputEncoder.java
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/DataOutputEncoder.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/DataOutputEncoder.java
index c2d490b..01e5137 100644
--- a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/DataOutputEncoder.java
+++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/DataOutputEncoder.java
@@ -28,9 +28,7 @@ import java.nio.ByteBuffer;
 /**
  * An {@link Encoder} that writes data to a {@link DataOutput}.
  */
-public final class DataOutputEncoder extends Encoder implements java.io.Serializable {
-
-	private static final long serialVersionUID = 1L;
+public final class DataOutputEncoder extends Encoder {
 
 	private DataOutput out;
 


[05/21] flink git commit: [FLINK-7972] [core] Move SerializationSchema to 'flink-core'

Posted by se...@apache.org.
[FLINK-7972] [core] Move SerializationSchema to 'flink-core'

Moves the SerializationSchema and its related from
flink-streaming-java to flink-core.

That helps API level projects that depend on those classes
to not pull in a dependency on runtime classes, and to
not be Scala version dependent.


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

Branch: refs/heads/master
Commit: fe931d075f031e9494fd26dbeed4bb1024bd52cf
Parents: 16b0882
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Nov 2 18:07:25 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Nov 3 16:40:34 2017 +0100

----------------------------------------------------------------------
 docs/quickstart/run_example_quickstart.md       |   2 +-
 .../connectors/kafka/FlinkKafkaConsumer010.java |   2 +-
 .../connectors/kafka/FlinkKafkaProducer010.java |   2 +-
 .../kafka/Kafka010AvroTableSource.java          |   2 +-
 .../kafka/Kafka010JsonTableSource.java          |   2 +-
 .../connectors/kafka/Kafka010TableSource.java   |   2 +-
 .../kafka/Kafka010AvroTableSourceTest.java      |   4 +-
 .../connectors/kafka/Kafka010ITCase.java        |   2 +-
 .../kafka/Kafka010JsonTableSourceTest.java      |   2 +-
 .../kafka/internal/Kafka010FetcherTest.java     |   2 +-
 .../connectors/kafka/FlinkKafkaConsumer011.java |   2 +-
 .../connectors/kafka/FlinkKafkaProducer011.java |   3 +-
 .../kafka/Kafka011AvroTableSource.java          |   2 +-
 .../kafka/Kafka011JsonTableSource.java          |   2 +-
 .../connectors/kafka/Kafka011TableSource.java   |   2 +-
 .../kafka/FlinkKafkaProducer011ITCase.java      |   2 +-
 .../kafka/Kafka011AvroTableSourceTest.java      |   4 +-
 .../connectors/kafka/Kafka011ITCase.java        |   2 +-
 .../kafka/Kafka011JsonTableSourceTest.java      |   2 +-
 .../connectors/kafka/FlinkKafkaConsumer08.java  |   2 +-
 .../connectors/kafka/FlinkKafkaConsumer081.java |   2 +-
 .../connectors/kafka/FlinkKafkaConsumer082.java |   2 +-
 .../connectors/kafka/FlinkKafkaProducer.java    |   2 +-
 .../connectors/kafka/FlinkKafkaProducer08.java  |   2 +-
 .../kafka/Kafka08AvroTableSource.java           |   2 +-
 .../connectors/kafka/Kafka08JsonTableSink.java  |   2 +-
 .../kafka/Kafka08JsonTableSource.java           |   2 +-
 .../connectors/kafka/Kafka08TableSource.java    |   2 +-
 .../kafka/Kafka08AvroTableSourceTest.java       |   4 +-
 .../kafka/Kafka08JsonTableSinkTest.java         |   2 +-
 .../kafka/Kafka08JsonTableSourceTest.java       |   2 +-
 .../connectors/kafka/KafkaConsumer08Test.java   |   4 +-
 .../connectors/kafka/KafkaProducerTest.java     |   2 +-
 .../connectors/kafka/FlinkKafkaConsumer09.java  |   2 +-
 .../connectors/kafka/FlinkKafkaProducer09.java  |   2 +-
 .../kafka/Kafka09AvroTableSource.java           |   2 +-
 .../connectors/kafka/Kafka09JsonTableSink.java  |   2 +-
 .../kafka/Kafka09JsonTableSource.java           |   2 +-
 .../connectors/kafka/Kafka09TableSource.java    |   2 +-
 .../kafka/Kafka09AvroTableSourceTest.java       |   4 +-
 .../kafka/Kafka09JsonTableSinkTest.java         |   2 +-
 .../kafka/Kafka09JsonTableSourceTest.java       |   2 +-
 .../connectors/kafka/KafkaProducerTest.java     |   2 +-
 .../kafka/internal/Kafka09FetcherTest.java      |   2 +-
 .../connectors/kafka/KafkaAvroTableSource.java  |   6 +-
 .../connectors/kafka/KafkaJsonTableSink.java    |   2 +-
 .../connectors/kafka/KafkaJsonTableSource.java  |   2 +-
 .../connectors/kafka/KafkaTableSink.java        |   2 +-
 .../connectors/kafka/KafkaTableSource.java      |   2 +-
 .../AvroRowDeserializationSchema.java           |   1 +
 .../AvroRowSerializationSchema.java             |   1 +
 .../JSONDeserializationSchema.java              |   2 +
 .../JsonRowDeserializationSchema.java           |   1 +
 .../JsonRowSerializationSchema.java             |   1 +
 .../KeyedDeserializationSchemaWrapper.java      |   1 +
 .../KeyedSerializationSchemaWrapper.java        |   2 +
 .../kafka/FlinkKafkaProducerBaseTest.java       |   2 +-
 .../connectors/kafka/KafkaConsumerTestBase.java |   6 +-
 .../connectors/kafka/KafkaProducerTestBase.java |   4 +-
 .../kafka/KafkaShortRetentionTestBase.java      |   2 +-
 .../kafka/KafkaTableSinkTestBase.java           |   2 +-
 .../kafka/KafkaTableSourceTestBase.java         |   2 +-
 .../connectors/kafka/KafkaTestEnvironment.java  |   3 +-
 .../kafka/testutils/DataGenerators.java         |   4 +-
 .../streaming/connectors/rabbitmq/RMQSink.java  |   2 +-
 .../connectors/rabbitmq/RMQSource.java          |   2 +-
 .../connectors/rabbitmq/RMQSinkTest.java        |   2 +-
 .../connectors/rabbitmq/RMQSourceTest.java      |   2 +-
 flink-core/pom.xml                              |  15 +++
 .../AbstractDeserializationSchema.java          |  70 ++++++++++
 .../serialization/DeserializationSchema.java    |  56 ++++++++
 .../serialization/SerializationSchema.java      |  42 ++++++
 .../serialization/SimpleStringSchema.java       | 107 +++++++++++++++
 .../TypeInformationSerializationSchema.java     | 131 +++++++++++++++++++
 .../AbstractDeserializationSchemaTest.java      | 119 +++++++++++++++++
 .../serialization/SimpleStringSchemaTest.java   |  53 ++++++++
 .../TypeInformationSerializationSchemaTest.java | 121 +++++++++++++++++
 .../examples/kafka/Kafka010Example.java         |   2 +-
 .../scala/examples/kafka/Kafka010Example.scala  |   2 +-
 flink-streaming-java/pom.xml                    |   6 -
 .../streaming/api/datastream/DataStream.java    |   2 +-
 .../api/functions/sink/SocketClientSink.java    |   2 +-
 .../AbstractDeserializationSchema.java          |  43 ++----
 .../serialization/DeserializationSchema.java    |  10 +-
 .../util/serialization/SerializationSchema.java |   7 +-
 .../util/serialization/SimpleStringSchema.java  |  73 ++---------
 .../TypeInformationSerializationSchema.java     |  96 ++------------
 .../functions/sink/SocketClientSinkTest.java    |   2 +-
 .../util/AbstractDeserializationSchemaTest.java | 120 -----------------
 .../TypeInformationSerializationSchemaTest.java | 122 -----------------
 .../serialization/SimpleStringSchemaTest.java   |  53 --------
 .../flink/streaming/api/scala/DataStream.scala  |   2 +-
 .../api/scala/OutputFormatTestPrograms.scala    |   2 +-
 93 files changed, 845 insertions(+), 568 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/docs/quickstart/run_example_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/quickstart/run_example_quickstart.md b/docs/quickstart/run_example_quickstart.md
index cf73799..d5c48c9 100644
--- a/docs/quickstart/run_example_quickstart.md
+++ b/docs/quickstart/run_example_quickstart.md
@@ -326,7 +326,7 @@ result
 The related classes also need to be imported:
 {% highlight java %}
 import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer08;
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
 import org.apache.flink.api.common.functions.MapFunction;
 {% endhighlight %}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java
index 3a6a13b..f569477 100644
--- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java
+++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java
@@ -17,6 +17,7 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
 import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
 import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
@@ -27,7 +28,6 @@ import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher;
 import org.apache.flink.streaming.connectors.kafka.internals.AbstractPartitionDiscoverer;
 import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
 import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
 import org.apache.flink.util.PropertiesUtil;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java
index 8575268..184a2e7 100644
--- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java
+++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.java
@@ -17,6 +17,7 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.SerializationSchema;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.DataStreamSink;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
@@ -28,7 +29,6 @@ import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartiti
 import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
 
 import org.apache.kafka.clients.producer.ProducerRecord;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSource.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSource.java
index 01e6329..fbc58ea 100644
--- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSource.java
+++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSource.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.sources.RowtimeAttributeDescriptor;
 import org.apache.flink.table.sources.StreamTableSource;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java
index e263cf2..bbdb32f 100644
--- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java
+++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.sources.RowtimeAttributeDescriptor;
 import org.apache.flink.table.sources.StreamTableSource;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java
index 5475c9f..bc675eb 100644
--- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java
+++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java
@@ -18,8 +18,8 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.sources.StreamTableSource;
 import org.apache.flink.types.Row;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSourceTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSourceTest.java
index e977d49..f5f8af1 100644
--- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSourceTest.java
+++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSourceTest.java
@@ -18,8 +18,8 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-import org.apache.flink.streaming.util.serialization.AvroRowDeserializationSchema;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.formats.avro.AvroRowDeserializationSchema;
 import org.apache.flink.types.Row;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java
index 3aa0d1a..c2b3dfa 100644
--- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java
+++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java
@@ -19,6 +19,7 @@ package org.apache.flink.streaming.connectors.kafka;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.typeutils.GenericTypeInfo;
@@ -37,7 +38,6 @@ import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartiti
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema;
 
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceTest.java
index 9a0ab04..0f6c531 100644
--- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceTest.java
+++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceTest.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema;
 import org.apache.flink.types.Row;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010FetcherTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010FetcherTest.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010FetcherTest.java
index f895e2f..45ceadc 100644
--- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010FetcherTest.java
+++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010FetcherTest.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.streaming.connectors.kafka.internal;
 
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
 import org.apache.flink.core.testutils.MultiShotLatch;
 import org.apache.flink.core.testutils.OneShotLatch;
 import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
@@ -29,7 +30,6 @@ import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition
 import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
 
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.consumer.ConsumerRecords;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer011.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer011.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer011.java
index 8d165c3..6f75828 100644
--- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer011.java
+++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer011.java
@@ -17,7 +17,7 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
index c27c620..873ef08 100644
--- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
+++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java
@@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal;
 import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.serialization.SerializationSchema;
 import org.apache.flink.api.common.state.ListState;
 import org.apache.flink.api.common.state.ListStateDescriptor;
 import org.apache.flink.api.common.time.Time;
@@ -42,7 +43,6 @@ import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegat
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
 import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.NetUtils;
 
@@ -59,6 +59,7 @@ import org.apache.kafka.common.MetricName;
 import org.apache.kafka.common.PartitionInfo;
 import org.apache.kafka.common.errors.InvalidTxnStateException;
 import org.apache.kafka.common.serialization.ByteArraySerializer;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSource.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSource.java
index 81d3496..af3b5af 100644
--- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSource.java
+++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSource.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.sources.RowtimeAttributeDescriptor;
 import org.apache.flink.table.sources.StreamTableSource;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSource.java
index 1aff670..71158f6 100644
--- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSource.java
+++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSource.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.sources.RowtimeAttributeDescriptor;
 import org.apache.flink.table.sources.StreamTableSource;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java
index 576a421..dbf980b 100644
--- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java
+++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java
@@ -18,8 +18,8 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.sources.StreamTableSource;
 import org.apache.flink.types.Row;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java
index 295451f..922344d 100644
--- a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java
+++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java
@@ -19,6 +19,7 @@
 package org.apache.flink.streaming.connectors.kafka;
 
 import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
 import org.apache.flink.runtime.state.OperatorStateHandle;
@@ -27,7 +28,6 @@ import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema;
 
 import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSourceTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSourceTest.java
index bde0761..e348aa6 100644
--- a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSourceTest.java
+++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSourceTest.java
@@ -18,8 +18,8 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-import org.apache.flink.streaming.util.serialization.AvroRowDeserializationSchema;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.formats.avro.AvroRowDeserializationSchema;
 import org.apache.flink.types.Row;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011ITCase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011ITCase.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011ITCase.java
index 6d259fa..99d5b56 100644
--- a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011ITCase.java
+++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011ITCase.java
@@ -19,6 +19,7 @@ package org.apache.flink.streaming.connectors.kafka;
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.typeutils.GenericTypeInfo;
@@ -37,7 +38,6 @@ import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartiti
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema;
 
 import org.junit.BeforeClass;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceTest.java
index 6870ecc..fc2957e 100644
--- a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceTest.java
+++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceTest.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema;
 import org.apache.flink.types.Row;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java
index f41a4e3..0a70f61 100644
--- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java
+++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java
@@ -17,6 +17,7 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
 import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
 import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
@@ -27,7 +28,6 @@ import org.apache.flink.streaming.connectors.kafka.internals.Kafka08Fetcher;
 import org.apache.flink.streaming.connectors.kafka.internals.Kafka08PartitionDiscoverer;
 import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
 import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
 import org.apache.flink.util.PropertiesUtil;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java
index 4102bf8..c65ccc1 100644
--- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java
+++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer081.java
@@ -17,7 +17,7 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 
 import java.util.Properties;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java
index 7ba5103..e5374fb 100644
--- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java
+++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer082.java
@@ -17,7 +17,7 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 
 import java.util.Properties;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
index 434286e..1911c8d 100644
--- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
+++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
@@ -17,11 +17,11 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.SerializationSchema;
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
 import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
 
 import java.util.Properties;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java
index a14768b..2fce9f9 100644
--- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java
+++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer08.java
@@ -17,13 +17,13 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.SerializationSchema;
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
 import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
 
 import java.util.Properties;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSource.java
index 998f4d4..8f45881 100644
--- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSource.java
+++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSource.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.sources.RowtimeAttributeDescriptor;
 import org.apache.flink.table.sources.StreamTableSource;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java
index 79406d8..a887048 100644
--- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java
+++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java
@@ -18,10 +18,10 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.SerializationSchema;
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
 import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
 import org.apache.flink.types.Row;
 
 import java.util.Properties;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java
index aab9ea8..b3b37c6 100644
--- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java
+++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.sources.RowtimeAttributeDescriptor;
 import org.apache.flink.table.sources.StreamTableSource;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java
index b2b949b..8270b78 100644
--- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java
+++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java
@@ -18,8 +18,8 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.sources.StreamTableSource;
 import org.apache.flink.types.Row;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSourceTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSourceTest.java
index 348f0f2..22daafe 100644
--- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSourceTest.java
+++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSourceTest.java
@@ -18,8 +18,8 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-import org.apache.flink.streaming.util.serialization.AvroRowDeserializationSchema;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.formats.avro.AvroRowDeserializationSchema;
 import org.apache.flink.types.Row;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java
index ac92c8a..890fc3a 100644
--- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java
+++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java
@@ -18,9 +18,9 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.SerializationSchema;
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
 import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
 import org.apache.flink.types.Row;
 
 import java.util.Properties;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java
index 01a0123..7e3349c 100644
--- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java
+++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema;
 import org.apache.flink.types.Row;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java
index 9e4d3b7..8627ccb 100644
--- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java
+++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumer08Test.java
@@ -19,11 +19,11 @@
 package org.apache.flink.streaming.connectors.kafka;
 
 import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
 import org.apache.flink.streaming.connectors.kafka.internals.Kafka08PartitionDiscoverer;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
 import org.apache.flink.util.NetUtils;
 
 import org.apache.kafka.clients.consumer.ConsumerConfig;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
index fc8678f..304351c 100644
--- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
+++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
@@ -18,12 +18,12 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
 import org.apache.flink.streaming.api.operators.StreamSink;
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
 import org.apache.flink.streaming.connectors.kafka.testutils.FakeStandardProducerConfig;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
 import org.apache.flink.util.TestLogger;
 
 import org.apache.kafka.clients.producer.Callback;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
index 0cf40e6..65be712 100644
--- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
+++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java
@@ -17,6 +17,7 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks;
 import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks;
 import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
@@ -27,7 +28,6 @@ import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher;
 import org.apache.flink.streaming.connectors.kafka.internals.AbstractPartitionDiscoverer;
 import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
 import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
 import org.apache.flink.util.PropertiesUtil;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java
index 6b9768e..946f7e9 100644
--- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java
+++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer09.java
@@ -17,13 +17,13 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.SerializationSchema;
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
 import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
 
 import java.util.Properties;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSource.java
index a90a8d8..808be01 100644
--- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSource.java
+++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSource.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.sources.RowtimeAttributeDescriptor;
 import org.apache.flink.table.sources.StreamTableSource;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java
index b2227cd..f65a02d 100644
--- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java
+++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java
@@ -18,10 +18,10 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.SerializationSchema;
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
 import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
 import org.apache.flink.types.Row;
 
 import java.util.Properties;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java
index 2f057d7..a699d65 100644
--- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java
+++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.sources.RowtimeAttributeDescriptor;
 import org.apache.flink.table.sources.StreamTableSource;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java
index 4d1166c..1d2c028 100644
--- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java
+++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java
@@ -18,8 +18,8 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.sources.StreamTableSource;
 import org.apache.flink.types.Row;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSourceTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSourceTest.java
index 27445e1..0ab8a4b 100644
--- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSourceTest.java
+++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSourceTest.java
@@ -18,8 +18,8 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-import org.apache.flink.streaming.util.serialization.AvroRowDeserializationSchema;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.formats.avro.AvroRowDeserializationSchema;
 import org.apache.flink.types.Row;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java
index c8fb4cd..c52b4ca 100644
--- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java
+++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java
@@ -18,9 +18,9 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.SerializationSchema;
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
 import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
 import org.apache.flink.types.Row;
 
 import java.util.Properties;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java
index ed3fafb..aeee175 100644
--- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java
+++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema;
 import org.apache.flink.types.Row;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
index 6b6c43f..37f3fe8 100644
--- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
+++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTest.java
@@ -18,12 +18,12 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
 import org.apache.flink.streaming.api.operators.StreamSink;
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
 import org.apache.flink.streaming.connectors.kafka.testutils.FakeStandardProducerConfig;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
 import org.apache.flink.util.TestLogger;
 
 import org.apache.kafka.clients.producer.Callback;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09FetcherTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09FetcherTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09FetcherTest.java
index 33ec17e..e4e276a 100644
--- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09FetcherTest.java
+++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09FetcherTest.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.streaming.connectors.kafka.internal;
 
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
 import org.apache.flink.core.testutils.MultiShotLatch;
 import org.apache.flink.core.testutils.OneShotLatch;
 import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
@@ -29,7 +30,6 @@ import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition
 import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
 
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.consumer.ConsumerRecords;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java
index 0cc9801..8cea36c 100644
--- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java
@@ -18,13 +18,13 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.AvroTypeInfo;
 import org.apache.flink.api.java.typeutils.GenericTypeInfo;
 import org.apache.flink.api.java.typeutils.RowTypeInfo;
-import org.apache.flink.streaming.util.serialization.AvroRowDeserializationSchema;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
+import org.apache.flink.formats.avro.AvroRowDeserializationSchema;
+import org.apache.flink.formats.avro.typeutils.AvroTypeInfo;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.sources.DefinedFieldMapping;
 import org.apache.flink.table.sources.StreamTableSource;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java
index 51fd952..f354dad 100644
--- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java
@@ -18,9 +18,9 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.SerializationSchema;
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
 import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
 import org.apache.flink.types.Row;
 
 import java.util.Properties;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java
index a91cc25..9a6525c 100644
--- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java
@@ -18,8 +18,8 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.api.java.typeutils.RowTypeInfo;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.sources.DefinedFieldMapping;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java
index a94936c..cac71dc 100644
--- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java
@@ -18,11 +18,11 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.SerializationSchema;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.typeutils.RowTypeInfo;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
 import org.apache.flink.table.sinks.AppendStreamTableSink;
 import org.apache.flink.types.Row;
 import org.apache.flink.util.Preconditions;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java
index 0bd04e4..3291f7d 100644
--- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java
@@ -18,10 +18,10 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.api.Types;
 import org.apache.flink.table.api.ValidationException;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowDeserializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowDeserializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowDeserializationSchema.java
index c7f1d82..0d36f4c 100644
--- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowDeserializationSchema.java
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowDeserializationSchema.java
@@ -17,6 +17,7 @@
 
 package org.apache.flink.streaming.util.serialization;
 
+import org.apache.flink.api.common.serialization.AbstractDeserializationSchema;
 import org.apache.flink.types.Row;
 import org.apache.flink.util.Preconditions;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowSerializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowSerializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowSerializationSchema.java
index 09acc6a..6f03b12 100644
--- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowSerializationSchema.java
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/AvroRowSerializationSchema.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.streaming.util.serialization;
 
+import org.apache.flink.api.common.serialization.SerializationSchema;
 import org.apache.flink.types.Row;
 import org.apache.flink.util.Preconditions;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONDeserializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONDeserializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONDeserializationSchema.java
index 4523da7..f60a0b7 100644
--- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONDeserializationSchema.java
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JSONDeserializationSchema.java
@@ -17,6 +17,8 @@
 
 package org.apache.flink.streaming.util.serialization;
 
+import org.apache.flink.api.common.serialization.AbstractDeserializationSchema;
+
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowDeserializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowDeserializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowDeserializationSchema.java
index d777c8d..100f960 100644
--- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowDeserializationSchema.java
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowDeserializationSchema.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.streaming.util.serialization;
 
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.typeutils.RowTypeInfo;
 import org.apache.flink.types.Row;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java
index 13a3677..5ece193 100644
--- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java
@@ -17,6 +17,7 @@
 
 package org.apache.flink.streaming.util.serialization;
 
+import org.apache.flink.api.common.serialization.SerializationSchema;
 import org.apache.flink.types.Row;
 import org.apache.flink.util.Preconditions;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java
index e128aba..93b4f68 100644
--- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedDeserializationSchemaWrapper.java
@@ -17,6 +17,7 @@
 
 package org.apache.flink.streaming.util.serialization;
 
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchemaWrapper.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchemaWrapper.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchemaWrapper.java
index 0a181d1..70ae897 100644
--- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchemaWrapper.java
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/KeyedSerializationSchemaWrapper.java
@@ -17,6 +17,8 @@
 
 package org.apache.flink.streaming.util.serialization;
 
+import org.apache.flink.api.common.serialization.SerializationSchema;
+
 /**
  * A simple wrapper for using the SerializationSchema with the KeyedDeserializationSchema
  * interface.

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java
index 6b4b6ff..d462953 100644
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java
@@ -19,6 +19,7 @@
 package org.apache.flink.streaming.connectors.kafka;
 
 import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.testutils.CheckedThread;
 import org.apache.flink.core.testutils.MultiShotLatch;
@@ -31,7 +32,6 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
 
 import org.apache.kafka.clients.producer.Callback;
 import org.apache.kafka.clients.producer.KafkaProducer;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
index e9a0331..0a5608a 100644
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java
@@ -23,6 +23,9 @@ import org.apache.flink.api.common.functions.FlatMapFunction;
 import org.apache.flink.api.common.functions.RichFlatMapFunction;
 import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeHint;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -60,14 +63,11 @@ import org.apache.flink.streaming.connectors.kafka.testutils.PartitionValidating
 import org.apache.flink.streaming.connectors.kafka.testutils.ThrottledMapper;
 import org.apache.flink.streaming.connectors.kafka.testutils.Tuple2FlinkPartitioner;
 import org.apache.flink.streaming.connectors.kafka.testutils.ValidatingExactlyOnceSink;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
 import org.apache.flink.streaming.util.serialization.TypeInformationKeyValueSerializationSchema;
-import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema;
 import org.apache.flink.test.util.SuccessException;
 import org.apache.flink.testutils.junit.RetryOnException;
 import org.apache.flink.testutils.junit.RetryRule;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java
index f81fcf1..7ba3c95 100644
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java
@@ -21,6 +21,8 @@ package org.apache.flink.streaming.connectors.kafka;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.tuple.Tuple2;
@@ -41,8 +43,6 @@ import org.apache.flink.streaming.connectors.kafka.testutils.FailingIdentityMapp
 import org.apache.flink.streaming.connectors.kafka.testutils.IntegerSource;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
-import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema;
 import org.apache.flink.test.util.SuccessException;
 import org.apache.flink.test.util.TestUtils;
 import org.apache.flink.util.Preconditions;


[08/21] flink git commit: [FLINK-7420] [avro] Move all Avro code to flink-avro

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/twitter/TwitterExample.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/twitter/TwitterExample.scala b/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/twitter/TwitterExample.scala
index d916116..048e7ac 100644
--- a/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/twitter/TwitterExample.scala
+++ b/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/twitter/TwitterExample.scala
@@ -23,12 +23,11 @@ import java.util.StringTokenizer
 import org.apache.flink.api.common.functions.FlatMapFunction
 import org.apache.flink.api.java.utils.ParameterTool
 import org.apache.flink.api.scala._
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.{JsonNode, ObjectMapper}
 import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment}
 import org.apache.flink.streaming.connectors.twitter.TwitterSource
 import org.apache.flink.streaming.examples.twitter.util.TwitterExampleData
 import org.apache.flink.util.Collector
-import org.codehaus.jackson.JsonNode
-import org.codehaus.jackson.map.ObjectMapper
 
 import scala.collection.mutable.ListBuffer
 

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-formats/flink-avro/pom.xml
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/pom.xml b/flink-formats/flink-avro/pom.xml
new file mode 100644
index 0000000..19d9129
--- /dev/null
+++ b/flink-formats/flink-avro/pom.xml
@@ -0,0 +1,280 @@
+<?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-formats</artifactId>
+		<version>1.4-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-avro_${scala.binary.version}</artifactId>
+	<name>flink-avro</name>
+
+	<packaging>jar</packaging>
+
+	<dependencies>
+
+		<!-- core dependencies -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-core</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.avro</groupId>
+			<artifactId>avro</artifactId>
+			<version>1.8.2</version>
+		</dependency>
+
+		<!-- test dependencies -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils-junit</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-clients_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-core</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<artifactId>maven-assembly-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>create-test-dependency</id>
+						<phase>process-test-classes</phase>
+						<goals>
+							<goal>single</goal>
+						</goals>
+						<configuration>
+							<archive>
+								<manifest>
+									<mainClass>org.apache.flink.formats.avro.testjar.AvroExternalJarProgram</mainClass>
+								</manifest>
+							</archive>
+							<finalName>maven</finalName>
+							<attach>false</attach>
+							<descriptors>
+								<descriptor>src/test/assembly/test-assembly.xml</descriptor>
+							</descriptors>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+			<!--Remove the AvroExternalJarProgram code from the test-classes directory since it musn't be in the
+			classpath when running the tests to actually test whether the user code class loader
+			is properly used.-->
+			<plugin>
+				<artifactId>maven-clean-plugin</artifactId>
+				<version>2.5</version><!--$NO-MVN-MAN-VER$-->
+				<executions>
+					<execution>
+						<id>remove-avroexternalprogram</id>
+						<phase>process-test-classes</phase>
+						<goals>
+							<goal>clean</goal>
+						</goals>
+						<configuration>
+							<excludeDefaultDirectories>true</excludeDefaultDirectories>
+							<filesets>
+								<fileset>
+									<directory>${project.build.testOutputDirectory}</directory>
+									<includes>
+										<include>**/testjar/*.class</include>
+									</includes>
+								</fileset>
+							</filesets>
+						</configuration>
+					</execution>
+				</executions>
+				<configuration>
+					<filesets>
+						<fileset>
+							<directory>${project.basedir}/src/test/java/org/apache/flink/formats/avro/generated</directory>
+						</fileset>
+					</filesets>
+				</configuration>
+			</plugin>
+			<!-- Generate Test class from avro schema -->
+			<plugin>
+				<groupId>org.apache.avro</groupId>
+				<artifactId>avro-maven-plugin</artifactId>
+				<version>1.8.2</version>
+				<executions>
+					<execution>
+						<phase>generate-sources</phase>
+						<goals>
+							<goal>schema</goal>
+						</goals>
+						<configuration>
+							<testSourceDirectory>${project.basedir}/src/test/resources/avro</testSourceDirectory>
+							<testOutputDirectory>${project.basedir}/src/test/java/</testOutputDirectory>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+
+			<!-- Add Avro test classes to test jar in order to test AvroTypeInfo. -->
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-jar-plugin</artifactId>
+				<executions>
+					<execution>
+						<goals>
+							<goal>test-jar</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-shade-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>shade-flink</id>
+						<phase>package</phase>
+						<goals>
+							<goal>shade</goal>
+						</goals>
+						<configuration combine.self="override">
+							<dependencyReducedPomLocation>${project.basedir}/target/dependency-reduced-pom.xml</dependencyReducedPomLocation>
+							<promoteTransitiveDependencies>true</promoteTransitiveDependencies>
+							<artifactSet>
+								<includes>
+									<include>org.codehaus.jackson:*</include>
+								</includes>
+							</artifactSet>
+							<relocations>
+								<relocation>
+									<pattern>org.codehaus.jackson</pattern>
+									<shadedPattern>org.apache.flink.avro.shaded.org.codehouse.jackson</shadedPattern>
+								</relocation>
+							</relocations>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+
+		<pluginManagement>
+			<plugins>
+				<!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
+				<plugin>
+					<groupId>org.eclipse.m2e</groupId>
+					<artifactId>lifecycle-mapping</artifactId>
+					<version>1.0.0</version>
+					<configuration>
+						<lifecycleMappingMetadata>
+							<pluginExecutions>
+								<pluginExecution>
+									<pluginExecutionFilter>
+										<groupId>org.apache.maven.plugins</groupId>
+										<artifactId>maven-assembly-plugin</artifactId>
+										<versionRange>[2.4,)</versionRange>
+										<goals>
+											<goal>single</goal>
+										</goals>
+									</pluginExecutionFilter>
+									<action>
+										<ignore/>
+									</action>
+								</pluginExecution>
+								<pluginExecution>
+									<pluginExecutionFilter>
+										<groupId>org.apache.maven.plugins</groupId>
+										<artifactId>maven-clean-plugin</artifactId>
+										<versionRange>[1,)</versionRange>
+										<goals>
+											<goal>clean</goal>
+										</goals>
+									</pluginExecutionFilter>
+									<action>
+										<ignore/>
+									</action>
+								</pluginExecution>
+								<pluginExecution>
+									<pluginExecutionFilter>
+										<groupId>org.apache.avro</groupId>
+										<artifactId>avro-maven-plugin</artifactId>
+										<versionRange>[1.7.7,)</versionRange>
+										<goals>
+											<goal>schema</goal>
+										</goals>
+									</pluginExecutionFilter>
+									<action>
+										<ignore/>
+									</action>
+								</pluginExecution>
+							</pluginExecutions>
+						</lifecycleMappingMetadata>
+					</configuration>
+				</plugin>
+			</plugins>
+		</pluginManagement>
+	</build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-formats/flink-avro/src/main/java/org/apache/flink/api/java/typeutils/AvroTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/api/java/typeutils/AvroTypeInfo.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/api/java/typeutils/AvroTypeInfo.java
new file mode 100644
index 0000000..58085f6
--- /dev/null
+++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/api/java/typeutils/AvroTypeInfo.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.typeutils;
+
+import org.apache.flink.annotation.Public;
+
+import org.apache.avro.specific.SpecificRecordBase;
+
+import static org.apache.flink.formats.avro.typeutils.AvroTypeInfo.generateFieldsFromAvroSchema;
+
+/**
+ * @deprecated Please use <code>org.apache.flink.formats.avro.typeutils.AvroTypeInfo</code>
+ * in the <code>flink-avro</code> module. This class will be removed in the near future.
+ */
+@Deprecated
+@Public
+public class AvroTypeInfo<T extends SpecificRecordBase> extends PojoTypeInfo<T> {
+
+	public AvroTypeInfo(Class<T> typeClass) {
+		super(typeClass, generateFieldsFromAvroSchema(typeClass));
+	}
+}

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDeserializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDeserializationSchema.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDeserializationSchema.java
new file mode 100644
index 0000000..4a3c02e
--- /dev/null
+++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDeserializationSchema.java
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.formats.avro;
+
+import org.apache.flink.api.common.serialization.AbstractDeserializationSchema;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.Decoder;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.specific.SpecificData;
+import org.apache.avro.specific.SpecificDatumReader;
+import org.apache.avro.specific.SpecificRecord;
+import org.apache.avro.util.Utf8;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.List;
+
+/**
+ * Deserialization schema from Avro bytes over {@link SpecificRecord} to {@link Row}.
+ *
+ * <p>Deserializes the <code>byte[]</code> messages into (nested) Flink Rows.
+ *
+ * {@link Utf8} is converted to regular Java Strings.
+ */
+public class AvroRowDeserializationSchema extends AbstractDeserializationSchema<Row> {
+
+	/**
+	 * Avro record class.
+	 */
+	private Class<? extends SpecificRecord> recordClazz;
+
+	/**
+	 * Schema for deterministic field order.
+	 */
+	private transient Schema schema;
+
+	/**
+	 * Reader that deserializes byte array into a record.
+	 */
+	private transient DatumReader<SpecificRecord> datumReader;
+
+	/**
+	 * Input stream to read message from.
+	 */
+	private transient MutableByteArrayInputStream inputStream;
+
+	/**
+	 * Avro decoder that decodes binary data.
+	 */
+	private transient Decoder decoder;
+
+	/**
+	 * Record to deserialize byte array to.
+	 */
+	private SpecificRecord record;
+
+	/**
+	 * Creates a Avro deserialization schema for the given record.
+	 *
+	 * @param recordClazz Avro record class used to deserialize Avro's record to Flink's row
+	 */
+	public AvroRowDeserializationSchema(Class<? extends SpecificRecord> recordClazz) {
+		Preconditions.checkNotNull(recordClazz, "Avro record class must not be null.");
+		this.recordClazz = recordClazz;
+		this.schema = SpecificData.get().getSchema(recordClazz);
+		this.datumReader = new SpecificDatumReader<>(schema);
+		this.record = (SpecificRecord) SpecificData.newInstance(recordClazz, schema);
+		this.inputStream = new MutableByteArrayInputStream();
+		this.decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+	}
+
+	@Override
+	public Row deserialize(byte[] message) throws IOException {
+		// read record
+		try {
+			inputStream.setBuffer(message);
+			this.record = datumReader.read(record, decoder);
+		} catch (IOException e) {
+			throw new RuntimeException("Failed to deserialize Row.", e);
+		}
+
+		// convert to row
+		final Object row = convertToRow(schema, record);
+		return (Row) row;
+	}
+
+	private void writeObject(ObjectOutputStream oos) throws IOException {
+		oos.writeObject(recordClazz);
+	}
+
+	@SuppressWarnings("unchecked")
+	private void readObject(ObjectInputStream ois) throws ClassNotFoundException, IOException {
+		this.recordClazz = (Class<? extends SpecificRecord>) ois.readObject();
+		this.schema = SpecificData.get().getSchema(recordClazz);
+		this.datumReader = new SpecificDatumReader<>(schema);
+		this.record = (SpecificRecord) SpecificData.newInstance(recordClazz, schema);
+		this.inputStream = new MutableByteArrayInputStream();
+		this.decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+	}
+
+	/**
+	 * Converts a (nested) Avro {@link SpecificRecord} into Flink's Row type.
+	 * Avro's {@link Utf8} fields are converted into regular Java strings.
+	 */
+	private static Object convertToRow(Schema schema, Object recordObj) {
+		if (recordObj instanceof GenericRecord) {
+			// records can be wrapped in a union
+			if (schema.getType() == Schema.Type.UNION) {
+				final List<Schema> types = schema.getTypes();
+				if (types.size() == 2 && types.get(0).getType() == Schema.Type.NULL && types.get(1).getType() == Schema.Type.RECORD) {
+					schema = types.get(1);
+				}
+				else {
+					throw new RuntimeException("Currently we only support schemas of the following form: UNION[null, RECORD]. Given: " + schema);
+				}
+			} else if (schema.getType() != Schema.Type.RECORD) {
+				throw new RuntimeException("Record type for row type expected. But is: " + schema);
+			}
+			final List<Schema.Field> fields = schema.getFields();
+			final Row row = new Row(fields.size());
+			final GenericRecord record = (GenericRecord) recordObj;
+			for (int i = 0; i < fields.size(); i++) {
+				final Schema.Field field = fields.get(i);
+				row.setField(i, convertToRow(field.schema(), record.get(field.pos())));
+			}
+			return row;
+		} else if (recordObj instanceof Utf8) {
+			return recordObj.toString();
+		} else {
+			return recordObj;
+		}
+	}
+
+	/**
+	 * An extension of the ByteArrayInputStream that allows to change a buffer that should be
+	 * read without creating a new ByteArrayInputStream instance. This allows to re-use the same
+	 * InputStream instance, copying message to process, and creation of Decoder on every new message.
+	 */
+	private static final class MutableByteArrayInputStream extends ByteArrayInputStream {
+
+		public MutableByteArrayInputStream() {
+			super(new byte[0]);
+		}
+
+		/**
+		 * Set buffer that can be read via the InputStream interface and reset the input stream.
+		 * This has the same effect as creating a new ByteArrayInputStream with a new buffer.
+		 *
+		 * @param buf the new buffer to read.
+		 */
+		public void setBuffer(byte[] buf) {
+			this.buf = buf;
+			this.pos = 0;
+			this.count = buf.length;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowSerializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowSerializationSchema.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowSerializationSchema.java
new file mode 100644
index 0000000..41000a6
--- /dev/null
+++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowSerializationSchema.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.formats.avro;
+
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.io.Encoder;
+import org.apache.avro.io.EncoderFactory;
+import org.apache.avro.specific.SpecificData;
+import org.apache.avro.specific.SpecificDatumWriter;
+import org.apache.avro.specific.SpecificRecord;
+import org.apache.avro.util.Utf8;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.List;
+
+/**
+ * Serialization schema that serializes {@link Row} over {@link SpecificRecord} into a Avro bytes.
+ */
+public class AvroRowSerializationSchema implements SerializationSchema<Row> {
+
+	/**
+	 * Avro record class.
+	 */
+	private Class<? extends SpecificRecord> recordClazz;
+
+	/**
+	 * Avro serialization schema.
+	 */
+	private transient Schema schema;
+
+	/**
+	 * Writer to serialize Avro record into a byte array.
+	 */
+	private transient DatumWriter<GenericRecord> datumWriter;
+
+	/**
+	 * Output stream to serialize records into byte array.
+	 */
+	private transient ByteArrayOutputStream arrayOutputStream = new ByteArrayOutputStream();
+
+	/**
+	 * Low-level class for serialization of Avro values.
+	 */
+	private transient Encoder encoder = EncoderFactory.get().binaryEncoder(arrayOutputStream, null);
+
+	/**
+	 * Creates a Avro serialization schema for the given schema.
+	 *
+	 * @param recordClazz Avro record class used to deserialize Avro's record to Flink's row
+	 */
+	public AvroRowSerializationSchema(Class<? extends SpecificRecord> recordClazz) {
+		Preconditions.checkNotNull(recordClazz, "Avro record class must not be null.");
+		this.recordClazz = recordClazz;
+		this.schema = SpecificData.get().getSchema(recordClazz);
+		this.datumWriter = new SpecificDatumWriter<>(schema);
+	}
+
+	@Override
+	@SuppressWarnings("unchecked")
+	public byte[] serialize(Row row) {
+		// convert to record
+		final Object record = convertToRecord(schema, row);
+
+		// write
+		try {
+			arrayOutputStream.reset();
+			datumWriter.write((GenericRecord) record, encoder);
+			encoder.flush();
+			return arrayOutputStream.toByteArray();
+		} catch (IOException e) {
+			throw new RuntimeException("Failed to serialize Row.", e);
+		}
+	}
+
+	private void writeObject(ObjectOutputStream oos) throws IOException {
+		oos.writeObject(recordClazz);
+	}
+
+	@SuppressWarnings("unchecked")
+	private void readObject(ObjectInputStream ois) throws ClassNotFoundException, IOException {
+		this.recordClazz = (Class<? extends SpecificRecord>) ois.readObject();
+		this.schema = SpecificData.get().getSchema(recordClazz);
+		this.datumWriter = new SpecificDatumWriter<>(schema);
+		this.arrayOutputStream = new ByteArrayOutputStream();
+		this.encoder = EncoderFactory.get().binaryEncoder(arrayOutputStream, null);
+	}
+
+	/**
+	 * Converts a (nested) Flink Row into Avro's {@link GenericRecord}.
+	 * Strings are converted into Avro's {@link Utf8} fields.
+	 */
+	private static Object convertToRecord(Schema schema, Object rowObj) {
+		if (rowObj instanceof Row) {
+			// records can be wrapped in a union
+			if (schema.getType() == Schema.Type.UNION) {
+				final List<Schema> types = schema.getTypes();
+				if (types.size() == 2 && types.get(0).getType() == Schema.Type.NULL && types.get(1).getType() == Schema.Type.RECORD) {
+					schema = types.get(1);
+				}
+				else if (types.size() == 2 && types.get(0).getType() == Schema.Type.RECORD && types.get(1).getType() == Schema.Type.NULL) {
+					schema = types.get(0);
+				}
+				else {
+					throw new RuntimeException("Currently we only support schemas of the following form: UNION[null, RECORD] or UNION[RECORD, NULL] Given: " + schema);
+				}
+			} else if (schema.getType() != Schema.Type.RECORD) {
+				throw new RuntimeException("Record type for row type expected. But is: " + schema);
+			}
+			final List<Schema.Field> fields = schema.getFields();
+			final GenericRecord record = new GenericData.Record(schema);
+			final Row row = (Row) rowObj;
+			for (int i = 0; i < fields.size(); i++) {
+				final Schema.Field field = fields.get(i);
+				record.put(field.pos(), convertToRecord(field.schema(), row.getField(i)));
+			}
+			return record;
+		} else if (rowObj instanceof String) {
+			return new Utf8((String) rowObj);
+		} else {
+			return rowObj;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSerializer.java
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSerializer.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSerializer.java
new file mode 100644
index 0000000..02f74f5
--- /dev/null
+++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSerializer.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.formats.avro.typeutils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.java.typeutils.runtime.KryoRegistration;
+import org.apache.flink.api.java.typeutils.runtime.KryoRegistrationSerializerConfigSnapshot;
+import org.apache.flink.api.java.typeutils.runtime.KryoUtils;
+import org.apache.flink.api.java.typeutils.runtime.kryo.Serializers;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.formats.avro.utils.DataInputDecoder;
+import org.apache.flink.formats.avro.utils.DataOutputEncoder;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
+
+import com.esotericsoftware.kryo.Kryo;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.reflect.ReflectDatumReader;
+import org.apache.avro.reflect.ReflectDatumWriter;
+import org.apache.avro.util.Utf8;
+import org.objenesis.strategy.StdInstantiatorStrategy;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * General purpose serialization. Currently using Apache Avro's Reflect-serializers for serialization and
+ * Kryo for deep object copies. We want to change this to Kryo-only.
+ *
+ * @param <T> The type serialized.
+ */
+@Internal
+public final class AvroSerializer<T> extends TypeSerializer<T> {
+
+	private static final long serialVersionUID = 1L;
+
+	private final Class<T> type;
+
+	private final Class<? extends T> typeToInstantiate;
+
+	/**
+	 * Map of class tag (using classname as tag) to their Kryo registration.
+	 *
+	 * <p>This map serves as a preview of the final registration result of
+	 * the Kryo instance, taking into account registration overwrites.
+	 */
+	private LinkedHashMap<String, KryoRegistration> kryoRegistrations;
+
+	private transient ReflectDatumWriter<T> writer;
+	private transient ReflectDatumReader<T> reader;
+
+	private transient DataOutputEncoder encoder;
+	private transient DataInputDecoder decoder;
+
+	private transient Kryo kryo;
+
+	private transient T deepCopyInstance;
+
+	// --------------------------------------------------------------------------------------------
+
+	public AvroSerializer(Class<T> type) {
+		this(type, type);
+	}
+
+	public AvroSerializer(Class<T> type, Class<? extends T> typeToInstantiate) {
+		this.type = checkNotNull(type);
+		this.typeToInstantiate = checkNotNull(typeToInstantiate);
+
+		InstantiationUtil.checkForInstantiation(typeToInstantiate);
+
+		this.kryoRegistrations = buildKryoRegistrations(type);
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public boolean isImmutableType() {
+		return false;
+	}
+
+	@Override
+	public AvroSerializer<T> duplicate() {
+		return new AvroSerializer<T>(type, typeToInstantiate);
+	}
+
+	@Override
+	public T createInstance() {
+		return InstantiationUtil.instantiate(this.typeToInstantiate);
+	}
+
+	@Override
+	public T copy(T from) {
+		checkKryoInitialized();
+
+		return KryoUtils.copy(from, kryo, this);
+	}
+
+	@Override
+	public T copy(T from, T reuse) {
+		checkKryoInitialized();
+
+		return KryoUtils.copy(from, reuse, kryo, this);
+	}
+
+	@Override
+	public int getLength() {
+		return -1;
+	}
+
+	@Override
+	public void serialize(T value, DataOutputView target) throws IOException {
+		checkAvroInitialized();
+		this.encoder.setOut(target);
+		this.writer.write(value, this.encoder);
+	}
+
+	@Override
+	public T deserialize(DataInputView source) throws IOException {
+		checkAvroInitialized();
+		this.decoder.setIn(source);
+		return this.reader.read(null, this.decoder);
+	}
+
+	@Override
+	public T deserialize(T reuse, DataInputView source) throws IOException {
+		checkAvroInitialized();
+		this.decoder.setIn(source);
+		return this.reader.read(reuse, this.decoder);
+	}
+
+	@Override
+	public void copy(DataInputView source, DataOutputView target) throws IOException {
+		checkAvroInitialized();
+
+		if (this.deepCopyInstance == null) {
+			this.deepCopyInstance = InstantiationUtil.instantiate(type, Object.class);
+		}
+
+		this.decoder.setIn(source);
+		this.encoder.setOut(target);
+
+		T tmp = this.reader.read(this.deepCopyInstance, this.decoder);
+		this.writer.write(tmp, this.encoder);
+	}
+
+	private void checkAvroInitialized() {
+		if (this.reader == null) {
+			this.reader = new ReflectDatumReader<T>(type);
+			this.writer = new ReflectDatumWriter<T>(type);
+			this.encoder = new DataOutputEncoder();
+			this.decoder = new DataInputDecoder();
+		}
+	}
+
+	private void checkKryoInitialized() {
+		if (this.kryo == null) {
+			this.kryo = new Kryo();
+
+			Kryo.DefaultInstantiatorStrategy instantiatorStrategy = new Kryo.DefaultInstantiatorStrategy();
+			instantiatorStrategy.setFallbackInstantiatorStrategy(new StdInstantiatorStrategy());
+			kryo.setInstantiatorStrategy(instantiatorStrategy);
+
+			kryo.setAsmEnabled(true);
+
+			KryoUtils.applyRegistrations(kryo, kryoRegistrations.values());
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int hashCode() {
+		return 31 * this.type.hashCode() + this.typeToInstantiate.hashCode();
+	}
+
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof AvroSerializer) {
+			@SuppressWarnings("unchecked")
+			AvroSerializer<T> avroSerializer = (AvroSerializer<T>) obj;
+
+			return avroSerializer.canEqual(this) &&
+				type == avroSerializer.type &&
+				typeToInstantiate == avroSerializer.typeToInstantiate;
+		} else {
+			return false;
+		}
+	}
+
+	@Override
+	public boolean canEqual(Object obj) {
+		return obj instanceof AvroSerializer;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	// Serializer configuration snapshotting & compatibility
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public AvroSerializerConfigSnapshot<T> snapshotConfiguration() {
+		return new AvroSerializerConfigSnapshot<>(type, typeToInstantiate, kryoRegistrations);
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public CompatibilityResult<T> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+		if (configSnapshot instanceof AvroSerializerConfigSnapshot) {
+			final AvroSerializerConfigSnapshot<T> config = (AvroSerializerConfigSnapshot<T>) configSnapshot;
+
+			if (type.equals(config.getTypeClass()) && typeToInstantiate.equals(config.getTypeToInstantiate())) {
+				// resolve Kryo registrations; currently, since the Kryo registrations in Avro
+				// are fixed, there shouldn't be a problem with the resolution here.
+
+				LinkedHashMap<String, KryoRegistration> oldRegistrations = config.getKryoRegistrations();
+				oldRegistrations.putAll(kryoRegistrations);
+
+				for (Map.Entry<String, KryoRegistration> reconfiguredRegistrationEntry : kryoRegistrations.entrySet()) {
+					if (reconfiguredRegistrationEntry.getValue().isDummy()) {
+						return CompatibilityResult.requiresMigration();
+					}
+				}
+
+				this.kryoRegistrations = oldRegistrations;
+				return CompatibilityResult.compatible();
+			}
+		}
+
+		// ends up here if the preceding serializer is not
+		// the ValueSerializer, or serialized data type has changed
+		return CompatibilityResult.requiresMigration();
+	}
+
+	/**
+	 * {@link TypeSerializerConfigSnapshot} for Avro.
+	 */
+	public static class AvroSerializerConfigSnapshot<T> extends KryoRegistrationSerializerConfigSnapshot<T> {
+
+		private static final int VERSION = 1;
+
+		private Class<? extends T> typeToInstantiate;
+
+		public AvroSerializerConfigSnapshot() {}
+
+		public AvroSerializerConfigSnapshot(
+			Class<T> baseType,
+			Class<? extends T> typeToInstantiate,
+			LinkedHashMap<String, KryoRegistration> kryoRegistrations) {
+
+			super(baseType, kryoRegistrations);
+			this.typeToInstantiate = Preconditions.checkNotNull(typeToInstantiate);
+		}
+
+		@Override
+		public void write(DataOutputView out) throws IOException {
+			super.write(out);
+
+			out.writeUTF(typeToInstantiate.getName());
+		}
+
+		@SuppressWarnings("unchecked")
+		@Override
+		public void read(DataInputView in) throws IOException {
+			super.read(in);
+
+			String classname = in.readUTF();
+			try {
+				typeToInstantiate = (Class<? extends T>) Class.forName(classname, true, getUserCodeClassLoader());
+			} catch (ClassNotFoundException e) {
+				throw new IOException("Cannot find requested class " + classname + " in classpath.", e);
+			}
+		}
+
+		@Override
+		public int getVersion() {
+			return VERSION;
+		}
+
+		public Class<? extends T> getTypeToInstantiate() {
+			return typeToInstantiate;
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+		in.defaultReadObject();
+
+		// kryoRegistrations may be null if this Avro serializer is deserialized from an old version
+		if (kryoRegistrations == null) {
+			this.kryoRegistrations = buildKryoRegistrations(type);
+		}
+	}
+
+	private static <T> LinkedHashMap<String, KryoRegistration> buildKryoRegistrations(Class<T> serializedDataType) {
+		final LinkedHashMap<String, KryoRegistration> registrations = new LinkedHashMap<>();
+
+		// register Avro types.
+		registrations.put(
+				GenericData.Array.class.getName(),
+				new KryoRegistration(
+						GenericData.Array.class,
+						new ExecutionConfig.SerializableSerializer<>(new Serializers.SpecificInstanceCollectionSerializerForArrayList())));
+		registrations.put(Utf8.class.getName(), new KryoRegistration(Utf8.class));
+		registrations.put(GenericData.EnumSymbol.class.getName(), new KryoRegistration(GenericData.EnumSymbol.class));
+		registrations.put(GenericData.Fixed.class.getName(), new KryoRegistration(GenericData.Fixed.class));
+		registrations.put(GenericData.StringType.class.getName(), new KryoRegistration(GenericData.StringType.class));
+
+		// register the serialized data type
+		registrations.put(serializedDataType.getName(), new KryoRegistration(serializedDataType));
+
+		return registrations;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroTypeInfo.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroTypeInfo.java
new file mode 100644
index 0000000..ddc89a8
--- /dev/null
+++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroTypeInfo.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.formats.avro.typeutils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.typeutils.GenericTypeInfo;
+import org.apache.flink.api.java.typeutils.PojoField;
+import org.apache.flink.api.java.typeutils.PojoTypeInfo;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+
+import org.apache.avro.specific.SpecificRecordBase;
+
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Special type information to generate a special AvroTypeInfo for Avro POJOs (implementing SpecificRecordBase, the typed Avro POJOs)
+ *
+ * <p>Proceeding: It uses a regular pojo type analysis and replaces all {@code GenericType<CharSequence>} with a {@code GenericType<avro.Utf8>}.
+ * All other types used by Avro are standard Java types.
+ * Only strings are represented as CharSequence fields and represented as Utf8 classes at runtime.
+ * CharSequence is not comparable. To make them nicely usable with field expressions, we replace them here
+ * by generic type infos containing Utf8 classes (which are comparable),
+ *
+ * <p>This class is checked by the AvroPojoTest.
+ */
+public class AvroTypeInfo<T extends SpecificRecordBase> extends PojoTypeInfo<T> {
+
+	public AvroTypeInfo(Class<T> typeClass) {
+		super(typeClass, generateFieldsFromAvroSchema(typeClass));
+	}
+
+	@Override
+	public TypeSerializer<T> createSerializer(ExecutionConfig config) {
+		return super.createSerializer(config);
+	}
+
+	@SuppressWarnings("unchecked")
+	@Internal
+	public static <T extends SpecificRecordBase> List<PojoField> generateFieldsFromAvroSchema(Class<T> typeClass) {
+		PojoTypeExtractor pte = new PojoTypeExtractor();
+		ArrayList<Type> typeHierarchy = new ArrayList<>();
+		typeHierarchy.add(typeClass);
+		TypeInformation ti = pte.analyzePojo(typeClass, typeHierarchy, null, null, null);
+
+		if (!(ti instanceof PojoTypeInfo)) {
+			throw new IllegalStateException("Expecting type to be a PojoTypeInfo");
+		}
+		PojoTypeInfo pti =  (PojoTypeInfo) ti;
+		List<PojoField> newFields = new ArrayList<>(pti.getTotalFields());
+
+		for (int i = 0; i < pti.getArity(); i++) {
+			PojoField f = pti.getPojoFieldAt(i);
+			TypeInformation newType = f.getTypeInformation();
+			// check if type is a CharSequence
+			if (newType instanceof GenericTypeInfo) {
+				if ((newType).getTypeClass().equals(CharSequence.class)) {
+					// replace the type by a org.apache.avro.util.Utf8
+					newType = new GenericTypeInfo(org.apache.avro.util.Utf8.class);
+				}
+			}
+			PojoField newField = new PojoField(f.getField(), newType);
+			newFields.add(newField);
+		}
+		return newFields;
+	}
+
+	private static class PojoTypeExtractor extends TypeExtractor {
+		private PojoTypeExtractor() {
+			super();
+		}
+
+		@Override
+		public <OUT, IN1, IN2> TypeInformation<OUT> analyzePojo(Class<OUT> clazz, ArrayList<Type> typeHierarchy,
+				ParameterizedType parameterizedType, TypeInformation<IN1> in1Type, TypeInformation<IN2> in2Type) {
+			return super.analyzePojo(clazz, typeHierarchy, parameterizedType, in1Type, in2Type);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/AvroKryoSerializerUtils.java
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/AvroKryoSerializerUtils.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/AvroKryoSerializerUtils.java
new file mode 100644
index 0000000..7305f23
--- /dev/null
+++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/AvroKryoSerializerUtils.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.formats.avro.utils;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.java.typeutils.runtime.kryo.Serializers;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+
+import java.io.Serializable;
+
+/**
+ * Utilities for integrating Avro serializers in Kryo.
+ */
+public class AvroKryoSerializerUtils {
+
+	public static void addAvroSerializers(ExecutionConfig reg, Class<?> type) {
+		// Avro POJOs contain java.util.List which have GenericData.Array as their runtime type
+		// because Kryo is not able to serialize them properly, we use this serializer for them
+		reg.registerTypeWithKryoSerializer(GenericData.Array.class, Serializers.SpecificInstanceCollectionSerializerForArrayList.class);
+
+		// We register this serializer for users who want to use untyped Avro records (GenericData.Record).
+		// Kryo is able to serialize everything in there, except for the Schema.
+		// This serializer is very slow, but using the GenericData.Records of Kryo is in general a bad idea.
+		// we add the serializer as a default serializer because Avro is using a private sub-type at runtime.
+		reg.addDefaultKryoSerializer(Schema.class, AvroSchemaSerializer.class);
+	}
+
+	/**
+	 * Slow serialization approach for Avro schemas.
+	 * This is only used with {{@link org.apache.avro.generic.GenericData.Record}} types.
+	 * Having this serializer, we are able to handle avro Records.
+	 */
+	public static class AvroSchemaSerializer extends Serializer<Schema> implements Serializable {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void write(Kryo kryo, Output output, Schema object) {
+			String schemaAsString = object.toString(false);
+			output.writeString(schemaAsString);
+		}
+
+		@Override
+		public Schema read(Kryo kryo, Input input, Class<Schema> type) {
+			String schemaAsString = input.readString();
+			// the parser seems to be stateful, to we need a new one for every type.
+			Schema.Parser sParser = new Schema.Parser();
+			return sParser.parse(schemaAsString);
+		}
+	}
+}

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

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

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

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

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

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


[20/21] flink git commit: [hotfix] [avro] Simplify the FSDataInputStreamWrapper

Posted by se...@apache.org.
[hotfix] [avro] Simplify the FSDataInputStreamWrapper

The FSDataInputStreamWrapper comes from a time when Flink's FsDataInputStream was not
position aware. Not that it is, the FSDataInputStreamWrapper is not required to track
its own position, but can simply delegate these calls to the FsDataInputStream.

This also adds missing @Override tags.


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

Branch: refs/heads/master
Commit: 25dcdea056a341142d4b85e942aa80e9f82879ad
Parents: 4cd7a80
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Nov 2 19:51:06 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Nov 3 16:40:35 2017 +0100

----------------------------------------------------------------------
 .../avro/utils/FSDataInputStreamWrapper.java      | 18 +++++++++---------
 1 file changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/25dcdea0/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/FSDataInputStreamWrapper.java
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/FSDataInputStreamWrapper.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/FSDataInputStreamWrapper.java
index c00fecb..5d412e2 100644
--- a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/FSDataInputStreamWrapper.java
+++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/FSDataInputStreamWrapper.java
@@ -31,36 +31,36 @@ import java.io.IOException;
  * <p>The wrapper keeps track of the position in the data stream.
  */
 public class FSDataInputStreamWrapper implements Closeable, SeekableInput {
+
 	private final FSDataInputStream stream;
-	private long pos;
-	private long len;
+	private final long len;
 
 	public FSDataInputStreamWrapper(FSDataInputStream stream, long len) {
 		this.stream = stream;
-		this.pos = 0;
 		this.len = len;
 	}
 
+	@Override
 	public long length() throws IOException {
 		return this.len;
 	}
 
+	@Override
 	public int read(byte[] b, int off, int len) throws IOException {
-		int read;
-		read = stream.read(b, off, len);
-		pos += read;
-		return read;
+		return stream.read(b, off, len);
 	}
 
+	@Override
 	public void seek(long p) throws IOException {
 		stream.seek(p);
-		pos = p;
 	}
 
+	@Override
 	public long tell() throws IOException {
-		return pos;
+		return stream.getPos();
 	}
 
+	@Override
 	public void close() throws IOException {
 		stream.close();
 	}


[04/21] flink git commit: [FLINK-7972] [core] Move SerializationSchema to 'flink-core'

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java
index 30f6dc2..de72985 100644
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java
@@ -18,6 +18,7 @@
 package org.apache.flink.streaming.connectors.kafka;
 
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.typeutils.TypeInfoParser;
 import org.apache.flink.configuration.ConfigConstants;
@@ -32,7 +33,6 @@ import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunctio
 import org.apache.flink.streaming.util.TestStreamEnvironment;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
 import org.apache.flink.util.InstantiationUtil;
 
 import org.junit.AfterClass;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java
index dcf3167..3138152 100644
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java
@@ -18,12 +18,12 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.SerializationSchema;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.typeutils.RowTypeInfo;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
 import org.apache.flink.table.api.Types;
 import org.apache.flink.types.Row;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java
index 218401c..7a882f4 100644
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java
@@ -18,9 +18,9 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.api.Types;
 import org.apache.flink.table.sources.RowtimeAttributeDescriptor;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java
index 21171f8..6851474 100644
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironment.java
@@ -17,17 +17,18 @@
 
 package org.apache.flink.streaming.connectors.kafka;
 
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.networking.NetworkFailuresProxy;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.DataStreamSink;
 import org.apache.flink.streaming.api.operators.StreamSink;
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
 
 import kafka.server.KafkaServer;
+
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java
index b204ea9..e432a65 100644
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/DataGenerators.java
@@ -21,6 +21,8 @@ package org.apache.flink.streaming.connectors.kafka.testutils;
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.api.common.functions.RichFunction;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.java.typeutils.TypeInfoParser;
 import org.apache.flink.streaming.api.datastream.DataStream;
@@ -36,8 +38,6 @@ import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartiti
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
-import org.apache.flink.streaming.util.serialization.TypeInformationSerializationSchema;
 
 import java.util.Collection;
 import java.util.Properties;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java b/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
index b3c6f8c..c1118ed 100644
--- a/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
+++ b/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSink.java
@@ -17,10 +17,10 @@
 
 package org.apache.flink.streaming.connectors.rabbitmq;
 
+import org.apache.flink.api.common.serialization.SerializationSchema;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
 import org.apache.flink.streaming.connectors.rabbitmq.common.RMQConnectionConfig;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
 
 import com.rabbitmq.client.Channel;
 import com.rabbitmq.client.Connection;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java b/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
index 5018bcf..d454153 100644
--- a/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
+++ b/flink-connectors/flink-connector-rabbitmq/src/main/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSource.java
@@ -18,6 +18,7 @@
 package org.apache.flink.streaming.connectors.rabbitmq;
 
 import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
 import org.apache.flink.configuration.Configuration;
@@ -25,7 +26,6 @@ import org.apache.flink.streaming.api.functions.source.MessageAcknowledgingSourc
 import org.apache.flink.streaming.api.functions.source.MultipleIdsMessageAcknowledgingSourceBase;
 import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
 import org.apache.flink.streaming.connectors.rabbitmq.common.RMQConnectionConfig;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 import org.apache.flink.util.Preconditions;
 
 import com.rabbitmq.client.Channel;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSinkTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSinkTest.java b/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSinkTest.java
index 93f884b..53b834d 100644
--- a/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSinkTest.java
+++ b/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSinkTest.java
@@ -17,10 +17,10 @@
 
 package org.apache.flink.streaming.connectors.rabbitmq;
 
+import org.apache.flink.api.common.serialization.SerializationSchema;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.functions.sink.SinkContextUtil;
 import org.apache.flink.streaming.connectors.rabbitmq.common.RMQConnectionConfig;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
 
 import com.rabbitmq.client.Channel;
 import com.rabbitmq.client.Connection;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java b/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java
index 0996355..bbf893f 100644
--- a/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java
+++ b/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java
@@ -18,6 +18,7 @@
 package org.apache.flink.streaming.connectors.rabbitmq;
 
 import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
 import org.apache.flink.api.common.state.OperatorStateStore;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.tuple.Tuple2;
@@ -32,7 +33,6 @@ import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.connectors.rabbitmq.common.RMQConnectionConfig;
 import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles;
 import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 
 import com.rabbitmq.client.AMQP;
 import com.rabbitmq.client.Channel;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-core/pom.xml
----------------------------------------------------------------------
diff --git a/flink-core/pom.xml b/flink-core/pom.xml
index 5b97273..ae3f56e 100644
--- a/flink-core/pom.xml
+++ b/flink-core/pom.xml
@@ -107,6 +107,9 @@ under the License.
 			<scope>test</scope>
 		</dependency>
 
+		<!-- Joda and jackson are used to test that serialization and type extraction
+			work with types from those libraries -->
+
 		<dependency>
 			<groupId>joda-time</groupId>
 			<artifactId>joda-time</artifactId>
@@ -118,6 +121,13 @@ under the License.
 			<artifactId>joda-convert</artifactId>
 			<scope>test</scope>
 		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-shaded-jackson</artifactId>
+			<scope>test</scope>
+		</dependency>
+
     </dependencies>
 
 	<profiles>
@@ -208,6 +218,11 @@ under the License.
 							<exclude>org.apache.flink.configuration.ConfigConstants#ENABLE_QUARANTINE_MONITOR</exclude>
 							<exclude>org.apache.flink.configuration.ConfigConstants#NETWORK_REQUEST_BACKOFF_INITIAL_KEY</exclude>
 							<exclude>org.apache.flink.configuration.ConfigConstants#NETWORK_REQUEST_BACKOFF_MAX_KEY</exclude>
+
+							<!-- apparently there is a bug in the plugin which makes it fail on this new file, event though
+								its new, and not conflicting/breaking -->
+							<exclude>org.apache.flink.api.common.serialization.DeserializationSchema</exclude>
+							<exclude>org.apache.flink.api.common.serialization.SerializationSchema</exclude>
 						</excludes>
 					</parameter>
 				</configuration>

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-core/src/main/java/org/apache/flink/api/common/serialization/AbstractDeserializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/serialization/AbstractDeserializationSchema.java b/flink-core/src/main/java/org/apache/flink/api/common/serialization/AbstractDeserializationSchema.java
new file mode 100644
index 0000000..871b7b1
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/serialization/AbstractDeserializationSchema.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.serialization;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+
+import java.io.IOException;
+
+/**
+ * The deserialization schema describes how to turn the byte messages delivered by certain
+ * data sources (for example Apache Kafka) into data types (Java/Scala objects) that are
+ * processed by Flink.
+ *
+ * <p>This base variant of the deserialization schema produces the type information
+ * automatically by extracting it from the generic class arguments.
+ *
+ * @param <T> The type created by the deserialization schema.
+ */
+@PublicEvolving
+public abstract class AbstractDeserializationSchema<T> implements DeserializationSchema<T> {
+
+	private static final long serialVersionUID = 1L;
+
+	/**
+	 * De-serializes the byte message.
+	 *
+	 * @param message The message, as a byte array.
+	 * @return The de-serialized message as an object.
+	 */
+	@Override
+	public abstract T deserialize(byte[] message) throws IOException;
+
+	/**
+	 * Method to decide whether the element signals the end of the stream. If
+	 * true is returned the element won't be emitted.
+	 *
+	 * <p>This default implementation returns always false, meaning the stream is interpreted
+	 * to be unbounded.
+	 *
+	 * @param nextElement The element to test for the end-of-stream signal.
+	 * @return True, if the element signals end of stream, false otherwise.
+	 */
+	@Override
+	public boolean isEndOfStream(T nextElement) {
+		return false;
+	}
+
+	@Override
+	public TypeInformation<T> getProducedType() {
+		return TypeExtractor.createTypeInfo(AbstractDeserializationSchema.class, getClass(), 0, null, null);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-core/src/main/java/org/apache/flink/api/common/serialization/DeserializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/serialization/DeserializationSchema.java b/flink-core/src/main/java/org/apache/flink/api/common/serialization/DeserializationSchema.java
new file mode 100644
index 0000000..9de4743
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/serialization/DeserializationSchema.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.api.common.serialization;
+
+import org.apache.flink.annotation.Public;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+/**
+ * The deserialization schema describes how to turn the byte messages delivered by certain
+ * data sources (for example Apache Kafka) into data types (Java/Scala objects) that are
+ * processed by Flink.
+ *
+ * <p>Note: In most cases, one should start from {@link AbstractDeserializationSchema}, which
+ * takes care of producing the return type information automatically.
+ *
+ * @param <T> The type created by the deserialization schema.
+ */
+@Public
+public interface DeserializationSchema<T> extends Serializable, ResultTypeQueryable<T> {
+
+	/**
+	 * Deserializes the byte message.
+	 *
+	 * @param message The message, as a byte array.
+	 *
+	 * @return The deserialized message as an object (null if the message cannot be deserialized).
+	 */
+	T deserialize(byte[] message) throws IOException;
+
+	/**
+	 * Method to decide whether the element signals the end of the stream. If
+	 * true is returned the element won't be emitted.
+	 *
+	 * @param nextElement The element to test for the end-of-stream signal.
+	 * @return True, if the element signals end of stream, false otherwise.
+	 */
+	boolean isEndOfStream(T nextElement);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-core/src/main/java/org/apache/flink/api/common/serialization/SerializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/serialization/SerializationSchema.java b/flink-core/src/main/java/org/apache/flink/api/common/serialization/SerializationSchema.java
new file mode 100644
index 0000000..3a4eaeb
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/serialization/SerializationSchema.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.api.common.serialization;
+
+import org.apache.flink.annotation.Public;
+
+import java.io.Serializable;
+
+/**
+ * The serialization schema describes how to turn a data object into a different serialized
+ * representation. Most data sinks (for example Apache Kafka) require the data to be handed
+ * to them in a specific format (for example as byte strings).
+ *
+ * @param <T> The type to be serialized.
+ */
+@Public
+public interface SerializationSchema<T> extends Serializable {
+
+	/**
+	 * Serializes the incoming element to a specified type.
+	 *
+	 * @param element
+	 *            The incoming element to be serialized
+	 * @return The serialized element.
+	 */
+	byte[] serialize(T element);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-core/src/main/java/org/apache/flink/api/common/serialization/SimpleStringSchema.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/serialization/SimpleStringSchema.java b/flink-core/src/main/java/org/apache/flink/api/common/serialization/SimpleStringSchema.java
new file mode 100644
index 0000000..3130a10
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/serialization/SimpleStringSchema.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.serialization;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+
+import java.io.IOException;
+import java.io.ObjectOutputStream;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Very simple serialization schema for strings.
+ *
+ * <p>By default, the serializer uses "UTF-8" for string/byte conversion.
+ */
+@PublicEvolving
+public class SimpleStringSchema implements DeserializationSchema<String>, SerializationSchema<String> {
+
+	private static final long serialVersionUID = 1L;
+
+	/** The charset to use to convert between strings and bytes.
+	 * The field is transient because we serialize a different delegate object instead */
+	private transient Charset charset;
+
+	/**
+	 * Creates a new SimpleStringSchema that uses "UTF-8" as the encoding.
+	 */
+	public SimpleStringSchema() {
+		this(StandardCharsets.UTF_8);
+	}
+
+	/**
+	 * Creates a new SimpleStringSchema that uses the given charset to convert between strings and bytes.
+	 *
+	 * @param charset The charset to use to convert between strings and bytes.
+	 */
+	public SimpleStringSchema(Charset charset) {
+		this.charset = checkNotNull(charset);
+	}
+
+	/**
+	 * Gets the charset used by this schema for serialization.
+	 * @return The charset used by this schema for serialization.
+	 */
+	public Charset getCharset() {
+		return charset;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Kafka Serialization
+	// ------------------------------------------------------------------------
+
+	@Override
+	public String deserialize(byte[] message) {
+		return new String(message, charset);
+	}
+
+	@Override
+	public boolean isEndOfStream(String nextElement) {
+		return false;
+	}
+
+	@Override
+	public byte[] serialize(String element) {
+		return element.getBytes(charset);
+	}
+
+	@Override
+	public TypeInformation<String> getProducedType() {
+		return BasicTypeInfo.STRING_TYPE_INFO;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Java Serialization
+	// ------------------------------------------------------------------------
+
+	private void writeObject (ObjectOutputStream out) throws IOException {
+		out.defaultWriteObject();
+		out.writeUTF(charset.name());
+	}
+
+	private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException {
+		in.defaultReadObject();
+		String charsetName = in.readUTF();
+		this.charset = Charset.forName(charsetName);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-core/src/main/java/org/apache/flink/api/common/serialization/TypeInformationSerializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/serialization/TypeInformationSerializationSchema.java b/flink-core/src/main/java/org/apache/flink/api/common/serialization/TypeInformationSerializationSchema.java
new file mode 100644
index 0000000..217a889
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/serialization/TypeInformationSerializationSchema.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.api.common.serialization;
+
+import org.apache.flink.annotation.Public;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataInputDeserializer;
+import org.apache.flink.core.memory.DataOutputSerializer;
+
+import java.io.IOException;
+
+/**
+ * A serialization and deserialization schema that uses Flink's serialization stack to
+ * transform typed from and to byte arrays.
+ *
+ * @param <T> The type to be serialized.
+ */
+@Public
+public class TypeInformationSerializationSchema<T> implements DeserializationSchema<T>, SerializationSchema<T> {
+
+	private static final long serialVersionUID = -5359448468131559102L;
+
+	/** The serializer for the actual de-/serialization. */
+	private final TypeSerializer<T> serializer;
+
+	/** The reusable output serialization buffer. */
+	private transient DataOutputSerializer dos;
+
+	/** The reusable input deserialization buffer. */
+	private transient DataInputDeserializer dis;
+
+	/**
+	 * The type information, to be returned by {@link #getProducedType()}. It is transient, because
+	 * it is not serializable. Note that this means that the type information is not available at
+	 * runtime, but only prior to the first serialization / deserialization.
+	 */
+	private transient TypeInformation<T> typeInfo;
+
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Creates a new de-/serialization schema for the given type.
+	 *
+	 * @param typeInfo The type information for the type de-/serialized by this schema.
+	 * @param ec The execution config, which is used to parametrize the type serializers.
+	 */
+	public TypeInformationSerializationSchema(TypeInformation<T> typeInfo, ExecutionConfig ec) {
+		this.typeInfo = typeInfo;
+		this.serializer = typeInfo.createSerializer(ec);
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	public T deserialize(byte[] message) {
+		if (dis != null) {
+			dis.setBuffer(message, 0, message.length);
+		} else {
+			dis = new DataInputDeserializer(message, 0, message.length);
+		}
+
+		try {
+			return serializer.deserialize(dis);
+		}
+		catch (IOException e) {
+			throw new RuntimeException("Unable to deserialize message", e);
+		}
+	}
+
+	/**
+	 * This schema never considers an element to signal end-of-stream, so this method returns always false.
+	 * @param nextElement The element to test for the end-of-stream signal.
+	 * @return Returns false.
+	 */
+	@Override
+	public boolean isEndOfStream(T nextElement) {
+		return false;
+	}
+
+	@Override
+	public byte[] serialize(T element) {
+		if (dos == null) {
+			dos = new DataOutputSerializer(16);
+		}
+
+		try {
+			serializer.serialize(element, dos);
+		}
+		catch (IOException e) {
+			throw new RuntimeException("Unable to serialize record", e);
+		}
+
+		byte[] ret = dos.getByteArray();
+		if (ret.length != dos.length()) {
+			byte[] n = new byte[dos.length()];
+			System.arraycopy(ret, 0, n, 0, dos.length());
+			ret = n;
+		}
+		dos.clear();
+		return ret;
+	}
+
+	@Override
+	public TypeInformation<T> getProducedType() {
+		if (typeInfo != null) {
+			return typeInfo;
+		}
+		else {
+			throw new IllegalStateException(
+					"The type information is not available after this class has been serialized and distributed.");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-core/src/test/java/org/apache/flink/api/common/serialization/AbstractDeserializationSchemaTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/serialization/AbstractDeserializationSchemaTest.java b/flink-core/src/test/java/org/apache/flink/api/common/serialization/AbstractDeserializationSchemaTest.java
new file mode 100644
index 0000000..ec241b4
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/api/common/serialization/AbstractDeserializationSchemaTest.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.serialization;
+
+import org.apache.flink.api.common.functions.InvalidTypesException;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.util.JSONPObject;
+
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for {@link AbstractDeserializationSchema}.
+ */
+@SuppressWarnings("serial")
+public class AbstractDeserializationSchemaTest {
+
+	@Test
+	public void testTypeExtractionTuple() {
+		TypeInformation<Tuple2<byte[], byte[]>> type = new TupleSchema().getProducedType();
+		TypeInformation<Tuple2<byte[], byte[]>> expected = TypeInformation.of(new TypeHint<Tuple2<byte[], byte[]>>(){});
+		assertEquals(expected, type);
+	}
+
+	@Test
+	public void testTypeExtractionTupleAnonymous() {
+		TypeInformation<Tuple2<byte[], byte[]>> type = new AbstractDeserializationSchema<Tuple2<byte[], byte[]>>() {
+			@Override
+			public Tuple2<byte[], byte[]> deserialize(byte[] message) throws IOException {
+				throw new UnsupportedOperationException();
+			}
+		}.getProducedType();
+
+		TypeInformation<Tuple2<byte[], byte[]>> expected = TypeInformation.of(new TypeHint<Tuple2<byte[], byte[]>>(){});
+		assertEquals(expected, type);
+	}
+
+	@Test
+	public void testTypeExtractionGeneric() {
+		TypeInformation<JSONPObject> type = new JsonSchema().getProducedType();
+		TypeInformation<JSONPObject> expected = TypeInformation.of(new TypeHint<JSONPObject>(){});
+		assertEquals(expected, type);
+	}
+
+	@Test
+	public void testTypeExtractionGenericAnonymous() {
+		TypeInformation<JSONPObject> type = new AbstractDeserializationSchema<JSONPObject>() {
+			@Override
+			public JSONPObject deserialize(byte[] message) throws IOException {
+				throw new UnsupportedOperationException();
+			}
+		}.getProducedType();
+
+		TypeInformation<JSONPObject> expected = TypeInformation.of(new TypeHint<JSONPObject>(){});
+		assertEquals(expected, type);
+	}
+
+	@Test
+	public void testTypeExtractionRawException() {
+		try {
+			new RawSchema().getProducedType();
+			fail();
+		} catch (InvalidTypesException e) {
+			// expected
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Test types
+	// ------------------------------------------------------------------------
+
+	private static class TupleSchema extends AbstractDeserializationSchema<Tuple2<byte[], byte[]>> {
+
+		@Override
+		public Tuple2<byte[], byte[]> deserialize(byte[] message) throws IOException {
+			throw new UnsupportedOperationException();
+		}
+	}
+
+	private static class JsonSchema extends AbstractDeserializationSchema<JSONPObject> {
+
+		@Override
+		public JSONPObject deserialize(byte[] message) throws IOException {
+			throw new UnsupportedOperationException();
+		}
+	}
+
+	@SuppressWarnings("rawtypes")
+	private static class RawSchema extends AbstractDeserializationSchema {
+
+		@Override
+		public Object deserialize(byte[] message) throws IOException {
+			throw new UnsupportedOperationException();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-core/src/test/java/org/apache/flink/api/common/serialization/SimpleStringSchemaTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/serialization/SimpleStringSchemaTest.java b/flink-core/src/test/java/org/apache/flink/api/common/serialization/SimpleStringSchemaTest.java
new file mode 100644
index 0000000..482ff13
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/api/common/serialization/SimpleStringSchemaTest.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.serialization;
+
+import org.apache.flink.core.testutils.CommonTestUtils;
+
+import org.junit.Test;
+
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for the {@link SimpleStringSchema}.
+ */
+public class SimpleStringSchemaTest {
+
+	@Test
+	public void testSerializationWithAnotherCharset() {
+		final Charset charset = StandardCharsets.UTF_16BE;
+		final String string = "之掃描古籍版實乃姚鼐的";
+		final byte[] bytes = string.getBytes(charset);
+
+		assertArrayEquals(bytes, new SimpleStringSchema(charset).serialize(string));
+		assertEquals(string, new SimpleStringSchema(charset).deserialize(bytes));
+	}
+
+	@Test
+	public void testSerializability() throws Exception {
+		final SimpleStringSchema schema = new SimpleStringSchema(StandardCharsets.UTF_16LE);
+		final SimpleStringSchema copy = CommonTestUtils.createCopySerializable(schema);
+
+		assertEquals(schema.getCharset(), copy.getCharset());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-core/src/test/java/org/apache/flink/api/common/serialization/TypeInformationSerializationSchemaTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/serialization/TypeInformationSerializationSchemaTest.java b/flink-core/src/test/java/org/apache/flink/api/common/serialization/TypeInformationSerializationSchemaTest.java
new file mode 100644
index 0000000..ef5f4b0
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/api/common/serialization/TypeInformationSerializationSchemaTest.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.api.common.serialization;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.core.testutils.CommonTestUtils;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for {@link TypeInformationSerializationSchema}.
+ */
+public class TypeInformationSerializationSchemaTest {
+
+	@Test
+	public void testDeSerialization() {
+		try {
+			TypeInformation<MyPOJO> info = TypeExtractor.getForClass(MyPOJO.class);
+
+			TypeInformationSerializationSchema<MyPOJO> schema =
+					new TypeInformationSerializationSchema<MyPOJO>(info, new ExecutionConfig());
+
+			MyPOJO[] types = {
+					new MyPOJO(72, new Date(763784523L), new Date(88234L)),
+					new MyPOJO(-1, new Date(11111111111111L)),
+					new MyPOJO(42),
+					new MyPOJO(17, new Date(222763784523L))
+			};
+
+			for (MyPOJO val : types) {
+				byte[] serialized = schema.serialize(val);
+				MyPOJO deser = schema.deserialize(serialized);
+				assertEquals(val, deser);
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testSerializability() {
+		try {
+			TypeInformation<MyPOJO> info = TypeExtractor.getForClass(MyPOJO.class);
+			TypeInformationSerializationSchema<MyPOJO> schema =
+					new TypeInformationSerializationSchema<MyPOJO>(info, new ExecutionConfig());
+
+			// this needs to succeed
+			CommonTestUtils.createCopySerializable(schema);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Test data types
+	// ------------------------------------------------------------------------
+
+	private static class MyPOJO {
+
+		public int aField;
+		public List<Date> aList;
+
+		public MyPOJO() {}
+
+		public MyPOJO(int iVal, Date... dates) {
+			this.aField = iVal;
+			this.aList = new ArrayList<>(Arrays.asList(dates));
+		}
+
+		@Override
+		public int hashCode() {
+			return aField;
+		}
+
+		@Override
+		public boolean equals(Object obj) {
+			if (obj instanceof MyPOJO) {
+				MyPOJO that = (MyPOJO) obj;
+				return this.aField == that.aField && (this.aList == null ?
+						that.aList == null :
+						that.aList != null && this.aList.equals(that.aList));
+			}
+			return super.equals(obj);
+		}
+
+		@Override
+		public String toString() {
+			return "MyPOJO " + aField + " " + aList;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka010Example.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka010Example.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka010Example.java
index b5abbc5..3fbd2b4 100644
--- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka010Example.java
+++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/kafka/Kafka010Example.java
@@ -19,12 +19,12 @@ package org.apache.flink.streaming.examples.kafka;
 
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
 import org.apache.flink.api.java.utils.ParameterTool;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer010;
 import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer010;
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/kafka/Kafka010Example.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/kafka/Kafka010Example.scala b/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/kafka/Kafka010Example.scala
index 2a52811..9f4fdc4 100644
--- a/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/kafka/Kafka010Example.scala
+++ b/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/kafka/Kafka010Example.scala
@@ -19,10 +19,10 @@
 package org.apache.flink.streaming.scala.examples.kafka
 
 import org.apache.flink.api.common.restartstrategy.RestartStrategies
+import org.apache.flink.api.common.serialization.SimpleStringSchema
 import org.apache.flink.api.java.utils.ParameterTool
 import org.apache.flink.streaming.api.scala._
 import org.apache.flink.streaming.connectors.kafka.{FlinkKafkaConsumer010, FlinkKafkaProducer010}
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema
 
 /**
  * An example that shows how to read from and write to Kafka. This will read String messages

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-streaming-java/pom.xml
----------------------------------------------------------------------
diff --git a/flink-streaming-java/pom.xml b/flink-streaming-java/pom.xml
index 5b15b5a..2683546 100644
--- a/flink-streaming-java/pom.xml
+++ b/flink-streaming-java/pom.xml
@@ -77,12 +77,6 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-jackson</artifactId>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-test-utils-junit</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index d0769c6..2274968 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -31,6 +31,7 @@ import org.apache.flink.api.common.functions.RichMapFunction;
 import org.apache.flink.api.common.io.OutputFormat;
 import org.apache.flink.api.common.operators.Keys;
 import org.apache.flink.api.common.operators.ResourceSpec;
+import org.apache.flink.api.common.serialization.SerializationSchema;
 import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
 import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -91,7 +92,6 @@ import org.apache.flink.streaming.runtime.partitioner.RescalePartitioner;
 import org.apache.flink.streaming.runtime.partitioner.ShufflePartitioner;
 import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
 import org.apache.flink.streaming.util.keys.KeySelectorUtil;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
 import org.apache.flink.util.Preconditions;
 
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/SocketClientSink.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/SocketClientSink.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/SocketClientSink.java
index 214d5c2..80e0dbe 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/SocketClientSink.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/SocketClientSink.java
@@ -18,8 +18,8 @@
 package org.apache.flink.streaming.api.functions.sink;
 
 import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.SerializationSchema;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
 import org.apache.flink.util.SerializableObject;
 
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/AbstractDeserializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/AbstractDeserializationSchema.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/AbstractDeserializationSchema.java
index 02ea004..7d30f91 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/AbstractDeserializationSchema.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/AbstractDeserializationSchema.java
@@ -18,10 +18,7 @@
 
 package org.apache.flink.streaming.util.serialization;
 
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-
-import java.io.IOException;
+import org.apache.flink.annotation.PublicEvolving;
 
 /**
  * The deserialization schema describes how to turn the byte messages delivered by certain
@@ -32,37 +29,15 @@ import java.io.IOException;
  * automatically by extracting it from the generic class arguments.
  *
  * @param <T> The type created by the deserialization schema.
+ *
+ * @deprecated Use {@link org.apache.flink.api.common.serialization.AbstractDeserializationSchema} instead.
  */
-public abstract class AbstractDeserializationSchema<T> implements DeserializationSchema<T> {
+@Deprecated
+@PublicEvolving
+@SuppressWarnings("deprecation")
+public abstract class AbstractDeserializationSchema<T>
+		extends org.apache.flink.api.common.serialization.AbstractDeserializationSchema<T>
+		implements DeserializationSchema<T> {
 
 	private static final long serialVersionUID = 1L;
-
-	/**
-	 * De-serializes the byte message.
-	 *
-	 * @param message The message, as a byte array.
-	 * @return The de-serialized message as an object.
-	 */
-	@Override
-	public abstract T deserialize(byte[] message) throws IOException;
-
-	/**
-	 * Method to decide whether the element signals the end of the stream. If
-	 * true is returned the element won't be emitted.
-	 *
-	 * <p>This default implementation returns always false, meaning the stream is interpreted
-	 * to be unbounded.
-	 *
-	 * @param nextElement The element to test for the end-of-stream signal.
-	 * @return True, if the element signals end of stream, false otherwise.
-	 */
-	@Override
-	public boolean isEndOfStream(T nextElement) {
-		return false;
-	}
-
-	@Override
-	public TypeInformation<T> getProducedType() {
-		return TypeExtractor.createTypeInfo(AbstractDeserializationSchema.class, getClass(), 0, null, null);
-	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/DeserializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/DeserializationSchema.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/DeserializationSchema.java
index 15ecb2c..cbaa004 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/DeserializationSchema.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/DeserializationSchema.java
@@ -32,9 +32,15 @@ import java.io.Serializable;
  * takes care of producing the return type information automatically.
  *
  * @param <T> The type created by the deserialization schema.
+ *
+ * @deprecated Use {@link org.apache.flink.api.common.serialization.DeserializationSchema} instead.
  */
 @Public
-public interface DeserializationSchema<T> extends Serializable, ResultTypeQueryable<T> {
+@Deprecated
+public interface DeserializationSchema<T> extends
+		org.apache.flink.api.common.serialization.DeserializationSchema<T>,
+		Serializable,
+		ResultTypeQueryable<T> {
 
 	/**
 	 * Deserializes the byte message.
@@ -43,6 +49,7 @@ public interface DeserializationSchema<T> extends Serializable, ResultTypeQuerya
 	 *
 	 * @return The deserialized message as an object (null if the message cannot be deserialized).
 	 */
+	@Override
 	T deserialize(byte[] message) throws IOException;
 
 	/**
@@ -52,5 +59,6 @@ public interface DeserializationSchema<T> extends Serializable, ResultTypeQuerya
 	 * @param nextElement The element to test for the end-of-stream signal.
 	 * @return True, if the element signals end of stream, false otherwise.
 	 */
+	@Override
 	boolean isEndOfStream(T nextElement);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/SerializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/SerializationSchema.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/SerializationSchema.java
index 986cfb3..c7c1de0 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/SerializationSchema.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/SerializationSchema.java
@@ -27,9 +27,13 @@ import java.io.Serializable;
  * to them in a specific format (for example as byte strings).
  *
  * @param <T> The type to be serialized.
+ *
+ * @deprecated Use {@link org.apache.flink.api.common.serialization.SerializationSchema} instead.
  */
 @Public
-public interface SerializationSchema<T> extends Serializable {
+@Deprecated
+public interface SerializationSchema<T>
+		extends org.apache.flink.api.common.serialization.SerializationSchema<T>, Serializable {
 
 	/**
 	 * Serializes the incoming element to a specified type.
@@ -38,5 +42,6 @@ public interface SerializationSchema<T> extends Serializable {
 	 *            The incoming element to be serialized
 	 * @return The serialized element.
 	 */
+	@Override
 	byte[] serialize(T element);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/SimpleStringSchema.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/SimpleStringSchema.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/SimpleStringSchema.java
index 27ba9e9..01ce30d 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/SimpleStringSchema.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/SimpleStringSchema.java
@@ -18,35 +18,27 @@
 package org.apache.flink.streaming.util.serialization;
 
 import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
 
-import java.io.IOException;
-import java.io.ObjectOutputStream;
 import java.nio.charset.Charset;
-import java.nio.charset.StandardCharsets;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * Very simple serialization schema for strings.
  *
  * <p>By default, the serializer uses "UTF-8" for string/byte conversion.
+ *
+ * @deprecated Use {@link org.apache.flink.api.common.serialization.SimpleStringSchema} instead.
  */
 @PublicEvolving
-public class SimpleStringSchema implements DeserializationSchema<String>, SerializationSchema<String> {
+@Deprecated
+@SuppressWarnings("deprecation")
+public class SimpleStringSchema
+		extends org.apache.flink.api.common.serialization.SimpleStringSchema
+		implements SerializationSchema<String>, DeserializationSchema<String> {
 
 	private static final long serialVersionUID = 1L;
 
-	/** The charset to use to convert between strings and bytes.
-	 * The field is transient because we serialize a different delegate object instead */
-	private transient Charset charset;
-
-	/**
-	 * Creates a new SimpleStringSchema that uses "UTF-8" as the encoding.
-	 */
 	public SimpleStringSchema() {
-		this(StandardCharsets.UTF_8);
+		super();
 	}
 
 	/**
@@ -55,53 +47,6 @@ public class SimpleStringSchema implements DeserializationSchema<String>, Serial
 	 * @param charset The charset to use to convert between strings and bytes.
 	 */
 	public SimpleStringSchema(Charset charset) {
-		this.charset = checkNotNull(charset);
-	}
-
-	/**
-	 * Gets the charset used by this schema for serialization.
-	 * @return The charset used by this schema for serialization.
-	 */
-	public Charset getCharset() {
-		return charset;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Kafka Serialization
-	// ------------------------------------------------------------------------
-
-	@Override
-	public String deserialize(byte[] message) {
-		return new String(message, charset);
-	}
-
-	@Override
-	public boolean isEndOfStream(String nextElement) {
-		return false;
-	}
-
-	@Override
-	public byte[] serialize(String element) {
-		return element.getBytes(charset);
-	}
-
-	@Override
-	public TypeInformation<String> getProducedType() {
-		return BasicTypeInfo.STRING_TYPE_INFO;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Java Serialization
-	// ------------------------------------------------------------------------
-
-	private void writeObject (ObjectOutputStream out) throws IOException {
-		out.defaultWriteObject();
-		out.writeUTF(charset.name());
-	}
-
-	private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException {
-		in.defaultReadObject();
-		String charsetName = in.readUTF();
-		this.charset = Charset.forName(charsetName);
+		super(charset);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationSerializationSchema.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationSerializationSchema.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationSerializationSchema.java
index 1c50dc2..b771fe0 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationSerializationSchema.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/serialization/TypeInformationSerializationSchema.java
@@ -21,41 +21,24 @@ package org.apache.flink.streaming.util.serialization;
 import org.apache.flink.annotation.Public;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.DataInputDeserializer;
-import org.apache.flink.core.memory.DataOutputSerializer;
-
-import java.io.IOException;
 
 /**
  * A serialization and deserialization schema that uses Flink's serialization stack to
  * transform typed from and to byte arrays.
  *
  * @param <T> The type to be serialized.
+ *
+ * @deprecated Use {@link org.apache.flink.api.common.serialization.TypeInformationSerializationSchema} instead.
  */
 @Public
-public class TypeInformationSerializationSchema<T> implements DeserializationSchema<T>, SerializationSchema<T> {
+@Deprecated
+@SuppressWarnings("deprecation")
+public class TypeInformationSerializationSchema<T>
+		extends org.apache.flink.api.common.serialization.TypeInformationSerializationSchema<T>
+		implements DeserializationSchema<T>, SerializationSchema<T> {
 
 	private static final long serialVersionUID = -5359448468131559102L;
 
-	/** The serializer for the actual de-/serialization. */
-	private final TypeSerializer<T> serializer;
-
-	/** The reusable output serialization buffer. */
-	private transient DataOutputSerializer dos;
-
-	/** The reusable input deserialization buffer. */
-	private transient DataInputDeserializer dis;
-
-	/**
-	 * The type information, to be returned by {@link #getProducedType()}. It is transient, because
-	 * it is not serializable. Note that this means that the type information is not available at
-	 * runtime, but only prior to the first serialization / deserialization.
-	 */
-	private transient TypeInformation<T> typeInfo;
-
-	// ------------------------------------------------------------------------
-
 	/**
 	 * Creates a new de-/serialization schema for the given type.
 	 *
@@ -63,69 +46,6 @@ public class TypeInformationSerializationSchema<T> implements DeserializationSch
 	 * @param ec The execution config, which is used to parametrize the type serializers.
 	 */
 	public TypeInformationSerializationSchema(TypeInformation<T> typeInfo, ExecutionConfig ec) {
-		this.typeInfo = typeInfo;
-		this.serializer = typeInfo.createSerializer(ec);
-	}
-
-	// ------------------------------------------------------------------------
-
-	@Override
-	public T deserialize(byte[] message) {
-		if (dis != null) {
-			dis.setBuffer(message, 0, message.length);
-		} else {
-			dis = new DataInputDeserializer(message, 0, message.length);
-		}
-
-		try {
-			return serializer.deserialize(dis);
-		}
-		catch (IOException e) {
-			throw new RuntimeException("Unable to deserialize message", e);
-		}
-	}
-
-	/**
-	 * This schema never considers an element to signal end-of-stream, so this method returns always false.
-	 * @param nextElement The element to test for the end-of-stream signal.
-	 * @return Returns false.
-	 */
-	@Override
-	public boolean isEndOfStream(T nextElement) {
-		return false;
-	}
-
-	@Override
-	public byte[] serialize(T element) {
-		if (dos == null) {
-			dos = new DataOutputSerializer(16);
-		}
-
-		try {
-			serializer.serialize(element, dos);
-		}
-		catch (IOException e) {
-			throw new RuntimeException("Unable to serialize record", e);
-		}
-
-		byte[] ret = dos.getByteArray();
-		if (ret.length != dos.length()) {
-			byte[] n = new byte[dos.length()];
-			System.arraycopy(ret, 0, n, 0, dos.length());
-			ret = n;
-		}
-		dos.clear();
-		return ret;
-	}
-
-	@Override
-	public TypeInformation<T> getProducedType() {
-		if (typeInfo != null) {
-			return typeInfo;
-		}
-		else {
-			throw new IllegalStateException(
-					"The type information is not available after this class has been serialized and distributed.");
-		}
+		super(typeInfo, ec);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/SocketClientSinkTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/SocketClientSinkTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/SocketClientSinkTest.java
index 6cdce11..b3c4ee9 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/SocketClientSinkTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/SocketClientSinkTest.java
@@ -18,9 +18,9 @@
 
 package org.apache.flink.streaming.api.functions.sink;
 
+import org.apache.flink.api.common.serialization.SerializationSchema;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.util.serialization.SerializationSchema;
 import org.apache.flink.util.TestLogger;
 
 import org.apache.commons.io.IOUtils;

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractDeserializationSchemaTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractDeserializationSchemaTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractDeserializationSchemaTest.java
deleted file mode 100644
index 818a43b..0000000
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractDeserializationSchemaTest.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.streaming.util;
-
-import org.apache.flink.api.common.functions.InvalidTypesException;
-import org.apache.flink.api.common.typeinfo.TypeHint;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.util.serialization.AbstractDeserializationSchema;
-
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.util.JSONPObject;
-
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-/**
- * Tests for {@link AbstractDeserializationSchema}.
- */
-@SuppressWarnings("serial")
-public class AbstractDeserializationSchemaTest {
-
-	@Test
-	public void testTypeExtractionTuple() {
-		TypeInformation<Tuple2<byte[], byte[]>> type = new TupleSchema().getProducedType();
-		TypeInformation<Tuple2<byte[], byte[]>> expected = TypeInformation.of(new TypeHint<Tuple2<byte[], byte[]>>(){});
-		assertEquals(expected, type);
-	}
-
-	@Test
-	public void testTypeExtractionTupleAnonymous() {
-		TypeInformation<Tuple2<byte[], byte[]>> type = new AbstractDeserializationSchema<Tuple2<byte[], byte[]>>() {
-			@Override
-			public Tuple2<byte[], byte[]> deserialize(byte[] message) throws IOException {
-				throw new UnsupportedOperationException();
-			}
-		}.getProducedType();
-
-		TypeInformation<Tuple2<byte[], byte[]>> expected = TypeInformation.of(new TypeHint<Tuple2<byte[], byte[]>>(){});
-		assertEquals(expected, type);
-	}
-
-	@Test
-	public void testTypeExtractionGeneric() {
-		TypeInformation<JSONPObject> type = new JsonSchema().getProducedType();
-		TypeInformation<JSONPObject> expected = TypeInformation.of(new TypeHint<JSONPObject>(){});
-		assertEquals(expected, type);
-	}
-
-	@Test
-	public void testTypeExtractionGenericAnonymous() {
-		TypeInformation<JSONPObject> type = new AbstractDeserializationSchema<JSONPObject>() {
-			@Override
-			public JSONPObject deserialize(byte[] message) throws IOException {
-				throw new UnsupportedOperationException();
-			}
-		}.getProducedType();
-
-		TypeInformation<JSONPObject> expected = TypeInformation.of(new TypeHint<JSONPObject>(){});
-		assertEquals(expected, type);
-	}
-
-	@Test
-	public void testTypeExtractionRawException() {
-		try {
-			new RawSchema().getProducedType();
-			fail();
-		} catch (InvalidTypesException e) {
-			// expected
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Test types
-	// ------------------------------------------------------------------------
-
-	private static class TupleSchema extends AbstractDeserializationSchema<Tuple2<byte[], byte[]>> {
-
-		@Override
-		public Tuple2<byte[], byte[]> deserialize(byte[] message) throws IOException {
-			throw new UnsupportedOperationException();
-		}
-	}
-
-	private static class JsonSchema extends AbstractDeserializationSchema<JSONPObject> {
-
-		@Override
-		public JSONPObject deserialize(byte[] message) throws IOException {
-			throw new UnsupportedOperationException();
-		}
-	}
-
-	@SuppressWarnings("rawtypes")
-	private static class RawSchema extends AbstractDeserializationSchema {
-
-		@Override
-		public Object deserialize(byte[] message) throws IOException {
-			throw new UnsupportedOperationException();
-		}
-	}
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/serialization/SimpleStringSchemaTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/serialization/SimpleStringSchemaTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/serialization/SimpleStringSchemaTest.java
deleted file mode 100644
index 6081ed1..0000000
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/serialization/SimpleStringSchemaTest.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.util.serialization;
-
-import org.apache.flink.core.testutils.CommonTestUtils;
-
-import org.junit.Test;
-
-import java.nio.charset.Charset;
-import java.nio.charset.StandardCharsets;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-
-/**
- * Tests for the {@link SimpleStringSchema}.
- */
-public class SimpleStringSchemaTest {
-
-	@Test
-	public void testSerializationWithAnotherCharset() {
-		final Charset charset = StandardCharsets.UTF_16BE;
-		final String string = "之掃描古籍版實乃姚鼐的";
-		final byte[] bytes = string.getBytes(charset);
-
-		assertArrayEquals(bytes, new SimpleStringSchema(charset).serialize(string));
-		assertEquals(string, new SimpleStringSchema(charset).deserialize(bytes));
-	}
-
-	@Test
-	public void testSerializability() throws Exception {
-		final SimpleStringSchema schema = new SimpleStringSchema(StandardCharsets.UTF_16LE);
-		final SimpleStringSchema copy = CommonTestUtils.createCopySerializable(schema);
-
-		assertEquals(schema.getCharset(), copy.getCharset());
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
----------------------------------------------------------------------
diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
index b5a7cd6..ef2e741 100644
--- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
+++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
@@ -23,6 +23,7 @@ import org.apache.flink.api.common.ExecutionConfig
 import org.apache.flink.api.common.functions.{FilterFunction, FlatMapFunction, MapFunction, Partitioner}
 import org.apache.flink.api.common.io.OutputFormat
 import org.apache.flink.api.common.operators.ResourceSpec
+import org.apache.flink.api.common.serialization.SerializationSchema
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.functions.KeySelector
 import org.apache.flink.api.java.tuple.{Tuple => JavaTuple}
@@ -38,7 +39,6 @@ import org.apache.flink.streaming.api.operators.OneInputStreamOperator
 import org.apache.flink.streaming.api.windowing.assigners._
 import org.apache.flink.streaming.api.windowing.time.Time
 import org.apache.flink.streaming.api.windowing.windows.{GlobalWindow, TimeWindow, Window}
-import org.apache.flink.streaming.util.serialization.SerializationSchema
 import org.apache.flink.util.Collector
 
 import scala.collection.JavaConverters._

http://git-wip-us.apache.org/repos/asf/flink/blob/fe931d07/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala
----------------------------------------------------------------------
diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala
index 3b47429..991f241 100644
--- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala
+++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/OutputFormatTestPrograms.scala
@@ -18,8 +18,8 @@
 package org.apache.flink.streaming.api.scala
 
 
+import org.apache.flink.api.common.serialization.SimpleStringSchema
 import org.apache.flink.core.fs.FileSystem
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema
 
 import scala.language.existentials
 


[07/21] flink git commit: [FLINK-7420] [avro] Move all Avro code to flink-avro

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

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroOutputFormatTest.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroOutputFormatTest.java
new file mode 100644
index 0000000..b5ad564
--- /dev/null
+++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroOutputFormatTest.java
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.formats.avro;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.formats.avro.generated.Colors;
+import org.apache.flink.formats.avro.generated.User;
+
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+import org.junit.Test;
+import org.mockito.internal.util.reflection.Whitebox;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for {@link AvroOutputFormat}.
+ */
+public class AvroOutputFormatTest {
+
+	@Test
+	public void testSetCodec() throws Exception {
+		// given
+		final AvroOutputFormat<User> outputFormat = new AvroOutputFormat<>(User.class);
+
+		// when
+		try {
+			outputFormat.setCodec(AvroOutputFormat.Codec.SNAPPY);
+		} catch (Exception ex) {
+			// then
+			fail("unexpected exception");
+		}
+	}
+
+	@Test
+	public void testSetCodecError() throws Exception {
+		// given
+		boolean error = false;
+		final AvroOutputFormat<User> outputFormat = new AvroOutputFormat<>(User.class);
+
+		// when
+		try {
+			outputFormat.setCodec(null);
+		} catch (Exception ex) {
+			error = true;
+		}
+
+		// then
+		assertTrue(error);
+	}
+
+	@Test
+	public void testSerialization() throws Exception {
+
+		serializeAndDeserialize(null, null);
+		serializeAndDeserialize(null, User.SCHEMA$);
+		for (final AvroOutputFormat.Codec codec : AvroOutputFormat.Codec.values()) {
+			serializeAndDeserialize(codec, null);
+			serializeAndDeserialize(codec, User.SCHEMA$);
+		}
+	}
+
+	private void serializeAndDeserialize(final AvroOutputFormat.Codec codec, final Schema schema) throws IOException, ClassNotFoundException {
+		// given
+		final AvroOutputFormat<User> outputFormat = new AvroOutputFormat<>(User.class);
+		if (codec != null) {
+			outputFormat.setCodec(codec);
+		}
+		if (schema != null) {
+			outputFormat.setSchema(schema);
+		}
+
+		final ByteArrayOutputStream bos = new ByteArrayOutputStream();
+
+		// when
+		try (final ObjectOutputStream oos = new ObjectOutputStream(bos)) {
+			oos.writeObject(outputFormat);
+		}
+		try (final ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(bos.toByteArray()))) {
+			// then
+			Object o = ois.readObject();
+			assertTrue(o instanceof AvroOutputFormat);
+			final AvroOutputFormat<User> restored = (AvroOutputFormat<User>) o;
+			final AvroOutputFormat.Codec restoredCodec = (AvroOutputFormat.Codec) Whitebox.getInternalState(restored, "codec");
+			final Schema restoredSchema = (Schema) Whitebox.getInternalState(restored, "userDefinedSchema");
+
+			assertTrue(codec != null ? restoredCodec == codec : restoredCodec == null);
+			assertTrue(schema != null ? restoredSchema.equals(schema) : restoredSchema == null);
+		}
+	}
+
+	@Test
+	public void testCompression() throws Exception {
+		// given
+		final Path outputPath = new Path(File.createTempFile("avro-output-file", "avro").getAbsolutePath());
+		final AvroOutputFormat<User> outputFormat = new AvroOutputFormat<>(outputPath, User.class);
+		outputFormat.setWriteMode(FileSystem.WriteMode.OVERWRITE);
+
+		final Path compressedOutputPath = new Path(File.createTempFile("avro-output-file", "compressed.avro").getAbsolutePath());
+		final AvroOutputFormat<User> compressedOutputFormat = new AvroOutputFormat<>(compressedOutputPath, User.class);
+		compressedOutputFormat.setWriteMode(FileSystem.WriteMode.OVERWRITE);
+		compressedOutputFormat.setCodec(AvroOutputFormat.Codec.SNAPPY);
+
+		// when
+		output(outputFormat);
+		output(compressedOutputFormat);
+
+		// then
+		assertTrue(fileSize(outputPath) > fileSize(compressedOutputPath));
+
+		// cleanup
+		FileSystem fs = FileSystem.getLocalFileSystem();
+		fs.delete(outputPath, false);
+		fs.delete(compressedOutputPath, false);
+	}
+
+	private long fileSize(Path path) throws IOException {
+		return path.getFileSystem().getFileStatus(path).getLen();
+	}
+
+	private void output(final AvroOutputFormat<User> outputFormat) throws IOException {
+		outputFormat.configure(new Configuration());
+		outputFormat.open(1, 1);
+		for (int i = 0; i < 100; i++) {
+			User user = new User();
+			user.setName("testUser");
+			user.setFavoriteNumber(1);
+			user.setFavoriteColor("blue");
+			user.setTypeBoolTest(true);
+			user.setTypeArrayString(Collections.emptyList());
+			user.setTypeArrayBoolean(Collections.emptyList());
+			user.setTypeEnum(Colors.BLUE);
+			user.setTypeMap(Collections.emptyMap());
+			outputFormat.writeRecord(user);
+		}
+		outputFormat.close();
+	}
+
+	@Test
+	public void testGenericRecord() throws IOException {
+		final Path outputPath = new Path(File.createTempFile("avro-output-file", "generic.avro").getAbsolutePath());
+		final AvroOutputFormat<GenericRecord> outputFormat = new AvroOutputFormat<>(outputPath, GenericRecord.class);
+		Schema schema = new Schema.Parser().parse("{\"type\":\"record\", \"name\":\"user\", \"fields\": [{\"name\":\"user_name\", \"type\":\"string\"}, {\"name\":\"favorite_number\", \"type\":\"int\"}, {\"name\":\"favorite_color\", \"type\":\"string\"}]}");
+		outputFormat.setWriteMode(FileSystem.WriteMode.OVERWRITE);
+		outputFormat.setSchema(schema);
+		output(outputFormat, schema);
+
+		GenericDatumReader<GenericRecord> reader = new GenericDatumReader<>(schema);
+		DataFileReader<GenericRecord> dataFileReader = new DataFileReader<>(new File(outputPath.getPath()), reader);
+
+		while (dataFileReader.hasNext()) {
+			GenericRecord record = dataFileReader.next();
+			assertEquals(record.get("user_name").toString(), "testUser");
+			assertEquals(record.get("favorite_number"), 1);
+			assertEquals(record.get("favorite_color").toString(), "blue");
+		}
+
+		//cleanup
+		FileSystem fs = FileSystem.getLocalFileSystem();
+		fs.delete(outputPath, false);
+
+	}
+
+	private void output(final AvroOutputFormat<GenericRecord> outputFormat, Schema schema) throws IOException {
+		outputFormat.configure(new Configuration());
+		outputFormat.open(1, 1);
+		for (int i = 0; i < 100; i++) {
+			GenericRecord record = new GenericData.Record(schema);
+			record.put("user_name", "testUser");
+			record.put("favorite_number", 1);
+			record.put("favorite_color", "blue");
+			outputFormat.writeRecord(record);
+		}
+		outputFormat.close();
+	}
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroRowDeSerializationSchemaTest.java
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroRowDeSerializationSchemaTest.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroRowDeSerializationSchemaTest.java
new file mode 100644
index 0000000..5341bcf
--- /dev/null
+++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroRowDeSerializationSchemaTest.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.formats.avro;
+
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.formats.avro.utils.AvroTestUtils;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.InstantiationUtil;
+
+import org.apache.avro.specific.SpecificRecord;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test for the Avro serialization and deserialization schema.
+ */
+public class AvroRowDeSerializationSchemaTest {
+
+	@Test
+	public void testSerializeDeserializeSimpleRow() throws IOException {
+		final Tuple3<Class<? extends SpecificRecord>, SpecificRecord, Row> testData = AvroTestUtils.getSimpleTestData();
+
+		final AvroRowSerializationSchema serializationSchema = new AvroRowSerializationSchema(testData.f0);
+		final AvroRowDeserializationSchema deserializationSchema = new AvroRowDeserializationSchema(testData.f0);
+
+		final byte[] bytes = serializationSchema.serialize(testData.f2);
+		final Row actual = deserializationSchema.deserialize(bytes);
+
+		assertEquals(testData.f2, actual);
+	}
+
+	@Test
+	public void testSerializeSimpleRowSeveralTimes() throws IOException {
+		final Tuple3<Class<? extends SpecificRecord>, SpecificRecord, Row> testData = AvroTestUtils.getSimpleTestData();
+
+		final AvroRowSerializationSchema serializationSchema = new AvroRowSerializationSchema(testData.f0);
+		final AvroRowDeserializationSchema deserializationSchema = new AvroRowDeserializationSchema(testData.f0);
+
+		serializationSchema.serialize(testData.f2);
+		serializationSchema.serialize(testData.f2);
+		final byte[] bytes = serializationSchema.serialize(testData.f2);
+		final Row actual = deserializationSchema.deserialize(bytes);
+
+		assertEquals(testData.f2, actual);
+	}
+
+	@Test
+	public void testDeserializeRowSeveralTimes() throws IOException {
+		final Tuple3<Class<? extends SpecificRecord>, SpecificRecord, Row> testData = AvroTestUtils.getSimpleTestData();
+
+		final AvroRowSerializationSchema serializationSchema = new AvroRowSerializationSchema(testData.f0);
+		final AvroRowDeserializationSchema deserializationSchema = new AvroRowDeserializationSchema(testData.f0);
+
+		final byte[] bytes = serializationSchema.serialize(testData.f2);
+		deserializationSchema.deserialize(bytes);
+		deserializationSchema.deserialize(bytes);
+		final Row actual = deserializationSchema.deserialize(bytes);
+
+		assertEquals(testData.f2, actual);
+	}
+
+	@Test
+	public void testSerializeDeserializeComplexRow() throws IOException {
+		final Tuple3<Class<? extends SpecificRecord>, SpecificRecord, Row> testData = AvroTestUtils.getComplexTestData();
+
+		final AvroRowSerializationSchema serializationSchema = new AvroRowSerializationSchema(testData.f0);
+		final AvroRowDeserializationSchema deserializationSchema = new AvroRowDeserializationSchema(testData.f0);
+
+		final byte[] bytes = serializationSchema.serialize(testData.f2);
+		final Row actual = deserializationSchema.deserialize(bytes);
+
+		assertEquals(testData.f2, actual);
+	}
+
+	@Test
+	public void testSerializeComplexRowSeveralTimes() throws IOException {
+		final Tuple3<Class<? extends SpecificRecord>, SpecificRecord, Row> testData = AvroTestUtils.getComplexTestData();
+
+		final AvroRowSerializationSchema serializationSchema = new AvroRowSerializationSchema(testData.f0);
+		final AvroRowDeserializationSchema deserializationSchema = new AvroRowDeserializationSchema(testData.f0);
+
+		serializationSchema.serialize(testData.f2);
+		serializationSchema.serialize(testData.f2);
+		final byte[] bytes = serializationSchema.serialize(testData.f2);
+		final Row actual = deserializationSchema.deserialize(bytes);
+
+		assertEquals(testData.f2, actual);
+	}
+
+	@Test
+	public void testDeserializeComplexRowSeveralTimes() throws IOException {
+		final Tuple3<Class<? extends SpecificRecord>, SpecificRecord, Row> testData = AvroTestUtils.getComplexTestData();
+
+		final AvroRowSerializationSchema serializationSchema = new AvroRowSerializationSchema(testData.f0);
+		final AvroRowDeserializationSchema deserializationSchema = new AvroRowDeserializationSchema(testData.f0);
+
+		final byte[] bytes = serializationSchema.serialize(testData.f2);
+		deserializationSchema.deserialize(bytes);
+		deserializationSchema.deserialize(bytes);
+		final Row actual = deserializationSchema.deserialize(bytes);
+
+		assertEquals(testData.f2, actual);
+	}
+
+	@Test
+	public void testSerializability() throws IOException, ClassNotFoundException {
+		final Tuple3<Class<? extends SpecificRecord>, SpecificRecord, Row> testData = AvroTestUtils.getComplexTestData();
+
+		final AvroRowSerializationSchema serOrig = new AvroRowSerializationSchema(testData.f0);
+		final AvroRowDeserializationSchema deserOrig = new AvroRowDeserializationSchema(testData.f0);
+
+		byte[] serBytes = InstantiationUtil.serializeObject(serOrig);
+		byte[] deserBytes = InstantiationUtil.serializeObject(deserOrig);
+
+		AvroRowSerializationSchema serCopy =
+			InstantiationUtil.deserializeObject(serBytes, Thread.currentThread().getContextClassLoader());
+		AvroRowDeserializationSchema deserCopy =
+			InstantiationUtil.deserializeObject(deserBytes, Thread.currentThread().getContextClassLoader());
+
+		final byte[] bytes = serCopy.serialize(testData.f2);
+		deserCopy.deserialize(bytes);
+		deserCopy.deserialize(bytes);
+		final Row actual = deserCopy.deserialize(bytes);
+
+		assertEquals(testData.f2, actual);
+	}
+}

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

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

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/testjar/AvroExternalJarProgram.java
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/testjar/AvroExternalJarProgram.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/testjar/AvroExternalJarProgram.java
new file mode 100644
index 0000000..17f56a6
--- /dev/null
+++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/testjar/AvroExternalJarProgram.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.formats.avro.testjar;
+
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.functions.RichReduceFunction;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.io.DiscardingOutputFormat;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.formats.avro.AvroInputFormat;
+
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.reflect.ReflectData;
+import org.apache.avro.reflect.ReflectDatumWriter;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+/**
+ * This file defines the classes for the AvroExternalJarProgramITCase.
+ */
+public class AvroExternalJarProgram  {
+
+	private static final class Color {
+
+		private String name;
+		private double saturation;
+
+		public Color() {
+			name = "";
+			saturation = 1.0;
+		}
+
+		public Color(String name, double saturation) {
+			this.name = name;
+			this.saturation = saturation;
+		}
+
+		public String getName() {
+			return name;
+		}
+
+		public void setName(String name) {
+			this.name = name;
+		}
+
+		public double getSaturation() {
+			return saturation;
+		}
+
+		public void setSaturation(double saturation) {
+			this.saturation = saturation;
+		}
+
+		@Override
+		public String toString() {
+			return name + '(' + saturation + ')';
+		}
+	}
+
+	private static final class MyUser {
+
+		private String name;
+		private List<Color> colors;
+
+		public MyUser() {
+			name = "unknown";
+			colors = new ArrayList<Color>();
+		}
+
+		public MyUser(String name, List<Color> colors) {
+			this.name = name;
+			this.colors = colors;
+		}
+
+		public String getName() {
+			return name;
+		}
+
+		public List<Color> getColors() {
+			return colors;
+		}
+
+		public void setName(String name) {
+			this.name = name;
+		}
+
+		public void setColors(List<Color> colors) {
+			this.colors = colors;
+		}
+
+		@Override
+		public String toString() {
+			return name + " : " + colors;
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	// --------------------------------------------------------------------------------------------
+
+	private static final class NameExtractor extends RichMapFunction<MyUser, Tuple2<String, MyUser>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<String, MyUser> map(MyUser u) {
+			String namePrefix = u.getName().substring(0, 1);
+			return new Tuple2<String, MyUser>(namePrefix, u);
+		}
+	}
+
+	private static final class NameGrouper extends RichReduceFunction<Tuple2<String, MyUser>> {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public Tuple2<String, MyUser> reduce(Tuple2<String, MyUser> val1, Tuple2<String, MyUser> val2) {
+			return val1;
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Test Data
+	// --------------------------------------------------------------------------------------------
+
+	private static final class Generator {
+
+		private final Random rnd = new Random(2389756789345689276L);
+
+		public MyUser nextUser() {
+			return randomUser();
+		}
+
+		private MyUser randomUser() {
+
+			int numColors = rnd.nextInt(5);
+			ArrayList<Color> colors = new ArrayList<Color>(numColors);
+			for (int i = 0; i < numColors; i++) {
+				colors.add(new Color(randomString(), rnd.nextDouble()));
+			}
+
+			return new MyUser(randomString(), colors);
+		}
+
+		private String randomString() {
+			char[] c = new char[this.rnd.nextInt(20) + 5];
+
+			for (int i = 0; i < c.length; i++) {
+				c[i] = (char) (this.rnd.nextInt(150) + 40);
+			}
+
+			return new String(c);
+		}
+	}
+
+	public static void writeTestData(File testFile, int numRecords) throws IOException {
+
+		DatumWriter<MyUser> userDatumWriter = new ReflectDatumWriter<MyUser>(MyUser.class);
+		DataFileWriter<MyUser> dataFileWriter = new DataFileWriter<MyUser>(userDatumWriter);
+
+		dataFileWriter.create(ReflectData.get().getSchema(MyUser.class), testFile);
+
+		Generator generator = new Generator();
+
+		for (int i = 0; i < numRecords; i++) {
+			MyUser user = generator.nextUser();
+			dataFileWriter.append(user);
+		}
+
+		dataFileWriter.close();
+	}
+
+//	public static void main(String[] args) throws Exception {
+//		String testDataFile = new File("src/test/resources/testdata.avro").getAbsolutePath();
+//		writeTestData(new File(testDataFile), 50);
+//	}
+
+	public static void main(String[] args) throws Exception {
+		String inputPath = args[0];
+
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<MyUser> input = env.createInput(new AvroInputFormat<MyUser>(new Path(inputPath), MyUser.class));
+
+		DataSet<Tuple2<String, MyUser>> result = input.map(new NameExtractor()).groupBy(0).reduce(new NameGrouper());
+
+		result.output(new DiscardingOutputFormat<Tuple2<String, MyUser>>());
+		env.execute();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroGenericArraySerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroGenericArraySerializerTest.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroGenericArraySerializerTest.java
new file mode 100644
index 0000000..89be9c0
--- /dev/null
+++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroGenericArraySerializerTest.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.formats.avro.typeutils;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.typeutils.runtime.AbstractGenericArraySerializerTest;
+
+/**
+ * Test for the {@link AvroSerializer}.
+ */
+public class AvroGenericArraySerializerTest extends AbstractGenericArraySerializerTest {
+
+	@Override
+	protected <T> TypeSerializer<T> createComponentSerializer(Class<T> type) {
+		return new AvroSerializer<T>(type);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroGenericTypeComparatorTest.java
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroGenericTypeComparatorTest.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroGenericTypeComparatorTest.java
new file mode 100644
index 0000000..a247766
--- /dev/null
+++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroGenericTypeComparatorTest.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.formats.avro.typeutils;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.typeutils.runtime.AbstractGenericTypeComparatorTest;
+
+/**
+ * Test for the {@link AvroSerializer}.
+ */
+public class AvroGenericTypeComparatorTest extends AbstractGenericTypeComparatorTest {
+
+	@Override
+	protected <T> TypeSerializer<T> createSerializer(Class<T> type) {
+		return new AvroSerializer<>(type);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroGenericTypeSerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroGenericTypeSerializerTest.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroGenericTypeSerializerTest.java
new file mode 100644
index 0000000..1c1a19b
--- /dev/null
+++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroGenericTypeSerializerTest.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.formats.avro.typeutils;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.typeutils.runtime.AbstractGenericTypeSerializerTest;
+
+/**
+ * Test for the {@link AvroSerializer}.
+ */
+public class AvroGenericTypeSerializerTest extends AbstractGenericTypeSerializerTest {
+
+	@Override
+	protected <T> TypeSerializer<T> createSerializer(Class<T> type) {
+		return new AvroSerializer<>(type);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/537a10ea/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroSerializerEmptyArrayTest.java
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroSerializerEmptyArrayTest.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroSerializerEmptyArrayTest.java
new file mode 100644
index 0000000..bb3d911
--- /dev/null
+++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroSerializerEmptyArrayTest.java
@@ -0,0 +1,217 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.formats.avro.typeutils;
+
+import org.apache.flink.api.common.typeutils.SerializerTestInstance;
+
+import org.apache.avro.reflect.Nullable;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for the {@link AvroSerializer}.
+ */
+public class AvroSerializerEmptyArrayTest {
+
+	@Test
+	public void testBookSerialization() {
+		try {
+			Book b = new Book(123, "This is a test book", 26382648);
+			AvroSerializer<Book> serializer = new AvroSerializer<Book>(Book.class);
+			SerializerTestInstance<Book> test = new SerializerTestInstance<Book>(serializer, Book.class, -1, b);
+			test.testAll();
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testSerialization() {
+		try {
+			List<String> titles = new ArrayList<String>();
+
+			List<Book> books = new ArrayList<Book>();
+			books.add(new Book(123, "This is a test book", 1));
+			books.add(new Book(24234234, "This is a test book", 1));
+			books.add(new Book(1234324, "This is a test book", 3));
+
+			BookAuthor a = new BookAuthor(1, titles, "Test Author");
+			a.books = books;
+			a.bookType = BookAuthor.BookType.journal;
+
+			AvroSerializer<BookAuthor> serializer = new AvroSerializer<BookAuthor>(BookAuthor.class);
+
+			SerializerTestInstance<BookAuthor> test = new SerializerTestInstance<BookAuthor>(serializer, BookAuthor.class, -1, a);
+			test.testAll();
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	/**
+	 * Avro POJO for testing.
+	 */
+	public static class Book {
+
+		long bookId;
+		@Nullable
+		String title;
+		long authorId;
+
+		public Book() {}
+
+		public Book(long bookId, String title, long authorId) {
+			this.bookId = bookId;
+			this.title = title;
+			this.authorId = authorId;
+		}
+
+		@Override
+		public int hashCode() {
+			final int prime = 31;
+			int result = 1;
+			result = prime * result + (int) (authorId ^ (authorId >>> 32));
+			result = prime * result + (int) (bookId ^ (bookId >>> 32));
+			result = prime * result + ((title == null) ? 0 : title.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;
+			}
+			Book other = (Book) obj;
+			if (authorId != other.authorId) {
+				return false;
+			}
+			if (bookId != other.bookId) {
+				return false;
+			}
+			if (title == null) {
+				if (other.title != null) {
+					return false;
+				}
+			} else if (!title.equals(other.title)) {
+				return false;
+			}
+			return true;
+		}
+	}
+
+	/**
+	 * Avro POJO for testing.
+	 */
+	public static class BookAuthor {
+
+		enum BookType {
+			book,
+			article,
+			journal
+		}
+
+		long authorId;
+
+		@Nullable
+		List<String> bookTitles;
+
+		@Nullable
+		List<Book> books;
+
+		String authorName;
+
+		BookType bookType;
+
+		public BookAuthor() {}
+
+		public BookAuthor(long authorId, List<String> bookTitles, String authorName) {
+			this.authorId = authorId;
+			this.bookTitles = bookTitles;
+			this.authorName = authorName;
+		}
+
+		@Override
+		public int hashCode() {
+			final int prime = 31;
+			int result = 1;
+			result = prime * result + (int) (authorId ^ (authorId >>> 32));
+			result = prime * result + ((authorName == null) ? 0 : authorName.hashCode());
+			result = prime * result + ((bookTitles == null) ? 0 : bookTitles.hashCode());
+			result = prime * result + ((bookType == null) ? 0 : bookType.hashCode());
+			result = prime * result + ((books == null) ? 0 : books.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;
+			}
+			BookAuthor other = (BookAuthor) obj;
+			if (authorId != other.authorId) {
+				return false;
+			}
+			if (authorName == null) {
+				if (other.authorName != null) {
+					return false;
+				}
+			} else if (!authorName.equals(other.authorName)) {
+				return false;
+			}
+			if (bookTitles == null) {
+				if (other.bookTitles != null) {
+					return false;
+				}
+			} else if (!bookTitles.equals(other.bookTitles)) {
+				return false;
+			}
+			if (bookType != other.bookType) {
+				return false;
+			}
+			if (books == null) {
+				if (other.books != null) {
+					return false;
+				}
+			} else if (!books.equals(other.books)) {
+				return false;
+			}
+			return true;
+		}
+	}
+}


[14/21] flink git commit: [hotfix] [avro] Improve Avro type hierarchy checks in AvroKryoSerializerUtils

Posted by se...@apache.org.
[hotfix] [avro] Improve Avro type hierarchy checks in AvroKryoSerializerUtils


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

Branch: refs/heads/master
Commit: 6dd5e2b272f717f6d97bb8dbf5f28c59d8c90151
Parents: a042ba9
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Nov 2 21:03:45 2017 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Nov 3 16:40:35 2017 +0100

----------------------------------------------------------------------
 .../flink/formats/avro/utils/AvroKryoSerializerUtils.java      | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/6dd5e2b2/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/AvroKryoSerializerUtils.java
----------------------------------------------------------------------
diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/AvroKryoSerializerUtils.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/AvroKryoSerializerUtils.java
index c28f6cf..5744abc 100644
--- a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/AvroKryoSerializerUtils.java
+++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/utils/AvroKryoSerializerUtils.java
@@ -37,8 +37,6 @@ import org.apache.avro.generic.GenericData;
 import java.io.Serializable;
 import java.util.LinkedHashMap;
 
-import static org.apache.flink.api.java.typeutils.TypeExtractionUtils.hasSuperclass;
-
 /**
  * Utilities for integrating Avro serializers in Kryo.
  */
@@ -46,7 +44,9 @@ public class AvroKryoSerializerUtils extends AvroUtils {
 
 	@Override
 	public void addAvroSerializersIfRequired(ExecutionConfig reg, Class<?> type) {
-		if (hasSuperclass(type, AVRO_SPECIFIC_RECORD_BASE) || hasSuperclass(type, AVRO_GENERIC_RECORD)) {
+		if (org.apache.avro.specific.SpecificRecordBase.class.isAssignableFrom(type) ||
+			org.apache.avro.generic.GenericData.Record.class.isAssignableFrom(type)) {
+
 			// Avro POJOs contain java.util.List which have GenericData.Array as their runtime type
 			// because Kryo is not able to serialize them properly, we use this serializer for them
 			reg.registerTypeWithKryoSerializer(GenericData.Array.class, Serializers.SpecificInstanceCollectionSerializerForArrayList.class);