[ https://issues.apache.org/jira/browse/FLINK-18540?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17156807#comment-17156807 ]
Benchao Li commented on FLINK-18540: ------------------------------------ [~sunwq] Thanks for reporting this. I think this is behavior is by design, if we don't have a retention config, then the aggregate state will never be cleaned, so if the agg result is the same with last one, we emit nothing. However, if we have a retention config, then we should always send these retract/accumulate messages to keep the downstream operator's state updated. > Unnecessary retract messages when setIdleStateRetentionTime before converting > dynamic table to retract stream > ------------------------------------------------------------------------------------------------------------- > > Key: FLINK-18540 > URL: https://issues.apache.org/jira/browse/FLINK-18540 > Project: Flink > Issue Type: Improvement > Components: Table SQL / Planner, Table SQL / Runtime > Affects Versions: 1.10.0 > Reporter: WeiqiangSun > Priority: Critical > > Unnecessary retract messages when setIdleStateRetentionTime before converting > dynamic table to retract stream。 > > {code:java} > final StreamExecutionEnvironment env = > StreamExecutionEnvironment.getExecutionEnvironment(); > env.setParallelism(1); > EnvironmentSettings es = > EnvironmentSettings.newInstance().inStreamingMode().build(); > StreamTableEnvironment fsTableEnv = StreamTableEnvironment.create(env,es); > DataStream<Integer> ds = env.fromElements(5,4,3,2,1); > // if do not setIdleStateRetentionTime, it will output only one message as > expected like '(true,5)' > // if do setIdleStateRetentionTime, it will output unnecessary retract > messages like '(false,5)/(true,5)' > fsTableEnv.getConfig().setIdleStateRetentionTime(Time.hours(1000000),Time.hours(10000000)); > fsTableEnv.createTemporaryView("test",ds,"id"); > Table test = fsTableEnv.sqlQuery("select max(id) from test"); > fsTableEnv.toRetractStream(test,Row.class).printToErr(); > {code} -- This message was sent by Atlassian Jira (v8.3.4#803005)