You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by gr...@apache.org on 2017/05/09 18:47:54 UTC

[3/3] flink git commit: [FLINK-5070] [types] Unable to use Scala's BeanProperty with classes

[FLINK-5070] [types] Unable to use Scala's BeanProperty with classes

This closes #3318


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

Branch: refs/heads/master
Commit: c90b6da5ff845e55c52150a3c6f0b7192959a40e
Parents: 6cd98a9
Author: twalthr <tw...@apache.org>
Authored: Wed Feb 15 11:05:38 2017 +0100
Committer: Greg Hogan <co...@greghogan.com>
Committed: Tue May 9 13:12:05 2017 -0400

----------------------------------------------------------------------
 .../flink/api/java/typeutils/TypeExtractor.java | 12 +++------
 .../scala/typeutils/TypeExtractionTest.scala    | 28 ++++++++++++++++----
 2 files changed, 26 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c90b6da5/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
index 2b9eed9..a5f236f 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
@@ -1711,9 +1711,6 @@ public class TypeExtractor {
 					// return type is same as field type (or the generic variant of it)
 					(m.getGenericReturnType().equals( fieldType ) || (fieldTypeWrapper != null && m.getReturnType().equals( fieldTypeWrapper )) || (fieldTypeGeneric != null && m.getGenericReturnType().equals(fieldTypeGeneric)) )
 				) {
-					if(hasGetter) {
-						throw new IllegalStateException("Detected more than one getter");
-					}
 					hasGetter = true;
 				}
 				// check for setters (<FieldName>_$eq for scala)
@@ -1723,9 +1720,6 @@ public class TypeExtractor {
 					// return type is void.
 					m.getReturnType().equals(Void.TYPE)
 				) {
-					if(hasSetter) {
-						throw new IllegalStateException("Detected more than one setter");
-					}
 					hasSetter = true;
 				}
 			}
@@ -1733,10 +1727,10 @@ public class TypeExtractor {
 				return true;
 			} else {
 				if(!hasGetter) {
-					LOG.debug(clazz+" does not contain a getter for field "+f.getName() );
+					LOG.info(clazz+" does not contain a getter for field "+f.getName() );
 				}
 				if(!hasSetter) {
-					LOG.debug(clazz+" does not contain a setter for field "+f.getName() );
+					LOG.info(clazz+" does not contain a setter for field "+f.getName() );
 				}
 				return false;
 			}
@@ -1771,7 +1765,7 @@ public class TypeExtractor {
 		for (Field field : fields) {
 			Type fieldType = field.getGenericType();
 			if(!isValidPojoField(field, clazz, typeHierarchy)) {
-				LOG.info(clazz + " is not a valid POJO type");
+				LOG.info(clazz + " is not a valid POJO type because not all fields are valid POJO fields.");
 				return null;
 			}
 			try {

http://git-wip-us.apache.org/repos/asf/flink/blob/c90b6da5/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/TypeExtractionTest.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/TypeExtractionTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/TypeExtractionTest.scala
index 0462ffa..d5b7867 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/TypeExtractionTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/TypeExtractionTest.scala
@@ -20,22 +20,34 @@ package org.apache.flink.api.scala.typeutils
 
 import org.apache.flink.api.common.io.FileInputFormat
 import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable
+import org.apache.flink.api.java.typeutils.{PojoTypeInfo, ResultTypeQueryable}
 import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.typeutils.TypeExtractionTest.CustomTypeInputFormat
+import org.apache.flink.api.scala.typeutils.TypeExtractionTest.{CustomBeanClass, CustomTypeInputFormat}
 import org.apache.flink.util.TestLogger
-import org.junit.Assert.assertEquals
+import org.junit.Assert._
 import org.junit.Test
 import org.scalatest.junit.JUnitSuiteLike
 
+import scala.beans.BeanProperty
+
 
 class TypeExtractionTest extends TestLogger with JUnitSuiteLike {
 
   @Test
   def testResultTypeQueryable(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
-    val productedType = env.createInput(new CustomTypeInputFormat).getType()
-    assertEquals(productedType, BasicTypeInfo.LONG_TYPE_INFO)
+    val producedType = env.createInput(new CustomTypeInputFormat).getType()
+    assertEquals(producedType, BasicTypeInfo.LONG_TYPE_INFO)
+  }
+
+  @Test
+  def testBeanPropertyClass(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val producedType = env.fromElements(new CustomBeanClass()).getType()
+    assertTrue(producedType.isInstanceOf[PojoTypeInfo[_]])
+    val pojoTypeInfo = producedType.asInstanceOf[PojoTypeInfo[_]]
+    assertEquals(pojoTypeInfo.getTypeAt(0), BasicTypeInfo.INT_TYPE_INFO)
+    assertEquals(pojoTypeInfo.getTypeAt(1), BasicTypeInfo.LONG_TYPE_INFO)
   }
 
 }
@@ -50,4 +62,10 @@ object TypeExtractionTest {
 
     override def nextRecord(reuse: String): String = throw new UnsupportedOperationException()
   }
+
+  class CustomBeanClass(
+      @BeanProperty var prop: Int,
+      var prop2: Long) {
+    def this() = this(0, 0L)
+  }
 }