[FLINK-6157] [core] Make TypeInformation fully serializable

This closes #3619.


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

Branch: refs/heads/master
Commit: 6cf4a93ebbf36f6c1dd4cc2c9b17dd58880c17c2
Parents: c90b6da
Author: twalthr <twal...@apache.org>
Authored: Mon Mar 27 10:33:58 2017 +0200
Committer: Greg Hogan <c...@greghogan.com>
Committed: Tue May 9 13:12:05 2017 -0400

----------------------------------------------------------------------
 flink-connectors/flink-avro/pom.xml             |  15 +-
 .../api/java/typeutils/AvroTypeInfoTest.java    |  37 +++++
 .../flink-hadoop-compatibility/pom.xml          |  13 ++
 .../java/typeutils/WritableTypeInfoTest.java    |  37 ++---
 .../common/typeinfo/BasicArrayTypeInfoTest.java |  42 +++++
 .../api/common/typeinfo/BasicTypeInfoTest.java  |  46 ++++++
 .../common/typeinfo/FractionalTypeInfoTest.java |  35 +++++
 .../common/typeinfo/IntegerTypeInfoTest.java    |  37 +++++
 .../common/typeinfo/NothingTypeInfoTest.java    |  34 ++++
 .../common/typeinfo/NumericTypeInfoTest.java    |  39 +++++
 .../typeinfo/PrimitiveArrayTypeInfoTest.java    |  41 +++++
 .../common/typeinfo/SqlTimeTypeInfoTest.java    |  36 +++++
 .../typeutils/TypeInformationTestBase.java      | 155 +++++++++++++++++++
 .../api/java/typeutils/EitherTypeInfoTest.java  |  42 ++---
 .../api/java/typeutils/EnumTypeInfoTest.java    |  35 ++---
 .../api/java/typeutils/GenericTypeInfoTest.java |  34 ++--
 .../api/java/typeutils/ListTypeInfoTest.java    |  37 +++++
 .../api/java/typeutils/MapTypeInfoTest.java     |  37 +++++
 .../api/java/typeutils/MissingTypeInfoTest.java |  29 ++--
 .../java/typeutils/ObjectArrayTypeInfoTest.java |  46 ++----
 .../api/java/typeutils/PojoTypeInfoTest.java    |  82 ++--------
 .../api/java/typeutils/RowTypeInfoTest.java     |  60 ++-----
 .../api/java/typeutils/TupleTypeInfoTest.java   |  49 ++----
 .../api/java/typeutils/ValueTypeInfoTest.java   |  46 +++---
 .../flink/types/BasicArrayTypeInfoTest.java     |  56 -------
 .../apache/flink/types/NothingTypeInfoTest.java |  47 ------
 .../flink/types/PrimitiveArrayTypeInfoTest.java |  56 -------
 flink-scala/pom.xml                             |  13 ++
 .../scala/typeutils/CaseClassTypeInfoTest.scala |  51 ++----
 .../scala/typeutils/EitherTypeInfoTest.scala    |  39 ++---
 .../scala/typeutils/EnumValueTypeInfoTest.scala |  36 ++---
 .../scala/typeutils/OptionTypeInfoTest.scala    |  38 ++---
 .../typeutils/ScalaNothingTypeInfoTest.scala    |  31 ++++
 .../typeutils/TraversableTypeInfoTest.scala     |  46 ++----
 .../api/scala/typeutils/TryTypeInfoTest.scala   |  40 ++---
 .../api/scala/typeutils/UnitTypeInfoTest.scala  |  31 ++++
 flink-tests/pom.xml                             |  33 +++-
 .../completeness/TypeInfoTestCoverageTest.java  |  62 ++++++++
 38 files changed, 981 insertions(+), 662 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-connectors/flink-avro/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-avro/pom.xml 
b/flink-connectors/flink-avro/pom.xml
index 468a2ba..170b344 100644
--- a/flink-connectors/flink-avro/pom.xml
+++ b/flink-connectors/flink-avro/pom.xml
@@ -154,8 +154,21 @@ under the License.
                                        </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>
                </plugins>
-               
+
                <pluginManagement>
                        <plugins>
                                <!--This plugin's configuration is used to 
store Eclipse m2e settings only. It has no influence on the Maven build 
itself.-->

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/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
new file mode 100644
index 0000000..e0bb1a1
--- /dev/null
+++ 
b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/java/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.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/6cf4a93e/flink-connectors/flink-hadoop-compatibility/pom.xml
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-hadoop-compatibility/pom.xml 
b/flink-connectors/flink-hadoop-compatibility/pom.xml
index 8c5b673..003225b 100644
--- a/flink-connectors/flink-hadoop-compatibility/pom.xml
+++ b/flink-connectors/flink-hadoop-compatibility/pom.xml
@@ -176,6 +176,19 @@ under the License.
                                </configuration>
                        </plugin>
 
+                       <!-- Add test classes to test jar in order to test 
WritableTypeInfo. -->
+                       <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-surefire-plugin</artifactId>

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableTypeInfoTest.java
----------------------------------------------------------------------
diff --git 
a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableTypeInfoTest.java
 
b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableTypeInfoTest.java
index eb9cdf0..666ab84 100644
--- 
a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableTypeInfoTest.java
+++ 
b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/api/java/typeutils/WritableTypeInfoTest.java
@@ -18,34 +18,23 @@
 
 package org.apache.flink.api.java.typeutils;
 
-import org.apache.flink.util.TestLogger;
-import org.apache.hadoop.io.Writable;
-import org.junit.Test;
-
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
+import org.apache.hadoop.io.Writable;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-
-public class WritableTypeInfoTest extends TestLogger {
-       
-       @Test
-       public void testWritableTypeInfoEquality() {
-               WritableTypeInfo<TestClass> tpeInfo1 = new 
WritableTypeInfo<>(TestClass.class);
-               WritableTypeInfo<TestClass> tpeInfo2 = new 
WritableTypeInfo<>(TestClass.class);
-
-               assertEquals(tpeInfo1, tpeInfo2);
-               assertEquals(tpeInfo1.hashCode(), tpeInfo2.hashCode());
-       }
-
-       @Test
-       public void testWritableTypeInfoInequality() {
-               WritableTypeInfo<TestClass> tpeInfo1 = new 
WritableTypeInfo<>(TestClass.class);
-               WritableTypeInfo<AlternateClass> tpeInfo2 = new 
WritableTypeInfo<>(AlternateClass.class);
-
-               assertNotEquals(tpeInfo1, tpeInfo2);
+/**
+ * Test for {@link WritableTypeInfo}.
+ */
+public class WritableTypeInfoTest extends 
TypeInformationTestBase<WritableTypeInfo<?>> {
+
+       @Override
+       protected WritableTypeInfo<?>[] getTestData() {
+               return new WritableTypeInfo<?>[] {
+                       new WritableTypeInfo<>(TestClass.class),
+                       new WritableTypeInfo<>(AlternateClass.class)
+               };
        }
 
        // 
------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/BasicArrayTypeInfoTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/BasicArrayTypeInfoTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/BasicArrayTypeInfoTest.java
new file mode 100644
index 0000000..7a51808
--- /dev/null
+++ 
b/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/BasicArrayTypeInfoTest.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.typeinfo;
+
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
+
+/**
+ * Test for {@link BasicArrayTypeInfo}.
+ */
+public class BasicArrayTypeInfoTest extends 
TypeInformationTestBase<BasicArrayTypeInfo<?, ?>> {
+
+       @Override
+       protected BasicArrayTypeInfo<?, ?>[] getTestData() {
+               return new BasicArrayTypeInfo<?, ?>[] {
+                       BasicArrayTypeInfo.STRING_ARRAY_TYPE_INFO,
+                       BasicArrayTypeInfo.BOOLEAN_ARRAY_TYPE_INFO,
+                       BasicArrayTypeInfo.BYTE_ARRAY_TYPE_INFO,
+                       BasicArrayTypeInfo.SHORT_ARRAY_TYPE_INFO,
+                       BasicArrayTypeInfo.INT_ARRAY_TYPE_INFO,
+                       BasicArrayTypeInfo.LONG_ARRAY_TYPE_INFO,
+                       BasicArrayTypeInfo.FLOAT_ARRAY_TYPE_INFO,
+                       BasicArrayTypeInfo.DOUBLE_ARRAY_TYPE_INFO,
+                       BasicArrayTypeInfo.CHAR_ARRAY_TYPE_INFO
+               };
+       }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/BasicTypeInfoTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/BasicTypeInfoTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/BasicTypeInfoTest.java
new file mode 100644
index 0000000..cd4511a
--- /dev/null
+++ 
b/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/BasicTypeInfoTest.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.typeinfo;
+
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
+
+/**
+ * Test for {@link BasicTypeInfo}.
+ */
+public class BasicTypeInfoTest extends 
TypeInformationTestBase<BasicTypeInfo<?>> {
+
+       @Override
+       protected BasicTypeInfo<?>[] getTestData() {
+               return new BasicTypeInfo<?>[] {
+                       BasicTypeInfo.STRING_TYPE_INFO,
+                       BasicTypeInfo.BOOLEAN_TYPE_INFO,
+                       BasicTypeInfo.BYTE_TYPE_INFO,
+                       BasicTypeInfo.SHORT_TYPE_INFO,
+                       BasicTypeInfo.INT_TYPE_INFO,
+                       BasicTypeInfo.LONG_TYPE_INFO,
+                       BasicTypeInfo.FLOAT_TYPE_INFO,
+                       BasicTypeInfo.DOUBLE_TYPE_INFO,
+                       BasicTypeInfo.CHAR_TYPE_INFO,
+                       BasicTypeInfo.DATE_TYPE_INFO,
+                       BasicTypeInfo.VOID_TYPE_INFO,
+                       BasicTypeInfo.BIG_INT_TYPE_INFO,
+                       BasicTypeInfo.BIG_DEC_TYPE_INFO
+               };
+       }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/FractionalTypeInfoTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/FractionalTypeInfoTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/FractionalTypeInfoTest.java
new file mode 100644
index 0000000..87c4db5
--- /dev/null
+++ 
b/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/FractionalTypeInfoTest.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.typeinfo;
+
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
+
+/**
+ * Test for {@link FractionalTypeInfo}.
+ */
+public class FractionalTypeInfoTest extends 
TypeInformationTestBase<FractionalTypeInfo<?>> {
+
+       @Override
+       protected FractionalTypeInfo<?>[] getTestData() {
+               return new FractionalTypeInfo<?>[] {
+                       (FractionalTypeInfo<?>) BasicTypeInfo.FLOAT_TYPE_INFO,
+                       (FractionalTypeInfo<?>) BasicTypeInfo.DOUBLE_TYPE_INFO
+               };
+       }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/IntegerTypeInfoTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/IntegerTypeInfoTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/IntegerTypeInfoTest.java
new file mode 100644
index 0000000..b5fd41f
--- /dev/null
+++ 
b/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/IntegerTypeInfoTest.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.api.common.typeinfo;
+
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
+
+/**
+ * Test for {@link IntegerTypeInfo}.
+ */
+public class IntegerTypeInfoTest extends 
TypeInformationTestBase<IntegerTypeInfo<?>> {
+
+       @Override
+       protected IntegerTypeInfo<?>[] getTestData() {
+               return new IntegerTypeInfo<?>[] {
+                       (IntegerTypeInfo<?>) BasicTypeInfo.BYTE_TYPE_INFO,
+                       (IntegerTypeInfo<?>) BasicTypeInfo.SHORT_TYPE_INFO,
+                       (IntegerTypeInfo<?>) BasicTypeInfo.INT_TYPE_INFO,
+                       (IntegerTypeInfo<?>) BasicTypeInfo.LONG_TYPE_INFO
+               };
+       }
+}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/NumericTypeInfoTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/NumericTypeInfoTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/NumericTypeInfoTest.java
new file mode 100644
index 0000000..49dd722
--- /dev/null
+++ 
b/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/NumericTypeInfoTest.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.typeinfo;
+
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
+
+/**
+ * Test for {@link NumericTypeInfo}.
+ */
+public class NumericTypeInfoTest extends 
TypeInformationTestBase<NumericTypeInfo<?>> {
+
+       @Override
+       protected NumericTypeInfo<?>[] getTestData() {
+               return new NumericTypeInfo<?>[] {
+                       (NumericTypeInfo<?>) BasicTypeInfo.BYTE_TYPE_INFO,
+                       (NumericTypeInfo<?>) BasicTypeInfo.SHORT_TYPE_INFO,
+                       (NumericTypeInfo<?>) BasicTypeInfo.INT_TYPE_INFO,
+                       (NumericTypeInfo<?>) BasicTypeInfo.LONG_TYPE_INFO,
+                       (NumericTypeInfo<?>) BasicTypeInfo.FLOAT_TYPE_INFO,
+                       (NumericTypeInfo<?>) BasicTypeInfo.DOUBLE_TYPE_INFO
+               };
+       }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/PrimitiveArrayTypeInfoTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/PrimitiveArrayTypeInfoTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/PrimitiveArrayTypeInfoTest.java
new file mode 100644
index 0000000..867ccad
--- /dev/null
+++ 
b/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/PrimitiveArrayTypeInfoTest.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.typeinfo;
+
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
+
+/**
+ * Test for {@link PrimitiveArrayTypeInfoTest}.
+ */
+public class PrimitiveArrayTypeInfoTest extends 
TypeInformationTestBase<PrimitiveArrayTypeInfo<?>> {
+
+       @Override
+       protected PrimitiveArrayTypeInfo<?>[] getTestData() {
+               return new PrimitiveArrayTypeInfo<?>[] {
+                       
PrimitiveArrayTypeInfo.BOOLEAN_PRIMITIVE_ARRAY_TYPE_INFO,
+                       PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO,
+                       PrimitiveArrayTypeInfo.SHORT_PRIMITIVE_ARRAY_TYPE_INFO,
+                       PrimitiveArrayTypeInfo.INT_PRIMITIVE_ARRAY_TYPE_INFO,
+                       PrimitiveArrayTypeInfo.LONG_PRIMITIVE_ARRAY_TYPE_INFO,
+                       PrimitiveArrayTypeInfo.FLOAT_PRIMITIVE_ARRAY_TYPE_INFO,
+                       PrimitiveArrayTypeInfo.DOUBLE_PRIMITIVE_ARRAY_TYPE_INFO,
+                       PrimitiveArrayTypeInfo.CHAR_PRIMITIVE_ARRAY_TYPE_INFO
+               };
+       }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/SqlTimeTypeInfoTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/SqlTimeTypeInfoTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/SqlTimeTypeInfoTest.java
new file mode 100644
index 0000000..892ab2b
--- /dev/null
+++ 
b/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/SqlTimeTypeInfoTest.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.typeinfo;
+
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
+
+/**
+ * Test for {@link SqlTimeTypeInfo}.
+ */
+public class SqlTimeTypeInfoTest extends 
TypeInformationTestBase<SqlTimeTypeInfo<?>> {
+
+       @Override
+       protected SqlTimeTypeInfo<?>[] getTestData() {
+               return new SqlTimeTypeInfo<?>[] {
+                       SqlTimeTypeInfo.DATE,
+                       SqlTimeTypeInfo.TIME,
+                       SqlTimeTypeInfo.TIMESTAMP
+               };
+       }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeInformationTestBase.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeInformationTestBase.java
 
b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeInformationTestBase.java
new file mode 100644
index 0000000..bd35070
--- /dev/null
+++ 
b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeInformationTestBase.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.typeutils;
+
+import java.io.IOException;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.TestLogger;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+import org.junit.Test;
+
+/**
+ * Abstract test base for type information.
+ */
+public abstract class TypeInformationTestBase<T extends TypeInformation<?>> 
extends TestLogger {
+
+       protected abstract T[] getTestData();
+
+       @Test
+       public void testHashcodeAndEquals() throws Exception {
+               final T[] testData = getTestData();
+               final TypeInformation<?> unrelatedTypeInfo = new 
UnrelatedTypeInfo();
+
+               for (T typeInfo : testData) {
+                       // check for implemented hashCode and equals
+                       if 
(typeInfo.getClass().getMethod("hashCode").getDeclaringClass() == Object.class) 
{
+                               throw new AssertionError("Type information does 
not implement own hashCode method: " +
+                                       typeInfo.getClass().getCanonicalName());
+                       }
+                       if (typeInfo.getClass().getMethod("equals", 
Object.class).getDeclaringClass() == Object.class) {
+                               throw new AssertionError("Type information does 
not implement own equals method: " +
+                                       typeInfo.getClass().getCanonicalName());
+                       }
+
+                       // compare among test data
+                       for (T otherTypeInfo : testData) {
+                               assertTrue("canEqual() returns inconsistent 
results.", typeInfo.canEqual(otherTypeInfo));
+                               // test equality
+                               if (typeInfo == otherTypeInfo) {
+                                       assertTrue("hashCode() returns 
inconsistent results.", typeInfo.hashCode() == otherTypeInfo.hashCode());
+                                       assertEquals("equals() is false for 
same object.", typeInfo, otherTypeInfo);
+                               }
+                               // test inequality
+                               else {
+                                       assertNotEquals("equals() returned true 
for different objects.", typeInfo, otherTypeInfo);
+                               }
+                       }
+
+                       // compare with unrelated type
+                       assertFalse("Type information allows to compare with 
unrelated type.", typeInfo.canEqual(unrelatedTypeInfo));
+                       assertNotEquals(typeInfo, unrelatedTypeInfo);
+               }
+       }
+
+       @Test
+       public void testSerialization() {
+               final T[] testData = getTestData();
+
+               for (T typeInfo : testData) {
+                       final byte[] serialized;
+                       try {
+                               serialized = 
InstantiationUtil.serializeObject(typeInfo);
+                       } catch (IOException e) {
+                               throw new AssertionError("Could not serialize 
type information: " + typeInfo, e);
+                       }
+                       final T deserialized;
+                       try {
+                               deserialized = 
InstantiationUtil.deserializeObject(serialized, getClass().getClassLoader());
+                       } catch (IOException | ClassNotFoundException e) {
+                               throw new AssertionError("Could not deserialize 
type information: " + typeInfo, e);
+                       }
+                       if (typeInfo.hashCode() != deserialized.hashCode() || 
!typeInfo.equals(deserialized)) {
+                               throw new AssertionError("Deserialized type 
information differs from original one.");
+                       }
+               }
+       }
+
+       private static class UnrelatedTypeInfo extends TypeInformation<Object> {
+
+               @Override
+               public boolean isBasicType() {
+                       return false;
+               }
+
+               @Override
+               public boolean isTupleType() {
+                       return false;
+               }
+
+               @Override
+               public int getArity() {
+                       return 0;
+               }
+
+               @Override
+               public int getTotalFields() {
+                       return 0;
+               }
+
+               @Override
+               public Class<Object> getTypeClass() {
+                       return null;
+               }
+
+               @Override
+               public boolean isKeyType() {
+                       return false;
+               }
+
+               @Override
+               public TypeSerializer<Object> createSerializer(ExecutionConfig 
config) {
+                       return null;
+               }
+
+               @Override
+               public String toString() {
+                       return null;
+               }
+
+               @Override
+               public boolean equals(Object obj) {
+                       return false;
+               }
+
+               @Override
+               public int hashCode() {
+                       return 0;
+               }
+
+               @Override
+               public boolean canEqual(Object obj) {
+                       return false;
+               }
+       }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-core/src/test/java/org/apache/flink/api/java/typeutils/EitherTypeInfoTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/EitherTypeInfoTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/EitherTypeInfoTest.java
index 13009ee..a5e340e 100644
--- 
a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/EitherTypeInfoTest.java
+++ 
b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/EitherTypeInfoTest.java
@@ -18,37 +18,21 @@
 
 package org.apache.flink.api.java.typeutils;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.util.TestLogger;
-import org.junit.Test;
-
-public class EitherTypeInfoTest extends TestLogger {
-
-       @Test
-       public void testEitherTypeEquality() {
-               EitherTypeInfo<Integer, String> eitherInfo1 = new 
EitherTypeInfo<Integer, String>(
-                               BasicTypeInfo.INT_TYPE_INFO, 
BasicTypeInfo.STRING_TYPE_INFO);
 
-               EitherTypeInfo<Integer, String> eitherInfo2 = new 
EitherTypeInfo<Integer, String>(
-                               BasicTypeInfo.INT_TYPE_INFO, 
BasicTypeInfo.STRING_TYPE_INFO);
-
-               assertEquals(eitherInfo1, eitherInfo2);
-               assertEquals(eitherInfo1.hashCode(), eitherInfo2.hashCode());
-       }
-
-       @Test
-       public void testEitherTypeInequality() {
-               EitherTypeInfo<Integer, String> eitherInfo1 = new 
EitherTypeInfo<Integer, String>(
-                               BasicTypeInfo.INT_TYPE_INFO, 
BasicTypeInfo.STRING_TYPE_INFO);
-
-               EitherTypeInfo<Integer, Tuple2<Double, Long>> eitherInfo2 = new 
EitherTypeInfo<Integer, Tuple2<Double, Long>>(
-                               BasicTypeInfo.INT_TYPE_INFO, new 
TupleTypeInfo<Tuple2<Double, Long>>(
-                               TypeExtractor.getForClass(Double.class), 
TypeExtractor.getForClass(String.class)));
-
-               assertNotEquals(eitherInfo1, eitherInfo2);
+/**
+ * Test for {@link EitherTypeInfo}.
+ */
+public class EitherTypeInfoTest extends 
TypeInformationTestBase<EitherTypeInfo<?, ?>> {
+
+       @Override
+       protected EitherTypeInfo<?, ?>[] getTestData() {
+               return new EitherTypeInfo<?, ?>[] {
+                       new EitherTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, 
BasicTypeInfo.STRING_TYPE_INFO),
+                       new EitherTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO,
+                               new TupleTypeInfo<Tuple2<Double, 
Long>>(TypeExtractor.getForClass(Double.class), 
TypeExtractor.getForClass(String.class)))
+               };
        }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-core/src/test/java/org/apache/flink/api/java/typeutils/EnumTypeInfoTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/EnumTypeInfoTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/EnumTypeInfoTest.java
index b200566..2023f7f 100644
--- 
a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/EnumTypeInfoTest.java
+++ 
b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/EnumTypeInfoTest.java
@@ -18,11 +18,21 @@
 
 package org.apache.flink.api.java.typeutils;
 
-import org.apache.flink.util.TestLogger;
-import org.junit.Test;
-import static org.junit.Assert.*;
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
 
-public class EnumTypeInfoTest extends TestLogger {
+/**
+ * Test for {@link EnumTypeInfo}.
+ */
+public class EnumTypeInfoTest extends TypeInformationTestBase<EnumTypeInfo<?>> 
{
+
+       @Override
+       @SuppressWarnings("unchecked")
+       protected EnumTypeInfo<?>[] getTestData() {
+               return new EnumTypeInfo<?>[] {
+                       (EnumTypeInfo<?>) new EnumTypeInfo(TestEnum.class),
+                       (EnumTypeInfo<?>) new 
EnumTypeInfo(AlternativeEnum.class)
+               };
+       }
 
        enum TestEnum {
                ONE, TWO
@@ -31,21 +41,4 @@ public class EnumTypeInfoTest extends TestLogger {
        enum AlternativeEnum {
                ONE, TWO
        }
-
-       @Test
-       public void testEnumTypeEquality() {
-               EnumTypeInfo<TestEnum> enumTypeInfo1 = new 
EnumTypeInfo<TestEnum>(TestEnum.class);
-               EnumTypeInfo<TestEnum> enumTypeInfo2 = new 
EnumTypeInfo<TestEnum>(TestEnum.class);
-
-               assertEquals(enumTypeInfo1, enumTypeInfo2);
-               assertEquals(enumTypeInfo1.hashCode(), 
enumTypeInfo2.hashCode());
-       }
-
-       @Test
-       public void testEnumTypeInequality() {
-               EnumTypeInfo<TestEnum> enumTypeInfo1 = new 
EnumTypeInfo<TestEnum>(TestEnum.class);
-               EnumTypeInfo<AlternativeEnum> enumTypeInfo2 = new 
EnumTypeInfo<AlternativeEnum>(AlternativeEnum.class);
-
-               assertNotEquals(enumTypeInfo1, enumTypeInfo2);
-       }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-core/src/test/java/org/apache/flink/api/java/typeutils/GenericTypeInfoTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/GenericTypeInfoTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/GenericTypeInfoTest.java
index fad43df..8dbcc99 100644
--- 
a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/GenericTypeInfoTest.java
+++ 
b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/GenericTypeInfoTest.java
@@ -18,30 +18,22 @@
 
 package org.apache.flink.api.java.typeutils;
 
-import org.apache.flink.util.TestLogger;
-import org.junit.Test;
-import static org.junit.Assert.*;
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
 
-public class GenericTypeInfoTest extends TestLogger {
+/**
+ * Test for {@link GenericTypeInfo}.
+ */
+public class GenericTypeInfoTest extends 
TypeInformationTestBase<GenericTypeInfo<?>> {
+
+       @Override
+       protected GenericTypeInfo<?>[] getTestData() {
+               return new GenericTypeInfo<?>[] {
+                       new GenericTypeInfo<>(TestClass.class),
+                       new GenericTypeInfo<>(AlternativeClass.class)
+               };
+       }
 
        static class TestClass {}
        static class AlternativeClass {}
 
-       @Test
-       public void testGenericTypeInfoEquality() {
-               GenericTypeInfo<TestClass> tpeInfo1 = new 
GenericTypeInfo<>(TestClass.class);
-               GenericTypeInfo<TestClass> tpeInfo2 = new 
GenericTypeInfo<>(TestClass.class);
-
-               assertEquals(tpeInfo1, tpeInfo2);
-               assertEquals(tpeInfo1.hashCode(), tpeInfo2.hashCode());
-       }
-
-       @Test
-       public void testGenericTypeInfoInequality() {
-               GenericTypeInfo<TestClass> tpeInfo1 = new 
GenericTypeInfo<>(TestClass.class);
-               GenericTypeInfo<AlternativeClass> tpeInfo2 = new 
GenericTypeInfo<>(AlternativeClass.class);
-
-               assertNotEquals(tpeInfo1, tpeInfo2);
-       }
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-core/src/test/java/org/apache/flink/api/java/typeutils/ListTypeInfoTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/ListTypeInfoTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/ListTypeInfoTest.java
new file mode 100644
index 0000000..58cabfd
--- /dev/null
+++ 
b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/ListTypeInfoTest.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.api.java.typeutils;
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
+
+/**
+ * Test for {@link ListTypeInfo}.
+ */
+public class ListTypeInfoTest extends TypeInformationTestBase<ListTypeInfo<?>> 
{
+
+       @Override
+       protected ListTypeInfo<?>[] getTestData() {
+               return new ListTypeInfo<?>[] {
+                       new ListTypeInfo<>(BasicTypeInfo.STRING_TYPE_INFO),
+                       new ListTypeInfo<>(BasicTypeInfo.BOOLEAN_TYPE_INFO),
+                       new ListTypeInfo<>(Object.class),
+               };
+       }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-core/src/test/java/org/apache/flink/api/java/typeutils/MapTypeInfoTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/MapTypeInfoTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/MapTypeInfoTest.java
new file mode 100644
index 0000000..5f3fc06
--- /dev/null
+++ 
b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/MapTypeInfoTest.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.api.java.typeutils;
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
+
+/**
+ * Test for {@link MapTypeInfo}.
+ */
+public class MapTypeInfoTest extends TypeInformationTestBase<MapTypeInfo<?, 
?>> {
+
+       @Override
+       protected MapTypeInfo<?, ?>[] getTestData() {
+               return new MapTypeInfo<?, ?>[] {
+                       new MapTypeInfo<>(BasicTypeInfo.STRING_TYPE_INFO, 
BasicTypeInfo.INT_TYPE_INFO),
+                       new MapTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, 
BasicTypeInfo.STRING_TYPE_INFO),
+                       new MapTypeInfo<>(String.class, Boolean.class)
+               };
+       }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-core/src/test/java/org/apache/flink/api/java/typeutils/MissingTypeInfoTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/MissingTypeInfoTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/MissingTypeInfoTest.java
index ee57475..cccc14d 100644
--- 
a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/MissingTypeInfoTest.java
+++ 
b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/MissingTypeInfoTest.java
@@ -19,29 +19,26 @@
 package org.apache.flink.api.java.typeutils;
 
 import org.apache.flink.api.common.functions.InvalidTypesException;
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
 import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 
 import static org.junit.Assert.*;
 
-public class MissingTypeInfoTest extends TestLogger {
-       static final String functionName = "foobar";
-       static final InvalidTypesException testException = new 
InvalidTypesException("Test exception.");
+public class MissingTypeInfoTest extends 
TypeInformationTestBase<MissingTypeInfo> {
+       private static final String functionName = "foobar";
+       private static final InvalidTypesException testException = new 
InvalidTypesException("Test exception.");
 
-       @Test
-       public void testMissingTypeInfoEquality() {
-               MissingTypeInfo tpeInfo1 = new MissingTypeInfo(functionName, 
testException);
-               MissingTypeInfo tpeInfo2 = new MissingTypeInfo(functionName, 
testException);
-
-               assertEquals(tpeInfo1, tpeInfo2);
-               assertEquals(tpeInfo1.hashCode(), tpeInfo2.hashCode());
+       @Override
+       protected MissingTypeInfo[] getTestData() {
+               return new MissingTypeInfo[] {
+                       new MissingTypeInfo(functionName, testException),
+                       new MissingTypeInfo("alt" + functionName, testException)
+               };
        }
 
-       @Test
-       public void testMissingTypeInfoInequality() {
-               MissingTypeInfo tpeInfo1 = new MissingTypeInfo(functionName, 
testException);
-               MissingTypeInfo tpeInfo2 = new MissingTypeInfo("alt" + 
functionName, testException);
-
-               assertNotEquals(tpeInfo1, tpeInfo2);
+       @Override
+       public void testSerialization() {
+               // this class is not intended to be serialized
        }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-core/src/test/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfoTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfoTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfoTest.java
index f3b39c0..655c28a 100644
--- 
a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfoTest.java
+++ 
b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfoTest.java
@@ -18,41 +18,21 @@
 
 package org.apache.flink.api.java.typeutils;
 
-import org.apache.flink.util.TestLogger;
-import org.junit.Test;
-
 import java.util.ArrayList;
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
 
-import static org.junit.Assert.*;
-
-public class ObjectArrayTypeInfoTest extends TestLogger {
-
-       public static class TestClass{}
-
-       @Test
-       public void testObjectArrayTypeInfoEquality() {
-               ObjectArrayTypeInfo<TestClass[], TestClass> tpeInfo1 = 
ObjectArrayTypeInfo.getInfoFor(
-                       TestClass[].class,
-                       new GenericTypeInfo<TestClass>(TestClass.class));
-
-               ObjectArrayTypeInfo<TestClass[], TestClass> tpeInfo2 = 
ObjectArrayTypeInfo.getInfoFor(
-                       TestClass[].class,
-                       new GenericTypeInfo<TestClass>(TestClass.class));
-
-               assertEquals(tpeInfo1, tpeInfo2);
-               assertEquals(tpeInfo1.hashCode(), tpeInfo2.hashCode());
+/**
+ * Test for {@link ObjectArrayTypeInfo}.
+ */
+public class ObjectArrayTypeInfoTest extends 
TypeInformationTestBase<ObjectArrayTypeInfo<?, ?>> {
+
+       @Override
+       protected ObjectArrayTypeInfo<?, ?>[] getTestData() {
+               return new ObjectArrayTypeInfo<?, ?>[] {
+                       ObjectArrayTypeInfo.getInfoFor(TestClass[].class, new 
GenericTypeInfo<>(TestClass.class)),
+                       ObjectArrayTypeInfo.getInfoFor(TestClass[].class, new 
PojoTypeInfo<>(TestClass.class, new ArrayList<PojoField>()))
+               };
        }
 
-       @Test
-       public void testObjectArrayTypeInfoInequality() {
-               ObjectArrayTypeInfo<TestClass[], TestClass> tpeInfo1 = 
ObjectArrayTypeInfo.getInfoFor(
-                       TestClass[].class,
-                       new GenericTypeInfo<TestClass>(TestClass.class));
-
-               ObjectArrayTypeInfo<TestClass[], TestClass> tpeInfo2 = 
ObjectArrayTypeInfo.getInfoFor(
-                       TestClass[].class,
-                       new PojoTypeInfo<TestClass>(TestClass.class, new 
ArrayList<PojoField>()));
-
-               assertNotEquals(tpeInfo1, tpeInfo2);
-       }
+       public static class TestClass {}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-core/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeInfoTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeInfoTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeInfoTest.java
index dbe5115..0af4ed5 100644
--- 
a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeInfoTest.java
+++ 
b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeInfoTest.java
@@ -18,75 +18,21 @@
 
 package org.apache.flink.api.java.typeutils;
 
-import static org.junit.Assert.*;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.util.InstantiationUtil;
-import org.junit.Test;
-
-import java.io.IOException;
-
-public class PojoTypeInfoTest {
-
-       @Test
-       public void testPojoTypeInfoEquality() {
-               try {
-                       TypeInformation<TestPojo> info1 = 
TypeExtractor.getForClass(TestPojo.class);
-                       TypeInformation<TestPojo> info2 = 
TypeExtractor.getForClass(TestPojo.class);
-                       
-                       assertTrue(info1 instanceof PojoTypeInfo);
-                       assertTrue(info2 instanceof PojoTypeInfo);
-                       
-                       assertTrue(info1.equals(info2));
-                       assertTrue(info1.hashCode() == info2.hashCode());
-               }
-               catch (Exception e) {
-                       e.printStackTrace();
-                       fail(e.getMessage());
-               }
-       }
-
-       @Test
-       public void testPojoTypeInfoInequality() {
-               try {
-                       TypeInformation<TestPojo> info1 = 
TypeExtractor.getForClass(TestPojo.class);
-                       TypeInformation<AlternatePojo> info2 = 
TypeExtractor.getForClass(AlternatePojo.class);
-
-                       assertTrue(info1 instanceof PojoTypeInfo);
-                       assertTrue(info2 instanceof PojoTypeInfo);
-
-                       assertFalse(info1.equals(info2));
-               }
-               catch (Exception e) {
-                       e.printStackTrace();
-                       fail(e.getMessage());
-               }
-       }
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
 
-       @Test
-       public void testSerializabilityOfPojoTypeInfo() throws IOException, 
ClassNotFoundException {
-               PojoTypeInfo<TestPojo> pojoTypeInfo = 
(PojoTypeInfo<TestPojo>)TypeExtractor.getForClass(TestPojo.class);
-
-               byte[] serializedPojoTypeInfo = 
InstantiationUtil.serializeObject(pojoTypeInfo);
-               PojoTypeInfo<TestPojo> deserializedPojoTypeInfo = 
(PojoTypeInfo<TestPojo>)InstantiationUtil.deserializeObject(
-                       serializedPojoTypeInfo,
-                       getClass().getClassLoader());
-
-               assertEquals(pojoTypeInfo, deserializedPojoTypeInfo);
-       }
-
-       @Test
-       public void testPrimitivePojo() {
-               TypeInformation<PrimitivePojo> info1 = 
TypeExtractor.getForClass(PrimitivePojo.class);
-
-               assertTrue(info1 instanceof PojoTypeInfo);
-       }
-
-       @Test
-       public void testUnderscorePojo() {
-               TypeInformation<UnderscorePojo> info1 = 
TypeExtractor.getForClass(UnderscorePojo.class);
-
-               assertTrue(info1 instanceof PojoTypeInfo);
+/**
+ * Test for {@link PojoTypeInfo}.
+ */
+public class PojoTypeInfoTest extends TypeInformationTestBase<PojoTypeInfo<?>>{
+
+       @Override
+       protected PojoTypeInfo<?>[] getTestData() {
+               return new PojoTypeInfo<?>[] {
+                       (PojoTypeInfo<?>) 
TypeExtractor.getForClass(TestPojo.class),
+                       (PojoTypeInfo<?>) 
TypeExtractor.getForClass(AlternatePojo.class),
+                       (PojoTypeInfo<?>) 
TypeExtractor.getForClass(PrimitivePojo.class),
+                       (PojoTypeInfo<?>) 
TypeExtractor.getForClass(UnderscorePojo.class)
+               };
        }
 
        public static final class TestPojo {

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-core/src/test/java/org/apache/flink/api/java/typeutils/RowTypeInfoTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/RowTypeInfoTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/RowTypeInfoTest.java
index 45f616c..03d1e04 100644
--- 
a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/RowTypeInfoTest.java
+++ 
b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/RowTypeInfoTest.java
@@ -20,6 +20,7 @@ package org.apache.flink.api.java.typeutils;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.CompositeType.FlatFieldDescriptor;
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
 import org.junit.Test;
 
 import java.util.ArrayList;
@@ -29,7 +30,10 @@ import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 
-public class RowTypeInfoTest {
+/**
+ * Test for {@link RowTypeInfo}.
+ */
+public class RowTypeInfoTest extends TypeInformationTestBase<RowTypeInfo> {
        private static TypeInformation<?>[] typeList = new TypeInformation<?>[]{
                BasicTypeInfo.INT_TYPE_INFO,
                new RowTypeInfo(
@@ -38,6 +42,15 @@ public class RowTypeInfoTest {
                BasicTypeInfo.STRING_TYPE_INFO
        };
 
+       @Override
+       protected RowTypeInfo[] getTestData() {
+               return new RowTypeInfo[] {
+                       new RowTypeInfo(BasicTypeInfo.INT_TYPE_INFO, 
BasicTypeInfo.STRING_TYPE_INFO),
+                       new RowTypeInfo(BasicTypeInfo.INT_TYPE_INFO, 
BasicTypeInfo.BOOLEAN_TYPE_INFO),
+                       new RowTypeInfo(typeList)
+               };
+       }
+
        @Test(expected = IllegalArgumentException.class)
        public void testWrongNumberOfFieldNames() {
                new RowTypeInfo(typeList, new String[]{"int", "string"});
@@ -92,13 +105,7 @@ public class RowTypeInfoTest {
 
        @Test
        public void testGetTypeAt() {
-               RowTypeInfo typeInfo = new RowTypeInfo(
-                       BasicTypeInfo.INT_TYPE_INFO,
-                       new RowTypeInfo(
-                               BasicTypeInfo.SHORT_TYPE_INFO,
-                               BasicTypeInfo.BIG_DEC_TYPE_INFO
-                       ),
-                       BasicTypeInfo.STRING_TYPE_INFO);
+               RowTypeInfo typeInfo = new RowTypeInfo(typeList);
 
 
                assertArrayEquals(new String[]{"f0", "f1", "f2"}, 
typeInfo.getFieldNames());
@@ -109,44 +116,11 @@ public class RowTypeInfoTest {
        }
 
        @Test
-       public void testRowTypeInfoEquality() {
-               RowTypeInfo typeInfo1 = new RowTypeInfo(
-                       BasicTypeInfo.INT_TYPE_INFO,
-                       BasicTypeInfo.STRING_TYPE_INFO);
-
-               RowTypeInfo typeInfo2 = new RowTypeInfo(
-                       BasicTypeInfo.INT_TYPE_INFO,
-                       BasicTypeInfo.STRING_TYPE_INFO);
-
-               assertEquals(typeInfo1, typeInfo2);
-               assertEquals(typeInfo1.hashCode(), typeInfo2.hashCode());
-       }
-
-       @Test
-       public void testRowTypeInfoInequality() {
-               RowTypeInfo typeInfo1 = new RowTypeInfo(
-                       BasicTypeInfo.INT_TYPE_INFO,
-                       BasicTypeInfo.STRING_TYPE_INFO);
-
-               RowTypeInfo typeInfo2 = new RowTypeInfo(
-                       BasicTypeInfo.INT_TYPE_INFO,
-                       BasicTypeInfo.BOOLEAN_TYPE_INFO);
-
-               assertNotEquals(typeInfo1, typeInfo2);
-               assertNotEquals(typeInfo1.hashCode(), typeInfo2.hashCode());
-       }
-
-       @Test
        public void testNestedRowTypeInfo() {
-               RowTypeInfo typeInfo = new RowTypeInfo(
-                       BasicTypeInfo.INT_TYPE_INFO,
-                       new RowTypeInfo(
-                               BasicTypeInfo.SHORT_TYPE_INFO,
-                               BasicTypeInfo.BIG_DEC_TYPE_INFO
-                       ),
-                       BasicTypeInfo.STRING_TYPE_INFO);
+               RowTypeInfo typeInfo = new RowTypeInfo(typeList);
 
                assertEquals("Row(f0: Short, f1: BigDecimal)", 
typeInfo.getTypeAt("f1").toString());
                assertEquals("Short", typeInfo.getTypeAt("f1.f0").toString());
        }
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TupleTypeInfoTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TupleTypeInfoTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TupleTypeInfoTest.java
index b6cff34..3a98877 100644
--- 
a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TupleTypeInfoTest.java
+++ 
b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TupleTypeInfoTest.java
@@ -21,21 +21,31 @@ package org.apache.flink.api.java.typeutils;
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple1;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.util.TestLogger;
-import org.junit.Assert;
+import static org.junit.Assert.assertTrue;
 import org.junit.Test;
 
-public class TupleTypeInfoTest extends TestLogger {
+/**
+ * Test for {@link TupleTypeInfo}.
+ */
+public class TupleTypeInfoTest extends 
TypeInformationTestBase<TupleTypeInfo<?>> {
+
+       @Override
+       protected TupleTypeInfo<?>[] getTestData() {
+               return new TupleTypeInfo<?>[] {
+                       new TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, 
BasicTypeInfo.STRING_TYPE_INFO),
+                       new TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO, 
BasicTypeInfo.BOOLEAN_TYPE_INFO)
+               };
+       }
 
        @Test
        public void testTupleTypeInfoSymmetricEqualityRelation() {
                TupleTypeInfo<Tuple1<Integer>> tupleTypeInfo = new 
TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO);
 
                TupleTypeInfoBase<Tuple1> anonymousTupleTypeInfo = new 
TupleTypeInfoBase<Tuple1>(
-                       (Class<Tuple1>)Tuple1.class,
+                       Tuple1.class,
                        (TypeInformation<?>)BasicTypeInfo.INT_TYPE_INFO) {
 
                        private static final long serialVersionUID = 
-7985593598027660836L;
@@ -64,33 +74,6 @@ public class TupleTypeInfoTest extends TestLogger {
                boolean tupleVsAnonymous = 
tupleTypeInfo.equals(anonymousTupleTypeInfo);
                boolean anonymousVsTuple = 
anonymousTupleTypeInfo.equals(tupleTypeInfo);
 
-               Assert.assertTrue("Equality relation should be symmetric", 
tupleVsAnonymous == anonymousVsTuple);
-       }
-
-       @Test
-       public void testTupleTypeInfoEquality() {
-               TupleTypeInfo<Tuple2<Integer, String>> tupleTypeInfo1 = new 
TupleTypeInfo<>(
-                       BasicTypeInfo.INT_TYPE_INFO,
-                       BasicTypeInfo.STRING_TYPE_INFO);
-
-               TupleTypeInfo<Tuple2<Integer, String>> tupleTypeInfo2 = new 
TupleTypeInfo<>(
-                       BasicTypeInfo.INT_TYPE_INFO,
-                       BasicTypeInfo.STRING_TYPE_INFO);
-
-               Assert.assertEquals(tupleTypeInfo1, tupleTypeInfo2);
-               Assert.assertEquals(tupleTypeInfo1.hashCode(), 
tupleTypeInfo2.hashCode());
-       }
-
-       @Test
-       public void testTupleTypeInfoInequality() {
-               TupleTypeInfo<Tuple2<Integer, String>> tupleTypeInfo1 = new 
TupleTypeInfo<>(
-                       BasicTypeInfo.INT_TYPE_INFO,
-                       BasicTypeInfo.STRING_TYPE_INFO);
-
-               TupleTypeInfo<Tuple2<Integer, Boolean>> tupleTypeInfo2 = new 
TupleTypeInfo<>(
-                       BasicTypeInfo.INT_TYPE_INFO,
-                       BasicTypeInfo.BOOLEAN_TYPE_INFO);
-
-               Assert.assertNotEquals(tupleTypeInfo1, tupleTypeInfo2);
+               assertTrue("Equality relation should be symmetric", 
tupleVsAnonymous == anonymousVsTuple);
        }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-core/src/test/java/org/apache/flink/api/java/typeutils/ValueTypeInfoTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/ValueTypeInfoTest.java
 
b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/ValueTypeInfoTest.java
index 4a579c8..b67d754 100644
--- 
a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/ValueTypeInfoTest.java
+++ 
b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/ValueTypeInfoTest.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.api.java.typeutils;
 
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.types.Record;
@@ -30,7 +31,26 @@ import java.io.IOException;
 
 import static org.junit.Assert.*;
 
-public class ValueTypeInfoTest extends TestLogger {
+/**
+ * Test for {@link ListTypeInfo}.
+ */
+public class ValueTypeInfoTest extends 
TypeInformationTestBase<ValueTypeInfo<?>> {
+
+       @Override
+       protected ValueTypeInfo<?>[] getTestData() {
+               return new ValueTypeInfo<?>[] {
+                       new ValueTypeInfo<>(TestClass.class),
+                       new ValueTypeInfo<>(AlternativeClass.class),
+                       new ValueTypeInfo<>(Record.class),
+               };
+       }
+
+       @Test
+       public void testValueTypeEqualsWithNull() throws Exception {
+               ValueTypeInfo<Record> tpeInfo = new 
ValueTypeInfo<>(Record.class);
+
+               Assert.assertFalse(tpeInfo.equals(null));
+       }
 
        public static class TestClass implements Value {
                private static final long serialVersionUID = 
-492760806806568285L;
@@ -60,28 +80,4 @@ public class ValueTypeInfoTest extends TestLogger {
 
                }
        }
-
-       @Test
-       public void testValueTypeInfoEquality() {
-               ValueTypeInfo<TestClass> tpeInfo1 = new 
ValueTypeInfo<>(TestClass.class);
-               ValueTypeInfo<TestClass> tpeInfo2 = new 
ValueTypeInfo<>(TestClass.class);
-
-               assertEquals(tpeInfo1, tpeInfo2);
-               assertEquals(tpeInfo1.hashCode(), tpeInfo2.hashCode());
-       }
-
-       @Test
-       public void testValueTyepInfoInequality() {
-               ValueTypeInfo<TestClass> tpeInfo1 = new 
ValueTypeInfo<>(TestClass.class);
-               ValueTypeInfo<AlternativeClass> tpeInfo2 = new 
ValueTypeInfo<>(AlternativeClass.class);
-
-               assertNotEquals(tpeInfo1, tpeInfo2);
-       }
-
-       @Test
-       public void testValueTypeEqualsWithNull() throws Exception {
-               ValueTypeInfo<Record> tpeInfo = new 
ValueTypeInfo<>(Record.class);
-
-               Assert.assertFalse(tpeInfo.equals(null));
-       }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-core/src/test/java/org/apache/flink/types/BasicArrayTypeInfoTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/types/BasicArrayTypeInfoTest.java 
b/flink-core/src/test/java/org/apache/flink/types/BasicArrayTypeInfoTest.java
deleted file mode 100644
index 3e086ff..0000000
--- 
a/flink-core/src/test/java/org/apache/flink/types/BasicArrayTypeInfoTest.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.types;
-
-import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
-import org.apache.flink.util.TestLogger;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-
-public class BasicArrayTypeInfoTest extends TestLogger {
-
-       static Class<?>[] classes = {String[].class, Integer[].class, 
Boolean[].class, Byte[].class,
-               Short[].class, Long[].class, Float[].class, Double[].class, 
Character[].class};
-
-       @Test
-       public void testBasicArrayTypeInfoEquality() {
-               for (Class<?> clazz: classes) {
-                       BasicArrayTypeInfo<?, ?> tpeInfo1 = 
BasicArrayTypeInfo.getInfoFor(clazz);
-                       BasicArrayTypeInfo<?, ?> tpeInfo2 = 
BasicArrayTypeInfo.getInfoFor(clazz);
-
-                       assertEquals(tpeInfo1, tpeInfo2);
-                       assertEquals(tpeInfo1.hashCode(), tpeInfo2.hashCode());
-               }
-       }
-
-       @Test
-       public void testBasicArrayTypeInfoInequality() {
-               for (Class<?> clazz1: classes) {
-                       for (Class<?> clazz2: classes) {
-                               if (!clazz1.equals(clazz2)) {
-                                       BasicArrayTypeInfo<?, ?> tpeInfo1 = 
BasicArrayTypeInfo.getInfoFor(clazz1);
-                                       BasicArrayTypeInfo<?, ?> tpeInfo2 = 
BasicArrayTypeInfo.getInfoFor(clazz2);
-                                       assertNotEquals(tpeInfo1, tpeInfo2);
-                               }
-                       }
-               }
-       }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-core/src/test/java/org/apache/flink/types/NothingTypeInfoTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/types/NothingTypeInfoTest.java 
b/flink-core/src/test/java/org/apache/flink/types/NothingTypeInfoTest.java
deleted file mode 100644
index a7976ee..0000000
--- a/flink-core/src/test/java/org/apache/flink/types/NothingTypeInfoTest.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.types;
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.NothingTypeInfo;
-import org.apache.flink.util.TestLogger;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-public class NothingTypeInfoTest extends TestLogger {
-
-       @Test
-       public void testNothingTypeInfoEquality() {
-               NothingTypeInfo tpeInfo1 = new NothingTypeInfo();
-               NothingTypeInfo tpeInfo2 = new NothingTypeInfo();
-
-               assertEquals(tpeInfo1, tpeInfo2);
-               assertEquals(tpeInfo1.hashCode(), tpeInfo2.hashCode());
-       }
-
-       @Test
-       public void testNothingTypeInfoInequality() {
-               NothingTypeInfo tpeInfo1 = new NothingTypeInfo();
-               BasicTypeInfo<Integer> tpeInfo2 = 
BasicTypeInfo.getInfoFor(Integer.class);
-
-               assertNotEquals(tpeInfo1, tpeInfo2);
-               assertNotEquals(tpeInfo2, tpeInfo1);
-       }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-core/src/test/java/org/apache/flink/types/PrimitiveArrayTypeInfoTest.java
----------------------------------------------------------------------
diff --git 
a/flink-core/src/test/java/org/apache/flink/types/PrimitiveArrayTypeInfoTest.java
 
b/flink-core/src/test/java/org/apache/flink/types/PrimitiveArrayTypeInfoTest.java
deleted file mode 100644
index 7ef14f9..0000000
--- 
a/flink-core/src/test/java/org/apache/flink/types/PrimitiveArrayTypeInfoTest.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.types;
-
-import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
-import org.apache.flink.util.TestLogger;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-
-public class PrimitiveArrayTypeInfoTest extends TestLogger {
-
-       static Class<?>[] classes = {int[].class, boolean[].class, byte[].class,
-               short[].class, long[].class, float[].class, double[].class, 
char[].class};
-
-       @Test
-       public void testPrimitiveArrayTypeInfoEquality() {
-               for (Class<?> clazz: classes) {
-                       PrimitiveArrayTypeInfo<?> tpeInfo1 = 
PrimitiveArrayTypeInfo.getInfoFor(clazz);
-                       PrimitiveArrayTypeInfo<?> tpeInfo2 = 
PrimitiveArrayTypeInfo.getInfoFor(clazz);
-
-                       assertEquals(tpeInfo1, tpeInfo2);
-                       assertEquals(tpeInfo1.hashCode(), tpeInfo2.hashCode());
-               }
-       }
-
-       @Test
-       public void testBasicArrayTypeInfoInequality() {
-               for (Class<?> clazz1: classes) {
-                       for (Class<?> clazz2: classes) {
-                               if (!clazz1.equals(clazz2)) {
-                                       PrimitiveArrayTypeInfo<?> tpeInfo1 = 
PrimitiveArrayTypeInfo.getInfoFor(clazz1);
-                                       PrimitiveArrayTypeInfo<?> tpeInfo2 = 
PrimitiveArrayTypeInfo.getInfoFor(clazz2);
-                                       assertNotEquals(tpeInfo1, tpeInfo2);
-                               }
-                       }
-               }
-       }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-scala/pom.xml
----------------------------------------------------------------------
diff --git a/flink-scala/pom.xml b/flink-scala/pom.xml
index cb9b538..7be1e29 100644
--- a/flink-scala/pom.xml
+++ b/flink-scala/pom.xml
@@ -220,6 +220,19 @@ under the License.
                                </configuration>
                        </plugin>
 
+                       <!-- Add Scala test classes to test jar in order to 
test Scala type information. -->
+                       <plugin>
+                               <groupId>org.apache.maven.plugins</groupId>
+                               <artifactId>maven-jar-plugin</artifactId>
+                               <executions>
+                                       <execution>
+                                               <goals>
+                                                       <goal>test-jar</goal>
+                                               </goals>
+                                       </execution>
+                               </executions>
+                       </plugin>
+
                </plugins>
        </build>
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/CaseClassTypeInfoTest.scala
----------------------------------------------------------------------
diff --git 
a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/CaseClassTypeInfoTest.scala
 
b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/CaseClassTypeInfoTest.scala
index 479483f..088cc70 100644
--- 
a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/CaseClassTypeInfoTest.scala
+++ 
b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/CaseClassTypeInfoTest.scala
@@ -20,54 +20,27 @@ package org.apache.flink.api.scala.typeutils
 
 import org.apache.flink.api.common.ExecutionConfig
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo
-import org.apache.flink.api.common.typeutils.TypeSerializer
-import org.apache.flink.util.TestLogger
-import org.junit.Test
-import org.scalatest.junit.JUnitSuiteLike
+import org.apache.flink.api.common.typeutils.{TypeInformationTestBase, 
TypeSerializer}
 
-class CaseClassTypeInfoTest extends TestLogger with JUnitSuiteLike {
+/**
+  * Test for [[CaseClassTypeInfo]].
+  */
+class CaseClassTypeInfoTest extends 
TypeInformationTestBase[CaseClassTypeInfo[_]] {
 
-  @Test
-  def testCaseClassTypeInfoEquality(): Unit = {
-    val tpeInfo1 = new CaseClassTypeInfo[Tuple2[Int, String]](
-      classOf[Tuple2[Int, String]],
+  override protected def getTestData: Array[CaseClassTypeInfo[_]] = Array(
+    new CaseClassTypeInfo[(Int, String)](
+      classOf[(Int, String)],
       Array(),
       Array(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO),
       Array("_1", "_2")) {
       override def createSerializer(config: ExecutionConfig): 
TypeSerializer[(Int, String)] = ???
-    }
-
-    val tpeInfo2 = new CaseClassTypeInfo[Tuple2[Int, String]](
-      classOf[Tuple2[Int, String]],
-      Array(),
-      Array(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO),
-      Array("_1", "_2")) {
-      override def createSerializer(config: ExecutionConfig): 
TypeSerializer[(Int, String)] = ???
-    }
-
-    assert(tpeInfo1.equals(tpeInfo2))
-    assert(tpeInfo1.hashCode() == tpeInfo2.hashCode())
-  }
-
-  @Test
-  def testCaseClassTypeInfoInequality(): Unit = {
-    val tpeInfo1 = new CaseClassTypeInfo[Tuple2[Int, String]](
-      classOf[Tuple2[Int, String]],
-      Array(),
-      Array(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO),
-      Array("_1", "_2")) {
-      override def createSerializer(config: ExecutionConfig): 
TypeSerializer[(Int, String)] = ???
-    }
-
-    val tpeInfo2 = new CaseClassTypeInfo[Tuple2[Int, Boolean]](
-      classOf[Tuple2[Int, Boolean]],
+    },
+    new CaseClassTypeInfo[(Int, Boolean)](
+      classOf[(Int, Boolean)],
       Array(),
       Array(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.BOOLEAN_TYPE_INFO),
       Array("_1", "_2")) {
       override def createSerializer(config: ExecutionConfig): 
TypeSerializer[(Int, Boolean)] = ???
     }
-
-    assert(!tpeInfo1.equals(tpeInfo2))
-  }
-
+  )
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EitherTypeInfoTest.scala
----------------------------------------------------------------------
diff --git 
a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EitherTypeInfoTest.scala
 
b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EitherTypeInfoTest.scala
index e23a6a0..fc9811e 100644
--- 
a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EitherTypeInfoTest.scala
+++ 
b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EitherTypeInfoTest.scala
@@ -19,41 +19,24 @@
 package org.apache.flink.api.scala.typeutils
 
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo
-import org.apache.flink.util.TestLogger
-import org.junit.Test
-import org.scalatest.junit.JUnitSuiteLike
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase
 
-class EitherTypeInfoTest extends TestLogger with JUnitSuiteLike {
+/**
+  * Test for [[EitherTypeInfo]].
+  */
+class EitherTypeInfoTest extends TypeInformationTestBase[EitherTypeInfo[_, _, 
_]] {
 
-  @Test
-  def testEitherTypeEquality(): Unit = {
-    val eitherTypeInfo1 = new EitherTypeInfo[Integer, String, Either[Integer, 
String]](
-      classOf[Either[Integer,String]],
-      BasicTypeInfo.INT_TYPE_INFO,
-      BasicTypeInfo.STRING_TYPE_INFO
-    )
-    val eitherTypeInfo2 = new EitherTypeInfo[Integer, String, Either[Integer, 
String]](
-      classOf[Either[Integer,String]],
-      BasicTypeInfo.INT_TYPE_INFO,
-      BasicTypeInfo.STRING_TYPE_INFO
-    )
-
-    assert(eitherTypeInfo1.equals(eitherTypeInfo2))
-    assert(eitherTypeInfo1.hashCode() == eitherTypeInfo2.hashCode())
-  }
-
-  @Test
-  def testEitherTypeInequality(): Unit = {
-    val eitherTypeInfo1 = new EitherTypeInfo[Integer, Integer, Either[Integer, 
Integer]](
+  override protected def getTestData: Array[EitherTypeInfo[_, _, _]] = Array(
+    new EitherTypeInfo[Integer, Integer, Either[Integer, Integer]](
       classOf[Either[Integer,Integer]],
       BasicTypeInfo.INT_TYPE_INFO,
       BasicTypeInfo.INT_TYPE_INFO
-    )
-    val eitherTypeInfo2 = new EitherTypeInfo[Integer, String, Either[Integer, 
String]](
+    ),
+    new EitherTypeInfo[Integer, String, Either[Integer, String]](
       classOf[Either[Integer,String]],
       BasicTypeInfo.INT_TYPE_INFO,
       BasicTypeInfo.STRING_TYPE_INFO
     )
-    assert(!eitherTypeInfo1.equals(eitherTypeInfo2))
-  }
+  )
+
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueTypeInfoTest.scala
----------------------------------------------------------------------
diff --git 
a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueTypeInfoTest.scala
 
b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueTypeInfoTest.scala
index acd6a39..14ca0e8 100644
--- 
a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueTypeInfoTest.scala
+++ 
b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueTypeInfoTest.scala
@@ -18,39 +18,23 @@
 
 package org.apache.flink.api.scala.typeutils
 
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase
 import 
org.apache.flink.api.scala.typeutils.AlternateEnumeration.AlternateEnumeration
 import org.apache.flink.api.scala.typeutils.TestEnumeration.TestEnumeration
-import org.apache.flink.util.TestLogger
-import org.junit.Test
-import org.scalatest.junit.JUnitSuiteLike
 
-class EnumValueTypeInfoTest extends TestLogger with JUnitSuiteLike {
+/**
+  * Test for [[EnumValueTypeInfo]].
+  */
+class EnumValueTypeInfoTest extends 
TypeInformationTestBase[EnumValueTypeInfo[_]] {
 
-  @Test
-  def testEnumValueTypeInfoEquality(): Unit = {
-    val enumTypeInfo1 = new EnumValueTypeInfo[TestEnumeration.type](
+  override protected def getTestData: Array[EnumValueTypeInfo[_]] = Array(
+    new EnumValueTypeInfo[TestEnumeration.type](
       TestEnumeration,
-      classOf[TestEnumeration])
-    val enumTypeInfo2 = new EnumValueTypeInfo[TestEnumeration.type](
-      TestEnumeration,
-      classOf[TestEnumeration])
-
-    assert(enumTypeInfo1.equals(enumTypeInfo2))
-    assert(enumTypeInfo1.hashCode() == enumTypeInfo2.hashCode())
-  }
-
-  @Test
-  def testEnumValueTypeInfoInequality(): Unit = {
-    val enumTypeInfo1 = new EnumValueTypeInfo[TestEnumeration.type](
-      TestEnumeration,
-      classOf[TestEnumeration])
-    val enumTypeInfo2 = new EnumValueTypeInfo[AlternateEnumeration.type](
+      classOf[TestEnumeration]),
+    new EnumValueTypeInfo[AlternateEnumeration.type](
       AlternateEnumeration,
       classOf[AlternateEnumeration])
-
-    assert(!enumTypeInfo1.equals(enumTypeInfo2))
-  }
-
+  )
 }
 
 object TestEnumeration extends Enumeration {

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/OptionTypeInfoTest.scala
----------------------------------------------------------------------
diff --git 
a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/OptionTypeInfoTest.scala
 
b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/OptionTypeInfoTest.scala
index b765658..3effb0d 100644
--- 
a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/OptionTypeInfoTest.scala
+++ 
b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/OptionTypeInfoTest.scala
@@ -19,36 +19,16 @@
 package org.apache.flink.api.scala.typeutils
 
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo
-import org.apache.flink.api.java.typeutils.GenericTypeInfo
-import org.apache.flink.util.TestLogger
-import org.junit.Test
-import org.scalatest.junit.{JUnitSuiteLike, JUnitSuite}
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase
 
-class OptionTypeInfoTest extends TestLogger with JUnitSuiteLike {
+/**
+  * Test for [[OptionTypeInfo]].
+  */
+class OptionTypeInfoTest extends TypeInformationTestBase[OptionTypeInfo[_, _]] 
{
 
-  @Test
-  def testOptionTypeEquality: Unit = {
-    val optionTypeInfo1 = new OptionTypeInfo[Integer, 
Option[Integer]](BasicTypeInfo.INT_TYPE_INFO)
-    val optionTypeInfo2 = new OptionTypeInfo[Integer, 
Option[Integer]](BasicTypeInfo.INT_TYPE_INFO)
-
-    assert(optionTypeInfo1.equals(optionTypeInfo2))
-    assert(optionTypeInfo1.hashCode == optionTypeInfo2.hashCode)
-  }
-
-  @Test
-  def testOptionTypeInequality: Unit = {
-    val optionTypeInfo1 = new OptionTypeInfo[Integer, 
Option[Integer]](BasicTypeInfo.INT_TYPE_INFO)
-    val optionTypeInfo2 = new OptionTypeInfo[String, 
Option[String]](BasicTypeInfo.STRING_TYPE_INFO)
-
-    assert(!optionTypeInfo1.equals(optionTypeInfo2))
-  }
-
-  @Test
-  def testOptionTypeInequalityWithDifferentType: Unit = {
-    val optionTypeInfo = new OptionTypeInfo[Integer, 
Option[Integer]](BasicTypeInfo.INT_TYPE_INFO)
-    val genericTypeInfo = new 
GenericTypeInfo[Double](Double.getClass.asInstanceOf[Class[Double]])
-
-    assert(!optionTypeInfo.equals(genericTypeInfo))
-  }
+  override protected def getTestData: Array[OptionTypeInfo[_, _]] = Array(
+    new OptionTypeInfo[Integer, Option[Integer]](BasicTypeInfo.INT_TYPE_INFO),
+    new OptionTypeInfo[String, Option[String]](BasicTypeInfo.STRING_TYPE_INFO)
+  )
 }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/ScalaNothingTypeInfoTest.scala
----------------------------------------------------------------------
diff --git 
a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/ScalaNothingTypeInfoTest.scala
 
b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/ScalaNothingTypeInfoTest.scala
new file mode 100644
index 0000000..0193de9
--- /dev/null
+++ 
b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/ScalaNothingTypeInfoTest.scala
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.scala.typeutils
+
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase
+
+/**
+  * Test for [[ScalaNothingTypeInfo]].
+  */
+class ScalaNothingTypeInfoTest extends 
TypeInformationTestBase[ScalaNothingTypeInfo] {
+
+  override protected def getTestData: Array[ScalaNothingTypeInfo] = Array(
+    new ScalaNothingTypeInfo
+  )
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/TraversableTypeInfoTest.scala
----------------------------------------------------------------------
diff --git 
a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/TraversableTypeInfoTest.scala
 
b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/TraversableTypeInfoTest.scala
index e83b326..f337de1 100644
--- 
a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/TraversableTypeInfoTest.scala
+++ 
b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/TraversableTypeInfoTest.scala
@@ -19,50 +19,26 @@
 package org.apache.flink.api.scala.typeutils
 
 import org.apache.flink.api.common.ExecutionConfig
-import org.apache.flink.api.common.typeinfo.{TypeInformation, BasicTypeInfo}
-import org.apache.flink.api.common.typeutils.TypeSerializer
-import org.apache.flink.util.TestLogger
-import org.junit.Test
-import org.scalatest.junit.JUnitSuiteLike
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
+import org.apache.flink.api.common.typeutils.{TypeInformationTestBase, 
TypeSerializer}
 
-class TraversableTypeInfoTest extends TestLogger with JUnitSuiteLike {
+/**
+  * Test for [[TraversableTypeInfo]].
+  */
+class TraversableTypeInfoTest extends 
TypeInformationTestBase[TraversableTypeInfo[_, _]] {
 
-  @Test
-  def testTraversableTypeInfoEquality(): Unit = {
-    val tpeInfo1 = new TraversableTypeInfo[Seq[Int], Int](
+  override protected def getTestData: Array[TraversableTypeInfo[_, _]] = Array(
+    new TraversableTypeInfo[Seq[Int], Int](
       classOf[Seq[Int]],
       BasicTypeInfo.INT_TYPE_INFO.asInstanceOf[TypeInformation[Int]]) {
       override def createSerializer(executionConfig: ExecutionConfig): 
TypeSerializer[Seq[Int]] =
         ???
-    }
-
-    val tpeInfo2 = new TraversableTypeInfo[Seq[Int], Int](
-      classOf[Seq[Int]],
-      BasicTypeInfo.INT_TYPE_INFO.asInstanceOf[TypeInformation[Int]]) {
-      override def createSerializer(executionConfig: ExecutionConfig): 
TypeSerializer[Seq[Int]] =
-        ???
-    }
-
-    assert(tpeInfo1.equals(tpeInfo2))
-  }
-
-  @Test
-  def testTraversableTypeInfoInequality(): Unit = {
-    val tpeInfo1 = new TraversableTypeInfo[Seq[Int], Int](
-      classOf[Seq[Int]],
-      BasicTypeInfo.INT_TYPE_INFO.asInstanceOf[TypeInformation[Int]]) {
-      override def createSerializer(executionConfig: ExecutionConfig): 
TypeSerializer[Seq[Int]] =
-        ???
-    }
-
-    val tpeInfo2 = new TraversableTypeInfo[List[Int], Int](
+    },
+    new TraversableTypeInfo[List[Int], Int](
       classOf[List[Int]],
       BasicTypeInfo.INT_TYPE_INFO.asInstanceOf[TypeInformation[Int]]) {
       override def createSerializer(executionConfig: ExecutionConfig): 
TypeSerializer[List[Int]] =
         ???
     }
-
-    assert(!tpeInfo1.equals(tpeInfo2))
-  }
-
+  )
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/TryTypeInfoTest.scala
----------------------------------------------------------------------
diff --git 
a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/TryTypeInfoTest.scala
 
b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/TryTypeInfoTest.scala
index 5e68951..05d1821 100644
--- 
a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/TryTypeInfoTest.scala
+++ 
b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/TryTypeInfoTest.scala
@@ -19,39 +19,17 @@
 package org.apache.flink.api.scala.typeutils
 
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo
-import org.apache.flink.api.java.typeutils.GenericTypeInfo
-import org.apache.flink.util.TestLogger
-import org.junit.Test
-import org.scalatest.junit.JUnitSuiteLike
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase
 
 import scala.util.Try
 
-class TryTypeInfoTest extends TestLogger with JUnitSuiteLike {
+/**
+  * Test for [[TryTypeInfo]].
+  */
+class TryTypeInfoTest extends TypeInformationTestBase[TryTypeInfo[_, _]] {
 
-  @Test
-  def testTryTypeEquality(): Unit = {
-    val TryTypeInfo1 = new TryTypeInfo[Integer, 
Try[Integer]](BasicTypeInfo.INT_TYPE_INFO)
-    val TryTypeInfo2 = new TryTypeInfo[Integer, 
Try[Integer]](BasicTypeInfo.INT_TYPE_INFO)
-
-    assert(TryTypeInfo1.equals(TryTypeInfo2))
-    assert(TryTypeInfo1.hashCode == TryTypeInfo2.hashCode)
-  }
-
-  @Test
-  def testTryTypeInequality(): Unit = {
-    val TryTypeInfo1 = new TryTypeInfo[Integer, 
Try[Integer]](BasicTypeInfo.INT_TYPE_INFO)
-    val TryTypeInfo2 = new TryTypeInfo[String, 
Try[String]](BasicTypeInfo.STRING_TYPE_INFO)
-
-    //noinspection ComparingUnrelatedTypes
-    assert(!TryTypeInfo1.equals(TryTypeInfo2))
-  }
-
-  @Test
-  def testTryTypeInequalityWithDifferentType(): Unit = {
-    val TryTypeInfo = new TryTypeInfo[Integer, 
Try[Integer]](BasicTypeInfo.INT_TYPE_INFO)
-    val genericTypeInfo = new 
GenericTypeInfo[Double](Double.getClass.asInstanceOf[Class[Double]])
-
-    //noinspection ComparingUnrelatedTypes
-    assert(!TryTypeInfo.equals(genericTypeInfo))
-  }
+  override protected def getTestData: Array[TryTypeInfo[_, _]] = Array(
+    new TryTypeInfo[Integer, Try[Integer]](BasicTypeInfo.INT_TYPE_INFO),
+    new TryTypeInfo[String, Try[String]](BasicTypeInfo.STRING_TYPE_INFO)
+  )
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/UnitTypeInfoTest.scala
----------------------------------------------------------------------
diff --git 
a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/UnitTypeInfoTest.scala
 
b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/UnitTypeInfoTest.scala
new file mode 100644
index 0000000..1f22c5a
--- /dev/null
+++ 
b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/UnitTypeInfoTest.scala
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.scala.typeutils
+
+import org.apache.flink.api.common.typeutils.TypeInformationTestBase
+
+/**
+  * Test for [[UnitTypeInfo]].
+  */
+class UnitTypeInfoTest extends TypeInformationTestBase[UnitTypeInfo] {
+
+  override protected def getTestData: Array[UnitTypeInfo] = Array(
+    new UnitTypeInfo
+  )
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6cf4a93e/flink-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml
index ebee6ef..b67edbb 100644
--- a/flink-tests/pom.xml
+++ b/flink-tests/pom.xml
@@ -94,11 +94,19 @@ under the License.
                        <version>${project.version}</version>
                        <scope>test</scope>
                </dependency>
-               
+
+               <dependency>
+                       <groupId>org.apache.flink</groupId>
+                       <artifactId>flink-scala_2.10</artifactId>
+                       <version>${project.version}</version>
+                       <scope>test</scope>
+               </dependency>
+
                <dependency>
                        <groupId>org.apache.flink</groupId>
                        <artifactId>flink-scala_2.10</artifactId>
                        <version>${project.version}</version>
+                       <type>test-jar</type>
                        <scope>test</scope>
                </dependency>
 
@@ -140,6 +148,29 @@ under the License.
 
                <dependency>
                        <groupId>org.apache.flink</groupId>
+                       <artifactId>flink-hadoop-compatibility_2.10</artifactId>
+                       <version>${project.version}</version>
+                       <type>test-jar</type>
+                       <scope>test</scope>
+               </dependency>
+
+               <dependency>
+                       <groupId>org.apache.flink</groupId>
+                       <artifactId>flink-avro_2.10</artifactId>
+                       <version>${project.version}</version>
+                       <type>test-jar</type>
+                       <scope>test</scope>
+               </dependency>
+
+               <dependency>
+                       <groupId>org.apache.flink</groupId>
+                       <artifactId>flink-avro_2.10</artifactId>
+                       <version>${project.version}</version>
+                       <scope>test</scope>
+               </dependency>
+
+               <dependency>
+                       <groupId>org.apache.flink</groupId>
                        <artifactId>flink-optimizer_2.10</artifactId>
                        <version>${project.version}</version>
                        <type>test-jar</type>

Reply via email to