[
https://issues.apache.org/jira/browse/FLINK-2336?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17045593#comment-17045593
]
Guowei Ma commented on FLINK-2336:
----------------------------------
The reason why the first test throws IndexOutOfBoundsException is that when
using the input to infer the output type, the type hierarchy array length is 0.
In this case, null should be returned in advance(TypeExtractor.java:956).
The reason why the next two tests did not throw an exception is that the types
of these two lambdas have been erased, which is different from the branch the
first test took.
A simple modification is to add a line in TypeExtractor.java:956 to determine
if the length of inputTypeHierarchy is 0 and return null in advance.
Theoretically speaking, the first type of the lambda is not type erased, so we
could infer the output through Input. In other words, we can build a custom
ParameterizedType with rawtype = MapFunction.class and acutalTypeArguments as
the lambda input and output. However, this requires the introduction of a
custom ParameterizedType, and I feel we can wait until the TypeExtractor is
refactored.
What do you think [~sewen]
> ArrayIndexOufOBoundsException in TypeExtractor when mapping
> -----------------------------------------------------------
>
> Key: FLINK-2336
> URL: https://issues.apache.org/jira/browse/FLINK-2336
> Project: Flink
> Issue Type: Bug
> Components: API / Type Serialization System
> Affects Versions: 0.10.0
> Reporter: William Saar
> Priority: Critical
> Labels: usability
> Fix For: 1.10.1, 1.11.0
>
>
> The line that causes this is
> DataStream<O> outputStream = insideIterationStream.filter(outputFilter).map(m
> -> m.outputMessage);
> Problem occurs both when compiled using Javac and Eclipse's JDT compiler (in
> an environment where simple lambda type tests work)
> Exception in thread "main" java.lang.ArrayIndexOutOfBoundsException: -1
> at java.util.ArrayList.elementData(Unknown Source)
> at java.util.ArrayList.get(Unknown Source)
> at
> org.apache.flink.api.java.typeutils.TypeExtractor.createTypeInfoFromInputs(TypeExtractor.java:553)
> at
> org.apache.flink.api.java.typeutils.TypeExtractor.createTypeInfoWithTypeHierarchy(TypeExtractor.java:468)
> at
> org.apache.flink.api.java.typeutils.TypeExtractor.privateCreateTypeInfo(TypeExtractor.java:370)
> at
> org.apache.flink.api.java.typeutils.TypeExtractor.getUnaryOperatorReturnType(TypeExtractor.java:254)
> at
> org.apache.flink.api.java.typeutils.TypeExtractor.getMapReturnTypes(TypeExtractor.java:91)
> at
> org.apache.flink.streaming.api.datastream.DataStream.map(DataStream.java:605)
--
This message was sent by Atlassian Jira
(v8.3.4#803005)