You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by mb...@apache.org on 2014/12/17 21:38:07 UTC

[5/5] incubator-flink git commit: [FLINK-1333] Fixed getter/setter recognition for POJOs

[FLINK-1333] Fixed getter/setter recognition for POJOs

This closes #271


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

Branch: refs/heads/master
Commit: 63ef8e86a27e068250240a3b76f932ed07758d23
Parents: a70aa67
Author: Robert Metzger <me...@web.de>
Authored: Tue Dec 16 22:00:50 2014 +0100
Committer: mbalassi <mb...@apache.org>
Committed: Wed Dec 17 20:44:52 2014 +0100

----------------------------------------------------------------------
 .../flink/api/java/typeutils/TypeExtractor.java |  8 +++---
 .../type/extractor/PojoTypeExtractionTest.java  | 29 ++++++++++++++++++++
 2 files changed, 33 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/63ef8e86/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
index e52e2af..b528d00 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
@@ -989,12 +989,12 @@ public class TypeExtractor {
 			}
 			for(Method m : clazz.getMethods()) {
 				// check for getter
-				if(	// The name should be "get<FieldName>" or "<fieldName>" (for scala).
-					(m.getName().toLowerCase().equals("get"+fieldNameLow) || m.getName().toLowerCase().equals(fieldNameLow)) &&
+				if(	// The name should be "get<FieldName>" or "<fieldName>" (for scala) or "is<fieldName>" for boolean fields.
+					(m.getName().toLowerCase().equals("get"+fieldNameLow) || m.getName().toLowerCase().equals("is"+fieldNameLow) || m.getName().toLowerCase().equals(fieldNameLow)) &&
 					// no arguments for the getter
 					m.getParameterTypes().length == 0 &&
 					// return type is same as field type (or the generic variant of it)
-					(m.getReturnType().equals( fieldType ) || (fieldTypeGeneric != null && m.getGenericReturnType().equals(fieldTypeGeneric)) )
+					(m.getGenericReturnType().equals( fieldType ) || (fieldTypeGeneric != null && m.getGenericReturnType().equals(fieldTypeGeneric)) )
 				) {
 					if(hasGetter) {
 						throw new IllegalStateException("Detected more than one getter");
@@ -1004,7 +1004,7 @@ public class TypeExtractor {
 				// check for setters (<FieldName>_$eq for scala)
 				if((m.getName().toLowerCase().equals("set"+fieldNameLow) || m.getName().toLowerCase().equals(fieldNameLow+"_$eq")) &&
 					m.getParameterTypes().length == 1 && // one parameter of the field's type
-					( m.getParameterTypes()[0].equals( fieldType ) || (fieldTypeGeneric != null && m.getGenericParameterTypes()[0].equals(fieldTypeGeneric) ) )&&
+					( m.getGenericParameterTypes()[0].equals( fieldType ) || (fieldTypeGeneric != null && m.getGenericParameterTypes()[0].equals(fieldTypeGeneric) ) )&&
 					// return type is void.
 					m.getReturnType().equals(Void.TYPE)
 				) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/63ef8e86/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/PojoTypeExtractionTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/PojoTypeExtractionTest.java b/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/PojoTypeExtractionTest.java
index 893e63c..7cff856 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/PojoTypeExtractionTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/PojoTypeExtractionTest.java
@@ -19,6 +19,7 @@ package org.apache.flink.api.java.type.extractor;
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Date;
 import java.util.List;
 
@@ -138,6 +139,34 @@ public class PojoTypeExtractionTest {
 		}
 	}
 
+	public static class PojoWithGenericFields {
+		private Collection<String> users;
+		private boolean favorited;
+
+		public boolean isFavorited() {
+			return favorited;
+		}
+
+		public void setFavorited(boolean favorited) {
+			this.favorited = favorited;
+		}
+
+		public Collection<String> getUsers() {
+			return users;
+		}
+
+		public void setUsers(Collection<String> users) {
+			this.users = users;
+		}
+	}
+	@Test
+	public void testPojoWithGenericFields() {
+		TypeInformation<?> typeForClass = TypeExtractor.createTypeInfo(PojoWithGenericFields.class);
+
+		Assert.assertTrue(typeForClass instanceof PojoTypeInfo<?>);
+	}
+
+
 	// in this test, the location of the getters and setters is mixed across the type hierarchy.
 	public static class TypedPojoGetterSetterCheck extends GenericPojoGetterSetterCheck<String> {
 		public void setPackageProtected(String in) {