[ https://issues.apache.org/jira/browse/FLINK-6091?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15972717#comment-15972717 ]
ASF GitHub Bot commented on FLINK-6091: --------------------------------------- Github user lincoln-lil commented on a diff in the pull request: https://github.com/apache/flink/pull/3733#discussion_r111959829 --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/RetractionITCase.scala --- @@ -0,0 +1,310 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.api.scala.stream + +import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment +import org.apache.flink.table.api.{TableEnvironment, TableException} +import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, StreamingWithStateTestBase} +import org.apache.flink.types.Row +import org.junit.Assert._ +import org.junit.Test +import org.apache.flink.table.api.scala._ +import org.apache.flink.api.scala._ +import org.apache.flink.streaming.api.TimeCharacteristic +import org.apache.flink.table.utils.TableFunc0 + +import scala.collection.mutable + +/** + * tests for retraction + */ +class RetractionITCase extends StreamingWithStateTestBase { + // input data + val data = List( + ("Hello", 1), + ("word", 1), + ("Hello", 1), + ("bark", 1) + ) + + // keyed groupby + keyed groupby + @Test + def testWordCount(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + StreamITCase.testResults = mutable.MutableList() + env.setParallelism(1) + env.setStateBackend(getStateBackend) + + val stream = env.fromCollection(data) + val table = stream.toTable(tEnv, 'word, 'num) + val resultTable = table + .groupBy('word) + .select('word as 'word, 'num.sum as 'count) + .groupBy('count) + .select('count, 'word.count as 'frequency) + + val results = resultTable.toDataStream[Row] + results.addSink(new StreamITCase.StringSink) + env.execute() + + val expected = Seq("1,1", "1,2", "1,1", "2,1", "1,2") + assertEquals(expected.sorted, StreamITCase.testResults.sorted) + } + + // keyed groupby + non-keyed groupby + @Test + def testGroupByAndNonKeyedGroupBy(): Unit = { + + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + StreamITCase.testResults = mutable.MutableList() + env.setParallelism(1) + env.setStateBackend(getStateBackend) + + val stream = env.fromCollection(data) + val table = stream.toTable(tEnv, 'word, 'num) + val resultTable = table + .groupBy('word) + .select('word as 'word, 'num.sum as 'count) + .select('count.sum) + + val results = resultTable.toDataStream[Row] + results.addSink(new StreamITCase.StringSink) + env.execute() + + val expected = Seq("1", "2", "1", "3", "4") --- End diff -- Could you explain this example? table.groupBy('word).select('word as 'word, 'num.sum as 'count) I think this query does not produce retraction > Implement and turn on the retraction for aggregates > --------------------------------------------------- > > Key: FLINK-6091 > URL: https://issues.apache.org/jira/browse/FLINK-6091 > Project: Flink > Issue Type: Sub-task > Components: Table API & SQL > Reporter: Shaoxuan Wang > Assignee: Hequn Cheng > > Implement functions for generating and consuming retract messages for > different aggregates. > 1. add delete/add property to Row > 2. implement functions for generating retract messages for unbounded groupBy > 3. implement functions for handling retract messages for different aggregates. > 4. handle retraction messages in CommonCorrelate and CommonCalc (retain > Delete property). > Note: Currently, only unbounded groupby generates retraction and it is > working under unbounded and processing time mode. Hence, retraction is only > supported for unbounded and processing time aggregations so far. We can add > more retraction support later. > supported now: unbounded groupby, unbounded and processing time over window > unsupported now: group window, event time or bounded over window. -- This message was sent by Atlassian JIRA (v6.3.15#6346)