You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by fh...@apache.org on 2015/10/23 10:51:41 UTC
[1/2] flink git commit: [FLINK-2874] Fix Avro getter/setter
recognition
Repository: flink
Updated Branches:
refs/heads/release-0.9 96099951e -> ab694a3b2
[FLINK-2874] Fix Avro getter/setter recognition
This closes #1252
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/4392cf25
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/4392cf25
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/4392cf25
Branch: refs/heads/release-0.9
Commit: 4392cf25388cdf61ab5c6c12d5ea8c95b62f1db5
Parents: 9609995
Author: Ulf Karlsson <uk...@spotify.com>
Authored: Sun Oct 11 00:04:57 2015 +0200
Committer: Fabian Hueske <fh...@apache.org>
Committed: Fri Oct 23 10:27:51 2015 +0200
----------------------------------------------------------------------
.../flink/api/java/typeutils/TypeExtractor.java | 17 ++--
.../api/java/typeutils/PojoTypeInfoTest.java | 95 +++++++++++++++++++-
2 files changed, 104 insertions(+), 8 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/4392cf25/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 2e45107..e15650f 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
@@ -30,6 +30,7 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.avro.specific.SpecificRecordBase;
+import org.apache.commons.lang3.ClassUtils;
import org.apache.flink.api.common.functions.CoGroupFunction;
import org.apache.flink.api.common.functions.CrossFunction;
import org.apache.flink.api.common.functions.FlatJoinFunction;
@@ -1260,22 +1261,26 @@ public class TypeExtractor {
return true;
} else {
boolean hasGetter = false, hasSetter = false;
- final String fieldNameLow = f.getName().toLowerCase();
-
+ final String fieldNameLow = f.getName().toLowerCase().replaceAll("_", "");
+
Type fieldType = f.getGenericType();
+ Class<?> fieldTypeWrapper = ClassUtils.primitiveToWrapper(f.getType());
+
TypeVariable<?> fieldTypeGeneric = null;
if(fieldType instanceof TypeVariable) {
fieldTypeGeneric = (TypeVariable<?>) fieldType;
fieldType = materializeTypeVariable(typeHierarchy, (TypeVariable<?>)fieldType);
}
for(Method m : clazz.getMethods()) {
+ final String methodNameLow = m.getName().toLowerCase().replaceAll("_", "");
+
// check for getter
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)) &&
+ (methodNameLow.equals("get"+fieldNameLow) || methodNameLow.equals("is"+fieldNameLow) || methodNameLow.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.getGenericReturnType().equals( fieldType ) || (fieldTypeGeneric != null && m.getGenericReturnType().equals(fieldTypeGeneric)) )
+ (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");
@@ -1283,9 +1288,9 @@ public class TypeExtractor {
hasGetter = true;
}
// check for setters (<FieldName>_$eq for scala)
- if((m.getName().toLowerCase().equals("set"+fieldNameLow) || m.getName().toLowerCase().equals(fieldNameLow+"_$eq")) &&
+ if((methodNameLow.equals("set"+fieldNameLow) || methodNameLow.equals(fieldNameLow+"_$eq")) &&
m.getParameterTypes().length == 1 && // one parameter of the field's type
- ( m.getGenericParameterTypes()[0].equals( fieldType ) || (fieldTypeGeneric != null && m.getGenericParameterTypes()[0].equals(fieldTypeGeneric) ) )&&
+ (m.getGenericParameterTypes()[0].equals( fieldType ) || (fieldTypeWrapper != null && m.getParameterTypes()[0].equals( fieldTypeWrapper )) || (fieldTypeGeneric != null && m.getGenericParameterTypes()[0].equals(fieldTypeGeneric) ) )&&
// return type is void.
m.getReturnType().equals(Void.TYPE)
) {
http://git-wip-us.apache.org/repos/asf/flink/blob/4392cf25/flink-java/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeInfoTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeInfoTest.java
index 12b7913..3d3592a 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeInfoTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeInfoTest.java
@@ -21,8 +21,11 @@ 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
@@ -42,11 +45,54 @@ public class PojoTypeInfoTest {
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());
+ }
+ }
+
+ @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);
+ }
+
public static final class TestPojo {
public int someInt;
-
+
private String aString;
public Double[] doubleArray;
@@ -60,4 +106,49 @@ public class PojoTypeInfoTest {
return aString;
}
}
+
+ public static final class AlternatePojo {
+
+ public int someInt;
+
+ private String aString;
+
+ public Double[] doubleArray;
+
+
+ public void setaString(String aString) {
+ this.aString = aString;
+ }
+
+ public String getaString() {
+ return aString;
+ }
+ }
+
+ public static final class PrimitivePojo {
+
+ private int someInt;
+
+ public void setSomeInt(Integer someInt) {
+ this.someInt = someInt;
+ }
+
+ public Integer getSomeInt() {
+ return this.someInt;
+ }
+ }
+
+ public static final class UnderscorePojo {
+
+ private int some_int;
+
+ public void setSomeInt(int some_int) {
+ this.some_int = some_int;
+ }
+
+ public Integer getSomeInt() {
+ return this.some_int;
+ }
+ }
+
}
[2/2] flink git commit: [FLINK-2874] Fix recognition of Scala default
setters
Posted by fh...@apache.org.
[FLINK-2874] Fix recognition of Scala default setters
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/ab694a3b
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/ab694a3b
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/ab694a3b
Branch: refs/heads/release-0.9
Commit: ab694a3b22a88bfc5e911eae0c1e313073b6fd57
Parents: 4392cf2
Author: Fabian Hueske <fh...@apache.org>
Authored: Fri Oct 23 00:06:55 2015 +0200
Committer: Fabian Hueske <fh...@apache.org>
Committed: Fri Oct 23 10:27:59 2015 +0200
----------------------------------------------------------------------
.../java/org/apache/flink/api/java/typeutils/TypeExtractor.java | 4 +++-
1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/ab694a3b/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 e15650f..015f196 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
@@ -1272,7 +1272,9 @@ public class TypeExtractor {
fieldType = materializeTypeVariable(typeHierarchy, (TypeVariable<?>)fieldType);
}
for(Method m : clazz.getMethods()) {
- final String methodNameLow = m.getName().toLowerCase().replaceAll("_", "");
+ final String methodNameLow = m.getName().endsWith("_$eq") ?
+ m.getName().toLowerCase().replaceAll("_", "").replaceFirst("\\$eq$", "_\\$eq") :
+ m.getName().toLowerCase().replaceAll("_", "");
// check for getter
if( // The name should be "get<FieldName>" or "<fieldName>" (for scala) or "is<fieldName>" for boolean fields.