From 7407076d3990752eb5fa4072cd036efd2f656cbc Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 26 Nov 2014 13:27:06 +0100 Subject: [PATCH] [FLINK-1369] [types] Add support for Subclasses, Interfaces, Abstract Classes. - Abstract classes with fields are handled as POJO types. - Interfaces and abstract classes without fields are handled as generic types. This closes #236 This closes #316 --- .../compiler/postpass/JavaApiPostPass.java | 15 +- .../flink/compiler/util/NoOpBinaryUdfOp.java | 3 +- .../flink/compiler/util/NoOpUnaryUdfOp.java | 3 +- flink-core/pom.xml | 48 +- .../flink/api/common/ExecutionConfig.java | 160 +++++- .../api/common/functions/RuntimeContext.java | 7 + .../util/AbstractRuntimeUDFContext.java | 17 +- .../functions/util/RuntimeUDFContext.java | 9 +- .../common/operators/CollectionExecutor.java | 32 +- .../common/operators/DualInputOperator.java | 3 +- .../common/operators/GenericDataSinkBase.java | 7 +- .../operators/GenericDataSourceBase.java | 5 +- .../common/operators/SingleInputOperator.java | 3 +- .../flink/api/common/operators/Union.java | 3 +- .../operators/base/BulkIterationBase.java | 3 +- .../operators/base/CoGroupOperatorBase.java | 27 +- .../base/CollectorMapOperatorBase.java | 3 +- .../operators/base/CrossOperatorBase.java | 13 +- .../operators/base/DeltaIterationBase.java | 3 +- .../operators/base/FilterOperatorBase.java | 3 +- .../operators/base/FlatMapOperatorBase.java | 11 +- .../base/GroupReduceOperatorBase.java | 23 +- .../operators/base/JoinOperatorBase.java | 21 +- .../operators/base/MapOperatorBase.java | 11 +- .../base/MapPartitionOperatorBase.java | 11 +- .../operators/base/PartitionOperatorBase.java | 3 +- .../operators/base/ReduceOperatorBase.java | 14 +- .../flink/api/common/typeinfo/AtomicType.java | 3 +- .../common/typeinfo/BasicArrayTypeInfo.java | 5 +- .../api/common/typeinfo/BasicTypeInfo.java | 5 +- .../api/common/typeinfo/NothingTypeInfo.java | 3 +- .../typeinfo/PrimitiveArrayTypeInfo.java | 3 +- .../api/common/typeinfo/TypeInformation.java | 9 +- .../api/common/typeutils/CompositeType.java | 11 +- .../functions/util/RuntimeUDFContextTest.java | 11 +- .../base/FlatMapOperatorCollectionTest.java | 9 +- .../operators/base/JoinOperatorBaseTest.java | 15 +- .../operators/base/MapOperatorTest.java | 16 +- .../base/PartitionMapOperatorTest.java | 10 +- .../common/typeutils/SerializerTestBase.java | 2 +- .../flink/api/java/CollectionEnvironment.java | 2 +- .../org/apache/flink/api/java/DataSet.java | 12 +- .../flink/api/java/ExecutionEnvironment.java | 43 +- .../flink/api/java/io/CsvOutputFormat.java | 3 +- .../java/io/LocalCollectionOutputFormat.java | 5 +- .../java/io/TypeSerializerInputFormat.java | 4 +- .../java/io/TypeSerializerOutputFormat.java | 5 +- .../api/java/operators/CrossOperator.java | 4 +- .../api/java/operators/JoinOperator.java | 4 +- .../api/java/operators/ProjectOperator.java | 4 +- .../translation/PlanProjectOperator.java | 5 +- .../api/java/typeutils/EnumTypeInfo.java | 5 +- .../api/java/typeutils/GenericTypeInfo.java | 12 +- .../java/typeutils/InputTypeConfigurable.java | 9 +- .../api/java/typeutils/MissingTypeInfo.java | 3 +- .../java/typeutils/ObjectArrayTypeInfo.java | 7 +- .../api/java/typeutils/PojoTypeInfo.java | 17 +- .../api/java/typeutils/RecordTypeInfo.java | 3 +- .../api/java/typeutils/TupleTypeInfo.java | 11 +- .../api/java/typeutils/TypeExtractor.java | 1 + .../api/java/typeutils/ValueTypeInfo.java | 5 +- .../api/java/typeutils/WritableTypeInfo.java | 5 +- .../typeutils/runtime/KryoSerializer.java | 96 +--- .../typeutils/runtime/PojoSerializer.java | 459 ++++++++++++++---- .../base/CoGroupOperatorCollectionTest.java | 20 +- .../base/GroupReduceOperatorTest.java | 17 +- .../operators/base/JoinOperatorBaseTest.java | 9 +- .../operators/base/ReduceOperatorTest.java | 15 +- .../java/io/CollectionInputFormatTest.java | 5 +- .../api/java/io/TypeSerializerFormatTest.java | 3 +- .../type/extractor/TypeExtractorTest.java | 38 +- .../api/java/typeutils/CompositeTypeTest.java | 3 +- .../java/typeutils/TypeInfoParserTest.java | 8 +- .../runtime/CopyableValueComparatorTest.java | 2 - .../KryoGenericArraySerializerTest.java | 3 +- .../KryoGenericTypeComparatorTest.java | 3 +- .../KryoGenericTypeSerializerTest.java | 5 +- .../runtime/KryoVersusAvroMinibenchmark.java | 13 +- .../KryoWithCustomSerializersTest.java | 12 +- .../MultidimensionalArraySerializerTest.java | 11 +- .../typeutils/runtime/PojoComparatorTest.java | 5 +- .../PojoGenericTypeSerializerTest.java | 3 +- .../typeutils/runtime/PojoSerializerTest.java | 13 +- .../runtime/PojoSubclassComparatorTest.java | 76 +++ .../runtime/PojoSubclassSerializerTest.java | 196 ++++++++ .../SubclassFromInterfaceSerializerTest.java | 170 +++++++ .../runtime/TupleComparatorILD2Test.java | 2 - .../runtime/TupleComparatorILD3Test.java | 2 - .../runtime/TupleComparatorILDC3Test.java | 2 - .../runtime/TupleComparatorILDX1Test.java | 2 - .../runtime/TupleComparatorILDXC2Test.java | 2 - .../runtime/TupleComparatorISD1Test.java | 2 - .../runtime/TupleComparatorISD2Test.java | 2 - .../runtime/TupleComparatorISD3Test.java | 2 - .../runtime/TupleSerializerTest.java | 3 +- .../runtime/ValueComparatorTest.java | 2 - .../runtime/WritableComparatorTest.java | 2 - .../runtime/WritableSerializerTest.java | 4 +- .../type/lambdas/LambdaExtractionTest.java | 1 - .../task/AbstractIterativePactTask.java | 7 +- .../jobgraph/tasks/AbstractInvokable.java | 27 ++ .../runtime/operators/DataSourceTask.java | 18 - .../runtime/operators/PactTaskContext.java | 6 +- .../runtime/operators/RegularPactTask.java | 19 +- .../operators/chaining/ChainedDriver.java | 2 +- .../operators/sort/LargeRecordHandler.java | 11 +- .../util/DistributedRuntimeUDFContext.java | 9 +- .../drivers/AllGroupReduceDriverTest.java | 15 +- .../drivers/AllReduceDriverTest.java | 27 +- .../drivers/GroupReduceDriverTest.java | 39 +- .../drivers/ReduceCombineDriverTest.java | 39 +- .../operators/drivers/ReduceDriverTest.java | 39 +- .../sort/ExternalSortLargeRecordsITCase.java | 17 +- .../sort/LargeRecordHandlerITCase.java | 7 +- .../sort/LargeRecordHandlerTest.java | 13 +- .../sort/MassiveStringSortingITCase.java | 5 +- .../sort/MassiveStringValueSortingITCase.java | 5 +- .../operators/ScalaAggregateOperator.java | 15 +- .../scala/operators/ScalaCsvInputFormat.java | 7 +- .../scala/operators/ScalaCsvOutputFormat.java | 3 +- .../apache/flink/api/scala/CrossDataSet.scala | 5 +- .../api/scala/ExecutionEnvironment.scala | 8 +- .../scala/UnfinishedCoGroupOperation.scala | 6 +- .../api/scala/codegen/TypeAnalyzer.scala | 8 +- .../scala/codegen/TypeInformationGen.scala | 12 +- .../apache/flink/api/scala/joinDataSet.scala | 6 +- .../scala/typeutils/CaseClassTypeInfo.scala | 11 +- .../api/scala/typeutils/EitherTypeInfo.scala | 17 +- .../api/scala/typeutils/OptionTypeInfo.scala | 6 +- .../scala/typeutils/TraversableTypeInfo.scala | 3 +- .../api/scala/typeutils/TrySerializer.scala | 5 +- .../api/scala/typeutils/TryTypeInfo.scala | 8 +- .../mapred/HadoopReduceCombineFunction.java | 4 +- .../mapred/HadoopReduceFunction.java | 4 +- .../HadoopTupleUnwrappingIterator.java | 5 +- .../HadoopTupleUnwrappingIteratorTest.java | 4 +- .../flink/streaming/api/StreamGraph.java | 19 +- .../streaming/api/datastream/DataStream.java | 2 +- .../temporaloperator/StreamJoinOperator.java | 8 +- .../StreamExecutionEnvironment.java | 63 ++- .../aggregation/AggregationFunction.java | 4 +- .../aggregation/ComparableAggregator.java | 2 +- .../function/aggregation/SumAggregator.java | 2 +- .../function/source/FileSourceFunction.java | 4 +- .../api/invokable/StreamInvokable.java | 9 +- .../operator/GroupedWindowInvokable.java | 2 +- .../invokable/operator/ProjectInvokable.java | 2 +- .../invokable/operator/co/CoInvokable.java | 3 +- .../streamrecord/StreamRecordSerializer.java | 5 +- .../api/streamvertex/CoStreamVertex.java | 2 +- .../api/streamvertex/StreamVertex.java | 4 +- .../streamvertex/StreamingRuntimeContext.java | 5 +- .../streaming/util/keys/KeySelectorUtil.java | 6 +- .../api/AggregationFunctionTest.java | 3 +- .../flink/streaming/util/MockCoContext.java | 7 +- .../flink/streaming/util/MockContext.java | 5 +- .../streaming/api/scala/DataStream.scala | 6 +- .../api/scala/StreamCrossOperator.scala | 6 +- .../scala/StreamExecutionEnvironment.scala | 30 ++ .../api/scala/StreamJoinOperator.scala | 25 +- .../api/scala/WindowedDataStream.scala | 8 +- .../WordCountSubclassInterfacePOJOITCase.java | 152 ++++++ .../WordCountSubclassPOJOITCase.java | 123 +++++ .../scala/io/CollectionInputFormatTest.scala | 10 +- .../misc/MassiveCaseClassSortingITCase.scala | 9 +- .../runtime/CaseClassComparatorTest.scala | 9 +- .../KryoGenericTypeSerializerTest.scala | 13 +- .../ScalaSpecialTypesSerializerTest.scala | 6 +- .../runtime/TraversableSerializerTest.scala | 5 +- .../runtime/TupleComparatorILD2Test.scala | 8 +- .../runtime/TupleComparatorILD3Test.scala | 9 +- .../runtime/TupleComparatorILDC3Test.scala | 9 +- .../runtime/TupleComparatorILDX1Test.scala | 5 +- .../runtime/TupleComparatorILDXC2Test.scala | 5 +- .../runtime/TupleComparatorISD1Test.scala | 5 +- .../runtime/TupleComparatorISD2Test.scala | 5 +- .../runtime/TupleComparatorISD3Test.scala | 9 +- .../scala/runtime/TupleSerializerTest.scala | 10 +- .../scala/types/TypeInformationGenTest.scala | 1 + 179 files changed, 2211 insertions(+), 801 deletions(-) create mode 100644 flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSubclassComparatorTest.java create mode 100644 flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSubclassSerializerTest.java create mode 100644 flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/SubclassFromInterfaceSerializerTest.java create mode 100644 flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountSubclassInterfacePOJOITCase.java create mode 100644 flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountSubclassPOJOITCase.java diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/postpass/JavaApiPostPass.java b/flink-compiler/src/main/java/org/apache/flink/compiler/postpass/JavaApiPostPass.java index 208ff2e41e9f4..11ac231c13ccd 100644 --- a/flink-compiler/src/main/java/org/apache/flink/compiler/postpass/JavaApiPostPass.java +++ b/flink-compiler/src/main/java/org/apache/flink/compiler/postpass/JavaApiPostPass.java @@ -22,6 +22,7 @@ import java.util.HashSet; import java.util.Set; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.operators.DualInputOperator; import org.apache.flink.api.common.operators.GenericDataSourceBase; import org.apache.flink.api.common.operators.Operator; @@ -70,9 +71,13 @@ public class JavaApiPostPass implements OptimizerPostPass { private final Set alreadyDone = new HashSet(); + private ExecutionConfig executionConfig = null; @Override public void postPass(OptimizedPlan plan) { + + executionConfig = plan.getOriginalPactPlan().getExecutionConfig(); + for (SinkPlanNode sink : plan.getDataSinks()) { traverse(sink); } @@ -275,22 +280,22 @@ private static TypeInformation getTypeInfoFromSource(SourcePlanNode node) } } - private static TypeSerializerFactory createSerializer(TypeInformation typeInfo) { - TypeSerializer serializer = typeInfo.createSerializer(); + private TypeSerializerFactory createSerializer(TypeInformation typeInfo) { + TypeSerializer serializer = typeInfo.createSerializer(executionConfig); return new RuntimeSerializerFactory(serializer, typeInfo.getTypeClass()); } @SuppressWarnings("unchecked") - private static TypeComparatorFactory createComparator(TypeInformation typeInfo, FieldList keys, boolean[] sortOrder) { + private TypeComparatorFactory createComparator(TypeInformation typeInfo, FieldList keys, boolean[] sortOrder) { TypeComparator comparator; if (typeInfo instanceof CompositeType) { - comparator = ((CompositeType) typeInfo).createComparator(keys.toArray(), sortOrder, 0); + comparator = ((CompositeType) typeInfo).createComparator(keys.toArray(), sortOrder, 0, executionConfig); } else if (typeInfo instanceof AtomicType) { // handle grouping of atomic types - comparator = ((AtomicType) typeInfo).createComparator(sortOrder[0]); + comparator = ((AtomicType) typeInfo).createComparator(sortOrder[0], executionConfig); } else { throw new RuntimeException("Unrecognized type: " + typeInfo); diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/util/NoOpBinaryUdfOp.java b/flink-compiler/src/main/java/org/apache/flink/compiler/util/NoOpBinaryUdfOp.java index 4b48ec711a341..166b7b8f68c31 100644 --- a/flink-compiler/src/main/java/org/apache/flink/compiler/util/NoOpBinaryUdfOp.java +++ b/flink-compiler/src/main/java/org/apache/flink/compiler/util/NoOpBinaryUdfOp.java @@ -20,6 +20,7 @@ import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.NoOpFunction; import org.apache.flink.api.common.operators.BinaryOperatorInformation; @@ -43,7 +44,7 @@ public Class>[] getKeyClasses() { } @Override - protected List executeOnCollections(List inputData1, List inputData2, RuntimeContext runtimeContext, boolean mutables) { + protected List executeOnCollections(List inputData1, List inputData2, RuntimeContext runtimeContext, ExecutionConfig executionConfig) { throw new UnsupportedOperationException(); } } diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/util/NoOpUnaryUdfOp.java b/flink-compiler/src/main/java/org/apache/flink/compiler/util/NoOpUnaryUdfOp.java index 474d3a4b3a255..5013ae519dd4e 100644 --- a/flink-compiler/src/main/java/org/apache/flink/compiler/util/NoOpUnaryUdfOp.java +++ b/flink-compiler/src/main/java/org/apache/flink/compiler/util/NoOpUnaryUdfOp.java @@ -20,6 +20,7 @@ import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.NoOpFunction; import org.apache.flink.api.common.operators.RecordOperator; @@ -54,7 +55,7 @@ public UnaryOperatorInformation getOperatorInfo() { } @Override - protected List executeOnCollections(List inputData, RuntimeContext runtimeContext, boolean mutables) { + protected List executeOnCollections(List inputData, RuntimeContext runtimeContext, ExecutionConfig executionConfig) { return inputData; } } diff --git a/flink-core/pom.xml b/flink-core/pom.xml index 810860e3cd92f..182a77abde741 100644 --- a/flink-core/pom.xml +++ b/flink-core/pom.xml @@ -46,7 +46,18 @@ under the License. commons-collections - + + + com.esotericsoftware.kryo + kryo + + + + com.twitter + chill_2.10 + 0.5.1 + + com.google.guava @@ -72,4 +83,39 @@ under the License. + + + + hadoop-1 + + + + hadoop.profile1 + + + + + + org.apache.hadoop + hadoop-core + + + + + hadoop-2 + + + + !hadoop.profile + + + + + org.apache.hadoop + hadoop-common + + + + + 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 17e683f0b40f3..5fa01b7fef23b 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 @@ -18,7 +18,13 @@ package org.apache.flink.api.common; +import com.esotericsoftware.kryo.Serializer; + import java.io.Serializable; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; /** * A configuration config for configuring behavior of the system, such as whether to use @@ -34,11 +40,17 @@ public class ExecutionConfig implements Serializable { private boolean useClosureCleaner = true; private int degreeOfParallelism = -1; private int numberOfExecutionRetries = -1; + private boolean forceKryo = false; - // For future use... -// private boolean forceGenericSerializer = false; private boolean objectReuse = false; + + // Serializers and types registered with Kryo and the PojoSerializer + private final Map, Serializer> registeredKryoSerializers = new HashMap, Serializer>(); + private final Map, Class>> registeredKryoSerializersClasses = new HashMap, Class>>(); + private final Set> registeredKryoTypes = new HashSet>(); + private final Set> registeredPojoTypes = new HashSet>(); + /** * Enables the ClosureCleaner. This analyzes user code functions and sets fields to null * that are not used. This will in most cases make closures or anonymous inner classes @@ -128,21 +140,26 @@ public ExecutionConfig setNumberOfExecutionRetries(int numberOfExecutionRetries) return this; } - // These are for future use... -// public ExecutionConfig forceGenericSerializer() { -// forceGenericSerializer = true; -// return this; -// } -// -// public ExecutionConfig disableForceGenericSerializer() { -// forceGenericSerializer = false; -// return this; -// } -// -// public boolean isForceGenericSerializerEnabled() { -// return forceGenericSerializer; -// } -// + /** + * Force TypeExtractor to use Kryo serializer for POJOS even though we could analyze as POJO. + * In some cases this might be preferable. For example, when using interfaces + * with subclasses that cannot be analyzed as POJO. + */ + public void enableForceKryo() { + forceKryo = true; + } + + + /** + * Disable use of Kryo serializer for all POJOs. + */ + public void disableForceKryo() { + forceKryo = false; + } + + public boolean isForceKryoEnabled() { + return forceKryo; + } /** * Enables reusing objects that Flink internally uses for deserialization and passing @@ -169,4 +186,113 @@ public ExecutionConfig disableObjectReuse() { public boolean isObjectReuseEnabled() { return objectReuse; } + + // -------------------------------------------------------------------------------------------- + // Registry for types and serializers + // -------------------------------------------------------------------------------------------- + + /** + * Registers the given Serializer as a default serializer for the given type at the + * {@link org.apache.flink.api.common.typeutils.runtime.KryoSerializer}. + * + * Note that the serializer instance must be serializable (as defined by java.io.Serializable), + * because it may be distributed to the worker nodes by java serialization. + * + * @param type The class of the types serialized with the given serializer. + * @param serializer The serializer to use. + */ + public void registerKryoSerializer(Class type, Serializer serializer) { + if (type == null || serializer == null) { + throw new NullPointerException("Cannot register null class or serializer."); + } + if (!(serializer instanceof java.io.Serializable)) { + throw new IllegalArgumentException("The serializer instance must be serializable, (for distributing it in the cluster), " + + "as defined by java.io.Serializable. For stateless serializers, you can use the " + + "'registerSerializer(Class, Class)' method to register the serializer via its class."); + } + + registeredKryoSerializers.put(type, serializer); + } + + /** + * Registers the given Serializer via its class as a serializer for the given type at the + * {@link org.apache.flink.api.common.typeutils.runtime.KryoSerializer}. + * + * @param type The class of the types serialized with the given serializer. + * @param serializerClass The class of the serializer to use. + */ + public void registerKryoSerializer(Class type, Class> serializerClass) { + if (type == null || serializerClass == null) { + throw new NullPointerException("Cannot register null class or serializer."); + } + + registeredKryoSerializersClasses.put(type, serializerClass); + } + + /** + * Registers the given type with the serialization stack. If the type is eventually + * serialized as a POJO, then the type is registered with the POJO serializer. If the + * type ends up being serialized with Kryo, then it will be registered at Kryo to make + * sure that only tags are written. + * + * @param type The class of the type to register. + */ + public void registerPojoType(Class type) { + if (type == null) { + throw new NullPointerException("Cannot register null type class."); + } + registeredPojoTypes.add(type); + } + + /** + * Registers the given type with the serialization stack. If the type is eventually + * serialized as a POJO, then the type is registered with the POJO serializer. If the + * type ends up being serialized with Kryo, then it will be registered at Kryo to make + * sure that only tags are written. + * + * @param type The class of the type to register. + */ + public void registerKryoType(Class type) { + if (type == null) { + throw new NullPointerException("Cannot register null type class."); + } + registeredKryoTypes.add(type); + } + + /** + * Returns the registered Kryo Serializers. + */ + public Map, Serializer> getRegisteredKryoSerializers() { + return registeredKryoSerializers; + } + + /** + * Returns the registered Kryo Serializer classes. + */ + public Map, Class>> getRegisteredKryoSerializersClasses() { + return registeredKryoSerializersClasses; + } + + /** + * Returns the registered Kryo types. + */ + public Set> getRegisteredKryoTypes() { + if (isForceKryoEnabled()) { + // if we force kryo, we must also return all the types that + // were previously only registered as POJO + Set> result = new HashSet>(); + result.addAll(registeredKryoTypes); + result.addAll(registeredPojoTypes); + return result; + } else { + return registeredKryoTypes; + } + } + + /** + * Returns the registered POJO types. + */ + public Set> getRegisteredPojoTypes() { + return registeredPojoTypes; + } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java index ab938c028656b..e9209a8aabcd3 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java @@ -22,6 +22,7 @@ import java.util.HashMap; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.DoubleCounter; import org.apache.flink.api.common.accumulators.Histogram; @@ -60,6 +61,12 @@ public interface RuntimeContext { * @return The number of the parallel subtask. */ int getIndexOfThisSubtask(); + + /** + * Returns the {@link org.apache.flink.api.common.ExecutionConfig} for the currently executing + * job. + */ + ExecutionConfig getExecutionConfig(); /** * Gets the ClassLoader to load classes that were are not in system's classpath, but are part of the diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java index 6b755e14456ff..c04548c686dce 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java @@ -23,6 +23,7 @@ import java.util.Map; import java.util.concurrent.FutureTask; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.AccumulatorHelper; import org.apache.flink.api.common.accumulators.DoubleCounter; @@ -46,23 +47,31 @@ public abstract class AbstractRuntimeUDFContext implements RuntimeContext { private final ClassLoader userCodeClassLoader; + private final ExecutionConfig executionConfig; + private final HashMap> accumulators = new HashMap>(); private final DistributedCache distributedCache = new DistributedCache(); - public AbstractRuntimeUDFContext(String name, int numParallelSubtasks, int subtaskIndex, ClassLoader userCodeClassLoader) { + public AbstractRuntimeUDFContext(String name, int numParallelSubtasks, int subtaskIndex, ClassLoader userCodeClassLoader, ExecutionConfig executionConfig) { this.name = name; this.numParallelSubtasks = numParallelSubtasks; this.subtaskIndex = subtaskIndex; this.userCodeClassLoader = userCodeClassLoader; + this.executionConfig = executionConfig; } - public AbstractRuntimeUDFContext(String name, int numParallelSubtasks, int subtaskIndex, ClassLoader userCodeClassLoader, Map> cpTasks) { - this(name, numParallelSubtasks, subtaskIndex, userCodeClassLoader); + public AbstractRuntimeUDFContext(String name, int numParallelSubtasks, int subtaskIndex, ClassLoader userCodeClassLoader, ExecutionConfig executionConfig, Map> cpTasks) { + this(name, numParallelSubtasks, subtaskIndex, userCodeClassLoader, executionConfig); this.distributedCache.setCopyTasks(cpTasks); } - + + @Override + public ExecutionConfig getExecutionConfig() { + return executionConfig; + } + @Override public String getTaskName() { return this.name; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/RuntimeUDFContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/RuntimeUDFContext.java index 74fddefb47541..b9c98cd051c42 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/RuntimeUDFContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/RuntimeUDFContext.java @@ -23,6 +23,7 @@ import java.util.Map; import java.util.concurrent.FutureTask; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.BroadcastVariableInitializer; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.core.fs.Path; @@ -37,12 +38,12 @@ public class RuntimeUDFContext extends AbstractRuntimeUDFContext { private final HashMap> uninitializedBroadcastVars = new HashMap>(); - public RuntimeUDFContext(String name, int numParallelSubtasks, int subtaskIndex, ClassLoader userCodeClassLoader) { - super(name, numParallelSubtasks, subtaskIndex, userCodeClassLoader); + public RuntimeUDFContext(String name, int numParallelSubtasks, int subtaskIndex, ClassLoader userCodeClassLoader, ExecutionConfig executionConfig) { + super(name, numParallelSubtasks, subtaskIndex, userCodeClassLoader, executionConfig); } - public RuntimeUDFContext(String name, int numParallelSubtasks, int subtaskIndex, ClassLoader userCodeClassLoader, Map> cpTasks) { - super(name, numParallelSubtasks, subtaskIndex, userCodeClassLoader, cpTasks); + public RuntimeUDFContext(String name, int numParallelSubtasks, int subtaskIndex, ClassLoader userCodeClassLoader, ExecutionConfig executionConfig, Map> cpTasks) { + super(name, numParallelSubtasks, subtaskIndex, userCodeClassLoader, executionConfig, cpTasks); } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java index ea68554d47b74..afccd7c083d06 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java @@ -28,6 +28,7 @@ import java.util.Map; import java.util.Set; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.Plan; @@ -69,15 +70,18 @@ public class CollectionExecutor { private final ClassLoader classLoader; private final boolean mutableObjectSafeMode; + + private final ExecutionConfig executionConfig; // -------------------------------------------------------------------------------------------- - public CollectionExecutor() { - this(DEFAULT_MUTABLE_OBJECT_SAFE_MODE); + public CollectionExecutor(ExecutionConfig executionConfig) { + this(executionConfig, DEFAULT_MUTABLE_OBJECT_SAFE_MODE); } - public CollectionExecutor(boolean mutableObjectSafeMode) { + public CollectionExecutor(ExecutionConfig executionConfig, boolean mutableObjectSafeMode) { this.mutableObjectSafeMode = mutableObjectSafeMode; + this.executionConfig = executionConfig; this.intermediateResults = new HashMap, List>(); this.accumulators = new HashMap>(); @@ -161,13 +165,13 @@ private void executeDataSink(GenericDataSinkBase sink) throws Exception @SuppressWarnings("unchecked") GenericDataSinkBase typedSink = (GenericDataSinkBase) sink; - typedSink.executeOnCollections(input); + typedSink.executeOnCollections(input, executionConfig); } private List executeDataSource(GenericDataSourceBase source) throws Exception { @SuppressWarnings("unchecked") GenericDataSourceBase typedSource = (GenericDataSourceBase) source; - return typedSource.executeOnCollections(mutableObjectSafeMode); + return typedSource.executeOnCollections(executionConfig, mutableObjectSafeMode); } private List executeUnaryOperator(SingleInputOperator operator, int superStep) throws Exception { @@ -185,8 +189,8 @@ private List executeUnaryOperator(SingleInputOperator op // build the runtime context and compute broadcast variables, if necessary RuntimeUDFContext ctx; if (RichFunction.class.isAssignableFrom(typedOp.getUserCodeWrapper().getUserCodeClass())) { - ctx = superStep == 0 ? new RuntimeUDFContext(operator.getName(), 1, 0, getClass().getClassLoader()) : - new IterationRuntimeUDFContext(operator.getName(), 1, 0, superStep, classLoader); + ctx = superStep == 0 ? new RuntimeUDFContext(operator.getName(), 1, 0, getClass().getClassLoader(), executionConfig) : + new IterationRuntimeUDFContext(operator.getName(), 1, 0, superStep, classLoader, executionConfig); for (Map.Entry> bcInputs : operator.getBroadcastInputs().entrySet()) { List bcData = execute(bcInputs.getValue()); @@ -196,7 +200,7 @@ private List executeUnaryOperator(SingleInputOperator op ctx = null; } - List result = typedOp.executeOnCollections(inputData, ctx, mutableObjectSafeMode); + List result = typedOp.executeOnCollections(inputData, ctx, executionConfig); if (ctx != null) { AccumulatorHelper.mergeInto(this.accumulators, ctx.getAllAccumulators()); @@ -227,8 +231,8 @@ private List executeBinaryOperator(DualInputOperator> bcInputs : operator.getBroadcastInputs().entrySet()) { List bcData = execute(bcInputs.getValue()); @@ -238,7 +242,7 @@ private List executeBinaryOperator(DualInputOperator result = typedOp.executeOnCollections(inputData1, inputData2, ctx, mutableObjectSafeMode); + List result = typedOp.executeOnCollections(inputData1, inputData2, ctx, executionConfig); if (ctx != null) { AccumulatorHelper.mergeInto(this.accumulators, ctx.getAllAccumulators()); @@ -349,7 +353,7 @@ private List executeDeltaIteration(DeltaIterationBase iteration) th int[] keyColumns = iteration.getSolutionSetKeyFields(); boolean[] inputOrderings = new boolean[keyColumns.length]; - TypeComparator inputComparator = ((CompositeType) solutionType).createComparator(keyColumns, inputOrderings, 0); + TypeComparator inputComparator = ((CompositeType) solutionType).createComparator(keyColumns, inputOrderings, 0, executionConfig); Map, T> solutionMap = new HashMap, T>(solutionInputData.size()); // fill the solution from the initial input @@ -482,8 +486,8 @@ private class IterationRuntimeUDFContext extends RuntimeUDFContext implements It private final int superstep; - public IterationRuntimeUDFContext(String name, int numParallelSubtasks, int subtaskIndex, int superstep, ClassLoader classloader) { - super(name, numParallelSubtasks, subtaskIndex, classloader); + public IterationRuntimeUDFContext(String name, int numParallelSubtasks, int subtaskIndex, int superstep, ClassLoader classloader, ExecutionConfig executionConfig) { + super(name, numParallelSubtasks, subtaskIndex, classloader, executionConfig); this.superstep = superstep; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/DualInputOperator.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/DualInputOperator.java index 9cdea6dbdc0ee..f43f84710073d 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/DualInputOperator.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/DualInputOperator.java @@ -21,6 +21,7 @@ import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.operators.util.UserCodeWrapper; @@ -286,5 +287,5 @@ public void accept(Visitor> visitor) { // -------------------------------------------------------------------------------------------- - protected abstract List executeOnCollections(List inputData1, List inputData2, RuntimeContext runtimeContext, boolean mutableObjectSafeMode) throws Exception; + protected abstract List executeOnCollections(List inputData1, List inputData2, RuntimeContext runtimeContext, ExecutionConfig executionConfig) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/GenericDataSinkBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/GenericDataSinkBase.java index 242e83d91c1d2..2f8a3968e8261 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/GenericDataSinkBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/GenericDataSinkBase.java @@ -23,6 +23,7 @@ import java.util.Comparator; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.distributions.DataDistribution; import org.apache.flink.api.common.io.FinalizeOnMaster; import org.apache.flink.api.common.io.InitializeOnMaster; @@ -298,7 +299,7 @@ public void accept(Visitor> visitor) { // -------------------------------------------------------------------------------------------- - protected void executeOnCollections(List inputData) throws Exception { + protected void executeOnCollections(List inputData, ExecutionConfig executionConfig) throws Exception { OutputFormat format = this.formatWrapper.getUserCodeObject(); TypeInformation inputType = getInput().getOperatorInfo().getOutputType(); @@ -308,9 +309,9 @@ protected void executeOnCollections(List inputData) throws Exception { final TypeComparator sortComparator; if (inputType instanceof CompositeType) { - sortComparator = ((CompositeType) inputType).createComparator(sortColumns, sortOrderings, 0); + sortComparator = ((CompositeType) inputType).createComparator(sortColumns, sortOrderings, 0, executionConfig); } else if (inputType instanceof AtomicType) { - sortComparator = ((AtomicType) inputType).createComparator(sortOrderings[0]); + sortComparator = ((AtomicType) inputType).createComparator(sortOrderings[0], executionConfig); } else { throw new UnsupportedOperationException("Local output sorting does not support type "+inputType+" yet."); } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/GenericDataSourceBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/GenericDataSourceBase.java index ad1b2e4b6484d..13c5dad539c0c 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/GenericDataSourceBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/GenericDataSourceBase.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; import org.apache.flink.api.common.operators.util.UserCodeObjectWrapper; @@ -176,7 +177,7 @@ public void accept(Visitor> visitor) { // -------------------------------------------------------------------------------------------- - protected List executeOnCollections(boolean mutableObjectSafe) throws Exception { + protected List executeOnCollections(ExecutionConfig executionConfig, boolean mutableObjectSafe) throws Exception { @SuppressWarnings("unchecked") InputFormat inputFormat = (InputFormat) this.formatWrapper.getUserCodeObject(); inputFormat.configure(this.parameters); @@ -185,7 +186,7 @@ protected List executeOnCollections(boolean mutableObjectSafe) throws Excep // splits InputSplit[] splits = inputFormat.createInputSplits(1); - TypeSerializer serializer = getOperatorInfo().getOutputType().createSerializer(); + TypeSerializer serializer = getOperatorInfo().getOutputType().createSerializer(executionConfig); for (InputSplit split : splits) { inputFormat.open(split); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/SingleInputOperator.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/SingleInputOperator.java index eddf89b6aa72b..ada4ab0ab7b9e 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/SingleInputOperator.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/SingleInputOperator.java @@ -21,6 +21,7 @@ import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.operators.util.UserCodeWrapper; @@ -203,5 +204,5 @@ public void accept(Visitor> visitor) { // -------------------------------------------------------------------------------------------- - protected abstract List executeOnCollections(List inputData, RuntimeContext runtimeContext, boolean mutableObjectSafeMode) throws Exception; + protected abstract List executeOnCollections(List inputData, RuntimeContext runtimeContext, ExecutionConfig executionConfig) throws Exception; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/Union.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/Union.java index d7d0e20fddd13..9586c5dff035b 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/Union.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/Union.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; @@ -47,7 +48,7 @@ public Union(Operator input1, Operator input2, String unionLocationName) { } @Override - protected List executeOnCollections(List inputData1, List inputData2, RuntimeContext runtimeContext, boolean mutableObjectSafeMode) { + protected List executeOnCollections(List inputData1, List inputData2, RuntimeContext runtimeContext, ExecutionConfig executionConfig) { ArrayList result = new ArrayList(inputData1.size() + inputData2.size()); result.addAll(inputData1); result.addAll(inputData2); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/BulkIterationBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/BulkIterationBase.java index 31080cd4bae19..63041971cb431 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/BulkIterationBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/BulkIterationBase.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.Map; +import org.apache.flink.api.common.ExecutionConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.flink.api.common.InvalidProgramException; @@ -298,7 +299,7 @@ public boolean isConverged(int iteration, LongValue countAggregate) { } @Override - protected List executeOnCollections(List inputData, RuntimeContext runtimeContext, boolean mutableObjectSafeMode) { + protected List executeOnCollections(List inputData, RuntimeContext runtimeContext, ExecutionConfig executionConfig) { throw new UnsupportedOperationException(); } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CoGroupOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CoGroupOperatorBase.java index 65b9d1cb92083..4165f3d99ef75 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CoGroupOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CoGroupOperatorBase.java @@ -18,6 +18,7 @@ package org.apache.flink.api.common.operators.base; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.functions.Partitioner; @@ -186,7 +187,7 @@ public Partitioner getCustomPartitioner() { // ------------------------------------------------------------------------ @Override - protected List executeOnCollections(List input1, List input2, RuntimeContext ctx, boolean mutableObjectSafe) throws Exception { + protected List executeOnCollections(List input1, List input2, RuntimeContext ctx, ExecutionConfig executionConfig) throws Exception { // -------------------------------------------------------------------- // Setup // -------------------------------------------------------------------- @@ -196,17 +197,19 @@ protected List executeOnCollections(List input1, List input2, Run // for the grouping / merging comparator int[] inputKeys1 = getKeyColumns(0); int[] inputKeys2 = getKeyColumns(1); + + boolean objectReuseDisabled = !executionConfig.isObjectReuseEnabled(); boolean[] inputDirections1 = new boolean[inputKeys1.length]; boolean[] inputDirections2 = new boolean[inputKeys2.length]; Arrays.fill(inputDirections1, true); Arrays.fill(inputDirections2, true); - final TypeSerializer inputSerializer1 = inputType1.createSerializer(); - final TypeSerializer inputSerializer2 = inputType2.createSerializer(); + final TypeSerializer inputSerializer1 = inputType1.createSerializer(executionConfig); + final TypeSerializer inputSerializer2 = inputType2.createSerializer(executionConfig); - final TypeComparator inputComparator1 = getTypeComparator(inputType1, inputKeys1, inputDirections1); - final TypeComparator inputComparator2 = getTypeComparator(inputType2, inputKeys2, inputDirections2); + final TypeComparator inputComparator1 = getTypeComparator(executionConfig, inputType1, inputKeys1, inputDirections1); + final TypeComparator inputComparator2 = getTypeComparator(executionConfig, inputType2, inputKeys2, inputDirections2); final TypeComparator inputSortComparator1; final TypeComparator inputSortComparator2; @@ -227,7 +230,7 @@ protected List executeOnCollections(List input1, List input2, Run Arrays.fill(allSortDirections, 0, inputKeys1.length, true); System.arraycopy(groupSortDirections, 0, allSortDirections, inputKeys1.length, groupSortDirections.length); - inputSortComparator1 = getTypeComparator(inputType1, allSortKeys, allSortDirections); + inputSortComparator1 = getTypeComparator(executionConfig, inputType1, allSortKeys, allSortDirections); } if (groupOrder2 == null || groupOrder2.getNumberOfFields() == 0) { @@ -246,12 +249,12 @@ protected List executeOnCollections(List input1, List input2, Run Arrays.fill(allSortDirections, 0, inputKeys2.length, true); System.arraycopy(groupSortDirections, 0, allSortDirections, inputKeys2.length, groupSortDirections.length); - inputSortComparator2 = getTypeComparator(inputType2, allSortKeys, allSortDirections); + inputSortComparator2 = getTypeComparator(executionConfig, inputType2, allSortKeys, allSortDirections); } CoGroupSortListIterator coGroupIterator = new CoGroupSortListIterator(input1, inputSortComparator1, inputComparator1, inputSerializer1, - input2, inputSortComparator2, inputComparator2, inputSerializer2, mutableObjectSafe); + input2, inputSortComparator2, inputComparator2, inputSerializer2, objectReuseDisabled); // -------------------------------------------------------------------- // Run UDF @@ -262,8 +265,8 @@ protected List executeOnCollections(List input1, List input2, Run FunctionUtils.openFunction(function, parameters); List result = new ArrayList(); - Collector resultCollector = mutableObjectSafe ? - new CopyingListCollector(result, getOperatorInfo().getOutputType().createSerializer()) : + Collector resultCollector = objectReuseDisabled ? + new CopyingListCollector(result, getOperatorInfo().getOutputType().createSerializer(executionConfig)) : new ListCollector(result); while (coGroupIterator.next()) { @@ -275,12 +278,12 @@ protected List executeOnCollections(List input1, List input2, Run return result; } - private TypeComparator getTypeComparator(TypeInformation inputType, int[] inputKeys, boolean[] inputSortDirections) { + private TypeComparator getTypeComparator(ExecutionConfig executionConfig, TypeInformation inputType, int[] inputKeys, boolean[] inputSortDirections) { if (!(inputType instanceof CompositeType)) { throw new InvalidProgramException("Input types of coGroup must be composite types."); } - return ((CompositeType) inputType).createComparator(inputKeys, inputSortDirections, 0); + return ((CompositeType) inputType).createComparator(inputKeys, inputSortDirections, 0, executionConfig); } private static class CoGroupSortListIterator { diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CollectorMapOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CollectorMapOperatorBase.java index 8ad91c68783b5..62bdfbe7f2848 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CollectorMapOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CollectorMapOperatorBase.java @@ -20,6 +20,7 @@ import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.GenericCollectorMap; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.operators.SingleInputOperator; @@ -52,7 +53,7 @@ public CollectorMapOperatorBase(Class udf, UnaryOperatorInformatio // -------------------------------------------------------------------------------------------- @Override - protected List executeOnCollections(List inputData, RuntimeContext ctx, boolean mutableObjectSafeMode) { + protected List executeOnCollections(List inputData, RuntimeContext ctx, ExecutionConfig executionConfig) { throw new UnsupportedOperationException(); } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CrossOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CrossOperatorBase.java index c6ceef0c1455b..f20659cc13d90 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CrossOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CrossOperatorBase.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.CrossFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.FunctionUtils; @@ -84,18 +85,20 @@ public CrossHint getCrossHint() { // -------------------------------------------------------------------------------------------- @Override - protected List executeOnCollections(List inputData1, List inputData2, RuntimeContext ctx, boolean mutableObjectSafeMode) throws Exception { + protected List executeOnCollections(List inputData1, List inputData2, RuntimeContext ctx, ExecutionConfig executionConfig) throws Exception { CrossFunction function = this.userFunction.getUserCodeObject(); FunctionUtils.setFunctionRuntimeContext(function, ctx); FunctionUtils.openFunction(function, this.parameters); + + boolean objectReuseDisabled = !executionConfig.isObjectReuseEnabled(); ArrayList result = new ArrayList(inputData1.size() * inputData2.size()); - if (mutableObjectSafeMode) { - TypeSerializer inSerializer1 = getOperatorInfo().getFirstInputType().createSerializer(); - TypeSerializer inSerializer2 = getOperatorInfo().getSecondInputType().createSerializer(); - TypeSerializer outSerializer = getOperatorInfo().getOutputType().createSerializer(); + if (objectReuseDisabled) { + TypeSerializer inSerializer1 = getOperatorInfo().getFirstInputType().createSerializer(executionConfig); + TypeSerializer inSerializer2 = getOperatorInfo().getSecondInputType().createSerializer(executionConfig); + TypeSerializer outSerializer = getOperatorInfo().getOutputType().createSerializer(executionConfig); for (IN1 element1 : inputData1) { for (IN2 element2 : inputData2) { diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/DeltaIterationBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/DeltaIterationBase.java index f945b1d455158..298653442d580 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/DeltaIterationBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/DeltaIterationBase.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.Map; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.aggregators.AggregatorRegistry; import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.RuntimeContext; @@ -332,7 +333,7 @@ public UserCodeWrapper getUserCodeWrapper() { } @Override - protected List executeOnCollections(List inputData1, List inputData2, RuntimeContext runtimeContext, boolean mutableObjectSafeMode) { + protected List executeOnCollections(List inputData1, List inputData2, RuntimeContext runtimeContext, ExecutionConfig executionConfig) { throw new UnsupportedOperationException(); } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FilterOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FilterOperatorBase.java index f4bd53782e497..4db52650ccf88 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FilterOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FilterOperatorBase.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.FunctionUtils; @@ -50,7 +51,7 @@ public FilterOperatorBase(Class udf, UnaryOperatorInformation executeOnCollections(List inputData, RuntimeContext ctx, boolean mutableObjectSafeMode) throws Exception { + protected List executeOnCollections(List inputData, RuntimeContext ctx, ExecutionConfig executionConfig) throws Exception { FlatMapFunction function = this.userFunction.getUserCodeObject(); FunctionUtils.setFunctionRuntimeContext(function, ctx); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FlatMapOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FlatMapOperatorBase.java index 8312a994a937f..615ba87201b31 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FlatMapOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FlatMapOperatorBase.java @@ -18,6 +18,7 @@ package org.apache.flink.api.common.operators.base; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.CopyingListCollector; @@ -53,17 +54,19 @@ public FlatMapOperatorBase(Class udf, UnaryOperatorInformation executeOnCollections(List input, RuntimeContext ctx, boolean mutableObjectSafeMode) throws Exception { + protected List executeOnCollections(List input, RuntimeContext ctx, ExecutionConfig executionConfig) throws Exception { FlatMapFunction function = userFunction.getUserCodeObject(); FunctionUtils.setFunctionRuntimeContext(function, ctx); FunctionUtils.openFunction(function, parameters); ArrayList result = new ArrayList(input.size()); + + boolean objectReuseDisabled = !executionConfig.isObjectReuseEnabled(); - if (mutableObjectSafeMode) { - TypeSerializer inSerializer = getOperatorInfo().getInputType().createSerializer(); - TypeSerializer outSerializer = getOperatorInfo().getOutputType().createSerializer(); + if (objectReuseDisabled) { + TypeSerializer inSerializer = getOperatorInfo().getInputType().createSerializer(executionConfig); + TypeSerializer outSerializer = getOperatorInfo().getOutputType().createSerializer(executionConfig); CopyingListCollector resultCollector = new CopyingListCollector(result, outSerializer); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorBase.java index ddfd87465df0e..f4f7d0f717a26 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorBase.java @@ -19,6 +19,7 @@ package org.apache.flink.api.common.operators.base; import org.apache.commons.lang3.ArrayUtils; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.functions.FlatCombineFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; @@ -152,9 +153,11 @@ public Partitioner getCustomPartitioner() { // -------------------------------------------------------------------------------------------- @Override - protected List executeOnCollections(List inputData, RuntimeContext ctx, boolean mutableObjectSafeMode) throws Exception { + protected List executeOnCollections(List inputData, RuntimeContext ctx, ExecutionConfig executionConfig) throws Exception { GroupReduceFunction function = this.userFunction.getUserCodeObject(); + boolean objectReuseDisabled = !executionConfig.isObjectReuseEnabled(); + UnaryOperatorInformation operatorInfo = getOperatorInfo(); TypeInformation inputType = operatorInfo.getInputType(); @@ -176,7 +179,7 @@ protected List executeOnCollections(List inputData, RuntimeContext ctx, if(sortColumns.length == 0) { // => all reduce. No comparator Preconditions.checkArgument(sortOrderings.length == 0); } else { - final TypeComparator sortComparator = ((CompositeType) inputType).createComparator(sortColumns, sortOrderings, 0); + final TypeComparator sortComparator = ((CompositeType) inputType).createComparator(sortColumns, sortOrderings, 0, executionConfig); Collections.sort(inputData, new Comparator() { @Override @@ -193,9 +196,9 @@ public int compare(IN o1, IN o2) { ArrayList result = new ArrayList(); if (keyColumns.length == 0) { - if (mutableObjectSafeMode) { - final TypeSerializer inputSerializer = inputType.createSerializer(); - TypeSerializer outSerializer = getOperatorInfo().getOutputType().createSerializer(); + if (objectReuseDisabled) { + final TypeSerializer inputSerializer = inputType.createSerializer(executionConfig); + TypeSerializer outSerializer = getOperatorInfo().getOutputType().createSerializer(executionConfig); List inputDataCopy = new ArrayList(inputData.size()); for (IN in: inputData) { inputDataCopy.add(inputSerializer.copy(in)); @@ -208,14 +211,14 @@ public int compare(IN o1, IN o2) { function.reduce(inputData, collector); } } else { - final TypeSerializer inputSerializer = inputType.createSerializer(); + final TypeSerializer inputSerializer = inputType.createSerializer(executionConfig); boolean[] keyOrderings = new boolean[keyColumns.length]; - final TypeComparator comparator = ((CompositeType) inputType).createComparator(keyColumns, keyOrderings, 0); + final TypeComparator comparator = ((CompositeType) inputType).createComparator(keyColumns, keyOrderings, 0, executionConfig); - ListKeyGroupedIterator keyedIterator = new ListKeyGroupedIterator(inputData, inputSerializer, comparator, mutableObjectSafeMode); + ListKeyGroupedIterator keyedIterator = new ListKeyGroupedIterator(inputData, inputSerializer, comparator, objectReuseDisabled); - if (mutableObjectSafeMode) { - TypeSerializer outSerializer = getOperatorInfo().getOutputType().createSerializer(); + if (objectReuseDisabled) { + TypeSerializer outSerializer = getOperatorInfo().getOutputType().createSerializer(executionConfig); CopyingListCollector collector = new CopyingListCollector(result, outSerializer); while (keyedIterator.nextKey()) { diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/JoinOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/JoinOperatorBase.java index 555175dbd3d46..373846fe12dc0 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/JoinOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/JoinOperatorBase.java @@ -18,6 +18,7 @@ package org.apache.flink.api.common.operators.base; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.functions.RuntimeContext; @@ -139,7 +140,7 @@ public Partitioner getCustomPartitioner() { @SuppressWarnings("unchecked") @Override - protected List executeOnCollections(List inputData1, List inputData2, RuntimeContext runtimeContext, boolean mutableObjectSafe) throws Exception { + protected List executeOnCollections(List inputData1, List inputData2, RuntimeContext runtimeContext, ExecutionConfig executionConfig) throws Exception { FlatJoinFunction function = userFunction.getUserCodeObject(); FunctionUtils.setFunctionRuntimeContext(function, runtimeContext); @@ -148,22 +149,24 @@ protected List executeOnCollections(List inputData1, List inputDa TypeInformation leftInformation = getOperatorInfo().getFirstInputType(); TypeInformation rightInformation = getOperatorInfo().getSecondInputType(); TypeInformation outInformation = getOperatorInfo().getOutputType(); + + boolean objectReuseDisabled = !executionConfig.isObjectReuseEnabled(); - TypeSerializer leftSerializer = mutableObjectSafe ? leftInformation.createSerializer() : null; - TypeSerializer rightSerializer = mutableObjectSafe ? rightInformation.createSerializer() : null; + TypeSerializer leftSerializer = objectReuseDisabled ? leftInformation.createSerializer(executionConfig) : null; + TypeSerializer rightSerializer = objectReuseDisabled ? rightInformation.createSerializer(executionConfig) : null; TypeComparator leftComparator; TypeComparator rightComparator; if (leftInformation instanceof AtomicType) { - leftComparator = ((AtomicType) leftInformation).createComparator(true); + leftComparator = ((AtomicType) leftInformation).createComparator(true, executionConfig); } else if (leftInformation instanceof CompositeType) { int[] keyPositions = getKeyColumns(0); boolean[] orders = new boolean[keyPositions.length]; Arrays.fill(orders, true); - leftComparator = ((CompositeType) leftInformation).createComparator(keyPositions, orders, 0); + leftComparator = ((CompositeType) leftInformation).createComparator(keyPositions, orders, 0, executionConfig); } else { throw new RuntimeException("Type information for left input of type " + leftInformation.getClass() @@ -171,14 +174,14 @@ else if (leftInformation instanceof CompositeType) { } if (rightInformation instanceof AtomicType) { - rightComparator = ((AtomicType) rightInformation).createComparator(true); + rightComparator = ((AtomicType) rightInformation).createComparator(true, executionConfig); } else if (rightInformation instanceof CompositeType) { int[] keyPositions = getKeyColumns(1); boolean[] orders = new boolean[keyPositions.length]; Arrays.fill(orders, true); - rightComparator = ((CompositeType) rightInformation).createComparator(keyPositions, orders, 0); + rightComparator = ((CompositeType) rightInformation).createComparator(keyPositions, orders, 0, executionConfig); } else { throw new RuntimeException("Type information for right input of type " + rightInformation.getClass() @@ -188,7 +191,7 @@ else if (rightInformation instanceof CompositeType) { TypePairComparator pairComparator = new GenericPairComparator(leftComparator, rightComparator); List result = new ArrayList(); - Collector collector = mutableObjectSafe ? new CopyingListCollector(result, outInformation.createSerializer()) + Collector collector = objectReuseDisabled ? new CopyingListCollector(result, outInformation.createSerializer(executionConfig)) : new ListCollector(result); Map> probeTable = new HashMap>(); @@ -212,7 +215,7 @@ else if (rightInformation instanceof CompositeType) { pairComparator.setReference(left); for (IN2 right : matchingHashes) { if (pairComparator.equalToReference(right)) { - if (mutableObjectSafe) { + if (objectReuseDisabled) { function.join(leftSerializer.copy(left), rightSerializer.copy(right), collector); } else { function.join(left, right, collector); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapOperatorBase.java index 0218bfa252d69..cde3b7440b355 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapOperatorBase.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.FunctionUtils; @@ -54,17 +55,19 @@ public MapOperatorBase(Class udf, UnaryOperatorInformation executeOnCollections(List inputData, RuntimeContext ctx, boolean mutableObjectSafeMode) throws Exception { + protected List executeOnCollections(List inputData, RuntimeContext ctx, ExecutionConfig executionConfig) throws Exception { MapFunction function = this.userFunction.getUserCodeObject(); FunctionUtils.setFunctionRuntimeContext(function, ctx); FunctionUtils.openFunction(function, this.parameters); ArrayList result = new ArrayList(inputData.size()); + + boolean objectReuseDisabled = !executionConfig.isObjectReuseEnabled(); - if (mutableObjectSafeMode) { - TypeSerializer inSerializer = getOperatorInfo().getInputType().createSerializer(); - TypeSerializer outSerializer = getOperatorInfo().getOutputType().createSerializer(); + if (objectReuseDisabled) { + TypeSerializer inSerializer = getOperatorInfo().getInputType().createSerializer(executionConfig); + TypeSerializer outSerializer = getOperatorInfo().getOutputType().createSerializer(executionConfig); for (IN element : inputData) { IN inCopy = inSerializer.copy(element); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapPartitionOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapPartitionOperatorBase.java index 7c1fcef1684cc..25b3bb88102c7 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapPartitionOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapPartitionOperatorBase.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.MapPartitionFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.CopyingIterator; @@ -57,17 +58,19 @@ public MapPartitionOperatorBase(Class udf, UnaryOperatorInformatio // -------------------------------------------------------------------------------------------- @Override - protected List executeOnCollections(List inputData, RuntimeContext ctx, boolean mutableObjectSafeMode) throws Exception { + protected List executeOnCollections(List inputData, RuntimeContext ctx, ExecutionConfig executionConfig) throws Exception { MapPartitionFunction function = this.userFunction.getUserCodeObject(); FunctionUtils.setFunctionRuntimeContext(function, ctx); FunctionUtils.openFunction(function, this.parameters); ArrayList result = new ArrayList(inputData.size() / 4); + + boolean objectReuseDisabled = !executionConfig.isObjectReuseEnabled(); - if (mutableObjectSafeMode) { - TypeSerializer inSerializer = getOperatorInfo().getInputType().createSerializer(); - TypeSerializer outSerializer = getOperatorInfo().getOutputType().createSerializer(); + if (objectReuseDisabled) { + TypeSerializer inSerializer = getOperatorInfo().getInputType().createSerializer(executionConfig); + TypeSerializer outSerializer = getOperatorInfo().getOutputType().createSerializer(executionConfig); CopyingIterator source = new CopyingIterator(inputData.iterator(), inSerializer); CopyingListCollector resultCollector = new CopyingListCollector(result, outSerializer); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/PartitionOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/PartitionOperatorBase.java index 3602a821cf178..f91d7d8820b4a 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/PartitionOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/PartitionOperatorBase.java @@ -20,6 +20,7 @@ import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.NoOpFunction; @@ -88,7 +89,7 @@ public SingleInputSemanticProperties getSemanticProperties() { // -------------------------------------------------------------------------------------------- @Override - protected List executeOnCollections(List inputData, RuntimeContext runtimeContext, boolean mutableObjectSafeMode) { + protected List executeOnCollections(List inputData, RuntimeContext runtimeContext, ExecutionConfig executionConfig) { return inputData; } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/ReduceOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/ReduceOperatorBase.java index f1bf0e96ccc2a..d3d61e91fee28 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/ReduceOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/ReduceOperatorBase.java @@ -18,6 +18,7 @@ package org.apache.flink.api.common.operators.base; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.functions.ReduceFunction; @@ -149,12 +150,13 @@ public Partitioner getCustomPartitioner() { // -------------------------------------------------------------------------------------------- @Override - protected List executeOnCollections(List inputData, RuntimeContext ctx, boolean mutableObjectSafeMode) throws Exception { + protected List executeOnCollections(List inputData, RuntimeContext ctx, ExecutionConfig executionConfig) throws Exception { // make sure we can handle empty inputs if (inputData.isEmpty()) { return Collections.emptyList(); } - + + boolean objectReuseDisabled = !executionConfig.isObjectReuseEnabled(); ReduceFunction function = this.userFunction.getUserCodeObject(); UnaryOperatorInformation operatorInfo = getOperatorInfo(); @@ -169,11 +171,11 @@ protected List executeOnCollections(List inputData, RuntimeContext ctx, bo FunctionUtils.setFunctionRuntimeContext(function, ctx); FunctionUtils.openFunction(function, this.parameters); - TypeSerializer serializer = getOperatorInfo().getInputType().createSerializer(); + TypeSerializer serializer = getOperatorInfo().getInputType().createSerializer(executionConfig); if (inputColumns.length > 0) { boolean[] inputOrderings = new boolean[inputColumns.length]; - TypeComparator inputComparator = ((CompositeType) inputType).createComparator(inputColumns, inputOrderings, 0); + TypeComparator inputComparator = ((CompositeType) inputType).createComparator(inputColumns, inputOrderings, 0, executionConfig); Map, T> aggregateMap = new HashMap, T>(inputData.size() / 10); @@ -183,7 +185,7 @@ protected List executeOnCollections(List inputData, RuntimeContext ctx, bo T existing = aggregateMap.get(wrapper); T result; - if (mutableObjectSafeMode) { + if (objectReuseDisabled) { if (existing != null) { result = function.reduce(existing, serializer.copy(next)); } else { @@ -209,7 +211,7 @@ protected List executeOnCollections(List inputData, RuntimeContext ctx, bo else { T aggregate = inputData.get(0); - if (mutableObjectSafeMode) { + if (objectReuseDisabled) { aggregate = serializer.copy(aggregate); for (int i = 1; i < inputData.size(); i++) { diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/AtomicType.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/AtomicType.java index 10dbbfe09b5b2..2fc8a1b20c05a 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/AtomicType.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/AtomicType.java @@ -18,6 +18,7 @@ package org.apache.flink.api.common.typeinfo; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -26,5 +27,5 @@ */ public interface AtomicType { - TypeComparator createComparator(boolean sortOrderAscending); + TypeComparator createComparator(boolean sortOrderAscending, ExecutionConfig executionConfig); } 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 646a549eb4ee7..80f5f63b99a32 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 @@ -21,6 +21,7 @@ import java.util.HashMap; import java.util.Map; +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; @@ -94,12 +95,12 @@ public boolean isKeyType() { @Override @SuppressWarnings("unchecked") - public TypeSerializer createSerializer() { + public TypeSerializer createSerializer(ExecutionConfig executionConfig) { // special case the string array if (componentClass.equals(String.class)) { return (TypeSerializer) StringArraySerializer.INSTANCE; } else { - return (TypeSerializer) new GenericArraySerializer(this.componentClass, this.componentInfo.createSerializer()); + return (TypeSerializer) new GenericArraySerializer(this.componentClass, this.componentInfo.createSerializer(executionConfig)); } } 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 61d830aa0d825..7bf729818d664 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 @@ -23,6 +23,7 @@ import java.util.HashMap; import java.util.Map; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.InvalidTypesException; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -114,12 +115,12 @@ public boolean isKeyType() { } @Override - public TypeSerializer createSerializer() { + public TypeSerializer createSerializer(ExecutionConfig executionConfig) { return this.serializer; } @Override - public TypeComparator createComparator(boolean sortOrderAscending) { + public TypeComparator createComparator(boolean sortOrderAscending, ExecutionConfig executionConfig) { if (comparatorClass != null) { return instantiateComparator(comparatorClass, sortOrderAscending); } else { 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 dba0e6f8c3184..367670cc23884 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 @@ -19,6 +19,7 @@ package org.apache.flink.api.common.typeinfo; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.types.Nothing; @@ -54,7 +55,7 @@ public boolean isKeyType() { } @Override - public TypeSerializer createSerializer() { + public TypeSerializer createSerializer(ExecutionConfig executionConfig) { throw new RuntimeException("The Nothing type cannot have a serializer."); } } 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 5163801f80f0f..56e2680f2f8be 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 @@ -21,6 +21,7 @@ import java.util.HashMap; import java.util.Map; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.InvalidTypesException; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.array.BooleanPrimitiveArraySerializer; @@ -86,7 +87,7 @@ public boolean isKeyType() { } @Override - public TypeSerializer createSerializer() { + public TypeSerializer createSerializer(ExecutionConfig executionConfig) { return this.serializer; } 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 0f86486882974..0ed5afda73711 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 @@ -18,10 +18,13 @@ package org.apache.flink.api.common.typeinfo; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; -public abstract class TypeInformation { - +import java.io.Serializable; + +public abstract class TypeInformation implements Serializable { + public abstract boolean isBasicType(); public abstract boolean isTupleType(); @@ -32,7 +35,7 @@ public abstract class TypeInformation { public abstract boolean isKeyType(); - public abstract TypeSerializer createSerializer(); + public abstract TypeSerializer createSerializer(ExecutionConfig config); /** * @return The number of fields in this type, including its sub-fields (for compsite types) 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 c3ea0e477c195..f3e4cfab47de0 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 @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.AtomicType; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -91,7 +92,7 @@ public List getFlatFields(String fieldExpression) { /** * Get the actual comparator we've initialized. */ - protected abstract TypeComparator getNewComparator(); + protected abstract TypeComparator getNewComparator(ExecutionConfig config); /** @@ -99,7 +100,7 @@ public List getFlatFields(String fieldExpression) { * to create the actual comparators * @return The comparator */ - public TypeComparator createComparator(int[] logicalKeyFields, boolean[] orders, int logicalFieldOffset) { + public TypeComparator createComparator(int[] logicalKeyFields, boolean[] orders, int logicalFieldOffset, ExecutionConfig config) { initializeNewComparator(logicalKeyFields.length); for(int logicalKeyFieldIndex = 0; logicalKeyFieldIndex < logicalKeyFields.length; logicalKeyFieldIndex++) { @@ -110,13 +111,13 @@ public TypeComparator createComparator(int[] logicalKeyFields, boolean[] orde if(localFieldType instanceof AtomicType && logicalField == logicalKeyField) { // we found an atomic key --> create comparator - addCompareField(localFieldId, ((AtomicType) localFieldType).createComparator(orders[logicalKeyFieldIndex]) ); + 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 ) { // 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)); + addCompareField(localFieldId, ((CompositeType) localFieldType).createComparator(new int[] {logicalKeyField}, new boolean[] {orders[logicalKeyFieldIndex]}, logicalField, config)); } // maintain logicalField @@ -127,7 +128,7 @@ public TypeComparator createComparator(int[] logicalKeyFields, boolean[] orde logicalField++; } } - return getNewComparator(); + return getNewComparator(config); } diff --git a/flink-core/src/test/java/org/apache/flink/api/common/functions/util/RuntimeUDFContextTest.java b/flink-core/src/test/java/org/apache/flink/api/common/functions/util/RuntimeUDFContextTest.java index 7d701dcbf9814..c77a1b677a6a8 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/functions/util/RuntimeUDFContextTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/functions/util/RuntimeUDFContextTest.java @@ -24,6 +24,7 @@ import java.util.Arrays; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.BroadcastVariableInitializer; import org.junit.Test; @@ -33,7 +34,7 @@ public class RuntimeUDFContextTest { @Test public void testBroadcastVariableNotFound() { try { - RuntimeUDFContext ctx = new RuntimeUDFContext("test name", 3, 1, getClass().getClassLoader()); + RuntimeUDFContext ctx = new RuntimeUDFContext("test name", 3, 1, getClass().getClassLoader(), new ExecutionConfig()); try { ctx.getBroadcastVariable("some name"); @@ -63,7 +64,7 @@ public void testBroadcastVariableNotFound() { @Test public void testBroadcastVariableSimple() { try { - RuntimeUDFContext ctx = new RuntimeUDFContext("test name", 3, 1, getClass().getClassLoader()); + RuntimeUDFContext ctx = new RuntimeUDFContext("test name", 3, 1, getClass().getClassLoader(), new ExecutionConfig()); ctx.setBroadcastVariable("name1", Arrays.asList(1, 2, 3, 4)); ctx.setBroadcastVariable("name2", Arrays.asList(1.0, 2.0, 3.0, 4.0)); @@ -97,7 +98,7 @@ public void testBroadcastVariableSimple() { @Test public void testBroadcastVariableWithInitializer() { try { - RuntimeUDFContext ctx = new RuntimeUDFContext("test name", 3, 1, getClass().getClassLoader()); + RuntimeUDFContext ctx = new RuntimeUDFContext("test name", 3, 1, getClass().getClassLoader(), new ExecutionConfig()); ctx.setBroadcastVariable("name", Arrays.asList(1, 2, 3, 4)); @@ -122,7 +123,7 @@ public void testBroadcastVariableWithInitializer() { @Test public void testResetBroadcastVariableWithInitializer() { try { - RuntimeUDFContext ctx = new RuntimeUDFContext("test name", 3, 1, getClass().getClassLoader()); + RuntimeUDFContext ctx = new RuntimeUDFContext("test name", 3, 1, getClass().getClassLoader(), new ExecutionConfig()); ctx.setBroadcastVariable("name", Arrays.asList(1, 2, 3, 4)); @@ -145,7 +146,7 @@ public void testResetBroadcastVariableWithInitializer() { @Test public void testBroadcastVariableWithInitializerAndMismatch() { try { - RuntimeUDFContext ctx = new RuntimeUDFContext("test name", 3, 1, getClass().getClassLoader()); + RuntimeUDFContext ctx = new RuntimeUDFContext("test name", 3, 1, getClass().getClassLoader(), new ExecutionConfig()); ctx.setBroadcastVariable("name", Arrays.asList(1, 2, 3, 4)); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/FlatMapOperatorCollectionTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/FlatMapOperatorCollectionTest.java index 9a0a2b5a051f6..d2314559592cb 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/FlatMapOperatorCollectionTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/FlatMapOperatorCollectionTest.java @@ -18,6 +18,7 @@ package org.apache.flink.api.common.operators.base; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.functions.RuntimeContext; @@ -63,9 +64,15 @@ public void testExecuteOnCollection() { } private void testExecuteOnCollection(FlatMapFunction udf, List input, boolean mutableSafe) throws Exception { + ExecutionConfig executionConfig = new ExecutionConfig(); + if (mutableSafe) { + executionConfig.disableObjectReuse(); + } else { + executionConfig.enableObjectReuse(); + } // run on collections final List result = getTestFlatMapOperator(udf) - .executeOnCollections(input, new RuntimeUDFContext("Test UDF", 4, 0, null), mutableSafe); + .executeOnCollections(input, new RuntimeUDFContext("Test UDF", 4, 0, null, executionConfig), executionConfig); Assert.assertEquals(input.size(), result.size()); Assert.assertEquals(input, result); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/JoinOperatorBaseTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/JoinOperatorBaseTest.java index feb22233f6bf4..54975b481d63a 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/JoinOperatorBaseTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/JoinOperatorBaseTest.java @@ -20,6 +20,7 @@ import static org.junit.Assert.*; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.functions.RichFlatJoinFunction; import org.apache.flink.api.common.functions.util.RuntimeUDFContext; @@ -60,8 +61,11 @@ public void join(String first, String second, Collector out) throws Exc List expected = new ArrayList(Arrays.asList(3, 3, 6 ,6)); try { - List resultSafe = base.executeOnCollections(inputData1, inputData2, null, true); - List resultRegular = base.executeOnCollections(inputData1, inputData2, null, false); + ExecutionConfig executionConfig = new ExecutionConfig(); + executionConfig.disableObjectReuse(); + List resultSafe = base.executeOnCollections(inputData1, inputData2, null, executionConfig); + executionConfig.enableObjectReuse(); + List resultRegular = base.executeOnCollections(inputData1, inputData2, null, executionConfig); assertEquals(expected, resultSafe); assertEquals(expected, resultRegular); @@ -110,8 +114,11 @@ public void join(String first, String second, Collector out) throws Exc try { - List resultSafe = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskName, 1, 0, null), true); - List resultRegular = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskName, 1, 0, null), false); + ExecutionConfig executionConfig = new ExecutionConfig(); + executionConfig.disableObjectReuse(); + List resultSafe = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskName, 1, 0, null, executionConfig), executionConfig); + executionConfig.enableObjectReuse(); + List resultRegular = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext(taskName, 1, 0, null, executionConfig), executionConfig); assertEquals(expected, resultSafe); assertEquals(expected, resultRegular); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/MapOperatorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/MapOperatorTest.java index fd23d4008584f..8e07f077e8bf1 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/MapOperatorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/MapOperatorTest.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.functions.RuntimeContext; @@ -52,8 +53,12 @@ public Integer map(String value) { parser, new UnaryOperatorInformation(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO), "TestMapper"); List input = new ArrayList(asList("1", "2", "3", "4", "5", "6")); - List resultMutableSafe = op.executeOnCollections(input, null, true); - List resultRegular = op.executeOnCollections(input, null, false); + + ExecutionConfig executionConfig = new ExecutionConfig(); + executionConfig.disableObjectReuse(); + List resultMutableSafe = op.executeOnCollections(input, null, executionConfig); + executionConfig.enableObjectReuse(); + List resultRegular = op.executeOnCollections(input, null, executionConfig); assertEquals(asList(1, 2, 3, 4, 5, 6), resultMutableSafe); assertEquals(asList(1, 2, 3, 4, 5, 6), resultRegular); @@ -97,8 +102,11 @@ public void close() throws Exception { parser, new UnaryOperatorInformation(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO), taskName); List input = new ArrayList(asList("1", "2", "3", "4", "5", "6")); - List resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskName, 1, 0, null), true); - List resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskName, 1, 0, null), false); + ExecutionConfig executionConfig = new ExecutionConfig(); + executionConfig.disableObjectReuse(); + List resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskName, 1, 0, null, executionConfig), executionConfig); + executionConfig.enableObjectReuse(); + List resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskName, 1, 0, null, executionConfig), executionConfig); assertEquals(asList(1, 2, 3, 4, 5, 6), resultMutableSafe); assertEquals(asList(1, 2, 3, 4, 5, 6), resultRegular); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/PartitionMapOperatorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/PartitionMapOperatorTest.java index 50c6b980ec6b5..61ba359dd72e4 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/PartitionMapOperatorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/PartitionMapOperatorTest.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.util.Collector; import org.apache.flink.api.common.functions.MapPartitionFunction; import org.apache.flink.api.common.functions.RichMapPartitionFunction; @@ -74,9 +75,12 @@ public void close() throws Exception { parser, new UnaryOperatorInformation(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO), taskName); List input = new ArrayList(asList("1", "2", "3", "4", "5", "6")); - - List resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskName, 1, 0, null), true); - List resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskName, 1, 0, null), false); + + ExecutionConfig executionConfig = new ExecutionConfig(); + executionConfig.disableObjectReuse(); + List resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskName, 1, 0, null, executionConfig), executionConfig); + executionConfig.enableObjectReuse(); + List resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskName, 1, 0, null, executionConfig), executionConfig); assertEquals(asList(1, 2, 3, 4, 5, 6), resultMutableSafe); assertEquals(asList(1, 2, 3, 4, 5, 6), resultRegular); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java index 1531ae6a87a05..59bea0ce21a90 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java @@ -358,7 +358,7 @@ public void testSerializabilityAndEquals() { try { ser2 = SerializationUtils.clone(ser1); } catch (SerializationException e) { - fail("The serializer is not serializable."); + fail("The serializer is not serializable: " + e); return; } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java index 2d5749024d170..c61d624079d0d 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/CollectionEnvironment.java @@ -29,7 +29,7 @@ public JobExecutionResult execute(String jobName) throws Exception { Plan p = createProgramPlan(jobName); // We need to reverse here. Object-Reuse enabled, means safe mode is disabled. - CollectionExecutor exec = new CollectionExecutor(!getConfig().isObjectReuseEnabled()); + CollectionExecutor exec = new CollectionExecutor(getConfig()); return exec.execute(p); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java index e5bb1fd90121c..f2091e25082fa 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/DataSet.java @@ -69,8 +69,10 @@ import org.apache.flink.api.java.operators.UnsortedGrouping; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.GenericTypeInfo; import org.apache.flink.api.java.typeutils.InputTypeConfigurable; import org.apache.flink.api.java.typeutils.MissingTypeInfo; +import org.apache.flink.api.java.typeutils.PojoTypeInfo; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.core.fs.FileSystem.WriteMode; @@ -92,7 +94,7 @@ */ public abstract class DataSet { - private final ExecutionEnvironment context; + protected final ExecutionEnvironment context; // NOTE: the type must not be accessed directly, but only via getType() private TypeInformation type; @@ -109,7 +111,11 @@ protected DataSet(ExecutionEnvironment context, TypeInformation typeInfo) { } this.context = context; - this.type = typeInfo; + if (typeInfo instanceof PojoTypeInfo && context.getConfig().isForceKryoEnabled()) { + this.type = new GenericTypeInfo(typeInfo.getTypeClass()); + } else { + this.type = typeInfo; + } } /** @@ -1300,7 +1306,7 @@ public DataSink output(OutputFormat outputFormat) { // configure the type if needed if (outputFormat instanceof InputTypeConfigurable) { - ((InputTypeConfigurable) outputFormat).setInputType(getType()); + ((InputTypeConfigurable) outputFormat).setInputType(getType(), context.getConfig() ); } DataSink sink = new DataSink(this, outputFormat, getType()); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java index af8095cf03be1..1105ab9746c33 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/ExecutionEnvironment.java @@ -54,10 +54,10 @@ import org.apache.flink.api.java.operators.OperatorTranslation; import org.apache.flink.api.java.operators.translation.JavaPlan; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.PojoTypeInfo; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.api.java.typeutils.ValueTypeInfo; -import org.apache.flink.api.java.typeutils.runtime.KryoSerializer; import org.apache.flink.core.fs.Path; import org.apache.flink.types.StringValue; import org.apache.flink.util.NumberSequenceIterator; @@ -103,8 +103,7 @@ public abstract class ExecutionEnvironment { private final List> cacheFile = new ArrayList>(); private ExecutionConfig config = new ExecutionConfig(); - - + // -------------------------------------------------------------------------------------------- // Constructor and Properties // -------------------------------------------------------------------------------------------- @@ -116,14 +115,6 @@ protected ExecutionEnvironment() { this.executionId = UUID.randomUUID(); } - /** - * Sets the config object. - */ - public void setConfig(ExecutionConfig config) { - Validate.notNull(config); - this.config = config; - } - /** * Gets the config object. */ @@ -208,7 +199,7 @@ public String getIdString() { // -------------------------------------------------------------------------------------------- // Registry for types and serializers // -------------------------------------------------------------------------------------------- - + /** * Registers the given Serializer as a default serializer for the given type at the * {@link org.apache.flink.api.java.typeutils.runtime.KryoSerializer}. @@ -220,11 +211,7 @@ public String getIdString() { * @param serializer The serializer to use. */ public void registerKryoSerializer(Class type, Serializer serializer) { - if (type == null || serializer == null) { - throw new NullPointerException("Cannot register null class or serializer."); - } - - KryoSerializer.registerSerializer(type, serializer); + config.registerKryoSerializer(type, serializer); } /** @@ -235,11 +222,7 @@ public void registerKryoSerializer(Class type, Serializer serializer) { * @param serializerClass The class of the serializer to use. */ public void registerKryoSerializer(Class type, Class> serializerClass) { - if (type == null || serializerClass == null) { - throw new NullPointerException("Cannot register null class or serializer."); - } - - KryoSerializer.registerSerializer(type, serializerClass); + config.registerKryoSerializer(type, serializerClass); } /** @@ -254,10 +237,16 @@ public void registerType(Class type) { if (type == null) { throw new NullPointerException("Cannot register null type class."); } - - KryoSerializer.registerType(type); + + TypeInformation typeInfo = TypeExtractor.createTypeInfo(type); + + if (typeInfo instanceof PojoTypeInfo) { + config.registerPojoType(type); + } else { + config.registerKryoType(type); + } } - + // -------------------------------------------------------------------------------------------- // Data set creations // -------------------------------------------------------------------------------------------- @@ -555,7 +544,7 @@ public DataSource fromCollection(Collection data) { TypeInformation type = TypeExtractor.getForObject(firstValue); CollectionInputFormat.checkCollection(data, type.getTypeClass()); - return new DataSource(this, new CollectionInputFormat(data, type.createSerializer()), type, Utils.getCallLocationName()); + return new DataSource(this, new CollectionInputFormat(data, type.createSerializer(config)), type, Utils.getCallLocationName()); } /** @@ -582,7 +571,7 @@ public DataSource fromCollection(Collection data, TypeInformation t private DataSource fromCollection(Collection data, TypeInformation type, String callLocationName) { CollectionInputFormat.checkCollection(data, type.getTypeClass()); - return new DataSource(this, new CollectionInputFormat(data, type.createSerializer()), type, callLocationName); + return new DataSource(this, new CollectionInputFormat(data, type.createSerializer(config)), type, callLocationName); } /** diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/CsvOutputFormat.java b/flink-java/src/main/java/org/apache/flink/api/java/io/CsvOutputFormat.java index 8e0ff5ec06c86..9239e253b0d59 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/io/CsvOutputFormat.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/io/CsvOutputFormat.java @@ -24,6 +24,7 @@ import java.io.OutputStreamWriter; import java.io.Writer; +import org.apache.flink.api.common.ExecutionConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.flink.api.common.InvalidProgramException; @@ -216,7 +217,7 @@ public String toString() { * is in fact a tuple type. */ @Override - public void setInputType(TypeInformation type) { + public void setInputType(TypeInformation type, ExecutionConfig executionConfig) { if (!type.isTupleType()) { throw new InvalidProgramException("The " + CsvOutputFormat.class.getSimpleName() + " can only be used to write tuple data sets."); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/LocalCollectionOutputFormat.java b/flink-java/src/main/java/org/apache/flink/api/java/io/LocalCollectionOutputFormat.java index 4b2e0ed7410ac..23c19f7a00243 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/io/LocalCollectionOutputFormat.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/io/LocalCollectionOutputFormat.java @@ -25,6 +25,7 @@ import java.util.HashMap; import java.util.Map; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.io.OutputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -89,7 +90,7 @@ public void close() throws IOException { @Override @SuppressWarnings("unchecked") - public void setInputType(TypeInformation type) { - this.typeSerializer = (TypeSerializer)type.createSerializer(); + public void setInputType(TypeInformation type, ExecutionConfig executionConfig) { + this.typeSerializer = (TypeSerializer)type.createSerializer(executionConfig); } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/TypeSerializerInputFormat.java b/flink-java/src/main/java/org/apache/flink/api/java/io/TypeSerializerInputFormat.java index 8e92c27dbf09a..7d0dfaa3681f4 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/io/TypeSerializerInputFormat.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/io/TypeSerializerInputFormat.java @@ -18,6 +18,7 @@ package org.apache.flink.api.java.io; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.io.BinaryInputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -40,7 +41,8 @@ public class TypeSerializerInputFormat extends BinaryInputFormat implement public TypeSerializerInputFormat(TypeInformation resultType) { this.resultType = resultType; - this.serializer = resultType.createSerializer(); + // TODO: fix this shit + this.serializer = resultType.createSerializer(new ExecutionConfig()); } @Override diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/TypeSerializerOutputFormat.java b/flink-java/src/main/java/org/apache/flink/api/java/io/TypeSerializerOutputFormat.java index a9b93ee23b697..0c9ed8027ac22 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/io/TypeSerializerOutputFormat.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/io/TypeSerializerOutputFormat.java @@ -18,6 +18,7 @@ package org.apache.flink.api.java.io; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.io.BinaryOutputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -52,7 +53,7 @@ public void setSerializer(TypeSerializer serializer){ @Override @SuppressWarnings("unchecked") - public void setInputType(TypeInformation type) { - serializer = (TypeSerializer) type.createSerializer(); + public void setInputType(TypeInformation type, ExecutionConfig executionConfig) { + serializer = (TypeSerializer) type.createSerializer(executionConfig); } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java index 9ea28f77ce927..acfb47a90fe50 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CrossOperator.java @@ -226,7 +226,7 @@ protected ProjectCross(DataSet input1, DataSet input2, int[] fields, boo TupleTypeInfo returnType, CrossHint hint) { super(input1, input2, - new ProjectCrossFunction(fields, isFromFirst, returnType.createSerializer().createInstance()), + new ProjectCrossFunction(fields, isFromFirst, returnType.createSerializer(input1.getExecutionEnvironment().getConfig()).createInstance()), returnType, hint, "unknown"); crossProjection = null; @@ -236,7 +236,7 @@ protected ProjectCross(DataSet input1, DataSet input2, int[] fields, boo TupleTypeInfo returnType, CrossProjection crossProjection, CrossHint hint) { super(input1, input2, - new ProjectCrossFunction(fields, isFromFirst, returnType.createSerializer().createInstance()), + new ProjectCrossFunction(fields, isFromFirst, returnType.createSerializer(input1.getExecutionEnvironment().getConfig()).createInstance()), returnType, hint, "unknown"); this.crossProjection = crossProjection; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java index e60314fcd3b1a..8b61779ec8bc2 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/JoinOperator.java @@ -633,7 +633,7 @@ public static class ProjectJoin extends EquiJoin input1, DataSet input2, Keys keys1, Keys keys2, JoinHint hint, int[] fields, boolean[] isFromFirst, TupleTypeInfo returnType) { super(input1, input2, keys1, keys2, - new ProjectFlatJoinFunction(fields, isFromFirst, returnType.createSerializer().createInstance()), + new ProjectFlatJoinFunction(fields, isFromFirst, returnType.createSerializer(input1.getExecutionEnvironment().getConfig()).createInstance()), returnType, hint, Utils.getCallLocationName(4)); // We need to use the 4th element in the stack because the call comes through .types(). @@ -642,7 +642,7 @@ protected ProjectJoin(DataSet input1, DataSet input2, Keys keys1, Ke protected ProjectJoin(DataSet input1, DataSet input2, Keys keys1, Keys keys2, JoinHint hint, int[] fields, boolean[] isFromFirst, TupleTypeInfo returnType, JoinProjection joinProj) { super(input1, input2, keys1, keys2, - new ProjectFlatJoinFunction(fields, isFromFirst, returnType.createSerializer().createInstance()), + new ProjectFlatJoinFunction(fields, isFromFirst, returnType.createSerializer(input1.getExecutionEnvironment().getConfig()).createInstance()), returnType, hint, Utils.getCallLocationName(4)); this.joinProj = joinProj; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java index bddef8fffd29d..16d9ff334fa70 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/ProjectOperator.java @@ -49,7 +49,7 @@ public class ProjectOperator protected final int[] fields; private Projection proj; - + public ProjectOperator(DataSet input, int[] fields, TupleTypeInfo returnType) { super(input, returnType); @@ -68,7 +68,7 @@ public ProjectOperator(DataSet input, int[] fields, TupleTypeInfo retur protected org.apache.flink.api.common.operators.base.MapOperatorBase> translateToDataFlow(Operator input) { String name = getName() != null ? getName() : "Projection " + Arrays.toString(fields); // create operator - PlanProjectOperator ppo = new PlanProjectOperator(fields, name, getInputType(), getResultType()); + PlanProjectOperator ppo = new PlanProjectOperator(fields, name, getInputType(), getResultType(), context.getConfig()); // set input ppo.setInput(input); // set dop diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanProjectOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanProjectOperator.java index 1b452da360a34..959b9297e45ba 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanProjectOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanProjectOperator.java @@ -18,6 +18,7 @@ package org.apache.flink.api.java.operators.translation; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.operators.UnaryOperatorInformation; @@ -27,8 +28,8 @@ public class PlanProjectOperator extends MapOperatorBase> { - public PlanProjectOperator(int[] fields, String name, TypeInformation inType, TypeInformation outType) { - super(new MapProjector(fields, outType.createSerializer().createInstance()), new UnaryOperatorInformation(inType, outType), name); + public PlanProjectOperator(int[] fields, String name, TypeInformation inType, TypeInformation outType, ExecutionConfig executionConfig) { + super(new MapProjector(fields, outType.createSerializer(executionConfig).createInstance()), new UnaryOperatorInformation(inType, outType), name); } public static final class MapProjector 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 c2a0b7f904582..291acce0ea177 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 org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.AtomicType; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -40,7 +41,7 @@ public EnumTypeInfo(Class typeClass) { } @Override - public TypeComparator createComparator(boolean sortOrderAscending) { + public TypeComparator createComparator(boolean sortOrderAscending, ExecutionConfig executionConfig) { return new EnumComparator(sortOrderAscending); } @@ -75,7 +76,7 @@ public boolean isKeyType() { } @Override - public TypeSerializer createSerializer() { + public TypeSerializer createSerializer(ExecutionConfig executionConfig) { return new EnumSerializer(typeClass); } 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 5bc6cb949520f..cb0ac3134ecfe 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 org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.AtomicType; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -25,7 +26,6 @@ import org.apache.flink.api.java.typeutils.runtime.GenericTypeComparator; import org.apache.flink.api.java.typeutils.runtime.KryoSerializer; - public class GenericTypeInfo extends TypeInformation implements AtomicType { private final Class typeClass; @@ -33,7 +33,7 @@ public class GenericTypeInfo extends TypeInformation implements AtomicType public GenericTypeInfo(Class typeClass) { this.typeClass = typeClass; } - + @Override public boolean isBasicType() { return false; @@ -65,16 +65,16 @@ public boolean isKeyType() { } @Override - public TypeSerializer createSerializer() { - return new KryoSerializer(this.typeClass); + public TypeSerializer createSerializer(ExecutionConfig config) { + return new KryoSerializer(this.typeClass, config); } @SuppressWarnings("unchecked") @Override - public TypeComparator createComparator(boolean sortOrderAscending) { + public TypeComparator createComparator(boolean sortOrderAscending, ExecutionConfig executionConfig) { if (isKeyType()) { @SuppressWarnings("rawtypes") - GenericTypeComparator comparator = new GenericTypeComparator(sortOrderAscending, createSerializer(), this.typeClass); + GenericTypeComparator comparator = new GenericTypeComparator(sortOrderAscending, createSerializer(executionConfig), this.typeClass); return (TypeComparator) comparator; } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/InputTypeConfigurable.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/InputTypeConfigurable.java index e26326ed43625..f8b4247bebfad 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/InputTypeConfigurable.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/InputTypeConfigurable.java @@ -19,11 +19,13 @@ package org.apache.flink.api.java.typeutils; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; /** * {@link org.apache.flink.api.common.io.OutputFormat}s can implement this interface to be configured - * with the data type they will operate on. The method {@link #setInputType(TypeInformation)} will be + * with the data type they will operate on. The method {@link #setInputType(org.apache.flink.api + * .common.typeinfo.TypeInformation, org.apache.flink.api.common.ExecutionConfig)} will be * called when the output format is used with an output method such as * {@link org.apache.flink.api.java.DataSet#output(org.apache.flink.api.common.io.OutputFormat)}. */ @@ -32,8 +34,9 @@ public interface InputTypeConfigurable { /** * Method that is called on an {@link org.apache.flink.api.common.io.OutputFormat} when it is passed to * the DataSet's output method. May be used to configures the output format based on the data type. - * + * * @param type The data type of the input. + * @param executionConfig */ - void setInputType(TypeInformation type); + void setInputType(TypeInformation type, ExecutionConfig executionConfig); } 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 10ab02f29d1c0..f0da264331201 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 @@ -18,6 +18,7 @@ package org.apache.flink.api.java.typeutils; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.InvalidTypesException; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -79,7 +80,7 @@ public boolean isKeyType() { } @Override - public TypeSerializer createSerializer() { + public TypeSerializer createSerializer(ExecutionConfig executionConfig) { throw new UnsupportedOperationException("The missing type information cannot be used as a type information."); } 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 664350ecc64fa..b4bc86bf21017 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 @@ -22,6 +22,7 @@ import java.lang.reflect.ParameterizedType; import java.lang.reflect.Type; +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; @@ -106,15 +107,15 @@ public boolean isKeyType() { @SuppressWarnings("unchecked") @Override - public TypeSerializer createSerializer() { + public TypeSerializer 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) new GenericArraySerializer((Class) paramType.getRawType(), - this.componentInfo.createSerializer()); + this.componentInfo.createSerializer(executionConfig)); } else { - return (TypeSerializer) new GenericArraySerializer((Class) this.componentType, this.componentInfo.createSerializer()); + return (TypeSerializer) new GenericArraySerializer((Class) this.componentType, this.componentInfo.createSerializer(executionConfig)); } } 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 fb5ca449743c7..503b6f2fc0a8a 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 @@ -28,13 +28,14 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.CompositeType; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.operators.Keys.ExpressionKeys; import org.apache.flink.api.java.typeutils.runtime.PojoComparator; import org.apache.flink.api.java.typeutils.runtime.PojoSerializer; +import org.apache.flink.api.java.operators.Keys.ExpressionKeys; import com.google.common.base.Joiner; @@ -43,7 +44,7 @@ * TypeInformation for arbitrary (they have to be java-beans-style) java objects (what we call POJO). * */ -public class PojoTypeInfo extends CompositeType{ +public class PojoTypeInfo extends CompositeType { private final static String REGEX_FIELD = "[\\p{L}_\\$][\\p{L}\\p{Digit}_\\$]*"; private final static String REGEX_NESTED_FIELDS = "("+REGEX_FIELD+")(\\.(.+))?"; @@ -269,7 +270,7 @@ protected void addCompareField(int fieldId, TypeComparator comparator) { } @Override - protected TypeComparator getNewComparator() { + protected TypeComparator getNewComparator(ExecutionConfig config) { // first remove the null array fields final Field[] finalKeyFields = Arrays.copyOf(keyFields, comparatorHelperIndex); @SuppressWarnings("rawtypes") @@ -277,21 +278,21 @@ protected TypeComparator getNewComparator() { if(finalFieldComparators.length == 0 || finalKeyFields.length == 0 || finalFieldComparators.length != finalKeyFields.length) { throw new IllegalArgumentException("Pojo comparator creation has a bug"); } - return new PojoComparator(finalKeyFields, finalFieldComparators, createSerializer(), typeClass); + return new PojoComparator(finalKeyFields, finalFieldComparators, createSerializer(config), typeClass); } @Override - public TypeSerializer createSerializer() { + public TypeSerializer createSerializer(ExecutionConfig config) { 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(); + fieldSerializers[i] = fields[i].type.createSerializer(config); reflectiveFields[i] = fields[i].field; } - return new PojoSerializer(this.typeClass, fieldSerializers, reflectiveFields); + return new PojoSerializer(this.typeClass, fieldSerializers, reflectiveFields, config); } // -------------------------------------------------------------------------------------------- @@ -312,7 +313,7 @@ public String toString() { for (PojoField field : fields) { fieldStrings.add(field.field.getName() + ": " + field.type.toString()); } - return "PojoType<" + typeClass.getCanonicalName() + return "PojoType<" + typeClass.getName() + ", fields = [" + Joiner.on(", ").join(fieldStrings) + "]" + ">"; } 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 2464f255a5cfc..60fa1105b606f 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 @@ -18,6 +18,7 @@ package org.apache.flink.api.java.typeutils; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.record.RecordSerializer; @@ -60,7 +61,7 @@ public boolean isKeyType() { } @Override - public TypeSerializer createSerializer() { + public TypeSerializer createSerializer(ExecutionConfig config) { return RecordSerializer.get(); } 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 af258e2818395..c1e573d2afd58 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 @@ -20,11 +20,12 @@ import java.util.Arrays; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; +//CHECKSTYLE.OFF: AvoidStarImport - Needed for TupleGenerator import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; -//CHECKSTYLE.OFF: AvoidStarImport - Needed for TupleGenerator import org.apache.flink.api.java.tuple.*; //CHECKSTYLE.ON: AvoidStarImport import org.apache.flink.api.java.typeutils.runtime.TupleComparator; @@ -48,10 +49,10 @@ public TupleTypeInfo(Class tupleType, TypeInformation... types) { } @Override - public TupleSerializer createSerializer() { + public TupleSerializer createSerializer(ExecutionConfig executionConfig) { TypeSerializer[] fieldSerializers = new TypeSerializer[getArity()]; for (int i = 0; i < types.length; i++) { - fieldSerializers[i] = types[i].createSerializer(); + fieldSerializers[i] = types[i].createSerializer(executionConfig); } Class tupleClass = getTypeClass(); @@ -81,7 +82,7 @@ protected void addCompareField(int fieldId, TypeComparator comparator) { } @Override - protected TypeComparator getNewComparator() { + protected TypeComparator getNewComparator(ExecutionConfig executionConfig) { @SuppressWarnings("rawtypes") final TypeComparator[] finalFieldComparators = Arrays.copyOf(fieldComparators, comparatorHelperIndex); final int[] finalLogicalKeyFields = Arrays.copyOf(logicalKeyFields, comparatorHelperIndex); @@ -93,7 +94,7 @@ protected TypeComparator getNewComparator() { } TypeSerializer[] fieldSerializers = new TypeSerializer[maxKey + 1]; for (int i = 0; i <= maxKey; i++) { - fieldSerializers[i] = types[i].createSerializer(); + fieldSerializers[i] = types[i].createSerializer(executionConfig); } if(finalFieldComparators.length == 0 || finalLogicalKeyFields.length == 0 || fieldSerializers.length == 0 || finalFieldComparators.length != finalLogicalKeyFields.length) { 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 c99a80f248b6d..1b3003f994b41 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 @@ -1255,6 +1255,7 @@ private boolean isValidPojoField(Field f, Class clazz, ArrayList typeHi @SuppressWarnings("unchecked") private TypeInformation analyzePojo(Class clazz, ArrayList typeHierarchy, ParameterizedType parameterizedType, TypeInformation in1Type, TypeInformation in2Type) { + // add the hierarchy of the POJO itself if it is generic if (parameterizedType != null) { getTypeHierarchy(typeHierarchy, parameterizedType, Object.class); 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 79a27607a83c2..1486ee56baee3 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 org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.InvalidTypesException; import org.apache.flink.api.common.typeinfo.AtomicType; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -79,7 +80,7 @@ public boolean isKeyType() { @Override @SuppressWarnings("unchecked") - public TypeSerializer createSerializer() { + public TypeSerializer createSerializer(ExecutionConfig executionConfig) { if (CopyableValue.class.isAssignableFrom(type)) { return (TypeSerializer) createCopyableValueSerializer(type.asSubclass(CopyableValue.class)); } @@ -90,7 +91,7 @@ public TypeSerializer createSerializer() { @SuppressWarnings({ "unchecked", "rawtypes" }) @Override - public TypeComparator createComparator(boolean sortOrderAscending) { + public TypeComparator createComparator(boolean sortOrderAscending, ExecutionConfig executionConfig) { if (!isKeyType()) { throw new RuntimeException("The type " + type.getName() + " is not Comparable."); } 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 195ce258b87bc..89d78184a9414 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 org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.InvalidTypesException; import org.apache.flink.api.common.typeinfo.AtomicType; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -43,7 +44,7 @@ public WritableTypeInfo(Class typeClass) { @SuppressWarnings({ "rawtypes", "unchecked" }) @Override - public TypeComparator createComparator(boolean sortOrderAscending) { + public TypeComparator createComparator(boolean sortOrderAscending, ExecutionConfig executionConfig) { if(Comparable.class.isAssignableFrom(typeClass)) { return new WritableComparator(sortOrderAscending, typeClass); } @@ -84,7 +85,7 @@ public boolean isKeyType() { } @Override - public TypeSerializer createSerializer() { + public TypeSerializer createSerializer(ExecutionConfig executionConfig) { return new WritableSerializer(typeClass); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoSerializer.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoSerializer.java index c55cd71b0b123..133dd576353bd 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoSerializer.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoSerializer.java @@ -27,6 +27,7 @@ import com.esotericsoftware.kryo.serializers.JavaSerializer; import com.twitter.chill.ScalaKryoInstantiator; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -36,9 +37,6 @@ import java.io.EOFException; import java.io.IOException; import java.lang.reflect.Modifier; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; import java.util.Map; import java.util.Set; @@ -55,13 +53,8 @@ public class KryoSerializer extends TypeSerializer { private static final long serialVersionUID = 3L; - private static Map, Serializer> staticRegisteredSerializers = new HashMap, Serializer>(); - private static Map, Class>> staticRegisteredSerializersClasses = new HashMap, Class>>(); - - private static Set> staticRegisteredTypes = new HashSet>(); - // ------------------------------------------------------------------------ - + private final Map, Serializer> registeredSerializers; private final Map, Class>> registeredSerializersClasses; private final Set> registeredTypes; @@ -82,29 +75,15 @@ public class KryoSerializer extends TypeSerializer { // ------------------------------------------------------------------------ - public KryoSerializer(Class type){ + public KryoSerializer(Class type, ExecutionConfig executionConfig){ if(type == null){ throw new NullPointerException("Type class cannot be null."); } this.type = type; - // create copies of the statically registered serializers - // we use static synchronization to safeguard against concurrent use - // of the static collections. - synchronized (KryoSerializer.class) { - this.registeredSerializers = staticRegisteredSerializers.isEmpty() ? - Collections., Serializer>emptyMap() : - new HashMap, Serializer>(staticRegisteredSerializers); - - this.registeredSerializersClasses = staticRegisteredSerializersClasses.isEmpty() ? - Collections., Class>>emptyMap() : - new HashMap, Class>>(staticRegisteredSerializersClasses); - - this.registeredTypes = staticRegisteredTypes.isEmpty() ? - Collections.>emptySet() : - new HashSet>(staticRegisteredTypes); - } - + this.registeredSerializers = executionConfig.getRegisteredKryoSerializers(); + this.registeredSerializersClasses = executionConfig.getRegisteredKryoSerializersClasses(); + this.registeredTypes = executionConfig.getRegisteredKryoTypes(); } /** @@ -289,69 +268,6 @@ private void checkKryoInitialized() { kryo.setClassLoader(Thread.currentThread().getContextClassLoader()); } } - - // -------------------------------------------------------------------------------------------- - // For registering custom serializers and types - // -------------------------------------------------------------------------------------------- - - /** - * Registers the given Serializer as a default serializer for the given class at the Kryo - * instance. - * Note that the serializer instance must be serializable (as defined by java.io.Serializable), - * because it may be distributed to the worker nodes by java serialization. - * - * @param clazz The class of the types serialized with the given serializer. - * @param serializer The serializer to use. - * @throws IllegalArgumentException Thrown, if the serializer is not serializable. - */ - public static void registerSerializer(Class clazz, Serializer serializer) { - if (clazz == null || serializer == null) { - throw new NullPointerException("Cannot register null class or serializer."); - } - if (!(serializer instanceof java.io.Serializable)) { - throw new IllegalArgumentException("The serializer instance must be serializable, (for distributing it in the cluster), " - + "as defined by java.io.Serializable. For stateless serializers, you can use the " - + "'registerSerializer(Class, Class)' method to register the serializer via its class."); - } - - synchronized (KryoSerializer.class) { - staticRegisteredSerializers.put(clazz, serializer); - } - } - - /** - * Registers a serializer via its class as a default serializer for the given class at the Kryo - * instance. - * - * @param clazz The class of the types serialized with the given serializer. - * @param serializerClass The serializer to use. - */ - public static void registerSerializer(Class clazz, Class> serializerClass) { - if (clazz == null || serializerClass == null) { - throw new NullPointerException("Cannot register null class or serializer."); - } - - synchronized (KryoSerializer.class) { - staticRegisteredSerializersClasses.put(clazz, serializerClass); - } - } - - /** - * Registers the given type with Kryo. Registering the type allows Kryo to write abbreviated - * name tags, rather than full class names, thereby vastly increasing the serialization - * performance in many cases. - * - * @param type The class of the type to register. - */ - public static void registerType(Class type) { - if (type == null) { - throw new NullPointerException("Cannot register null type class."); - } - - synchronized (KryoSerializer.class) { - staticRegisteredTypes.add(type); - } - } // -------------------------------------------------------------------------------------------- // For testing 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 15e8537c480e6..603309448d0fc 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 @@ -22,36 +22,96 @@ import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.lang.reflect.Field; +import java.lang.reflect.Modifier; +import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; public final class PojoSerializer extends TypeSerializer { + // Flags for the header + private static byte IS_NULL = 1; + private static byte NO_SUBCLASS = 2; + private static byte IS_SUBCLASS = 4; + private static byte IS_TAGGED_SUBCLASS = 8; + private static final long serialVersionUID = 1L; private final Class clazz; - private final TypeSerializer[] fieldSerializers; + private TypeSerializer[] fieldSerializers; // We need to handle these ourselves in writeObject()/readObject() private transient Field[] fields; - private final int numFields; + private int numFields; + + private transient Map, TypeSerializer> subclassSerializerCache; + private transient ClassLoader cl; + + private Map, Integer> registeredClasses; + + private TypeSerializer[] registeredSerializers; + + private final ExecutionConfig executionConfig; @SuppressWarnings("unchecked") - public PojoSerializer(Class clazz, TypeSerializer[] fieldSerializers, Field[] fields) { + public PojoSerializer( + Class clazz, + TypeSerializer[] fieldSerializers, + Field[] fields, + ExecutionConfig executionConfig) { this.clazz = clazz; this.fieldSerializers = (TypeSerializer[]) fieldSerializers; this.fields = fields; this.numFields = fieldSerializers.length; + this.executionConfig = executionConfig; + + Set> registeredPojoTypes = executionConfig.getRegisteredPojoTypes(); for (int i = 0; i < numFields; i++) { this.fields[i].setAccessible(true); } + + cl = Thread.currentThread().getContextClassLoader(); + + subclassSerializerCache = new HashMap, TypeSerializer>(); + + // We only want those classes that are not our own class and are actually sub-classes. + List> cleanedTaggedClasses = new ArrayList>(registeredPojoTypes.size()); + for (Class registeredClass: registeredPojoTypes) { + if (registeredClass.equals(clazz)) { + continue; + } + if (!clazz.isAssignableFrom(registeredClass)) { + continue; + } + cleanedTaggedClasses.add(registeredClass); + + } + this.registeredClasses = new LinkedHashMap, Integer>(cleanedTaggedClasses.size()); + registeredSerializers = new TypeSerializer[cleanedTaggedClasses.size()]; + + int id = 0; + for (Class registeredClass: cleanedTaggedClasses) { + this.registeredClasses.put(registeredClass, id); + TypeInformation typeInfo = TypeExtractor.createTypeInfo(registeredClass); + registeredSerializers[id] = typeInfo.createSerializer(executionConfig); + + id++; + } } private void writeObject(ObjectOutputStream out) @@ -67,9 +127,9 @@ private void writeObject(ObjectOutputStream out) private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { in.defaultReadObject(); - int numKeyFields = in.readInt(); - fields = new Field[numKeyFields]; - for (int i = 0; i < numKeyFields; i++) { + int numFields = in.readInt(); + fields = new Field[numFields]; + for (int i = 0; i < numFields; i++) { Class clazz = (Class)in.readObject(); String fieldName = in.readUTF(); fields[i] = null; @@ -88,8 +148,42 @@ private void readObject(ObjectInputStream in) + " (" + fieldName + ")"); } } + + cl = Thread.currentThread().getContextClassLoader(); + subclassSerializerCache = new HashMap, TypeSerializer>(); + } + + private TypeSerializer getSubclassSerializer(Class subclass) { + TypeSerializer result = subclassSerializerCache.get(subclass); + if (result == null) { + + TypeInformation typeInfo = TypeExtractor.createTypeInfo(subclass); + result = typeInfo.createSerializer(executionConfig); + if (result instanceof PojoSerializer) { + PojoSerializer subclassSerializer = (PojoSerializer) result; + subclassSerializer.copyBaseFieldOrder(this); + } + subclassSerializerCache.put(subclass, result); + + } + return result; + } + + private boolean hasField(Field f) { + for (Field field: fields) { + if (f.equals(field)) { + return true; + } + } + return false; + } + + @SuppressWarnings("unchecked") + private void copyBaseFieldOrder(PojoSerializer baseSerializer) { + // do nothing for now, but in the future, adapt subclass serializer to have same + // ordering as base class serializer so that binary comparison on base class fields + // can work } - @Override public boolean isImmutableType() { @@ -110,7 +204,7 @@ public PojoSerializer duplicate() { } if (stateful) { - return new PojoSerializer(clazz, duplicateFieldSerializers, fields); + return new PojoSerializer(clazz, duplicateFieldSerializers, fields, executionConfig); } else { return this; } @@ -119,13 +213,12 @@ public PojoSerializer duplicate() { @Override public T createInstance() { + if (clazz.isInterface() || Modifier.isAbstract(clazz.getModifiers())) { + return null; + } try { T t = clazz.newInstance(); - - for (int i = 0; i < numFields; i++) { - fields[i].set(t, fieldSerializers[i].createInstance()); - } - + initializeFields(t); return t; } catch (Exception e) { @@ -133,51 +226,89 @@ public T createInstance() { } } + protected void initializeFields(T t) { + for (int i = 0; i < numFields; i++) { + try { + fields[i].set(t, fieldSerializers[i].createInstance()); + } catch (IllegalAccessException e) { + throw new RuntimeException("Cannot initialize fields.", e); + } + } + } + @Override + @SuppressWarnings("unchecked") public T copy(T from) { - T target; - try { - target = clazz.newInstance(); - } - catch (Throwable t) { - throw new RuntimeException("Cannot instantiate class.", t); + if (from == null) { + return null; } - - try { - for (int i = 0; i < numFields; i++) { - Object value = fields[i].get(from); - if (value != null) { - Object copy = fieldSerializers[i].copy(value); - fields[i].set(target, copy); - } - else { - fields[i].set(target, null); + + Class actualType = from.getClass(); + if (actualType == clazz) { + T target; + try { + target = (T) from.getClass().newInstance(); + } + catch (Throwable t) { + throw new RuntimeException("Cannot instantiate class.", t); + } + // no subclass + try { + for (int i = 0; i < numFields; i++) { + Object value = fields[i].get(from); + if (value != null) { + Object copy = fieldSerializers[i].copy(value); + fields[i].set(target, copy); + } + else { + fields[i].set(target, null); + } } + } catch (IllegalAccessException e) { + throw new RuntimeException("Error during POJO copy, this should not happen since we check the fields before."); + } + return target; + } else { + // subclass + TypeSerializer subclassSerializer = getSubclassSerializer(actualType); + return (T) subclassSerializer.copy(from); } - catch (IllegalAccessException e) { - throw new RuntimeException("Error during POJO copy, this should not happen since we check the fields before."); - } - return target; } @Override + @SuppressWarnings("unchecked") public T copy(T from, T reuse) { - try { - for (int i = 0; i < numFields; i++) { - Object value = fields[i].get(from); - if (value != null) { - Object copy = fieldSerializers[i].copy(fields[i].get(from), fields[i].get(reuse)); - fields[i].set(reuse, copy); - } - else { - fields[i].set(reuse, null); + if (from == null) { + return null; + } + + Class actualType = from.getClass(); + if (reuse == null || actualType != reuse.getClass()) { + // cannot reuse, do a non-reuse copy + return copy(from); + } + + if (actualType == clazz) { + try { + for (int i = 0; i < numFields; i++) { + Object value = fields[i].get(from); + if (value != null) { + Object copy = fieldSerializers[i].copy(fields[i].get(from), fields[i].get(reuse)); + fields[i].set(reuse, copy); + } + else { + fields[i].set(reuse, null); + } } + } catch (IllegalAccessException e) { + throw new RuntimeException("Error during POJO copy, this should not happen since we check the fields" + "before."); } - } catch (IllegalAccessException e) { - throw new RuntimeException("Error during POJO copy, this should not happen since we check the fields" + - "before."); + } else { + TypeSerializer subclassSerializer = getSubclassSerializer(actualType); + reuse = (T) subclassSerializer.copy(from, reuse); } + return reuse; } @@ -188,94 +319,228 @@ public int getLength() { @Override + @SuppressWarnings("unchecked") public void serialize(T value, DataOutputView target) throws IOException { + int flags = 0; // handle null values if (value == null) { - target.writeBoolean(true); + flags |= IS_NULL; + target.writeByte(flags); return; + } + + Integer subclassTag = -1; + Class actualClass = value.getClass(); + TypeSerializer subclassSerializer = null; + if (clazz != actualClass) { + subclassTag = registeredClasses.get(actualClass); + if (subclassTag != null) { + flags |= IS_TAGGED_SUBCLASS; + subclassSerializer = registeredSerializers[subclassTag]; + } else { + flags |= IS_SUBCLASS; + subclassSerializer = getSubclassSerializer(actualClass); + } } else { - target.writeBoolean(false); + flags |= NO_SUBCLASS; } - try { - for (int i = 0; i < numFields; i++) { - Object o = fields[i].get(value); - if(o == null) { - target.writeBoolean(true); // null field handling - } else { - target.writeBoolean(false); - fieldSerializers[i].serialize(o, target); + + target.writeByte(flags); + + if ((flags & IS_SUBCLASS) != 0) { + target.writeUTF(actualClass.getName()); + } else if ((flags & IS_TAGGED_SUBCLASS) != 0) { + target.writeByte(subclassTag); + } + + + if ((flags & NO_SUBCLASS) != 0) { + try { + for (int i = 0; i < numFields; i++) { + Object o = fields[i].get(value); + if (o == null) { + target.writeBoolean(true); // null field handling + } else { + target.writeBoolean(false); + fieldSerializers[i].serialize(o, target); + } } + } catch (IllegalAccessException e) { + throw new RuntimeException("Error during POJO copy, this should not happen since we check the fields" + "before."); + + } + } else { + // subclass + if (subclassSerializer != null) { + subclassSerializer.serialize(value, target); } - } catch (IllegalAccessException e) { - throw new RuntimeException("Error during POJO copy, this should not happen since we check the fields" + - "before."); } } @Override + @SuppressWarnings("unchecked") public T deserialize(DataInputView source) throws IOException { - boolean isNull = source.readBoolean(); - if(isNull) { + int flags = source.readByte(); + if((flags & IS_NULL) != 0) { return null; } + T target; - try { - target = clazz.newInstance(); - } - catch (Throwable t) { - throw new RuntimeException("Cannot instantiate class.", t); + + Class actualSubclass = null; + TypeSerializer subclassSerializer = null; + + if ((flags & IS_SUBCLASS) != 0) { + String subclassName = source.readUTF(); + try { + actualSubclass = Class.forName(subclassName, true, cl); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Cannot instantiate class.", e); + } + subclassSerializer = getSubclassSerializer(actualSubclass); + target = (T) subclassSerializer.createInstance(); + // also initialize fields for which the subclass serializer is not responsible + initializeFields(target); + } else if ((flags & IS_TAGGED_SUBCLASS) != 0) { + + int subclassTag = source.readByte(); + subclassSerializer = registeredSerializers[subclassTag]; + target = (T) subclassSerializer.createInstance(); + // also initialize fields for which the subclass serializer is not responsible + initializeFields(target); + } else { + target = createInstance(); } - - try { - for (int i = 0; i < numFields; i++) { - isNull = source.readBoolean(); - if(isNull) { - fields[i].set(target, null); - } else { - Object field = fieldSerializers[i].deserialize(source); - fields[i].set(target, field); + + if ((flags & NO_SUBCLASS) != 0) { + try { + for (int i = 0; i < numFields; i++) { + boolean isNull = source.readBoolean(); + if (isNull) { + fields[i].set(target, null); + } else { + Object field = fieldSerializers[i].deserialize(source); + fields[i].set(target, field); + } } + } catch (IllegalAccessException e) { + throw new RuntimeException("Error during POJO copy, this should not happen since we check the fields" + "before."); + + } + } else { + if (subclassSerializer != null) { + target = (T) subclassSerializer.deserialize(target, source); } - } catch (IllegalAccessException e) { - throw new RuntimeException("Error during POJO copy, this should not happen since we check the fields" + - "before."); } return target; } @Override + @SuppressWarnings("unchecked") public T deserialize(T reuse, DataInputView source) throws IOException { + // handle null values - boolean isNull = source.readBoolean(); - if (isNull) { + int flags = source.readByte(); + if((flags & IS_NULL) != 0) { return null; } - try { - for (int i = 0; i < numFields; i++) { - isNull = source.readBoolean(); - if(isNull) { - fields[i].set(reuse, null); - } else { - Object field = fieldSerializers[i].deserialize(fields[i].get(reuse), source); - fields[i].set(reuse, field); + + Class subclass = null; + TypeSerializer subclassSerializer = null; + if ((flags & IS_SUBCLASS) != 0) { + String subclassName = source.readUTF(); + try { + subclass = Class.forName(subclassName, true, cl); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Cannot instantiate class.", e); + } + subclassSerializer = getSubclassSerializer(subclass); + + if (reuse == null || subclass != reuse.getClass()) { + // cannot reuse + reuse = (T) subclassSerializer.createInstance(); + // also initialize fields for which the subclass serializer is not responsible + initializeFields(reuse); + } + } else if ((flags & IS_TAGGED_SUBCLASS) != 0) { + int subclassTag = source.readByte(); + subclassSerializer = registeredSerializers[subclassTag]; + + if (reuse == null || ((PojoSerializer)subclassSerializer).clazz != reuse.getClass()) { + // cannot reuse + reuse = (T) subclassSerializer.createInstance(); + // also initialize fields for which the subclass serializer is not responsible + initializeFields(reuse); + } + } else { + if (reuse == null || clazz != reuse.getClass()) { + reuse = createInstance(); + } + } + + if ((flags & NO_SUBCLASS) != 0) { + try { + for (int i = 0; i < numFields; i++) { + boolean isNull = source.readBoolean(); + if (isNull) { + fields[i].set(reuse, null); + } else { + Object field = fieldSerializers[i].deserialize(fields[i].get(reuse), source); + + fields[i].set(reuse, field); + } } + } catch (IllegalAccessException e) { + throw new RuntimeException("Error during POJO copy, this should not happen since we check the fields" + "before."); + } + } else { + if (subclassSerializer != null) { + reuse = (T) subclassSerializer.deserialize(reuse, source); } - } catch (IllegalAccessException e) { - throw new RuntimeException("Error during POJO copy, this should not happen since we check the fields" + - "before."); } + return reuse; } @Override public void copy(DataInputView source, DataOutputView target) throws IOException { - // copy the Non-Null/Null tag - target.writeBoolean(source.readBoolean()); - for (int i = 0; i < numFields; i++) { - boolean isNull = source.readBoolean(); - target.writeBoolean(isNull); - if (!isNull) { - fieldSerializers[i].copy(source, target); + // copy the flags + int flags = source.readByte(); + target.writeByte(flags); + + if ((flags & IS_NULL) != 0) { + // is a null value, nothing further to copy + return; + } + + TypeSerializer subclassSerializer = null; + if ((flags & IS_SUBCLASS) != 0) { + String className = source.readUTF(); + target.writeUTF(className); + try { + Class subclass = Class.forName(className, true, Thread.currentThread() + .getContextClassLoader()); + subclassSerializer = getSubclassSerializer(subclass); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Cannot instantiate class.", e); + } + } else if ((flags & IS_TAGGED_SUBCLASS) != 0) { + int subclassTag = source.readByte(); + target.writeByte(subclassTag); + subclassSerializer = registeredSerializers[subclassTag]; + } + + if ((flags & NO_SUBCLASS) != 0) { + for (int i = 0; i < numFields; i++) { + boolean isNull = source.readBoolean(); + target.writeBoolean(isNull); + if (!isNull) { + fieldSerializers[i].copy(source, target); + } + } + } else { + if (subclassSerializer != null) { + subclassSerializer.copy(source, target); } } } diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/CoGroupOperatorCollectionTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/CoGroupOperatorCollectionTest.java index 51d4b0e1e7a57..0aa6097bfa6c5 100644 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/CoGroupOperatorCollectionTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/CoGroupOperatorCollectionTest.java @@ -18,6 +18,7 @@ package org.apache.flink.api.common.operators.base; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.functions.RichCoGroupFunction; import org.apache.flink.api.common.functions.RuntimeContext; @@ -66,16 +67,19 @@ public void testExecuteOnCollection() { .build() ); - final RuntimeContext ctx = new RuntimeUDFContext("Test UDF", 4, 0, null); + ExecutionConfig executionConfig = new ExecutionConfig(); + final RuntimeContext ctx = new RuntimeUDFContext("Test UDF", 4, 0, null, executionConfig); { SumCoGroup udf1 = new SumCoGroup(); SumCoGroup udf2 = new SumCoGroup(); - + + executionConfig.disableObjectReuse(); List> resultSafe = getCoGroupOperator(udf1) - .executeOnCollections(input1, input2, ctx, true); + .executeOnCollections(input1, input2, ctx, executionConfig); + executionConfig.enableObjectReuse(); List> resultRegular = getCoGroupOperator(udf2) - .executeOnCollections(input1, input2, ctx, false); + .executeOnCollections(input1, input2, ctx, executionConfig); Assert.assertTrue(udf1.isClosed); Assert.assertTrue(udf2.isClosed); @@ -95,13 +99,15 @@ public void testExecuteOnCollection() { } { + executionConfig.disableObjectReuse(); List> resultSafe = getCoGroupOperator(new SumCoGroup()) .executeOnCollections(Collections.>emptyList(), - Collections.>emptyList(), ctx, true); - + Collections.>emptyList(), ctx, executionConfig); + + executionConfig.enableObjectReuse(); List> resultRegular = getCoGroupOperator(new SumCoGroup()) .executeOnCollections(Collections.>emptyList(), - Collections.>emptyList(), ctx, false); + Collections.>emptyList(), ctx, executionConfig); Assert.assertEquals(0, resultSafe.size()); Assert.assertEquals(0, resultRegular.size()); diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorTest.java index cfca5aa1eac41..447c8c589dc6f 100644 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorTest.java @@ -18,6 +18,7 @@ package org.apache.flink.api.common.operators.base; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.api.common.functions.RuntimeContext; @@ -79,9 +80,12 @@ public void reduce(Iterable> values, Integer>("foo", 3), new Tuple2("bar", 2), new Tuple2("bar", 4))); - - List> resultMutableSafe = op.executeOnCollections(input, null, true); - List> resultRegular = op.executeOnCollections(input, null, false); + + ExecutionConfig executionConfig = new ExecutionConfig(); + executionConfig.disableObjectReuse(); + List> resultMutableSafe = op.executeOnCollections(input, null, executionConfig); + executionConfig.enableObjectReuse(); + List> resultRegular = op.executeOnCollections(input, null, executionConfig); Set> resultSetMutableSafe = new HashSet>(resultMutableSafe); Set> resultSetRegular = new HashSet>(resultRegular); @@ -155,8 +159,11 @@ public void close() throws Exception { Integer>("foo", 3), new Tuple2("bar", 2), new Tuple2("bar", 4))); - List> resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskName, 1, 0, null), true); - List> resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskName, 1, 0, null), false); + ExecutionConfig executionConfig = new ExecutionConfig(); + executionConfig.disableObjectReuse(); + List> resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskName, 1, 0, null, executionConfig), executionConfig); + executionConfig.enableObjectReuse(); + List> resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskName, 1, 0, null, executionConfig), executionConfig); Set> resultSetMutableSafe = new HashSet>(resultMutableSafe); diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/JoinOperatorBaseTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/JoinOperatorBaseTest.java index b4ef54f8e3f89..1b38281eadcc3 100644 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/JoinOperatorBaseTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/JoinOperatorBaseTest.java @@ -20,10 +20,10 @@ import static org.junit.Assert.*; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.functions.util.RuntimeUDFContext; import org.apache.flink.api.common.operators.BinaryOperatorInformation; -import org.apache.flink.api.common.operators.base.JoinOperatorBase; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.typeutils.TupleTypeInfo; @@ -101,8 +101,11 @@ public void join(Tuple3 first, Tuple2 )); try { - List> resultSafe = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext("op", 1, 0, null), true); - List> resultRegular = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext("op", 1, 0, null), false); + ExecutionConfig executionConfig = new ExecutionConfig(); + executionConfig.disableObjectReuse(); + List> resultSafe = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext("op", 1, 0, null, executionConfig), executionConfig); + executionConfig.enableObjectReuse(); + List> resultRegular = base.executeOnCollections(inputData1, inputData2, new RuntimeUDFContext("op", 1, 0, null, executionConfig), executionConfig); assertEquals(expected, new HashSet>(resultSafe)); assertEquals(expected, new HashSet>(resultRegular)); diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/ReduceOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/ReduceOperatorTest.java index 90bbe41d14be0..4e1eebd75220f 100644 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/ReduceOperatorTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/ReduceOperatorTest.java @@ -18,6 +18,7 @@ package org.apache.flink.api.common.operators.base; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.RichReduceFunction; import org.apache.flink.api.common.functions.RuntimeContext; @@ -68,8 +69,11 @@ public Tuple2 reduce(Tuple2 value1, Integer>("foo", 3), new Tuple2("bar", 2), new Tuple2("bar", 4))); - List> resultMutableSafe = op.executeOnCollections(input, null, true); - List> resultRegular = op.executeOnCollections(input, null, false); + ExecutionConfig executionConfig = new ExecutionConfig(); + executionConfig.disableObjectReuse(); + List> resultMutableSafe = op.executeOnCollections(input, null, executionConfig); + executionConfig.enableObjectReuse(); + List> resultRegular = op.executeOnCollections(input, null, executionConfig); Set> resultSetMutableSafe = new HashSet>(resultMutableSafe); Set> resultSetRegular = new HashSet>(resultRegular); @@ -132,8 +136,11 @@ public void close() throws Exception { Integer>("foo", 3), new Tuple2("bar", 2), new Tuple2("bar", 4))); - List> resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskName, 1, 0, null), true); - List> resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskName, 1, 0, null), false); + ExecutionConfig executionConfig = new ExecutionConfig(); + executionConfig.disableObjectReuse(); + List> resultMutableSafe = op.executeOnCollections(input, new RuntimeUDFContext(taskName, 1, 0, null, executionConfig), executionConfig); + executionConfig.enableObjectReuse(); + List> resultRegular = op.executeOnCollections(input, new RuntimeUDFContext(taskName, 1, 0, null, executionConfig), executionConfig); Set> resultSetMutableSafe = new HashSet>(resultMutableSafe); Set> resultSetRegular = new HashSet>(resultRegular); 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 118e707cccfab..cd495b5cfb18a 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 @@ -23,6 +23,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +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; @@ -91,7 +92,7 @@ public void testSerializability() { TypeInformation info = (TypeInformation) TypeExtractor.createTypeInfo(ElementType.class); CollectionInputFormat inputFormat = new CollectionInputFormat(inputCollection, - info.createSerializer()); + info.createSerializer(new ExecutionConfig())); ByteArrayOutputStream buffer = new ByteArrayOutputStream(); ObjectOutputStream out = new ObjectOutputStream(buffer); @@ -168,7 +169,7 @@ public void testSerializabilityStrings() { try { List inputCollection = Arrays.asList(data); - CollectionInputFormat inputFormat = new CollectionInputFormat(inputCollection, BasicTypeInfo.STRING_TYPE_INFO.createSerializer()); + CollectionInputFormat inputFormat = new CollectionInputFormat(inputCollection, BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig())); // serialize ByteArrayOutputStream baos = new ByteArrayOutputStream(); diff --git a/flink-java/src/test/java/org/apache/flink/api/java/io/TypeSerializerFormatTest.java b/flink-java/src/test/java/org/apache/flink/api/java/io/TypeSerializerFormatTest.java index 7dd11352afd13..f6e3c2a4a3b69 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/io/TypeSerializerFormatTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/io/TypeSerializerFormatTest.java @@ -18,6 +18,7 @@ package org.apache.flink.api.java.io; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.io.BinaryInputFormat; import org.apache.flink.api.common.io.BinaryOutputFormat; import org.apache.flink.api.common.io.BlockInfo; @@ -51,7 +52,7 @@ public TypeSerializerFormatTest(int numberOfTuples, long blockSize, int degreeOf resultType = TypeExtractor.getForObject(getRecord(0)); - serializer = resultType.createSerializer(); + serializer = resultType.createSerializer(new ExecutionConfig()); } @Before 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 b96b3bcc084d0..dade55c009c63 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 @@ -1077,10 +1077,16 @@ public void testFunctionInputInOutputMultipleTimes2() { public interface Testable {} - public abstract class AbstractClass {} + public static abstract class AbstractClassWithoutMember {} + + public static abstract class AbstractClassWithMember { + public int x; + } @Test - public void testAbstractAndInterfaceTypesException() { + public void testAbstractAndInterfaceTypes() { + + // interface RichMapFunction function = new RichMapFunction() { private static final long serialVersionUID = 1L; @@ -1089,21 +1095,35 @@ public Testable map(String value) throws Exception { return null; } }; - - TypeInformation ti = TypeExtractor.getMapReturnTypes(function, BasicTypeInfo.STRING_TYPE_INFO, null, true); + + TypeInformation ti = TypeExtractor.getMapReturnTypes(function, BasicTypeInfo.STRING_TYPE_INFO); + Assert.assertTrue(ti instanceof GenericTypeInfo); + + // abstract class with out class member + RichMapFunction function2 = new RichMapFunction() { + private static final long serialVersionUID = 1L; + + @Override + public AbstractClassWithoutMember map(String value) throws Exception { + return null; + } + }; + + ti = TypeExtractor.getMapReturnTypes(function2, BasicTypeInfo.STRING_TYPE_INFO); Assert.assertTrue(ti instanceof GenericTypeInfo); - RichMapFunction function2 = new RichMapFunction() { + // abstract class with class member + RichMapFunction function3 = new RichMapFunction() { private static final long serialVersionUID = 1L; @Override - public AbstractClass map(String value) throws Exception { + public AbstractClassWithMember map(String value) throws Exception { return null; } }; - TypeInformation ti2 = TypeExtractor.getMapReturnTypes(function2, BasicTypeInfo.STRING_TYPE_INFO, null, true); - Assert.assertTrue(ti2 instanceof GenericTypeInfo); + ti = TypeExtractor.getMapReturnTypes(function3, BasicTypeInfo.STRING_TYPE_INFO); + Assert.assertTrue(ti instanceof PojoTypeInfo); } @SuppressWarnings({ "rawtypes", "unchecked" }) @@ -1727,7 +1747,7 @@ public CustomType[][][] map( + "myField1=String,myField2=int" + ">[][][]")); Assert.assertEquals("ObjectArrayTypeInfo" + + "PojoType" + ">>>", ti.toString()); // generic array diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/CompositeTypeTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/CompositeTypeTest.java index c71625a9b5686..33aa4498aaf9b 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/CompositeTypeTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/CompositeTypeTest.java @@ -52,7 +52,8 @@ public class CompositeTypeTest { inNestedTuple3, BasicTypeInfo.INT_TYPE_INFO ); - private final PojoTypeInfo pojoTypeInfo = ((PojoTypeInfo)TypeExtractor.getForClass(MyPojo.class)); + private final PojoTypeInfo pojoTypeInfo = ((PojoTypeInfo) TypeExtractor.getForClass + (MyPojo.class)); private final TupleTypeInfo pojoInTupleTypeInfo = new TupleTypeInfo>(BasicTypeInfo.INT_TYPE_INFO, pojoTypeInfo); 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 c2f3737ac7327..eadf96dc82ef4 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 @@ -28,12 +28,6 @@ import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.typeutils.GenericTypeInfo; -import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.api.java.typeutils.TypeInfoParser; -import org.apache.flink.api.java.typeutils.ValueTypeInfo; -import org.apache.flink.api.java.typeutils.WritableTypeInfo; import org.apache.flink.types.BooleanValue; import org.apache.flink.types.ByteValue; import org.apache.flink.types.CharValue; @@ -300,7 +294,7 @@ public void testMultiDimensionalArray() { // pojos ti = TypeInfoParser.parse("org.apache.flink.api.java.typeutils.TypeInfoParserTest$MyPojo[][][]"); Assert.assertEquals("ObjectArrayTypeInfo" + + "PojoType" + ">>>", ti.toString()); // basic types diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueComparatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueComparatorTest.java index a17f49924b23f..b3172752f4dda 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueComparatorTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueComparatorTest.java @@ -21,8 +21,6 @@ import org.apache.flink.api.common.typeutils.ComparatorTestBase; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.typeutils.runtime.CopyableValueComparator; -import org.apache.flink.api.java.typeutils.runtime.CopyableValueSerializer; import org.apache.flink.types.StringValue; public class CopyableValueComparatorTest extends ComparatorTestBase { diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/KryoGenericArraySerializerTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/KryoGenericArraySerializerTest.java index c1907270a7353..d1163d52d1321 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/KryoGenericArraySerializerTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/KryoGenericArraySerializerTest.java @@ -18,11 +18,12 @@ package org.apache.flink.api.java.typeutils.runtime; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; public class KryoGenericArraySerializerTest extends AbstractGenericArraySerializerTest { @Override protected TypeSerializer createComponentSerializer(Class type) { - return new KryoSerializer(type); + return new KryoSerializer(type, new ExecutionConfig()); } } diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/KryoGenericTypeComparatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/KryoGenericTypeComparatorTest.java index 37dba4e075138..01c76d9164fa7 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/KryoGenericTypeComparatorTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/KryoGenericTypeComparatorTest.java @@ -18,11 +18,12 @@ package org.apache.flink.api.java.typeutils.runtime; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; public class KryoGenericTypeComparatorTest extends AbstractGenericTypeComparatorTest { @Override protected TypeSerializer createSerializer(Class type) { - return new KryoSerializer(type); + return new KryoSerializer(type, new ExecutionConfig()); } } \ No newline at end of file diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/KryoGenericTypeSerializerTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/KryoGenericTypeSerializerTest.java index 5953599fa2d48..8630d95fe041f 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/KryoGenericTypeSerializerTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/KryoGenericTypeSerializerTest.java @@ -20,6 +20,7 @@ import static org.junit.Assert.*; +import org.apache.flink.api.common.ExecutionConfig; import org.junit.Test; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -70,7 +71,7 @@ private void fillCollection(Collection coll){ @Override protected TypeSerializer createSerializer(Class type) { - return new KryoSerializer(type); + return new KryoSerializer(type, new ExecutionConfig()); } /** @@ -94,7 +95,7 @@ public void testForwardEOFException() { // construct a memory target that is too small for the string TestDataOutputSerializer target = new TestDataOutputSerializer(10000, 30000); - KryoSerializer serializer = new KryoSerializer(String.class); + KryoSerializer serializer = new KryoSerializer(String.class, new ExecutionConfig()); try { serializer.serialize(str, target); diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/KryoVersusAvroMinibenchmark.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/KryoVersusAvroMinibenchmark.java index 4c6b39fc1e04c..b2ed2196eaa35 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/KryoVersusAvroMinibenchmark.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/KryoVersusAvroMinibenchmark.java @@ -27,6 +27,10 @@ import java.util.List; import java.util.Random; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.GenericTypeInfo; +import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.MemoryUtils; @@ -84,9 +88,12 @@ public static void main(String[] args) throws Exception { System.out.println("Kryo serializer"); { final TestDataOutputSerializer outView = new TestDataOutputSerializer(100000000); - final KryoSerializer serializer = new KryoSerializer(MyType.class); - serializer.getKryo().register(Tuple2.class); - + ExecutionConfig conf = new ExecutionConfig(); + conf.registerKryoType(MyType.class); + conf.enableForceKryo(); + TypeInformation typeInfo = new GenericTypeInfo(MyType.class); + final TypeSerializer serializer = typeInfo.createSerializer(conf); + long start = System.nanoTime(); for (int k = 0; k < NUM_ELEMENTS; k++) { diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/KryoWithCustomSerializersTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/KryoWithCustomSerializersTest.java index 9d7ab61d02624..7020d80d92ab1 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/KryoWithCustomSerializersTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/KryoWithCustomSerializersTest.java @@ -21,7 +21,10 @@ import java.util.Collection; import java.util.HashSet; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.typeutils.GenericTypeInfo; import org.joda.time.LocalDate; import org.junit.Test; @@ -40,15 +43,16 @@ public void testJodaTime(){ b.add(new LocalDate(1L)); b.add(new LocalDate(2L)); - - KryoSerializer.registerSerializer(LocalDate.class, LocalDateSerializer.class); - + runTests(b); } @Override protected TypeSerializer createSerializer(Class type) { - return new KryoSerializer(type); + ExecutionConfig conf = new ExecutionConfig(); + conf.registerKryoSerializer(LocalDate.class, LocalDateSerializer.class); + TypeInformation typeInfo = new GenericTypeInfo(type); + return typeInfo.createSerializer(conf); } public static final class LocalDateSerializer extends Serializer implements java.io.Serializable { diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/MultidimensionalArraySerializerTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/MultidimensionalArraySerializerTest.java index 22b6c767d22c3..1c978167c66fc 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/MultidimensionalArraySerializerTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/MultidimensionalArraySerializerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.api.java.typeutils.runtime; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.SerializerTestInstance; import org.apache.flink.api.java.typeutils.TypeExtractor; @@ -35,7 +36,7 @@ public void testStringArray() { String[][] array = new String[][]{{null,"b"},{"c","d"},{"e","f"},{"g","h"},null}; TypeInformation ti = TypeExtractor.getForClass(String[][].class); - SerializerTestInstance testInstance = new SerializerTestInstance(ti.createSerializer(), String[][].class, -1, array); + SerializerTestInstance testInstance = new SerializerTestInstance(ti.createSerializer(new ExecutionConfig()), String[][].class, -1, array); testInstance.testAll(); } @@ -44,7 +45,7 @@ public void testPrimitiveArray() { int[][] array = new int[][]{{12,1},{48,42},{23,80},{484,849},{987,4}}; TypeInformation ti = TypeExtractor.getForClass(int[][].class); - SerializerTestInstance testInstance = new SerializerTestInstance(ti.createSerializer(), int[][].class, -1, array); + SerializerTestInstance testInstance = new SerializerTestInstance(ti.createSerializer(new ExecutionConfig()), int[][].class, -1, array); testInstance.testAll(); } @@ -73,13 +74,13 @@ public void testObjectArrays() { Integer[][] array = new Integer[][]{{0,1}, null, {null, 42}}; TypeInformation ti = TypeExtractor.getForClass(Integer[][].class); - SerializerTestInstance testInstance = new SerializerTestInstance(ti.createSerializer(), Integer[][].class, -1, array); + SerializerTestInstance testInstance = new SerializerTestInstance(ti.createSerializer(new ExecutionConfig()), Integer[][].class, -1, array); testInstance.testAll(); MyPojo[][] array2 = new MyPojo[][]{{new MyPojo(null, 42), new MyPojo("test2", -1)}, {null, null}, null}; TypeInformation ti2 = TypeExtractor.getForClass(MyPojo[][].class); - SerializerTestInstance testInstance2 = new SerializerTestInstance(ti2.createSerializer(), MyPojo[][].class, -1, array2); + SerializerTestInstance testInstance2 = new SerializerTestInstance(ti2.createSerializer(new ExecutionConfig()), MyPojo[][].class, -1, array2); testInstance2.testAll(); } @@ -112,7 +113,7 @@ public void testGenericObjectArrays() { }; TypeInformation ti = TypeInfoParser.parse("org.apache.flink.api.java.typeutils.runtime.MultidimensionalArraySerializerTest$MyGenericPojo[][]"); - SerializerTestInstance testInstance = new SerializerTestInstance(ti.createSerializer(), MyGenericPojo[][].class, -1, (Object) array); + SerializerTestInstance testInstance = new SerializerTestInstance(ti.createSerializer(new ExecutionConfig()), MyGenericPojo[][].class, -1, (Object) array); testInstance.testAll(); } diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoComparatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoComparatorTest.java index e53f48abe3528..1baf443ef8c30 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoComparatorTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoComparatorTest.java @@ -20,6 +20,7 @@ import java.util.Arrays; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.ComparatorTestBase; import org.apache.flink.api.common.typeutils.CompositeType; @@ -47,12 +48,12 @@ protected TypeComparator createComparator(boolean ascending ExpressionKeys keys = new ExpressionKeys(new String[] {"theTuple.*"}, cType); boolean[] orders = new boolean[keys.getNumberOfKeyFields()]; Arrays.fill(orders, ascending); - return cType.createComparator(keys.computeLogicalKeyPositions(), orders, 0); + return cType.createComparator(keys.computeLogicalKeyPositions(), orders, 0, new ExecutionConfig()); } @Override protected TypeSerializer createSerializer() { - return type.createSerializer(); + return type.createSerializer(new ExecutionConfig()); } @Override diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoGenericTypeSerializerTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoGenericTypeSerializerTest.java index a176178bd1338..d405412fa67d5 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoGenericTypeSerializerTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoGenericTypeSerializerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.api.java.typeutils.runtime; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.typeutils.TypeExtractor; @@ -27,6 +28,6 @@ public class PojoGenericTypeSerializerTest extends AbstractGenericTypeSerializer @Override protected TypeSerializer createSerializer(Class type) { TypeInformation typeInfo = TypeExtractor.getForClass(type); - return typeInfo.createSerializer(); + return typeInfo.createSerializer(new ExecutionConfig()); } } diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializerTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializerTest.java index 006625ec6faf6..1fa71633b318b 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializerTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializerTest.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.Random; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.SerializerTestBase; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -40,14 +41,14 @@ import com.google.common.base.Objects; /** - * A test for the {@link org.apache.flink.api.java.typeutils.runtime.PojoSerializer}. + * A test for the {@link PojoSerializer}. */ public class PojoSerializerTest extends SerializerTestBase { private TypeInformation type = TypeExtractor.getForClass(TestUserClass.class); @Override protected TypeSerializer createSerializer() { - TypeSerializer serializer = type.createSerializer(); + TypeSerializer serializer = type.createSerializer(new ExecutionConfig()); assert(serializer instanceof PojoSerializer); return serializer; } @@ -199,14 +200,14 @@ public void testTuplePojoTestEquality() { pType.getFlatFields("nestedClass.dumm2", 0, result); int[] fields = new int[1]; // see below fields[0] = result.get(0).getPosition(); - TypeComparator pojoComp = pType.createComparator( fields, new boolean[]{true}, 0); + TypeComparator pojoComp = pType.createComparator( fields, new boolean[]{true}, 0, new ExecutionConfig()); TestUserClass pojoTestRecord = new TestUserClass(0, "abc", 3d, new int[] {1,2,3}, new NestedTestUserClass(1, "haha", 4d, new int[] {5,4,3})); int pHash = pojoComp.hash(pojoTestRecord); Tuple1 tupleTest = new Tuple1("haha"); TupleTypeInfo> tType = (TupleTypeInfo>)TypeExtractor.getForObject(tupleTest); - TypeComparator> tupleComp = tType.createComparator(new int[] {0}, new boolean[] {true}, 0); + TypeComparator> tupleComp = tType.createComparator(new int[] {0}, new boolean[] {true}, 0, new ExecutionConfig()); int tHash = tupleComp.hash(tupleTest); @@ -223,12 +224,12 @@ public void testTuplePojoTestEquality() { e.printStackTrace(); Assert.fail("Keys must be compatible: "+e.getMessage()); } - TypeComparator multiPojoComp = pType.createComparator( expressKey.computeLogicalKeyPositions(), new boolean[]{true, true, true}, 0); + TypeComparator multiPojoComp = pType.createComparator( expressKey.computeLogicalKeyPositions(), new boolean[]{true, true, true}, 0, new ExecutionConfig()); int multiPojoHash = multiPojoComp.hash(pojoTestRecord); // pojo order is: dumm2 (str), dumm1 (int), dumm3 (double). - TypeComparator> multiTupleComp = multiTupleType.createComparator(fieldKey.computeLogicalKeyPositions(), new boolean[] {true, true,true}, 0); + TypeComparator> multiTupleComp = multiTupleType.createComparator(fieldKey.computeLogicalKeyPositions(), new boolean[] {true, true,true}, 0, new ExecutionConfig()); int multiTupleHash = multiTupleComp.hash(multiTupleTest); Assert.assertTrue("The hashing for tuples and pojos must be the same, so that they are mixable. Also for those with multiple key fields", multiPojoHash == multiTupleHash); diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSubclassComparatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSubclassComparatorTest.java new file mode 100644 index 0000000000000..3a03683a98307 --- /dev/null +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSubclassComparatorTest.java @@ -0,0 +1,76 @@ +/* + * 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.runtime; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.ComparatorTestBase; +import org.apache.flink.api.common.typeutils.CompositeType; +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.operators.Keys.ExpressionKeys; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.junit.Assert; + +import java.util.Arrays; + + +public class PojoSubclassComparatorTest extends ComparatorTestBase { + TypeInformation type = TypeExtractor.getForClass(PojoContainingTuple.class); + + PojoContainingTuple[] data = new PojoContainingTuple[]{ + new Subclass(1, 1L, 1L, 17L), + new Subclass(2, 2L, 2L, 42L), + new Subclass(8519, 85190L, 85190L, 117L), + new Subclass(8520, 85191L, 85191L, 93L), + }; + + @Override + protected TypeComparator createComparator(boolean ascending) { + Assert.assertTrue(type instanceof CompositeType); + CompositeType cType = (CompositeType) type; + ExpressionKeys keys = new ExpressionKeys(new String[] {"theTuple.*"}, cType); + boolean[] orders = new boolean[keys.getNumberOfKeyFields()]; + Arrays.fill(orders, ascending); + return cType.createComparator(keys.computeLogicalKeyPositions(), orders, 0, new ExecutionConfig()); + } + + @Override + protected TypeSerializer createSerializer() { + return type.createSerializer(new ExecutionConfig()); + } + + @Override + protected PojoContainingTuple[] getSortedTestData() { + return data; + } + + public static class Subclass extends PojoContainingTuple { + + public long additionalField; + + public Subclass() { + } + + public Subclass(int i, long l1, long l2, long additionalField) { + super(i, l1, l2); + this.additionalField = additionalField; + } + } +} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSubclassSerializerTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSubclassSerializerTest.java new file mode 100644 index 0000000000000..8c61a19e95d36 --- /dev/null +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSubclassSerializerTest.java @@ -0,0 +1,196 @@ +/* + * 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.runtime; + +import com.google.common.base.Objects; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.junit.Test; + +import java.util.Random; + +/** + * A test for the {@link PojoSerializer}. + */ +public class PojoSubclassSerializerTest extends SerializerTestBase { + private TypeInformation type = TypeExtractor.getForClass(TestUserClassBase.class); + + @Override + protected TypeSerializer createSerializer() { + // only register one of the three child classes, the third child class is NO POJO + ExecutionConfig conf = new ExecutionConfig(); + conf.registerPojoType(TestUserClass1.class); + TypeSerializer serializer = type.createSerializer(conf); + assert(serializer instanceof PojoSerializer); + return serializer; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return TestUserClassBase.class; + } + + @Override + protected TestUserClassBase[] getTestData() { + Random rnd = new Random(874597969123412341L); + + return new TestUserClassBase[]{ + new TestUserClass1(rnd.nextInt(), "foo", rnd.nextLong()), + new TestUserClass2(rnd.nextInt(), "bar", rnd.nextFloat()), + new TestUserClass3(rnd.nextInt(), "bar", rnd.nextFloat()) + }; + + } + + @Override + @Test + public void testInstantiate() { + // don't do anything, since the PojoSerializer with subclass will return null + } + + // User code class for testing the serializer + public static abstract class TestUserClassBase { + public int dumm1; + public String dumm2; + + + public TestUserClassBase() { + } + + public TestUserClassBase(int dumm1, String dumm2) { + this.dumm1 = dumm1; + this.dumm2 = dumm2; + } + + @Override + public int hashCode() { + return Objects.hashCode(dumm1, dumm2); + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof TestUserClassBase)) { + return false; + } + TestUserClassBase otherTUC = (TestUserClassBase) other; + if (dumm1 != otherTUC.dumm1) { + return false; + } + if (!dumm2.equals(otherTUC.dumm2)) { + return false; + } + return true; + } + } + + public static class TestUserClass1 extends TestUserClassBase { + public long dumm3; + + public TestUserClass1() { + } + + public TestUserClass1(int dumm1, String dumm2, long dumm3) { + super(dumm1, dumm2); + this.dumm3 = dumm3; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof TestUserClass1)) { + return false; + } + TestUserClass1 otherTUC = (TestUserClass1) other; + if (dumm1 != otherTUC.dumm1) { + return false; + } + if (!dumm2.equals(otherTUC.dumm2)) { + return false; + } + if (dumm3 != otherTUC.dumm3) { + return false; + } + return true; + } + } + + public static class TestUserClass2 extends TestUserClassBase { + public float dumm4; + + public TestUserClass2() { + } + + public TestUserClass2(int dumm1, String dumm2, float dumm4) { + super(dumm1, dumm2); + this.dumm4 = dumm4; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof TestUserClass2)) { + return false; + } + TestUserClass2 otherTUC = (TestUserClass2) other; + if (dumm1 != otherTUC.dumm1) { + return false; + } + if (!dumm2.equals(otherTUC.dumm2)) { + return false; + } + if (dumm4 != otherTUC.dumm4) { + return false; + } + return true; + } + } + + public static class TestUserClass3 extends TestUserClassBase { + public float dumm4; + + public TestUserClass3(int dumm1, String dumm2, float dumm4) { + super(dumm1, dumm2); + this.dumm4 = dumm4; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof TestUserClass3)) { + return false; + } + TestUserClass3 otherTUC = (TestUserClass3) other; + if (dumm1 != otherTUC.dumm1) { + return false; + } + if (!dumm2.equals(otherTUC.dumm2)) { + return false; + } + if (dumm4 != otherTUC.dumm4) { + return false; + } + return true; + } + } +} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/SubclassFromInterfaceSerializerTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/SubclassFromInterfaceSerializerTest.java new file mode 100644 index 0000000000000..efb1b9baa2227 --- /dev/null +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/SubclassFromInterfaceSerializerTest.java @@ -0,0 +1,170 @@ +/* + * 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.runtime; + +import com.google.common.base.Objects; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.junit.Test; + +import java.util.Random; + +/** + * Testing the serialization of classes which are subclasses of a class that implements an interface. + */ +public class SubclassFromInterfaceSerializerTest extends SerializerTestBase { + private TypeInformation type = TypeExtractor.getForClass(TestUserInterface.class); + + @Override + protected TypeSerializer createSerializer() { + // only register one of the two child classes + ExecutionConfig conf = new ExecutionConfig(); + conf.registerPojoType(TestUserClass2.class); + TypeSerializer serializer = type.createSerializer(conf); + assert(serializer instanceof KryoSerializer); + return serializer; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return TestUserInterface.class; + } + + @Override + protected TestUserInterface[] getTestData() { + Random rnd = new Random(874597969123412341L); + + return new TestUserInterface[]{ + new TestUserClass1(rnd.nextInt(), "foo", rnd.nextLong()), + new TestUserClass2(rnd.nextInt(), "bar", rnd.nextFloat()) + }; + + } + + @Override + @Test + public void testInstantiate() { + // don't do anything, since the PojoSerializer with subclass will return null + } + + public interface TestUserInterface {} + + // User code class for testing the serializer + public static class TestUserClassBase implements TestUserInterface { + public int dumm1; + public String dumm2; + + + public TestUserClassBase() { + } + + public TestUserClassBase(int dumm1, String dumm2) { + this.dumm1 = dumm1; + this.dumm2 = dumm2; + } + + @Override + public int hashCode() { + return Objects.hashCode(dumm1, dumm2); + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof TestUserClassBase)) { + return false; + } + TestUserClassBase otherTUC = (TestUserClassBase) other; + if (dumm1 != otherTUC.dumm1) { + return false; + } + if (!dumm2.equals(otherTUC.dumm2)) { + return false; + } + return true; + } + } + + public static class TestUserClass1 extends TestUserClassBase { + public long dumm3; + + public TestUserClass1() { + } + + public TestUserClass1(int dumm1, String dumm2, long dumm3) { + super(dumm1, dumm2); + this.dumm3 = dumm3; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof TestUserClass1)) { + return false; + } + TestUserClass1 otherTUC = (TestUserClass1) other; + if (dumm1 != otherTUC.dumm1) { + return false; + } + if (!dumm2.equals(otherTUC.dumm2)) { + return false; + } + if (dumm3 != otherTUC.dumm3) { + return false; + } + return true; + } + } + + public static class TestUserClass2 extends TestUserClassBase { + public float dumm4; + + public TestUserClass2() { + } + + public TestUserClass2(int dumm1, String dumm2, float dumm4) { + super(dumm1, dumm2); + this.dumm4 = dumm4; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof TestUserClass2)) { + return false; + } + TestUserClass2 otherTUC = (TestUserClass2) other; + if (dumm1 != otherTUC.dumm1) { + return false; + } + if (!dumm2.equals(otherTUC.dumm2)) { + return false; + } + if (dumm4 != otherTUC.dumm4) { + return false; + } + return true; + } + } +} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILD2Test.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILD2Test.java index 9f1c7b80a5667..cfc4914d1f158 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILD2Test.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILD2Test.java @@ -26,8 +26,6 @@ import org.apache.flink.api.common.typeutils.base.LongComparator; import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.typeutils.runtime.TupleComparator; -import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; import org.apache.flink.api.java.typeutils.runtime.tuple.base.TupleComparatorTestBase; diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILD3Test.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILD3Test.java index 8d13ab747548f..e5a0e6ca590b2 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILD3Test.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILD3Test.java @@ -27,8 +27,6 @@ import org.apache.flink.api.common.typeutils.base.LongComparator; import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.typeutils.runtime.TupleComparator; -import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; import org.apache.flink.api.java.typeutils.runtime.tuple.base.TupleComparatorTestBase; diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDC3Test.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDC3Test.java index 82b8b4e1d7f59..a1e6c406a3ad8 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDC3Test.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDC3Test.java @@ -27,8 +27,6 @@ import org.apache.flink.api.common.typeutils.base.LongComparator; import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.typeutils.runtime.TupleComparator; -import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; import org.apache.flink.api.java.typeutils.runtime.tuple.base.TupleComparatorTestBase; diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDX1Test.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDX1Test.java index 25de450ba7c65..b5c0c1f285ac7 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDX1Test.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDX1Test.java @@ -25,8 +25,6 @@ import org.apache.flink.api.common.typeutils.base.LongComparator; import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.typeutils.runtime.TupleComparator; -import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; import org.apache.flink.api.java.typeutils.runtime.tuple.base.TupleComparatorTestBase; diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDXC2Test.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDXC2Test.java index 2a5fc29f46cb5..793a2f45effe4 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDXC2Test.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDXC2Test.java @@ -26,8 +26,6 @@ import org.apache.flink.api.common.typeutils.base.LongComparator; import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.typeutils.runtime.TupleComparator; -import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; import org.apache.flink.api.java.typeutils.runtime.tuple.base.TupleComparatorTestBase; diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD1Test.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD1Test.java index b2218addb1166..8cdee9be7adb8 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD1Test.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD1Test.java @@ -25,8 +25,6 @@ import org.apache.flink.api.common.typeutils.base.IntSerializer; import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.typeutils.runtime.TupleComparator; -import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; import org.apache.flink.api.java.typeutils.runtime.tuple.base.TupleComparatorTestBase; diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD2Test.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD2Test.java index fe152a3a50fdc..06c292fa4c772 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD2Test.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD2Test.java @@ -26,8 +26,6 @@ import org.apache.flink.api.common.typeutils.base.StringComparator; import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.typeutils.runtime.TupleComparator; -import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; import org.apache.flink.api.java.typeutils.runtime.tuple.base.TupleComparatorTestBase; diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD3Test.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD3Test.java index b92d82599e459..d823a29f0e0af 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD3Test.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD3Test.java @@ -27,8 +27,6 @@ import org.apache.flink.api.common.typeutils.base.StringComparator; import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.typeutils.runtime.TupleComparator; -import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; import org.apache.flink.api.java.typeutils.runtime.tuple.base.TupleComparatorTestBase; diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTest.java index a701a82fa441e..96f8306ca8fc5 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTest.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.Random; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple1; @@ -209,7 +210,7 @@ public void testTuple5CustomObjects() { private void runTests(T... instances) { try { TupleTypeInfo tupleTypeInfo = (TupleTypeInfo) TypeExtractor.getForObject(instances[0]); - TypeSerializer serializer = tupleTypeInfo.createSerializer(); + TypeSerializer serializer = tupleTypeInfo.createSerializer(new ExecutionConfig()); Class tupleClass = tupleTypeInfo.getTypeClass(); diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/ValueComparatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/ValueComparatorTest.java index e710187fb2dd7..cf9874d7c8292 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/ValueComparatorTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/ValueComparatorTest.java @@ -21,8 +21,6 @@ import org.apache.flink.api.common.typeutils.ComparatorTestBase; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.typeutils.runtime.ValueComparator; -import org.apache.flink.api.java.typeutils.runtime.ValueSerializer; import org.apache.flink.types.StringValue; public class ValueComparatorTest extends ComparatorTestBase { diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorTest.java index ce1147a6c2ec6..f5a90b7cc3618 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorTest.java @@ -21,8 +21,6 @@ import org.apache.flink.api.common.typeutils.ComparatorTestBase; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.typeutils.runtime.WritableComparator; -import org.apache.flink.api.java.typeutils.runtime.WritableSerializer; public class WritableComparatorTest extends ComparatorTestBase { diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerTest.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerTest.java index 01349bf095c60..557c0e4458120 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerTest.java @@ -18,10 +18,10 @@ package org.apache.flink.api.java.typeutils.runtime; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.SerializerTestInstance; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.api.java.typeutils.WritableTypeInfo; -import org.apache.flink.api.java.typeutils.runtime.WritableSerializer; import org.junit.Test; public class WritableSerializerTest { @@ -41,7 +41,7 @@ public void testStringArrayWritable() { }; WritableTypeInfo writableTypeInfo = (WritableTypeInfo) TypeExtractor.getForObject(data[0]); - WritableSerializer writableSerializer = (WritableSerializer) writableTypeInfo.createSerializer(); + WritableSerializer writableSerializer = (WritableSerializer) writableTypeInfo.createSerializer(new ExecutionConfig()); SerializerTestInstance testInstance = new SerializerTestInstance(writableSerializer,writableTypeInfo.getTypeClass(), -1, data); diff --git a/flink-java8/src/test/java/org/apache/flink/api/java/type/lambdas/LambdaExtractionTest.java b/flink-java8/src/test/java/org/apache/flink/api/java/type/lambdas/LambdaExtractionTest.java index 49a3fe5553b00..6edd2f7078a26 100644 --- a/flink-java8/src/test/java/org/apache/flink/api/java/type/lambdas/LambdaExtractionTest.java +++ b/flink-java8/src/test/java/org/apache/flink/api/java/type/lambdas/LambdaExtractionTest.java @@ -29,7 +29,6 @@ import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; -import org.apache.flink.api.common.functions.InvalidTypesException; import org.apache.flink.api.common.functions.JoinFunction; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.MapPartitionFunction; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativePactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativePactTask.java index 4f10683b6da03..d213069fa3b2a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativePactTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/task/AbstractIterativePactTask.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.iterative.task; +import org.apache.flink.api.common.ExecutionConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.flink.api.common.aggregators.Aggregator; @@ -165,7 +166,7 @@ protected void closeLocalStrategiesAndCaches() { public DistributedRuntimeUDFContext createRuntimeContext(String taskName) { Environment env = getEnvironment(); return new IterativeRuntimeUdfContext(taskName, env.getNumberOfSubtasks(), - env.getIndexInSubtaskGroup(), getUserCodeClassLoader()); + env.getIndexInSubtaskGroup(), getUserCodeClassLoader(), getExecutionConfig()); } // -------------------------------------------------------------------------------------------- @@ -355,8 +356,8 @@ private TypeSerializer getOutputSerializer() { private class IterativeRuntimeUdfContext extends DistributedRuntimeUDFContext implements IterationRuntimeContext { - public IterativeRuntimeUdfContext(String name, int numParallelSubtasks, int subtaskIndex, ClassLoader userCodeClassLoader) { - super(name, numParallelSubtasks, subtaskIndex, userCodeClassLoader); + public IterativeRuntimeUdfContext(String name, int numParallelSubtasks, int subtaskIndex, ClassLoader userCodeClassLoader, ExecutionConfig executionConfig) { + super(name, numParallelSubtasks, subtaskIndex, userCodeClassLoader, executionConfig); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java index ff1b27287fd82..42bbe3ceb101f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java @@ -18,14 +18,21 @@ package org.apache.flink.runtime.jobgraph.tasks; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.util.InstantiationUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Abstract base class for every task class in Flink. */ public abstract class AbstractInvokable { + private static final Logger LOG = LoggerFactory.getLogger(AbstractInvokable.class); + + /** * The environment assigned to this invokable. */ @@ -110,6 +117,26 @@ public final Configuration getJobConfiguration() { return this.environment.getJobConfiguration(); } + /** + * Returns the global ExecutionConfig. + */ + public ExecutionConfig getExecutionConfig() { + try { + ExecutionConfig c = (ExecutionConfig) InstantiationUtil.readObjectFromConfig( + getJobConfiguration(), + ExecutionConfig.CONFIG_KEY, + this.getClass().getClassLoader()); + if (c != null) { + return c; + } else { + return new ExecutionConfig(); + } + } catch (Exception e) { + LOG.warn("Could not load ExecutionConfig from Environment, returning default ExecutionConfig: {}", e); + return new ExecutionConfig(); + } + } + /** * This method is called when a task is canceled either as a result of a user abort or an execution failure. It can * be overwritten to respond to shut down the user code properly. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java index 330f438c6b393..01ab533b4def9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java @@ -74,24 +74,6 @@ public class DataSourceTask extends AbstractInvokable { // cancel flag private volatile boolean taskCanceled = false; - private ExecutionConfig getExecutionConfig() { - ExecutionConfig executionConfig = new ExecutionConfig(); - try { - ExecutionConfig c = (ExecutionConfig) InstantiationUtil.readObjectFromConfig( - getJobConfiguration(), - ExecutionConfig.CONFIG_KEY, - this.getClass().getClassLoader()); - if (c != null) { - executionConfig = c; - } - } catch (IOException e) { - throw new RuntimeException("Could not load ExecutionConfig from Job Configuration: " + e); - } catch (ClassNotFoundException e) { - throw new RuntimeException("Could not load ExecutionConfig from Job Configuration: " + e); - } - return executionConfig; - } - @Override public void registerInputOutput() { initInputFormat(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java index f6d2f4a96d4a0..ffe09cb509d03 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java @@ -45,8 +45,6 @@ public interface PactTaskContext { TaskConfig getTaskConfig(); - ExecutionConfig getExecutionConfig(); - ClassLoader getUserCodeClassLoader(); MemoryManager getMemoryManager(); @@ -60,7 +58,9 @@ public interface PactTaskContext { TypeComparator getDriverComparator(int index); S getStub(); - + + ExecutionConfig getExecutionConfig(); + Collector getOutputCollector(); AbstractInvokable getOwningNepheleTask(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java index 6b6918d7eaff1..9118400774b9b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java @@ -1068,7 +1068,7 @@ protected void initOutputs() throws Exception { public DistributedRuntimeUDFContext createRuntimeContext(String taskName) { Environment env = getEnvironment(); return new DistributedRuntimeUDFContext(taskName, env.getNumberOfSubtasks(), - env.getIndexInSubtaskGroup(), getUserCodeClassLoader(), env.getCopyTask()); + env.getIndexInSubtaskGroup(), getUserCodeClassLoader(), getExecutionConfig(), env.getCopyTask()); } // -------------------------------------------------------------------------------------------- @@ -1080,23 +1080,6 @@ public TaskConfig getTaskConfig() { return this.config; } - @Override - public ExecutionConfig getExecutionConfig() { - try { - ExecutionConfig c = (ExecutionConfig) InstantiationUtil.readObjectFromConfig( - getOwningNepheleTask().getJobConfiguration(), - ExecutionConfig.CONFIG_KEY, - this.getClass().getClassLoader()); - if (c != null) { - return c; - } else { - return new ExecutionConfig(); - } - } catch (Exception e) { - throw new RuntimeException("Could not load ExecutionConfig from Job Configuration: " + e); - } - } - @Override public MemoryManager getMemoryManager() { return getEnvironment().getMemoryManager(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java index 838286e5e1695..c84d888c8aed2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/ChainedDriver.java @@ -62,7 +62,7 @@ public void setup(TaskConfig config, String taskName, Collector outputCollec } else { Environment env = parent.getEnvironment(); this.udfContext = new DistributedRuntimeUDFContext(taskName, env.getNumberOfSubtasks(), - env.getIndexInSubtaskGroup(), userCodeClassLoader, env.getCopyTask()); + env.getIndexInSubtaskGroup(), userCodeClassLoader, parent.getExecutionConfig(), env.getCopyTask()); } this.executionConfig = executionConfig; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/LargeRecordHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/LargeRecordHandler.java index 0080d63a88c50..ce630211409bc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/LargeRecordHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/LargeRecordHandler.java @@ -25,6 +25,7 @@ import java.util.ArrayList; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -102,6 +103,8 @@ public class LargeRecordHandler { private volatile boolean closed; + private final ExecutionConfig executionConfig; + // -------------------------------------------------------------------------------------------- public LargeRecordHandler(TypeSerializer serializer, TypeComparator comparator, @@ -115,7 +118,9 @@ public LargeRecordHandler(TypeSerializer serializer, TypeComparator compar this.memory = checkNotNull(memory); this.memoryOwner = checkNotNull(memoryOwner); this.maxFilehandles = maxFilehandles; - + + this.executionConfig = memoryOwner.getExecutionConfig(); + checkArgument(maxFilehandles >= 2); } @@ -374,13 +379,13 @@ public boolean hasData() { // -------------------------------------------------------------------------------------------- - private static TypeSerializer createSerializer(Object key, int pos) { + private TypeSerializer createSerializer(Object key, int pos) { if (key == null) { throw new NullKeyFieldException(pos); } try { TypeInformation info = TypeExtractor.getForObject(key); - return info.createSerializer(); + return info.createSerializer(executionConfig); } catch (Throwable t) { throw new RuntimeException("Could not create key serializer for type " + key); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/DistributedRuntimeUDFContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/DistributedRuntimeUDFContext.java index aa7aec6957723..e84106638f164 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/DistributedRuntimeUDFContext.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/DistributedRuntimeUDFContext.java @@ -23,6 +23,7 @@ import java.util.Map; import java.util.concurrent.FutureTask; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.BroadcastVariableInitializer; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.AbstractRuntimeUDFContext; @@ -40,12 +41,12 @@ public class DistributedRuntimeUDFContext extends AbstractRuntimeUDFContext { private final HashMap> broadcastVars = new HashMap>(); - public DistributedRuntimeUDFContext(String name, int numParallelSubtasks, int subtaskIndex, ClassLoader userCodeClassLoader) { - super(name, numParallelSubtasks, subtaskIndex, userCodeClassLoader); + public DistributedRuntimeUDFContext(String name, int numParallelSubtasks, int subtaskIndex, ClassLoader userCodeClassLoader, ExecutionConfig executionConfig) { + super(name, numParallelSubtasks, subtaskIndex, userCodeClassLoader, executionConfig); } - public DistributedRuntimeUDFContext(String name, int numParallelSubtasks, int subtaskIndex, ClassLoader userCodeClassLoader, Map> cpTasks) { - super(name, numParallelSubtasks, subtaskIndex, userCodeClassLoader, cpTasks); + public DistributedRuntimeUDFContext(String name, int numParallelSubtasks, int subtaskIndex, ClassLoader userCodeClassLoader, ExecutionConfig executionConfig, Map> cpTasks) { + super(name, numParallelSubtasks, subtaskIndex, userCodeClassLoader, executionConfig, cpTasks); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java index ee237360642b5..31707b1f33154 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllGroupReduceDriverTest.java @@ -21,6 +21,7 @@ import java.util.Arrays; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -52,7 +53,7 @@ public void testAllReduceDriverImmutableEmpty() { MutableObjectIterator> input = EmptyMutableObjectIterator.get(); context.setDriverStrategy(DriverStrategy.ALL_GROUP_REDUCE); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setCollector(new DiscardingOutputCollector>()); AllGroupReduceDriver, Tuple2> driver = new AllGroupReduceDriver, Tuple2>(); @@ -75,12 +76,12 @@ public void testAllReduceDriverImmutable() { List> data = DriverTestData.createReduceImmutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); - MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer()); + MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer(new ExecutionConfig())); - GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer()); + GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer(new ExecutionConfig())); context.setDriverStrategy(DriverStrategy.ALL_GROUP_REDUCE); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setCollector(result); context.setUdf(new ConcatSumReducer()); @@ -115,12 +116,12 @@ public void testAllReduceDriverMutable() { List> data = DriverTestData.createReduceMutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); - MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer()); + MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer(new ExecutionConfig())); - GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer()); + GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer(new ExecutionConfig())); context.setDriverStrategy(DriverStrategy.ALL_GROUP_REDUCE); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setCollector(result); context.setUdf(new ConcatSumMutableReducer()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllReduceDriverTest.java index bfc91682d9ad5..119ac5b24e860 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllReduceDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/AllReduceDriverTest.java @@ -22,6 +22,7 @@ import java.util.Arrays; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.RichReduceFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -52,7 +53,7 @@ public void testAllReduceDriverImmutableEmpty() { MutableObjectIterator> input = EmptyMutableObjectIterator.get(); context.setDriverStrategy(DriverStrategy.ALL_REDUCE); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setCollector(new DiscardingOutputCollector>()); AllReduceDriver> driver = new AllReduceDriver>(); @@ -76,12 +77,12 @@ public void testAllReduceDriverImmutable() { List> data = DriverTestData.createReduceImmutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); - MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer()); + MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer(new ExecutionConfig())); - GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer()); + GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer(new ExecutionConfig())); context.setDriverStrategy(DriverStrategy.ALL_REDUCE); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setCollector(result); context.setUdf(new ConcatSumFirstReducer()); @@ -108,12 +109,12 @@ public void testAllReduceDriverImmutable() { List> data = DriverTestData.createReduceImmutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); - MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer()); + MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer(new ExecutionConfig())); - GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer()); + GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer(new ExecutionConfig())); context.setDriverStrategy(DriverStrategy.ALL_REDUCE); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setCollector(result); context.setUdf(new ConcatSumSecondReducer()); @@ -150,12 +151,12 @@ public void testAllReduceDriverMutable() { List> data = DriverTestData.createReduceMutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); - MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer()); + MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer(new ExecutionConfig())); - GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer()); + GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer(new ExecutionConfig())); context.setDriverStrategy(DriverStrategy.ALL_REDUCE); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setCollector(result); context.setUdf(new ConcatSumFirstMutableReducer()); @@ -181,12 +182,12 @@ public void testAllReduceDriverMutable() { List> data = DriverTestData.createReduceMutableData(); TypeInformation> typeInfo = TypeExtractor.getForObject(data.get(0)); - MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer()); + MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer(new ExecutionConfig())); - GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer()); + GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer(new ExecutionConfig())); context.setDriverStrategy(DriverStrategy.ALL_REDUCE); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setCollector(result); context.setUdf(new ConcatSumSecondMutableReducer()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java index d249e9ac9b13a..53ca6ac75518a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/GroupReduceDriverTest.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.functions.RichGroupReduceFunction; @@ -50,12 +51,12 @@ public void testAllReduceDriverImmutableEmpty() { List> data = DriverTestData.createReduceImmutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); MutableObjectIterator> input = EmptyMutableObjectIterator.get(); - TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0); + TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0, new ExecutionConfig()); context.setDriverStrategy(DriverStrategy.SORTED_GROUP_REDUCE); - GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer()); + GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer(new ExecutionConfig())); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setComparator1(comparator); context.setCollector(result); @@ -81,13 +82,13 @@ public void testAllReduceDriverImmutable() { List> data = DriverTestData.createReduceImmutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); - MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer()); - TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0); + MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer(new ExecutionConfig())); + TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0, new ExecutionConfig()); - GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer()); + GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer(new ExecutionConfig())); context.setDriverStrategy(DriverStrategy.SORTED_GROUP_REDUCE); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setCollector(result); context.setComparator1(comparator); context.setUdf(new ConcatSumReducer()); @@ -117,13 +118,13 @@ public void testAllReduceDriverMutable() { List> data = DriverTestData.createReduceMutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); - MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer()); - TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0); + MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer(new ExecutionConfig())); + TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0, new ExecutionConfig()); - GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer()); + GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer(new ExecutionConfig())); context.setDriverStrategy(DriverStrategy.SORTED_GROUP_REDUCE); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setComparator1(comparator); context.setCollector(result); context.setUdf(new ConcatSumMutableReducer()); @@ -153,13 +154,13 @@ public void testAllReduceDriverIncorrectlyAccumulatingMutable() { List> data = DriverTestData.createReduceMutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); - MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer()); - TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0); + MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer(new ExecutionConfig())); + TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0, new ExecutionConfig()); - GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer()); + GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer(new ExecutionConfig())); context.setDriverStrategy(DriverStrategy.SORTED_GROUP_REDUCE); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setComparator1(comparator); context.setCollector(result); context.setUdf(new ConcatSumMutableAccumulatingReducer()); @@ -195,13 +196,13 @@ public void testAllReduceDriverAccumulatingImmutable() { List> data = DriverTestData.createReduceMutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); - MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer()); - TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0); + MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer(new ExecutionConfig())); + TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0, new ExecutionConfig()); - GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer()); + GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer(new ExecutionConfig())); context.setDriverStrategy(DriverStrategy.SORTED_GROUP_REDUCE); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setComparator1(comparator); context.setCollector(result); context.setUdf(new ConcatSumMutableAccumulatingReducer()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceCombineDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceCombineDriverTest.java index 44cbe169ba0ce..7494847c8af23 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceCombineDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceCombineDriverTest.java @@ -22,6 +22,7 @@ import java.util.Collections; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.functions.RichReduceFunction; @@ -55,11 +56,11 @@ public void testImmutableEmpty() { MutableObjectIterator> input = EmptyMutableObjectIterator.get(); context.setDriverStrategy(DriverStrategy.SORTED_PARTIAL_REDUCE); - TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0); + TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0, new ExecutionConfig()); - GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer()); + GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer(new ExecutionConfig())); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setComparator1(comparator); context.setCollector(result); @@ -89,13 +90,13 @@ public void testReduceDriverImmutable() { Collections.shuffle(data); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); - MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer()); - TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0); + MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer(new ExecutionConfig())); + TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0, new ExecutionConfig()); - GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer()); + GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer(new ExecutionConfig())); context.setDriverStrategy(DriverStrategy.SORTED_PARTIAL_REDUCE); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setComparator1(comparator); context.setCollector(result); context.setUdf(new ConcatSumFirstReducer()); @@ -120,13 +121,13 @@ public void testReduceDriverImmutable() { Collections.shuffle(data); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); - MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer()); - TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0); + MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer(new ExecutionConfig())); + TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0, new ExecutionConfig()); - GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer()); + GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer(new ExecutionConfig())); context.setDriverStrategy(DriverStrategy.SORTED_PARTIAL_REDUCE); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setComparator1(comparator); context.setCollector(result); context.setUdf(new ConcatSumSecondReducer()); @@ -159,13 +160,13 @@ public void testReduceDriverMutable() { List> data = DriverTestData.createReduceMutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); - MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer()); - TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0); + MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer(new ExecutionConfig())); + TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0, new ExecutionConfig()); - GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer()); + GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer(new ExecutionConfig())); context.setDriverStrategy(DriverStrategy.SORTED_PARTIAL_REDUCE); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setComparator1(comparator); context.setCollector(result); context.setUdf(new ConcatSumFirstMutableReducer()); @@ -187,13 +188,13 @@ public void testReduceDriverMutable() { List> data = DriverTestData.createReduceMutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); - MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer()); - TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0); + MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer(new ExecutionConfig())); + TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0, new ExecutionConfig()); - GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer()); + GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer(new ExecutionConfig())); context.setDriverStrategy(DriverStrategy.SORTED_PARTIAL_REDUCE); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setComparator1(comparator); context.setCollector(result); context.setUdf(new ConcatSumSecondMutableReducer()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceDriverTest.java index ae4e54cf49ac6..5d78835cd8d0f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/ReduceDriverTest.java @@ -21,6 +21,7 @@ import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.functions.RichReduceFunction; @@ -50,11 +51,11 @@ public void testReduceDriverImmutableEmpty() { TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); MutableObjectIterator> input = EmptyMutableObjectIterator.get(); context.setDriverStrategy(DriverStrategy.SORTED_REDUCE); - TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0); + TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0, new ExecutionConfig()); - GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer()); + GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer(new ExecutionConfig())); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setComparator1(comparator); context.setCollector(result); @@ -81,13 +82,13 @@ public void testReduceDriverImmutable() { List> data = DriverTestData.createReduceImmutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); - MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer()); - TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0); + MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer(new ExecutionConfig())); + TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0, new ExecutionConfig()); - GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer()); + GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer(new ExecutionConfig())); context.setDriverStrategy(DriverStrategy.SORTED_REDUCE); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setComparator1(comparator); context.setCollector(result); context.setUdf(new ConcatSumFirstReducer()); @@ -109,13 +110,13 @@ public void testReduceDriverImmutable() { List> data = DriverTestData.createReduceImmutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); - MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer()); - TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0); + MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer(new ExecutionConfig())); + TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0, new ExecutionConfig()); - GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer()); + GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer(new ExecutionConfig())); context.setDriverStrategy(DriverStrategy.SORTED_REDUCE); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setComparator1(comparator); context.setCollector(result); context.setUdf(new ConcatSumSecondReducer()); @@ -147,13 +148,13 @@ public void testReduceDriverMutable() { List> data = DriverTestData.createReduceMutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); - MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer()); - TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0); + MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer(new ExecutionConfig())); + TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0, new ExecutionConfig()); - GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer()); + GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer(new ExecutionConfig())); context.setDriverStrategy(DriverStrategy.SORTED_REDUCE); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setComparator1(comparator); context.setCollector(result); context.setUdf(new ConcatSumFirstMutableReducer()); @@ -174,13 +175,13 @@ public void testReduceDriverMutable() { List> data = DriverTestData.createReduceMutableData(); TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeExtractor.getForObject(data.get(0)); - MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer()); - TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0); + MutableObjectIterator> input = new RegularToMutableObjectIterator>(data.iterator(), typeInfo.createSerializer(new ExecutionConfig())); + TypeComparator> comparator = typeInfo.createComparator(new int[]{0}, new boolean[] {true}, 0, new ExecutionConfig()); - GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer()); + GatheringCollector> result = new GatheringCollector>(typeInfo.createSerializer(new ExecutionConfig())); context.setDriverStrategy(DriverStrategy.SORTED_REDUCE); - context.setInput1(input, typeInfo.createSerializer()); + context.setInput1(input, typeInfo.createSerializer(new ExecutionConfig())); context.setComparator1(comparator); context.setCollector(result); context.setUdf(new ConcatSumSecondMutableReducer()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortLargeRecordsITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortLargeRecordsITCase.java index af7b008cd54e0..c0083a3ed1a22 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortLargeRecordsITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortLargeRecordsITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.operators.sort; +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.TypeComparator; @@ -98,8 +99,8 @@ public void testSortWithLongRecordsOnly() { final TupleTypeInfo> typeInfo = new TupleTypeInfo>(types); - final TypeSerializer> serializer = typeInfo.createSerializer(); - final TypeComparator> comparator = typeInfo.createComparator(new int[] {0}, new boolean[]{false}, 0); + final TypeSerializer> serializer = typeInfo.createSerializer(new ExecutionConfig()); + final TypeComparator> comparator = typeInfo.createComparator(new int[] {0}, new boolean[]{false}, 0, new ExecutionConfig()); MutableObjectIterator> source = new MutableObjectIterator>() { @@ -168,8 +169,8 @@ public void testSortWithLongAndShortRecordsMixed() { final TupleTypeInfo> typeInfo = new TupleTypeInfo>(types); - final TypeSerializer> serializer = typeInfo.createSerializer(); - final TypeComparator> comparator = typeInfo.createComparator(new int[] {0}, new boolean[]{false}, 0); + final TypeSerializer> serializer = typeInfo.createSerializer(new ExecutionConfig()); + final TypeComparator> comparator = typeInfo.createComparator(new int[] {0}, new boolean[]{false}, 0, new ExecutionConfig()); MutableObjectIterator> source = new MutableObjectIterator>() { @@ -240,8 +241,8 @@ public void testSortWithShortMediumAndLargeRecords() { final TupleTypeInfo> typeInfo = new TupleTypeInfo>(types); - final TypeSerializer> serializer = typeInfo.createSerializer(); - final TypeComparator> comparator = typeInfo.createComparator(new int[] {0}, new boolean[]{false}, 0); + final TypeSerializer> serializer = typeInfo.createSerializer(new ExecutionConfig()); + final TypeComparator> comparator = typeInfo.createComparator(new int[] {0}, new boolean[]{false}, 0, new ExecutionConfig()); MutableObjectIterator> source = new MutableObjectIterator>() { @@ -323,8 +324,8 @@ public void testSortWithMediumRecordsOnly() { final TupleTypeInfo> typeInfo = new TupleTypeInfo>(types); - final TypeSerializer> serializer = typeInfo.createSerializer(); - final TypeComparator> comparator = typeInfo.createComparator(new int[] {0}, new boolean[]{false}, 0); + final TypeSerializer> serializer = typeInfo.createSerializer(new ExecutionConfig()); + final TypeComparator> comparator = typeInfo.createComparator(new int[] {0}, new boolean[]{false}, 0, new ExecutionConfig()); MutableObjectIterator> source = new MutableObjectIterator>() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerITCase.java index 3d237f770bea9..498cb61fa3801 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerITCase.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Random; +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.TypeComparator; @@ -76,9 +77,9 @@ public void testRecordHandlerCompositeKey() { final TupleTypeInfo> typeInfo = new TupleTypeInfo>(types); - final TypeSerializer> serializer = typeInfo.createSerializer(); + final TypeSerializer> serializer = typeInfo.createSerializer(new ExecutionConfig()); final TypeComparator> comparator = typeInfo.createComparator( - new int[] {2, 0}, new boolean[] {true, true}, 0); + new int[] {2, 0}, new boolean[] {true, true}, 0, new ExecutionConfig()); LargeRecordHandler> handler = new LargeRecordHandler>( serializer, comparator, ioMan, memMan, initialMemory, owner, 128); @@ -216,7 +217,7 @@ public void fileTest() { final TupleTypeInfo> typeInfo = new TupleTypeInfo>(types); - final TypeSerializer> serializer = typeInfo.createSerializer(); + final TypeSerializer> serializer = typeInfo.createSerializer(new ExecutionConfig()); channel = ioMan.createChannel(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerTest.java index d2abd6275e148..6eb736f2cca7f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerTest.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.Random; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; @@ -55,9 +56,9 @@ public void testEmptyRecordHandler() { final TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeInfoParser.>parse("Tuple2"); - final TypeSerializer> serializer = typeInfo.createSerializer(); + final TypeSerializer> serializer = typeInfo.createSerializer(new ExecutionConfig()); final TypeComparator> comparator = typeInfo.createComparator( - new int[] {0}, new boolean[] {true}, 0); + new int[] {0}, new boolean[] {true}, 0, new ExecutionConfig()); LargeRecordHandler> handler = new LargeRecordHandler>( serializer, comparator, ioMan, memMan, memory, owner, 128); @@ -109,9 +110,9 @@ public void testRecordHandlerSingleKey() { final TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeInfoParser.>parse("Tuple2"); - final TypeSerializer> serializer = typeInfo.createSerializer(); + final TypeSerializer> serializer = typeInfo.createSerializer(new ExecutionConfig()); final TypeComparator> comparator = typeInfo.createComparator( - new int[] {0}, new boolean[] {true}, 0); + new int[] {0}, new boolean[] {true}, 0, new ExecutionConfig()); LargeRecordHandler> handler = new LargeRecordHandler>( serializer, comparator, ioMan, memMan, initialMemory, owner, 128); @@ -197,9 +198,9 @@ public void testRecordHandlerCompositeKey() { final TupleTypeInfo> typeInfo = (TupleTypeInfo>) TypeInfoParser.>parse("Tuple3"); - final TypeSerializer> serializer = typeInfo.createSerializer(); + final TypeSerializer> serializer = typeInfo.createSerializer(new ExecutionConfig()); final TypeComparator> comparator = typeInfo.createComparator( - new int[] {2, 0}, new boolean[] {true, true}, 0); + new int[] {2, 0}, new boolean[] {true, true}, 0, new ExecutionConfig()); LargeRecordHandler> handler = new LargeRecordHandler>( serializer, comparator, ioMan, memMan, initialMemory, owner, 128); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MassiveStringSortingITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MassiveStringSortingITCase.java index 084da41f89d3d..b96fdf254065b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MassiveStringSortingITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MassiveStringSortingITCase.java @@ -26,6 +26,7 @@ import java.io.IOException; import java.util.Random; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.StringComparator; @@ -175,8 +176,8 @@ public void testStringTuplesSorting() { TupleTypeInfo> typeInfo = (TupleTypeInfo>) (TupleTypeInfo) TypeInfoParser.parse("Tuple2"); - TypeSerializer> serializer = typeInfo.createSerializer(); - TypeComparator> comparator = typeInfo.createComparator(new int[] { 0 }, new boolean[] { true }, 0); + TypeSerializer> serializer = typeInfo.createSerializer(new ExecutionConfig()); + TypeComparator> comparator = typeInfo.createComparator(new int[] { 0 }, new boolean[] { true }, 0, new ExecutionConfig()); reader = new BufferedReader(new FileReader(input)); MutableObjectIterator> inputIterator = new StringTupleReaderMutableObjectIterator(reader); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MassiveStringValueSortingITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MassiveStringValueSortingITCase.java index 9e925e8aa6bca..5265cdbdeab6b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MassiveStringValueSortingITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/MassiveStringValueSortingITCase.java @@ -26,6 +26,7 @@ import java.io.IOException; import java.util.Random; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; @@ -177,8 +178,8 @@ public void testStringValueTuplesSorting() { TupleTypeInfo> typeInfo = (TupleTypeInfo>) (TupleTypeInfo) TypeInfoParser.parse("Tuple2"); - TypeSerializer> serializer = typeInfo.createSerializer(); - TypeComparator> comparator = typeInfo.createComparator(new int[] { 0 }, new boolean[] { true }, 0); + TypeSerializer> serializer = typeInfo.createSerializer(new ExecutionConfig()); + TypeComparator> comparator = typeInfo.createComparator(new int[] { 0 }, new boolean[] { true }, 0, new ExecutionConfig()); reader = new BufferedReader(new FileReader(input)); MutableObjectIterator> inputIterator = new StringValueTupleReaderMutableObjectIterator(reader); diff --git a/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaAggregateOperator.java b/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaAggregateOperator.java index d352817061313..7f468fa08eda7 100644 --- a/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaAggregateOperator.java +++ b/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaAggregateOperator.java @@ -28,7 +28,7 @@ import org.apache.flink.api.common.operators.SingleInputSemanticProperties; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; -import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.aggregation.AggregationFunction; import org.apache.flink.api.java.aggregation.AggregationFunctionFactory; import org.apache.flink.api.java.aggregation.Aggregations; @@ -162,7 +162,7 @@ protected org.apache.flink.api.common.operators.base.GroupReduceOperatorBase function = new AggregatingUdf(getInputType().createSerializer(), aggFunctions, fields); + RichGroupReduceFunction function = new AggregatingUdf(getInputType(), aggFunctions, fields); String name = getName() != null ? getName() : genName.toString(); @@ -240,12 +240,14 @@ public static final class AggregatingUdf extends RichGroupRed private TupleSerializerBase serializer; - public AggregatingUdf(TypeSerializer serializer, AggregationFunction[] aggFunctions, int[] fieldPositions) { - Validate.notNull(serializer); + private TypeInformation typeInfo; + + public AggregatingUdf(TypeInformation typeInfo, AggregationFunction[] aggFunctions, int[] fieldPositions) { + Validate.notNull(typeInfo); Validate.notNull(aggFunctions); Validate.isTrue(aggFunctions.length == fieldPositions.length); - Validate.isInstanceOf(TupleSerializerBase.class, serializer, "Serializer for Scala Aggregate Operator must be a tuple serializer."); - this.serializer = (TupleSerializerBase) serializer; + Validate.isTrue(typeInfo.isTupleType(), "TypeInfo for Scala Aggregate Operator must be a tuple TypeInfo."); + this.typeInfo = typeInfo; this.aggFunctions = aggFunctions; this.fieldPositions = fieldPositions; } @@ -256,6 +258,7 @@ public void open(Configuration parameters) throws Exception { for (AggregationFunction aggFunction : aggFunctions) { aggFunction.initializeAggregate(); } + this.serializer = (TupleSerializerBase) typeInfo.createSerializer(getRuntimeContext().getExecutionConfig()); } @Override diff --git a/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaCsvInputFormat.java b/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaCsvInputFormat.java index 2ee10092b5900..79c6659b61898 100644 --- a/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaCsvInputFormat.java +++ b/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaCsvInputFormat.java @@ -22,6 +22,7 @@ import com.google.common.base.Charsets; import com.google.common.base.Preconditions; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.io.GenericCsvInputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.TupleTypeInfoBase; @@ -69,7 +70,9 @@ public ScalaCsvInputFormat(Path filePath, TypeInformation typeInfo) { throw new UnsupportedOperationException("This only works on tuple types."); } TupleTypeInfoBase tupleType = (TupleTypeInfoBase) typeInfo; - serializer = (TupleSerializerBase)tupleType.createSerializer(); + // We can use an empty config here, since we only use the serializer to create + // the top-level case class + serializer = (TupleSerializerBase) tupleType.createSerializer(new ExecutionConfig()); Class[] classes = new Class[tupleType.getArity()]; for (int i = 0; i < tupleType.getArity(); i++) { @@ -214,7 +217,7 @@ public OUT readRecord(OUT reuse, byte[] bytes, int offset, int numBytes) { return null; } } - + if (parseRecord(parsedValues, bytes, offset, numBytes)) { OUT result = serializer.createInstance(parsedValues); return result; diff --git a/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaCsvOutputFormat.java b/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaCsvOutputFormat.java index 9eb49234b0cfa..afcdc4a9d22b8 100644 --- a/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaCsvOutputFormat.java +++ b/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaCsvOutputFormat.java @@ -19,6 +19,7 @@ package org.apache.flink.api.scala.operators; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.io.FileOutputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -219,7 +220,7 @@ public String toString() { * is in fact a tuple type. */ @Override - public void setInputType(TypeInformation type) { + public void setInputType(TypeInformation type, ExecutionConfig executionConfig) { if (!type.isTupleType()) { throw new InvalidProgramException("The " + ScalaCsvOutputFormat.class.getSimpleName() + " can only be used to write tuple data sets."); diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/CrossDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/CrossDataSet.scala index 2e69efa9ce071..e57fd5bf83d64 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/CrossDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/CrossDataSet.scala @@ -18,6 +18,7 @@ package org.apache.flink.api.scala import org.apache.commons.lang3.Validate +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.functions.{RichCrossFunction, CrossFunction} import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer} import org.apache.flink.api.java.operators._ @@ -114,10 +115,10 @@ private[flink] object CrossDataSet { val returnType = new CaseClassTypeInfo[(L, R)]( classOf[(L, R)], Seq(leftInput.getType, rightInput.getType), Array("_1", "_2")) { - override def createSerializer: TypeSerializer[(L, R)] = { + override def createSerializer(executionConfig: ExecutionConfig): TypeSerializer[(L, R)] = { val fieldSerializers: Array[TypeSerializer[_]] = new Array[TypeSerializer[_]](getArity) for (i <- 0 until getArity) { - fieldSerializers(i) = types(i).createSerializer + fieldSerializers(i) = types(i).createSerializer(executionConfig) } new CaseClassSerializer[(L, R)](classOf[(L, R)], fieldSerializers) { 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 e193770fe6fc8..7ec65c6a11676 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 @@ -69,12 +69,6 @@ import scala.reflect.ClassTag * be created. */ class ExecutionEnvironment(javaEnv: JavaEnv) { - /** - * Sets the config object. - */ - def setConfig(config: ExecutionConfig): Unit = { - javaEnv.setConfig(config) - } /** * Gets the config object. @@ -411,7 +405,7 @@ class ExecutionEnvironment(javaEnv: JavaEnv) { CollectionInputFormat.checkCollection(data.asJavaCollection, typeInfo.getTypeClass) val dataSource = new DataSource[T]( javaEnv, - new CollectionInputFormat[T](data.asJavaCollection, typeInfo.createSerializer), + new CollectionInputFormat[T](data.asJavaCollection, typeInfo.createSerializer(getConfig)), typeInfo, getCallLocationName()) wrap(dataSource) diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/UnfinishedCoGroupOperation.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/UnfinishedCoGroupOperation.scala index 9f895fb4b0d3a..2b2e37259c0f7 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/UnfinishedCoGroupOperation.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/UnfinishedCoGroupOperation.scala @@ -18,6 +18,7 @@ package org.apache.flink.api.scala +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.functions.CoGroupFunction import org.apache.flink.api.common.typeutils.TypeSerializer import org.apache.flink.api.java.operators._ @@ -68,10 +69,11 @@ class UnfinishedCoGroupOperation[L: ClassTag, R: ClassTag]( val returnType = new CaseClassTypeInfo[(Array[L], Array[R])]( classOf[(Array[L], Array[R])], Seq(leftArrayType, rightArrayType), Array("_1", "_2")) { - override def createSerializer: TypeSerializer[(Array[L], Array[R])] = { + override def createSerializer( + executionConfig: ExecutionConfig): TypeSerializer[(Array[L], Array[R])] = { val fieldSerializers: Array[TypeSerializer[_]] = new Array[TypeSerializer[_]](getArity) for (i <- 0 until getArity) { - fieldSerializers(i) = types(i).createSerializer + fieldSerializers(i) = types(i).createSerializer(executionConfig) } new CaseClassSerializer[(Array[L], Array[R])]( diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala index c2e432f3ddf5a..e025192066df3 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeAnalyzer.scala @@ -272,8 +272,12 @@ private[flink] trait TypeAnalyzer[C <: Context] { this: MacroContextHolder[C] def unapply(tpe: Type): Option[Type] = tpe match { case _ if tpe <:< typeOf[BitSet] => Some(typeOf[Int]) - case _ if tpe <:< typeOf[SortedMap[_, _]] => None - case _ if tpe <:< typeOf[SortedSet[_]] => None + case _ if tpe <:< typeOf[SortedMap[_, _]] => + // handled by generic serializer + None + case _ if tpe <:< typeOf[SortedSet[_]] => + // handled by generic serializer + None case _ if tpe <:< typeOf[TraversableOnce[_]] => // val traversable = tpe.baseClasses 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 f6630a080086a..bafb7bff468f5 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 @@ -19,9 +19,10 @@ package org.apache.flink.api.scala.codegen import java.lang.reflect.{Field, Modifier} +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeinfo._ -import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.common.typeutils._ import org.apache.flink.api.java.typeutils._ import org.apache.flink.api.scala.typeutils.{CaseClassSerializer, CaseClassTypeInfo} import org.apache.flink.types.Value @@ -98,10 +99,10 @@ private[flink] trait TypeInformationGen[C <: Context] { val fieldNamesExpr = c.Expr[Seq[String]](mkSeq(fieldNames)) reify { new CaseClassTypeInfo[T](tpeClazz.splice, fieldsExpr.splice, fieldNamesExpr.splice) { - override def createSerializer: TypeSerializer[T] = { + override def createSerializer(executionConfig: ExecutionConfig): TypeSerializer[T] = { val fieldSerializers: Array[TypeSerializer[_]] = new Array[TypeSerializer[_]](getArity) for (i <- 0 until getArity) { - fieldSerializers(i) = types(i).createSerializer + fieldSerializers(i) = types(i).createSerializer(executionConfig) } new CaseClassSerializer[T](tupleType, fieldSerializers) { @@ -170,12 +171,13 @@ private[flink] trait TypeInformationGen[C <: Context] { import scala.collection.generic.CanBuildFrom import org.apache.flink.api.scala.typeutils.TraversableTypeInfo import org.apache.flink.api.scala.typeutils.TraversableSerializer + import org.apache.flink.api.common.ExecutionConfig val elementTpe = $elementTypeInfo new TraversableTypeInfo($collectionClass, elementTpe) { - def createSerializer() = { + def createSerializer(executionConfig: ExecutionConfig) = { new TraversableSerializer[${desc.tpe}, ${desc.elem.tpe}]( - elementTpe.createSerializer) { + elementTpe.createSerializer(executionConfig)) { def getCbf = implicitly[CanBuildFrom[${desc.tpe}, ${desc.elem.tpe}, ${desc.tpe}]] } } diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala index bff26cb56c4fe..420496ff85a79 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/joinDataSet.scala @@ -18,7 +18,7 @@ package org.apache.flink.api.scala import org.apache.commons.lang3.Validate -import org.apache.flink.api.common.InvalidProgramException +import org.apache.flink.api.common.{ExecutionConfig, InvalidProgramException} import org.apache.flink.api.common.functions.{JoinFunction, RichFlatJoinFunction, FlatJoinFunction} import org.apache.flink.api.common.typeutils.TypeSerializer import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint @@ -236,10 +236,10 @@ class UnfinishedJoinOperation[L, R]( val returnType = new CaseClassTypeInfo[(L, R)]( classOf[(L, R)], Seq(leftSet.getType, rightSet.getType), Array("_1", "_2")) { - override def createSerializer: TypeSerializer[(L, R)] = { + override def createSerializer(executionConfig: ExecutionConfig): TypeSerializer[(L, R)] = { val fieldSerializers: Array[TypeSerializer[_]] = new Array[TypeSerializer[_]](getArity) for (i <- 0 until getArity) { - fieldSerializers(i) = types(i).createSerializer + fieldSerializers(i) = types(i).createSerializer(executionConfig) } new CaseClassSerializer[(L, R)](classOf[(L, R)], fieldSerializers) { 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 b407332cc0a26..1d627ab35f3d6 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 @@ -20,14 +20,15 @@ package org.apache.flink.api.scala.typeutils 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._ import org.apache.flink.api.java.operators.Keys.ExpressionKeys -import org.apache.flink.api.java.typeutils.PojoTypeInfo.NamedFlatFieldDescriptor -import org.apache.flink.api.java.typeutils.TupleTypeInfoBase -import org.apache.flink.api.common.typeutils.{CompositeType, TypeComparator} +import org.apache.flink.api.java.typeutils.{TupleTypeInfoBase, PojoTypeInfo} +import PojoTypeInfo.NamedFlatFieldDescriptor /** * TypeInformation for Case Classes. Creation and access is different from @@ -74,13 +75,13 @@ abstract class CaseClassTypeInfo[T <: Product]( comparatorHelperIndex += 1 } - override protected def getNewComparator: TypeComparator[T] = { + 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) + val fieldSerializers = types.take(maxKey + 1).map(_.createSerializer(executionConfig)) new CaseClassComparator[T](finalLogicalKeyFields, finalComparators, fieldSerializers.toArray) } 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 ce19a658216fb..e2d3388045c57 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 @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.typeutils +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.common.typeutils.TypeSerializer @@ -36,12 +37,18 @@ class EitherTypeInfo[A, B, T <: Either[A, B]]( override def getArity: Int = 1 override def getTypeClass = clazz - def createSerializer(): TypeSerializer[T] = { - val leftSerializer = - if (leftTypeInfo != null) leftTypeInfo.createSerializer() else new NothingSerializer + def createSerializer(executionConfig: ExecutionConfig): TypeSerializer[T] = { + val leftSerializer = if (leftTypeInfo != null) { + leftTypeInfo.createSerializer(executionConfig) + } else { + new NothingSerializer + } - val rightSerializer = - if (rightTypeInfo != null) rightTypeInfo.createSerializer() else new NothingSerializer + val rightSerializer = if (rightTypeInfo != null) { + rightTypeInfo.createSerializer(executionConfig) + } else { + new NothingSerializer + } new EitherSerializer(leftSerializer, rightSerializer) } 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 97fe7a7ed2149..4d39f7feaa186 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 @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.typeutils +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.common.typeutils.TypeSerializer @@ -33,12 +34,13 @@ class OptionTypeInfo[A, T <: Option[A]](elemTypeInfo: TypeInformation[A]) override def getArity: Int = 1 override def getTypeClass = classOf[Option[_]].asInstanceOf[Class[T]] - def createSerializer(): TypeSerializer[T] = { + def createSerializer(executionConfig: ExecutionConfig): TypeSerializer[T] = { if (elemTypeInfo == null) { // this happens when the type of a DataSet is None, i.e. DataSet[None] new OptionSerializer(new NothingSerializer).asInstanceOf[TypeSerializer[T]] } else { - new OptionSerializer(elemTypeInfo.createSerializer()).asInstanceOf[TypeSerializer[T]] + new OptionSerializer(elemTypeInfo.createSerializer(executionConfig)) + .asInstanceOf[TypeSerializer[T]] } } 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 06e40d838d1a5..96dc96d193fa0 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 @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.typeutils +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.common.typeutils.TypeSerializer @@ -37,7 +38,7 @@ abstract class TraversableTypeInfo[T <: TraversableOnce[E], E]( override def getArity: Int = 1 override def getTypeClass: Class[T] = clazz - def createSerializer(): TypeSerializer[T] + def createSerializer(executionConfig: ExecutionConfig): TypeSerializer[T] override def equals(other: Any): Boolean = { if (other.isInstanceOf[TraversableTypeInfo[_, _]]) { 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 1f565f205b88f..5db129074602f 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 @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.typeutils +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeutils.TypeSerializer import org.apache.flink.api.java.typeutils.runtime.KryoSerializer import org.apache.flink.core.memory.{DataInputView, DataOutputView} @@ -26,12 +27,12 @@ import scala.util.{Success, Try, Failure} /** * Serializer for [[scala.util.Try]]. */ -class TrySerializer[A](val elemSerializer: TypeSerializer[A]) +class TrySerializer[A](val elemSerializer: TypeSerializer[A], executionConfig: ExecutionConfig) extends TypeSerializer[Try[A]] { override def duplicate: TrySerializer[A] = this - val throwableSerializer = new KryoSerializer[Throwable](classOf[Throwable]) + val throwableSerializer = new KryoSerializer[Throwable](classOf[Throwable], executionConfig) override def createInstance: Try[A] = { Failure(new RuntimeException("Empty Failure")) 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 b630cd47ddd68..d1112609a2d41 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 @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.typeutils +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.common.typeutils.TypeSerializer import org.apache.flink.api.java.typeutils.runtime.KryoSerializer @@ -36,12 +37,13 @@ class TryTypeInfo[A, T <: Try[A]](elemTypeInfo: TypeInformation[A]) override def getArity: Int = 1 override def getTypeClass = classOf[Try[_]].asInstanceOf[Class[T]] - def createSerializer(): TypeSerializer[T] = { + def createSerializer(executionConfig: ExecutionConfig): TypeSerializer[T] = { if (elemTypeInfo == null) { // this happens when the type of a DataSet is None, i.e. DataSet[Failure] - new TrySerializer(new NothingSerializer).asInstanceOf[TypeSerializer[T]] + new TrySerializer(new NothingSerializer, executionConfig).asInstanceOf[TypeSerializer[T]] } else { - new TrySerializer(elemTypeInfo.createSerializer()).asInstanceOf[TypeSerializer[T]] + new TrySerializer(elemTypeInfo.createSerializer(executionConfig), executionConfig) + .asInstanceOf[TypeSerializer[T]] } } diff --git a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java index 376fd70decbe1..f00859ff11965 100644 --- a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java +++ b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java @@ -25,6 +25,7 @@ import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.api.java.typeutils.TupleTypeInfo; @@ -103,7 +104,8 @@ public void open(Configuration parameters) throws Exception { this.reporter = new HadoopDummyReporter(); Class inKeyClass = (Class) TypeExtractor.getParameterType(Reducer.class, reducer.getClass(), 0); - this.valueIterator = new HadoopTupleUnwrappingIterator(inKeyClass); + TypeSerializer keySerializer = TypeExtractor.getForClass((Class) inKeyClass).createSerializer(getRuntimeContext().getExecutionConfig()); + this.valueIterator = new HadoopTupleUnwrappingIterator(keySerializer); this.combineCollector = new HadoopOutputCollector(); this.reduceCollector = new HadoopOutputCollector(); } diff --git a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java index fd2c4932dd609..69434213149d6 100644 --- a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java +++ b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java @@ -25,6 +25,7 @@ import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.api.java.typeutils.TupleTypeInfo; @@ -92,7 +93,8 @@ public void open(Configuration parameters) throws Exception { this.reporter = new HadoopDummyReporter(); this.reduceCollector = new HadoopOutputCollector(); Class inKeyClass = (Class) TypeExtractor.getParameterType(Reducer.class, reducer.getClass(), 0); - this.valueIterator = new HadoopTupleUnwrappingIterator(inKeyClass); + TypeSerializer keySerializer = TypeExtractor.getForClass(inKeyClass).createSerializer(getRuntimeContext().getExecutionConfig()); + this.valueIterator = new HadoopTupleUnwrappingIterator(keySerializer); } @Override diff --git a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIterator.java b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIterator.java index 5ecac2e1a341f..a063183bd14c7 100644 --- a/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIterator.java +++ b/flink-staging/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIterator.java @@ -23,7 +23,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.operators.translation.TupleUnwrappingIterator; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.typeutils.TypeExtractor; /** * Wraps a Flink Tuple2 (key-value-pair) iterator into an iterator over the second (value) field. @@ -42,8 +41,8 @@ public class HadoopTupleUnwrappingIterator private KEY curKey = null; private VALUE firstValue = null; - public HadoopTupleUnwrappingIterator(Class keyClass) { - this.keySerializer = TypeExtractor.getForClass((Class) keyClass).createSerializer(); + public HadoopTupleUnwrappingIterator(TypeSerializer keySerializer) { + this.keySerializer = keySerializer; } /** diff --git a/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIteratorTest.java b/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIteratorTest.java index 2592b880da986..524318cdc34fb 100644 --- a/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIteratorTest.java +++ b/flink-staging/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIteratorTest.java @@ -22,6 +22,7 @@ import java.util.NoSuchElementException; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.runtime.WritableSerializer; import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopTupleUnwrappingIterator; import org.apache.hadoop.io.IntWritable; import org.junit.Assert; @@ -33,7 +34,8 @@ public class HadoopTupleUnwrappingIteratorTest { public void testValueIterator() { HadoopTupleUnwrappingIterator valIt = - new HadoopTupleUnwrappingIterator(IntWritable.class); + new HadoopTupleUnwrappingIterator(new WritableSerializer + (IntWritable.class)); // many values diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamGraph.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamGraph.java index b5e43afd836e0..4800c96d6a8d8 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamGraph.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/StreamGraph.java @@ -29,6 +29,7 @@ import java.util.Map; import java.util.Set; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.compiler.plan.StreamingPlan; @@ -87,7 +88,11 @@ public class StreamGraph extends StreamingPlan { private Set sources; - public StreamGraph() { + private ExecutionConfig executionConfig; + + public StreamGraph(ExecutionConfig executionConfig) { + + this.executionConfig = executionConfig; initGraph(); @@ -145,9 +150,9 @@ public void addStreamVertex(String vertexName, addVertex(vertexName, StreamVertex.class, invokableObject, operatorName, parallelism); StreamRecordSerializer inSerializer = inTypeInfo != null ? new StreamRecordSerializer( - inTypeInfo) : null; + inTypeInfo, executionConfig) : null; StreamRecordSerializer outSerializer = outTypeInfo != null ? new StreamRecordSerializer( - outTypeInfo) : null; + outTypeInfo, executionConfig) : null; addTypeSerializers(vertexName, inSerializer, null, outSerializer, null); @@ -251,9 +256,9 @@ public void addCoTask(String vertexName, addVertex(vertexName, CoStreamVertex.class, taskInvokableObject, operatorName, parallelism); - addTypeSerializers(vertexName, new StreamRecordSerializer(in1TypeInfo), - new StreamRecordSerializer(in2TypeInfo), new StreamRecordSerializer( - outTypeInfo), null); + addTypeSerializers(vertexName, new StreamRecordSerializer(in1TypeInfo, executionConfig), + new StreamRecordSerializer(in2TypeInfo, executionConfig), new StreamRecordSerializer( + outTypeInfo, executionConfig), null); if (LOG.isDebugEnabled()) { LOG.debug("CO-TASK: {}", vertexName); @@ -399,7 +404,7 @@ public void setInvokable(String vertexName, StreamInvokable i } public void setOutType(String id, TypeInformation outType) { - StreamRecordSerializer serializer = new StreamRecordSerializer(outType); + StreamRecordSerializer serializer = new StreamRecordSerializer(outType, executionConfig); typeSerializersOut1.put(id, serializer); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java index aed449660482f..eff6026ddcf3f 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java @@ -319,7 +319,7 @@ public GroupedDataStream groupBy(KeySelector keySelector) { private GroupedDataStream groupBy(Keys keys) { return new GroupedDataStream(this, clean(KeySelectorUtil.getSelectorForKeys(keys, - getType()))); + getType(), environment.getConfig()))); } /** diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporaloperator/StreamJoinOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporaloperator/StreamJoinOperator.java index d2b203234172d..6c53b6b16fae8 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporaloperator/StreamJoinOperator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/temporaloperator/StreamJoinOperator.java @@ -71,7 +71,7 @@ private JoinWindow(StreamJoinOperator operator) { */ public JoinPredicate where(int... fields) { return new JoinPredicate(op, KeySelectorUtil.getSelectorForKeys( - new Keys.ExpressionKeys(fields, type1), type1)); + new Keys.ExpressionKeys(fields, type1), type1, op.input1.getExecutionEnvironment().getConfig())); } /** @@ -88,7 +88,7 @@ public JoinPredicate where(int... fields) { */ public JoinPredicate where(String... fields) { return new JoinPredicate(op, KeySelectorUtil.getSelectorForKeys( - new Keys.ExpressionKeys(fields, type1), type1)); + new Keys.ExpressionKeys(fields, type1), type1, op.input1.getExecutionEnvironment().getConfig())); } /** @@ -158,7 +158,7 @@ private JoinPredicate(StreamJoinOperator operator, KeySelector ke */ public JoinedStream equalTo(int... fields) { keys2 = KeySelectorUtil.getSelectorForKeys(new Keys.ExpressionKeys(fields, type2), - type2); + type2, op.input1.getExecutionEnvironment().getConfig()); return createJoinOperator(); } @@ -177,7 +177,7 @@ public JoinedStream equalTo(int... fields) { */ public JoinedStream equalTo(String... fields) { this.keys2 = KeySelectorUtil.getSelectorForKeys(new Keys.ExpressionKeys(fields, - type2), type2); + type2), type2, op.input1.getExecutionEnvironment().getConfig()); return createJoinOperator(); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java index 45c14c1be3767..65dde79756197 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java @@ -22,6 +22,7 @@ import java.util.Collection; import java.util.List; +import com.esotericsoftware.kryo.Serializer; import org.apache.commons.lang3.Validate; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.io.InputFormat; @@ -30,6 +31,7 @@ import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.io.TextInputFormat; import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.typeutils.PojoTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.client.program.Client; import org.apache.flink.client.program.Client.OptimizerPlanEnvironment; @@ -78,15 +80,7 @@ public abstract class StreamExecutionEnvironment { * Constructor for creating StreamExecutionEnvironment */ protected StreamExecutionEnvironment() { - streamGraph = new StreamGraph(); - } - - /** - * Sets the config object. - */ - public void setConfig(ExecutionConfig config) { - Validate.notNull(config); - this.config = config; + streamGraph = new StreamGraph(config); } /** @@ -180,6 +174,57 @@ public static void setDefaultLocalParallelism(int degreeOfParallelism) { defaultLocalDop = degreeOfParallelism; } + // -------------------------------------------------------------------------------------------- + // Registry for types and serializers + // -------------------------------------------------------------------------------------------- + + /** + * Registers the given Serializer as a default serializer for the given type at the + * {@link org.apache.flink.api.java.typeutils.runtime.KryoSerializer}. + * + * Note that the serializer instance must be serializable (as defined by java.io.Serializable), + * because it may be distributed to the worker nodes by java serialization. + * + * @param type The class of the types serialized with the given serializer. + * @param serializer The serializer to use. + */ + public void registerKryoSerializer(Class type, Serializer serializer) { + config.registerKryoSerializer(type, serializer); + } + + /** + * Registers the given Serializer via its class as a serializer for the given type at the + * {@link org.apache.flink.api.java.typeutils.runtime.KryoSerializer}. + * + * @param type The class of the types serialized with the given serializer. + * @param serializerClass The class of the serializer to use. + */ + public void registerKryoSerializer(Class type, Class> serializerClass) { + config.registerKryoSerializer(type, serializerClass); + } + + /** + * Registers the given type with the serialization stack. If the type is eventually + * serialized as a POJO, then the type is registered with the POJO serializer. If the + * type ends up being serialized with Kryo, then it will be registered at Kryo to make + * sure that only tags are written. + * + * @param type The class of the type to register. + */ + public void registerType(Class type) { + if (type == null) { + throw new NullPointerException("Cannot register null type class."); + } + + TypeInformation typeInfo = TypeExtractor.createTypeInfo(type); + + if (typeInfo instanceof PojoTypeInfo) { + config.registerPojoType(type); + } else { + config.registerKryoType(type); + } + } + // -------------------------------------------------------------------------------------------- // Data stream creations // -------------------------------------------------------------------------------------------- diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/AggregationFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/AggregationFunction.java index 1c273d37d74ef..3704e3b6de1be 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/AggregationFunction.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/AggregationFunction.java @@ -17,9 +17,9 @@ package org.apache.flink.streaming.api.function.aggregation; -import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.common.functions.RichReduceFunction; -public abstract class AggregationFunction implements ReduceFunction { +public abstract class AggregationFunction extends RichReduceFunction { private static final long serialVersionUID = 1L; public int position; diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/ComparableAggregator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/ComparableAggregator.java index 226c45a7df1a4..7f7cf0b257d65 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/ComparableAggregator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/ComparableAggregator.java @@ -193,7 +193,7 @@ public PojoComparableAggregator(String field, TypeInformation typeInfo, if (cType instanceof PojoTypeInfo) { pojoComparator = (PojoComparator) cType.createComparator( - new int[] { logicalKeyPosition }, new boolean[] { false }, 0); + new int[] { logicalKeyPosition }, new boolean[] { false }, 0, getRuntimeContext().getExecutionConfig()); } else { throw new IllegalArgumentException( "Key expressions are only supported on POJO types. " diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/SumAggregator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/SumAggregator.java index 142028b027430..74e4597b17e38 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/SumAggregator.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/aggregation/SumAggregator.java @@ -146,7 +146,7 @@ public PojoSumAggregator(String field, TypeInformation type) { if (cType instanceof PojoTypeInfo) { comparator = (PojoComparator) cType.createComparator( - new int[] { logicalKeyPosition }, new boolean[] { false }, 0); + new int[] { logicalKeyPosition }, new boolean[] { false }, 0, getRuntimeContext().getExecutionConfig()); } else { throw new IllegalArgumentException( "Key expressions are only supported on POJO types. " diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java index a5ef3a73785ab..6d1441afc53ad 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/function/source/FileSourceFunction.java @@ -45,8 +45,8 @@ public FileSourceFunction(InputFormat format, TypeInformation this.serializerFactory = createSerializer(typeInfo); } - private static TypeSerializerFactory createSerializer(TypeInformation typeInfo) { - TypeSerializer serializer = typeInfo.createSerializer(); + private TypeSerializerFactory createSerializer(TypeInformation typeInfo) { + TypeSerializer serializer = typeInfo.createSerializer(getRuntimeContext().getExecutionConfig()); return new RuntimeSerializerFactory(serializer, typeInfo.getTypeClass()); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamInvokable.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamInvokable.java index 6cee5f2dd758b..793c9525b0b98 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamInvokable.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/StreamInvokable.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.io.Serializable; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.FunctionUtils; @@ -48,6 +49,8 @@ public abstract class StreamInvokable implements Serializable { protected StreamTaskContext taskContext; + protected ExecutionConfig executionConfig = null; + protected MutableObjectIterator> recordIterator; protected StreamRecordSerializer inSerializer; protected TypeSerializer objectSerializer; @@ -67,11 +70,12 @@ public StreamInvokable(Function userFunction) { /** * Initializes the {@link StreamInvokable} for input and output handling - * + * * @param taskContext * StreamTaskContext representing the vertex + * @param executionConfig */ - public void setup(StreamTaskContext taskContext) { + public void setup(StreamTaskContext taskContext, ExecutionConfig executionConfig) { this.collector = taskContext.getOutputCollector(); this.recordIterator = taskContext.getInput(0); this.inSerializer = taskContext.getInputSerializer(0); @@ -80,6 +84,7 @@ public void setup(StreamTaskContext taskContext) { this.objectSerializer = inSerializer.getObjectSerializer(); } this.taskContext = taskContext; + this.executionConfig = executionConfig; } /** diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowInvokable.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowInvokable.java index 997463c8c219b..df2edd2723040 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowInvokable.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/GroupedWindowInvokable.java @@ -354,7 +354,7 @@ private WindowInvokable makeNewGroup(StreamRecord element) throws E clonedDistributedEvictionPolicies); } - groupInvokable.setup(taskContext); + groupInvokable.setup(taskContext, executionConfig); groupInvokable.open(this.parameters); windowingGroups.put(keySelector.getKey(element.getObject()), groupInvokable); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/ProjectInvokable.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/ProjectInvokable.java index c9d9e5aa1e836..69c7cee00cf3e 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/ProjectInvokable.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/ProjectInvokable.java @@ -35,7 +35,7 @@ public ProjectInvokable(int[] fields, TypeInformation outTypeInformation) { super(null); this.fields = fields; this.numFields = this.fields.length; - this.outTypeSerializer = outTypeInformation.createSerializer(); + this.outTypeSerializer = outTypeInformation.createSerializer(executionConfig); } @Override diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java index 604873e641348..9f98db3350638 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/invokable/operator/co/CoInvokable.java @@ -17,6 +17,7 @@ package org.apache.flink.streaming.api.invokable.operator.co; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.streaming.api.invokable.StreamInvokable; @@ -46,7 +47,7 @@ public CoInvokable(Function userFunction) { protected TypeSerializer serializer2; @Override - public void setup(StreamTaskContext taskContext) { + public void setup(StreamTaskContext taskContext, ExecutionConfig executionConfig) { this.collector = taskContext.getOutputCollector(); this.recordIterator = taskContext.getCoReader(); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java index 98f12ec65f0f1..cd68937a3ae47 100755 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamrecord/StreamRecordSerializer.java @@ -20,6 +20,7 @@ import java.io.IOException; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.DataInputView; @@ -32,8 +33,8 @@ public final class StreamRecordSerializer extends TypeSerializer typeSerializer; private final boolean isTuple; - public StreamRecordSerializer(TypeInformation typeInfo) { - this.typeSerializer = typeInfo.createSerializer(); + public StreamRecordSerializer(TypeInformation typeInfo, ExecutionConfig executionConfig) { + this.typeSerializer = typeInfo.createSerializer(executionConfig); this.isTuple = typeInfo.isTupleType(); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java index 2b650be97f9be..83cdcd12c8b86 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/CoStreamVertex.java @@ -68,7 +68,7 @@ public void setInputsOutputs() { @Override protected void setInvokable() { userInvokable = configuration.getUserInvokable(userClassLoader); - userInvokable.setup(this); + userInvokable.setup(this, getExecutionConfig()); } protected void setConfigInputs() throws StreamVertexException { diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamVertex.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamVertex.java index 994b1faa6146c..024e415be2fcf 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamVertex.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamVertex.java @@ -97,7 +97,7 @@ public void setInputsOutputs() { protected void setInvokable() { userInvokable = configuration.getUserInvokable(userClassLoader); - userInvokable.setup(this); + userInvokable.setup(this, getExecutionConfig()); } public String getName() { @@ -111,7 +111,7 @@ public int getInstanceID() { public StreamingRuntimeContext createRuntimeContext(String taskName, Map> states) { Environment env = getEnvironment(); - return new StreamingRuntimeContext(taskName, env, getUserCodeClassLoader(), states); + return new StreamingRuntimeContext(taskName, env, getUserCodeClassLoader(), getExecutionConfig(), states); } @Override diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamingRuntimeContext.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamingRuntimeContext.java index a1a64e2dc61b4..0daf3c2bc916e 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamingRuntimeContext.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/streamvertex/StreamingRuntimeContext.java @@ -20,6 +20,7 @@ import java.util.Map; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.RuntimeUDFContext; import org.apache.flink.configuration.Configuration; @@ -38,9 +39,9 @@ public class StreamingRuntimeContext extends RuntimeUDFContext { private final Map> operatorStates; public StreamingRuntimeContext(String name, Environment env, ClassLoader userCodeClassLoader, - Map> operatorStates) { + ExecutionConfig executionConfig, Map> operatorStates) { super(name, env.getNumberOfSubtasks(), env.getIndexInSubtaskGroup(), - userCodeClassLoader, env.getCopyTask()); + userCodeClassLoader, executionConfig, env.getCopyTask()); this.env = env; this.operatorStates = operatorStates; } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java index 08afd0dbcbd5c..77467b5a0703e 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java @@ -19,6 +19,7 @@ import java.lang.reflect.Array; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.CompositeType; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -60,12 +61,13 @@ public class KeySelectorUtil { Tuple20.class, Tuple21.class, Tuple22.class, Tuple23.class, Tuple24.class, Tuple25.class }; - public static KeySelector getSelectorForKeys(Keys keys, TypeInformation typeInfo) { + public static KeySelector getSelectorForKeys(Keys keys, TypeInformation typeInfo, ExecutionConfig executionConfig) { int[] logicalKeyPositions = keys.computeLogicalKeyPositions(); int keyLength = logicalKeyPositions.length; boolean[] orders = new boolean[keyLength]; + // TODO: Fix using KeySelector everywhere TypeComparator comparator = ((CompositeType) typeInfo).createComparator( - logicalKeyPositions, orders, 0); + logicalKeyPositions, orders, 0, executionConfig); return new ComparableKeySelector(comparator, keyLength); } diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java index 49cd49763b2e9..115f6145f7434 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java @@ -23,6 +23,7 @@ import java.util.ArrayList; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; @@ -118,7 +119,7 @@ public void groupSumIntegerTest() { KeySelector, ?> keySelector = KeySelectorUtil.getSelectorForKeys( new Keys.ExpressionKeys>(new int[] { 0 }, typeInfo), - typeInfo); + typeInfo, new ExecutionConfig()); List> groupedSumList = MockContext.createAndExecute( new GroupedReduceInvokable>(sumFunction, keySelector), diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockCoContext.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockCoContext.java index ea94f98183875..a6560ae842fbd 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockCoContext.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockCoContext.java @@ -23,6 +23,7 @@ import java.util.Iterator; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.typeutils.TypeExtractor; @@ -57,9 +58,9 @@ public MockCoContext(Collection input1, Collection input2) { this.inputIterator2 = input2.iterator(); TypeInformation inTypeInfo1 = TypeExtractor.getForObject(input1.iterator().next()); - inDeserializer1 = new StreamRecordSerializer(inTypeInfo1); + inDeserializer1 = new StreamRecordSerializer(inTypeInfo1, new ExecutionConfig()); TypeInformation inTypeInfo2 = TypeExtractor.getForObject(input2.iterator().next()); - inDeserializer2 = new StreamRecordSerializer(inTypeInfo2); + inDeserializer2 = new StreamRecordSerializer(inTypeInfo2, new ExecutionConfig()); mockIterator = new MockCoReaderIterator(inDeserializer1, inDeserializer2); @@ -154,7 +155,7 @@ public CoReaderIterator, StreamRecord> getIterator() { public static List createAndExecute(CoInvokable invokable, List input1, List input2) { MockCoContext mockContext = new MockCoContext(input1, input2); - invokable.setup(mockContext); + invokable.setup(mockContext, new ExecutionConfig()); try { invokable.open(null); diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java index 5537052d82f83..81467dc570ad7 100644 --- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java +++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java @@ -23,6 +23,7 @@ import java.util.Iterator; import java.util.List; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.streaming.api.StreamConfig; @@ -49,7 +50,7 @@ public MockContext(Collection inputs) { } TypeInformation inTypeInfo = TypeExtractor.getForObject(inputs.iterator().next()); - inDeserializer = new StreamRecordSerializer(inTypeInfo); + inDeserializer = new StreamRecordSerializer(inTypeInfo, new ExecutionConfig()); iterator = new MockInputIterator(); outputs = new ArrayList(); @@ -104,7 +105,7 @@ public MutableObjectIterator> getIterator() { public static List createAndExecute(StreamInvokable invokable, List inputs) { MockContext mockContext = new MockContext(inputs); - invokable.setup(mockContext); + invokable.setup(mockContext, new ExecutionConfig()); try { invokable.open(null); invokable.invoke(); diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala index 177a9ee93ee0c..23495a5dc4659 100644 --- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala @@ -18,6 +18,7 @@ package org.apache.flink.streaming.api.scala +import org.apache.flink.api.java.typeutils.TupleTypeInfoBase import org.apache.flink.streaming.api.datastream.{DataStream => JavaStream, SingleOutputStreamOperator, GroupedDataStream} import scala.reflect.ClassTag @@ -41,7 +42,6 @@ import org.apache.flink.streaming.api.collector.OutputSelector import scala.collection.JavaConversions._ import java.util.HashMap import org.apache.flink.streaming.api.function.aggregation.SumFunction -import org.apache.flink.api.java.typeutils.TupleTypeInfoBase import org.apache.flink.streaming.api.function.aggregation.AggregationFunction import org.apache.flink.streaming.api.function.aggregation.AggregationFunction.AggregationType import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo @@ -290,7 +290,9 @@ class DataStream[T](javaStream: JavaStream[T]) { val jStream = javaStream.asInstanceOf[JavaStream[Product]] val outType = jStream.getType().asInstanceOf[TupleTypeInfoBase[_]] - val agg = new ScalaStreamingAggregator[Product](jStream.getType().createSerializer(), position) + val agg = new ScalaStreamingAggregator[Product]( + jStream.getType().createSerializer(javaStream.getExecutionEnvironment.getConfig), + position) val reducer = aggregationType match { case AggregationType.SUM => new agg.Sum(SumFunction.getForClass(outType.getTypeAt(position). diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamCrossOperator.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamCrossOperator.scala index a408ec0e0ac7e..06271fdbe74f4 100644 --- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamCrossOperator.scala +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamCrossOperator.scala @@ -18,6 +18,8 @@ package org.apache.flink.streaming.api.scala +import org.apache.flink.api.common.ExecutionConfig + import scala.reflect.ClassTag import org.apache.commons.lang.Validate import org.apache.flink.api.common.functions.CrossFunction @@ -44,10 +46,10 @@ class StreamCrossOperator[I1, I2](i1: JavaStream[I1], i2: JavaStream[I2]) extend classOf[(I1, I2)], Seq(input1.getType, input2.getType), Array("_1", "_2")) { - override def createSerializer: TypeSerializer[(I1, I2)] = { + override def createSerializer(executionConfig: ExecutionConfig): TypeSerializer[(I1, I2)] = { val fieldSerializers: Array[TypeSerializer[_]] = new Array[TypeSerializer[_]](getArity) for (i <- 0 until getArity) { - fieldSerializers(i) = types(i).createSerializer + fieldSerializers(i) = types(i).createSerializer(executionConfig) } new CaseClassSerializer[(I1, I2)](classOf[(I1, I2)], fieldSerializers) { diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala index 394673ca0a64f..bc9b4228b2205 100644 --- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala @@ -18,6 +18,9 @@ package org.apache.flink.streaming.api.scala +import com.esotericsoftware.kryo.Serializer +import org.apache.flink.api.java.typeutils.runtime.KryoSerializer + import scala.reflect.ClassTag import org.apache.commons.lang.Validate import org.apache.flink.api.common.typeinfo.TypeInformation @@ -72,6 +75,33 @@ class StreamExecutionEnvironment(javaEnv: JavaEnv) { */ def getBufferTimout: Long = javaEnv.getBufferTimeout() + /** + * Registers the given Serializer as a default serializer for the given class at the + * [[KryoSerializer]]. + */ + def registerKryoSerializer(clazz: Class[_], serializer: Serializer[_]): Unit = { + javaEnv.registerKryoSerializer(clazz, serializer) + } + + /** + * Registers the given Serializer as a default serializer for the given class at the + * [[KryoSerializer]] + */ + def registerKryoSerializer(clazz: Class[_], serializer: Class[_ <: Serializer[_]]) { + javaEnv.registerKryoSerializer(clazz, serializer) + } + + /** + * Registers the given type with the serialization stack. If the type is eventually + * serialized as a POJO, then the type is registered with the POJO serializer. If the + * type ends up being serialized with Kryo, then it will be registered at Kryo to make + * sure that only tags are written. + * + */ + def registerType(typeClass: Class[_]) { + javaEnv.registerType(typeClass) + } + /** * Creates a DataStream that represents the Strings produced by reading the * given file line wise. The file will be read with the system's default diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala index 35a94cd003209..67f7aae8f10a7 100644 --- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamJoinOperator.scala @@ -18,6 +18,8 @@ package org.apache.flink.streaming.api.scala +import org.apache.flink.api.common.ExecutionConfig + import scala.Array.canBuildFrom import scala.reflect.ClassTag import org.apache.commons.lang.Validate @@ -46,7 +48,7 @@ TemporalOperator[I1, I2, StreamJoinOperator.JoinWindow[I1, I2]](i1, i2) { object StreamJoinOperator { - class JoinWindow[I1, I2](private[flink]op: StreamJoinOperator[I1, I2]) extends + class JoinWindow[I1, I2](private[flink] val op: StreamJoinOperator[I1, I2]) extends TemporalWindow[JoinWindow[I1, I2]] { private[flink] val type1 = op.input1.getType() @@ -59,7 +61,9 @@ object StreamJoinOperator { */ def where(fields: Int*) = { new JoinPredicate[I1, I2](op, KeySelectorUtil.getSelectorForKeys( - new Keys.ExpressionKeys(fields.toArray, type1), type1)) + new Keys.ExpressionKeys(fields.toArray, type1), + type1, + op.input1.getExecutionEnvironment.getConfig)) } /** @@ -70,7 +74,9 @@ object StreamJoinOperator { */ def where(firstField: String, otherFields: String*) = new JoinPredicate[I1, I2](op, KeySelectorUtil.getSelectorForKeys( - new Keys.ExpressionKeys(firstField +: otherFields.toArray, type1), type1)) + new Keys.ExpressionKeys(firstField +: otherFields.toArray, type1), + type1, + op.input1.getExecutionEnvironment.getConfig)) /** * Continues a temporal Join transformation by defining @@ -112,7 +118,9 @@ object StreamJoinOperator { */ def equalTo(fields: Int*): JoinedStream[I1, I2] = { finish(KeySelectorUtil.getSelectorForKeys( - new Keys.ExpressionKeys(fields.toArray, type2), type2)) + new Keys.ExpressionKeys(fields.toArray, type2), + type2, + op.input1.getExecutionEnvironment.getConfig)) } /** @@ -123,7 +131,9 @@ object StreamJoinOperator { */ def equalTo(firstField: String, otherFields: String*): JoinedStream[I1, I2] = finish(KeySelectorUtil.getSelectorForKeys( - new Keys.ExpressionKeys(firstField +: otherFields.toArray, type2), type2)) + new Keys.ExpressionKeys(firstField +: otherFields.toArray, type2), + type2, + op.input1.getExecutionEnvironment.getConfig)) /** * Creates a temporal join transformation by defining the second join key. @@ -151,10 +161,11 @@ object StreamJoinOperator { classOf[(I1, I2)], Seq(op.input1.getType, op.input2.getType), Array("_1", "_2")) { - override def createSerializer: TypeSerializer[(I1, I2)] = { + override def createSerializer( + executionConfig: ExecutionConfig): TypeSerializer[(I1, I2)] = { val fieldSerializers: Array[TypeSerializer[_]] = new Array[TypeSerializer[_]](getArity) for (i <- 0 until getArity) { - fieldSerializers(i) = types(i).createSerializer + fieldSerializers(i) = types(i).createSerializer(executionConfig) } new CaseClassSerializer[(I1, I2)](classOf[(I1, I2)], fieldSerializers) { diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedDataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedDataStream.scala index 5c734bf7bd673..33bbc670022a3 100644 --- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedDataStream.scala +++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/WindowedDataStream.scala @@ -18,6 +18,8 @@ package org.apache.flink.streaming.api.scala +import org.apache.flink.api.java.typeutils.TupleTypeInfoBase + import scala.Array.canBuildFrom import scala.collection.JavaConversions.iterableAsScalaIterable import scala.reflect.ClassTag @@ -26,7 +28,6 @@ import org.apache.flink.api.common.functions.GroupReduceFunction import org.apache.flink.api.common.functions.ReduceFunction import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.functions.KeySelector -import org.apache.flink.api.java.typeutils.TupleTypeInfoBase import org.apache.flink.api.streaming.scala.ScalaStreamingAggregator import org.apache.flink.streaming.api.datastream.{WindowedDataStream => JavaWStream} import org.apache.flink.streaming.api.function.aggregation.AggregationFunction.AggregationType @@ -234,7 +235,10 @@ class WindowedDataStream[T](javaStream: JavaWStream[T]) { val jStream = javaStream.asInstanceOf[JavaWStream[Product]] val outType = jStream.getType().asInstanceOf[TupleTypeInfoBase[_]] - val agg = new ScalaStreamingAggregator[Product](jStream.getType().createSerializer(), position) + val agg = new ScalaStreamingAggregator[Product]( + jStream.getType().createSerializer( + javaStream.getDataStream.getExecutionEnvironment.getConfig), + position) val reducer = aggregationType match { case AggregationType.SUM => new agg.Sum(SumFunction.getForClass( diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountSubclassInterfacePOJOITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountSubclassInterfacePOJOITCase.java new file mode 100644 index 0000000000000..05ffc889fe215 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountSubclassInterfacePOJOITCase.java @@ -0,0 +1,152 @@ +/* + * 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.test.exampleJavaPrograms; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.test.testdata.WordCountData; +import org.apache.flink.test.util.JavaProgramTestBase; +import org.apache.flink.util.Collector; + +import java.io.Serializable; + +@SuppressWarnings("serial") +public class WordCountSubclassInterfacePOJOITCase extends JavaProgramTestBase implements Serializable { + private static final long serialVersionUID = 1L; + protected String textPath; + protected String resultPath; + + + @Override + protected void preSubmit() throws Exception { + textPath = createTempFile("text.txt", WordCountData.TEXT); + resultPath = getTempDirPath("result"); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(WordCountData.COUNTS, resultPath); + } + + @Override + protected void testProgram() throws Exception { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + DataSet text = env.readTextFile(textPath); + + DataSet counts = text + .flatMap(new Tokenizer()) + .groupBy("word") + .reduce(new ReduceFunction() { + private static final long serialVersionUID = 1L; + public WCBase reduce(WCBase value1, WCBase value2) { + WC wc1 = (WC) value1; + WC wc2 = (WC) value2; + int c = wc1.secretCount.getCount() + wc2.secretCount.getCount(); + wc1.secretCount.setCount(c); + return wc1; + } + }) + .map(new MapFunction() { + @Override + public WCBase map(WCBase value) throws Exception { + WC wc = (WC) value; + wc.count = wc.secretCount.getCount(); + return wc; + } + }); + + counts.writeAsText(resultPath); + + env.execute("WordCount with custom data types example"); + } + + public static final class Tokenizer implements FlatMapFunction { + + @Override + public void flatMap(String value, Collector out) { + // normalize and split the line + String[] tokens = value.toLowerCase().split("\\W+"); + // emit the pairs + for (String token : tokens) { + if (token.length() > 0) { + out.collect(new WC(token, 1)); + } + } + } + } + + public static abstract class WCBase { + public String word; + public int count; + + public WCBase(String w, int c) { + this.word = w; + this.count = c; + } + @Override + public String toString() { + return word+" "+count; + } + } + + public static interface CrazyCounter { + public int getCount(); + public void setCount(int c); + } + + public static class CrazyCounterImpl implements CrazyCounter { + public int countz; + + public CrazyCounterImpl() { + } + + public CrazyCounterImpl(int c) { + this.countz = c; + } + + @Override + public int getCount() { + return countz; + } + + @Override + public void setCount(int c) { + this.countz = c; + } + + } + + public static class WC extends WCBase { + public CrazyCounter secretCount; + + public WC() { + super(null, 0); + } + + public WC(String w, int c) { + super(w, 0); + this.secretCount = new CrazyCounterImpl(c); + } + + } + +} \ No newline at end of file diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountSubclassPOJOITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountSubclassPOJOITCase.java new file mode 100644 index 0000000000000..f74ee1629e60d --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountSubclassPOJOITCase.java @@ -0,0 +1,123 @@ +/* + * 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.test.exampleJavaPrograms; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.test.testdata.WordCountData; +import org.apache.flink.test.util.JavaProgramTestBase; +import org.apache.flink.util.Collector; + +import java.io.Serializable; + +@SuppressWarnings("serial") +public class WordCountSubclassPOJOITCase extends JavaProgramTestBase implements Serializable { + private static final long serialVersionUID = 1L; + protected String textPath; + protected String resultPath; + + + @Override + protected void preSubmit() throws Exception { + textPath = createTempFile("text.txt", WordCountData.TEXT); + resultPath = getTempDirPath("result"); + } + + @Override + protected void postSubmit() throws Exception { + compareResultsByLinesInMemory(WordCountData.COUNTS, resultPath); + } + + @Override + protected void testProgram() throws Exception { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + DataSet text = env.readTextFile(textPath); + + DataSet counts = text + .flatMap(new Tokenizer()) + .groupBy("word") + .reduce(new ReduceFunction() { + private static final long serialVersionUID = 1L; + public WCBase reduce(WCBase value1, WCBase value2) { + WC wc1 = (WC) value1; + WC wc2 = (WC) value2; + return new WC(value1.word, wc1.secretCount + wc2.secretCount); + } + }) + .map(new MapFunction() { + @Override + public WCBase map(WCBase value) throws Exception { + WC wc = (WC) value; + wc.count = wc.secretCount; + return wc; + } + }); + + counts.writeAsText(resultPath); + + env.execute("WordCount with custom data types example"); + } + + public static final class Tokenizer implements FlatMapFunction { + + @Override + public void flatMap(String value, Collector out) { + // normalize and split the line + String[] tokens = value.toLowerCase().split("\\W+"); + // emit the pairs + for (String token : tokens) { + if (token.length() > 0) { + out.collect(new WC(token, 1)); + } + } + } + } + + public static abstract class WCBase { + public String word; + public int count; + + public WCBase(String w, int c) { + this.word = w; + this.count = c; + } + @Override + public String toString() { + return word+" "+count; + } + } + + public static class WC extends WCBase { + + public int secretCount; + + public WC() { + super(null, 0); + } + + public WC(String w, int c) { + super(w, 0); + this.secretCount = c; + } + } + +} diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/io/CollectionInputFormatTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/io/CollectionInputFormatTest.scala index 0a0e50d0fab7c..84a0032d24c94 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/io/CollectionInputFormatTest.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/io/CollectionInputFormatTest.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.io +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.java.io.CollectionInputFormat import org.junit.Assert.assertEquals import org.junit.Assert.assertNotNull @@ -55,8 +56,11 @@ class CollectionInputFormatTest { val inputCollection = Seq(new ElementType(1), new ElementType(2), new ElementType(3)) val info = createTypeInformation[ElementType] - val inputFormat: CollectionInputFormat[ElementType] = new - CollectionInputFormat[ElementType](inputCollection.asJava, info.createSerializer()) + val inputFormat: CollectionInputFormat[ElementType] = { + new CollectionInputFormat[ElementType]( + inputCollection.asJava, + info.createSerializer(new ExecutionConfig)) + } val buffer = new ByteArrayOutputStream val out = new ObjectOutputStream(buffer) @@ -107,7 +111,7 @@ class CollectionInputFormatTest { val inputFormat = new CollectionInputFormat[String]( data.asJava, - BasicTypeInfo.STRING_TYPE_INFO.createSerializer) + BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig)) val baos = new ByteArrayOutputStream val oos = new ObjectOutputStream(baos) diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/misc/MassiveCaseClassSortingITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/misc/MassiveCaseClassSortingITCase.scala index 4f8816fd6ec96..157aa0deac85a 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/misc/MassiveCaseClassSortingITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/misc/MassiveCaseClassSortingITCase.scala @@ -22,6 +22,7 @@ import java.io.File import java.util.Random import java.io.BufferedWriter import java.io.FileWriter +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.scala._ import java.io.BufferedReader import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync @@ -81,8 +82,12 @@ class MassiveCaseClassSortingITCase { val typeInfo = implicitly[TypeInformation[StringTuple]] .asInstanceOf[CompositeType[StringTuple]] - val serializer = typeInfo.createSerializer() - val comparator = typeInfo.createComparator(Array(0, 1), Array(true, true), 0) + val serializer = typeInfo.createSerializer(new ExecutionConfig) + val comparator = typeInfo.createComparator( + Array(0, 1), + Array(true, true), + 0, + new ExecutionConfig) val mm = new DefaultMemoryManager(1024 * 1024, 1) val ioMan = new IOManagerAsync() diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/CaseClassComparatorTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/CaseClassComparatorTest.scala index 47183954467f0..21c658162915f 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/CaseClassComparatorTest.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/CaseClassComparatorTest.scala @@ -18,6 +18,7 @@ package org.apache.flink.api.scala.runtime +import org.apache.flink.api.common.ExecutionConfig import org.junit.Test import org.junit.Assert._ import org.apache.flink.api.scala._ @@ -48,9 +49,13 @@ class CaseClassComparatorTest { val typeInfo = implicitly[TypeInformation[CaseTestClass]] .asInstanceOf[CompositeType[CaseTestClass]] - val serializer = typeInfo.createSerializer() + val serializer = typeInfo.createSerializer(new ExecutionConfig) val comparator = new FailingCompareDeserializedWrapper( - typeInfo.createComparator(Array[Int](0, 2), Array[Boolean](true, true), 0)) + typeInfo.createComparator( + Array[Int](0, 2), + Array[Boolean](true, true), + 0, + new ExecutionConfig)) assertTrue(comparator.supportsNormalizedKey()) assertEquals(8, comparator.getNormalizeKeyLen()) diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/KryoGenericTypeSerializerTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/KryoGenericTypeSerializerTest.scala index c396f9f32e47c..ce4efb3c2ce93 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/KryoGenericTypeSerializerTest.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/KryoGenericTypeSerializerTest.scala @@ -17,13 +17,15 @@ */ package org.apache.flink.api.scala.runtime +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeutils.SerializerTestInstance +import org.apache.flink.api.java.ExecutionEnvironment import org.apache.flink.api.java.typeutils.GenericTypeInfo +import org.apache.flink.api.java.typeutils.runtime.KryoSerializer import org.joda.time.DateTime import org.junit.Test import scala.reflect._ import org.joda.time.LocalDate -import org.apache.flink.api.java.typeutils.runtime.KryoSerializer import com.esotericsoftware.kryo.Serializer import com.esotericsoftware.kryo.Kryo import com.esotericsoftware.kryo.io.Output @@ -94,8 +96,6 @@ class KryoGenericTypeSerializerTest { def jodaSerialization: Unit = { val a = List(new LocalDate(1), new LocalDate(2)) - KryoSerializer.registerSerializer(classOf[LocalDate], new LocalDateSerializer()) - runTests(a) } @@ -191,8 +191,13 @@ class KryoGenericTypeSerializerTest { def runTests[T : ClassTag](objects: Seq[T]): Unit ={ val clsTag = classTag[T] + + + // Register the custom Kryo Serializer + val conf = new ExecutionConfig + conf.registerKryoSerializer(classOf[LocalDate], classOf[LocalDateSerializer]) val typeInfo = new GenericTypeInfo[T](clsTag.runtimeClass.asInstanceOf[Class[T]]) - val serializer = typeInfo.createSerializer() + val serializer = typeInfo.createSerializer(conf) val typeClass = typeInfo.getTypeClass val instance = new SerializerTestInstance[T](serializer, typeClass, -1, objects: _*) diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/ScalaSpecialTypesSerializerTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/ScalaSpecialTypesSerializerTest.scala index fc51c0cb3688b..c86fde03886c5 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/ScalaSpecialTypesSerializerTest.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/ScalaSpecialTypesSerializerTest.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.runtime +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.java.typeutils.runtime.KryoSerializer import org.junit.Assert._ @@ -90,7 +91,7 @@ class ScalaSpecialTypesSerializerTest { private final def runTests[T : TypeInformation](instances: Array[T]) { try { val typeInfo = implicitly[TypeInformation[T]] - val serializer = typeInfo.createSerializer + val serializer = typeInfo.createSerializer(new ExecutionConfig) val typeClass = typeInfo.getTypeClass val test = new ScalaSpecialTypesSerializerTestInstance[T](serializer, typeClass, -1, instances) @@ -116,8 +117,9 @@ class ScalaSpecialTypesSerializerTestInstance[T]( override def testInstantiate(): Unit = { try { val serializer: TypeSerializer[T] = getSerializer - val instance: T = serializer.createInstance if (!serializer.isInstanceOf[KryoSerializer[_]]) { + // kryo serializer does return null, so only test for non-kryo-serializers + val instance: T = serializer.createInstance assertNotNull("The created instance must not be null.", instance) } val tpe: Class[T] = getTypeClass diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TraversableSerializerTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TraversableSerializerTest.scala index 84ff4a6237916..65648b6928a40 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TraversableSerializerTest.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TraversableSerializerTest.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.runtime +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.functions.InvalidTypesException import org.junit.Assert._ @@ -104,7 +105,7 @@ class TraversableSerializerTest { val testData = Array(Array((1, "String"), (2, "Foo")), Array((4, "String"), (3, "Foo"))) runTests(testData) } -// + @Test def testWithCaseClass(): Unit = { val testData = Array(Seq((1, "String"), (2, "Foo")), Seq((4, "String"), (3, "Foo"))) @@ -132,7 +133,7 @@ class TraversableSerializerTest { private final def runTests[T : TypeInformation](instances: Array[T]) { try { val typeInfo = implicitly[TypeInformation[T]] - val serializer = typeInfo.createSerializer + val serializer = typeInfo.createSerializer(new ExecutionConfig) val typeClass = typeInfo.getTypeClass val test = new ScalaSpecialTypesSerializerTestInstance[T](serializer, typeClass, -1, instances) diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILD2Test.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILD2Test.scala index 8ccbc8375fdd8..8b1a1801cf47f 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILD2Test.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILD2Test.scala @@ -17,8 +17,8 @@ */ package org.apache.flink.api.scala.runtime -import org.apache.flink.api.common.typeutils.TypeComparator -import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.common.ExecutionConfig +import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer} import org.apache.flink.api.java.typeutils.TupleTypeInfoBase import org.apache.flink.api.scala.runtime.tuple.base.TupleComparatorTestBase @@ -30,12 +30,12 @@ class TupleComparatorILD2Test extends TupleComparatorTestBase[(Int, Long, Double protected def createComparator(ascending: Boolean): TypeComparator[(Int, Long, Double)] = { val ti = createTypeInformation[(Int, Long, Double)] ti.asInstanceOf[TupleTypeInfoBase[(Int, Long, Double)]] - .createComparator(Array(0, 1), Array(ascending, ascending), 0) + .createComparator(Array(0, 1), Array(ascending, ascending), 0, new ExecutionConfig) } protected def createSerializer: TypeSerializer[(Int, Long, Double)] = { val ti = createTypeInformation[(Int, Long, Double)] - ti.createSerializer() + ti.createSerializer(new ExecutionConfig) } protected def getSortedTestData: Array[(Int, Long, Double)] = { diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILD3Test.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILD3Test.scala index d3295718050fd..2fdc087d1e277 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILD3Test.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILD3Test.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.runtime +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer} import org.apache.flink.api.java.typeutils.TupleTypeInfoBase import org.apache.flink.api.scala._ @@ -28,12 +29,16 @@ class TupleComparatorILD3Test extends TupleComparatorTestBase[(Int, Long, Double protected def createComparator(ascending: Boolean): TypeComparator[(Int, Long, Double)] = { val ti = createTypeInformation[(Int, Long, Double)] ti.asInstanceOf[TupleTypeInfoBase[(Int, Long, Double)]] - .createComparator(Array(0, 1, 2), Array(ascending, ascending, ascending), 0) + .createComparator( + Array(0, 1, 2), + Array(ascending, ascending, ascending), + 0, + new ExecutionConfig) } protected def createSerializer: TypeSerializer[(Int, Long, Double)] = { val ti = createTypeInformation[(Int, Long, Double)] - ti.createSerializer() + ti.createSerializer(new ExecutionConfig) } protected def getSortedTestData: Array[(Int, Long, Double)] = { diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDC3Test.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDC3Test.scala index de7affdad58af..34e03065bec65 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDC3Test.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDC3Test.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.runtime +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeutils.{TypeSerializer, TypeComparator} import org.apache.flink.api.java.typeutils.TupleTypeInfoBase import org.apache.flink.api.scala._ @@ -28,12 +29,16 @@ class TupleComparatorILDC3Test extends TupleComparatorTestBase[(Int, Long, Doubl protected def createComparator(ascending: Boolean): TypeComparator[(Int, Long, Double)] = { val ti = createTypeInformation[(Int, Long, Double)] ti.asInstanceOf[TupleTypeInfoBase[(Int, Long, Double)]] - .createComparator(Array(2, 0, 1), Array(ascending, ascending, ascending), 0) + .createComparator( + Array(2, 0, 1), + Array(ascending, ascending, ascending), + 0, + new ExecutionConfig) } protected def createSerializer: TypeSerializer[(Int, Long, Double)] = { val ti = createTypeInformation[(Int, Long, Double)] - ti.createSerializer() + ti.createSerializer(new ExecutionConfig) } protected def getSortedTestData: Array[(Int, Long, Double)] = { diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDX1Test.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDX1Test.scala index eebd56f3abf0f..27d8296974b2e 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDX1Test.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDX1Test.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.runtime +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeutils.{TypeSerializer, TypeComparator} import org.apache.flink.api.java.typeutils.TupleTypeInfoBase import org.apache.flink.api.scala.runtime.tuple.base.TupleComparatorTestBase @@ -28,12 +29,12 @@ class TupleComparatorILDX1Test extends TupleComparatorTestBase[(Int, Long, Doubl protected def createComparator(ascending: Boolean): TypeComparator[(Int, Long, Double)] = { val ti = createTypeInformation[(Int, Long, Double)] ti.asInstanceOf[TupleTypeInfoBase[(Int, Long, Double)]] - .createComparator(Array(1), Array(ascending), 0) + .createComparator(Array(1), Array(ascending), 0, new ExecutionConfig) } protected def createSerializer: TypeSerializer[(Int, Long, Double)] = { val ti = createTypeInformation[(Int, Long, Double)] - ti.createSerializer() + ti.createSerializer(new ExecutionConfig) } protected def getSortedTestData: Array[(Int, Long, Double)] = { diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDXC2Test.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDXC2Test.scala index c83ab8b03e062..8231d46f74e55 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDXC2Test.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorILDXC2Test.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.runtime +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeutils.{TypeSerializer, TypeComparator} import org.apache.flink.api.java.typeutils.TupleTypeInfoBase import org.apache.flink.api.scala.runtime.tuple.base.TupleComparatorTestBase @@ -28,12 +29,12 @@ class TupleComparatorILDXC2Test extends TupleComparatorTestBase[(Int, Long, Doub protected def createComparator(ascending: Boolean): TypeComparator[(Int, Long, Double)] = { val ti = createTypeInformation[(Int, Long, Double)] ti.asInstanceOf[TupleTypeInfoBase[(Int, Long, Double)]] - .createComparator(Array(2, 1), Array(ascending, ascending), 0) + .createComparator(Array(2, 1), Array(ascending, ascending), 0, new ExecutionConfig) } protected def createSerializer: TypeSerializer[(Int, Long, Double)] = { val ti = createTypeInformation[(Int, Long, Double)] - ti.createSerializer() + ti.createSerializer(new ExecutionConfig) } protected def getSortedTestData: Array[(Int, Long, Double)] = { diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD1Test.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD1Test.scala index 03c96668f8111..1e1399e3ba92e 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD1Test.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD1Test.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.runtime +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeutils.{TypeSerializer, TypeComparator} import org.apache.flink.api.java.typeutils.TupleTypeInfoBase import org.apache.flink.api.scala._ @@ -27,12 +28,12 @@ class TupleComparatorISD1Test extends TupleComparatorTestBase[(Int, String, Doub protected def createComparator(ascending: Boolean): TypeComparator[(Int, String, Double)] = { val ti = createTypeInformation[(Int, String, Double)] ti.asInstanceOf[TupleTypeInfoBase[(Int, String, Double)]] - .createComparator(Array(0), Array(ascending),0) + .createComparator(Array(0), Array(ascending), 0, new ExecutionConfig) } protected def createSerializer: TypeSerializer[(Int, String, Double)] = { val ti = createTypeInformation[(Int, String, Double)] - ti.createSerializer() + ti.createSerializer(new ExecutionConfig) } protected def getSortedTestData: Array[(Int, String, Double)] = { diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD2Test.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD2Test.scala index 9a3b9f9b19772..eb905bd149a3d 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD2Test.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD2Test.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.runtime +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer} import org.apache.flink.api.java.typeutils.TupleTypeInfoBase import org.apache.flink.api.scala._ @@ -27,12 +28,12 @@ class TupleComparatorISD2Test extends TupleComparatorTestBase[(Int, String, Doub protected def createComparator(ascending: Boolean): TypeComparator[(Int, String, Double)] = { val ti = createTypeInformation[(Int, String, Double)] ti.asInstanceOf[TupleTypeInfoBase[(Int, String, Double)]] - .createComparator(Array(0, 1), Array(ascending, ascending), 0) + .createComparator(Array(0, 1), Array(ascending, ascending), 0, new ExecutionConfig) } protected def createSerializer: TypeSerializer[(Int, String, Double)] = { val ti = createTypeInformation[(Int, String, Double)] - ti.createSerializer() + ti.createSerializer(new ExecutionConfig) } protected def getSortedTestData: Array[(Int, String, Double)] = { diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD3Test.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD3Test.scala index 01b4f3ec4c39a..d7ff16a063f00 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD3Test.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleComparatorISD3Test.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.api.scala.runtime +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeutils.{TypeSerializer, TypeComparator} import org.apache.flink.api.java.typeutils.TupleTypeInfoBase import org.apache.flink.api.scala._ @@ -27,12 +28,16 @@ class TupleComparatorISD3Test extends TupleComparatorTestBase[(Int, String, Doub protected def createComparator(ascending: Boolean): TypeComparator[(Int, String, Double)] = { val ti = createTypeInformation[(Int, String, Double)] ti.asInstanceOf[TupleTypeInfoBase[(Int, String, Double)]] - .createComparator(Array(0, 1, 2), Array(ascending, ascending, ascending), 0) + .createComparator( + Array(0, 1, 2), + Array(ascending, ascending, ascending), + 0, + new ExecutionConfig) } protected def createSerializer: TypeSerializer[(Int, String, Double)] = { val ti = createTypeInformation[(Int, String, Double)] - ti.createSerializer() + ti.createSerializer(new ExecutionConfig) } protected def getSortedTestData: Array[(Int, String, Double)] = { diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerTest.scala index 29e13ecce48a7..9371604f71d5a 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerTest.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerTest.scala @@ -18,6 +18,8 @@ package org.apache.flink.api.scala.runtime import java.util +import org.apache.flink.api.common.ExecutionConfig +import org.apache.flink.api.java.ExecutionEnvironment import org.apache.flink.api.java.typeutils.TupleTypeInfoBase import org.apache.flink.api.java.typeutils.runtime.AbstractGenericTypeSerializerTest._ import org.apache.flink.api.common.typeinfo.TypeInformation @@ -28,7 +30,6 @@ import org.junit.Test import org.apache.flink.api.scala._ import scala.collection.JavaConverters._ import java.util.Random -import org.apache.flink.api.java.typeutils.runtime.KryoSerializer class TupleSerializerTest { @@ -102,8 +103,6 @@ class TupleSerializerTest { (StringUtils.getRandomString(rnd, 10, 100), new LocalDate(rnd.nextInt)), (StringUtils.getRandomString(rnd, 10, 100), new LocalDate(rnd.nextInt))) - KryoSerializer.registerSerializer(classOf[LocalDate], new LocalDateSerializer()) - runTests(testTuples) } @@ -192,8 +191,11 @@ class TupleSerializerTest { private final def runTests[T <: Product : TypeInformation](instances: Array[T]) { try { + // Register the custom Kryo Serializer + val conf = new ExecutionConfig + conf.registerKryoSerializer(classOf[LocalDate], classOf[LocalDateSerializer]) val tupleTypeInfo = implicitly[TypeInformation[T]].asInstanceOf[TupleTypeInfoBase[T]] - val serializer = tupleTypeInfo.createSerializer + val serializer = tupleTypeInfo.createSerializer(conf) val tupleClass = tupleTypeInfo.getTypeClass val test = new TupleSerializerTestInstance[T](serializer, tupleClass, -1, instances) test.testAll() 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 89d7c5eddc653..08ba49d0c57eb 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 @@ -20,6 +20,7 @@ package org.apache.flink.api.scala.types import java.io.DataInput import java.io.DataOutput import org.apache.flink.api.common.typeinfo._ +import org.apache.flink.api.common.typeutils._ import org.apache.flink.api.java.typeutils._ import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo import org.apache.flink.types.{IntValue, StringValue}