You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Jürgen Kreileder (Jira)" <ji...@apache.org> on 2020/01/17 11:19:00 UTC

[jira] [Commented] (FLINK-15634) disableAutoGeneratedUIDs fails with coGroup and join

    [ https://issues.apache.org/jira/browse/FLINK-15634?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17017919#comment-17017919 ] 

Jürgen Kreileder commented on FLINK-15634:
------------------------------------------

The Map operators are added in CoGroupedStreams.WithWindow::apply:
{code:java}
public <T> DataStream<T> apply(CoGroupFunction<T1, T2, T> function, TypeInformation<T> resultType) {
[...]
   DataStream<TaggedUnion<T1, T2>> taggedInput1 = input1
         .map(new Input1Tagger<T1, T2>())
         .setParallelism(input1.getParallelism())
         .returns(unionType);
   DataStream<TaggedUnion<T1, T2>> taggedInput2 = input2
         .map(new Input2Tagger<T1, T2>())
         .setParallelism(input2.getParallelism())
         .returns(unionType);
{code}

> disableAutoGeneratedUIDs fails with coGroup and join
> ----------------------------------------------------
>
>                 Key: FLINK-15634
>                 URL: https://issues.apache.org/jira/browse/FLINK-15634
>             Project: Flink
>          Issue Type: Bug
>          Components: API / DataStream
>    Affects Versions: 1.10.0
>            Reporter: Jürgen Kreileder
>            Priority: Major
>
> coGroup/join seems to generate two Map operators for which you can't set the UID. 
> Here's a test case:
> {code:java}
> @Test
> public void testDisablingAutoUidsWorksWithCoGroup() throws Exception {
>    StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
>    env.getConfig().disableAutoGeneratedUIDs();
>    env
>       .addSource(new NoOpSourceFunction()).setUidHash("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa")
>       .coGroup(env.addSource(new NoOpSourceFunction()).setUidHash("bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb"))
>       .where(o -> o).equalTo(o -> o)
>       .window(TumblingEventTimeWindows.of(Time.days(1)))
>       .with(new CoGroupFunction<String, String, Object>() {
>          @Override
>          public void coGroup(Iterable<String> first, Iterable<String> second, Collector<Object> out) throws Exception {
>          }
>       }).setUidHash("cccccccccccccccccccccccccccccccc")
>       .addSink(new DiscardingSink<>()).setUidHash("dddddddddddddddddddddddddddddddd");
>    env.execute();
> }
> {code}
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)