From 2d168a924d5a607343579793b64f621aa15419bd Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Mon, 8 May 2017 18:41:37 +0200 Subject: [PATCH 1/3] [FLINK-6491] [table] Add QueryConfig and state clean up for non-windowed aggregates. --- .../table/api/BatchTableEnvironment.scala | 17 ++- .../apache/flink/table/api/QueryConfig.scala | 102 ++++++++++++++++ .../table/api/StreamTableEnvironment.scala | 51 ++++++-- .../flink/table/api/TableEnvironment.scala | 2 +- .../api/java/StreamTableEnvironment.scala | 115 ++++++++++++++++-- .../api/scala/StreamTableEnvironment.scala | 46 ++++++- .../table/api/scala/TableConversions.scala | 40 +++++- .../org/apache/flink/table/api/table.scala | 26 +++- .../nodes/datastream/DataStreamCalc.scala | 8 +- .../datastream/DataStreamCorrelate.scala | 8 +- .../datastream/DataStreamGroupAggregate.scala | 20 ++- .../DataStreamGroupWindowAggregate.scala | 8 +- .../datastream/DataStreamOverAggregate.scala | 9 +- .../plan/nodes/datastream/DataStreamRel.scala | 7 +- .../nodes/datastream/DataStreamScan.scala | 7 +- .../nodes/datastream/DataStreamUnion.scala | 10 +- .../nodes/datastream/DataStreamValues.scala | 6 +- .../datastream/StreamTableSourceScan.scala | 7 +- .../runtime/aggregate/AggregateUtil.scala | 6 +- .../aggregate/GroupAggProcessFunction.scala | 54 +++++++- .../table/utils/MockTableEnvironment.scala | 9 +- 21 files changed, 494 insertions(+), 64 deletions(-) create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala index 2a3cedf0979a6..f33c187fa04b1 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala @@ -32,7 +32,7 @@ import org.apache.flink.api.java.io.DiscardingOutputFormat import org.apache.flink.api.java.typeutils.GenericTypeInfo import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} import org.apache.flink.table.explain.PlanJsonParser -import org.apache.flink.table.expressions.{Expression, RowtimeAttribute, TimeAttribute} +import org.apache.flink.table.expressions.{Expression, TimeAttribute} import org.apache.flink.table.plan.nodes.FlinkConventions import org.apache.flink.table.plan.nodes.dataset.DataSetRel import org.apache.flink.table.plan.rules.FlinkRuleSets @@ -113,9 +113,20 @@ abstract class BatchTableEnvironment( * * @param table The [[Table]] to write. * @param sink The [[TableSink]] to write the [[Table]] to. + * @param qConfig The configuration for the query to generate. * @tparam T The expected type of the [[DataSet]] which represents the [[Table]]. */ - override private[flink] def writeToSink[T](table: Table, sink: TableSink[T]): Unit = { + override private[flink] def writeToSink[T]( + table: Table, + sink: TableSink[T], + qConfig: QueryConfig): Unit = { + + // We do not pass the configuration on, because there is nothing to configure for batch queries. + val bQConfig = qConfig match { + case batchConfig: BatchQueryConfig => batchConfig + case _ => + throw new TableException("BatchQueryConfig required to configure batch query.") + } sink match { case batchSink: BatchTableSink[T] => @@ -125,7 +136,7 @@ abstract class BatchTableEnvironment( // Give the DataSet to the TableSink to emit it. batchSink.emitDataSet(result) case _ => - throw new TableException("BatchTableSink required to emit batch Table") + throw new TableException("BatchTableSink required to emit batch Table.") } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala new file mode 100644 index 0000000000000..8e8b5ac73c94e --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala @@ -0,0 +1,102 @@ +/* + * 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 + +import _root_.java.io.Serializable +import org.apache.flink.api.common.time.Time + +class QueryConfig private[table] extends Serializable {} + +/** + * The [[BatchQueryConfig]] holds parameters to configure the behavior of batch queries. + */ +class BatchQueryConfig private[table] extends QueryConfig + +/** + * The [[StreamQueryConfig]] holds parameters to configure the behavior of streaming queries. + * + * An empty [[StreamQueryConfig]] can be generated using the [[StreamTableEnvironment.qConf]] + * method. + */ +class StreamQueryConfig private[table] extends QueryConfig { + + /** + * The minimum time until state which was not updated will be retained. + * State might be cleared and removed if it was not updated for the defined period of time. + */ + private var minIdleStateRetentionTime: Long = Long.MinValue + + /** + * The maximum time until state which was not updated will be retained. + * State will be cleared and removed if it was not updated for the defined period of time. + */ + private var maxIdleStateRetentionTime: Long = Long.MinValue + + /** + * Specifies the time interval for how long idle state, i.e., state which was not updated, will + * be retained. When state was not updated for the specified interval of time, it will be cleared + * and removed. + * + * When new data arrives for previously cleaned-up state, the new data will be handled as if it + * was the first data. This can result in previous results being overwritten. + * + * Note: [[setIdleStateRetentionTime(minTime: Time, maxTime: Time)]] allows to set a minimum and + * maximum time for state to be retained. This method is more efficient, because the system has + * to do less bookkeeping to identify the time at which state must be cleared. + * + * @param time The time interval for how long idle state is retained. Set to 0 (zero) to never + * clean-up the state. + */ + def setIdleStateRetentionTime(time: Time): StreamQueryConfig = { + setIdleStateRetentionTime(time, time) + } + + /** + * Specifies a minimum and a maximum time interval for how long idle state, i.e., state which + * was not updated, will be retained. + * State will never be cleared until it was idle for less than the minimum time and will never + * be kept if it was idle for more than the maximum time. + * + * When new data arrives for previously cleaned-up state, the new data will be handled as if it + * was the first data. This can result in previous results being overwritten. + * + * Set to 0 (zero) to never clean-up the state. + * + * @param minTime The minimum time interval for which idle state is retained. Set to 0 (zero) to + * never clean-up the state. + * @param maxTime The maximum time interval for which idle state is retained. May not be smaller + * than than minTime. Set to 0 (zero) to never clean-up the state. + */ + def setIdleStateRetentionTime(minTime: Time, maxTime: Time): StreamQueryConfig = { + if (maxTime.toMilliseconds < minTime.toMilliseconds) { + throw new IllegalArgumentException("maxTime may not be smaller than minTime.") + } + minIdleStateRetentionTime = minTime.toMilliseconds + maxIdleStateRetentionTime = maxTime.toMilliseconds + this + } + + def getMinIdleStateRetentionTime: Long = { + minIdleStateRetentionTime + } + + def getMaxIdleStateRetentionTime: Long = { + maxIdleStateRetentionTime + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala index aef2b1b5a946c..c594d4c300a71 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala @@ -81,6 +81,8 @@ abstract class StreamTableEnvironment( // the naming pattern for internally registered tables. private val internalNamePattern = "^_DataStreamTable_[0-9]+$".r + def qConf: StreamQueryConfig = new StreamQueryConfig + /** * Checks if the chosen table name is valid. * @@ -126,9 +128,20 @@ abstract class StreamTableEnvironment( * * @param table The [[Table]] to write. * @param sink The [[TableSink]] to write the [[Table]] to. + * @param qConfig The configuration for the query to generate. * @tparam T The expected type of the [[DataStream]] which represents the [[Table]]. */ - override private[flink] def writeToSink[T](table: Table, sink: TableSink[T]): Unit = { + override private[flink] def writeToSink[T]( + table: Table, + sink: TableSink[T], + qConfig: QueryConfig): Unit = { + + // Check query configuration + val sQConf = qConfig match { + case streamConfig: StreamQueryConfig => streamConfig + case _ => + throw new TableException("StreamQueryConfig required to configure stream query.") + } sink match { @@ -137,7 +150,7 @@ abstract class StreamTableEnvironment( val outputType = sink.getOutputType // translate the Table into a DataStream and provide the type that the TableSink expects. val result: DataStream[T] = - translate(table, updatesAsRetraction = true, withChangeFlag = true)(outputType) + translate(table, sQConf, updatesAsRetraction = true, withChangeFlag = true)(outputType) // Give the DataStream to the TableSink to emit it. retractSink.asInstanceOf[RetractStreamTableSink[Any]] .emitDataStream(result.asInstanceOf[DataStream[JTuple2[JBool, Any]]]) @@ -160,7 +173,11 @@ abstract class StreamTableEnvironment( val outputType = sink.getOutputType // translate the Table into a DataStream and provide the type that the TableSink expects. val result: DataStream[T] = - translate(optimizedPlan, table.getRelNode.getRowType, withChangeFlag = true)(outputType) + translate( + optimizedPlan, + table.getRelNode.getRowType, + sQConf, + withChangeFlag = true)(outputType) // Give the DataStream to the TableSink to emit it. upsertSink.asInstanceOf[UpsertStreamTableSink[Any]] .emitDataStream(result.asInstanceOf[DataStream[JTuple2[JBool, Any]]]) @@ -176,7 +193,11 @@ abstract class StreamTableEnvironment( val outputType = sink.getOutputType // translate the Table into a DataStream and provide the type that the TableSink expects. val result: DataStream[T] = - translate(optimizedPlan, table.getRelNode.getRowType, withChangeFlag = false)(outputType) + translate( + optimizedPlan, + table.getRelNode.getRowType, + sQConf, + withChangeFlag = false)(outputType) // Give the DataStream to the TableSink to emit it. appendSink.asInstanceOf[AppendStreamTableSink[T]].emitDataStream(result) @@ -545,17 +566,21 @@ abstract class StreamTableEnvironment( * Table API calls and / or SQL queries and generating corresponding [[DataStream]] operators. * * @param table The root node of the relational expression tree. + * @param qConfig The configuration for the query to generate. * @param updatesAsRetraction Set to true to encode updates as retraction messages. * @param withChangeFlag Set to true to emit records with change flags. * @param tpe The [[TypeInformation]] of the resulting [[DataStream]]. * @tparam A The type of the resulting [[DataStream]]. * @return The [[DataStream]] that corresponds to the translated [[Table]]. */ - protected def translate[A](table: Table, updatesAsRetraction: Boolean, withChangeFlag: Boolean) - (implicit tpe: TypeInformation[A]): DataStream[A] = { + protected def translate[A]( + table: Table, + qConfig: StreamQueryConfig, + updatesAsRetraction: Boolean, + withChangeFlag: Boolean)(implicit tpe: TypeInformation[A]): DataStream[A] = { val relNode = table.getRelNode val dataStreamPlan = optimize(relNode, updatesAsRetraction) - translate(dataStreamPlan, relNode.getRowType, withChangeFlag) + translate(dataStreamPlan, relNode.getRowType, qConfig, withChangeFlag) } /** @@ -564,6 +589,7 @@ abstract class StreamTableEnvironment( * @param logicalPlan The root node of the relational expression tree. * @param logicalType The row type of the result. Since the logicalPlan can lose the * field naming during optimization we pass the row type separately. + * @param qConfig The configuration for the query to generate. * @param withChangeFlag Set to true to emit records with change flags. * @param tpe The [[TypeInformation]] of the resulting [[DataStream]]. * @tparam A The type of the resulting [[DataStream]]. @@ -572,6 +598,7 @@ abstract class StreamTableEnvironment( protected def translate[A]( logicalPlan: RelNode, logicalType: RelDataType, + qConfig: StreamQueryConfig, withChangeFlag: Boolean) (implicit tpe: TypeInformation[A]): DataStream[A] = { @@ -583,7 +610,7 @@ abstract class StreamTableEnvironment( } // get CRow plan - val plan: DataStream[CRow] = translateToCRow(logicalPlan) + val plan: DataStream[CRow] = translateToCRow(logicalPlan, qConfig) // convert CRow to output type val conversion = if (withChangeFlag) { @@ -615,14 +642,16 @@ abstract class StreamTableEnvironment( * Translates a logical [[RelNode]] plan into a [[DataStream]] of type [[CRow]]. * * @param logicalPlan The logical plan to translate. + * @param qConfig The configuration for the query to generate. * @return The [[DataStream]] of type [[CRow]]. */ protected def translateToCRow( - logicalPlan: RelNode): DataStream[CRow] = { + logicalPlan: RelNode, + qConfig: StreamQueryConfig): DataStream[CRow] = { logicalPlan match { case node: DataStreamRel => - node.translateToPlan(this) + node.translateToPlan(this, qConfig) case _ => throw TableException("Cannot generate DataStream due to an invalid logical plan. " + "This is a bug and should not happen. Please file an issue.") @@ -638,7 +667,7 @@ abstract class StreamTableEnvironment( def explain(table: Table): String = { val ast = table.getRelNode val optimizedPlan = optimize(ast, updatesAsRetraction = false) - val dataStream = translateToCRow(optimizedPlan) + val dataStream = translateToCRow(optimizedPlan, qConf) val env = dataStream.getExecutionEnvironment val jsonSqlPlan = env.getExecutionPlan diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala index bb0de3e529037..cd3ee671cec91 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala @@ -509,7 +509,7 @@ abstract class TableEnvironment(val config: TableConfig) { * @param sink The [[TableSink]] to write the [[Table]] to. * @tparam T The data type that the [[TableSink]] expects. */ - private[flink] def writeToSink[T](table: Table, sink: TableSink[T]): Unit + private[flink] def writeToSink[T](table: Table, sink: TableSink[T], conf: QueryConfig): Unit /** * Registers a Calcite [[AbstractTable]] in the TableEnvironment's catalog. diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/StreamTableEnvironment.scala index a70bcca79c36f..c3b59511d2954 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/StreamTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/StreamTableEnvironment.scala @@ -150,9 +150,50 @@ class StreamTableEnvironment( * @return The converted [[DataStream]]. */ def toDataStream[T](table: Table, clazz: Class[T]): DataStream[T] = { + toDataStream(table, clazz, qConf) + } + + /** + * Converts the given [[Table]] into an append [[DataStream]] of a specified type. + * + * The [[Table]] must only have insert (append) changes. If the [[Table]] is also modified + * by update or delete changes, the conversion will fail. + * + * The fields of the [[Table]] are mapped to [[DataStream]] fields as follows: + * - [[org.apache.flink.types.Row]] and [[org.apache.flink.api.java.tuple.Tuple]] + * types: Fields are mapped by position, field types must match. + * - POJO [[DataStream]] types: Fields are mapped by field name, field types must match. + * + * @param table The [[Table]] to convert. + * @param typeInfo The [[TypeInformation]] that specifies the type of the [[DataStream]]. + * @tparam T The type of the resulting [[DataStream]]. + * @return The converted [[DataStream]]. + */ + def toDataStream[T](table: Table, typeInfo: TypeInformation[T]): DataStream[T] = { + toDataStream(table, typeInfo, qConf) + } + + /** + * Converts the given [[Table]] into an append [[DataStream]] of a specified type. + * + * The [[Table]] must only have insert (append) changes. If the [[Table]] is also modified + * by update or delete changes, the conversion will fail. + * + * The fields of the [[Table]] are mapped to [[DataStream]] fields as follows: + * - [[org.apache.flink.types.Row]] and [[org.apache.flink.api.java.tuple.Tuple]] + * types: Fields are mapped by position, field types must match. + * - POJO [[DataStream]] types: Fields are mapped by field name, field types must match. + * + * @param table The [[Table]] to convert. + * @param clazz The class of the type of the resulting [[DataStream]]. + * @param qConfig The configuration of the query to generate. + * @tparam T The type of the resulting [[DataStream]]. + * @return The converted [[DataStream]]. + */ + def toDataStream[T](table: Table, clazz: Class[T], qConfig: StreamQueryConfig): DataStream[T] = { val typeInfo = TypeExtractor.createTypeInfo(clazz) TableEnvironment.validateType(typeInfo) - translate[T](table, updatesAsRetraction = false, withChangeFlag = false)(typeInfo) + translate[T](table, qConfig, updatesAsRetraction = false, withChangeFlag = false)(typeInfo) } /** @@ -168,12 +209,64 @@ class StreamTableEnvironment( * * @param table The [[Table]] to convert. * @param typeInfo The [[TypeInformation]] that specifies the type of the [[DataStream]]. + * @param qConfig The configuration of the query to generate. * @tparam T The type of the resulting [[DataStream]]. * @return The converted [[DataStream]]. */ - def toDataStream[T](table: Table, typeInfo: TypeInformation[T]): DataStream[T] = { + def toDataStream[T]( + table: Table, + typeInfo: TypeInformation[T], + qConfig: StreamQueryConfig): DataStream[T] = { TableEnvironment.validateType(typeInfo) - translate[T](table, updatesAsRetraction = false, withChangeFlag = false)(typeInfo) + translate[T](table, qConfig, updatesAsRetraction = false, withChangeFlag = false)(typeInfo) + } + + /** + * Converts the given [[Table]] into a [[DataStream]] of add and retract messages. + * The message will be encoded as [[JTuple2]]. The first field is a [[JBool]] flag, + * the second field holds the record of the specified type [[T]]. + * + * A true [[JBool]] flag indicates an add message, a false flag indicates a retract message. + * + * The fields of the [[Table]] are mapped to the requested type as follows: + * - [[org.apache.flink.types.Row]] and [[org.apache.flink.api.java.tuple.Tuple]] + * types: Fields are mapped by position, field types must match. + * - POJO [[DataStream]] types: Fields are mapped by field name, field types must match. + * + * @param table The [[Table]] to convert. + * @param clazz The class of the requested record type. + * @tparam T The type of the requested record type. + * @return The converted [[DataStream]]. + */ + def toRetractStream[T]( + table: Table, + clazz: Class[T]): DataStream[JTuple2[JBool, T]] = { + + toRetractStream(table, clazz, qConf) + } + + /** + * Converts the given [[Table]] into a [[DataStream]] of add and retract messages. + * The message will be encoded as [[JTuple2]]. The first field is a [[JBool]] flag, + * the second field holds the record of the specified type [[T]]. + * + * A true [[JBool]] flag indicates an add message, a false flag indicates a retract message. + * + * The fields of the [[Table]] are mapped to the requested type as follows: + * - [[org.apache.flink.types.Row]] and [[org.apache.flink.api.java.tuple.Tuple]] + * types: Fields are mapped by position, field types must match. + * - POJO [[DataStream]] types: Fields are mapped by field name, field types must match. + * + * @param table The [[Table]] to convert. + * @param typeInfo The [[TypeInformation]] of the requested record type. + * @tparam T The type of the requested record type. + * @return The converted [[DataStream]]. + */ + def toRetractStream[T]( + table: Table, + typeInfo: TypeInformation[T]): DataStream[JTuple2[JBool, T]] = { + + toRetractStream(table, typeInfo, qConf) } /** @@ -190,17 +283,21 @@ class StreamTableEnvironment( * * @param table The [[Table]] to convert. * @param clazz The class of the requested record type. + * @param qConfig The configuration of the query to generate. * @tparam T The type of the requested record type. * @return The converted [[DataStream]]. */ - def toRetractStream[T](table: Table, clazz: Class[T]): - DataStream[JTuple2[JBool, T]] = { + def toRetractStream[T]( + table: Table, + clazz: Class[T], + qConfig: StreamQueryConfig): DataStream[JTuple2[JBool, T]] = { val typeInfo = TypeExtractor.createTypeInfo(clazz) TableEnvironment.validateType(typeInfo) val resultType = new TupleTypeInfo[JTuple2[JBool, T]](Types.BOOLEAN, typeInfo) translate[JTuple2[JBool, T]]( table, + qConfig, updatesAsRetraction = true, withChangeFlag = true)(resultType) } @@ -219,11 +316,14 @@ class StreamTableEnvironment( * * @param table The [[Table]] to convert. * @param typeInfo The [[TypeInformation]] of the requested record type. + * @param qConfig The configuration of the query to generate. * @tparam T The type of the requested record type. * @return The converted [[DataStream]]. */ - def toRetractStream[T](table: Table, typeInfo: TypeInformation[T]): - DataStream[JTuple2[JBool, T]] = { + def toRetractStream[T]( + table: Table, + typeInfo: TypeInformation[T], + qConfig: StreamQueryConfig): DataStream[JTuple2[JBool, T]] = { TableEnvironment.validateType(typeInfo) val resultTypeInfo = new TupleTypeInfo[JTuple2[JBool, T]]( @@ -232,6 +332,7 @@ class StreamTableEnvironment( ) translate[JTuple2[JBool, T]]( table, + qConfig, updatesAsRetraction = true, withChangeFlag = true)(resultTypeInfo) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala index e5ad6c23e35c5..56f7d55744cf9 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala @@ -19,7 +19,7 @@ package org.apache.flink.table.api.scala import org.apache.flink.api.scala._ import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.table.api.{Table, TableConfig, TableEnvironment} +import org.apache.flink.table.api.{StreamQueryConfig, Table, TableConfig, TableEnvironment} import org.apache.flink.table.expressions.Expression import org.apache.flink.table.functions.{AggregateFunction, TableFunction} import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment} @@ -143,8 +143,29 @@ class StreamTableEnvironment( * @return The converted [[DataStream]]. */ def toDataStream[T: TypeInformation](table: Table): DataStream[T] = { + toDataStream(table, qConf) + } + + /** + * Converts the given [[Table]] into an append [[DataStream]] of a specified type. + * + * The [[Table]] must only have insert (append) changes. If the [[Table]] is also modified + * by update or delete changes, the conversion will fail. + * + * The fields of the [[Table]] are mapped to [[DataStream]] fields as follows: + * - [[org.apache.flink.types.Row]] and Scala Tuple types: Fields are mapped by position, field + * types must match. + * - POJO [[DataStream]] types: Fields are mapped by field name, field types must match. + * + * @param table The [[Table]] to convert. + * @param qConfig The configuration of the query to generate. + * @tparam T The type of the resulting [[DataStream]]. + * @return The converted [[DataStream]]. + */ + def toDataStream[T: TypeInformation](table: Table, qConfig: StreamQueryConfig): DataStream[T] = { val returnType = createTypeInformation[T] - asScalaStream(translate(table, updatesAsRetraction = false, withChangeFlag = false)(returnType)) + asScalaStream( + translate(table, qConfig, updatesAsRetraction = false, withChangeFlag = false)(returnType)) } /** @@ -159,8 +180,27 @@ class StreamTableEnvironment( * @return The converted [[DataStream]]. */ def toRetractStream[T: TypeInformation](table: Table): DataStream[(Boolean, T)] = { + toRetractStream(table, qConf) + } + + /** + * Converts the given [[Table]] into a [[DataStream]] of add and retract messages. + * The message will be encoded as [[Tuple2]]. The first field is a [[Boolean]] flag, + * the second field holds the record of the specified type [[T]]. + * + * A true [[Boolean]] flag indicates an add message, a false flag indicates a retract message. + * + * @param table The [[Table]] to convert. + * @param qConfig The configuration of the query to generate. + * @tparam T The type of the requested data type. + * @return The converted [[DataStream]]. + */ + def toRetractStream[T: TypeInformation]( + table: Table, + qConfig: StreamQueryConfig): DataStream[(Boolean, T)] = { val returnType = createTypeInformation[(Boolean, T)] - asScalaStream(translate(table, updatesAsRetraction = true, withChangeFlag = true)(returnType)) + asScalaStream( + translate(table, qConfig, updatesAsRetraction = true, withChangeFlag = true)(returnType)) } /** diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/TableConversions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/TableConversions.scala index 5efff62599185..966b42f19756d 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/TableConversions.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/TableConversions.scala @@ -21,7 +21,7 @@ package org.apache.flink.table.api.scala import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala._ import org.apache.flink.streaming.api.scala.DataStream -import org.apache.flink.table.api.{Table, TableException} +import org.apache.flink.table.api.{StreamQueryConfig, Table, TableException} import org.apache.flink.table.api.scala.{BatchTableEnvironment => ScalaBatchTableEnv} import org.apache.flink.table.api.scala.{StreamTableEnvironment => ScalaStreamTableEnv} @@ -57,6 +57,21 @@ class TableConversions(table: Table) { } } + /** Converts the [[Table]] to a [[DataStream]] of the specified type. + * + * @param qConfig The configuration for the generated query. + */ + def toDataStream[T: TypeInformation](qConfig: StreamQueryConfig): DataStream[T] = { + table.tableEnv match { + case tEnv: ScalaStreamTableEnv => + tEnv.toDataStream(table, qConfig) + case _ => + throw new TableException( + "Only tables that originate from Scala DataStreams " + + "can be converted to Scala DataStreams.") + } + } + /** Converts the [[Table]] to a [[DataStream]] of add and retract messages. * The message will be encoded as [[Tuple2]]. The first field is a [[Boolean]] flag, * the second field holds the record of the specified type [[T]]. @@ -76,5 +91,28 @@ class TableConversions(table: Table) { } } + /** Converts the [[Table]] to a [[DataStream]] of add and retract messages. + * The message will be encoded as [[Tuple2]]. The first field is a [[Boolean]] flag, + * the second field holds the record of the specified type [[T]]. + * + * A true [[Boolean]] flag indicates an add message, a false flag indicates a retract message. + * + * @param qConfig The configuration for the generated query. + * + */ + def toRetractStream[T: TypeInformation](qConfig: StreamQueryConfig): DataStream[(Boolean, T)] = { + + table.tableEnv match { + case tEnv: ScalaStreamTableEnv => + tEnv.toRetractStream(table, qConfig) + case _ => + throw new TableException( + "Only tables that originate from Scala DataStreams " + + "can be converted to Scala DataStreams.") + } + } + + + } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala index 310a75f3a98a3..5a2eb1c01a849 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala @@ -763,6 +763,30 @@ class Table( * @tparam T The data type that the [[TableSink]] expects. */ def writeToSink[T](sink: TableSink[T]): Unit = { + + def qConfig = this.tableEnv match { + case s: StreamTableEnvironment => s.qConf + case b: BatchTableEnvironment => new BatchQueryConfig + case _ => null + } + + writeToSink(sink, qConfig) + } + + /** + * Writes the [[Table]] to a [[TableSink]]. A [[TableSink]] defines an external storage location. + * + * A batch [[Table]] can only be written to a + * [[org.apache.flink.table.sinks.BatchTableSink]], a streaming [[Table]] requires a + * [[org.apache.flink.table.sinks.AppendStreamTableSink]], a + * [[org.apache.flink.table.sinks.RetractStreamTableSink]], or an + * [[org.apache.flink.table.sinks.UpsertStreamTableSink]]. + * + * @param sink The [[TableSink]] to which the [[Table]] is written. + * @param conf The configuration for the query that writes to the sink. + * @tparam T The data type that the [[TableSink]] expects. + */ + def writeToSink[T](sink: TableSink[T], conf: QueryConfig): Unit = { // get schema information of table val rowType = getRelNode.getRowType val fieldNames: Array[String] = rowType.getFieldNames.asScala.toArray @@ -773,7 +797,7 @@ class Table( val configuredSink = sink.configure(fieldNames, fieldTypes) // emit the table to the configured table sink - tableEnv.writeToSink(this, configuredSink) + tableEnv.writeToSink(this, configuredSink, conf) } /** diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala index ce0f966e9c179..0e377b510aea6 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala @@ -25,7 +25,7 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery import org.apache.calcite.rel.{RelNode, RelWriter} import org.apache.calcite.rex.RexProgram import org.apache.flink.streaming.api.datastream.DataStream -import org.apache.flink.table.api.StreamTableEnvironment +import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment} import org.apache.flink.table.plan.schema.RowSchema import org.apache.flink.table.codegen.CodeGenerator import org.apache.flink.table.plan.nodes.CommonCalc @@ -83,11 +83,13 @@ class DataStreamCalc( estimateRowCount(calcProgram, rowCnt) } - override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = { + override def translateToPlan( + tableEnv: StreamTableEnvironment, + qConfig: StreamQueryConfig): DataStream[CRow] = { val config = tableEnv.getConfig - val inputDataStream = getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv) + val inputDataStream = getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig) val inputRowType = inputDataStream.getType.asInstanceOf[CRowTypeInfo].rowType val generator = new CodeGenerator(config, false, inputRowType) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala index 19ad89b387abd..cbd818a5ecce1 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala @@ -23,7 +23,7 @@ import org.apache.calcite.rex.{RexCall, RexNode} import org.apache.calcite.sql.SemiJoinType import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.streaming.api.datastream.DataStream -import org.apache.flink.table.api.StreamTableEnvironment +import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment} import org.apache.flink.table.functions.utils.TableSqlFunction import org.apache.flink.table.plan.nodes.CommonCorrelate import org.apache.flink.table.plan.nodes.logical.FlinkLogicalTableFunctionScan @@ -82,12 +82,14 @@ class DataStreamCorrelate( .itemIf("condition", condition.orNull, condition.isDefined) } - override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = { + override def translateToPlan( + tableEnv: StreamTableEnvironment, + qConfig: StreamQueryConfig): DataStream[CRow] = { val config = tableEnv.getConfig // we do not need to specify input type - val inputDS = getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv) + val inputDS = getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig) val inputType = inputDS.getType.asInstanceOf[CRowTypeInfo] val funcRel = scan.asInstanceOf[FlinkLogicalTableFunctionScan] diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala index 18f1fc89b76ed..47cdb820ef452 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala @@ -21,9 +21,10 @@ import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.AggregateCall import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel} +import org.apache.flink.api.common.time.Time import org.apache.flink.api.java.functions.NullByteKeySelector import org.apache.flink.streaming.api.datastream.DataStream -import org.apache.flink.table.api.StreamTableEnvironment +import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment} import org.apache.flink.table.codegen.CodeGenerator import org.apache.flink.table.runtime.aggregate._ import org.apache.flink.table.plan.nodes.CommonAggregate @@ -31,6 +32,7 @@ import org.apache.flink.table.plan.schema.RowSchema import org.apache.flink.table.plan.rules.datastream.DataStreamRetractionRules import org.apache.flink.table.runtime.aggregate.AggregateUtil.CalcitePair import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} +import org.slf4j.LoggerFactory /** * @@ -59,6 +61,8 @@ class DataStreamGroupAggregate( with CommonAggregate with DataStreamRel { + private val LOG = LoggerFactory.getLogger(this.getClass) + override def deriveRowType() = schema.logicalType override def needsUpdatesAsRetraction = true @@ -100,9 +104,18 @@ class DataStreamGroupAggregate( inputSchema.logicalType, groupings, getRowType, namedAggregates, Nil)) } - override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = { + override def translateToPlan( + tableEnv: StreamTableEnvironment, + qConfig: StreamQueryConfig): DataStream[CRow] = { + + if (qConfig.getMinIdleStateRetentionTime < 0 || qConfig.getMaxIdleStateRetentionTime < 0) { + LOG.warn( + "No state retention interval configured for a query which accumulates state. " + + "Please provide a query configuration with valid retention interval to prevent excessive " + + "state size. You may specify a retention time of 0 to not clean up the state.") + } - val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv) + val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig) val physicalNamedAggregates = namedAggregates.map { namedAggregate => new CalcitePair[AggregateCall, String]( @@ -136,6 +149,7 @@ class DataStreamGroupAggregate( inputSchema.logicalType, inputSchema.physicalFieldTypeInfo, groupings, + qConfig, DataStreamRetractionRules.isAccRetract(this), DataStreamRetractionRules.isAccRetract(getInput)) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala index 1be1896a96fdf..51c4df0555485 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala @@ -26,7 +26,7 @@ import org.apache.flink.api.java.tuple.Tuple import org.apache.flink.streaming.api.datastream.{AllWindowedStream, DataStream, KeyedStream, WindowedStream} import org.apache.flink.streaming.api.windowing.assigners._ import org.apache.flink.streaming.api.windowing.windows.{Window => DataStreamWindow} -import org.apache.flink.table.api.{StreamTableEnvironment, TableException} +import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment, TableException} import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty import org.apache.flink.table.codegen.CodeGenerator import org.apache.flink.table.expressions.ExpressionUtils._ @@ -107,9 +107,11 @@ class DataStreamGroupWindowAggregate( namedProperties)) } - override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = { + override def translateToPlan( + tableEnv: StreamTableEnvironment, + qConfig: StreamQueryConfig): DataStream[CRow] = { - val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv) + val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig) val physicalNamedAggregates = namedAggregates.map { namedAggregate => new CalcitePair[AggregateCall, String]( diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala index e823cd6a5b9b9..07cbf4803648a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala @@ -26,7 +26,7 @@ import org.apache.calcite.rel.core.{AggregateCall, Window} import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel} import org.apache.calcite.rel.RelFieldCollation.Direction.ASCENDING import org.apache.flink.streaming.api.datastream.DataStream -import org.apache.flink.table.api.{StreamTableEnvironment, TableException} +import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment, TableException} import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.plan.nodes.OverAggregate import org.apache.flink.table.plan.schema.RowSchema @@ -88,7 +88,10 @@ class DataStreamOverAggregate( namedAggregates)) } - override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = { + override def translateToPlan( + tableEnv: StreamTableEnvironment, + qConfig: StreamQueryConfig): DataStream[CRow] = { + if (logicWindow.groups.size > 1) { throw new TableException( "Unsupported use of OVER windows. All aggregates must be computed on the same window.") @@ -109,7 +112,7 @@ class DataStreamOverAggregate( "Unsupported use of OVER windows. The window can only be ordered in ASCENDING mode.") } - val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv) + val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig) val consumeRetraction = DataStreamRetractionRules.isAccRetract(input) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala index 9754de4461df8..6f6edf7b5de7f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala @@ -19,7 +19,7 @@ package org.apache.flink.table.plan.nodes.datastream import org.apache.flink.streaming.api.datastream.DataStream -import org.apache.flink.table.api.StreamTableEnvironment +import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment} import org.apache.flink.table.plan.nodes.FlinkRelNode import org.apache.flink.table.runtime.types.CRow @@ -29,9 +29,12 @@ trait DataStreamRel extends FlinkRelNode { * Translates the FlinkRelNode into a Flink operator. * * @param tableEnv The [[StreamTableEnvironment]] of the translated Table. + * @param qConfig The configuration for the query to generate. * @return DataStream of type [[CRow]] */ - def translateToPlan(tableEnv: StreamTableEnvironment) : DataStream[CRow] + def translateToPlan( + tableEnv: StreamTableEnvironment, + qConfig: StreamQueryConfig): DataStream[CRow] /** * Whether the [[DataStreamRel]] requires that update and delete changes are sent with retraction diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamScan.scala index c613646f9cff2..e64bf0fe1c389 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamScan.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamScan.scala @@ -23,7 +23,7 @@ import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.TableScan import org.apache.flink.streaming.api.datastream.DataStream -import org.apache.flink.table.api.StreamTableEnvironment +import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment} import org.apache.flink.table.plan.schema.RowSchema import org.apache.flink.table.plan.schema.DataStreamTable import org.apache.flink.table.runtime.types.CRow @@ -54,7 +54,10 @@ class DataStreamScan( ) } - override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = { + override def translateToPlan( + tableEnv: StreamTableEnvironment, + qConfig: StreamQueryConfig): DataStream[CRow] = { + val config = tableEnv.getConfig val inputDataStream: DataStream[Any] = dataStreamTable.dataStream convertToInternalRow(schema, inputDataStream, dataStreamTable, config) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala index 654c259f637c9..6cc739603f43f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala @@ -21,7 +21,7 @@ package org.apache.flink.table.plan.nodes.datastream import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.{BiRel, RelNode, RelWriter} import org.apache.flink.streaming.api.datastream.DataStream -import org.apache.flink.table.api.StreamTableEnvironment +import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment} import org.apache.flink.table.plan.schema.RowSchema import org.apache.flink.table.runtime.types.CRow @@ -58,10 +58,12 @@ class DataStreamUnion( s"Union All(union: (${schema.logicalFieldNames.mkString(", ")}))" } - override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = { + override def translateToPlan( + tableEnv: StreamTableEnvironment, + qConfig: StreamQueryConfig): DataStream[CRow] = { - val leftDataSet = left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv) - val rightDataSet = right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv) + val leftDataSet = left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig) + val rightDataSet = right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig) leftDataSet.union(rightDataSet) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamValues.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamValues.scala index 32c9aaf0e3d20..ba6b0257a8812 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamValues.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamValues.scala @@ -24,7 +24,7 @@ import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.core.Values import org.apache.calcite.rex.RexLiteral import org.apache.flink.streaming.api.datastream.DataStream -import org.apache.flink.table.api.StreamTableEnvironment +import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment} import org.apache.flink.table.codegen.CodeGenerator import org.apache.flink.table.plan.schema.RowSchema import org.apache.flink.table.runtime.io.CRowValuesInputFormat @@ -56,7 +56,9 @@ class DataStreamValues( ) } - override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = { + override def translateToPlan( + tableEnv: StreamTableEnvironment, + qConfig: StreamQueryConfig): DataStream[CRow] = { val config = tableEnv.getConfig diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala index b2d7019ae0fd6..225f23f172c91 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala @@ -22,7 +22,7 @@ import org.apache.calcite.plan._ import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.metadata.RelMetadataQuery import org.apache.flink.streaming.api.datastream.DataStream -import org.apache.flink.table.api.{StreamTableEnvironment, TableEnvironment} +import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment, TableEnvironment} import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.plan.nodes.PhysicalTableSourceScan import org.apache.flink.table.plan.schema.RowSchema @@ -98,7 +98,10 @@ class StreamTableSourceScan( ) } - override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = { + override def translateToPlan( + tableEnv: StreamTableEnvironment, + qConfig: StreamQueryConfig): DataStream[CRow] = { + val config = tableEnv.getConfig val inputDataStream = tableSource.getDataStream(tableEnv.execEnv).asInstanceOf[DataStream[Any]] convertToInternalRow( diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala index 768c9cbb482f0..27392c72b8d98 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala @@ -33,7 +33,7 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.streaming.api.functions.windowing.{AllWindowFunction, WindowFunction} import org.apache.flink.streaming.api.windowing.windows.{Window => DataStreamWindow} -import org.apache.flink.table.api.TableException +import org.apache.flink.table.api.{StreamQueryConfig, TableException} import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.codegen.CodeGenerator @@ -155,6 +155,7 @@ object AggregateUtil { inputRowType: RelDataType, inputFieldTypes: Seq[TypeInformation[_]], groupings: Array[Int], + qConfig: StreamQueryConfig, generateRetraction: Boolean, consumeRetraction: Boolean): ProcessFunction[CRow, CRow] = { @@ -190,7 +191,8 @@ object AggregateUtil { new GroupAggProcessFunction( genFunction, aggregationStateType, - generateRetraction) + generateRetraction, + qConfig) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala index 6ee37e62f1655..84fee8759f4a8 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala @@ -26,9 +26,9 @@ import org.apache.flink.util.Collector import org.apache.flink.api.common.state.ValueStateDescriptor import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.common.state.ValueState -import org.apache.flink.table.api.Types +import org.apache.flink.table.api.{StreamQueryConfig, Types} import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} -import org.slf4j.LoggerFactory +import org.slf4j.{Logger, LoggerFactory} import org.apache.flink.table.runtime.types.CRow /** @@ -40,13 +40,20 @@ import org.apache.flink.table.runtime.types.CRow class GroupAggProcessFunction( private val genAggregations: GeneratedAggregationsFunction, private val aggregationStateType: RowTypeInfo, - private val generateRetraction: Boolean) + private val generateRetraction: Boolean, + private val qConfig: StreamQueryConfig) extends ProcessFunction[CRow, CRow] with Compiler[GeneratedAggregations] { - val LOG = LoggerFactory.getLogger(this.getClass) + val LOG: Logger = LoggerFactory.getLogger(this.getClass) private var function: GeneratedAggregations = _ + private val minRetentionTime = qConfig.getMinIdleStateRetentionTime + private val maxRetentionTime = qConfig.getMaxIdleStateRetentionTime + private val stateCleaningEnabled = minRetentionTime > 1 && maxRetentionTime > 1 + // interval in which clean-up timers are registered + private val cleanupTimerInterval = maxRetentionTime - minRetentionTime + private var newRow: CRow = _ private var prevRow: CRow = _ private var firstRow: Boolean = _ @@ -54,6 +61,8 @@ class GroupAggProcessFunction( private var state: ValueState[Row] = _ // counts the number of added and retracted input records private var cntState: ValueState[JLong] = _ + // holds the latest registered cleanup timer + private var cleanupTimeState: ValueState[JLong] = _ override def open(config: Configuration) { LOG.debug(s"Compiling AggregateHelper: $genAggregations.name \n\n " + @@ -74,6 +83,12 @@ class GroupAggProcessFunction( val inputCntDescriptor: ValueStateDescriptor[JLong] = new ValueStateDescriptor[JLong]("GroupAggregateInputCounter", Types.LONG) cntState = getRuntimeContext.getState(inputCntDescriptor) + + if (stateCleaningEnabled) { + val inputCntDescriptor: ValueStateDescriptor[JLong] = + new ValueStateDescriptor[JLong]("GroupAggregateCleanupTime", Types.LONG) + cleanupTimeState = getRuntimeContext.getState(inputCntDescriptor) + } } override def processElement( @@ -81,6 +96,23 @@ class GroupAggProcessFunction( ctx: ProcessFunction[CRow, CRow]#Context, out: Collector[CRow]): Unit = { + if (stateCleaningEnabled) { + + val currentTime = ctx.timerService().currentProcessingTime() + val earliestCleanup = currentTime + minRetentionTime + + // last registered timer + val lastCleanupTime = cleanupTimeState.value() + + if (lastCleanupTime == null || earliestCleanup >= lastCleanupTime + cleanupTimerInterval) { + // we need to register a new timer + val cleanupTime = earliestCleanup + cleanupTimerInterval + // register timer and remember clean-up time + ctx.timerService().registerProcessingTimeTimer(cleanupTime) + cleanupTimeState.update(cleanupTime) + } + } + val input = inputC.row // get accumulators and input counter @@ -144,4 +176,18 @@ class GroupAggProcessFunction( cntState.clear() } } + + override def onTimer( + timestamp: Long, + ctx: ProcessFunction[CRow, CRow]#OnTimerContext, + out: Collector[CRow]): Unit = { + + if (timestamp == cleanupTimeState.value()) { + // clear all state + this.state.clear() + this.cntState.clear() + this.cleanupTimeState.clear() + } + } + } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala index 8626b07410a1d..3d79e2284cdb8 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala @@ -18,16 +18,17 @@ package org.apache.flink.table.utils -import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.tools.RuleSet -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.table.api.{Table, TableConfig, TableEnvironment} +import org.apache.flink.table.api.{QueryConfig, Table, TableConfig, TableEnvironment} import org.apache.flink.table.sinks.TableSink import org.apache.flink.table.sources.TableSource class MockTableEnvironment extends TableEnvironment(new TableConfig) { - override private[flink] def writeToSink[T](table: Table, sink: TableSink[T]): Unit = ??? + override private[flink] def writeToSink[T]( + table: Table, + sink: TableSink[T], + qConfig: QueryConfig): Unit = ??? override protected def checkValidTableName(name: String): Unit = ??? From f8b2ef3d27ef73142679ab50882a46c895074947 Mon Sep 17 00:00:00 2001 From: sunjincheng121 Date: Tue, 9 May 2017 14:36:42 +0800 Subject: [PATCH 2/3] [FLINK-6491] [table] Add QueryConfig and state clean up for over-windowed aggregates. --- .../apache/flink/table/api/QueryConfig.scala | 8 +- .../datastream/DataStreamOverAggregate.scala | 8 + .../runtime/aggregate/AggregateUtil.scala | 28 +- .../aggregate/GroupAggProcessFunction.scala | 42 +- .../aggregate/ProcTimeBoundedRangeOver.scala | 25 +- .../aggregate/ProcTimeBoundedRowsOver.scala | 17 +- .../ProcTimeUnboundedNonPartitionedOver.scala | 17 +- .../ProcTimeUnboundedPartitionedOver.scala | 17 +- .../ProcessFunctionWithCleanupState.scala | 99 ++++ .../aggregate/RowTimeBoundedRangeOver.scala | 20 +- .../aggregate/RowTimeBoundedRowsOver.scala | 21 +- .../aggregate/RowTimeUnboundedOver.scala | 30 +- .../table/GroupAggregationsITCase.scala | 13 +- ...ocessingOverRangeProcessFunctionTest.scala | 336 -------------- .../runtime/harness/HarnessTestBase.scala | 286 +++++++++++- .../harness/NonWindowHarnessTest.scala | 147 ++++++ .../harness/OverWindowHarnessTest.scala | 435 +++++++----------- 17 files changed, 866 insertions(+), 683 deletions(-) create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcessFunctionWithCleanupState.scala delete mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRangeProcessFunctionTest.scala create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala index 8e8b5ac73c94e..f6600ba59a05a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala @@ -56,15 +56,15 @@ class StreamQueryConfig private[table] extends QueryConfig { * When new data arrives for previously cleaned-up state, the new data will be handled as if it * was the first data. This can result in previous results being overwritten. * - * Note: [[setIdleStateRetentionTime(minTime: Time, maxTime: Time)]] allows to set a minimum and + * Note: [[withIdleStateRetentionTime(minTime: Time, maxTime: Time)]] allows to set a minimum and * maximum time for state to be retained. This method is more efficient, because the system has * to do less bookkeeping to identify the time at which state must be cleared. * * @param time The time interval for how long idle state is retained. Set to 0 (zero) to never * clean-up the state. */ - def setIdleStateRetentionTime(time: Time): StreamQueryConfig = { - setIdleStateRetentionTime(time, time) + def withIdleStateRetentionTime(time: Time): StreamQueryConfig = { + withIdleStateRetentionTime(time, time) } /** @@ -83,7 +83,7 @@ class StreamQueryConfig private[table] extends QueryConfig { * @param maxTime The maximum time interval for which idle state is retained. May not be smaller * than than minTime. Set to 0 (zero) to never clean-up the state. */ - def setIdleStateRetentionTime(minTime: Time, maxTime: Time): StreamQueryConfig = { + def withIdleStateRetentionTime(minTime: Time, maxTime: Time): StreamQueryConfig = { if (maxTime.toMilliseconds < minTime.toMilliseconds) { throw new IllegalArgumentException("maxTime may not be smaller than minTime.") } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala index 07cbf4803648a..9f1eb423fb33a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala @@ -138,6 +138,7 @@ class DataStreamOverAggregate( if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { // unbounded OVER window createUnboundedAndCurrentRowOverWindow( + qConfig, generator, inputDS, isRowTimeType = false, @@ -147,6 +148,7 @@ class DataStreamOverAggregate( overWindow.upperBound.isCurrentRow) { // bounded OVER window createBoundedAndCurrentRowOverWindow( + qConfig, generator, inputDS, isRowTimeType = false, @@ -162,6 +164,7 @@ class DataStreamOverAggregate( overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { // unbounded OVER window createUnboundedAndCurrentRowOverWindow( + qConfig, generator, inputDS, isRowTimeType = true, @@ -169,6 +172,7 @@ class DataStreamOverAggregate( } else if (overWindow.lowerBound.isPreceding && overWindow.upperBound.isCurrentRow) { // bounded OVER window createBoundedAndCurrentRowOverWindow( + qConfig, generator, inputDS, isRowTimeType = true, @@ -185,6 +189,7 @@ class DataStreamOverAggregate( } def createUnboundedAndCurrentRowOverWindow( + qConfig: StreamQueryConfig, generator: CodeGenerator, inputDS: DataStream[CRow], isRowTimeType: Boolean, @@ -210,6 +215,7 @@ class DataStreamOverAggregate( inputSchema.physicalType, inputSchema.physicalTypeInfo, inputSchema.physicalFieldTypeInfo, + qConfig, isRowTimeType, partitionKeys.nonEmpty, isRowsClause) @@ -242,6 +248,7 @@ class DataStreamOverAggregate( } def createBoundedAndCurrentRowOverWindow( + qConfig: StreamQueryConfig, generator: CodeGenerator, inputDS: DataStream[CRow], isRowTimeType: Boolean, @@ -269,6 +276,7 @@ class DataStreamOverAggregate( inputSchema.physicalTypeInfo, inputSchema.physicalFieldTypeInfo, precedingOffset, + qConfig, isRowsClause, isRowTimeType ) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala index 27392c72b8d98..5e5e3f980d41a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala @@ -77,6 +77,7 @@ object AggregateUtil { inputType: RelDataType, inputTypeInfo: TypeInformation[Row], inputFieldTypeInfo: Seq[TypeInformation[_]], + qConfig: StreamQueryConfig, isRowTimeType: Boolean, isPartitioned: Boolean, isRowsClause: Boolean) @@ -117,23 +118,27 @@ object AggregateUtil { new RowTimeUnboundedRowsOver( genFunction, aggregationStateType, - CRowTypeInfo(inputTypeInfo)) + CRowTypeInfo(inputTypeInfo), + qConfig) } else { // RANGE unbounded over process function new RowTimeUnboundedRangeOver( genFunction, aggregationStateType, - CRowTypeInfo(inputTypeInfo)) + CRowTypeInfo(inputTypeInfo), + qConfig) } } else { if (isPartitioned) { new ProcTimeUnboundedPartitionedOver( genFunction, - aggregationStateType) + aggregationStateType, + qConfig) } else { new ProcTimeUnboundedNonPartitionedOver( genFunction, - aggregationStateType) + aggregationStateType, + qConfig) } } } @@ -217,6 +222,7 @@ object AggregateUtil { inputTypeInfo: TypeInformation[Row], inputFieldTypeInfo: Seq[TypeInformation[_]], precedingOffset: Long, + qConfig: StreamQueryConfig, isRowsClause: Boolean, isRowTimeType: Boolean) : ProcessFunction[CRow, CRow] = { @@ -258,15 +264,15 @@ object AggregateUtil { genFunction, aggregationStateType, inputRowType, - precedingOffset - ) + precedingOffset, + qConfig) } else { new RowTimeBoundedRangeOver( genFunction, aggregationStateType, inputRowType, - precedingOffset - ) + precedingOffset, + qConfig) } } else { if (isRowsClause) { @@ -274,13 +280,15 @@ object AggregateUtil { genFunction, precedingOffset, aggregationStateType, - inputRowType) + inputRowType, + qConfig) } else { new ProcTimeBoundedRangeOver( genFunction, precedingOffset, aggregationStateType, - inputRowType) + inputRowType, + qConfig) } } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala index 84fee8759f4a8..dbaddd60e0e0c 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala @@ -42,18 +42,12 @@ class GroupAggProcessFunction( private val aggregationStateType: RowTypeInfo, private val generateRetraction: Boolean, private val qConfig: StreamQueryConfig) - extends ProcessFunction[CRow, CRow] + extends ProcessFunctionWithCleanupState[CRow, CRow](qConfig) with Compiler[GeneratedAggregations] { val LOG: Logger = LoggerFactory.getLogger(this.getClass) private var function: GeneratedAggregations = _ - private val minRetentionTime = qConfig.getMinIdleStateRetentionTime - private val maxRetentionTime = qConfig.getMaxIdleStateRetentionTime - private val stateCleaningEnabled = minRetentionTime > 1 && maxRetentionTime > 1 - // interval in which clean-up timers are registered - private val cleanupTimerInterval = maxRetentionTime - minRetentionTime - private var newRow: CRow = _ private var prevRow: CRow = _ private var firstRow: Boolean = _ @@ -62,7 +56,6 @@ class GroupAggProcessFunction( // counts the number of added and retracted input records private var cntState: ValueState[JLong] = _ // holds the latest registered cleanup timer - private var cleanupTimeState: ValueState[JLong] = _ override def open(config: Configuration) { LOG.debug(s"Compiling AggregateHelper: $genAggregations.name \n\n " + @@ -84,11 +77,7 @@ class GroupAggProcessFunction( new ValueStateDescriptor[JLong]("GroupAggregateInputCounter", Types.LONG) cntState = getRuntimeContext.getState(inputCntDescriptor) - if (stateCleaningEnabled) { - val inputCntDescriptor: ValueStateDescriptor[JLong] = - new ValueStateDescriptor[JLong]("GroupAggregateCleanupTime", Types.LONG) - cleanupTimeState = getRuntimeContext.getState(inputCntDescriptor) - } + initCleanupTimeState("GroupAggregateCleanupTime") } override def processElement( @@ -96,22 +85,9 @@ class GroupAggProcessFunction( ctx: ProcessFunction[CRow, CRow]#Context, out: Collector[CRow]): Unit = { - if (stateCleaningEnabled) { - - val currentTime = ctx.timerService().currentProcessingTime() - val earliestCleanup = currentTime + minRetentionTime - - // last registered timer - val lastCleanupTime = cleanupTimeState.value() - - if (lastCleanupTime == null || earliestCleanup >= lastCleanupTime + cleanupTimerInterval) { - // we need to register a new timer - val cleanupTime = earliestCleanup + cleanupTimerInterval - // register timer and remember clean-up time - ctx.timerService().registerProcessingTimeTimer(cleanupTime) - cleanupTimeState.update(cleanupTime) - } - } + val currentTime = ctx.timerService().currentProcessingTime() + // register state-cleanup timer + registerProcessingCleanupTimer(ctx, currentTime) val input = inputC.row @@ -181,13 +157,7 @@ class GroupAggProcessFunction( timestamp: Long, ctx: ProcessFunction[CRow, CRow]#OnTimerContext, out: Collector[CRow]): Unit = { - - if (timestamp == cleanupTimeState.value()) { - // clear all state - this.state.clear() - this.cntState.clear() - this.cleanupTimeState.clear() - } + cleanupStateOnTimer(timestamp, state, cntState) } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala index 3fb506fd1333c..931772b81b199 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala @@ -31,6 +31,7 @@ import org.apache.flink.api.java.typeutils.ListTypeInfo import java.util.{ArrayList, List => JList} import org.apache.flink.api.common.typeinfo.BasicTypeInfo +import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} import org.slf4j.LoggerFactory @@ -48,9 +49,11 @@ class ProcTimeBoundedRangeOver( genAggregations: GeneratedAggregationsFunction, precedingTimeBoundary: Long, aggregatesTypeInfo: RowTypeInfo, - inputType: TypeInformation[CRow]) - extends ProcessFunction[CRow, CRow] + inputType: TypeInformation[CRow], + qConfig: StreamQueryConfig) + extends ProcessFunctionWithCleanupState[CRow, CRow](qConfig) with Compiler[GeneratedAggregations] { + private var output: CRow = _ private var accumulatorState: ValueState[Row] = _ private var rowMapState: MapState[Long, JList[Row]] = _ @@ -81,6 +84,8 @@ class ProcTimeBoundedRangeOver( val stateDescriptor: ValueStateDescriptor[Row] = new ValueStateDescriptor[Row]("overState", aggregatesTypeInfo) accumulatorState = getRuntimeContext.getState(stateDescriptor) + + initCleanupTimeState("ProcTimeBoundedRangeOverCleanupTime") } override def processElement( @@ -89,6 +94,9 @@ class ProcTimeBoundedRangeOver( out: Collector[CRow]): Unit = { val currentTime = ctx.timerService.currentProcessingTime + // register state-cleanup timer + registerProcessingCleanupTimer(ctx, currentTime) + // buffer the event incoming event // add current element to the window list of elements with corresponding timestamp @@ -109,7 +117,14 @@ class ProcTimeBoundedRangeOver( ctx: ProcessFunction[CRow, CRow]#OnTimerContext, out: Collector[CRow]): Unit = { - // we consider the original timestamp of events that have registered this time trigger 1 ms ago + val isCleanup = cleanupStateOnTimer(timestamp, rowMapState, accumulatorState) + + if (isCleanup) { + return + } + // we consider the original timestamp of events + // that have registered this time trigger 1 ms ago + val currentTime = timestamp - 1 var i = 0 @@ -153,7 +168,8 @@ class ProcTimeBoundedRangeOver( // get the list of elements of current proctime val currentElements = rowMapState.get(currentTime) - // add current elements to aggregator. Multiple elements might have arrived in the same proctime + // add current elements to aggregator. Multiple elements might + // have arrived in the same proctime // the same accumulator value will be computed for all elements var iElemenets = 0 while (iElemenets < currentElements.size()) { @@ -178,7 +194,6 @@ class ProcTimeBoundedRangeOver( // update the value of accumulators for future incremental computation accumulatorState.update(accumulators) - } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala index 0c7f44ed1531b..7fd435bd7a3a6 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala @@ -33,6 +33,7 @@ import org.apache.flink.api.java.typeutils.ListTypeInfo import java.util.{List => JList} import org.apache.flink.api.common.typeinfo.BasicTypeInfo +import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} import org.slf4j.LoggerFactory @@ -49,8 +50,9 @@ class ProcTimeBoundedRowsOver( genAggregations: GeneratedAggregationsFunction, precedingOffset: Long, aggregatesTypeInfo: RowTypeInfo, - inputType: TypeInformation[CRow]) - extends ProcessFunction[CRow, CRow] + inputType: TypeInformation[CRow], + qConfig: StreamQueryConfig) + extends ProcessFunctionWithCleanupState[CRow, CRow](qConfig) with Compiler[GeneratedAggregations] { Preconditions.checkArgument(precedingOffset > 0) @@ -99,6 +101,8 @@ class ProcTimeBoundedRowsOver( val smallestTimestampDescriptor : ValueStateDescriptor[Long] = new ValueStateDescriptor[Long]("smallestTSState", classOf[Long]) smallestTsState = getRuntimeContext.getState(smallestTimestampDescriptor) + + initCleanupTimeState("ProcTimeBoundedRowsOverCleanupTime") } override def processElement( @@ -110,6 +114,9 @@ class ProcTimeBoundedRowsOver( val currentTime = ctx.timerService.currentProcessingTime + // register state-cleanup timer + registerProcessingCleanupTimer(ctx, currentTime) + // initialize state for the processed element var accumulators = accumulatorState.value if (accumulators == null) { @@ -180,4 +187,10 @@ class ProcTimeBoundedRowsOver( out.collect(output) } + override def onTimer( + timestamp: Long, + ctx: ProcessFunction[CRow, CRow]#OnTimerContext, + out: Collector[CRow]): Unit = { + cleanupStateOnTimer(timestamp, rowMapState, accumulatorState, counterState, smallestTsState) + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedNonPartitionedOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedNonPartitionedOver.scala index 8a231327e01c2..41fbe6bfa50d0 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedNonPartitionedOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedNonPartitionedOver.scala @@ -23,6 +23,7 @@ import org.apache.flink.configuration.Configuration import org.apache.flink.runtime.state.{FunctionInitializationContext, FunctionSnapshotContext} import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.util.Collector import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} @@ -37,8 +38,9 @@ import org.slf4j.LoggerFactory */ class ProcTimeUnboundedNonPartitionedOver( genAggregations: GeneratedAggregationsFunction, - aggregationStateType: RowTypeInfo) - extends ProcessFunction[CRow, CRow] + aggregationStateType: RowTypeInfo, + qConfig: StreamQueryConfig) + extends ProcessFunctionWithCleanupState[CRow, CRow](qConfig) with CheckpointedFunction with Compiler[GeneratedAggregations] { @@ -68,12 +70,16 @@ class ProcTimeUnboundedNonPartitionedOver( accumulators = function.createAccumulators() } } + initCleanupTimeState("ProcTimeUnboundedNonPartitionedOverCleanupTime") } override def processElement( inputC: CRow, ctx: ProcessFunction[CRow, CRow]#Context, out: Collector[CRow]): Unit = { + // register state-cleanup timer + registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) + val input = inputC.row @@ -85,6 +91,13 @@ class ProcTimeUnboundedNonPartitionedOver( out.collect(output) } + override def onTimer( + timestamp: Long, + ctx: ProcessFunction[CRow, CRow]#OnTimerContext, + out: Collector[CRow]): Unit = { + cleanupStateOnTimer(timestamp, state) + } + override def snapshotState(context: FunctionSnapshotContext): Unit = { state.clear() if (null != accumulators) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedPartitionedOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedPartitionedOver.scala index 847c1bfb142f5..6f1f3822a55d1 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedPartitionedOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedPartitionedOver.scala @@ -24,6 +24,7 @@ import org.apache.flink.util.Collector import org.apache.flink.api.common.state.ValueStateDescriptor import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.common.state.ValueState +import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} import org.apache.flink.table.runtime.types.CRow import org.slf4j.LoggerFactory @@ -36,8 +37,9 @@ import org.slf4j.LoggerFactory */ class ProcTimeUnboundedPartitionedOver( genAggregations: GeneratedAggregationsFunction, - aggregationStateType: RowTypeInfo) - extends ProcessFunction[CRow, CRow] + aggregationStateType: RowTypeInfo, + qConfig: StreamQueryConfig) + extends ProcessFunctionWithCleanupState[CRow, CRow](qConfig) with Compiler[GeneratedAggregations] { private var output: CRow = _ @@ -59,6 +61,8 @@ class ProcTimeUnboundedPartitionedOver( val stateDescriptor: ValueStateDescriptor[Row] = new ValueStateDescriptor[Row]("overState", aggregationStateType) state = getRuntimeContext.getState(stateDescriptor) + + initCleanupTimeState("ProcTimeUnboundedPartitionedOverCleanupTime") } override def processElement( @@ -66,6 +70,9 @@ class ProcTimeUnboundedPartitionedOver( ctx: ProcessFunction[CRow, CRow]#Context, out: Collector[CRow]): Unit = { + // register state-cleanup timer + registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) + val input = inputC.row var accumulators = state.value() @@ -83,4 +90,10 @@ class ProcTimeUnboundedPartitionedOver( out.collect(output) } + override def onTimer( + timestamp: Long, + ctx: ProcessFunction[CRow, CRow]#OnTimerContext, + out: Collector[CRow]): Unit = { + cleanupStateOnTimer(timestamp, state) + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcessFunctionWithCleanupState.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcessFunctionWithCleanupState.scala new file mode 100644 index 0000000000000..ad249727ff213 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcessFunctionWithCleanupState.scala @@ -0,0 +1,99 @@ +/* + * 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.aggregate + +import java.lang.{Long => JLong} + +import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor} +import org.apache.flink.api.common.state.State +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.table.api.{StreamQueryConfig, Types} + +abstract class ProcessFunctionWithCleanupState[IN,OUT](qConfig: StreamQueryConfig) + extends ProcessFunction[IN, OUT]{ + + protected val minRetentionTime = qConfig.getMinIdleStateRetentionTime + protected val maxRetentionTime = qConfig.getMaxIdleStateRetentionTime + protected val stateCleaningEnabled = minRetentionTime > 1 && maxRetentionTime > 1 + // interval in which clean-up timers are registered + protected val cleanupTimerInterval = maxRetentionTime - minRetentionTime + + // holds the latest registered cleanup timer + private var cleanupTimeState: ValueState[JLong] = _ + + protected def initCleanupTimeState(stateName: String) { + if (stateCleaningEnabled) { + val inputCntDescriptor: ValueStateDescriptor[JLong] = + new ValueStateDescriptor[JLong](stateName, Types.LONG) + cleanupTimeState = getRuntimeContext.getState(inputCntDescriptor) + } + } + + protected def registerProcessingCleanupTimer( + ctx: ProcessFunction[IN, OUT]#Context, + currentTime: Long): Unit = { + if (stateCleaningEnabled) { + + val earliestCleanup = currentTime + minRetentionTime + + // last registered timer + val lastCleanupTime = cleanupTimeState.value() + + if (lastCleanupTime == null || earliestCleanup >= lastCleanupTime + cleanupTimerInterval) { + // we need to register a new timer + val cleanupTime = earliestCleanup + cleanupTimerInterval + // register timer and remember clean-up time + ctx.timerService().registerProcessingTimeTimer(cleanupTime) + cleanupTimeState.update(cleanupTime) + } + } + } + protected def registerEventCleanupTimer( + ctx: ProcessFunction[IN, OUT]#Context, + currentTime: Long): Unit = { + if (stateCleaningEnabled) { + + val earliestCleanup = currentTime + minRetentionTime + + // last registered timer + val lastCleanupTime = cleanupTimeState.value() + + if (lastCleanupTime == null || earliestCleanup >= lastCleanupTime + cleanupTimerInterval) { + // we need to register a new timer + val cleanupTime = earliestCleanup + cleanupTimerInterval + // register timer and remember clean-up time + ctx.timerService().registerEventTimeTimer(cleanupTime) + cleanupTimeState.update(cleanupTime) + } + } + } + + protected def cleanupStateOnTimer(timestamp: Long, states: State*): Boolean = { + var result: Boolean = false + if (stateCleaningEnabled) { + val cleanupTime = cleanupTimeState.value() + if (null != cleanupTime && timestamp == cleanupTime) { + // clear all state + states.foreach(_.clear()) + this.cleanupTimeState.clear() + result = true + } + } + result + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala index 4020d440a78fa..7768325a29ee7 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala @@ -24,6 +24,7 @@ import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} import org.apache.flink.api.java.typeutils.{ListTypeInfo, RowTypeInfo} import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} import org.apache.flink.types.Row @@ -42,8 +43,9 @@ class RowTimeBoundedRangeOver( genAggregations: GeneratedAggregationsFunction, aggregationStateType: RowTypeInfo, inputRowType: CRowTypeInfo, - precedingOffset: Long) - extends ProcessFunction[CRow, CRow] + precedingOffset: Long, + qConfig: StreamQueryConfig) + extends ProcessFunctionWithCleanupState[CRow, CRow](qConfig) with Compiler[GeneratedAggregations] { Preconditions.checkNotNull(aggregationStateType) Preconditions.checkNotNull(precedingOffset) @@ -97,6 +99,8 @@ class RowTimeBoundedRangeOver( valueTypeInformation) dataState = getRuntimeContext.getMapState(mapStateDescriptor) + + initCleanupTimeState("RowTimeBoundedRangeOverCleanupTime") } override def processElement( @@ -109,6 +113,9 @@ class RowTimeBoundedRangeOver( // triggering timestamp for trigger calculation val triggeringTs = ctx.timestamp + // register state-cleanup timer + registerEventCleanupTimer(ctx, triggeringTs) + val lastTriggeringTs = lastTriggeringTsState.value // check if the data is expired, if not, save the data and register event time timer @@ -133,6 +140,15 @@ class RowTimeBoundedRangeOver( out: Collector[CRow]): Unit = { // gets all window data from state for the calculation val inputs: JList[Row] = dataState.get(timestamp) + val isCleanup = cleanupStateOnTimer( + timestamp, + lastTriggeringTsState, + accumulatorState, + dataState) + + if (isCleanup) { + return + } if (null != inputs) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala index 5ec6ec7ab1ead..dd4a7e64a3e92 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala @@ -26,6 +26,7 @@ import org.apache.flink.api.java.typeutils.{ListTypeInfo, RowTypeInfo} import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.types.Row +import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.util.{Collector, Preconditions} import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} @@ -43,8 +44,9 @@ class RowTimeBoundedRowsOver( genAggregations: GeneratedAggregationsFunction, aggregationStateType: RowTypeInfo, inputRowType: CRowTypeInfo, - precedingOffset: Long) - extends ProcessFunction[CRow, CRow] + precedingOffset: Long, + qConfig: StreamQueryConfig) +extends ProcessFunctionWithCleanupState[CRow, CRow](qConfig) with Compiler[GeneratedAggregations] { Preconditions.checkNotNull(aggregationStateType) @@ -106,6 +108,8 @@ class RowTimeBoundedRowsOver( valueTypeInformation) dataState = getRuntimeContext.getMapState(mapStateDescriptor) + + initCleanupTimeState("RowTimeBoundedRowsOverCleanupTime") } override def processElement( @@ -118,6 +122,9 @@ class RowTimeBoundedRowsOver( // triggering timestamp for trigger calculation val triggeringTs = ctx.timestamp + // register state-cleanup timer + registerEventCleanupTimer(ctx, triggeringTs) + val lastTriggeringTs = lastTriggeringTsState.value // check if the data is expired, if not, save the data and register event time timer @@ -141,6 +148,16 @@ class RowTimeBoundedRowsOver( ctx: ProcessFunction[CRow, CRow]#OnTimerContext, out: Collector[CRow]): Unit = { + val isCleanup = cleanupStateOnTimer( + timestamp, + lastTriggeringTsState, + dataCountState, + accumulatorState, + dataState) + + if (isCleanup) { + return + } // gets all window data from state for the calculation val inputs: JList[Row] = dataState.get(timestamp) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala index 3e2a8117b56e5..6e482810473ae 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala @@ -28,6 +28,7 @@ import org.apache.flink.util.{Collector, Preconditions} import org.apache.flink.api.common.state._ import org.apache.flink.api.java.typeutils.ListTypeInfo import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} import org.slf4j.LoggerFactory @@ -43,8 +44,9 @@ import org.slf4j.LoggerFactory abstract class RowTimeUnboundedOver( genAggregations: GeneratedAggregationsFunction, intermediateType: TypeInformation[Row], - inputType: TypeInformation[CRow]) - extends ProcessFunction[CRow, CRow] + inputType: TypeInformation[CRow], + qConfig: StreamQueryConfig) + extends ProcessFunctionWithCleanupState[CRow, CRow](qConfig) with Compiler[GeneratedAggregations] { protected var output: CRow = _ @@ -83,6 +85,8 @@ abstract class RowTimeUnboundedOver( new MapStateDescriptor[Long, JList[Row]]("rowmapstate", BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) + + initCleanupTimeState("RowTimeUnboundedOverCleanupTime") } /** @@ -102,6 +106,10 @@ abstract class RowTimeUnboundedOver( val input = inputC.row val timestamp = ctx.timestamp() + + // register state-cleanup timer + registerEventCleanupTimer(ctx, timestamp) + val curWatermark = ctx.timerService().currentWatermark() // discard late record @@ -136,6 +144,12 @@ abstract class RowTimeUnboundedOver( Preconditions.checkArgument(out.isInstanceOf[TimestampedCollector[CRow]]) val collector = out.asInstanceOf[TimestampedCollector[CRow]] + val isCleanup = cleanupStateOnTimer(timestamp, rowMapState, accumulatorState) + + if (isCleanup) { + return + } + val keyIterator = rowMapState.keys.iterator if (keyIterator.hasNext) { val curWatermark = ctx.timerService.currentWatermark @@ -221,11 +235,13 @@ abstract class RowTimeUnboundedOver( class RowTimeUnboundedRowsOver( genAggregations: GeneratedAggregationsFunction, intermediateType: TypeInformation[Row], - inputType: TypeInformation[CRow]) + inputType: TypeInformation[CRow], + qConfig: StreamQueryConfig) extends RowTimeUnboundedOver( genAggregations: GeneratedAggregationsFunction, intermediateType, - inputType) { + inputType, + qConfig) { override def processElementsWithSameTimestamp( curRowList: JList[Row], @@ -259,11 +275,13 @@ class RowTimeUnboundedRowsOver( class RowTimeUnboundedRangeOver( genAggregations: GeneratedAggregationsFunction, intermediateType: TypeInformation[Row], - inputType: TypeInformation[CRow]) + inputType: TypeInformation[CRow], + qConfig: StreamQueryConfig) extends RowTimeUnboundedOver( genAggregations: GeneratedAggregationsFunction, intermediateType, - inputType) { + inputType, + qConfig) { override def processElementsWithSameTimestamp( curRowList: JList[Row], diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupAggregationsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupAggregationsITCase.scala index 910cbf267dbc5..375701318b54f 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupAggregationsITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupAggregationsITCase.scala @@ -18,11 +18,12 @@ package org.apache.flink.table.api.scala.stream.table +import org.apache.flink.api.common.time.Time import org.apache.flink.api.scala._ import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.table.api.scala._ import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, StreamTestData, StreamingWithStateTestBase} -import org.apache.flink.table.api.TableEnvironment +import org.apache.flink.table.api.{StreamQueryConfig, TableEnvironment} import org.apache.flink.table.api.scala.stream.utils.StreamITCase.RetractingSink import org.apache.flink.types.Row import org.junit.Assert.assertEquals @@ -34,6 +35,8 @@ import scala.collection.mutable * Tests of groupby (without window) aggregations */ class GroupAggregationsITCase extends StreamingWithStateTestBase { + private val qConfig = new StreamQueryConfig() + qConfig.withIdleStateRetentionTime(Time.hours(1), Time.hours(2)) @Test def testNonKeyedGroupAggregate(): Unit = { @@ -45,7 +48,7 @@ class GroupAggregationsITCase extends StreamingWithStateTestBase { val t = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c) .select('a.sum, 'b.sum) - val results = t.toRetractStream[Row] + val results = t.toRetractStream[Row](qConfig) results.addSink(new StreamITCase.RetractingSink).setParallelism(1) env.execute() @@ -64,7 +67,7 @@ class GroupAggregationsITCase extends StreamingWithStateTestBase { .groupBy('b) .select('b, 'a.sum) - val results = t.toRetractStream[Row] + val results = t.toRetractStream[Row](qConfig) results.addSink(new StreamITCase.RetractingSink) env.execute() @@ -85,7 +88,7 @@ class GroupAggregationsITCase extends StreamingWithStateTestBase { .groupBy('cnt) .select('cnt, 'b.count as 'freq) - val results = t.toRetractStream[Row] + val results = t.toRetractStream[Row](qConfig) results.addSink(new RetractingSink) env.execute() @@ -104,7 +107,7 @@ class GroupAggregationsITCase extends StreamingWithStateTestBase { .groupBy('e, 'b % 3) .select('c.min, 'e, 'a.avg, 'd.count) - val results = t.toRetractStream[Row] + val results = t.toRetractStream[Row](qConfig) results.addSink(new RetractingSink) env.execute() diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRangeProcessFunctionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRangeProcessFunctionTest.scala deleted file mode 100644 index eadcfc8c6d774..0000000000000 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRangeProcessFunctionTest.scala +++ /dev/null @@ -1,336 +0,0 @@ -/* - * 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.aggregate - -import java.util.Comparator -import java.util.concurrent.ConcurrentLinkedQueue -import java.lang.{Integer => JInt, Long => JLong} - -import org.apache.flink.api.common.typeinfo.BasicTypeInfo._ -import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} -import org.apache.flink.api.java.functions.KeySelector -import org.apache.flink.api.java.typeutils.RowTypeInfo -import org.apache.flink.streaming.api.operators.KeyedProcessOperator -import org.apache.flink.streaming.api.watermark.Watermark -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord -import org.apache.flink.streaming.util.{KeyedOneInputStreamOperatorTestHarness, TestHarnessUtil} -import org.apache.flink.table.codegen.GeneratedAggregationsFunction -import org.apache.flink.table.functions.AggregateFunction -import org.apache.flink.table.functions.aggfunctions.{LongMaxWithRetractAggFunction, LongMinWithRetractAggFunction} -import org.apache.flink.table.runtime.aggregate.BoundedProcessingOverRangeProcessFunctionTest._ -import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} -import org.apache.flink.types.Row -import org.junit.Test - -class BoundedProcessingOverRangeProcessFunctionTest { - - @Test - def testProcTimePartitionedOverRange(): Unit = { - - val rT = new CRowTypeInfo(new RowTypeInfo(Array[TypeInformation[_]]( - INT_TYPE_INFO, - LONG_TYPE_INFO, - INT_TYPE_INFO, - STRING_TYPE_INFO, - LONG_TYPE_INFO), - Array("a", "b", "c", "d", "e"))) - - val aggregates = - Array(new LongMinWithRetractAggFunction, - new LongMaxWithRetractAggFunction).asInstanceOf[Array[AggregateFunction[_, _]]] - val aggregationStateType: RowTypeInfo = AggregateUtil.createAccumulatorRowType(aggregates) - - val funcCode = - """ - |public class BoundedOverAggregateHelper$33 - | extends org.apache.flink.table.runtime.aggregate.GeneratedAggregations { - | - | transient org.apache.flink.table.functions.aggfunctions.LongMinWithRetractAggFunction - | fmin = null; - | - | transient org.apache.flink.table.functions.aggfunctions.LongMaxWithRetractAggFunction - | fmax = null; - | - | public BoundedOverAggregateHelper$33() throws Exception { - | - | fmin = (org.apache.flink.table.functions.aggfunctions.LongMinWithRetractAggFunction) - | org.apache.flink.table.functions.utils.UserDefinedFunctionUtils - | .deserialize("rO0ABXNyAEtvcmcuYXBhY2hlLmZsaW5rLnRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbn" + - | "MuTG9uZ01pbldpdGhSZXRyYWN0QWdnRnVuY3Rpb26oIdX_DaMPxQIAAHhyAEdvcmcuYXBhY2hlLmZsaW5rL" + - | "nRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbnMuTWluV2l0aFJldHJhY3RBZ2dGdW5jdGlvbq_ZGuzxtA_S" + - | "AgABTAADb3JkdAAVTHNjYWxhL21hdGgvT3JkZXJpbmc7eHIAMm9yZy5hcGFjaGUuZmxpbmsudGFibGUuZnV" + - | "uY3Rpb25zLkFnZ3JlZ2F0ZUZ1bmN0aW9uTcYVPtJjNfwCAAB4cgA0b3JnLmFwYWNoZS5mbGluay50YWJsZS" + - | "5mdW5jdGlvbnMuVXNlckRlZmluZWRGdW5jdGlvbi0B91QxuAyTAgAAeHBzcgAZc2NhbGEubWF0aC5PcmRlc" + - | "mluZyRMb25nJOda0iCPo2ukAgAAeHA"); - | - | fmax = (org.apache.flink.table.functions.aggfunctions.LongMaxWithRetractAggFunction) - | org.apache.flink.table.functions.utils.UserDefinedFunctionUtils - | .deserialize("rO0ABXNyAEtvcmcuYXBhY2hlLmZsaW5rLnRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbn" + - | "MuTG9uZ01heFdpdGhSZXRyYWN0QWdnRnVuY3Rpb25RmsI8azNGXwIAAHhyAEdvcmcuYXBhY2hlLmZsaW5rL" + - | "nRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbnMuTWF4V2l0aFJldHJhY3RBZ2dGdW5jdGlvbvnwowlX0_Qf" + - | "AgABTAADb3JkdAAVTHNjYWxhL21hdGgvT3JkZXJpbmc7eHIAMm9yZy5hcGFjaGUuZmxpbmsudGFibGUuZnV" + - | "uY3Rpb25zLkFnZ3JlZ2F0ZUZ1bmN0aW9uTcYVPtJjNfwCAAB4cgA0b3JnLmFwYWNoZS5mbGluay50YWJsZS" + - | "5mdW5jdGlvbnMuVXNlckRlZmluZWRGdW5jdGlvbi0B91QxuAyTAgAAeHBzcgAZc2NhbGEubWF0aC5PcmRlc" + - | "mluZyRMb25nJOda0iCPo2ukAgAAeHA"); - | } - | - | public void setAggregationResults( - | org.apache.flink.types.Row accs, - | org.apache.flink.types.Row output) { - | - | org.apache.flink.table.functions.AggregateFunction baseClass0 = - | (org.apache.flink.table.functions.AggregateFunction) fmin; - | output.setField(5, baseClass0.getValue( - | (org.apache.flink.table.functions.aggfunctions.MinWithRetractAccumulator) - | accs.getField(0))); - | - | org.apache.flink.table.functions.AggregateFunction baseClass1 = - | (org.apache.flink.table.functions.AggregateFunction) fmax; - | output.setField(6, baseClass1.getValue( - | (org.apache.flink.table.functions.aggfunctions.MaxWithRetractAccumulator) - | accs.getField(1))); - | } - | - | public void accumulate( - | org.apache.flink.types.Row accs, - | org.apache.flink.types.Row input) { - | - | fmin.accumulate( - | ((org.apache.flink.table.functions.aggfunctions.MinWithRetractAccumulator) - | accs.getField(0)), - | (java.lang.Long) input.getField(4)); - | - | fmax.accumulate( - | ((org.apache.flink.table.functions.aggfunctions.MaxWithRetractAccumulator) - | accs.getField(1)), - | (java.lang.Long) input.getField(4)); - | } - | - | public void retract( - | org.apache.flink.types.Row accs, - | org.apache.flink.types.Row input) { - | - | fmin.retract( - | ((org.apache.flink.table.functions.aggfunctions.MinWithRetractAccumulator) - | accs.getField(0)), - | (java.lang.Long) input.getField(4)); - | - | fmax.retract( - | ((org.apache.flink.table.functions.aggfunctions.MaxWithRetractAccumulator) - | accs.getField(1)), - | (java.lang.Long) input.getField(4)); - | } - | - | public org.apache.flink.types.Row createAccumulators() { - | - | org.apache.flink.types.Row accs = new org.apache.flink.types.Row(2); - | - | accs.setField( - | 0, - | fmin.createAccumulator()); - | - | accs.setField( - | 1, - | fmax.createAccumulator()); - | - | return accs; - | } - | - | public void setForwardedFields( - | org.apache.flink.types.Row input, - | org.apache.flink.types.Row output) { - | - | output.setField(0, input.getField(0)); - | output.setField(1, input.getField(1)); - | output.setField(2, input.getField(2)); - | output.setField(3, input.getField(3)); - | output.setField(4, input.getField(4)); - | } - | - | public org.apache.flink.types.Row createOutputRow() { - | return new org.apache.flink.types.Row(7); - | } - | - |/******* This test does not use the following methods *******/ - | public org.apache.flink.types.Row mergeAccumulatorsPair( - | org.apache.flink.types.Row a, - | org.apache.flink.types.Row b) { - | return null; - | } - | - | public void resetAccumulator(org.apache.flink.types.Row accs) { - | } - | - | public void setConstantFlags(org.apache.flink.types.Row output) { - | } - |} - """.stripMargin - - val funcName = "BoundedOverAggregateHelper$33" - - val genAggFunction = GeneratedAggregationsFunction(funcName, funcCode) - val processFunction = new KeyedProcessOperator[String, CRow, CRow]( - new ProcTimeBoundedRangeOver( - genAggFunction, - 1000, - aggregationStateType, - rT)) - - val testHarness = new KeyedOneInputStreamOperatorTestHarness[JInt, CRow, CRow]( - processFunction, - new TupleRowSelector(0), - BasicTypeInfo.INT_TYPE_INFO) - - testHarness.open() - - // Time = 3 - testHarness.setProcessingTime(3) - // key = 1 - testHarness.processElement(new StreamRecord( - new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 0)) - // key = 2 - testHarness.processElement(new StreamRecord( - new CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong), true), 0)) - - // Time = 4 - testHarness.setProcessingTime(4) - // key = 1 - testHarness.processElement(new StreamRecord( - new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong), true), 0)) - testHarness.processElement(new StreamRecord( - new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong), true), 0)) - // key = 2 - testHarness.processElement(new StreamRecord( - new CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong), true), 0)) - - // Time = 5 - testHarness.setProcessingTime(5) - testHarness.processElement(new StreamRecord( - new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong), true), 0)) - - // Time = 6 - testHarness.setProcessingTime(6) - - // Time = 1002 - testHarness.setProcessingTime(1002) - // key = 1 - testHarness.processElement(new StreamRecord( - new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong), true), 0)) - testHarness.processElement(new StreamRecord( - new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong), true), 0)) - // key = 2 - testHarness.processElement(new StreamRecord( - new CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), true), 0)) - - // Time = 1003 - testHarness.setProcessingTime(1003) - testHarness.processElement(new StreamRecord( - new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 0)) - - // Time = 1004 - testHarness.setProcessingTime(1004) - testHarness.processElement(new StreamRecord( - new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 0)) - - // Time = 1005 - testHarness.setProcessingTime(1005) - // key = 1 - testHarness.processElement(new StreamRecord( - new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 0)) - testHarness.processElement(new StreamRecord( - new CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 0)) - // key = 2 - testHarness.processElement(new StreamRecord( - new CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 0)) - - testHarness.setProcessingTime(1006) - - val result = testHarness.getOutput - - val expectedOutput = new ConcurrentLinkedQueue[Object]() - - // all elements at the same proc timestamp have the same value - expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true), 4)) - expectedOutput.add(new StreamRecord(new CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true), 4)) - expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 1L: JLong, 3L: JLong), true), 5)) - expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true), 5)) - expectedOutput.add(new StreamRecord(new CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true), 5)) - expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong, 1L: JLong, 4L: JLong), true), 6)) - expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 1L: JLong, 6L: JLong), true), 1003)) - expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 1L: JLong, 6L: JLong), true), 1003)) - expectedOutput.add(new StreamRecord(new CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true), 1003)) - expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 1L: JLong, 7L: JLong), true), 1004)) - expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 2L: JLong, 8L: JLong), true), 1005)) - expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 4L: JLong, 10L: JLong), true), 1006)) - expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 4L: JLong, 10L: JLong), true), 1006)) - expectedOutput.add(new StreamRecord(new CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 30L: JLong, 40L: JLong), true), 1006)) - - TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", - expectedOutput, result, new RowResultSortComparator(6)) - - testHarness.close() - - } -} - -object BoundedProcessingOverRangeProcessFunctionTest { - -/** - * Return 0 for equal CRows and non zero for different CRows - */ -class RowResultSortComparator(indexCounter: Int) extends Comparator[Object] with Serializable { - - override def compare(o1: Object, o2: Object):Int = { - - if (o1.isInstanceOf[Watermark] || o2.isInstanceOf[Watermark]) { - // watermark is not expected - -1 - } else { - val row1 = o1.asInstanceOf[StreamRecord[CRow]].getValue - val row2 = o2.asInstanceOf[StreamRecord[CRow]].getValue - row1.toString.compareTo(row2.toString) - } - } -} - -/** - * Simple test class that returns a specified field as the selector function - */ -class TupleRowSelector( - private val selectorField:Int) extends KeySelector[CRow, Integer] { - - override def getKey(value: CRow): Integer = { - value.row.getField(selectorField).asInstanceOf[Integer] - } -} - -} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala index eb5acd5b9a3c9..3f61cab304c0f 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala @@ -19,15 +19,299 @@ package org.apache.flink.table.runtime.harness import java.util.{Comparator, Queue => JQueue} +import org.apache.flink.api.common.time.Time +import org.apache.flink.api.common.typeinfo.BasicTypeInfo.{INT_TYPE_INFO, LONG_TYPE_INFO, STRING_TYPE_INFO} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.functions.KeySelector +import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.streaming.api.operators.OneInputStreamOperator import org.apache.flink.streaming.api.watermark.Watermark import org.apache.flink.streaming.runtime.streamrecord.StreamRecord import org.apache.flink.streaming.util.{KeyedOneInputStreamOperatorTestHarness, TestHarnessUtil} -import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.table.api.StreamQueryConfig +import org.apache.flink.table.codegen.GeneratedAggregationsFunction +import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.table.functions.aggfunctions.{LongMaxWithRetractAggFunction, LongMinWithRetractAggFunction, IntSumWithRetractAggFunction} +import org.apache.flink.table.runtime.aggregate.AggregateUtil +import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} class HarnessTestBase { + + protected var qConfig = + new StreamQueryConfig().withIdleStateRetentionTime(Time.seconds(2), Time.seconds(3)) + + protected val MinMaxRowType = new RowTypeInfo(Array[TypeInformation[_]]( + INT_TYPE_INFO, + LONG_TYPE_INFO, + INT_TYPE_INFO, + STRING_TYPE_INFO, + LONG_TYPE_INFO), + Array("a", "b", "c", "d", "e")) + + protected val SumRowType = new RowTypeInfo(Array[TypeInformation[_]]( + LONG_TYPE_INFO, + INT_TYPE_INFO, + STRING_TYPE_INFO), + Array("a", "b", "c")) + + protected val minMaxCRowType = new CRowTypeInfo(MinMaxRowType) + protected val sumCRowType = new CRowTypeInfo(SumRowType) + + protected val minMaxAggregates = + Array(new LongMinWithRetractAggFunction, + new LongMaxWithRetractAggFunction).asInstanceOf[Array[AggregateFunction[_, _]]] + + protected val sumAggregates = + Array(new IntSumWithRetractAggFunction).asInstanceOf[Array[AggregateFunction[_, _]]] + + protected val minMaxAggregationStateType: RowTypeInfo = + AggregateUtil.createAccumulatorRowType(minMaxAggregates) + + protected val sumAggregationStateType: RowTypeInfo = + AggregateUtil.createAccumulatorRowType(sumAggregates) + + val minMaxCode: String = + """ + |public class MinMaxAggregateHelper + | extends org.apache.flink.table.runtime.aggregate.GeneratedAggregations { + | + | transient org.apache.flink.table.functions.aggfunctions.LongMinWithRetractAggFunction + | fmin = null; + | + | transient org.apache.flink.table.functions.aggfunctions.LongMaxWithRetractAggFunction + | fmax = null; + | + | public MinMaxAggregateHelper() throws Exception { + | + | fmin = (org.apache.flink.table.functions.aggfunctions.LongMinWithRetractAggFunction) + | org.apache.flink.table.functions.utils.UserDefinedFunctionUtils + | .deserialize("rO0ABXNyAEtvcmcuYXBhY2hlLmZsaW5rLnRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbn" + + | "MuTG9uZ01pbldpdGhSZXRyYWN0QWdnRnVuY3Rpb26oIdX_DaMPxQIAAHhyAEdvcmcuYXBhY2hlLmZsaW5rL" + + | "nRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbnMuTWluV2l0aFJldHJhY3RBZ2dGdW5jdGlvbq_ZGuzxtA_S" + + | "AgABTAADb3JkdAAVTHNjYWxhL21hdGgvT3JkZXJpbmc7eHIAMm9yZy5hcGFjaGUuZmxpbmsudGFibGUuZnV" + + | "uY3Rpb25zLkFnZ3JlZ2F0ZUZ1bmN0aW9uTcYVPtJjNfwCAAB4cgA0b3JnLmFwYWNoZS5mbGluay50YWJsZS" + + | "5mdW5jdGlvbnMuVXNlckRlZmluZWRGdW5jdGlvbi0B91QxuAyTAgAAeHBzcgAZc2NhbGEubWF0aC5PcmRlc" + + | "mluZyRMb25nJOda0iCPo2ukAgAAeHA"); + | + | fmax = (org.apache.flink.table.functions.aggfunctions.LongMaxWithRetractAggFunction) + | org.apache.flink.table.functions.utils.UserDefinedFunctionUtils + | .deserialize("rO0ABXNyAEtvcmcuYXBhY2hlLmZsaW5rLnRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbn" + + | "MuTG9uZ01heFdpdGhSZXRyYWN0QWdnRnVuY3Rpb25RmsI8azNGXwIAAHhyAEdvcmcuYXBhY2hlLmZsaW5rL" + + | "nRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbnMuTWF4V2l0aFJldHJhY3RBZ2dGdW5jdGlvbvnwowlX0_Qf" + + | "AgABTAADb3JkdAAVTHNjYWxhL21hdGgvT3JkZXJpbmc7eHIAMm9yZy5hcGFjaGUuZmxpbmsudGFibGUuZnV" + + | "uY3Rpb25zLkFnZ3JlZ2F0ZUZ1bmN0aW9uTcYVPtJjNfwCAAB4cgA0b3JnLmFwYWNoZS5mbGluay50YWJsZS" + + | "5mdW5jdGlvbnMuVXNlckRlZmluZWRGdW5jdGlvbi0B91QxuAyTAgAAeHBzcgAZc2NhbGEubWF0aC5PcmRlc" + + | "mluZyRMb25nJOda0iCPo2ukAgAAeHA"); + | } + | + | public void setAggregationResults( + | org.apache.flink.types.Row accs, + | org.apache.flink.types.Row output) { + | + | org.apache.flink.table.functions.AggregateFunction baseClass0 = + | (org.apache.flink.table.functions.AggregateFunction) fmin; + | output.setField(5, baseClass0.getValue( + | (org.apache.flink.table.functions.aggfunctions.MinWithRetractAccumulator) + | accs.getField(0))); + | + | org.apache.flink.table.functions.AggregateFunction baseClass1 = + | (org.apache.flink.table.functions.AggregateFunction) fmax; + | output.setField(6, baseClass1.getValue( + | (org.apache.flink.table.functions.aggfunctions.MaxWithRetractAccumulator) + | accs.getField(1))); + | } + | + | public void accumulate( + | org.apache.flink.types.Row accs, + | org.apache.flink.types.Row input) { + | + | fmin.accumulate( + | ((org.apache.flink.table.functions.aggfunctions.MinWithRetractAccumulator) + | accs.getField(0)), + | (java.lang.Long) input.getField(4)); + | + | fmax.accumulate( + | ((org.apache.flink.table.functions.aggfunctions.MaxWithRetractAccumulator) + | accs.getField(1)), + | (java.lang.Long) input.getField(4)); + | } + | + | public void retract( + | org.apache.flink.types.Row accs, + | org.apache.flink.types.Row input) { + | + | fmin.retract( + | ((org.apache.flink.table.functions.aggfunctions.MinWithRetractAccumulator) + | accs.getField(0)), + | (java.lang.Long) input.getField(4)); + | + | fmax.retract( + | ((org.apache.flink.table.functions.aggfunctions.MaxWithRetractAccumulator) + | accs.getField(1)), + | (java.lang.Long) input.getField(4)); + | } + | + | public org.apache.flink.types.Row createAccumulators() { + | + | org.apache.flink.types.Row accs = new org.apache.flink.types.Row(2); + | + | accs.setField( + | 0, + | fmin.createAccumulator()); + | + | accs.setField( + | 1, + | fmax.createAccumulator()); + | + | return accs; + | } + | + | public void setForwardedFields( + | org.apache.flink.types.Row input, + | org.apache.flink.types.Row output) { + | + | output.setField(0, input.getField(0)); + | output.setField(1, input.getField(1)); + | output.setField(2, input.getField(2)); + | output.setField(3, input.getField(3)); + | output.setField(4, input.getField(4)); + | } + | + | public org.apache.flink.types.Row createOutputRow() { + | return new org.apache.flink.types.Row(7); + | } + | + |/******* This test does not use the following methods *******/ + | public org.apache.flink.types.Row mergeAccumulatorsPair( + | org.apache.flink.types.Row a, + | org.apache.flink.types.Row b) { + | return null; + | } + | + | public void resetAccumulator(org.apache.flink.types.Row accs) { + | } + | + | public void setConstantFlags(org.apache.flink.types.Row output) { + | } + |} + """.stripMargin + + val sumAggCode: String = + """ + |public final class SumAggregationHelper + | extends org.apache.flink.table.runtime.aggregate.GeneratedAggregations { + | + | + |transient org.apache.flink.table.functions.aggfunctions.IntSumWithRetractAggFunction + |sum = null; + |private final org.apache.flink.table.runtime.aggregate.SingleElementIterable accIt0 = + | new org.apache.flink.table.runtime.aggregate.SingleElementIterable(); + | + | public SumAggregationHelper() throws Exception { + | + |sum = (org.apache.flink.table.functions.aggfunctions.IntSumWithRetractAggFunction) + |org.apache.flink.table.functions.utils.UserDefinedFunctionUtils + |.deserialize + |("rO0ABXNyAEpvcmcuYXBhY2hlLmZsaW5rLnRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbnMuSW50U3VtV2l0a" + + |"FJldHJhY3RBZ2dGdW5jdGlvblkfWkeNZDeDAgAAeHIAR29yZy5hcGFjaGUuZmxpbmsudGFibGUuZnVuY3Rpb25" + + |"zLmFnZ2Z1bmN0aW9ucy5TdW1XaXRoUmV0cmFjdEFnZ0Z1bmN0aW9ut2oWrOsLrs0CAAFMAAdudW1lcmljdAAUT" + + |"HNjYWxhL21hdGgvTnVtZXJpYzt4cgAyb3JnLmFwYWNoZS5mbGluay50YWJsZS5mdW5jdGlvbnMuQWdncmVnYXR" + + |"lRnVuY3Rpb25NxhU-0mM1_AIAAHhyADRvcmcuYXBhY2hlLmZsaW5rLnRhYmxlLmZ1bmN0aW9ucy5Vc2VyRGVma" + + |"W5lZEZ1bmN0aW9uLQH3VDG4DJMCAAB4cHNyACFzY2FsYS5tYXRoLk51bWVyaWMkSW50SXNJbnRlZ3JhbCTw6XA" + + |"59sPAzAIAAHhw"); + | + | + | } + | + | public final void setAggregationResults( + | org.apache.flink.types.Row accs, + | org.apache.flink.types.Row output) { + | + | org.apache.flink.table.functions.AggregateFunction baseClass0 = + | (org.apache.flink.table.functions.AggregateFunction) + | sum; + | + | output.setField( + | 1, + | baseClass0.getValue((org.apache.flink.table.functions.aggfunctions + | .SumWithRetractAccumulator) accs.getField(0))); + | } + | + | public final void accumulate( + | org.apache.flink.types.Row accs, + | org.apache.flink.types.Row input) { + | + | sum.accumulate( + | ((org.apache.flink.table.functions.aggfunctions.SumWithRetractAccumulator) accs + | .getField + | (0)), + | (java.lang.Integer) input.getField(1)); + | } + | + | + | public final void retract( + | org.apache.flink.types.Row accs, + | org.apache.flink.types.Row input) { + | } + | + | public final org.apache.flink.types.Row createAccumulators() + | { + | + | org.apache.flink.types.Row accs = + | new org.apache.flink.types.Row(1); + | + | accs.setField( + | 0, + | sum.createAccumulator()); + | + | return accs; + | } + | + | public final void setForwardedFields( + | org.apache.flink.types.Row input, + | org.apache.flink.types.Row output) + | { + | + | output.setField( + | 0, + | input.getField(0)); + | } + | + | public final void setConstantFlags(org.apache.flink.types.Row output) + | { + | + | } + | + | public final org.apache.flink.types.Row createOutputRow() { + | return new org.apache.flink.types.Row(2); + | } + | + | + | public final org.apache.flink.types.Row mergeAccumulatorsPair( + | org.apache.flink.types.Row a, + | org.apache.flink.types.Row b) + | { + | + | return a; + | + | } + | + | public final void resetAccumulator( + | org.apache.flink.types.Row accs) { + | } + |} + |""".stripMargin + + + protected val minMaxFuncName = "MinMaxAggregateHelper" + protected val sumFuncName = "SumAggregationHelper" + + protected val genMinMaxAggFunction = GeneratedAggregationsFunction(minMaxFuncName, minMaxCode) + protected val genSumAggFunction = GeneratedAggregationsFunction(sumFuncName, sumAggCode) + def createHarnessTester[IN, OUT, KEY]( operator: OneInputStreamOperator[IN, OUT], keySelector: KeySelector[IN, KEY], diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala new file mode 100644 index 0000000000000..13cf27c10a6a2 --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala @@ -0,0 +1,147 @@ +/* + * 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.typeinfo.BasicTypeInfo +import org.apache.flink.streaming.api.operators.KeyedProcessOperator +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord +import org.apache.flink.table.runtime.aggregate._ +import org.apache.flink.table.runtime.harness.HarnessTestBase._ +import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.types.Row +import org.junit.Test + +class NonWindowHarnessTest extends HarnessTestBase { + + @Test + def testProcTimeNonWindow(): Unit = { + + val processFunction = new KeyedProcessOperator[String, CRow, CRow]( + new GroupAggProcessFunction( + genSumAggFunction, + sumAggregationStateType, + false, + qConfig)) + + val testHarness = + createHarnessTester( + processFunction, + new TupleRowKeySelector[String](2), + BasicTypeInfo.STRING_TYPE_INFO) + + testHarness.open() + + // register cleanup timer with 3001 + testHarness.setProcessingTime(1) + + testHarness.processElement(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt, "aaa"), true), 1)) + testHarness.processElement(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt, "bbb"), true), 1)) + testHarness.processElement(new StreamRecord(CRow(Row.of(3L: JLong, 2: JInt, "aaa"), true), 1)) + testHarness.processElement(new StreamRecord(CRow(Row.of(4L: JLong, 3: JInt, "aaa"), true), 1)) + + // trigger cleanup timer and register cleanup timer with 6002 + testHarness.setProcessingTime(3002) + testHarness.processElement(new StreamRecord(CRow(Row.of(5L: JLong, 4: JInt, "aaa"), true), 1)) + testHarness.processElement(new StreamRecord(CRow(Row.of(6L: JLong, 2: JInt, "bbb"), true), 1)) + testHarness.processElement(new StreamRecord(CRow(Row.of(7L: JLong, 5: JInt, "aaa"), true), 1)) + testHarness.processElement(new StreamRecord(CRow(Row.of(8L: JLong, 6: JInt, "aaa"), true), 1)) + testHarness.processElement(new StreamRecord(CRow(Row.of(9L: JLong, 7: JInt, "aaa"), true), 1)) + testHarness.processElement(new StreamRecord(CRow(Row.of(10L: JLong, 3: JInt, "bbb"), true), 1)) + + val result = testHarness.getOutput + + val expectedOutput = new ConcurrentLinkedQueue[Object]() + + expectedOutput.add(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt), true), 1)) + expectedOutput.add(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt), true), 1)) + expectedOutput.add(new StreamRecord(CRow(Row.of(3L: JLong, 3: JInt), true), 1)) + expectedOutput.add(new StreamRecord(CRow(Row.of(4L: JLong, 6: JInt), true), 1)) + expectedOutput.add(new StreamRecord(CRow(Row.of(5L: JLong, 4: JInt), true), 1)) + expectedOutput.add(new StreamRecord(CRow(Row.of(6L: JLong, 2: JInt), true), 1)) + expectedOutput.add(new StreamRecord(CRow(Row.of(7L: JLong, 9: JInt), true), 1)) + expectedOutput.add(new StreamRecord(CRow(Row.of(8L: JLong, 15: JInt), true), 1)) + expectedOutput.add(new StreamRecord(CRow(Row.of(9L: JLong, 22: JInt), true), 1)) + expectedOutput.add(new StreamRecord(CRow(Row.of(10L: JLong, 5: JInt), true), 1)) + + verify(expectedOutput, result, new RowResultSortComparator(6)) + + testHarness.close() + } + + @Test + def testProcTimeNonWindowWithRetract(): Unit = { + + val processFunction = new KeyedProcessOperator[String, CRow, CRow]( + new GroupAggProcessFunction( + genSumAggFunction, + sumAggregationStateType, + true, + qConfig)) + + val testHarness = + createHarnessTester( + processFunction, + new TupleRowKeySelector[String](2), + BasicTypeInfo.STRING_TYPE_INFO) + + testHarness.open() + + // register cleanup timer with 3001 + testHarness.setProcessingTime(1) + + testHarness.processElement(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt, "aaa"), true), 1)) + testHarness.processElement(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt, "bbb"), true), 2)) + testHarness.processElement(new StreamRecord(CRow(Row.of(3L: JLong, 2: JInt, "aaa"), true), 3)) + testHarness.processElement(new StreamRecord(CRow(Row.of(4L: JLong, 3: JInt, "ccc"), true), 4)) + + // trigger cleanup timer and register cleanup timer with 6002 + testHarness.setProcessingTime(3002) + testHarness.processElement(new StreamRecord(CRow(Row.of(5L: JLong, 4: JInt, "aaa"), true), 5)) + testHarness.processElement(new StreamRecord(CRow(Row.of(6L: JLong, 2: JInt, "bbb"), true), 6)) + testHarness.processElement(new StreamRecord(CRow(Row.of(7L: JLong, 5: JInt, "aaa"), true), 7)) + testHarness.processElement(new StreamRecord(CRow(Row.of(8L: JLong, 6: JInt, "eee"), true), 8)) + testHarness.processElement(new StreamRecord(CRow(Row.of(9L: JLong, 7: JInt, "aaa"), true), 9)) + testHarness.processElement(new StreamRecord(CRow(Row.of(10L: JLong, 3: JInt, "bbb"), true), 10)) + + val result = testHarness.getOutput + + val expectedOutput = new ConcurrentLinkedQueue[Object]() + + expectedOutput.add(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt), true), 1)) + expectedOutput.add(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt), true), 2)) + expectedOutput.add(new StreamRecord(CRow(Row.of(3L: JLong, 1: JInt), false), 3)) + expectedOutput.add(new StreamRecord(CRow(Row.of(3L: JLong, 3: JInt), true), 3)) + expectedOutput.add(new StreamRecord(CRow(Row.of(4L: JLong, 3: JInt), true), 4)) + expectedOutput.add(new StreamRecord(CRow(Row.of(5L: JLong, 4: JInt), true), 5)) + expectedOutput.add(new StreamRecord(CRow(Row.of(6L: JLong, 2: JInt), true), 6)) + expectedOutput.add(new StreamRecord(CRow(Row.of(7L: JLong, 4: JInt), false), 7)) + expectedOutput.add(new StreamRecord(CRow(Row.of(7L: JLong, 9: JInt), true), 7)) + expectedOutput.add(new StreamRecord(CRow(Row.of(8L: JLong, 6: JInt), true), 8)) + expectedOutput.add(new StreamRecord(CRow(Row.of(9L: JLong, 9: JInt), false), 9)) + expectedOutput.add(new StreamRecord(CRow(Row.of(9L: JLong, 16: JInt), true), 9)) + expectedOutput.add(new StreamRecord(CRow(Row.of(10L: JLong, 2: JInt), false), 10)) + expectedOutput.add(new StreamRecord(CRow(Row.of(10L: JLong, 5: JInt), true), 10)) + + verify(expectedOutput, result, new RowResultSortComparator(0)) + + testHarness.close() + } +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala index 56ca85c18e29d..0b6dceefa3db4 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala @@ -18,180 +18,31 @@ package org.apache.flink.table.runtime.harness import java.lang.{Integer => JInt, Long => JLong} -import java.util.concurrent.ConcurrentLinkedQueue +import java.util.concurrent.{ConcurrentLinkedQueue, TimeUnit} -import org.apache.flink.api.common.typeinfo.BasicTypeInfo._ -import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} -import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.common.time.Time +import org.apache.flink.api.common.typeinfo.BasicTypeInfo import org.apache.flink.streaming.api.operators.KeyedProcessOperator import org.apache.flink.streaming.runtime.streamrecord.StreamRecord -import org.apache.flink.table.codegen.GeneratedAggregationsFunction -import org.apache.flink.table.functions.AggregateFunction -import org.apache.flink.table.functions.aggfunctions.{LongMaxWithRetractAggFunction, LongMinWithRetractAggFunction} +import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.table.runtime.aggregate._ import org.apache.flink.table.runtime.harness.HarnessTestBase._ -import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} +import org.apache.flink.table.runtime.types.CRow import org.apache.flink.types.Row import org.junit.Test class OverWindowHarnessTest extends HarnessTestBase{ - private val rT = new RowTypeInfo(Array[TypeInformation[_]]( - INT_TYPE_INFO, - LONG_TYPE_INFO, - INT_TYPE_INFO, - STRING_TYPE_INFO, - LONG_TYPE_INFO), - Array("a", "b", "c", "d", "e")) - - private val cRT = new CRowTypeInfo(rT) - - private val aggregates = - Array(new LongMinWithRetractAggFunction, - new LongMaxWithRetractAggFunction).asInstanceOf[Array[AggregateFunction[_, _]]] - private val aggregationStateType: RowTypeInfo = AggregateUtil.createAccumulatorRowType(aggregates) - - val funcCode: String = - """ - |public class BoundedOverAggregateHelper - | extends org.apache.flink.table.runtime.aggregate.GeneratedAggregations { - | - | transient org.apache.flink.table.functions.aggfunctions.LongMinWithRetractAggFunction - | fmin = null; - | - | transient org.apache.flink.table.functions.aggfunctions.LongMaxWithRetractAggFunction - | fmax = null; - | - | public BoundedOverAggregateHelper() throws Exception { - | - | fmin = (org.apache.flink.table.functions.aggfunctions.LongMinWithRetractAggFunction) - | org.apache.flink.table.functions.utils.UserDefinedFunctionUtils - | .deserialize("rO0ABXNyAEtvcmcuYXBhY2hlLmZsaW5rLnRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbn" + - | "MuTG9uZ01pbldpdGhSZXRyYWN0QWdnRnVuY3Rpb26oIdX_DaMPxQIAAHhyAEdvcmcuYXBhY2hlLmZsaW5rL" + - | "nRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbnMuTWluV2l0aFJldHJhY3RBZ2dGdW5jdGlvbq_ZGuzxtA_S" + - | "AgABTAADb3JkdAAVTHNjYWxhL21hdGgvT3JkZXJpbmc7eHIAMm9yZy5hcGFjaGUuZmxpbmsudGFibGUuZnV" + - | "uY3Rpb25zLkFnZ3JlZ2F0ZUZ1bmN0aW9uTcYVPtJjNfwCAAB4cgA0b3JnLmFwYWNoZS5mbGluay50YWJsZS" + - | "5mdW5jdGlvbnMuVXNlckRlZmluZWRGdW5jdGlvbi0B91QxuAyTAgAAeHBzcgAZc2NhbGEubWF0aC5PcmRlc" + - | "mluZyRMb25nJOda0iCPo2ukAgAAeHA"); - | - | fmax = (org.apache.flink.table.functions.aggfunctions.LongMaxWithRetractAggFunction) - | org.apache.flink.table.functions.utils.UserDefinedFunctionUtils - | .deserialize("rO0ABXNyAEtvcmcuYXBhY2hlLmZsaW5rLnRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbn" + - | "MuTG9uZ01heFdpdGhSZXRyYWN0QWdnRnVuY3Rpb25RmsI8azNGXwIAAHhyAEdvcmcuYXBhY2hlLmZsaW5rL" + - | "nRhYmxlLmZ1bmN0aW9ucy5hZ2dmdW5jdGlvbnMuTWF4V2l0aFJldHJhY3RBZ2dGdW5jdGlvbvnwowlX0_Qf" + - | "AgABTAADb3JkdAAVTHNjYWxhL21hdGgvT3JkZXJpbmc7eHIAMm9yZy5hcGFjaGUuZmxpbmsudGFibGUuZnV" + - | "uY3Rpb25zLkFnZ3JlZ2F0ZUZ1bmN0aW9uTcYVPtJjNfwCAAB4cgA0b3JnLmFwYWNoZS5mbGluay50YWJsZS" + - | "5mdW5jdGlvbnMuVXNlckRlZmluZWRGdW5jdGlvbi0B91QxuAyTAgAAeHBzcgAZc2NhbGEubWF0aC5PcmRlc" + - | "mluZyRMb25nJOda0iCPo2ukAgAAeHA"); - | } - | - | public void setAggregationResults( - | org.apache.flink.types.Row accs, - | org.apache.flink.types.Row output) { - | - | org.apache.flink.table.functions.AggregateFunction baseClass0 = - | (org.apache.flink.table.functions.AggregateFunction) fmin; - | output.setField(5, baseClass0.getValue( - | (org.apache.flink.table.functions.aggfunctions.MinWithRetractAccumulator) - | accs.getField(0))); - | - | org.apache.flink.table.functions.AggregateFunction baseClass1 = - | (org.apache.flink.table.functions.AggregateFunction) fmax; - | output.setField(6, baseClass1.getValue( - | (org.apache.flink.table.functions.aggfunctions.MaxWithRetractAccumulator) - | accs.getField(1))); - | } - | - | public void accumulate( - | org.apache.flink.types.Row accs, - | org.apache.flink.types.Row input) { - | - | fmin.accumulate( - | ((org.apache.flink.table.functions.aggfunctions.MinWithRetractAccumulator) - | accs.getField(0)), - | (java.lang.Long) input.getField(4)); - | - | fmax.accumulate( - | ((org.apache.flink.table.functions.aggfunctions.MaxWithRetractAccumulator) - | accs.getField(1)), - | (java.lang.Long) input.getField(4)); - | } - | - | public void retract( - | org.apache.flink.types.Row accs, - | org.apache.flink.types.Row input) { - | - | fmin.retract( - | ((org.apache.flink.table.functions.aggfunctions.MinWithRetractAccumulator) - | accs.getField(0)), - | (java.lang.Long) input.getField(4)); - | - | fmax.retract( - | ((org.apache.flink.table.functions.aggfunctions.MaxWithRetractAccumulator) - | accs.getField(1)), - | (java.lang.Long) input.getField(4)); - | } - | - | public org.apache.flink.types.Row createAccumulators() { - | - | org.apache.flink.types.Row accs = new org.apache.flink.types.Row(2); - | - | accs.setField( - | 0, - | fmin.createAccumulator()); - | - | accs.setField( - | 1, - | fmax.createAccumulator()); - | - | return accs; - | } - | - | public void setForwardedFields( - | org.apache.flink.types.Row input, - | org.apache.flink.types.Row output) { - | - | output.setField(0, input.getField(0)); - | output.setField(1, input.getField(1)); - | output.setField(2, input.getField(2)); - | output.setField(3, input.getField(3)); - | output.setField(4, input.getField(4)); - | } - | - | public org.apache.flink.types.Row createOutputRow() { - | return new org.apache.flink.types.Row(7); - | } - | - |/******* This test does not use the following methods *******/ - | public org.apache.flink.types.Row mergeAccumulatorsPair( - | org.apache.flink.types.Row a, - | org.apache.flink.types.Row b) { - | return null; - | } - | - | public void resetAccumulator(org.apache.flink.types.Row accs) { - | } - | - | public void setConstantFlags(org.apache.flink.types.Row output) { - | } - |} - """.stripMargin - - - private val funcName = "BoundedOverAggregateHelper" - - private val genAggFunction = GeneratedAggregationsFunction(funcName, funcCode) - - @Test def testProcTimeBoundedRowsOver(): Unit = { val processFunction = new KeyedProcessOperator[String, CRow, CRow]( new ProcTimeBoundedRowsOver( - genAggFunction, + genMinMaxAggFunction, 2, - aggregationStateType, - cRT)) + minMaxAggregationStateType, + minMaxCRowType, + qConfig)) val testHarness = createHarnessTester(processFunction,new TupleRowKeySelector[Integer](0),BasicTypeInfo @@ -199,6 +50,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.open() + // register cleanup timer with 3001 testHarness.setProcessingTime(1) testHarness.processElement(new StreamRecord( @@ -209,6 +61,8 @@ class OverWindowHarnessTest extends HarnessTestBase{ CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong), true), 1)) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong), true), 1)) + + testHarness.setProcessingTime(1100) testHarness.processElement(new StreamRecord( CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong), true), 1)) testHarness.processElement(new StreamRecord( @@ -220,15 +74,19 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), true), 1)) - testHarness.setProcessingTime(2) + // trigger cleanup timer and register cleanup timer with 6001 + testHarness.setProcessingTime(3001) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 2)) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 2)) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 2)) + + // using historical data and register cleanup timer with 9000 + testHarness.setProcessingTime(6000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 2)) + CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 2)) testHarness.processElement(new StreamRecord( CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 2)) @@ -265,7 +123,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 20L: JLong, 30L: JLong), true), 1)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 6L: JLong, 7L: JLong), true), 2)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 7L: JLong, 7L: JLong), true), 2)) expectedOutput.add(new StreamRecord( CRow( Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 7L: JLong, 8L: JLong), true), 2)) @@ -277,7 +135,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 9L: JLong, 10L: JLong), true), 2)) expectedOutput.add(new StreamRecord( CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 30L: JLong, 40L: JLong), true), 2)) + Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 2)) verify(expectedOutput, result, new RowResultSortComparator(6)) @@ -292,10 +150,11 @@ class OverWindowHarnessTest extends HarnessTestBase{ val processFunction = new KeyedProcessOperator[String, CRow, CRow]( new ProcTimeBoundedRangeOver( - genAggFunction, - 1000, - aggregationStateType, - cRT)) + genMinMaxAggFunction, + 4000, + minMaxAggregationStateType, + minMaxCRowType, + qConfig)) val testHarness = createHarnessTester( @@ -305,6 +164,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.open() + // register cleanup timer with 3003 testHarness.setProcessingTime(3) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 0)) @@ -314,6 +174,9 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.setProcessingTime(4) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong), true), 0)) + + // trigger cleanup timer and register cleanup timer with 6003 + testHarness.setProcessingTime(3003) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong), true), 0)) testHarness.processElement(new StreamRecord( @@ -323,9 +186,10 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong), true), 0)) - testHarness.setProcessingTime(6) + // using historical data and register cleanup timer with 9002 + testHarness.setProcessingTime(6002) - testHarness.setProcessingTime(1002) + testHarness.setProcessingTime(7002) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong), true), 0)) testHarness.processElement(new StreamRecord( @@ -333,15 +197,15 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), true), 0)) - testHarness.setProcessingTime(1003) + // using historical data and register cleanup timer with 14002 + testHarness.setProcessingTime(11002) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 0)) - testHarness.setProcessingTime(1004) + testHarness.setProcessingTime(11004) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 0)) - testHarness.setProcessingTime(1005) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 0)) testHarness.processElement(new StreamRecord( @@ -349,7 +213,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 0)) - testHarness.setProcessingTime(1006) + testHarness.setProcessingTime(11006) val result = testHarness.getOutput @@ -364,40 +228,40 @@ class OverWindowHarnessTest extends HarnessTestBase{ Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true), 4)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 1L: JLong, 3L: JLong), true), 5)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true), 5)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true), 5)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 3L: JLong, 4L: JLong), true), 3004)) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true), 5)) + CRow(Row.of( + 2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong, 20L: JLong, 20L: JLong), true), 3004)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong, 1L: JLong, 4L: JLong), true), 6)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong, 4L: JLong, 4L: JLong), true), 6)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 1L: JLong, 6L: JLong), true), 1003)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 5L: JLong, 6L: JLong), true), 7003)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 1L: JLong, 6L: JLong), true), 1003)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 5L: JLong, 6L: JLong), true), 7003)) expectedOutput.add(new StreamRecord( CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true), 1003)) + Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 30L: JLong, 30L: JLong), true), 7003)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 1L: JLong, 7L: JLong), true), 1004)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 7L: JLong, 7L: JLong), true), 11003)) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 2L: JLong, 8L: JLong), true), 1005)) + CRow(Row.of( + 1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 7L: JLong, 10L: JLong), true), 11005)) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 4L: JLong, 10L: JLong), true), 1006)) + CRow(Row.of( + 1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 7L: JLong, 10L: JLong), true), 11005)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 4L: JLong, 10L: JLong), true), 1006)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 7L: JLong, 10L: JLong), true), 11005)) expectedOutput.add(new StreamRecord( CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 30L: JLong, 40L: JLong), true), 1006)) + Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 11005)) verify(expectedOutput, result, new RowResultSortComparator(6)) @@ -409,8 +273,9 @@ class OverWindowHarnessTest extends HarnessTestBase{ val processFunction = new KeyedProcessOperator[String, CRow, CRow]( new ProcTimeUnboundedPartitionedOver( - genAggFunction, - aggregationStateType)) + genMinMaxAggFunction, + minMaxAggregationStateType, + qConfig)) val testHarness = createHarnessTester( @@ -420,6 +285,9 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.open() + // register cleanup timer with 4003 + testHarness.setProcessingTime(1003) + testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 0)) testHarness.processElement(new StreamRecord( @@ -438,18 +306,19 @@ class OverWindowHarnessTest extends HarnessTestBase{ CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong), true), 0)) testHarness.processElement(new StreamRecord( CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), true), 0)) - - testHarness.setProcessingTime(1003) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 1003)) + CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 0)) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 1003)) + CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 0)) + + // trigger cleanup timer and register cleanup timer with 8003 + testHarness.setProcessingTime(5003) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 1003)) + CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 5003)) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 1003)) + CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 5003)) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 1003)) + CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 5003)) val result = testHarness.getOutput @@ -484,19 +353,19 @@ class OverWindowHarnessTest extends HarnessTestBase{ Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true), 0)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 1L: JLong, 7L: JLong), true), 1003)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 1L: JLong, 7L: JLong), true), 0)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 1L: JLong, 8L: JLong), true), 1003)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 1L: JLong, 8L: JLong), true), 0)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 1L: JLong, 9L: JLong), true), 1003)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 9L: JLong, 9L: JLong), true), 5003)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 1L: JLong, 10L: JLong), true), 1003)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 9L: JLong, 10L: JLong), true), 5003)) expectedOutput.add(new StreamRecord( CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 10L: JLong, 40L: JLong), true), 1003)) + Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 5003)) verify(expectedOutput, result, new RowResultSortComparator(6)) testHarness.close() @@ -507,13 +376,16 @@ class OverWindowHarnessTest extends HarnessTestBase{ */ @Test def testRowTimeBoundedRangeOver(): Unit = { + qConfig = new StreamQueryConfig() + .withIdleStateRetentionTime(Time.seconds(3), Time.seconds(5)) val processFunction = new KeyedProcessOperator[String, CRow, CRow]( new RowTimeBoundedRangeOver( - genAggFunction, - aggregationStateType, - cRT, - 4000)) + genMinMaxAggFunction, + minMaxAggregationStateType, + minMaxCRowType, + 4000, + qConfig)) val testHarness = createHarnessTester( @@ -523,6 +395,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.open() + // register cleanup timer with 4003 testHarness.processWatermark(1) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 2)) @@ -547,31 +420,36 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(2: JInt, 11L: JLong, 1: JInt, "bbb", 25L: JLong), true), 4801)) - testHarness.processWatermark(6500) + testHarness.processWatermark(9802) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong), true), 6501)) + CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong), true), 9803)) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong), true), 6501)) + CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong), true), 9804)) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), true), 6501)) + CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), true), 9805)) - testHarness.processWatermark(7000) + // using historical data and register cleanup timer with 19802 + testHarness.processWatermark(14801) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 7001)) + CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 14802)) - testHarness.processWatermark(8000) + testHarness.processWatermark(14802) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 8001)) + CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 14803)) - testHarness.processWatermark(12000) + testHarness.processWatermark(15000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 12001)) + CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 15001)) + + testHarness.processWatermark(19900) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 12001)) + CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 19901)) + + testHarness.processWatermark(22000) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 12001)) + CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 22001)) - testHarness.processWatermark(19000) + testHarness.processWatermark(23000) val result = testHarness.getOutput @@ -598,28 +476,28 @@ class OverWindowHarnessTest extends HarnessTestBase{ Row.of(2: JInt, 11L: JLong, 1: JInt, "bbb", 25L: JLong, 25L: JLong, 25L: JLong), true), 4801)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 2L: JLong, 6L: JLong), true), 6501)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 5L: JLong, 5L: JLong), true), 9803)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 2L: JLong, 6L: JLong), true), 6501)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 5L: JLong, 6L: JLong), true), 9804)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 2L: JLong, 7L: JLong), true), 7001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 7L: JLong, 7L: JLong), true), 14802)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 2L: JLong, 8L: JLong), true), 8001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 7L: JLong, 8L: JLong), true), 14803)) expectedOutput.add(new StreamRecord( CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 25L: JLong, 30L: JLong), true), 6501)) + Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 30L: JLong, 30L: JLong), true), 9805)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 8L: JLong, 10L: JLong), true), 12001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 7L: JLong, 9L: JLong), true), 15001)) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 8L: JLong, 10L: JLong), true), 12001)) + CRow(Row.of( + 1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 10L: JLong, 10L: JLong), true), 19901)) expectedOutput.add(new StreamRecord( CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 12001)) + Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 22001)) verify(expectedOutput, result, new RowResultSortComparator(6)) testHarness.close() @@ -630,10 +508,11 @@ class OverWindowHarnessTest extends HarnessTestBase{ val processFunction = new KeyedProcessOperator[String, CRow, CRow]( new RowTimeBoundedRowsOver( - genAggFunction, - aggregationStateType, - cRT, - 3)) + genMinMaxAggFunction, + minMaxAggregationStateType, + minMaxCRowType, + 3, + qConfig)) val testHarness = createHarnessTester( @@ -643,6 +522,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.open() + // register cleanup timer with 3801 testHarness.processWatermark(800) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 801)) @@ -651,6 +531,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong), true), 2501)) + // trigger cleanup timer and register cleanup timer with 7001 testHarness.processWatermark(4000) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong), true), 4001)) @@ -682,10 +563,13 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processWatermark(12000) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 12001)) + + // trigger cleanup timer and register cleanup timer with 18002 + testHarness.processWatermark(15001) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 12001)) + CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 15002)) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 12001)) + CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 15002)) testHarness.processWatermark(19000) @@ -701,10 +585,10 @@ class OverWindowHarnessTest extends HarnessTestBase{ Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true), 2501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true), 4001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 2L: JLong, 2L: JLong), true), 4001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true), 4001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 2L: JLong, 3L: JLong), true), 4001)) expectedOutput.add(new StreamRecord( CRow( Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true), 4001)) @@ -719,7 +603,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 4L: JLong, 6L: JLong), true), 6501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true), 6501)) + Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 30L: JLong, 30L: JLong), true), 6501)) expectedOutput.add(new StreamRecord( CRow( Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 5L: JLong, 7L: JLong), true), 7001)) @@ -728,13 +612,13 @@ class OverWindowHarnessTest extends HarnessTestBase{ Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 6L: JLong, 8L: JLong), true), 8001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 7L: JLong, 9L: JLong), true), 12001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 9L: JLong, 9L: JLong), true), 12001)) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 8L: JLong, 10L: JLong), true), 12001)) + CRow(Row.of( + 1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 10L: JLong, 10L: JLong), true), 15002)) expectedOutput.add(new StreamRecord( CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 20L: JLong, 40L: JLong), true), 12001)) + Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 15002)) verify(expectedOutput, result, new RowResultSortComparator(6)) testHarness.close() @@ -748,9 +632,10 @@ class OverWindowHarnessTest extends HarnessTestBase{ val processFunction = new KeyedProcessOperator[String, CRow, CRow]( new RowTimeUnboundedRangeOver( - genAggFunction, - aggregationStateType, - cRT)) + genMinMaxAggFunction, + minMaxAggregationStateType, + minMaxCRowType, + qConfig)) val testHarness = createHarnessTester( @@ -760,6 +645,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.open() + // register cleanup timer with 3801 testHarness.processWatermark(800) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 801)) @@ -768,6 +654,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong), true), 2501)) + // trigger cleanup timer and register cleanup timer with 7001 testHarness.processWatermark(4000) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong), true), 4001)) @@ -780,6 +667,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong), true), 4801)) + // using historical data and register cleanup timer with 9501 testHarness.processWatermark(6500) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong), true), 6501)) @@ -792,10 +680,12 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 7001)) - testHarness.processWatermark(8000) + // trigger cleanup timer and register cleanup timer with 13001 + testHarness.processWatermark(10000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 8001)) + CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 10001)) + // using historical data and register cleanup timer with 15001 testHarness.processWatermark(12000) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 12001)) @@ -819,40 +709,40 @@ class OverWindowHarnessTest extends HarnessTestBase{ Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true), 2501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 1L: JLong, 3L: JLong), true), 4001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 2L: JLong, 3L: JLong), true), 4001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true), 4001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 2L: JLong, 3L: JLong), true), 4001)) expectedOutput.add(new StreamRecord( CRow( Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true), 4001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong, 1L: JLong, 4L: JLong), true), 4801)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong, 2L: JLong, 4L: JLong), true), 4801)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 1L: JLong, 6L: JLong), true), 6501)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 2L: JLong, 6L: JLong), true), 6501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 1L: JLong, 6L: JLong), true), 6501)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 2L: JLong, 6L: JLong), true), 6501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true), 6501)) + Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 30L: JLong, 30L: JLong), true), 6501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 1L: JLong, 7L: JLong), true), 7001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 2L: JLong, 7L: JLong), true), 7001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 1L: JLong, 8L: JLong), true), 8001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 8L: JLong, 8L: JLong), true), 10001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 1L: JLong, 10L: JLong), true), 12001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 8L: JLong, 10L: JLong), true), 12001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 1L: JLong, 10L: JLong), true), 12001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 8L: JLong, 10L: JLong), true), 12001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 10L: JLong, 40L: JLong), true), 12001)) + Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 12001)) verify(expectedOutput, result, new RowResultSortComparator(6)) testHarness.close() @@ -863,9 +753,10 @@ class OverWindowHarnessTest extends HarnessTestBase{ val processFunction = new KeyedProcessOperator[String, CRow, CRow]( new RowTimeUnboundedRowsOver( - genAggFunction, - aggregationStateType, - cRT)) + genMinMaxAggFunction, + minMaxAggregationStateType, + minMaxCRowType, + qConfig)) val testHarness = createHarnessTester( @@ -875,6 +766,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.open() + // register cleanup timer with 3801 testHarness.processWatermark(800) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 801)) @@ -883,6 +775,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong), true), 2501)) + // trigger cleanup timer and register cleanup timer with 7001 testHarness.processWatermark(4000) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong), true), 4001)) @@ -895,6 +788,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong), true), 4801)) + // using historical data and register cleanup timer with 9501 testHarness.processWatermark(6500) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong), true), 6501)) @@ -911,6 +805,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 8001)) + // trigger cleanup timer and register cleanup timer with 15001 testHarness.processWatermark(12000) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 12001)) @@ -933,40 +828,40 @@ class OverWindowHarnessTest extends HarnessTestBase{ Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true), 2501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true), 4001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 2L: JLong, 2L: JLong), true), 4001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true), 4001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 2L: JLong, 3L: JLong), true), 4001)) expectedOutput.add(new StreamRecord( CRow( Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true), 4001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong, 1L: JLong, 4L: JLong), true), 4801)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong, 2L: JLong, 4L: JLong), true), 4801)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 1L: JLong, 5L: JLong), true), 6501)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 2L: JLong, 5L: JLong), true), 6501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 1L: JLong, 6L: JLong), true), 6501)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 2L: JLong, 6L: JLong), true), 6501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true), 6501)) + Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 30L: JLong, 30L: JLong), true), 6501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 1L: JLong, 7L: JLong), true), 7001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 2L: JLong, 7L: JLong), true), 7001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 1L: JLong, 8L: JLong), true), 8001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 2L: JLong, 8L: JLong), true), 8001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 1L: JLong, 9L: JLong), true), 12001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 9L: JLong, 9L: JLong), true), 12001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 1L: JLong, 10L: JLong), true), 12001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 9L: JLong, 10L: JLong), true), 12001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 10L: JLong, 40L: JLong), true), 12001)) + Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 12001)) verify(expectedOutput, result, new RowResultSortComparator(6)) testHarness.close() From 4c42ef21e9ee755c48e3da67acb4da2ede977db6 Mon Sep 17 00:00:00 2001 From: sunjincheng121 Date: Thu, 11 May 2017 13:14:49 +0800 Subject: [PATCH 3/3] 1. Uniform hump name, change all `qConfig` and `qConf` to `queryConfig`. 2. add warn log into DataStreamOverAggregate. 3. fix cleanup logic 4. extend test cases that cover the corner cases of the clean up logic. --- .../table/api/BatchTableEnvironment.scala | 8 +- .../apache/flink/table/api/QueryConfig.scala | 2 +- .../table/api/StreamTableEnvironment.scala | 38 ++-- .../api/java/StreamTableEnvironment.scala | 34 ++-- .../api/scala/StreamTableEnvironment.scala | 20 ++- .../table/api/scala/TableConversions.scala | 13 +- .../org/apache/flink/table/api/table.scala | 6 +- .../nodes/datastream/DataStreamCalc.scala | 5 +- .../datastream/DataStreamCorrelate.scala | 4 +- .../datastream/DataStreamGroupAggregate.scala | 11 +- .../DataStreamGroupWindowAggregate.scala | 4 +- .../datastream/DataStreamOverAggregate.scala | 47 +++-- .../plan/nodes/datastream/DataStreamRel.scala | 4 +- .../nodes/datastream/DataStreamScan.scala | 2 +- .../nodes/datastream/DataStreamUnion.scala | 6 +- .../nodes/datastream/DataStreamValues.scala | 2 +- .../datastream/StreamTableSourceScan.scala | 2 +- .../runtime/aggregate/AggregateUtil.scala | 28 ++- .../aggregate/GroupAggProcessFunction.scala | 5 +- .../aggregate/ProcTimeBoundedRangeOver.scala | 4 +- .../aggregate/ProcTimeBoundedRowsOver.scala | 11 +- .../ProcTimeUnboundedNonPartitionedOver.scala | 4 +- .../ProcTimeUnboundedPartitionedOver.scala | 4 +- .../ProcessFunctionWithCleanupState.scala | 35 +--- .../aggregate/RowTimeBoundedRangeOver.scala | 20 +-- .../aggregate/RowTimeBoundedRowsOver.scala | 21 +-- .../aggregate/RowTimeUnboundedOver.scala | 30 +--- .../table/GroupAggregationsITCase.scala | 12 +- .../runtime/harness/HarnessTestBase.scala | 5 - .../harness/NonWindowHarnessTest.scala | 30 ++-- .../harness/OverWindowHarnessTest.scala | 168 ++++++++---------- .../table/utils/MockTableEnvironment.scala | 2 +- 32 files changed, 263 insertions(+), 324 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala index f33c187fa04b1..3c0f51b719b9b 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala @@ -113,17 +113,17 @@ abstract class BatchTableEnvironment( * * @param table The [[Table]] to write. * @param sink The [[TableSink]] to write the [[Table]] to. - * @param qConfig The configuration for the query to generate. + * @param queryConfig The configuration for the query to generate. * @tparam T The expected type of the [[DataSet]] which represents the [[Table]]. */ override private[flink] def writeToSink[T]( table: Table, sink: TableSink[T], - qConfig: QueryConfig): Unit = { + queryConfig: QueryConfig): Unit = { // We do not pass the configuration on, because there is nothing to configure for batch queries. - val bQConfig = qConfig match { - case batchConfig: BatchQueryConfig => batchConfig + queryConfig match { + case _: BatchQueryConfig => case _ => throw new TableException("BatchQueryConfig required to configure batch query.") } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala index f6600ba59a05a..c8fbab7e05ddb 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala @@ -31,7 +31,7 @@ class BatchQueryConfig private[table] extends QueryConfig /** * The [[StreamQueryConfig]] holds parameters to configure the behavior of streaming queries. * - * An empty [[StreamQueryConfig]] can be generated using the [[StreamTableEnvironment.qConf]] + * An empty [[StreamQueryConfig]] can be generated using the [[StreamTableEnvironment.queryConfig]] * method. */ class StreamQueryConfig private[table] extends QueryConfig { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala index c594d4c300a71..d68da04799717 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala @@ -81,7 +81,7 @@ abstract class StreamTableEnvironment( // the naming pattern for internally registered tables. private val internalNamePattern = "^_DataStreamTable_[0-9]+$".r - def qConf: StreamQueryConfig = new StreamQueryConfig + def queryConfig: StreamQueryConfig = new StreamQueryConfig /** * Checks if the chosen table name is valid. @@ -128,16 +128,16 @@ abstract class StreamTableEnvironment( * * @param table The [[Table]] to write. * @param sink The [[TableSink]] to write the [[Table]] to. - * @param qConfig The configuration for the query to generate. + * @param queryConfig The configuration for the query to generate. * @tparam T The expected type of the [[DataStream]] which represents the [[Table]]. */ override private[flink] def writeToSink[T]( table: Table, sink: TableSink[T], - qConfig: QueryConfig): Unit = { + queryConfig: QueryConfig): Unit = { // Check query configuration - val sQConf = qConfig match { + val streamQueryConfig = queryConfig match { case streamConfig: StreamQueryConfig => streamConfig case _ => throw new TableException("StreamQueryConfig required to configure stream query.") @@ -150,7 +150,11 @@ abstract class StreamTableEnvironment( val outputType = sink.getOutputType // translate the Table into a DataStream and provide the type that the TableSink expects. val result: DataStream[T] = - translate(table, sQConf, updatesAsRetraction = true, withChangeFlag = true)(outputType) + translate( + table, + streamQueryConfig, + updatesAsRetraction = true, + withChangeFlag = true)(outputType) // Give the DataStream to the TableSink to emit it. retractSink.asInstanceOf[RetractStreamTableSink[Any]] .emitDataStream(result.asInstanceOf[DataStream[JTuple2[JBool, Any]]]) @@ -176,7 +180,7 @@ abstract class StreamTableEnvironment( translate( optimizedPlan, table.getRelNode.getRowType, - sQConf, + streamQueryConfig, withChangeFlag = true)(outputType) // Give the DataStream to the TableSink to emit it. upsertSink.asInstanceOf[UpsertStreamTableSink[Any]] @@ -196,7 +200,7 @@ abstract class StreamTableEnvironment( translate( optimizedPlan, table.getRelNode.getRowType, - sQConf, + streamQueryConfig, withChangeFlag = false)(outputType) // Give the DataStream to the TableSink to emit it. appendSink.asInstanceOf[AppendStreamTableSink[T]].emitDataStream(result) @@ -566,7 +570,7 @@ abstract class StreamTableEnvironment( * Table API calls and / or SQL queries and generating corresponding [[DataStream]] operators. * * @param table The root node of the relational expression tree. - * @param qConfig The configuration for the query to generate. + * @param queryConfig The configuration for the query to generate. * @param updatesAsRetraction Set to true to encode updates as retraction messages. * @param withChangeFlag Set to true to emit records with change flags. * @param tpe The [[TypeInformation]] of the resulting [[DataStream]]. @@ -575,12 +579,12 @@ abstract class StreamTableEnvironment( */ protected def translate[A]( table: Table, - qConfig: StreamQueryConfig, + queryConfig: StreamQueryConfig, updatesAsRetraction: Boolean, withChangeFlag: Boolean)(implicit tpe: TypeInformation[A]): DataStream[A] = { val relNode = table.getRelNode val dataStreamPlan = optimize(relNode, updatesAsRetraction) - translate(dataStreamPlan, relNode.getRowType, qConfig, withChangeFlag) + translate(dataStreamPlan, relNode.getRowType, queryConfig, withChangeFlag) } /** @@ -589,7 +593,7 @@ abstract class StreamTableEnvironment( * @param logicalPlan The root node of the relational expression tree. * @param logicalType The row type of the result. Since the logicalPlan can lose the * field naming during optimization we pass the row type separately. - * @param qConfig The configuration for the query to generate. + * @param queryConfig The configuration for the query to generate. * @param withChangeFlag Set to true to emit records with change flags. * @param tpe The [[TypeInformation]] of the resulting [[DataStream]]. * @tparam A The type of the resulting [[DataStream]]. @@ -598,7 +602,7 @@ abstract class StreamTableEnvironment( protected def translate[A]( logicalPlan: RelNode, logicalType: RelDataType, - qConfig: StreamQueryConfig, + queryConfig: StreamQueryConfig, withChangeFlag: Boolean) (implicit tpe: TypeInformation[A]): DataStream[A] = { @@ -610,7 +614,7 @@ abstract class StreamTableEnvironment( } // get CRow plan - val plan: DataStream[CRow] = translateToCRow(logicalPlan, qConfig) + val plan: DataStream[CRow] = translateToCRow(logicalPlan, queryConfig) // convert CRow to output type val conversion = if (withChangeFlag) { @@ -642,16 +646,16 @@ abstract class StreamTableEnvironment( * Translates a logical [[RelNode]] plan into a [[DataStream]] of type [[CRow]]. * * @param logicalPlan The logical plan to translate. - * @param qConfig The configuration for the query to generate. + * @param queryConfig The configuration for the query to generate. * @return The [[DataStream]] of type [[CRow]]. */ protected def translateToCRow( logicalPlan: RelNode, - qConfig: StreamQueryConfig): DataStream[CRow] = { + queryConfig: StreamQueryConfig): DataStream[CRow] = { logicalPlan match { case node: DataStreamRel => - node.translateToPlan(this, qConfig) + node.translateToPlan(this, queryConfig) case _ => throw TableException("Cannot generate DataStream due to an invalid logical plan. " + "This is a bug and should not happen. Please file an issue.") @@ -667,7 +671,7 @@ abstract class StreamTableEnvironment( def explain(table: Table): String = { val ast = table.getRelNode val optimizedPlan = optimize(ast, updatesAsRetraction = false) - val dataStream = translateToCRow(optimizedPlan, qConf) + val dataStream = translateToCRow(optimizedPlan, queryConfig) val env = dataStream.getExecutionEnvironment val jsonSqlPlan = env.getExecutionPlan diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/StreamTableEnvironment.scala index c3b59511d2954..a16884a319ca8 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/StreamTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/StreamTableEnvironment.scala @@ -150,7 +150,7 @@ class StreamTableEnvironment( * @return The converted [[DataStream]]. */ def toDataStream[T](table: Table, clazz: Class[T]): DataStream[T] = { - toDataStream(table, clazz, qConf) + toDataStream(table, clazz, queryConfig) } /** @@ -170,7 +170,7 @@ class StreamTableEnvironment( * @return The converted [[DataStream]]. */ def toDataStream[T](table: Table, typeInfo: TypeInformation[T]): DataStream[T] = { - toDataStream(table, typeInfo, qConf) + toDataStream(table, typeInfo, queryConfig) } /** @@ -186,14 +186,16 @@ class StreamTableEnvironment( * * @param table The [[Table]] to convert. * @param clazz The class of the type of the resulting [[DataStream]]. - * @param qConfig The configuration of the query to generate. + * @param queryConfig The configuration of the query to generate. * @tparam T The type of the resulting [[DataStream]]. * @return The converted [[DataStream]]. */ - def toDataStream[T](table: Table, clazz: Class[T], qConfig: StreamQueryConfig): DataStream[T] = { + def toDataStream[T]( + table: Table, clazz: Class[T], + queryConfig: StreamQueryConfig): DataStream[T] = { val typeInfo = TypeExtractor.createTypeInfo(clazz) TableEnvironment.validateType(typeInfo) - translate[T](table, qConfig, updatesAsRetraction = false, withChangeFlag = false)(typeInfo) + translate[T](table, queryConfig, updatesAsRetraction = false, withChangeFlag = false)(typeInfo) } /** @@ -209,16 +211,16 @@ class StreamTableEnvironment( * * @param table The [[Table]] to convert. * @param typeInfo The [[TypeInformation]] that specifies the type of the [[DataStream]]. - * @param qConfig The configuration of the query to generate. + * @param queryConfig The configuration of the query to generate. * @tparam T The type of the resulting [[DataStream]]. * @return The converted [[DataStream]]. */ def toDataStream[T]( table: Table, typeInfo: TypeInformation[T], - qConfig: StreamQueryConfig): DataStream[T] = { + queryConfig: StreamQueryConfig): DataStream[T] = { TableEnvironment.validateType(typeInfo) - translate[T](table, qConfig, updatesAsRetraction = false, withChangeFlag = false)(typeInfo) + translate[T](table, queryConfig, updatesAsRetraction = false, withChangeFlag = false)(typeInfo) } /** @@ -242,7 +244,7 @@ class StreamTableEnvironment( table: Table, clazz: Class[T]): DataStream[JTuple2[JBool, T]] = { - toRetractStream(table, clazz, qConf) + toRetractStream(table, clazz, queryConfig) } /** @@ -266,7 +268,7 @@ class StreamTableEnvironment( table: Table, typeInfo: TypeInformation[T]): DataStream[JTuple2[JBool, T]] = { - toRetractStream(table, typeInfo, qConf) + toRetractStream(table, typeInfo, queryConfig) } /** @@ -283,21 +285,21 @@ class StreamTableEnvironment( * * @param table The [[Table]] to convert. * @param clazz The class of the requested record type. - * @param qConfig The configuration of the query to generate. + * @param queryConfig The configuration of the query to generate. * @tparam T The type of the requested record type. * @return The converted [[DataStream]]. */ def toRetractStream[T]( table: Table, clazz: Class[T], - qConfig: StreamQueryConfig): DataStream[JTuple2[JBool, T]] = { + queryConfig: StreamQueryConfig): DataStream[JTuple2[JBool, T]] = { val typeInfo = TypeExtractor.createTypeInfo(clazz) TableEnvironment.validateType(typeInfo) val resultType = new TupleTypeInfo[JTuple2[JBool, T]](Types.BOOLEAN, typeInfo) translate[JTuple2[JBool, T]]( table, - qConfig, + queryConfig, updatesAsRetraction = true, withChangeFlag = true)(resultType) } @@ -316,14 +318,14 @@ class StreamTableEnvironment( * * @param table The [[Table]] to convert. * @param typeInfo The [[TypeInformation]] of the requested record type. - * @param qConfig The configuration of the query to generate. + * @param queryConfig The configuration of the query to generate. * @tparam T The type of the requested record type. * @return The converted [[DataStream]]. */ def toRetractStream[T]( table: Table, typeInfo: TypeInformation[T], - qConfig: StreamQueryConfig): DataStream[JTuple2[JBool, T]] = { + queryConfig: StreamQueryConfig): DataStream[JTuple2[JBool, T]] = { TableEnvironment.validateType(typeInfo) val resultTypeInfo = new TupleTypeInfo[JTuple2[JBool, T]]( @@ -332,7 +334,7 @@ class StreamTableEnvironment( ) translate[JTuple2[JBool, T]]( table, - qConfig, + queryConfig, updatesAsRetraction = true, withChangeFlag = true)(resultTypeInfo) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala index 56f7d55744cf9..8c6b27397e770 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala @@ -143,7 +143,7 @@ class StreamTableEnvironment( * @return The converted [[DataStream]]. */ def toDataStream[T: TypeInformation](table: Table): DataStream[T] = { - toDataStream(table, qConf) + toDataStream(table, queryConfig) } /** @@ -158,14 +158,16 @@ class StreamTableEnvironment( * - POJO [[DataStream]] types: Fields are mapped by field name, field types must match. * * @param table The [[Table]] to convert. - * @param qConfig The configuration of the query to generate. + * @param queryConfig The configuration of the query to generate. * @tparam T The type of the resulting [[DataStream]]. * @return The converted [[DataStream]]. */ - def toDataStream[T: TypeInformation](table: Table, qConfig: StreamQueryConfig): DataStream[T] = { + def toDataStream[T: TypeInformation]( + table: Table, + queryConfig: StreamQueryConfig): DataStream[T] = { val returnType = createTypeInformation[T] - asScalaStream( - translate(table, qConfig, updatesAsRetraction = false, withChangeFlag = false)(returnType)) + asScalaStream(translate( + table, queryConfig, updatesAsRetraction = false, withChangeFlag = false)(returnType)) } /** @@ -180,7 +182,7 @@ class StreamTableEnvironment( * @return The converted [[DataStream]]. */ def toRetractStream[T: TypeInformation](table: Table): DataStream[(Boolean, T)] = { - toRetractStream(table, qConf) + toRetractStream(table, queryConfig) } /** @@ -191,16 +193,16 @@ class StreamTableEnvironment( * A true [[Boolean]] flag indicates an add message, a false flag indicates a retract message. * * @param table The [[Table]] to convert. - * @param qConfig The configuration of the query to generate. + * @param queryConfig The configuration of the query to generate. * @tparam T The type of the requested data type. * @return The converted [[DataStream]]. */ def toRetractStream[T: TypeInformation]( table: Table, - qConfig: StreamQueryConfig): DataStream[(Boolean, T)] = { + queryConfig: StreamQueryConfig): DataStream[(Boolean, T)] = { val returnType = createTypeInformation[(Boolean, T)] asScalaStream( - translate(table, qConfig, updatesAsRetraction = true, withChangeFlag = true)(returnType)) + translate(table, queryConfig, updatesAsRetraction = true, withChangeFlag = true)(returnType)) } /** diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/TableConversions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/TableConversions.scala index 966b42f19756d..9874a9ea52e79 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/TableConversions.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/TableConversions.scala @@ -59,12 +59,12 @@ class TableConversions(table: Table) { /** Converts the [[Table]] to a [[DataStream]] of the specified type. * - * @param qConfig The configuration for the generated query. + * @param queryConfig The configuration for the generated query. */ - def toDataStream[T: TypeInformation](qConfig: StreamQueryConfig): DataStream[T] = { + def toDataStream[T: TypeInformation](queryConfig: StreamQueryConfig): DataStream[T] = { table.tableEnv match { case tEnv: ScalaStreamTableEnv => - tEnv.toDataStream(table, qConfig) + tEnv.toDataStream(table, queryConfig) case _ => throw new TableException( "Only tables that originate from Scala DataStreams " + @@ -97,14 +97,15 @@ class TableConversions(table: Table) { * * A true [[Boolean]] flag indicates an add message, a false flag indicates a retract message. * - * @param qConfig The configuration for the generated query. + * @param queryConfig The configuration for the generated query. * */ - def toRetractStream[T: TypeInformation](qConfig: StreamQueryConfig): DataStream[(Boolean, T)] = { + def toRetractStream[T: TypeInformation]( + queryConfig: StreamQueryConfig): DataStream[(Boolean, T)] = { table.tableEnv match { case tEnv: ScalaStreamTableEnv => - tEnv.toRetractStream(table, qConfig) + tEnv.toRetractStream(table, queryConfig) case _ => throw new TableException( "Only tables that originate from Scala DataStreams " + diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala index 5a2eb1c01a849..ca61c651f2b09 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala @@ -764,13 +764,13 @@ class Table( */ def writeToSink[T](sink: TableSink[T]): Unit = { - def qConfig = this.tableEnv match { - case s: StreamTableEnvironment => s.qConf + def queryConfig = this.tableEnv match { + case s: StreamTableEnvironment => s.queryConfig case b: BatchTableEnvironment => new BatchQueryConfig case _ => null } - writeToSink(sink, qConfig) + writeToSink(sink, queryConfig) } /** diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala index 0e377b510aea6..5f270f6986c32 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala @@ -85,11 +85,12 @@ class DataStreamCalc( override def translateToPlan( tableEnv: StreamTableEnvironment, - qConfig: StreamQueryConfig): DataStream[CRow] = { + queryConfig: StreamQueryConfig): DataStream[CRow] = { val config = tableEnv.getConfig - val inputDataStream = getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig) + val inputDataStream = + getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig) val inputRowType = inputDataStream.getType.asInstanceOf[CRowTypeInfo].rowType val generator = new CodeGenerator(config, false, inputRowType) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala index cbd818a5ecce1..5b32b1053a292 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala @@ -84,12 +84,12 @@ class DataStreamCorrelate( override def translateToPlan( tableEnv: StreamTableEnvironment, - qConfig: StreamQueryConfig): DataStream[CRow] = { + queryConfig: StreamQueryConfig): DataStream[CRow] = { val config = tableEnv.getConfig // we do not need to specify input type - val inputDS = getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig) + val inputDS = getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig) val inputType = inputDS.getType.asInstanceOf[CRowTypeInfo] val funcRel = scan.asInstanceOf[FlinkLogicalTableFunctionScan] diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala index 47cdb820ef452..54141d9146904 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala @@ -106,16 +106,17 @@ class DataStreamGroupAggregate( override def translateToPlan( tableEnv: StreamTableEnvironment, - qConfig: StreamQueryConfig): DataStream[CRow] = { + queryConfig: StreamQueryConfig): DataStream[CRow] = { - if (qConfig.getMinIdleStateRetentionTime < 0 || qConfig.getMaxIdleStateRetentionTime < 0) { + if (groupings.length > 0 && (queryConfig.getMinIdleStateRetentionTime < 0 + || queryConfig.getMaxIdleStateRetentionTime < 0)) { LOG.warn( "No state retention interval configured for a query which accumulates state. " + "Please provide a query configuration with valid retention interval to prevent excessive " + - "state size. You may specify a retention time of 0 to not clean up the state.") + "state size. You may specify a retention time of 0 to not clean up the state.") } - val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig) + val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig) val physicalNamedAggregates = namedAggregates.map { namedAggregate => new CalcitePair[AggregateCall, String]( @@ -149,7 +150,7 @@ class DataStreamGroupAggregate( inputSchema.logicalType, inputSchema.physicalFieldTypeInfo, groupings, - qConfig, + queryConfig, DataStreamRetractionRules.isAccRetract(this), DataStreamRetractionRules.isAccRetract(getInput)) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala index 51c4df0555485..36c4b94c54fe5 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala @@ -109,9 +109,9 @@ class DataStreamGroupWindowAggregate( override def translateToPlan( tableEnv: StreamTableEnvironment, - qConfig: StreamQueryConfig): DataStream[CRow] = { + queryConfig: StreamQueryConfig): DataStream[CRow] = { - val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig) + val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig) val physicalNamedAggregates = namedAggregates.map { namedAggregate => new CalcitePair[AggregateCall, String]( diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala index 9f1eb423fb33a..68799a4727424 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala @@ -36,6 +36,7 @@ import org.apache.flink.table.codegen.CodeGenerator import org.apache.flink.table.plan.rules.datastream.DataStreamRetractionRules import org.apache.flink.table.runtime.aggregate.AggregateUtil.CalcitePair import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} +import org.slf4j.LoggerFactory class DataStreamOverAggregate( logicWindow: Window, @@ -47,6 +48,7 @@ class DataStreamOverAggregate( extends SingleRel(cluster, traitSet, inputNode) with OverAggregate with DataStreamRel { + private val LOG = LoggerFactory.getLogger(this.getClass) override def deriveRowType(): RelDataType = schema.logicalType @@ -90,7 +92,7 @@ class DataStreamOverAggregate( override def translateToPlan( tableEnv: StreamTableEnvironment, - qConfig: StreamQueryConfig): DataStream[CRow] = { + queryConfig: StreamQueryConfig): DataStream[CRow] = { if (logicWindow.groups.size > 1) { throw new TableException( @@ -112,10 +114,26 @@ class DataStreamOverAggregate( "Unsupported use of OVER windows. The window can only be ordered in ASCENDING mode.") } - val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig) + val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig) val consumeRetraction = DataStreamRetractionRules.isAccRetract(input) + if (consumeRetraction) { + throw new TableException( + "Retraction on Over window aggregation is not supported yet. " + + "Note: Over window aggregation should not follow a non-windowed GroupBy aggregation.") + } + + if (overWindow.lowerBound.isUnbounded) { + if (queryConfig.getMinIdleStateRetentionTime < 0 + || queryConfig.getMaxIdleStateRetentionTime < 0) { + LOG.warn( + "No state retention interval configured for a query which accumulates state. " + + "Please provide a query configuration with valid retention interval to prevent " + + "excessive state size. You may specify a retention time of 0 to not clean up the state.") + } + } + val generator = new CodeGenerator( tableEnv.getConfig, false, @@ -126,19 +144,13 @@ class DataStreamOverAggregate( .get(orderKey.getFieldIndex) .getType - if (consumeRetraction) { - throw new TableException( - "Retraction on Over window aggregation is not supported yet. " + - "Note: Over window aggregation should not follow a non-windowed GroupBy aggregation.") - } - timeType match { case _ if FlinkTypeFactory.isProctimeIndicatorType(timeType) => // proc-time OVER window if (overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { // unbounded OVER window createUnboundedAndCurrentRowOverWindow( - qConfig, + queryConfig, generator, inputDS, isRowTimeType = false, @@ -146,9 +158,10 @@ class DataStreamOverAggregate( } else if ( overWindow.lowerBound.isPreceding && !overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { + // bounded OVER window createBoundedAndCurrentRowOverWindow( - qConfig, + queryConfig, generator, inputDS, isRowTimeType = false, @@ -164,7 +177,7 @@ class DataStreamOverAggregate( overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { // unbounded OVER window createUnboundedAndCurrentRowOverWindow( - qConfig, + queryConfig, generator, inputDS, isRowTimeType = true, @@ -172,7 +185,7 @@ class DataStreamOverAggregate( } else if (overWindow.lowerBound.isPreceding && overWindow.upperBound.isCurrentRow) { // bounded OVER window createBoundedAndCurrentRowOverWindow( - qConfig, + queryConfig, generator, inputDS, isRowTimeType = true, @@ -189,12 +202,14 @@ class DataStreamOverAggregate( } def createUnboundedAndCurrentRowOverWindow( - qConfig: StreamQueryConfig, + queryConfig: StreamQueryConfig, generator: CodeGenerator, inputDS: DataStream[CRow], isRowTimeType: Boolean, isRowsClause: Boolean): DataStream[CRow] = { + + val overWindow: Group = logicWindow.groups.get(0) val partitionKeys: Array[Int] = overWindow.keys.toArray.map(schema.mapIndex) @@ -215,7 +230,7 @@ class DataStreamOverAggregate( inputSchema.physicalType, inputSchema.physicalTypeInfo, inputSchema.physicalFieldTypeInfo, - qConfig, + queryConfig, isRowTimeType, partitionKeys.nonEmpty, isRowsClause) @@ -248,7 +263,7 @@ class DataStreamOverAggregate( } def createBoundedAndCurrentRowOverWindow( - qConfig: StreamQueryConfig, + queryConfig: StreamQueryConfig, generator: CodeGenerator, inputDS: DataStream[CRow], isRowTimeType: Boolean, @@ -276,7 +291,7 @@ class DataStreamOverAggregate( inputSchema.physicalTypeInfo, inputSchema.physicalFieldTypeInfo, precedingOffset, - qConfig, + queryConfig, isRowsClause, isRowTimeType ) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala index 6f6edf7b5de7f..65d336fcdf582 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala @@ -29,12 +29,12 @@ trait DataStreamRel extends FlinkRelNode { * Translates the FlinkRelNode into a Flink operator. * * @param tableEnv The [[StreamTableEnvironment]] of the translated Table. - * @param qConfig The configuration for the query to generate. + * @param queryConfig The configuration for the query to generate. * @return DataStream of type [[CRow]] */ def translateToPlan( tableEnv: StreamTableEnvironment, - qConfig: StreamQueryConfig): DataStream[CRow] + queryConfig: StreamQueryConfig): DataStream[CRow] /** * Whether the [[DataStreamRel]] requires that update and delete changes are sent with retraction diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamScan.scala index e64bf0fe1c389..424c6a26633e5 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamScan.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamScan.scala @@ -56,7 +56,7 @@ class DataStreamScan( override def translateToPlan( tableEnv: StreamTableEnvironment, - qConfig: StreamQueryConfig): DataStream[CRow] = { + queryConfig: StreamQueryConfig): DataStream[CRow] = { val config = tableEnv.getConfig val inputDataStream: DataStream[Any] = dataStreamTable.dataStream diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala index 6cc739603f43f..6f4980aa70dab 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala @@ -60,10 +60,10 @@ class DataStreamUnion( override def translateToPlan( tableEnv: StreamTableEnvironment, - qConfig: StreamQueryConfig): DataStream[CRow] = { + queryConfig: StreamQueryConfig): DataStream[CRow] = { - val leftDataSet = left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig) - val rightDataSet = right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, qConfig) + val leftDataSet = left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig) + val rightDataSet = right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig) leftDataSet.union(rightDataSet) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamValues.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamValues.scala index ba6b0257a8812..d7c490f40055b 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamValues.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamValues.scala @@ -58,7 +58,7 @@ class DataStreamValues( override def translateToPlan( tableEnv: StreamTableEnvironment, - qConfig: StreamQueryConfig): DataStream[CRow] = { + queryConfig: StreamQueryConfig): DataStream[CRow] = { val config = tableEnv.getConfig diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala index 225f23f172c91..51e609f4fe3be 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala @@ -100,7 +100,7 @@ class StreamTableSourceScan( override def translateToPlan( tableEnv: StreamTableEnvironment, - qConfig: StreamQueryConfig): DataStream[CRow] = { + queryConfig: StreamQueryConfig): DataStream[CRow] = { val config = tableEnv.getConfig val inputDataStream = tableSource.getDataStream(tableEnv.execEnv).asInstanceOf[DataStream[Any]] diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala index 5e5e3f980d41a..03f78225f9f96 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala @@ -77,7 +77,7 @@ object AggregateUtil { inputType: RelDataType, inputTypeInfo: TypeInformation[Row], inputFieldTypeInfo: Seq[TypeInformation[_]], - qConfig: StreamQueryConfig, + queryConfig: StreamQueryConfig, isRowTimeType: Boolean, isPartitioned: Boolean, isRowsClause: Boolean) @@ -118,27 +118,25 @@ object AggregateUtil { new RowTimeUnboundedRowsOver( genFunction, aggregationStateType, - CRowTypeInfo(inputTypeInfo), - qConfig) + CRowTypeInfo(inputTypeInfo)) } else { // RANGE unbounded over process function new RowTimeUnboundedRangeOver( genFunction, aggregationStateType, - CRowTypeInfo(inputTypeInfo), - qConfig) + CRowTypeInfo(inputTypeInfo)) } } else { if (isPartitioned) { new ProcTimeUnboundedPartitionedOver( genFunction, aggregationStateType, - qConfig) + queryConfig) } else { new ProcTimeUnboundedNonPartitionedOver( genFunction, aggregationStateType, - qConfig) + queryConfig) } } } @@ -160,7 +158,7 @@ object AggregateUtil { inputRowType: RelDataType, inputFieldTypes: Seq[TypeInformation[_]], groupings: Array[Int], - qConfig: StreamQueryConfig, + queryConfig: StreamQueryConfig, generateRetraction: Boolean, consumeRetraction: Boolean): ProcessFunction[CRow, CRow] = { @@ -197,7 +195,7 @@ object AggregateUtil { genFunction, aggregationStateType, generateRetraction, - qConfig) + queryConfig) } @@ -222,7 +220,7 @@ object AggregateUtil { inputTypeInfo: TypeInformation[Row], inputFieldTypeInfo: Seq[TypeInformation[_]], precedingOffset: Long, - qConfig: StreamQueryConfig, + queryConfig: StreamQueryConfig, isRowsClause: Boolean, isRowTimeType: Boolean) : ProcessFunction[CRow, CRow] = { @@ -264,15 +262,13 @@ object AggregateUtil { genFunction, aggregationStateType, inputRowType, - precedingOffset, - qConfig) + precedingOffset) } else { new RowTimeBoundedRangeOver( genFunction, aggregationStateType, inputRowType, - precedingOffset, - qConfig) + precedingOffset) } } else { if (isRowsClause) { @@ -281,14 +277,14 @@ object AggregateUtil { precedingOffset, aggregationStateType, inputRowType, - qConfig) + queryConfig) } else { new ProcTimeBoundedRangeOver( genFunction, precedingOffset, aggregationStateType, inputRowType, - qConfig) + queryConfig) } } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala index dbaddd60e0e0c..e710e917df781 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala @@ -41,8 +41,8 @@ class GroupAggProcessFunction( private val genAggregations: GeneratedAggregationsFunction, private val aggregationStateType: RowTypeInfo, private val generateRetraction: Boolean, - private val qConfig: StreamQueryConfig) - extends ProcessFunctionWithCleanupState[CRow, CRow](qConfig) + private val queryConfig: StreamQueryConfig) + extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig) with Compiler[GeneratedAggregations] { val LOG: Logger = LoggerFactory.getLogger(this.getClass) @@ -55,7 +55,6 @@ class GroupAggProcessFunction( private var state: ValueState[Row] = _ // counts the number of added and retracted input records private var cntState: ValueState[JLong] = _ - // holds the latest registered cleanup timer override def open(config: Configuration) { LOG.debug(s"Compiling AggregateHelper: $genAggregations.name \n\n " + diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala index 931772b81b199..ae551b62b4f45 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala @@ -50,8 +50,8 @@ class ProcTimeBoundedRangeOver( precedingTimeBoundary: Long, aggregatesTypeInfo: RowTypeInfo, inputType: TypeInformation[CRow], - qConfig: StreamQueryConfig) - extends ProcessFunctionWithCleanupState[CRow, CRow](qConfig) + queryConfig: StreamQueryConfig) + extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig) with Compiler[GeneratedAggregations] { private var output: CRow = _ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala index 7fd435bd7a3a6..0a7971d3cb205 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala @@ -51,8 +51,8 @@ class ProcTimeBoundedRowsOver( precedingOffset: Long, aggregatesTypeInfo: RowTypeInfo, inputType: TypeInformation[CRow], - qConfig: StreamQueryConfig) - extends ProcessFunctionWithCleanupState[CRow, CRow](qConfig) + queryConfig: StreamQueryConfig) + extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig) with Compiler[GeneratedAggregations] { Preconditions.checkArgument(precedingOffset > 0) @@ -191,6 +191,11 @@ class ProcTimeBoundedRowsOver( timestamp: Long, ctx: ProcessFunction[CRow, CRow]#OnTimerContext, out: Collector[CRow]): Unit = { - cleanupStateOnTimer(timestamp, rowMapState, accumulatorState, counterState, smallestTsState) + cleanupStateOnTimer( + timestamp, + rowMapState, + accumulatorState, + counterState, + smallestTsState) } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedNonPartitionedOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedNonPartitionedOver.scala index 41fbe6bfa50d0..63402734c7f65 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedNonPartitionedOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedNonPartitionedOver.scala @@ -39,8 +39,8 @@ import org.slf4j.LoggerFactory class ProcTimeUnboundedNonPartitionedOver( genAggregations: GeneratedAggregationsFunction, aggregationStateType: RowTypeInfo, - qConfig: StreamQueryConfig) - extends ProcessFunctionWithCleanupState[CRow, CRow](qConfig) + queryConfig: StreamQueryConfig) + extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig) with CheckpointedFunction with Compiler[GeneratedAggregations] { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedPartitionedOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedPartitionedOver.scala index 6f1f3822a55d1..0f098385d127c 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedPartitionedOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedPartitionedOver.scala @@ -38,8 +38,8 @@ import org.slf4j.LoggerFactory class ProcTimeUnboundedPartitionedOver( genAggregations: GeneratedAggregationsFunction, aggregationStateType: RowTypeInfo, - qConfig: StreamQueryConfig) - extends ProcessFunctionWithCleanupState[CRow, CRow](qConfig) + queryConfig: StreamQueryConfig) + extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig) with Compiler[GeneratedAggregations] { private var output: CRow = _ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcessFunctionWithCleanupState.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcessFunctionWithCleanupState.scala index ad249727ff213..07d4442b53ba1 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcessFunctionWithCleanupState.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcessFunctionWithCleanupState.scala @@ -24,14 +24,12 @@ import org.apache.flink.api.common.state.State import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.table.api.{StreamQueryConfig, Types} -abstract class ProcessFunctionWithCleanupState[IN,OUT](qConfig: StreamQueryConfig) +abstract class ProcessFunctionWithCleanupState[IN,OUT](queryConfig: StreamQueryConfig) extends ProcessFunction[IN, OUT]{ - protected val minRetentionTime = qConfig.getMinIdleStateRetentionTime - protected val maxRetentionTime = qConfig.getMaxIdleStateRetentionTime + protected val minRetentionTime = queryConfig.getMinIdleStateRetentionTime + protected val maxRetentionTime = queryConfig.getMaxIdleStateRetentionTime protected val stateCleaningEnabled = minRetentionTime > 1 && maxRetentionTime > 1 - // interval in which clean-up timers are registered - protected val cleanupTimerInterval = maxRetentionTime - minRetentionTime // holds the latest registered cleanup timer private var cleanupTimeState: ValueState[JLong] = _ @@ -54,36 +52,19 @@ abstract class ProcessFunctionWithCleanupState[IN,OUT](qConfig: StreamQueryConfi // last registered timer val lastCleanupTime = cleanupTimeState.value() - if (lastCleanupTime == null || earliestCleanup >= lastCleanupTime + cleanupTimerInterval) { + if (lastCleanupTime == null || earliestCleanup > lastCleanupTime) { // we need to register a new timer - val cleanupTime = earliestCleanup + cleanupTimerInterval + val cleanupTime = currentTime + maxRetentionTime // register timer and remember clean-up time ctx.timerService().registerProcessingTimeTimer(cleanupTime) cleanupTimeState.update(cleanupTime) } } } - protected def registerEventCleanupTimer( - ctx: ProcessFunction[IN, OUT]#Context, - currentTime: Long): Unit = { - if (stateCleaningEnabled) { - - val earliestCleanup = currentTime + minRetentionTime - - // last registered timer - val lastCleanupTime = cleanupTimeState.value() - - if (lastCleanupTime == null || earliestCleanup >= lastCleanupTime + cleanupTimerInterval) { - // we need to register a new timer - val cleanupTime = earliestCleanup + cleanupTimerInterval - // register timer and remember clean-up time - ctx.timerService().registerEventTimeTimer(cleanupTime) - cleanupTimeState.update(cleanupTime) - } - } - } - protected def cleanupStateOnTimer(timestamp: Long, states: State*): Boolean = { + protected def cleanupStateOnTimer( + timestamp: Long, + states: State*): Boolean = { var result: Boolean = false if (stateCleaningEnabled) { val cleanupTime = cleanupTimeState.value() diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala index 7768325a29ee7..4020d440a78fa 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala @@ -24,7 +24,6 @@ import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} import org.apache.flink.api.java.typeutils.{ListTypeInfo, RowTypeInfo} import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.ProcessFunction -import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} import org.apache.flink.types.Row @@ -43,9 +42,8 @@ class RowTimeBoundedRangeOver( genAggregations: GeneratedAggregationsFunction, aggregationStateType: RowTypeInfo, inputRowType: CRowTypeInfo, - precedingOffset: Long, - qConfig: StreamQueryConfig) - extends ProcessFunctionWithCleanupState[CRow, CRow](qConfig) + precedingOffset: Long) + extends ProcessFunction[CRow, CRow] with Compiler[GeneratedAggregations] { Preconditions.checkNotNull(aggregationStateType) Preconditions.checkNotNull(precedingOffset) @@ -99,8 +97,6 @@ class RowTimeBoundedRangeOver( valueTypeInformation) dataState = getRuntimeContext.getMapState(mapStateDescriptor) - - initCleanupTimeState("RowTimeBoundedRangeOverCleanupTime") } override def processElement( @@ -113,9 +109,6 @@ class RowTimeBoundedRangeOver( // triggering timestamp for trigger calculation val triggeringTs = ctx.timestamp - // register state-cleanup timer - registerEventCleanupTimer(ctx, triggeringTs) - val lastTriggeringTs = lastTriggeringTsState.value // check if the data is expired, if not, save the data and register event time timer @@ -140,15 +133,6 @@ class RowTimeBoundedRangeOver( out: Collector[CRow]): Unit = { // gets all window data from state for the calculation val inputs: JList[Row] = dataState.get(timestamp) - val isCleanup = cleanupStateOnTimer( - timestamp, - lastTriggeringTsState, - accumulatorState, - dataState) - - if (isCleanup) { - return - } if (null != inputs) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala index dd4a7e64a3e92..5ec6ec7ab1ead 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala @@ -26,7 +26,6 @@ import org.apache.flink.api.java.typeutils.{ListTypeInfo, RowTypeInfo} import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.types.Row -import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.util.{Collector, Preconditions} import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} @@ -44,9 +43,8 @@ class RowTimeBoundedRowsOver( genAggregations: GeneratedAggregationsFunction, aggregationStateType: RowTypeInfo, inputRowType: CRowTypeInfo, - precedingOffset: Long, - qConfig: StreamQueryConfig) -extends ProcessFunctionWithCleanupState[CRow, CRow](qConfig) + precedingOffset: Long) + extends ProcessFunction[CRow, CRow] with Compiler[GeneratedAggregations] { Preconditions.checkNotNull(aggregationStateType) @@ -108,8 +106,6 @@ extends ProcessFunctionWithCleanupState[CRow, CRow](qConfig) valueTypeInformation) dataState = getRuntimeContext.getMapState(mapStateDescriptor) - - initCleanupTimeState("RowTimeBoundedRowsOverCleanupTime") } override def processElement( @@ -122,9 +118,6 @@ extends ProcessFunctionWithCleanupState[CRow, CRow](qConfig) // triggering timestamp for trigger calculation val triggeringTs = ctx.timestamp - // register state-cleanup timer - registerEventCleanupTimer(ctx, triggeringTs) - val lastTriggeringTs = lastTriggeringTsState.value // check if the data is expired, if not, save the data and register event time timer @@ -148,16 +141,6 @@ extends ProcessFunctionWithCleanupState[CRow, CRow](qConfig) ctx: ProcessFunction[CRow, CRow]#OnTimerContext, out: Collector[CRow]): Unit = { - val isCleanup = cleanupStateOnTimer( - timestamp, - lastTriggeringTsState, - dataCountState, - accumulatorState, - dataState) - - if (isCleanup) { - return - } // gets all window data from state for the calculation val inputs: JList[Row] = dataState.get(timestamp) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala index 6e482810473ae..3e2a8117b56e5 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala @@ -28,7 +28,6 @@ import org.apache.flink.util.{Collector, Preconditions} import org.apache.flink.api.common.state._ import org.apache.flink.api.java.typeutils.ListTypeInfo import org.apache.flink.streaming.api.operators.TimestampedCollector -import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} import org.slf4j.LoggerFactory @@ -44,9 +43,8 @@ import org.slf4j.LoggerFactory abstract class RowTimeUnboundedOver( genAggregations: GeneratedAggregationsFunction, intermediateType: TypeInformation[Row], - inputType: TypeInformation[CRow], - qConfig: StreamQueryConfig) - extends ProcessFunctionWithCleanupState[CRow, CRow](qConfig) + inputType: TypeInformation[CRow]) + extends ProcessFunction[CRow, CRow] with Compiler[GeneratedAggregations] { protected var output: CRow = _ @@ -85,8 +83,6 @@ abstract class RowTimeUnboundedOver( new MapStateDescriptor[Long, JList[Row]]("rowmapstate", BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo) rowMapState = getRuntimeContext.getMapState(mapStateDescriptor) - - initCleanupTimeState("RowTimeUnboundedOverCleanupTime") } /** @@ -106,10 +102,6 @@ abstract class RowTimeUnboundedOver( val input = inputC.row val timestamp = ctx.timestamp() - - // register state-cleanup timer - registerEventCleanupTimer(ctx, timestamp) - val curWatermark = ctx.timerService().currentWatermark() // discard late record @@ -144,12 +136,6 @@ abstract class RowTimeUnboundedOver( Preconditions.checkArgument(out.isInstanceOf[TimestampedCollector[CRow]]) val collector = out.asInstanceOf[TimestampedCollector[CRow]] - val isCleanup = cleanupStateOnTimer(timestamp, rowMapState, accumulatorState) - - if (isCleanup) { - return - } - val keyIterator = rowMapState.keys.iterator if (keyIterator.hasNext) { val curWatermark = ctx.timerService.currentWatermark @@ -235,13 +221,11 @@ abstract class RowTimeUnboundedOver( class RowTimeUnboundedRowsOver( genAggregations: GeneratedAggregationsFunction, intermediateType: TypeInformation[Row], - inputType: TypeInformation[CRow], - qConfig: StreamQueryConfig) + inputType: TypeInformation[CRow]) extends RowTimeUnboundedOver( genAggregations: GeneratedAggregationsFunction, intermediateType, - inputType, - qConfig) { + inputType) { override def processElementsWithSameTimestamp( curRowList: JList[Row], @@ -275,13 +259,11 @@ class RowTimeUnboundedRowsOver( class RowTimeUnboundedRangeOver( genAggregations: GeneratedAggregationsFunction, intermediateType: TypeInformation[Row], - inputType: TypeInformation[CRow], - qConfig: StreamQueryConfig) + inputType: TypeInformation[CRow]) extends RowTimeUnboundedOver( genAggregations: GeneratedAggregationsFunction, intermediateType, - inputType, - qConfig) { + inputType) { override def processElementsWithSameTimestamp( curRowList: JList[Row], diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupAggregationsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupAggregationsITCase.scala index 375701318b54f..9da2c445a9d06 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupAggregationsITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupAggregationsITCase.scala @@ -35,8 +35,8 @@ import scala.collection.mutable * Tests of groupby (without window) aggregations */ class GroupAggregationsITCase extends StreamingWithStateTestBase { - private val qConfig = new StreamQueryConfig() - qConfig.withIdleStateRetentionTime(Time.hours(1), Time.hours(2)) + private val queryConfig = new StreamQueryConfig() + queryConfig.withIdleStateRetentionTime(Time.hours(1), Time.hours(2)) @Test def testNonKeyedGroupAggregate(): Unit = { @@ -48,7 +48,7 @@ class GroupAggregationsITCase extends StreamingWithStateTestBase { val t = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c) .select('a.sum, 'b.sum) - val results = t.toRetractStream[Row](qConfig) + val results = t.toRetractStream[Row](queryConfig) results.addSink(new StreamITCase.RetractingSink).setParallelism(1) env.execute() @@ -67,7 +67,7 @@ class GroupAggregationsITCase extends StreamingWithStateTestBase { .groupBy('b) .select('b, 'a.sum) - val results = t.toRetractStream[Row](qConfig) + val results = t.toRetractStream[Row](queryConfig) results.addSink(new StreamITCase.RetractingSink) env.execute() @@ -88,7 +88,7 @@ class GroupAggregationsITCase extends StreamingWithStateTestBase { .groupBy('cnt) .select('cnt, 'b.count as 'freq) - val results = t.toRetractStream[Row](qConfig) + val results = t.toRetractStream[Row](queryConfig) results.addSink(new RetractingSink) env.execute() @@ -107,7 +107,7 @@ class GroupAggregationsITCase extends StreamingWithStateTestBase { .groupBy('e, 'b % 3) .select('c.min, 'e, 'a.avg, 'd.count) - val results = t.toRetractStream[Row](qConfig) + val results = t.toRetractStream[Row](queryConfig) results.addSink(new RetractingSink) env.execute() diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala index 3f61cab304c0f..77798f9ddcebb 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala @@ -19,7 +19,6 @@ package org.apache.flink.table.runtime.harness import java.util.{Comparator, Queue => JQueue} -import org.apache.flink.api.common.time.Time import org.apache.flink.api.common.typeinfo.BasicTypeInfo.{INT_TYPE_INFO, LONG_TYPE_INFO, STRING_TYPE_INFO} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.functions.KeySelector @@ -28,7 +27,6 @@ import org.apache.flink.streaming.api.operators.OneInputStreamOperator import org.apache.flink.streaming.api.watermark.Watermark import org.apache.flink.streaming.runtime.streamrecord.StreamRecord import org.apache.flink.streaming.util.{KeyedOneInputStreamOperatorTestHarness, TestHarnessUtil} -import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.table.codegen.GeneratedAggregationsFunction import org.apache.flink.table.functions.AggregateFunction import org.apache.flink.table.functions.aggfunctions.{LongMaxWithRetractAggFunction, LongMinWithRetractAggFunction, IntSumWithRetractAggFunction} @@ -37,9 +35,6 @@ import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} class HarnessTestBase { - protected var qConfig = - new StreamQueryConfig().withIdleStateRetentionTime(Time.seconds(2), Time.seconds(3)) - protected val MinMaxRowType = new RowTypeInfo(Array[TypeInformation[_]]( INT_TYPE_INFO, LONG_TYPE_INFO, diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala index 13cf27c10a6a2..04214f9a178ef 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala @@ -20,9 +20,11 @@ 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.KeyedProcessOperator import org.apache.flink.streaming.runtime.streamrecord.StreamRecord +import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.table.runtime.aggregate._ import org.apache.flink.table.runtime.harness.HarnessTestBase._ import org.apache.flink.table.runtime.types.CRow @@ -31,6 +33,9 @@ import org.junit.Test class NonWindowHarnessTest extends HarnessTestBase { + protected var queryConfig = + new StreamQueryConfig().withIdleStateRetentionTime(Time.seconds(2), Time.seconds(3)) + @Test def testProcTimeNonWindow(): Unit = { @@ -39,7 +44,7 @@ class NonWindowHarnessTest extends HarnessTestBase { genSumAggFunction, sumAggregationStateType, false, - qConfig)) + queryConfig)) val testHarness = createHarnessTester( @@ -54,13 +59,18 @@ class NonWindowHarnessTest extends HarnessTestBase { testHarness.processElement(new StreamRecord(CRow(Row.of(1L: JLong, 1: JInt, "aaa"), true), 1)) testHarness.processElement(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt, "bbb"), true), 1)) + // reuse timer 3001 + testHarness.setProcessingTime(1000) testHarness.processElement(new StreamRecord(CRow(Row.of(3L: JLong, 2: JInt, "aaa"), true), 1)) testHarness.processElement(new StreamRecord(CRow(Row.of(4L: JLong, 3: JInt, "aaa"), true), 1)) - // trigger cleanup timer and register cleanup timer with 6002 - testHarness.setProcessingTime(3002) + // register cleanup timer with 4002 + testHarness.setProcessingTime(1002) testHarness.processElement(new StreamRecord(CRow(Row.of(5L: JLong, 4: JInt, "aaa"), true), 1)) testHarness.processElement(new StreamRecord(CRow(Row.of(6L: JLong, 2: JInt, "bbb"), true), 1)) + + // trigger cleanup timer and register cleanup timer with 7003 + testHarness.setProcessingTime(4003) testHarness.processElement(new StreamRecord(CRow(Row.of(7L: JLong, 5: JInt, "aaa"), true), 1)) testHarness.processElement(new StreamRecord(CRow(Row.of(8L: JLong, 6: JInt, "aaa"), true), 1)) testHarness.processElement(new StreamRecord(CRow(Row.of(9L: JLong, 7: JInt, "aaa"), true), 1)) @@ -74,12 +84,12 @@ class NonWindowHarnessTest extends HarnessTestBase { expectedOutput.add(new StreamRecord(CRow(Row.of(2L: JLong, 1: JInt), true), 1)) expectedOutput.add(new StreamRecord(CRow(Row.of(3L: JLong, 3: JInt), true), 1)) expectedOutput.add(new StreamRecord(CRow(Row.of(4L: JLong, 6: JInt), true), 1)) - expectedOutput.add(new StreamRecord(CRow(Row.of(5L: JLong, 4: JInt), true), 1)) - expectedOutput.add(new StreamRecord(CRow(Row.of(6L: JLong, 2: JInt), true), 1)) - expectedOutput.add(new StreamRecord(CRow(Row.of(7L: JLong, 9: JInt), true), 1)) - expectedOutput.add(new StreamRecord(CRow(Row.of(8L: JLong, 15: JInt), true), 1)) - expectedOutput.add(new StreamRecord(CRow(Row.of(9L: JLong, 22: JInt), true), 1)) - expectedOutput.add(new StreamRecord(CRow(Row.of(10L: JLong, 5: JInt), true), 1)) + expectedOutput.add(new StreamRecord(CRow(Row.of(5L: JLong, 10: JInt), true), 1)) + expectedOutput.add(new StreamRecord(CRow(Row.of(6L: JLong, 3: JInt), true), 1)) + expectedOutput.add(new StreamRecord(CRow(Row.of(7L: JLong, 5: JInt), true), 1)) + expectedOutput.add(new StreamRecord(CRow(Row.of(8L: JLong, 11: JInt), true), 1)) + expectedOutput.add(new StreamRecord(CRow(Row.of(9L: JLong, 18: JInt), true), 1)) + expectedOutput.add(new StreamRecord(CRow(Row.of(10L: JLong, 3: JInt), true), 1)) verify(expectedOutput, result, new RowResultSortComparator(6)) @@ -94,7 +104,7 @@ class NonWindowHarnessTest extends HarnessTestBase { genSumAggFunction, sumAggregationStateType, true, - qConfig)) + queryConfig)) val testHarness = createHarnessTester( diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala index 0b6dceefa3db4..931ea125ac20e 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala @@ -33,6 +33,9 @@ import org.junit.Test class OverWindowHarnessTest extends HarnessTestBase{ + protected var queryConfig = + new StreamQueryConfig().withIdleStateRetentionTime(Time.seconds(2), Time.seconds(3)) + @Test def testProcTimeBoundedRowsOver(): Unit = { @@ -42,7 +45,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ 2, minMaxAggregationStateType, minMaxCRowType, - qConfig)) + queryConfig)) val testHarness = createHarnessTester(processFunction,new TupleRowKeySelector[Integer](0),BasicTypeInfo @@ -62,6 +65,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong), true), 1)) + // register cleanup timer with 4100 testHarness.setProcessingTime(1100) testHarness.processElement(new StreamRecord( CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong), true), 1)) @@ -74,7 +78,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), true), 1)) - // trigger cleanup timer and register cleanup timer with 6001 + // register cleanup timer with 6001 testHarness.setProcessingTime(3001) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 2)) @@ -83,8 +87,8 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 2)) - // using historical data and register cleanup timer with 9000 - testHarness.setProcessingTime(6000) + // trigger cleanup timer and register cleanup timer with 9002 + testHarness.setProcessingTime(6002) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 2)) testHarness.processElement(new StreamRecord( @@ -123,7 +127,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 20L: JLong, 30L: JLong), true), 1)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 7L: JLong, 7L: JLong), true), 2)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 6L: JLong, 7L: JLong), true), 2)) expectedOutput.add(new StreamRecord( CRow( Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 7L: JLong, 8L: JLong), true), 2)) @@ -132,7 +136,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 8L: JLong, 9L: JLong), true), 2)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 9L: JLong, 10L: JLong), true), 2)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 10L: JLong, 10L: JLong), true), 2)) expectedOutput.add(new StreamRecord( CRow( Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 2)) @@ -154,7 +158,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ 4000, minMaxAggregationStateType, minMaxCRowType, - qConfig)) + queryConfig)) val testHarness = createHarnessTester( @@ -186,7 +190,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong), true), 0)) - // using historical data and register cleanup timer with 9002 + // register cleanup timer with 9002 testHarness.setProcessingTime(6002) testHarness.setProcessingTime(7002) @@ -197,7 +201,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), true), 0)) - // using historical data and register cleanup timer with 14002 + // register cleanup timer with 14002 testHarness.setProcessingTime(11002) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 0)) @@ -275,7 +279,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ new ProcTimeUnboundedPartitionedOver( genMinMaxAggFunction, minMaxAggregationStateType, - qConfig)) + queryConfig)) val testHarness = createHarnessTester( @@ -376,16 +380,13 @@ class OverWindowHarnessTest extends HarnessTestBase{ */ @Test def testRowTimeBoundedRangeOver(): Unit = { - qConfig = new StreamQueryConfig() - .withIdleStateRetentionTime(Time.seconds(3), Time.seconds(5)) val processFunction = new KeyedProcessOperator[String, CRow, CRow]( new RowTimeBoundedRangeOver( genMinMaxAggFunction, minMaxAggregationStateType, minMaxCRowType, - 4000, - qConfig)) + 4000)) val testHarness = createHarnessTester( @@ -395,7 +396,6 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.open() - // register cleanup timer with 4003 testHarness.processWatermark(1) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 2)) @@ -420,36 +420,31 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(2: JInt, 11L: JLong, 1: JInt, "bbb", 25L: JLong), true), 4801)) - testHarness.processWatermark(9802) + testHarness.processWatermark(6500) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong), true), 9803)) + CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong), true), 6501)) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong), true), 9804)) + CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong), true), 6501)) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), true), 9805)) + CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), true), 6501)) - // using historical data and register cleanup timer with 19802 - testHarness.processWatermark(14801) + testHarness.processWatermark(7000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 14802)) + CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 7001)) - testHarness.processWatermark(14802) + testHarness.processWatermark(8000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 14803)) + CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 8001)) - testHarness.processWatermark(15000) + testHarness.processWatermark(12000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 15001)) - - testHarness.processWatermark(19900) + CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 12001)) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 19901)) - - testHarness.processWatermark(22000) + CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 12001)) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 22001)) + CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 12001)) - testHarness.processWatermark(23000) + testHarness.processWatermark(19000) val result = testHarness.getOutput @@ -476,28 +471,28 @@ class OverWindowHarnessTest extends HarnessTestBase{ Row.of(2: JInt, 11L: JLong, 1: JInt, "bbb", 25L: JLong, 25L: JLong, 25L: JLong), true), 4801)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 5L: JLong, 5L: JLong), true), 9803)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 2L: JLong, 6L: JLong), true), 6501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 5L: JLong, 6L: JLong), true), 9804)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 2L: JLong, 6L: JLong), true), 6501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 7L: JLong, 7L: JLong), true), 14802)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 2L: JLong, 7L: JLong), true), 7001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 7L: JLong, 8L: JLong), true), 14803)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 2L: JLong, 8L: JLong), true), 8001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 30L: JLong, 30L: JLong), true), 9805)) + Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 25L: JLong, 30L: JLong), true), 6501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 7L: JLong, 9L: JLong), true), 15001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 8L: JLong, 10L: JLong), true), 12001)) expectedOutput.add(new StreamRecord( - CRow(Row.of( - 1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 10L: JLong, 10L: JLong), true), 19901)) + CRow( + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 8L: JLong, 10L: JLong), true), 12001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 22001)) + Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 12001)) verify(expectedOutput, result, new RowResultSortComparator(6)) testHarness.close() @@ -511,8 +506,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ genMinMaxAggFunction, minMaxAggregationStateType, minMaxCRowType, - 3, - qConfig)) + 3)) val testHarness = createHarnessTester( @@ -522,7 +516,6 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.open() - // register cleanup timer with 3801 testHarness.processWatermark(800) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 801)) @@ -531,7 +524,6 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong), true), 2501)) - // trigger cleanup timer and register cleanup timer with 7001 testHarness.processWatermark(4000) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong), true), 4001)) @@ -563,13 +555,10 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processWatermark(12000) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 12001)) - - // trigger cleanup timer and register cleanup timer with 18002 - testHarness.processWatermark(15001) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 15002)) + CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 12001)) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 15002)) + CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 12001)) testHarness.processWatermark(19000) @@ -585,10 +574,10 @@ class OverWindowHarnessTest extends HarnessTestBase{ Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true), 2501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 2L: JLong, 2L: JLong), true), 4001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true), 4001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 2L: JLong, 3L: JLong), true), 4001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true), 4001)) expectedOutput.add(new StreamRecord( CRow( Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true), 4001)) @@ -603,7 +592,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 4L: JLong, 6L: JLong), true), 6501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 30L: JLong, 30L: JLong), true), 6501)) + Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true), 6501)) expectedOutput.add(new StreamRecord( CRow( Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 5L: JLong, 7L: JLong), true), 7001)) @@ -612,13 +601,13 @@ class OverWindowHarnessTest extends HarnessTestBase{ Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 6L: JLong, 8L: JLong), true), 8001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 9L: JLong, 9L: JLong), true), 12001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 7L: JLong, 9L: JLong), true), 12001)) expectedOutput.add(new StreamRecord( - CRow(Row.of( - 1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 10L: JLong, 10L: JLong), true), 15002)) + CRow( + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 8L: JLong, 10L: JLong), true), 12001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 15002)) + Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 20L: JLong, 40L: JLong), true), 12001)) verify(expectedOutput, result, new RowResultSortComparator(6)) testHarness.close() @@ -634,8 +623,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ new RowTimeUnboundedRangeOver( genMinMaxAggFunction, minMaxAggregationStateType, - minMaxCRowType, - qConfig)) + minMaxCRowType)) val testHarness = createHarnessTester( @@ -645,7 +633,6 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.open() - // register cleanup timer with 3801 testHarness.processWatermark(800) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 801)) @@ -654,7 +641,6 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong), true), 2501)) - // trigger cleanup timer and register cleanup timer with 7001 testHarness.processWatermark(4000) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong), true), 4001)) @@ -667,7 +653,6 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong), true), 4801)) - // using historical data and register cleanup timer with 9501 testHarness.processWatermark(6500) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong), true), 6501)) @@ -680,12 +665,10 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 7001)) - // trigger cleanup timer and register cleanup timer with 13001 - testHarness.processWatermark(10000) + testHarness.processWatermark(8000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 10001)) + CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 8001)) - // using historical data and register cleanup timer with 15001 testHarness.processWatermark(12000) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 12001)) @@ -709,40 +692,40 @@ class OverWindowHarnessTest extends HarnessTestBase{ Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true), 2501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 2L: JLong, 3L: JLong), true), 4001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 1L: JLong, 3L: JLong), true), 4001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 2L: JLong, 3L: JLong), true), 4001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true), 4001)) expectedOutput.add(new StreamRecord( CRow( Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true), 4001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong, 2L: JLong, 4L: JLong), true), 4801)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong, 1L: JLong, 4L: JLong), true), 4801)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 2L: JLong, 6L: JLong), true), 6501)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 1L: JLong, 6L: JLong), true), 6501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 2L: JLong, 6L: JLong), true), 6501)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 1L: JLong, 6L: JLong), true), 6501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 30L: JLong, 30L: JLong), true), 6501)) + Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true), 6501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 2L: JLong, 7L: JLong), true), 7001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 1L: JLong, 7L: JLong), true), 7001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 8L: JLong, 8L: JLong), true), 10001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 1L: JLong, 8L: JLong), true), 8001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 8L: JLong, 10L: JLong), true), 12001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 1L: JLong, 10L: JLong), true), 12001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 8L: JLong, 10L: JLong), true), 12001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 1L: JLong, 10L: JLong), true), 12001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 12001)) + Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 10L: JLong, 40L: JLong), true), 12001)) verify(expectedOutput, result, new RowResultSortComparator(6)) testHarness.close() @@ -755,8 +738,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ new RowTimeUnboundedRowsOver( genMinMaxAggFunction, minMaxAggregationStateType, - minMaxCRowType, - qConfig)) + minMaxCRowType)) val testHarness = createHarnessTester( @@ -766,7 +748,6 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.open() - // register cleanup timer with 3801 testHarness.processWatermark(800) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 801)) @@ -775,7 +756,6 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong), true), 2501)) - // trigger cleanup timer and register cleanup timer with 7001 testHarness.processWatermark(4000) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong), true), 4001)) @@ -788,7 +768,6 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong), true), 4801)) - // using historical data and register cleanup timer with 9501 testHarness.processWatermark(6500) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong), true), 6501)) @@ -805,7 +784,6 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 8001)) - // trigger cleanup timer and register cleanup timer with 15001 testHarness.processWatermark(12000) testHarness.processElement(new StreamRecord( CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 12001)) @@ -828,40 +806,40 @@ class OverWindowHarnessTest extends HarnessTestBase{ Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true), 2501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 2L: JLong, 2L: JLong), true), 4001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true), 4001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 2L: JLong, 3L: JLong), true), 4001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true), 4001)) expectedOutput.add(new StreamRecord( CRow( Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true), 4001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong, 2L: JLong, 4L: JLong), true), 4801)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong, 1L: JLong, 4L: JLong), true), 4801)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 2L: JLong, 5L: JLong), true), 6501)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 1L: JLong, 5L: JLong), true), 6501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 2L: JLong, 6L: JLong), true), 6501)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 1L: JLong, 6L: JLong), true), 6501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 30L: JLong, 30L: JLong), true), 6501)) + Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true), 6501)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 2L: JLong, 7L: JLong), true), 7001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 1L: JLong, 7L: JLong), true), 7001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 2L: JLong, 8L: JLong), true), 8001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 1L: JLong, 8L: JLong), true), 8001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 9L: JLong, 9L: JLong), true), 12001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 1L: JLong, 9L: JLong), true), 12001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 9L: JLong, 10L: JLong), true), 12001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 1L: JLong, 10L: JLong), true), 12001)) expectedOutput.add(new StreamRecord( CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 12001)) + Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 10L: JLong, 40L: JLong), true), 12001)) verify(expectedOutput, result, new RowResultSortComparator(6)) testHarness.close() diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala index 3d79e2284cdb8..c4e2433833793 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala @@ -28,7 +28,7 @@ class MockTableEnvironment extends TableEnvironment(new TableConfig) { override private[flink] def writeToSink[T]( table: Table, sink: TableSink[T], - qConfig: QueryConfig): Unit = ??? + queryConfig: QueryConfig): Unit = ??? override protected def checkValidTableName(name: String): Unit = ???