You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by "Márton Balassi (JIRA)" <ji...@apache.org> on 2015/10/02 17:41:26 UTC

[jira] [Created] (FLINK-2812) KeySelectorUtil.getSelectorForKeys and TypeExtractor.getKeySelectorTypes are incompatible

Márton Balassi created FLINK-2812:
-------------------------------------

             Summary: KeySelectorUtil.getSelectorForKeys and TypeExtractor.getKeySelectorTypes are incompatible
                 Key: FLINK-2812
                 URL: https://issues.apache.org/jira/browse/FLINK-2812
             Project: Flink
          Issue Type: Bug
          Components: Type Serialization System
    Affects Versions: 0.10
            Reporter: Márton Balassi
            Priority: Minor


The following code snippet fails, because {{KeySelectorUtil.getSelectorForKeys}} returns the base {{Tuple}} type.

```java
TypeInformation<Tuple2<Integer, Integer>> typeInfo = TypeExtractor
.getForObject(Tuple2.of(0, 0));

ExecutionConfig config = new ExecutionConfig();

KeySelector<Tuple2<Integer, Integer>, ?> keySelector = KeySelectorUtil.getSelectorForKeys(
new Keys.ExpressionKeys<>(new int[]{0}, typeInfo), typeInfo, config);

// fails with InvalidTypesException
TypeExtractor.getKeySelectorTypes(keySelector, typeInfo); 
```

However if I manually define the key selector as follows the snippet works fine due to the key type being an integer.

```java
KeySelector<Tuple2<Integer, Integer>, Integer> keySelector =

new KeySelector<Tuple2<Integer, Integer>, Integer>() {
	@Override
	public Integer getKey(Tuple2<Integer, Integer> value) throws Exception {
		return value.f0;
	}
};
```

The error message looks like this:
org.apache.flink.api.common.functions.InvalidTypesException: Usage of class Tuple as a type is not allowed. Use a concrete subclass (e.g. Tuple1, Tuple2, etc.) instead.
	at org.apache.flink.api.java.typeutils.TypeExtractor.createTypeInfoWithTypeHierarchy(TypeExtractor.java:401)
	at org.apache.flink.api.java.typeutils.TypeExtractor.privateCreateTypeInfo(TypeExtractor.java:379)
	at org.apache.flink.api.java.typeutils.TypeExtractor.getUnaryOperatorReturnType(TypeExtractor.java:279)
	at org.apache.flink.api.java.typeutils.TypeExtractor.getKeySelectorTypes(TypeExtractor.java:229)
	at org.apache.flink.api.java.typeutils.TypeExtractor.getKeySelectorTypes(TypeExtractor.java:223)




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)