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/09/17 13:24:04 UTC
[1/9] flink git commit: [FLINK-2595] [tests] Fixed unclosed JarFile
in ClassLoaderUtilsTest
Repository: flink
Updated Branches:
refs/heads/master 76a40d59e -> 77989d3cb
[FLINK-2595] [tests] Fixed unclosed JarFile in ClassLoaderUtilsTest
This closes #1137
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/2c9e2c8b
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/2c9e2c8b
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/2c9e2c8b
Branch: refs/heads/master
Commit: 2c9e2c8bbe1547709d820949d1739f7ea2ce89cf
Parents: 76a40d5
Author: Ted <yu...@gmail.com>
Authored: Wed Sep 16 09:29:54 2015 -0700
Committer: Fabian Hueske <fh...@apache.org>
Committed: Thu Sep 17 10:41:28 2015 +0200
----------------------------------------------------------------------
.../org/apache/flink/runtime/util/ClassLoaderUtilsTest.java | 5 ++++-
1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/2c9e2c8b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ClassLoaderUtilsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ClassLoaderUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ClassLoaderUtilsTest.java
index 654001d..7f6e1bd 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ClassLoaderUtilsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ClassLoaderUtilsTest.java
@@ -46,12 +46,15 @@ public class ClassLoaderUtilsTest {
jarFileCreator.createJarFile();
// validate that the JAR is correct and the test setup is not broken
+ JarFile jarFile = null;
try {
- new JarFile(validJar.getAbsolutePath());
+ jarFile = new JarFile(validJar.getAbsolutePath());
}
catch (Exception e) {
e.printStackTrace();
fail("test setup broken: cannot create a valid jar file");
+ } finally {
+ if (jarFile != null) jarFile.close();
}
// file with some random contents
[7/9] flink git commit: [FLINK-2689] [runtime] Fix reuse of null
object for solution set Joins and CoGroups.
Posted by fh...@apache.org.
[FLINK-2689] [runtime] Fix reuse of null object for solution set Joins and CoGroups.
This closes #1136
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/988a04eb
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/988a04eb
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/988a04eb
Branch: refs/heads/master
Commit: 988a04eb486d286e071f4a68aa22c64a2cd4ed8e
Parents: 93c95b6
Author: Fabian Hueske <fh...@apache.org>
Authored: Wed Sep 16 16:56:06 2015 +0200
Committer: Fabian Hueske <fh...@apache.org>
Committed: Thu Sep 17 11:19:34 2015 +0200
----------------------------------------------------------------------
.../runtime/operators/CoGroupWithSolutionSetFirstDriver.java | 6 +++---
.../runtime/operators/CoGroupWithSolutionSetSecondDriver.java | 6 +++---
.../runtime/operators/JoinWithSolutionSetFirstDriver.java | 4 ++--
.../runtime/operators/JoinWithSolutionSetSecondDriver.java | 4 ++--
4 files changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/988a04eb/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java
index b27b6b9..97d6e51 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetFirstDriver.java
@@ -175,9 +175,9 @@ public class CoGroupWithSolutionSetFirstDriver<IT1, IT2, OT> implements Resettab
while (this.running && probeSideInput.nextKey()) {
IT2 current = probeSideInput.getCurrent();
- buildSideRecord = prober.getMatchFor(current, buildSideRecord);
- if (buildSideRecord != null) {
- siIter.set(buildSideRecord);
+ IT1 matchedRecord = prober.getMatchFor(current, buildSideRecord);
+ if (matchedRecord != null) {
+ siIter.set(matchedRecord);
coGroupStub.coGroup(siIter, probeSideInput.getValues(), collector);
} else {
coGroupStub.coGroup(emptySolutionSide, probeSideInput.getValues(), collector);
http://git-wip-us.apache.org/repos/asf/flink/blob/988a04eb/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java
index ba0f8f9..9e8a81c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CoGroupWithSolutionSetSecondDriver.java
@@ -174,9 +174,9 @@ public class CoGroupWithSolutionSetSecondDriver<IT1, IT2, OT> implements Resetta
while (this.running && probeSideInput.nextKey()) {
IT1 current = probeSideInput.getCurrent();
- buildSideRecord = prober.getMatchFor(current, buildSideRecord);
- if (buildSideRecord != null) {
- siIter.set(buildSideRecord);
+ IT2 matchedRecord = prober.getMatchFor(current, buildSideRecord);
+ if (matchedRecord != null) {
+ siIter.set(matchedRecord);
coGroupStub.coGroup(probeSideInput.getValues(), siIter, collector);
} else {
coGroupStub.coGroup(probeSideInput.getValues(), emptySolutionSide, collector);
http://git-wip-us.apache.org/repos/asf/flink/blob/988a04eb/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetFirstDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetFirstDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetFirstDriver.java
index a1c8a4a..fe926cb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetFirstDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetFirstDriver.java
@@ -166,8 +166,8 @@ public class JoinWithSolutionSetFirstDriver<IT1, IT2, OT> implements ResettableP
IT1 buildSideRecord = this.solutionSideRecord;
while (this.running && ((probeSideRecord = probeSideInput.next(probeSideRecord)) != null)) {
- buildSideRecord = prober.getMatchFor(probeSideRecord, buildSideRecord);
- joinFunction.join(buildSideRecord, probeSideRecord, collector);
+ IT1 matchedRecord = prober.getMatchFor(probeSideRecord, buildSideRecord);
+ joinFunction.join(matchedRecord, probeSideRecord, collector);
}
} else if (objectMap != null) {
final JoinHashMap<IT1> hashTable = this.objectMap;
http://git-wip-us.apache.org/repos/asf/flink/blob/988a04eb/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetSecondDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetSecondDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetSecondDriver.java
index 32a75dc..20079fc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetSecondDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinWithSolutionSetSecondDriver.java
@@ -168,8 +168,8 @@ public class JoinWithSolutionSetSecondDriver<IT1, IT2, OT> implements Resettable
IT2 buildSideRecord = this.solutionSideRecord;
while (this.running && ((probeSideRecord = probeSideInput.next(probeSideRecord)) != null)) {
- buildSideRecord = prober.getMatchFor(probeSideRecord, buildSideRecord);
- joinFunction.join(probeSideRecord, buildSideRecord, collector);
+ IT2 matchedRecord = prober.getMatchFor(probeSideRecord, buildSideRecord);
+ joinFunction.join(probeSideRecord, matchedRecord, collector);
}
} else if (objectMap != null) {
final JoinHashMap<IT2> hashTable = this.objectMap;
[5/9] flink git commit: [FLINK-2637] [api-breaking] [types] Adds
equals and hashCode method to TypeInformations and TypeSerializers - Fixes
ObjectArrayTypeInfo - Makes CompositeTypes serializable - Adds test for
equality relation's symmetric property - A
Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/main/java/org/apache/flink/api/java/typeutils/EnumTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/EnumTypeInfo.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/EnumTypeInfo.java
index 024eb71..de59c36 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/EnumTypeInfo.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/EnumTypeInfo.java
@@ -18,6 +18,7 @@
package org.apache.flink.api.java.typeutils;
+import com.google.common.base.Preconditions;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.typeinfo.AtomicType;
import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -38,12 +39,12 @@ public class EnumTypeInfo<T extends Enum<T>> extends TypeInformation<T> implemen
private final Class<T> typeClass;
public EnumTypeInfo(Class<T> typeClass) {
- if (typeClass == null) {
- throw new NullPointerException();
- }
+ Preconditions.checkNotNull(typeClass, "Enum type class must not be null.");
+
if (!Enum.class.isAssignableFrom(typeClass) ) {
throw new IllegalArgumentException("EnumTypeInfo can only be used for subclasses of " + Enum.class.getName());
}
+
this.typeClass = typeClass;
}
@@ -98,13 +99,22 @@ public class EnumTypeInfo<T extends Enum<T>> extends TypeInformation<T> implemen
@Override
public int hashCode() {
- return typeClass.hashCode() ^ 0xd3a2646c;
+ return typeClass.hashCode();
}
-
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof EnumTypeInfo;
+ }
+
@Override
public boolean equals(Object obj) {
if (obj instanceof EnumTypeInfo) {
- return typeClass == ((EnumTypeInfo<?>) obj).typeClass;
+ @SuppressWarnings("unchecked")
+ EnumTypeInfo<T> enumTypeInfo = (EnumTypeInfo<T>) obj;
+
+ return enumTypeInfo.canEqual(this) &&
+ typeClass == enumTypeInfo.typeClass;
} else {
return false;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/main/java/org/apache/flink/api/java/typeutils/GenericTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/GenericTypeInfo.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/GenericTypeInfo.java
index 5caf8f2..7e7aa68 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/GenericTypeInfo.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/GenericTypeInfo.java
@@ -18,6 +18,7 @@
package org.apache.flink.api.java.typeutils;
+import com.google.common.base.Preconditions;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.typeinfo.AtomicType;
import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -34,7 +35,7 @@ public class GenericTypeInfo<T> extends TypeInformation<T> implements AtomicType
private final Class<T> typeClass;
public GenericTypeInfo(Class<T> typeClass) {
- this.typeClass = typeClass;
+ this.typeClass = Preconditions.checkNotNull(typeClass);
}
@Override
@@ -88,13 +89,21 @@ public class GenericTypeInfo<T> extends TypeInformation<T> implements AtomicType
@Override
public int hashCode() {
- return typeClass.hashCode() ^ 0x165667b1;
+ return typeClass.hashCode();
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof GenericTypeInfo;
}
@Override
public boolean equals(Object obj) {
- if (obj.getClass() == GenericTypeInfo.class) {
- return typeClass == ((GenericTypeInfo<?>) obj).typeClass;
+ if (obj instanceof GenericTypeInfo) {
+ @SuppressWarnings("unchecked")
+ GenericTypeInfo<T> genericTypeInfo = (GenericTypeInfo<T>) obj;
+
+ return typeClass == genericTypeInfo.typeClass;
} else {
return false;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/main/java/org/apache/flink/api/java/typeutils/MissingTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/MissingTypeInfo.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/MissingTypeInfo.java
index 227c68c..1dd7f01 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/MissingTypeInfo.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/MissingTypeInfo.java
@@ -31,8 +31,8 @@ public class MissingTypeInfo extends TypeInformation<InvalidTypesException> {
private static final long serialVersionUID = -4212082837126702723L;
- private String functionName;
- private InvalidTypesException typeException;
+ private final String functionName;
+ private final InvalidTypesException typeException;
public MissingTypeInfo(String functionName) {
@@ -87,6 +87,34 @@ public class MissingTypeInfo extends TypeInformation<InvalidTypesException> {
}
@Override
+ public String toString() {
+ return getClass().getSimpleName() + "<" + functionName + ", " + typeException.getMessage() + ">";
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof MissingTypeInfo) {
+ MissingTypeInfo missingTypeInfo = (MissingTypeInfo) obj;
+
+ return missingTypeInfo.canEqual(this) &&
+ functionName.equals(missingTypeInfo.functionName) &&
+ typeException.equals(missingTypeInfo.typeException);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return 31 * functionName.hashCode() + typeException.hashCode();
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof MissingTypeInfo;
+ }
+
+ @Override
public int getTotalFields() {
throw new UnsupportedOperationException("The missing type information cannot be used as a type information.");
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfo.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfo.java
index 6806122..150c976 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfo.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfo.java
@@ -19,13 +19,9 @@
package org.apache.flink.api.java.typeutils;
import java.lang.reflect.Array;
-import java.lang.reflect.GenericArrayType;
-import java.lang.reflect.ParameterizedType;
-import java.lang.reflect.Type;
+import com.google.common.base.Preconditions;
import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.functions.InvalidTypesException;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.common.typeutils.base.GenericArraySerializer;
@@ -34,21 +30,12 @@ public class ObjectArrayTypeInfo<T, C> extends TypeInformation<T> {
private static final long serialVersionUID = 1L;
- private final Type arrayType;
- private final Type componentType;
+ private final Class<T> arrayType;
private final TypeInformation<C> componentInfo;
- @SuppressWarnings("unchecked")
- private ObjectArrayTypeInfo(Type arrayType, Type componentType) {
- this.arrayType = arrayType;
- this.componentType = componentType;
- this.componentInfo = (TypeInformation<C>) TypeExtractor.createTypeInfo(componentType);
- }
-
- private ObjectArrayTypeInfo(Type arrayType, Type componentType, TypeInformation<C> componentInfo) {
- this.arrayType = arrayType;
- this.componentType = componentType;
- this.componentInfo = componentInfo;
+ private ObjectArrayTypeInfo(Class<T> arrayType, TypeInformation<C> componentInfo) {
+ this.arrayType = Preconditions.checkNotNull(arrayType);
+ this.componentInfo = Preconditions.checkNotNull(componentInfo);
}
// --------------------------------------------------------------------------------------------
@@ -76,29 +63,9 @@ public class ObjectArrayTypeInfo<T, C> extends TypeInformation<T> {
@SuppressWarnings("unchecked")
@Override
public Class<T> getTypeClass() {
- // if arrayType is a Class
- if (arrayType instanceof Class) {
- return (Class<T>) arrayType;
- }
-
- // if arrayType is a GenericArrayType
- Class<?> componentClass = (Class<?>) ((ParameterizedType) componentType).getRawType();
-
- try {
- return (Class<T>) Class.forName("[L" + componentClass.getName() + ";");
- } catch (ClassNotFoundException e) {
- throw new RuntimeException("Cannot create non-generic type class.", e);
- }
- }
-
- public Type getType() {
return arrayType;
}
- public Type getComponentType() {
- return this.componentType;
- }
-
public TypeInformation<C> getComponentInfo() {
return componentInfo;
}
@@ -111,15 +78,9 @@ public class ObjectArrayTypeInfo<T, C> extends TypeInformation<T> {
@SuppressWarnings("unchecked")
@Override
public TypeSerializer<T> createSerializer(ExecutionConfig executionConfig) {
- // use raw type for serializer if generic array type
- if (this.componentType instanceof GenericArrayType) {
- ParameterizedType paramType = (ParameterizedType) ((GenericArrayType) this.componentType).getGenericComponentType();
-
- return (TypeSerializer<T>) new GenericArraySerializer<C>((Class<C>) paramType.getRawType(),
- this.componentInfo.createSerializer(executionConfig));
- } else {
- return (TypeSerializer<T>) new GenericArraySerializer<C>((Class<C>) this.componentType, this.componentInfo.createSerializer(executionConfig));
- }
+ return (TypeSerializer<T>) new GenericArraySerializer<C>(
+ componentInfo.getTypeClass(),
+ componentInfo.createSerializer(executionConfig));
}
@Override
@@ -128,38 +89,37 @@ public class ObjectArrayTypeInfo<T, C> extends TypeInformation<T> {
}
@Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
+ public boolean equals(Object obj) {
+ if (obj instanceof ObjectArrayTypeInfo) {
+ @SuppressWarnings("unchecked")
+ ObjectArrayTypeInfo<T, C> objectArrayTypeInfo = (ObjectArrayTypeInfo<T, C>)obj;
+
+ return objectArrayTypeInfo.canEqual(this) &&
+ arrayType == objectArrayTypeInfo.arrayType &&
+ componentInfo.equals(objectArrayTypeInfo.componentInfo);
+ } else {
return false;
}
+ }
- ObjectArrayTypeInfo<?, ?> that = (ObjectArrayTypeInfo<?, ?>) o;
- return this.arrayType.equals(that.arrayType) && this.componentType.equals(that.componentType);
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof ObjectArrayTypeInfo;
}
@Override
public int hashCode() {
- return 31 * this.arrayType.hashCode() + this.componentType.hashCode();
+ return 31 * this.arrayType.hashCode() + this.componentInfo.hashCode();
}
// --------------------------------------------------------------------------------------------
- public static <T, C> ObjectArrayTypeInfo<T, C> getInfoFor(Type type, TypeInformation<C> componentInfo) {
- // generic array type e.g. for Tuples
- if (type instanceof GenericArrayType) {
- GenericArrayType genericArray = (GenericArrayType) type;
- return new ObjectArrayTypeInfo<T, C>(type, genericArray.getGenericComponentType(), componentInfo);
- }
- // for tuples without generics (e.g. generated by the TypeInformation parser)
- // and multidimensional arrays (e.g. in scala)
- else if (type instanceof Class<?> && ((Class<?>) type).isArray()
- && BasicTypeInfo.getInfoFor((Class<?>) type) == null) {
- return new ObjectArrayTypeInfo<T, C>(type, ((Class<?>) type).getComponentType(), componentInfo);
- }
- throw new InvalidTypesException("The given type is not a valid object array.");
+ public static <T, C> ObjectArrayTypeInfo<T, C> getInfoFor(Class<T> arrayClass, TypeInformation<C> componentInfo) {
+ Preconditions.checkNotNull(arrayClass);
+ Preconditions.checkNotNull(componentInfo);
+ Preconditions.checkArgument(arrayClass.isArray(), "Class " + arrayClass + " must be an array.");
+
+ return new ObjectArrayTypeInfo<T, C>(arrayClass, componentInfo);
}
/**
@@ -170,20 +130,12 @@ public class ObjectArrayTypeInfo<T, C> extends TypeInformation<T> {
* This must be used in cases where the complete type of the array is not available as a
* {@link java.lang.reflect.Type} or {@link java.lang.Class}.
*/
+ @SuppressWarnings("unchecked")
public static <T, C> ObjectArrayTypeInfo<T, C> getInfoFor(TypeInformation<C> componentInfo) {
- return new ObjectArrayTypeInfo<T, C>(
- Array.newInstance(componentInfo.getTypeClass(), 0).getClass(),
- componentInfo.getTypeClass(),
- componentInfo);
- }
+ Preconditions.checkNotNull(componentInfo);
- @SuppressWarnings("unchecked")
- public static <T, C> ObjectArrayTypeInfo<T, C> getInfoFor(Type type) {
- // class type e.g. for POJOs
- if (type instanceof Class<?> && ((Class<?>) type).isArray() && BasicTypeInfo.getInfoFor((Class<C>) type) == null) {
- Class<C> array = (Class<C>) type;
- return new ObjectArrayTypeInfo<T, C>(type, array.getComponentType());
- }
- throw new InvalidTypesException("The given type is not a valid object array.");
+ return new ObjectArrayTypeInfo<T, C>(
+ (Class<T>)Array.newInstance(componentInfo.getTypeClass(), 0).getClass(),
+ componentInfo);
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/main/java/org/apache/flink/api/java/typeutils/PojoField.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/PojoField.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/PojoField.java
index 91a7a5e..4ad0ac2 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/PojoField.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/PojoField.java
@@ -23,16 +23,29 @@ import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
import java.io.Serializable;
import java.lang.reflect.Field;
+import java.util.Objects;
+import com.google.common.base.Preconditions;
import org.apache.flink.api.common.typeinfo.TypeInformation;
public class PojoField implements Serializable {
- public transient Field field;
- public TypeInformation<?> type;
+
+ private static final long serialVersionUID = 1975295846436559363L;
+
+ private transient Field field;
+ private final TypeInformation<?> type;
public PojoField(Field field, TypeInformation<?> type) {
- this.field = field;
- this.type = type;
+ this.field = Preconditions.checkNotNull(field);
+ this.type = Preconditions.checkNotNull(type);
+ }
+
+ public Field getField() {
+ return field;
+ }
+
+ public TypeInformation<?> getTypeInformation() {
+ return type;
}
private void writeObject(ObjectOutputStream out)
@@ -68,4 +81,25 @@ public class PojoField implements Serializable {
public String toString() {
return "PojoField " + field.getDeclaringClass() + "." + field.getName() + " (" + type + ")";
}
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof PojoField) {
+ PojoField other = (PojoField) obj;
+
+ return other.canEqual(this) && type.equals(other.type) &&
+ Objects.equals(field, other.field);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(field, type);
+ }
+
+ public boolean canEqual(Object obj) {
+ return obj instanceof PojoField;
+ }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java
index 273a907..f7e4e42 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java
@@ -22,12 +22,12 @@ import java.lang.reflect.Field;
import java.lang.reflect.Modifier;
import java.util.ArrayList;
import java.util.Arrays;
-import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
+import com.google.common.base.Preconditions;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.CompositeType;
@@ -40,8 +40,6 @@ import org.apache.flink.api.java.operators.Keys.ExpressionKeys;
import com.google.common.base.Joiner;
import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* TypeInformation for "Java Beans"-style types. Flink refers to them as POJOs,
@@ -59,8 +57,6 @@ public class PojoTypeInfo<T> extends CompositeType<T> {
private static final long serialVersionUID = 1L;
- private static final Logger LOG = LoggerFactory.getLogger(PojoTypeInfo.class);
-
private final static String REGEX_FIELD = "[\\p{L}_\\$][\\p{L}\\p{Digit}_\\$]*";
private final static String REGEX_NESTED_FIELDS = "("+REGEX_FIELD+")(\\.(.+))?";
private final static String REGEX_NESTED_FIELDS_WILDCARD = REGEX_NESTED_FIELDS
@@ -70,31 +66,32 @@ public class PojoTypeInfo<T> extends CompositeType<T> {
private static final Pattern PATTERN_NESTED_FIELDS = Pattern.compile(REGEX_NESTED_FIELDS);
private static final Pattern PATTERN_NESTED_FIELDS_WILDCARD = Pattern.compile(REGEX_NESTED_FIELDS_WILDCARD);
- private final Class<T> typeClass;
-
- private PojoField[] fields;
+ private final PojoField[] fields;
- private int totalFields;
+ private final int totalFields;
public PojoTypeInfo(Class<T> typeClass, List<PojoField> fields) {
super(typeClass);
- this.typeClass = typeClass;
- List<PojoField> tempFields = new ArrayList<PojoField>(fields);
- Collections.sort(tempFields, new Comparator<PojoField>() {
+
+ Preconditions.checkArgument(Modifier.isPublic(typeClass.getModifiers()),
+ "POJO " + typeClass + " is not public");
+
+ this.fields = fields.toArray(new PojoField[fields.size()]);
+
+ Arrays.sort(this.fields, new Comparator<PojoField>() {
@Override
public int compare(PojoField o1, PojoField o2) {
- return o1.field.getName().compareTo(o2.field.getName());
+ return o1.getField().getName().compareTo(o2.getField().getName());
}
});
- this.fields = tempFields.toArray(new PojoField[tempFields.size()]);
-
- // check if POJO is public
- if(!Modifier.isPublic(typeClass.getModifiers())) {
- throw new RuntimeException("POJO "+typeClass+" is not public");
- }
+
+ int counterFields = 0;
+
for(PojoField field : fields) {
- totalFields += field.type.getTotalFields();
+ counterFields += field.getTypeInformation().getTotalFields();
}
+
+ totalFields = counterFields;
}
@Override
@@ -119,11 +116,6 @@ public class PojoTypeInfo<T> extends CompositeType<T> {
}
@Override
- public Class<T> getTypeClass() {
- return typeClass;
- }
-
- @Override
public boolean isSortKeyType() {
// Support for sorting POJOs that implement Comparable is not implemented yet.
// Since the order of fields in a POJO type is not well defined, sorting on fields
@@ -145,12 +137,16 @@ public class PojoTypeInfo<T> extends CompositeType<T> {
// handle select all
int keyPosition = 0;
for(PojoField pField : fields) {
- if(pField.type instanceof CompositeType) {
- CompositeType<?> cType = (CompositeType<?>)pField.type;
+ if(pField.getTypeInformation() instanceof CompositeType) {
+ CompositeType<?> cType = (CompositeType<?>)pField.getTypeInformation();
cType.getFlatFields(String.valueOf(ExpressionKeys.SELECT_ALL_CHAR), offset + keyPosition, result);
keyPosition += cType.getTotalFields()-1;
} else {
- result.add(new NamedFlatFieldDescriptor(pField.field.getName(), offset + keyPosition, pField.type));
+ result.add(
+ new NamedFlatFieldDescriptor(
+ pField.getField().getName(),
+ offset + keyPosition,
+ pField.getTypeInformation()));
}
keyPosition++;
}
@@ -163,9 +159,9 @@ public class PojoTypeInfo<T> extends CompositeType<T> {
int fieldPos = -1;
TypeInformation<?> fieldType = null;
for (int i = 0; i < fields.length; i++) {
- if (fields[i].field.getName().equals(field)) {
+ if (fields[i].getField().getName().equals(field)) {
fieldPos = i;
- fieldType = fields[i].type;
+ fieldType = fields[i].getTypeInformation();
break;
}
}
@@ -181,7 +177,6 @@ public class PojoTypeInfo<T> extends CompositeType<T> {
}
// add all fields of composite type
((CompositeType<?>) fieldType).getFlatFields("*", offset, result);
- return;
} else {
// we found the field to add
// compute flat field position by adding skipped fields
@@ -190,8 +185,6 @@ public class PojoTypeInfo<T> extends CompositeType<T> {
flatFieldPos += this.getTypeAt(i).getTotalFields();
}
result.add(new FlatFieldDescriptor(flatFieldPos, fieldType));
- // nothing left to do
- return;
}
} else {
if(fieldType instanceof CompositeType<?>) {
@@ -200,8 +193,6 @@ public class PojoTypeInfo<T> extends CompositeType<T> {
offset += this.getTypeAt(i).getTotalFields();
}
((CompositeType<?>) fieldType).getFlatFields(tail, offset, result);
- // nothing left to do
- return;
} else {
throw new InvalidFieldReferenceException("Nested field expression \""+tail+"\" not possible on atomic type "+fieldType+".");
}
@@ -226,9 +217,9 @@ public class PojoTypeInfo<T> extends CompositeType<T> {
int fieldPos = -1;
TypeInformation<?> fieldType = null;
for (int i = 0; i < fields.length; i++) {
- if (fields[i].field.getName().equals(field)) {
+ if (fields[i].getField().getName().equals(field)) {
fieldPos = i;
- fieldType = fields[i].type;
+ fieldType = fields[i].getTypeInformation();
break;
}
}
@@ -255,10 +246,15 @@ public class PojoTypeInfo<T> extends CompositeType<T> {
throw new IndexOutOfBoundsException();
}
@SuppressWarnings("unchecked")
- TypeInformation<X> typed = (TypeInformation<X>) fields[pos].type;
+ TypeInformation<X> typed = (TypeInformation<X>) fields[pos].getTypeInformation();
return typed;
}
+ @Override
+ protected TypeComparatorBuilder<T> createTypeComparatorBuilder() {
+ return new PojoTypeComparatorBuilder();
+ }
+
// used for testing. Maybe use mockito here
public PojoField getPojoFieldAt(int pos) {
if (pos < 0 || pos >= this.fields.length) {
@@ -267,42 +263,10 @@ public class PojoTypeInfo<T> extends CompositeType<T> {
return this.fields[pos];
}
- /**
- * Comparator creation
- */
- private TypeComparator<?>[] fieldComparators;
- private Field[] keyFields;
- private int comparatorHelperIndex = 0;
- @Override
- protected void initializeNewComparator(int keyCount) {
- fieldComparators = new TypeComparator<?>[keyCount];
- keyFields = new Field[keyCount];
- comparatorHelperIndex = 0;
- }
-
- @Override
- protected void addCompareField(int fieldId, TypeComparator<?> comparator) {
- fieldComparators[comparatorHelperIndex] = comparator;
- keyFields[comparatorHelperIndex] = fields[fieldId].field;
- comparatorHelperIndex++;
- }
-
- @Override
- protected TypeComparator<T> getNewComparator(ExecutionConfig config) {
- // first remove the null array fields
- final Field[] finalKeyFields = Arrays.copyOf(keyFields, comparatorHelperIndex);
- @SuppressWarnings("rawtypes")
- final TypeComparator[] finalFieldComparators = Arrays.copyOf(fieldComparators, comparatorHelperIndex);
- if(finalFieldComparators.length == 0 || finalKeyFields.length == 0 || finalFieldComparators.length != finalKeyFields.length) {
- throw new IllegalArgumentException("Pojo comparator creation has a bug");
- }
- return new PojoComparator<T>(finalKeyFields, finalFieldComparators, createSerializer(config), typeClass);
- }
-
public String[] getFieldNames() {
String[] result = new String[fields.length];
for (int i = 0; i < fields.length; i++) {
- result[i] = fields[i].field.getName();
+ result[i] = fields[i].getField().getName();
}
return result;
}
@@ -310,7 +274,7 @@ public class PojoTypeInfo<T> extends CompositeType<T> {
@Override
public int getFieldIndex(String fieldName) {
for (int i = 0; i < fields.length; i++) {
- if (fields[i].field.getName().equals(fieldName)) {
+ if (fields[i].getField().getName().equals(fieldName)) {
return i;
}
}
@@ -320,46 +284,106 @@ public class PojoTypeInfo<T> extends CompositeType<T> {
@Override
public TypeSerializer<T> createSerializer(ExecutionConfig config) {
if(config.isForceKryoEnabled()) {
- return new KryoSerializer<T>(this.typeClass, config);
+ return new KryoSerializer<T>(getTypeClass(), config);
}
if(config.isForceAvroEnabled()) {
- return new AvroSerializer<T>(this.typeClass);
+ return new AvroSerializer<T>(getTypeClass());
}
TypeSerializer<?>[] fieldSerializers = new TypeSerializer<?>[fields.length ];
Field[] reflectiveFields = new Field[fields.length];
for (int i = 0; i < fields.length; i++) {
- fieldSerializers[i] = fields[i].type.createSerializer(config);
- reflectiveFields[i] = fields[i].field;
+ fieldSerializers[i] = fields[i].getTypeInformation().createSerializer(config);
+ reflectiveFields[i] = fields[i].getField();
}
- return new PojoSerializer<T>(this.typeClass, fieldSerializers, reflectiveFields, config);
+ return new PojoSerializer<T>(getTypeClass(), fieldSerializers, reflectiveFields, config);
}
-
- // --------------------------------------------------------------------------------------------
@Override
public boolean equals(Object obj) {
- return (obj instanceof PojoTypeInfo) && ((PojoTypeInfo<?>) obj).typeClass == this.typeClass;
+ if (obj instanceof PojoTypeInfo) {
+ @SuppressWarnings("unchecked")
+ PojoTypeInfo<T> pojoTypeInfo = (PojoTypeInfo<T>)obj;
+
+ return pojoTypeInfo.canEqual(this) &&
+ super.equals(pojoTypeInfo) &&
+ Arrays.equals(fields, pojoTypeInfo.fields) &&
+ totalFields == pojoTypeInfo.totalFields;
+ } else {
+ return false;
+ }
}
@Override
public int hashCode() {
- return typeClass.hashCode() + 1387562934;
+ return 31 * (31 * Arrays.hashCode(fields) + totalFields) + super.hashCode();
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof PojoTypeInfo;
}
@Override
public String toString() {
List<String> fieldStrings = new ArrayList<String>();
for (PojoField field : fields) {
- fieldStrings.add(field.field.getName() + ": " + field.type.toString());
+ fieldStrings.add(field.getField().getName() + ": " + field.getTypeInformation().toString());
}
- return "PojoType<" + typeClass.getName()
+ return "PojoType<" + getTypeClass().getName()
+ ", fields = [" + Joiner.on(", ").join(fieldStrings) + "]"
+ ">";
}
+ // --------------------------------------------------------------------------------------------
+
+ private class PojoTypeComparatorBuilder implements TypeComparatorBuilder<T> {
+
+ private ArrayList<TypeComparator> fieldComparators;
+ private ArrayList<Field> keyFields;
+
+ public PojoTypeComparatorBuilder() {
+ fieldComparators = new ArrayList<TypeComparator>();
+ keyFields = new ArrayList<Field>();
+ }
+
+
+ @Override
+ public void initializeTypeComparatorBuilder(int size) {
+ fieldComparators.ensureCapacity(size);
+ keyFields.ensureCapacity(size);
+ }
+
+ @Override
+ public void addComparatorField(int fieldId, TypeComparator<?> comparator) {
+ fieldComparators.add(comparator);
+ keyFields.add(fields[fieldId].getField());
+ }
+
+ @Override
+ public TypeComparator<T> createTypeComparator(ExecutionConfig config) {
+ Preconditions.checkState(
+ keyFields.size() > 0,
+ "No keys were defined for the PojoTypeComparatorBuilder.");
+
+ Preconditions.checkState(
+ fieldComparators.size() > 0,
+ "No type comparators were defined for the PojoTypeComparatorBuilder.");
+
+ Preconditions.checkState(
+ keyFields.size() == fieldComparators.size(),
+ "Number of key fields and field comparators is not equal.");
+
+ return new PojoComparator<T>(
+ keyFields.toArray(new Field[keyFields.size()]),
+ fieldComparators.toArray(new TypeComparator[fieldComparators.size()]),
+ createSerializer(config),
+ getTypeClass());
+ }
+ }
+
public static class NamedFlatFieldDescriptor extends FlatFieldDescriptor {
private String fieldName;
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/main/java/org/apache/flink/api/java/typeutils/RecordTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/RecordTypeInfo.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/RecordTypeInfo.java
index 9857eb6..e9ce102 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/RecordTypeInfo.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/RecordTypeInfo.java
@@ -70,12 +70,22 @@ public class RecordTypeInfo extends TypeInformation<Record> {
@Override
public int hashCode() {
- return Record.class.hashCode() ^ 0x165667b1;
+ return Record.class.hashCode();
}
-
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof RecordTypeInfo;
+ }
+
@Override
public boolean equals(Object obj) {
- return obj.getClass() == RecordTypeInfo.class;
+ if (obj instanceof RecordTypeInfo) {
+ RecordTypeInfo recordTypeInfo = (RecordTypeInfo) obj;
+ return recordTypeInfo.canEqual(this);
+ } else {
+ return false;
+ }
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java
index 618b190..30710e5 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java
@@ -18,8 +18,12 @@
package org.apache.flink.api.java.typeutils;
+import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Collections;
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Ints;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.functions.InvalidTypesException;
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
@@ -52,10 +56,13 @@ public final class TupleTypeInfo<T extends Tuple> extends TupleTypeInfoBase<T> {
public TupleTypeInfo(Class<T> tupleType, TypeInformation<?>... types) {
super(tupleType, types);
- if (types == null || types.length > Tuple.MAX_ARITY) {
- throw new IllegalArgumentException();
- }
+
+ Preconditions.checkArgument(
+ types.length <= Tuple.MAX_ARITY,
+ "The tuple type exceeds the maximum supported arity.");
+
this.fieldNames = new String[types.length];
+
for (int i = 0; i < types.length; i++) {
fieldNames[i] = "f" + i;
}
@@ -78,7 +85,7 @@ public final class TupleTypeInfo<T extends Tuple> extends TupleTypeInfoBase<T> {
@SuppressWarnings("unchecked")
@Override
public TupleSerializer<T> createSerializer(ExecutionConfig executionConfig) {
- if (this.tupleType == Tuple0.class) {
+ if (getTypeClass() == Tuple0.class) {
return (TupleSerializer<T>) Tuple0Serializer.INSTANCE;
}
@@ -91,48 +98,65 @@ public final class TupleTypeInfo<T extends Tuple> extends TupleTypeInfoBase<T> {
return new TupleSerializer<T>(tupleClass, fieldSerializers);
}
-
- /**
- * Comparator creation
- */
- private TypeComparator<?>[] fieldComparators;
- private int[] logicalKeyFields;
- private int comparatorHelperIndex = 0;
-
- @Override
- protected void initializeNewComparator(int localKeyCount) {
- fieldComparators = new TypeComparator<?>[localKeyCount];
- logicalKeyFields = new int[localKeyCount];
- comparatorHelperIndex = 0;
- }
@Override
- protected void addCompareField(int fieldId, TypeComparator<?> comparator) {
- fieldComparators[comparatorHelperIndex] = comparator;
- logicalKeyFields[comparatorHelperIndex] = fieldId;
- comparatorHelperIndex++;
+ protected TypeComparatorBuilder<T> createTypeComparatorBuilder() {
+ return new TupleTypeComparatorBuilder();
}
- @Override
- protected TypeComparator<T> getNewComparator(ExecutionConfig executionConfig) {
- @SuppressWarnings("rawtypes")
- final TypeComparator[] finalFieldComparators = Arrays.copyOf(fieldComparators, comparatorHelperIndex);
- final int[] finalLogicalKeyFields = Arrays.copyOf(logicalKeyFields, comparatorHelperIndex);
- //final TypeSerializer[] finalFieldSerializers = Arrays.copyOf(fieldSerializers, comparatorHelperIndex);
- // create the serializers for the prefix up to highest key position
- int maxKey = 0;
- for(int key : finalLogicalKeyFields) {
- maxKey = Math.max(maxKey, key);
+ private class TupleTypeComparatorBuilder implements TypeComparatorBuilder<T> {
+
+ private final ArrayList<TypeComparator> fieldComparators = new ArrayList<TypeComparator>();
+ private final ArrayList<Integer> logicalKeyFields = new ArrayList<Integer>();
+
+ @Override
+ public void initializeTypeComparatorBuilder(int size) {
+ fieldComparators.ensureCapacity(size);
+ logicalKeyFields.ensureCapacity(size);
}
- TypeSerializer<?>[] fieldSerializers = new TypeSerializer<?>[maxKey + 1];
- for (int i = 0; i <= maxKey; i++) {
- fieldSerializers[i] = types[i].createSerializer(executionConfig);
+
+ @Override
+ public void addComparatorField(int fieldId, TypeComparator<?> comparator) {
+ fieldComparators.add(comparator);
+ logicalKeyFields.add(fieldId);
}
- if(finalFieldComparators.length == 0 || finalLogicalKeyFields.length == 0 || fieldSerializers.length == 0
- || finalFieldComparators.length != finalLogicalKeyFields.length) {
- throw new IllegalArgumentException("Tuple comparator creation has a bug");
+
+ @Override
+ public TypeComparator<T> createTypeComparator(ExecutionConfig config) {
+ Preconditions.checkState(
+ fieldComparators.size() > 0,
+ "No field comparators were defined for the TupleTypeComparatorBuilder."
+ );
+
+ Preconditions.checkState(
+ logicalKeyFields.size() > 0,
+ "No key fields were defined for the TupleTypeComparatorBuilder."
+ );
+
+ Preconditions.checkState(
+ fieldComparators.size() == logicalKeyFields.size(),
+ "The number of field comparators and key fields is not equal."
+ );
+
+ final int maxKey = Collections.max(logicalKeyFields);
+
+ Preconditions.checkState(
+ maxKey >= 0,
+ "The maximum key field must be greater or equal than 0."
+ );
+
+ TypeSerializer<?>[] fieldSerializers = new TypeSerializer<?>[maxKey + 1];
+
+ for (int i = 0; i <= maxKey; i++) {
+ fieldSerializers[i] = types[i].createSerializer(config);
+ }
+
+ return new TupleComparator<T>(
+ Ints.toArray(logicalKeyFields),
+ fieldComparators.toArray(new TypeComparator[fieldComparators.size()]),
+ fieldSerializers
+ );
}
- return new TupleComparator<T>(finalLogicalKeyFields, finalFieldComparators, fieldSerializers);
}
// --------------------------------------------------------------------------------------------
@@ -142,17 +166,22 @@ public final class TupleTypeInfo<T extends Tuple> extends TupleTypeInfoBase<T> {
if (obj instanceof TupleTypeInfo) {
@SuppressWarnings("unchecked")
TupleTypeInfo<T> other = (TupleTypeInfo<T>) obj;
- return ((this.tupleType == null && other.tupleType == null) || this.tupleType.equals(other.tupleType)) &&
- Arrays.deepEquals(this.types, other.types);
-
+ return other.canEqual(this) &&
+ super.equals(other) &&
+ Arrays.equals(fieldNames, other.fieldNames);
} else {
return false;
}
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof TupleTypeInfo;
+ }
@Override
public int hashCode() {
- return this.types.hashCode() ^ Arrays.deepHashCode(this.types);
+ return 31 * super.hashCode() + Arrays.hashCode(fieldNames);
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfoBase.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfoBase.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfoBase.java
index a2d937f..469476f 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfoBase.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfoBase.java
@@ -23,6 +23,7 @@ import java.util.List;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
+import com.google.common.base.Preconditions;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.CompositeType;
import org.apache.flink.api.java.operators.Keys.ExpressionKeys;
@@ -45,17 +46,20 @@ public abstract class TupleTypeInfoBase<T> extends CompositeType<T> {
protected final TypeInformation<?>[] types;
- protected final Class<T> tupleType;
-
- private int totalFields;
+ private final int totalFields;
public TupleTypeInfoBase(Class<T> tupleType, TypeInformation<?>... types) {
super(tupleType);
- this.tupleType = tupleType;
- this.types = types;
+
+ this.types = Preconditions.checkNotNull(types);
+
+ int fieldCounter = 0;
+
for(TypeInformation<?> type : types) {
- totalFields += type.getTotalFields();
+ fieldCounter += type.getTotalFields();
}
+
+ totalFields = fieldCounter;
}
@Override
@@ -83,11 +87,6 @@ public abstract class TupleTypeInfoBase<T> extends CompositeType<T> {
}
@Override
- public Class<T> getTypeClass() {
- return tupleType;
- }
-
- @Override
public void getFlatFields(String fieldExpression, int offset, List<FlatFieldDescriptor> result) {
Matcher matcher = PATTERN_NESTED_FIELDS_WILDCARD.matcher(fieldExpression);
@@ -109,53 +108,49 @@ public abstract class TupleTypeInfoBase<T> extends CompositeType<T> {
}
keyPosition++;
}
- return;
- }
+ } else {
+ String fieldStr = matcher.group(1);
+ Matcher fieldMatcher = PATTERN_FIELD.matcher(fieldStr);
- String fieldStr = matcher.group(1);
- Matcher fieldMatcher = PATTERN_FIELD.matcher(fieldStr);
- if (!fieldMatcher.matches()) {
- throw new RuntimeException("Invalid matcher pattern");
- }
- field = fieldMatcher.group(2);
- int fieldPos = Integer.valueOf(field);
+ if (!fieldMatcher.matches()) {
+ throw new RuntimeException("Invalid matcher pattern");
+ }
- if (fieldPos >= this.getArity()) {
- throw new InvalidFieldReferenceException("Tuple field expression \"" + fieldStr + "\" out of bounds of " + this.toString() + ".");
- }
- TypeInformation<?> fieldType = this.getTypeAt(fieldPos);
- String tail = matcher.group(5);
- if(tail == null) {
- if(fieldType instanceof CompositeType) {
- // forward offsets
- for(int i=0; i<fieldPos; i++) {
- offset += this.getTypeAt(i).getTotalFields();
- }
- // add all fields of composite type
- ((CompositeType<?>) fieldType).getFlatFields("*", offset, result);
- return;
- } else {
- // we found the field to add
- // compute flat field position by adding skipped fields
- int flatFieldPos = offset;
- for(int i=0; i<fieldPos; i++) {
- flatFieldPos += this.getTypeAt(i).getTotalFields();
- }
- result.add(new FlatFieldDescriptor(flatFieldPos, fieldType));
- // nothing left to do
- return;
+ field = fieldMatcher.group(2);
+ int fieldPos = Integer.valueOf(field);
+
+ if (fieldPos >= this.getArity()) {
+ throw new InvalidFieldReferenceException("Tuple field expression \"" + fieldStr + "\" out of bounds of " + this.toString() + ".");
}
- } else {
- if(fieldType instanceof CompositeType<?>) {
- // forward offset
- for(int i=0; i<fieldPos; i++) {
- offset += this.getTypeAt(i).getTotalFields();
+ TypeInformation<?> fieldType = this.getTypeAt(fieldPos);
+ String tail = matcher.group(5);
+ if (tail == null) {
+ if (fieldType instanceof CompositeType) {
+ // forward offsets
+ for (int i = 0; i < fieldPos; i++) {
+ offset += this.getTypeAt(i).getTotalFields();
+ }
+ // add all fields of composite type
+ ((CompositeType<?>) fieldType).getFlatFields("*", offset, result);
+ } else {
+ // we found the field to add
+ // compute flat field position by adding skipped fields
+ int flatFieldPos = offset;
+ for (int i = 0; i < fieldPos; i++) {
+ flatFieldPos += this.getTypeAt(i).getTotalFields();
+ }
+ result.add(new FlatFieldDescriptor(flatFieldPos, fieldType));
}
- ((CompositeType<?>) fieldType).getFlatFields(tail, offset, result);
- // nothing left to do
- return;
} else {
- throw new InvalidFieldReferenceException("Nested field expression \""+tail+"\" not possible on atomic type "+fieldType+".");
+ if (fieldType instanceof CompositeType<?>) {
+ // forward offset
+ for (int i = 0; i < fieldPos; i++) {
+ offset += this.getTypeAt(i).getTotalFields();
+ }
+ ((CompositeType<?>) fieldType).getFlatFields(tail, offset, result);
+ } else {
+ throw new InvalidFieldReferenceException("Nested field expression \"" + tail + "\" not possible on atomic type " + fieldType + ".");
+ }
}
}
}
@@ -213,17 +208,24 @@ public abstract class TupleTypeInfoBase<T> extends CompositeType<T> {
if (obj instanceof TupleTypeInfoBase) {
@SuppressWarnings("unchecked")
TupleTypeInfoBase<T> other = (TupleTypeInfoBase<T>) obj;
- return ((this.tupleType == null && other.tupleType == null) || this.tupleType.equals(other.tupleType)) &&
- Arrays.deepEquals(this.types, other.types);
-
+
+ return other.canEqual(this) &&
+ super.equals(other) &&
+ Arrays.equals(types, other.types) &&
+ totalFields == other.totalFields;
} else {
return false;
}
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof TupleTypeInfoBase;
+ }
@Override
public int hashCode() {
- return this.types.hashCode() ^ Arrays.deepHashCode(this.types);
+ return 31 * (31 * super.hashCode() + Arrays.hashCode(types)) + totalFields;
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/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 8f5d599..0196b5d 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
@@ -526,12 +526,33 @@ public class TypeExtractor {
} catch (ClassNotFoundException e) {
throw new InvalidTypesException("Could not convert GenericArrayType to Class.");
}
+
return getForClass(classArray);
+ } else {
+ TypeInformation<?> componentInfo = createTypeInfoWithTypeHierarchy(
+ typeHierarchy,
+ genericArray.getGenericComponentType(),
+ in1Type,
+ in2Type);
+
+ Class<OUT> classArray;
+
+ try {
+ String componentClassName = componentInfo.getTypeClass().getName();
+ String resultingClassName;
+
+ if (componentClassName.startsWith("[")) {
+ resultingClassName = "[" + componentClassName;
+ } else {
+ resultingClassName = "[L" + componentClassName + ";";
+ }
+ classArray = (Class<OUT>) Class.forName(resultingClassName);
+ } catch (ClassNotFoundException e) {
+ throw new InvalidTypesException("Could not convert GenericArrayType to Class.");
+ }
+
+ return ObjectArrayTypeInfo.getInfoFor(classArray, componentInfo);
}
-
- TypeInformation<?> componentInfo = createTypeInfoWithTypeHierarchy(typeHierarchy, genericArray.getGenericComponentType(),
- in1Type, in2Type);
- return ObjectArrayTypeInfo.getInfoFor(t, componentInfo);
}
// objects with generics are treated as Class first
else if (t instanceof ParameterizedType) {
@@ -1188,7 +1209,13 @@ public class TypeExtractor {
// object arrays
else {
- return ObjectArrayTypeInfo.getInfoFor(clazz);
+ TypeInformation<?> componentTypeInfo = createTypeInfoWithTypeHierarchy(
+ typeHierarchy,
+ clazz.getComponentType(),
+ in1Type,
+ in2Type);
+
+ return ObjectArrayTypeInfo.getInfoFor(clazz, componentTypeInfo);
}
}
@@ -1481,8 +1508,8 @@ public class TypeExtractor {
private static TypeInformation<?> getTypeOfPojoField(TypeInformation<?> pojoInfo, Field field) {
for (int j = 0; j < pojoInfo.getArity(); j++) {
PojoField pf = ((PojoTypeInfo<?>) pojoInfo).getPojoFieldAt(j);
- if (pf.field.getName().equals(field.getName())) {
- return pf.type;
+ if (pf.getField().getName().equals(field.getName())) {
+ return pf.getTypeInformation();
}
}
return null;
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ValueTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ValueTypeInfo.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ValueTypeInfo.java
index e61acd8..0b4823e 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ValueTypeInfo.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ValueTypeInfo.java
@@ -18,6 +18,7 @@
package org.apache.flink.api.java.typeutils;
+import com.google.common.base.Preconditions;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.functions.InvalidTypesException;
import org.apache.flink.api.common.typeinfo.AtomicType;
@@ -52,17 +53,13 @@ public class ValueTypeInfo<T extends Value> extends TypeInformation<T> implement
private static final long serialVersionUID = 1L;
private final Class<T> type;
-
public ValueTypeInfo(Class<T> type) {
- if (type == null) {
- throw new NullPointerException();
- }
- if (!Value.class.isAssignableFrom(type) && !type.equals(Value.class)) {
- throw new IllegalArgumentException("ValueTypeInfo can only be used for subclasses of " + Value.class.getName());
- }
-
- this.type = type;
+ this.type = Preconditions.checkNotNull(type);
+
+ Preconditions.checkArgument(
+ Value.class.isAssignableFrom(type) || type.equals(Value.class),
+ "ValueTypeInfo can only be used for subclasses of " + Value.class.getName());
}
@Override
@@ -136,17 +133,26 @@ public class ValueTypeInfo<T extends Value> extends TypeInformation<T> implement
@Override
public int hashCode() {
- return this.type.hashCode() ^ 0xd3a2646c;
+ return this.type.hashCode();
}
@Override
public boolean equals(Object obj) {
- if (obj.getClass() == ValueTypeInfo.class) {
- return type == ((ValueTypeInfo<?>) obj).type;
+ if (obj instanceof ValueTypeInfo) {
+ @SuppressWarnings("unchecked")
+ ValueTypeInfo<T> valueTypeInfo = (ValueTypeInfo<T>) obj;
+
+ return valueTypeInfo.canEqual(this) &&
+ type == valueTypeInfo.type;
} else {
return false;
}
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof ValueTypeInfo;
+ }
@Override
public String toString() {
@@ -155,7 +161,7 @@ public class ValueTypeInfo<T extends Value> extends TypeInformation<T> implement
// --------------------------------------------------------------------------------------------
- static final <X extends Value> TypeInformation<X> getValueTypeInfo(Class<X> typeClass) {
+ static <X extends Value> TypeInformation<X> getValueTypeInfo(Class<X> typeClass) {
if (Value.class.isAssignableFrom(typeClass) && !typeClass.equals(Value.class)) {
return new ValueTypeInfo<X>(typeClass);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/main/java/org/apache/flink/api/java/typeutils/WritableTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/WritableTypeInfo.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/WritableTypeInfo.java
index a91b888..6c140d9 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/WritableTypeInfo.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/WritableTypeInfo.java
@@ -18,6 +18,7 @@
package org.apache.flink.api.java.typeutils;
+import com.google.common.base.Preconditions;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.functions.InvalidTypesException;
import org.apache.flink.api.common.typeinfo.AtomicType;
@@ -41,13 +42,11 @@ public class WritableTypeInfo<T extends Writable> extends TypeInformation<T> imp
private final Class<T> typeClass;
public WritableTypeInfo(Class<T> typeClass) {
- if (typeClass == null) {
- throw new NullPointerException();
- }
- if (!Writable.class.isAssignableFrom(typeClass) || typeClass == Writable.class) {
- throw new IllegalArgumentException("WritableTypeInfo can only be used for subclasses of " + Writable.class.getName());
- }
- this.typeClass = typeClass;
+ this.typeClass = Preconditions.checkNotNull(typeClass);
+
+ Preconditions.checkArgument(
+ Writable.class.isAssignableFrom(typeClass) && !typeClass.equals(Writable.class),
+ "WritableTypeInfo can only be used for subclasses of " + Writable.class.getName());
}
@SuppressWarnings({ "rawtypes", "unchecked" })
@@ -104,12 +103,26 @@ public class WritableTypeInfo<T extends Writable> extends TypeInformation<T> imp
@Override
public int hashCode() {
- return typeClass.hashCode() ^ 0xd3a2646c;
+ return typeClass.hashCode();
}
@Override
public boolean equals(Object obj) {
- return obj.getClass() == WritableTypeInfo.class && typeClass == ((WritableTypeInfo<?>) obj).typeClass;
+ if (obj instanceof WritableTypeInfo) {
+ @SuppressWarnings("unchecked")
+ WritableTypeInfo<T> writableTypeInfo = (WritableTypeInfo<T>) obj;
+
+ return writableTypeInfo.canEqual(this) &&
+ typeClass == writableTypeInfo.typeClass;
+
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof WritableTypeInfo;
}
// --------------------------------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java
index 34dc500..26bf4ce 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java
@@ -25,6 +25,7 @@ import java.io.IOException;
import com.esotericsoftware.kryo.KryoException;
import com.esotericsoftware.kryo.io.Input;
import com.esotericsoftware.kryo.io.Output;
+import com.google.common.base.Preconditions;
import org.apache.avro.generic.GenericData;
import org.apache.avro.reflect.ReflectDatumReader;
import org.apache.avro.reflect.ReflectDatumWriter;
@@ -69,14 +70,10 @@ public final class AvroSerializer<T> extends TypeSerializer<T> {
}
public AvroSerializer(Class<T> type, Class<? extends T> typeToInstantiate) {
- if (type == null || typeToInstantiate == null) {
- throw new NullPointerException();
- }
+ this.type = Preconditions.checkNotNull(type);
+ this.typeToInstantiate = Preconditions.checkNotNull(typeToInstantiate);
InstantiationUtil.checkForInstantiation(typeToInstantiate);
-
- this.type = type;
- this.typeToInstantiate = typeToInstantiate;
}
// --------------------------------------------------------------------------------------------
@@ -192,16 +189,25 @@ public final class AvroSerializer<T> extends TypeSerializer<T> {
@Override
public int hashCode() {
- return 0x42fba55c + this.type.hashCode() + this.typeToInstantiate.hashCode();
+ return 31 * this.type.hashCode() + this.typeToInstantiate.hashCode();
}
@Override
public boolean equals(Object obj) {
- if (obj.getClass() == AvroSerializer.class) {
- AvroSerializer<?> other = (AvroSerializer<?>) obj;
- return this.type == other.type && this.typeToInstantiate == other.typeToInstantiate;
+ if (obj instanceof AvroSerializer) {
+ @SuppressWarnings("unchecked")
+ AvroSerializer<T> avroSerializer = (AvroSerializer<T>) obj;
+
+ return avroSerializer.canEqual(this) &&
+ type == avroSerializer.type &&
+ typeToInstantiate == avroSerializer.typeToInstantiate;
} else {
return false;
}
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof AvroSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueSerializer.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueSerializer.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueSerializer.java
index 193d495..9e46f27 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueSerializer.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueSerializer.java
@@ -20,6 +20,7 @@ package org.apache.flink.api.java.typeutils.runtime;
import java.io.IOException;
+import com.google.common.base.Preconditions;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;
@@ -38,7 +39,7 @@ public class CopyableValueSerializer<T extends CopyableValue<T>> extends TypeSer
public CopyableValueSerializer(Class<T> valueClass) {
- this.valueClass = valueClass;
+ this.valueClass = Preconditions.checkNotNull(valueClass);
}
@Override
@@ -105,16 +106,24 @@ public class CopyableValueSerializer<T extends CopyableValue<T>> extends TypeSer
@Override
public int hashCode() {
- return this.valueClass.hashCode() + 9231;
+ return this.valueClass.hashCode();
}
@Override
public boolean equals(Object obj) {
- if (obj.getClass() == CopyableValueSerializer.class) {
- CopyableValueSerializer<?> other = (CopyableValueSerializer<?>) obj;
- return this.valueClass == other.valueClass;
+ if (obj instanceof CopyableValueSerializer) {
+ @SuppressWarnings("unchecked")
+ CopyableValueSerializer<T> copyableValueSerializer = (CopyableValueSerializer<T>) obj;
+
+ return copyableValueSerializer.canEqual(this) &&
+ valueClass == copyableValueSerializer.valueClass;
} else {
return false;
}
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof CopyableValueSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java
index 5d4553d..de24956 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java
@@ -30,7 +30,9 @@ import java.util.LinkedHashMap;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
+import java.util.Objects;
+import com.google.common.base.Preconditions;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -51,33 +53,33 @@ public final class PojoSerializer<T> extends TypeSerializer<T> {
private final Class<T> clazz;
- private TypeSerializer<Object>[] fieldSerializers;
+ private final TypeSerializer<Object>[] fieldSerializers;
- // We need to handle these ourselves in writeObject()/readObject()
- private transient Field[] fields;
-
- private int numFields;
+ private final int numFields;
- private transient Map<Class<?>, TypeSerializer<?>> subclassSerializerCache;
- private transient ClassLoader cl;
+ private final Map<Class<?>, Integer> registeredClasses;
- private Map<Class<?>, Integer> registeredClasses;
-
- private TypeSerializer<?>[] registeredSerializers;
+ private final TypeSerializer<?>[] registeredSerializers;
private final ExecutionConfig executionConfig;
+ private transient Map<Class<?>, TypeSerializer<?>> subclassSerializerCache;
+ private transient ClassLoader cl;
+ // We need to handle these ourselves in writeObject()/readObject()
+ private transient Field[] fields;
+
@SuppressWarnings("unchecked")
public PojoSerializer(
Class<T> clazz,
TypeSerializer<?>[] fieldSerializers,
Field[] fields,
ExecutionConfig executionConfig) {
- this.clazz = clazz;
- this.fieldSerializers = (TypeSerializer<Object>[]) fieldSerializers;
- this.fields = fields;
+
+ this.clazz = Preconditions.checkNotNull(clazz);
+ this.fieldSerializers = (TypeSerializer<Object>[]) Preconditions.checkNotNull(fieldSerializers);
+ this.fields = Preconditions.checkNotNull(fields);
this.numFields = fieldSerializers.length;
- this.executionConfig = executionConfig;
+ this.executionConfig = Preconditions.checkNotNull(executionConfig);
LinkedHashSet<Class<?>> registeredPojoTypes = executionConfig.getRegisteredPojoTypes();
@@ -563,23 +565,28 @@ public final class PojoSerializer<T> extends TypeSerializer<T> {
@Override
public int hashCode() {
- int hashCode = numFields * 47;
- for (TypeSerializer<?> ser : this.fieldSerializers) {
- hashCode = (hashCode << 7) | (hashCode >>> -7);
- hashCode += ser.hashCode();
- }
- return hashCode;
+ return 31 * (31 * Arrays.hashCode(fieldSerializers) + Arrays.hashCode(registeredSerializers)) +
+ Objects.hash(clazz, numFields, registeredClasses);
}
@Override
public boolean equals(Object obj) {
- if (obj != null && obj instanceof PojoSerializer) {
- PojoSerializer<?> otherTS = (PojoSerializer<?>) obj;
- return (otherTS.clazz == this.clazz) &&
- Arrays.deepEquals(this.fieldSerializers, otherTS.fieldSerializers);
- }
- else {
+ if (obj instanceof PojoSerializer) {
+ PojoSerializer<?> other = (PojoSerializer<?>) obj;
+
+ return other.canEqual(this) &&
+ clazz == other.clazz &&
+ Arrays.equals(fieldSerializers, other.fieldSerializers) &&
+ Arrays.equals(registeredSerializers, other.registeredSerializers) &&
+ numFields == other.numFields &&
+ registeredClasses.equals(other.registeredClasses);
+ } else {
return false;
}
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof PojoSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/Tuple0Serializer.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/Tuple0Serializer.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/Tuple0Serializer.java
index 246cecf..a06ff1a 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/Tuple0Serializer.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/Tuple0Serializer.java
@@ -13,6 +13,7 @@
package org.apache.flink.api.java.typeutils.runtime;
import java.io.IOException;
+
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.tuple.Tuple0;
import org.apache.flink.core.memory.DataInputView;
@@ -94,12 +95,23 @@ public class Tuple0Serializer extends TupleSerializer<Tuple0> {
@Override
public int hashCode() {
- return 1837461876;
+ return Tuple0Serializer.class.hashCode();
}
@Override
public boolean equals(Object obj) {
- return obj == this || obj instanceof Tuple0Serializer;
+ if (obj instanceof Tuple0Serializer) {
+ Tuple0Serializer other = (Tuple0Serializer) obj;
+
+ return other.canEqual(this);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof Tuple0Serializer;
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java
index f041736..bf3c7a1 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java
@@ -18,12 +18,14 @@
package org.apache.flink.api.java.typeutils.runtime;
+import com.google.common.base.Preconditions;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;
import java.io.IOException;
import java.util.Arrays;
+import java.util.Objects;
public abstract class TupleSerializerBase<T> extends TypeSerializer<T> {
@@ -32,14 +34,14 @@ public abstract class TupleSerializerBase<T> extends TypeSerializer<T> {
protected final Class<T> tupleClass;
- protected TypeSerializer<Object>[] fieldSerializers;
+ protected final TypeSerializer<Object>[] fieldSerializers;
protected final int arity;
@SuppressWarnings("unchecked")
public TupleSerializerBase(Class<T> tupleClass, TypeSerializer<?>[] fieldSerializers) {
- this.tupleClass = tupleClass;
- this.fieldSerializers = (TypeSerializer<Object>[]) fieldSerializers;
+ this.tupleClass = Preconditions.checkNotNull(tupleClass);
+ this.fieldSerializers = (TypeSerializer<Object>[]) Preconditions.checkNotNull(fieldSerializers);
this.arity = fieldSerializers.length;
}
@@ -74,23 +76,25 @@ public abstract class TupleSerializerBase<T> extends TypeSerializer<T> {
@Override
public int hashCode() {
- int hashCode = arity * 47;
- for (TypeSerializer<?> ser : this.fieldSerializers) {
- hashCode = (hashCode << 7) | (hashCode >>> -7);
- hashCode += ser.hashCode();
- }
- return hashCode;
+ return 31 * Arrays.hashCode(fieldSerializers) + Objects.hash(tupleClass, arity);
}
@Override
public boolean equals(Object obj) {
- if (obj != null && obj instanceof TupleSerializerBase) {
- TupleSerializerBase<?> otherTS = (TupleSerializerBase<?>) obj;
- return (otherTS.tupleClass == this.tupleClass) &&
- Arrays.deepEquals(this.fieldSerializers, otherTS.fieldSerializers);
- }
- else {
+ if (obj instanceof TupleSerializerBase) {
+ TupleSerializerBase<?> other = (TupleSerializerBase<?>) obj;
+
+ return other.canEqual(this) &&
+ tupleClass == other.tupleClass &&
+ Arrays.equals(fieldSerializers, other.fieldSerializers) &&
+ arity == other.arity;
+ } else {
return false;
}
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof TupleSerializerBase;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializer.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializer.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializer.java
index ad1b0f0..179ef19 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializer.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializer.java
@@ -20,6 +20,7 @@ package org.apache.flink.api.java.typeutils.runtime;
import java.io.IOException;
+import com.google.common.base.Preconditions;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;
@@ -47,11 +48,7 @@ public class ValueSerializer<T extends Value> extends TypeSerializer<T> {
// --------------------------------------------------------------------------------------------
public ValueSerializer(Class<T> type) {
- if (type == null) {
- throw new NullPointerException();
- }
-
- this.type = type;
+ this.type = Preconditions.checkNotNull(type);
}
// --------------------------------------------------------------------------------------------
@@ -126,16 +123,22 @@ public class ValueSerializer<T extends Value> extends TypeSerializer<T> {
@Override
public int hashCode() {
- return this.type.hashCode() + 17;
+ return this.type.hashCode();
}
@Override
public boolean equals(Object obj) {
- if (obj.getClass() == ValueSerializer.class) {
+ if (obj instanceof ValueSerializer) {
ValueSerializer<?> other = (ValueSerializer<?>) obj;
- return this.type == other.type;
+
+ return other.canEqual(this) && type == other.type;
} else {
return false;
}
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof ValueSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializer.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializer.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializer.java
index 60012ee..d854f52 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializer.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializer.java
@@ -121,16 +121,22 @@ public class WritableSerializer<T extends Writable> extends TypeSerializer<T> {
@Override
public int hashCode() {
- return this.typeClass.hashCode() + 177;
+ return this.typeClass.hashCode();
}
@Override
public boolean equals(Object obj) {
- if (obj.getClass() == WritableSerializer.class) {
+ if (obj instanceof WritableSerializer) {
WritableSerializer<?> other = (WritableSerializer<?>) obj;
- return this.typeClass == other.typeClass;
+
+ return other.canEqual(this) && typeClass == other.typeClass;
} else {
return false;
}
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof WritableSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
index 1bc6771..f825fc6 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
@@ -25,6 +25,7 @@ import com.esotericsoftware.kryo.factories.ReflectionSerializerFactory;
import com.esotericsoftware.kryo.io.Input;
import com.esotericsoftware.kryo.io.Output;
import com.esotericsoftware.kryo.serializers.JavaSerializer;
+import com.google.common.base.Preconditions;
import com.twitter.chill.ScalaKryoInstantiator;
import org.apache.avro.generic.GenericData;
@@ -45,6 +46,7 @@ import java.lang.reflect.Modifier;
import java.util.LinkedHashMap;
import java.util.LinkedHashSet;
import java.util.Map;
+import java.util.Objects;
/**
* A type serializer that serializes its type using the Kryo serialization
@@ -84,10 +86,7 @@ public class KryoSerializer<T> extends TypeSerializer<T> {
// ------------------------------------------------------------------------
public KryoSerializer(Class<T> type, ExecutionConfig executionConfig){
- if(type == null){
- throw new NullPointerException("Type class cannot be null.");
- }
- this.type = type;
+ this.type = Preconditions.checkNotNull(type);
this.defaultSerializers = executionConfig.getDefaultKryoSerializers();
this.defaultSerializerClasses = executionConfig.getDefaultKryoSerializerClasses();
@@ -241,19 +240,34 @@ public class KryoSerializer<T> extends TypeSerializer<T> {
@Override
public int hashCode() {
- return type.hashCode();
+ return Objects.hash(
+ type,
+ registeredTypes,
+ registeredTypesWithSerializerClasses,
+ defaultSerializerClasses);
}
@Override
public boolean equals(Object obj) {
- if (obj != null && obj instanceof KryoSerializer) {
+ if (obj instanceof KryoSerializer) {
KryoSerializer<?> other = (KryoSerializer<?>) obj;
- return other.type == this.type;
+
+ // we cannot include the Serializers here because they don't implement the equals method
+ return other.canEqual(this) &&
+ type == other.type &&
+ registeredTypes.equals(other.registeredTypes) &&
+ registeredTypesWithSerializerClasses.equals(other.registeredTypesWithSerializerClasses) &&
+ defaultSerializerClasses.equals(other.defaultSerializerClasses);
} else {
return false;
}
}
-
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof KryoSerializer;
+ }
+
// --------------------------------------------------------------------------------------------
private void checkKryoInitialized() {
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/test/java/org/apache/flink/api/java/io/CollectionInputFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/io/CollectionInputFormatTest.java b/flink-java/src/test/java/org/apache/flink/api/java/io/CollectionInputFormatTest.java
index ad491d0..ebaa44d 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/io/CollectionInputFormatTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/io/CollectionInputFormatTest.java
@@ -43,6 +43,7 @@ import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
+import java.util.Objects;
public class CollectionInputFormatTest {
@@ -265,8 +266,8 @@ public class CollectionInputFormatTest {
private static final long serialVersionUID = 1L;
- private boolean failOnRead;
- private boolean failOnWrite;
+ private final boolean failOnRead;
+ private final boolean failOnWrite;
public TestSerializer(boolean failOnRead, boolean failOnWrite) {
this.failOnRead = failOnRead;
@@ -331,5 +332,26 @@ public class CollectionInputFormatTest {
public void copy(DataInputView source, DataOutputView target) throws IOException {
target.writeInt(source.readInt());
}
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof TestSerializer) {
+ TestSerializer other = (TestSerializer) obj;
+
+ return other.canEqual(this) && failOnRead == other.failOnRead && failOnWrite == other.failOnWrite;
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof TestSerializer;
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(failOnRead, failOnWrite);
+ }
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/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 34fde20..96ba264 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
@@ -336,37 +336,37 @@ public class PojoTypeExtractionTest {
tupleSeen = false, objectSeen = false, writableSeen = false, collectionSeen = false;
for(int i = 0; i < pojoTypeComplexNested.getArity(); i++) {
PojoField field = pojoTypeComplexNested.getPojoFieldAt(i);
- String name = field.field.getName();
+ String name = field.getField().getName();
if(name.equals("date")) {
if(dateSeen) {
Assert.fail("already seen");
}
dateSeen = true;
- Assert.assertEquals(BasicTypeInfo.DATE_TYPE_INFO, field.type);
- Assert.assertEquals(Date.class, field.type.getTypeClass());
+ Assert.assertEquals(BasicTypeInfo.DATE_TYPE_INFO, field.getTypeInformation());
+ Assert.assertEquals(Date.class, field.getTypeInformation().getTypeClass());
} else if(name.equals("someNumberWithÜnicödeNäme")) {
if(intSeen) {
Assert.fail("already seen");
}
intSeen = true;
- Assert.assertEquals(BasicTypeInfo.INT_TYPE_INFO, field.type);
- Assert.assertEquals(Integer.class, field.type.getTypeClass());
+ Assert.assertEquals(BasicTypeInfo.INT_TYPE_INFO, field.getTypeInformation());
+ Assert.assertEquals(Integer.class, field.getTypeInformation().getTypeClass());
} else if(name.equals("someFloat")) {
if(floatSeen) {
Assert.fail("already seen");
}
floatSeen = true;
- Assert.assertEquals(BasicTypeInfo.FLOAT_TYPE_INFO, field.type);
- Assert.assertEquals(Float.class, field.type.getTypeClass());
+ Assert.assertEquals(BasicTypeInfo.FLOAT_TYPE_INFO, field.getTypeInformation());
+ Assert.assertEquals(Float.class, field.getTypeInformation().getTypeClass());
} else if(name.equals("word")) {
if(tupleSeen) {
Assert.fail("already seen");
}
tupleSeen = true;
- Assert.assertTrue(field.type instanceof TupleTypeInfo<?>);
- Assert.assertEquals(Tuple3.class, field.type.getTypeClass());
+ Assert.assertTrue(field.getTypeInformation() instanceof TupleTypeInfo<?>);
+ Assert.assertEquals(Tuple3.class, field.getTypeInformation().getTypeClass());
// do some more advanced checks on the tuple
- TupleTypeInfo<?> tupleTypeFromComplexNested = (TupleTypeInfo<?>) field.type;
+ TupleTypeInfo<?> tupleTypeFromComplexNested = (TupleTypeInfo<?>) field.getTypeInformation();
Assert.assertEquals(BasicTypeInfo.LONG_TYPE_INFO, tupleTypeFromComplexNested.getTypeAt(0));
Assert.assertEquals(BasicTypeInfo.LONG_TYPE_INFO, tupleTypeFromComplexNested.getTypeAt(1));
Assert.assertEquals(BasicTypeInfo.STRING_TYPE_INFO, tupleTypeFromComplexNested.getTypeAt(2));
@@ -375,21 +375,21 @@ public class PojoTypeExtractionTest {
Assert.fail("already seen");
}
objectSeen = true;
- Assert.assertEquals(new GenericTypeInfo<Object>(Object.class), field.type);
- Assert.assertEquals(Object.class, field.type.getTypeClass());
+ Assert.assertEquals(new GenericTypeInfo<Object>(Object.class), field.getTypeInformation());
+ Assert.assertEquals(Object.class, field.getTypeInformation().getTypeClass());
} else if(name.equals("hadoopCitizen")) {
if(writableSeen) {
Assert.fail("already seen");
}
writableSeen = true;
- Assert.assertEquals(new WritableTypeInfo<MyWritable>(MyWritable.class), field.type);
- Assert.assertEquals(MyWritable.class, field.type.getTypeClass());
+ Assert.assertEquals(new WritableTypeInfo<MyWritable>(MyWritable.class), field.getTypeInformation());
+ Assert.assertEquals(MyWritable.class, field.getTypeInformation().getTypeClass());
} else if(name.equals("collection")) {
if(collectionSeen) {
Assert.fail("already seen");
}
collectionSeen = true;
- Assert.assertEquals(new GenericTypeInfo(List.class), field.type);
+ Assert.assertEquals(new GenericTypeInfo(List.class), field.getTypeInformation());
} else {
Assert.fail("field "+field+" is not expected");
@@ -428,28 +428,28 @@ public class PojoTypeExtractionTest {
PojoTypeInfo<?> pojoTypeForClass = (PojoTypeInfo<?>) typeInformation;
for(int i = 0; i < pojoTypeForClass.getArity(); i++) {
PojoField field = pojoTypeForClass.getPojoFieldAt(i);
- String name = field.field.getName();
+ String name = field.getField().getName();
if(name.equals("somethingFancy")) {
if(arrayListSeen) {
Assert.fail("already seen");
}
arrayListSeen = true;
- Assert.assertTrue(field.type instanceof GenericTypeInfo);
- Assert.assertEquals(ArrayList.class, field.type.getTypeClass());
+ Assert.assertTrue(field.getTypeInformation() instanceof GenericTypeInfo);
+ Assert.assertEquals(ArrayList.class, field.getTypeInformation().getTypeClass());
} else if(name.equals("fancyIds")) {
if(multisetSeen) {
Assert.fail("already seen");
}
multisetSeen = true;
- Assert.assertTrue(field.type instanceof GenericTypeInfo);
- Assert.assertEquals(HashMultiset.class, field.type.getTypeClass());
+ Assert.assertTrue(field.getTypeInformation() instanceof GenericTypeInfo);
+ Assert.assertEquals(HashMultiset.class, field.getTypeInformation().getTypeClass());
} else if(name.equals("fancyArray")) {
if(strArraySeen) {
Assert.fail("already seen");
}
strArraySeen = true;
- Assert.assertEquals(BasicArrayTypeInfo.STRING_ARRAY_TYPE_INFO, field.type);
- Assert.assertEquals(String[].class, field.type.getTypeClass());
+ Assert.assertEquals(BasicArrayTypeInfo.STRING_ARRAY_TYPE_INFO, field.getTypeInformation());
+ Assert.assertEquals(String[].class, field.getTypeInformation().getTypeClass());
} else if(Arrays.asList("date", "someNumberWithÜnicödeNäme", "someFloat", "word", "nothing", "hadoopCitizen", "collection").contains(name)) {
// ignore these, they are inherited from the ComplexNestedClass
}
@@ -479,13 +479,13 @@ public class PojoTypeExtractionTest {
PojoTypeInfo<?> pojoTypeForClass = (PojoTypeInfo<?>) typeInformation;
for(int i = 0; i < pojoTypeForClass.getArity(); i++) {
PojoField field = pojoTypeForClass.getPojoFieldAt(i);
- String name = field.field.getName();
+ String name = field.getField().getName();
if(name.equals("special")) {
- Assert.assertEquals(BasicTypeInfo.INT_TYPE_INFO, field.type);
+ Assert.assertEquals(BasicTypeInfo.INT_TYPE_INFO, field.getTypeInformation());
} else if(name.equals("f0") || name.equals("f1")) {
- Assert.assertEquals(BasicTypeInfo.STRING_TYPE_INFO, field.type);
+ Assert.assertEquals(BasicTypeInfo.STRING_TYPE_INFO, field.getTypeInformation());
} else if(name.equals("f2")) {
- Assert.assertEquals(BasicTypeInfo.LONG_TYPE_INFO, field.type);
+ Assert.assertEquals(BasicTypeInfo.LONG_TYPE_INFO, field.getTypeInformation());
} else {
Assert.fail("unexpected field");
}
@@ -499,15 +499,15 @@ public class PojoTypeExtractionTest {
PojoTypeInfo<?> pojoTypeForClass = (PojoTypeInfo<?>) typeForClass;
for(int i = 0; i < pojoTypeForClass.getArity(); i++) {
PojoField field = pojoTypeForClass.getPojoFieldAt(i);
- String name = field.field.getName();
+ String name = field.getField().getName();
if(name.equals("field1")) {
- Assert.assertEquals(BasicTypeInfo.INT_TYPE_INFO, field.type);
+ Assert.assertEquals(BasicTypeInfo.INT_TYPE_INFO, field.getTypeInformation());
} else if (name.equals("field2")) {
- Assert.assertEquals(BasicTypeInfo.LONG_TYPE_INFO, field.type);
+ Assert.assertEquals(BasicTypeInfo.LONG_TYPE_INFO, field.getTypeInformation());
} else if (name.equals("field3")) {
- Assert.assertEquals(BasicTypeInfo.STRING_TYPE_INFO, field.type);
+ Assert.assertEquals(BasicTypeInfo.STRING_TYPE_INFO, field.getTypeInformation());
} else if (name.equals("key")) {
- Assert.assertEquals(BasicTypeInfo.INT_TYPE_INFO, field.type);
+ Assert.assertEquals(BasicTypeInfo.INT_TYPE_INFO, field.getTypeInformation());
} else {
Assert.fail("Unexpected field "+field);
}
@@ -525,13 +525,13 @@ public class PojoTypeExtractionTest {
PojoTypeInfo<?> pojoTypeForClass = (PojoTypeInfo<?>) typeForClass;
for(int i = 0; i < pojoTypeForClass.getArity(); i++) {
PojoField field = pojoTypeForClass.getPojoFieldAt(i);
- String name = field.field.getName();
+ String name = field.getField().getName();
if(name.equals("field1")) {
- Assert.assertEquals(new GenericTypeInfo<Object>(Object.class), field.type);
+ Assert.assertEquals(new GenericTypeInfo<Object>(Object.class), field.getTypeInformation());
} else if (name.equals("field2")) {
- Assert.assertEquals(new GenericTypeInfo<Object>(Object.class), field.type);
+ Assert.assertEquals(new GenericTypeInfo<Object>(Object.class), field.getTypeInformation());
} else if (name.equals("key")) {
- Assert.assertEquals(BasicTypeInfo.INT_TYPE_INFO, field.type);
+ Assert.assertEquals(BasicTypeInfo.INT_TYPE_INFO, field.getTypeInformation());
} else {
Assert.fail("Unexpected field "+field);
}
@@ -546,14 +546,14 @@ public class PojoTypeExtractionTest {
PojoTypeInfo<?> pojoTypeForClass = (PojoTypeInfo<?>) typeForClass;
for(int i = 0; i < pojoTypeForClass.getArity(); i++) {
PojoField field = pojoTypeForClass.getPojoFieldAt(i);
- String name = field.field.getName();
+ String name = field.getField().getName();
if(name.equals("field1")) {
- Assert.assertTrue(field.type instanceof PojoTypeInfo<?>); // From tuple is pojo (not tuple type!)
+ Assert.assertTrue(field.getTypeInformation() instanceof PojoTypeInfo<?>); // From tuple is pojo (not tuple type!)
} else if (name.equals("field2")) {
- Assert.assertTrue(field.type instanceof TupleTypeInfo<?>);
- Assert.assertTrue( ((TupleTypeInfo<?>)field.type).getTypeAt(0).equals(BasicTypeInfo.STRING_TYPE_INFO) );
+ Assert.assertTrue(field.getTypeInformation() instanceof TupleTypeInfo<?>);
+ Assert.assertTrue( ((TupleTypeInfo<?>)field.getTypeInformation()).getTypeAt(0).equals(BasicTypeInfo.STRING_TYPE_INFO) );
} else if (name.equals("key")) {
- Assert.assertEquals(BasicTypeInfo.INT_TYPE_INFO, field.type);
+ Assert.assertEquals(BasicTypeInfo.INT_TYPE_INFO, field.getTypeInformation());
} else {
Assert.fail("Unexpected field "+field);
}
@@ -641,13 +641,13 @@ public class PojoTypeExtractionTest {
PojoTypeInfo<?> pti = (PojoTypeInfo<?>) ti;
for(int i = 0; i < pti.getArity(); i++) {
PojoField field = pti.getPojoFieldAt(i);
- String name = field.field.getName();
+ String name = field.getField().getName();
if(name.equals("field1")) {
- Assert.assertEquals(BasicTypeInfo.STRING_TYPE_INFO, field.type);
+ Assert.assertEquals(BasicTypeInfo.STRING_TYPE_INFO, field.getTypeInformation());
} else if (name.equals("field2")) {
- Assert.assertEquals(BasicTypeInfo.STRING_TYPE_INFO, field.type);
+ Assert.assertEquals(BasicTypeInfo.STRING_TYPE_INFO, field.getTypeInformation());
} else if (name.equals("key")) {
- Assert.assertEquals(BasicTypeInfo.INT_TYPE_INFO, field.type);
+ Assert.assertEquals(BasicTypeInfo.INT_TYPE_INFO, field.getTypeInformation());
} else {
Assert.fail("Unexpected field "+field);
}
@@ -680,15 +680,15 @@ public class PojoTypeExtractionTest {
PojoTypeInfo<?> pti = (PojoTypeInfo<?>) ti;
for(int i = 0; i < pti.getArity(); i++) {
PojoField field = pti.getPojoFieldAt(i);
- String name = field.field.getName();
+ String name = field.getField().getName();
if(name.equals("extraField")) {
- Assert.assertEquals(BasicTypeInfo.CHAR_TYPE_INFO, field.type);
+ Assert.assertEquals(BasicTypeInfo.CHAR_TYPE_INFO, field.getTypeInformation());
} else if (name.equals("f0")) {
- Assert.assertEquals(BasicTypeInfo.BOOLEAN_TYPE_INFO, field.type);
+ Assert.assertEquals(BasicTypeInfo.BOOLEAN_TYPE_INFO, field.getTypeInformation());
} else if (name.equals("f1")) {
- Assert.assertEquals(BasicTypeInfo.BOOLEAN_TYPE_INFO, field.type);
+ Assert.assertEquals(BasicTypeInfo.BOOLEAN_TYPE_INFO, field.getTypeInformation());
} else if (name.equals("f2")) {
- Assert.assertEquals(BasicTypeInfo.LONG_TYPE_INFO, field.type);
+ Assert.assertEquals(BasicTypeInfo.LONG_TYPE_INFO, field.getTypeInformation());
} else {
Assert.fail("Unexpected field "+field);
}
@@ -831,7 +831,7 @@ public class PojoTypeExtractionTest {
public void testRecursivePojo1() {
TypeInformation<?> ti = TypeExtractor.createTypeInfo(RecursivePojo1.class);
Assert.assertTrue(ti instanceof PojoTypeInfo);
- Assert.assertEquals(GenericTypeInfo.class, ((PojoTypeInfo) ti).getPojoFieldAt(0).type.getClass());
+ Assert.assertEquals(GenericTypeInfo.class, ((PojoTypeInfo) ti).getPojoFieldAt(0).getTypeInformation().getClass());
}
@Test
@@ -839,8 +839,8 @@ public class PojoTypeExtractionTest {
TypeInformation<?> ti = TypeExtractor.createTypeInfo(RecursivePojo2.class);
Assert.assertTrue(ti instanceof PojoTypeInfo);
PojoField pf = ((PojoTypeInfo) ti).getPojoFieldAt(0);
- Assert.assertTrue(pf.type instanceof TupleTypeInfo);
- Assert.assertEquals(GenericTypeInfo.class, ((TupleTypeInfo) pf.type).getTypeAt(0).getClass());
+ Assert.assertTrue(pf.getTypeInformation() instanceof TupleTypeInfo);
+ Assert.assertEquals(GenericTypeInfo.class, ((TupleTypeInfo) pf.getTypeInformation()).getTypeAt(0).getClass());
}
@Test
@@ -848,8 +848,8 @@ public class PojoTypeExtractionTest {
TypeInformation<?> ti = TypeExtractor.createTypeInfo(RecursivePojo3.class);
Assert.assertTrue(ti instanceof PojoTypeInfo);
PojoField pf = ((PojoTypeInfo) ti).getPojoFieldAt(0);
- Assert.assertTrue(pf.type instanceof PojoTypeInfo);
- Assert.assertEquals(GenericTypeInfo.class, ((PojoTypeInfo) pf.type).getPojoFieldAt(0).type.getClass());
+ Assert.assertTrue(pf.getTypeInformation() instanceof PojoTypeInfo);
+ Assert.assertEquals(GenericTypeInfo.class, ((PojoTypeInfo) pf.getTypeInformation()).getPojoFieldAt(0).getTypeInformation().getClass());
}
public static class FooBarPojo {
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java
index d27a82b..7f0cf5b 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java
@@ -66,6 +66,8 @@ import org.apache.hadoop.io.Writable;
import org.junit.Assert;
import org.junit.Test;
+import javax.xml.bind.TypeConstraintException;
+
public class TypeExtractorTest {
@@ -1237,7 +1239,7 @@ public class TypeExtractorTest {
TypeInformation<?> ti = TypeExtractor.getMapReturnTypes(function, (TypeInformation) TypeInfoParser.parse("org.apache.flink.api.java.type.extractor.TypeExtractorTest$CustomArrayObject[]"));
Assert.assertTrue(ti instanceof ObjectArrayTypeInfo<?, ?>);
- Assert.assertEquals(CustomArrayObject.class, ((ObjectArrayTypeInfo<?, ?>) ti).getComponentType());
+ Assert.assertEquals(CustomArrayObject.class, ((ObjectArrayTypeInfo<?, ?>) ti).getComponentInfo().getTypeClass());
}
@SuppressWarnings({ "rawtypes", "unchecked" })
[3/9] flink git commit: [FLINK-2637] [api-breaking] [types] Adds
equals and hashCode method to TypeInformations and TypeSerializers - Fixes
ObjectArrayTypeInfo - Makes CompositeTypes serializable - Adds test for
equality relation's symmetric property - A
Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/StreamWindowTypeInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/StreamWindowTypeInfoTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/StreamWindowTypeInfoTest.java
new file mode 100644
index 0000000..2e98a8f
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/windowing/StreamWindowTypeInfoTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.streaming.api.windowing;
+
+import org.apache.flink.api.java.typeutils.GenericTypeInfo;
+import org.apache.flink.api.java.typeutils.PojoField;
+import org.apache.flink.api.java.typeutils.PojoTypeInfo;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+
+import java.util.ArrayList;
+
+import static org.junit.Assert.*;
+
+public class StreamWindowTypeInfoTest extends TestLogger {
+
+ public static class TestClass{}
+
+ @Test
+ public void testStreamWindowTypeInfoEquality() {
+ StreamWindowTypeInfo<TestClass> tpeInfo1 = new StreamWindowTypeInfo<>(new GenericTypeInfo<>(TestClass.class));
+ StreamWindowTypeInfo<TestClass> tpeInfo2 = new StreamWindowTypeInfo<>(new GenericTypeInfo<>(TestClass.class));
+
+ assertEquals(tpeInfo1, tpeInfo2);
+ assertEquals(tpeInfo1.hashCode(), tpeInfo2.hashCode());
+ }
+
+ @Test
+ public void testStreamWindowTypeInfoInequality() {
+ StreamWindowTypeInfo<TestClass> tpeInfo1 = new StreamWindowTypeInfo<>(new GenericTypeInfo<>(TestClass.class));
+ StreamWindowTypeInfo<TestClass> tpeInfo2 = new StreamWindowTypeInfo<>(new PojoTypeInfo<>(TestClass.class, new ArrayList<PojoField>()));
+
+ assertNotEquals(tpeInfo1, tpeInfo2);
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-staging/flink-table/src/main/scala/org/apache/flink/api/table/typeinfo/RenamingProxyTypeInfo.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-table/src/main/scala/org/apache/flink/api/table/typeinfo/RenamingProxyTypeInfo.scala b/flink-staging/flink-table/src/main/scala/org/apache/flink/api/table/typeinfo/RenamingProxyTypeInfo.scala
index 6a9cbfe..dd598ab 100644
--- a/flink-staging/flink-table/src/main/scala/org/apache/flink/api/table/typeinfo/RenamingProxyTypeInfo.scala
+++ b/flink-staging/flink-table/src/main/scala/org/apache/flink/api/table/typeinfo/RenamingProxyTypeInfo.scala
@@ -21,8 +21,9 @@ import java.util
import org.apache.flink.api.common.ExecutionConfig
import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.common.typeutils.CompositeType.FlatFieldDescriptor
-import org.apache.flink.api.common.typeutils.{CompositeType, TypeComparator, TypeSerializer}
+import org.apache.flink.api.common.typeutils.CompositeType.{TypeComparatorBuilder,
+FlatFieldDescriptor}
+import org.apache.flink.api.common.typeutils.{CompositeType, TypeSerializer}
/**
* A TypeInformation that is used to rename fields of an underlying CompositeType. This
@@ -30,8 +31,9 @@ import org.apache.flink.api.common.typeutils.{CompositeType, TypeComparator, Typ
* that does not get translated to a runtime operator.
*/
class RenamingProxyTypeInfo[T](
- tpe: CompositeType[T],
- fieldNames: Array[String]) extends CompositeType[T](tpe.getTypeClass) {
+ val tpe: CompositeType[T],
+ val fieldNames: Array[String])
+ extends CompositeType[T](tpe.getTypeClass) {
def getUnderlyingType: CompositeType[T] = tpe
@@ -86,16 +88,6 @@ class RenamingProxyTypeInfo[T](
executionConfig: ExecutionConfig) =
tpe.createComparator(logicalKeyFields, orders, logicalFieldOffset, executionConfig)
- // These are never called since we override create comparator
- override protected def initializeNewComparator(localKeyCount: Int): Unit =
- throw new RuntimeException("Cannot happen.")
-
- override protected def getNewComparator(executionConfig: ExecutionConfig): TypeComparator[T] =
- throw new RuntimeException("Cannot happen.")
-
- override protected def addCompareField(fieldId: Int, comparator: TypeComparator[_]): Unit =
- throw new RuntimeException("Cannot happen.")
-
override def getFlatFields(
fieldExpression: String,
offset: Int,
@@ -106,4 +98,27 @@ class RenamingProxyTypeInfo[T](
override def getTypeAt[X](fieldExpression: String): TypeInformation[X] = {
tpe.getTypeAt(fieldExpression)
}
+
+ override protected def createTypeComparatorBuilder(): TypeComparatorBuilder[T] = {
+ throw new RuntimeException("This method should never be called because createComparator is " +
+ "overwritten.")
+ }
+
+ override def equals(obj: Any): Boolean = {
+ obj match {
+ case renamingProxy: RenamingProxyTypeInfo[_] =>
+ renamingProxy.canEqual(this) &&
+ tpe.equals(renamingProxy.tpe) &&
+ fieldNames.sameElements(renamingProxy.fieldNames)
+ case _ => false
+ }
+ }
+
+ override def hashCode(): Int = {
+ 31 * tpe.hashCode() + util.Arrays.hashCode(fieldNames.asInstanceOf[Array[AnyRef]])
+ }
+
+ override def canEqual(obj: Any): Boolean = {
+ obj.isInstanceOf[RenamingProxyTypeInfo[_]]
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-staging/flink-table/src/main/scala/org/apache/flink/api/table/typeinfo/RowSerializer.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-table/src/main/scala/org/apache/flink/api/table/typeinfo/RowSerializer.scala b/flink-staging/flink-table/src/main/scala/org/apache/flink/api/table/typeinfo/RowSerializer.scala
index 527e2b4..02219c7 100644
--- a/flink-staging/flink-table/src/main/scala/org/apache/flink/api/table/typeinfo/RowSerializer.scala
+++ b/flink-staging/flink-table/src/main/scala/org/apache/flink/api/table/typeinfo/RowSerializer.scala
@@ -28,7 +28,7 @@ import org.apache.flink.core.memory.{DataInputView, DataOutputView}
/**
* Serializer for [[Row]].
*/
-class RowSerializer(fieldSerializers: Array[TypeSerializer[Any]])
+class RowSerializer(val fieldSerializers: Array[TypeSerializer[Any]])
extends TypeSerializer[Row] {
private def getFieldSerializers = fieldSerializers
@@ -150,9 +150,17 @@ class RowSerializer(fieldSerializers: Array[TypeSerializer[Any]])
override def equals(any: scala.Any): Boolean = {
any match {
case otherRS: RowSerializer =>
- val otherFieldSerializers = otherRS.getFieldSerializers.asInstanceOf[Array[AnyRef]]
- util.Arrays.deepEquals(fieldSerializers.asInstanceOf[Array[AnyRef]], otherFieldSerializers)
+ otherRS.canEqual(this) &&
+ fieldSerializers.sameElements(otherRS.fieldSerializers)
case _ => false
}
}
+
+ override def canEqual(obj: scala.Any): Boolean = {
+ obj.isInstanceOf[RowSerializer]
+ }
+
+ override def hashCode(): Int = {
+ util.Arrays.hashCode(fieldSerializers.asInstanceOf[Array[AnyRef]])
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-staging/flink-table/src/test/java/org/apache/flink/api/java/table/test/PojoGroupingITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-table/src/test/java/org/apache/flink/api/java/table/test/PojoGroupingITCase.java b/flink-staging/flink-table/src/test/java/org/apache/flink/api/java/table/test/PojoGroupingITCase.java
new file mode 100644
index 0000000..9c62a51
--- /dev/null
+++ b/flink-staging/flink-table/src/test/java/org/apache/flink/api/java/table/test/PojoGroupingITCase.java
@@ -0,0 +1,89 @@
+/*
+ * 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.table.test;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.flink.api.common.operators.Order;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.table.TableEnvironment;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.table.Table;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class PojoGroupingITCase extends MultipleProgramsTestBase {
+
+ public PojoGroupingITCase(TestExecutionMode mode) {
+ super(mode);
+ }
+
+ @Test
+ public void testPojoGrouping() throws Exception {
+ ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+ DataSet<Tuple3<String, Double, String>> data = env.fromElements(
+ new Tuple3<String, Double, String>("A", 23.0, "Z"),
+ new Tuple3<String, Double, String>("A", 24.0, "Y"),
+ new Tuple3<String, Double, String>("B", 1.0, "Z"));
+
+ TableEnvironment tableEnv = new TableEnvironment();
+
+ Table table = tableEnv
+ .fromDataSet(data, "groupMe, value, name")
+ .select("groupMe, value, name")
+ .where("groupMe != 'B'");
+
+ DataSet<MyPojo> myPojos = tableEnv.toDataSet(table, MyPojo.class);
+
+ DataSet<MyPojo> result = myPojos.groupBy("groupMe")
+ .sortGroup("value", Order.DESCENDING)
+ .first(1);
+ List<MyPojo> resultList = result.collect();
+
+ compareResultAsText(resultList, "A,24.0,Y");
+ }
+
+ public static class MyPojo implements Serializable {
+ private static final long serialVersionUID = 8741918940120107213L;
+
+ public String groupMe;
+ public double value;
+ public String name;
+
+ public MyPojo() {
+ // for serialization
+ }
+
+ public MyPojo(String groupMe, double value, String name) {
+ this.groupMe = groupMe;
+ this.value = value;
+ this.name = name;
+ }
+
+ @Override
+ public String toString() {
+ return groupMe + "," + value + "," + name;
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-staging/flink-table/src/test/scala/org/apache/flink/api/table/typeinfo/RenamingProxyTypeInfoTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-table/src/test/scala/org/apache/flink/api/table/typeinfo/RenamingProxyTypeInfoTest.scala b/flink-staging/flink-table/src/test/scala/org/apache/flink/api/table/typeinfo/RenamingProxyTypeInfoTest.scala
new file mode 100644
index 0000000..ef616a9
--- /dev/null
+++ b/flink-staging/flink-table/src/test/scala/org/apache/flink/api/table/typeinfo/RenamingProxyTypeInfoTest.scala
@@ -0,0 +1,75 @@
+/*
+ * 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.table.typeinfo
+
+import org.apache.flink.api.common.typeutils.CompositeType
+import org.apache.flink.api.java.typeutils.TypeExtractor
+import org.apache.flink.util.TestLogger
+import org.junit.Test
+import org.scalatest.junit.JUnitSuiteLike
+
+class RenamingProxyTypeInfoTest extends TestLogger with JUnitSuiteLike {
+
+ @Test
+ def testRenamingProxyTypeEquality(): Unit = {
+ val pojoTypeInfo1 = TypeExtractor.createTypeInfo(classOf[TestPojo])
+ .asInstanceOf[CompositeType[TestPojo]]
+
+ val tpeInfo1 = new RenamingProxyTypeInfo[TestPojo](
+ pojoTypeInfo1,
+ Array("someInt", "aString", "doubleArray"))
+
+ val tpeInfo2 = new RenamingProxyTypeInfo[TestPojo](
+ pojoTypeInfo1,
+ Array("someInt", "aString", "doubleArray"))
+
+ assert(tpeInfo1.equals(tpeInfo2))
+ assert(tpeInfo1.hashCode() == tpeInfo2.hashCode())
+ }
+
+ @Test
+ def testRenamingProxyTypeInequality(): Unit = {
+ val pojoTypeInfo1 = TypeExtractor.createTypeInfo(classOf[TestPojo])
+ .asInstanceOf[CompositeType[TestPojo]]
+
+ val tpeInfo1 = new RenamingProxyTypeInfo[TestPojo](
+ pojoTypeInfo1,
+ Array("someInt", "aString", "doubleArray"))
+
+ val tpeInfo2 = new RenamingProxyTypeInfo[TestPojo](
+ pojoTypeInfo1,
+ Array("foobar", "aString", "doubleArray"))
+
+ assert(!tpeInfo1.equals(tpeInfo2))
+ }
+}
+
+final class TestPojo {
+ var someInt: Int = 0
+ private var aString: String = null
+ var doubleArray: Array[Double] = null
+
+ def setaString(aString: String) {
+ this.aString = aString
+ }
+
+ def getaString: String = {
+ return aString
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/UnionITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/UnionITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/UnionITCase.scala
index cab6ab7..bc53a60 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/UnionITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/UnionITCase.scala
@@ -18,40 +18,35 @@
package org.apache.flink.api.scala.operators
import org.apache.flink.api.scala.util.CollectionDataSets
-import org.apache.flink.core.fs.FileSystem.WriteMode
import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
import org.apache.flink.test.util.{TestBaseUtils, MultipleProgramsTestBase}
import org.junit._
-import org.junit.rules.TemporaryFolder
import org.junit.runner.RunWith
import org.junit.runners.Parameterized
import org.apache.flink.api.scala._
+import scala.collection.mutable.ArrayBuffer
+
@RunWith(classOf[Parameterized])
class UnionITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) {
- private var resultPath: String = null
+ private var result: Seq[String] = null
private var expected: String = null
- private val _tempFolder = new TemporaryFolder()
-
- private final val FULL_TUPLE_3_STRING: String = "1,1,Hi\n" + "2,2,Hello\n" + "3,2," +
- "Hello world\n" + "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3," +
- "Luke Skywalker\n" + "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4," +
- "Comment#4\n" + "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5," +
- "Comment#8\n" + "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6," +
- "Comment#12\n" + "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"
- @Rule
- def tempFolder = _tempFolder
+ private final val FULL_TUPLE_3_STRING: String = "(1,1,Hi)\n" + "(2,2,Hello)\n" + "(3,2," +
+ "Hello world)\n" + "(4,3,Hello world, how are you?)\n" + "(5,3,I am fine.)\n" + "(6,3," +
+ "Luke Skywalker)\n" + "(7,4,Comment#1)\n" + "(8,4,Comment#2)\n" + "(9,4,Comment#3)\n" +
+ "(10,4," +
+ "Comment#4)\n" + "(11,5,Comment#5)\n" + "(12,5,Comment#6)\n" + "(13,5,Comment#7)\n" + "(14,5," +
+ "Comment#8)\n" + "(15,5,Comment#9)\n" + "(16,6,Comment#10)\n" + "(17,6,Comment#11)\n" +
+ "(18,6," +
+ "Comment#12)\n" + "(19,6,Comment#13)\n" + "(20,6,Comment#14)\n" + "(21,6,Comment#15)\n"
- @Before
- def before(): Unit = {
- resultPath = tempFolder.newFile().toURI.toString
- }
@After
def after(): Unit = {
- TestBaseUtils.compareResultsByLinesInMemory(expected, resultPath)
+ import collection.JavaConverters._
+ TestBaseUtils.compareResultAsText(ArrayBuffer(result: _*).asJava, expected)
}
@Test
@@ -62,8 +57,7 @@ class UnionITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode
val env = ExecutionEnvironment.getExecutionEnvironment
val ds = CollectionDataSets.get3TupleDataSet(env)
val unionDs = ds.union(CollectionDataSets.get3TupleDataSet(env))
- unionDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE)
- env.execute()
+ result = unionDs.collect().map(_.toString)
expected = FULL_TUPLE_3_STRING + FULL_TUPLE_3_STRING
}
@@ -79,8 +73,7 @@ class UnionITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode
.union(CollectionDataSets.get3TupleDataSet(env))
.union(CollectionDataSets.get3TupleDataSet(env))
.union(CollectionDataSets.get3TupleDataSet(env))
- unionDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE)
- env.execute()
+ result = unionDs.collect().map(_.toString)
expected = FULL_TUPLE_3_STRING + FULL_TUPLE_3_STRING + FULL_TUPLE_3_STRING +
FULL_TUPLE_3_STRING + FULL_TUPLE_3_STRING
}
@@ -94,8 +87,21 @@ class UnionITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode
// Don't know how to make an empty result in an other way than filtering it
val empty = CollectionDataSets.get3TupleDataSet(env).filter( t => false )
val unionDs = CollectionDataSets.get3TupleDataSet(env).union(empty)
- unionDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE)
- env.execute()
+ result = unionDs.collect().map(_.toString())
expected = FULL_TUPLE_3_STRING
}
+
+ @Test
+ def testUnionWithOptionType(): Unit = {
+ /*
+ * Union of a tuple with an Option field
+ */
+ val env = ExecutionEnvironment.getExecutionEnvironment
+ val data = Seq((Some(1), 1), (None, -1), (Some(42), 42))
+ val input1 = env.fromCollection(data)
+ val input2 = env.fromCollection(data)
+
+ result = input1.union(input2).collect().map(_.toString())
+ expected = data ++ data mkString("\n")
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-tests/src/test/scala/org/apache/flink/api/scala/types/TypeInformationGenTest.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/types/TypeInformationGenTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/types/TypeInformationGenTest.scala
index 8460bbc..43c35f9 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/types/TypeInformationGenTest.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/types/TypeInformationGenTest.scala
@@ -340,7 +340,7 @@ class TypeInformationGenTest {
Assert.assertTrue(ti.isInstanceOf[ObjectArrayTypeInfo[_, _]])
Assert.assertEquals(
classOf[CustomType],
- ti.asInstanceOf[ObjectArrayTypeInfo[_, _]].getComponentType)
+ ti.asInstanceOf[ObjectArrayTypeInfo[_, _]].getComponentInfo.getTypeClass)
}
@Test
[2/9] flink git commit: [FLINK-2640] [yarn] integrate off-heap
configuration
Posted by fh...@apache.org.
[FLINK-2640] [yarn] integrate off-heap configuration
This closes #1132
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/93c95b6a
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/93c95b6a
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/93c95b6a
Branch: refs/heads/master
Commit: 93c95b6a6f150a2c55dc387e4ef1d603b3ef3f22
Parents: 8ca853e
Author: Maximilian Michels <mx...@apache.org>
Authored: Tue Sep 15 11:04:34 2015 +0200
Committer: Fabian Hueske <fh...@apache.org>
Committed: Thu Sep 17 11:19:33 2015 +0200
----------------------------------------------------------------------
.../flink/yarn/YARNSessionFIFOITCase.java | 13 +++--
.../apache/flink/yarn/ApplicationMaster.scala | 5 +-
.../flink/yarn/ApplicationMasterActor.scala | 55 ++++++++++++++++++--
3 files changed, 61 insertions(+), 12 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/93c95b6a/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
index 6f07d36..3d381a6 100644
--- a/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
+++ b/flink-yarn-tests/src/main/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
@@ -546,12 +546,15 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
});
Assert.assertNotNull("Unable to locate JobManager log", jobmanagerLog);
content = FileUtils.readFileToString(jobmanagerLog);
- // expecting 512 mb, because TM was started with 1024, we cut off 50% (NOT THE DEFAULT VALUE).
- Assert.assertTrue("Expected string 'Starting TM with command=$JAVA_HOME/bin/java -Xms424m -Xmx424m' not found in JobManager log: '"+jobmanagerLog+"'",
- content.contains("Starting TM with command=$JAVA_HOME/bin/java -Xms424m -Xmx424m"));
- Assert.assertTrue("Expected string ' (2/2) (attempt #0) to ' not found in JobManager log." +
+ // TM was started with 1024 but we cut off 50% (NOT THE DEFAULT VALUE) and then divide
+ // between heap and off-heap memory (see {@link ApplicationMasterActor}).
+ String expected = "Starting TM with command=$JAVA_HOME/bin/java -Xms359m -Xmx359m -XX:MaxDirectMemorySize=65m";
+ Assert.assertTrue("Expected string '" + expected + "' not found in JobManager log: '"+jobmanagerLog+"'",
+ content.contains(expected));
+ expected = " (2/2) (attempt #0) to ";
+ Assert.assertTrue("Expected string '" + expected + "' not found in JobManager log." +
"This string checks that the job has been started with a parallelism of 2. Log contents: '"+jobmanagerLog+"'",
- content.contains(" (2/2) (attempt #0) to "));
+ content.contains(expected));
// make sure the detached app is really finished.
LOG.info("Checking again that app has finished");
http://git-wip-us.apache.org/repos/asf/flink/blob/93c95b6a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala
index 1d1db7e..869c643 100644
--- a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala
+++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMaster.scala
@@ -26,21 +26,20 @@ import org.apache.flink.client.CliFrontend
import org.apache.flink.configuration.{GlobalConfiguration, Configuration, ConfigConstants}
import org.apache.flink.runtime.StreamingMode
import org.apache.flink.runtime.akka.AkkaUtils
-import org.apache.flink.runtime.instance.AkkaActorGateway
import org.apache.flink.runtime.jobmanager.JobManager
import org.apache.flink.runtime.jobmanager.web.WebInfoServer
-import org.apache.flink.runtime.util.{StandaloneUtils, LeaderRetrievalUtils, EnvironmentInformation}
+import org.apache.flink.runtime.util.{StandaloneUtils, EnvironmentInformation}
import org.apache.flink.runtime.webmonitor.WebMonitor
import org.apache.flink.yarn.Messages.StartYarnSession
import org.apache.hadoop.security.UserGroupInformation
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
import org.apache.hadoop.yarn.conf.YarnConfiguration
+import scala.collection.JavaConversions._
import scala.io.Source
object ApplicationMaster {
- import scala.collection.JavaConversions._
val LOG = Logger(getClass)
http://git-wip-us.apache.org/repos/asf/flink/blob/93c95b6a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMasterActor.scala
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMasterActor.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMasterActor.scala
index e99f8d2..4af4bcc 100644
--- a/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMasterActor.scala
+++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/ApplicationMasterActor.scala
@@ -405,7 +405,8 @@ trait ApplicationMasterActor extends FlinkActor {
Try {
log.info("Start yarn session.")
memoryPerTaskManager = env.get(FlinkYarnClient.ENV_TM_MEMORY).toInt
- val heapLimit = Utils.calculateHeapSize(memoryPerTaskManager, flinkConfiguration)
+
+ val memoryLimit = Utils.calculateHeapSize(memoryPerTaskManager, flinkConfiguration)
val applicationMasterHost = env.get(Environment.NM_HOST.key)
require(applicationMasterHost != null, s"Application master (${Environment.NM_HOST} not set.")
@@ -500,7 +501,7 @@ trait ApplicationMasterActor extends FlinkActor {
val hs = ApplicationMaster.hasStreamingMode(env)
containerLaunchContext = Some(
createContainerLaunchContext(
- heapLimit,
+ memoryLimit,
hasLogback,
hasLog4j,
yarnClientUsername,
@@ -550,7 +551,7 @@ trait ApplicationMasterActor extends FlinkActor {
}
private def createContainerLaunchContext(
- heapLimit: Int,
+ memoryLimit: Int,
hasLogback: Boolean,
hasLog4j: Boolean,
yarnClientUsername: String,
@@ -561,9 +562,11 @@ trait ApplicationMasterActor extends FlinkActor {
log.info("Create container launch context.")
val ctx = Records.newRecord(classOf[ContainerLaunchContext])
+ val (heapLimit, offHeapLimit) = calculateMemoryLimits(memoryLimit, streamingMode)
+
val javaOpts = flinkConfiguration.getString(ConfigConstants.FLINK_JVM_OPTIONS, "")
val tmCommand = new StringBuilder(s"$$JAVA_HOME/bin/java -Xms${heapLimit}m " +
- s"-Xmx${heapLimit}m $javaOpts")
+ s"-Xmx${heapLimit}m -XX:MaxDirectMemorySize=${offHeapLimit}m $javaOpts")
if (hasLogback || hasLog4j) {
tmCommand ++=
@@ -616,4 +619,48 @@ trait ApplicationMasterActor extends FlinkActor {
ctx
}
+
+ /**
+ * Calculate the correct JVM heap and off-heap memory limits.
+ * @param memoryLimit The maximum memory in megabytes.
+ * @param streamingMode True if this is a streaming cluster.
+ * @return A Tuple2 containing the heap and the offHeap limit in megabytes.
+ */
+ private def calculateMemoryLimits(memoryLimit: Long, streamingMode: Boolean): (Long, Long) = {
+
+ // The new config entry overrides the old one
+ val networkBufferSizeOld = flinkConfiguration.getLong(
+ ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY,
+ ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE)
+
+ val networkBufferSize = flinkConfiguration.getLong(
+ ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
+ networkBufferSizeOld)
+
+ val numNetworkBuffers = flinkConfiguration.getLong(
+ ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
+ ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS)
+
+ // direct memory for Netty's off-heap buffers
+ val networkMemory = ((numNetworkBuffers * networkBufferSize) >> 20) + 1
+
+ val useOffHeap = flinkConfiguration.getBoolean(
+ ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)
+
+ if (useOffHeap && !streamingMode){
+ val fixedOffHeapSize = flinkConfiguration.getLong(
+ ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L)
+ if (fixedOffHeapSize > 0) {
+ (memoryLimit - fixedOffHeapSize - networkMemory, fixedOffHeapSize + networkMemory)
+ } else {
+ val fraction = flinkConfiguration.getFloat(
+ ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
+ ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION)
+ val offHeapSize = (fraction * memoryLimit).toLong
+ (memoryLimit - offHeapSize - networkMemory, offHeapSize + networkMemory)
+ }
+ } else {
+ (memoryLimit - networkMemory, networkMemory)
+ }
+ }
}
[9/9] flink git commit: [FLINK-2691] [documentation] Fix broken links
to Python script on QuickStart docs
Posted by fh...@apache.org.
[FLINK-2691] [documentation] Fix broken links to Python script on QuickStart docs
This closes #1140
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/77989d3c
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/77989d3c
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/77989d3c
Branch: refs/heads/master
Commit: 77989d3cb2dd8a5513f5bacafc0e5e7d6f8278e8
Parents: c269158
Author: Felix Cheung <fe...@hotmail.com>
Authored: Wed Sep 16 17:00:39 2015 -0700
Committer: Fabian Hueske <fh...@apache.org>
Committed: Thu Sep 17 11:28:13 2015 +0200
----------------------------------------------------------------------
docs/quickstart/run_example_quickstart.md | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/77989d3c/docs/quickstart/run_example_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/quickstart/run_example_quickstart.md b/docs/quickstart/run_example_quickstart.md
index 9d03095..4493812 100644
--- a/docs/quickstart/run_example_quickstart.md
+++ b/docs/quickstart/run_example_quickstart.md
@@ -53,7 +53,7 @@ The `kmeans/` directory should now contain two files: `centers` and `points`. Th
## Inspect the Input Data
-Use the `plotPoints.py` tool to review the generated data points. [Download Python Script](quickstart/plotPoints.py)
+Use the `plotPoints.py` tool to review the generated data points. [Download Python Script](plotPoints.py)
~~~ bash
python plotPoints.py points ./points input
@@ -145,7 +145,7 @@ Stop Flink when you are done.
~~~
## Analyze the Result
-Use the [Python Script](quickstart/plotPoints.py) again to visualize the result.
+Use the [Python Script](plotPoints.py) again to visualize the result.
~~~bash
cd kmeans
[6/9] flink git commit: [FLINK-2637] [api-breaking] [types] Adds
equals and hashCode method to TypeInformations and TypeSerializers - Fixes
ObjectArrayTypeInfo - Makes CompositeTypes serializable - Adds test for
equality relation's symmetric property - A
Posted by fh...@apache.org.
[FLINK-2637] [api-breaking] [types] Adds equals and hashCode method to TypeInformations and TypeSerializers
- Fixes ObjectArrayTypeInfo
- Makes CompositeTypes serializable
- Adds test for equality relation's symmetric property
- Adds test for PojoTypeInfo serializability
This closes #1134
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/8ca853e0
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/8ca853e0
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/8ca853e0
Branch: refs/heads/master
Commit: 8ca853e0f6c18be8e6b066c6ec0f23badb797323
Parents: 2c9e2c8
Author: Till Rohrmann <tr...@apache.org>
Authored: Tue Sep 8 01:12:09 2015 +0200
Committer: Fabian Hueske <fh...@apache.org>
Committed: Thu Sep 17 11:19:33 2015 +0200
----------------------------------------------------------------------
.../flink/api/common/ExecutionConfig.java | 56 ++++++
.../api/common/typeinfo/BasicArrayTypeInfo.java | 44 ++++-
.../api/common/typeinfo/BasicTypeInfo.java | 29 ++-
.../api/common/typeinfo/FractionalTypeInfo.java | 16 +-
.../api/common/typeinfo/IntegerTypeInfo.java | 19 +-
.../api/common/typeinfo/NothingTypeInfo.java | 26 +++
.../api/common/typeinfo/NumericTypeInfo.java | 23 ++-
.../common/typeinfo/PrimitiveArrayTypeInfo.java | 45 +++--
.../api/common/typeinfo/TypeInformation.java | 17 ++
.../api/common/typeutils/CompositeType.java | 130 +++++++++----
.../api/common/typeutils/TypeSerializer.java | 12 ++
.../typeutils/base/BooleanSerializer.java | 5 +
.../typeutils/base/BooleanValueSerializer.java | 5 +
.../common/typeutils/base/ByteSerializer.java | 5 +
.../typeutils/base/ByteValueSerializer.java | 5 +
.../common/typeutils/base/CharSerializer.java | 5 +
.../typeutils/base/CharValueSerializer.java | 5 +
.../common/typeutils/base/DateSerializer.java | 5 +
.../common/typeutils/base/DoubleSerializer.java | 5 +
.../typeutils/base/DoubleValueSerializer.java | 5 +
.../common/typeutils/base/EnumSerializer.java | 16 +-
.../common/typeutils/base/FloatSerializer.java | 5 +
.../typeutils/base/FloatValueSerializer.java | 5 +
.../typeutils/base/GenericArraySerializer.java | 35 ++--
.../common/typeutils/base/IntSerializer.java | 5 +
.../typeutils/base/IntValueSerializer.java | 5 +
.../common/typeutils/base/LongSerializer.java | 5 +
.../typeutils/base/LongValueSerializer.java | 5 +
.../common/typeutils/base/ShortSerializer.java | 5 +
.../typeutils/base/ShortValueSerializer.java | 5 +
.../common/typeutils/base/StringSerializer.java | 5 +
.../typeutils/base/StringValueSerializer.java | 5 +
.../typeutils/base/TypeSerializerSingleton.java | 10 +-
.../common/typeutils/base/VoidSerializer.java | 5 +
.../array/BooleanPrimitiveArraySerializer.java | 5 +
.../array/BytePrimitiveArraySerializer.java | 5 +
.../array/CharPrimitiveArraySerializer.java | 5 +
.../array/DoublePrimitiveArraySerializer.java | 5 +
.../array/FloatPrimitiveArraySerializer.java | 5 +
.../base/array/IntPrimitiveArraySerializer.java | 5 +
.../array/LongPrimitiveArraySerializer.java | 5 +
.../array/ShortPrimitiveArraySerializer.java | 5 +
.../base/array/StringArraySerializer.java | 5 +
.../typeutils/record/RecordSerializer.java | 22 ++-
.../flink/types/BasicArrayTypeInfoTest.java | 56 ++++++
.../apache/flink/types/BasicTypeInfoTest.java | 58 ++++++
.../apache/flink/types/NothingTypeInfoTest.java | 47 +++++
.../flink/types/PrimitiveArrayTypeInfoTest.java | 56 ++++++
.../org/apache/flink/api/java/io/CsvReader.java | 2 +-
.../flink/api/java/sca/UdfAnalyzerUtils.java | 2 +-
.../flink/api/java/typeutils/AvroTypeInfo.java | 4 +-
.../flink/api/java/typeutils/EnumTypeInfo.java | 22 ++-
.../api/java/typeutils/GenericTypeInfo.java | 17 +-
.../api/java/typeutils/MissingTypeInfo.java | 32 +++-
.../api/java/typeutils/ObjectArrayTypeInfo.java | 114 ++++-------
.../flink/api/java/typeutils/PojoField.java | 42 +++-
.../flink/api/java/typeutils/PojoTypeInfo.java | 190 +++++++++++--------
.../api/java/typeutils/RecordTypeInfo.java | 16 +-
.../flink/api/java/typeutils/TupleTypeInfo.java | 115 ++++++-----
.../api/java/typeutils/TupleTypeInfoBase.java | 116 +++++------
.../flink/api/java/typeutils/TypeExtractor.java | 41 +++-
.../flink/api/java/typeutils/ValueTypeInfo.java | 32 ++--
.../api/java/typeutils/WritableTypeInfo.java | 31 ++-
.../java/typeutils/runtime/AvroSerializer.java | 26 ++-
.../runtime/CopyableValueSerializer.java | 19 +-
.../java/typeutils/runtime/PojoSerializer.java | 59 +++---
.../typeutils/runtime/Tuple0Serializer.java | 16 +-
.../typeutils/runtime/TupleSerializerBase.java | 34 ++--
.../java/typeutils/runtime/ValueSerializer.java | 19 +-
.../typeutils/runtime/WritableSerializer.java | 12 +-
.../typeutils/runtime/kryo/KryoSerializer.java | 30 ++-
.../api/java/io/CollectionInputFormatTest.java | 26 ++-
.../type/extractor/PojoTypeExtractionTest.java | 108 +++++------
.../java/type/extractor/TypeExtractorTest.java | 4 +-
.../api/java/typeutils/EnumTypeInfoTest.java | 51 +++++
.../api/java/typeutils/GenericTypeInfoTest.java | 47 +++++
.../api/java/typeutils/MissingTypeInfoTest.java | 47 +++++
.../java/typeutils/ObjectArrayTypeInfoTest.java | 58 ++++++
.../api/java/typeutils/PojoTypeInfoTest.java | 52 ++++-
.../api/java/typeutils/RecordTypeInfoTest.java | 44 +++++
.../api/java/typeutils/TupleTypeInfoTest.java | 96 ++++++++++
.../api/java/typeutils/TypeInfoParserTest.java | 24 +--
.../api/java/typeutils/ValueTypeInfoTest.java | 87 +++++++++
.../java/typeutils/WritableTypeInfoTest.java | 74 ++++++++
.../testutils/types/IntListSerializer.java | 21 ++
.../testutils/types/IntPairSerializer.java | 21 ++
.../testutils/types/StringPairSerializer.java | 21 ++
.../flink/api/scala/ExecutionEnvironment.scala | 2 +-
.../api/scala/codegen/TypeInformationGen.scala | 2 +-
.../scala/typeutils/CaseClassSerializer.scala | 15 +-
.../api/scala/typeutils/CaseClassTypeInfo.scala | 186 ++++++++++--------
.../api/scala/typeutils/EitherSerializer.scala | 19 +-
.../api/scala/typeutils/EitherTypeInfo.scala | 25 ++-
.../scala/typeutils/EnumValueSerializer.scala | 13 +-
.../api/scala/typeutils/EnumValueTypeInfo.scala | 20 +-
.../api/scala/typeutils/NothingSerializer.scala | 13 +-
.../api/scala/typeutils/OptionSerializer.scala | 17 +-
.../api/scala/typeutils/OptionTypeInfo.scala | 18 +-
.../scala/typeutils/TraversableSerializer.scala | 17 +-
.../scala/typeutils/TraversableTypeInfo.scala | 24 ++-
.../api/scala/typeutils/TrySerializer.scala | 21 +-
.../flink/api/scala/typeutils/TryTypeInfo.scala | 19 +-
.../scala/typeutils/CaseClassTypeInfoTest.scala | 73 +++++++
.../scala/typeutils/EitherTypeInfoTest.scala | 59 ++++++
.../scala/typeutils/EnumValueTypeInfoTest.scala | 64 +++++++
.../scala/typeutils/OptionTypeInfoTest.scala | 54 ++++++
.../typeutils/TraversableTypeInfoTest.scala | 68 +++++++
.../api/scala/typeutils/TryTypeInfoTest.scala | 55 ++++++
.../api/windowing/StreamWindowSerializer.java | 24 +++
.../api/windowing/StreamWindowTypeInfo.java | 38 +++-
.../MultiplexingStreamRecordSerializer.java | 21 ++
.../streamrecord/StreamRecordSerializer.java | 21 ++
.../api/windowing/StreamWindowTypeInfoTest.java | 51 +++++
.../table/typeinfo/RenamingProxyTypeInfo.scala | 43 +++--
.../api/table/typeinfo/RowSerializer.scala | 14 +-
.../api/java/table/test/PojoGroupingITCase.java | 89 +++++++++
.../typeinfo/RenamingProxyTypeInfoTest.scala | 75 ++++++++
.../flink/api/scala/operators/UnionITCase.scala | 54 +++---
.../scala/types/TypeInformationGenTest.scala | 2 +-
119 files changed, 3120 insertions(+), 730 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
index 4ae0b8d..3233327 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
@@ -24,6 +24,7 @@ import java.io.Serializable;
import java.util.LinkedHashMap;
import java.util.LinkedHashSet;
import java.util.Map;
+import java.util.Objects;
/**
* A config to define the behavior of the program execution. It allows to define (among other
@@ -605,6 +606,61 @@ public class ExecutionConfig implements Serializable {
this.autoTypeRegistrationEnabled = false;
}
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof ExecutionConfig) {
+ ExecutionConfig other = (ExecutionConfig) obj;
+
+ return other.canEqual(this) &&
+ Objects.equals(executionMode, other.executionMode) &&
+ useClosureCleaner == other.useClosureCleaner &&
+ parallelism == other.parallelism &&
+ numberOfExecutionRetries == other.numberOfExecutionRetries &&
+ forceKryo == other.forceKryo &&
+ objectReuse == other.objectReuse &&
+ autoTypeRegistrationEnabled == other.autoTypeRegistrationEnabled &&
+ forceAvro == other.forceAvro &&
+ Objects.equals(codeAnalysisMode, other.codeAnalysisMode) &&
+ printProgressDuringExecution == other.printProgressDuringExecution &&
+ Objects.equals(globalJobParameters, other.globalJobParameters) &&
+ autoWatermarkInterval == other.autoWatermarkInterval &&
+ timestampsEnabled == other.timestampsEnabled &&
+ registeredTypesWithKryoSerializerClasses.equals(other.registeredTypesWithKryoSerializerClasses) &&
+ defaultKryoSerializerClasses.equals(other.defaultKryoSerializerClasses) &&
+ registeredKryoTypes.equals(other.registeredKryoTypes) &&
+ registeredPojoTypes.equals(other.registeredPojoTypes);
+
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(
+ executionMode,
+ useClosureCleaner,
+ parallelism,
+ numberOfExecutionRetries,
+ forceKryo,
+ objectReuse,
+ autoTypeRegistrationEnabled,
+ forceAvro,
+ codeAnalysisMode,
+ printProgressDuringExecution,
+ globalJobParameters,
+ autoWatermarkInterval,
+ timestampsEnabled,
+ registeredTypesWithKryoSerializerClasses,
+ defaultKryoSerializerClasses,
+ registeredKryoTypes,
+ registeredPojoTypes);
+ }
+
+ public boolean canEqual(Object obj) {
+ return obj instanceof ExecutionConfig;
+ }
+
// ------------------------------ Utilities ----------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BasicArrayTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BasicArrayTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BasicArrayTypeInfo.java
index 4fcb266..c72e8ed 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BasicArrayTypeInfo.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BasicArrayTypeInfo.java
@@ -20,14 +20,16 @@ package org.apache.flink.api.common.typeinfo;
import java.util.HashMap;
import java.util.Map;
+import java.util.Objects;
+import com.google.common.base.Preconditions;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.common.typeutils.base.array.StringArraySerializer;
import org.apache.flink.api.common.functions.InvalidTypesException;
import org.apache.flink.api.common.typeutils.base.GenericArraySerializer;
-public class BasicArrayTypeInfo<T, C> extends TypeInformation<T> {
+public final class BasicArrayTypeInfo<T, C> extends TypeInformation<T> {
private static final long serialVersionUID = 1L;
@@ -45,14 +47,12 @@ public class BasicArrayTypeInfo<T, C> extends TypeInformation<T> {
// --------------------------------------------------------------------------------------------
private final Class<T> arrayClass;
- private final Class<C> componentClass;
private final TypeInformation<C> componentInfo;
@SuppressWarnings("unchecked")
private BasicArrayTypeInfo(Class<T> arrayClass, BasicTypeInfo<C> componentInfo) {
- this.arrayClass = arrayClass;
- this.componentClass = (Class<C>) arrayClass.getComponentType();
- this.componentInfo = componentInfo;
+ this.arrayClass = Preconditions.checkNotNull(arrayClass);
+ this.componentInfo = Preconditions.checkNotNull(componentInfo);
}
// --------------------------------------------------------------------------------------------
@@ -83,7 +83,7 @@ public class BasicArrayTypeInfo<T, C> extends TypeInformation<T> {
}
public Class<C> getComponentTypeClass() {
- return this.componentClass;
+ return this.componentInfo.getTypeClass();
}
public TypeInformation<C> getComponentInfo() {
@@ -99,13 +99,39 @@ public class BasicArrayTypeInfo<T, C> extends TypeInformation<T> {
@SuppressWarnings("unchecked")
public TypeSerializer<T> createSerializer(ExecutionConfig executionConfig) {
// special case the string array
- if (componentClass.equals(String.class)) {
+ if (componentInfo.getTypeClass().equals(String.class)) {
return (TypeSerializer<T>) StringArraySerializer.INSTANCE;
} else {
- return (TypeSerializer<T>) new GenericArraySerializer<C>(this.componentClass, this.componentInfo.createSerializer(executionConfig));
+ return (TypeSerializer<T>) new GenericArraySerializer<C>(
+ this.componentInfo.getTypeClass(),
+ this.componentInfo.createSerializer(executionConfig));
}
}
-
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof BasicArrayTypeInfo) {
+ BasicArrayTypeInfo<?, ?> other = (BasicArrayTypeInfo<?, ?>) obj;
+
+ return other.canEqual(this) &&
+ arrayClass == other.arrayClass &&
+ componentInfo.equals(other.componentInfo);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(arrayClass, componentInfo);
+ }
+
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof BasicArrayTypeInfo;
+ }
+
@Override
public String toString() {
return this.getClass().getSimpleName()+"<"+this.componentInfo+">";
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BasicTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BasicTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BasicTypeInfo.java
index c622151..3e34644 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BasicTypeInfo.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BasicTypeInfo.java
@@ -19,10 +19,13 @@
package org.apache.flink.api.common.typeinfo;
import java.lang.reflect.Constructor;
+import java.util.Arrays;
import java.util.Date;
import java.util.HashMap;
import java.util.Map;
+import java.util.Objects;
+import com.google.common.base.Preconditions;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.functions.InvalidTypesException;
import org.apache.flink.api.common.typeutils.TypeComparator;
@@ -55,6 +58,8 @@ import org.apache.flink.api.common.typeutils.base.VoidSerializer;
*/
public class BasicTypeInfo<T> extends TypeInformation<T> implements AtomicType<T> {
+ private static final long serialVersionUID = -430955220409131770L;
+
public static final BasicTypeInfo<String> STRING_TYPE_INFO = new BasicTypeInfo<String>(String.class, new Class<?>[]{}, StringSerializer.INSTANCE, StringComparator.class);
public static final BasicTypeInfo<Boolean> BOOLEAN_TYPE_INFO = new BasicTypeInfo<Boolean>(Boolean.class, new Class<?>[]{}, BooleanSerializer.INSTANCE, BooleanComparator.class);
public static final BasicTypeInfo<Byte> BYTE_TYPE_INFO = new IntegerTypeInfo<Byte>(Byte.class, new Class<?>[]{Short.class, Integer.class, Long.class, Float.class, Double.class, Character.class}, ByteSerializer.INSTANCE, ByteComparator.class);
@@ -66,7 +71,7 @@ public class BasicTypeInfo<T> extends TypeInformation<T> implements AtomicType<T
public static final BasicTypeInfo<Character> CHAR_TYPE_INFO = new BasicTypeInfo<Character>(Character.class, new Class<?>[]{}, CharSerializer.INSTANCE, CharComparator.class);
public static final BasicTypeInfo<Date> DATE_TYPE_INFO = new BasicTypeInfo<Date>(Date.class, new Class<?>[]{}, DateSerializer.INSTANCE, DateComparator.class);
public static final BasicTypeInfo<Void> VOID_TYPE_INFO = new BasicTypeInfo<Void>(Void.class, new Class<?>[]{}, VoidSerializer.INSTANCE, null);
-
+
// --------------------------------------------------------------------------------------------
private final Class<T> clazz;
@@ -79,9 +84,10 @@ public class BasicTypeInfo<T> extends TypeInformation<T> implements AtomicType<T
protected BasicTypeInfo(Class<T> clazz, Class<?>[] possibleCastTargetTypes, TypeSerializer<T> serializer, Class<? extends TypeComparator<T>> comparatorClass) {
- this.clazz = clazz;
- this.possibleCastTargetTypes = possibleCastTargetTypes;
- this.serializer = serializer;
+ this.clazz = Preconditions.checkNotNull(clazz);
+ this.possibleCastTargetTypes = Preconditions.checkNotNull(possibleCastTargetTypes);
+ this.serializer = Preconditions.checkNotNull(serializer);
+ // comparator can be null as in VOID_TYPE_INFO
this.comparatorClass = comparatorClass;
}
@@ -148,15 +154,24 @@ public class BasicTypeInfo<T> extends TypeInformation<T> implements AtomicType<T
@Override
public int hashCode() {
- return this.clazz.hashCode();
+ return (31 * Objects.hash(clazz, serializer, comparatorClass)) + Arrays.hashCode(possibleCastTargetTypes);
}
-
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof BasicTypeInfo;
+ }
+
@Override
public boolean equals(Object obj) {
if (obj instanceof BasicTypeInfo) {
@SuppressWarnings("unchecked")
BasicTypeInfo<T> other = (BasicTypeInfo<T>) obj;
- return this.clazz.equals(other.clazz);
+
+ return other.canEqual(this) &&
+ this.clazz == other.clazz &&
+ serializer.equals(other.serializer) &&
+ this.comparatorClass == other.comparatorClass;
} else {
return false;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/FractionalTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/FractionalTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/FractionalTypeInfo.java
index 7e5e95d..f357e34 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/FractionalTypeInfo.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/FractionalTypeInfo.java
@@ -18,15 +18,29 @@
package org.apache.flink.api.common.typeinfo;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
import org.apache.flink.api.common.typeutils.TypeComparator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
+import java.util.Set;
+
/**
- * Type information for numeric primitive types (int, long, double, byte, ...).
+ * Type information for numeric fractional primitive types (double, float).
*/
public class FractionalTypeInfo<T> extends NumericTypeInfo<T> {
+ private static final long serialVersionUID = 554334260950199994L;
+
+ private static final Set<Class<?>> fractionalTypes = Sets.<Class<?>>newHashSet(
+ Double.class,
+ Float.class
+ );
+
protected FractionalTypeInfo(Class<T> clazz, Class<?>[] possibleCastTargetTypes, TypeSerializer<T> serializer, Class<? extends TypeComparator<T>> comparatorClass) {
super(clazz, possibleCastTargetTypes, serializer, comparatorClass);
+
+ Preconditions.checkArgument(fractionalTypes.contains(clazz), "The given class " +
+ clazz.getSimpleName() + " is not a fractional type.");
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/IntegerTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/IntegerTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/IntegerTypeInfo.java
index 5a7e304..7f9dd92 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/IntegerTypeInfo.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/IntegerTypeInfo.java
@@ -18,15 +18,32 @@
package org.apache.flink.api.common.typeinfo;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
import org.apache.flink.api.common.typeutils.TypeComparator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
+import java.util.Set;
+
/**
- * Type information for numeric primitive types (int, long, double, byte, ...).
+ * Type information for numeric integer primitive types: int, long, byte, short, character.
*/
public class IntegerTypeInfo<T> extends NumericTypeInfo<T> {
+ private static final long serialVersionUID = -8068827354966766955L;
+
+ private static final Set<Class<?>> integerTypes = Sets.<Class<?>>newHashSet(
+ Integer.class,
+ Long.class,
+ Byte.class,
+ Short.class,
+ Character.class
+ );
+
protected IntegerTypeInfo(Class<T> clazz, Class<?>[] possibleCastTargetTypes, TypeSerializer<T> serializer, Class<? extends TypeComparator<T>> comparatorClass) {
super(clazz, possibleCastTargetTypes, serializer, comparatorClass);
+
+ Preconditions.checkArgument(integerTypes.contains(clazz), "The given class " +
+ clazz.getSimpleName() + " is not a integer type.");
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/NothingTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/NothingTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/NothingTypeInfo.java
index 184bdff..a32c898 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/NothingTypeInfo.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/NothingTypeInfo.java
@@ -63,4 +63,30 @@ public class NothingTypeInfo extends TypeInformation<Nothing> {
public TypeSerializer<Nothing> createSerializer(ExecutionConfig executionConfig) {
throw new RuntimeException("The Nothing type cannot have a serializer.");
}
+
+ @Override
+ public String toString() {
+ return getClass().getSimpleName();
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof NothingTypeInfo) {
+ NothingTypeInfo nothingTypeInfo = (NothingTypeInfo) obj;
+
+ return nothingTypeInfo.canEqual(this);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return NothingTypeInfo.class.hashCode();
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof NothingTypeInfo;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/NumericTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/NumericTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/NumericTypeInfo.java
index 0f598f4..830e297 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/NumericTypeInfo.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/NumericTypeInfo.java
@@ -18,16 +18,35 @@
package org.apache.flink.api.common.typeinfo;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
import org.apache.flink.api.common.typeutils.TypeComparator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
+import java.util.Set;
+
/**
- * Type information for numeric primitive types (int, long, double, byte, ...).
+ * Type information for numeric primitive types: int, long, double, byte, short, float, char.
*/
-public class NumericTypeInfo<T> extends BasicTypeInfo<T> {
+public abstract class NumericTypeInfo<T> extends BasicTypeInfo<T> {
+
+ private static final long serialVersionUID = -5937777910658986986L;
+
+ private static final Set<Class<?>> numericalTypes = Sets.<Class<?>>newHashSet(
+ Integer.class,
+ Long.class,
+ Double.class,
+ Byte.class,
+ Short.class,
+ Float.class,
+ Character.class
+ );
protected NumericTypeInfo(Class<T> clazz, Class<?>[] possibleCastTargetTypes, TypeSerializer<T> serializer, Class<? extends
TypeComparator<T>> comparatorClass) {
super(clazz, possibleCastTargetTypes, serializer, comparatorClass);
+
+ Preconditions.checkArgument(numericalTypes.contains(clazz), "The given class " +
+ clazz.getSimpleName() + " is not a numerical type.");
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/PrimitiveArrayTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/PrimitiveArrayTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/PrimitiveArrayTypeInfo.java
index 3843f28..44339ac 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/PrimitiveArrayTypeInfo.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/PrimitiveArrayTypeInfo.java
@@ -20,7 +20,9 @@ package org.apache.flink.api.common.typeinfo;
import java.util.HashMap;
import java.util.Map;
+import java.util.Objects;
+import com.google.common.base.Preconditions;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.functions.InvalidTypesException;
import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -69,23 +71,22 @@ public class PrimitiveArrayTypeInfo<T> extends TypeInformation<T> implements Ato
private final TypeSerializer<T> serializer;
/** The class of the comparator for the array */
- private Class<? extends PrimitiveArrayComparator<T, ?>> comparatorClass;
+ private final Class<? extends PrimitiveArrayComparator<T, ?>> comparatorClass;
/**
* Creates a new type info for a
* @param arrayClass The class of the array (such as int[].class)
* @param serializer The serializer for the array.
+ * @param comparatorClass The class of the array comparator
*/
private PrimitiveArrayTypeInfo(Class<T> arrayClass, TypeSerializer<T> serializer, Class<? extends PrimitiveArrayComparator<T, ?>> comparatorClass) {
- if (arrayClass == null || serializer == null) {
- throw new NullPointerException();
- }
- if (!(arrayClass.isArray() && arrayClass.getComponentType().isPrimitive())) {
- throw new IllegalArgumentException("Class must represent an array of primitives.");
- }
- this.arrayClass = arrayClass;
- this.serializer = serializer;
- this.comparatorClass = comparatorClass;
+ this.arrayClass = Preconditions.checkNotNull(arrayClass);
+ this.serializer = Preconditions.checkNotNull(serializer);
+ this.comparatorClass = Preconditions.checkNotNull(comparatorClass);
+
+ Preconditions.checkArgument(
+ arrayClass.isArray() && arrayClass.getComponentType().isPrimitive(),
+ "Class must represent an array of primitives");
}
// --------------------------------------------------------------------------------------------
@@ -133,12 +134,28 @@ public class PrimitiveArrayTypeInfo<T> extends TypeInformation<T> implements Ato
@Override
public boolean equals(Object other) {
if (other instanceof PrimitiveArrayTypeInfo) {
- PrimitiveArrayTypeInfo otherArray = (PrimitiveArrayTypeInfo) other;
- return otherArray.arrayClass == arrayClass;
+ @SuppressWarnings("unchecked")
+ PrimitiveArrayTypeInfo<T> otherArray = (PrimitiveArrayTypeInfo<T>) other;
+
+ return otherArray.canEqual(this) &&
+ arrayClass == otherArray.arrayClass &&
+ serializer.equals(otherArray.serializer) &&
+ comparatorClass == otherArray.comparatorClass;
+ } else {
+ return false;
}
- return false;
}
-
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(arrayClass, serializer, comparatorClass);
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof PrimitiveArrayTypeInfo;
+ }
+
// --------------------------------------------------------------------------------------------
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/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 b090e67..309c968 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
@@ -146,4 +146,21 @@ public abstract class TypeInformation<T> implements Serializable {
* @return A serializer for this type.
*/
public abstract TypeSerializer<T> createSerializer(ExecutionConfig config);
+
+ @Override
+ public abstract String toString();
+
+ @Override
+ public abstract boolean equals(Object obj);
+
+ @Override
+ public abstract int hashCode();
+
+ /**
+ * Returns true if the given object can be equaled with this object. If not, it returns false.
+ *
+ * @param obj Object which wants to take part in the equality relation
+ * @return true if obj can be equaled with this, otherwise false
+ */
+ public abstract boolean canEqual(Object obj);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeType.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeType.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeType.java
index de39ec8..909afdd 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeType.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeType.java
@@ -20,14 +20,16 @@ package org.apache.flink.api.common.typeutils;
import java.util.ArrayList;
import java.util.List;
+import java.util.Objects;
+import com.google.common.base.Preconditions;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.typeinfo.AtomicType;
import org.apache.flink.api.common.typeinfo.TypeInformation;
/**
- * Type Information for Tuple and Pojo types
+ * Base type information class for Tuple and Pojo types
*
* The class is taking care of serialization and comparators for Tuples as well.
*/
@@ -35,10 +37,19 @@ public abstract class CompositeType<T> extends TypeInformation<T> {
private static final long serialVersionUID = 1L;
- protected final Class<T> typeClass;
+ private final Class<T> typeClass;
public CompositeType(Class<T> typeClass) {
- this.typeClass = typeClass;
+ this.typeClass = Preconditions.checkNotNull(typeClass);
+ }
+
+ /**
+ * Returns the type class of the composite type
+ *
+ * @return Type class of the composite type
+ */
+ public Class<T> getTypeClass() {
+ return typeClass;
}
/**
@@ -78,24 +89,8 @@ public abstract class CompositeType<T> extends TypeInformation<T> {
* @return The type of the field at the given position.
*/
public abstract <X> TypeInformation<X> getTypeAt(int pos);
-
- /**
- * Initializes the internal state inside a Composite type to create a new comparator
- * (such as the lists / arrays for the fields and field comparators)
- * @param localKeyCount
- */
- protected abstract void initializeNewComparator(int localKeyCount);
-
- /**
- * Add a field for comparison in this type.
- */
- protected abstract void addCompareField(int fieldId, TypeComparator<?> comparator);
-
- /**
- * Get the actual comparator we've initialized.
- */
- protected abstract TypeComparator<T> getNewComparator(ExecutionConfig config);
-
+
+ protected abstract TypeComparatorBuilder<T> createTypeComparatorBuilder();
/**
* Generic implementation of the comparator creation. Composite types are supplying the infrastructure
@@ -103,38 +98,74 @@ public abstract class CompositeType<T> extends TypeInformation<T> {
* @return The comparator
*/
public TypeComparator<T> createComparator(int[] logicalKeyFields, boolean[] orders, int logicalFieldOffset, ExecutionConfig config) {
- initializeNewComparator(logicalKeyFields.length);
-
- for(int logicalKeyFieldIndex = 0; logicalKeyFieldIndex < logicalKeyFields.length; logicalKeyFieldIndex++) {
+
+ TypeComparatorBuilder<T> builder = createTypeComparatorBuilder();
+
+ builder.initializeTypeComparatorBuilder(logicalKeyFields.length);
+
+ for (int logicalKeyFieldIndex = 0; logicalKeyFieldIndex < logicalKeyFields.length; logicalKeyFieldIndex++) {
int logicalKeyField = logicalKeyFields[logicalKeyFieldIndex];
int logicalField = logicalFieldOffset; // this is the global/logical field number
- for(int localFieldId = 0; localFieldId < this.getArity(); localFieldId++) {
+ boolean comparatorAdded = false;
+
+ for (int localFieldId = 0; localFieldId < this.getArity() && logicalField <= logicalKeyField && !comparatorAdded; localFieldId++) {
TypeInformation<?> localFieldType = this.getTypeAt(localFieldId);
- if(localFieldType instanceof AtomicType && logicalField == logicalKeyField) {
+ if (localFieldType instanceof AtomicType && logicalField == logicalKeyField) {
// we found an atomic key --> create comparator
- addCompareField(localFieldId, ((AtomicType<?>) localFieldType).createComparator(orders[logicalKeyFieldIndex], config) );
- } else if(localFieldType instanceof CompositeType && // must be a composite type
- ( logicalField <= logicalKeyField //check if keyField can be at or behind the current logicalField
- && logicalKeyField <= logicalField + (localFieldType.getTotalFields() - 1) ) // check if logical field + lookahead could contain our key
- ) {
+ builder.addComparatorField(
+ localFieldId,
+ ((AtomicType<?>) localFieldType).createComparator(
+ orders[logicalKeyFieldIndex],
+ config));
+
+ comparatorAdded = true;
+ }
+ // must be composite type and check that the logicalKeyField is within the bounds
+ // of the composite type's logical fields
+ else if (localFieldType instanceof CompositeType &&
+ logicalField <= logicalKeyField &&
+ logicalKeyField <= logicalField + (localFieldType.getTotalFields() - 1)) {
// we found a compositeType that is containing the logicalKeyField we are looking for --> create comparator
- addCompareField(localFieldId, ((CompositeType<?>) localFieldType).createComparator(new int[] {logicalKeyField}, new boolean[] {orders[logicalKeyFieldIndex]}, logicalField, config));
+ builder.addComparatorField(
+ localFieldId,
+ ((CompositeType<?>) localFieldType).createComparator(
+ new int[]{logicalKeyField},
+ new boolean[]{orders[logicalKeyFieldIndex]},
+ logicalField,
+ config)
+ );
+
+ comparatorAdded = true;
}
-
- // maintain logicalField
- if(localFieldType instanceof CompositeType) {
+
+ if (localFieldType instanceof CompositeType) {
// we need to subtract 1 because we are not accounting for the local field (not accessible for the user)
logicalField += localFieldType.getTotalFields() - 1;
}
+
logicalField++;
}
+
+ if (!comparatorAdded) {
+ throw new IllegalArgumentException("Could not add a comparator for the logical" +
+ "key field index " + logicalKeyFieldIndex + ".");
+ }
}
- return getNewComparator(config);
+
+ return builder.createTypeComparator(config);
}
// --------------------------------------------------------------------------------------------
+ protected interface TypeComparatorBuilder<T> {
+ void initializeTypeComparatorBuilder(int size);
+
+ void addComparatorField(int fieldId, TypeComparator<?> comparator);
+
+ TypeComparator<T> createTypeComparator(ExecutionConfig config);
+ }
+
public static class FlatFieldDescriptor {
private int keyPosition;
private TypeInformation<?> type;
@@ -222,4 +253,31 @@ public abstract class CompositeType<T> extends TypeInformation<T> {
super(s);
}
}
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof CompositeType) {
+ @SuppressWarnings("unchecked")
+ CompositeType<T> compositeType = (CompositeType<T>)obj;
+
+ return compositeType.canEqual(this) && typeClass == compositeType.typeClass;
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(typeClass);
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof CompositeType;
+ }
+
+ @Override
+ public String toString() {
+ return getClass().getSimpleName() + "<" + typeClass.getSimpleName() + ">";
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java
index 45b0669..6d603d3 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java
@@ -146,4 +146,16 @@ public abstract class TypeSerializer<T> implements Serializable {
* @throws IOException Thrown if any of the two views raises an exception.
*/
public abstract void copy(DataInputView source, DataOutputView target) throws IOException;
+
+ public abstract boolean equals(Object obj);
+
+ /**
+ * Returns true if the given object can be equaled with this object. If not, it returns false.
+ *
+ * @param obj Object which wants to take part in the equality relation
+ * @return true if obj can be equaled with this, otherwise false
+ */
+ public abstract boolean canEqual(Object obj);
+
+ public abstract int hashCode();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanSerializer.java
index a844ac8..5fe586b 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanSerializer.java
@@ -75,4 +75,9 @@ public final class BooleanSerializer extends TypeSerializerSingleton<Boolean> {
public void copy(DataInputView source, DataOutputView target) throws IOException {
target.writeBoolean(source.readBoolean());
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof BooleanSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanValueSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanValueSerializer.java
index 3aae95d..a3d2c39 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanValueSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanValueSerializer.java
@@ -80,4 +80,9 @@ public final class BooleanValueSerializer extends TypeSerializerSingleton<Boolea
public void copy(DataInputView source, DataOutputView target) throws IOException {
target.writeBoolean(source.readBoolean());
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof BooleanValueSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ByteSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ByteSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ByteSerializer.java
index 92b3685..abc816c 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ByteSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ByteSerializer.java
@@ -77,4 +77,9 @@ public final class ByteSerializer extends TypeSerializerSingleton<Byte> {
public void copy(DataInputView source, DataOutputView target) throws IOException {
target.writeByte(source.readByte());
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof ByteSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ByteValueSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ByteValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ByteValueSerializer.java
index e523d5e..ff44c27 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ByteValueSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ByteValueSerializer.java
@@ -78,4 +78,9 @@ public final class ByteValueSerializer extends TypeSerializerSingleton<ByteValue
public void copy(DataInputView source, DataOutputView target) throws IOException {
target.writeByte(source.readByte());
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof ByteValueSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CharSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CharSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CharSerializer.java
index 181db56..2734af7 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CharSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CharSerializer.java
@@ -77,4 +77,9 @@ public final class CharSerializer extends TypeSerializerSingleton<Character> {
public void copy(DataInputView source, DataOutputView target) throws IOException {
target.writeChar(source.readChar());
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof CharSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CharValueSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CharValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CharValueSerializer.java
index 690509c..e14a861 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CharValueSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CharValueSerializer.java
@@ -77,4 +77,9 @@ public class CharValueSerializer extends TypeSerializerSingleton<CharValue> {
public void copy(DataInputView source, DataOutputView target) throws IOException {
target.writeChar(source.readChar());
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof CharValueSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DateSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DateSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DateSerializer.java
index d427918..6703240 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DateSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DateSerializer.java
@@ -97,4 +97,9 @@ public final class DateSerializer extends TypeSerializerSingleton<Date> {
public void copy(DataInputView source, DataOutputView target) throws IOException {
target.writeLong(source.readLong());
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof DateSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleSerializer.java
index 24af95c..f982399 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleSerializer.java
@@ -76,4 +76,9 @@ public final class DoubleSerializer extends TypeSerializerSingleton<Double> {
public void copy(DataInputView source, DataOutputView target) throws IOException {
target.writeDouble(source.readDouble());
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof DoubleSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleValueSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleValueSerializer.java
index 34434f1..747d969 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleValueSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleValueSerializer.java
@@ -78,4 +78,9 @@ public final class DoubleValueSerializer extends TypeSerializerSingleton<DoubleV
public void copy(DataInputView source, DataOutputView target) throws IOException {
target.writeDouble(source.readDouble());
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof DoubleValueSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/EnumSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/EnumSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/EnumSerializer.java
index 643e4fa..1c09e71 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/EnumSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/EnumSerializer.java
@@ -22,6 +22,7 @@ import java.io.IOException;
import java.io.ObjectInputStream;
import java.lang.reflect.Method;
+import com.google.common.base.Preconditions;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;
@@ -36,7 +37,7 @@ public final class EnumSerializer<T extends Enum<T>> extends TypeSerializer<T> {
private final Class<T> enumClass;
public EnumSerializer(Class<T> enumClass) {
- this.enumClass = enumClass;
+ this.enumClass = Preconditions.checkNotNull(enumClass);
this.values = createValues(enumClass);
}
@@ -94,12 +95,23 @@ public final class EnumSerializer<T extends Enum<T>> extends TypeSerializer<T> {
public boolean equals(Object obj) {
if(obj instanceof EnumSerializer) {
EnumSerializer<?> other = (EnumSerializer<?>) obj;
- return other.enumClass == this.enumClass;
+
+ return other.canEqual(this) && other.enumClass == this.enumClass;
} else {
return false;
}
}
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof EnumSerializer;
+ }
+
+ @Override
+ public int hashCode() {
+ return enumClass.hashCode();
+ }
+
// --------------------------------------------------------------------------------------------
private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatSerializer.java
index c823783..1cc574c 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatSerializer.java
@@ -76,4 +76,9 @@ public final class FloatSerializer extends TypeSerializerSingleton<Float> {
public void copy(DataInputView source, DataOutputView target) throws IOException {
target.writeFloat(source.readFloat());
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof FloatSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatValueSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatValueSerializer.java
index 15d00b5..11b17c8 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatValueSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatValueSerializer.java
@@ -78,4 +78,9 @@ public class FloatValueSerializer extends TypeSerializerSingleton<FloatValue> {
public void copy(DataInputView source, DataOutputView target) throws IOException {
target.writeFloat(source.readFloat());
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof FloatValueSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/GenericArraySerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/GenericArraySerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/GenericArraySerializer.java
index 6687661..9ea6811 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/GenericArraySerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/GenericArraySerializer.java
@@ -21,6 +21,7 @@ package org.apache.flink.api.common.typeutils.base;
import java.io.IOException;
import java.lang.reflect.Array;
+import com.google.common.base.Preconditions;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;
@@ -38,17 +39,12 @@ public final class GenericArraySerializer<C> extends TypeSerializer<C[]> {
private final TypeSerializer<C> componentSerializer;
- private final C[] EMPTY;
+ private transient C[] EMPTY;
public GenericArraySerializer(Class<C> componentClass, TypeSerializer<C> componentSerializer) {
- if (componentClass == null || componentSerializer == null) {
- throw new NullPointerException();
- }
-
- this.componentClass = componentClass;
- this.componentSerializer = componentSerializer;
- this.EMPTY = create(0);
+ this.componentClass = Preconditions.checkNotNull(componentClass);
+ this.componentSerializer = Preconditions.checkNotNull(componentSerializer);
}
@Override
@@ -70,6 +66,10 @@ public final class GenericArraySerializer<C> extends TypeSerializer<C[]> {
@Override
public C[] createInstance() {
+ if (EMPTY == null) {
+ EMPTY = create(0);
+ }
+
return EMPTY;
}
@@ -158,20 +158,27 @@ public final class GenericArraySerializer<C> extends TypeSerializer<C[]> {
@Override
public int hashCode() {
- return componentClass.hashCode() + componentSerializer.hashCode();
+ return 31 * componentClass.hashCode() + componentSerializer.hashCode();
}
@Override
public boolean equals(Object obj) {
- if (obj != null && obj instanceof GenericArraySerializer) {
- GenericArraySerializer<?> other = (GenericArraySerializer<?>) obj;
- return this.componentClass == other.componentClass &&
- this.componentSerializer.equals(other.componentSerializer);
+ if (obj instanceof GenericArraySerializer) {
+ GenericArraySerializer<?> other = (GenericArraySerializer<?>)obj;
+
+ return other.canEqual(this) &&
+ componentClass == other.componentClass &&
+ componentSerializer.equals(other.componentSerializer);
} else {
return false;
}
}
-
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof GenericArraySerializer;
+ }
+
@Override
public String toString() {
return "Serializer " + componentClass.getName() + "[]";
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntSerializer.java
index 778f044..e9d9059 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntSerializer.java
@@ -77,4 +77,9 @@ public final class IntSerializer extends TypeSerializerSingleton<Integer> {
public void copy(DataInputView source, DataOutputView target) throws IOException {
target.writeInt(source.readInt());
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof IntSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntValueSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntValueSerializer.java
index c2d1b60..a3a5cbb 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntValueSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntValueSerializer.java
@@ -78,4 +78,9 @@ public final class IntValueSerializer extends TypeSerializerSingleton<IntValue>
public void copy(DataInputView source, DataOutputView target) throws IOException {
target.writeInt(source.readInt());
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof IntValueSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongSerializer.java
index 6d8b758..2954477 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongSerializer.java
@@ -77,4 +77,9 @@ public final class LongSerializer extends TypeSerializerSingleton<Long> {
public void copy(DataInputView source, DataOutputView target) throws IOException {
target.writeLong(source.readLong());
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof LongSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongValueSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongValueSerializer.java
index 37dec40..6233240 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongValueSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongValueSerializer.java
@@ -78,4 +78,9 @@ public final class LongValueSerializer extends TypeSerializerSingleton<LongValue
public void copy(DataInputView source, DataOutputView target) throws IOException {
target.writeLong(source.readLong());
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof LongValueSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortSerializer.java
index 44e5e3e..45004fa 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortSerializer.java
@@ -77,4 +77,9 @@ public final class ShortSerializer extends TypeSerializerSingleton<Short> {
public void copy(DataInputView source, DataOutputView target) throws IOException {
target.writeShort(source.readShort());
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof ShortSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortValueSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortValueSerializer.java
index 1dbe4a5..338d328 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortValueSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortValueSerializer.java
@@ -78,4 +78,9 @@ public final class ShortValueSerializer extends TypeSerializerSingleton<ShortVal
public void copy(DataInputView source, DataOutputView target) throws IOException {
target.writeShort(source.readShort());
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof ShortValueSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/StringSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/StringSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/StringSerializer.java
index 7b26600..b5332f8 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/StringSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/StringSerializer.java
@@ -77,4 +77,9 @@ public final class StringSerializer extends TypeSerializerSingleton<String> {
public void copy(DataInputView source, DataOutputView target) throws IOException {
StringValue.copyString(source, target);
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof StringSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/StringValueSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/StringValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/StringValueSerializer.java
index 7628cab..6ac8b6c 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/StringValueSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/StringValueSerializer.java
@@ -103,4 +103,9 @@ public final class StringValueSerializer extends TypeSerializerSingleton<StringV
}
}
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof StringValueSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/TypeSerializerSingleton.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/TypeSerializerSingleton.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/TypeSerializerSingleton.java
index e076e5b..68842d6 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/TypeSerializerSingleton.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/TypeSerializerSingleton.java
@@ -33,11 +33,17 @@ public abstract class TypeSerializerSingleton<T> extends TypeSerializer<T>{
@Override
public int hashCode() {
- return super.hashCode();
+ return TypeSerializerSingleton.class.hashCode();
}
@Override
public boolean equals(Object obj) {
- return obj != null && obj.getClass() == this.getClass();
+ if (obj instanceof TypeSerializerSingleton) {
+ TypeSerializerSingleton<?> other = (TypeSerializerSingleton<?>) obj;
+
+ return other.canEqual(this);
+ } else {
+ return false;
+ }
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/VoidSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/VoidSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/VoidSerializer.java
index 272ffbd..ee1ebbd 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/VoidSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/VoidSerializer.java
@@ -77,4 +77,9 @@ public final class VoidSerializer extends TypeSerializerSingleton<Void> {
public void copy(DataInputView source, DataOutputView target) throws IOException {
target.write(source.readByte());
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof VoidSerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/BooleanPrimitiveArraySerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/BooleanPrimitiveArraySerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/BooleanPrimitiveArraySerializer.java
index 4a493ac..6386fc1 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/BooleanPrimitiveArraySerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/BooleanPrimitiveArraySerializer.java
@@ -101,4 +101,9 @@ public final class BooleanPrimitiveArraySerializer extends TypeSerializerSinglet
target.writeInt(len);
target.write(source, len);
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof BooleanPrimitiveArraySerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/BytePrimitiveArraySerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/BytePrimitiveArraySerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/BytePrimitiveArraySerializer.java
index fb4d506..ea4c43e 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/BytePrimitiveArraySerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/BytePrimitiveArraySerializer.java
@@ -93,4 +93,9 @@ public final class BytePrimitiveArraySerializer extends TypeSerializerSingleton<
target.writeInt(len);
target.write(source, len);
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof BytePrimitiveArraySerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/CharPrimitiveArraySerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/CharPrimitiveArraySerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/CharPrimitiveArraySerializer.java
index 8e3c4ea..e2f7515 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/CharPrimitiveArraySerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/CharPrimitiveArraySerializer.java
@@ -100,4 +100,9 @@ public final class CharPrimitiveArraySerializer extends TypeSerializerSingleton<
target.writeInt(len);
target.write(source, len * 2);
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof CharPrimitiveArraySerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/DoublePrimitiveArraySerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/DoublePrimitiveArraySerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/DoublePrimitiveArraySerializer.java
index 10e25c2..2500312 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/DoublePrimitiveArraySerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/DoublePrimitiveArraySerializer.java
@@ -100,4 +100,9 @@ public final class DoublePrimitiveArraySerializer extends TypeSerializerSingleto
target.writeInt(len);
target.write(source, len * 8);
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof DoublePrimitiveArraySerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/FloatPrimitiveArraySerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/FloatPrimitiveArraySerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/FloatPrimitiveArraySerializer.java
index d57af00..ea2513e 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/FloatPrimitiveArraySerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/FloatPrimitiveArraySerializer.java
@@ -100,4 +100,9 @@ public final class FloatPrimitiveArraySerializer extends TypeSerializerSingleton
target.writeInt(len);
target.write(source, len * 4);
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof FloatPrimitiveArraySerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/IntPrimitiveArraySerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/IntPrimitiveArraySerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/IntPrimitiveArraySerializer.java
index eaff287..05d70c5 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/IntPrimitiveArraySerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/IntPrimitiveArraySerializer.java
@@ -100,4 +100,9 @@ public class IntPrimitiveArraySerializer extends TypeSerializerSingleton<int[]>{
target.writeInt(len);
target.write(source, len * 4);
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof IntPrimitiveArraySerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/LongPrimitiveArraySerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/LongPrimitiveArraySerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/LongPrimitiveArraySerializer.java
index 55a22c2..e6f6cc5 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/LongPrimitiveArraySerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/LongPrimitiveArraySerializer.java
@@ -100,4 +100,9 @@ public final class LongPrimitiveArraySerializer extends TypeSerializerSingleton<
target.writeInt(len);
target.write(source, len * 8);
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof LongPrimitiveArraySerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/ShortPrimitiveArraySerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/ShortPrimitiveArraySerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/ShortPrimitiveArraySerializer.java
index 08275b0..e2261e8 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/ShortPrimitiveArraySerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/ShortPrimitiveArraySerializer.java
@@ -100,4 +100,9 @@ public final class ShortPrimitiveArraySerializer extends TypeSerializerSingleton
target.writeInt(len);
target.write(source, len * 2);
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof ShortPrimitiveArraySerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/StringArraySerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/StringArraySerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/StringArraySerializer.java
index ad172a8..228429d 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/StringArraySerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/StringArraySerializer.java
@@ -104,4 +104,9 @@ public final class StringArraySerializer extends TypeSerializerSingleton<String[
StringValue.copyString(source, target);
}
}
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof StringArraySerializer;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-core/src/main/java/org/apache/flink/api/common/typeutils/record/RecordSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/record/RecordSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/record/RecordSerializer.java
index 11b21d6..6ffa0df 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/record/RecordSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/record/RecordSerializer.java
@@ -40,7 +40,7 @@ public final class RecordSerializer extends TypeSerializer<Record> {
// --------------------------------------------------------------------------------------------
- public static final RecordSerializer get() {
+ public static RecordSerializer get() {
return INSTANCE;
}
@@ -121,4 +121,24 @@ public final class RecordSerializer extends TypeSerializer<Record> {
target.write(source, val);
}
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof RecordSerializer) {
+ RecordSerializer other = (RecordSerializer) obj;
+ return other.canEqual(this);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof RecordSerializer;
+ }
+
+ @Override
+ public int hashCode() {
+ return RecordSerializer.class.hashCode();
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/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
new file mode 100644
index 0000000..3e086ff
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/types/BasicArrayTypeInfoTest.java
@@ -0,0 +1,56 @@
+/*
+ * 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/8ca853e0/flink-core/src/test/java/org/apache/flink/types/BasicTypeInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/types/BasicTypeInfoTest.java b/flink-core/src/test/java/org/apache/flink/types/BasicTypeInfoTest.java
new file mode 100644
index 0000000..cdd06d0
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/types/BasicTypeInfoTest.java
@@ -0,0 +1,58 @@
+/*
+ * 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 WNTIES 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.util.TestLogger;
+import org.junit.Test;
+
+import java.util.Date;
+
+import static org.junit.Assert.*;
+
+public class BasicTypeInfoTest extends TestLogger {
+
+ static Class<?>[] classes = {String.class, Integer.class, Boolean.class, Byte.class,
+ Short.class, Long.class, Float.class, Double.class, Character.class, Date.class,
+ Void.class};
+
+ @Test
+ public void testBasicTypeInfoEquality() {
+ for (Class<?> clazz: classes) {
+ BasicTypeInfo<?> tpeInfo1 = BasicTypeInfo.getInfoFor(clazz);
+ BasicTypeInfo<?> tpeInfo2 = BasicTypeInfo.getInfoFor(clazz);
+
+ assertEquals(tpeInfo1, tpeInfo2);
+ assertEquals(tpeInfo1.hashCode(), tpeInfo2.hashCode());
+ }
+ }
+
+ @Test
+ public void testBasicTypeInfoInequality() {
+ for (Class<?> clazz1: classes) {
+ for (Class<?> clazz2: classes) {
+ if (!clazz1.equals(clazz2)) {
+ BasicTypeInfo<?> tpeInfo1 = BasicTypeInfo.getInfoFor(clazz1);
+ BasicTypeInfo<?> tpeInfo2 = BasicTypeInfo.getInfoFor(clazz2);
+ assertNotEquals(tpeInfo1, tpeInfo2);
+ }
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/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
new file mode 100644
index 0000000..a7976ee
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/types/NothingTypeInfoTest.java
@@ -0,0 +1,47 @@
+/*
+ * 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/8ca853e0/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
new file mode 100644
index 0000000..7ef14f9
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/types/PrimitiveArrayTypeInfoTest.java
@@ -0,0 +1,56 @@
+/*
+ * 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/8ca853e0/flink-java/src/main/java/org/apache/flink/api/java/io/CsvReader.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/CsvReader.java b/flink-java/src/main/java/org/apache/flink/api/java/io/CsvReader.java
index 052f960..34b5b47 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/io/CsvReader.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/io/CsvReader.java
@@ -300,7 +300,7 @@ public class CsvReader {
if(pos < 0) {
throw new IllegalArgumentException("Field \""+pojoFields[i]+"\" not part of POJO type "+pojoType.getCanonicalName());
}
- classes[i] = typeInfo.getPojoFieldAt(pos).type.getTypeClass();
+ classes[i] = typeInfo.getPojoFieldAt(pos).getTypeInformation().getTypeClass();
}
configureInputFormat(inputFormat, classes);
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/main/java/org/apache/flink/api/java/sca/UdfAnalyzerUtils.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/sca/UdfAnalyzerUtils.java b/flink-java/src/main/java/org/apache/flink/api/java/sca/UdfAnalyzerUtils.java
index dbfd29e..eccdeac 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/sca/UdfAnalyzerUtils.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/sca/UdfAnalyzerUtils.java
@@ -75,7 +75,7 @@ public final class UdfAnalyzerUtils {
final PojoTypeInfo<?> pojoTypeInfo = (PojoTypeInfo<?>) typeInfo;
HashMap<String, TaggedValue> containerMapping = new HashMap<String, TaggedValue>();
for (int i = 0; i < pojoTypeInfo.getArity(); i++) {
- final String fieldName = pojoTypeInfo.getPojoFieldAt(i).field.getName();
+ final String fieldName = pojoTypeInfo.getPojoFieldAt(i).getField().getName();
containerMapping.put(fieldName,
convertTypeInfoToTaggedValue(input,
pojoTypeInfo.getTypeAt(i),
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/main/java/org/apache/flink/api/java/typeutils/AvroTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/AvroTypeInfo.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/AvroTypeInfo.java
index ed03627..59c7eb0 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/AvroTypeInfo.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/AvroTypeInfo.java
@@ -56,7 +56,7 @@ public class AvroTypeInfo<T extends SpecificRecordBase> extends PojoTypeInfo<T>
for(int i = 0; i < pti.getArity(); i++) {
PojoField f = pti.getPojoFieldAt(i);
- TypeInformation newType = f.type;
+ TypeInformation newType = f.getTypeInformation();
// check if type is a CharSequence
if(newType instanceof GenericTypeInfo) {
if((newType).getTypeClass().equals(CharSequence.class)) {
@@ -64,7 +64,7 @@ public class AvroTypeInfo<T extends SpecificRecordBase> extends PojoTypeInfo<T>
newType = new GenericTypeInfo(org.apache.avro.util.Utf8.class);
}
}
- PojoField newField = new PojoField(f.field, newType);
+ PojoField newField = new PojoField(f.getField(), newType);
newFields.add(newField);
}
return newFields;
[8/9] flink git commit: [FLINK-2659] [runtime] Fix object reuse in
UnionWithTempOperator
Posted by fh...@apache.org.
[FLINK-2659] [runtime] Fix object reuse in UnionWithTempOperator
This closes #1130
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/c269158c
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/c269158c
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/c269158c
Branch: refs/heads/master
Commit: c269158cc4ddb1faf79f75a2bf04aa5f9e1865bf
Parents: 988a04e
Author: Greg Hogan <co...@greghogan.com>
Authored: Mon Sep 14 11:29:49 2015 -0400
Committer: Fabian Hueske <fh...@apache.org>
Committed: Thu Sep 17 11:19:34 2015 +0200
----------------------------------------------------------------------
.../apache/flink/runtime/operators/UnionWithTempOperator.java | 7 ++++---
1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/c269158c/flink-runtime/src/main/java/org/apache/flink/runtime/operators/UnionWithTempOperator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/UnionWithTempOperator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/UnionWithTempOperator.java
index d8437a9..098686c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/UnionWithTempOperator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/UnionWithTempOperator.java
@@ -60,15 +60,16 @@ public class UnionWithTempOperator<T> implements PactDriver<Function, T> {
public void run() throws Exception {
final Collector<T> output = this.taskContext.getOutputCollector();
- T record = this.taskContext.<T>getInputSerializer(STREAMED_INPUT).getSerializer().createInstance();
+ T reuse = this.taskContext.<T>getInputSerializer(STREAMED_INPUT).getSerializer().createInstance();
+ T record;
final MutableObjectIterator<T> input = this.taskContext.getInput(STREAMED_INPUT);
- while (this.running && ((record = input.next(record)) != null)) {
+ while (this.running && ((record = input.next(reuse)) != null)) {
output.collect(record);
}
final MutableObjectIterator<T> cache = this.taskContext.getInput(CACHED_INPUT);
- while (this.running && ((record = cache.next(record)) != null)) {
+ while (this.running && ((record = cache.next(reuse)) != null)) {
output.collect(record);
}
}
[4/9] flink git commit: [FLINK-2637] [api-breaking] [types] Adds
equals and hashCode method to TypeInformations and TypeSerializers - Fixes
ObjectArrayTypeInfo - Makes CompositeTypes serializable - Adds test for
equality relation's symmetric property - A
Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/test/java/org/apache/flink/api/java/typeutils/EnumTypeInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/EnumTypeInfoTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/EnumTypeInfoTest.java
new file mode 100644
index 0000000..b200566
--- /dev/null
+++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/EnumTypeInfoTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.util.TestLogger;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class EnumTypeInfoTest extends TestLogger {
+
+ enum TestEnum {
+ ONE, TWO
+ }
+
+ 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/8ca853e0/flink-java/src/test/java/org/apache/flink/api/java/typeutils/GenericTypeInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/GenericTypeInfoTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/GenericTypeInfoTest.java
new file mode 100644
index 0000000..fad43df
--- /dev/null
+++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/GenericTypeInfoTest.java
@@ -0,0 +1,47 @@
+/*
+ * 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.util.TestLogger;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class GenericTypeInfoTest extends TestLogger {
+
+ 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/8ca853e0/flink-java/src/test/java/org/apache/flink/api/java/typeutils/MissingTypeInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/MissingTypeInfoTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/MissingTypeInfoTest.java
new file mode 100644
index 0000000..ee57475
--- /dev/null
+++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/MissingTypeInfoTest.java
@@ -0,0 +1,47 @@
+/*
+ * 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.functions.InvalidTypesException;
+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.");
+
+ @Test
+ public void testMissingTypeInfoEquality() {
+ MissingTypeInfo tpeInfo1 = new MissingTypeInfo(functionName, testException);
+ MissingTypeInfo tpeInfo2 = new MissingTypeInfo(functionName, testException);
+
+ assertEquals(tpeInfo1, tpeInfo2);
+ assertEquals(tpeInfo1.hashCode(), tpeInfo2.hashCode());
+ }
+
+ @Test
+ public void testMissingTypeInfoInequality() {
+ MissingTypeInfo tpeInfo1 = new MissingTypeInfo(functionName, testException);
+ MissingTypeInfo tpeInfo2 = new MissingTypeInfo("alt" + functionName, testException);
+
+ assertNotEquals(tpeInfo1, tpeInfo2);
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/test/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfoTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfoTest.java
new file mode 100644
index 0000000..f3b39c0
--- /dev/null
+++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfoTest.java
@@ -0,0 +1,58 @@
+/*
+ * 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.util.TestLogger;
+import org.junit.Test;
+
+import java.util.ArrayList;
+
+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
+ 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);
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/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..2fe1357 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,12 +21,15 @@ 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 testEquals() {
+ public void testPojoTypeInfoEquality() {
try {
TypeInformation<TestPojo> info1 = TypeExtractor.getForClass(TestPojo.class);
TypeInformation<TestPojo> info2 = TypeExtractor.getForClass(TestPojo.class);
@@ -42,6 +45,35 @@ 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);
+ }
public static final class TestPojo {
@@ -60,4 +92,22 @@ 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;
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/test/java/org/apache/flink/api/java/typeutils/RecordTypeInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/RecordTypeInfoTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/RecordTypeInfoTest.java
new file mode 100644
index 0000000..7aeb062
--- /dev/null
+++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/RecordTypeInfoTest.java
@@ -0,0 +1,44 @@
+/*
+ * 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.util.TestLogger;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class RecordTypeInfoTest extends TestLogger {
+
+ @Test
+ public void testRecordTypeInfoEquality() {
+ RecordTypeInfo tpeInfo1 = new RecordTypeInfo();
+ RecordTypeInfo tpeInfo2 = new RecordTypeInfo();
+
+ assertEquals(tpeInfo1, tpeInfo2);
+ assertEquals(tpeInfo1.hashCode(), tpeInfo2.hashCode());
+ }
+
+ @Test
+ public void testRecordTypeInfoInequality() {
+ RecordTypeInfo tpeInfo1 = new RecordTypeInfo();
+ MissingTypeInfo tpeInfo2 = new MissingTypeInfo("foobar");
+
+ assertNotEquals(tpeInfo1, tpeInfo2);
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/test/java/org/apache/flink/api/java/typeutils/TupleTypeInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/TupleTypeInfoTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/TupleTypeInfoTest.java
new file mode 100644
index 0000000..b6cff34
--- /dev/null
+++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/TupleTypeInfoTest.java
@@ -0,0 +1,96 @@
+/*
+ * 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.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+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 org.junit.Test;
+
+public class TupleTypeInfoTest extends TestLogger {
+
+ @Test
+ public void testTupleTypeInfoSymmetricEqualityRelation() {
+ TupleTypeInfo<Tuple1<Integer>> tupleTypeInfo = new TupleTypeInfo<>(BasicTypeInfo.INT_TYPE_INFO);
+
+ TupleTypeInfoBase<Tuple1> anonymousTupleTypeInfo = new TupleTypeInfoBase<Tuple1>(
+ (Class<Tuple1>)Tuple1.class,
+ (TypeInformation<?>)BasicTypeInfo.INT_TYPE_INFO) {
+
+ private static final long serialVersionUID = -7985593598027660836L;
+
+ @Override
+ public TypeSerializer<Tuple1> createSerializer(ExecutionConfig config) {
+ return null;
+ }
+
+ @Override
+ protected TypeComparatorBuilder<Tuple1> createTypeComparatorBuilder() {
+ return null;
+ }
+
+ @Override
+ public String[] getFieldNames() {
+ return new String[0];
+ }
+
+ @Override
+ public int getFieldIndex(String fieldName) {
+ return 0;
+ }
+ };
+
+ 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);
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/test/java/org/apache/flink/api/java/typeutils/TypeInfoParserTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/TypeInfoParserTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/TypeInfoParserTest.java
index 9fe8174..e225460 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/TypeInfoParserTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/TypeInfoParserTest.java
@@ -181,14 +181,14 @@ public class TypeInfoParserTest {
+ ">");
Assert.assertTrue(ti instanceof PojoTypeInfo);
PojoTypeInfo<?> pti = (PojoTypeInfo<?>) ti;
- Assert.assertEquals("array", pti.getPojoFieldAt(0).field.getName());
- Assert.assertTrue(pti.getPojoFieldAt(0).type instanceof BasicArrayTypeInfo);
- Assert.assertEquals("basic", pti.getPojoFieldAt(1).field.getName());
- Assert.assertTrue(pti.getPojoFieldAt(1).type instanceof BasicTypeInfo);
- Assert.assertEquals("hadoopCitizen", pti.getPojoFieldAt(2).field.getName());
- Assert.assertTrue(pti.getPojoFieldAt(2).type instanceof WritableTypeInfo);
- Assert.assertEquals("tuple", pti.getPojoFieldAt(3).field.getName());
- Assert.assertTrue(pti.getPojoFieldAt(3).type instanceof TupleTypeInfo);
+ Assert.assertEquals("array", pti.getPojoFieldAt(0).getField().getName());
+ Assert.assertTrue(pti.getPojoFieldAt(0).getTypeInformation() instanceof BasicArrayTypeInfo);
+ Assert.assertEquals("basic", pti.getPojoFieldAt(1).getField().getName());
+ Assert.assertTrue(pti.getPojoFieldAt(1).getTypeInformation() instanceof BasicTypeInfo);
+ Assert.assertEquals("hadoopCitizen", pti.getPojoFieldAt(2).getField().getName());
+ Assert.assertTrue(pti.getPojoFieldAt(2).getTypeInformation() instanceof WritableTypeInfo);
+ Assert.assertEquals("tuple", pti.getPojoFieldAt(3).getField().getName());
+ Assert.assertTrue(pti.getPojoFieldAt(3).getTypeInformation() instanceof TupleTypeInfo);
}
@Test
@@ -198,12 +198,12 @@ public class TypeInfoParserTest {
TupleTypeInfo<?> tti = (TupleTypeInfo<?>) ti;
Assert.assertTrue(tti.getTypeAt(0) instanceof BasicTypeInfo);
Assert.assertTrue(tti.getTypeAt(1) instanceof TupleTypeInfo);
- TupleTypeInfo<?> tti2 = (TupleTypeInfo<?>) tti.getTypeAt(1);
+ TupleTypeInfo<?> tti2 = (TupleTypeInfo<?>)(Object)tti.getTypeAt(1);
Assert.assertTrue(tti2.getTypeAt(0) instanceof BasicTypeInfo);
Assert.assertTrue(tti2.getTypeAt(1) instanceof PojoTypeInfo);
PojoTypeInfo<?> pti = (PojoTypeInfo<?>) tti2.getTypeAt(1);
- Assert.assertEquals("basic", pti.getPojoFieldAt(0).field.getName());
- Assert.assertTrue(pti.getPojoFieldAt(0).type instanceof BasicTypeInfo);
+ Assert.assertEquals("basic", pti.getPojoFieldAt(0).getField().getName());
+ Assert.assertTrue(pti.getPojoFieldAt(0).getTypeInformation() instanceof BasicTypeInfo);
}
public static class MyWritable implements Writable {
@@ -232,7 +232,7 @@ public class TypeInfoParserTest {
TypeInformation<?> ti = TypeInfoParser.parse("java.lang.Class[]");
Assert.assertTrue(ti instanceof ObjectArrayTypeInfo<?, ?>);
- Assert.assertEquals(Class.class, ((ObjectArrayTypeInfo<?, ?>) ti).getComponentType());
+ Assert.assertEquals(Class.class, ((ObjectArrayTypeInfo<?, ?>) ti).getComponentInfo().getTypeClass());
TypeInformation<?> ti2 = TypeInfoParser.parse("Tuple2<Integer,Double>[]");
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-java/src/test/java/org/apache/flink/api/java/typeutils/ValueTypeInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/ValueTypeInfoTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/ValueTypeInfoTest.java
new file mode 100644
index 0000000..4a579c8
--- /dev/null
+++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/ValueTypeInfoTest.java
@@ -0,0 +1,87 @@
+/*
+ * 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.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.types.Record;
+import org.apache.flink.types.Value;
+import org.apache.flink.util.TestLogger;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.*;
+
+public class ValueTypeInfoTest extends TestLogger {
+
+ public static class TestClass implements Value {
+ private static final long serialVersionUID = -492760806806568285L;
+
+ @Override
+ public void write(DataOutputView out) throws IOException {
+
+ }
+
+ @Override
+ public void read(DataInputView in) throws IOException {
+
+ }
+ }
+
+ public static class AlternativeClass implements Value {
+
+ private static final long serialVersionUID = -163437084575260172L;
+
+ @Override
+ public void write(DataOutputView out) throws IOException {
+
+ }
+
+ @Override
+ public void read(DataInputView in) throws IOException {
+
+ }
+ }
+
+ @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/8ca853e0/flink-java/src/test/java/org/apache/flink/api/java/typeutils/WritableTypeInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/WritableTypeInfoTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/WritableTypeInfoTest.java
new file mode 100644
index 0000000..2ab0021
--- /dev/null
+++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/WritableTypeInfoTest.java
@@ -0,0 +1,74 @@
+/*
+ * 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.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 static org.junit.Assert.*;
+
+public class WritableTypeInfoTest extends TestLogger {
+
+ public static class TestClass implements Writable {
+ @Override
+ public void write(DataOutput dataOutput) throws IOException {
+
+ }
+
+ @Override
+ public void readFields(DataInput dataInput) throws IOException {
+
+ }
+ }
+
+ public static class AlternateClass implements Writable {
+ @Override
+ public void write(DataOutput dataOutput) throws IOException {
+
+ }
+
+ @Override
+ public void readFields(DataInput dataInput) throws IOException {
+
+ }
+ }
+
+
+ @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);
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntListSerializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntListSerializer.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntListSerializer.java
index 69dfeb9..e872526 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntListSerializer.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntListSerializer.java
@@ -106,4 +106,25 @@ public class IntListSerializer extends TypeSerializer<IntList> {
target.writeInt(source.readInt());
}
}
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof IntListSerializer) {
+ IntListSerializer other = (IntListSerializer) obj;
+
+ return other.canEqual(this);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof IntListSerializer;
+ }
+
+ @Override
+ public int hashCode() {
+ return IntListSerializer.class.hashCode();
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntPairSerializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntPairSerializer.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntPairSerializer.java
index c2571cc..e4a9264 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntPairSerializer.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntPairSerializer.java
@@ -88,6 +88,27 @@ public class IntPairSerializer extends TypeSerializer<IntPair> {
target.write(source, 8);
}
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof IntPairSerializer) {
+ IntPairSerializer other = (IntPairSerializer) obj;
+
+ return other.canEqual(this);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof IntPairSerializer;
+ }
+
+ @Override
+ public int hashCode() {
+ return IntPairSerializer.class.hashCode();
+ }
+
public static final class IntPairSerializerFactory implements TypeSerializerFactory<IntPair> {
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/StringPairSerializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/StringPairSerializer.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/StringPairSerializer.java
index 388e8bd..b62b097 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/StringPairSerializer.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/StringPairSerializer.java
@@ -83,4 +83,25 @@ public class StringPairSerializer extends TypeSerializer<StringPair> {
StringValue.writeString(StringValue.readString(source), target);
StringValue.writeString(StringValue.readString(source), target);
}
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof StringPairSerializer) {
+ StringPairSerializer other = (StringPairSerializer) obj;
+
+ return other.canEqual(this);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof StringPairSerializer;
+ }
+
+ @Override
+ public int hashCode() {
+ return StringPairSerializer.class.hashCode();
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
index 344b186..5265134 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/ExecutionEnvironment.scala
@@ -296,7 +296,7 @@ class ExecutionEnvironment(javaEnv: JavaEnv) {
"Field \"" + pojoFields(i) + "\" not part of POJO type " +
info.getTypeClass.getCanonicalName);
}
- classesBuf += info.getPojoFieldAt(pos).`type`.getTypeClass
+ classesBuf += info.getPojoFieldAt(pos).getTypeInformation().getTypeClass
}
}
case _ => throw new IllegalArgumentException("Type information is not valid.")
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
index 07f7205..aa76fcc 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
@@ -121,7 +121,7 @@ private[flink] trait TypeInformationGen[C <: Context] {
fieldSerializers(i) = types(i).createSerializer(executionConfig)
}
- new CaseClassSerializer[T](tupleType, fieldSerializers) {
+ new CaseClassSerializer[T](getTypeClass(), fieldSerializers) {
override def createInstance(fields: Array[AnyRef]): T = {
instance.splice
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassSerializer.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassSerializer.scala
index 2a76c37..3a015f7 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassSerializer.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassSerializer.scala
@@ -17,7 +17,6 @@
*/
package org.apache.flink.api.scala.typeutils
-import org.apache.commons.lang.SerializationUtils
import org.apache.flink.api.common.typeutils.TypeSerializer
import org.apache.flink.api.java.typeutils.runtime.TupleSerializerBase
import org.apache.flink.core.memory.{DataOutputView, DataInputView}
@@ -29,19 +28,25 @@ import org.apache.flink.core.memory.{DataOutputView, DataInputView}
abstract class CaseClassSerializer[T <: Product](
clazz: Class[T],
scalaFieldSerializers: Array[TypeSerializer[_]])
- extends TupleSerializerBase[T](clazz, scalaFieldSerializers) with Cloneable {
+ extends TupleSerializerBase[T](clazz, scalaFieldSerializers)
+ with Cloneable {
@transient var fields : Array[AnyRef] = _
@transient var instanceCreationFailed : Boolean = false
override def duplicate = {
- val result = this.clone().asInstanceOf[CaseClassSerializer[T]]
+ clone().asInstanceOf[CaseClassSerializer[T]]
+ }
+
+ @throws[CloneNotSupportedException]
+ override protected def clone(): Object = {
+ val result = super.clone().asInstanceOf[CaseClassSerializer[T]]
- // set transient fields to null and make copy of serializers
+ // achieve a deep copy by duplicating the field serializers
+ result.fieldSerializers.transform(_.duplicate())
result.fields = null
result.instanceCreationFailed = false
- result.fieldSerializers = fieldSerializers.map(_.duplicate())
result
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassTypeInfo.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassTypeInfo.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassTypeInfo.scala
index 0c8049d..37c7431 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassTypeInfo.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassTypeInfo.scala
@@ -18,19 +18,19 @@
package org.apache.flink.api.scala.typeutils
+import java.util
import java.util.regex.{Pattern, Matcher}
import org.apache.flink.api.common.ExecutionConfig
import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.common.typeinfo.AtomicType
-import org.apache.flink.api.common.typeutils.CompositeType.InvalidFieldReferenceException
-import org.apache.flink.api.common.typeutils.CompositeType.FlatFieldDescriptor
+import org.apache.flink.api.common.typeutils.CompositeType.{TypeComparatorBuilder,
+InvalidFieldReferenceException, FlatFieldDescriptor}
import org.apache.flink.api.common.typeutils._
import org.apache.flink.api.java.operators.Keys.ExpressionKeys
-import org.apache.flink.api.java.typeutils.{TupleTypeInfoBase, PojoTypeInfo}
-import PojoTypeInfo.NamedFlatFieldDescriptor
+import org.apache.flink.api.java.typeutils.TupleTypeInfoBase
import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
/**
* TypeInformation for Case Classes. Creation and access is different from
@@ -38,7 +38,7 @@ import scala.collection.JavaConverters._
*/
abstract class CaseClassTypeInfo[T <: Product](
clazz: Class[T],
- typeParamTypeInfos: Array[TypeInformation[_]],
+ val typeParamTypeInfos: Array[TypeInformation[_]],
fieldTypes: Seq[TypeInformation[_]],
val fieldNames: Seq[String])
extends TupleTypeInfoBase[T](clazz, fieldTypes: _*) {
@@ -63,46 +63,19 @@ abstract class CaseClassTypeInfo[T <: Product](
fields map { x => fieldNames.indexOf(x) }
}
- /*
- * Comparator construction
- */
- var fieldComparators: Array[TypeComparator[_]] = null
- var logicalKeyFields : Array[Int] = null
- var comparatorHelperIndex = 0
-
- override protected def initializeNewComparator(localKeyCount: Int): Unit = {
- fieldComparators = new Array(localKeyCount)
- logicalKeyFields = new Array(localKeyCount)
- comparatorHelperIndex = 0
- }
-
- override protected def addCompareField(fieldId: Int, comparator: TypeComparator[_]): Unit = {
- fieldComparators(comparatorHelperIndex) = comparator
- logicalKeyFields(comparatorHelperIndex) = fieldId
- comparatorHelperIndex += 1
- }
-
- override protected def getNewComparator(executionConfig: ExecutionConfig): TypeComparator[T] = {
- val finalLogicalKeyFields = logicalKeyFields.take(comparatorHelperIndex)
- val finalComparators = fieldComparators.take(comparatorHelperIndex)
- val maxKey = finalLogicalKeyFields.max
-
- // create serializers only up to the last key, fields after that are not needed
- val fieldSerializers = types.take(maxKey + 1).map(_.createSerializer(executionConfig))
- new CaseClassComparator[T](finalLogicalKeyFields, finalComparators, fieldSerializers.toArray)
- }
-
override def getFlatFields(
fieldExpression: String,
offset: Int,
result: java.util.List[FlatFieldDescriptor]): Unit = {
val matcher: Matcher = PATTERN_NESTED_FIELDS_WILDCARD.matcher(fieldExpression)
+
if (!matcher.matches) {
throw new InvalidFieldReferenceException("Invalid tuple field reference \"" +
fieldExpression + "\".")
}
var field: String = matcher.group(0)
+
if ((field == ExpressionKeys.SELECT_ALL_CHAR) ||
(field == ExpressionKeys.SELECT_ALL_CHAR_SCALA)) {
var keyPosition: Int = 0
@@ -116,59 +89,58 @@ abstract class CaseClassTypeInfo[T <: Product](
}
keyPosition += 1
}
- return
} else {
field = matcher.group(1)
- }
-
- val intFieldMatcher = PATTERN_INT_FIELD.matcher(field)
- if(intFieldMatcher.matches()) {
- // convert 0-indexed integer field into 1-indexed name field
- field = "_" + (Integer.valueOf(field) + 1)
- }
- var pos = offset
- val tail = matcher.group(3)
- if (tail == null) {
+ val intFieldMatcher = PATTERN_INT_FIELD.matcher(field)
+ if(intFieldMatcher.matches()) {
+ // convert 0-indexed integer field into 1-indexed name field
+ field = "_" + (Integer.valueOf(field) + 1)
+ }
- for (i <- 0 until fieldNames.length) {
- if (field == fieldNames(i)) {
- // found field
- fieldTypes(i) match {
- case ct: CompositeType[_] =>
- ct.getFlatFields("*", pos, result)
- return
- case _ =>
- result.add(new FlatFieldDescriptor(pos, fieldTypes(i)))
- return
+ val tail = matcher.group(3)
+
+ if (tail == null) {
+ def extractFlatFields(index: Int, pos: Int): Unit = {
+ if (index >= fieldNames.size) {
+ throw new InvalidFieldReferenceException("Unable to find field \"" + field +
+ "\" in type " + this + ".")
+ } else if (field == fieldNames(index)) {
+ // found field
+ fieldTypes(index) match {
+ case ct: CompositeType[_] =>
+ ct.getFlatFields("*", pos, result)
+ case _ =>
+ result.add(new FlatFieldDescriptor(pos, fieldTypes(index)))
+ }
+ } else {
+ // skipping over non-matching fields
+ extractFlatFields(index + 1, pos + fieldTypes(index).getTotalFields())
}
- } else {
- // skipping over non-matching fields
- pos += fieldTypes(i).getTotalFields
}
- }
- throw new InvalidFieldReferenceException("Unable to find field \"" + field +
- "\" in type " + this + ".")
- } else {
- var pos = offset
- for (i <- 0 until fieldNames.length) {
- if (field == fieldNames(i)) {
- // found field
- fieldTypes(i) match {
- case ct: CompositeType[_] =>
- ct.getFlatFields(tail, pos, result)
- return
- case _ =>
- throw new InvalidFieldReferenceException("Nested field expression \"" + tail +
- "\" not possible on atomic type " + fieldTypes(i) + ".")
+
+ extractFlatFields(0, offset)
+ } else {
+ def extractFlatFields(index: Int, pos: Int): Unit = {
+ if (index >= fieldNames.size) {
+ throw new InvalidFieldReferenceException("Unable to find field \"" + field +
+ "\" in type " + this + ".")
+ } else if (field == fieldNames(index)) {
+ // found field
+ fieldTypes(index) match {
+ case ct: CompositeType[_] =>
+ ct.getFlatFields(tail, pos, result)
+ case _ =>
+ throw new InvalidFieldReferenceException("Nested field expression \"" + tail +
+ "\" not possible on atomic type " + fieldTypes(index) + ".")
+ }
+ } else {
+ extractFlatFields(index + 1, pos + fieldTypes(index).getTotalFields())
}
- } else {
- // skipping over non-matching fields
- pos += fieldTypes(i).getTotalFields
}
+
+ extractFlatFields(0, offset)
}
- throw new InvalidFieldReferenceException("Unable to find field \"" + field +
- "\" in type " + this + ".")
}
}
@@ -195,7 +167,7 @@ abstract class CaseClassTypeInfo[T <: Product](
field = "_" + (Integer.valueOf(field) + 1)
}
- for (i <- 0 until fieldNames.length) {
+ for (i <- fieldNames.indices) {
if (fieldNames(i) == field) {
if (tail == null) {
return getTypeAt(i)
@@ -225,9 +197,57 @@ abstract class CaseClassTypeInfo[T <: Product](
}
}
- override def toString = clazz.getName + "(" + fieldNames.zip(types).map {
- case (n, t) => n + ": " + t}
- .mkString(", ") + ")"
+ override def createTypeComparatorBuilder(): TypeComparatorBuilder[T] = {
+ new CaseClassTypeComparatorBuilder
+ }
+
+ private class CaseClassTypeComparatorBuilder extends TypeComparatorBuilder[T] {
+ val fieldComparators: ArrayBuffer[TypeComparator[_]] = new ArrayBuffer[TypeComparator[_]]()
+ val logicalKeyFields: ArrayBuffer[Int] = new ArrayBuffer[Int]()
+
+ override def initializeTypeComparatorBuilder(size: Int): Unit = {}
+
+ override def addComparatorField(fieldId: Int, comparator: TypeComparator[_]): Unit = {
+ fieldComparators += comparator
+ logicalKeyFields += fieldId
+ }
+
+ override def createTypeComparator(config: ExecutionConfig): TypeComparator[T] = {
+ val maxIndex = logicalKeyFields.max
+
+ new CaseClassComparator[T](
+ logicalKeyFields.toArray,
+ fieldComparators.toArray,
+ types.take(maxIndex + 1).map(_.createSerializer(config))
+ )
+ }
+ }
+
+ override def toString: String = {
+ clazz.getName + "(" + fieldNames.zip(types).map {
+ case (n, t) => n + ": " + t
+ }.mkString(", ") + ")"
+ }
override def isCaseClass = true
+
+ override def equals(obj: Any): Boolean = {
+ obj match {
+ case caseClass: CaseClassTypeInfo[_] =>
+ caseClass.canEqual(this) &&
+ super.equals(caseClass) &&
+ typeParamTypeInfos.sameElements(caseClass.typeParamTypeInfos) &&
+ fieldNames.equals(caseClass.fieldNames)
+ case _ => false
+ }
+ }
+
+ override def hashCode(): Int = {
+ 31 * (31 * super.hashCode() + fieldNames.hashCode()) +
+ util.Arrays.hashCode(typeParamTypeInfos.asInstanceOf[Array[AnyRef]])
+ }
+
+ override def canEqual(obj: Any): Boolean = {
+ obj.isInstanceOf[CaseClassTypeInfo[_]]
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherSerializer.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherSerializer.scala
index 65628a0..2efc207 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherSerializer.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherSerializer.scala
@@ -86,11 +86,20 @@ class EitherSerializer[A, B, T <: Either[A, B]](
}
override def equals(obj: Any): Boolean = {
- if (obj != null && obj.isInstanceOf[EitherSerializer[_, _, _]]) {
- val other = obj.asInstanceOf[EitherSerializer[_, _, _]]
- other.leftSerializer.equals(leftSerializer) && other.rightSerializer.equals(rightSerializer)
- } else {
- false
+ obj match {
+ case eitherSerializer: EitherSerializer[_, _, _] =>
+ eitherSerializer.canEqual(this) &&
+ leftSerializer.equals(eitherSerializer.leftSerializer) &&
+ rightSerializer.equals(eitherSerializer.rightSerializer)
+ case _ => false
}
}
+
+ override def canEqual(obj: Any): Boolean = {
+ obj.isInstanceOf[EitherSerializer[_, _, _]]
+ }
+
+ override def hashCode(): Int = {
+ 31 * leftSerializer.hashCode() + rightSerializer.hashCode()
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherTypeInfo.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherTypeInfo.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherTypeInfo.scala
index a1cded7..2beebde 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherTypeInfo.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherTypeInfo.scala
@@ -27,9 +27,9 @@ import scala.collection.JavaConverters._
* TypeInformation [[Either]].
*/
class EitherTypeInfo[A, B, T <: Either[A, B]](
- clazz: Class[T],
- leftTypeInfo: TypeInformation[A],
- rightTypeInfo: TypeInformation[B])
+ val clazz: Class[T],
+ val leftTypeInfo: TypeInformation[A],
+ val rightTypeInfo: TypeInformation[B])
extends TypeInformation[T] {
override def isBasicType: Boolean = false
@@ -55,5 +55,24 @@ class EitherTypeInfo[A, B, T <: Either[A, B]](
new EitherSerializer(leftSerializer, rightSerializer)
}
+ override def equals(obj: Any): Boolean = {
+ obj match {
+ case eitherTypeInfo: EitherTypeInfo[_, _, _] =>
+ eitherTypeInfo.canEqual(this) &&
+ clazz.equals(eitherTypeInfo.clazz) &&
+ leftTypeInfo.equals(eitherTypeInfo.leftTypeInfo) &&
+ rightTypeInfo.equals(eitherTypeInfo.rightTypeInfo)
+ case _ => false
+ }
+ }
+
+ override def canEqual(obj: Any): Boolean = {
+ obj.isInstanceOf[EitherTypeInfo[_, _, _]]
+ }
+
+ override def hashCode(): Int = {
+ 31 * (31 * clazz.hashCode() + leftTypeInfo.hashCode()) + rightTypeInfo.hashCode()
+ }
+
override def toString = s"Either[$leftTypeInfo, $rightTypeInfo]"
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala
index 8d03676..7c3bc95 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala
@@ -51,15 +51,18 @@ class EnumValueSerializer[E <: Enumeration](val enum: E) extends TypeSerializer[
override def deserialize(reuse: T, source: DataInputView): T = deserialize(source)
override def equals(obj: Any): Boolean = {
- if (obj != null && obj.isInstanceOf[EnumValueSerializer[_]]) {
- val other = obj.asInstanceOf[EnumValueSerializer[_]]
- this.enum == other.enum
- } else {
- false
+ obj match {
+ case enumValueSerializer: EnumValueSerializer[_] =>
+ enumValueSerializer.canEqual(this) && enum == enumValueSerializer.enum
+ case _ => false
}
}
override def hashCode(): Int = {
enum.hashCode()
}
+
+ override def canEqual(obj: scala.Any): Boolean = {
+ obj.isInstanceOf[EnumValueSerializer[_]]
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueTypeInfo.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueTypeInfo.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueTypeInfo.scala
index c66e4bc..e3d665e 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueTypeInfo.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueTypeInfo.scala
@@ -26,7 +26,7 @@ import scala.collection.JavaConverters._
/**
* TypeInformation for [[Enumeration]] values.
*/
-class EnumValueTypeInfo[E <: Enumeration](enum: E, clazz: Class[E#Value])
+class EnumValueTypeInfo[E <: Enumeration](val enum: E, val clazz: Class[E#Value])
extends TypeInformation[E#Value] with AtomicType[E#Value] {
type T = E#Value
@@ -49,4 +49,22 @@ class EnumValueTypeInfo[E <: Enumeration](enum: E, clazz: Class[E#Value])
}
override def toString = clazz.getCanonicalName
+
+ override def hashCode(): Int = {
+ 31 * enum.hashCode() + clazz.hashCode()
+ }
+
+ override def equals(obj: Any): Boolean = {
+ obj match {
+ case enumValueTypeInfo: EnumValueTypeInfo[E] =>
+ enumValueTypeInfo.canEqual(this) &&
+ enum.equals(enumValueTypeInfo.enum) &&
+ clazz.equals(enumValueTypeInfo.clazz)
+ case _ => false
+ }
+ }
+
+ override def canEqual(obj: Any): Boolean = {
+ obj.isInstanceOf[EnumValueTypeInfo[E]]
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/NothingSerializer.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/NothingSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/NothingSerializer.scala
index 147a060..a6a7954 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/NothingSerializer.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/NothingSerializer.scala
@@ -56,6 +56,17 @@ class NothingSerializer extends TypeSerializer[Any] {
throw new RuntimeException("This must not be used. You encountered a bug.")
override def equals(obj: Any): Boolean = {
- obj != null && obj.isInstanceOf[NothingSerializer]
+ obj match {
+ case nothingSerializer: NothingSerializer => nothingSerializer.canEqual(this)
+ case _ => false
+ }
+ }
+
+ override def canEqual(obj: scala.Any): Boolean = {
+ obj.isInstanceOf[NothingSerializer]
+ }
+
+ override def hashCode(): Int = {
+ classOf[NothingSerializer].hashCode()
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionSerializer.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionSerializer.scala
index 488710d..af2091b 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionSerializer.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionSerializer.scala
@@ -71,11 +71,18 @@ class OptionSerializer[A](val elemSerializer: TypeSerializer[A])
override def deserialize(reuse: Option[A], source: DataInputView): Option[A] = deserialize(source)
override def equals(obj: Any): Boolean = {
- if (obj != null && obj.isInstanceOf[OptionSerializer[_]]) {
- val other = obj.asInstanceOf[OptionSerializer[_]]
- other.elemSerializer.equals(elemSerializer)
- } else {
- false
+ obj match {
+ case optionSerializer: OptionSerializer[_] =>
+ optionSerializer.canEqual(this) && elemSerializer.equals(optionSerializer.elemSerializer)
+ case _ => false
}
}
+
+ override def canEqual(obj: scala.Any): Boolean = {
+ obj.isInstanceOf[OptionSerializer[_]]
+ }
+
+ override def hashCode(): Int = {
+ elemSerializer.hashCode()
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionTypeInfo.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionTypeInfo.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionTypeInfo.scala
index 510b604..2aff2dd 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionTypeInfo.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionTypeInfo.scala
@@ -26,7 +26,7 @@ import scala.collection.JavaConverters._
/**
* TypeInformation for [[Option]].
*/
-class OptionTypeInfo[A, T <: Option[A]](elemTypeInfo: TypeInformation[A])
+class OptionTypeInfo[A, T <: Option[A]](private val elemTypeInfo: TypeInformation[A])
extends TypeInformation[T] {
override def isBasicType: Boolean = false
@@ -49,4 +49,20 @@ class OptionTypeInfo[A, T <: Option[A]](elemTypeInfo: TypeInformation[A])
}
override def toString = s"Option[$elemTypeInfo]"
+
+ override def equals(obj: Any): Boolean = {
+ obj match {
+ case optTpe: OptionTypeInfo[_, _] =>
+ optTpe.canEqual(this) && elemTypeInfo.equals(optTpe.elemTypeInfo)
+ case _ => false
+ }
+ }
+
+ def canEqual(obj: Any): Boolean = {
+ obj.isInstanceOf[OptionTypeInfo[_, _]]
+ }
+
+ override def hashCode: Int = {
+ elemTypeInfo.hashCode()
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala
index 38fd14b..d242863 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala
@@ -134,11 +134,18 @@ abstract class TraversableSerializer[T <: TraversableOnce[E], E](
}
override def equals(obj: Any): Boolean = {
- if (obj != null && obj.isInstanceOf[TraversableSerializer[_, _]]) {
- val other = obj.asInstanceOf[TraversableSerializer[_, _]]
- other.elementSerializer.equals(elementSerializer)
- } else {
- false
+ obj match {
+ case other: TraversableSerializer[_, _] =>
+ other.canEqual(this) && elementSerializer.equals(other.elementSerializer)
+ case _ => false
}
}
+
+ override def hashCode(): Int = {
+ elementSerializer.hashCode()
+ }
+
+ override def canEqual(obj: Any): Boolean = {
+ obj.isInstanceOf[TraversableSerializer[_, _]]
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableTypeInfo.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableTypeInfo.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableTypeInfo.scala
index 76067bb..8948b0c 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableTypeInfo.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableTypeInfo.scala
@@ -23,13 +23,11 @@ import org.apache.flink.api.common.typeutils.TypeSerializer
import scala.collection.JavaConverters._
-import scala.collection.generic.CanBuildFrom
-
/**
* TypeInformation for Scala Collections.
*/
abstract class TraversableTypeInfo[T <: TraversableOnce[E], E](
- clazz: Class[T],
+ val clazz: Class[T],
val elementTypeInfo: TypeInformation[E])
extends TypeInformation[T] {
@@ -41,17 +39,25 @@ abstract class TraversableTypeInfo[T <: TraversableOnce[E], E](
override def getTypeClass: Class[T] = clazz
override def getGenericParameters = List[TypeInformation[_]](elementTypeInfo).asJava
-
def createSerializer(executionConfig: ExecutionConfig): TypeSerializer[T]
override def equals(other: Any): Boolean = {
- if (other.isInstanceOf[TraversableTypeInfo[_, _]]) {
- val otherTrav = other.asInstanceOf[TraversableTypeInfo[_, _]]
- otherTrav.getTypeClass == getTypeClass && otherTrav.elementTypeInfo == elementTypeInfo
- } else {
- false
+ other match {
+ case traversable: TraversableTypeInfo[_, _] =>
+ traversable.canEqual(this) &&
+ clazz == traversable.clazz &&
+ elementTypeInfo.equals(traversable.elementTypeInfo)
+ case _ => false
}
}
+ override def hashCode(): Int = {
+ 31 * clazz.hashCode() + elementTypeInfo.hashCode()
+ }
+
+ override def canEqual(obj: Any): Boolean = {
+ obj.isInstanceOf[TraversableTypeInfo[_, _]]
+ }
+
override def toString = s"$clazz[$elementTypeInfo]"
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TrySerializer.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TrySerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TrySerializer.scala
index b7ceadf..99aae21 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TrySerializer.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TrySerializer.scala
@@ -27,7 +27,9 @@ import scala.util.{Success, Try, Failure}
/**
* Serializer for [[scala.util.Try]].
*/
-class TrySerializer[A](val elemSerializer: TypeSerializer[A], executionConfig: ExecutionConfig)
+class TrySerializer[A](
+ private val elemSerializer: TypeSerializer[A],
+ private val executionConfig: ExecutionConfig)
extends TypeSerializer[Try[A]] {
override def duplicate: TrySerializer[A] = this
@@ -80,11 +82,18 @@ class TrySerializer[A](val elemSerializer: TypeSerializer[A], executionConfig: E
override def deserialize(reuse: Try[A], source: DataInputView): Try[A] = deserialize(source)
override def equals(obj: Any): Boolean = {
- if (obj != null && obj.isInstanceOf[TrySerializer[_]]) {
- val other = obj.asInstanceOf[TrySerializer[_]]
- other.elemSerializer.equals(elemSerializer)
- } else {
- false
+ obj match {
+ case other: TrySerializer[_] =>
+ other.canEqual(this) && elemSerializer.equals(other.elemSerializer)
+ case _ => false
}
}
+
+ override def canEqual(obj: Any): Boolean = {
+ obj.isInstanceOf[TrySerializer[_]]
+ }
+
+ override def hashCode(): Int = {
+ 31 * elemSerializer.hashCode() + executionConfig.hashCode()
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TryTypeInfo.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TryTypeInfo.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TryTypeInfo.scala
index 3749b37..f3f2ce2 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TryTypeInfo.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TryTypeInfo.scala
@@ -28,7 +28,7 @@ import scala.util.Try
/**
* TypeInformation for [[scala.util.Try]].
*/
-class TryTypeInfo[A, T <: Try[A]](elemTypeInfo: TypeInformation[A])
+class TryTypeInfo[A, T <: Try[A]](val elemTypeInfo: TypeInformation[A])
extends TypeInformation[T] {
override def isBasicType: Boolean = false
@@ -49,5 +49,22 @@ class TryTypeInfo[A, T <: Try[A]](elemTypeInfo: TypeInformation[A])
}
}
+ override def equals(obj: Any): Boolean = {
+ obj match {
+ case tryTypeInfo: TryTypeInfo[_, _] =>
+ tryTypeInfo.canEqual(this) &&
+ elemTypeInfo.equals(tryTypeInfo.elemTypeInfo)
+ case _ => false
+ }
+ }
+
+ override def canEqual(obj: Any): Boolean = {
+ obj.isInstanceOf[TryTypeInfo[_, _]]
+ }
+
+ override def hashCode(): Int = {
+ elemTypeInfo.hashCode()
+ }
+
override def toString = s"Try[$elemTypeInfo]"
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/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
new file mode 100644
index 0000000..479483f
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/CaseClassTypeInfoTest.scala
@@ -0,0 +1,73 @@
+/*
+ * 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.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
+
+class CaseClassTypeInfoTest extends TestLogger with JUnitSuiteLike {
+
+ @Test
+ def testCaseClassTypeInfoEquality(): 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, 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]],
+ 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/8ca853e0/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
new file mode 100644
index 0000000..e23a6a0
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EitherTypeInfoTest.scala
@@ -0,0 +1,59 @@
+/*
+ * 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.typeinfo.BasicTypeInfo
+import org.apache.flink.util.TestLogger
+import org.junit.Test
+import org.scalatest.junit.JUnitSuiteLike
+
+class EitherTypeInfoTest extends TestLogger with JUnitSuiteLike {
+
+ @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]](
+ classOf[Either[Integer,Integer]],
+ BasicTypeInfo.INT_TYPE_INFO,
+ BasicTypeInfo.INT_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))
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/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
new file mode 100644
index 0000000..acd6a39
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueTypeInfoTest.scala
@@ -0,0 +1,64 @@
+/*
+ * 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.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
+ def testEnumValueTypeInfoEquality(): Unit = {
+ val enumTypeInfo1 = 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](
+ AlternateEnumeration,
+ classOf[AlternateEnumeration])
+
+ assert(!enumTypeInfo1.equals(enumTypeInfo2))
+ }
+
+}
+
+object TestEnumeration extends Enumeration {
+ type TestEnumeration = Value
+ val ONE = this.Value
+}
+
+object AlternateEnumeration extends Enumeration {
+ type AlternateEnumeration = Value
+ val TWO = this.Value
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/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
new file mode 100644
index 0000000..b765658
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/OptionTypeInfoTest.scala
@@ -0,0 +1,54 @@
+/*
+ * 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.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}
+
+class OptionTypeInfoTest extends TestLogger with JUnitSuiteLike {
+
+ @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))
+ }
+}
+
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/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
new file mode 100644
index 0000000..e83b326
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/TraversableTypeInfoTest.scala
@@ -0,0 +1,68 @@
+/*
+ * 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.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
+
+class TraversableTypeInfoTest extends TestLogger with JUnitSuiteLike {
+
+ @Test
+ def testTraversableTypeInfoEquality(): 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[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](
+ 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/8ca853e0/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
new file mode 100644
index 0000000..3a5fc80
--- /dev/null
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/TryTypeInfoTest.scala
@@ -0,0 +1,55 @@
+/*
+ * 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.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 scala.util.Try
+
+class TryTypeInfoTest extends TestLogger with JUnitSuiteLike {
+
+ @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)
+
+ 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]])
+
+ assert(!TryTypeInfo.equals(genericTypeInfo))
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindowSerializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindowSerializer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindowSerializer.java
index 229cb4a..529850f 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindowSerializer.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindowSerializer.java
@@ -20,6 +20,7 @@ package org.apache.flink.streaming.api.windowing;
import java.io.IOException;
+import com.google.common.base.Preconditions;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -37,6 +38,8 @@ public final class StreamWindowSerializer<T> extends TypeSerializer<StreamWindow
TypeSerializer<Boolean> boolSerializer = BooleanSerializer.INSTANCE;
public StreamWindowSerializer(TypeInformation<T> typeInfo, ExecutionConfig conf) {
+ Preconditions.checkNotNull(typeInfo);
+
this.typeSerializer = typeInfo.createSerializer(conf);
}
@@ -118,6 +121,27 @@ public final class StreamWindowSerializer<T> extends TypeSerializer<StreamWindow
}
@Override
+ public boolean equals(Object obj) {
+ if (obj instanceof StreamWindowSerializer) {
+ StreamWindowSerializer<?> other = (StreamWindowSerializer<?>) obj;
+
+ return other.canEqual(this) && typeSerializer.equals(other.typeSerializer);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof StreamWindowSerializer;
+ }
+
+ @Override
+ public int hashCode() {
+ return typeSerializer.hashCode();
+ }
+
+ @Override
public TypeSerializer<StreamWindow<T>> duplicate() {
return this;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindowTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindowTypeInfo.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindowTypeInfo.java
index 3bcc253..2c0a999 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindowTypeInfo.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/windowing/StreamWindowTypeInfo.java
@@ -18,6 +18,7 @@
package org.apache.flink.streaming.api.windowing;
+import com.google.common.base.Preconditions;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -25,10 +26,11 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
public class StreamWindowTypeInfo<T> extends TypeInformation<StreamWindow<T>> {
private static final long serialVersionUID = 1L;
- TypeInformation<T> innerType;
+
+ final TypeInformation<T> innerType;
public StreamWindowTypeInfo(TypeInformation<T> innerType) {
- this.innerType = innerType;
+ this.innerType = Preconditions.checkNotNull(innerType);
}
public TypeInformation<T> getInnerType() {
@@ -50,10 +52,10 @@ public class StreamWindowTypeInfo<T> extends TypeInformation<StreamWindow<T>> {
return innerType.getArity();
}
+ @SuppressWarnings("unchecked")
@Override
public Class<StreamWindow<T>> getTypeClass() {
- // TODO Auto-generated method stub
- return null;
+ return (Class<StreamWindow<T>>)(Object)StreamWindow.class;
}
@Override
@@ -67,6 +69,34 @@ public class StreamWindowTypeInfo<T> extends TypeInformation<StreamWindow<T>> {
}
@Override
+ public String toString() {
+ return getClass().getSimpleName() + "<" + innerType + ">";
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof StreamWindowTypeInfo) {
+ @SuppressWarnings("unchecked")
+ StreamWindowTypeInfo<T> streamWindowTypeInfo = (StreamWindowTypeInfo<T>) obj;
+
+ return streamWindowTypeInfo.canEqual(this) &&
+ innerType.equals(streamWindowTypeInfo.innerType);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return innerType.hashCode();
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof StreamWindowTypeInfo;
+ }
+
+ @Override
public int getTotalFields() {
return innerType.getTotalFields();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java
index 156e5d6..d4363cd 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java
@@ -164,4 +164,25 @@ public final class MultiplexingStreamRecordSerializer<T> extends TypeSerializer<
typeSerializer.copy(source, target);
}
}
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof MultiplexingStreamRecordSerializer) {
+ MultiplexingStreamRecordSerializer<?> other = (MultiplexingStreamRecordSerializer<?>) obj;
+
+ return other.canEqual(this) && typeSerializer.equals(other.typeSerializer);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof MultiplexingStreamRecordSerializer;
+ }
+
+ @Override
+ public int hashCode() {
+ return typeSerializer.hashCode();
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/8ca853e0/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializer.java
index e58d3c8..d47da50 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializer.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecordSerializer.java
@@ -122,4 +122,25 @@ public final class StreamRecordSerializer<T> extends TypeSerializer<StreamRecord
public void copy(DataInputView source, DataOutputView target) throws IOException {
typeSerializer.copy(source, target);
}
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof StreamRecordSerializer) {
+ StreamRecordSerializer<?> other = (StreamRecordSerializer<?>) obj;
+
+ return other.canEqual(this) && typeSerializer.equals(other.typeSerializer);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof StreamRecordSerializer;
+ }
+
+ @Override
+ public int hashCode() {
+ return typeSerializer.hashCode();
+ }
}