You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2018/04/22 20:40:38 UTC

[02/17] flink git commit: [FLINK-9197] [core] Improve error messages for TypeInformation and TypeHint with generic variables.

[FLINK-9197] [core] Improve error messages for TypeInformation and TypeHint with generic variables.


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

Branch: refs/heads/master
Commit: 04e13447cfdb080200432cc8ccb65c6714741bf3
Parents: 36767dd
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Apr 17 19:46:08 2018 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sun Apr 22 16:28:34 2018 +0200

----------------------------------------------------------------------
 .../flink/api/common/typeinfo/TypeHint.java     | 11 ++-
 .../api/common/typeinfo/TypeInformation.java    | 13 +++-
 .../flink/api/common/typeinfo/TypeHintTest.java | 20 ++++--
 .../common/typeinfo/TypeInformationTest.java    | 71 ++++++++++++++++++++
 4 files changed, 106 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/04e13447/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeHint.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeHint.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeHint.java
index 7de0cbd..5f0597c 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeHint.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeHint.java
@@ -19,7 +19,9 @@
 package org.apache.flink.api.common.typeinfo;
 
 import org.apache.flink.annotation.Public;
+import org.apache.flink.api.common.functions.InvalidTypesException;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.util.FlinkRuntimeException;
 
 /**
  * A utility class for describing generic types. It can be used to obtain a type information via:
@@ -44,7 +46,14 @@ public abstract class TypeHint<T> {
 	 * Creates a hint for the generic type in the class signature.
 	 */
 	public TypeHint() {
-		this.typeInfo = TypeExtractor.createTypeInfo(this, TypeHint.class, getClass(), 0);
+		try {
+			this.typeInfo = TypeExtractor.createTypeInfo(
+					this, TypeHint.class, getClass(), 0);
+		}
+		catch (InvalidTypesException e) {
+			throw new FlinkRuntimeException("The TypeHint is using a generic variable." +
+					"This is not supported, generic types must be fully specified for the TypeHint.");
+		}
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/04e13447/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInformation.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInformation.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInformation.java
index 25cf03f..c5c077f 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInformation.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInformation.java
@@ -21,9 +21,11 @@ package org.apache.flink.api.common.typeinfo;
 import org.apache.flink.annotation.Public;
 import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.functions.InvalidTypesException;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.util.FlinkRuntimeException;
 
 import java.io.Serializable;
 import java.util.Collections;
@@ -204,7 +206,16 @@ public abstract class TypeInformation<T> implements Serializable {
 	 * @return The TypeInformation object for the type described by the hint.
 	 */
 	public static <T> TypeInformation<T> of(Class<T> typeClass) {
-		return TypeExtractor.createTypeInfo(typeClass);
+		try {
+			return TypeExtractor.createTypeInfo(typeClass);
+		}
+		catch (InvalidTypesException e) {
+			throw new FlinkRuntimeException(
+					"Cannot extract TypeInformation from Class alone, because generic parameters are missing. " +
+					"Please use TypeInformation.of(TypeHint) instead, or another equivalent method in the API that " +
+					"accepts a TypeHint instead of a Class. " +
+					"For example for a Tuple2<Long, String> pass a 'new TypeHint<Tuple2<Long, String>>(){}'.");
+		}
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/04e13447/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/TypeHintTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/TypeHintTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/TypeHintTest.java
index ccbd81a..bd4af90 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/TypeHintTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/TypeHintTest.java
@@ -20,11 +20,15 @@ package org.apache.flink.api.common.typeinfo;
 
 import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.util.FlinkRuntimeException;
 
 import org.junit.Test;
 
+import java.util.List;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Tests for the {@link TypeHint}.
@@ -54,13 +58,15 @@ public class TypeHintTest {
 	}
 
 	@Test
-	public void testTypeInfoOf() {
-		assertEquals(BasicTypeInfo.STRING_TYPE_INFO, TypeInformation.of(String.class));
-		assertEquals(BasicTypeInfo.STRING_TYPE_INFO, TypeInformation.of(new TypeHint<String>(){}));
-
-		TypeInformation<Tuple3<String, Double, Boolean>> tupleInfo =
-				new TupleTypeInfo<>(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO, BasicTypeInfo.BOOLEAN_TYPE_INFO);
+	public <T> void testWithGenericParameter() {
+		try {
+			new TypeHint<T>(){};
+			fail();
+		}
+		catch (FlinkRuntimeException ignored) {}
 
-		assertEquals(tupleInfo, TypeInformation.of(new TypeHint<Tuple3<String, Double, Boolean>>(){}));
+		// this works, because "List" goes to the GenericType (blackbox) which does
+		// not care about generic parametrization
+		new TypeHint<List<T>>(){};
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/04e13447/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/TypeInformationTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/TypeInformationTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/TypeInformationTest.java
new file mode 100644
index 0000000..1488866
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/TypeInformationTest.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.common.typeinfo;
+
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.typeutils.GenericTypeInfo;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests for the {@link TypeInformation} class.
+ */
+public class TypeInformationTest {
+
+	@Test
+	public void testOfClass() {
+		assertEquals(BasicTypeInfo.STRING_TYPE_INFO, TypeInformation.of(String.class));
+	}
+
+	@Test
+	public void testOfGenericClassForFlink() {
+		try {
+			TypeInformation.of(Tuple3.class);
+			fail("should fail with an exception");
+		}
+		catch (FlinkRuntimeException e) {
+			// check that the error message mentions the TypeHint
+			assertNotEquals(-1, e.getMessage().indexOf("TypeHint"));
+		}
+	}
+
+	@Test
+	public void testOfGenericClassForGenericType() {
+		assertEquals(new GenericTypeInfo<>(List.class), TypeInformation.of(List.class));
+	}
+
+	@Test
+	public void testOfTypeHint() {
+		assertEquals(BasicTypeInfo.STRING_TYPE_INFO, TypeInformation.of(String.class));
+		assertEquals(BasicTypeInfo.STRING_TYPE_INFO, TypeInformation.of(new TypeHint<String>(){}));
+
+		TypeInformation<Tuple3<String, Double, Boolean>> tupleInfo =
+				new TupleTypeInfo<>(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO, BasicTypeInfo.BOOLEAN_TYPE_INFO);
+
+		assertEquals(tupleInfo, TypeInformation.of(new TypeHint<Tuple3<String, Double, Boolean>>(){}));
+	}
+}