Type problem in RichFlatMapFunction when using GenericArray type

classic Classic list List threaded Threaded
7 messages Options
Reply | Threaded
Open this post in threaded view
|

Type problem in RichFlatMapFunction when using GenericArray type

Martin Junghanns
Hi,

I ran into a problem when using generic arrays in a tuple. I wrote a
minimal program to reproduce the error [1].

The problem seems to be related to the order of tuple fields. When
I switch Tuple2<K[], K> to Tuple2<K, K[]> and perform the join on field
0, everything works as expected.

Using Flink 1.1.2.

Cheers,
Martin


[1] https://gist.github.com/s1ck/37aefb19198cd01a8b998fab354c2cfd
Reply | Threaded
Open this post in threaded view
|

Re: Type problem in RichFlatMapFunction when using GenericArray type

Chesnay Schepler-3
Hello Martin,

Could you include the error you are getting?

Regards,
Chesnay

On 10.10.2016 13:31, Martin Junghanns wrote:

> Hi,
>
> I ran into a problem when using generic arrays in a tuple. I wrote a
> minimal program to reproduce the error [1].
>
> The problem seems to be related to the order of tuple fields. When
> I switch Tuple2<K[], K> to Tuple2<K, K[]> and perform the join on
> field 0, everything works as expected.
>
> Using Flink 1.1.2.
>
> Cheers,
> Martin
>
>
> [1] https://gist.github.com/s1ck/37aefb19198cd01a8b998fab354c2cfd
>

Reply | Threaded
Open this post in threaded view
|

Re: Type problem in RichFlatMapFunction when using GenericArray type

Martin Junghanns
Hi Chesnay,

added it to the Gist but also here:

10/10/2016 14:00:44 Job execution switched to status FAILING.
java.lang.ClassCastException: java.lang.Long cannot be cast to
[Ljava.lang.Object;
        at
org.apache.flink.api.common.typeutils.base.GenericArraySerializer.serialize(GenericArraySerializer.java:36)
        at
org.apache.flink.api.common.typeutils.base.GenericArraySerializer.serialize(GenericArraySerializer.java:112)
        at
org.apache.flink.api.common.typeutils.base.GenericArraySerializer.serialize(GenericArraySerializer.java:36)
        at
org.apache.flink.api.java.typeutils.runtime.TupleSerializer.serialize(TupleSerializer.java:124)
        at
org.apache.flink.api.java.typeutils.runtime.TupleSerializer.serialize(TupleSerializer.java:30)
        at
org.apache.flink.runtime.plugable.SerializationDelegate.write(SerializationDelegate.java:56)
        at
org.apache.flink.runtime.io.network.api.serialization.SpanningRecordSerializer.addRecord(SpanningRecordSerializer.java:83)
        at
org.apache.flink.runtime.io.network.api.writer.RecordWriter.emit(RecordWriter.java:85)
        at
org.apache.flink.runtime.operators.shipping.OutputCollector.collect(OutputCollector.java:65)
        at
org.apache.flink.runtime.operators.util.metrics.CountingCollector.collect(CountingCollector.java:35)
        at org.gradoop.examples.TypeProblem$2.join(TypeProblem.java:61)
        at org.gradoop.examples.TypeProblem$2.join(TypeProblem.java:57)
        at
org.apache.flink.runtime.operators.hash.NonReusingBuildFirstHashJoinIterator.callWithNextKey(NonReusingBuildFirstHashJoinIterator.java:149)
        at org.apache.flink.runtime.operators.JoinDriver.run(JoinDriver.java:222)
        at org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:486)
        at org.apache.flink.runtime.operators.BatchTask.invoke(BatchTask.java:351)
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
        at java.lang.Thread.run(Thread.java:745)


On 10.10.2016 13:39, Chesnay Schepler wrote:

> Hello Martin,
>
> Could you include the error you are getting?
>
> Regards,
> Chesnay
>
> On 10.10.2016 13:31, Martin Junghanns wrote:
>> Hi,
>>
>> I ran into a problem when using generic arrays in a tuple. I wrote a
>> minimal program to reproduce the error [1].
>>
>> The problem seems to be related to the order of tuple fields. When
>> I switch Tuple2<K[], K> to Tuple2<K, K[]> and perform the join on
>> field 0, everything works as expected.
>>
>> Using Flink 1.1.2.
>>
>> Cheers,
>> Martin
>>
>>
>> [1] https://gist.github.com/s1ck/37aefb19198cd01a8b998fab354c2cfd
>>
>
Reply | Threaded
Open this post in threaded view
|

Re: Type problem in RichFlatMapFunction when using GenericArray type

Martin Junghanns
In reply to this post by Chesnay Schepler-3
Shall I open an issue for that?

The Exception gets thrown when using
RichFlatJoinFunction or RichFlatMapFunction (updated the Gist)
and the first field of the tuple is an array type.

I can look into it once the issue is there.

Cheers,

Martin


On 10.10.2016 13:39, Chesnay Schepler wrote:

> Hello Martin,
>
> Could you include the error you are getting?
>
> Regards,
> Chesnay
>
> On 10.10.2016 13:31, Martin Junghanns wrote:
>> Hi,
>>
>> I ran into a problem when using generic arrays in a tuple. I wrote a
>> minimal program to reproduce the error [1].
>>
>> The problem seems to be related to the order of tuple fields. When
>> I switch Tuple2<K[], K> to Tuple2<K, K[]> and perform the join on
>> field 0, everything works as expected.
>>
>> Using Flink 1.1.2.
>>
>> Cheers,
>> Martin
>>
>>
>> [1] https://gist.github.com/s1ck/37aefb19198cd01a8b998fab354c2cfd
>>
>

Reply | Threaded
Open this post in threaded view
|

Re: Type problem in RichFlatMapFunction when using GenericArray type

Chesnay Schepler-3
Yes, i think a JIRA issue would be good for this.

On 11.10.2016 08:42, Martin Junghanns wrote:

> Shall I open an issue for that?
>
> The Exception gets thrown when using
> RichFlatJoinFunction or RichFlatMapFunction (updated the Gist)
> and the first field of the tuple is an array type.
>
> I can look into it once the issue is there.
>
> Cheers,
>
> Martin
>
>
> On 10.10.2016 13:39, Chesnay Schepler wrote:
>> Hello Martin,
>>
>> Could you include the error you are getting?
>>
>> Regards,
>> Chesnay
>>
>> On 10.10.2016 13:31, Martin Junghanns wrote:
>>> Hi,
>>>
>>> I ran into a problem when using generic arrays in a tuple. I wrote a
>>> minimal program to reproduce the error [1].
>>>
>>> The problem seems to be related to the order of tuple fields. When
>>> I switch Tuple2<K[], K> to Tuple2<K, K[]> and perform the join on
>>> field 0, everything works as expected.
>>>
>>> Using Flink 1.1.2.
>>>
>>> Cheers,
>>> Martin
>>>
>>>
>>> [1] https://gist.github.com/s1ck/37aefb19198cd01a8b998fab354c2cfd
>>>
>>
>
>

Reply | Threaded
Open this post in threaded view
|

Re: Type problem in RichFlatMapFunction when using GenericArray type

Timo Walther-2
I will also have a look at this issue.

Am 11/10/16 um 09:10 schrieb Chesnay Schepler:

> Yes, i think a JIRA issue would be good for this.
>
> On 11.10.2016 08:42, Martin Junghanns wrote:
>> Shall I open an issue for that?
>>
>> The Exception gets thrown when using
>> RichFlatJoinFunction or RichFlatMapFunction (updated the Gist)
>> and the first field of the tuple is an array type.
>>
>> I can look into it once the issue is there.
>>
>> Cheers,
>>
>> Martin
>>
>>
>> On 10.10.2016 13:39, Chesnay Schepler wrote:
>>> Hello Martin,
>>>
>>> Could you include the error you are getting?
>>>
>>> Regards,
>>> Chesnay
>>>
>>> On 10.10.2016 13:31, Martin Junghanns wrote:
>>>> Hi,
>>>>
>>>> I ran into a problem when using generic arrays in a tuple. I wrote
>>>> a minimal program to reproduce the error [1].
>>>>
>>>> The problem seems to be related to the order of tuple fields. When
>>>> I switch Tuple2<K[], K> to Tuple2<K, K[]> and perform the join on
>>>> field 0, everything works as expected.
>>>>
>>>> Using Flink 1.1.2.
>>>>
>>>> Cheers,
>>>> Martin
>>>>
>>>>
>>>> [1] https://gist.github.com/s1ck/37aefb19198cd01a8b998fab354c2cfd
>>>>
>>>
>>
>>


--
Freundliche Grüße / Kind Regards

Timo Walther

Follow me: @twalthr
https://www.linkedin.com/in/twalthr

Reply | Threaded
Open this post in threaded view
|

Re: Type problem in RichFlatMapFunction when using GenericArray type

Timo Walther-2
I identified the problem and opened a issue for it:
https://issues.apache.org/jira/browse/FLINK-4801


Am 11/10/16 um 15:31 schrieb Timo Walther:

> I will also have a look at this issue.
>
> Am 11/10/16 um 09:10 schrieb Chesnay Schepler:
>> Yes, i think a JIRA issue would be good for this.
>>
>> On 11.10.2016 08:42, Martin Junghanns wrote:
>>> Shall I open an issue for that?
>>>
>>> The Exception gets thrown when using
>>> RichFlatJoinFunction or RichFlatMapFunction (updated the Gist)
>>> and the first field of the tuple is an array type.
>>>
>>> I can look into it once the issue is there.
>>>
>>> Cheers,
>>>
>>> Martin
>>>
>>>
>>> On 10.10.2016 13:39, Chesnay Schepler wrote:
>>>> Hello Martin,
>>>>
>>>> Could you include the error you are getting?
>>>>
>>>> Regards,
>>>> Chesnay
>>>>
>>>> On 10.10.2016 13:31, Martin Junghanns wrote:
>>>>> Hi,
>>>>>
>>>>> I ran into a problem when using generic arrays in a tuple. I wrote
>>>>> a minimal program to reproduce the error [1].
>>>>>
>>>>> The problem seems to be related to the order of tuple fields. When
>>>>> I switch Tuple2<K[], K> to Tuple2<K, K[]> and perform the join on
>>>>> field 0, everything works as expected.
>>>>>
>>>>> Using Flink 1.1.2.
>>>>>
>>>>> Cheers,
>>>>> Martin
>>>>>
>>>>>
>>>>> [1] https://gist.github.com/s1ck/37aefb19198cd01a8b998fab354c2cfd
>>>>>
>>>>
>>>
>>>
>
>


--
Freundliche Grüße / Kind Regards

Timo Walther

Follow me: @twalthr
https://www.linkedin.com/in/twalthr