You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ja...@apache.org on 2020/03/17 09:06:16 UTC
[flink] branch release-1.10 updated:
[FLINK-16047][table-planner-blink] Fix Blink planner produces wrong
aggregate results with state clean up
This is an automated email from the ASF dual-hosted git repository.
jark pushed a commit to branch release-1.10
in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/release-1.10 by this push:
new 42efb26 [FLINK-16047][table-planner-blink] Fix Blink planner produces wrong aggregate results with state clean up
42efb26 is described below
commit 42efb26d0b7365d5909f256ea2e491bbd4bb0bf4
Author: Jark Wu <ja...@apache.org>
AuthorDate: Tue Mar 17 16:49:40 2020 +0800
[FLINK-16047][table-planner-blink] Fix Blink planner produces wrong aggregate results with state clean up
This closes #11419
---
.../harness/GroupAggregateHarnessTest.scala | 137 +++++++++++++++++++++
.../operators/aggregate/GroupAggFunction.java | 7 ++
.../table/runtime/util/BaseRowHarnessAssertor.java | 12 +-
.../table/runtime/util/StreamRecordUtils.java | 9 ++
4 files changed, 164 insertions(+), 1 deletion(-)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/harness/GroupAggregateHarnessTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/harness/GroupAggregateHarnessTest.scala
new file mode 100644
index 0000000..b566aba
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/harness/GroupAggregateHarnessTest.scala
@@ -0,0 +1,137 @@
+/*
+ * 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.planner.runtime.harness
+
+import org.apache.flink.api.common.time.Time
+import org.apache.flink.api.scala._
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.scala.internal.StreamTableEnvironmentImpl
+import org.apache.flink.table.api.{EnvironmentSettings, Types}
+import org.apache.flink.table.planner.runtime.utils.StreamingWithStateTestBase.StateBackendMode
+import org.apache.flink.table.runtime.util.BaseRowHarnessAssertor
+import org.apache.flink.table.runtime.util.StreamRecordUtils.{binaryrow, retractBinaryRow}
+import org.apache.flink.types.Row
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+import org.junit.{Before, Test}
+
+import java.lang.{Long => JLong}
+import java.util.concurrent.ConcurrentLinkedQueue
+
+import scala.collection.mutable
+
+@RunWith(classOf[Parameterized])
+class GroupAggregateHarnessTest(mode: StateBackendMode) extends HarnessTestBase(mode) {
+
+ @Before
+ override def before(): Unit = {
+ super.before()
+ val setting = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build()
+ val config = new TestTableConfig
+ this.tEnv = StreamTableEnvironmentImpl.create(env, setting, config)
+ }
+
+ @Test
+ def testAggregateWithRetraction(): Unit = {
+ val data = new mutable.MutableList[(String, String, Long)]
+ val t = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c)
+ tEnv.createTemporaryView("T", t)
+
+ val sql =
+ """
+ |SELECT a, SUM(c)
+ |FROM (
+ | SELECT a, b, SUM(c) as c
+ | FROM T GROUP BY a, b
+ |)GROUP BY a
+ """.stripMargin
+ val t1 = tEnv.sqlQuery(sql)
+
+ val queryConfig = new TestStreamQueryConfig(Time.seconds(2), Time.seconds(3))
+ val testHarness = createHarnessTester(t1.toRetractStream[Row](queryConfig), "GroupAggregate")
+ val assertor = new BaseRowHarnessAssertor(Array( Types.STRING, Types.LONG))
+
+ testHarness.open()
+
+ val expectedOutput = new ConcurrentLinkedQueue[Object]()
+
+ // register cleanup timer with 3001
+ testHarness.setProcessingTime(1)
+
+ // accumulate
+ testHarness.processElement(new StreamRecord(binaryrow("aaa", 1L: JLong), 1))
+ expectedOutput.add(new StreamRecord(binaryrow("aaa", 1L: JLong), 1))
+
+ // accumulate
+ testHarness.processElement(new StreamRecord(binaryrow("bbb", 1L: JLong), 2))
+ expectedOutput.add(new StreamRecord(binaryrow("bbb", 1L: JLong), 2))
+
+ // retract for insertion
+ testHarness.processElement(new StreamRecord(binaryrow("aaa", 2L: JLong), 3))
+ expectedOutput.add(new StreamRecord(retractBinaryRow( "aaa", 1L: JLong), 3))
+ expectedOutput.add(new StreamRecord(binaryrow("aaa", 3L: JLong), 3))
+
+ // retract for deletion
+ testHarness.processElement(new StreamRecord(retractBinaryRow("aaa", 2L: JLong), 3))
+ expectedOutput.add(new StreamRecord(retractBinaryRow("aaa", 3L: JLong), 3))
+ expectedOutput.add(new StreamRecord(binaryrow("aaa", 1L: JLong), 3))
+
+ // accumulate
+ testHarness.processElement(new StreamRecord(binaryrow("ccc", 3L: JLong), 4))
+ expectedOutput.add(new StreamRecord(binaryrow("ccc", 3L: JLong), 4))
+
+ // trigger cleanup timer and register cleanup timer with 6002
+ testHarness.setProcessingTime(3002)
+
+ // retract after clean up
+ testHarness.processElement(new StreamRecord(retractBinaryRow("ccc", 3L: JLong), 4))
+ // not output
+
+ // accumulate
+ testHarness.processElement(new StreamRecord(binaryrow("aaa", 4L: JLong), 5))
+ expectedOutput.add(new StreamRecord(binaryrow("aaa", 4L: JLong), 5))
+ testHarness.processElement(new StreamRecord(binaryrow("bbb", 2L: JLong), 6))
+ expectedOutput.add(new StreamRecord(binaryrow("bbb", 2L: JLong), 6))
+
+ // retract
+ testHarness.processElement(new StreamRecord(binaryrow("aaa", 5L: JLong), 7))
+ expectedOutput.add(new StreamRecord(retractBinaryRow("aaa", 4L: JLong), 7))
+ expectedOutput.add(new StreamRecord(binaryrow("aaa", 9L: JLong), 7))
+
+ // accumulate
+ testHarness.processElement(new StreamRecord(binaryrow("eee", 6L: JLong), 8))
+ expectedOutput.add(new StreamRecord(binaryrow("eee", 6L: JLong), 8))
+
+ // retract
+ testHarness.processElement(new StreamRecord(binaryrow("aaa", 7L: JLong), 9))
+ expectedOutput.add(new StreamRecord(retractBinaryRow("aaa", 9L: JLong), 9))
+ expectedOutput.add(new StreamRecord(binaryrow("aaa", 16L: JLong), 9))
+ testHarness.processElement(new StreamRecord(binaryrow("bbb", 3L: JLong), 10))
+ expectedOutput.add(new StreamRecord(retractBinaryRow("bbb", 2L: JLong), 10))
+ expectedOutput.add(new StreamRecord(binaryrow("bbb", 5L: JLong), 10))
+
+ val result = testHarness.getOutput
+
+ assertor.assertOutputEqualsSorted("result mismatch", expectedOutput, result)
+
+ testHarness.close()
+ }
+
+}
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/GroupAggFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/GroupAggFunction.java
index d7a400b..75d8ef1 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/GroupAggFunction.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/GroupAggFunction.java
@@ -36,6 +36,7 @@ import org.apache.flink.util.Collector;
import static org.apache.flink.table.dataformat.util.BaseRowUtil.ACCUMULATE_MSG;
import static org.apache.flink.table.dataformat.util.BaseRowUtil.RETRACT_MSG;
import static org.apache.flink.table.dataformat.util.BaseRowUtil.isAccumulateMsg;
+import static org.apache.flink.table.dataformat.util.BaseRowUtil.isRetractMsg;
/**
* Aggregate Function used for the groupby (without window) aggregate.
@@ -141,6 +142,12 @@ public class GroupAggFunction extends KeyedProcessFunctionWithCleanupState<BaseR
boolean firstRow;
BaseRow accumulators = accState.value();
if (null == accumulators) {
+ // Don't create a new accumulator for a retraction message. This
+ // might happen if the retraction message is the first message for the
+ // key or after a state clean up.
+ if (isRetractMsg(input)) {
+ return;
+ }
firstRow = true;
accumulators = function.createAccumulators();
} else {
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/BaseRowHarnessAssertor.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/BaseRowHarnessAssertor.java
index df5a5b7..f43c3df 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/BaseRowHarnessAssertor.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/BaseRowHarnessAssertor.java
@@ -105,7 +105,17 @@ public class BaseRowHarnessAssertor {
for (Object ex : expected) {
if (ex instanceof StreamRecord) {
- expectedRecords.add((GenericRow) ((StreamRecord) ex).getValue());
+ BaseRow row = (BaseRow) ((StreamRecord) ex).getValue();
+ if (row instanceof GenericRow) {
+ expectedRecords.add((GenericRow) row);
+ } else {
+ GenericRow genericRow = BaseRowUtil.toGenericRow(
+ row,
+ Arrays.stream(typeInfos)
+ .map(TypeInfoLogicalTypeConverter::fromTypeInfoToLogicalType)
+ .toArray(LogicalType[]::new));
+ expectedRecords.add(genericRow);
+ }
}
}
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/StreamRecordUtils.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/StreamRecordUtils.java
index 2acbcf0..9ac8ec7 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/StreamRecordUtils.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/StreamRecordUtils.java
@@ -120,6 +120,15 @@ public class StreamRecordUtils {
return row;
}
+ /**
+ * Generate a retraction BinaryRow based on the given object fields.
+ */
+ public static BinaryRow retractBinaryRow(Object... fields) {
+ BinaryRow br = binaryrow(fields);
+ BaseRowUtil.setRetract(br);
+ return br;
+ }
+
private StreamRecordUtils() {
// deprecate default constructor
}