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

ASF GitHub Bot commented on FLINK-11074:
----------------------------------------

walterddr commented on a change in pull request #7253: [FLINK-11074] 
[table][tests] Enable harness tests with RocksdbStateBackend and add harness 
tests for CollectAggFunction
URL: https://github.com/apache/flink/pull/7253#discussion_r239669171
 
 

 ##########
 File path: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/AggFunctionHarnessTest.scala
 ##########
 @@ -0,0 +1,95 @@
+/*
+ * 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.runtime.harness
+
+import java.lang.{Integer => JInt, Long => JLong}
+import java.util.concurrent.ConcurrentLinkedQueue
+
+import org.apache.flink.api.common.time.Time
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+import org.apache.flink.streaming.api.operators.{AbstractUdfStreamOperator, 
LegacyKeyedProcessOperator}
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
+import org.apache.flink.table.api.dataview.{DataView, MapView}
+import org.apache.flink.table.runtime.aggregate.{GeneratedAggregations, 
GroupAggProcessFunction}
+import 
org.apache.flink.table.runtime.harness.HarnessTestBase.{TestStreamQueryConfig, 
TupleRowKeySelector}
+import org.apache.flink.table.runtime.types.CRow
+import org.junit.Test
+
+import scala.collection.JavaConverters._
+
+class AggFunctionHarnessTest extends HarnessTestBase {
+  protected var queryConfig =
+    new TestStreamQueryConfig(Time.seconds(0), Time.seconds(0))
+
+  @Test
+  def testCollectAggregate(): Unit = {
+    val processFunction = new LegacyKeyedProcessOperator[String, CRow, CRow](
+      new GroupAggProcessFunction(
+        genCollectAggFunction,
+        collectAggregationStateType,
+        false,
+        queryConfig))
+
+    val testHarness = createHarnessTester(
+      processFunction,
+      new TupleRowKeySelector[String](2),
+      BasicTypeInfo.STRING_TYPE_INFO)
+    testHarness.setStateBackend(getStateBackend)
+
+    testHarness.open()
+
+    val state = getState(processFunction, 
"mapView").asInstanceOf[MapView[JInt, JInt]]
+
+    val expectedOutput = new ConcurrentLinkedQueue[Object]()
+
+    testHarness.processElement(new StreamRecord(CRow(1L: JLong, 1: JInt, 
"aaa"), 1))
+    expectedOutput.add(new StreamRecord(CRow("aaa", Map(1 -> 1).asJava), 1))
+    testHarness.processElement(new StreamRecord(CRow(2L: JLong, 1: JInt, 
"bbb"), 1))
+    expectedOutput.add(new StreamRecord(CRow("bbb", Map(1 -> 1).asJava), 1))
+
 
 Review comment:
   can we add something like:
   ```
   
                // do a snapshot & close
                State snapshot = testHarness.snapshot(0L, 0L);
                testHarness.close();
                // reopen and restore
                testHarness = createTestHarness(operator);
                testHarness.setup();
                testHarness.initializeState(snapshot);
                testHarness.open();
   ```
   this will catch some of the weird serialization/deserialization problem as 
well.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Improve the harness test to make it possible test with state backend
> --------------------------------------------------------------------
>
>                 Key: FLINK-11074
>                 URL: https://issues.apache.org/jira/browse/FLINK-11074
>             Project: Flink
>          Issue Type: Test
>          Components: Table API & SQL
>            Reporter: Dian Fu
>            Assignee: Dian Fu
>            Priority: Major
>              Labels: pull-request-available
>
> Currently, the harness test can only test without state backend. If you use a 
> DataView in the accumulator of the aggregate function, the DataView is a java 
> object and held in heap, not replaced with StateMapView/StateListView which 
> values are actually held in the state backend. We should improve the harness 
> test to make it possible to test with state backend. Otherwise, issues such 
> as FLINK-10674 could have never been found. With this harness test available, 
> we could test the built-in aggregate functions which use the DataView more 
> fine grained.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to