diff --git a/flink-core/pom.xml b/flink-core/pom.xml index adc9a9bd98b136ec541b20f8332c4399c2c5adb3..ba1050ca4268b7d5c648e3eed08418c968082431 100644 --- a/flink-core/pom.xml +++ b/flink-core/pom.xml @@ -40,12 +40,6 @@ under the License. flink-annotations ${project.version} - - - org.apache.flink - ${shading-artifact.name} - ${project.version} - commons-collections @@ -56,20 +50,61 @@ under the License. com.esotericsoftware.kryo kryo + + + + org.apache.avro + avro + + + + org.xerial.snappy + snappy-java + + + org.apache.commons + commons-compress + + + + + + + org.apache.flink + ${shading-artifact.name} + ${project.version} + + com.google.guava guava ${guava.version} + + commons-io commons-io test + + joda-time + joda-time + 2.5 + test + + + + org.joda + joda-convert + 1.7 + test + + diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/Keys.java similarity index 81% rename from flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java rename to flink-core/src/main/java/org/apache/flink/api/common/operators/Keys.java index 5992f0bcc562a860e0ff1f2578c30b212f05173b..6d681de9a33c0a98b4b92986b9ba9fee490837b0 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/Keys.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/Keys.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.api.java.operators; +package org.apache.flink.api.common.operators; import java.util.ArrayList; import java.util.List; @@ -24,22 +24,12 @@ import java.util.List; import com.google.common.base.Joiner; import org.apache.flink.api.common.InvalidProgramException; -import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.api.common.operators.Operator; -import org.apache.flink.api.common.operators.UnaryOperatorInformation; -import org.apache.flink.api.common.operators.base.MapOperatorBase; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.CompositeType; import org.apache.flink.api.common.typeutils.CompositeType.FlatFieldDescriptor; import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.operators.translation.KeyExtractingMapper; -import org.apache.flink.api.java.operators.translation.KeyRemovingMapper; -import org.apache.flink.api.java.operators.translation.TwoKeyExtractingMapper; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.typeutils.GenericTypeInfo; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; import com.google.common.base.Preconditions; @@ -177,87 +167,6 @@ public abstract class Keys { } } - @SuppressWarnings("unchecked") - public static Operator> appendKeyExtractor( - Operator input, - SelectorFunctionKeys key) - { - - TypeInformation inputType = key.getInputType(); - TypeInformation> typeInfoWithKey = createTypeWithKey(key); - KeyExtractingMapper extractor = new KeyExtractingMapper(key.getKeyExtractor()); - - MapOperatorBase, MapFunction>> mapper = - new MapOperatorBase, MapFunction>>( - extractor, - new UnaryOperatorInformation(inputType, typeInfoWithKey), - "Key Extractor" - ); - - mapper.setInput(input); - mapper.setParallelism(input.getParallelism()); - - return mapper; - } - - @SuppressWarnings("unchecked") - public static Operator> appendKeyExtractor( - Operator input, - SelectorFunctionKeys key1, - SelectorFunctionKeys key2) - { - - TypeInformation inputType = key1.getInputType(); - TypeInformation> typeInfoWithKey = createTypeWithKey(key1, key2); - TwoKeyExtractingMapper extractor = - new TwoKeyExtractingMapper<>(key1.getKeyExtractor(), key2.getKeyExtractor()); - - MapOperatorBase, MapFunction>> mapper = - new MapOperatorBase, MapFunction>>( - extractor, - new UnaryOperatorInformation<>(inputType, typeInfoWithKey), - "Key Extractor" - ); - - mapper.setInput(input); - mapper.setParallelism(input.getParallelism()); - - return mapper; - } - - public static org.apache.flink.api.common.operators.SingleInputOperator appendKeyRemover( - Operator> inputWithKey, - SelectorFunctionKeys key) - { - - TypeInformation inputType = key.getInputType(); - TypeInformation> typeInfoWithKey = createTypeWithKey(key); - - MapOperatorBase, T, MapFunction, T>> mapper = - new MapOperatorBase, T, MapFunction, T>>( - new KeyRemovingMapper(), - new UnaryOperatorInformation<>(typeInfoWithKey, inputType), - "Key Remover" - ); - mapper.setInput(inputWithKey); - mapper.setParallelism(inputWithKey.getParallelism()); - - return mapper; - } - - public static TypeInformation> createTypeWithKey( - SelectorFunctionKeys key) - { - return new TupleTypeInfo<>(key.getKeyType(), key.getInputType()); - } - - public static TypeInformation> createTypeWithKey( - SelectorFunctionKeys key1, - SelectorFunctionKeys key2) - { - return new TupleTypeInfo<>(key1.getKeyType(), key2.getKeyType(), key1.getInputType()); - } - @Override public String toString() { return "Key function (Type: " + keyType + ")"; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/KeySelector.java b/flink-core/src/main/java/org/apache/flink/api/java/functions/KeySelector.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/functions/KeySelector.java rename to flink-core/src/main/java/org/apache/flink/api/java/functions/KeySelector.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/AvroTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/AvroTypeInfo.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/AvroTypeInfo.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/AvroTypeInfo.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/EitherTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/EitherTypeInfo.java similarity index 98% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/EitherTypeInfo.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/EitherTypeInfo.java index ec7be972268a958f11c5ba879790d3ad96a6149a..74d850b03c316c009c6acdfb48840effa1ce93fe 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/EitherTypeInfo.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/EitherTypeInfo.java @@ -22,6 +22,7 @@ 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.EitherSerializer; +import org.apache.flink.types.Either; /** * A {@link TypeInformation} for the {@link Either} type of the Java API. diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/EnumTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/EnumTypeInfo.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/EnumTypeInfo.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/EnumTypeInfo.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/GenericTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/GenericTypeInfo.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/GenericTypeInfo.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/GenericTypeInfo.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/InputTypeConfigurable.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/InputTypeConfigurable.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/InputTypeConfigurable.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/InputTypeConfigurable.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/MissingTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/MissingTypeInfo.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/MissingTypeInfo.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/MissingTypeInfo.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfo.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfo.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfo.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/PojoField.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/PojoField.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/PojoField.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/PojoField.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java similarity index 99% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java index f7e4e42e444926c9a51afaebce58e545b25693bd..cc0d2396234d404d7cc550da118c38274025f81a 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/PojoTypeInfo.java @@ -28,7 +28,9 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import com.google.common.base.Preconditions; + import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.operators.Keys.ExpressionKeys; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.CompositeType; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -36,7 +38,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.typeutils.runtime.AvroSerializer; 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; import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ResultTypeQueryable.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/ResultTypeQueryable.java similarity index 97% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/ResultTypeQueryable.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/ResultTypeQueryable.java index 22a18f196862b83e0884083695a3523983878c8d..5e0cbedda01b54511254b057e3e876e2b8e2d297 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ResultTypeQueryable.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/ResultTypeQueryable.java @@ -24,7 +24,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; * This interface can be implemented by functions and input formats to tell the framework * about their produced data type. This method acts as an alternative to the reflection analysis * that is otherwise performed and is useful in situations where the produced data type may vary - * depending on parameterization. + * depending on parametrization. */ public interface ResultTypeQueryable { diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfo.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfoBase.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfoBase.java similarity index 98% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfoBase.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfoBase.java index 469476f4b68362f4e1160006f44a4e3a1de7dbb0..753eb66ac12d4898e1c6f5ec33025f46795b6f8c 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfoBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TupleTypeInfoBase.java @@ -24,9 +24,10 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import com.google.common.base.Preconditions; + +import org.apache.flink.api.common.operators.Keys.ExpressionKeys; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.CompositeType; -import org.apache.flink.api.java.operators.Keys.ExpressionKeys; public abstract class TupleTypeInfoBase extends CompositeType { @@ -35,7 +36,7 @@ public abstract class TupleTypeInfoBase extends CompositeType { private final static String REGEX_FIELD = "(f?)([0-9]+)"; private final static String REGEX_NESTED_FIELDS = "("+REGEX_FIELD+")(\\.(.+))?"; private final static String REGEX_NESTED_FIELDS_WILDCARD = REGEX_NESTED_FIELDS - +"|\\"+ExpressionKeys.SELECT_ALL_CHAR + +"|\\"+ ExpressionKeys.SELECT_ALL_CHAR +"|\\"+ExpressionKeys.SELECT_ALL_CHAR_SCALA; private static final Pattern PATTERN_FIELD = Pattern.compile(REGEX_FIELD); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java similarity index 99% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java index b8f20755f225d57d0cbf87f750ab38beca5e1bb0..d4ea24c102e16c6cdd7661d198d8855d89fb1840 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java @@ -30,7 +30,9 @@ import java.util.ArrayList; import java.util.List; import org.apache.avro.specific.SpecificRecordBase; + import org.apache.commons.lang3.ClassUtils; + import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.functions.CrossFunction; import org.apache.flink.api.common.functions.FlatJoinFunction; @@ -54,9 +56,12 @@ import org.apache.flink.api.common.typeutils.CompositeType; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple0; +import org.apache.flink.types.Either; import org.apache.flink.types.Value; import org.apache.flink.util.Collector; + import org.apache.hadoop.io.Writable; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeInfoParser.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeInfoParser.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeInfoParser.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeInfoParser.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/ValueTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/ValueTypeInfo.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/ValueTypeInfo.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/ValueTypeInfo.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/WritableTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/WritableTypeInfo.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/WritableTypeInfo.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/WritableTypeInfo.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueComparator.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueComparator.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueComparator.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueComparator.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueSerializer.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueSerializer.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueSerializer.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataInputDecoder.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataInputDecoder.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataInputDecoder.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataInputDecoder.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataInputViewStream.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataInputViewStream.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataInputViewStream.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataInputViewStream.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataOutputEncoder.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataOutputEncoder.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataOutputEncoder.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataOutputEncoder.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataOutputViewStream.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataOutputViewStream.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataOutputViewStream.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/DataOutputViewStream.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java similarity index 96% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java index b4b95f3965d3bcf631e4ed5352200b7c10fbb402..7c9676bcb4b04ca874300c78bcf03cfcfd4eb6e2 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java @@ -18,13 +18,13 @@ package org.apache.flink.api.java.typeutils.runtime; -import static org.apache.flink.api.java.typeutils.Either.Left; -import static org.apache.flink.api.java.typeutils.Either.Right; +import static org.apache.flink.types.Either.Left; +import static org.apache.flink.types.Either.Right; import java.io.IOException; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.typeutils.Either; +import org.apache.flink.types.Either; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/GenericTypeComparator.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/GenericTypeComparator.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/GenericTypeComparator.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/GenericTypeComparator.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoUtils.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoUtils.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoUtils.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoUtils.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/NoFetchingInput.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NoFetchingInput.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/NoFetchingInput.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NoFetchingInput.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoComparator.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoComparator.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoComparator.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoComparator.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeComparatorFactory.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeComparatorFactory.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeComparatorFactory.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeComparatorFactory.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimePairComparatorFactory.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimePairComparatorFactory.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimePairComparatorFactory.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimePairComparatorFactory.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeSerializerFactory.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeSerializerFactory.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeSerializerFactory.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeSerializerFactory.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/Tuple0Serializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/Tuple0Serializer.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/Tuple0Serializer.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/Tuple0Serializer.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparator.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorBase.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorBase.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorBase.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorBase.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueComparator.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueComparator.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueComparator.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueComparator.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializer.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializer.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializer.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableComparator.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableComparator.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableComparator.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableComparator.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializer.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializer.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializer.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java similarity index 100% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/Serializers.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/Serializers.java similarity index 78% rename from flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/Serializers.java rename to flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/Serializers.java index 8bac729cc7ce03ae7741676ba0c17f48262fe69e..009cb2413bab87e5d235db37a7d9e119747692e7 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/Serializers.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/Serializers.java @@ -23,13 +23,14 @@ import com.esotericsoftware.kryo.Serializer; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; import com.esotericsoftware.kryo.serializers.CollectionSerializer; + import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.specific.SpecificRecordBase; + 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.java.Utils; import org.apache.flink.api.java.typeutils.GenericTypeInfo; import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; @@ -64,7 +65,7 @@ public class Serializers { } else if (typeInfo instanceof CompositeType) { List> genericTypesInComposite = new ArrayList<>(); - Utils.getContainedGenericTypes((CompositeType)typeInfo, genericTypesInComposite); + getContainedGenericTypes((CompositeType)typeInfo, genericTypesInComposite); for (GenericTypeInfo gt : genericTypesInComposite) { Serializers.recursivelyRegisterType(gt.getTypeClass(), config, alreadySeen); } @@ -127,41 +128,38 @@ public class Serializers { } } - private static void checkAndAddSerializerForTypeAvro(ExecutionConfig reg, Class type) { - if (GenericData.Record.class.isAssignableFrom(type)) { - registerGenericAvro(reg); - } - if (SpecificRecordBase.class.isAssignableFrom(type)) { - @SuppressWarnings("unchecked") - Class specRecordClass = (Class) type; - registerSpecificAvro(reg, specRecordClass); - } - } - /** - * Register these serializers for using Avro's {@link GenericData.Record} and classes - * implementing {@link org.apache.avro.specific.SpecificRecordBase} + * Returns all GenericTypeInfos contained in a composite type. + * + * @param typeInfo {@link CompositeType} */ - private static void registerGenericAvro(ExecutionConfig reg) { - // Avro POJOs contain java.util.List which have GenericData.Array as their runtime type - // because Kryo is not able to serialize them properly, we use this serializer for them - reg.registerTypeWithKryoSerializer(GenericData.Array.class, SpecificInstanceCollectionSerializerForArrayList.class); - - // We register this serializer for users who want to use untyped Avro records (GenericData.Record). - // Kryo is able to serialize everything in there, except for the Schema. - // This serializer is very slow, but using the GenericData.Records of Kryo is in general a bad idea. - // we add the serializer as a default serializer because Avro is using a private sub-type at runtime. - reg.addDefaultKryoSerializer(Schema.class, AvroSchemaSerializer.class); + private static void getContainedGenericTypes(CompositeType typeInfo, List> target) { + for (int i = 0; i < typeInfo.getArity(); i++) { + TypeInformation type = typeInfo.getTypeAt(i); + if (type instanceof CompositeType) { + getContainedGenericTypes((CompositeType) type, target); + } else if (type instanceof GenericTypeInfo) { + if (!target.contains(type)) { + target.add((GenericTypeInfo) type); + } + } + } } + + // ------------------------------------------------------------------------ + + private static void checkAndAddSerializerForTypeAvro(ExecutionConfig reg, Class type) { + if (GenericData.Record.class.isAssignableFrom(type) || SpecificRecordBase.class.isAssignableFrom(type)) { + // Avro POJOs contain java.util.List which have GenericData.Array as their runtime type + // because Kryo is not able to serialize them properly, we use this serializer for them + reg.registerTypeWithKryoSerializer(GenericData.Array.class, SpecificInstanceCollectionSerializerForArrayList.class); - private static void registerSpecificAvro(ExecutionConfig reg, Class avroType) { - registerGenericAvro(reg); - // This rule only applies if users explicitly use the GenericTypeInformation for the avro types - // usually, we are able to handle Avro POJOs with the POJO serializer. - // (However only if the GenericData.Array type is registered!) - - // ClassTag tag = scala.reflect.ClassTag$.MODULE$.apply(avroType); - // reg.registerTypeWithKryoSerializer(avroType, com.twitter.chill.avro.AvroSerializer.SpecificRecordSerializer(tag)); + // We register this serializer for users who want to use untyped Avro records (GenericData.Record). + // Kryo is able to serialize everything in there, except for the Schema. + // This serializer is very slow, but using the GenericData.Records of Kryo is in general a bad idea. + // we add the serializer as a default serializer because Avro is using a private sub-type at runtime. + reg.addDefaultKryoSerializer(Schema.class, AvroSchemaSerializer.class); + } } // -------------------------------------------------------------------------------------------- diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/ExpressionKeysTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/ExpressionKeysTest.java similarity index 97% rename from flink-java/src/test/java/org/apache/flink/api/java/operators/ExpressionKeysTest.java rename to flink-core/src/test/java/org/apache/flink/api/common/operators/ExpressionKeysTest.java index dd8223417b45a0af7b966fe62827a31196dbf006..8e65a79b3e00b9a19b49b95d44c4f5e917b3a1a9 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/operators/ExpressionKeysTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/ExpressionKeysTest.java @@ -15,29 +15,33 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.api.java.operators; + +package org.apache.flink.api.common.operators; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.Arrays; import org.apache.commons.lang3.ArrayUtils; + import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.operators.Keys.ExpressionKeys; +import org.apache.flink.api.common.operators.Keys.ExpressionKeys; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple7; -import org.apache.flink.api.java.type.extractor.PojoTypeExtractionTest.ComplexNestedClass; +import org.apache.flink.api.java.typeutils.PojoTypeExtractionTest.ComplexNestedClass; import org.apache.flink.api.java.typeutils.GenericTypeInfo; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.api.java.operators.SelectorFunctionKeysTest.KeySelector1; -import org.apache.flink.api.java.operators.SelectorFunctionKeysTest.KeySelector3; +import org.apache.flink.api.common.operators.SelectorFunctionKeysTest.KeySelector1; +import org.apache.flink.api.common.operators.SelectorFunctionKeysTest.KeySelector3; + import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; + import org.powermock.modules.junit4.PowerMockRunner; import org.powermock.reflect.Whitebox; @@ -474,6 +478,4 @@ public class ExpressionKeysTest { return 0; } } - - } diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/SelectorFunctionKeysTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/SelectorFunctionKeysTest.java similarity index 94% rename from flink-java/src/test/java/org/apache/flink/api/java/operators/SelectorFunctionKeysTest.java rename to flink-core/src/test/java/org/apache/flink/api/common/operators/SelectorFunctionKeysTest.java index f59af6e543ae56679e4c07bf832d4dfb0943e5a4..ad6fefa2135ec85243d427a2d692f1e24b23dcf9 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/operators/SelectorFunctionKeysTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/SelectorFunctionKeysTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.api.java.operators; +package org.apache.flink.api.common.operators; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -25,9 +25,10 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.api.java.operators.ExpressionKeysTest.PojoWithMultiplePojos; -import org.apache.flink.api.java.operators.ExpressionKeysTest.Pojo1; -import org.apache.flink.api.java.operators.ExpressionKeysTest.Pojo2; +import org.apache.flink.api.common.operators.ExpressionKeysTest.PojoWithMultiplePojos; +import org.apache.flink.api.common.operators.ExpressionKeysTest.Pojo1; +import org.apache.flink.api.common.operators.ExpressionKeysTest.Pojo2; + import org.junit.Assert; import org.junit.Test; diff --git a/flink-java/src/test/java/org/apache/flink/api/java/tuple/Tuple2Test.java b/flink-core/src/test/java/org/apache/flink/api/java/tuple/Tuple2Test.java similarity index 99% rename from flink-java/src/test/java/org/apache/flink/api/java/tuple/Tuple2Test.java rename to flink-core/src/test/java/org/apache/flink/api/java/tuple/Tuple2Test.java index 0cc5dcfb45e4259e9902353fce2f8cfb26528374..b8fe14ccb6303f85b7e3729b90997aa7bc68a38a 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/tuple/Tuple2Test.java +++ b/flink-core/src/test/java/org/apache/flink/api/java/tuple/Tuple2Test.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package org.apache.flink.api.java.tuple; import org.apache.flink.types.NullFieldException; diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/CompositeTypeTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/CompositeTypeTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/CompositeTypeTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/CompositeTypeTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/EitherTypeInfoTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/EitherTypeInfoTest.java similarity index 96% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/EitherTypeInfoTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/EitherTypeInfoTest.java index caa3402219d5736d810a0728bd09b32a5fd59a1e..78c10b1270891bc837a0ac8094129acf35688030 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/EitherTypeInfoTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/EitherTypeInfoTest.java @@ -23,7 +23,8 @@ import static org.junit.Assert.assertNotEquals; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.typeutils.Either.Right; +import org.apache.flink.types.Either; +import org.apache.flink.types.Either.Right; import org.apache.flink.util.TestLogger; import org.junit.Test; diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/EnumTypeInfoTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/EnumTypeInfoTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/EnumTypeInfoTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/EnumTypeInfoTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/GenericTypeInfoTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/GenericTypeInfoTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/GenericTypeInfoTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/GenericTypeInfoTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/MissingTypeInfoTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/MissingTypeInfoTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/MissingTypeInfoTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/MissingTypeInfoTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfoTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfoTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfoTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/ObjectArrayTypeInfoTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/PojoTypeExtractionTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeExtractionTest.java similarity index 92% rename from flink-java/src/test/java/org/apache/flink/api/java/type/extractor/PojoTypeExtractionTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeExtractionTest.java index 96ba264475466a9d3afc39cfec5e804943380fe5..bc118488e9aab226dfd2c99fd0a26f42eb5606fc 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/PojoTypeExtractionTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeExtractionTest.java @@ -15,7 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.api.java.type.extractor; + +package org.apache.flink.api.java.typeutils; import java.util.ArrayList; import java.util.Arrays; @@ -28,18 +29,11 @@ import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.CompositeType.FlatFieldDescriptor; -import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.typeutils.GenericTypeInfo; -import org.apache.flink.api.java.typeutils.PojoField; -import org.apache.flink.api.java.typeutils.PojoTypeInfo; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.api.java.typeutils.TypeInfoParser; import org.apache.flink.api.java.typeutils.TypeInfoParserTest.MyWritable; -import org.apache.flink.api.java.typeutils.WritableTypeInfo; + import org.junit.Assert; import org.junit.Test; @@ -559,67 +553,7 @@ public class PojoTypeExtractionTest { } } } - - - public static class Vertex { - - private K key1; - private K key2; - private V value; - - public Vertex() {} - - public Vertex(K key, V value) { - this.key1 = key; - this.key2 = key; - this.value = value; - } - - public Vertex(K key1, K key2, V value) { - this.key1 = key1; - this.key2 = key2; - this.value = value; - } - - public void setKey1(K key1) { - this.key1 = key1; - } - - public void setKey2(K key2) { - this.key2 = key2; - } - - public K getKey1() { - return key1; - } - - public K getKey2() { - return key2; - } - - public void setValue(V value) { - this.value = value; - } - - public V getValue() { - return value; - } - } - - public static class VertexTyped extends Vertex{ - public VertexTyped(Long l, Double d) { - super(l, d); - } - public VertexTyped() { - } - } - - @Test - public void testGetterSetterWithVertex() { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - env.fromElements(new VertexTyped(0L, 3.0), new VertexTyped(1L, 1.0)); - } - + public static class MyMapper implements MapFunction, PojoWithGenerics> { private static final long serialVersionUID = 1L; @@ -636,7 +570,8 @@ public class PojoTypeExtractionTest { MapFunction function = new MyMapper(); TypeInformation ti = TypeExtractor.getMapReturnTypes(function, (TypeInformation) - TypeInfoParser.parse("org.apache.flink.api.java.type.extractor.PojoTypeExtractionTest$PojoWithGenerics")); + TypeInfoParser.parse("org.apache.flink.api.java.typeutils.PojoTypeExtractionTest$PojoWithGenerics")); + Assert.assertTrue(ti instanceof PojoTypeInfo); PojoTypeInfo pti = (PojoTypeInfo) ti; for(int i = 0; i < pti.getArity(); i++) { @@ -710,7 +645,8 @@ public class PojoTypeExtractionTest { MapFunction function = new MyMapper3(); TypeInformation ti = TypeExtractor.getMapReturnTypes(function, (TypeInformation) - TypeInfoParser.parse("org.apache.flink.api.java.type.extractor.PojoTypeExtractionTest$PojoTuple")); + TypeInfoParser.parse("org.apache.flink.api.java.typeutils.PojoTypeExtractionTest$PojoTuple")); + Assert.assertTrue(ti instanceof TupleTypeInfo); TupleTypeInfo tti = (TupleTypeInfo) ti; Assert.assertEquals(BasicTypeInfo.CHAR_TYPE_INFO, tti.getTypeAt(0)); @@ -735,7 +671,7 @@ public class PojoTypeExtractionTest { MapFunction function = new MyMapper4(); TypeInformation ti = TypeExtractor.getMapReturnTypes(function, (TypeInformation) - TypeInfoParser.parse("org.apache.flink.api.java.type.extractor.PojoTypeExtractionTest$PojoWithParameterizedFields1>")); + TypeInfoParser.parse("org.apache.flink.api.java.typeutils.PojoTypeExtractionTest$PojoWithParameterizedFields1>")); Assert.assertEquals(BasicTypeInfo.BYTE_TYPE_INFO, ti); } @@ -757,8 +693,8 @@ public class PojoTypeExtractionTest { MapFunction function = new MyMapper5(); TypeInformation ti = TypeExtractor.getMapReturnTypes(function, (TypeInformation) - TypeInfoParser.parse("org.apache.flink.api.java.type.extractor.PojoTypeExtractionTest$PojoWithParameterizedFields2<" - + "field=org.apache.flink.api.java.type.extractor.PojoTypeExtractionTest$PojoWithGenerics" + TypeInfoParser.parse("org.apache.flink.api.java.typeutils.PojoTypeExtractionTest$PojoWithParameterizedFields2<" + + "field=org.apache.flink.api.java.typeutils.PojoTypeExtractionTest$PojoWithGenerics" + ">")); Assert.assertEquals(BasicTypeInfo.BYTE_TYPE_INFO, ti); } @@ -781,7 +717,7 @@ public class PojoTypeExtractionTest { MapFunction function = new MyMapper6(); TypeInformation ti = TypeExtractor.getMapReturnTypes(function, (TypeInformation) - TypeInfoParser.parse("org.apache.flink.api.java.type.extractor.PojoTypeExtractionTest$PojoWithParameterizedFields3<" + TypeInfoParser.parse("org.apache.flink.api.java.typeutils.PojoTypeExtractionTest$PojoWithParameterizedFields3<" + "field=int[]" + ">")); Assert.assertEquals(BasicTypeInfo.INT_TYPE_INFO, ti); @@ -805,7 +741,7 @@ public class PojoTypeExtractionTest { MapFunction function = new MyMapper7(); TypeInformation ti = TypeExtractor.getMapReturnTypes(function, (TypeInformation) - TypeInfoParser.parse("org.apache.flink.api.java.type.extractor.PojoTypeExtractionTest$PojoWithParameterizedFields4<" + TypeInfoParser.parse("org.apache.flink.api.java.typeutils.PojoTypeExtractionTest$PojoWithParameterizedFields4<" + "field=Tuple1[]" + ">")); Assert.assertEquals(BasicTypeInfo.INT_TYPE_INFO, ti); diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeInfoTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeInfoTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeInfoTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeInfoTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/PojoTypeInformationTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeInformationTest.java similarity index 98% rename from flink-java/src/test/java/org/apache/flink/api/java/type/extractor/PojoTypeInformationTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeInformationTest.java index 7f5ef254cd699f49140d82f71c80a0707421f9e7..51e481d898e89fbf2345ac9f7e5e1c7350991da0 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/PojoTypeInformationTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/PojoTypeInformationTest.java @@ -17,7 +17,7 @@ */ -package org.apache.flink.api.java.type.extractor; +package org.apache.flink.api.java.typeutils; import static org.junit.Assert.assertTrue; diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/TupleTypeInfoTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TupleTypeInfoTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/TupleTypeInfoTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/TupleTypeInfoTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorInputFormatsTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeExtractorInputFormatsTest.java similarity index 96% rename from flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorInputFormatsTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeExtractorInputFormatsTest.java index bbf514886f22fd8b588ab88faacf5a856dfeb322..a606896d2363d190a4d3294790c9642f6f3e944d 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorInputFormatsTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeExtractorInputFormatsTest.java @@ -16,10 +16,7 @@ * limitations under the License. */ -package org.apache.flink.api.java.type.extractor; - - -import static org.junit.Assert.*; +package org.apache.flink.api.java.typeutils; import java.io.IOException; @@ -30,13 +27,13 @@ import org.apache.flink.api.common.io.statistics.BaseStatistics; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.InputSplit; + import org.junit.Test; +import static org.junit.Assert.*; + @SuppressWarnings("serial") public class TypeExtractorInputFormatsTest { diff --git a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeExtractorTest.java similarity index 96% rename from flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeExtractorTest.java index 57b32b123c3ef5dced093c36c2005aa71989cbfc..cdafa6aae31fcc704a375661a5f75926c5f4f41d 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeExtractorTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.api.java.type.extractor; +package org.apache.flink.api.java.typeutils; import java.io.DataInput; import java.io.DataOutput; @@ -38,7 +38,6 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.CompositeType.FlatFieldDescriptor; -import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple0; @@ -46,30 +45,20 @@ import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple9; -import org.apache.flink.api.java.typeutils.Either; -import org.apache.flink.api.java.typeutils.EitherTypeInfo; -import org.apache.flink.api.java.typeutils.EnumTypeInfo; -import org.apache.flink.api.java.typeutils.GenericTypeInfo; -import org.apache.flink.api.java.typeutils.MissingTypeInfo; -import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; -import org.apache.flink.api.java.typeutils.PojoTypeInfo; -import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.api.java.typeutils.TypeExtractor; -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.Either; import org.apache.flink.configuration.Configuration; import org.apache.flink.types.DoubleValue; import org.apache.flink.types.IntValue; import org.apache.flink.types.StringValue; import org.apache.flink.types.Value; import org.apache.flink.util.Collector; + import org.apache.hadoop.io.Writable; + import org.junit.Assert; import org.junit.Test; - +@SuppressWarnings("serial") public class TypeExtractorTest { @@ -97,7 +86,7 @@ public class TypeExtractorTest { Assert.assertEquals(ti, TypeExtractor.getForClass(Boolean.class)); // use getForObject() - Assert.assertEquals(BasicTypeInfo.BOOLEAN_TYPE_INFO, TypeExtractor.getForObject(Boolean.valueOf(true))); + Assert.assertEquals(BasicTypeInfo.BOOLEAN_TYPE_INFO, TypeExtractor.getForObject(true)); } public static class MyWritable implements Writable { @@ -356,7 +345,9 @@ public class TypeExtractorTest { } }; - TypeInformation ti = TypeExtractor.getCrossReturnTypes(function, (TypeInformation) TypeInfoParser.parse("org.apache.flink.api.java.type.extractor.TypeExtractorTest$CustomType"), (TypeInformation) TypeInfoParser.parse("Integer")); + TypeInformation ti = TypeExtractor.getCrossReturnTypes(function, + (TypeInformation) TypeInfoParser.parse("org.apache.flink.api.java.typeutils.TypeExtractorTest$CustomType"), + (TypeInformation) TypeInfoParser.parse("Integer")); Assert.assertFalse(ti.isBasicType()); Assert.assertFalse(ti.isTupleType()); @@ -408,7 +399,8 @@ public class TypeExtractorTest { }; - TypeInformation ti = TypeExtractor.getMapReturnTypes(function, (TypeInformation) TypeInfoParser.parse("Tuple2")); + TypeInformation ti = TypeExtractor.getMapReturnTypes(function, + (TypeInformation) TypeInfoParser.parse("Tuple2")); Assert.assertTrue(ti.isTupleType()); Assert.assertEquals(2, ti.getArity()); @@ -849,25 +841,6 @@ public class TypeExtractorTest { } } - @SuppressWarnings({"unchecked", "rawtypes"}) - @Test - public void testFunctionWithMissingGenericsAndReturns() { - RichMapFunction function = new RichMapFunction() { - private static final long serialVersionUID = 1L; - - @Override - public Object map(Object value) throws Exception { - return null; - } - }; - - TypeInformation info = ExecutionEnvironment.getExecutionEnvironment() - .fromElements("arbitrary", "data") - .map(function).returns("String").getResultType(); - - Assert.assertEquals(TypeInfoParser.parse("String"), info); - } - @SuppressWarnings({ "rawtypes", "unchecked" }) @Test public void testFunctionDependingOnInputAsSuperclass() { @@ -1256,7 +1229,8 @@ public class TypeExtractorTest { } }; - TypeInformation ti = TypeExtractor.getMapReturnTypes(function, (TypeInformation) TypeInfoParser.parse("org.apache.flink.api.java.type.extractor.TypeExtractorTest$CustomArrayObject[]")); + TypeInformation ti = TypeExtractor.getMapReturnTypes(function, + (TypeInformation) TypeInfoParser.parse("org.apache.flink.api.java.typeutils.TypeExtractorTest$CustomArrayObject[]")); Assert.assertTrue(ti instanceof ObjectArrayTypeInfo); Assert.assertEquals(CustomArrayObject.class, ((ObjectArrayTypeInfo) ti).getComponentInfo().getTypeClass()); @@ -1816,11 +1790,13 @@ public class TypeExtractorTest { return null; } }; - ti = TypeExtractor.getMapReturnTypes((MapFunction)function, TypeInfoParser.parse("org.apache.flink.api.java.type.extractor.TypeExtractorTest$CustomType<" - + "myField1=String,myField2=int" - + ">[][][]")); + ti = TypeExtractor.getMapReturnTypes((MapFunction)function, + TypeInfoParser.parse("org.apache.flink.api.java.typeutils.TypeExtractorTest$CustomType<" + + "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/TypeInfoParserTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeInfoParserTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/TypeInfoParserTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeInfoParserTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/ValueTypeInfoTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/ValueTypeInfoTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/ValueTypeInfoTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/ValueTypeInfoTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/WritableTypeInfoTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/WritableTypeInfoTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/WritableTypeInfoTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/WritableTypeInfoTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericArraySerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericArraySerializerTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericArraySerializerTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericArraySerializerTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericTypeComparatorTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericTypeComparatorTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericTypeComparatorTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericTypeComparatorTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericTypeSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericTypeSerializerTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericTypeSerializerTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AbstractGenericTypeSerializerTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericArraySerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericArraySerializerTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericArraySerializerTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericArraySerializerTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericTypeComparatorTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericTypeComparatorTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericTypeComparatorTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericTypeComparatorTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericTypeSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericTypeSerializerTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericTypeSerializerTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroGenericTypeSerializerTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializerEmptyArrayTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializerEmptyArrayTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializerEmptyArrayTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializerEmptyArrayTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueComparatorTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueComparatorTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueComparatorTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueComparatorTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializerTest.java similarity index 95% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializerTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializerTest.java index e4672cccfea332531e50612fccf6bf18cd2f022a..d636d5ecfcd405d1292acd3dd813e78a2a0fd9f2 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializerTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializerTest.java @@ -20,15 +20,15 @@ package org.apache.flink.api.java.typeutils.runtime; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.fail; -import static org.apache.flink.api.java.typeutils.Either.Left; -import static org.apache.flink.api.java.typeutils.Either.Right; +import static org.apache.flink.types.Either.Left; +import static org.apache.flink.types.Either.Right; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeutils.SerializerTestInstance; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.typeutils.Either; +import org.apache.flink.types.Either; import org.apache.flink.api.java.typeutils.EitherTypeInfo; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.junit.Test; diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/GenericPairComparatorTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/GenericPairComparatorTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/GenericPairComparatorTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/GenericPairComparatorTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/MultidimensionalArraySerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/MultidimensionalArraySerializerTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/MultidimensionalArraySerializerTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/MultidimensionalArraySerializerTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoComparatorTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoComparatorTest.java similarity index 97% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoComparatorTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoComparatorTest.java index 1baf443ef8c302283dfa9d120c8ead374cabd429..26b03c6dfeb37f2de9df2b19c86dba39e1acbc9a 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoComparatorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoComparatorTest.java @@ -26,7 +26,7 @@ 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.common.operators.Keys.ExpressionKeys; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.junit.Assert; diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoContainingTuple.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoContainingTuple.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoContainingTuple.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoContainingTuple.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoGenericTypeSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoGenericTypeSerializerTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoGenericTypeSerializerTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoGenericTypeSerializerTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializerTest.java similarity index 98% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializerTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializerTest.java index b1467b931113b25e3a5e935dbab7eac791e37f2e..260f7e982bcd58b86f8ac641107d269c67e2733f 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializerTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializerTest.java @@ -29,8 +29,8 @@ import org.apache.flink.api.common.typeutils.SerializerTestBase; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.CompositeType.FlatFieldDescriptor; -import org.apache.flink.api.java.operators.Keys.ExpressionKeys; -import org.apache.flink.api.java.operators.Keys.IncompatibleKeysException; +import org.apache.flink.api.common.operators.Keys.ExpressionKeys; +import org.apache.flink.api.common.operators.Keys.IncompatibleKeysException; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.typeutils.PojoTypeInfo; diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSubclassComparatorTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSubclassComparatorTest.java similarity index 97% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSubclassComparatorTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSubclassComparatorTest.java index 3a03683a983077e1d8bb5e86e37be97d35f4a486..40c5c8cc6d0087aa77db2af3d931ba20adba9c79 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSubclassComparatorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSubclassComparatorTest.java @@ -24,7 +24,7 @@ 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.common.operators.Keys.ExpressionKeys; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.junit.Assert; diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSubclassSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSubclassSerializerTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSubclassSerializerTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSubclassSerializerTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/StringArrayWritable.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/StringArrayWritable.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/StringArrayWritable.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/StringArrayWritable.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/SubclassFromInterfaceSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/SubclassFromInterfaceSerializerTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/SubclassFromInterfaceSerializerTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/SubclassFromInterfaceSerializerTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TestDataOutputSerializer.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TestDataOutputSerializer.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TestDataOutputSerializer.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TestDataOutputSerializer.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILD2Test.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILD2Test.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILD2Test.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILD2Test.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILD3Test.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILD3Test.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILD3Test.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILD3Test.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDC3Test.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDC3Test.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDC3Test.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDC3Test.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDX1Test.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDX1Test.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDX1Test.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDX1Test.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDXC2Test.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDXC2Test.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDXC2Test.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorILDXC2Test.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD1Test.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD1Test.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD1Test.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD1Test.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD2Test.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD2Test.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD2Test.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD2Test.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD3Test.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD3Test.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD3Test.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorISD3Test.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorTTT1Test.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorTTT1Test.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorTTT1Test.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorTTT1Test.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorTTT2Test.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorTTT2Test.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorTTT2Test.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorTTT2Test.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorTTT3Test.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorTTT3Test.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorTTT3Test.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleComparatorTTT3Test.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTestInstance.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTestInstance.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTestInstance.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerTestInstance.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/ValueComparatorTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/ValueComparatorTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/ValueComparatorTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/ValueComparatorTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/ValueComparatorUUIDTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/ValueComparatorUUIDTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/ValueComparatorUUIDTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/ValueComparatorUUIDTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/ValueID.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/ValueID.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/ValueID.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/ValueID.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializerUUIDTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializerUUIDTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializerUUIDTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializerUUIDTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorUUIDTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorUUIDTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorUUIDTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableComparatorUUIDTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableID.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableID.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableID.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableID.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerUUIDTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerUUIDTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerUUIDTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializerUUIDTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoClearedBufferTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoClearedBufferTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoClearedBufferTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoClearedBufferTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoGenericArraySerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoGenericArraySerializerTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoGenericArraySerializerTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoGenericArraySerializerTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoGenericTypeComparatorTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoGenericTypeComparatorTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoGenericTypeComparatorTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoGenericTypeComparatorTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoGenericTypeSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoGenericTypeSerializerTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoGenericTypeSerializerTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoGenericTypeSerializerTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoWithCustomSerializersTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoWithCustomSerializersTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoWithCustomSerializersTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoWithCustomSerializersTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/SerializersTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/SerializersTest.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/SerializersTest.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/SerializersTest.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/tuple/base/TupleComparatorTestBase.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/tuple/base/TupleComparatorTestBase.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/tuple/base/TupleComparatorTestBase.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/tuple/base/TupleComparatorTestBase.java diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/tuple/base/TuplePairComparatorTestBase.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/tuple/base/TuplePairComparatorTestBase.java similarity index 100% rename from flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/tuple/base/TuplePairComparatorTestBase.java rename to flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/tuple/base/TuplePairComparatorTestBase.java diff --git a/flink-java/pom.xml b/flink-java/pom.xml index a31e89d01083639e173a2e8a8050c3379f2db3d7..3203d752af84e84ef2b30127590a2fb121152ecf 100644 --- a/flink-java/pom.xml +++ b/flink-java/pom.xml @@ -40,22 +40,12 @@ under the License. flink-core ${project.version} + org.apache.flink ${shading-artifact.name} ${project.version} - - - org.apache.avro - avro - - - - - com.esotericsoftware.kryo - kryo - org.ow2.asm @@ -63,12 +53,6 @@ under the License. ${asm.version} - - com.twitter - chill-java - ${chill.version} - - com.google.guava guava @@ -88,20 +72,6 @@ under the License. test-jar test - - - joda-time - joda-time - 2.5 - test - - - - org.joda - joda-convert - 1.7 - test - 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 6db32c5ad5e4a8f395bd77037b0029e9f7175d26..6bcdb52d8c074bae8b194f454368b6876363ac83 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 @@ -63,7 +63,7 @@ import org.apache.flink.api.java.operators.GroupCombineOperator; import org.apache.flink.api.java.operators.GroupReduceOperator; import org.apache.flink.api.java.operators.IterativeDataSet; import org.apache.flink.api.java.operators.JoinOperator.JoinOperatorSets; -import org.apache.flink.api.java.operators.Keys; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.java.operators.MapOperator; import org.apache.flink.api.java.operators.MapPartitionOperator; import org.apache.flink.api.java.operators.PartitionOperator; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/Utils.java b/flink-java/src/main/java/org/apache/flink/api/java/Utils.java index 2edc53339b81b3957fae8030622d24e0f3ed5f1c..80f8199daa6ec1628c7bc4669d811ca760bee8b4 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/Utils.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/Utils.java @@ -33,7 +33,6 @@ import org.apache.flink.configuration.Configuration; import java.io.IOException; import java.lang.reflect.Field; import java.lang.reflect.Modifier; -import java.util.List; import java.util.Random; import static org.apache.flink.api.java.functions.FunctionAnnotation.SkipCodeAnalysis; @@ -61,24 +60,6 @@ public final class Utils { return String.format("%s(%s:%d)", elem.getMethodName(), elem.getFileName(), elem.getLineNumber()); } - /** - * Returns all GenericTypeInfos contained in a composite type. - * - * @param typeInfo {@link CompositeType} - */ - public static void getContainedGenericTypes(CompositeType typeInfo, List> target) { - for (int i = 0; i < typeInfo.getArity(); i++) { - TypeInformation type = typeInfo.getTypeAt(i); - if (type instanceof CompositeType) { - getContainedGenericTypes((CompositeType) type, target); - } else if (type instanceof GenericTypeInfo) { - if (!target.contains(type)) { - target.add((GenericTypeInfo) type); - } - } - } - } - // -------------------------------------------------------------------------------------------- /** diff --git a/flink-java/src/main/java/org/apache/flink/api/java/functions/SemanticPropUtil.java b/flink-java/src/main/java/org/apache/flink/api/java/functions/SemanticPropUtil.java index 761eeb31fe9cee0fd13d49ad125f2d7a95f8219a..a75b8e0eb27f04f96bffa2691527c4b02380ff17 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/functions/SemanticPropUtil.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/functions/SemanticPropUtil.java @@ -37,7 +37,7 @@ import org.apache.flink.api.java.functions.FunctionAnnotation.NonForwardedFields import org.apache.flink.api.java.functions.FunctionAnnotation.ReadFields; import org.apache.flink.api.java.functions.FunctionAnnotation.ReadFieldsFirst; import org.apache.flink.api.java.functions.FunctionAnnotation.ReadFieldsSecond; -import org.apache.flink.api.java.operators.Keys; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import java.lang.annotation.Annotation; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/SplitDataProperties.java b/flink-java/src/main/java/org/apache/flink/api/java/io/SplitDataProperties.java index c44929fd71ce05579ce94863f73093100dbfad08..6763cdfc2ec69d8b2b29e808f65f7b3419b8c289 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/io/SplitDataProperties.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/io/SplitDataProperties.java @@ -25,7 +25,7 @@ import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.common.operators.Ordering; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.operators.DataSource; -import org.apache.flink.api.java.operators.Keys; +import org.apache.flink.api.common.operators.Keys; import java.util.Arrays; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java index 00e0d3bbdc269ddef282269b3200674416c1a007..6485936c9e87d94773dad175ee577926cfc98f77 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java @@ -25,6 +25,7 @@ import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.functions.RichGroupReduceFunction; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.SingleInputSemanticProperties; import org.apache.flink.api.common.operators.UnaryOperatorInformation; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java index 845deb4f6fbc52019970f7f2b5c7e14b18d830f9..6c6b051ed277fdd5044821d052045f2037c2916a 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupOperator.java @@ -30,6 +30,7 @@ import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.DualInputSemanticProperties; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.common.operators.Ordering; @@ -40,9 +41,9 @@ import org.apache.flink.api.java.Utils; import org.apache.flink.api.java.functions.SemanticPropUtil; import org.apache.flink.api.java.operators.DeltaIteration.SolutionSetPlaceHolder; import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.operators.Keys.ExpressionKeys; -import org.apache.flink.api.java.operators.Keys.IncompatibleKeysException; -import org.apache.flink.api.java.operators.Keys.SelectorFunctionKeys; +import org.apache.flink.api.common.operators.Keys.ExpressionKeys; +import org.apache.flink.api.common.operators.Keys.IncompatibleKeysException; +import org.apache.flink.api.common.operators.Keys.SelectorFunctionKeys; import org.apache.flink.api.java.operators.translation.PlanBothUnwrappingCoGroupOperator; import org.apache.flink.api.java.operators.translation.PlanLeftUnwrappingCoGroupOperator; import org.apache.flink.api.java.operators.translation.PlanRightUnwrappingCoGroupOperator; @@ -297,11 +298,11 @@ public class CoGroupOperator extends TwoInputUdfOperator keys2 = (SelectorFunctionKeys) rawKeys2; - final TypeInformation> typeInfoWithKey1 = SelectorFunctionKeys.createTypeWithKey(keys1); - final TypeInformation> typeInfoWithKey2 = SelectorFunctionKeys.createTypeWithKey(keys2); + final TypeInformation> typeInfoWithKey1 = KeyFunctions.createTypeWithKey(keys1); + final TypeInformation> typeInfoWithKey2 = KeyFunctions.createTypeWithKey(keys2); - final Operator> keyedInput1 = SelectorFunctionKeys.appendKeyExtractor(input1, keys1); - final Operator> keyedInput2 = SelectorFunctionKeys.appendKeyExtractor(input2, keys2); + final Operator> keyedInput1 = KeyFunctions.appendKeyExtractor(input1, keys1); + final Operator> keyedInput2 = KeyFunctions.appendKeyExtractor(input2, keys2); final PlanBothUnwrappingCoGroupOperator cogroup = new PlanBothUnwrappingCoGroupOperator<>(function, keys1, keys2, name, outputType, typeInfoWithKey1, typeInfoWithKey2); @@ -324,8 +325,8 @@ public class CoGroupOperator extends TwoInputUdfOperator keys2 = (SelectorFunctionKeys) rawKeys2; - final TypeInformation> typeInfoWithKey2 = SelectorFunctionKeys.createTypeWithKey(keys2); - final Operator> keyedInput2 = SelectorFunctionKeys.appendKeyExtractor(input2, keys2); + final TypeInformation> typeInfoWithKey2 = KeyFunctions.createTypeWithKey(keys2); + final Operator> keyedInput2 = KeyFunctions.appendKeyExtractor(input2, keys2); final PlanRightUnwrappingCoGroupOperator cogroup = new PlanRightUnwrappingCoGroupOperator<>( @@ -355,8 +356,8 @@ public class CoGroupOperator extends TwoInputUdfOperator keys1 = (SelectorFunctionKeys) rawKeys1; - final TypeInformation> typeInfoWithKey1 = SelectorFunctionKeys.createTypeWithKey(keys1); - final Operator> keyedInput1 = SelectorFunctionKeys.appendKeyExtractor(input1, keys1); + final TypeInformation> typeInfoWithKey1 = KeyFunctions.createTypeWithKey(keys1); + final Operator> keyedInput1 = KeyFunctions.appendKeyExtractor(input1, keys1); final PlanLeftUnwrappingCoGroupOperator cogroup = new PlanLeftUnwrappingCoGroupOperator<>( diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupRawOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupRawOperator.java index 30639c302f08443dc1390b811873f936f3ebf9c1..74f54b8e659195171d7d843289aedfb33b1ef9b2 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupRawOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/CoGroupRawOperator.java @@ -21,11 +21,12 @@ import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.operators.BinaryOperatorInformation; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.base.CoGroupRawOperatorBase; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.operators.Keys.IncompatibleKeysException; +import org.apache.flink.api.common.operators.Keys.IncompatibleKeysException; /** * A {@link DataSet} that is the result of a CoGroup transformation. diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java index 915a05376ccf4d15d7f10d1f4e50d21200250483..8745271918ca9d7b404c44cd6dcfeadc7148b3a5 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DataSink.java @@ -21,6 +21,7 @@ package org.apache.flink.api.java.operators; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.io.OutputFormat; import org.apache.flink.api.common.operators.GenericDataSinkBase; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.common.operators.Ordering; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java index a302478bc25ff13409fd88eca94adc331bf18537..7b3001f4f01730290f95e0d921c5b84d3a1adc24 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java @@ -23,6 +23,7 @@ import java.util.Arrays; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.aggregators.Aggregator; import org.apache.flink.api.common.aggregators.AggregatorRegistry; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIterationResultSet.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIterationResultSet.java index 6717c6d2df0583bb01aaf41994d4c58b9b98b3b5..4af910879b68ce2796b9d83f2f93160c640e984f 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIterationResultSet.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIterationResultSet.java @@ -18,6 +18,7 @@ package org.apache.flink.api.java.operators; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java index 5102c801de0fb8d728e36e80fd201b9496e6ea0c..9979f59514ad04266685db8386f0ab7189b5293d 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/DistinctOperator.java @@ -20,12 +20,13 @@ package org.apache.flink.api.java.operators; import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.Operator; 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.typeinfo.TypeInformation; -import org.apache.flink.api.java.operators.Keys.SelectorFunctionKeys; +import org.apache.flink.api.common.operators.Keys.SelectorFunctionKeys; import org.apache.flink.api.java.operators.translation.PlanUnwrappingReduceGroupOperator; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; @@ -116,8 +117,8 @@ public class DistinctOperator extends SingleInputOperator keys = (SelectorFunctionKeys) rawKeys; - TypeInformation> typeInfoWithKey = SelectorFunctionKeys.createTypeWithKey(keys); - Operator> keyedInput = SelectorFunctionKeys.appendKeyExtractor(input, keys); + TypeInformation> typeInfoWithKey = KeyFunctions.createTypeWithKey(keys); + Operator> keyedInput = KeyFunctions.appendKeyExtractor(input, keys); PlanUnwrappingReduceGroupOperator reducer = new PlanUnwrappingReduceGroupOperator<>(function, keys, name, outputType, typeInfoWithKey, true); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupCombineOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupCombineOperator.java index ef0c12f74d3b06aa9ae33fceee15e058a16d308d..a43b8694514a6140d434aad466561f7381958b41 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupCombineOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupCombineOperator.java @@ -19,6 +19,7 @@ package org.apache.flink.api.java.operators; import org.apache.flink.api.common.functions.GroupCombineFunction; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.common.operators.Ordering; @@ -30,7 +31,7 @@ import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.functions.SemanticPropUtil; import org.apache.flink.api.java.operators.translation.PlanUnwrappingGroupCombineOperator; import org.apache.flink.api.java.operators.translation.PlanUnwrappingSortedGroupCombineOperator; -import org.apache.flink.api.java.operators.Keys.SelectorFunctionKeys; +import org.apache.flink.api.common.operators.Keys.SelectorFunctionKeys; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; @@ -195,8 +196,8 @@ public class GroupCombineOperator extends SingleInputUdfOperator keys = (SelectorFunctionKeys) rawKeys; - TypeInformation> typeInfoWithKey = SelectorFunctionKeys.createTypeWithKey(keys); - Operator> keyedInput = SelectorFunctionKeys.appendKeyExtractor(input, keys); + TypeInformation> typeInfoWithKey = KeyFunctions.createTypeWithKey(keys); + Operator> keyedInput = KeyFunctions.appendKeyExtractor(input, keys); PlanUnwrappingGroupCombineOperator reducer = new PlanUnwrappingGroupCombineOperator<>(function, keys, name, outputType, typeInfoWithKey); @@ -217,9 +218,9 @@ public class GroupCombineOperator extends SingleInputUdfOperator groupingKey = (SelectorFunctionKeys) rawGroupingKey; final SelectorFunctionKeys sortingKey = (SelectorFunctionKeys)rawSortingKeys; - TypeInformation> typeInfoWithKey = SelectorFunctionKeys.createTypeWithKey(groupingKey, sortingKey); + TypeInformation> typeInfoWithKey = KeyFunctions.createTypeWithKey(groupingKey, sortingKey); - Operator> inputWithKey = SelectorFunctionKeys.appendKeyExtractor(input, groupingKey, sortingKey); + Operator> inputWithKey = KeyFunctions.appendKeyExtractor(input, groupingKey, sortingKey); PlanUnwrappingSortedGroupCombineOperator reducer = new PlanUnwrappingSortedGroupCombineOperator<>(function, groupingKey, sortingKey, name, outputType, typeInfoWithKey); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java index b1bf8444a9a350d585dbe7773faf6bc6cd8f2e32..42553a008da4995ca2a49910f4a9ac2b7c233c24 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/GroupReduceOperator.java @@ -20,6 +20,7 @@ package org.apache.flink.api.java.operators; import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.common.operators.Ordering; @@ -28,7 +29,7 @@ import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.SemanticPropUtil; -import org.apache.flink.api.java.operators.Keys.SelectorFunctionKeys; +import org.apache.flink.api.common.operators.Keys.SelectorFunctionKeys; import org.apache.flink.api.java.operators.translation.PlanUnwrappingReduceGroupOperator; import org.apache.flink.api.java.operators.translation.PlanUnwrappingSortedReduceGroupOperator; import org.apache.flink.api.java.tuple.Tuple2; @@ -281,9 +282,9 @@ public class GroupReduceOperator extends SingleInputUdfOperator keys = (SelectorFunctionKeys) rawKeys; - TypeInformation> typeInfoWithKey = SelectorFunctionKeys.createTypeWithKey(keys); + TypeInformation> typeInfoWithKey = KeyFunctions.createTypeWithKey(keys); - Operator> keyedInput = SelectorFunctionKeys.appendKeyExtractor(input, keys); + Operator> keyedInput = KeyFunctions.appendKeyExtractor(input, keys); PlanUnwrappingReduceGroupOperator reducer = new PlanUnwrappingReduceGroupOperator(function, keys, name, outputType, typeInfoWithKey, combinable); @@ -305,9 +306,9 @@ public class GroupReduceOperator extends SingleInputUdfOperator groupingKey = (SelectorFunctionKeys) rawGroupingKey; final SelectorFunctionKeys sortingKey = (SelectorFunctionKeys) rawSortingKey; - TypeInformation> typeInfoWithKey = SelectorFunctionKeys.createTypeWithKey(groupingKey,sortingKey); + TypeInformation> typeInfoWithKey = KeyFunctions.createTypeWithKey(groupingKey,sortingKey); - Operator> inputWithKey = SelectorFunctionKeys.appendKeyExtractor(input, groupingKey, sortingKey); + Operator> inputWithKey = KeyFunctions.appendKeyExtractor(input, groupingKey, sortingKey); PlanUnwrappingSortedReduceGroupOperator reducer = new PlanUnwrappingSortedReduceGroupOperator<>( diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java index c1174582b923ac51ec09387ac4a31a52cf7633a9..823aee4188eb722d9895aa750ad3121e6e9e86d5 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/Grouping.java @@ -20,6 +20,7 @@ package org.apache.flink.api.java.operators; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.java.DataSet; /** 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 0c0b7109120a0b26e504cd67eff64c1bbdf1f980..c02d365191cc3ac8f565137dae560b8289ae8dc4 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 @@ -29,6 +29,7 @@ import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.functions.RichFlatJoinFunction; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.DualInputSemanticProperties; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.base.JoinOperatorBase; import org.apache.flink.api.common.operators.base.InnerJoinOperatorBase; @@ -42,9 +43,9 @@ import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsSec import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.functions.SemanticPropUtil; import org.apache.flink.api.java.operators.DeltaIteration.SolutionSetPlaceHolder; -import org.apache.flink.api.java.operators.Keys.ExpressionKeys; -import org.apache.flink.api.java.operators.Keys.IncompatibleKeysException; -import org.apache.flink.api.java.operators.Keys.SelectorFunctionKeys; +import org.apache.flink.api.common.operators.Keys.ExpressionKeys; +import org.apache.flink.api.common.operators.Keys.IncompatibleKeysException; +import org.apache.flink.api.common.operators.Keys.SelectorFunctionKeys; import org.apache.flink.api.java.operators.join.JoinOperatorSetsBase; import org.apache.flink.api.java.operators.join.JoinType; import org.apache.flink.api.java.operators.join.JoinFunctionAssigner; @@ -394,8 +395,8 @@ public abstract class JoinOperator extends TwoInputUdfOperator keys1 = (SelectorFunctionKeys)rawKeys1; - TypeInformation> typeInfoWithKey1 = SelectorFunctionKeys.createTypeWithKey(keys1); - Operator> keyMapper1 = SelectorFunctionKeys.appendKeyExtractor(input1, keys1); + TypeInformation> typeInfoWithKey1 = KeyFunctions.createTypeWithKey(keys1); + Operator> keyMapper1 = KeyFunctions.appendKeyExtractor(input1, keys1); return this.withInput1(keyMapper1, typeInfoWithKey1, rawKeys1); } @@ -406,8 +407,8 @@ public abstract class JoinOperator extends TwoInputUdfOperator keys2 = (SelectorFunctionKeys)rawKeys2; - TypeInformation> typeInfoWithKey2 = SelectorFunctionKeys.createTypeWithKey(keys2); - Operator> keyMapper2 = SelectorFunctionKeys.appendKeyExtractor(input2, keys2); + TypeInformation> typeInfoWithKey2 = KeyFunctions.createTypeWithKey(keys2); + Operator> keyMapper2 = KeyFunctions.appendKeyExtractor(input2, keys2); return withInput2(keyMapper2, typeInfoWithKey2, rawKeys2); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/KeyFunctions.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/KeyFunctions.java new file mode 100644 index 0000000000000000000000000000000000000000..49d598afd9bf505220992bb22a208ea3067c2c4c --- /dev/null +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/KeyFunctions.java @@ -0,0 +1,119 @@ +/* + * 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.operators; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.operators.UnaryOperatorInformation; +import org.apache.flink.api.common.operators.base.MapOperatorBase; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.operators.Keys.SelectorFunctionKeys; +import org.apache.flink.api.java.operators.translation.KeyExtractingMapper; +import org.apache.flink.api.java.operators.translation.KeyRemovingMapper; +import org.apache.flink.api.java.operators.translation.TwoKeyExtractingMapper; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; + +/** + * This class holds static utilities to append functions that extract and + * prune keys. + */ +public class KeyFunctions { + + @SuppressWarnings("unchecked") + public static org.apache.flink.api.common.operators.Operator> appendKeyExtractor( + org.apache.flink.api.common.operators.Operator input, + SelectorFunctionKeys key) + { + + TypeInformation inputType = key.getInputType(); + TypeInformation> typeInfoWithKey = createTypeWithKey(key); + KeyExtractingMapper extractor = new KeyExtractingMapper(key.getKeyExtractor()); + + MapOperatorBase, MapFunction>> mapper = + new MapOperatorBase, MapFunction>>( + extractor, + new UnaryOperatorInformation(inputType, typeInfoWithKey), + "Key Extractor" + ); + + mapper.setInput(input); + mapper.setParallelism(input.getParallelism()); + + return mapper; + } + + @SuppressWarnings("unchecked") + public static org.apache.flink.api.common.operators.Operator> appendKeyExtractor( + org.apache.flink.api.common.operators.Operator input, + SelectorFunctionKeys key1, + SelectorFunctionKeys key2) + { + + TypeInformation inputType = key1.getInputType(); + TypeInformation> typeInfoWithKey = createTypeWithKey(key1, key2); + TwoKeyExtractingMapper extractor = + new TwoKeyExtractingMapper<>(key1.getKeyExtractor(), key2.getKeyExtractor()); + + MapOperatorBase, MapFunction>> mapper = + new MapOperatorBase, MapFunction>>( + extractor, + new UnaryOperatorInformation<>(inputType, typeInfoWithKey), + "Key Extractor" + ); + + mapper.setInput(input); + mapper.setParallelism(input.getParallelism()); + + return mapper; + } + + public static org.apache.flink.api.common.operators.SingleInputOperator appendKeyRemover( + org.apache.flink.api.common.operators.Operator> inputWithKey, + SelectorFunctionKeys key) + { + + TypeInformation inputType = key.getInputType(); + TypeInformation> typeInfoWithKey = createTypeWithKey(key); + + MapOperatorBase, T, MapFunction, T>> mapper = + new MapOperatorBase, T, MapFunction, T>>( + new KeyRemovingMapper(), + new UnaryOperatorInformation<>(typeInfoWithKey, inputType), + "Key Remover" + ); + mapper.setInput(inputWithKey); + mapper.setParallelism(inputWithKey.getParallelism()); + + return mapper; + } + + public static TypeInformation> createTypeWithKey( + SelectorFunctionKeys key) + { + return new TupleTypeInfo<>(key.getKeyType(), key.getInputType()); + } + + public static TypeInformation> createTypeWithKey( + SelectorFunctionKeys key1, + SelectorFunctionKeys key2) + { + return new TupleTypeInfo<>(key1.getKeyType(), key2.getKeyType(), key1.getInputType()); + } +} diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java index 1384ca2f15790b3f541068d6d3f1478fc13a739a..96931b0bdfb0ba5282cae99f861eb49a6f78ba5f 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/PartitionOperator.java @@ -21,13 +21,14 @@ package org.apache.flink.api.java.operators; import com.google.common.base.Preconditions; import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.PartitionOperatorBase; import org.apache.flink.api.common.operators.base.PartitionOperatorBase.PartitionMethod; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.operators.Keys.SelectorFunctionKeys; +import org.apache.flink.api.common.operators.Keys.SelectorFunctionKeys; import org.apache.flink.api.java.tuple.Tuple2; /** @@ -150,9 +151,9 @@ public class PartitionOperator extends SingleInputOperator customPartitioner) { final SelectorFunctionKeys keys = (SelectorFunctionKeys) rawKeys; - TypeInformation> typeInfoWithKey = SelectorFunctionKeys.createTypeWithKey(keys); + TypeInformation> typeInfoWithKey = KeyFunctions.createTypeWithKey(keys); - Operator> keyedInput = SelectorFunctionKeys.appendKeyExtractor(input, keys); + Operator> keyedInput = KeyFunctions.appendKeyExtractor(input, keys); PartitionOperatorBase> keyedPartitionedInput = new PartitionOperatorBase<>(new UnaryOperatorInformation<>(typeInfoWithKey, typeInfoWithKey), pMethod, new int[]{0}, name); @@ -160,7 +161,7 @@ public class PartitionOperator extends SingleInputOperator extends SingleInputUdfOperator keys = (SelectorFunctionKeys) rawKeys; - TypeInformation> typeInfoWithKey = SelectorFunctionKeys.createTypeWithKey(keys); - Operator> keyedInput = SelectorFunctionKeys.appendKeyExtractor(input, keys); + TypeInformation> typeInfoWithKey = KeyFunctions.createTypeWithKey(keys); + Operator> keyedInput = KeyFunctions.appendKeyExtractor(input, keys); PlanUnwrappingReduceOperator reducer = new PlanUnwrappingReduceOperator<>(function, keys, name, inputType, typeInfoWithKey); reducer.setInput(keyedInput); reducer.setParallelism(parallelism); - return SelectorFunctionKeys.appendKeyRemover(reducer, keys); + return KeyFunctions.appendKeyRemover(reducer, keys); } } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortPartitionOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortPartitionOperator.java index d65bc685218f4831610fcbbd4e464687e0a46978..c8a8684d6df6212d1d8fd4b559b7d2d87c7d47fe 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortPartitionOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortPartitionOperator.java @@ -19,6 +19,7 @@ package org.apache.flink.api.java.operators; import org.apache.flink.api.common.InvalidProgramException; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.Operator; import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.common.operators.Ordering; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java index 07d0b9a824ceefdf909a8f9afcf3e5bf591edca3..2453f1b6677909e84dee1681550087b1c6216ea1 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/SortedGrouping.java @@ -19,6 +19,7 @@ package org.apache.flink.api.java.operators; import org.apache.flink.api.common.functions.GroupCombineFunction; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.Ordering; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.Utils; @@ -31,7 +32,7 @@ import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.operators.Keys.ExpressionKeys; +import org.apache.flink.api.common.operators.Keys.ExpressionKeys; import org.apache.flink.api.java.typeutils.TypeExtractor; import com.google.common.base.Preconditions; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperatorUtils.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperatorUtils.java index c55d91994c52754711d3d8c70ae210aeb9560131..6144975c11004b9843258c7bbf57b90978f0276a 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperatorUtils.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperatorUtils.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.CodeAnalysisMode; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.InvalidTypesException; import org.apache.flink.api.common.operators.DualInputSemanticProperties; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.SingleInputSemanticProperties; import org.apache.flink.api.java.functions.FunctionAnnotation; import org.apache.flink.api.java.sca.CodeAnalyzerException; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java index 5b0a36805a48492fe39ee09037c654a656ccfd2a..4e6f6ffd81e8316da221f6e44d6b9ce118a3fa51 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/UnsortedGrouping.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.Order; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSetsBase.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSetsBase.java index e14e06d8ff540892d668eacb9f7f40ab60a81be2..734456cdc63038c36e98a0b05412773c0a13a3ff 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSetsBase.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/join/JoinOperatorSetsBase.java @@ -28,7 +28,7 @@ import org.apache.flink.api.java.Utils; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.operators.JoinOperator.DefaultJoin; import org.apache.flink.api.java.operators.JoinOperator.EquiJoin; -import org.apache.flink.api.java.operators.Keys; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.typeutils.TypeExtractor; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanBothUnwrappingCoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanBothUnwrappingCoGroupOperator.java index fb74d1eadd52b2197bb571dc0a8ce58385db6cc3..f620e11da98e4d6d2b0734b2f21b2eb1fd50b967 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanBothUnwrappingCoGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanBothUnwrappingCoGroupOperator.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.base.CoGroupOperatorBase; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.operators.Keys; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanLeftUnwrappingCoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanLeftUnwrappingCoGroupOperator.java index 2c6273296deb2809e9bdfe5c3d5b173ab737284e..2307c0ce3bec992e7e181bbd37fa2bdfd7c744c8 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanLeftUnwrappingCoGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanLeftUnwrappingCoGroupOperator.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.base.CoGroupOperatorBase; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.operators.Keys; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanRightUnwrappingCoGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanRightUnwrappingCoGroupOperator.java index 56f34cc15461f049ba1768f4dfd7946b3d187257..30e28ebf6fee9c09d803a52a2db6b9294bebbfcf 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanRightUnwrappingCoGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanRightUnwrappingCoGroupOperator.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.operators.base.CoGroupOperatorBase; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.operators.Keys; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingGroupCombineOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingGroupCombineOperator.java index c8e40ceb53d2663d2bbe2e01514ba72a84011141..95b58408d379b630f5be48e81165a4679cda1895 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingGroupCombineOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingGroupCombineOperator.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.GroupCombineOperatorBase; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.operators.Keys; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java index 72b79a44789bf1d2d2b82f63c48522f3670c2cfb..21f15d4970a58647ee915007833f150aa992cdcf 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceGroupOperator.java @@ -23,7 +23,7 @@ import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.operators.Keys; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.util.Collector; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceOperator.java index 2aa8d5463b7b51f4630d65249c7176317a07e0a1..e85bb79758cfff0d78ac371cf268eefde958c5a2 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingReduceOperator.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.ReduceOperatorBase; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.operators.Keys; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.java.tuple.Tuple2; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingSortedGroupCombineOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingSortedGroupCombineOperator.java index e52a5c42bbc1a59bdddf34d4b88db457bfb95121..46773fa5f614e8985e4dbcef9dfa6982366af407 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingSortedGroupCombineOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingSortedGroupCombineOperator.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.GroupCombineOperatorBase; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.operators.Keys; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.util.Collector; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingSortedReduceGroupOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingSortedReduceGroupOperator.java index 278d706659ceb2f2ac8b101cf78185f70fdb8eb8..f0e8055bd8f2e8a306c72ec2f8c26563dffec0d3 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingSortedReduceGroupOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/PlanUnwrappingSortedReduceGroupOperator.java @@ -23,7 +23,7 @@ import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.operators.base.GroupReduceOperatorBase; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.operators.Keys; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.util.Collector; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/sca/UdfAnalyzer.java b/flink-java/src/main/java/org/apache/flink/api/java/sca/UdfAnalyzer.java index 1f4b44a2f830a5a13c1dec06a53b1527c5830e49..93cba33f631b314f783204fc906f3f60cfc3d062 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/sca/UdfAnalyzer.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/sca/UdfAnalyzer.java @@ -32,8 +32,8 @@ import org.apache.flink.api.common.operators.SemanticProperties; import org.apache.flink.api.common.operators.SingleInputSemanticProperties; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.SemanticPropUtil; -import org.apache.flink.api.java.operators.Keys; -import org.apache.flink.api.java.operators.Keys.ExpressionKeys; +import org.apache.flink.api.common.operators.Keys; +import org.apache.flink.api.common.operators.Keys.ExpressionKeys; import org.apache.flink.api.java.sca.TaggedValue.Input; import org.objectweb.asm.Type; import org.objectweb.asm.tree.MethodNode; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/Either.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/Either.java deleted file mode 100644 index 83828318dd94acdfc548a133bad9d72588d59a6d..0000000000000000000000000000000000000000 --- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/Either.java +++ /dev/null @@ -1,185 +0,0 @@ -/* - * 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; - -/** - * This type represents a value of one two possible types, Left or Right (a - * disjoint union), inspired by Scala's Either type. - * - * @param - * the type of Left - * @param - * the type of Right - */ -public abstract class Either { - - /** - * Create a Left value of Either - */ - public static Either Left(L value) { - return new Left(value); - } - - /** - * Create a Right value of Either - */ - public static Either Right(R value) { - return new Right(value); - } - - /** - * Retrieve the Left value of Either. - * - * @return the Left value - * @throws IllegalStateException - * if called on a Right - */ - public abstract L left() throws IllegalStateException; - - /** - * Retrieve the Right value of Either. - * - * @return the Right value - * @throws IllegalStateException - * if called on a Left - */ - public abstract R right() throws IllegalStateException; - - /** - * - * @return true if this is a Left value, false if this is a Right value - */ - public final boolean isLeft() { - return getClass() == Left.class; - } - - /** - * - * @return true if this is a Right value, false if this is a Left value - */ - public final boolean isRight() { - return getClass() == Right.class; - } - - /** - * A left value of {@link Either} - * - * @param - * the type of Left - * @param - * the type of Right - */ - public static class Left extends Either { - private final L value; - - public Left(L value) { - this.value = java.util.Objects.requireNonNull(value); - } - - @Override - public L left() { - return value; - } - - @Override - public R right() { - throw new IllegalStateException("Cannot retrieve Right value on a Left"); - } - - @Override - public boolean equals(Object object) { - if (object instanceof Left) { - final Left other = (Left) object; - return value.equals(other.value); - } - return false; - } - - @Override - public int hashCode() { - return value.hashCode(); - } - - @Override - public String toString() { - return "Left(" + value.toString() + ")"; - } - - /** - * Creates a left value of {@link Either} - * - */ - public static Left of(L left) { - return new Left(left); - } - } - - /** - * A right value of {@link Either} - * - * @param - * the type of Left - * @param - * the type of Right - */ - public static class Right extends Either { - private final R value; - - public Right(R value) { - this.value = java.util.Objects.requireNonNull(value); - } - - @Override - public L left() { - throw new IllegalStateException("Cannot retrieve Left value on a Right"); - } - - @Override - public R right() { - return value; - } - - @Override - public boolean equals(Object object) { - if (object instanceof Right) { - final Right other = (Right) object; - return value.equals(other.value); - } - return false; - } - - @Override - public int hashCode() { - return value.hashCode(); - } - - @Override - public String toString() { - return "Right(" + value.toString() + ")"; - } - - /** - * Creates a right value of {@link Either} - * - */ - public static Right of(R right) { - return new Right(right); - } - } -} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/TypeExtractionTest.java b/flink-java/src/test/java/org/apache/flink/api/java/TypeExtractionTest.java new file mode 100644 index 0000000000000000000000000000000000000000..fb20d7808b20cec7e7fb3fe2ac133fa60369e900 --- /dev/null +++ b/flink-java/src/test/java/org/apache/flink/api/java/TypeExtractionTest.java @@ -0,0 +1,117 @@ +/* + * 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; + +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.TypeInfoParser; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +/** + * Tests concerning type extraction by ExecutionEnvironment methods. + */ +@SuppressWarnings("serial") +public class TypeExtractionTest { + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Test + public void testFunctionWithMissingGenericsAndReturns() { + + RichMapFunction function = new RichMapFunction() { + private static final long serialVersionUID = 1L; + + @Override + public Object map(Object value) throws Exception { + return null; + } + }; + + TypeInformation info = ExecutionEnvironment.getExecutionEnvironment() + .fromElements("arbitrary", "data") + .map(function).returns("String").getResultType(); + + assertEquals(TypeInfoParser.parse("String"), info); + } + + @Test + public void testGetterSetterWithVertex() { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.fromElements(new VertexTyped(0L, 3.0), new VertexTyped(1L, 1.0)); + } + + // ------------------------------------------------------------------------ + // Test types + // ------------------------------------------------------------------------ + + public static class Vertex { + + private K key1; + private K key2; + private V value; + + public Vertex() {} + + public Vertex(K key, V value) { + this.key1 = key; + this.key2 = key; + this.value = value; + } + + public Vertex(K key1, K key2, V value) { + this.key1 = key1; + this.key2 = key2; + this.value = value; + } + + public void setKey1(K key1) { + this.key1 = key1; + } + + public void setKey2(K key2) { + this.key2 = key2; + } + + public K getKey1() { + return key1; + } + + public K getKey2() { + return key2; + } + + public void setValue(V value) { + this.value = value; + } + + public V getValue() { + return value; + } + } + + public static class VertexTyped extends Vertex{ + public VertexTyped(Long l, Double d) { + super(l, d); + } + public VertexTyped() { + } + } +} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/operators/NamesTest.java b/flink-java/src/test/java/org/apache/flink/api/java/operators/NamesTest.java index 5c2c7e9afc8391eb0ddc2fa4408ae6432650d4b9..09a705c38d16f761f5b620065d97ed6a813f62b2 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/operators/NamesTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/operators/NamesTest.java @@ -48,7 +48,7 @@ public class NamesTest implements Serializable { public void testDefaultName() { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet strs = env.fromCollection(Arrays.asList( new String[] {"a", "b"})); + DataSet strs = env.fromCollection(Arrays.asList("a", "b")); // WARNING: The test will fail if this line is being moved down in the file (the line-number is hard-coded) @@ -68,7 +68,7 @@ public class NamesTest implements Serializable { public void testGivenName() { ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - DataSet strs = env.fromCollection(Arrays.asList( new String[] {"a", "b"})); + DataSet strs = env.fromCollection(Arrays.asList("a", "b")); strs.filter(new FilterFunction() { private static final long serialVersionUID = 1L; @Override diff --git a/flink-java/src/test/java/org/apache/flink/api/java/sca/UdfAnalyzerTest.java b/flink-java/src/test/java/org/apache/flink/api/java/sca/UdfAnalyzerTest.java index 8d8f801638ea671b89aa208df6f90140b7a95278..a1ccfe479468618d2e949ac3d94fb8a1abe56d45 100644 --- a/flink-java/src/test/java/org/apache/flink/api/java/sca/UdfAnalyzerTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/java/sca/UdfAnalyzerTest.java @@ -35,7 +35,7 @@ import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFields; import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsFirst; import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsSecond; import org.apache.flink.api.java.functions.SemanticPropUtil; -import org.apache.flink.api.java.operators.Keys; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; diff --git a/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/PythonPlanBinder.java b/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/PythonPlanBinder.java index 1534ebf10b23c8e9f14a9356e1226c94a072b837..0479c0be7609798b2b57dae5f29bbf9cc1059a84 100644 --- a/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/PythonPlanBinder.java +++ b/flink-libraries/flink-python/src/main/java/org/apache/flink/python/api/PythonPlanBinder.java @@ -29,7 +29,7 @@ import org.apache.flink.api.java.operators.AggregateOperator; import org.apache.flink.api.java.operators.CoGroupRawOperator; import org.apache.flink.api.java.operators.CrossOperator.DefaultCross; import org.apache.flink.api.java.operators.Grouping; -import org.apache.flink.api.java.operators.Keys; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.java.operators.SortedGrouping; import org.apache.flink.api.java.operators.UdfOperator; import org.apache.flink.api.java.operators.UnsortedGrouping; diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/JavaBatchTranslator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/JavaBatchTranslator.scala index d25fa9d851f0da22b6701fed7688647511f31292..22be45aab19806f7cc32535ea2779788453afe54 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/JavaBatchTranslator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/JavaBatchTranslator.scala @@ -20,13 +20,14 @@ package org.apache.flink.api.java.table import java.lang.reflect.Modifier +import org.apache.flink.api.common.operators.Keys import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.common.typeutils.CompositeType import org.apache.flink.api.java.aggregation.AggregationFunction import org.apache.flink.api.java.operators.JoinOperator.EquiJoin -import org.apache.flink.api.java.operators.Keys.ExpressionKeys -import org.apache.flink.api.java.operators.{GroupReduceOperator, Keys, MapOperator, UnsortedGrouping} +import Keys.ExpressionKeys +import org.apache.flink.api.java.operators.{GroupReduceOperator, MapOperator, UnsortedGrouping} import org.apache.flink.api.java.{DataSet => JavaDataSet} import org.apache.flink.api.table.expressions.analysis.ExtractEquiJoinFields import org.apache.flink.api.table.plan._ 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 a79c8431c50ea50d1895c4b80769d2cabbe81c8f..1e0a2b3450d1f5501ebc7fda360fc2e50ee382a9 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 @@ -34,7 +34,7 @@ import org.apache.flink.api.java.aggregation.AggregationFunction; import org.apache.flink.api.java.aggregation.AggregationFunctionFactory; import org.apache.flink.api.java.aggregation.Aggregations; import org.apache.flink.api.java.operators.Grouping; -import org.apache.flink.api.java.operators.Keys; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.java.operators.SingleInputOperator; import org.apache.flink.api.java.typeutils.TupleTypeInfoBase; import org.apache.flink.api.java.typeutils.runtime.TupleSerializerBase; diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/CoGroupDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/CoGroupDataSet.scala index 3eb6472be9ae213b477ad3882e755ebae6589c37..a6cce43f84fab8a2837d2477a30749600e0287f1 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/CoGroupDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/CoGroupDataSet.scala @@ -21,10 +21,10 @@ package org.apache.flink.api.scala import org.apache.commons.lang3.tuple.{ImmutablePair, Pair} import org.apache.flink.api.common.InvalidProgramException import org.apache.flink.api.common.functions.{CoGroupFunction, Partitioner, RichCoGroupFunction} -import org.apache.flink.api.common.operators.Order +import org.apache.flink.api.common.operators.{Keys, Order} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.common.typeutils.CompositeType -import org.apache.flink.api.java.operators.Keys.ExpressionKeys +import Keys.ExpressionKeys import org.apache.flink.api.java.operators._ import org.apache.flink.util.Collector diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala index 35f0faf267a5ee56c738ca938d3f433e39b4e400..151e6b30b692347499e2d74f0bdea17694b68e6b 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/DataSet.scala @@ -22,7 +22,7 @@ import org.apache.flink.api.common.accumulators.SerializedListAccumulator import org.apache.flink.api.common.aggregators.Aggregator import org.apache.flink.api.common.functions._ import org.apache.flink.api.common.io.{FileOutputFormat, OutputFormat} -import org.apache.flink.api.common.operators.Order +import org.apache.flink.api.common.operators.{Keys, Order} import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint import org.apache.flink.api.common.operators.base.CrossOperatorBase.CrossHint import org.apache.flink.api.common.operators.base.PartitionOperatorBase.PartitionMethod @@ -31,7 +31,7 @@ import org.apache.flink.api.java.Utils.CountHelper import org.apache.flink.api.java.aggregation.Aggregations import org.apache.flink.api.java.functions.{FirstReducer, KeySelector} import org.apache.flink.api.java.io.{PrintingOutputFormat, TextOutputFormat} -import org.apache.flink.api.java.operators.Keys.ExpressionKeys +import Keys.ExpressionKeys import org.apache.flink.api.java.operators._ import org.apache.flink.api.java.operators.join.JoinType import org.apache.flink.api.java.{DataSet => JavaDataSet, Utils} @@ -93,6 +93,7 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) { /** * Returns the execution environment associated with the current DataSet. + * * @return associated execution environment */ def getExecutionEnvironment: ExecutionEnvironment = @@ -515,7 +516,6 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) { * Convenience method to get the count (number of elements) of a DataSet * * @return A long integer that represents the number of elements in the set - * * @see org.apache.flink.api.java.Utils.CountHelper */ @throws(classOf[Exception]) @@ -531,7 +531,6 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) { * As DataSet can contain a lot of data, this method should be used with caution. * * @return A Seq containing the elements of the DataSet - * * @see org.apache.flink.api.java.Utils.CollectHelper */ @throws(classOf[Exception]) @@ -1369,7 +1368,7 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) { /** * Range-partitions a DataSet on the specified fields. * - '''important:''' This operation requires an extra pass over the DataSet to compute the range + *'''important:''' This operation requires an extra pass over the DataSet to compute the range * boundaries and shuffles the whole DataSet over the network. * This can take significant amount of time. */ @@ -1385,7 +1384,7 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) { /** * Range-partitions a DataSet using the specified key selector function. * - '''important:''' This operation requires an extra pass over the DataSet to compute the range + *'''important:''' This operation requires an extra pass over the DataSet to compute the range * boundaries and shuffles the whole DataSet over the network. * This can take significant amount of time. */ @@ -1516,6 +1515,7 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) { /** * Writes `this` DataSet to the specified location. This uses [[AnyRef.toString]] on * each element. + * * @see org.apache.flink.api.java.DataSet#writeAsText(String) */ def writeAsText( @@ -1532,6 +1532,7 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) { * Writes `this` DataSet to the specified location as CSV file(s). * * This only works on Tuple DataSets. For individual tuple fields [[AnyRef.toString]] is used. + * * @see org.apache.flink.api.java.DataSet#writeAsText(String) */ def writeAsCsv( @@ -1623,8 +1624,8 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) { * * * Writes a DataSet to the standard output stream (stdout) with a sink identifier prefixed. * This uses [[AnyRef.toString]] on each element. - * @param sinkIdentifier The string to prefix the output with. - * + * + * @param sinkIdentifier The string to prefix the output with. * @deprecated Use [[printOnTaskManager(String)]] instead. */ @Deprecated @@ -1636,8 +1637,8 @@ class DataSet[T: ClassTag](set: JavaDataSet[T]) { /** * Writes a DataSet to the standard error stream (stderr) with a sink identifier prefixed. * This uses [[AnyRef.toString]] on each element. - * @param sinkIdentifier The string to prefix the output with. - * + * + * @param sinkIdentifier The string to prefix the output with. * @deprecated Use [[printOnTaskManager(String)]] instead. */ @Deprecated diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala index 7d5419b64ec448cc3e97012806e24bb909ff2e71..bb8287a909f902bd512207c5c9456f8859427cb7 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/GroupedDataSet.scala @@ -19,11 +19,11 @@ package org.apache.flink.api.scala import org.apache.flink.api.common.InvalidProgramException import org.apache.flink.api.common.functions.{GroupCombineFunction, GroupReduceFunction, Partitioner, ReduceFunction} -import org.apache.flink.api.common.operators.Order +import org.apache.flink.api.common.operators.{Keys, Order} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.aggregation.Aggregations import org.apache.flink.api.java.functions.{FirstReducer, KeySelector} -import org.apache.flink.api.java.operators.Keys.ExpressionKeys +import Keys.ExpressionKeys import org.apache.flink.api.java.operators._ import org.apache.flink.api.scala.operators.ScalaAggregateOperator import org.apache.flink.util.Collector 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 91f8c85e01e6dd9b0acb4314ae6bddaa021ff0f6..ace0790dab88b3ee0e61b17f6c721ec94714c2e3 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 @@ -19,6 +19,7 @@ package org.apache.flink.api.scala import org.apache.flink.api.common.functions.CoGroupFunction +import org.apache.flink.api.common.operators.Keys import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.operators._ import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo @@ -36,6 +37,7 @@ import scala.reflect.ClassTag * val right = ... * val coGroupResult = left.coGroup(right).where(...).isEqualTo(...) * }}} + * * @tparam L The type of the left input of the coGroup. * @tparam R The type of the right input of the coGroup. */ 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 49b270134049cd2cabed71b60d1cad48aa27b91d..71f2bfbd4ebe8888d748ebe09d351e016598da6c 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,6 +18,7 @@ package org.apache.flink.api.scala import org.apache.flink.api.common.functions.{FlatJoinFunction, JoinFunction, Partitioner, RichFlatJoinFunction} +import org.apache.flink.api.common.operators.Keys import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.operators.JoinOperator.DefaultJoin.WrappingFlatJoinFunction @@ -256,6 +257,7 @@ private[flink] abstract class UnfinishedJoinOperationBase[L, R, O <: JoinFunctio * val right = ... * val joinResult = left.join(right).where(...).equalTo(...) * }}} + * * @tparam L The type of the left input of the join. * @tparam R The type of the right input of the join. */ @@ -287,6 +289,7 @@ class UnfinishedJoinOperation[L, R]( * (first, second) => ... * } * }}} + * * @tparam L The type of the left input of the join. * @tparam R The type of the right input of the join. */ 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 5db7a91a603790d77289f9c4f2d6cb13564b24fc..e8bc3a4929bd0977bf8ec639e80503ade278bc3b 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 @@ -22,11 +22,12 @@ import java.util import java.util.regex.{Pattern, Matcher} import org.apache.flink.api.common.ExecutionConfig +import org.apache.flink.api.common.operators.Keys import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.common.typeutils.CompositeType.{TypeComparatorBuilder, InvalidFieldReferenceException, FlatFieldDescriptor} import org.apache.flink.api.common.typeutils._ -import org.apache.flink.api.java.operators.Keys.ExpressionKeys +import Keys.ExpressionKeys import org.apache.flink.api.java.typeutils.TupleTypeInfoBase import scala.collection.JavaConverters._ diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala index 08d0242417718f0a97e0c38f7e2e4f4cba01e6bf..ddb45a4b499c225415b0d228c2dc871cdc8c3d3b 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/unfinishedKeyPairOperation.scala @@ -19,10 +19,10 @@ package org.apache.flink.api.scala import org.apache.flink.api.common.InvalidProgramException +import org.apache.flink.api.common.operators.Keys import org.apache.flink.api.java.functions.KeySelector -import org.apache.flink.api.java.operators.Keys -import org.apache.flink.api.java.operators.Keys.ExpressionKeys +import Keys.ExpressionKeys import org.apache.flink.api.common.typeinfo.TypeInformation /** diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java index 254af195d51e493a49d85da5a52f30e5c567ca2b..f4b6e7f19866295d7a29372fe27d82ca6bcc995e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java @@ -37,7 +37,7 @@ import org.apache.flink.api.java.Utils; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.io.CsvOutputFormat; import org.apache.flink.api.java.io.TextOutputFormat; -import org.apache.flink.api.java.operators.Keys; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.typeutils.InputTypeConfigurable; import org.apache.flink.api.java.typeutils.TypeExtractor; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java index afbd8ab0a10ee407dcd2c805bcfb3a25cb0eed77..cf40a3b42138909aefab793f1be37c26bb1d920b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java @@ -29,7 +29,7 @@ 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.java.functions.KeySelector; -import org.apache.flink.api.java.operators.Keys; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.api.java.typeutils.TupleTypeInfo; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java index dd8dec95b5a2198dc5ab37ea4940c7d8434adab0..d0617d084077c5b6bd2699df4cd4daf9c44fb4e8 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java @@ -29,7 +29,7 @@ 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; -import org.apache.flink.api.java.operators.Keys; +import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CoGroupOperatorTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CoGroupOperatorTest.scala index 46059a6c10b2415ef5f5a8d5f72f6d9217517931..ab5ebf58948f5449e4d0e8a0e2ee0322dddf2a28 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CoGroupOperatorTest.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CoGroupOperatorTest.scala @@ -20,7 +20,8 @@ package org.apache.flink.api.scala.operators import java.util import org.apache.flink.api.common.InvalidProgramException -import org.apache.flink.api.java.operators.Keys.IncompatibleKeysException +import org.apache.flink.api.common.operators.Keys +import Keys.IncompatibleKeysException import org.junit.Assert import org.junit.Test import org.apache.flink.api.scala._ diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupCombineITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupCombineITCase.scala index 576ecdfc0655ebd6be970d60785cbafb747efdae..2dabb567c1f834805eb63fd6f592453bbe3c2a9b 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupCombineITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupCombineITCase.scala @@ -19,9 +19,9 @@ package org.apache.flink.api.scala.operators import org.apache.flink.api.common.operators.Order import org.apache.flink.api.java.io.DiscardingOutputFormat -import org.apache.flink.api.scala.{ExecutionEnvironment, _} +import org.apache.flink.api.scala._ import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.test.javaApiOperators.GroupCombineITCase +import org.apache.flink.test.javaApiOperators.GroupCombineITCase.ScalaGroupCombineFunctionExample import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.MultipleProgramsTestBase import org.apache.flink.util.Collector @@ -43,7 +43,7 @@ class GroupCombineITCase(mode: TestExecutionMode) extends MultipleProgramsTestBa .map(str => Tuple1(str)) // all methods on DataSet - ds.combineGroup(new GroupCombineITCase.ScalaGroupCombineFunctionExample()) + ds.combineGroup(new ScalaGroupCombineFunctionExample()) .output(new DiscardingOutputFormat[Tuple1[String]]) ds.combineGroup((in, out: Collector[Tuple1[String]]) => in.toSet foreach (out.collect)) @@ -51,7 +51,7 @@ class GroupCombineITCase(mode: TestExecutionMode) extends MultipleProgramsTestBa // all methods on UnsortedGrouping ds.groupBy(0) - .combineGroup(new GroupCombineITCase.ScalaGroupCombineFunctionExample()) + .combineGroup(new ScalaGroupCombineFunctionExample()) .output(new DiscardingOutputFormat[Tuple1[String]]) ds.groupBy(0) @@ -60,7 +60,7 @@ class GroupCombineITCase(mode: TestExecutionMode) extends MultipleProgramsTestBa // all methods on SortedGrouping ds.groupBy(0).sortGroup(0, Order.ASCENDING) - .combineGroup(new GroupCombineITCase.ScalaGroupCombineFunctionExample()) + .combineGroup(new ScalaGroupCombineFunctionExample()) .output(new DiscardingOutputFormat[Tuple1[String]]) ds.groupBy(0).sortGroup(0, Order.ASCENDING) diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/JoinOperatorTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/JoinOperatorTest.scala index 0d9ea9e1f3d39804593f0515cd3d639b02e432bd..81a7d7e574f6e237980f69f9b902f4a4003d8d46 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/JoinOperatorTest.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/JoinOperatorTest.scala @@ -20,7 +20,8 @@ package org.apache.flink.api.scala.operators import java.util import org.apache.flink.api.common.InvalidProgramException -import org.apache.flink.api.java.operators.Keys.IncompatibleKeysException +import org.apache.flink.api.common.operators.Keys +import Keys.IncompatibleKeysException import org.apache.flink.api.scala._ import org.apache.flink.api.scala.util.CollectionDataSets.CustomType import org.junit.{Assert, Test} 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 70276cb52dd8db549c9f4eafe721532aeb43073e..46495d09740291e1d2087df2efe40676b4394dc5 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 @@ -19,7 +19,7 @@ package org.apache.flink.api.scala.types import java.io.{DataInput, DataOutput} -import org.apache.flink.api.java.`type`.extractor.TypeExtractorTest.CustomTuple +import org.apache.flink.api.java.typeutils.TypeExtractorTest.CustomTuple import org.apache.flink.api.java.io.CollectionInputFormat import org.apache.hadoop.io.Writable import org.junit.{Assert, Test} @@ -602,7 +602,7 @@ class TypeInformationGenTest { // This checks the condition in checkCollection. If this fails with IllegalArgumentException, // then things like "env.fromElements((),(),())" won't work. import scala.collection.JavaConversions._ - CollectionInputFormat.checkCollection(Seq((),(),()), (new UnitTypeInfo).getTypeClass) + CollectionInputFormat.checkCollection(Seq((),(),()), (new UnitTypeInfo).getTypeClass()) } }