You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by fh...@apache.org on 2017/05/12 06:11:02 UTC
[1/5] flink git commit: [FLINK-6491] [table] Add QueryConfig and
state clean up for non-windowed aggregates.
Repository: flink
Updated Branches:
refs/heads/master 704098725 -> b50ef4b8d
[FLINK-6491] [table] Add QueryConfig and state clean up for non-windowed aggregates.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/d16339db
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/d16339db
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/d16339db
Branch: refs/heads/master
Commit: d16339db83b98f41ef14fbd278530dc219f02ed8
Parents: 7040987
Author: Fabian Hueske <fh...@apache.org>
Authored: Mon May 8 18:41:37 2017 +0200
Committer: Fabian Hueske <fh...@apache.org>
Committed: Thu May 11 23:42:19 2017 +0200
----------------------------------------------------------------------
.../flink/table/api/BatchTableEnvironment.scala | 17 ++-
.../apache/flink/table/api/QueryConfig.scala | 102 ++++++++++++++++
.../table/api/StreamTableEnvironment.scala | 51 ++++++--
.../flink/table/api/TableEnvironment.scala | 2 +-
.../table/api/java/StreamTableEnvironment.scala | 115 +++++++++++++++++--
.../api/scala/StreamTableEnvironment.scala | 46 +++++++-
.../table/api/scala/TableConversions.scala | 40 ++++++-
.../org/apache/flink/table/api/table.scala | 26 ++++-
.../plan/nodes/datastream/DataStreamCalc.scala | 8 +-
.../nodes/datastream/DataStreamCorrelate.scala | 8 +-
.../datastream/DataStreamGroupAggregate.scala | 20 +++-
.../DataStreamGroupWindowAggregate.scala | 8 +-
.../datastream/DataStreamOverAggregate.scala | 9 +-
.../plan/nodes/datastream/DataStreamRel.scala | 7 +-
.../plan/nodes/datastream/DataStreamScan.scala | 7 +-
.../plan/nodes/datastream/DataStreamUnion.scala | 10 +-
.../nodes/datastream/DataStreamValues.scala | 6 +-
.../datastream/StreamTableSourceScan.scala | 7 +-
.../table/runtime/aggregate/AggregateUtil.scala | 6 +-
.../aggregate/GroupAggProcessFunction.scala | 54 ++++++++-
.../table/utils/MockTableEnvironment.scala | 9 +-
21 files changed, 494 insertions(+), 64 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/d16339db/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.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 2a3cedf..f33c187 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.")
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/d16339db/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/QueryConfig.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala
new file mode 100644
index 0000000..8e8b5ac
--- /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
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/d16339db/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
----------------------------------------------------------------------
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 aef2b1b..c594d4c 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
http://git-wip-us.apache.org/repos/asf/flink/blob/d16339db/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
----------------------------------------------------------------------
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 bf4a8e0..9f50f0c 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
@@ -510,7 +510,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.
http://git-wip-us.apache.org/repos/asf/flink/blob/d16339db/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/StreamTableEnvironment.scala
----------------------------------------------------------------------
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 a70bcca..c3b5951 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)
}
http://git-wip-us.apache.org/repos/asf/flink/blob/d16339db/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala
----------------------------------------------------------------------
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 e5ad6c2..56f7d55 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))
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/d16339db/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/TableConversions.scala
----------------------------------------------------------------------
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 5efff62..966b42f 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.")
+ }
+ }
+
+
+
}
http://git-wip-us.apache.org/repos/asf/flink/blob/d16339db/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
----------------------------------------------------------------------
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 310a75f..5a2eb1c 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)
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/d16339db/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala
----------------------------------------------------------------------
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 ce0f966..0e377b5 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)
http://git-wip-us.apache.org/repos/asf/flink/blob/d16339db/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala
----------------------------------------------------------------------
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 19ad89b..cbd818a 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]
http://git-wip-us.apache.org/repos/asf/flink/blob/d16339db/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala
----------------------------------------------------------------------
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 506c0cb..f01b24a 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))
http://git-wip-us.apache.org/repos/asf/flink/blob/d16339db/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala
----------------------------------------------------------------------
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 ef207b0..d2aaad0 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](
http://git-wip-us.apache.org/repos/asf/flink/blob/d16339db/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
----------------------------------------------------------------------
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 4061242..8e97884 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)
http://git-wip-us.apache.org/repos/asf/flink/blob/d16339db/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala
----------------------------------------------------------------------
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 9754de4..6f6edf7 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
http://git-wip-us.apache.org/repos/asf/flink/blob/d16339db/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamScan.scala
----------------------------------------------------------------------
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 c613646..e64bf0f 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)
http://git-wip-us.apache.org/repos/asf/flink/blob/d16339db/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala
----------------------------------------------------------------------
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 654c259..6cc7396 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)
}
http://git-wip-us.apache.org/repos/asf/flink/blob/d16339db/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamValues.scala
----------------------------------------------------------------------
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 32c9aaf..ba6b025 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
http://git-wip-us.apache.org/repos/asf/flink/blob/d16339db/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala
----------------------------------------------------------------------
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 b2d7019..225f23f 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(
http://git-wip-us.apache.org/repos/asf/flink/blob/d16339db/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
----------------------------------------------------------------------
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 768c9cb..27392c7 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)
}
http://git-wip-us.apache.org/repos/asf/flink/blob/d16339db/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala
----------------------------------------------------------------------
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 6ee37e6..84fee87 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()
+ }
+ }
+
}
http://git-wip-us.apache.org/repos/asf/flink/blob/d16339db/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala
----------------------------------------------------------------------
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 8626b07..3d79e22 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 = ???
[4/5] flink git commit: [FLINK-6483] [table] Add materialization of
time indicators.
Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverterTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverterTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverterTest.scala
new file mode 100644
index 0000000..7ac0874
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverterTest.scala
@@ -0,0 +1,351 @@
+/*
+ * 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.calcite
+
+import java.sql.Timestamp
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.calcite.RelTimeIndicatorConverterTest.TableFunc
+import org.apache.flink.table.expressions.{TimeIntervalUnit, WindowReference}
+import org.apache.flink.table.functions.TableFunction
+import org.apache.flink.table.plan.logical.TumblingGroupWindow
+import org.apache.flink.table.utils.TableTestBase
+import org.apache.flink.table.utils.TableTestUtil._
+import org.junit.Test
+
+/**
+ * Tests for [[RelTimeIndicatorConverter]].
+ */
+class RelTimeIndicatorConverterTest extends TableTestBase {
+
+ @Test
+ def testSimpleMaterialization(): Unit = {
+ val util = streamTestUtil()
+ val t = util.addTable[(Long, Long, Int)]('rowtime.rowtime, 'long, 'int, 'proctime.proctime)
+
+ val result = t
+ .select('rowtime.floor(TimeIntervalUnit.DAY) as 'rowtime, 'long)
+ .filter('long > 0)
+ .select('rowtime)
+
+ val expected = unaryNode(
+ "DataStreamCalc",
+ streamTableNode(0),
+ term("select", "FLOOR(TIME_MATERIALIZATION(rowtime)", "FLAG(DAY)) AS rowtime"),
+ term("where", ">(long, 0)")
+ )
+
+ util.verifyTable(result, expected)
+ }
+
+ @Test
+ def testSelectAll(): Unit = {
+ val util = streamTestUtil()
+ val t = util.addTable[(Long, Long, Int)]('rowtime.rowtime, 'long, 'int, 'proctime.proctime)
+
+ val result = t.select('*)
+
+ val expected = unaryNode(
+ "DataStreamCalc",
+ streamTableNode(0),
+ term("select", "TIME_MATERIALIZATION(rowtime) AS rowtime", "long", "int",
+ "TIME_MATERIALIZATION(proctime) AS proctime")
+ )
+
+ util.verifyTable(result, expected)
+ }
+
+ @Test
+ def testFilteringOnRowtime(): Unit = {
+ val util = streamTestUtil()
+ val t = util.addTable[(Long, Long, Int)]('rowtime.rowtime, 'long, 'int)
+
+ val result = t
+ .filter('rowtime > "1990-12-02 12:11:11".toTimestamp)
+ .select('rowtime)
+
+ val expected = unaryNode(
+ "DataStreamCalc",
+ streamTableNode(0),
+ term("select", "TIME_MATERIALIZATION(rowtime) AS rowtime"),
+ term("where", ">(TIME_MATERIALIZATION(rowtime), 1990-12-02 12:11:11)")
+ )
+
+ util.verifyTable(result, expected)
+ }
+
+ @Test
+ def testGroupingOnRowtime(): Unit = {
+ val util = streamTestUtil()
+ val t = util.addTable[(Long, Long, Int)]('rowtime.rowtime, 'long, 'int, 'proctime.proctime)
+
+ val result = t
+ .groupBy('rowtime)
+ .select('long.count)
+
+ val expected = unaryNode(
+ "DataStreamCalc",
+ unaryNode(
+ "DataStreamGroupAggregate",
+ unaryNode(
+ "DataStreamCalc",
+ streamTableNode(0),
+ term("select", "long", "TIME_MATERIALIZATION(rowtime) AS rowtime")
+ ),
+ term("groupBy", "rowtime"),
+ term("select", "rowtime", "COUNT(long) AS TMP_0")
+ ),
+ term("select", "TMP_0")
+ )
+
+ util.verifyTable(result, expected)
+ }
+
+ @Test
+ def testGroupingOnProctimeSql(): Unit = {
+ val util = streamTestUtil()
+ util.addTable[(Long, Int)]("MyTable" , 'long, 'int, 'proctime.proctime)
+
+ val result = util.tEnv.sql("SELECT COUNT(long) FROM MyTable GROUP BY proctime")
+
+ val expected = unaryNode(
+ "DataStreamCalc",
+ unaryNode(
+ "DataStreamGroupAggregate",
+ unaryNode(
+ "DataStreamCalc",
+ streamTableNode(0),
+ term("select", "TIME_MATERIALIZATION(proctime) AS proctime", "long")
+ ),
+ term("groupBy", "proctime"),
+ term("select", "proctime", "COUNT(long) AS EXPR$0")
+ ),
+ term("select", "EXPR$0")
+ )
+
+ util.verifyTable(result, expected)
+ }
+
+ @Test
+ def testAggregationOnRowtime(): Unit = {
+ val util = streamTestUtil()
+ val t = util.addTable[(Long, Long, Int)]('rowtime.rowtime, 'long, 'int)
+
+ val result = t
+ .groupBy('long)
+ .select('rowtime.min)
+
+ val expected = unaryNode(
+ "DataStreamCalc",
+ unaryNode(
+ "DataStreamGroupAggregate",
+ unaryNode(
+ "DataStreamCalc",
+ streamTableNode(0),
+ term("select", "TIME_MATERIALIZATION(rowtime) AS rowtime", "long")
+ ),
+ term("groupBy", "long"),
+ term("select", "long", "MIN(rowtime) AS TMP_0")
+ ),
+ term("select", "TMP_0")
+ )
+
+ util.verifyTable(result, expected)
+ }
+
+ @Test
+ def testAggregationOnProctimeSql(): Unit = {
+ val util = streamTestUtil()
+ util.addTable[(Long, Int)]("MyTable" , 'long, 'int, 'proctime.proctime)
+
+ val result = util.tEnv.sql("SELECT MIN(proctime) FROM MyTable GROUP BY long")
+
+ val expected = unaryNode(
+ "DataStreamCalc",
+ unaryNode(
+ "DataStreamGroupAggregate",
+ unaryNode(
+ "DataStreamCalc",
+ streamTableNode(0),
+ term("select", "long", "TIME_MATERIALIZATION(proctime) AS proctime")
+ ),
+ term("groupBy", "long"),
+ term("select", "long", "MIN(proctime) AS EXPR$0")
+ ),
+ term("select", "EXPR$0")
+ )
+
+ util.verifyTable(result, expected)
+ }
+
+ @Test
+ def testTableFunction(): Unit = {
+ val util = streamTestUtil()
+ val t = util.addTable[(Long, Long, Int)]('rowtime.rowtime, 'long, 'int, 'proctime.proctime)
+ val func = new TableFunc
+
+ val result = t.join(func('rowtime, 'proctime) as 's).select('rowtime, 'proctime, 's)
+
+ val expected = unaryNode(
+ "DataStreamCalc",
+ unaryNode(
+ "DataStreamCorrelate",
+ streamTableNode(0),
+ term("invocation",
+ s"${func.functionIdentifier}(TIME_MATERIALIZATION($$0), TIME_MATERIALIZATION($$3))"),
+ term("function", func),
+ term("rowType", "RecordType(TIMESTAMP(3) rowtime, BIGINT long, INTEGER int, " +
+ "TIMESTAMP(3) proctime, VARCHAR(2147483647) s)"),
+ term("joinType", "INNER")
+ ),
+ term("select",
+ "TIME_MATERIALIZATION(rowtime) AS rowtime",
+ "TIME_MATERIALIZATION(proctime) AS proctime",
+ "s")
+ )
+
+ util.verifyTable(result, expected)
+ }
+
+ @Test
+ def testWindow(): Unit = {
+ val util = streamTestUtil()
+ val t = util.addTable[(Long, Long, Int)]('rowtime.rowtime, 'long, 'int)
+
+ val result = t
+ .window(Tumble over 100.millis on 'rowtime as 'w)
+ .groupBy('w, 'long)
+ .select('w.end as 'rowtime, 'long, 'int.sum)
+
+ val expected = unaryNode(
+ "DataStreamCalc",
+ unaryNode(
+ "DataStreamGroupWindowAggregate",
+ streamTableNode(0),
+ term("groupBy", "long"),
+ term(
+ "window",
+ TumblingGroupWindow(
+ WindowReference("w"),
+ 'rowtime,
+ 100.millis)),
+ term("select", "long", "SUM(int) AS TMP_1", "end(WindowReference(w)) AS TMP_0")
+ ),
+ term("select", "TMP_0 AS rowtime", "long", "TMP_1")
+ )
+
+ util.verifyTable(result, expected)
+ }
+
+ @Test
+ def testWindowSql(): Unit = {
+ val util = streamTestUtil()
+ util.addTable[(Long, Long, Int)]("MyTable", 'rowtime.rowtime, 'long, 'int)
+
+ val result = util.tEnv.sql(
+ "SELECT TUMBLE_END(rowtime, INTERVAL '0.1' SECOND) AS `rowtime`, `long`, " +
+ "SUM(`int`) FROM MyTable " +
+ "GROUP BY `long`, TUMBLE(rowtime, INTERVAL '0.1' SECOND)")
+
+ val expected = unaryNode(
+ "DataStreamCalc",
+ unaryNode(
+ "DataStreamGroupWindowAggregate",
+ streamTableNode(0),
+ term("groupBy", "long"),
+ term(
+ "window",
+ TumblingGroupWindow(
+ 'w$,
+ 'rowtime,
+ 100.millis)),
+ term("select", "long", "SUM(int) AS EXPR$2", "start('w$) AS w$start", "end('w$) AS w$end")
+ ),
+ term("select", "w$end", "long", "EXPR$2")
+ )
+
+ util.verifyTable(result, expected)
+ }
+
+ @Test
+ def testWindowWithAggregationOnRowtimeSql(): Unit = {
+ val util = streamTestUtil()
+ util.addTable[(Long, Long, Int)]("MyTable", 'rowtime.rowtime, 'long, 'int)
+
+ val result = util.tEnv.sql("SELECT MIN(rowtime), long FROM MyTable " +
+ "GROUP BY long, TUMBLE(rowtime, INTERVAL '0.1' SECOND)")
+
+ val expected = unaryNode(
+ "DataStreamCalc",
+ unaryNode(
+ "DataStreamGroupWindowAggregate",
+ unaryNode(
+ "DataStreamCalc",
+ streamTableNode(0),
+ term("select", "long", "1970-01-01 00:00:00 AS $f1",
+ "TIME_MATERIALIZATION(rowtime) AS $f2")
+ ),
+ term("groupBy", "long"),
+ term(
+ "window",
+ TumblingGroupWindow(
+ 'w$,
+ 'rowtime,
+ 100.millis)),
+ term("select", "long", "MIN($f2) AS EXPR$0")
+ ),
+ term("select", "EXPR$0", "long")
+ )
+
+ util.verifyTable(result, expected)
+ }
+
+ @Test
+ def testUnion(): Unit = {
+ val util = streamTestUtil()
+ val t = util.addTable[(Long, Long, Int)]("MyTable", 'rowtime.rowtime, 'long, 'int)
+
+ val result = t.unionAll(t).select('rowtime)
+
+ val expected = unaryNode(
+ "DataStreamCalc",
+ binaryNode(
+ "DataStreamUnion",
+ streamTableNode(0),
+ streamTableNode(0),
+ term("union all", "rowtime", "long", "int")
+ ),
+ term("select", "TIME_MATERIALIZATION(rowtime) AS rowtime")
+ )
+
+ util.verifyTable(result, expected)
+ }
+
+}
+
+object RelTimeIndicatorConverterTest {
+
+ class TableFunc extends TableFunction[String] {
+ val t = new Timestamp(0L)
+ def eval(time1: Long, time2: Timestamp): Unit = {
+ collect(time1.toString + time2.after(t))
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/datastream/TimeAttributesITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/datastream/TimeAttributesITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/datastream/TimeAttributesITCase.scala
new file mode 100644
index 0000000..7d7088e
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/datastream/TimeAttributesITCase.scala
@@ -0,0 +1,237 @@
+/*
+ * 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.datastream
+
+import java.math.BigDecimal
+
+import org.apache.flink.api.scala._
+import org.apache.flink.streaming.api.TimeCharacteristic
+import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.streaming.api.watermark.Watermark
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.scala.stream.utils.StreamITCase
+import org.apache.flink.table.api.{TableEnvironment, TableException, Types}
+import org.apache.flink.table.calcite.RelTimeIndicatorConverterTest.TableFunc
+import org.apache.flink.table.expressions.TimeIntervalUnit
+import org.apache.flink.table.runtime.datastream.TimeAttributesITCase.TimestampWithEqualWatermark
+import org.apache.flink.types.Row
+import org.junit.Assert._
+import org.junit.Test
+
+import scala.collection.mutable
+
+/**
+ * Tests for access and materialization of time attributes.
+ */
+class TimeAttributesITCase extends StreamingMultipleProgramsTestBase {
+
+ val data = List(
+ (1L, 1, 1d, 1f, new BigDecimal("1"), "Hi"),
+ (2L, 2, 2d, 2f, new BigDecimal("2"), "Hallo"),
+ (3L, 2, 2d, 2f, new BigDecimal("2"), "Hello"),
+ (4L, 5, 5d, 5f, new BigDecimal("5"), "Hello"),
+ (7L, 3, 3d, 3f, new BigDecimal("3"), "Hello"),
+ (8L, 3, 3d, 3f, new BigDecimal("3"), "Hello world"),
+ (16L, 4, 4d, 4f, new BigDecimal("4"), "Hello world"))
+
+ @Test(expected = classOf[TableException])
+ def testInvalidTimeCharacteristic(): Unit = {
+ val env = StreamExecutionEnvironment.getExecutionEnvironment
+ val tEnv = TableEnvironment.getTableEnvironment(env)
+ val stream = env
+ .fromCollection(data)
+ .assignTimestampsAndWatermarks(new TimestampWithEqualWatermark())
+ stream.toTable(tEnv, 'rowtime.rowtime, 'int, 'double, 'float, 'bigdec, 'string)
+ }
+
+ @Test
+ def testCalcMaterialization(): Unit = {
+ val env = StreamExecutionEnvironment.getExecutionEnvironment
+ env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+ val tEnv = TableEnvironment.getTableEnvironment(env)
+ StreamITCase.testResults = mutable.MutableList()
+
+ val stream = env
+ .fromCollection(data)
+ .assignTimestampsAndWatermarks(new TimestampWithEqualWatermark())
+ val table = stream.toTable(tEnv, 'rowtime.rowtime, 'int, 'double, 'float, 'bigdec, 'string)
+
+ val t = table.select('rowtime.cast(Types.STRING))
+
+ val results = t.toDataStream[Row]
+ results.addSink(new StreamITCase.StringSink)
+ env.execute()
+
+ val expected = Seq(
+ "1970-01-01 00:00:00.001",
+ "1970-01-01 00:00:00.002",
+ "1970-01-01 00:00:00.003",
+ "1970-01-01 00:00:00.004",
+ "1970-01-01 00:00:00.007",
+ "1970-01-01 00:00:00.008",
+ "1970-01-01 00:00:00.016")
+ assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+ }
+
+ @Test
+ def testCalcMaterialization2(): Unit = {
+ val env = StreamExecutionEnvironment.getExecutionEnvironment
+ env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+ val tEnv = TableEnvironment.getTableEnvironment(env)
+ StreamITCase.testResults = mutable.MutableList()
+
+ val stream = env
+ .fromCollection(data)
+ .assignTimestampsAndWatermarks(new TimestampWithEqualWatermark())
+ val table = stream.toTable(tEnv, 'rowtime.rowtime, 'int, 'double, 'float, 'bigdec, 'string)
+
+ val t = table
+ .filter('rowtime.cast(Types.LONG) > 4)
+ .select('rowtime, 'rowtime.floor(TimeIntervalUnit.DAY), 'rowtime.ceil(TimeIntervalUnit.DAY))
+
+ val results = t.toDataStream[Row]
+ results.addSink(new StreamITCase.StringSink)
+ env.execute()
+
+ val expected = Seq(
+ "1970-01-01 00:00:00.007,1970-01-01 00:00:00.0,1970-01-02 00:00:00.0",
+ "1970-01-01 00:00:00.008,1970-01-01 00:00:00.0,1970-01-02 00:00:00.0",
+ "1970-01-01 00:00:00.016,1970-01-01 00:00:00.0,1970-01-02 00:00:00.0")
+ assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+ }
+
+ @Test
+ def testTableFunction(): Unit = {
+ val env = StreamExecutionEnvironment.getExecutionEnvironment
+ env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+ val tEnv = TableEnvironment.getTableEnvironment(env)
+ StreamITCase.testResults = mutable.MutableList()
+
+ val stream = env
+ .fromCollection(data)
+ .assignTimestampsAndWatermarks(new TimestampWithEqualWatermark())
+ val table = stream.toTable(
+ tEnv, 'rowtime.rowtime, 'int, 'double, 'float, 'bigdec, 'string, 'proctime.proctime)
+ val func = new TableFunc
+
+ val t = table.join(func('rowtime, 'proctime) as 's).select('rowtime, 's)
+
+ val results = t.toDataStream[Row]
+ results.addSink(new StreamITCase.StringSink)
+ env.execute()
+
+ val expected = Seq(
+ "1970-01-01 00:00:00.001,1true",
+ "1970-01-01 00:00:00.002,2true",
+ "1970-01-01 00:00:00.003,3true",
+ "1970-01-01 00:00:00.004,4true",
+ "1970-01-01 00:00:00.007,7true",
+ "1970-01-01 00:00:00.008,8true",
+ "1970-01-01 00:00:00.016,16true")
+ assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+ }
+
+ @Test
+ def testUnion(): Unit = {
+ val env = StreamExecutionEnvironment.getExecutionEnvironment
+ env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+ val tEnv = TableEnvironment.getTableEnvironment(env)
+ StreamITCase.testResults = mutable.MutableList()
+
+ val stream = env
+ .fromCollection(data)
+ .assignTimestampsAndWatermarks(new TimestampWithEqualWatermark())
+ val table = stream.toTable(
+ tEnv, 'rowtime.rowtime, 'int, 'double, 'float, 'bigdec, 'string)
+
+ val t = table.unionAll(table).select('rowtime)
+
+ val results = t.toDataStream[Row]
+ results.addSink(new StreamITCase.StringSink)
+ env.execute()
+
+ val expected = Seq(
+ "1970-01-01 00:00:00.001",
+ "1970-01-01 00:00:00.001",
+ "1970-01-01 00:00:00.002",
+ "1970-01-01 00:00:00.002",
+ "1970-01-01 00:00:00.003",
+ "1970-01-01 00:00:00.003",
+ "1970-01-01 00:00:00.004",
+ "1970-01-01 00:00:00.004",
+ "1970-01-01 00:00:00.007",
+ "1970-01-01 00:00:00.007",
+ "1970-01-01 00:00:00.008",
+ "1970-01-01 00:00:00.008",
+ "1970-01-01 00:00:00.016",
+ "1970-01-01 00:00:00.016")
+ assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+ }
+
+ @Test
+ def testWindowWithAggregationOnRowtimeSql(): Unit = {
+ val env = StreamExecutionEnvironment.getExecutionEnvironment
+ env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+ val tEnv = TableEnvironment.getTableEnvironment(env)
+ StreamITCase.testResults = mutable.MutableList()
+
+ val stream = env
+ .fromCollection(data)
+ .assignTimestampsAndWatermarks(new TimestampWithEqualWatermark())
+ val table = stream.toTable(tEnv, 'rowtime.rowtime, 'int, 'double, 'float, 'bigdec, 'string)
+ tEnv.registerTable("MyTable", table)
+
+ val t = tEnv.sql("SELECT COUNT(`rowtime`) FROM MyTable " +
+ "GROUP BY TUMBLE(rowtime, INTERVAL '0.003' SECOND)")
+
+ val results = t.toDataStream[Row]
+ results.addSink(new StreamITCase.StringSink)
+ env.execute()
+
+ val expected = Seq(
+ "1",
+ "2",
+ "2",
+ "2"
+ )
+ assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+ }
+
+}
+
+object TimeAttributesITCase {
+ class TimestampWithEqualWatermark
+ extends AssignerWithPunctuatedWatermarks[(Long, Int, Double, Float, BigDecimal, String)] {
+
+ override def checkAndGetNextWatermark(
+ lastElement: (Long, Int, Double, Float, BigDecimal, String),
+ extractedTimestamp: Long)
+ : Watermark = {
+ new Watermark(extractedTimestamp)
+ }
+
+ override def extractTimestamp(
+ element: (Long, Int, Double, Float, BigDecimal, String),
+ previousElementTimestamp: Long): Long = {
+ element._1
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala
index 0e6d461..65014cd 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala
@@ -24,9 +24,11 @@ import org.apache.flink.api.java.{DataSet => JDataSet}
import org.apache.flink.table.api.{Table, TableEnvironment}
import org.apache.flink.table.api.scala._
import org.apache.flink.api.scala.{DataSet, ExecutionEnvironment}
+import org.apache.flink.streaming.api.TimeCharacteristic
import org.apache.flink.table.expressions.Expression
import org.apache.flink.table.functions.{ScalarFunction, TableFunction}
import org.apache.flink.streaming.api.datastream.{DataStream => JDataStream}
+import org.apache.flink.streaming.api.environment.{StreamExecutionEnvironment => JStreamExecutionEnvironment}
import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment}
import org.apache.flink.table.api.scala.batch.utils.LogicalPlanFormatUtils
import org.junit.Assert.assertEquals
@@ -174,7 +176,10 @@ case class BatchTableTestUtil() extends TableTestUtil {
case class StreamTableTestUtil() extends TableTestUtil {
+ val javaEnv = mock(classOf[JStreamExecutionEnvironment])
+ when(javaEnv.getStreamTimeCharacteristic).thenReturn(TimeCharacteristic.EventTime)
val env = mock(classOf[StreamExecutionEnvironment])
+ when(env.getWrappedStreamExecutionEnvironment).thenReturn(javaEnv)
val tEnv = TableEnvironment.getTableEnvironment(env)
def addTable[T: TypeInformation](
[3/5] flink git commit: [FLINK-6491] [table] Add QueryConfig and
state clean up for over-windowed aggregates.
Posted by fh...@apache.org.
[FLINK-6491] [table] Add QueryConfig and state clean up for over-windowed aggregates.
This closes #3863.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/24808871
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/24808871
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/24808871
Branch: refs/heads/master
Commit: 2480887180d881c30d228a73c746f94abbcbbb64
Parents: d16339d
Author: sunjincheng121 <su...@gmail.com>
Authored: Tue May 9 14:36:42 2017 +0800
Committer: Fabian Hueske <fh...@apache.org>
Committed: Thu May 11 23:53:25 2017 +0200
----------------------------------------------------------------------
.../flink/table/api/BatchTableEnvironment.scala | 8 +-
.../apache/flink/table/api/QueryConfig.scala | 102 -----
.../table/api/StreamTableEnvironment.scala | 38 +-
.../table/api/java/StreamTableEnvironment.scala | 35 +-
.../apache/flink/table/api/queryConfig.scala | 102 +++++
.../api/scala/StreamTableEnvironment.scala | 20 +-
.../table/api/scala/TableConversions.scala | 13 +-
.../org/apache/flink/table/api/table.scala | 6 +-
.../plan/nodes/datastream/DataStreamCalc.scala | 5 +-
.../nodes/datastream/DataStreamCorrelate.scala | 4 +-
.../datastream/DataStreamGroupAggregate.scala | 10 +-
.../DataStreamGroupWindowAggregate.scala | 4 +-
.../datastream/DataStreamOverAggregate.scala | 34 +-
.../plan/nodes/datastream/DataStreamRel.scala | 4 +-
.../plan/nodes/datastream/DataStreamScan.scala | 2 +-
.../plan/nodes/datastream/DataStreamUnion.scala | 6 +-
.../nodes/datastream/DataStreamValues.scala | 2 +-
.../datastream/StreamTableSourceScan.scala | 2 +-
.../table/runtime/aggregate/AggregateUtil.scala | 32 +-
.../aggregate/GroupAggProcessFunction.scala | 44 +-
.../aggregate/ProcTimeBoundedRangeOver.scala | 26 +-
.../aggregate/ProcTimeBoundedRowsOver.scala | 20 +-
.../ProcTimeUnboundedNonPartitionedOver.scala | 20 +-
.../ProcTimeUnboundedPartitionedOver.scala | 20 +-
.../ProcessFunctionWithCleanupState.scala | 85 ++++
.../aggregate/RowTimeBoundedRangeOver.scala | 44 +-
.../aggregate/RowTimeBoundedRowsOver.scala | 41 +-
.../aggregate/RowTimeUnboundedOver.scala | 44 +-
.../stream/table/GroupAggregationsITCase.scala | 13 +-
...ProcessingOverRangeProcessFunctionTest.scala | 336 --------------
.../table/runtime/harness/HarnessTestBase.scala | 281 +++++++++++-
.../runtime/harness/NonWindowHarnessTest.scala | 157 +++++++
.../runtime/harness/OverWindowHarnessTest.scala | 458 ++++++++++---------
.../table/utils/MockTableEnvironment.scala | 2 +-
34 files changed, 1209 insertions(+), 811 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.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 f33c187..3c0f51b 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.")
}
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/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/QueryConfig.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala
deleted file mode 100644
index 8e8b5ac..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/QueryConfig.scala
+++ /dev/null
@@ -1,102 +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.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
- }
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
----------------------------------------------------------------------
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 c594d4c..d68da04 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
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/StreamTableEnvironment.scala
----------------------------------------------------------------------
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 c3b5951..311986c 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,17 @@ 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 +212,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 +245,7 @@ class StreamTableEnvironment(
table: Table,
clazz: Class[T]): DataStream[JTuple2[JBool, T]] = {
- toRetractStream(table, clazz, qConf)
+ toRetractStream(table, clazz, queryConfig)
}
/**
@@ -266,7 +269,7 @@ class StreamTableEnvironment(
table: Table,
typeInfo: TypeInformation[T]): DataStream[JTuple2[JBool, T]] = {
- toRetractStream(table, typeInfo, qConf)
+ toRetractStream(table, typeInfo, queryConfig)
}
/**
@@ -283,21 +286,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 +319,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 +335,7 @@ class StreamTableEnvironment(
)
translate[JTuple2[JBool, T]](
table,
- qConfig,
+ queryConfig,
updatesAsRetraction = true,
withChangeFlag = true)(resultTypeInfo)
}
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/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/queryConfig.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/queryConfig.scala
new file mode 100644
index 0000000..c8fbab7
--- /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.queryConfig]]
+ * 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: [[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 withIdleStateRetentionTime(time: Time): StreamQueryConfig = {
+ withIdleStateRetentionTime(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 withIdleStateRetentionTime(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
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala
----------------------------------------------------------------------
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 56f7d55..8c6b273 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))
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/TableConversions.scala
----------------------------------------------------------------------
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 966b42f..9874a9e 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 " +
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
----------------------------------------------------------------------
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 5a2eb1c..ca61c65 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)
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala
----------------------------------------------------------------------
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 0e377b5..5f270f6 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)
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala
----------------------------------------------------------------------
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 cbd818a..5b32b10 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]
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala
----------------------------------------------------------------------
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 f01b24a..e5d8088 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,16 @@ 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) {
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 +149,7 @@ class DataStreamGroupAggregate(
inputSchema.logicalType,
inputSchema.physicalFieldTypeInfo,
groupings,
- qConfig,
+ queryConfig,
DataStreamRetractionRules.isAccRetract(this),
DataStreamRetractionRules.isAccRetract(getInput))
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala
----------------------------------------------------------------------
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 d2aaad0..2a71592 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](
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
----------------------------------------------------------------------
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 8e97884..a9fbf02 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,23 @@ 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 (!logicWindow.groups.get(0).keys.isEmpty && queryConfig.getMinIdleStateRetentionTime < 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,18 +141,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(
+ queryConfig,
generator,
inputDS,
isRowTimeType = false,
@@ -145,8 +155,10 @@ class DataStreamOverAggregate(
} else if (
overWindow.lowerBound.isPreceding && !overWindow.lowerBound.isUnbounded &&
overWindow.upperBound.isCurrentRow) {
+
// bounded OVER window
createBoundedAndCurrentRowOverWindow(
+ queryConfig,
generator,
inputDS,
isRowTimeType = false,
@@ -162,6 +174,7 @@ class DataStreamOverAggregate(
overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) {
// unbounded OVER window
createUnboundedAndCurrentRowOverWindow(
+ queryConfig,
generator,
inputDS,
isRowTimeType = true,
@@ -169,6 +182,7 @@ class DataStreamOverAggregate(
} else if (overWindow.lowerBound.isPreceding && overWindow.upperBound.isCurrentRow) {
// bounded OVER window
createBoundedAndCurrentRowOverWindow(
+ queryConfig,
generator,
inputDS,
isRowTimeType = true,
@@ -185,6 +199,7 @@ class DataStreamOverAggregate(
}
def createUnboundedAndCurrentRowOverWindow(
+ queryConfig: StreamQueryConfig,
generator: CodeGenerator,
inputDS: DataStream[CRow],
isRowTimeType: Boolean,
@@ -210,6 +225,7 @@ class DataStreamOverAggregate(
inputSchema.physicalType,
inputSchema.physicalTypeInfo,
inputSchema.physicalFieldTypeInfo,
+ queryConfig,
isRowTimeType,
partitionKeys.nonEmpty,
isRowsClause)
@@ -242,6 +258,7 @@ class DataStreamOverAggregate(
}
def createBoundedAndCurrentRowOverWindow(
+ queryConfig: StreamQueryConfig,
generator: CodeGenerator,
inputDS: DataStream[CRow],
isRowTimeType: Boolean,
@@ -269,6 +286,7 @@ class DataStreamOverAggregate(
inputSchema.physicalTypeInfo,
inputSchema.physicalFieldTypeInfo,
precedingOffset,
+ queryConfig,
isRowsClause,
isRowTimeType
)
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala
----------------------------------------------------------------------
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 6f6edf7..65d336f 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
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamScan.scala
----------------------------------------------------------------------
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 e64bf0f..424c6a2 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
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala
----------------------------------------------------------------------
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 6cc7396..6f4980a 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)
}
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamValues.scala
----------------------------------------------------------------------
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 ba6b025..d7c490f 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
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala
----------------------------------------------------------------------
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 225f23f..51e609f 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]]
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
----------------------------------------------------------------------
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 27392c7..8073959 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[_]],
+ queryConfig: StreamQueryConfig,
isRowTimeType: Boolean,
isPartitioned: Boolean,
isRowsClause: Boolean)
@@ -117,23 +118,27 @@ object AggregateUtil {
new RowTimeUnboundedRowsOver(
genFunction,
aggregationStateType,
- CRowTypeInfo(inputTypeInfo))
+ CRowTypeInfo(inputTypeInfo),
+ queryConfig)
} else {
// RANGE unbounded over process function
new RowTimeUnboundedRangeOver(
genFunction,
aggregationStateType,
- CRowTypeInfo(inputTypeInfo))
+ CRowTypeInfo(inputTypeInfo),
+ queryConfig)
}
} else {
if (isPartitioned) {
new ProcTimeUnboundedPartitionedOver(
genFunction,
- aggregationStateType)
+ aggregationStateType,
+ queryConfig)
} else {
new ProcTimeUnboundedNonPartitionedOver(
genFunction,
- aggregationStateType)
+ aggregationStateType,
+ queryConfig)
}
}
}
@@ -155,7 +160,7 @@ object AggregateUtil {
inputRowType: RelDataType,
inputFieldTypes: Seq[TypeInformation[_]],
groupings: Array[Int],
- qConfig: StreamQueryConfig,
+ queryConfig: StreamQueryConfig,
generateRetraction: Boolean,
consumeRetraction: Boolean): ProcessFunction[CRow, CRow] = {
@@ -192,7 +197,7 @@ object AggregateUtil {
genFunction,
aggregationStateType,
generateRetraction,
- qConfig)
+ queryConfig)
}
@@ -217,6 +222,7 @@ object AggregateUtil {
inputTypeInfo: TypeInformation[Row],
inputFieldTypeInfo: Seq[TypeInformation[_]],
precedingOffset: Long,
+ queryConfig: StreamQueryConfig,
isRowsClause: Boolean,
isRowTimeType: Boolean)
: ProcessFunction[CRow, CRow] = {
@@ -258,15 +264,15 @@ object AggregateUtil {
genFunction,
aggregationStateType,
inputRowType,
- precedingOffset
- )
+ precedingOffset,
+ queryConfig)
} else {
new RowTimeBoundedRangeOver(
genFunction,
aggregationStateType,
inputRowType,
- precedingOffset
- )
+ precedingOffset,
+ queryConfig)
}
} else {
if (isRowsClause) {
@@ -274,13 +280,15 @@ object AggregateUtil {
genFunction,
precedingOffset,
aggregationStateType,
- inputRowType)
+ inputRowType,
+ queryConfig)
} else {
new ProcTimeBoundedRangeOver(
genFunction,
precedingOffset,
aggregationStateType,
- inputRowType)
+ inputRowType,
+ queryConfig)
}
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala
----------------------------------------------------------------------
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 84fee87..57ea86e 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,19 +41,13 @@ class GroupAggProcessFunction(
private val genAggregations: GeneratedAggregationsFunction,
private val aggregationStateType: RowTypeInfo,
private val generateRetraction: Boolean,
- private val qConfig: StreamQueryConfig)
- extends ProcessFunction[CRow, CRow]
+ private val queryConfig: StreamQueryConfig)
+ extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
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 = _
@@ -61,8 +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
- private var cleanupTimeState: ValueState[JLong] = _
override def open(config: Configuration) {
LOG.debug(s"Compiling AggregateHelper: $genAggregations.name \n\n " +
@@ -84,11 +76,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 +84,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
@@ -182,11 +157,8 @@ class GroupAggProcessFunction(
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()
+ if (needToCleanupState(timestamp)) {
+ cleanupState(state, cntState)
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala
----------------------------------------------------------------------
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 3fb506f..d50912c 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],
+ queryConfig: StreamQueryConfig)
+ extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
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,15 @@ 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
+ if (needToCleanupState(timestamp)) {
+ // clean up and return
+ cleanupState(rowMapState, accumulatorState)
+ 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 +169,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 +195,6 @@ class ProcTimeBoundedRangeOver(
// update the value of accumulators for future incremental computation
accumulatorState.update(accumulators)
-
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala
----------------------------------------------------------------------
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 0c7f44e..e388c93 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],
+ queryConfig: StreamQueryConfig)
+ extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
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,13 @@ class ProcTimeBoundedRowsOver(
out.collect(output)
}
+ override def onTimer(
+ timestamp: Long,
+ ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
+ out: Collector[CRow]): Unit = {
+
+ if (needToCleanupState(timestamp)) {
+ cleanupState(rowMapState, accumulatorState, counterState, smallestTsState)
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedNonPartitionedOver.scala
----------------------------------------------------------------------
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 8a23132..2a6c9c8 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,
+ queryConfig: StreamQueryConfig)
+ extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
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,16 @@ class ProcTimeUnboundedNonPartitionedOver(
out.collect(output)
}
+ override def onTimer(
+ timestamp: Long,
+ ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
+ out: Collector[CRow]): Unit = {
+
+ if (needToCleanupState(timestamp)) {
+ cleanupState(state)
+ }
+ }
+
override def snapshotState(context: FunctionSnapshotContext): Unit = {
state.clear()
if (null != accumulators) {
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedPartitionedOver.scala
----------------------------------------------------------------------
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 847c1bf..97f0ad7 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,
+ queryConfig: StreamQueryConfig)
+ extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
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,13 @@ class ProcTimeUnboundedPartitionedOver(
out.collect(output)
}
+ override def onTimer(
+ timestamp: Long,
+ ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
+ out: Collector[CRow]): Unit = {
+
+ if (needToCleanupState(timestamp)) {
+ cleanupState(state)
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcessFunctionWithCleanupState.scala
----------------------------------------------------------------------
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 0000000..292fd3b
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcessFunctionWithCleanupState.scala
@@ -0,0 +1,85 @@
+/*
+ * 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.TimeDomain
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.api.{StreamQueryConfig, Types}
+
+abstract class ProcessFunctionWithCleanupState[IN,OUT](queryConfig: StreamQueryConfig)
+ extends ProcessFunction[IN, OUT]{
+
+ protected val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime
+ protected val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime
+ protected val stateCleaningEnabled: Boolean = minRetentionTime > 1
+
+ // 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) {
+
+ // last registered timer
+ val curCleanupTime = cleanupTimeState.value()
+
+ // check if a cleanup timer is registered and
+ // that the current cleanup timer won't delete state we need to keep
+ if (curCleanupTime == null || (currentTime + minRetentionTime) > curCleanupTime) {
+ // we need to register a new (later) timer
+ val cleanupTime = currentTime + maxRetentionTime
+ // register timer and remember clean-up time
+ ctx.timerService().registerProcessingTimeTimer(cleanupTime)
+ cleanupTimeState.update(cleanupTime)
+ }
+ }
+ }
+
+ protected def isProcessingTimeTimer(ctx: OnTimerContext): Boolean = {
+ ctx.timeDomain() == TimeDomain.PROCESSING_TIME
+ }
+
+ protected def needToCleanupState(timestamp: Long): Boolean = {
+ if (stateCleaningEnabled) {
+ val cleanupTime = cleanupTimeState.value()
+ // check that the triggered timer is the last registered processing time timer.
+ null != cleanupTime && timestamp == cleanupTime
+ } else {
+ false
+ }
+ }
+
+ protected def cleanupState(states: State*): Unit = {
+ // clear all state
+ states.foreach(_.clear())
+ this.cleanupTimeState.clear()
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala
----------------------------------------------------------------------
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 4020d44..65edf6d 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,
+ queryConfig: StreamQueryConfig)
+ extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
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(
@@ -106,6 +110,9 @@ class RowTimeBoundedRangeOver(
val input = inputC.row
+ // register state-cleanup timer
+ registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime())
+
// triggering timestamp for trigger calculation
val triggeringTs = ctx.timestamp
@@ -131,6 +138,34 @@ class RowTimeBoundedRangeOver(
timestamp: Long,
ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
out: Collector[CRow]): Unit = {
+
+ if (isProcessingTimeTimer(ctx.asInstanceOf[OnTimerContext])) {
+ if (needToCleanupState(timestamp)) {
+
+ val keysIt = dataState.keys.iterator()
+ val lastProcessedTime = lastTriggeringTsState.value
+
+ // is data left which has not been processed yet?
+ var noRecordsToProcess = true
+ while (keysIt.hasNext && noRecordsToProcess) {
+ if (keysIt.next() > lastProcessedTime) {
+ noRecordsToProcess = false
+ }
+ }
+
+ if (noRecordsToProcess) {
+ // we clean the state
+ cleanupState(dataState, accumulatorState, lastTriggeringTsState)
+ } else {
+ // There are records left to process because a watermark has not been received yet.
+ // This would only happen if the input stream has stopped. So we don't need to clean up.
+ // We leave the state as it is and schedule a new cleanup timer
+ registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime())
+ }
+ }
+ return
+ }
+
// gets all window data from state for the calculation
val inputs: JList[Row] = dataState.get(timestamp)
@@ -196,8 +231,11 @@ class RowTimeBoundedRangeOver(
// update state
accumulatorState.update(accumulators)
- lastTriggeringTsState.update(timestamp)
}
+ lastTriggeringTsState.update(timestamp)
+
+ // update cleanup timer
+ registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime())
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala
----------------------------------------------------------------------
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 5ec6ec7..395ae39 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
@@ -25,6 +25,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.types.Row
import org.apache.flink.util.{Collector, Preconditions}
import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction}
@@ -43,8 +44,9 @@ class RowTimeBoundedRowsOver(
genAggregations: GeneratedAggregationsFunction,
aggregationStateType: RowTypeInfo,
inputRowType: CRowTypeInfo,
- precedingOffset: Long)
- extends ProcessFunction[CRow, CRow]
+ precedingOffset: Long,
+ queryConfig: StreamQueryConfig)
+ extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
with Compiler[GeneratedAggregations] {
Preconditions.checkNotNull(aggregationStateType)
@@ -106,6 +108,8 @@ class RowTimeBoundedRowsOver(
valueTypeInformation)
dataState = getRuntimeContext.getMapState(mapStateDescriptor)
+
+ initCleanupTimeState("RowTimeBoundedRowsOverCleanupTime")
}
override def processElement(
@@ -115,6 +119,9 @@ class RowTimeBoundedRowsOver(
val input = inputC.row
+ // register state-cleanup timer
+ registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime())
+
// triggering timestamp for trigger calculation
val triggeringTs = ctx.timestamp
@@ -141,6 +148,33 @@ class RowTimeBoundedRowsOver(
ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
out: Collector[CRow]): Unit = {
+ if (isProcessingTimeTimer(ctx.asInstanceOf[OnTimerContext])) {
+ if (needToCleanupState(timestamp)) {
+
+ val keysIt = dataState.keys.iterator()
+ val lastProcessedTime = lastTriggeringTsState.value
+
+ // is data left which has not been processed yet?
+ var noRecordsToProcess = true
+ while (keysIt.hasNext && noRecordsToProcess) {
+ if (keysIt.next() > lastProcessedTime) {
+ noRecordsToProcess = false
+ }
+ }
+
+ if (noRecordsToProcess) {
+ // We clean the state
+ cleanupState(dataState, accumulatorState, dataCountState, lastTriggeringTsState)
+ } else {
+ // There are records left to process because a watermark has not been received yet.
+ // This would only happen if the input stream has stopped. So we don't need to clean up.
+ // We leave the state as it is and schedule a new cleanup timer
+ registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime())
+ }
+ }
+ return
+ }
+
// gets all window data from state for the calculation
val inputs: JList[Row] = dataState.get(timestamp)
@@ -220,6 +254,9 @@ class RowTimeBoundedRowsOver(
}
lastTriggeringTsState.update(timestamp)
+
+ // update cleanup timer
+ registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime())
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala
----------------------------------------------------------------------
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 3e2a811..741d2b4 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],
+ queryConfig: StreamQueryConfig)
+ extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig)
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")
}
/**
@@ -101,6 +105,9 @@ abstract class RowTimeUnboundedOver(
val input = inputC.row
+ // register state-cleanup timer
+ registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime())
+
val timestamp = ctx.timestamp()
val curWatermark = ctx.timerService().currentWatermark()
@@ -133,6 +140,24 @@ abstract class RowTimeUnboundedOver(
ctx: ProcessFunction[CRow, CRow]#OnTimerContext,
out: Collector[CRow]): Unit = {
+ if (isProcessingTimeTimer(ctx.asInstanceOf[OnTimerContext])) {
+ if (needToCleanupState(timestamp)) {
+
+ // we check whether there are still records which have not been processed yet
+ val noRecordsToProcess = !rowMapState.keys.iterator().hasNext
+ if (noRecordsToProcess) {
+ // we clean the state
+ cleanupState(rowMapState, accumulatorState)
+ } else {
+ // There are records left to process because a watermark has not been received yet.
+ // This would only happen if the input stream has stopped. So we don't need to clean up.
+ // We leave the state as it is and schedule a new cleanup timer
+ registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime())
+ }
+ }
+ return
+ }
+
Preconditions.checkArgument(out.isInstanceOf[TimestampedCollector[CRow]])
val collector = out.asInstanceOf[TimestampedCollector[CRow]]
@@ -178,6 +203,9 @@ abstract class RowTimeUnboundedOver(
ctx.timerService.registerEventTimeTimer(curWatermark + 1)
}
}
+
+ // update cleanup timer
+ registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime())
}
/**
@@ -221,11 +249,13 @@ abstract class RowTimeUnboundedOver(
class RowTimeUnboundedRowsOver(
genAggregations: GeneratedAggregationsFunction,
intermediateType: TypeInformation[Row],
- inputType: TypeInformation[CRow])
+ inputType: TypeInformation[CRow],
+ queryConfig: StreamQueryConfig)
extends RowTimeUnboundedOver(
genAggregations: GeneratedAggregationsFunction,
intermediateType,
- inputType) {
+ inputType,
+ queryConfig) {
override def processElementsWithSameTimestamp(
curRowList: JList[Row],
@@ -259,11 +289,13 @@ class RowTimeUnboundedRowsOver(
class RowTimeUnboundedRangeOver(
genAggregations: GeneratedAggregationsFunction,
intermediateType: TypeInformation[Row],
- inputType: TypeInformation[CRow])
+ inputType: TypeInformation[CRow],
+ queryConfig: StreamQueryConfig)
extends RowTimeUnboundedOver(
genAggregations: GeneratedAggregationsFunction,
intermediateType,
- inputType) {
+ inputType,
+ queryConfig) {
override def processElementsWithSameTimestamp(
curRowList: JList[Row],
[5/5] flink git commit: [FLINK-6483] [table] Add materialization of
time indicators.
Posted by fh...@apache.org.
[FLINK-6483] [table] Add materialization of time indicators.
This closes #3862.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/b50ef4b8
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/b50ef4b8
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/b50ef4b8
Branch: refs/heads/master
Commit: b50ef4b8de73e0e19df154d87ea588236e3ccb43
Parents: 2480887
Author: twalthr <tw...@apache.org>
Authored: Wed May 10 10:11:34 2017 +0200
Committer: Fabian Hueske <fh...@apache.org>
Committed: Fri May 12 08:09:54 2017 +0200
----------------------------------------------------------------------
.../table/api/StreamTableEnvironment.scala | 39 +-
.../calcite/RelTimeIndicatorConverter.scala | 404 +++++++++++++------
.../flink/table/codegen/CodeGenerator.scala | 47 ++-
.../flink/table/plan/nodes/CommonCalc.scala | 13 +-
.../table/plan/nodes/CommonCorrelate.scala | 19 +-
.../table/plan/nodes/dataset/DataSetCalc.scala | 6 +-
.../plan/nodes/dataset/DataSetCorrelate.scala | 8 +-
.../plan/nodes/datastream/DataStreamCalc.scala | 14 +-
.../nodes/datastream/DataStreamCorrelate.scala | 23 +-
.../datastream/StreamTableSourceScan.scala | 4 +-
.../plan/nodes/logical/FlinkLogicalCalc.scala | 2 +-
.../logical/FlinkLogicalTableSourceScan.scala | 6 +-
.../DataStreamLogicalWindowAggregateRule.scala | 14 +-
.../flink/table/plan/schema/RowSchema.scala | 11 +-
.../plan/schema/StreamTableSourceTable.scala | 8 +-
.../runtime/CRowCorrelateFlatMapRunner.scala | 83 ----
.../runtime/CRowCorrelateProcessRunner.scala | 91 +++++
.../flink/table/runtime/CRowFlatMapRunner.scala | 72 ----
.../flink/table/runtime/CRowProcessRunner.scala | 80 ++++
.../table/sources/DefinedTimeAttributes.scala | 60 ---
.../table/sources/definedTimeAttributes.scala | 60 +++
.../stream/StreamTableEnvironmentTest.scala | 10 +-
.../api/scala/stream/TableSourceTest.scala | 22 +-
.../calcite/RelTimeIndicatorConverterTest.scala | 351 ++++++++++++++++
.../datastream/TimeAttributesITCase.scala | 237 +++++++++++
.../flink/table/utils/TableTestBase.scala | 5 +
26 files changed, 1267 insertions(+), 422 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
----------------------------------------------------------------------
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 d68da04..994ac80 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
@@ -18,38 +18,37 @@
package org.apache.flink.table.api
-import _root_.java.util.concurrent.atomic.AtomicInteger
import _root_.java.lang.{Boolean => JBool}
+import _root_.java.util.concurrent.atomic.AtomicInteger
import org.apache.calcite.plan.RelOptUtil
import org.apache.calcite.plan.hep.HepMatchOrder
-import org.apache.calcite.rel.{RelNode, RelVisitor}
import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{RelNode, RelVisitor}
import org.apache.calcite.rex.{RexCall, RexInputRef, RexNode}
import org.apache.calcite.sql.SqlKind
import org.apache.calcite.sql2rel.RelDecorrelator
import org.apache.calcite.tools.{RuleSet, RuleSets}
-import org.apache.flink.api.common.typeinfo.AtomicType
-import org.apache.flink.api.common.typeutils.CompositeType
import org.apache.flink.api.common.functions.MapFunction
-import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation}
+import org.apache.flink.api.common.typeutils.CompositeType
import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
import org.apache.flink.api.java.typeutils.TupleTypeInfo
import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo
+import org.apache.flink.streaming.api.TimeCharacteristic
import org.apache.flink.streaming.api.datastream.DataStream
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
import org.apache.flink.table.calcite.RelTimeIndicatorConverter
import org.apache.flink.table.explain.PlanJsonParser
import org.apache.flink.table.expressions.{Expression, ProctimeAttribute, RowtimeAttribute, UnresolvedFieldReference}
import org.apache.flink.table.plan.nodes.FlinkConventions
-import org.apache.flink.table.plan.nodes.datastream.{DataStreamRel, UpdateAsRetractionTrait}
-import org.apache.flink.table.plan.nodes.datastream._
+import org.apache.flink.table.plan.nodes.datastream.{DataStreamRel, UpdateAsRetractionTrait, _}
import org.apache.flink.table.plan.rules.FlinkRuleSets
import org.apache.flink.table.plan.schema.{DataStreamTable, RowSchema, StreamTableSourceTable}
-import org.apache.flink.table.runtime.{CRowInputJavaTupleOutputMapRunner, CRowInputMapRunner, CRowInputScalaTupleOutputMapRunner}
import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.runtime.{CRowInputJavaTupleOutputMapRunner, CRowInputMapRunner, CRowInputScalaTupleOutputMapRunner}
import org.apache.flink.table.sinks.{AppendStreamTableSink, RetractStreamTableSink, TableSink, UpsertStreamTableSink}
-import org.apache.flink.table.sources.{StreamTableSource, TableSource}
+import org.apache.flink.table.sources.{DefinedRowtimeAttribute, StreamTableSource, TableSource}
import org.apache.flink.table.typeutils.TypeCheckUtils
import org.apache.flink.types.Row
@@ -111,6 +110,17 @@ abstract class StreamTableEnvironment(
override def registerTableSource(name: String, tableSource: TableSource[_]): Unit = {
checkValidTableName(name)
+ // check if event-time is enabled
+ tableSource match {
+ case dra: DefinedRowtimeAttribute if
+ execEnv.getStreamTimeCharacteristic != TimeCharacteristic.EventTime =>
+
+ throw TableException(
+ s"A rowtime attribute requires an EventTime time characteristic in stream environment. " +
+ s"But is: ${execEnv.getStreamTimeCharacteristic}")
+ case _ => // ok
+ }
+
tableSource match {
case streamTableSource: StreamTableSource[_] =>
registerTableInternal(name, new StreamTableSourceTable(streamTableSource))
@@ -390,6 +400,13 @@ abstract class StreamTableEnvironment(
// validate and extract time attributes
val (rowtime, proctime) = validateAndExtractTimeAttributes(streamType, fields)
+ // check if event-time is enabled
+ if (rowtime.isDefined && execEnv.getStreamTimeCharacteristic != TimeCharacteristic.EventTime) {
+ throw TableException(
+ s"A rowtime attribute requires an EventTime time characteristic in stream environment. " +
+ s"But is: ${execEnv.getStreamTimeCharacteristic}")
+ }
+
val dataStreamTable = new DataStreamTable[T](
dataStream,
fieldIndexes,
@@ -518,9 +535,9 @@ abstract class StreamTableEnvironment(
// 3. normalize the logical plan
val normRuleSet = getNormRuleSet
val normalizedPlan = if (normRuleSet.iterator().hasNext) {
- runHepPlanner(HepMatchOrder.BOTTOM_UP, normRuleSet, decorPlan, decorPlan.getTraitSet)
+ runHepPlanner(HepMatchOrder.BOTTOM_UP, normRuleSet, convPlan, convPlan.getTraitSet)
} else {
- decorPlan
+ convPlan
}
// 4. optimize the logical Flink plan
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala
index fa2e3ee..7ceb397 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala
@@ -18,73 +18,43 @@
package org.apache.flink.table.calcite
-import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFieldImpl, RelRecordType, StructKind}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core._
import org.apache.calcite.rel.logical._
-import org.apache.calcite.rel.{RelNode, RelShuttleImpl}
+import org.apache.calcite.rel.{RelNode, RelShuttle}
import org.apache.calcite.rex._
import org.apache.calcite.sql.fun.SqlStdOperatorTable
import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo
-import org.apache.flink.table.api.ValidationException
-import org.apache.flink.table.calcite.FlinkTypeFactory.isTimeIndicatorType
+import org.apache.flink.table.api.{TableException, ValidationException}
import org.apache.flink.table.functions.TimeMaterializationSqlFunction
import org.apache.flink.table.plan.schema.TimeIndicatorRelDataType
+import org.apache.flink.table.calcite.FlinkTypeFactory.isTimeIndicatorType
+import org.apache.flink.table.plan.logical.rel.LogicalWindowAggregate
import scala.collection.JavaConversions._
+import scala.collection.mutable
/**
* Traverses a [[RelNode]] tree and converts fields with [[TimeIndicatorRelDataType]] type. If a
* time attribute is accessed for a calculation, it will be materialized. Forwarding is allowed in
* some cases, but not all.
*/
-class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttleImpl {
+class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle {
- override def visit(project: LogicalProject): RelNode = {
- // visit children and update inputs
- val updatedProject = super.visit(project).asInstanceOf[LogicalProject]
+ private val timestamp = rexBuilder
+ .getTypeFactory
+ .asInstanceOf[FlinkTypeFactory]
+ .createTypeFromTypeInfo(SqlTimeTypeInfo.TIMESTAMP)
- // check if input field contains time indicator type
- // materialize field if no time indicator is present anymore
- // if input field is already materialized, change to timestamp type
- val materializer = new RexTimeIndicatorMaterializer(
- rexBuilder,
- updatedProject.getInput.getRowType.getFieldList.map(_.getType))
- val newProjects = updatedProject.getProjects.map(_.accept(materializer))
-
- // copy project
- updatedProject.copy(
- updatedProject.getTraitSet,
- updatedProject.getInput,
- newProjects,
- buildRowType(updatedProject.getRowType.getFieldNames, newProjects.map(_.getType))
- )
- }
-
- override def visit(filter: LogicalFilter): RelNode = {
- // visit children and update inputs
- val updatedFilter = super.visit(filter).asInstanceOf[LogicalFilter]
-
- // check if input field contains time indicator type
- // materialize field if no time indicator is present anymore
- // if input field is already materialized, change to timestamp type
- val materializer = new RexTimeIndicatorMaterializer(
- rexBuilder,
- updatedFilter.getInput.getRowType.getFieldList.map(_.getType))
- val newCondition = updatedFilter.getCondition.accept(materializer)
-
- // copy filter
- updatedFilter.copy(
- updatedFilter.getTraitSet,
- updatedFilter.getInput,
- newCondition
- )
- }
+ override def visit(intersect: LogicalIntersect): RelNode =
+ throw new TableException("Logical intersect in a stream environment is not supported yet.")
override def visit(union: LogicalUnion): RelNode = {
// visit children and update inputs
- val updatedUnion = super.visit(union).asInstanceOf[LogicalUnion]
+ val inputs = union.getInputs.map(_.accept(this))
// make sure that time indicator types match
- val inputTypes = updatedUnion.getInputs.map(_.getRowType)
+ val inputTypes = inputs.map(_.getRowType)
val head = inputTypes.head.getFieldList.map(_.getType)
@@ -114,101 +84,269 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttleImpl {
"Union fields with time attributes have different types.")
}
- updatedUnion
+ LogicalUnion.create(inputs, union.all)
}
+ override def visit(aggregate: LogicalAggregate): RelNode = convertAggregate(aggregate)
+
+ override def visit(minus: LogicalMinus): RelNode =
+ throw new TableException("Logical minus in a stream environment is not supported yet.")
+
+ override def visit(sort: LogicalSort): RelNode =
+ throw new TableException("Logical sort in a stream environment is not supported yet.")
+
+ override def visit(`match`: LogicalMatch): RelNode =
+ throw new TableException("Logical match in a stream environment is not supported yet.")
+
override def visit(other: RelNode): RelNode = other match {
- case scan: LogicalTableFunctionScan if
- stack.size() > 0 && stack.peek().isInstanceOf[LogicalCorrelate] =>
+
+ case uncollect: Uncollect =>
// visit children and update inputs
- val updatedScan = super.visit(scan).asInstanceOf[LogicalTableFunctionScan]
-
- val correlate = stack.peek().asInstanceOf[LogicalCorrelate]
-
- // check if input field contains time indicator type
- // materialize field if no time indicator is present anymore
- // if input field is already materialized, change to timestamp type
- val materializer = new RexTimeIndicatorMaterializer(
- rexBuilder,
- correlate.getInputs.get(0).getRowType.getFieldList.map(_.getType))
- val newCall = updatedScan.getCall.accept(materializer)
-
- // copy scan
- updatedScan.copy(
- updatedScan.getTraitSet,
- updatedScan.getInputs,
- newCall,
- updatedScan.getElementType,
- updatedScan.getRowType,
- updatedScan.getColumnMappings
- )
+ val input = uncollect.getInput.accept(this)
+ Uncollect.create(uncollect.getTraitSet, input, uncollect.withOrdinality)
+
+ case scan: LogicalTableFunctionScan =>
+ scan
+
+ case aggregate: LogicalWindowAggregate =>
+ val convAggregate = convertAggregate(aggregate)
+
+ LogicalWindowAggregate.create(
+ aggregate.getWindow,
+ aggregate.getNamedProperties,
+ convAggregate)
case _ =>
- super.visit(other)
+ throw new TableException(s"Unsupported logical operator: ${other.getClass.getSimpleName}")
}
- private def buildRowType(names: Seq[String], types: Seq[RelDataType]): RelDataType = {
- val fields = names.zipWithIndex.map { case (name, idx) =>
- new RelDataTypeFieldImpl(name, idx, types(idx))
- }
- new RelRecordType(StructKind.FULLY_QUALIFIED, fields)
+
+ override def visit(exchange: LogicalExchange): RelNode =
+ throw new TableException("Logical exchange in a stream environment is not supported yet.")
+
+ override def visit(scan: TableScan): RelNode = scan
+
+ override def visit(scan: TableFunctionScan): RelNode =
+ throw new TableException("Table function scan in a stream environment is not supported yet.")
+
+ override def visit(values: LogicalValues): RelNode = values
+
+ override def visit(filter: LogicalFilter): RelNode = {
+ // visit children and update inputs
+ val input = filter.getInput.accept(this)
+
+ // check if input field contains time indicator type
+ // materialize field if no time indicator is present anymore
+ // if input field is already materialized, change to timestamp type
+ val materializer = new RexTimeIndicatorMaterializer(
+ rexBuilder,
+ input.getRowType.getFieldList.map(_.getType))
+
+ val condition = filter.getCondition.accept(materializer)
+ LogicalFilter.create(input, condition)
}
-}
-class RexTimeIndicatorMaterializer(
- private val rexBuilder: RexBuilder,
- private val input: Seq[RelDataType])
- extends RexShuttle {
-
- val timestamp = rexBuilder
- .getTypeFactory
- .asInstanceOf[FlinkTypeFactory]
- .createTypeFromTypeInfo(SqlTimeTypeInfo.TIMESTAMP)
-
- override def visitInputRef(inputRef: RexInputRef): RexNode = {
- // reference is interesting
- if (isTimeIndicatorType(inputRef.getType)) {
- val resolvedRefType = input(inputRef.getIndex)
- // input is a valid time indicator
- if (isTimeIndicatorType(resolvedRefType)) {
- inputRef
- }
- // input has been materialized
- else {
- new RexInputRef(inputRef.getIndex, resolvedRefType)
- }
- }
- // reference is a regular field
- else {
- super.visitInputRef(inputRef)
+ override def visit(project: LogicalProject): RelNode = {
+ // visit children and update inputs
+ val input = project.getInput.accept(this)
+
+ // check if input field contains time indicator type
+ // materialize field if no time indicator is present anymore
+ // if input field is already materialized, change to timestamp type
+ val materializer = new RexTimeIndicatorMaterializer(
+ rexBuilder,
+ input.getRowType.getFieldList.map(_.getType))
+
+ val projects = project.getProjects.map(_.accept(materializer))
+ val fieldNames = project.getRowType.getFieldNames
+ LogicalProject.create(input, projects, fieldNames)
+ }
+
+ override def visit(join: LogicalJoin): RelNode =
+ throw new TableException("Logical join in a stream environment is not supported yet.")
+
+ override def visit(correlate: LogicalCorrelate): RelNode = {
+ // visit children and update inputs
+ val inputs = correlate.getInputs.map(_.accept(this))
+
+ val right = inputs(1) match {
+ case scan: LogicalTableFunctionScan =>
+ // visit children and update inputs
+ val scanInputs = scan.getInputs.map(_.accept(this))
+
+ // check if input field contains time indicator type
+ // materialize field if no time indicator is present anymore
+ // if input field is already materialized, change to timestamp type
+ val materializer = new RexTimeIndicatorMaterializer(
+ rexBuilder,
+ inputs.head.getRowType.getFieldList.map(_.getType))
+
+ val call = scan.getCall.accept(materializer)
+ LogicalTableFunctionScan.create(
+ scan.getCluster,
+ scanInputs,
+ call,
+ scan.getElementType,
+ scan.getRowType,
+ scan.getColumnMappings)
+
+ case _ =>
+ inputs(1)
}
+
+ LogicalCorrelate.create(
+ inputs.head,
+ right,
+ correlate.getCorrelationId,
+ correlate.getRequiredColumns,
+ correlate.getJoinType)
}
- override def visitCall(call: RexCall): RexNode = {
- val updatedCall = super.visitCall(call).asInstanceOf[RexCall]
+ private def convertAggregate(aggregate: Aggregate): LogicalAggregate = {
+ // visit children and update inputs
+ val input = aggregate.getInput.accept(this)
+
+ // add a project to materialize aggregation arguments/grouping keys
+
+ val refIndices = mutable.Set[Int]()
+
+ // check arguments of agg calls
+ aggregate.getAggCallList.foreach(call => if (call.getArgList.size() == 0) {
+ // count(*) has an empty argument list
+ (0 until input.getRowType.getFieldCount).foreach(refIndices.add)
+ } else {
+ // for other aggregations
+ call.getArgList.map(_.asInstanceOf[Int]).foreach(refIndices.add)
+ })
- // skip materialization for special operators
- updatedCall.getOperator match {
- case SqlStdOperatorTable.SESSION | SqlStdOperatorTable.HOP | SqlStdOperatorTable.TUMBLE =>
- return updatedCall
+ // check grouping sets
+ aggregate.getGroupSets.foreach(set =>
+ set.asList().map(_.asInstanceOf[Int]).foreach(refIndices.add)
+ )
- case _ => // do nothing
+ val needsMaterialization = refIndices.exists(idx =>
+ isTimeIndicatorType(input.getRowType.getFieldList.get(idx).getType))
+
+ // create project if necessary
+ val projectedInput = if (needsMaterialization) {
+
+ // insert or merge with input project if
+ // a time attribute is accessed and needs to be materialized
+ input match {
+
+ // merge
+ case lp: LogicalProject =>
+ val projects = lp.getProjects.zipWithIndex.map { case (expr, idx) =>
+ if (isTimeIndicatorType(expr.getType) && refIndices.contains(idx)) {
+ rexBuilder.makeCall(
+ TimeMaterializationSqlFunction,
+ expr)
+ } else {
+ expr
+ }
+ }
+
+ LogicalProject.create(
+ lp.getInput,
+ projects,
+ input.getRowType.getFieldNames)
+
+ // new project
+ case _ =>
+ val projects = input.getRowType.getFieldList.map { field =>
+ if (isTimeIndicatorType(field.getType) && refIndices.contains(field.getIndex)) {
+ rexBuilder.makeCall(
+ TimeMaterializationSqlFunction,
+ new RexInputRef(field.getIndex, field.getType))
+ } else {
+ new RexInputRef(field.getIndex, field.getType)
+ }
+ }
+
+ LogicalProject.create(
+ input,
+ projects,
+ input.getRowType.getFieldNames)
+ }
+ } else {
+ // no project necessary
+ input
}
- // materialize operands with time indicators
- val materializedOperands = updatedCall.getOperands.map { o =>
- if (isTimeIndicatorType(o.getType)) {
- rexBuilder.makeCall(TimeMaterializationSqlFunction, o)
+ // remove time indicator type as agg call return type
+ val updatedAggCalls = aggregate.getAggCallList.map { call =>
+ val callType = if (isTimeIndicatorType(call.getType)) {
+ timestamp
} else {
- o
+ call.getType
}
+ AggregateCall.create(
+ call.getAggregation,
+ call.isDistinct,
+ call.getArgList,
+ call.filterArg,
+ callType,
+ call.name)
}
- // remove time indicator return type
- if (isTimeIndicatorType(updatedCall.getType)) {
- updatedCall.clone(timestamp, materializedOperands)
- } else {
- updatedCall.clone(updatedCall.getType, materializedOperands)
+ LogicalAggregate.create(
+ projectedInput,
+ aggregate.indicator,
+ aggregate.getGroupSet,
+ aggregate.getGroupSets,
+ updatedAggCalls)
+ }
+
+ class RexTimeIndicatorMaterializer(
+ private val rexBuilder: RexBuilder,
+ private val input: Seq[RelDataType])
+ extends RexShuttle {
+
+ override def visitInputRef(inputRef: RexInputRef): RexNode = {
+ // reference is interesting
+ if (isTimeIndicatorType(inputRef.getType)) {
+ val resolvedRefType = input(inputRef.getIndex)
+ // input is a valid time indicator
+ if (isTimeIndicatorType(resolvedRefType)) {
+ inputRef
+ }
+ // input has been materialized
+ else {
+ new RexInputRef(inputRef.getIndex, resolvedRefType)
+ }
+ }
+ // reference is a regular field
+ else {
+ super.visitInputRef(inputRef)
+ }
+ }
+
+ override def visitCall(call: RexCall): RexNode = {
+ val updatedCall = super.visitCall(call).asInstanceOf[RexCall]
+
+ // materialize operands with time indicators
+ val materializedOperands = updatedCall.getOperator match {
+
+ // skip materialization for special operators
+ case SqlStdOperatorTable.SESSION | SqlStdOperatorTable.HOP | SqlStdOperatorTable.TUMBLE =>
+ updatedCall.getOperands.toList
+
+ case _ =>
+ updatedCall.getOperands.map { o =>
+ if (isTimeIndicatorType(o.getType)) {
+ rexBuilder.makeCall(TimeMaterializationSqlFunction, o)
+ } else {
+ o
+ }
+ }
+ }
+
+ // remove time indicator return type
+ if (isTimeIndicatorType(updatedCall.getType)) {
+ updatedCall.clone(timestamp, materializedOperands)
+ } else {
+ updatedCall.clone(updatedCall.getType, materializedOperands)
+ }
}
}
}
@@ -217,6 +355,30 @@ object RelTimeIndicatorConverter {
def convert(rootRel: RelNode, rexBuilder: RexBuilder): RelNode = {
val converter = new RelTimeIndicatorConverter(rexBuilder)
- rootRel.accept(converter)
+ val convertedRoot = rootRel.accept(converter)
+
+ var needsConversion = false
+
+ // materialize all remaining time indicators
+ val projects = convertedRoot.getRowType.getFieldList.map(field =>
+ if (isTimeIndicatorType(field.getType)) {
+ needsConversion = true
+ rexBuilder.makeCall(
+ TimeMaterializationSqlFunction,
+ new RexInputRef(field.getIndex, field.getType))
+ } else {
+ new RexInputRef(field.getIndex, field.getType)
+ }
+ )
+
+ // add final conversion
+ if (needsConversion) {
+ LogicalProject.create(
+ convertedRoot,
+ projects,
+ convertedRoot.getRowType.getFieldNames)
+ } else {
+ convertedRoot
+ }
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala
index 25addbc..036889f 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala
@@ -34,6 +34,7 @@ import org.apache.flink.api.common.typeutils.CompositeType
import org.apache.flink.api.java.typeutils._
import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo
import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.ProcessFunction
import org.apache.flink.table.api.TableConfig
import org.apache.flink.table.calcite.FlinkTypeFactory
import org.apache.flink.table.codegen.CodeGenUtils._
@@ -238,6 +239,11 @@ class CodeGenerator(
var outRecordTerm = "out"
/**
+ * @return term of the [[ProcessFunction]]'s context
+ */
+ var contextTerm = "ctx"
+
+ /**
* @return returns if null checking is enabled
*/
def nullCheck: Boolean = config.getNullCheck
@@ -699,6 +705,17 @@ class CodeGenerator(
List(s"$inputTypeTerm1 $input1Term = ($inputTypeTerm1) _in1;",
s"$inputTypeTerm2 $input2Term = ($inputTypeTerm2) _in2;"))
}
+
+ // ProcessFunction
+ else if (clazz == classOf[ProcessFunction[_, _]]) {
+ val baseClass = classOf[ProcessFunction[_, _]]
+ val inputTypeTerm = boxedTypeTermForTypeInfo(input1)
+ (baseClass,
+ s"void processElement(Object _in1, " +
+ s"org.apache.flink.streaming.api.functions.ProcessFunction.Context $contextTerm," +
+ s"org.apache.flink.util.Collector $collectorTerm)",
+ List(s"$inputTypeTerm $input1Term = ($inputTypeTerm) _in1;"))
+ }
else {
// TODO more functions
throw new CodeGenException("Unsupported Function.")
@@ -1312,9 +1329,11 @@ class CodeGenerator(
throw new CodeGenException("Dynamic parameter references are not supported yet.")
override def visitCall(call: RexCall): GeneratedExpression = {
- // time materialization is not implemented yet
+ // special case: time materialization
if (call.getOperator == TimeMaterializationSqlFunction) {
- throw new CodeGenException("Access to time attributes is not possible yet.")
+ return generateRecordTimestamp(
+ FlinkTypeFactory.isRowtimeIndicatorType(call.getOperands.get(0).getType)
+ )
}
val operands = call.getOperands.map(_.accept(this))
@@ -1840,6 +1859,30 @@ class CodeGenerator(
}
}
+ private[flink] def generateRecordTimestamp(isEventTime: Boolean): GeneratedExpression = {
+ val resultTerm = newName("result")
+ val resultTypeTerm = primitiveTypeTermForTypeInfo(SqlTimeTypeInfo.TIMESTAMP)
+
+ val resultCode = if (isEventTime) {
+ s"""
+ |$resultTypeTerm $resultTerm;
+ |if ($contextTerm.timestamp() == null) {
+ | throw new RuntimeException("Rowtime timestamp is null. Please make sure that a proper " +
+ | "TimestampAssigner is defined and the stream environment uses the EventTime time " +
+ | "characteristic.");
+ |}
+ |else {
+ | $resultTerm = $contextTerm.timestamp();
+ |}
+ |""".stripMargin
+ } else {
+ s"""
+ |$resultTypeTerm $resultTerm = $contextTerm.timerService().currentProcessingTime();
+ |""".stripMargin
+ }
+ GeneratedExpression(resultTerm, NEVER_NULL, resultCode, SqlTimeTypeInfo.TIMESTAMP)
+ }
+
// ----------------------------------------------------------------------------------------------
// Reusable code snippets
// ----------------------------------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala
index e875587..9b486e4 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCalc.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.nodes
import org.apache.calcite.plan.{RelOptCost, RelOptPlanner}
import org.apache.calcite.rex._
-import org.apache.flink.api.common.functions.FlatMapFunction
+import org.apache.flink.api.common.functions.Function
import org.apache.flink.table.api.TableConfig
import org.apache.flink.table.calcite.FlinkTypeFactory
import org.apache.flink.table.codegen.{CodeGenerator, GeneratedFunction}
@@ -30,16 +30,17 @@ import org.apache.flink.types.Row
import scala.collection.JavaConversions._
import scala.collection.JavaConverters._
-trait CommonCalc[T] {
+trait CommonCalc {
- private[flink] def generateFunction(
+ private[flink] def generateFunction[T <: Function](
generator: CodeGenerator,
ruleDescription: String,
inputSchema: RowSchema,
returnSchema: RowSchema,
calcProgram: RexProgram,
- config: TableConfig):
- GeneratedFunction[FlatMapFunction[Row, Row], Row] = {
+ config: TableConfig,
+ functionClass: Class[T]):
+ GeneratedFunction[T, Row] = {
val expandedExpressions = calcProgram
.getProjectList
@@ -92,7 +93,7 @@ trait CommonCalc[T] {
generator.generateFunction(
ruleDescription,
- classOf[FlatMapFunction[Row, Row]],
+ functionClass,
body,
returnSchema.physicalTypeInfo)
}
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCorrelate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCorrelate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCorrelate.scala
index c95f2f7..874bea2 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCorrelate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonCorrelate.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.nodes
import org.apache.calcite.rel.`type`.RelDataType
import org.apache.calcite.rex.{RexCall, RexInputRef, RexNode, RexShuttle}
import org.apache.calcite.sql.SemiJoinType
-import org.apache.flink.api.common.functions.FlatMapFunction
+import org.apache.flink.api.common.functions.{FlatMapFunction, Function}
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.table.api.{TableConfig, TableException}
import org.apache.flink.table.codegen.CodeGenUtils.primitiveDefaultValue
@@ -36,22 +36,22 @@ import scala.collection.JavaConverters._
/**
* Join a user-defined table function
*/
-trait CommonCorrelate[T] {
+trait CommonCorrelate {
/**
* Generates the flat map function to run the user-defined table function.
*/
- private[flink] def generateFunction(
+ private[flink] def generateFunction[T <: Function](
config: TableConfig,
inputSchema: RowSchema,
udtfTypeInfo: TypeInformation[Any],
returnSchema: RowSchema,
- rowType: RelDataType,
joinType: SemiJoinType,
rexCall: RexCall,
pojoFieldMapping: Option[Array[Int]],
- ruleDescription: String):
- GeneratedFunction[FlatMapFunction[Row, Row], Row] = {
+ ruleDescription: String,
+ functionClass: Class[T]):
+ GeneratedFunction[T, Row] = {
val functionGenerator = new CodeGenerator(
config,
@@ -89,7 +89,7 @@ trait CommonCorrelate[T] {
val outerResultExpr = functionGenerator.generateResultExpression(
input1AccessExprs ++ input2NullExprs,
returnSchema.physicalTypeInfo,
- rowType.getFieldNames.asScala)
+ returnSchema.physicalFieldNames)
body +=
s"""
|boolean hasOutput = $collectorTerm.isCollected();
@@ -104,7 +104,7 @@ trait CommonCorrelate[T] {
functionGenerator.generateFunction(
ruleDescription,
- classOf[FlatMapFunction[Row, Row]],
+ functionClass,
body,
returnSchema.physicalTypeInfo)
}
@@ -117,7 +117,6 @@ trait CommonCorrelate[T] {
inputSchema: RowSchema,
udtfTypeInfo: TypeInformation[Any],
returnSchema: RowSchema,
- rowType: RelDataType,
condition: Option[RexNode],
pojoFieldMapping: Option[Array[Int]])
: GeneratedCollector = {
@@ -135,7 +134,7 @@ trait CommonCorrelate[T] {
val crossResultExpr = generator.generateResultExpression(
input1AccessExprs ++ input2AccessExprs,
returnSchema.physicalTypeInfo,
- rowType.getFieldNames.asScala)
+ returnSchema.physicalFieldNames)
val collectorCode = if (condition.isEmpty) {
s"""
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala
index e340a8c..9a9f738 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala
@@ -24,6 +24,7 @@ import org.apache.calcite.rel.core.Calc
import org.apache.calcite.rel.metadata.RelMetadataQuery
import org.apache.calcite.rel.{RelNode, RelWriter}
import org.apache.calcite.rex._
+import org.apache.flink.api.common.functions.FlatMapFunction
import org.apache.flink.api.java.DataSet
import org.apache.flink.api.java.typeutils.RowTypeInfo
import org.apache.flink.table.api.BatchTableEnvironment
@@ -46,7 +47,7 @@ class DataSetCalc(
calcProgram: RexProgram,
ruleDescription: String)
extends Calc(cluster, traitSet, input, calcProgram)
- with CommonCalc[Row]
+ with CommonCalc
with DataSetRel {
override def deriveRowType(): RelDataType = rowRelDataType
@@ -95,7 +96,8 @@ class DataSetCalc(
new RowSchema(getInput.getRowType),
new RowSchema(getRowType),
calcProgram,
- config)
+ config,
+ classOf[FlatMapFunction[Row, Row]])
val runner = new FlatMapRunner(genFunction.name, genFunction.code, returnType)
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCorrelate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCorrelate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCorrelate.scala
index 49ead26..731d2e5 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCorrelate.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCorrelate.scala
@@ -23,6 +23,7 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery
import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
import org.apache.calcite.rex.{RexCall, RexNode}
import org.apache.calcite.sql.SemiJoinType
+import org.apache.flink.api.common.functions.FlatMapFunction
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.api.java.DataSet
import org.apache.flink.api.java.typeutils.RowTypeInfo
@@ -49,7 +50,7 @@ class DataSetCorrelate(
joinType: SemiJoinType,
ruleDescription: String)
extends SingleRel(cluster, traitSet, inputNode)
- with CommonCorrelate[Row]
+ with CommonCorrelate
with DataSetRel {
override def deriveRowType() = relRowType
@@ -109,18 +110,17 @@ class DataSetCorrelate(
new RowSchema(getInput.getRowType),
udtfTypeInfo,
new RowSchema(getRowType),
- rowType,
joinType,
rexCall,
pojoFieldMapping,
- ruleDescription)
+ ruleDescription,
+ classOf[FlatMapFunction[Row, Row]])
val collector = generateCollector(
config,
new RowSchema(getInput.getRowType),
udtfTypeInfo,
new RowSchema(getRowType),
- rowType,
condition,
pojoFieldMapping)
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala
----------------------------------------------------------------------
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 5f270f6..f75efc8 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,11 +25,12 @@ 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.streaming.api.functions.ProcessFunction
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
-import org.apache.flink.table.runtime.CRowFlatMapRunner
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.runtime.CRowProcessRunner
import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
/**
@@ -45,7 +46,7 @@ class DataStreamCalc(
calcProgram: RexProgram,
ruleDescription: String)
extends Calc(cluster, traitSet, input, calcProgram)
- with CommonCalc[CRow]
+ with CommonCalc
with DataStreamRel {
override def deriveRowType(): RelDataType = schema.logicalType
@@ -101,17 +102,18 @@ class DataStreamCalc(
inputSchema,
schema,
calcProgram,
- config)
+ config,
+ classOf[ProcessFunction[CRow, CRow]])
val inputParallelism = inputDataStream.getParallelism
- val mapFunc = new CRowFlatMapRunner(
+ val processFunc = new CRowProcessRunner(
genFunction.name,
genFunction.code,
CRowTypeInfo(schema.physicalTypeInfo))
inputDataStream
- .flatMap(mapFunc)
+ .process(processFunc)
.name(calcOpName(calcProgram, getExpressionString))
// keep parallelism to ensure order of accumulate and retract messages
.setParallelism(inputParallelism)
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala
----------------------------------------------------------------------
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 5b32b10..b7165cd 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,12 +23,13 @@ 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.streaming.api.functions.ProcessFunction
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
import org.apache.flink.table.plan.schema.RowSchema
-import org.apache.flink.table.runtime.CRowCorrelateFlatMapRunner
+import org.apache.flink.table.runtime.CRowCorrelateProcessRunner
import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
/**
@@ -46,7 +47,7 @@ class DataStreamCorrelate(
joinType: SemiJoinType,
ruleDescription: String)
extends SingleRel(cluster, traitSet, input)
- with CommonCorrelate[CRow]
+ with CommonCorrelate
with DataStreamRel {
override def deriveRowType() = schema.logicalType
@@ -90,7 +91,6 @@ class DataStreamCorrelate(
// we do not need to specify input type
val inputDS = getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
- val inputType = inputDS.getType.asInstanceOf[CRowTypeInfo]
val funcRel = scan.asInstanceOf[FlinkLogicalTableFunctionScan]
val rexCall = funcRel.getCall.asInstanceOf[RexCall]
@@ -98,37 +98,36 @@ class DataStreamCorrelate(
val pojoFieldMapping = Some(sqlFunction.getPojoFieldMapping)
val udtfTypeInfo = sqlFunction.getRowTypeInfo.asInstanceOf[TypeInformation[Any]]
- val flatMap = generateFunction(
+ val process = generateFunction(
config,
inputSchema,
udtfTypeInfo,
schema,
- getRowType,
joinType,
rexCall,
pojoFieldMapping,
- ruleDescription)
+ ruleDescription,
+ classOf[ProcessFunction[CRow, CRow]])
val collector = generateCollector(
config,
inputSchema,
udtfTypeInfo,
schema,
- getRowType,
condition,
pojoFieldMapping)
- val mapFunc = new CRowCorrelateFlatMapRunner(
- flatMap.name,
- flatMap.code,
+ val processFunc = new CRowCorrelateProcessRunner(
+ process.name,
+ process.code,
collector.name,
collector.code,
- CRowTypeInfo(flatMap.returnType))
+ CRowTypeInfo(process.returnType))
val inputParallelism = inputDS.getParallelism
inputDS
- .flatMap(mapFunc)
+ .process(processFunc)
// preserve input parallelism to ensure that acc and retract messages remain in order
.setParallelism(inputParallelism)
.name(correlateOpName(rexCall, sqlFunction, schema.logicalType))
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala
----------------------------------------------------------------------
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 51e609f..72ecac5 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
@@ -49,7 +49,7 @@ class StreamTableSourceScan(
val fieldCnt = fieldNames.length
val rowtime = tableSource match {
- case timeSource: DefinedRowTimeAttribute if timeSource.getRowtimeAttribute != null =>
+ case timeSource: DefinedRowtimeAttribute if timeSource.getRowtimeAttribute != null =>
val rowtimeAttribute = timeSource.getRowtimeAttribute
Some((fieldCnt, rowtimeAttribute))
case _ =>
@@ -57,7 +57,7 @@ class StreamTableSourceScan(
}
val proctime = tableSource match {
- case timeSource: DefinedProcTimeAttribute if timeSource.getProctimeAttribute != null =>
+ case timeSource: DefinedProctimeAttribute if timeSource.getProctimeAttribute != null =>
val proctimeAttribute = timeSource.getProctimeAttribute
Some((fieldCnt + (if (rowtime.isDefined) 1 else 0), proctimeAttribute))
case _ =>
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalCalc.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalCalc.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalCalc.scala
index 0ca079e..ec90392 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalCalc.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalCalc.scala
@@ -34,7 +34,7 @@ class FlinkLogicalCalc(
calcProgram: RexProgram)
extends Calc(cluster, traitSet, input, calcProgram)
with FlinkLogicalRel
- with CommonCalc[Any] {
+ with CommonCalc {
override def copy(traitSet: RelTraitSet, child: RelNode, program: RexProgram): Calc = {
new FlinkLogicalCalc(cluster, traitSet, child, program)
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala
index a2777ec..3ae949e 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala
@@ -29,7 +29,7 @@ import org.apache.flink.table.api.TableEnvironment
import org.apache.flink.table.calcite.FlinkTypeFactory
import org.apache.flink.table.plan.nodes.FlinkConventions
import org.apache.flink.table.plan.schema.TableSourceTable
-import org.apache.flink.table.sources.{DefinedProcTimeAttribute, DefinedRowTimeAttribute, TableSource}
+import org.apache.flink.table.sources.{DefinedProctimeAttribute, DefinedRowtimeAttribute, TableSource}
import scala.collection.JavaConverters._
@@ -54,7 +54,7 @@ class FlinkLogicalTableSourceScan(
val fieldCnt = fieldNames.length
val rowtime = tableSource match {
- case timeSource: DefinedRowTimeAttribute if timeSource.getRowtimeAttribute != null =>
+ case timeSource: DefinedRowtimeAttribute if timeSource.getRowtimeAttribute != null =>
val rowtimeAttribute = timeSource.getRowtimeAttribute
Some((fieldCnt, rowtimeAttribute))
case _ =>
@@ -62,7 +62,7 @@ class FlinkLogicalTableSourceScan(
}
val proctime = tableSource match {
- case timeSource: DefinedProcTimeAttribute if timeSource.getProctimeAttribute != null =>
+ case timeSource: DefinedProctimeAttribute if timeSource.getProctimeAttribute != null =>
val proctimeAttribute = timeSource.getProctimeAttribute
Some((fieldCnt + (if (rowtime.isDefined) 1 else 0), proctimeAttribute))
case _ =>
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamLogicalWindowAggregateRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamLogicalWindowAggregateRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamLogicalWindowAggregateRule.scala
index 28efcf5..d57d4cc 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamLogicalWindowAggregateRule.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamLogicalWindowAggregateRule.scala
@@ -26,7 +26,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable
import org.apache.flink.table.api.{TableException, Window}
import org.apache.flink.table.api.scala.{Session, Slide, Tumble}
import org.apache.flink.table.calcite.FlinkTypeFactory
-import org.apache.flink.table.expressions.{Literal, UnresolvedFieldReference}
+import org.apache.flink.table.expressions.{Literal, ResolvedFieldReference, UnresolvedFieldReference}
import org.apache.flink.table.plan.rules.common.LogicalWindowAggregateRule
import org.apache.flink.table.typeutils.TimeIntervalTypeInfo
@@ -68,10 +68,12 @@ class DataStreamLogicalWindowAggregateRule
case _ => throw new TableException("Only constant window descriptors are supported.")
}
- def getOperandAsTimeIndicator(call: RexCall, idx: Int): String =
+ def getOperandAsTimeIndicator(call: RexCall, idx: Int): ResolvedFieldReference =
call.getOperands.get(idx) match {
case v: RexInputRef if FlinkTypeFactory.isTimeIndicatorType(v.getType) =>
- rowType.getFieldList.get(v.getIndex).getName
+ ResolvedFieldReference(
+ rowType.getFieldList.get(v.getIndex).getName,
+ FlinkTypeFactory.toTypeInfo(v.getType))
case _ =>
throw new TableException("Window can only be defined over a time attribute column.")
}
@@ -82,7 +84,7 @@ class DataStreamLogicalWindowAggregateRule
val interval = getOperandAsLong(windowExpr, 1)
val w = Tumble.over(Literal(interval, TimeIntervalTypeInfo.INTERVAL_MILLIS))
- w.on(UnresolvedFieldReference(time)).as("w$")
+ w.on(time).as("w$")
case SqlStdOperatorTable.HOP =>
val time = getOperandAsTimeIndicator(windowExpr, 0)
@@ -91,14 +93,14 @@ class DataStreamLogicalWindowAggregateRule
.over(Literal(size, TimeIntervalTypeInfo.INTERVAL_MILLIS))
.every(Literal(slide, TimeIntervalTypeInfo.INTERVAL_MILLIS))
- w.on(UnresolvedFieldReference(time)).as("w$")
+ w.on(time).as("w$")
case SqlStdOperatorTable.SESSION =>
val time = getOperandAsTimeIndicator(windowExpr, 0)
val gap = getOperandAsLong(windowExpr, 1)
val w = Session.withGap(Literal(gap, TimeIntervalTypeInfo.INTERVAL_MILLIS))
- w.on(UnresolvedFieldReference(time)).as("w$")
+ w.on(time).as("w$")
}
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/RowSchema.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/RowSchema.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/RowSchema.scala
index b42be82..ccbe44d 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/RowSchema.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/RowSchema.scala
@@ -20,11 +20,12 @@ package org.apache.flink.table.plan.schema
import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeField, RelRecordType}
import org.apache.calcite.rel.core.AggregateCall
-import org.apache.calcite.rex.{RexInputRef, RexNode, RexShuttle}
+import org.apache.calcite.rex.{RexCall, RexInputRef, RexNode, RexShuttle}
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.api.java.typeutils.RowTypeInfo
import org.apache.flink.table.api.TableException
import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.functions.TimeMaterializationSqlFunction
import org.apache.flink.types.Row
import scala.collection.JavaConversions._
@@ -76,6 +77,14 @@ class RowSchema(private val logicalRowType: RelDataType) {
override def visitInputRef(inputRef: RexInputRef): RexNode = {
new RexInputRef(mapIndex(inputRef.getIndex), inputRef.getType)
}
+
+ override def visitCall(call: RexCall): RexNode = call.getOperator match {
+ // we leave time indicators unchanged yet
+ // the index becomes invalid but right now we are only
+ // interested in the type of the input reference
+ case TimeMaterializationSqlFunction => call
+ case _ => super.visitCall(call)
+ }
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/StreamTableSourceTable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/StreamTableSourceTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/StreamTableSourceTable.scala
index 75deca5..fa15288 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/StreamTableSourceTable.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/StreamTableSourceTable.scala
@@ -19,10 +19,10 @@
package org.apache.flink.table.plan.schema
import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory}
-import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.api.{TableEnvironment, TableException}
import org.apache.flink.table.calcite.FlinkTypeFactory
import org.apache.flink.table.plan.stats.FlinkStatistic
-import org.apache.flink.table.sources.{DefinedProcTimeAttribute, DefinedRowTimeAttribute, TableSource}
+import org.apache.flink.table.sources.{DefinedProctimeAttribute, DefinedRowtimeAttribute, TableSource}
class StreamTableSourceTable[T](
override val tableSource: TableSource[T],
@@ -39,7 +39,7 @@ class StreamTableSourceTable[T](
val fieldCnt = fieldNames.length
val rowtime = tableSource match {
- case timeSource: DefinedRowTimeAttribute if timeSource.getRowtimeAttribute != null =>
+ case timeSource: DefinedRowtimeAttribute if timeSource.getRowtimeAttribute != null =>
val rowtimeAttribute = timeSource.getRowtimeAttribute
Some((fieldCnt, rowtimeAttribute))
case _ =>
@@ -47,7 +47,7 @@ class StreamTableSourceTable[T](
}
val proctime = tableSource match {
- case timeSource: DefinedProcTimeAttribute if timeSource.getProctimeAttribute != null =>
+ case timeSource: DefinedProctimeAttribute if timeSource.getProctimeAttribute != null =>
val proctimeAttribute = timeSource.getProctimeAttribute
Some((fieldCnt + (if (rowtime.isDefined) 1 else 0), proctimeAttribute))
case _ =>
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateFlatMapRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateFlatMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateFlatMapRunner.scala
deleted file mode 100644
index ff3821a..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateFlatMapRunner.scala
+++ /dev/null
@@ -1,83 +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
-
-import org.apache.flink.api.common.functions.{FlatMapFunction, RichFlatMapFunction}
-import org.apache.flink.api.common.functions.util.FunctionUtils
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.table.codegen.Compiler
-import org.apache.flink.table.runtime.types.CRow
-import org.apache.flink.types.Row
-import org.apache.flink.util.Collector
-import org.slf4j.{Logger, LoggerFactory}
-
-/**
- * A CorrelateFlatMapRunner with [[CRow]] input and [[CRow]] output.
- */
-class CRowCorrelateFlatMapRunner(
- flatMapName: String,
- flatMapCode: String,
- collectorName: String,
- collectorCode: String,
- @transient var returnType: TypeInformation[CRow])
- extends RichFlatMapFunction[CRow, CRow]
- with ResultTypeQueryable[CRow]
- with Compiler[Any] {
-
- val LOG: Logger = LoggerFactory.getLogger(this.getClass)
-
- private var function: FlatMapFunction[Row, Row] = _
- private var collector: TableFunctionCollector[_] = _
- private var cRowWrapper: CRowWrappingCollector = _
-
- override def open(parameters: Configuration): Unit = {
- LOG.debug(s"Compiling TableFunctionCollector: $collectorName \n\n Code:\n$collectorCode")
- val clazz = compile(getRuntimeContext.getUserCodeClassLoader, collectorName, collectorCode)
- LOG.debug("Instantiating TableFunctionCollector.")
- collector = clazz.newInstance().asInstanceOf[TableFunctionCollector[_]]
- this.cRowWrapper = new CRowWrappingCollector()
-
- LOG.debug(s"Compiling FlatMapFunction: $flatMapName \n\n Code:\n$flatMapCode")
- val flatMapClazz = compile(getRuntimeContext.getUserCodeClassLoader, flatMapName, flatMapCode)
- val constructor = flatMapClazz.getConstructor(classOf[TableFunctionCollector[_]])
- LOG.debug("Instantiating FlatMapFunction.")
- function = constructor.newInstance(collector).asInstanceOf[FlatMapFunction[Row, Row]]
- FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext)
- FunctionUtils.openFunction(function, parameters)
- }
-
- override def flatMap(in: CRow, out: Collector[CRow]): Unit = {
- cRowWrapper.out = out
- cRowWrapper.setChange(in.change)
-
- collector.setCollector(cRowWrapper)
- collector.setInput(in.row)
- collector.reset()
-
- function.flatMap(in.row, cRowWrapper)
- }
-
- override def getProducedType: TypeInformation[CRow] = returnType
-
- override def close(): Unit = {
- FunctionUtils.closeFunction(function)
- }
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateProcessRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateProcessRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateProcessRunner.scala
new file mode 100644
index 0000000..4f0a785
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateProcessRunner.scala
@@ -0,0 +1,91 @@
+/*
+ * 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
+
+import org.apache.flink.api.common.functions.util.FunctionUtils
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.{Logger, LoggerFactory}
+
+/**
+ * A CorrelateProcessRunner with [[CRow]] input and [[CRow]] output.
+ */
+class CRowCorrelateProcessRunner(
+ processName: String,
+ processCode: String,
+ collectorName: String,
+ collectorCode: String,
+ @transient var returnType: TypeInformation[CRow])
+ extends ProcessFunction[CRow, CRow]
+ with ResultTypeQueryable[CRow]
+ with Compiler[Any] {
+
+ val LOG: Logger = LoggerFactory.getLogger(this.getClass)
+
+ private var function: ProcessFunction[Row, Row] = _
+ private var collector: TableFunctionCollector[_] = _
+ private var cRowWrapper: CRowWrappingCollector = _
+
+ override def open(parameters: Configuration): Unit = {
+ LOG.debug(s"Compiling TableFunctionCollector: $collectorName \n\n Code:\n$collectorCode")
+ val clazz = compile(getRuntimeContext.getUserCodeClassLoader, collectorName, collectorCode)
+ LOG.debug("Instantiating TableFunctionCollector.")
+ collector = clazz.newInstance().asInstanceOf[TableFunctionCollector[_]]
+ this.cRowWrapper = new CRowWrappingCollector()
+
+ LOG.debug(s"Compiling ProcessFunction: $processName \n\n Code:\n$processCode")
+ val processClazz = compile(getRuntimeContext.getUserCodeClassLoader, processName, processCode)
+ val constructor = processClazz.getConstructor(classOf[TableFunctionCollector[_]])
+ LOG.debug("Instantiating ProcessFunction.")
+ function = constructor.newInstance(collector).asInstanceOf[ProcessFunction[Row, Row]]
+ FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext)
+ FunctionUtils.openFunction(function, parameters)
+ }
+
+ override def processElement(
+ in: CRow,
+ ctx: ProcessFunction[CRow, CRow]#Context,
+ out: Collector[CRow])
+ : Unit = {
+
+ cRowWrapper.out = out
+ cRowWrapper.setChange(in.change)
+
+ collector.setCollector(cRowWrapper)
+ collector.setInput(in.row)
+ collector.reset()
+
+ function.processElement(
+ in.row,
+ ctx.asInstanceOf[ProcessFunction[Row, Row]#Context],
+ cRowWrapper)
+ }
+
+ override def getProducedType: TypeInformation[CRow] = returnType
+
+ override def close(): Unit = {
+ FunctionUtils.closeFunction(function)
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowFlatMapRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowFlatMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowFlatMapRunner.scala
deleted file mode 100644
index 9701cb9..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowFlatMapRunner.scala
+++ /dev/null
@@ -1,72 +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
-
-import org.apache.flink.api.common.functions.util.FunctionUtils
-import org.apache.flink.api.common.functions.{FlatMapFunction, RichFlatMapFunction}
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.table.codegen.Compiler
-import org.apache.flink.table.runtime.types.CRow
-import org.apache.flink.types.Row
-import org.apache.flink.util.Collector
-import org.slf4j.LoggerFactory
-
-/**
- * FlatMapRunner with [[CRow]] input and [[CRow]] output.
- */
-class CRowFlatMapRunner(
- name: String,
- code: String,
- @transient var returnType: TypeInformation[CRow])
- extends RichFlatMapFunction[CRow, CRow]
- with ResultTypeQueryable[CRow]
- with Compiler[FlatMapFunction[Row, Row]] {
-
- val LOG = LoggerFactory.getLogger(this.getClass)
-
- private var function: FlatMapFunction[Row, Row] = _
- private var cRowWrapper: CRowWrappingCollector = _
-
- override def open(parameters: Configuration): Unit = {
- LOG.debug(s"Compiling FlatMapFunction: $name \n\n Code:\n$code")
- val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
- LOG.debug("Instantiating FlatMapFunction.")
- function = clazz.newInstance()
- FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext)
- FunctionUtils.openFunction(function, parameters)
-
- this.cRowWrapper = new CRowWrappingCollector()
- }
-
- override def flatMap(in: CRow, out: Collector[CRow]): Unit = {
- cRowWrapper.out = out
- cRowWrapper.setChange(in.change)
- function.flatMap(in.row, cRowWrapper)
- }
-
- override def getProducedType: TypeInformation[CRow] = returnType
-
- override def close(): Unit = {
- FunctionUtils.closeFunction(function)
- }
-}
-
-
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowProcessRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowProcessRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowProcessRunner.scala
new file mode 100644
index 0000000..cef62a5
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowProcessRunner.scala
@@ -0,0 +1,80 @@
+/*
+ * 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
+
+import org.apache.flink.api.common.functions.util.FunctionUtils
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.ProcessFunction
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.table.runtime.types.CRow
+import org.apache.flink.types.Row
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+/**
+ * ProcessRunner with [[CRow]] input and [[CRow]] output.
+ */
+class CRowProcessRunner(
+ name: String,
+ code: String,
+ @transient var returnType: TypeInformation[CRow])
+ extends ProcessFunction[CRow, CRow]
+ with ResultTypeQueryable[CRow]
+ with Compiler[ProcessFunction[Row, Row]] {
+
+ val LOG = LoggerFactory.getLogger(this.getClass)
+
+ private var function: ProcessFunction[Row, Row] = _
+ private var cRowWrapper: CRowWrappingCollector = _
+
+ override def open(parameters: Configuration): Unit = {
+ LOG.debug(s"Compiling ProcessFunction: $name \n\n Code:\n$code")
+ val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
+ LOG.debug("Instantiating ProcessFunction.")
+ function = clazz.newInstance()
+ FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext)
+ FunctionUtils.openFunction(function, parameters)
+
+ this.cRowWrapper = new CRowWrappingCollector()
+ }
+
+ override def processElement(
+ in: CRow,
+ ctx: ProcessFunction[CRow, CRow]#Context,
+ out: Collector[CRow])
+ : Unit = {
+
+ cRowWrapper.out = out
+ cRowWrapper.setChange(in.change)
+ function.processElement(
+ in.row,
+ ctx.asInstanceOf[ProcessFunction[Row, Row]#Context],
+ cRowWrapper)
+ }
+
+ override def getProducedType: TypeInformation[CRow] = returnType
+
+ override def close(): Unit = {
+ FunctionUtils.closeFunction(function)
+ }
+}
+
+
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/DefinedTimeAttributes.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/DefinedTimeAttributes.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/DefinedTimeAttributes.scala
deleted file mode 100644
index 6d87663..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/DefinedTimeAttributes.scala
+++ /dev/null
@@ -1,60 +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.sources
-
-/**
- * Defines a logical event-time attribute for a [[TableSource]].
- * The event-time attribute can be used for indicating, accessing, and working with Flink's
- * event-time.
- *
- * A [[TableSource]] that implements this interface defines the name of
- * the event-time attribute. The attribute will be added to the schema of the
- * [[org.apache.flink.table.api.Table]] produced by the [[TableSource]].
- */
-trait DefinedRowTimeAttribute {
-
- /**
- * Defines a name of the event-time attribute that represents Flink's
- * event-time. Null if no rowtime should be available.
- *
- * The field will be appended to the schema provided by the [[TableSource]].
- */
- def getRowtimeAttribute: String
-}
-
-/**
- * Defines a logical processing-time attribute for a [[TableSource]].
- * The processing-time attribute can be used for indicating, accessing, and working with Flink's
- * processing-time.
- *
- * A [[TableSource]] that implements this interface defines the name of
- * the processing-time attribute. The attribute will be added to the schema of the
- * [[org.apache.flink.table.api.Table]] produced by the [[TableSource]].
- */
-trait DefinedProcTimeAttribute {
-
- /**
- * Defines a name of the processing-time attribute that represents Flink's
- * processing-time. Null if no rowtime should be available.
- *
- * The field will be appended to the schema provided by the [[TableSource]].
- */
- def getProctimeAttribute: String
-
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/definedTimeAttributes.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/definedTimeAttributes.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/definedTimeAttributes.scala
new file mode 100644
index 0000000..d381115
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/definedTimeAttributes.scala
@@ -0,0 +1,60 @@
+/*
+ * 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.sources
+
+/**
+ * Defines a logical event-time attribute for a [[TableSource]].
+ * The event-time attribute can be used for indicating, accessing, and working with Flink's
+ * event-time.
+ *
+ * A [[TableSource]] that implements this interface defines the name of
+ * the event-time attribute. The attribute will be added to the schema of the
+ * [[org.apache.flink.table.api.Table]] produced by the [[TableSource]].
+ */
+trait DefinedRowtimeAttribute {
+
+ /**
+ * Defines a name of the event-time attribute that represents Flink's
+ * event-time. Null if no rowtime should be available.
+ *
+ * The field will be appended to the schema provided by the [[TableSource]].
+ */
+ def getRowtimeAttribute: String
+}
+
+/**
+ * Defines a logical processing-time attribute for a [[TableSource]].
+ * The processing-time attribute can be used for indicating, accessing, and working with Flink's
+ * processing-time.
+ *
+ * A [[TableSource]] that implements this interface defines the name of
+ * the processing-time attribute. The attribute will be added to the schema of the
+ * [[org.apache.flink.table.api.Table]] produced by the [[TableSource]].
+ */
+trait DefinedProctimeAttribute {
+
+ /**
+ * Defines a name of the processing-time attribute that represents Flink's
+ * processing-time. Null if no rowtime should be available.
+ *
+ * The field will be appended to the schema provided by the [[TableSource]].
+ */
+ def getProctimeAttribute: String
+
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/StreamTableEnvironmentTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/StreamTableEnvironmentTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/StreamTableEnvironmentTest.scala
index e9384c7..7797f22 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/StreamTableEnvironmentTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/StreamTableEnvironmentTest.scala
@@ -19,18 +19,16 @@
package org.apache.flink.table.api.scala.stream
import java.lang.{Integer => JInt, Long => JLong}
-import java.util.Collections
-import java.util.{List => JList}
import org.apache.flink.api.java.tuple.{Tuple5 => JTuple5}
import org.apache.flink.api.java.typeutils.TupleTypeInfo
import org.apache.flink.api.scala._
+import org.apache.flink.streaming.api.TimeCharacteristic
import org.apache.flink.streaming.api.datastream.DataStream
import org.apache.flink.streaming.api.environment.{StreamExecutionEnvironment => JStreamExecEnv}
-import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
-import org.apache.flink.table.api.{TableEnvironment, TableException, Types}
import org.apache.flink.table.api.java.{StreamTableEnvironment => JStreamTableEnv}
import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.{TableEnvironment, TableException, Types}
import org.apache.flink.table.utils.TableTestBase
import org.junit.Test
import org.mockito.Mockito.{mock, when}
@@ -151,7 +149,9 @@ class StreamTableEnvironmentTest extends TableTestBase {
private def prepareSchemaExpressionParser:
(JStreamTableEnv, DataStream[JTuple5[JLong, JInt, String, JInt, JLong]]) = {
- val jTEnv = TableEnvironment.getTableEnvironment(mock(classOf[JStreamExecEnv]))
+ val jStreamExecEnv = mock(classOf[JStreamExecEnv])
+ when(jStreamExecEnv.getStreamTimeCharacteristic).thenReturn(TimeCharacteristic.EventTime)
+ val jTEnv = TableEnvironment.getTableEnvironment(jStreamExecEnv)
val sType = new TupleTypeInfo(Types.LONG, Types.INT, Types.STRING, Types.INT, Types.LONG)
.asInstanceOf[TupleTypeInfo[JTuple5[JLong, JInt, String, JInt, JLong]]]
http://git-wip-us.apache.org/repos/asf/flink/blob/b50ef4b8/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceTest.scala
index 18066c9..cda90f7 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceTest.scala
@@ -24,7 +24,7 @@ import org.apache.flink.streaming.api.datastream.DataStream
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
import org.apache.flink.table.api.Types
import org.apache.flink.table.api.scala._
-import org.apache.flink.table.sources.{DefinedProcTimeAttribute, DefinedRowTimeAttribute, StreamTableSource}
+import org.apache.flink.table.sources.{DefinedProctimeAttribute, DefinedRowtimeAttribute, StreamTableSource}
import org.apache.flink.table.utils.TableTestBase
import org.apache.flink.table.utils.TableTestUtil.{term, unaryNode}
import org.apache.flink.types.Row
@@ -35,7 +35,7 @@ class TableSourceTest extends TableTestBase {
@Test
def testRowTimeTableSourceSimple(): Unit = {
val util = streamTestUtil()
- util.tEnv.registerTableSource("rowTimeT", new TestRowTimeSource("addTime"))
+ util.tEnv.registerTableSource("rowTimeT", new TestRowtimeSource("addTime"))
val t = util.tEnv.scan("rowTimeT").select("addTime, id, name, val")
@@ -43,7 +43,7 @@ class TableSourceTest extends TableTestBase {
unaryNode(
"DataStreamCalc",
"StreamTableSourceScan(table=[[rowTimeT]], fields=[id, val, name, addTime])",
- term("select", "addTime", "id", "name", "val")
+ term("select", "TIME_MATERIALIZATION(addTime) AS addTime", "id", "name", "val")
)
util.verifyTable(t, expected)
}
@@ -51,7 +51,7 @@ class TableSourceTest extends TableTestBase {
@Test
def testRowTimeTableSourceGroupWindow(): Unit = {
val util = streamTestUtil()
- util.tEnv.registerTableSource("rowTimeT", new TestRowTimeSource("addTime"))
+ util.tEnv.registerTableSource("rowTimeT", new TestRowtimeSource("addTime"))
val t = util.tEnv.scan("rowTimeT")
.filter("val > 100")
@@ -82,7 +82,7 @@ class TableSourceTest extends TableTestBase {
@Test
def testProcTimeTableSourceSimple(): Unit = {
val util = streamTestUtil()
- util.tEnv.registerTableSource("procTimeT", new TestProcTimeSource("pTime"))
+ util.tEnv.registerTableSource("procTimeT", new TestProctimeSource("pTime"))
val t = util.tEnv.scan("procTimeT").select("pTime, id, name, val")
@@ -90,7 +90,7 @@ class TableSourceTest extends TableTestBase {
unaryNode(
"DataStreamCalc",
"StreamTableSourceScan(table=[[procTimeT]], fields=[id, val, name, pTime])",
- term("select", "pTime", "id", "name", "val")
+ term("select", "TIME_MATERIALIZATION(pTime) AS pTime", "id", "name", "val")
)
util.verifyTable(t, expected)
}
@@ -98,7 +98,7 @@ class TableSourceTest extends TableTestBase {
@Test
def testProcTimeTableSourceOverWindow(): Unit = {
val util = streamTestUtil()
- util.tEnv.registerTableSource("procTimeT", new TestProcTimeSource("pTime"))
+ util.tEnv.registerTableSource("procTimeT", new TestProctimeSource("pTime"))
val t = util.tEnv.scan("procTimeT")
.window(Over partitionBy 'id orderBy 'pTime preceding 2.hours as 'w)
@@ -123,8 +123,8 @@ class TableSourceTest extends TableTestBase {
}
}
-class TestRowTimeSource(timeField: String)
- extends StreamTableSource[Row] with DefinedRowTimeAttribute {
+class TestRowtimeSource(timeField: String)
+ extends StreamTableSource[Row] with DefinedRowtimeAttribute {
override def getDataStream(execEnv: StreamExecutionEnvironment): DataStream[Row] = ???
@@ -137,8 +137,8 @@ class TestRowTimeSource(timeField: String)
}
}
-class TestProcTimeSource(timeField: String)
- extends StreamTableSource[Row] with DefinedProcTimeAttribute {
+class TestProctimeSource(timeField: String)
+ extends StreamTableSource[Row] with DefinedProctimeAttribute {
override def getDataStream(execEnv: StreamExecutionEnvironment): DataStream[Row] = ???
[2/5] flink git commit: [FLINK-6491] [table] Add QueryConfig and
state clean up for over-windowed aggregates.
Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupAggregationsITCase.scala
----------------------------------------------------------------------
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 910cbf2..9da2c44 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 queryConfig = new StreamQueryConfig()
+ queryConfig.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](queryConfig)
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](queryConfig)
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](queryConfig)
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](queryConfig)
results.addSink(new RetractingSink)
env.execute()
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/BoundedProcessingOverRangeProcessFunctionTest.scala
----------------------------------------------------------------------
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 eadcfc8..0000000
--- 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]
- }
-}
-
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala
----------------------------------------------------------------------
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 eb5acd5b..77798f9 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,294 @@ package org.apache.flink.table.runtime.harness
import java.util.{Comparator, Queue => JQueue}
+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.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 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<org.apache
+ | .flink.table.functions.aggfunctions.SumWithRetractAccumulator> accIt0 =
+ | new org.apache.flink.table.runtime.aggregate.SingleElementIterable<org.apache.flink
+ | .table
+ | .functions.aggfunctions.SumWithRetractAccumulator>();
+ |
+ | 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],
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala
----------------------------------------------------------------------
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 0000000..04214f9
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.harness
+
+import java.lang.{Integer => JInt, Long => JLong}
+import java.util.concurrent.ConcurrentLinkedQueue
+
+import org.apache.flink.api.common.time.Time
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+import org.apache.flink.streaming.api.operators.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
+import org.apache.flink.types.Row
+import org.junit.Test
+
+class NonWindowHarnessTest extends HarnessTestBase {
+
+ protected var queryConfig =
+ new StreamQueryConfig().withIdleStateRetentionTime(Time.seconds(2), Time.seconds(3))
+
+ @Test
+ def testProcTimeNonWindow(): Unit = {
+
+ val processFunction = new KeyedProcessOperator[String, CRow, CRow](
+ new GroupAggProcessFunction(
+ genSumAggFunction,
+ sumAggregationStateType,
+ false,
+ queryConfig))
+
+ 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))
+ // 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))
+
+ // 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))
+ 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, 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))
+
+ testHarness.close()
+ }
+
+ @Test
+ def testProcTimeNonWindowWithRetract(): Unit = {
+
+ val processFunction = new KeyedProcessOperator[String, CRow, CRow](
+ new GroupAggProcessFunction(
+ genSumAggFunction,
+ sumAggregationStateType,
+ true,
+ queryConfig))
+
+ 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()
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala
----------------------------------------------------------------------
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 56ca85c..786a843 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,34 @@
package org.apache.flink.table.runtime.harness
import java.lang.{Integer => JInt, Long => JLong}
-import java.util.concurrent.ConcurrentLinkedQueue
+import java.util.concurrent.{ConcurrentLinkedQueue}
-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)
-
+ protected var queryConfig =
+ new StreamQueryConfig().withIdleStateRetentionTime(Time.seconds(2), Time.seconds(3))
@Test
def testProcTimeBoundedRowsOver(): Unit = {
val processFunction = new KeyedProcessOperator[String, CRow, CRow](
new ProcTimeBoundedRowsOver(
- genAggFunction,
+ genMinMaxAggFunction,
2,
- aggregationStateType,
- cRT))
+ minMaxAggregationStateType,
+ minMaxCRowType,
+ queryConfig))
val testHarness =
createHarnessTester(processFunction,new TupleRowKeySelector[Integer](0),BasicTypeInfo
@@ -199,6 +53,7 @@ class OverWindowHarnessTest extends HarnessTestBase{
testHarness.open()
+ // register cleanup timer with 3001
testHarness.setProcessingTime(1)
testHarness.processElement(new StreamRecord(
@@ -209,6 +64,9 @@ 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))
+
+ // 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))
testHarness.processElement(new StreamRecord(
@@ -220,15 +78,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)
+ // 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))
+
+ // 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))
+ 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))
@@ -274,10 +136,10 @@ 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, 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 +154,11 @@ class OverWindowHarnessTest extends HarnessTestBase{
val processFunction = new KeyedProcessOperator[String, CRow, CRow](
new ProcTimeBoundedRangeOver(
- genAggFunction,
- 1000,
- aggregationStateType,
- cRT))
+ genMinMaxAggFunction,
+ 4000,
+ minMaxAggregationStateType,
+ minMaxCRowType,
+ queryConfig))
val testHarness =
createHarnessTester(
@@ -305,6 +168,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 +178,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 +190,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)
+ // 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 +201,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)
+ // 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 +217,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 +232,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 +277,9 @@ class OverWindowHarnessTest extends HarnessTestBase{
val processFunction = new KeyedProcessOperator[String, CRow, CRow](
new ProcTimeUnboundedPartitionedOver(
- genAggFunction,
- aggregationStateType))
+ genMinMaxAggFunction,
+ minMaxAggregationStateType,
+ queryConfig))
val testHarness =
createHarnessTester(
@@ -420,6 +289,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 +310,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 +357,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()
@@ -510,10 +383,11 @@ class OverWindowHarnessTest extends HarnessTestBase{
val processFunction = new KeyedProcessOperator[String, CRow, CRow](
new RowTimeBoundedRangeOver(
- genAggFunction,
- aggregationStateType,
- cRT,
- 4000))
+ genMinMaxAggFunction,
+ minMaxAggregationStateType,
+ minMaxCRowType,
+ 4000,
+ new StreamQueryConfig().withIdleStateRetentionTime(Time.seconds(1), Time.seconds(2))))
val testHarness =
createHarnessTester(
@@ -573,6 +447,40 @@ class OverWindowHarnessTest extends HarnessTestBase{
testHarness.processWatermark(19000)
+ // test cleanup
+ testHarness.setProcessingTime(1000)
+ testHarness.processWatermark(20000)
+
+ // check that state is removed after max retention time
+ testHarness.processElement(new StreamRecord(
+ CRow(Row.of(1: JInt, 0L: JLong, 0: JInt, "ccc", 1L: JLong), true), 20001)) // clean-up 3000
+ testHarness.setProcessingTime(2500)
+ testHarness.processElement(new StreamRecord(
+ CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "ccc", 2L: JLong), true), 20002)) // clean-up 4500
+ testHarness.processWatermark(20010) // compute output
+
+ assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+ testHarness.setProcessingTime(4499)
+ assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+ testHarness.setProcessingTime(4500)
+ assert(testHarness.numKeyedStateEntries() == 0) // check that all state is gone
+
+ // check that state is only removed if all data was processed
+ testHarness.processElement(new StreamRecord(
+ CRow(Row.of(3: JInt, 0L: JLong, 0: JInt, "ccc", 3L: JLong), true), 20011)) // clean-up 6500
+
+ assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+ testHarness.setProcessingTime(6500) // clean-up attempt but rescheduled to 8500
+ assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+
+ testHarness.processWatermark(20020) // schedule emission
+
+ assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+ testHarness.setProcessingTime(8499) // clean-up
+ assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+ testHarness.setProcessingTime(8500) // clean-up
+ assert(testHarness.numKeyedStateEntries() == 0) // check that all state is gone
+
val result = testHarness.getOutput
val expectedOutput = new ConcurrentLinkedQueue[Object]()
@@ -621,6 +529,16 @@ class OverWindowHarnessTest extends HarnessTestBase{
CRow(
Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 12001))
+ expectedOutput.add(new StreamRecord(
+ CRow(
+ Row.of(1: JInt, 0L: JLong, 0: JInt, "ccc", 1L: JLong, 1L: JLong, 1L: JLong), true), 20001))
+ expectedOutput.add(new StreamRecord(
+ CRow(
+ Row.of(2: JInt, 0L: JLong, 0: JInt, "ccc", 2L: JLong, 1L: JLong, 2L: JLong), true), 20002))
+ expectedOutput.add(new StreamRecord(
+ CRow(
+ Row.of(3: JInt, 0L: JLong, 0: JInt, "ccc", 3L: JLong, 3L: JLong, 3L: JLong), true), 20011))
+
verify(expectedOutput, result, new RowResultSortComparator(6))
testHarness.close()
}
@@ -630,10 +548,11 @@ class OverWindowHarnessTest extends HarnessTestBase{
val processFunction = new KeyedProcessOperator[String, CRow, CRow](
new RowTimeBoundedRowsOver(
- genAggFunction,
- aggregationStateType,
- cRT,
- 3))
+ genMinMaxAggFunction,
+ minMaxAggregationStateType,
+ minMaxCRowType,
+ 3,
+ new StreamQueryConfig().withIdleStateRetentionTime(Time.seconds(1), Time.seconds(2))))
val testHarness =
createHarnessTester(
@@ -689,6 +608,41 @@ class OverWindowHarnessTest extends HarnessTestBase{
testHarness.processWatermark(19000)
+ // test cleanup
+ testHarness.setProcessingTime(1000)
+ testHarness.processWatermark(20000)
+
+ // check that state is removed after max retention time
+ testHarness.processElement(new StreamRecord(
+ CRow(Row.of(1: JInt, 0L: JLong, 0: JInt, "ccc", 1L: JLong), true), 20001)) // clean-up 3000
+ testHarness.setProcessingTime(2500)
+ testHarness.processElement(new StreamRecord(
+ CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "ccc", 2L: JLong), true), 20002)) // clean-up 4500
+ testHarness.processWatermark(20010) // compute output
+
+ assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+ testHarness.setProcessingTime(4499)
+ assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+ testHarness.setProcessingTime(4500)
+ assert(testHarness.numKeyedStateEntries() == 0) // check that all state is gone
+
+ // check that state is only removed if all data was processed
+ testHarness.processElement(new StreamRecord(
+ CRow(Row.of(3: JInt, 0L: JLong, 0: JInt, "ccc", 3L: JLong), true), 20011)) // clean-up 6500
+
+ assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+ testHarness.setProcessingTime(6500) // clean-up attempt but rescheduled to 8500
+ assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+
+ testHarness.processWatermark(20020) // schedule emission
+
+ assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+ testHarness.setProcessingTime(8499) // clean-up
+ assert(testHarness.numKeyedStateEntries() > 0) // check that we have state
+ testHarness.setProcessingTime(8500) // clean-up
+ assert(testHarness.numKeyedStateEntries() == 0) // check that all state is gone
+
+
val result = testHarness.getOutput
val expectedOutput = new ConcurrentLinkedQueue[Object]()
@@ -736,6 +690,16 @@ class OverWindowHarnessTest extends HarnessTestBase{
CRow(
Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 20L: JLong, 40L: JLong), true), 12001))
+ expectedOutput.add(new StreamRecord(
+ CRow(
+ Row.of(1: JInt, 0L: JLong, 0: JInt, "ccc", 1L: JLong, 1L: JLong, 1L: JLong), true), 20001))
+ expectedOutput.add(new StreamRecord(
+ CRow(
+ Row.of(2: JInt, 0L: JLong, 0: JInt, "ccc", 2L: JLong, 1L: JLong, 2L: JLong), true), 20002))
+ expectedOutput.add(new StreamRecord(
+ CRow(
+ Row.of(3: JInt, 0L: JLong, 0: JInt, "ccc", 3L: JLong, 3L: JLong, 3L: JLong), true), 20011))
+
verify(expectedOutput, result, new RowResultSortComparator(6))
testHarness.close()
}
@@ -748,9 +712,10 @@ class OverWindowHarnessTest extends HarnessTestBase{
val processFunction = new KeyedProcessOperator[String, CRow, CRow](
new RowTimeUnboundedRangeOver(
- genAggFunction,
- aggregationStateType,
- cRT))
+ genMinMaxAggFunction,
+ minMaxAggregationStateType,
+ minMaxCRowType,
+ new StreamQueryConfig().withIdleStateRetentionTime(Time.seconds(1), Time.seconds(2))))
val testHarness =
createHarnessTester(
@@ -760,6 +725,7 @@ class OverWindowHarnessTest extends HarnessTestBase{
testHarness.open()
+ testHarness.setProcessingTime(1000)
testHarness.processWatermark(800)
testHarness.processElement(new StreamRecord(
CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 801))
@@ -806,6 +772,30 @@ class OverWindowHarnessTest extends HarnessTestBase{
testHarness.processWatermark(19000)
+ // test cleanup
+ assert(testHarness.numKeyedStateEntries() > 0)
+ testHarness.setProcessingTime(2999) // clean up timer is 3000, so nothing should happen
+ assert(testHarness.numKeyedStateEntries() > 0)
+ testHarness.setProcessingTime(3000) // clean up is triggered
+ assert(testHarness.numKeyedStateEntries() == 0)
+
+ testHarness.processWatermark(20000)
+ testHarness.processElement(new StreamRecord(
+ CRow(Row.of(1: JInt, 0L: JLong, 0: JInt, "ccc", 1L: JLong), true), 20001)) // clean-up 5000
+ testHarness.setProcessingTime(2500)
+ testHarness.processElement(new StreamRecord(
+ CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "ccc", 2L: JLong), true), 20002)) // clean-up 5000
+
+ assert(testHarness.numKeyedStateEntries() > 0)
+ testHarness.setProcessingTime(5000) // does not clean up, because data left. New timer 7000
+ testHarness.processWatermark(20010) // compute output
+
+ assert(testHarness.numKeyedStateEntries() > 0)
+ testHarness.setProcessingTime(6999) // clean up timer is 3000, so nothing should happen
+ assert(testHarness.numKeyedStateEntries() > 0)
+ testHarness.setProcessingTime(7000) // clean up is triggered
+ assert(testHarness.numKeyedStateEntries() == 0)
+
val result = testHarness.getOutput
val expectedOutput = new ConcurrentLinkedQueue[Object]()
@@ -854,6 +844,13 @@ class OverWindowHarnessTest extends HarnessTestBase{
CRow(
Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 10L: JLong, 40L: JLong), true), 12001))
+ expectedOutput.add(new StreamRecord(
+ CRow(
+ Row.of(1: JInt, 0L: JLong, 0: JInt, "ccc", 1L: JLong, 1L: JLong, 1L: JLong), true), 20001))
+ expectedOutput.add(new StreamRecord(
+ CRow(
+ Row.of(2: JInt, 0L: JLong, 0: JInt, "ccc", 2L: JLong, 1L: JLong, 2L: JLong), true), 20002))
+
verify(expectedOutput, result, new RowResultSortComparator(6))
testHarness.close()
}
@@ -863,9 +860,10 @@ class OverWindowHarnessTest extends HarnessTestBase{
val processFunction = new KeyedProcessOperator[String, CRow, CRow](
new RowTimeUnboundedRowsOver(
- genAggFunction,
- aggregationStateType,
- cRT))
+ genMinMaxAggFunction,
+ minMaxAggregationStateType,
+ minMaxCRowType,
+ new StreamQueryConfig().withIdleStateRetentionTime(Time.seconds(1), Time.seconds(2))))
val testHarness =
createHarnessTester(
@@ -875,6 +873,7 @@ class OverWindowHarnessTest extends HarnessTestBase{
testHarness.open()
+ testHarness.setProcessingTime(1000)
testHarness.processWatermark(800)
testHarness.processElement(new StreamRecord(
CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 801))
@@ -921,6 +920,30 @@ class OverWindowHarnessTest extends HarnessTestBase{
testHarness.processWatermark(19000)
+ // test cleanup
+ assert(testHarness.numKeyedStateEntries() > 0)
+ testHarness.setProcessingTime(2999) // clean up timer is 3000, so nothing should happen
+ assert(testHarness.numKeyedStateEntries() > 0)
+ testHarness.setProcessingTime(3000) // clean up is triggered
+ assert(testHarness.numKeyedStateEntries() == 0)
+
+ testHarness.processWatermark(20000)
+ testHarness.processElement(new StreamRecord(
+ CRow(Row.of(1: JInt, 0L: JLong, 0: JInt, "ccc", 1L: JLong), true), 20001)) // clean-up 5000
+ testHarness.setProcessingTime(2500)
+ testHarness.processElement(new StreamRecord(
+ CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "ccc", 2L: JLong), true), 20002)) // clean-up 5000
+
+ assert(testHarness.numKeyedStateEntries() > 0)
+ testHarness.setProcessingTime(5000) // does not clean up, because data left. New timer 7000
+ testHarness.processWatermark(20010) // compute output
+
+ assert(testHarness.numKeyedStateEntries() > 0)
+ testHarness.setProcessingTime(6999) // clean up timer is 3000, so nothing should happen
+ assert(testHarness.numKeyedStateEntries() > 0)
+ testHarness.setProcessingTime(7000) // clean up is triggered
+ assert(testHarness.numKeyedStateEntries() == 0)
+
val result = testHarness.getOutput
val expectedOutput = new ConcurrentLinkedQueue[Object]()
@@ -968,6 +991,13 @@ class OverWindowHarnessTest extends HarnessTestBase{
CRow(
Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 10L: JLong, 40L: JLong), true), 12001))
+ expectedOutput.add(new StreamRecord(
+ CRow(
+ Row.of(1: JInt, 0L: JLong, 0: JInt, "ccc", 1L: JLong, 1L: JLong, 1L: JLong), true), 20001))
+ expectedOutput.add(new StreamRecord(
+ CRow(
+ Row.of(2: JInt, 0L: JLong, 0: JInt, "ccc", 2L: JLong, 1L: JLong, 2L: JLong), true), 20002))
+
verify(expectedOutput, result, new RowResultSortComparator(6))
testHarness.close()
}
http://git-wip-us.apache.org/repos/asf/flink/blob/24808871/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala
----------------------------------------------------------------------
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 3d79e22..c4e2433 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 = ???