From 03ca69505e14aea5452a1b6e77d942ecc2440de4 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Fri, 4 Aug 2017 02:20:56 +0200 Subject: [PATCH 1/7] [FLINK-7337] [table] Refactor internal handling of time indicator attributes. - Expand phyiscal Row schema for time indicators. - Refactor computation of logical schema of tables to import. - Refactor operators to use time attribute in Row instead of StreamRecord timestamp. --- .../table/api/StreamTableEnvironment.scala | 160 ++++- .../table/calcite/FlinkTypeFactory.scala | 37 +- .../calcite/RelTimeIndicatorConverter.scala | 67 +- .../flink/table/codegen/CodeGenerator.scala | 66 +- .../table/codegen/calls/ScalarOperators.scala | 37 +- ... ProcTimeMaterializationSqlFunction.scala} | 8 +- .../table/plan/nodes/CommonCorrelate.scala | 4 +- .../plan/nodes/PhysicalTableSourceScan.scala | 4 +- .../nodes/dataset/BatchTableSourceScan.scala | 4 +- .../nodes/datastream/DataStreamCalc.scala | 4 - .../datastream/DataStreamGroupAggregate.scala | 12 +- .../DataStreamGroupWindowAggregate.scala | 46 +- .../datastream/DataStreamOverAggregate.scala | 111 ++-- .../plan/nodes/datastream/StreamScan.scala | 32 +- .../datastream/StreamTableSourceScan.scala | 23 +- .../logical/FlinkLogicalTableSourceScan.scala | 23 +- .../table/plan/schema/DataStreamTable.scala | 13 - .../flink/table/plan/schema/FlinkTable.scala | 10 +- .../flink/table/plan/schema/RowSchema.scala | 76 +-- .../plan/schema/StreamTableSourceTable.scala | 48 +- .../CRowInputTupleOutputMapRunner.scala | 37 ++ ...er.scala => CRowOutputProcessRunner.scala} | 34 +- .../TimestampSetterProcessFunction.scala | 52 ++ .../runtime/aggregate/AggregateUtil.scala | 48 +- ...tSessionWindowAggReduceGroupFunction.scala | 5 +- ...SetSlideWindowAggReduceGroupFunction.scala | 5 +- ...mbleTimeWindowAggReduceGroupFunction.scala | 2 +- ...mentalAggregateAllTimeWindowFunction.scala | 15 +- ...crementalAggregateTimeWindowFunction.scala | 19 +- .../aggregate/ProcTimeBoundedRangeOver.scala | 3 + .../ProcTimeSortProcessFunction.scala | 7 +- .../aggregate/RowTimeBoundedRangeOver.scala | 8 +- .../aggregate/RowTimeBoundedRowsOver.scala | 8 +- .../RowTimeSortProcessFunction.scala | 19 +- .../aggregate/RowTimeUnboundedOver.scala | 19 +- .../table/runtime/aggregate/SortUtil.scala | 3 + .../TimeWindowPropertyCollector.scala | 23 +- .../table/runtime/join/WindowJoinUtil.scala | 5 +- .../flink/table/api/stream/sql/SortTest.scala | 4 +- .../api/stream/table/TableSourceTest.scala | 4 +- .../plan/TimeIndicatorConversionTest.scala | 29 +- .../runtime/harness/HarnessTestBase.scala | 22 +- .../runtime/harness/JoinHarnessTest.scala | 4 +- .../harness/NonWindowHarnessTest.scala | 4 +- .../harness/OverWindowHarnessTest.scala | 586 ++++++++---------- .../SortProcessFunctionHarnessTest.scala | 129 ++-- .../stream/table/TableSinkITCase.scala | 140 ++++- 47 files changed, 1108 insertions(+), 911 deletions(-) rename flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/{TimeMaterializationSqlFunction.scala => ProcTimeMaterializationSqlFunction.scala} (85%) rename flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/{CRowOutputMapRunner.scala => CRowOutputProcessRunner.scala} (65%) create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/TimestampSetterProcessFunction.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 7328b2adc5f4f..6b73f4ecc16d7 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 @@ -23,10 +23,8 @@ 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.`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.rel.`type`.{RelDataType, RelDataTypeField, RelDataTypeFieldImpl, RelRecordType} +import org.apache.calcite.rel.RelNode import org.apache.calcite.sql2rel.RelDecorrelator import org.apache.calcite.tools.{RuleSet, RuleSets} import org.apache.flink.api.common.functions.MapFunction @@ -38,16 +36,16 @@ 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.calcite.{FlinkTypeFactory, RelTimeIndicatorConverter} import org.apache.flink.table.explain.PlanJsonParser import org.apache.flink.table.expressions._ 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.{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.plan.util.UpdatingPlanChecker import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} -import org.apache.flink.table.runtime.{CRowInputJavaTupleOutputMapRunner, CRowInputMapRunner, CRowInputScalaTupleOutputMapRunner} +import org.apache.flink.table.runtime.{CRowInputJavaTupleOutputMapRunner, CRowInputMapRunner, CRowInputScalaTupleOutputMapRunner, WrappingTimestampSetterProcessFunction} import org.apache.flink.table.sinks.{AppendStreamTableSink, RetractStreamTableSink, TableSink, UpsertStreamTableSink} import org.apache.flink.table.sources.{DefinedRowtimeAttribute, StreamTableSource, TableSource} import org.apache.flink.table.typeutils.TypeCheckUtils @@ -270,11 +268,11 @@ abstract class StreamTableEnvironment( * valid Java class identifier. */ private def getConversionMapperWithChanges[OUT]( - physicalTypeInfo: TypeInformation[CRow], - schema: RowSchema, - requestedTypeInfo: TypeInformation[OUT], - functionName: String): - MapFunction[CRow, OUT] = { + physicalTypeInfo: TypeInformation[CRow], + schema: RowSchema, + requestedTypeInfo: TypeInformation[OUT], + functionName: String): + MapFunction[CRow, OUT] = { requestedTypeInfo match { @@ -356,9 +354,7 @@ abstract class StreamTableEnvironment( val dataStreamTable = new DataStreamTable[T]( dataStream, fieldIndexes, - fieldNames, - None, - None + fieldNames ) registerTableInternal(name, dataStreamTable) } @@ -393,12 +389,14 @@ abstract class StreamTableEnvironment( s"But is: ${execEnv.getStreamTimeCharacteristic}") } + // adjust field indexes and field names + val indexesWithIndicatorFields = adjustFieldIndexes(fieldIndexes, rowtime, proctime) + val namesWithIndicatorFields = adjustFieldNames(fieldNames, rowtime, proctime) + val dataStreamTable = new DataStreamTable[T]( dataStream, - fieldIndexes, - fieldNames, - rowtime, - proctime + indexesWithIndicatorFields, + namesWithIndicatorFields ) registerTableInternal(name, dataStreamTable) } @@ -501,6 +499,68 @@ abstract class StreamTableEnvironment( (rowtime, proctime) } + /** + * Injects markers for time indicator fields into the field indexes. + * A rowtime indicator is represented as -1, a proctime indicator as -2. + * + * @param fieldIndexes The field indexes into which the time indicators markers are injected. + * @param rowtime An optional rowtime indicator + * @param proctime An optional proctime indicator + * @return An adjusted array of field indexes. + */ + private def adjustFieldIndexes( + fieldIndexes: Array[Int], + rowtime: Option[(Int, String)], + proctime: Option[(Int, String)]): Array[Int] = { + + // inject rowtime field + val withRowtime = if (rowtime.isDefined) { + fieldIndexes.patch(rowtime.get._1, Seq(-1), 0) // -1 indicates rowtime + } else { + fieldIndexes + } + + // inject proctime field + val withProctime = if (proctime.isDefined) { + withRowtime.patch(proctime.get._1, Seq(-2), 0) // -2 indicates proctime + } else { + withRowtime + } + + withProctime + } + + /** + * Injects names of time indicator fields into the list of field names. + * + * @param fieldNames The array of field names into which the time indicator field names are + * injected. + * @param rowtime An optional rowtime indicator + * @param proctime An optional proctime indicator + * @return An adjusted array of field names. + */ + private def adjustFieldNames( + fieldNames: Array[String], + rowtime: Option[(Int, String)], + proctime: Option[(Int, String)]): Array[String] = { + + // inject rowtime field + val withRowtime = if (rowtime.isDefined) { + fieldNames.patch(rowtime.get._1, Seq(rowtime.get._2), 0) + } else { + fieldNames + } + + // inject proctime field + val withProctime = if (proctime.isDefined) { + withRowtime.patch(proctime.get._1, Seq(proctime.get._2), 0) + } else { + withRowtime + } + + withProctime + } + /** * Returns the decoration rule set for this environment * including a custom RuleSet configuration. @@ -632,10 +692,21 @@ abstract class StreamTableEnvironment( val relNode = table.getRelNode val dataStreamPlan = optimize(relNode, updatesAsRetraction) - // we convert the logical row type to the output row type - val convertedOutputType = RelTimeIndicatorConverter.convertOutputType(relNode) - - translate(dataStreamPlan, convertedOutputType, queryConfig, withChangeFlag) + // zip original field names with optimized field types + val fieldTypes = relNode.getRowType.getFieldList.asScala + .zip(dataStreamPlan.getRowType.getFieldList.asScala) + // get name of original plan and type of optimized plan + .map(x => (x._1.getName, x._2.getType)) + // add field indexes + .zipWithIndex + // build new field types + .map(x => new RelDataTypeFieldImpl(x._1._1, x._2, x._1._2)) + + // build a record type from list of field types + val rowType = new RelRecordType( + fieldTypes.toList.asInstanceOf[List[RelDataTypeField]].asJava) + + translate(dataStreamPlan, rowType, queryConfig, withChangeFlag) } /** @@ -684,12 +755,43 @@ abstract class StreamTableEnvironment( val rootParallelism = plan.getParallelism - conversion match { - case mapFunction: MapFunction[CRow, A] => - plan.map(mapFunction) - .returns(tpe) - .name(s"to: ${tpe.getTypeClass.getSimpleName}") - .setParallelism(rootParallelism) + val rowtimeFields = logicalType.getFieldList.asScala + .filter(f => FlinkTypeFactory.isRowtimeIndicatorType(f.getType)) + + if (rowtimeFields.isEmpty) { + // to rowtime field to set + + conversion match { + case mapFunction: MapFunction[CRow, A] => + plan.map(mapFunction) + .returns(tpe) + .name(s"to: ${tpe.getTypeClass.getSimpleName}") + .setParallelism(rootParallelism) + } + } else if (rowtimeFields.size == 1) { + // set the only rowtime field as event-time timestamp for DataStream + + val mapFunction = conversion match { + case mapFunction: MapFunction[CRow, A] => mapFunction + case _ => new MapFunction[CRow, A] { + override def map(cRow: CRow): A = cRow.asInstanceOf[A] + } + } + + plan.process( + new WrappingTimestampSetterProcessFunction[A]( + mapFunction, + rowtimeFields.head.getIndex)) + .returns(tpe) + .name(s"to: ${tpe.getTypeClass.getSimpleName}") + .setParallelism(rootParallelism) + + } else { + throw new TableException( + s"Found more than one rowtime field: [${rowtimeFields.map(_.getName).mkString(", ")}] in " + + s"the table that should be converted to a DataStream.\n" + + s"Please select the rowtime field that should be used as event-time timestamp for the " + + s"DataStream by casting all other fields to TIMESTAMP or LONG.") } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala index dbefe203e9601..637e8cc0fba19 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala @@ -172,45 +172,20 @@ class FlinkTypeFactory(typeSystem: RelDataTypeSystem) extends JavaTypeFactoryImp * * @param fieldNames field names * @param fieldTypes field types, every element is Flink's [[TypeInformation]] - * @param rowtime optional system field to indicate event-time; the index determines the index - * in the final record. If the index is smaller than the number of specified - * fields, it shifts all following fields. - * @param proctime optional system field to indicate processing-time; the index determines the - * index in the final record. If the index is smaller than the number of - * specified fields, it shifts all following fields. * @return a struct type with the input fieldNames, input fieldTypes, and system fields */ def buildLogicalRowType( fieldNames: Seq[String], - fieldTypes: Seq[TypeInformation[_]], - rowtime: Option[(Int, String)], - proctime: Option[(Int, String)]) + fieldTypes: Seq[TypeInformation[_]]) : RelDataType = { val logicalRowTypeBuilder = builder val fields = fieldNames.zip(fieldTypes) - - var totalNumberOfFields = fields.length - if (rowtime.isDefined) { - totalNumberOfFields += 1 - } - if (proctime.isDefined) { - totalNumberOfFields += 1 - } - - var addedTimeAttributes = 0 - for (i <- 0 until totalNumberOfFields) { - if (rowtime.isDefined && rowtime.get._1 == i) { - logicalRowTypeBuilder.add(rowtime.get._2, createRowtimeIndicatorType()) - addedTimeAttributes += 1 - } else if (proctime.isDefined && proctime.get._1 == i) { - logicalRowTypeBuilder.add(proctime.get._2, createProctimeIndicatorType()) - addedTimeAttributes += 1 - } else { - val field = fields(i - addedTimeAttributes) - logicalRowTypeBuilder.add(field._1, createTypeFromTypeInfo(field._2, isNullable = true)) - } - } + fields.foreach(f => { + // time indicators are not nullable + val nullable = !FlinkTypeFactory.isTimeIndicatorType(f._2) + logicalRowTypeBuilder.add(f._1, createTypeFromTypeInfo(f._2, nullable)) + }) logicalRowTypeBuilder.build } 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 eb1429158a73d..f5319ec422a3b 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,7 +18,7 @@ package org.apache.flink.table.calcite -import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFieldImpl, RelRecordType} +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, RelShuttle} @@ -26,8 +26,8 @@ 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.{TableException, ValidationException} -import org.apache.flink.table.calcite.FlinkTypeFactory.isTimeIndicatorType -import org.apache.flink.table.functions.TimeMaterializationSqlFunction +import org.apache.flink.table.calcite.FlinkTypeFactory.{isRowtimeIndicatorType, _} +import org.apache.flink.table.functions.ProcTimeMaterializationSqlFunction import org.apache.flink.table.plan.logical.rel.LogicalWindowAggregate import org.apache.flink.table.plan.schema.TimeIndicatorRelDataType @@ -242,9 +242,13 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle { case lp: LogicalProject => val projects = lp.getProjects.zipWithIndex.map { case (expr, idx) => if (isTimeIndicatorType(expr.getType) && refIndices.contains(idx)) { - rexBuilder.makeCall( - TimeMaterializationSqlFunction, - expr) + if (isRowtimeIndicatorType(expr.getType)) { + // cast rowtime indicator to regular timestamp + rexBuilder.makeAbstractCast(timestamp, expr) + } else { + // generate proctime access + rexBuilder.makeCall(ProcTimeMaterializationSqlFunction, expr) + } } else { expr } @@ -259,9 +263,17 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle { 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)) + if (isRowtimeIndicatorType(field.getType)) { + // cast rowtime indicator to regular timestamp + rexBuilder.makeAbstractCast( + timestamp, + new RexInputRef(field.getIndex, field.getType)) + } else { + // generate proctime access + rexBuilder.makeCall( + ProcTimeMaterializationSqlFunction, + new RexInputRef(field.getIndex, field.getType)) + } } else { new RexInputRef(field.getIndex, field.getType) } @@ -311,19 +323,19 @@ object RelTimeIndicatorConverter { var needsConversion = false - // materialize all remaining time indicators + // materialize remaining proc time indicators val projects = convertedRoot.getRowType.getFieldList.map(field => - if (isTimeIndicatorType(field.getType)) { + if (isProctimeIndicatorType(field.getType)) { needsConversion = true rexBuilder.makeCall( - TimeMaterializationSqlFunction, + ProcTimeMaterializationSqlFunction, new RexInputRef(field.getIndex, field.getType)) } else { new RexInputRef(field.getIndex, field.getType) } ) - // add final conversion + // add final conversion if necessary if (needsConversion) { LogicalProject.create( convertedRoot, @@ -334,27 +346,6 @@ object RelTimeIndicatorConverter { } } - def convertOutputType(rootRel: RelNode): RelDataType = { - - val timestamp = rootRel - .getCluster - .getRexBuilder - .getTypeFactory - .asInstanceOf[FlinkTypeFactory] - .createTypeFromTypeInfo(SqlTimeTypeInfo.TIMESTAMP, isNullable = false) - - // convert all time indicators types to timestamps - val fields = rootRel.getRowType.getFieldList.map { field => - if (isTimeIndicatorType(field.getType)) { - new RelDataTypeFieldImpl(field.getName, field.getIndex, timestamp) - } else { - field - } - } - - new RelRecordType(fields) - } - /** * Materializes time indicator accesses in an expression. * @@ -415,7 +406,13 @@ class RexTimeIndicatorMaterializer( case _ => updatedCall.getOperands.map { o => if (isTimeIndicatorType(o.getType)) { - rexBuilder.makeCall(TimeMaterializationSqlFunction, o) + if (isRowtimeIndicatorType(o.getType)) { + // cast rowtime indicator to regular timestamp + rexBuilder.makeAbstractCast(timestamp, o) + } else { + // generate proctime access + rexBuilder.makeCall(ProcTimeMaterializationSqlFunction, o) + } } else { o } 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 63fd058b980cb..8a7ea75893aa2 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 @@ -36,11 +36,12 @@ import org.apache.flink.table.api.TableConfig import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.codegen.CodeGenUtils._ import org.apache.flink.table.codegen.GeneratedExpression.{NEVER_NULL, NO_CODE} +import org.apache.flink.table.codegen.calls.FunctionGenerator import org.apache.flink.table.codegen.calls.ScalarOperators._ -import org.apache.flink.table.codegen.calls.{BuiltInMethods, FunctionGenerator} import org.apache.flink.table.functions.sql.ScalarSqlFunctions import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils -import org.apache.flink.table.functions.{FunctionContext, TimeMaterializationSqlFunction, UserDefinedFunction} +import org.apache.flink.table.functions.{FunctionContext, ProcTimeMaterializationSqlFunction, UserDefinedFunction} +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo import org.apache.flink.table.typeutils.TypeCheckUtils._ import scala.collection.JavaConversions._ @@ -244,14 +245,18 @@ abstract class CodeGenerator( returnType: TypeInformation[_ <: Any], resultFieldNames: Seq[String]) : GeneratedExpression = { - val input1AccessExprs = input1Mapping.map { idx => - generateInputAccess(input1, input1Term, idx) + val input1AccessExprs = input1Mapping.map { + case -1 => generateStreamRecordTimestampAcccess() + case -2 => generateNullLiteral(TimeIndicatorTypeInfo.PROCTIME_INDICATOR) + case idx => generateInputAccess(input1, input1Term, idx) } val input2AccessExprs = input2 match { case Some(ti) => - input2Mapping.map { idx => - generateInputAccess(ti, input2Term, idx) + input2Mapping.map { + case -1 => generateNullLiteral(TimeIndicatorTypeInfo.ROWTIME_INDICATOR) + case -2 => generateNullLiteral(TimeIndicatorTypeInfo.PROCTIME_INDICATOR) + case idx => generateInputAccess(ti, input2Term, idx) }.toSeq case None => Seq() // add nothing @@ -722,10 +727,8 @@ abstract class CodeGenerator( override def visitCall(call: RexCall): GeneratedExpression = { // special case: time materialization - if (call.getOperator == TimeMaterializationSqlFunction) { - return generateRecordTimestamp( - FlinkTypeFactory.isRowtimeIndicatorType(call.getOperands.get(0).getType) - ) + if (call.getOperator == ProcTimeMaterializationSqlFunction) { + return generateProcTimestamp() } val resultType = FlinkTypeFactory.toTypeInfo(call.getType) @@ -965,10 +968,10 @@ abstract class CodeGenerator( generateArrayElement(this, array) case ScalarSqlFunctions.CONCAT => - generateConcat(BuiltInMethods.CONCAT, operands) + generateConcat(operands) case ScalarSqlFunctions.CONCAT_WS => - generateConcat(BuiltInMethods.CONCAT_WS, operands) + generateConcatWs(operands) // advanced scalar functions case sqlOperator: SqlOperator => @@ -1125,6 +1128,19 @@ abstract class CodeGenerator( } } + private[flink] def generateStreamRecordTimestampAcccess(): GeneratedExpression = { + val resultTerm = newName("result") + val nullTerm = newName("isNull") + + val accessCode = + s""" + |long $resultTerm = $contextTerm.timestamp(); + |boolean $nullTerm = false; + """.stripMargin + + GeneratedExpression(resultTerm, nullTerm, accessCode, TimeIndicatorTypeInfo.ROWTIME_INDICATOR) + } + private def generateNullLiteral(resultType: TypeInformation[_]): GeneratedExpression = { val resultTerm = newName("result") val nullTerm = newName("isNull") @@ -1214,8 +1230,15 @@ abstract class CodeGenerator( |""".stripMargin } else if (nullCheck) { s""" - |$resultTypeTerm $resultTerm = $unboxedFieldCode; + |$tmpTypeTerm $tmpTerm = $fieldTerm; |boolean $nullTerm = $fieldTerm == null; + |$resultTypeTerm $resultTerm; + |if ($nullTerm) { + | $resultTerm = $defaultValue; + |} + |else { + | $resultTerm = $unboxedFieldCode; + |} |""".stripMargin } else { s""" @@ -1268,27 +1291,14 @@ abstract class CodeGenerator( } } - private[flink] def generateRecordTimestamp(isEventTime: Boolean): GeneratedExpression = { + private[flink] def generateProcTimestamp(): 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 { + val resultCode = s""" |$resultTypeTerm $resultTerm = $contextTerm.timerService().currentProcessingTime(); |""".stripMargin - } GeneratedExpression(resultTerm, NEVER_NULL, resultCode, SqlTimeTypeInfo.TIMESTAMP) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala index 1ab927d0020e2..dd01e027648a1 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala @@ -17,8 +17,6 @@ */ package org.apache.flink.table.codegen.calls -import java.lang.reflect.Method - import org.apache.calcite.avatica.util.DateTimeUtils.MILLIS_PER_DAY import org.apache.calcite.avatica.util.{DateTimeUtils, TimeUnitRange} import org.apache.calcite.util.BuiltInMethod @@ -1025,15 +1023,40 @@ object ScalarOperators { } } - def generateConcat( - method: Method, - operands: Seq[GeneratedExpression]): GeneratedExpression = { + def generateConcat(operands: Seq[GeneratedExpression]): GeneratedExpression = { - generateCallIfArgsNotNull(false, STRING_TYPE_INFO, operands) { - (terms) =>s"${qualifyMethod(method)}(${terms.mkString(", ")})" + generateCallIfArgsNotNull(true, STRING_TYPE_INFO, operands) { + (terms) =>s"${qualifyMethod(BuiltInMethods.CONCAT)}(${terms.mkString(", ")})" } } + def generateConcatWs(operands: Seq[GeneratedExpression]): GeneratedExpression = { + + val resultTerm = newName("result") + val nullTerm = newName("isNull") + val defaultValue = primitiveDefaultValue(Types.STRING) + + val operatorCode = + s""" + |${operands.map(_.code).mkString("\n")} + | + |String $resultTerm; + |boolean $nullTerm; + |if (${operands.head.nullTerm}) { + | $nullTerm = true; + | $resultTerm = $defaultValue; + |} else { + | + | ${operands.tail.map(o => s"if (${o.nullTerm}) ${o.resultTerm} = null;").mkString("\n")} + | $nullTerm = false; + | $resultTerm = ${qualifyMethod(BuiltInMethods.CONCAT_WS)}( + | ${operands.map(_.resultTerm).mkString(", ")}); + |} + |""".stripMargin + + GeneratedExpression(resultTerm, nullTerm, operatorCode, Types.STRING) + } + def generateMapGet( codeGenerator: CodeGenerator, map: GeneratedExpression, diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TimeMaterializationSqlFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/ProcTimeMaterializationSqlFunction.scala similarity index 85% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TimeMaterializationSqlFunction.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/ProcTimeMaterializationSqlFunction.scala index d87502650edb9..d2e7eff5f52fc 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TimeMaterializationSqlFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/ProcTimeMaterializationSqlFunction.scala @@ -22,12 +22,12 @@ import org.apache.calcite.sql.`type`._ import org.apache.calcite.sql.validate.SqlMonotonicity /** - * Function that materializes a time attribute to the metadata timestamp. After materialization - * the result can be used in regular arithmetical calculations. + * Function that materializes a processing time attribute. + * After materialization the result can be used in regular arithmetical calculations. */ -object TimeMaterializationSqlFunction +object ProcTimeMaterializationSqlFunction extends SqlFunction( - "TIME_MATERIALIZATION", + "PROCTIME", SqlKind.OTHER_FUNCTION, ReturnTypes.explicit(SqlTypeName.TIMESTAMP), InferTypes.RETURN_TYPE, 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 96aaf3e5e38dd..71f0d4df18aba 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 @@ -53,8 +53,6 @@ trait CommonCorrelate { functionClass: Class[T]): GeneratedFunction[T, Row] = { - val physicalRexCall = inputSchema.mapRexNode(rexCall) - val functionGenerator = new FunctionCodeGenerator( config, false, @@ -69,7 +67,7 @@ trait CommonCorrelate { .addReusableConstructor(classOf[TableFunctionCollector[_]]) .head - val call = functionGenerator.generateExpression(physicalRexCall) + val call = functionGenerator.generateExpression(rexCall) var body = s""" |${call.resultTerm}.setCollector($collectorTerm); diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala index dc7a0d6d3579e..5872d8cd360c4 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala @@ -39,9 +39,7 @@ abstract class PhysicalTableSourceScan( val flinkTypeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory] flinkTypeFactory.buildLogicalRowType( TableEnvironment.getFieldNames(tableSource), - TableEnvironment.getFieldTypes(tableSource.getReturnType), - None, - None) + TableEnvironment.getFieldTypes(tableSource.getReturnType)) } override def explainTerms(pw: RelWriter): RelWriter = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala index fb291e407a5a7..74aac431ce273 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala @@ -42,9 +42,7 @@ class BatchTableSourceScan( val flinkTypeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory] flinkTypeFactory.buildLogicalRowType( TableEnvironment.getFieldNames(tableSource), - TableEnvironment.getFieldTypes(tableSource.getReturnType), - None, - None) + TableEnvironment.getFieldTypes(tableSource.getReturnType)) } override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = { 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 2e0033063e7c9..e40501ec62a62 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 @@ -110,10 +110,6 @@ class DataStreamCalc( // filter out time attributes val projection = calcProgram.getProjectList.asScala .map(calcProgram.expandLocalRef) - // time indicator fields must not be part of the code generation - .filter(expr => !FlinkTypeFactory.isTimeIndicatorType(expr.getType)) - // update indices - .map(expr => inputSchema.mapRexNode(expr)) val generator = new FunctionCodeGenerator(config, false, inputSchema.physicalTypeInfo) 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 12694fc30fc27..ca9ba23929bae 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 @@ -112,12 +112,6 @@ class DataStreamGroupAggregate( val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig) - val physicalNamedAggregates = namedAggregates.map { namedAggregate => - new CalcitePair[AggregateCall, String]( - inputSchema.mapAggregateCall(namedAggregate.left), - namedAggregate.right) - } - val outRowType = CRowTypeInfo(schema.physicalTypeInfo) val generator = new AggregationCodeGenerator( @@ -136,11 +130,9 @@ class DataStreamGroupAggregate( s"select: ($aggString)" val nonKeyedAggOpName = s"select: ($aggString)" - val physicalGrouping = groupings.map(inputSchema.mapIndex) - val processFunction = AggregateUtil.createGroupAggregateFunction( generator, - physicalNamedAggregates, + namedAggregates, inputSchema.logicalType, inputSchema.physicalFieldTypeInfo, groupings, @@ -150,7 +142,7 @@ class DataStreamGroupAggregate( val result: DataStream[CRow] = // grouped / keyed aggregation - if (physicalGrouping.nonEmpty) { + if (groupings.nonEmpty) { inputDS .keyBy(groupings: _*) .process(processFunction) 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 c4ffdb1e900f2..856e9bf7cdcf3 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 @@ -32,11 +32,13 @@ import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.codegen.AggregationCodeGenerator import org.apache.flink.table.expressions.ExpressionUtils._ +import org.apache.flink.table.expressions.ResolvedFieldReference import org.apache.flink.table.plan.logical._ import org.apache.flink.table.plan.nodes.CommonAggregate import org.apache.flink.table.plan.schema.RowSchema import org.apache.flink.table.plan.nodes.datastream.DataStreamGroupWindowAggregate._ import org.apache.flink.table.plan.rules.datastream.DataStreamRetractionRules +import org.apache.flink.table.runtime.TimestampSetterProcessFunction import org.apache.flink.table.runtime.aggregate.AggregateUtil._ import org.apache.flink.table.runtime.aggregate._ import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} @@ -118,14 +120,6 @@ class DataStreamGroupWindowAggregate( val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig) - val physicalNamedAggregates = namedAggregates.map { namedAggregate => - new CalcitePair[AggregateCall, String]( - inputSchema.mapAggregateCall(namedAggregate.left), - namedAggregate.right) - } - val physicalNamedProperties = namedProperties - .filter(np => !FlinkTypeFactory.isTimeIndicatorType(np.property.resultType)) - val inputIsAccRetract = DataStreamRetractionRules.isAccRetract(input) if (inputIsAccRetract) { @@ -148,6 +142,20 @@ class DataStreamGroupWindowAggregate( "state size. You may specify a retention time of 0 to not clean up the state.") } + val timestampedInput = if (isRowtimeAttribute(window.timeAttribute)) { + // copy the window rowtime attribute into the StreamRecord timestamp field + val timeAttribute = window.timeAttribute.asInstanceOf[ResolvedFieldReference].name + val timeIdx = inputSchema.logicalFieldNames.indexOf(timeAttribute) + + inputDS + .process( + new TimestampSetterProcessFunction(timeIdx,CRowTypeInfo(inputSchema.physicalTypeInfo))) + .setParallelism(inputDS.getParallelism) + .name(s"time attribute: ($timeAttribute)") + } else { + inputDS + } + val outRowType = CRowTypeInfo(schema.physicalTypeInfo) val aggString = aggregationToString( @@ -171,18 +179,16 @@ class DataStreamGroupWindowAggregate( case SessionGroupWindow(_, _, _) => true case _ => false } - val physicalGrouping = grouping.map(inputSchema.mapIndex) - // grouped / keyed aggregation - if (physicalGrouping.length > 0) { + if (grouping.length > 0) { val windowFunction = AggregateUtil.createAggregationGroupWindowFunction( window, - physicalGrouping.length, - physicalNamedAggregates.size, + grouping.length, + namedAggregates.size, schema.physicalArity, - physicalNamedProperties) + namedProperties) - val keyedStream = inputDS.keyBy(physicalGrouping: _*) + val keyedStream = timestampedInput.keyBy(grouping: _*) val windowedStream = createKeyedWindowedStream(queryConfig, window, keyedStream) .asInstanceOf[WindowedStream[CRow, Tuple, DataStreamWindow]] @@ -190,11 +196,11 @@ class DataStreamGroupWindowAggregate( val (aggFunction, accumulatorRowType, aggResultRowType) = AggregateUtil.createDataStreamAggregateFunction( generator, - physicalNamedAggregates, + namedAggregates, inputSchema.physicalType, inputSchema.physicalFieldTypeInfo, schema.physicalType, - physicalGrouping, + grouping, needMerge) windowedStream @@ -206,16 +212,16 @@ class DataStreamGroupWindowAggregate( val windowFunction = AggregateUtil.createAggregationAllWindowFunction( window, schema.physicalArity, - physicalNamedProperties) + namedProperties) val windowedStream = - createNonKeyedWindowedStream(queryConfig, window, inputDS) + createNonKeyedWindowedStream(queryConfig, window, timestampedInput) .asInstanceOf[AllWindowedStream[CRow, DataStreamWindow]] val (aggFunction, accumulatorRowType, aggResultRowType) = AggregateUtil.createDataStreamAggregateFunction( generator, - physicalNamedAggregates, + namedAggregates, inputSchema.physicalType, inputSchema.physicalFieldTypeInfo, schema.physicalType, 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 34a7fd8ca1956..c03a55e1710e1 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 @@ -141,60 +141,37 @@ class DataStreamOverAggregate( .get(orderKey.getFieldIndex) .getType - 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, - isRowsClause = overWindow.isRows) - } else if ( - overWindow.lowerBound.isPreceding && !overWindow.lowerBound.isUnbounded && - overWindow.upperBound.isCurrentRow) { - - // bounded OVER window - createBoundedAndCurrentRowOverWindow( - queryConfig, - generator, - inputDS, - isRowTimeType = false, - isRowsClause = overWindow.isRows) - } else { - throw new TableException( - "OVER RANGE FOLLOWING windows are not supported yet.") - } - - case _ if FlinkTypeFactory.isRowtimeIndicatorType(timeType) => - // row-time OVER window - if (overWindow.lowerBound.isPreceding && - overWindow.lowerBound.isUnbounded && overWindow.upperBound.isCurrentRow) { - // unbounded OVER window - createUnboundedAndCurrentRowOverWindow( - queryConfig, - generator, - inputDS, - isRowTimeType = true, - isRowsClause = overWindow.isRows) - } else if (overWindow.lowerBound.isPreceding && overWindow.upperBound.isCurrentRow) { - // bounded OVER window - createBoundedAndCurrentRowOverWindow( - queryConfig, - generator, - inputDS, - isRowTimeType = true, - isRowsClause = overWindow.isRows) - } else { - throw new TableException( - "OVER RANGE FOLLOWING windows are not supported yet.") - } - - case _ => - throw new TableException( - s"OVER windows can only be applied on time attributes.") + // identify window rowtime attribute + val rowTimeIdx: Option[Int] = if (FlinkTypeFactory.isRowtimeIndicatorType(timeType)) { + Some(orderKey.getFieldIndex) + } else if (FlinkTypeFactory.isProctimeIndicatorType(timeType)) { + None + } else { + throw new TableException(s"OVER windows can only be applied on time attributes.") + } + + if (overWindow.lowerBound.isPreceding && overWindow.lowerBound.isUnbounded && + overWindow.upperBound.isCurrentRow) { + // unbounded OVER window + createUnboundedAndCurrentRowOverWindow( + queryConfig, + generator, + inputDS, + rowTimeIdx, + isRowsClause = overWindow.isRows) + } else if ( + overWindow.lowerBound.isPreceding && !overWindow.lowerBound.isUnbounded && + overWindow.upperBound.isCurrentRow) { + + // bounded OVER window + createBoundedAndCurrentRowOverWindow( + queryConfig, + generator, + inputDS, + rowTimeIdx, + isRowsClause = overWindow.isRows) + } else { + throw new TableException("OVER RANGE FOLLOWING windows are not supported yet.") } } @@ -202,19 +179,14 @@ class DataStreamOverAggregate( queryConfig: StreamQueryConfig, generator: AggregationCodeGenerator, inputDS: DataStream[CRow], - isRowTimeType: Boolean, + rowTimeIdx: Option[Int], isRowsClause: Boolean): DataStream[CRow] = { val overWindow: Group = logicWindow.groups.get(0) - val partitionKeys: Array[Int] = overWindow.keys.toArray.map(schema.mapIndex) + val partitionKeys: Array[Int] = overWindow.keys.toArray - val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates.map { - namedAggregate => - new CalcitePair[AggregateCall, String]( - schema.mapAggregateCall(namedAggregate.left), - namedAggregate.right) - } + val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates // get the output types val returnTypeInfo = CRowTypeInfo(schema.physicalTypeInfo) @@ -226,7 +198,7 @@ class DataStreamOverAggregate( inputSchema.physicalTypeInfo, inputSchema.physicalFieldTypeInfo, queryConfig, - isRowTimeType, + rowTimeIdx, partitionKeys.nonEmpty, isRowsClause) @@ -254,17 +226,12 @@ class DataStreamOverAggregate( queryConfig: StreamQueryConfig, generator: AggregationCodeGenerator, inputDS: DataStream[CRow], - isRowTimeType: Boolean, + rowTimeIdx: Option[Int], isRowsClause: Boolean): DataStream[CRow] = { val overWindow: Group = logicWindow.groups.get(0) - val partitionKeys: Array[Int] = overWindow.keys.toArray.map(schema.mapIndex) - val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates.map { - namedAggregate => - new CalcitePair[AggregateCall, String]( - schema.mapAggregateCall(namedAggregate.left), - namedAggregate.right) - } + val partitionKeys: Array[Int] = overWindow.keys.toArray + val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates val precedingOffset = getLowerBoundary(logicWindow, overWindow, getInput()) + (if (isRowsClause) 1 else 0) @@ -281,7 +248,7 @@ class DataStreamOverAggregate( precedingOffset, queryConfig, isRowsClause, - isRowTimeType + rowTimeIdx ) val result: DataStream[CRow] = // partitioned aggregation diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala index 25e72fa887ef0..07ac238f0aeae 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala @@ -18,14 +18,15 @@ package org.apache.flink.table.plan.nodes.datastream -import org.apache.flink.api.common.functions.MapFunction import org.apache.flink.streaming.api.datastream.DataStream +import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.table.api.TableConfig +import org.apache.flink.table.codegen.FunctionCodeGenerator import org.apache.flink.table.plan.nodes.CommonScan import org.apache.flink.table.plan.schema.RowSchema import org.apache.flink.types.Row import org.apache.flink.table.plan.schema.FlinkTable -import org.apache.flink.table.runtime.CRowOutputMapRunner +import org.apache.flink.table.runtime.CRowOutputProcessRunner import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} import scala.collection.JavaConverters._ @@ -45,24 +46,37 @@ trait StreamScan extends CommonScan[CRow] with DataStreamRel { // conversion if (needsConversion(input.getType, internalType)) { - val function = generatedConversionFunction( + val generator = new FunctionCodeGenerator( config, - classOf[MapFunction[Any, Row]], + false, inputType, + None, + Some(flinkTable.fieldIndexes)) + + val conversion = generator.generateConverterResultExpression( schema.physicalTypeInfo, + schema.physicalFieldNames) + + val body = + s""" + |${conversion.code} + |${generator.collectorTerm}.collect(${conversion.resultTerm}); + |""".stripMargin + + val function = generator.generateFunction( "DataStreamSourceConversion", - schema.physicalFieldNames, - Some(flinkTable.fieldIndexes)) + classOf[ProcessFunction[Any, Row]], + body, + schema.physicalTypeInfo) - val mapFunc = new CRowOutputMapRunner( + val processFunc = new CRowOutputProcessRunner( function.name, function.code, internalType) val opName = s"from: (${getRowType.getFieldNames.asScala.toList.mkString(", ")})" - // TODO we need a ProcessFunction here - input.map(mapFunc).name(opName).returns(internalType) + input.process(processFunc).name(opName).returns(internalType) } // no conversion necessary, forward else { 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 72ecac58243c4..663b2762eb9c2 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 @@ -30,6 +30,7 @@ import org.apache.flink.table.sources._ import org.apache.flink.table.plan.schema.TableSourceTable import org.apache.flink.table.runtime.types.CRow import org.apache.flink.table.sources.{StreamTableSource, TableSource} +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo /** Flink RelNode to read data from an external source defined by a [[StreamTableSource]]. */ class StreamTableSourceScan( @@ -46,29 +47,29 @@ class StreamTableSourceScan( val fieldNames = TableEnvironment.getFieldNames(tableSource).toList val fieldTypes = TableEnvironment.getFieldTypes(tableSource.getReturnType).toList - val fieldCnt = fieldNames.length + val fields = fieldNames.zip(fieldTypes) - val rowtime = tableSource match { + val withRowtime = tableSource match { case timeSource: DefinedRowtimeAttribute if timeSource.getRowtimeAttribute != null => val rowtimeAttribute = timeSource.getRowtimeAttribute - Some((fieldCnt, rowtimeAttribute)) + fields :+ (rowtimeAttribute, TimeIndicatorTypeInfo.ROWTIME_INDICATOR) case _ => - None + fields } - val proctime = tableSource match { + val withProctime = tableSource match { case timeSource: DefinedProctimeAttribute if timeSource.getProctimeAttribute != null => val proctimeAttribute = timeSource.getProctimeAttribute - Some((fieldCnt + (if (rowtime.isDefined) 1 else 0), proctimeAttribute)) + withRowtime :+ (proctimeAttribute, TimeIndicatorTypeInfo.PROCTIME_INDICATOR) case _ => - None + withRowtime } + val (fieldNamesWithIndicators, fieldTypesWithIndicators) = withProctime.unzip + flinkTypeFactory.buildLogicalRowType( - fieldNames, - fieldTypes, - rowtime, - proctime) + fieldNamesWithIndicators, + fieldTypesWithIndicators) } override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = { 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 3ae949ef57516..470d006257077 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 @@ -30,6 +30,7 @@ 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.typeutils.TimeIndicatorTypeInfo import scala.collection.JavaConverters._ @@ -51,29 +52,29 @@ class FlinkLogicalTableSourceScan( val fieldNames = TableEnvironment.getFieldNames(tableSource).toList val fieldTypes = TableEnvironment.getFieldTypes(tableSource.getReturnType).toList - val fieldCnt = fieldNames.length + val fields = fieldNames.zip(fieldTypes) - val rowtime = tableSource match { + val withRowtime = tableSource match { case timeSource: DefinedRowtimeAttribute if timeSource.getRowtimeAttribute != null => val rowtimeAttribute = timeSource.getRowtimeAttribute - Some((fieldCnt, rowtimeAttribute)) + fields :+ (rowtimeAttribute, TimeIndicatorTypeInfo.ROWTIME_INDICATOR) case _ => - None + fields } - val proctime = tableSource match { + val withProctime = tableSource match { case timeSource: DefinedProctimeAttribute if timeSource.getProctimeAttribute != null => val proctimeAttribute = timeSource.getProctimeAttribute - Some((fieldCnt + (if (rowtime.isDefined) 1 else 0), proctimeAttribute)) + withRowtime :+ (proctimeAttribute, TimeIndicatorTypeInfo.PROCTIME_INDICATOR) case _ => - None + withRowtime } + val (fieldNamesWithIndicators, fieldTypesWithIndicators) = withProctime.unzip + flinkTypeFactory.buildLogicalRowType( - fieldNames, - fieldTypes, - rowtime, - proctime) + fieldNamesWithIndicators, + fieldTypesWithIndicators) } override def computeSelfCost(planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataStreamTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataStreamTable.scala index 70054b4a0d2c9..b7021e285a4a0 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataStreamTable.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataStreamTable.scala @@ -18,27 +18,14 @@ package org.apache.flink.table.plan.schema -import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory} import org.apache.flink.streaming.api.datastream.DataStream -import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.plan.stats.FlinkStatistic class DataStreamTable[T]( val dataStream: DataStream[T], override val fieldIndexes: Array[Int], override val fieldNames: Array[String], - val rowtime: Option[(Int, String)], - val proctime: Option[(Int, String)], override val statistic: FlinkStatistic = FlinkStatistic.UNKNOWN) extends FlinkTable[T](dataStream.getType, fieldIndexes, fieldNames, statistic) { - override def getRowType(typeFactory: RelDataTypeFactory): RelDataType = { - val flinkTypeFactory = typeFactory.asInstanceOf[FlinkTypeFactory] - - flinkTypeFactory.buildLogicalRowType( - fieldNames, - fieldTypes, - rowtime, - proctime) - } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala index 752b00e9747dd..90ddd477d99d2 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala @@ -26,6 +26,7 @@ import org.apache.flink.api.common.typeutils.CompositeType import org.apache.flink.table.api.TableException import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.plan.stats.FlinkStatistic +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo abstract class FlinkTable[T]( val typeInfo: TypeInformation[T], @@ -49,12 +50,15 @@ abstract class FlinkTable[T]( typeInfo match { case cType: CompositeType[_] => // it is ok to leave out fields - if (fieldNames.length > cType.getArity) { + if (fieldIndexes.count(_ >= 0) > cType.getArity) { throw new TableException( s"Arity of type (" + cType.getFieldNames.deep + ") " + "must not be greater than number of field names " + fieldNames.deep + ".") } - fieldIndexes.map(cType.getTypeAt(_).asInstanceOf[TypeInformation[_]]) + fieldIndexes.map { + case -1 => TimeIndicatorTypeInfo.ROWTIME_INDICATOR + case -2 => TimeIndicatorTypeInfo.PROCTIME_INDICATOR + case i => cType.getTypeAt(i).asInstanceOf[TypeInformation[_]]} case aType: AtomicType[_] => if (fieldIndexes.length != 1 || fieldIndexes(0) != 0) { throw new TableException( @@ -65,7 +69,7 @@ abstract class FlinkTable[T]( override def getRowType(typeFactory: RelDataTypeFactory): RelDataType = { val flinkTypeFactory = typeFactory.asInstanceOf[FlinkTypeFactory] - flinkTypeFactory.buildLogicalRowType(fieldNames, fieldTypes, None, None) + flinkTypeFactory.buildLogicalRowType(fieldNames, fieldTypes) } /** 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 ccbe44d3af0ec..3cc4f13517423 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 @@ -19,13 +19,9 @@ 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.{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._ @@ -35,9 +31,7 @@ import scala.collection.JavaConversions._ */ class RowSchema(private val logicalRowType: RelDataType) { - private lazy val physicalRowFields: Seq[RelDataTypeField] = logicalRowType.getFieldList filter { - field => !FlinkTypeFactory.isTimeIndicatorType(field.getType) - } + private lazy val physicalRowFields: Seq[RelDataTypeField] = logicalRowType.getFieldList private lazy val physicalRowType: RelDataType = new RelRecordType(physicalRowFields) @@ -50,43 +44,6 @@ class RowSchema(private val logicalRowType: RelDataType) { private lazy val physicalRowTypeInfo: TypeInformation[Row] = new RowTypeInfo( physicalRowFieldTypes.toArray, physicalRowFieldNames.toArray) - private lazy val indexMapping: Array[Int] = generateIndexMapping - - private lazy val inputRefUpdater = new RexInputRefUpdater() - - private def generateIndexMapping: Array[Int] = { - val mapping = new Array[Int](logicalRowType.getFieldCount) - var countTimeIndicators = 0 - var i = 0 - while (i < logicalRowType.getFieldCount) { - val t = logicalRowType.getFieldList.get(i).getType - if (FlinkTypeFactory.isTimeIndicatorType(t)) { - countTimeIndicators += 1 - // no mapping - mapping(i) = -1 - } else { - mapping(i) = i - countTimeIndicators - } - i += 1 - } - mapping - } - - private class RexInputRefUpdater extends RexShuttle { - - 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) - } - } - /** * Returns the arity of the logical record. */ @@ -127,35 +84,4 @@ class RowSchema(private val logicalRowType: RelDataType) { */ def physicalFieldNames: Seq[String] = physicalRowFieldNames - /** - * Converts logical indices to physical indices based on this schema. - */ - def mapIndex(logicalIndex: Int): Int = { - val mappedIndex = indexMapping(logicalIndex) - if (mappedIndex < 0) { - throw new TableException("Invalid access to a logical field.") - } else { - mappedIndex - } - } - - /** - * Converts logical indices of a aggregate call to physical ones. - */ - def mapAggregateCall(logicalAggCall: AggregateCall): AggregateCall = { - logicalAggCall.copy( - logicalAggCall.getArgList.map(mapIndex(_).asInstanceOf[Integer]), - if (logicalAggCall.filterArg < 0) { - logicalAggCall.filterArg - } else { - mapIndex(logicalAggCall.filterArg) - } - ) - } - - /** - * Converts logical field references of a [[RexNode]] to physical ones. - */ - def mapRexNode(logicalRexNode: RexNode): RexNode = logicalRexNode.accept(inputRefUpdater) - } 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 408381dea9e81..dc1f31ab27db5 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 @@ -23,6 +23,7 @@ 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.typeutils.TimeIndicatorTypeInfo class StreamTableSourceTable[T]( override val tableSource: TableSource[T], @@ -36,41 +37,38 @@ class StreamTableSourceTable[T]( val fieldNames = TableEnvironment.getFieldNames(tableSource).toList val fieldTypes = TableEnvironment.getFieldTypes(tableSource.getReturnType).toList - val fieldCnt = fieldNames.length + val fields = fieldNames.zip(fieldTypes) - val rowtime = tableSource match { - case nullTimeSource : DefinedRowtimeAttribute - if nullTimeSource.getRowtimeAttribute == null => - None - case emptyStringTimeSource: DefinedRowtimeAttribute - if emptyStringTimeSource.getRowtimeAttribute.trim.equals("") => - throw TableException("The name of the rowtime attribute must not be empty.") - case timeSource: DefinedRowtimeAttribute => + val withRowtime = tableSource match { + case timeSource: DefinedRowtimeAttribute if timeSource.getRowtimeAttribute == null => + fields + case timeSource: DefinedRowtimeAttribute if timeSource.getRowtimeAttribute.trim.equals("") => + throw TableException("The name of the rowtime attribute must not be empty.") + case timeSource: DefinedRowtimeAttribute => val rowtimeAttribute = timeSource.getRowtimeAttribute - Some((fieldCnt, rowtimeAttribute)) + fields :+ (rowtimeAttribute, TimeIndicatorTypeInfo.ROWTIME_INDICATOR) case _ => - None + fields } - val proctime = tableSource match { - case nullTimeSource : DefinedProctimeAttribute - if nullTimeSource.getProctimeAttribute == null => - None - case emptyStringTimeSource: DefinedProctimeAttribute - if emptyStringTimeSource.getProctimeAttribute.trim.equals("") => - throw TableException("The name of the proctime attribute must not be empty.") - case timeSource: DefinedProctimeAttribute => + val withProctime = tableSource match { + case timeSource : DefinedProctimeAttribute if timeSource.getProctimeAttribute == null => + withRowtime + case timeSource: DefinedProctimeAttribute + if timeSource.getProctimeAttribute.trim.equals("") => + throw TableException("The name of the rowtime attribute must not be empty.") + case timeSource: DefinedProctimeAttribute => val proctimeAttribute = timeSource.getProctimeAttribute - Some((fieldCnt + (if (rowtime.isDefined) 1 else 0), proctimeAttribute)) + withRowtime :+ (proctimeAttribute, TimeIndicatorTypeInfo.PROCTIME_INDICATOR) case _ => - None + withRowtime } + val (fieldNamesWithIndicators, fieldTypesWithIndicators) = withProctime.unzip + flinkTypeFactory.buildLogicalRowType( - fieldNames, - fieldTypes, - rowtime, - proctime) + fieldNamesWithIndicators, + fieldTypesWithIndicators) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputTupleOutputMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputTupleOutputMapRunner.scala index 7c964371827f9..002c34826ca9e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputTupleOutputMapRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputTupleOutputMapRunner.scala @@ -19,7 +19,9 @@ package org.apache.flink.table.runtime import java.lang.{Boolean => JBool} +import java.sql.Timestamp +import org.apache.calcite.runtime.SqlFunctions import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.ResultTypeQueryable @@ -29,6 +31,9 @@ import org.apache.flink.table.runtime.types.CRow import org.apache.flink.types.Row import org.slf4j.LoggerFactory import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.util.Collector /** * Convert [[CRow]] to a [[JTuple2]] @@ -90,3 +95,35 @@ class CRowInputScalaTupleOutputMapRunner( override def getProducedType: TypeInformation[(Boolean, Any)] = returnType } + +/** + * Wraps a ProcessFunction and sets a Timestamp field of a CRow as + * [[org.apache.flink.streaming.runtime.streamrecord.StreamRecord]] timestamp. + */ +class WrappingTimestampSetterProcessFunction[OUT]( + function: MapFunction[CRow, OUT], + rowtimeIdx: Int) + extends ProcessFunction[CRow, OUT] { + + override def open(parameters: Configuration): Unit = { + super.open(parameters) + function match { + case f: RichMapFunction[_, _] => + f.setRuntimeContext(getRuntimeContext) + f.open(parameters) + case _ => + } + } + + override def processElement( + in: CRow, + ctx: ProcessFunction[CRow, OUT]#Context, + out: Collector[OUT]): Unit = { + + val timestamp = SqlFunctions.toLong(in.row.getField(rowtimeIdx).asInstanceOf[Timestamp]) + out.asInstanceOf[TimestampedCollector[_]].setAbsoluteTimestamp(timestamp) + + out.collect(function.map(in)) + } + +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputProcessRunner.scala similarity index 65% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputMapRunner.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputProcessRunner.scala index cb8f69556e351..ac94c500ebff4 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputMapRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputProcessRunner.scala @@ -18,42 +18,54 @@ package org.apache.flink.table.runtime -import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction} 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.streaming.api.operators.TimestampedCollector 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 /** - * MapRunner with [[CRow]] output. + * ProcessRunner with [[CRow]] output. */ -class CRowOutputMapRunner( +class CRowOutputProcessRunner( name: String, code: String, @transient var returnType: TypeInformation[CRow]) - extends RichMapFunction[Any, CRow] + extends ProcessFunction[Any, CRow] with ResultTypeQueryable[CRow] - with Compiler[MapFunction[Any, Row]] { + with Compiler[ProcessFunction[Any, Row]] { val LOG = LoggerFactory.getLogger(this.getClass) - private var function: MapFunction[Any, Row] = _ - private var outCRow: CRow = _ + private var function: ProcessFunction[Any, Row] = _ + private var cRowWrapper: CRowWrappingCollector = _ override def open(parameters: Configuration): Unit = { LOG.debug(s"Compiling MapFunction: $name \n\n Code:\n$code") val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code) LOG.debug("Instantiating MapFunction.") function = clazz.newInstance() - outCRow = new CRow(null, true) + + this.cRowWrapper = new CRowWrappingCollector() + this.cRowWrapper.setChange(true) } - override def map(in: Any): CRow = { - outCRow.row = function.map(in) - outCRow + override def processElement( + in: Any, + ctx: ProcessFunction[Any, CRow]#Context, + out: Collector[CRow]): Unit = { + + // remove timestamp from stream record + val tc = out.asInstanceOf[TimestampedCollector[_]] + tc.eraseTimestamp() + + cRowWrapper.out = out + function.processElement(in, ctx.asInstanceOf[ProcessFunction[Any, Row]#Context], cRowWrapper) } override def getProducedType: TypeInformation[CRow] = returnType diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/TimestampSetterProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/TimestampSetterProcessFunction.scala new file mode 100644 index 0000000000000..00961f0f57d98 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/TimestampSetterProcessFunction.scala @@ -0,0 +1,52 @@ +/* + * 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 java.sql.Timestamp + +import org.apache.calcite.runtime.SqlFunctions +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ResultTypeQueryable +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.util.Collector + +/** + * ProcessFunction to copy a timestamp from a [[org.apache.flink.types.Row]] field into the + * [[org.apache.flink.streaming.runtime.streamrecord.StreamRecord]]. + */ +class TimestampSetterProcessFunction( + val rowtimeIdx: Int, + @transient var returnType: TypeInformation[CRow]) + extends ProcessFunction[CRow, CRow] + with ResultTypeQueryable[CRow] { + + override def processElement( + in: CRow, + ctx: ProcessFunction[CRow, CRow]#Context, + out: Collector[CRow]): Unit = { + + val timestamp = SqlFunctions.toLong(in.row.getField(rowtimeIdx).asInstanceOf[Timestamp]) + out.asInstanceOf[TimestampedCollector[CRow]].setAbsoluteTimestamp(timestamp) + out.collect(in) + } + + override def getProducedType: TypeInformation[CRow] = returnType +} 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 c9f98e31bd3be..9372dbd8d8a44 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 @@ -35,7 +35,7 @@ import org.apache.flink.streaming.api.windowing.windows.{Window => DataStreamWin 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.{AggregationCodeGenerator, CodeGenerator} +import org.apache.flink.table.codegen.AggregationCodeGenerator import org.apache.flink.table.expressions.ExpressionUtils.isTimeIntervalLiteral import org.apache.flink.table.expressions._ import org.apache.flink.table.functions.aggfunctions._ @@ -66,7 +66,7 @@ object AggregateUtil { * @param inputType Physical type of the row. * @param inputTypeInfo Physical type information of the row. * @param inputFieldTypeInfo Physical type information of the row's fields. - * @param isRowTimeType It is a tag that indicates whether the time type is rowTimeType + * @param rowTimeIdx The index of the rowtime field or None in case of processing time. * @param isPartitioned It is a tag that indicate whether the input is partitioned * @param isRowsClause It is a tag that indicates whether the OVER clause is ROWS clause */ @@ -77,7 +77,7 @@ object AggregateUtil { inputTypeInfo: TypeInformation[Row], inputFieldTypeInfo: Seq[TypeInformation[_]], queryConfig: StreamQueryConfig, - isRowTimeType: Boolean, + rowTimeIdx: Option[Int], isPartitioned: Boolean, isRowsClause: Boolean) : ProcessFunction[CRow, CRow] = { @@ -111,13 +111,14 @@ object AggregateUtil { needReset = false ) - if (isRowTimeType) { + if (rowTimeIdx.isDefined) { if (isRowsClause) { // ROWS unbounded over process function new RowTimeUnboundedRowsOver( genFunction, aggregationStateType, CRowTypeInfo(inputTypeInfo), + rowTimeIdx.get, queryConfig) } else { // RANGE unbounded over process function @@ -125,6 +126,7 @@ object AggregateUtil { genFunction, aggregationStateType, CRowTypeInfo(inputTypeInfo), + rowTimeIdx.get, queryConfig) } } else { @@ -207,7 +209,7 @@ object AggregateUtil { * @param inputFieldTypeInfo Physical type information of the row's fields. * @param precedingOffset the preceding offset * @param isRowsClause It is a tag that indicates whether the OVER clause is ROWS clause - * @param isRowTimeType It is a tag that indicates whether the time type is rowTimeType + * @param rowTimeIdx The index of the rowtime field or None in case of processing time. * @return [[org.apache.flink.streaming.api.functions.ProcessFunction]] */ private[flink] def createBoundedOverProcessFunction( @@ -219,7 +221,7 @@ object AggregateUtil { precedingOffset: Long, queryConfig: StreamQueryConfig, isRowsClause: Boolean, - isRowTimeType: Boolean) + rowTimeIdx: Option[Int]) : ProcessFunction[CRow, CRow] = { val needRetract = true @@ -253,13 +255,14 @@ object AggregateUtil { needReset = true ) - if (isRowTimeType) { + if (rowTimeIdx.isDefined) { if (isRowsClause) { new RowTimeBoundedRowsOver( genFunction, aggregationStateType, inputRowType, precedingOffset, + rowTimeIdx.get, queryConfig) } else { new RowTimeBoundedRangeOver( @@ -267,6 +270,7 @@ object AggregateUtil { aggregationStateType, inputRowType, precedingOffset, + rowTimeIdx.get, queryConfig) } } else { @@ -588,7 +592,7 @@ object AggregateUtil { window match { case TumblingGroupWindow(_, _, size) if isTimeInterval(size.resultType) => // tumbling time window - val (startPos, endPos) = computeWindowStartEndPropertyPos(properties) + val (startPos, endPos, _) = computeWindowPropertyPos(properties) if (doAllSupportPartialMerge(aggregates)) { // for incremental aggregations new DataSetTumbleTimeWindowAggReduceCombineFunction( @@ -615,7 +619,7 @@ object AggregateUtil { asLong(size)) case SessionGroupWindow(_, _, gap) => - val (startPos, endPos) = computeWindowStartEndPropertyPos(properties) + val (startPos, endPos, _) = computeWindowPropertyPos(properties) new DataSetSessionWindowAggReduceGroupFunction( genFinalAggFunction, keysAndAggregatesArity, @@ -625,7 +629,7 @@ object AggregateUtil { isInputCombined) case SlidingGroupWindow(_, _, size, _) if isTimeInterval(size.resultType) => - val (startPos, endPos) = computeWindowStartEndPropertyPos(properties) + val (startPos, endPos, _) = computeWindowPropertyPos(properties) if (doAllSupportPartialMerge(aggregates)) { // for partial aggregations new DataSetSlideWindowAggReduceCombineFunction( @@ -951,10 +955,11 @@ object AggregateUtil { : AllWindowFunction[Row, CRow, DataStreamWindow] = { if (isTimeWindow(window)) { - val (startPos, endPos) = computeWindowStartEndPropertyPos(properties) + val (startPos, endPos, timePos) = computeWindowPropertyPos(properties) new IncrementalAggregateAllTimeWindowFunction( startPos, endPos, + timePos, finalRowArity) .asInstanceOf[AllWindowFunction[Row, CRow, DataStreamWindow]] } else { @@ -975,12 +980,13 @@ object AggregateUtil { WindowFunction[Row, CRow, Tuple, DataStreamWindow] = { if (isTimeWindow(window)) { - val (startPos, endPos) = computeWindowStartEndPropertyPos(properties) + val (startPos, endPos, timePos) = computeWindowPropertyPos(properties) new IncrementalAggregateTimeWindowFunction( numGroupingKeys, numAggregates, startPos, endPos, + timePos, finalRowArity) .asInstanceOf[WindowFunction[Row, CRow, Tuple, DataStreamWindow]] } else { @@ -1136,25 +1142,31 @@ object AggregateUtil { } } - private[flink] def computeWindowStartEndPropertyPos( - properties: Seq[NamedWindowProperty]): (Option[Int], Option[Int]) = { + private[flink] def computeWindowPropertyPos( + properties: Seq[NamedWindowProperty]): (Option[Int], Option[Int], Option[Int]) = { - val propPos = properties.foldRight((None: Option[Int], None: Option[Int], 0)) { + val propPos = properties.foldRight( + (None: Option[Int], None: Option[Int], None: Option[Int], 0)) { (p, x) => p match { case NamedWindowProperty(_, prop) => prop match { case WindowStart(_) if x._1.isDefined => throw new TableException("Duplicate WindowStart property encountered. This is a bug.") case WindowStart(_) => - (Some(x._3), x._2, x._3 - 1) + (Some(x._4), x._2, x._3, x._4 - 1) case WindowEnd(_) if x._2.isDefined => throw new TableException("Duplicate WindowEnd property encountered. This is a bug.") case WindowEnd(_) => - (x._1, Some(x._3), x._3 - 1) + (x._1, Some(x._4), x._3, x._4 - 1) + case RowtimeAttribute(_) if x._3.isDefined => + throw new TableException( + "Duplicate Window rowtime property encountered. This is a bug.") + case RowtimeAttribute(_) => + (x._1, x._2, Some(x._4), x._4 - 1) } } } - (propPos._1, propPos._2) + (propPos._1, propPos._2, propPos._3) } private def transformToAggregateFunctions( diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggReduceGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggReduceGroupFunction.scala index fabf200add416..2160ef5bd5941 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggReduceGroupFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggReduceGroupFunction.scala @@ -78,7 +78,10 @@ class DataSetSessionWindowAggReduceGroupFunction( output = function.createOutputRow() accumulators = function.createAccumulators() - collector = new RowTimeWindowPropertyCollector(finalRowWindowStartPos, finalRowWindowEndPos) + collector = new RowTimeWindowPropertyCollector( + finalRowWindowStartPos, + finalRowWindowEndPos, + None) } /** diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideWindowAggReduceGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideWindowAggReduceGroupFunction.scala index 56ed08ade4a80..e4b9458ed834e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideWindowAggReduceGroupFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideWindowAggReduceGroupFunction.scala @@ -68,7 +68,10 @@ class DataSetSlideWindowAggReduceGroupFunction( output = function.createOutputRow() accumulators = function.createAccumulators() - collector = new RowTimeWindowPropertyCollector(finalRowWindowStartPos, finalRowWindowEndPos) + collector = new RowTimeWindowPropertyCollector( + finalRowWindowStartPos, + finalRowWindowEndPos, + None) } override def reduce(records: Iterable[Row], out: Collector[Row]): Unit = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleTimeWindowAggReduceGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleTimeWindowAggReduceGroupFunction.scala index 8af2c2e8a8a69..b4f7585282a14 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleTimeWindowAggReduceGroupFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleTimeWindowAggReduceGroupFunction.scala @@ -67,7 +67,7 @@ class DataSetTumbleTimeWindowAggReduceGroupFunction( output = function.createOutputRow() accumulators = function.createAccumulators() - collector = new RowTimeWindowPropertyCollector(windowStartPos, windowEndPos) + collector = new RowTimeWindowPropertyCollector(windowStartPos, windowEndPos, None) } override def reduce(records: Iterable[Row], out: Collector[Row]): Unit = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateAllTimeWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateAllTimeWindowFunction.scala index 711cc0505bf5b..3c2e8581343cf 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateAllTimeWindowFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateAllTimeWindowFunction.scala @@ -29,13 +29,15 @@ import org.apache.flink.util.Collector * * Computes the final aggregate value from incrementally computed aggregates. * - * @param windowStartPos the start position of window - * @param windowEndPos the end position of window + * @param windowStartOffset the offset of the window start property + * @param windowEndOffset the offset of the window end property + * @param windowRowtimeOffset the offset of the window rowtime property * @param finalRowArity The arity of the final output row. */ class IncrementalAggregateAllTimeWindowFunction( - private val windowStartPos: Option[Int], - private val windowEndPos: Option[Int], + private val windowStartOffset: Option[Int], + private val windowEndOffset: Option[Int], + private val windowRowtimeOffset: Option[Int], private val finalRowArity: Int) extends IncrementalAggregateAllWindowFunction[TimeWindow]( finalRowArity) { @@ -43,7 +45,10 @@ class IncrementalAggregateAllTimeWindowFunction( private var collector: CRowTimeWindowPropertyCollector = _ override def open(parameters: Configuration): Unit = { - collector = new CRowTimeWindowPropertyCollector(windowStartPos, windowEndPos) + collector = new CRowTimeWindowPropertyCollector( + windowStartOffset, + windowEndOffset, + windowRowtimeOffset) super.open(parameters) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateTimeWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateTimeWindowFunction.scala index 809bbfdd5676a..19502302385f4 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateTimeWindowFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateTimeWindowFunction.scala @@ -29,15 +29,19 @@ import org.apache.flink.util.Collector /** * Computes the final aggregate value from incrementally computed aggreagtes. * - * @param windowStartPos the start position of window - * @param windowEndPos the end position of window - * @param finalRowArity The arity of the final output row + * @param numGroupingKey the number of grouping keys + * @param numAggregates the number of aggregates + * @param windowStartOffset the offset of the window start property + * @param windowEndOffset the offset of the window end property + * @param windowRowtimeOffset the offset of the window rowtime property + * @param finalRowArity The arity of the final output row. */ class IncrementalAggregateTimeWindowFunction( private val numGroupingKey: Int, private val numAggregates: Int, - private val windowStartPos: Option[Int], - private val windowEndPos: Option[Int], + private val windowStartOffset: Option[Int], + private val windowEndOffset: Option[Int], + private val windowRowtimeOffset: Option[Int], private val finalRowArity: Int) extends IncrementalAggregateWindowFunction[TimeWindow]( numGroupingKey, @@ -47,7 +51,10 @@ class IncrementalAggregateTimeWindowFunction( private var collector: CRowTimeWindowPropertyCollector = _ override def open(parameters: Configuration): Unit = { - collector = new CRowTimeWindowPropertyCollector(windowStartPos, windowEndPos) + collector = new CRowTimeWindowPropertyCollector( + windowStartOffset, + windowEndOffset, + windowRowtimeOffset) super.open(parameters) } 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 8f2ec98a51c03..916e0216d5aa6 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.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} @@ -123,6 +124,8 @@ class ProcTimeBoundedRangeOver( return } + out.asInstanceOf[TimestampedCollector[_]].eraseTimestamp() + // we consider the original timestamp of events // that have registered this time trigger 1 ms ago diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala index 2d0b14b3e3f83..6bef05ac7a746 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala @@ -23,10 +23,11 @@ import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.types.Row import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} import org.apache.flink.util.{Collector, Preconditions} - import java.util.ArrayList import java.util.Collections +import org.apache.flink.streaming.api.operators.TimestampedCollector + /** * ProcessFunction to sort on processing time and additional attributes. @@ -75,7 +76,9 @@ class ProcTimeSortProcessFunction( timestamp: Long, ctx: ProcessFunction[CRow, CRow]#OnTimerContext, out: Collector[CRow]): Unit = { - + + out.asInstanceOf[TimestampedCollector[_]].eraseTimestamp() + val iter = bufferedEvents.get.iterator() // insert all rows into the sort buffer 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 1a207bbb3a2b5..77c62cb696223 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 @@ -17,13 +17,16 @@ */ package org.apache.flink.table.runtime.aggregate +import java.sql.Timestamp import java.util.{ArrayList => JArrayList, List => JList} +import org.apache.calcite.runtime.SqlFunctions import org.apache.flink.api.common.state._ 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.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} @@ -44,6 +47,7 @@ class RowTimeBoundedRangeOver( aggregationStateType: RowTypeInfo, inputRowType: CRowTypeInfo, precedingOffset: Long, + rowTimeIdx: Int, queryConfig: StreamQueryConfig) extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig) with Compiler[GeneratedAggregations] { @@ -114,7 +118,7 @@ class RowTimeBoundedRangeOver( registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) // triggering timestamp for trigger calculation - val triggeringTs = ctx.timestamp + val triggeringTs = SqlFunctions.toLong(input.getField(rowTimeIdx).asInstanceOf[Timestamp]) val lastTriggeringTs = lastTriggeringTsState.value @@ -166,6 +170,8 @@ class RowTimeBoundedRangeOver( return } + out.asInstanceOf[TimestampedCollector[_]].eraseTimestamp() + // gets all window data from state for the calculation val inputs: JList[Row] = dataState.get(timestamp) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala index a4b1076c514d6..b645684413aa5 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 @@ -17,14 +17,17 @@ */ package org.apache.flink.table.runtime.aggregate +import java.sql.Timestamp import java.util import java.util.{List => JList} +import org.apache.calcite.runtime.SqlFunctions import org.apache.flink.api.common.state._ 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.streaming.api.operators.TimestampedCollector import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.types.Row import org.apache.flink.util.{Collector, Preconditions} @@ -45,6 +48,7 @@ class RowTimeBoundedRowsOver( aggregationStateType: RowTypeInfo, inputRowType: CRowTypeInfo, precedingOffset: Long, + rowTimeIdx: Int, queryConfig: StreamQueryConfig) extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig) with Compiler[GeneratedAggregations] { @@ -123,7 +127,7 @@ class RowTimeBoundedRowsOver( registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) // triggering timestamp for trigger calculation - val triggeringTs = ctx.timestamp + val triggeringTs = SqlFunctions.toLong(input.getField(rowTimeIdx).asInstanceOf[Timestamp]) val lastTriggeringTs = lastTriggeringTsState.value // check if the data is expired, if not, save the data and register event time timer @@ -175,6 +179,8 @@ class RowTimeBoundedRowsOver( return } + out.asInstanceOf[TimestampedCollector[_]].eraseTimestamp() + // gets all window data from state for the calculation val inputs: JList[Row] = dataState.get(timestamp) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala index 737f32c255aed..92ad6ccb0a8f5 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala @@ -17,6 +17,8 @@ */ package org.apache.flink.table.runtime.aggregate +import java.sql.Timestamp + import org.apache.flink.api.common.state.ValueState import org.apache.flink.api.common.state.ValueStateDescriptor import org.apache.flink.api.common.state.MapState @@ -28,18 +30,22 @@ import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} import org.apache.flink.types.Row import org.apache.flink.util.{Collector, Preconditions} - import java.util.Collections -import java.util.{List => JList, ArrayList => JArrayList} +import java.util.{ArrayList => JArrayList, List => JList} + +import org.apache.calcite.runtime.SqlFunctions +import org.apache.flink.streaming.api.operators.TimestampedCollector /** * ProcessFunction to sort on event-time and possibly addtional secondary sort attributes. * * @param inputRowType The data type of the input data. + * @param rowtimeIdx The index of the rowtime field. * @param rowComparator A comparator to sort rows. */ class RowTimeSortProcessFunction( private val inputRowType: CRowTypeInfo, + private val rowtimeIdx: Int, private val rowComparator: Option[CollectionRowComparator]) extends ProcessFunction[CRow, CRow] { @@ -84,7 +90,7 @@ class RowTimeSortProcessFunction( val input = inputC.row // timestamp of the processed row - val rowtime = ctx.timestamp + val rowtime = SqlFunctions.toLong(input.getField(rowtimeIdx).asInstanceOf[Timestamp]) val lastTriggeringTs = lastTriggeringTsState.value @@ -105,13 +111,14 @@ class RowTimeSortProcessFunction( } } } - - + override def onTimer( timestamp: Long, ctx: ProcessFunction[CRow, CRow]#OnTimerContext, out: Collector[CRow]): Unit = { - + + out.asInstanceOf[TimestampedCollector[_]].eraseTimestamp() + // gets all rows for the triggering timestamps val inputs: JList[Row] = dataState.get(timestamp) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala index f38ba93794e94..6c62b1784ab2b 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 @@ -17,14 +17,16 @@ */ package org.apache.flink.table.runtime.aggregate +import java.sql.Timestamp import java.util import java.util.{List => JList} +import org.apache.calcite.runtime.SqlFunctions import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} import org.apache.flink.configuration.Configuration import org.apache.flink.types.Row import org.apache.flink.streaming.api.functions.ProcessFunction -import org.apache.flink.util.{Collector, Preconditions} +import org.apache.flink.util.Collector import org.apache.flink.api.common.state._ import org.apache.flink.api.java.typeutils.ListTypeInfo import org.apache.flink.streaming.api.operators.TimestampedCollector @@ -45,6 +47,7 @@ abstract class RowTimeUnboundedOver( genAggregations: GeneratedAggregationsFunction, intermediateType: TypeInformation[Row], inputType: TypeInformation[CRow], + rowTimeIdx: Int, queryConfig: StreamQueryConfig) extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig) with Compiler[GeneratedAggregations] { @@ -108,7 +111,7 @@ abstract class RowTimeUnboundedOver( // register state-cleanup timer registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) - val timestamp = ctx.timestamp() + val timestamp = SqlFunctions.toLong(input.getField(rowTimeIdx).asInstanceOf[Timestamp]) val curWatermark = ctx.timerService().currentWatermark() // discard late record @@ -158,8 +161,8 @@ abstract class RowTimeUnboundedOver( return } - Preconditions.checkArgument(out.isInstanceOf[TimestampedCollector[CRow]]) - val collector = out.asInstanceOf[TimestampedCollector[CRow]] + // remove StreamRecord timestamp + out.asInstanceOf[TimestampedCollector[_]].eraseTimestamp() val keyIterator = rowMapState.keys.iterator if (keyIterator.hasNext) { @@ -188,10 +191,9 @@ abstract class RowTimeUnboundedOver( while (!sortedTimestamps.isEmpty) { val curTimestamp = sortedTimestamps.removeFirst() val curRowList = rowMapState.get(curTimestamp) - collector.setAbsoluteTimestamp(curTimestamp) // process the same timestamp datas, the mechanism is different according ROWS or RANGE - processElementsWithSameTimestamp(curRowList, lastAccumulator, collector) + processElementsWithSameTimestamp(curRowList, lastAccumulator, out) rowMapState.remove(curTimestamp) } @@ -250,11 +252,13 @@ class RowTimeUnboundedRowsOver( genAggregations: GeneratedAggregationsFunction, intermediateType: TypeInformation[Row], inputType: TypeInformation[CRow], + rowTimeIdx: Int, queryConfig: StreamQueryConfig) extends RowTimeUnboundedOver( genAggregations: GeneratedAggregationsFunction, intermediateType, inputType, + rowTimeIdx, queryConfig) { override def processElementsWithSameTimestamp( @@ -266,7 +270,6 @@ class RowTimeUnboundedRowsOver( while (i < curRowList.size) { val curRow = curRowList.get(i) - var j = 0 // copy forwarded fields to output row function.setForwardedFields(curRow, output.row) @@ -290,11 +293,13 @@ class RowTimeUnboundedRangeOver( genAggregations: GeneratedAggregationsFunction, intermediateType: TypeInformation[Row], inputType: TypeInformation[CRow], + rowTimeIdx: Int, queryConfig: StreamQueryConfig) extends RowTimeUnboundedOver( genAggregations: GeneratedAggregationsFunction, intermediateType, inputType, + rowTimeIdx, queryConfig) { override def processElementsWithSameTimestamp( diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala index 5f83f1d348f97..438f697d7750d 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala @@ -60,6 +60,8 @@ object SortUtil { inputTypeInfo: TypeInformation[Row], execCfg: ExecutionConfig): ProcessFunction[CRow, CRow] = { + val rowtimeIdx = collationSort.getFieldCollations.get(0).getFieldIndex + val collectionRowComparator = if (collationSort.getFieldCollations.size() > 1) { val rowComp = createRowComparator( @@ -76,6 +78,7 @@ object SortUtil { new RowTimeSortProcessFunction( inputCRowType, + rowtimeIdx, collectionRowComparator) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/TimeWindowPropertyCollector.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/TimeWindowPropertyCollector.scala index 0c8ae007a3f9c..4ec5239dcda17 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/TimeWindowPropertyCollector.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/TimeWindowPropertyCollector.scala @@ -29,7 +29,8 @@ import org.apache.flink.util.Collector */ abstract class TimeWindowPropertyCollector[T]( windowStartOffset: Option[Int], - windowEndOffset: Option[Int]) + windowEndOffset: Option[Int], + windowRowtimeOffset: Option[Int]) extends Collector[T] { var wrappedCollector: Collector[T] = _ @@ -55,20 +56,32 @@ abstract class TimeWindowPropertyCollector[T]( SqlFunctions.internalToTimestamp(windowEnd)) } + if (windowRowtimeOffset.isDefined) { + output.setField( + lastFieldPos + windowRowtimeOffset.get, + SqlFunctions.internalToTimestamp(windowEnd - 1)) + } + wrappedCollector.collect(record) } override def close(): Unit = wrappedCollector.close() } -class RowTimeWindowPropertyCollector(startOffset: Option[Int], endOffset: Option[Int]) - extends TimeWindowPropertyCollector[Row](startOffset, endOffset) { +class RowTimeWindowPropertyCollector( + startOffset: Option[Int], + endOffset: Option[Int], + rowtimeOffset: Option[Int]) + extends TimeWindowPropertyCollector[Row](startOffset, endOffset, rowtimeOffset) { override def getRow(record: Row): Row = record } -class CRowTimeWindowPropertyCollector(startOffset: Option[Int], endOffset: Option[Int]) - extends TimeWindowPropertyCollector[CRow](startOffset, endOffset) { +class CRowTimeWindowPropertyCollector( + startOffset: Option[Int], + endOffset: Option[Int], + rowtimeOffset: Option[Int]) + extends TimeWindowPropertyCollector[CRow](startOffset, endOffset, rowtimeOffset) { override def getRow(record: CRow): Row = record.row } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala index 379b8d2557226..e0af07736ff0e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala @@ -429,9 +429,8 @@ object WindowJoinUtil { |${generator.collectorTerm}.collect(${conversion.resultTerm}); |""".stripMargin case Some(remainCondition) => - // map logical field accesses to physical accesses - val physicalCondition = returnType.mapRexNode(remainCondition) - val genCond = generator.generateExpression(physicalCondition) + // generate code for remaining condition + val genCond = generator.generateExpression(remainCondition) s""" |${genCond.code} |if (${genCond.resultTerm}) { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/SortTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/SortTest.scala index a5a1319d0c96e..d20002a9077b4 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/SortTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/SortTest.scala @@ -41,7 +41,7 @@ class SortTest extends TableTestBase { unaryNode("DataStreamSort", streamTableNode(0), term("orderBy", "proctime ASC", "c ASC")), - term("select", "a", "TIME_MATERIALIZATION(proctime) AS proctime", "c")) + term("select", "a", "PROCTIME(proctime) AS proctime", "c")) streamUtil.verifySql(sqlQuery, expected) } @@ -57,7 +57,7 @@ class SortTest extends TableTestBase { unaryNode("DataStreamSort", streamTableNode(0), term("orderBy", "rowtime ASC, c ASC")), - term("select", "a", "TIME_MATERIALIZATION(rowtime) AS rowtime", "c")) + term("select", "a", "rowtime", "c")) streamUtil.verifySql(sqlQuery, expected) } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/TableSourceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/TableSourceTest.scala index 5d4386ca66202..696706159b4c9 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/TableSourceTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/TableSourceTest.scala @@ -43,7 +43,7 @@ class TableSourceTest extends TableTestBase { unaryNode( "DataStreamCalc", "StreamTableSourceScan(table=[[rowTimeT]], fields=[id, val, name, addTime])", - term("select", "TIME_MATERIALIZATION(addTime) AS addTime", "id", "name", "val") + term("select", "addTime", "id", "name", "val") ) util.verifyTable(t, expected) } @@ -90,7 +90,7 @@ class TableSourceTest extends TableTestBase { unaryNode( "DataStreamCalc", "StreamTableSourceScan(table=[[procTimeT]], fields=[id, val, name, pTime])", - term("select", "TIME_MATERIALIZATION(pTime) AS pTime", "id", "name", "val") + term("select", "PROCTIME(pTime) AS pTime", "id", "name", "val") ) util.verifyTable(t, expected) } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala index b17debea45367..ab80c65c1327a 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/TimeIndicatorConversionTest.scala @@ -48,7 +48,7 @@ class TimeIndicatorConversionTest extends TableTestBase { val expected = unaryNode( "DataStreamCalc", streamTableNode(0), - term("select", "FLOOR(TIME_MATERIALIZATION(rowtime)", "FLAG(DAY)) AS rowtime"), + term("select", "FLOOR(CAST(rowtime)", "FLAG(DAY)) AS rowtime"), term("where", ">(long, 0)") ) @@ -65,8 +65,8 @@ class TimeIndicatorConversionTest extends TableTestBase { val expected = unaryNode( "DataStreamCalc", streamTableNode(0), - term("select", "TIME_MATERIALIZATION(rowtime) AS rowtime", "long", "int", - "TIME_MATERIALIZATION(proctime) AS proctime") + term("select", "rowtime", "long", "int", + "PROCTIME(proctime) AS proctime") ) util.verifyTable(result, expected) @@ -84,7 +84,7 @@ class TimeIndicatorConversionTest extends TableTestBase { val expected = unaryNode( "DataStreamCalc", streamTableNode(0), - term("select", "TIME_MATERIALIZATION(rowtime) AS rowtime"), + term("select", "rowtime"), term("where", ">(rowtime, 1990-12-02 12:11:11)") ) @@ -107,7 +107,7 @@ class TimeIndicatorConversionTest extends TableTestBase { unaryNode( "DataStreamCalc", streamTableNode(0), - term("select", "long", "TIME_MATERIALIZATION(rowtime) AS rowtime") + term("select", "long", "CAST(rowtime) AS rowtime") ), term("groupBy", "rowtime"), term("select", "rowtime", "COUNT(long) AS TMP_0") @@ -134,7 +134,7 @@ class TimeIndicatorConversionTest extends TableTestBase { unaryNode( "DataStreamCalc", streamTableNode(0), - term("select", "TIME_MATERIALIZATION(rowtime) AS rowtime", "long") + term("select", "CAST(rowtime) AS rowtime", "long") ), term("groupBy", "long"), term("select", "long", "MIN(rowtime) AS TMP_0") @@ -159,16 +159,13 @@ class TimeIndicatorConversionTest extends TableTestBase { "DataStreamCorrelate", streamTableNode(0), term("invocation", - s"${func.functionIdentifier}(TIME_MATERIALIZATION($$0), TIME_MATERIALIZATION($$3), '')"), + s"${func.functionIdentifier}(CAST($$0):TIMESTAMP(3) NOT NULL, PROCTIME($$3), '')"), term("function", func), term("rowType", "RecordType(TIME ATTRIBUTE(ROWTIME) rowtime, BIGINT long, INTEGER int, " + "TIME ATTRIBUTE(PROCTIME) proctime, VARCHAR(65536) s)"), term("joinType", "INNER") ), - term("select", - "TIME_MATERIALIZATION(rowtime) AS rowtime", - "TIME_MATERIALIZATION(proctime) AS proctime", - "s") + term("select", "rowtime", "PROCTIME(proctime) AS proctime", "s") ) util.verifyTable(result, expected) @@ -219,7 +216,7 @@ class TimeIndicatorConversionTest extends TableTestBase { streamTableNode(0), term("union all", "rowtime", "long", "int") ), - term("select", "TIME_MATERIALIZATION(rowtime) AS rowtime") + term("select", "rowtime") ) util.verifyTable(result, expected) @@ -287,7 +284,7 @@ class TimeIndicatorConversionTest extends TableTestBase { unaryNode( "DataStreamCalc", streamTableNode(0), - term("select", "TIME_MATERIALIZATION(proctime) AS proctime", "long") + term("select", "PROCTIME(proctime) AS proctime", "long") ), term("groupBy", "proctime"), term("select", "proctime", "COUNT(long) AS EXPR$0") @@ -312,7 +309,7 @@ class TimeIndicatorConversionTest extends TableTestBase { unaryNode( "DataStreamCalc", streamTableNode(0), - term("select", "long", "TIME_MATERIALIZATION(proctime) AS proctime") + term("select", "long", "PROCTIME(proctime) AS proctime") ), term("groupBy", "long"), term("select", "long", "MIN(proctime) AS EXPR$0") @@ -368,7 +365,7 @@ class TimeIndicatorConversionTest extends TableTestBase { unaryNode( "DataStreamCalc", streamTableNode(0), - term("select", "long", "rowtime", "TIME_MATERIALIZATION(rowtime) AS $f2") + term("select", "long", "rowtime", "CAST(rowtime) AS rowtime0") ), term("groupBy", "long"), term( @@ -377,7 +374,7 @@ class TimeIndicatorConversionTest extends TableTestBase { 'w$, 'rowtime, 100.millis)), - term("select", "long", "MIN($f2) AS EXPR$0") + term("select", "long", "MIN(rowtime0) AS EXPR$0") ), term("select", "EXPR$0", "long") ) 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 80ff55e673cfb..04aada6ca738f 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 @@ -46,12 +46,10 @@ class HarnessTestBase { UserDefinedFunctionUtils.serialize(new IntSumWithRetractAggFunction) 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")) + Array("rowtime", "a", "b")) protected val SumRowType = new RowTypeInfo(Array[TypeInformation[_]]( LONG_TYPE_INFO, @@ -103,13 +101,13 @@ class HarnessTestBase { | | org.apache.flink.table.functions.AggregateFunction baseClass0 = | (org.apache.flink.table.functions.AggregateFunction) fmin; - | output.setField(5, baseClass0.getValue( + | output.setField(3, 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( + | output.setField(4, baseClass1.getValue( | (org.apache.flink.table.functions.aggfunctions.MaxWithRetractAccumulator) | accs.getField(1))); | } @@ -121,12 +119,12 @@ class HarnessTestBase { | fmin.accumulate( | ((org.apache.flink.table.functions.aggfunctions.MinWithRetractAccumulator) | accs.getField(0)), - | (java.lang.Long) input.getField(4)); + | (java.lang.Long) input.getField(2)); | | fmax.accumulate( | ((org.apache.flink.table.functions.aggfunctions.MaxWithRetractAccumulator) | accs.getField(1)), - | (java.lang.Long) input.getField(4)); + | (java.lang.Long) input.getField(2)); | } | | public void retract( @@ -136,12 +134,12 @@ class HarnessTestBase { | fmin.retract( | ((org.apache.flink.table.functions.aggfunctions.MinWithRetractAccumulator) | accs.getField(0)), - | (java.lang.Long) input.getField(4)); + | (java.lang.Long) input.getField(2)); | | fmax.retract( | ((org.apache.flink.table.functions.aggfunctions.MaxWithRetractAccumulator) | accs.getField(1)), - | (java.lang.Long) input.getField(4)); + | (java.lang.Long) input.getField(2)); | } | | public org.apache.flink.types.Row createAccumulators() { @@ -166,12 +164,10 @@ class HarnessTestBase { | 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); + | return new org.apache.flink.types.Row(5); | } | |/******* This test does not use the following methods *******/ @@ -326,7 +322,7 @@ object HarnessTestBase { /** * Return 0 for equal Rows and non zero for different rows */ - class RowResultSortComparator(indexCounter: Int) extends Comparator[Object] with Serializable { + class RowResultSortComparator() extends Comparator[Object] with Serializable { override def compare(o1: Object, o2: Object): Int = { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala index 6c24c5d9532fa..065b7bcc15e38 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/JoinHarnessTest.scala @@ -154,7 +154,7 @@ class JoinHarnessTest extends HarnessTestBase{ expectedOutput.add(new StreamRecord( CRow(Row.of(2: JInt, "bbb2", 2: JInt, "Hello2"), true), 25)) - verify(expectedOutput, result, new RowResultSortComparator(6)) + verify(expectedOutput, result, new RowResultSortComparator()) testHarness.close() } @@ -227,7 +227,7 @@ class JoinHarnessTest extends HarnessTestBase{ expectedOutput.add(new StreamRecord( CRow(Row.of(1: JInt, "aaa3", 1: JInt, "bbb12"), true), 12)) - verify(expectedOutput, result, new RowResultSortComparator(6)) + verify(expectedOutput, result, new RowResultSortComparator()) testHarness.close() } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala index 04214f9a178ef..dd14d7edaa561 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala @@ -91,7 +91,7 @@ class NonWindowHarnessTest extends HarnessTestBase { 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)) + verify(expectedOutput, result, new RowResultSortComparator()) testHarness.close() } @@ -150,7 +150,7 @@ class NonWindowHarnessTest extends HarnessTestBase { 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)) + verify(expectedOutput, result, new RowResultSortComparator()) testHarness.close() } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala index 8cad64f7280ae..ba044bec11bd9 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 @@ -17,14 +17,15 @@ */ package org.apache.flink.table.runtime.harness -import java.lang.{Integer => JInt, Long => JLong} +import java.lang.{Long => JLong} import java.util.concurrent.ConcurrentLinkedQueue +import org.apache.calcite.runtime.SqlFunctions.{internalToTimestamp => toTS} 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.api.{StreamQueryConfig, Types} import org.apache.flink.table.runtime.aggregate._ import org.apache.flink.table.runtime.harness.HarnessTestBase._ import org.apache.flink.table.runtime.types.CRow @@ -48,8 +49,10 @@ class OverWindowHarnessTest extends HarnessTestBase{ queryConfig)) val testHarness = - createHarnessTester(processFunction,new TupleRowKeySelector[Integer](0),BasicTypeInfo - .INT_TYPE_INFO) + createHarnessTester( + processFunction, + new TupleRowKeySelector[String](1), + Types.STRING) testHarness.open() @@ -57,91 +60,77 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.setProcessingTime(1) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 1)) + CRow(Row.of(toTS(1), "aaa", 1L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong), true), 1)) + CRow(Row.of(toTS(1), "bbb", 10L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong), true), 1)) + CRow(Row.of(toTS(1), "aaa", 2L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong), true), 1)) + CRow(Row.of(toTS(1), "aaa", 3L: JLong), true))) // 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)) + CRow(Row.of(toTS(1), "bbb", 20L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong), true), 1)) + CRow(Row.of(toTS(1), "aaa", 4L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong), true), 1)) + CRow(Row.of(toTS(1), "aaa", 5L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong), true), 1)) + CRow(Row.of(toTS(1), "aaa", 6L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), true), 1)) + CRow(Row.of(toTS(1), "bbb", 30L: JLong), true))) // 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)) + CRow(Row.of(toTS(2), "aaa", 7L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 2)) + CRow(Row.of(toTS(2), "aaa", 8L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 2)) + CRow(Row.of(toTS(2), "aaa", 9L: JLong), true))) // 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(toTS(2), "aaa", 10L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 2)) + CRow(Row.of(toTS(2), "bbb", 40L: JLong), true))) val result = testHarness.getOutput val expectedOutput = new ConcurrentLinkedQueue[Object]() expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true), 1)) + CRow(Row.of(toTS(1), "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true), 1)) + CRow(Row.of(toTS(1), "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true), 1)) + CRow(Row.of(toTS(1), "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 2L: JLong, 3L: JLong), true), 1)) + CRow(Row.of(toTS(1), "aaa", 3L: JLong, 2L: JLong, 3L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true), 1)) + CRow(Row.of(toTS(1), "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong, 3L: JLong, 4L: JLong), true), 1)) + CRow(Row.of(toTS(1), "aaa", 4L: JLong, 3L: JLong, 4L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 4L: JLong, 5L: JLong), true), 1)) + CRow(Row.of(toTS(1), "aaa", 5L: JLong, 4L: JLong, 5L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 5L: JLong, 6L: JLong), true), 1)) + CRow(Row.of(toTS(1), "aaa", 6L: JLong, 5L: JLong, 6L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 20L: JLong, 30L: JLong), true), 1)) + CRow(Row.of(toTS(1), "bbb", 30L: JLong, 20L: JLong, 30L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 6L: JLong, 7L: JLong), true), 2)) + CRow(Row.of(toTS(2), "aaa", 7L: JLong, 6L: JLong, 7L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 7L: JLong, 8L: JLong), true), 2)) + CRow(Row.of(toTS(2), "aaa", 8L: JLong, 7L: JLong, 8L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 8L: JLong, 9L: JLong), true), 2)) + CRow(Row.of(toTS(2), "aaa", 9L: JLong, 8L: JLong, 9L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 10L: JLong, 10L: JLong), true), 2)) + CRow(Row.of(toTS(2), "aaa", 10L: JLong, 10L: JLong, 10L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 2)) + CRow(Row.of(toTS(2), "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true))) - verify(expectedOutput, result, new RowResultSortComparator(6)) + verify(expectedOutput, result, new RowResultSortComparator()) testHarness.close() } @@ -163,59 +152,59 @@ class OverWindowHarnessTest extends HarnessTestBase{ val testHarness = createHarnessTester( processFunction, - new TupleRowKeySelector[Integer](0), - BasicTypeInfo.INT_TYPE_INFO) + new TupleRowKeySelector[String](1), + Types.STRING) 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)) + CRow(Row.of(toTS(0), "aaa", 1L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong), true), 0)) + CRow(Row.of(toTS(0), "bbb", 10L: JLong), true))) testHarness.setProcessingTime(4) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 2L: JLong), true))) // 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)) + CRow(Row.of(toTS(0), "aaa", 3L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong), true), 0)) + CRow(Row.of(toTS(0), "bbb", 20L: JLong), true))) testHarness.setProcessingTime(5) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 4L: JLong), true))) // register cleanup timer with 9002 testHarness.setProcessingTime(6002) testHarness.setProcessingTime(7002) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 5L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 6L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), true), 0)) + CRow(Row.of(toTS(0), "bbb", 30L: JLong), true))) // 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)) + CRow(Row.of(toTS(0), "aaa", 7L: JLong), true))) testHarness.setProcessingTime(11004) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 8L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 9L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 10L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 0)) + CRow(Row.of(toTS(0), "bbb", 40L: JLong), true))) testHarness.setProcessingTime(11006) @@ -225,49 +214,35 @@ class OverWindowHarnessTest extends HarnessTestBase{ // all elements at the same proc timestamp have the same value per key expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true), 4)) + CRow(Row.of(toTS(0), "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true), 4)) + CRow(Row.of(toTS(0), "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true), 5)) + CRow(Row.of(toTS(0), "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 3L: JLong, 4L: JLong), true), 3004)) + CRow(Row.of(toTS(0), "aaa", 3L: JLong, 3L: JLong, 4L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow(Row.of( - 2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong, 20L: JLong, 20L: JLong), true), 3004)) + CRow(Row.of(toTS(0), "bbb", 20L: JLong, 20L: JLong, 20L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong, 4L: JLong, 4L: JLong), true), 6)) + CRow(Row.of(toTS(0), "aaa", 4L: JLong, 4L: JLong, 4L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 5L: JLong, 6L: JLong), true), 7003)) + CRow(Row.of(toTS(0), "aaa", 5L: JLong, 5L: JLong, 6L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 5L: JLong, 6L: JLong), true), 7003)) + CRow(Row.of(toTS(0), "aaa", 6L: JLong, 5L: JLong, 6L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 30L: JLong, 30L: JLong), true), 7003)) + CRow(Row.of(toTS(0), "bbb", 30L: JLong, 30L: JLong, 30L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 7L: JLong, 7L: JLong), true), 11003)) + CRow(Row.of(toTS(0), "aaa", 7L: JLong, 7L: JLong, 7L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow(Row.of( - 1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 7L: JLong, 10L: JLong), true), 11005)) + CRow(Row.of(toTS(0), "aaa", 8L: JLong, 7L: JLong, 10L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow(Row.of( - 1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 7L: JLong, 10L: JLong), true), 11005)) + CRow(Row.of(toTS(0), "aaa", 9L: JLong, 7L: JLong, 10L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 7L: JLong, 10L: JLong), true), 11005)) + CRow(Row.of(toTS(0), "aaa", 10L: JLong, 7L: JLong, 10L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 11005)) + CRow(Row.of(toTS(0), "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true))) - verify(expectedOutput, result, new RowResultSortComparator(6)) + verify(expectedOutput, result, new RowResultSortComparator()) testHarness.close() } @@ -284,8 +259,8 @@ class OverWindowHarnessTest extends HarnessTestBase{ val testHarness = createHarnessTester( processFunction, - new TupleRowKeySelector[Integer](0), - BasicTypeInfo.INT_TYPE_INFO) + new TupleRowKeySelector[String](1), + Types.STRING) testHarness.open() @@ -293,85 +268,71 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.setProcessingTime(1003) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 1L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong), true), 0)) + CRow(Row.of(toTS(0), "bbb", 10L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 2L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 3L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong), true), 0)) + CRow(Row.of(toTS(0), "bbb", 20L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 4L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 5L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 6L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), true), 0)) + CRow(Row.of(toTS(0), "bbb", 30L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 7L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 8L: JLong), true))) // 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), 5003)) + CRow(Row.of(toTS(0), "aaa", 9L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 5003)) + CRow(Row.of(toTS(0), "aaa", 10L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 5003)) + CRow(Row.of(toTS(0), "bbb", 40L: JLong), true))) val result = testHarness.getOutput val expectedOutput = new ConcurrentLinkedQueue[Object]() expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true), 0)) + CRow(Row.of(toTS(0), "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true), 0)) + CRow(Row.of(toTS(0), "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong, 1L: JLong, 4L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 4L: JLong, 1L: JLong, 4L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 1L: JLong, 5L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 5L: JLong, 1L: JLong, 5L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 1L: JLong, 6L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 6L: JLong, 1L: JLong, 6L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true), 0)) + CRow(Row.of(toTS(0), "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 1L: JLong, 7L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 7L: JLong, 1L: JLong, 7L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 1L: JLong, 8L: JLong), true), 0)) + CRow(Row.of(toTS(0), "aaa", 8L: JLong, 1L: JLong, 8L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 9L: JLong, 9L: JLong), true), 5003)) + CRow(Row.of(toTS(0), "aaa", 9L: JLong, 9L: JLong, 9L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 9L: JLong, 10L: JLong), true), 5003)) + CRow(Row.of(toTS(0), "aaa", 10L: JLong, 9L: JLong, 10L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 5003)) + CRow(Row.of(toTS(0), "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true))) - verify(expectedOutput, result, new RowResultSortComparator(6)) + verify(expectedOutput, result, new RowResultSortComparator()) testHarness.close() } @@ -387,63 +348,64 @@ class OverWindowHarnessTest extends HarnessTestBase{ minMaxAggregationStateType, minMaxCRowType, 4000, + 0, new StreamQueryConfig().withIdleStateRetentionTime(Time.seconds(1), Time.seconds(2)))) val testHarness = createHarnessTester( processFunction, - new TupleRowKeySelector[String](3), + new TupleRowKeySelector[String](1), BasicTypeInfo.STRING_TYPE_INFO) testHarness.open() testHarness.processWatermark(1) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 2)) + CRow(Row.of(toTS(2), "aaa", 1L: JLong), true))) testHarness.processWatermark(2) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong), true), 3)) + CRow(Row.of(toTS(3), "bbb", 10L: JLong), true))) testHarness.processWatermark(4000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong), true), 4001)) + CRow(Row.of(toTS(4001), "aaa", 2L: JLong), true))) testHarness.processWatermark(4001) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong), true), 4002)) + CRow(Row.of(toTS(4002), "aaa", 3L: JLong), true))) testHarness.processWatermark(4002) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 0L: JLong, 0: JInt, "aaa", 4L: JLong), true), 4003)) + CRow(Row.of(toTS(4003), "aaa", 4L: JLong), true))) testHarness.processWatermark(4800) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 11L: JLong, 1: JInt, "bbb", 25L: JLong), true), 4801)) + CRow(Row.of(toTS(4801), "bbb", 25L: JLong), true))) testHarness.processWatermark(6500) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "aaa", 5L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "aaa", 6L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "bbb", 30L: JLong), true))) testHarness.processWatermark(7000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 7001)) + CRow(Row.of(toTS(7001), "aaa", 7L: JLong), true))) testHarness.processWatermark(8000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 8001)) + CRow(Row.of(toTS(8001), "aaa", 8L: JLong), true))) testHarness.processWatermark(12000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "aaa", 9L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "aaa", 10L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "bbb", 40L: JLong), true))) testHarness.processWatermark(19000) @@ -453,21 +415,22 @@ class OverWindowHarnessTest extends HarnessTestBase{ // 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 + CRow(Row.of(toTS(20001), "ccc", 1L: JLong), true))) // 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 + CRow(Row.of(toTS(20002), "ccc", 2L: JLong), true))) // 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) + val x = testHarness.numKeyedStateEntries() 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 + CRow(Row.of(toTS(20011), "ccc", 3L: JLong), true))) // clean-up 6500 assert(testHarness.numKeyedStateEntries() > 0) // check that we have state testHarness.setProcessingTime(6500) // clean-up attempt but rescheduled to 8500 @@ -487,59 +450,42 @@ class OverWindowHarnessTest extends HarnessTestBase{ // all elements at the same row-time have the same value per key expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true), 2)) + CRow(Row.of(toTS(2), "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true), 3)) + CRow(Row.of(toTS(3), "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true), 4001)) + CRow(Row.of(toTS(4001), "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true), 4002)) + CRow(Row.of(toTS(4002), "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 0L: JLong, 0: JInt, "aaa", 4L: JLong, 2L: JLong, 4L: JLong), true), 4003)) + CRow(Row.of(toTS(4003), "aaa", 4L: JLong, 2L: JLong, 4L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 11L: JLong, 1: JInt, "bbb", 25L: JLong, 25L: JLong, 25L: JLong), true), 4801)) + CRow(Row.of(toTS(4801), "bbb", 25L: JLong, 25L: JLong, 25L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 2L: JLong, 6L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "aaa", 5L: JLong, 2L: JLong, 6L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 2L: JLong, 6L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "aaa", 6L: JLong, 2L: JLong, 6L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 2L: JLong, 7L: JLong), true), 7001)) + CRow(Row.of(toTS(7001), "aaa", 7L: JLong, 2L: JLong, 7L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 2L: JLong, 8L: JLong), true), 8001)) + CRow(Row.of(toTS(8001), "aaa", 8L: JLong, 2L: JLong, 8L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 25L: JLong, 30L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "bbb", 30L: JLong, 25L: JLong, 30L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 8L: JLong, 10L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "aaa", 9L: JLong, 8L: JLong, 10L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 8L: JLong, 10L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "aaa", 10L: JLong, 8L: JLong, 10L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 0L: JLong, 0: JInt, "ccc", 1L: JLong, 1L: JLong, 1L: JLong), true), 20001)) + CRow(Row.of(toTS(20001), "ccc", 1L: JLong, 1L: JLong, 1L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "ccc", 2L: JLong, 1L: JLong, 2L: JLong), true), 20002)) + CRow(Row.of(toTS(20002), "ccc", 2L: JLong, 1L: JLong, 2L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(3: JInt, 0L: JLong, 0: JInt, "ccc", 3L: JLong, 3L: JLong, 3L: JLong), true), 20011)) + CRow(Row.of(toTS(20011), "ccc", 3L: JLong, 3L: JLong, 3L: JLong), true))) - verify(expectedOutput, result, new RowResultSortComparator(6)) + verify(expectedOutput, result, new RowResultSortComparator()) testHarness.close() } @@ -552,59 +498,60 @@ class OverWindowHarnessTest extends HarnessTestBase{ minMaxAggregationStateType, minMaxCRowType, 3, + 0, new StreamQueryConfig().withIdleStateRetentionTime(Time.seconds(1), Time.seconds(2)))) val testHarness = createHarnessTester( processFunction, - new TupleRowKeySelector[String](3), + new TupleRowKeySelector[String](1), BasicTypeInfo.STRING_TYPE_INFO) testHarness.open() testHarness.processWatermark(800) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 801)) + CRow(Row.of(toTS(801), "aaa", 1L: JLong), true))) testHarness.processWatermark(2500) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong), true), 2501)) + CRow(Row.of(toTS(2501), "bbb", 10L: JLong), true))) testHarness.processWatermark(4000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong), true), 4001)) + CRow(Row.of(toTS(4001), "aaa", 2L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong), true), 4001)) + CRow(Row.of(toTS(4001), "aaa", 3L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong), true), 4001)) + CRow(Row.of(toTS(4001), "bbb", 20L: JLong), true))) testHarness.processWatermark(4800) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong), true), 4801)) + CRow(Row.of(toTS(4801), "aaa", 4L: JLong), true))) testHarness.processWatermark(6500) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "aaa", 5L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "aaa", 6L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "bbb", 30L: JLong), true))) testHarness.processWatermark(7000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 7001)) + CRow(Row.of(toTS(7001), "aaa", 7L: JLong), true))) testHarness.processWatermark(8000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 8001)) + CRow(Row.of(toTS(8001), "aaa", 8L: JLong), true))) testHarness.processWatermark(12000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "aaa", 9L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "aaa", 10L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "bbb", 40L: JLong), true))) testHarness.processWatermark(19000) @@ -614,10 +561,10 @@ class OverWindowHarnessTest extends HarnessTestBase{ // 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 + CRow(Row.of(toTS(20001), "ccc", 1L: JLong), true))) // 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 + CRow(Row.of(toTS(20002), "ccc", 2L: JLong), true))) // clean-up 4500 testHarness.processWatermark(20010) // compute output assert(testHarness.numKeyedStateEntries() > 0) // check that we have state @@ -628,7 +575,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ // 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 + CRow(Row.of(toTS(20011), "ccc", 3L: JLong), true))) // clean-up 6500 assert(testHarness.numKeyedStateEntries() > 0) // check that we have state testHarness.setProcessingTime(6500) // clean-up attempt but rescheduled to 8500 @@ -648,59 +595,42 @@ class OverWindowHarnessTest extends HarnessTestBase{ val expectedOutput = new ConcurrentLinkedQueue[Object]() expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true), 801)) + CRow(Row.of(toTS(801), "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true), 2501)) + CRow(Row.of(toTS(2501), "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true), 4001)) + CRow(Row.of(toTS(4001), "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true), 4001)) + CRow(Row.of(toTS(4001), "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true), 4001)) + CRow(Row.of(toTS(4001), "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong, 2L: JLong, 4L: JLong), true), 4801)) + CRow(Row.of(toTS(4801), "aaa", 4L: JLong, 2L: JLong, 4L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 3L: JLong, 5L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "aaa", 5L: JLong, 3L: JLong, 5L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 4L: JLong, 6L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "aaa", 6L: JLong, 4L: JLong, 6L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 5L: JLong, 7L: JLong), true), 7001)) + CRow(Row.of(toTS(7001), "aaa", 7L: JLong, 5L: JLong, 7L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 6L: JLong, 8L: JLong), true), 8001)) + CRow(Row.of(toTS(8001), "aaa", 8L: JLong, 6L: JLong, 8L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 7L: JLong, 9L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "aaa", 9L: JLong, 7L: JLong, 9L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 8L: JLong, 10L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "aaa", 10L: JLong, 8L: JLong, 10L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 20L: JLong, 40L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "bbb", 40L: JLong, 20L: JLong, 40L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 0L: JLong, 0: JInt, "ccc", 1L: JLong, 1L: JLong, 1L: JLong), true), 20001)) + CRow(Row.of(toTS(20001), "ccc", 1L: JLong, 1L: JLong, 1L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "ccc", 2L: JLong, 1L: JLong, 2L: JLong), true), 20002)) + CRow(Row.of(toTS(20002), "ccc", 2L: JLong, 1L: JLong, 2L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(3: JInt, 0L: JLong, 0: JInt, "ccc", 3L: JLong, 3L: JLong, 3L: JLong), true), 20011)) + CRow(Row.of(toTS(20011), "ccc", 3L: JLong, 3L: JLong, 3L: JLong), true))) - verify(expectedOutput, result, new RowResultSortComparator(6)) + verify(expectedOutput, result, new RowResultSortComparator()) testHarness.close() } @@ -715,12 +645,13 @@ class OverWindowHarnessTest extends HarnessTestBase{ genMinMaxAggFunction, minMaxAggregationStateType, minMaxCRowType, + 0, new StreamQueryConfig().withIdleStateRetentionTime(Time.seconds(1), Time.seconds(2)))) val testHarness = createHarnessTester( processFunction, - new TupleRowKeySelector[String](3), + new TupleRowKeySelector[String](1), BasicTypeInfo.STRING_TYPE_INFO) testHarness.open() @@ -728,47 +659,47 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.setProcessingTime(1000) testHarness.processWatermark(800) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 801)) + CRow(Row.of(toTS(801), "aaa", 1L: JLong), true))) testHarness.processWatermark(2500) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong), true), 2501)) + CRow(Row.of(toTS(2501), "bbb", 10L: JLong), true))) testHarness.processWatermark(4000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong), true), 4001)) + CRow(Row.of(toTS(4001), "aaa", 2L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong), true), 4001)) + CRow(Row.of(toTS(4001), "aaa", 3L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong), true), 4001)) + CRow(Row.of(toTS(4001), "bbb", 20L: JLong), true))) testHarness.processWatermark(4800) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong), true), 4801)) + CRow(Row.of(toTS(4801), "aaa", 4L: JLong), true))) testHarness.processWatermark(6500) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "aaa", 5L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "aaa", 6L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "bbb", 30L: JLong), true))) testHarness.processWatermark(7000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 7001)) + CRow(Row.of(toTS(7001), "aaa", 7L: JLong), true))) testHarness.processWatermark(8000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 8001)) + CRow(Row.of(toTS(8001), "aaa", 8L: JLong), true))) testHarness.processWatermark(12000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "aaa", 9L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "aaa", 10L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "bbb", 40L: JLong), true))) testHarness.processWatermark(19000) @@ -781,10 +712,10 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processWatermark(20000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 0L: JLong, 0: JInt, "ccc", 1L: JLong), true), 20001)) // clean-up 5000 + CRow(Row.of(toTS(20001), "ccc", 1L: JLong), true))) // 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 + CRow(Row.of(toTS(20002), "ccc", 2L: JLong), true))) // clean-up 5000 assert(testHarness.numKeyedStateEntries() > 0) testHarness.setProcessingTime(5000) // does not clean up, because data left. New timer 7000 @@ -802,56 +733,40 @@ class OverWindowHarnessTest extends HarnessTestBase{ // all elements at the same row-time have the same value per key expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true), 801)) + CRow(Row.of(toTS(801), "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true), 2501)) + CRow(Row.of(toTS(2501), "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 1L: JLong, 3L: JLong), true), 4001)) + CRow(Row.of(toTS(4001), "aaa", 2L: JLong, 1L: JLong, 3L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true), 4001)) + CRow(Row.of(toTS(4001), "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true), 4001)) + CRow(Row.of(toTS(4001), "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong, 1L: JLong, 4L: JLong), true), 4801)) + CRow(Row.of(toTS(4801), "aaa", 4L: JLong, 1L: JLong, 4L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 1L: JLong, 6L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "aaa", 5L: JLong, 1L: JLong, 6L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 1L: JLong, 6L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "aaa", 6L: JLong, 1L: JLong, 6L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 1L: JLong, 7L: JLong), true), 7001)) + CRow(Row.of(toTS(7001), "aaa", 7L: JLong, 1L: JLong, 7L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 1L: JLong, 8L: JLong), true), 8001)) + CRow(Row.of(toTS(8001), "aaa", 8L: JLong, 1L: JLong, 8L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 1L: JLong, 10L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "aaa", 9L: JLong, 1L: JLong, 10L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 1L: JLong, 10L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "aaa", 10L: JLong, 1L: JLong, 10L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 10L: JLong, 40L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "bbb", 40L: JLong, 10L: JLong, 40L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 0L: JLong, 0: JInt, "ccc", 1L: JLong, 1L: JLong, 1L: JLong), true), 20001)) + CRow(Row.of(toTS(20001), "ccc", 1L: JLong, 1L: JLong, 1L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "ccc", 2L: JLong, 1L: JLong, 2L: JLong), true), 20002)) + CRow(Row.of(toTS(20002), "ccc", 2L: JLong, 1L: JLong, 2L: JLong), true))) - verify(expectedOutput, result, new RowResultSortComparator(6)) + verify(expectedOutput, result, new RowResultSortComparator()) testHarness.close() } @@ -863,12 +778,13 @@ class OverWindowHarnessTest extends HarnessTestBase{ genMinMaxAggFunction, minMaxAggregationStateType, minMaxCRowType, + 0, new StreamQueryConfig().withIdleStateRetentionTime(Time.seconds(1), Time.seconds(2)))) val testHarness = createHarnessTester( processFunction, - new TupleRowKeySelector[String](3), + new TupleRowKeySelector[String](1), BasicTypeInfo.STRING_TYPE_INFO) testHarness.open() @@ -876,47 +792,47 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.setProcessingTime(1000) testHarness.processWatermark(800) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong), true), 801)) + CRow(Row.of(toTS(801), "aaa", 1L: JLong), true))) testHarness.processWatermark(2500) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong), true), 2501)) + CRow(Row.of(toTS(2501), "bbb", 10L: JLong), true))) testHarness.processWatermark(4000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong), true), 4001)) + CRow(Row.of(toTS(4001), "aaa", 2L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong), true), 4001)) + CRow(Row.of(toTS(4001), "aaa", 3L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong), true), 4001)) + CRow(Row.of(toTS(4001), "bbb", 20L: JLong), true))) testHarness.processWatermark(4800) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong), true), 4801)) + CRow(Row.of(toTS(4801), "aaa", 4L: JLong), true))) testHarness.processWatermark(6500) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "aaa", 5L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "aaa", 6L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "bbb", 30L: JLong), true))) testHarness.processWatermark(7000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong), true), 7001)) + CRow(Row.of(toTS(7001), "aaa", 7L: JLong), true))) testHarness.processWatermark(8000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong), true), 8001)) + CRow(Row.of(toTS(8001), "aaa", 8L: JLong), true))) testHarness.processWatermark(12000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "aaa", 9L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "aaa", 10L: JLong), true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "bbb", 40L: JLong), true))) testHarness.processWatermark(19000) @@ -929,10 +845,10 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processWatermark(20000) testHarness.processElement(new StreamRecord( - CRow(Row.of(1: JInt, 0L: JLong, 0: JInt, "ccc", 1L: JLong), true), 20001)) // clean-up 5000 + CRow(Row.of(toTS(20001), "ccc", 1L: JLong), true))) // 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 + CRow(Row.of(toTS(20002), "ccc", 2L: JLong), true))) // clean-up 5000 assert(testHarness.numKeyedStateEntries() > 0) testHarness.setProcessingTime(5000) // does not clean up, because data left. New timer 7000 @@ -949,56 +865,40 @@ class OverWindowHarnessTest extends HarnessTestBase{ val expectedOutput = new ConcurrentLinkedQueue[Object]() expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true), 801)) + CRow(Row.of(toTS(801), "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true), 2501)) + CRow(Row.of(toTS(2501), "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true), 4001)) + CRow(Row.of(toTS(4001), "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true), 4001)) + CRow(Row.of(toTS(4001), "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true), 4001)) + CRow(Row.of(toTS(4001), "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 4L: JLong, 1L: JLong, 4L: JLong), true), 4801)) + CRow(Row.of(toTS(4801), "aaa", 4L: JLong, 1L: JLong, 4L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 5L: JLong, 1L: JLong, 5L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "aaa", 5L: JLong, 1L: JLong, 5L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 6L: JLong, 1L: JLong, 6L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "aaa", 6L: JLong, 1L: JLong, 6L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true), 6501)) + CRow(Row.of(toTS(6501), "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 7L: JLong, 1L: JLong, 7L: JLong), true), 7001)) + CRow(Row.of(toTS(7001), "aaa", 7L: JLong, 1L: JLong, 7L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 8L: JLong, 1L: JLong, 8L: JLong), true), 8001)) + CRow(Row.of(toTS(8001), "aaa", 8L: JLong, 1L: JLong, 8L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 9L: JLong, 1L: JLong, 9L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "aaa", 9L: JLong, 1L: JLong, 9L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 10L: JLong, 1L: JLong, 10L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "aaa", 10L: JLong, 1L: JLong, 10L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "bbb", 40L: JLong, 10L: JLong, 40L: JLong), true), 12001)) + CRow(Row.of(toTS(12001), "bbb", 40L: JLong, 10L: JLong, 40L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(1: JInt, 0L: JLong, 0: JInt, "ccc", 1L: JLong, 1L: JLong, 1L: JLong), true), 20001)) + CRow(Row.of(toTS(20001), "ccc", 1L: JLong, 1L: JLong, 1L: JLong), true))) expectedOutput.add(new StreamRecord( - CRow( - Row.of(2: JInt, 0L: JLong, 0: JInt, "ccc", 2L: JLong, 1L: JLong, 2L: JLong), true), 20002)) + CRow(Row.of(toTS(20002), "ccc", 2L: JLong, 1L: JLong, 2L: JLong), true))) - verify(expectedOutput, result, new RowResultSortComparator(6)) + verify(expectedOutput, result, new RowResultSortComparator()) testHarness.close() } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/SortProcessFunctionHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/SortProcessFunctionHarnessTest.scala index 0451534d2b36a..18ba6bbe64e36 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/SortProcessFunctionHarnessTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/SortProcessFunctionHarnessTest.scala @@ -21,6 +21,7 @@ package org.apache.flink.table.runtime.harness import java.lang.{Integer => JInt, Long => JLong} import java.util.concurrent.ConcurrentLinkedQueue +import org.apache.calcite.runtime.SqlFunctions.{internalToTimestamp => toTS} import org.apache.flink.api.common.typeinfo.BasicTypeInfo._ import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer} @@ -35,6 +36,7 @@ import org.apache.flink.streaming.util.{KeyedOneInputStreamOperatorTestHarness, import org.apache.flink.table.runtime.aggregate.{CollectionRowComparator, ProcTimeSortProcessFunction, RowTimeSortProcessFunction} import org.apache.flink.table.runtime.harness.SortProcessFunctionHarnessTest.TupleRowSelector import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo import org.apache.flink.types.Row import org.junit.Test @@ -75,7 +77,7 @@ class SortProcessFunctionHarnessTest { inputCRowType, collectionRowComparator)) - val testHarness = new KeyedOneInputStreamOperatorTestHarness[Integer,CRow,CRow]( + val testHarness = new KeyedOneInputStreamOperatorTestHarness[Integer, CRow, CRow]( processFunction, new TupleRowSelector(0), BasicTypeInfo.INT_TYPE_INFO) @@ -86,77 +88,77 @@ class SortProcessFunctionHarnessTest { // timestamp is ignored in processing time testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 11L: JLong), true), 1001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 11L: JLong), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 12L: JLong, 1: JInt, "aaa", 11L: JLong), true), 2002)) + Row.of(1: JInt, 12L: JLong, 1: JInt, "aaa", 11L: JLong), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 12L: JLong, 2: JInt, "aaa", 11L: JLong), true), 2003)) + Row.of(1: JInt, 12L: JLong, 2: JInt, "aaa", 11L: JLong), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 12L: JLong, 0: JInt, "aaa", 11L: JLong), true), 2004)) + Row.of(1: JInt, 12L: JLong, 0: JInt, "aaa", 11L: JLong), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", 11L: JLong), true), 2006)) + Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", 11L: JLong), true))) //move the timestamp to ensure the execution testHarness.setProcessingTime(1005) - + testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 1L: JLong, 0: JInt, "aaa", 11L: JLong), true), 2007)) + Row.of(1: JInt, 1L: JLong, 0: JInt, "aaa", 11L: JLong), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 3L: JLong, 0: JInt, "aaa", 11L: JLong), true), 2007)) + Row.of(1: JInt, 3L: JLong, 0: JInt, "aaa", 11L: JLong), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 2L: JLong, 0: JInt, "aaa", 11L: JLong), true), 2007)) - + Row.of(1: JInt, 2L: JLong, 0: JInt, "aaa", 11L: JLong), true))) + testHarness.setProcessingTime(1008) - + val result = testHarness.getOutput - + val expectedOutput = new ConcurrentLinkedQueue[Object]() - + // all elements at the same proc timestamp have the same value // elements should be sorted ascending on field 1 and descending on field 2 // (10,0) (11,1) (12,2) (12,1) (12,0) // (1,0) (2,0) - + expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", 11L: JLong),true), 4)) + Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", 11L: JLong),true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 11L: JLong),true), 4)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 11L: JLong),true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 12L: JLong, 2: JInt, "aaa", 11L: JLong),true), 4)) + Row.of(1: JInt, 12L: JLong, 2: JInt, "aaa", 11L: JLong),true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 12L: JLong, 1: JInt, "aaa", 11L: JLong),true), 4)) + Row.of(1: JInt, 12L: JLong, 1: JInt, "aaa", 11L: JLong),true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 12L: JLong, 0: JInt, "aaa", 11L: JLong),true), 4)) - + Row.of(1: JInt, 12L: JLong, 0: JInt, "aaa", 11L: JLong),true))) + expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 1L: JLong, 0: JInt, "aaa", 11L: JLong),true), 1006)) + Row.of(1: JInt, 1L: JLong, 0: JInt, "aaa", 11L: JLong),true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 2L: JLong, 0: JInt, "aaa", 11L: JLong),true), 1006)) + Row.of(1: JInt, 2L: JLong, 0: JInt, "aaa", 11L: JLong),true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 3L: JLong, 0: JInt, "aaa", 11L: JLong),true), 1006)) + Row.of(1: JInt, 3L: JLong, 0: JInt, "aaa", 11L: JLong),true))) TestHarnessUtil.assertOutputEquals("Output was not correctly sorted.", expectedOutput, result) - + testHarness.close() } - + @Test def testSortRowTimeHarnessPartitioned(): Unit = { - + val rT = new RowTypeInfo(Array[TypeInformation[_]]( INT_TYPE_INFO, LONG_TYPE_INFO, INT_TYPE_INFO, STRING_TYPE_INFO, - LONG_TYPE_INFO), + TimeIndicatorTypeInfo.ROWTIME_INDICATOR), Array("a", "b", "c", "d", "e")) val indexes = Array(1, 2) - + val fieldComps = Array[TypeComparator[AnyRef]]( LONG_TYPE_INFO.createComparator(true, null).asInstanceOf[TypeComparator[AnyRef]], INT_TYPE_INFO.createComparator(false, null).asInstanceOf[TypeComparator[AnyRef]] ) - val booleanOrders = Array(true, false) + val booleanOrders = Array(true, false) val rowComp = new RowComparator( rT.getTotalFields, @@ -164,21 +166,22 @@ class SortProcessFunctionHarnessTest { fieldComps, new Array[TypeSerializer[AnyRef]](0), //used only for serialized comparisons booleanOrders) - + val collectionRowComparator = new CollectionRowComparator(rowComp) - + val inputCRowType = CRowTypeInfo(rT) - + val processFunction = new KeyedProcessOperator[Integer,CRow,CRow]( new RowTimeSortProcessFunction( inputCRowType, + 4, Some(collectionRowComparator))) - + val testHarness = new KeyedOneInputStreamOperatorTestHarness[Integer, CRow, CRow]( - processFunction, - new TupleRowSelector(0), + processFunction, + new TupleRowSelector(0), BasicTypeInfo.INT_TYPE_INFO) - + testHarness.open() testHarness.setTimeCharacteristic(TimeCharacteristic.EventTime) @@ -186,71 +189,71 @@ class SortProcessFunctionHarnessTest { // timestamp is ignored in processing time testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 11L: JLong), true), 1001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", toTS(1001)), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 12L: JLong, 1: JInt, "aaa", 11L: JLong), true), 2002)) + Row.of(1: JInt, 12L: JLong, 1: JInt, "aaa", toTS(2002)), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 13L: JLong, 2: JInt, "aaa", 11L: JLong), true), 2002)) + Row.of(1: JInt, 13L: JLong, 2: JInt, "aaa", toTS(2002)), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 12L: JLong, 3: JInt, "aaa", 11L: JLong), true), 2002)) + Row.of(1: JInt, 12L: JLong, 3: JInt, "aaa", toTS(2002)), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 14L: JLong, 0: JInt, "aaa", 11L: JLong), true), 2002)) + Row.of(1: JInt, 14L: JLong, 0: JInt, "aaa", toTS(2002)), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 12L: JLong, 3: JInt, "aaa", 11L: JLong), true), 2004)) + Row.of(1: JInt, 12L: JLong, 3: JInt, "aaa", toTS(2004)), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", 11L: JLong), true), 2006)) + Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", toTS(2006)), true))) // move watermark forward testHarness.processWatermark(2007) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 20L: JLong, 1: JInt, "aaa", 11L: JLong), true), 2008)) + Row.of(1: JInt, 20L: JLong, 1: JInt, "aaa", toTS(2008)), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 14L: JLong, 0: JInt, "aaa", 11L: JLong), true), 2002)) // too late + Row.of(1: JInt, 14L: JLong, 0: JInt, "aaa", toTS(2002)), true))) // too late testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 12L: JLong, 3: JInt, "aaa", 11L: JLong), true), 2019)) // too early + Row.of(1: JInt, 12L: JLong, 3: JInt, "aaa", toTS(2019)), true))) // too early testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 20L: JLong, 2: JInt, "aaa", 11L: JLong), true), 2008)) + Row.of(1: JInt, 20L: JLong, 2: JInt, "aaa", toTS(2008)), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", 11L: JLong), true), 2010)) + Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", toTS(2010)), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 19L: JLong, 0: JInt, "aaa", 11L: JLong), true), 2008)) + Row.of(1: JInt, 19L: JLong, 0: JInt, "aaa", toTS(2008)), true))) // move watermark forward testHarness.processWatermark(2012) val result = testHarness.getOutput - + val expectedOutput = new ConcurrentLinkedQueue[Object]() - + // all elements at the same proc timestamp have the same value // elements should be sorted ascending on field 1 and descending on field 2 // (10,0) (11,1) (12,2) (12,1) (12,0) expectedOutput.add(new Watermark(3)) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 11L: JLong),true), 1001)) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", toTS(1001)), true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 12L: JLong, 3: JInt, "aaa", 11L: JLong),true), 2002)) + Row.of(1: JInt, 12L: JLong, 3: JInt, "aaa", toTS(2002)), true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 12L: JLong, 1: JInt, "aaa", 11L: JLong),true), 2002)) + Row.of(1: JInt, 12L: JLong, 1: JInt, "aaa", toTS(2002)), true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 13L: JLong, 2: JInt, "aaa", 11L: JLong),true), 2002)) + Row.of(1: JInt, 13L: JLong, 2: JInt, "aaa", toTS(2002)), true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 14L: JLong, 0: JInt, "aaa", 11L: JLong),true), 2002)) + Row.of(1: JInt, 14L: JLong, 0: JInt, "aaa", toTS(2002)), true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 12L: JLong, 3: JInt, "aaa", 11L: JLong),true), 2004)) + Row.of(1: JInt, 12L: JLong, 3: JInt, "aaa", toTS(2004)), true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", 11L: JLong),true), 2006)) + Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", toTS(2006)), true))) expectedOutput.add(new Watermark(2007)) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 19L: JLong, 0: JInt, "aaa", 11L: JLong), true), 2008)) + Row.of(1: JInt, 19L: JLong, 0: JInt, "aaa", toTS(2008)), true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 20L: JLong, 2: JInt, "aaa", 11L: JLong), true), 2008)) + Row.of(1: JInt, 20L: JLong, 2: JInt, "aaa", toTS(2008)), true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 20L: JLong, 1: JInt, "aaa", 11L: JLong), true), 2008)) + Row.of(1: JInt, 20L: JLong, 1: JInt, "aaa", toTS(2008)), true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", 11L: JLong), true), 2010)) + Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", toTS(2010)), true))) expectedOutput.add(new Watermark(2012)) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala index 4121754336169..82ed81c3e017c 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala @@ -20,6 +20,7 @@ package org.apache.flink.table.runtime.stream.table import java.io.File import java.lang.{Boolean => JBool} +import java.sql.Timestamp import org.apache.flink.api.common.functions.MapFunction import org.apache.flink.api.common.typeinfo.TypeInformation @@ -28,19 +29,22 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo 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.functions.ProcessFunction import org.apache.flink.streaming.api.functions.sink.SinkFunction import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase import org.apache.flink.table.api.scala._ -import org.apache.flink.table.api.TableEnvironment -import org.apache.flink.table.runtime.utils.StreamTestData +import org.apache.flink.table.api.{TableEnvironment, TableException, Types} +import org.apache.flink.table.runtime.utils.{StreamITCase, StreamTestData} import org.apache.flink.table.sinks._ import org.apache.flink.test.util.TestBaseUtils import org.apache.flink.types.Row +import org.apache.flink.util.Collector import org.junit.Assert._ import org.junit.Test import scala.collection.mutable +import scala.collection.JavaConverters._ class TableSinkITCase extends StreamingMultipleProgramsTestBase { @@ -199,8 +203,6 @@ class TableSinkITCase extends StreamingMultipleProgramsTestBase { } - - @Test def testUpsertSinkOnAppendingTableWithFullKey1(): Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment @@ -349,6 +351,136 @@ class TableSinkITCase extends StreamingMultipleProgramsTestBase { assertEquals(expected, retracted) } + @Test + def testToAppendStreamRowtime(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + val tEnv = TableEnvironment.getTableEnvironment(env) + StreamITCase.clear + + val t = StreamTestData.get3TupleDataStream(env) + .assignAscendingTimestamps(_._1.toLong) + .toTable(tEnv, 'id, 'num, 'text, 'rowtime.rowtime) + + val r = t + .window(Tumble over 5.milli on 'rowtime as 'w) + .groupBy('num, 'w) + .select('num, 'w.rowtime, 'w.rowtime.cast(Types.LONG)) + + r.toAppendStream[Row] + .process(new ProcessFunction[Row, Row] { + override def processElement( + row: Row, + ctx: ProcessFunction[Row, Row]#Context, + out: Collector[Row]): Unit = { + + val rowTS: Long = row.getField(2).asInstanceOf[Long] + if (ctx.timestamp() == rowTS) { + out.collect(row) + } + } + }).addSink(new StreamITCase.StringSink[Row]) + + env.execute() + + val expected = List( + "1,1970-01-01 00:00:00.004,4", + "2,1970-01-01 00:00:00.004,4", + "3,1970-01-01 00:00:00.004,4", + "3,1970-01-01 00:00:00.009,9", + "4,1970-01-01 00:00:00.009,9", + "4,1970-01-01 00:00:00.014,14", + "5,1970-01-01 00:00:00.014,14", + "5,1970-01-01 00:00:00.019,19", + "6,1970-01-01 00:00:00.019,19", + "6,1970-01-01 00:00:00.024,24") + + assertEquals(expected, StreamITCase.testResults.sorted) + } + + @Test + def testToRetractStreamRowtime(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + val tEnv = TableEnvironment.getTableEnvironment(env) + StreamITCase.clear + + val t = StreamTestData.get3TupleDataStream(env) + .assignAscendingTimestamps(_._1.toLong) + .toTable(tEnv, 'id, 'num, 'text, 'rowtime.rowtime) + + val r = t + .window(Tumble over 5.milli on 'rowtime as 'w) + .groupBy('num, 'w) + .select('num, 'w.rowtime, 'w.rowtime.cast(Types.LONG)) + + r.toRetractStream[Row] + .process(new ProcessFunction[(Boolean, Row), Row] { + override def processElement( + row: (Boolean, Row), + ctx: ProcessFunction[(Boolean, Row), Row]#Context, + out: Collector[Row]): Unit = { + + val rowTS: Long = row._2.getField(2).asInstanceOf[Long] + if (ctx.timestamp() == rowTS) { + out.collect(row._2) + } + } + }).addSink(new StreamITCase.StringSink[Row]) + + env.execute() + + val expected = List( + "1,1970-01-01 00:00:00.004,4", + "2,1970-01-01 00:00:00.004,4", + "3,1970-01-01 00:00:00.004,4", + "3,1970-01-01 00:00:00.009,9", + "4,1970-01-01 00:00:00.009,9", + "4,1970-01-01 00:00:00.014,14", + "5,1970-01-01 00:00:00.014,14", + "5,1970-01-01 00:00:00.019,19", + "6,1970-01-01 00:00:00.019,19", + "6,1970-01-01 00:00:00.024,24") + + assertEquals(expected, StreamITCase.testResults.sorted) + } + + @Test(expected = classOf[TableException]) + def testToAppendStreamMultiRowtime(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + val tEnv = TableEnvironment.getTableEnvironment(env) + + val t = StreamTestData.get3TupleDataStream(env) + .assignAscendingTimestamps(_._1.toLong) + .toTable(tEnv, 'id, 'num, 'text, 'rowtime.rowtime) + + val r = t + .window(Tumble over 5.milli on 'rowtime as 'w) + .groupBy('num, 'w) + .select('num, 'w.rowtime, 'w.rowtime as 'rowtime2) + + r.toAppendStream[Row] + } + + @Test(expected = classOf[TableException]) + def testToRetractStreamMultiRowtime(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + val tEnv = TableEnvironment.getTableEnvironment(env) + + val t = StreamTestData.get3TupleDataStream(env) + .assignAscendingTimestamps(_._1.toLong) + .toTable(tEnv, 'id, 'num, 'text, 'rowtime.rowtime) + + val r = t + .window(Tumble over 5.milli on 'rowtime as 'w) + .groupBy('num, 'w) + .select('num, 'w.rowtime, 'w.rowtime as 'rowtime2) + + r.toRetractStream[Row] + } + /** Converts a list of retraction messages into a list of final results. */ private def restractResults(results: List[JTuple2[JBool, Row]]): List[String] = { From 876369d41019b3ec5ba824553b31fb2f3b44a18d Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Mon, 7 Aug 2017 23:39:48 +0200 Subject: [PATCH 2/7] Addressed review feedback --- .../table/api/StreamTableEnvironment.scala | 37 ++++------ .../flink/table/api/TableEnvironment.scala | 8 +-- .../calcite/RelTimeIndicatorConverter.scala | 12 ++-- .../flink/table/codegen/CodeGenerator.scala | 71 +++++++++++-------- .../table/codegen/calls/ScalarOperators.scala | 21 ++++-- ...nction.scala => ProctimeSqlFunction.scala} | 2 +- .../flink/table/plan/nodes/CommonCalc.scala | 6 +- .../table/plan/nodes/CommonCorrelate.scala | 16 ++--- .../nodes/datastream/DataStreamCalc.scala | 8 +-- .../datastream/DataStreamCorrelate.scala | 6 +- .../datastream/DataStreamGroupAggregate.scala | 22 +++--- .../DataStreamGroupWindowAggregate.scala | 42 +++++------ .../datastream/DataStreamOverAggregate.scala | 38 +++++----- .../nodes/datastream/DataStreamScan.scala | 2 +- .../nodes/datastream/DataStreamSort.scala | 20 +++--- .../nodes/datastream/DataStreamUnion.scala | 6 +- .../nodes/datastream/DataStreamValues.scala | 12 ++-- .../datastream/DataStreamWindowJoin.scala | 16 ++--- .../plan/nodes/datastream/StreamScan.scala | 8 +-- .../datastream/DataStreamWindowJoinRule.scala | 2 +- .../flink/table/plan/schema/FlinkTable.scala | 4 +- .../flink/table/plan/schema/RowSchema.scala | 50 ++++--------- .../CRowInputTupleOutputMapRunner.scala | 35 --------- .../runtime/CRowOutputProcessRunner.scala | 4 +- ...appingTimestampSetterProcessFunction.scala | 61 ++++++++++++++++ .../runtime/aggregate/AggregateUtil.scala | 14 ++-- .../aggregate/ProcTimeBoundedRangeOver.scala | 1 + .../ProcTimeSortProcessFunction.scala | 1 + .../aggregate/RowTimeBoundedRangeOver.scala | 1 + .../aggregate/RowTimeBoundedRowsOver.scala | 1 + .../RowTimeSortProcessFunction.scala | 1 + .../aggregate/RowTimeUnboundedOver.scala | 2 +- .../table/runtime/aggregate/SortUtil.scala | 2 +- .../table/runtime/join/WindowJoinUtil.scala | 6 +- .../typeutils/TimeIndicatorTypeInfo.scala | 3 + 35 files changed, 282 insertions(+), 259 deletions(-) rename flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/{ProcTimeMaterializationSqlFunction.scala => ProctimeSqlFunction.scala} (97%) create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/WrappingTimestampSetterProcessFunction.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 6b73f4ecc16d7..c4e1450c64ca1 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 @@ -48,7 +48,7 @@ import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} import org.apache.flink.table.runtime.{CRowInputJavaTupleOutputMapRunner, CRowInputMapRunner, CRowInputScalaTupleOutputMapRunner, WrappingTimestampSetterProcessFunction} import org.apache.flink.table.sinks.{AppendStreamTableSink, RetractStreamTableSink, TableSink, UpsertStreamTableSink} import org.apache.flink.table.sources.{DefinedRowtimeAttribute, StreamTableSource, TableSource} -import org.apache.flink.table.typeutils.TypeCheckUtils +import org.apache.flink.table.typeutils.{TimeIndicatorTypeInfo, TypeCheckUtils} import org.apache.flink.types.Row import _root_.scala.collection.JavaConverters._ @@ -501,7 +501,6 @@ abstract class StreamTableEnvironment( /** * Injects markers for time indicator fields into the field indexes. - * A rowtime indicator is represented as -1, a proctime indicator as -2. * * @param fieldIndexes The field indexes into which the time indicators markers are injected. * @param rowtime An optional rowtime indicator @@ -514,17 +513,15 @@ abstract class StreamTableEnvironment( proctime: Option[(Int, String)]): Array[Int] = { // inject rowtime field - val withRowtime = if (rowtime.isDefined) { - fieldIndexes.patch(rowtime.get._1, Seq(-1), 0) // -1 indicates rowtime - } else { - fieldIndexes + val withRowtime = rowtime match { + case Some(rt) => fieldIndexes.patch(rt._1, Seq(TimeIndicatorTypeInfo.ROWTIME_MARKER), 0) + case _ => fieldIndexes } // inject proctime field - val withProctime = if (proctime.isDefined) { - withRowtime.patch(proctime.get._1, Seq(-2), 0) // -2 indicates proctime - } else { - withRowtime + val withProctime = proctime match { + case Some(pt) => withRowtime.patch(pt._1, Seq(TimeIndicatorTypeInfo.PROCTIME_MARKER), 0) + case _ => withRowtime } withProctime @@ -545,17 +542,15 @@ abstract class StreamTableEnvironment( proctime: Option[(Int, String)]): Array[String] = { // inject rowtime field - val withRowtime = if (rowtime.isDefined) { - fieldNames.patch(rowtime.get._1, Seq(rowtime.get._2), 0) - } else { - fieldNames + val withRowtime = rowtime match { + case Some(rt) => fieldNames.patch(rt._1, Seq(rowtime.get._2), 0) + case _ => fieldNames } // inject proctime field - val withProctime = if (proctime.isDefined) { - withRowtime.patch(proctime.get._1, Seq(proctime.get._2), 0) - } else { - withRowtime + val withProctime = proctime match { + case Some(pt) => withRowtime.patch(pt._1, Seq(proctime.get._2), 0) + case _ => withRowtime } withProctime @@ -759,8 +754,7 @@ abstract class StreamTableEnvironment( .filter(f => FlinkTypeFactory.isRowtimeIndicatorType(f.getType)) if (rowtimeFields.isEmpty) { - // to rowtime field to set - + // no rowtime field to set conversion match { case mapFunction: MapFunction[CRow, A] => plan.map(mapFunction) @@ -770,7 +764,6 @@ abstract class StreamTableEnvironment( } } else if (rowtimeFields.size == 1) { // set the only rowtime field as event-time timestamp for DataStream - val mapFunction = conversion match { case mapFunction: MapFunction[CRow, A] => mapFunction case _ => new MapFunction[CRow, A] { @@ -791,7 +784,7 @@ abstract class StreamTableEnvironment( s"Found more than one rowtime field: [${rowtimeFields.map(_.getName).mkString(", ")}] in " + s"the table that should be converted to a DataStream.\n" + s"Please select the rowtime field that should be used as event-time timestamp for the " + - s"DataStream by casting all other fields to TIMESTAMP or LONG.") + s"DataStream by casting all other fields to TIMESTAMP.") } } 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 3bca1568ae2bb..b647c513f6b48 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 @@ -731,15 +731,15 @@ abstract class TableEnvironment(val config: TableConfig) { // validate that at least the field types of physical and logical type match // we do that here to make sure that plan translation was correct - if (schema.physicalTypeInfo != inputTypeInfo) { + if (schema.typeInfo != inputTypeInfo) { throw TableException( s"The field types of physical and logical row types do not match. " + - s"Physical type is [${schema.physicalTypeInfo}], Logical type is [${inputTypeInfo}]. " + + s"Physical type is [${schema.typeInfo}], Logical type is [${inputTypeInfo}]. " + s"This is a bug and should not happen. Please file an issue.") } - val fieldTypes = schema.physicalFieldTypeInfo - val fieldNames = schema.physicalFieldNames + val fieldTypes = schema.fieldTypeInfos + val fieldNames = schema.fieldNames // validate requested type if (requestedTypeInfo.getArity != fieldTypes.length) { 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 f5319ec422a3b..717a1af46f38c 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 @@ -27,7 +27,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo import org.apache.flink.table.api.{TableException, ValidationException} import org.apache.flink.table.calcite.FlinkTypeFactory.{isRowtimeIndicatorType, _} -import org.apache.flink.table.functions.ProcTimeMaterializationSqlFunction +import org.apache.flink.table.functions.ProctimeSqlFunction import org.apache.flink.table.plan.logical.rel.LogicalWindowAggregate import org.apache.flink.table.plan.schema.TimeIndicatorRelDataType @@ -247,7 +247,7 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle { rexBuilder.makeAbstractCast(timestamp, expr) } else { // generate proctime access - rexBuilder.makeCall(ProcTimeMaterializationSqlFunction, expr) + rexBuilder.makeCall(ProctimeSqlFunction, expr) } } else { expr @@ -271,7 +271,7 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle { } else { // generate proctime access rexBuilder.makeCall( - ProcTimeMaterializationSqlFunction, + ProctimeSqlFunction, new RexInputRef(field.getIndex, field.getType)) } } else { @@ -323,12 +323,12 @@ object RelTimeIndicatorConverter { var needsConversion = false - // materialize remaining proc time indicators + // materialize remaining proctime indicators val projects = convertedRoot.getRowType.getFieldList.map(field => if (isProctimeIndicatorType(field.getType)) { needsConversion = true rexBuilder.makeCall( - ProcTimeMaterializationSqlFunction, + ProctimeSqlFunction, new RexInputRef(field.getIndex, field.getType)) } else { new RexInputRef(field.getIndex, field.getType) @@ -411,7 +411,7 @@ class RexTimeIndicatorMaterializer( rexBuilder.makeAbstractCast(timestamp, o) } else { // generate proctime access - rexBuilder.makeCall(ProcTimeMaterializationSqlFunction, o) + rexBuilder.makeCall(ProctimeSqlFunction, o) } } else { o 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 8a7ea75893aa2..5290a56f9d509 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 @@ -40,7 +40,7 @@ import org.apache.flink.table.codegen.calls.FunctionGenerator import org.apache.flink.table.codegen.calls.ScalarOperators._ import org.apache.flink.table.functions.sql.ScalarSqlFunctions import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils -import org.apache.flink.table.functions.{FunctionContext, ProcTimeMaterializationSqlFunction, UserDefinedFunction} +import org.apache.flink.table.functions.{FunctionContext, ProctimeSqlFunction, UserDefinedFunction} import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo import org.apache.flink.table.typeutils.TypeCheckUtils._ @@ -56,10 +56,11 @@ import scala.collection.mutable * @param nullableInput input(s) can be null. * @param input1 type information about the first input of the Function * @param input2 type information about the second input if the Function is binary - * @param input1FieldMapping additional mapping information for input1 - * (e.g. POJO types have no deterministic field order and some input fields might not be read) - * @param input2FieldMapping additional mapping information for input2 - * (e.g. POJO types have no deterministic field order and some input fields might not be read) + * @param input1FieldMapping additional mapping information for input1. + * POJO types have no deterministic field order and some input fields might not be read. + * The input1FieldMapping is also used to inject time indicator attributes. + * @param input2FieldMapping additional mapping information for input2. + * POJO types have no deterministic field order and some input fields might not be read. */ abstract class CodeGenerator( config: TableConfig, @@ -245,20 +246,23 @@ abstract class CodeGenerator( returnType: TypeInformation[_ <: Any], resultFieldNames: Seq[String]) : GeneratedExpression = { + val input1AccessExprs = input1Mapping.map { - case -1 => generateStreamRecordTimestampAcccess() - case -2 => generateNullLiteral(TimeIndicatorTypeInfo.PROCTIME_INDICATOR) - case idx => generateInputAccess(input1, input1Term, idx) + case TimeIndicatorTypeInfo.ROWTIME_MARKER => + // attribute is a rowtime indicator. Access event-time timestamp in StreamRecord. + generateRowtimeAccess() + case TimeIndicatorTypeInfo.PROCTIME_MARKER => + // attribute is proctime indicator. + // We use a null literal and generate a timestamp when we need it. + generateNullLiteral(TimeIndicatorTypeInfo.PROCTIME_INDICATOR) + case idx => + // regular attribute. Access attribute in input data type. + generateInputAccess(input1, input1Term, idx) } val input2AccessExprs = input2 match { case Some(ti) => - input2Mapping.map { - case -1 => generateNullLiteral(TimeIndicatorTypeInfo.ROWTIME_INDICATOR) - case -2 => generateNullLiteral(TimeIndicatorTypeInfo.PROCTIME_INDICATOR) - case idx => generateInputAccess(ti, input2Term, idx) - }.toSeq - + input2Mapping.map(idx => generateInputAccess(ti, input2Term, idx)).toSeq case None => Seq() // add nothing } @@ -727,8 +731,8 @@ abstract class CodeGenerator( override def visitCall(call: RexCall): GeneratedExpression = { // special case: time materialization - if (call.getOperator == ProcTimeMaterializationSqlFunction) { - return generateProcTimestamp() + if (call.getOperator == ProctimeSqlFunction) { + return generateProctimeTimestamp() } val resultType = FlinkTypeFactory.toTypeInfo(call.getType) @@ -968,7 +972,7 @@ abstract class CodeGenerator( generateArrayElement(this, array) case ScalarSqlFunctions.CONCAT => - generateConcat(operands) + generateConcat(this.nullCheck, operands) case ScalarSqlFunctions.CONCAT_WS => generateConcatWs(operands) @@ -1128,19 +1132,6 @@ abstract class CodeGenerator( } } - private[flink] def generateStreamRecordTimestampAcccess(): GeneratedExpression = { - val resultTerm = newName("result") - val nullTerm = newName("isNull") - - val accessCode = - s""" - |long $resultTerm = $contextTerm.timestamp(); - |boolean $nullTerm = false; - """.stripMargin - - GeneratedExpression(resultTerm, nullTerm, accessCode, TimeIndicatorTypeInfo.ROWTIME_INDICATOR) - } - private def generateNullLiteral(resultType: TypeInformation[_]): GeneratedExpression = { val resultTerm = newName("result") val nullTerm = newName("isNull") @@ -1291,7 +1282,25 @@ abstract class CodeGenerator( } } - private[flink] def generateProcTimestamp(): GeneratedExpression = { + private[flink] def generateRowtimeAccess(): GeneratedExpression = { + val resultTerm = newName("result") + val nullTerm = newName("isNull") + + val accessCode = + s""" + |Long $resultTerm = $contextTerm.timestamp(); + |if ($resultTerm == 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."); + |} + |boolean $nullTerm = false; + """.stripMargin + + GeneratedExpression(resultTerm, nullTerm, accessCode, TimeIndicatorTypeInfo.ROWTIME_INDICATOR) + } + + private[flink] def generateProctimeTimestamp(): GeneratedExpression = { val resultTerm = newName("result") val resultTypeTerm = primitiveTypeTermForTypeInfo(SqlTimeTypeInfo.TIMESTAMP) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala index dd01e027648a1..01e9dffe817d2 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala @@ -1023,9 +1023,12 @@ object ScalarOperators { } } - def generateConcat(operands: Seq[GeneratedExpression]): GeneratedExpression = { + def generateConcat( + nullCheck: Boolean, + operands: Seq[GeneratedExpression]) + : GeneratedExpression = { - generateCallIfArgsNotNull(true, STRING_TYPE_INFO, operands) { + generateCallIfArgsNotNull(nullCheck, STRING_TYPE_INFO, operands) { (terms) =>s"${qualifyMethod(BuiltInMethods.CONCAT)}(${terms.mkString(", ")})" } } @@ -1036,6 +1039,8 @@ object ScalarOperators { val nullTerm = newName("isNull") val defaultValue = primitiveDefaultValue(Types.STRING) + val tempTerms = operands.tail.map(_ => newName("temp")) + val operatorCode = s""" |${operands.map(_.code).mkString("\n")} @@ -1046,11 +1051,15 @@ object ScalarOperators { | $nullTerm = true; | $resultTerm = $defaultValue; |} else { - | - | ${operands.tail.map(o => s"if (${o.nullTerm}) ${o.resultTerm} = null;").mkString("\n")} + | ${operands.tail.zip(tempTerms).map { + case (o: GeneratedExpression, t: String) => + s"String $t;\n" + + s" if (${o.nullTerm}) $t = null; else $t = ${o.resultTerm};" + }.mkString("\n") + } | $nullTerm = false; - | $resultTerm = ${qualifyMethod(BuiltInMethods.CONCAT_WS)}( - | ${operands.map(_.resultTerm).mkString(", ")}); + | $resultTerm = ${qualifyMethod(BuiltInMethods.CONCAT_WS)} + | (${operands.head.resultTerm}, ${tempTerms.mkString(", ")}); |} |""".stripMargin diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/ProcTimeMaterializationSqlFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/ProctimeSqlFunction.scala similarity index 97% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/ProcTimeMaterializationSqlFunction.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/ProctimeSqlFunction.scala index d2e7eff5f52fc..4fb0378b83e63 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/ProcTimeMaterializationSqlFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/ProctimeSqlFunction.scala @@ -25,7 +25,7 @@ import org.apache.calcite.sql.validate.SqlMonotonicity * Function that materializes a processing time attribute. * After materialization the result can be used in regular arithmetical calculations. */ -object ProcTimeMaterializationSqlFunction +object ProctimeSqlFunction extends SqlFunction( "PROCTIME", SqlKind.OTHER_FUNCTION, 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 3e355ff41d20b..2f1871b7e1f47 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 @@ -42,8 +42,8 @@ trait CommonCalc { GeneratedFunction[T, Row] = { val projection = generator.generateResultExpression( - returnSchema.physicalTypeInfo, - returnSchema.physicalFieldNames, + returnSchema.typeInfo, + returnSchema.fieldNames, calcProjection) // only projection @@ -80,7 +80,7 @@ trait CommonCalc { ruleDescription, functionClass, body, - returnSchema.physicalTypeInfo) + returnSchema.typeInfo) } private[flink] def conditionToString( 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 71f0d4df18aba..7c01fdeb9b178 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 @@ -56,7 +56,7 @@ trait CommonCorrelate { val functionGenerator = new FunctionCodeGenerator( config, false, - inputSchema.physicalTypeInfo, + inputSchema.typeInfo, Some(udtfTypeInfo), None, pojoFieldMapping) @@ -88,8 +88,8 @@ trait CommonCorrelate { } val outerResultExpr = functionGenerator.generateResultExpression( input1AccessExprs ++ input2NullExprs, - returnSchema.physicalTypeInfo, - returnSchema.physicalFieldNames) + returnSchema.typeInfo, + returnSchema.fieldNames) body += s""" |boolean hasOutput = $collectorTerm.isCollected(); @@ -106,7 +106,7 @@ trait CommonCorrelate { ruleDescription, functionClass, body, - returnSchema.physicalTypeInfo) + returnSchema.typeInfo) } /** @@ -124,7 +124,7 @@ trait CommonCorrelate { val generator = new CollectorCodeGenerator( config, false, - inputSchema.physicalTypeInfo, + inputSchema.typeInfo, Some(udtfTypeInfo), None, pojoFieldMapping) @@ -133,8 +133,8 @@ trait CommonCorrelate { val crossResultExpr = generator.generateResultExpression( input1AccessExprs ++ input2AccessExprs, - returnSchema.physicalTypeInfo, - returnSchema.physicalFieldNames) + returnSchema.typeInfo, + returnSchema.fieldNames) val collectorCode = if (condition.isEmpty) { s""" @@ -146,7 +146,7 @@ trait CommonCorrelate { // adjust indicies of InputRefs to adhere to schema expected by generator val changeInputRefIndexShuttle = new RexShuttle { override def visitInputRef(inputRef: RexInputRef): RexNode = { - new RexInputRef(inputSchema.physicalArity + inputRef.getIndex, inputRef.getType) + new RexInputRef(inputSchema.arity + inputRef.getIndex, inputRef.getType) } } // Run generateExpression to add init statements (ScalarFunctions) of condition to generator. 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 e40501ec62a62..45e69028b8305 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 @@ -52,7 +52,7 @@ class DataStreamCalc( with CommonCalc with DataStreamRel { - override def deriveRowType(): RelDataType = schema.logicalType + override def deriveRowType(): RelDataType = schema.relDataType override def copy(traitSet: RelTraitSet, child: RelNode, program: RexProgram): Calc = { new DataStreamCalc( @@ -100,7 +100,7 @@ class DataStreamCalc( val condition = if (calcProgram.getCondition != null) { val materializedCondition = RelTimeIndicatorConverter.convertExpression( calcProgram.expandLocalRef(calcProgram.getCondition), - inputSchema.logicalType, + inputSchema.relDataType, cluster.getRexBuilder) Some(materializedCondition) } else { @@ -111,7 +111,7 @@ class DataStreamCalc( val projection = calcProgram.getProjectList.asScala .map(calcProgram.expandLocalRef) - val generator = new FunctionCodeGenerator(config, false, inputSchema.physicalTypeInfo) + val generator = new FunctionCodeGenerator(config, false, inputSchema.typeInfo) val genFunction = generateFunction( generator, @@ -128,7 +128,7 @@ class DataStreamCalc( val processFunc = new CRowProcessRunner( genFunction.name, genFunction.code, - CRowTypeInfo(schema.physicalTypeInfo)) + CRowTypeInfo(schema.typeInfo)) inputDataStream .process(processFunc) 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 b7165cd190183..18ab2a3354eb5 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 @@ -50,7 +50,7 @@ class DataStreamCorrelate( with CommonCorrelate with DataStreamRel { - override def deriveRowType() = schema.logicalType + override def deriveRowType() = schema.relDataType override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = { new DataStreamCorrelate( @@ -78,7 +78,7 @@ class DataStreamCorrelate( super.explainTerms(pw) .item("invocation", scan.getCall) .item("function", sqlFunction.getTableFunction.getClass.getCanonicalName) - .item("rowType", schema.logicalType) + .item("rowType", schema.relDataType) .item("joinType", joinType) .itemIf("condition", condition.orNull, condition.isDefined) } @@ -130,7 +130,7 @@ class DataStreamCorrelate( .process(processFunc) // preserve input parallelism to ensure that acc and retract messages remain in order .setParallelism(inputParallelism) - .name(correlateOpName(rexCall, sqlFunction, schema.logicalType)) + .name(correlateOpName(rexCall, sqlFunction, schema.relDataType)) } } 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 ca9ba23929bae..590d9be43bfd5 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 @@ -59,7 +59,7 @@ class DataStreamGroupAggregate( private val LOG = LoggerFactory.getLogger(this.getClass) - override def deriveRowType() = schema.logicalType + override def deriveRowType() = schema.relDataType override def needsUpdatesAsRetraction = true @@ -83,20 +83,20 @@ class DataStreamGroupAggregate( override def toString: String = { s"Aggregate(${ if (!groupings.isEmpty) { - s"groupBy: (${groupingToString(inputSchema.logicalType, groupings)}), " + s"groupBy: (${groupingToString(inputSchema.relDataType, groupings)}), " } else { "" } }select:(${aggregationToString( - inputSchema.logicalType, groupings, getRowType, namedAggregates, Nil)}))" + inputSchema.relDataType, groupings, getRowType, namedAggregates, Nil)}))" } override def explainTerms(pw: RelWriter): RelWriter = { super.explainTerms(pw) .itemIf("groupBy", groupingToString( - inputSchema.logicalType, groupings), !groupings.isEmpty) + inputSchema.relDataType, groupings), !groupings.isEmpty) .item("select", aggregationToString( - inputSchema.logicalType, groupings, getRowType, namedAggregates, Nil)) + inputSchema.relDataType, groupings, getRowType, namedAggregates, Nil)) } override def translateToPlan( @@ -112,29 +112,29 @@ class DataStreamGroupAggregate( val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig) - val outRowType = CRowTypeInfo(schema.physicalTypeInfo) + val outRowType = CRowTypeInfo(schema.typeInfo) val generator = new AggregationCodeGenerator( tableEnv.getConfig, false, - inputSchema.physicalTypeInfo) + inputSchema.typeInfo) val aggString = aggregationToString( - inputSchema.logicalType, + inputSchema.relDataType, groupings, getRowType, namedAggregates, Nil) - val keyedAggOpName = s"groupBy: (${groupingToString(inputSchema.logicalType, groupings)}), " + + val keyedAggOpName = s"groupBy: (${groupingToString(inputSchema.relDataType, groupings)}), " + s"select: ($aggString)" val nonKeyedAggOpName = s"select: ($aggString)" val processFunction = AggregateUtil.createGroupAggregateFunction( generator, namedAggregates, - inputSchema.logicalType, - inputSchema.physicalFieldTypeInfo, + inputSchema.relDataType, + inputSchema.fieldTypeInfos, groupings, queryConfig, DataStreamRetractionRules.isAccRetract(this), 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 856e9bf7cdcf3..ac63be188c0c8 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 @@ -60,7 +60,7 @@ class DataStreamGroupWindowAggregate( private val LOG = LoggerFactory.getLogger(this.getClass) - override def deriveRowType(): RelDataType = schema.logicalType + override def deriveRowType(): RelDataType = schema.relDataType override def needsUpdatesAsRetraction = true @@ -86,14 +86,14 @@ class DataStreamGroupWindowAggregate( override def toString: String = { s"Aggregate(${ if (!grouping.isEmpty) { - s"groupBy: (${groupingToString(inputSchema.logicalType, grouping)}), " + s"groupBy: (${groupingToString(inputSchema.relDataType, grouping)}), " } else { "" } }window: ($window), " + s"select: (${ aggregationToString( - inputSchema.logicalType, + inputSchema.relDataType, grouping, getRowType, namedAggregates, @@ -103,13 +103,13 @@ class DataStreamGroupWindowAggregate( override def explainTerms(pw: RelWriter): RelWriter = { super.explainTerms(pw) - .itemIf("groupBy", groupingToString(inputSchema.logicalType, grouping), !grouping.isEmpty) + .itemIf("groupBy", groupingToString(inputSchema.relDataType, grouping), !grouping.isEmpty) .item("window", window) .item( "select", aggregationToString( - inputSchema.logicalType, + inputSchema.relDataType, grouping, - schema.logicalType, + schema.relDataType, namedAggregates, namedProperties)) } @@ -145,27 +145,27 @@ class DataStreamGroupWindowAggregate( val timestampedInput = if (isRowtimeAttribute(window.timeAttribute)) { // copy the window rowtime attribute into the StreamRecord timestamp field val timeAttribute = window.timeAttribute.asInstanceOf[ResolvedFieldReference].name - val timeIdx = inputSchema.logicalFieldNames.indexOf(timeAttribute) + val timeIdx = inputSchema.fieldNames.indexOf(timeAttribute) inputDS .process( - new TimestampSetterProcessFunction(timeIdx,CRowTypeInfo(inputSchema.physicalTypeInfo))) + new TimestampSetterProcessFunction(timeIdx, CRowTypeInfo(inputSchema.typeInfo))) .setParallelism(inputDS.getParallelism) .name(s"time attribute: ($timeAttribute)") } else { inputDS } - val outRowType = CRowTypeInfo(schema.physicalTypeInfo) + val outRowType = CRowTypeInfo(schema.typeInfo) val aggString = aggregationToString( - inputSchema.logicalType, + inputSchema.relDataType, grouping, - schema.logicalType, + schema.relDataType, namedAggregates, namedProperties) - val keyedAggOpName = s"groupBy: (${groupingToString(inputSchema.logicalType, grouping)}), " + + val keyedAggOpName = s"groupBy: (${groupingToString(inputSchema.relDataType, grouping)}), " + s"window: ($window), " + s"select: ($aggString)" val nonKeyedAggOpName = s"window: ($window), select: ($aggString)" @@ -173,7 +173,7 @@ class DataStreamGroupWindowAggregate( val generator = new AggregationCodeGenerator( tableEnv.getConfig, false, - inputSchema.physicalTypeInfo) + inputSchema.typeInfo) val needMerge = window match { case SessionGroupWindow(_, _, _) => true @@ -185,7 +185,7 @@ class DataStreamGroupWindowAggregate( window, grouping.length, namedAggregates.size, - schema.physicalArity, + schema.arity, namedProperties) val keyedStream = timestampedInput.keyBy(grouping: _*) @@ -197,9 +197,9 @@ class DataStreamGroupWindowAggregate( AggregateUtil.createDataStreamAggregateFunction( generator, namedAggregates, - inputSchema.physicalType, - inputSchema.physicalFieldTypeInfo, - schema.physicalType, + inputSchema.relDataType, + inputSchema.fieldTypeInfos, + schema.relDataType, grouping, needMerge) @@ -211,7 +211,7 @@ class DataStreamGroupWindowAggregate( else { val windowFunction = AggregateUtil.createAggregationAllWindowFunction( window, - schema.physicalArity, + schema.arity, namedProperties) val windowedStream = @@ -222,9 +222,9 @@ class DataStreamGroupWindowAggregate( AggregateUtil.createDataStreamAggregateFunction( generator, namedAggregates, - inputSchema.physicalType, - inputSchema.physicalFieldTypeInfo, - schema.physicalType, + inputSchema.relDataType, + inputSchema.fieldTypeInfos, + schema.relDataType, Array[Int](), needMerge) 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 c03a55e1710e1..7bf342a8985d1 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 @@ -50,7 +50,7 @@ class DataStreamOverAggregate( with DataStreamRel { private val LOG = LoggerFactory.getLogger(this.getClass) - override def deriveRowType(): RelDataType = schema.logicalType + override def deriveRowType(): RelDataType = schema.relDataType override def needsUpdatesAsRetraction = true @@ -78,15 +78,15 @@ class DataStreamOverAggregate( super.explainTerms(pw) .itemIf("partitionBy", - partitionToString(schema.logicalType, partitionKeys), partitionKeys.nonEmpty) + partitionToString(schema.relDataType, partitionKeys), partitionKeys.nonEmpty) .item("orderBy", - orderingToString(schema.logicalType, overWindow.orderKeys.getFieldCollations)) + orderingToString(schema.relDataType, overWindow.orderKeys.getFieldCollations)) .itemIf("rows", windowRange(logicWindow, overWindow, inputNode), overWindow.isRows) .itemIf("range", windowRange(logicWindow, overWindow, inputNode), !overWindow.isRows) .item( "select", aggregationToString( - inputSchema.logicalType, - schema.logicalType, + inputSchema.relDataType, + schema.relDataType, namedAggregates)) } @@ -134,9 +134,9 @@ class DataStreamOverAggregate( val generator = new AggregationCodeGenerator( tableEnv.getConfig, false, - inputSchema.physicalTypeInfo) + inputSchema.typeInfo) - val timeType = schema.logicalType + val timeType = schema.relDataType .getFieldList .get(orderKey.getFieldIndex) .getType @@ -189,14 +189,14 @@ class DataStreamOverAggregate( val namedAggregates: Seq[CalcitePair[AggregateCall, String]] = generateNamedAggregates // get the output types - val returnTypeInfo = CRowTypeInfo(schema.physicalTypeInfo) + val returnTypeInfo = CRowTypeInfo(schema.typeInfo) val processFunction = AggregateUtil.createUnboundedOverProcessFunction( generator, namedAggregates, - inputSchema.physicalType, - inputSchema.physicalTypeInfo, - inputSchema.physicalFieldTypeInfo, + inputSchema.relDataType, + inputSchema.typeInfo, + inputSchema.fieldTypeInfos, queryConfig, rowTimeIdx, partitionKeys.nonEmpty, @@ -237,14 +237,14 @@ class DataStreamOverAggregate( getLowerBoundary(logicWindow, overWindow, getInput()) + (if (isRowsClause) 1 else 0) // get the output types - val returnTypeInfo = CRowTypeInfo(schema.physicalTypeInfo) + val returnTypeInfo = CRowTypeInfo(schema.typeInfo) val processFunction = AggregateUtil.createBoundedOverProcessFunction( generator, namedAggregates, - inputSchema.physicalType, - inputSchema.physicalTypeInfo, - inputSchema.physicalFieldTypeInfo, + inputSchema.relDataType, + inputSchema.typeInfo, + inputSchema.fieldTypeInfos, precedingOffset, queryConfig, isRowsClause, @@ -285,18 +285,18 @@ class DataStreamOverAggregate( s"over: (${ if (!partitionKeys.isEmpty) { - s"PARTITION BY: ${partitionToString(inputSchema.logicalType, partitionKeys)}, " + s"PARTITION BY: ${partitionToString(inputSchema.relDataType, partitionKeys)}, " } else { "" } - }ORDER BY: ${orderingToString(inputSchema.logicalType, + }ORDER BY: ${orderingToString(inputSchema.relDataType, overWindow.orderKeys.getFieldCollations)}, " + s"${if (overWindow.isRows) "ROWS" else "RANGE"}" + s"${windowRange(logicWindow, overWindow, inputNode)}, " + s"select: (${ aggregationToString( - inputSchema.logicalType, - schema.logicalType, + inputSchema.relDataType, + schema.relDataType, namedAggregates) }))" } 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 424c6a26633e5..9352efb5372dc 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 @@ -43,7 +43,7 @@ class DataStreamScan( val dataStreamTable: DataStreamTable[Any] = getTable.unwrap(classOf[DataStreamTable[Any]]) - override def deriveRowType(): RelDataType = schema.logicalType + override def deriveRowType(): RelDataType = schema.relDataType override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = { new DataStreamScan( diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala index a11e6c179993b..8f9942fd608c3 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamSort.scala @@ -53,7 +53,7 @@ class DataStreamSort( with CommonSort with DataStreamRel { - override def deriveRowType(): RelDataType = schema.logicalType + override def deriveRowType(): RelDataType = schema.relDataType override def copy( traitSet: RelTraitSet, @@ -75,13 +75,13 @@ class DataStreamSort( } override def toString: String = { - sortToString(schema.logicalType, sortCollation, sortOffset, sortFetch) + sortToString(schema.relDataType, sortCollation, sortOffset, sortFetch) } override def explainTerms(pw: RelWriter) : RelWriter = { sortExplainTerms( pw.input("input", getInput()), - schema.logicalType, + schema.relDataType, sortCollation, sortOffset, sortFetch) @@ -94,7 +94,7 @@ class DataStreamSort( val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig) // need to identify time between others order fields. Time needs to be first sort element - val timeType = SortUtil.getFirstSortField(sortCollation, schema.logicalType).getType + val timeType = SortUtil.getFirstSortField(sortCollation, schema.relDataType).getType // time ordering needs to be ascending if (SortUtil.getFirstSortDirection(sortCollation) != Direction.ASCENDING) { @@ -141,15 +141,15 @@ class DataStreamSort( inputDS: DataStream[CRow], execCfg: ExecutionConfig): DataStream[CRow] = { - val returnTypeInfo = CRowTypeInfo(schema.physicalTypeInfo) + val returnTypeInfo = CRowTypeInfo(schema.typeInfo) // if the order has secondary sorting fields in addition to the proctime if (sortCollation.getFieldCollations.size() > 1) { val processFunction = SortUtil.createProcTimeSortFunction( sortCollation, - inputSchema.logicalType, - inputSchema.physicalTypeInfo, + inputSchema.relDataType, + inputSchema.typeInfo, execCfg) inputDS.keyBy(new NullByteKeySelector[CRow]) @@ -173,12 +173,12 @@ class DataStreamSort( inputDS: DataStream[CRow], execCfg: ExecutionConfig): DataStream[CRow] = { - val returnTypeInfo = CRowTypeInfo(schema.physicalTypeInfo) + val returnTypeInfo = CRowTypeInfo(schema.typeInfo) val processFunction = SortUtil.createRowTimeSortFunction( sortCollation, - inputSchema.logicalType, - inputSchema.physicalTypeInfo, + inputSchema.relDataType, + inputSchema.typeInfo, execCfg) inputDS.keyBy(new NullByteKeySelector[CRow]) 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 6f4980aa70dab..7258ec88e4b93 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 @@ -38,7 +38,7 @@ class DataStreamUnion( extends BiRel(cluster, traitSet, leftNode, rightNode) with DataStreamRel { - override def deriveRowType() = schema.logicalType + override def deriveRowType() = schema.relDataType override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = { new DataStreamUnion( @@ -55,7 +55,7 @@ class DataStreamUnion( } override def toString = { - s"Union All(union: (${schema.logicalFieldNames.mkString(", ")}))" + s"Union All(union: (${schema.fieldNames.mkString(", ")}))" } override def translateToPlan( @@ -68,6 +68,6 @@ class DataStreamUnion( } private def unionSelectionToString: String = { - schema.logicalFieldNames.mkString(", ") + schema.fieldNames.mkString(", ") } } 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 14766815c600d..1ef9107cbc488 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 @@ -41,10 +41,10 @@ class DataStreamValues( schema: RowSchema, tuples: ImmutableList[ImmutableList[RexLiteral]], ruleDescription: String) - extends Values(cluster, schema.logicalType, tuples, traitSet) + extends Values(cluster, schema.relDataType, tuples, traitSet) with DataStreamRel { - override def deriveRowType() = schema.logicalType + override def deriveRowType() = schema.relDataType override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = { new DataStreamValues( @@ -62,14 +62,14 @@ class DataStreamValues( val config = tableEnv.getConfig - val returnType = CRowTypeInfo(schema.physicalTypeInfo) + val returnType = CRowTypeInfo(schema.typeInfo) val generator = new InputFormatCodeGenerator(config) // generate code for every record val generatedRecords = getTuples.asScala.map { r => generator.generateResultExpression( - schema.physicalTypeInfo, - schema.physicalFieldNames, + schema.typeInfo, + schema.fieldNames, r.asScala) } @@ -77,7 +77,7 @@ class DataStreamValues( val generatedFunction = generator.generateValuesInputFormat( ruleDescription, generatedRecords.map(_.code), - schema.physicalTypeInfo) + schema.typeInfo) val inputFormat = new CRowValuesInputFormat( generatedFunction.name, diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala index 987947c3d6ac3..f8015b354e6d3 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala @@ -54,7 +54,7 @@ class DataStreamWindowJoin( with CommonJoin with DataStreamRel { - override def deriveRowType(): RelDataType = schema.logicalType + override def deriveRowType(): RelDataType = schema.relDataType override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = { new DataStreamWindowJoin( @@ -76,7 +76,7 @@ class DataStreamWindowJoin( override def toString: String = { joinToString( - schema.logicalType, + schema.relDataType, joinCondition, joinType, getExpressionString) @@ -85,7 +85,7 @@ class DataStreamWindowJoin( override def explainTerms(pw: RelWriter): RelWriter = { joinExplainTerms( super.explainTerms(pw), - schema.logicalType, + schema.relDataType, joinCondition, joinType, getExpressionString) @@ -117,8 +117,8 @@ class DataStreamWindowJoin( WindowJoinUtil.generateJoinFunction( config, joinType, - leftSchema.physicalTypeInfo, - rightSchema.physicalTypeInfo, + leftSchema.typeInfo, + rightSchema.typeInfo, schema, remainCondition, ruleDescription) @@ -160,13 +160,13 @@ class DataStreamWindowJoin( leftKeys: Array[Int], rightKeys: Array[Int]): DataStream[CRow] = { - val returnTypeInfo = CRowTypeInfo(schema.physicalTypeInfo) + val returnTypeInfo = CRowTypeInfo(schema.typeInfo) val procInnerJoinFunc = new ProcTimeWindowInnerJoin( leftLowerBound, leftUpperBound, - leftSchema.physicalTypeInfo, - rightSchema.physicalTypeInfo, + leftSchema.typeInfo, + rightSchema.typeInfo, joinFunctionName, joinFunctionCode) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala index 07ac238f0aeae..4aca85633f1b8 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala @@ -41,7 +41,7 @@ trait StreamScan extends CommonScan[CRow] with DataStreamRel { : DataStream[CRow] = { val inputType = input.getType - val internalType = CRowTypeInfo(schema.physicalTypeInfo) + val internalType = CRowTypeInfo(schema.typeInfo) // conversion if (needsConversion(input.getType, internalType)) { @@ -54,8 +54,8 @@ trait StreamScan extends CommonScan[CRow] with DataStreamRel { Some(flinkTable.fieldIndexes)) val conversion = generator.generateConverterResultExpression( - schema.physicalTypeInfo, - schema.physicalFieldNames) + schema.typeInfo, + schema.fieldNames) val body = s""" @@ -67,7 +67,7 @@ trait StreamScan extends CommonScan[CRow] with DataStreamRel { "DataStreamSourceConversion", classOf[ProcessFunction[Any, Row]], body, - schema.physicalTypeInfo) + schema.typeInfo) val processFunc = new CRowOutputProcessRunner( function.name, diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala index 2075689b79dc4..7dfcbc523d309 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamWindowJoinRule.scala @@ -87,7 +87,7 @@ class DataStreamWindowJoinRule val (windowBounds, remainCondition) = WindowJoinUtil.extractWindowBoundsFromPredicate( joinInfo.getRemaining(join.getCluster.getRexBuilder), - leftRowSchema.logicalArity, + leftRowSchema.arity, join.getRowType, join.getCluster.getRexBuilder, TableConfig.DEFAULT) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala index 90ddd477d99d2..df56ae6ec2405 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala @@ -56,8 +56,8 @@ abstract class FlinkTable[T]( "must not be greater than number of field names " + fieldNames.deep + ".") } fieldIndexes.map { - case -1 => TimeIndicatorTypeInfo.ROWTIME_INDICATOR - case -2 => TimeIndicatorTypeInfo.PROCTIME_INDICATOR + case TimeIndicatorTypeInfo.ROWTIME_MARKER => TimeIndicatorTypeInfo.ROWTIME_INDICATOR + case TimeIndicatorTypeInfo.PROCTIME_MARKER => TimeIndicatorTypeInfo.PROCTIME_INDICATOR case i => cType.getTypeAt(i).asInstanceOf[TypeInformation[_]]} case aType: AtomicType[_] => if (fieldIndexes.length != 1 || fieldIndexes(0) != 0) { 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 3cc4f13517423..ad0f552b8f3a5 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 @@ -18,7 +18,7 @@ package org.apache.flink.table.plan.schema -import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeField, RelRecordType} +import org.apache.calcite.rel.`type`.RelDataType import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.table.calcite.FlinkTypeFactory @@ -31,57 +31,35 @@ import scala.collection.JavaConversions._ */ class RowSchema(private val logicalRowType: RelDataType) { - private lazy val physicalRowFields: Seq[RelDataTypeField] = logicalRowType.getFieldList - - private lazy val physicalRowType: RelDataType = new RelRecordType(physicalRowFields) - - private lazy val physicalRowFieldTypes: Seq[TypeInformation[_]] = physicalRowFields map { f => - FlinkTypeFactory.toTypeInfo(f.getType) - } - - private lazy val physicalRowFieldNames: Seq[String] = physicalRowFields.map(_.getName) + private lazy val physicalRowFieldTypes: Seq[TypeInformation[_]] = + logicalRowType.getFieldList map { f => FlinkTypeFactory.toTypeInfo(f.getType) } private lazy val physicalRowTypeInfo: TypeInformation[Row] = new RowTypeInfo( - physicalRowFieldTypes.toArray, physicalRowFieldNames.toArray) - - /** - * Returns the arity of the logical record. - */ - def logicalArity: Int = logicalRowType.getFieldCount - - /** - * Returns the arity of the physical record. - */ - def physicalArity: Int = physicalTypeInfo.getArity - - /** - * Returns a logical [[RelDataType]] including logical fields (i.e. time indicators). - */ - def logicalType: RelDataType = logicalRowType + physicalRowFieldTypes.toArray, fieldNames.toArray) /** - * Returns a physical [[RelDataType]] with no logical fields (i.e. time indicators). + * Returns the arity of the schema. */ - def physicalType: RelDataType = physicalRowType + def arity: Int = logicalRowType.getFieldCount /** - * Returns a physical [[TypeInformation]] of row with no logical fields (i.e. time indicators). + * Returns the [[RelDataType]] of the schema */ - def physicalTypeInfo: TypeInformation[Row] = physicalRowTypeInfo + def relDataType: RelDataType = logicalRowType /** - * Returns [[TypeInformation]] of the row's fields with no logical fields (i.e. time indicators). + * Returns the [[TypeInformation]] of of the schema */ - def physicalFieldTypeInfo: Seq[TypeInformation[_]] = physicalRowFieldTypes + def typeInfo: TypeInformation[Row] = physicalRowTypeInfo /** - * Returns the logical fields names including logical fields (i.e. time indicators). + * Returns the [[TypeInformation]] of fields of the schema */ - def logicalFieldNames: Seq[String] = logicalRowType.getFieldNames + def fieldTypeInfos: Seq[TypeInformation[_]] = physicalRowFieldTypes /** - * Returns the physical fields names with no logical fields (i.e. time indicators). + * Returns the fields names */ - def physicalFieldNames: Seq[String] = physicalRowFieldNames + def fieldNames: Seq[String] = logicalRowType.getFieldNames } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputTupleOutputMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputTupleOutputMapRunner.scala index 002c34826ca9e..6b3aa44fa97fc 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputTupleOutputMapRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputTupleOutputMapRunner.scala @@ -31,9 +31,6 @@ import org.apache.flink.table.runtime.types.CRow import org.apache.flink.types.Row import org.slf4j.LoggerFactory import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} -import org.apache.flink.streaming.api.functions.ProcessFunction -import org.apache.flink.streaming.api.operators.TimestampedCollector -import org.apache.flink.util.Collector /** * Convert [[CRow]] to a [[JTuple2]] @@ -95,35 +92,3 @@ class CRowInputScalaTupleOutputMapRunner( override def getProducedType: TypeInformation[(Boolean, Any)] = returnType } - -/** - * Wraps a ProcessFunction and sets a Timestamp field of a CRow as - * [[org.apache.flink.streaming.runtime.streamrecord.StreamRecord]] timestamp. - */ -class WrappingTimestampSetterProcessFunction[OUT]( - function: MapFunction[CRow, OUT], - rowtimeIdx: Int) - extends ProcessFunction[CRow, OUT] { - - override def open(parameters: Configuration): Unit = { - super.open(parameters) - function match { - case f: RichMapFunction[_, _] => - f.setRuntimeContext(getRuntimeContext) - f.open(parameters) - case _ => - } - } - - override def processElement( - in: CRow, - ctx: ProcessFunction[CRow, OUT]#Context, - out: Collector[OUT]): Unit = { - - val timestamp = SqlFunctions.toLong(in.row.getField(rowtimeIdx).asInstanceOf[Timestamp]) - out.asInstanceOf[TimestampedCollector[_]].setAbsoluteTimestamp(timestamp) - - out.collect(function.map(in)) - } - -} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputProcessRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputProcessRunner.scala index ac94c500ebff4..c80f29102b82a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputProcessRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputProcessRunner.scala @@ -46,9 +46,9 @@ class CRowOutputProcessRunner( private var cRowWrapper: CRowWrappingCollector = _ override def open(parameters: Configuration): Unit = { - LOG.debug(s"Compiling MapFunction: $name \n\n Code:\n$code") + LOG.debug(s"Compiling ProcessFunction: $name \n\n Code:\n$code") val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code) - LOG.debug("Instantiating MapFunction.") + LOG.debug("Instantiating ProcessFunction.") function = clazz.newInstance() this.cRowWrapper = new CRowWrappingCollector() diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/WrappingTimestampSetterProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/WrappingTimestampSetterProcessFunction.scala new file mode 100644 index 0000000000000..8f12c3061763a --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/WrappingTimestampSetterProcessFunction.scala @@ -0,0 +1,61 @@ +/* + * 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 java.sql.Timestamp + +import org.apache.calcite.runtime.SqlFunctions +import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction} +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.streaming.api.operators.TimestampedCollector +import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.util.Collector + +/** + * Wraps a ProcessFunction and sets a Timestamp field of a CRow as + * [[org.apache.flink.streaming.runtime.streamrecord.StreamRecord]] timestamp. + */ +class WrappingTimestampSetterProcessFunction[OUT]( + function: MapFunction[CRow, OUT], + rowtimeIdx: Int) + extends ProcessFunction[CRow, OUT] { + + override def open(parameters: Configuration): Unit = { + super.open(parameters) + function match { + case f: RichMapFunction[_, _] => + f.setRuntimeContext(getRuntimeContext) + f.open(parameters) + case _ => + } + } + + override def processElement( + in: CRow, + ctx: ProcessFunction[CRow, OUT]#Context, + out: Collector[OUT]): Unit = { + + val timestamp = SqlFunctions.toLong(in.row.getField(rowtimeIdx).asInstanceOf[Timestamp]) + out.asInstanceOf[TimestampedCollector[_]].setAbsoluteTimestamp(timestamp) + + out.collect(function.map(in)) + } + +} 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 9372dbd8d8a44..52105e3941f8b 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 @@ -1147,22 +1147,22 @@ object AggregateUtil { val propPos = properties.foldRight( (None: Option[Int], None: Option[Int], None: Option[Int], 0)) { - (p, x) => p match { + case (p, (s, e, t, i)) => p match { case NamedWindowProperty(_, prop) => prop match { - case WindowStart(_) if x._1.isDefined => + case WindowStart(_) if s.isDefined => throw new TableException("Duplicate WindowStart property encountered. This is a bug.") case WindowStart(_) => - (Some(x._4), x._2, x._3, x._4 - 1) - case WindowEnd(_) if x._2.isDefined => + (Some(i), e, t, i - 1) + case WindowEnd(_) if e.isDefined => throw new TableException("Duplicate WindowEnd property encountered. This is a bug.") case WindowEnd(_) => - (x._1, Some(x._4), x._3, x._4 - 1) - case RowtimeAttribute(_) if x._3.isDefined => + (s, Some(i), t, i - 1) + case RowtimeAttribute(_) if t.isDefined => throw new TableException( "Duplicate Window rowtime property encountered. This is a bug.") case RowtimeAttribute(_) => - (x._1, x._2, Some(x._4), x._4 - 1) + (s, e, Some(i), i - 1) } } } 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 916e0216d5aa6..ab3dc1db63248 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 @@ -124,6 +124,7 @@ class ProcTimeBoundedRangeOver( return } + // remove timestamp set outside of ProcessFunction. out.asInstanceOf[TimestampedCollector[_]].eraseTimestamp() // we consider the original timestamp of events diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala index 6bef05ac7a746..1e12060a6bafd 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeSortProcessFunction.scala @@ -77,6 +77,7 @@ class ProcTimeSortProcessFunction( ctx: ProcessFunction[CRow, CRow]#OnTimerContext, out: Collector[CRow]): Unit = { + // remove timestamp set outside of ProcessFunction. out.asInstanceOf[TimestampedCollector[_]].eraseTimestamp() val iter = bufferedEvents.get.iterator() 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 77c62cb696223..ceb986d406d91 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 @@ -170,6 +170,7 @@ class RowTimeBoundedRangeOver( return } + // remove timestamp set outside of ProcessFunction. out.asInstanceOf[TimestampedCollector[_]].eraseTimestamp() // gets all window data from state for the calculation 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 b645684413aa5..678a3b78bc91c 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 @@ -179,6 +179,7 @@ class RowTimeBoundedRowsOver( return } + // remove timestamp set outside of ProcessFunction. out.asInstanceOf[TimestampedCollector[_]].eraseTimestamp() // gets all window data from state for the calculation diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala index 92ad6ccb0a8f5..fd586786c5e71 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala @@ -117,6 +117,7 @@ class RowTimeSortProcessFunction( ctx: ProcessFunction[CRow, CRow]#OnTimerContext, out: Collector[CRow]): Unit = { + // remove timestamp set outside of ProcessFunction. out.asInstanceOf[TimestampedCollector[_]].eraseTimestamp() // gets all rows for the triggering timestamps 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 6c62b1784ab2b..04b63a157fe34 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 @@ -161,7 +161,7 @@ abstract class RowTimeUnboundedOver( return } - // remove StreamRecord timestamp + // remove timestamp set outside of ProcessFunction. out.asInstanceOf[TimestampedCollector[_]].eraseTimestamp() val keyIterator = rowMapState.keys.iterator diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala index 438f697d7750d..d62c7b975957e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SortUtil.scala @@ -142,7 +142,7 @@ object SortUtil { } new RowComparator( - new RowSchema(inputType).physicalArity, + new RowSchema(inputType).arity, sortFields.toArray, fieldComps.toArray, new Array[TypeSerializer[AnyRef]](0), // not required because we only compare objects. diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala index e0af07736ff0e..b5661139d2efe 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala @@ -418,8 +418,8 @@ object WindowJoinUtil { Some(rightType)) val conversion = generator.generateConverterResultExpression( - returnType.physicalTypeInfo, - returnType.physicalType.getFieldNames.asScala) + returnType.typeInfo, + returnType.fieldNames) // if other condition is none, then output the result directly val body = otherCondition match { @@ -444,7 +444,7 @@ object WindowJoinUtil { ruleDescription, classOf[FlatJoinFunction[Row, Row, Row]], body, - returnType.physicalTypeInfo) + returnType.typeInfo) } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TimeIndicatorTypeInfo.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TimeIndicatorTypeInfo.scala index 083f1ebfd0ec7..e0e054b836da6 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TimeIndicatorTypeInfo.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TimeIndicatorTypeInfo.scala @@ -40,6 +40,9 @@ class TimeIndicatorTypeInfo(val isEventTime: Boolean) object TimeIndicatorTypeInfo { + val ROWTIME_MARKER: Int = -1 + val PROCTIME_MARKER: Int = -2 + val ROWTIME_INDICATOR = new TimeIndicatorTypeInfo(true) val PROCTIME_INDICATOR = new TimeIndicatorTypeInfo(false) From 54bfed8debbe29784e0de7f07ebb277df68a4eb5 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Fri, 11 Aug 2017 16:43:07 +0200 Subject: [PATCH 3/7] minor improvement --- .../scala/org/apache/flink/table/codegen/CodeGenerator.scala | 1 - 1 file changed, 1 deletion(-) 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 5290a56f9d509..448c051bae273 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 @@ -1221,7 +1221,6 @@ abstract class CodeGenerator( |""".stripMargin } else if (nullCheck) { s""" - |$tmpTypeTerm $tmpTerm = $fieldTerm; |boolean $nullTerm = $fieldTerm == null; |$resultTypeTerm $resultTerm; |if ($nullTerm) { From b0b24011e7c0444e3e1ebaba810edc06e9c85ad6 Mon Sep 17 00:00:00 2001 From: twalthr Date: Sat, 12 Aug 2017 13:51:42 +0200 Subject: [PATCH 4/7] Efficient handling of rowtime timestamps --- .../table/api/BatchTableEnvironment.scala | 36 +- .../table/api/StreamTableEnvironment.scala | 245 +++++----- .../flink/table/api/TableEnvironment.scala | 73 +-- .../codegen/AggregationCodeGenerator.scala | 25 +- .../flink/table/codegen/CodeGenUtils.scala | 29 +- .../flink/table/codegen/CodeGenerator.scala | 14 +- .../table/codegen/calls/ScalarOperators.scala | 7 +- .../table/functions/ProctimeSqlFunction.scala | 2 + .../utils/UserDefinedFunctionUtils.scala | 6 +- .../DataStreamGroupWindowAggregate.scala | 4 +- ...putMapRunner.scala => CRowMapRunner.scala} | 2 +- ...ala => OutputRowtimeProcessFunction.scala} | 9 +- ...ion.scala => RowtimeProcessFunction.scala} | 7 +- .../runtime/aggregate/AggregateUtil.scala | 33 +- .../aggregate/RowTimeBoundedRangeOver.scala | 4 +- .../aggregate/RowTimeBoundedRowsOver.scala | 13 +- .../RowTimeSortProcessFunction.scala | 15 +- .../aggregate/RowTimeUnboundedOver.scala | 17 +- .../TimeWindowPropertyCollector.scala | 2 +- .../CRowToJavaTupleMapFunction.scala | 40 ++ .../CRowToJavaTupleMapRunner.scala} | 42 +- .../conversion/CRowToRowMapFunction.scala | 32 ++ .../CRowToScalaTupleMapFunction.scala | 33 ++ .../CRowToScalaTupleMapRunner.scala | 56 +++ .../typeutils/TimeIndicatorTypeInfo.scala | 13 +- .../harness/OverWindowHarnessTest.scala | 436 +++++++++--------- .../SortProcessFunctionHarnessTest.scala | 49 +- .../runtime/stream/TimeAttributesITCase.scala | 12 +- .../stream/table/TableSinkITCase.scala | 4 +- 29 files changed, 694 insertions(+), 566 deletions(-) rename flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/{CRowInputMapRunner.scala => CRowMapRunner.scala} (98%) rename flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/{WrappingTimestampSetterProcessFunction.scala => OutputRowtimeProcessFunction.scala} (89%) rename flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/{TimestampSetterProcessFunction.scala => RowtimeProcessFunction.scala} (89%) create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToJavaTupleMapFunction.scala rename flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/{CRowInputTupleOutputMapRunner.scala => conversion/CRowToJavaTupleMapRunner.scala} (65%) create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToRowMapFunction.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToScalaTupleMapFunction.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToScalaTupleMapRunner.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 78667a2e9cac3..a9d60ddfbf820 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 @@ -153,28 +153,22 @@ abstract class BatchTableEnvironment( physicalTypeInfo: TypeInformation[IN], schema: RowSchema, requestedTypeInfo: TypeInformation[OUT], - functionName: String): - Option[MapFunction[IN, OUT]] = { - - if (requestedTypeInfo.getTypeClass == classOf[Row]) { - // Row to Row, no conversion needed - None - } else { - // some type that is neither Row or CRow - - val converterFunction = generateRowConverterFunction[OUT]( - physicalTypeInfo.asInstanceOf[TypeInformation[Row]], - schema, - requestedTypeInfo, - functionName - ) - - val mapFunction = new MapRunner[IN, OUT]( - converterFunction.name, - converterFunction.code, - converterFunction.returnType) + functionName: String) + : Option[MapFunction[IN, OUT]] = { + + val converterFunction = generateRowConverterFunction[OUT]( + physicalTypeInfo.asInstanceOf[TypeInformation[Row]], + schema, + requestedTypeInfo, + functionName + ) - Some(mapFunction) + // add a runner if we need conversion + converterFunction.map { func => + new MapRunner[IN, OUT]( + func.name, + func.code, + func.returnType) } } 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 c4e1450c64ca1..8d8cebb0c471f 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 @@ -23,15 +23,15 @@ 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.`type`.{RelDataType, RelDataTypeField, RelDataTypeFieldImpl, RelRecordType} import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeField, RelDataTypeFieldImpl, RelRecordType} import org.apache.calcite.sql2rel.RelDecorrelator import org.apache.calcite.tools.{RuleSet, RuleSets} import org.apache.flink.api.common.functions.MapFunction -import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation} +import org.apache.flink.api.common.typeinfo.{AtomicType, SqlTimeTypeInfo, 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.{PojoTypeInfo, TupleTypeInfo} +import org.apache.flink.api.java.typeutils.{PojoTypeInfo, RowTypeInfo, TupleTypeInfo} import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo import org.apache.flink.streaming.api.TimeCharacteristic import org.apache.flink.streaming.api.datastream.DataStream @@ -44,12 +44,12 @@ import org.apache.flink.table.plan.nodes.datastream.{DataStreamRel, UpdateAsRetr import org.apache.flink.table.plan.rules.FlinkRuleSets import org.apache.flink.table.plan.schema.{DataStreamTable, RowSchema, StreamTableSourceTable} import org.apache.flink.table.plan.util.UpdatingPlanChecker +import org.apache.flink.table.runtime.conversion._ import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} -import org.apache.flink.table.runtime.{CRowInputJavaTupleOutputMapRunner, CRowInputMapRunner, CRowInputScalaTupleOutputMapRunner, WrappingTimestampSetterProcessFunction} +import org.apache.flink.table.runtime.{CRowMapRunner, OutputRowtimeProcessFunction} import org.apache.flink.table.sinks.{AppendStreamTableSink, RetractStreamTableSink, TableSink, UpsertStreamTableSink} import org.apache.flink.table.sources.{DefinedRowtimeAttribute, StreamTableSource, TableSource} import org.apache.flink.table.typeutils.{TimeIndicatorTypeInfo, TypeCheckUtils} -import org.apache.flink.types.Row import _root_.scala.collection.JavaConverters._ @@ -223,38 +223,33 @@ abstract class StreamTableEnvironment( /** * Creates a final converter that maps the internal row type to external type. * - * @param physicalTypeInfo the input of the sink + * @param inputTypeInfo the input of the sink * @param schema the input schema with correct field names (esp. for POJO field mapping) * @param requestedTypeInfo the output type of the sink * @param functionName name of the map function. Must not be unique but has to be a * valid Java class identifier. */ - protected def getConversionMapper[IN, OUT]( - physicalTypeInfo: TypeInformation[IN], + protected def getConversionMapper[OUT]( + inputTypeInfo: TypeInformation[CRow], schema: RowSchema, requestedTypeInfo: TypeInformation[OUT], - functionName: String): - MapFunction[IN, OUT] = { - - if (requestedTypeInfo.getTypeClass == classOf[Row]) { - // CRow to Row, only needs to be unwrapped - new MapFunction[CRow, Row] { - override def map(value: CRow): Row = value.row - }.asInstanceOf[MapFunction[IN, OUT]] - } else { - // Some type that is neither CRow nor Row - val converterFunction = generateRowConverterFunction[OUT]( - physicalTypeInfo.asInstanceOf[CRowTypeInfo].rowType, - schema, - requestedTypeInfo, - functionName - ) + functionName: String) + : MapFunction[CRow, OUT] = { + + val converterFunction = generateRowConverterFunction[OUT]( + inputTypeInfo.asInstanceOf[CRowTypeInfo].rowType, + schema, + requestedTypeInfo, + functionName + ) - new CRowInputMapRunner[OUT]( - converterFunction.name, - converterFunction.code, - converterFunction.returnType) - .asInstanceOf[MapFunction[IN, OUT]] + converterFunction match { + + case Some(func) => + new CRowMapRunner[OUT](func.name, func.code, func.returnType) + + case _ => + new CRowToRowMapFunction().asInstanceOf[MapFunction[CRow, OUT]] } } @@ -271,71 +266,62 @@ abstract class StreamTableEnvironment( physicalTypeInfo: TypeInformation[CRow], schema: RowSchema, requestedTypeInfo: TypeInformation[OUT], - functionName: String): - MapFunction[CRow, OUT] = { - - requestedTypeInfo match { - - // Scala tuple - case t: CaseClassTypeInfo[_] - if t.getTypeClass == classOf[(_, _)] && t.getTypeAt(0) == Types.BOOLEAN => - - val reqType = t.getTypeAt(1).asInstanceOf[TypeInformation[Any]] - if (reqType.getTypeClass == classOf[Row]) { - // Requested type is Row. Just rewrap CRow in Tuple2 - new MapFunction[CRow, (Boolean, Row)] { - override def map(cRow: CRow): (Boolean, Row) = { - (cRow.change, cRow.row) - } - }.asInstanceOf[MapFunction[CRow, OUT]] - } else { - // Use a map function to convert Row into requested type and wrap result in Tuple2 - val converterFunction = generateRowConverterFunction( - physicalTypeInfo.asInstanceOf[CRowTypeInfo].rowType, - schema, - reqType, - functionName - ) - - new CRowInputScalaTupleOutputMapRunner( - converterFunction.name, - converterFunction.code, - requestedTypeInfo.asInstanceOf[TypeInformation[(Boolean, Any)]]) - .asInstanceOf[MapFunction[CRow, OUT]] + functionName: String) + : MapFunction[CRow, OUT] = requestedTypeInfo match { - } + // Scala tuple + case t: CaseClassTypeInfo[_] + if t.getTypeClass == classOf[(_, _)] && t.getTypeAt(0) == Types.BOOLEAN => - // Java tuple - case t: TupleTypeInfo[_] - if t.getTypeClass == classOf[JTuple2[_, _]] && t.getTypeAt(0) == Types.BOOLEAN => - - val reqType = t.getTypeAt(1).asInstanceOf[TypeInformation[Any]] - if (reqType.getTypeClass == classOf[Row]) { - // Requested type is Row. Just rewrap CRow in Tuple2 - new MapFunction[CRow, JTuple2[JBool, Row]] { - val outT = new JTuple2(true.asInstanceOf[JBool], null.asInstanceOf[Row]) - override def map(cRow: CRow): JTuple2[JBool, Row] = { - outT.f0 = cRow.change - outT.f1 = cRow.row - outT - } - }.asInstanceOf[MapFunction[CRow, OUT]] - } else { - // Use a map function to convert Row into requested type and wrap result in Tuple2 - val converterFunction = generateRowConverterFunction( - physicalTypeInfo.asInstanceOf[CRowTypeInfo].rowType, - schema, - reqType, - functionName - ) - - new CRowInputJavaTupleOutputMapRunner( - converterFunction.name, - converterFunction.code, - requestedTypeInfo.asInstanceOf[TypeInformation[JTuple2[JBool, Any]]]) - .asInstanceOf[MapFunction[CRow, OUT]] - } - } + val reqType = t.getTypeAt[Any](1) + + // convert Row into requested type and wrap result in Tuple2 + val converterFunction = generateRowConverterFunction( + physicalTypeInfo.asInstanceOf[CRowTypeInfo].rowType, + schema, + reqType, + functionName + ) + + converterFunction match { + + case Some(func) => + new CRowToScalaTupleMapRunner( + func.name, + func.code, + requestedTypeInfo.asInstanceOf[TypeInformation[(Boolean, Any)]] + ).asInstanceOf[MapFunction[CRow, OUT]] + + case _ => + new CRowToScalaTupleMapFunction().asInstanceOf[MapFunction[CRow, OUT]] + } + + // Java tuple + case t: TupleTypeInfo[_] + if t.getTypeClass == classOf[JTuple2[_, _]] && t.getTypeAt(0) == Types.BOOLEAN => + + val reqType = t.getTypeAt[Any](1) + + // convert Row into requested type and wrap result in Tuple2 + val converterFunction = generateRowConverterFunction( + physicalTypeInfo.asInstanceOf[CRowTypeInfo].rowType, + schema, + reqType, + functionName + ) + + converterFunction match { + + case Some(func) => + new CRowToJavaTupleMapRunner( + func.name, + func.code, + requestedTypeInfo.asInstanceOf[TypeInformation[JTuple2[JBool, Any]]] + ).asInstanceOf[MapFunction[CRow, OUT]] + + case _ => + new CRowToJavaTupleMapFunction().asInstanceOf[MapFunction[CRow, OUT]] + } } /** @@ -733,16 +719,42 @@ abstract class StreamTableEnvironment( // get CRow plan val plan: DataStream[CRow] = translateToCRow(logicalPlan, queryConfig) + val rowtimeFields = logicalType + .getFieldList.asScala + .filter(f => FlinkTypeFactory.isRowtimeIndicatorType(f.getType)) + + // convert the input type for the conversion mapper + // the input will be changed in the OutputRowtimeProcessFunction later + val convType = if (rowtimeFields.size > 1) { + throw new TableException( + s"Found more than one rowtime field: [${rowtimeFields.map(_.getName).mkString(", ")}] in " + + s"the table that should be converted to a DataStream.\n" + + s"Please select the rowtime field that should be used as event-time timestamp for the " + + s"DataStream by casting all other fields to TIMESTAMP.") + } else if (rowtimeFields.size == 1) { + val origRowType = plan.getType.asInstanceOf[CRowTypeInfo].rowType + val convFieldTypes = origRowType.getFieldTypes.map { t => + if (FlinkTypeFactory.isRowtimeIndicatorType(t)) { + SqlTimeTypeInfo.TIMESTAMP + } else { + t + } + } + CRowTypeInfo(new RowTypeInfo(convFieldTypes, origRowType.getFieldNames)) + } else { + plan.getType + } + // convert CRow to output type - val conversion = if (withChangeFlag) { + val conversion: MapFunction[CRow, A] = if (withChangeFlag) { getConversionMapperWithChanges( - plan.getType, + convType, new RowSchema(logicalType), tpe, "DataStreamSinkConversion") } else { getConversionMapper( - plan.getType, + convType, new RowSchema(logicalType), tpe, "DataStreamSinkConversion") @@ -750,42 +762,19 @@ abstract class StreamTableEnvironment( val rootParallelism = plan.getParallelism - val rowtimeFields = logicalType.getFieldList.asScala - .filter(f => FlinkTypeFactory.isRowtimeIndicatorType(f.getType)) - - if (rowtimeFields.isEmpty) { + val withRowtime = if (rowtimeFields.isEmpty) { // no rowtime field to set - conversion match { - case mapFunction: MapFunction[CRow, A] => - plan.map(mapFunction) - .returns(tpe) - .name(s"to: ${tpe.getTypeClass.getSimpleName}") - .setParallelism(rootParallelism) - } - } else if (rowtimeFields.size == 1) { - // set the only rowtime field as event-time timestamp for DataStream - val mapFunction = conversion match { - case mapFunction: MapFunction[CRow, A] => mapFunction - case _ => new MapFunction[CRow, A] { - override def map(cRow: CRow): A = cRow.asInstanceOf[A] - } - } - - plan.process( - new WrappingTimestampSetterProcessFunction[A]( - mapFunction, - rowtimeFields.head.getIndex)) - .returns(tpe) - .name(s"to: ${tpe.getTypeClass.getSimpleName}") - .setParallelism(rootParallelism) - + plan.map(conversion) } else { - throw new TableException( - s"Found more than one rowtime field: [${rowtimeFields.map(_.getName).mkString(", ")}] in " + - s"the table that should be converted to a DataStream.\n" + - s"Please select the rowtime field that should be used as event-time timestamp for the " + - s"DataStream by casting all other fields to TIMESTAMP.") + // set the only rowtime field as event-time timestamp for DataStream + // and convert it to SQL timestamp + plan.process(new OutputRowtimeProcessFunction[A](conversion, rowtimeFields.head.getIndex)) } + + withRowtime + .returns(tpe) + .name(s"to: ${tpe.getTypeClass.getSimpleName}") + .setParallelism(rootParallelism) } /** 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 b647c513f6b48..5fe8e77e93f93 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 @@ -38,7 +38,7 @@ import org.apache.calcite.tools._ import org.apache.flink.api.common.functions.MapFunction import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation} import org.apache.flink.api.common.typeutils.CompositeType -import org.apache.flink.api.java.typeutils._ +import org.apache.flink.api.java.typeutils.{RowTypeInfo, _} import org.apache.flink.api.java.{ExecutionEnvironment => JavaBatchExecEnv} import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo import org.apache.flink.api.scala.{ExecutionEnvironment => ScalaBatchExecEnv} @@ -48,26 +48,23 @@ import org.apache.flink.table.api.java.{BatchTableEnvironment => JavaBatchTableE import org.apache.flink.table.api.scala.{BatchTableEnvironment => ScalaBatchTableEnv, StreamTableEnvironment => ScalaStreamTableEnv} import org.apache.flink.table.calcite.{FlinkPlannerImpl, FlinkRelBuilder, FlinkTypeFactory, FlinkTypeSystem} import org.apache.flink.table.catalog.{ExternalCatalog, ExternalCatalogSchema} -import org.apache.flink.table.codegen.{FunctionCodeGenerator, ExpressionReducer, GeneratedFunction} -import org.apache.flink.table.expressions.{Alias, Expression, UnresolvedFieldReference} -import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils._ -import org.apache.flink.table.functions.AggregateFunction -import org.apache.flink.table.expressions._ -import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils.{checkForInstantiation, checkNotSingleton, createScalarSqlFunction, createTableSqlFunctions} -import org.apache.flink.table.functions.{ScalarFunction, TableFunction} +import org.apache.flink.table.codegen.{ExpressionReducer, FunctionCodeGenerator, GeneratedFunction} +import org.apache.flink.table.expressions.{Alias, Expression, UnresolvedFieldReference, _} +import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils.{checkForInstantiation, checkNotSingleton, createScalarSqlFunction, createTableSqlFunctions, _} +import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, TableFunction} import org.apache.flink.table.plan.cost.DataSetCostFactory import org.apache.flink.table.plan.logical.{CatalogNode, LogicalRelNode} import org.apache.flink.table.plan.rules.FlinkRuleSets import org.apache.flink.table.plan.schema.{RelTable, RowSchema} import org.apache.flink.table.sinks.TableSink import org.apache.flink.table.sources.{DefinedFieldNames, TableSource} +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo import org.apache.flink.table.validate.FunctionCatalog import org.apache.flink.types.Row -import org.apache.flink.api.java.typeutils.RowTypeInfo -import _root_.scala.collection.JavaConverters._ -import _root_.scala.collection.mutable.HashMap import _root_.scala.annotation.varargs +import _root_.scala.collection.JavaConverters._ +import _root_.scala.collection.mutable /** * The abstract base class for batch and stream TableEnvironments. @@ -108,10 +105,10 @@ abstract class TableEnvironment(val config: TableConfig) { private[flink] val attrNameCntr: AtomicInteger = new AtomicInteger(0) // registered external catalog names -> catalog - private val externalCatalogs = new HashMap[String, ExternalCatalog] + private val externalCatalogs = new mutable.HashMap[String, ExternalCatalog] /** Returns the table config to define the runtime behavior of the Table API. */ - def getConfig = config + def getConfig: TableConfig = config /** * Returns the operator table for this environment including a custom Calcite configuration. @@ -692,7 +689,7 @@ abstract class TableEnvironment(val config: TableConfig) { case _ => throw new TableException( "Field reference expression or alias on field expression expected.") } - case r: RowTypeInfo => { + case r: RowTypeInfo => exprs.zipWithIndex flatMap { case (UnresolvedFieldReference(name), idx) => Some((idx, name)) @@ -707,8 +704,7 @@ abstract class TableEnvironment(val config: TableConfig) { case _ => throw new TableException( "Field reference expression or alias on field expression expected.") } - - } + case tpe => throw new TableException( s"Source of type $tpe cannot be converted into Table.") } @@ -719,33 +715,47 @@ abstract class TableEnvironment(val config: TableConfig) { throw new TableException("Field name can not be '*'.") } - (fieldNames.toArray, fieldIndexes.toArray) + (fieldNames, fieldIndexes) } protected def generateRowConverterFunction[OUT]( inputTypeInfo: TypeInformation[Row], schema: RowSchema, requestedTypeInfo: TypeInformation[OUT], - functionName: String): - GeneratedFunction[MapFunction[Row, OUT], OUT] = { + functionName: String) + : Option[GeneratedFunction[MapFunction[Row, OUT], OUT]] = { // validate that at least the field types of physical and logical type match // we do that here to make sure that plan translation was correct if (schema.typeInfo != inputTypeInfo) { throw TableException( s"The field types of physical and logical row types do not match. " + - s"Physical type is [${schema.typeInfo}], Logical type is [${inputTypeInfo}]. " + + s"Physical type is [${schema.typeInfo}], Logical type is [$inputTypeInfo]. " + s"This is a bug and should not happen. Please file an issue.") } + // generic row needs no conversion + if (requestedTypeInfo.isInstanceOf[GenericTypeInfo[_]] && + requestedTypeInfo.getTypeClass == classOf[Row]) { + return None + } + val fieldTypes = schema.fieldTypeInfos val fieldNames = schema.fieldNames - // validate requested type + // check for valid type info if (requestedTypeInfo.getArity != fieldTypes.length) { throw new TableException( - s"Arity[${fieldTypes.length}] of result[${fieldTypes}] does not match " + - s"the number[${requestedTypeInfo.getArity}] of requested type[${requestedTypeInfo}].") + s"Arity [${fieldTypes.length}] of result [$fieldTypes] does not match " + + s"the number[${requestedTypeInfo.getArity}] of requested type [$requestedTypeInfo].") + } + + // check requested types + + def validateFieldType(fieldType: TypeInformation[_]): Unit = fieldType match { + case _: TimeIndicatorTypeInfo => + throw new TableException("The time indicator type is an internal type only.") + case _ => // ok } requestedTypeInfo match { @@ -758,9 +768,10 @@ abstract class TableEnvironment(val config: TableConfig) { throw new TableException(s"POJO does not define field name: $fName") } val requestedTypeInfo = pt.getTypeAt(pojoIdx) + validateFieldType(requestedTypeInfo) if (fType != requestedTypeInfo) { throw new TableException(s"Result field does not match requested type. " + - s"requested: $requestedTypeInfo; Actual: $fType") + s"Requested: $requestedTypeInfo; Actual: $fType") } } @@ -769,6 +780,7 @@ abstract class TableEnvironment(val config: TableConfig) { fieldTypes.zipWithIndex foreach { case (fieldTypeInfo, i) => val requestedTypeInfo = tt.getTypeAt(i) + validateFieldType(requestedTypeInfo) if (fieldTypeInfo != requestedTypeInfo) { throw new TableException(s"Result field does not match requested type. " + s"Requested: $requestedTypeInfo; Actual: $fieldTypeInfo") @@ -781,10 +793,11 @@ abstract class TableEnvironment(val config: TableConfig) { throw new TableException(s"Requested result type is an atomic type but " + s"result[$fieldTypes] has more or less than a single field.") } - val fieldTypeInfo = fieldTypes.head - if (fieldTypeInfo != at) { + val requestedTypeInfo = fieldTypes.head + validateFieldType(requestedTypeInfo) + if (requestedTypeInfo != at) { throw new TableException(s"Result field does not match requested type. " + - s"Requested: $at; Actual: $fieldTypeInfo") + s"Requested: $at; Actual: $requestedTypeInfo") } case _ => @@ -809,11 +822,13 @@ abstract class TableEnvironment(val config: TableConfig) { |return ${conversion.resultTerm}; |""".stripMargin - generator.generateFunction( + val generated = generator.generateFunction( functionName, classOf[MapFunction[Row, OUT]], body, requestedTypeInfo) + + Some(generated) } } @@ -972,7 +987,7 @@ object TableEnvironment { validateType(inputType) inputType match { - case t: CompositeType[_] => 0.until(t.getArity).map(t.getTypeAt(_)).toArray + case t: CompositeType[_] => 0.until(t.getArity).map(i => t.getTypeAt(i)).toArray case a: AtomicType[_] => Array(a.asInstanceOf[TypeInformation[_]]) case tpe => throw new TableException(s"Currently only CompositeType and AtomicType are supported.") diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/AggregationCodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/AggregationCodeGenerator.scala index 680eb44efc056..25527ccf3c2cf 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/AggregationCodeGenerator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/AggregationCodeGenerator.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.api.TableConfig import org.apache.flink.table.codegen.Indenter.toISC import org.apache.flink.table.codegen.CodeGenUtils.newName import org.apache.flink.table.functions.AggregateFunction +import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils.{getUserDefinedMethod, signatureToString} import org.apache.flink.table.runtime.aggregate.{GeneratedAggregations, SingleElementIterable} @@ -47,7 +48,6 @@ class AggregationCodeGenerator( * * @param name Class name of the function. * Does not need to be unique but has to be a valid Java class identifier. - * @param generator The code generator instance * @param physicalInputTypes Physical input row types * @param aggregates All aggregate functions * @param aggFields Indexes of the input fields for all aggregate functions @@ -68,7 +68,6 @@ class AggregationCodeGenerator( */ def generateAggregations( name: String, - generator: CodeGenerator, physicalInputTypes: Seq[TypeInformation[_]], aggregates: Array[AggregateFunction[_ <: Any, _ <: Any]], aggFields: Array[Array[Int]], @@ -86,28 +85,29 @@ class AggregationCodeGenerator( // get unique function name val funcName = newName(name) // register UDAGGs - val aggs = aggregates.map(a => generator.addReusableFunction(a)) + val aggs = aggregates.map(a => addReusableFunction(a)) // get java types of accumulators val accTypeClasses = aggregates.map { a => a.getClass.getMethod("createAccumulator").getReturnType } val accTypes = accTypeClasses.map(_.getCanonicalName) - // get java classes of input fields - val javaClasses = physicalInputTypes.map(t => t.getTypeClass) // get parameter lists for aggregation functions - val parameters = aggFields.map { inFields => + val parametersCode = aggFields.map { inFields => val fields = for (f <- inFields) yield - s"(${javaClasses(f).getCanonicalName}) input.getField($f)" + s"(${CodeGenUtils.boxedTypeTermForTypeInfo(physicalInputTypes(f))}) input.getField($f)" fields.mkString(", ") } - val methodSignaturesList = aggFields.map { - inFields => for (f <- inFields) yield javaClasses(f) + + // get method signatures + val classes = UserDefinedFunctionUtils.typeInfoToClass(physicalInputTypes) + val methodSignaturesList = aggFields.map { inFields => + inFields.map(classes(_)) } // check and validate the needed methods aggregates.zipWithIndex.map { - case (a, i) => { + case (a, i) => getUserDefinedMethod(a, "accumulate", Array(accTypeClasses(i)) ++ methodSignaturesList(i)) .getOrElse( throw new CodeGenException( @@ -159,7 +159,6 @@ class AggregationCodeGenerator( s"aggregate ${a.getClass.getCanonicalName}'.") ) } - } } def genSetAggregationResults: String = { @@ -208,7 +207,7 @@ class AggregationCodeGenerator( j""" | ${aggs(i)}.accumulate( | ((${accTypes(i)}) accs.getField($i)), - | ${parameters(i)});""".stripMargin + | ${parametersCode(i)});""".stripMargin }.mkString("\n") j"""$sig { @@ -229,7 +228,7 @@ class AggregationCodeGenerator( j""" | ${aggs(i)}.retract( | ((${accTypes(i)}) accs.getField($i)), - | ${parameters(i)});""".stripMargin + | ${parametersCode(i)});""".stripMargin }.mkString("\n") if (needRetract) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenUtils.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenUtils.scala index 1d8c926233b2f..161f9a3b055f4 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenUtils.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenUtils.scala @@ -28,7 +28,7 @@ import org.apache.flink.api.common.typeinfo.{FractionalTypeInfo, SqlTimeTypeInfo import org.apache.flink.api.common.typeutils.CompositeType import org.apache.flink.api.java.typeutils.{PojoTypeInfo, RowTypeInfo, TupleTypeInfo, TypeExtractor} import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo -import org.apache.flink.table.typeutils.{TimeIntervalTypeInfo, TypeCheckUtils} +import org.apache.flink.table.typeutils.{TimeIndicatorTypeInfo, TimeIntervalTypeInfo, TypeCheckUtils} object CodeGenUtils { @@ -90,6 +90,9 @@ object CodeGenUtils { case BOOLEAN_PRIMITIVE_ARRAY_TYPE_INFO => "boolean[]" case CHAR_PRIMITIVE_ARRAY_TYPE_INFO => "char[]" + // time indicators are represented as Long even if they seem to be Timestamp + case _: TimeIndicatorTypeInfo => "java.lang.Long" + case _ => tpe.getTypeClass.getCanonicalName } @@ -123,8 +126,10 @@ object CodeGenUtils { def qualifyEnum(enum: Enum[_]): String = enum.getClass.getCanonicalName + "." + enum.name() - def internalToTimePointCode(resultType: TypeInformation[_], resultTerm: String) = + def internalToTimePointCode(resultType: TypeInformation[_], resultTerm: String): String = resultType match { + case _: TimeIndicatorTypeInfo => + resultTerm // time indicators are not modified case SqlTimeTypeInfo.DATE => s"${qualifyMethod(BuiltInMethod.INTERNAL_TO_DATE.method)}($resultTerm)" case SqlTimeTypeInfo.TIME => @@ -133,7 +138,7 @@ object CodeGenUtils { s"${qualifyMethod(BuiltInMethod.INTERNAL_TO_TIMESTAMP.method)}($resultTerm)" } - def timePointToInternalCode(resultType: TypeInformation[_], resultTerm: String) = + def timePointToInternalCode(resultType: TypeInformation[_], resultTerm: String): String = resultType match { case SqlTimeTypeInfo.DATE => s"${qualifyMethod(BuiltInMethod.DATE_TO_INT.method)}($resultTerm)" @@ -157,43 +162,43 @@ object CodeGenUtils { // ---------------------------------------------------------------------------------------------- - def requireNumeric(genExpr: GeneratedExpression) = + def requireNumeric(genExpr: GeneratedExpression): Unit = if (!TypeCheckUtils.isNumeric(genExpr.resultType)) { throw new CodeGenException("Numeric expression type expected, but was " + s"'${genExpr.resultType}'.") } - def requireComparable(genExpr: GeneratedExpression) = + def requireComparable(genExpr: GeneratedExpression): Unit = if (!TypeCheckUtils.isComparable(genExpr.resultType)) { throw new CodeGenException(s"Comparable type expected, but was '${genExpr.resultType}'.") } - def requireString(genExpr: GeneratedExpression) = + def requireString(genExpr: GeneratedExpression): Unit = if (!TypeCheckUtils.isString(genExpr.resultType)) { throw new CodeGenException("String expression type expected.") } - def requireBoolean(genExpr: GeneratedExpression) = + def requireBoolean(genExpr: GeneratedExpression): Unit = if (!TypeCheckUtils.isBoolean(genExpr.resultType)) { throw new CodeGenException("Boolean expression type expected.") } - def requireTemporal(genExpr: GeneratedExpression) = + def requireTemporal(genExpr: GeneratedExpression): Unit = if (!TypeCheckUtils.isTemporal(genExpr.resultType)) { throw new CodeGenException("Temporal expression type expected.") } - def requireTimeInterval(genExpr: GeneratedExpression) = + def requireTimeInterval(genExpr: GeneratedExpression): Unit = if (!TypeCheckUtils.isTimeInterval(genExpr.resultType)) { throw new CodeGenException("Interval expression type expected.") } - def requireArray(genExpr: GeneratedExpression) = + def requireArray(genExpr: GeneratedExpression): Unit = if (!TypeCheckUtils.isArray(genExpr.resultType)) { throw new CodeGenException("Array expression type expected.") } - def requireInteger(genExpr: GeneratedExpression) = + def requireInteger(genExpr: GeneratedExpression): Unit = if (!TypeCheckUtils.isInteger(genExpr.resultType)) { throw new CodeGenException("Integer expression type expected.") } @@ -243,7 +248,7 @@ object CodeGenUtils { val fieldName = pt.getFieldNames()(index) getFieldAccessor(pt.getTypeClass, fieldName) - case _ => throw new CodeGenException(s"Unsupported composite type: '${compType}'") + case _ => throw new CodeGenException(s"Unsupported composite type: '$compType'") } } 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 448c051bae273..7d1b2013d4178 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 @@ -327,13 +327,13 @@ abstract class CodeGenerator( // initial type check if (returnType.getArity != fieldExprs.length) { throw new CodeGenException( - s"Arity[${returnType.getArity}] of result type[$returnType] does not match " + - s"number[${fieldExprs.length}] of expressions[$fieldExprs].") + s"Arity [${returnType.getArity}] of result type [$returnType] does not match " + + s"number [${fieldExprs.length}] of expressions [$fieldExprs].") } if (resultFieldNames.length != fieldExprs.length) { throw new CodeGenException( - s"Arity[${resultFieldNames.length}] of result field names[$resultFieldNames] does not " + - s"match number[${fieldExprs.length}] of expressions[$fieldExprs].") + s"Arity [${resultFieldNames.length}] of result field names [$resultFieldNames] does not " + + s"match number [${fieldExprs.length}] of expressions [$fieldExprs].") } // type check returnType match { @@ -341,8 +341,8 @@ abstract class CodeGenerator( fieldExprs.zipWithIndex foreach { case (fieldExpr, i) if fieldExpr.resultType != pt.getTypeAt(resultFieldNames(i)) => throw new CodeGenException( - s"Incompatible types of expression and result type. Expression[$fieldExpr] type is " + - s"[${fieldExpr.resultType}], result type is [${pt.getTypeAt(resultFieldNames(i))}]") + s"Incompatible types of expression and result type. Expression [$fieldExpr] type is" + + s" [${fieldExpr.resultType}], result type is [${pt.getTypeAt(resultFieldNames(i))}]") case _ => // ok } @@ -358,7 +358,7 @@ abstract class CodeGenerator( case at: AtomicType[_] if at != fieldExprs.head.resultType => throw new CodeGenException( - s"Incompatible types of expression and result type. Expression[${fieldExprs.head}] " + + s"Incompatible types of expression and result type. Expression [${fieldExprs.head}] " + s"type is [${fieldExprs.head.resultType}], result type is [$at]") case _ => // ok diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala index 01e9dffe817d2..7de7acaa9de45 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala @@ -26,7 +26,7 @@ import org.apache.flink.api.java.typeutils.{MapTypeInfo, ObjectArrayTypeInfo} import org.apache.flink.table.codegen.CodeGenUtils._ import org.apache.flink.table.codegen.calls.CallGenerator.generateCallIfArgsNotNull import org.apache.flink.table.codegen.{CodeGenException, CodeGenerator, GeneratedExpression} -import org.apache.flink.table.typeutils.{TimeIntervalTypeInfo, TypeCoercion} +import org.apache.flink.table.typeutils.{TimeIndicatorTypeInfo, TimeIntervalTypeInfo, TypeCoercion} import org.apache.flink.table.typeutils.TypeCheckUtils._ object ScalarOperators { @@ -543,6 +543,11 @@ object ScalarOperators { operand: GeneratedExpression, targetType: TypeInformation[_]) : GeneratedExpression = (operand.resultType, targetType) match { + + // special case: cast from TimeIndicatorTypeInfo to SqlTimeTypeInfo + case (ti: TimeIndicatorTypeInfo, SqlTimeTypeInfo.TIMESTAMP) => + operand.copy(resultType = SqlTimeTypeInfo.TIMESTAMP) // just replace the TypeInformation + // identity casting case (fromTp, toTp) if fromTp == toTp => operand diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/ProctimeSqlFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/ProctimeSqlFunction.scala index 4fb0378b83e63..2e7dba693fb04 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/ProctimeSqlFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/ProctimeSqlFunction.scala @@ -38,4 +38,6 @@ object ProctimeSqlFunction override def getMonotonicity(call: SqlOperatorBinding): SqlMonotonicity = SqlMonotonicity.INCREASING + + override def isDeterministic: Boolean = false } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala index 47469d1954710..b44c28ea34039 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala @@ -175,11 +175,11 @@ object UserDefinedFunctionUtils { } }) { throw new ValidationException( - s"Scala-style variable arguments in '${methodName}' methods are not supported. Please " + + s"Scala-style variable arguments in '$methodName' methods are not supported. Please " + s"add a @scala.annotation.varargs annotation.") } else if (found.length > 1) { throw new ValidationException( - s"Found multiple '${methodName}' methods which match the signature.") + s"Found multiple '$methodName' methods which match the signature.") } found.headOption } @@ -218,7 +218,7 @@ object UserDefinedFunctionUtils { if (methods.isEmpty) { throw new ValidationException( s"Function class '${function.getClass.getCanonicalName}' does not implement at least " + - s"one method named '${methodName}' which is public, not abstract and " + + s"one method named '$methodName' which is public, not abstract and " + s"(in case of table functions) not static.") } 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 ac63be188c0c8..0cf86f7069bcd 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 @@ -38,7 +38,7 @@ import org.apache.flink.table.plan.nodes.CommonAggregate import org.apache.flink.table.plan.schema.RowSchema import org.apache.flink.table.plan.nodes.datastream.DataStreamGroupWindowAggregate._ import org.apache.flink.table.plan.rules.datastream.DataStreamRetractionRules -import org.apache.flink.table.runtime.TimestampSetterProcessFunction +import org.apache.flink.table.runtime.RowtimeProcessFunction import org.apache.flink.table.runtime.aggregate.AggregateUtil._ import org.apache.flink.table.runtime.aggregate._ import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} @@ -149,7 +149,7 @@ class DataStreamGroupWindowAggregate( inputDS .process( - new TimestampSetterProcessFunction(timeIdx, CRowTypeInfo(inputSchema.typeInfo))) + new RowtimeProcessFunction(timeIdx, CRowTypeInfo(inputSchema.typeInfo))) .setParallelism(inputDS.getParallelism) .name(s"time attribute: ($timeAttribute)") } else { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowMapRunner.scala similarity index 98% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputMapRunner.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowMapRunner.scala index 109c6e1ebe35c..9ed9188b8d4d8 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputMapRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowMapRunner.scala @@ -30,7 +30,7 @@ import org.slf4j.LoggerFactory /** * MapRunner with [[CRow]] input. */ -class CRowInputMapRunner[OUT]( +class CRowMapRunner[OUT]( name: String, code: String, @transient var returnType: TypeInformation[OUT]) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/WrappingTimestampSetterProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/OutputRowtimeProcessFunction.scala similarity index 89% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/WrappingTimestampSetterProcessFunction.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/OutputRowtimeProcessFunction.scala index 8f12c3061763a..e7adbcee64de6 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/WrappingTimestampSetterProcessFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/OutputRowtimeProcessFunction.scala @@ -18,8 +18,6 @@ package org.apache.flink.table.runtime -import java.sql.Timestamp - import org.apache.calcite.runtime.SqlFunctions import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction} import org.apache.flink.configuration.Configuration @@ -32,7 +30,7 @@ import org.apache.flink.util.Collector * Wraps a ProcessFunction and sets a Timestamp field of a CRow as * [[org.apache.flink.streaming.runtime.streamrecord.StreamRecord]] timestamp. */ -class WrappingTimestampSetterProcessFunction[OUT]( +class OutputRowtimeProcessFunction[OUT]( function: MapFunction[CRow, OUT], rowtimeIdx: Int) extends ProcessFunction[CRow, OUT] { @@ -52,9 +50,12 @@ class WrappingTimestampSetterProcessFunction[OUT]( ctx: ProcessFunction[CRow, OUT]#Context, out: Collector[OUT]): Unit = { - val timestamp = SqlFunctions.toLong(in.row.getField(rowtimeIdx).asInstanceOf[Timestamp]) + val timestamp = in.row.getField(rowtimeIdx).asInstanceOf[Long] out.asInstanceOf[TimestampedCollector[_]].setAbsoluteTimestamp(timestamp) + val convertedTimestamp = SqlFunctions.internalToTimestamp(timestamp) + in.row.setField(rowtimeIdx, convertedTimestamp) + out.collect(function.map(in)) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/TimestampSetterProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/RowtimeProcessFunction.scala similarity index 89% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/TimestampSetterProcessFunction.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/RowtimeProcessFunction.scala index 00961f0f57d98..e192b075afdcd 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/TimestampSetterProcessFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/RowtimeProcessFunction.scala @@ -18,9 +18,6 @@ package org.apache.flink.table.runtime -import java.sql.Timestamp - -import org.apache.calcite.runtime.SqlFunctions import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.ResultTypeQueryable import org.apache.flink.streaming.api.functions.ProcessFunction @@ -32,7 +29,7 @@ import org.apache.flink.util.Collector * ProcessFunction to copy a timestamp from a [[org.apache.flink.types.Row]] field into the * [[org.apache.flink.streaming.runtime.streamrecord.StreamRecord]]. */ -class TimestampSetterProcessFunction( +class RowtimeProcessFunction( val rowtimeIdx: Int, @transient var returnType: TypeInformation[CRow]) extends ProcessFunction[CRow, CRow] @@ -43,7 +40,7 @@ class TimestampSetterProcessFunction( ctx: ProcessFunction[CRow, CRow]#Context, out: Collector[CRow]): Unit = { - val timestamp = SqlFunctions.toLong(in.row.getField(rowtimeIdx).asInstanceOf[Timestamp]) + val timestamp = in.row.getField(rowtimeIdx).asInstanceOf[Long] out.asInstanceOf[TimestampedCollector[CRow]].setAbsoluteTimestamp(timestamp) out.collect(in) } 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 52105e3941f8b..6304dc4bb69f8 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 @@ -96,7 +96,6 @@ object AggregateUtil { val genFunction = generator.generateAggregations( "UnboundedProcessingOverAggregateHelper", - generator, inputFieldTypeInfo, aggregates, aggFields, @@ -175,7 +174,6 @@ object AggregateUtil { val genFunction = generator.generateAggregations( "NonWindowedAggregationHelper", - generator, inputFieldTypes, aggregates, aggFields, @@ -240,7 +238,6 @@ object AggregateUtil { val genFunction = generator.generateAggregations( "BoundedOverAggregateHelper", - generator, inputFieldTypeInfo, aggregates, aggFields, @@ -372,7 +369,6 @@ object AggregateUtil { val genFunction = generator.generateAggregations( "DataSetAggregatePrepareMapHelper", - generator, inputFieldTypeInfo, aggregates, aggFieldIndexes, @@ -451,7 +447,6 @@ object AggregateUtil { // sliding time-window for partial aggregations val genFunction = generator.generateAggregations( "DataSetAggregatePrepareMapHelper", - generator, physicalInputTypes, aggregates, aggFieldIndexes, @@ -555,7 +550,6 @@ object AggregateUtil { val genPreAggFunction = generator.generateAggregations( "GroupingWindowAggregateHelper", - generator, physicalInputTypes, aggregates, aggFieldIndexes, @@ -572,7 +566,6 @@ object AggregateUtil { val genFinalAggFunction = generator.generateAggregations( "GroupingWindowAggregateHelper", - generator, physicalInputTypes, aggregates, aggFieldIndexes, @@ -714,7 +707,6 @@ object AggregateUtil { val genFunction = generator.generateAggregations( "GroupingWindowAggregateHelper", - generator, physicalInputTypes, aggregates, aggFieldIndexes, @@ -789,7 +781,6 @@ object AggregateUtil { val genFunction = generator.generateAggregations( "GroupingWindowAggregateHelper", - generator, physicalInputTypes, aggregates, aggFieldIndexes, @@ -870,7 +861,6 @@ object AggregateUtil { val genPreAggFunction = generator.generateAggregations( "DataSetAggregatePrepareMapHelper", - generator, inputFieldTypeInfo, aggregates, aggInFields, @@ -897,7 +887,6 @@ object AggregateUtil { val genFinalAggFunction = generator.generateAggregations( "DataSetAggregateFinalHelper", - generator, inputFieldTypeInfo, aggregates, aggInFields, @@ -921,7 +910,6 @@ object AggregateUtil { else { val genFunction = generator.generateAggregations( "DataSetAggregateHelper", - generator, inputFieldTypeInfo, aggregates, aggInFields, @@ -1019,7 +1007,6 @@ object AggregateUtil { val genFunction = generator.generateAggregations( "GroupingWindowAggregateHelper", - generator, inputFieldTypeInfo, aggregates, aggFields, @@ -1214,7 +1201,7 @@ object AggregateUtil { case DECIMAL => new DecimalSumWithRetractAggFunction case sqlType: SqlTypeName => - throw new TableException(s"Sum aggregate does no support type: '${sqlType}'") + throw new TableException(s"Sum aggregate does no support type: '$sqlType'") } } else { aggregates(index) = sqlTypeName match { @@ -1233,7 +1220,7 @@ object AggregateUtil { case DECIMAL => new DecimalSumAggFunction case sqlType: SqlTypeName => - throw new TableException(s"Sum aggregate does no support type: '${sqlType}'") + throw new TableException(s"Sum aggregate does no support type: '$sqlType'") } } @@ -1255,7 +1242,7 @@ object AggregateUtil { case DECIMAL => new DecimalSum0WithRetractAggFunction case sqlType: SqlTypeName => - throw new TableException(s"Sum0 aggregate does no support type: '${sqlType}'") + throw new TableException(s"Sum0 aggregate does no support type: '$sqlType'") } } else { aggregates(index) = sqlTypeName match { @@ -1274,7 +1261,7 @@ object AggregateUtil { case DECIMAL => new DecimalSum0AggFunction case sqlType: SqlTypeName => - throw new TableException(s"Sum0 aggregate does no support type: '${sqlType}'") + throw new TableException(s"Sum0 aggregate does no support type: '$sqlType'") } } @@ -1295,7 +1282,7 @@ object AggregateUtil { case DECIMAL => new DecimalAvgAggFunction case sqlType: SqlTypeName => - throw new TableException(s"Avg aggregate does no support type: '${sqlType}'") + throw new TableException(s"Avg aggregate does no support type: '$sqlType'") } case sqlMinMaxFunction: SqlMinMaxAggFunction => @@ -1322,7 +1309,7 @@ object AggregateUtil { new StringMinWithRetractAggFunction case sqlType: SqlTypeName => throw new TableException( - s"Min with retract aggregate does no support type: '${sqlType}'") + s"Min with retract aggregate does no support type: '$sqlType'") } } else { sqlTypeName match { @@ -1345,7 +1332,7 @@ object AggregateUtil { case VARCHAR | CHAR => new StringMinAggFunction case sqlType: SqlTypeName => - throw new TableException(s"Min aggregate does no support type: '${sqlType}'") + throw new TableException(s"Min aggregate does no support type: '$sqlType'") } } } else { @@ -1371,7 +1358,7 @@ object AggregateUtil { new StringMaxWithRetractAggFunction case sqlType: SqlTypeName => throw new TableException( - s"Max with retract aggregate does no support type: '${sqlType}'") + s"Max with retract aggregate does no support type: '$sqlType'") } } else { sqlTypeName match { @@ -1394,7 +1381,7 @@ object AggregateUtil { case VARCHAR | CHAR => new StringMaxAggFunction case sqlType: SqlTypeName => - throw new TableException(s"Max aggregate does no support type: '${sqlType}'") + throw new TableException(s"Max aggregate does no support type: '$sqlType'") } } } @@ -1463,7 +1450,7 @@ object AggregateUtil { relDataType.head.getIndex } else { throw TableException( - s"Encountered more than one time attribute with the same name: '${relDataType}'") + s"Encountered more than one time attribute with the same name: '$relDataType'") } case e => throw TableException( "The time attribute of window in batch environment should be " + 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 ceb986d406d91..8a0d682379636 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 @@ -17,10 +17,8 @@ */ package org.apache.flink.table.runtime.aggregate -import java.sql.Timestamp import java.util.{ArrayList => JArrayList, List => JList} -import org.apache.calcite.runtime.SqlFunctions import org.apache.flink.api.common.state._ import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} import org.apache.flink.api.java.typeutils.{ListTypeInfo, RowTypeInfo} @@ -118,7 +116,7 @@ class RowTimeBoundedRangeOver( registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) // triggering timestamp for trigger calculation - val triggeringTs = SqlFunctions.toLong(input.getField(rowTimeIdx).asInstanceOf[Timestamp]) + val triggeringTs = input.getField(rowTimeIdx).asInstanceOf[Long] val lastTriggeringTs = lastTriggeringTsState.value 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 678a3b78bc91c..ba6584664daa0 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 @@ -17,11 +17,9 @@ */ package org.apache.flink.table.runtime.aggregate -import java.sql.Timestamp import java.util import java.util.{List => JList} -import org.apache.calcite.runtime.SqlFunctions import org.apache.flink.api.common.state._ import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} import org.apache.flink.api.java.typeutils.{ListTypeInfo, RowTypeInfo} @@ -29,11 +27,11 @@ import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.streaming.api.operators.TimestampedCollector 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} import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} -import org.slf4j.LoggerFactory +import org.apache.flink.types.Row +import org.apache.flink.util.{Collector, Preconditions} +import org.slf4j.{Logger, LoggerFactory} /** * Process Function for ROWS clause event-time bounded OVER window @@ -56,6 +54,8 @@ class RowTimeBoundedRowsOver( Preconditions.checkNotNull(aggregationStateType) Preconditions.checkNotNull(precedingOffset) + val LOG: Logger = LoggerFactory.getLogger(this.getClass) + private var output: CRow = _ // the state which keeps the last triggering timestamp @@ -73,7 +73,6 @@ class RowTimeBoundedRowsOver( // to this time stamp. private var dataState: MapState[Long, JList[Row]] = _ - val LOG = LoggerFactory.getLogger(this.getClass) private var function: GeneratedAggregations = _ override def open(config: Configuration) { @@ -127,7 +126,7 @@ class RowTimeBoundedRowsOver( registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) // triggering timestamp for trigger calculation - val triggeringTs = SqlFunctions.toLong(input.getField(rowTimeIdx).asInstanceOf[Timestamp]) + val triggeringTs = input.getField(rowTimeIdx).asInstanceOf[Long] val lastTriggeringTs = lastTriggeringTsState.value // check if the data is expired, if not, save the data and register event time timer diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala index fd586786c5e71..0d69355108986 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeSortProcessFunction.scala @@ -17,24 +17,17 @@ */ package org.apache.flink.table.runtime.aggregate -import java.sql.Timestamp +import java.util.{Collections, ArrayList => JArrayList, List => JList} -import org.apache.flink.api.common.state.ValueState -import org.apache.flink.api.common.state.ValueStateDescriptor -import org.apache.flink.api.common.state.MapState -import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.state.{MapState, MapStateDescriptor, ValueState, ValueStateDescriptor} import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} import org.apache.flink.api.java.typeutils.ListTypeInfo import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.streaming.api.operators.TimestampedCollector import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} import org.apache.flink.types.Row import org.apache.flink.util.{Collector, Preconditions} -import java.util.Collections -import java.util.{ArrayList => JArrayList, List => JList} - -import org.apache.calcite.runtime.SqlFunctions -import org.apache.flink.streaming.api.operators.TimestampedCollector /** * ProcessFunction to sort on event-time and possibly addtional secondary sort attributes. @@ -90,7 +83,7 @@ class RowTimeSortProcessFunction( val input = inputC.row // timestamp of the processed row - val rowtime = SqlFunctions.toLong(input.getField(rowtimeIdx).asInstanceOf[Timestamp]) + val rowtime = input.getField(rowtimeIdx).asInstanceOf[Long] val lastTriggeringTs = lastTriggeringTsState.value 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 04b63a157fe34..9210c002c6b52 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 @@ -17,23 +17,21 @@ */ package org.apache.flink.table.runtime.aggregate -import java.sql.Timestamp import java.util import java.util.{List => JList} -import org.apache.calcite.runtime.SqlFunctions +import org.apache.flink.api.common.state._ import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} +import org.apache.flink.api.java.typeutils.ListTypeInfo import org.apache.flink.configuration.Configuration -import org.apache.flink.types.Row import org.apache.flink.streaming.api.functions.ProcessFunction -import org.apache.flink.util.Collector -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 +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.slf4j.{Logger, LoggerFactory} /** @@ -52,6 +50,8 @@ abstract class RowTimeUnboundedOver( extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig) with Compiler[GeneratedAggregations] { + val LOG: Logger = LoggerFactory.getLogger(this.getClass) + protected var output: CRow = _ // state to hold the accumulators of the aggregations private var accumulatorState: ValueState[Row] = _ @@ -60,7 +60,6 @@ abstract class RowTimeUnboundedOver( // list to sort timestamps to access rows in timestamp order private var sortedTimestamps: util.LinkedList[Long] = _ - val LOG = LoggerFactory.getLogger(this.getClass) protected var function: GeneratedAggregations = _ override def open(config: Configuration) { @@ -111,7 +110,7 @@ abstract class RowTimeUnboundedOver( // register state-cleanup timer registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) - val timestamp = SqlFunctions.toLong(input.getField(rowTimeIdx).asInstanceOf[Timestamp]) + val timestamp = input.getField(rowTimeIdx).asInstanceOf[Long] val curWatermark = ctx.timerService().currentWatermark() // discard late record diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/TimeWindowPropertyCollector.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/TimeWindowPropertyCollector.scala index 4ec5239dcda17..16e4a0b480efd 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/TimeWindowPropertyCollector.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/TimeWindowPropertyCollector.scala @@ -59,7 +59,7 @@ abstract class TimeWindowPropertyCollector[T]( if (windowRowtimeOffset.isDefined) { output.setField( lastFieldPos + windowRowtimeOffset.get, - SqlFunctions.internalToTimestamp(windowEnd - 1)) + windowEnd - 1) } wrappedCollector.collect(record) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToJavaTupleMapFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToJavaTupleMapFunction.scala new file mode 100644 index 0000000000000..6b4f87e54d2a4 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToJavaTupleMapFunction.scala @@ -0,0 +1,40 @@ +/* + * 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.conversion + +import org.apache.flink.api.common.functions.MapFunction +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} +import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.types.Row + +import _root_.java.lang.{Boolean => JBool} + +/** + * Convert [[CRow]] to a [[JTuple2]] containing a [[Row]]. + */ +class CRowToJavaTupleMapFunction extends MapFunction[CRow, JTuple2[JBool, Row]] { + + val out: JTuple2[JBool, Row] = new JTuple2(true.asInstanceOf[JBool], null.asInstanceOf[Row]) + + override def map(cRow: CRow): JTuple2[JBool, Row] = { + out.f0 = cRow.change + out.f1 = cRow.row + out + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputTupleOutputMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToJavaTupleMapRunner.scala similarity index 65% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputTupleOutputMapRunner.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToJavaTupleMapRunner.scala index 6b3aa44fa97fc..c6393b1e35e4f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowInputTupleOutputMapRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToJavaTupleMapRunner.scala @@ -16,26 +16,24 @@ * limitations under the License. */ -package org.apache.flink.table.runtime +package org.apache.flink.table.runtime.conversion import java.lang.{Boolean => JBool} -import java.sql.Timestamp -import org.apache.calcite.runtime.SqlFunctions import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction} import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} 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.slf4j.LoggerFactory -import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} +import org.slf4j.{Logger, LoggerFactory} /** - * Convert [[CRow]] to a [[JTuple2]] + * Convert [[CRow]] to a [[JTuple2]]. */ -class CRowInputJavaTupleOutputMapRunner( +class CRowToJavaTupleMapRunner( name: String, code: String, @transient var returnType: TypeInformation[JTuple2[JBool, Any]]) @@ -43,7 +41,7 @@ class CRowInputJavaTupleOutputMapRunner( with ResultTypeQueryable[JTuple2[JBool, Any]] with Compiler[MapFunction[Row, Any]] { - val LOG = LoggerFactory.getLogger(this.getClass) + val LOG: Logger = LoggerFactory.getLogger(this.getClass) private var function: MapFunction[Row, Any] = _ private var tupleWrapper: JTuple2[JBool, Any] = _ @@ -64,31 +62,3 @@ class CRowInputJavaTupleOutputMapRunner( override def getProducedType: TypeInformation[JTuple2[JBool, Any]] = returnType } - -/** - * Convert [[CRow]] to a [[Tuple2]] - */ -class CRowInputScalaTupleOutputMapRunner( - name: String, - code: String, - @transient var returnType: TypeInformation[(Boolean, Any)]) - extends RichMapFunction[CRow, (Boolean, Any)] - with ResultTypeQueryable[(Boolean, Any)] - with Compiler[MapFunction[Row, Any]] { - - val LOG = LoggerFactory.getLogger(this.getClass) - - private var function: MapFunction[Row, Any] = _ - - override def open(parameters: Configuration): Unit = { - LOG.debug(s"Compiling MapFunction: $name \n\n Code:\n$code") - val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code) - LOG.debug("Instantiating MapFunction.") - function = clazz.newInstance() - } - - override def map(in: CRow): (Boolean, Any) = - (in.change, function.map(in.row)) - - override def getProducedType: TypeInformation[(Boolean, Any)] = returnType -} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToRowMapFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToRowMapFunction.scala new file mode 100644 index 0000000000000..050f15f9a1170 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToRowMapFunction.scala @@ -0,0 +1,32 @@ +/* + * 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.conversion + +import org.apache.flink.api.common.functions.MapFunction +import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.types.Row + +/** + * Maps a CRow to a Row. + */ +class CRowToRowMapFunction extends MapFunction[CRow, Row] { + + override def map(value: CRow): Row = value.row + +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToScalaTupleMapFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToScalaTupleMapFunction.scala new file mode 100644 index 0000000000000..6461cc4a5307f --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToScalaTupleMapFunction.scala @@ -0,0 +1,33 @@ +/* + * 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.conversion + +import org.apache.flink.api.common.functions.MapFunction +import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.types.Row + +/** + * Convert [[CRow]] to a [[Tuple2]]. + */ +class CRowToScalaTupleMapFunction extends MapFunction[CRow, (Boolean, Row)] { + + override def map(cRow: CRow): (Boolean, Row) = { + (cRow.change, cRow.row) + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToScalaTupleMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToScalaTupleMapRunner.scala new file mode 100644 index 0000000000000..c7d71a9e46e68 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToScalaTupleMapRunner.scala @@ -0,0 +1,56 @@ +/* + * 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.conversion + +import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction} +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.slf4j.{Logger, LoggerFactory} + +/** + * Convert [[CRow]] to a [[Tuple2]]. + */ +class CRowToScalaTupleMapRunner( + name: String, + code: String, + @transient var returnType: TypeInformation[(Boolean, Any)]) + extends RichMapFunction[CRow, (Boolean, Any)] + with ResultTypeQueryable[(Boolean, Any)] + with Compiler[MapFunction[Row, Any]] { + + val LOG: Logger = LoggerFactory.getLogger(this.getClass) + + private var function: MapFunction[Row, Any] = _ + + override def open(parameters: Configuration): Unit = { + LOG.debug(s"Compiling MapFunction: $name \n\n Code:\n$code") + val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code) + LOG.debug("Instantiating MapFunction.") + function = clazz.newInstance() + } + + override def map(in: CRow): (Boolean, Any) = + (in.change, function.map(in.row)) + + override def getProducedType: TypeInformation[(Boolean, Any)] = returnType +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TimeIndicatorTypeInfo.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TimeIndicatorTypeInfo.scala index e0e054b836da6..824f3fb3ca4d5 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TimeIndicatorTypeInfo.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TimeIndicatorTypeInfo.scala @@ -20,13 +20,14 @@ package org.apache.flink.table.typeutils import java.sql.Timestamp +import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo -import org.apache.flink.api.common.typeutils.TypeComparator -import org.apache.flink.api.common.typeutils.base.{SqlTimestampComparator, SqlTimestampSerializer} +import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer} +import org.apache.flink.api.common.typeutils.base.{LongSerializer, SqlTimestampComparator, SqlTimestampSerializer} /** * Type information for indicating event or processing time. However, it behaves like a - * regular SQL timestamp. + * regular SQL timestamp but is serialized as Long. */ class TimeIndicatorTypeInfo(val isEventTime: Boolean) extends SqlTimeTypeInfo[Timestamp]( @@ -34,6 +35,12 @@ class TimeIndicatorTypeInfo(val isEventTime: Boolean) SqlTimestampSerializer.INSTANCE, classOf[SqlTimestampComparator].asInstanceOf[Class[TypeComparator[Timestamp]]]) { + // this replaces the effective serializer by a LongSerializer + // it is a hacky but efficient solution to keep the object creation overhead low but still + // be compatible with the corresponding SqlTimestampTypeInfo + override def createSerializer(executionConfig: ExecutionConfig): TypeSerializer[Timestamp] = + LongSerializer.INSTANCE.asInstanceOf[TypeSerializer[Timestamp]] + override def toString: String = s"TimeIndicatorTypeInfo(${if (isEventTime) "rowtime" else "proctime" })" } 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 ba044bec11bd9..ba36e183aa40d 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 @@ -15,12 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.flink.table.runtime.harness import java.lang.{Long => JLong} import java.util.concurrent.ConcurrentLinkedQueue -import org.apache.calcite.runtime.SqlFunctions.{internalToTimestamp => toTS} import org.apache.flink.api.common.time.Time import org.apache.flink.api.common.typeinfo.BasicTypeInfo import org.apache.flink.streaming.api.operators.KeyedProcessOperator @@ -34,7 +34,7 @@ import org.junit.Test class OverWindowHarnessTest extends HarnessTestBase{ - protected var queryConfig = + protected var queryConfig: StreamQueryConfig = new StreamQueryConfig().withIdleStateRetentionTime(Time.seconds(2), Time.seconds(3)) @Test @@ -60,75 +60,75 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.setProcessingTime(1) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(1), "aaa", 1L: JLong), true))) + CRow(Row.of(1L: JLong, "aaa", 1L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(1), "bbb", 10L: JLong), true))) + CRow(Row.of(1L: JLong, "bbb", 10L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(1), "aaa", 2L: JLong), true))) + CRow(Row.of(1L: JLong, "aaa", 2L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(1), "aaa", 3L: JLong), true))) + CRow(Row.of(1L: JLong, "aaa", 3L: JLong), change = true))) // register cleanup timer with 4100 testHarness.setProcessingTime(1100) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(1), "bbb", 20L: JLong), true))) + CRow(Row.of(1L: JLong, "bbb", 20L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(1), "aaa", 4L: JLong), true))) + CRow(Row.of(1L: JLong, "aaa", 4L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(1), "aaa", 5L: JLong), true))) + CRow(Row.of(1L: JLong, "aaa", 5L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(1), "aaa", 6L: JLong), true))) + CRow(Row.of(1L: JLong, "aaa", 6L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(1), "bbb", 30L: JLong), true))) + CRow(Row.of(1L: JLong, "bbb", 30L: JLong), change = true))) // register cleanup timer with 6001 testHarness.setProcessingTime(3001) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(2), "aaa", 7L: JLong), true))) + CRow(Row.of(2L: JLong, "aaa", 7L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(2), "aaa", 8L: JLong), true))) + CRow(Row.of(2L: JLong, "aaa", 8L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(2), "aaa", 9L: JLong), true))) + CRow(Row.of(2L: JLong, "aaa", 9L: JLong), change = true))) // trigger cleanup timer and register cleanup timer with 9002 testHarness.setProcessingTime(6002) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(2), "aaa", 10L: JLong), true))) + CRow(Row.of(2L: JLong, "aaa", 10L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(2), "bbb", 40L: JLong), true))) + CRow(Row.of(2L: JLong, "bbb", 40L: JLong), change = true))) val result = testHarness.getOutput val expectedOutput = new ConcurrentLinkedQueue[Object]() expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(1), "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true))) + CRow(Row.of(1L: JLong, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(1), "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true))) + CRow(Row.of(1L: JLong, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(1), "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true))) + CRow(Row.of(1L: JLong, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(1), "aaa", 3L: JLong, 2L: JLong, 3L: JLong), true))) + CRow(Row.of(1L: JLong, "aaa", 3L: JLong, 2L: JLong, 3L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(1), "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true))) + CRow(Row.of(1L: JLong, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(1), "aaa", 4L: JLong, 3L: JLong, 4L: JLong), true))) + CRow(Row.of(1L: JLong, "aaa", 4L: JLong, 3L: JLong, 4L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(1), "aaa", 5L: JLong, 4L: JLong, 5L: JLong), true))) + CRow(Row.of(1L: JLong, "aaa", 5L: JLong, 4L: JLong, 5L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(1), "aaa", 6L: JLong, 5L: JLong, 6L: JLong), true))) + CRow(Row.of(1L: JLong, "aaa", 6L: JLong, 5L: JLong, 6L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(1), "bbb", 30L: JLong, 20L: JLong, 30L: JLong), true))) + CRow(Row.of(1L: JLong, "bbb", 30L: JLong, 20L: JLong, 30L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(2), "aaa", 7L: JLong, 6L: JLong, 7L: JLong), true))) + CRow(Row.of(2L: JLong, "aaa", 7L: JLong, 6L: JLong, 7L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(2), "aaa", 8L: JLong, 7L: JLong, 8L: JLong), true))) + CRow(Row.of(2L: JLong, "aaa", 8L: JLong, 7L: JLong, 8L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(2), "aaa", 9L: JLong, 8L: JLong, 9L: JLong), true))) + CRow(Row.of(2L: JLong, "aaa", 9L: JLong, 8L: JLong, 9L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(2), "aaa", 10L: JLong, 10L: JLong, 10L: JLong), true))) + CRow(Row.of(2L: JLong, "aaa", 10L: JLong, 10L: JLong, 10L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(2), "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true))) + CRow(Row.of(2L: JLong, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), change = true))) verify(expectedOutput, result, new RowResultSortComparator()) @@ -160,51 +160,51 @@ class OverWindowHarnessTest extends HarnessTestBase{ // register cleanup timer with 3003 testHarness.setProcessingTime(3) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 1L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 1L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "bbb", 10L: JLong), true))) + CRow(Row.of(0L: JLong, "bbb", 10L: JLong), change = true))) testHarness.setProcessingTime(4) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 2L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 2L: JLong), change = true))) // trigger cleanup timer and register cleanup timer with 6003 testHarness.setProcessingTime(3003) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 3L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 3L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "bbb", 20L: JLong), true))) + CRow(Row.of(0L: JLong, "bbb", 20L: JLong), change = true))) testHarness.setProcessingTime(5) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 4L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 4L: JLong), change = true))) // register cleanup timer with 9002 testHarness.setProcessingTime(6002) testHarness.setProcessingTime(7002) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 5L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 5L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 6L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 6L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "bbb", 30L: JLong), true))) + CRow(Row.of(0L: JLong, "bbb", 30L: JLong), change = true))) // register cleanup timer with 14002 testHarness.setProcessingTime(11002) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 7L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 7L: JLong), change = true))) testHarness.setProcessingTime(11004) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 8L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 8L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 9L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 9L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 10L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 10L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "bbb", 40L: JLong), true))) + CRow(Row.of(0L: JLong, "bbb", 40L: JLong), change = true))) testHarness.setProcessingTime(11006) @@ -214,33 +214,33 @@ class OverWindowHarnessTest extends HarnessTestBase{ // all elements at the same proc timestamp have the same value per key expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true))) + CRow(Row.of(0L: JLong, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 3L: JLong, 3L: JLong, 4L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 3L: JLong, 3L: JLong, 4L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "bbb", 20L: JLong, 20L: JLong, 20L: JLong), true))) + CRow(Row.of(0L: JLong, "bbb", 20L: JLong, 20L: JLong, 20L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 4L: JLong, 4L: JLong, 4L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 4L: JLong, 4L: JLong, 4L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 5L: JLong, 5L: JLong, 6L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 5L: JLong, 5L: JLong, 6L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 6L: JLong, 5L: JLong, 6L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 6L: JLong, 5L: JLong, 6L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "bbb", 30L: JLong, 30L: JLong, 30L: JLong), true))) + CRow(Row.of(0L: JLong, "bbb", 30L: JLong, 30L: JLong, 30L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 7L: JLong, 7L: JLong, 7L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 7L: JLong, 7L: JLong, 7L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 8L: JLong, 7L: JLong, 10L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 8L: JLong, 7L: JLong, 10L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 9L: JLong, 7L: JLong, 10L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 9L: JLong, 7L: JLong, 10L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 10L: JLong, 7L: JLong, 10L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 10L: JLong, 7L: JLong, 10L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true))) + CRow(Row.of(0L: JLong, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), change = true))) verify(expectedOutput, result, new RowResultSortComparator()) @@ -268,69 +268,69 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.setProcessingTime(1003) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 1L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 1L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "bbb", 10L: JLong), true))) + CRow(Row.of(0L: JLong, "bbb", 10L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 2L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 2L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 3L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 3L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "bbb", 20L: JLong), true))) + CRow(Row.of(0L: JLong, "bbb", 20L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 4L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 4L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 5L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 5L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 6L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 6L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "bbb", 30L: JLong), true))) + CRow(Row.of(0L: JLong, "bbb", 30L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 7L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 7L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 8L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 8L: JLong), change = true))) // trigger cleanup timer and register cleanup timer with 8003 testHarness.setProcessingTime(5003) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 9L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 9L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 10L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 10L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(0), "bbb", 40L: JLong), true))) + CRow(Row.of(0L: JLong, "bbb", 40L: JLong), change = true))) val result = testHarness.getOutput val expectedOutput = new ConcurrentLinkedQueue[Object]() expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true))) + CRow(Row.of(0L: JLong, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true))) + CRow(Row.of(0L: JLong, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 4L: JLong, 1L: JLong, 4L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 4L: JLong, 1L: JLong, 4L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 5L: JLong, 1L: JLong, 5L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 5L: JLong, 1L: JLong, 5L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 6L: JLong, 1L: JLong, 6L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 6L: JLong, 1L: JLong, 6L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true))) + CRow(Row.of(0L: JLong, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 7L: JLong, 1L: JLong, 7L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 7L: JLong, 1L: JLong, 7L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 8L: JLong, 1L: JLong, 8L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 8L: JLong, 1L: JLong, 8L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 9L: JLong, 9L: JLong, 9L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 9L: JLong, 9L: JLong, 9L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "aaa", 10L: JLong, 9L: JLong, 10L: JLong), true))) + CRow(Row.of(0L: JLong, "aaa", 10L: JLong, 9L: JLong, 10L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(0), "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true))) + CRow(Row.of(0L: JLong, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), change = true))) verify(expectedOutput, result, new RowResultSortComparator()) testHarness.close() @@ -361,51 +361,51 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processWatermark(1) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(2), "aaa", 1L: JLong), true))) + CRow(Row.of(2L: JLong, "aaa", 1L: JLong), change = true))) testHarness.processWatermark(2) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(3), "bbb", 10L: JLong), true))) + CRow(Row.of(3L: JLong, "bbb", 10L: JLong), change = true))) testHarness.processWatermark(4000) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(4001), "aaa", 2L: JLong), true))) + CRow(Row.of(4001L: JLong, "aaa", 2L: JLong), change = true))) testHarness.processWatermark(4001) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(4002), "aaa", 3L: JLong), true))) + CRow(Row.of(4002L: JLong, "aaa", 3L: JLong), change = true))) testHarness.processWatermark(4002) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(4003), "aaa", 4L: JLong), true))) + CRow(Row.of(4003L: JLong, "aaa", 4L: JLong), change = true))) testHarness.processWatermark(4800) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(4801), "bbb", 25L: JLong), true))) + CRow(Row.of(4801L: JLong, "bbb", 25L: JLong), change = true))) testHarness.processWatermark(6500) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(6501), "aaa", 5L: JLong), true))) + CRow(Row.of(6501L: JLong, "aaa", 5L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(6501), "aaa", 6L: JLong), true))) + CRow(Row.of(6501L: JLong, "aaa", 6L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(6501), "bbb", 30L: JLong), true))) + CRow(Row.of(6501L: JLong, "bbb", 30L: JLong), change = true))) testHarness.processWatermark(7000) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(7001), "aaa", 7L: JLong), true))) + CRow(Row.of(7001L: JLong, "aaa", 7L: JLong), change = true))) testHarness.processWatermark(8000) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(8001), "aaa", 8L: JLong), true))) + CRow(Row.of(8001L: JLong, "aaa", 8L: JLong), change = true))) testHarness.processWatermark(12000) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(12001), "aaa", 9L: JLong), true))) + CRow(Row.of(12001L: JLong, "aaa", 9L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(12001), "aaa", 10L: JLong), true))) + CRow(Row.of(12001L: JLong, "aaa", 10L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(12001), "bbb", 40L: JLong), true))) + CRow(Row.of(12001L: JLong, "bbb", 40L: JLong), change = true))) testHarness.processWatermark(19000) @@ -415,10 +415,10 @@ class OverWindowHarnessTest extends HarnessTestBase{ // check that state is removed after max retention time testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(20001), "ccc", 1L: JLong), true))) // clean-up 3000 + CRow(Row.of(20001L: JLong, "ccc", 1L: JLong), change = true))) // clean-up 3000 testHarness.setProcessingTime(2500) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(20002), "ccc", 2L: JLong), true))) // clean-up 4500 + CRow(Row.of(20002L: JLong, "ccc", 2L: JLong), change = true))) // clean-up 4500 testHarness.processWatermark(20010) // compute output assert(testHarness.numKeyedStateEntries() > 0) // check that we have state @@ -430,7 +430,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ // check that state is only removed if all data was processed testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(20011), "ccc", 3L: JLong), true))) // clean-up 6500 + CRow(Row.of(20011L: JLong, "ccc", 3L: JLong), change = true))) // clean-up 6500 assert(testHarness.numKeyedStateEntries() > 0) // check that we have state testHarness.setProcessingTime(6500) // clean-up attempt but rescheduled to 8500 @@ -450,40 +450,40 @@ class OverWindowHarnessTest extends HarnessTestBase{ // all elements at the same row-time have the same value per key expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(2), "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true))) + CRow(Row.of(2L: JLong, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(3), "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true))) + CRow(Row.of(3L: JLong, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(4001), "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true))) + CRow(Row.of(4001L: JLong, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(4002), "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true))) + CRow(Row.of(4002L: JLong, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(4003), "aaa", 4L: JLong, 2L: JLong, 4L: JLong), true))) + CRow(Row.of(4003L: JLong, "aaa", 4L: JLong, 2L: JLong, 4L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(4801), "bbb", 25L: JLong, 25L: JLong, 25L: JLong), true))) + CRow(Row.of(4801L: JLong, "bbb", 25L: JLong, 25L: JLong, 25L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(6501), "aaa", 5L: JLong, 2L: JLong, 6L: JLong), true))) + CRow(Row.of(6501L: JLong, "aaa", 5L: JLong, 2L: JLong, 6L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(6501), "aaa", 6L: JLong, 2L: JLong, 6L: JLong), true))) + CRow(Row.of(6501L: JLong, "aaa", 6L: JLong, 2L: JLong, 6L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(7001), "aaa", 7L: JLong, 2L: JLong, 7L: JLong), true))) + CRow(Row.of(7001L: JLong, "aaa", 7L: JLong, 2L: JLong, 7L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(8001), "aaa", 8L: JLong, 2L: JLong, 8L: JLong), true))) + CRow(Row.of(8001L: JLong, "aaa", 8L: JLong, 2L: JLong, 8L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(6501), "bbb", 30L: JLong, 25L: JLong, 30L: JLong), true))) + CRow(Row.of(6501L: JLong, "bbb", 30L: JLong, 25L: JLong, 30L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(12001), "aaa", 9L: JLong, 8L: JLong, 10L: JLong), true))) + CRow(Row.of(12001L: JLong, "aaa", 9L: JLong, 8L: JLong, 10L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(12001), "aaa", 10L: JLong, 8L: JLong, 10L: JLong), true))) + CRow(Row.of(12001L: JLong, "aaa", 10L: JLong, 8L: JLong, 10L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(12001), "bbb", 40L: JLong, 40L: JLong, 40L: JLong), true))) + CRow(Row.of(12001L: JLong, "bbb", 40L: JLong, 40L: JLong, 40L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(20001), "ccc", 1L: JLong, 1L: JLong, 1L: JLong), true))) + CRow(Row.of(20001L: JLong, "ccc", 1L: JLong, 1L: JLong, 1L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(20002), "ccc", 2L: JLong, 1L: JLong, 2L: JLong), true))) + CRow(Row.of(20002L: JLong, "ccc", 2L: JLong, 1L: JLong, 2L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(20011), "ccc", 3L: JLong, 3L: JLong, 3L: JLong), true))) + CRow(Row.of(20011L: JLong, "ccc", 3L: JLong, 3L: JLong, 3L: JLong), change = true))) verify(expectedOutput, result, new RowResultSortComparator()) testHarness.close() @@ -511,47 +511,47 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processWatermark(800) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(801), "aaa", 1L: JLong), true))) + CRow(Row.of(801L: JLong, "aaa", 1L: JLong), change = true))) testHarness.processWatermark(2500) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(2501), "bbb", 10L: JLong), true))) + CRow(Row.of(2501L: JLong, "bbb", 10L: JLong), change = true))) testHarness.processWatermark(4000) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(4001), "aaa", 2L: JLong), true))) + CRow(Row.of(4001L: JLong, "aaa", 2L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(4001), "aaa", 3L: JLong), true))) + CRow(Row.of(4001L: JLong, "aaa", 3L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(4001), "bbb", 20L: JLong), true))) + CRow(Row.of(4001L: JLong, "bbb", 20L: JLong), change = true))) testHarness.processWatermark(4800) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(4801), "aaa", 4L: JLong), true))) + CRow(Row.of(4801L: JLong, "aaa", 4L: JLong), change = true))) testHarness.processWatermark(6500) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(6501), "aaa", 5L: JLong), true))) + CRow(Row.of(6501L: JLong, "aaa", 5L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(6501), "aaa", 6L: JLong), true))) + CRow(Row.of(6501L: JLong, "aaa", 6L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(6501), "bbb", 30L: JLong), true))) + CRow(Row.of(6501L: JLong, "bbb", 30L: JLong), change = true))) testHarness.processWatermark(7000) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(7001), "aaa", 7L: JLong), true))) + CRow(Row.of(7001L: JLong, "aaa", 7L: JLong), change = true))) testHarness.processWatermark(8000) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(8001), "aaa", 8L: JLong), true))) + CRow(Row.of(8001L: JLong, "aaa", 8L: JLong), change = true))) testHarness.processWatermark(12000) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(12001), "aaa", 9L: JLong), true))) + CRow(Row.of(12001L: JLong, "aaa", 9L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(12001), "aaa", 10L: JLong), true))) + CRow(Row.of(12001L: JLong, "aaa", 10L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(12001), "bbb", 40L: JLong), true))) + CRow(Row.of(12001L: JLong, "bbb", 40L: JLong), change = true))) testHarness.processWatermark(19000) @@ -561,10 +561,10 @@ class OverWindowHarnessTest extends HarnessTestBase{ // check that state is removed after max retention time testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(20001), "ccc", 1L: JLong), true))) // clean-up 3000 + CRow(Row.of(20001L: JLong, "ccc", 1L: JLong), change = true))) // clean-up 3000 testHarness.setProcessingTime(2500) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(20002), "ccc", 2L: JLong), true))) // clean-up 4500 + CRow(Row.of(20002L: JLong, "ccc", 2L: JLong), change = true))) // clean-up 4500 testHarness.processWatermark(20010) // compute output assert(testHarness.numKeyedStateEntries() > 0) // check that we have state @@ -575,7 +575,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ // check that state is only removed if all data was processed testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(20011), "ccc", 3L: JLong), true))) // clean-up 6500 + CRow(Row.of(20011L: JLong, "ccc", 3L: JLong), change = true))) // clean-up 6500 assert(testHarness.numKeyedStateEntries() > 0) // check that we have state testHarness.setProcessingTime(6500) // clean-up attempt but rescheduled to 8500 @@ -595,40 +595,40 @@ class OverWindowHarnessTest extends HarnessTestBase{ val expectedOutput = new ConcurrentLinkedQueue[Object]() expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(801), "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true))) + CRow(Row.of(801L: JLong, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(2501), "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true))) + CRow(Row.of(2501L: JLong, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(4001), "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true))) + CRow(Row.of(4001L: JLong, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(4001), "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true))) + CRow(Row.of(4001L: JLong, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(4001), "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true))) + CRow(Row.of(4001L: JLong, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(4801), "aaa", 4L: JLong, 2L: JLong, 4L: JLong), true))) + CRow(Row.of(4801L: JLong, "aaa", 4L: JLong, 2L: JLong, 4L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(6501), "aaa", 5L: JLong, 3L: JLong, 5L: JLong), true))) + CRow(Row.of(6501L: JLong, "aaa", 5L: JLong, 3L: JLong, 5L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(6501), "aaa", 6L: JLong, 4L: JLong, 6L: JLong), true))) + CRow(Row.of(6501L: JLong, "aaa", 6L: JLong, 4L: JLong, 6L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(6501), "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true))) + CRow(Row.of(6501L: JLong, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(7001), "aaa", 7L: JLong, 5L: JLong, 7L: JLong), true))) + CRow(Row.of(7001L: JLong, "aaa", 7L: JLong, 5L: JLong, 7L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(8001), "aaa", 8L: JLong, 6L: JLong, 8L: JLong), true))) + CRow(Row.of(8001L: JLong, "aaa", 8L: JLong, 6L: JLong, 8L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(12001), "aaa", 9L: JLong, 7L: JLong, 9L: JLong), true))) + CRow(Row.of(12001L: JLong, "aaa", 9L: JLong, 7L: JLong, 9L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(12001), "aaa", 10L: JLong, 8L: JLong, 10L: JLong), true))) + CRow(Row.of(12001L: JLong, "aaa", 10L: JLong, 8L: JLong, 10L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(12001), "bbb", 40L: JLong, 20L: JLong, 40L: JLong), true))) + CRow(Row.of(12001L: JLong, "bbb", 40L: JLong, 20L: JLong, 40L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(20001), "ccc", 1L: JLong, 1L: JLong, 1L: JLong), true))) + CRow(Row.of(20001L: JLong, "ccc", 1L: JLong, 1L: JLong, 1L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(20002), "ccc", 2L: JLong, 1L: JLong, 2L: JLong), true))) + CRow(Row.of(20002L: JLong, "ccc", 2L: JLong, 1L: JLong, 2L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(20011), "ccc", 3L: JLong, 3L: JLong, 3L: JLong), true))) + CRow(Row.of(20011L: JLong, "ccc", 3L: JLong, 3L: JLong, 3L: JLong), change = true))) verify(expectedOutput, result, new RowResultSortComparator()) testHarness.close() @@ -659,47 +659,47 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.setProcessingTime(1000) testHarness.processWatermark(800) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(801), "aaa", 1L: JLong), true))) + CRow(Row.of(801L: JLong, "aaa", 1L: JLong), change = true))) testHarness.processWatermark(2500) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(2501), "bbb", 10L: JLong), true))) + CRow(Row.of(2501L: JLong, "bbb", 10L: JLong), change = true))) testHarness.processWatermark(4000) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(4001), "aaa", 2L: JLong), true))) + CRow(Row.of(4001L: JLong, "aaa", 2L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(4001), "aaa", 3L: JLong), true))) + CRow(Row.of(4001L: JLong, "aaa", 3L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(4001), "bbb", 20L: JLong), true))) + CRow(Row.of(4001L: JLong, "bbb", 20L: JLong), change = true))) testHarness.processWatermark(4800) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(4801), "aaa", 4L: JLong), true))) + CRow(Row.of(4801L: JLong, "aaa", 4L: JLong), change = true))) testHarness.processWatermark(6500) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(6501), "aaa", 5L: JLong), true))) + CRow(Row.of(6501L: JLong, "aaa", 5L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(6501), "aaa", 6L: JLong), true))) + CRow(Row.of(6501L: JLong, "aaa", 6L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(6501), "bbb", 30L: JLong), true))) + CRow(Row.of(6501L: JLong, "bbb", 30L: JLong), change = true))) testHarness.processWatermark(7000) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(7001), "aaa", 7L: JLong), true))) + CRow(Row.of(7001L: JLong, "aaa", 7L: JLong), change = true))) testHarness.processWatermark(8000) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(8001), "aaa", 8L: JLong), true))) + CRow(Row.of(8001L: JLong, "aaa", 8L: JLong), change = true))) testHarness.processWatermark(12000) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(12001), "aaa", 9L: JLong), true))) + CRow(Row.of(12001L: JLong, "aaa", 9L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(12001), "aaa", 10L: JLong), true))) + CRow(Row.of(12001L: JLong, "aaa", 10L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(12001), "bbb", 40L: JLong), true))) + CRow(Row.of(12001L: JLong, "bbb", 40L: JLong), change = true))) testHarness.processWatermark(19000) @@ -712,10 +712,10 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processWatermark(20000) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(20001), "ccc", 1L: JLong), true))) // clean-up 5000 + CRow(Row.of(20001L: JLong, "ccc", 1L: JLong), change = true))) // clean-up 5000 testHarness.setProcessingTime(2500) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(20002), "ccc", 2L: JLong), true))) // clean-up 5000 + CRow(Row.of(20002L: JLong, "ccc", 2L: JLong), change = true))) // clean-up 5000 assert(testHarness.numKeyedStateEntries() > 0) testHarness.setProcessingTime(5000) // does not clean up, because data left. New timer 7000 @@ -733,38 +733,38 @@ class OverWindowHarnessTest extends HarnessTestBase{ // all elements at the same row-time have the same value per key expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(801), "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true))) + CRow(Row.of(801L: JLong, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(2501), "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true))) + CRow(Row.of(2501L: JLong, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(4001), "aaa", 2L: JLong, 1L: JLong, 3L: JLong), true))) + CRow(Row.of(4001L: JLong, "aaa", 2L: JLong, 1L: JLong, 3L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(4001), "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true))) + CRow(Row.of(4001L: JLong, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(4001), "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true))) + CRow(Row.of(4001L: JLong, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(4801), "aaa", 4L: JLong, 1L: JLong, 4L: JLong), true))) + CRow(Row.of(4801L: JLong, "aaa", 4L: JLong, 1L: JLong, 4L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(6501), "aaa", 5L: JLong, 1L: JLong, 6L: JLong), true))) + CRow(Row.of(6501L: JLong, "aaa", 5L: JLong, 1L: JLong, 6L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(6501), "aaa", 6L: JLong, 1L: JLong, 6L: JLong), true))) + CRow(Row.of(6501L: JLong, "aaa", 6L: JLong, 1L: JLong, 6L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(6501), "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true))) + CRow(Row.of(6501L: JLong, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(7001), "aaa", 7L: JLong, 1L: JLong, 7L: JLong), true))) + CRow(Row.of(7001L: JLong, "aaa", 7L: JLong, 1L: JLong, 7L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(8001), "aaa", 8L: JLong, 1L: JLong, 8L: JLong), true))) + CRow(Row.of(8001L: JLong, "aaa", 8L: JLong, 1L: JLong, 8L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(12001), "aaa", 9L: JLong, 1L: JLong, 10L: JLong), true))) + CRow(Row.of(12001L: JLong, "aaa", 9L: JLong, 1L: JLong, 10L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(12001), "aaa", 10L: JLong, 1L: JLong, 10L: JLong), true))) + CRow(Row.of(12001L: JLong, "aaa", 10L: JLong, 1L: JLong, 10L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(12001), "bbb", 40L: JLong, 10L: JLong, 40L: JLong), true))) + CRow(Row.of(12001L: JLong, "bbb", 40L: JLong, 10L: JLong, 40L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(20001), "ccc", 1L: JLong, 1L: JLong, 1L: JLong), true))) + CRow(Row.of(20001L: JLong, "ccc", 1L: JLong, 1L: JLong, 1L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(20002), "ccc", 2L: JLong, 1L: JLong, 2L: JLong), true))) + CRow(Row.of(20002L: JLong, "ccc", 2L: JLong, 1L: JLong, 2L: JLong), change = true))) verify(expectedOutput, result, new RowResultSortComparator()) testHarness.close() @@ -792,47 +792,47 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.setProcessingTime(1000) testHarness.processWatermark(800) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(801), "aaa", 1L: JLong), true))) + CRow(Row.of(801L: JLong, "aaa", 1L: JLong), change = true))) testHarness.processWatermark(2500) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(2501), "bbb", 10L: JLong), true))) + CRow(Row.of(2501L: JLong, "bbb", 10L: JLong), change = true))) testHarness.processWatermark(4000) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(4001), "aaa", 2L: JLong), true))) + CRow(Row.of(4001L: JLong, "aaa", 2L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(4001), "aaa", 3L: JLong), true))) + CRow(Row.of(4001L: JLong, "aaa", 3L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(4001), "bbb", 20L: JLong), true))) + CRow(Row.of(4001L: JLong, "bbb", 20L: JLong), change = true))) testHarness.processWatermark(4800) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(4801), "aaa", 4L: JLong), true))) + CRow(Row.of(4801L: JLong, "aaa", 4L: JLong), change = true))) testHarness.processWatermark(6500) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(6501), "aaa", 5L: JLong), true))) + CRow(Row.of(6501L: JLong, "aaa", 5L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(6501), "aaa", 6L: JLong), true))) + CRow(Row.of(6501L: JLong, "aaa", 6L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(6501), "bbb", 30L: JLong), true))) + CRow(Row.of(6501L: JLong, "bbb", 30L: JLong), change = true))) testHarness.processWatermark(7000) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(7001), "aaa", 7L: JLong), true))) + CRow(Row.of(7001L: JLong, "aaa", 7L: JLong), change = true))) testHarness.processWatermark(8000) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(8001), "aaa", 8L: JLong), true))) + CRow(Row.of(8001L: JLong, "aaa", 8L: JLong), change = true))) testHarness.processWatermark(12000) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(12001), "aaa", 9L: JLong), true))) + CRow(Row.of(12001L: JLong, "aaa", 9L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(12001), "aaa", 10L: JLong), true))) + CRow(Row.of(12001L: JLong, "aaa", 10L: JLong), change = true))) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(12001), "bbb", 40L: JLong), true))) + CRow(Row.of(12001L: JLong, "bbb", 40L: JLong), change = true))) testHarness.processWatermark(19000) @@ -845,10 +845,10 @@ class OverWindowHarnessTest extends HarnessTestBase{ testHarness.processWatermark(20000) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(20001), "ccc", 1L: JLong), true))) // clean-up 5000 + CRow(Row.of(20001L: JLong, "ccc", 1L: JLong), change = true))) // clean-up 5000 testHarness.setProcessingTime(2500) testHarness.processElement(new StreamRecord( - CRow(Row.of(toTS(20002), "ccc", 2L: JLong), true))) // clean-up 5000 + CRow(Row.of(20002L: JLong, "ccc", 2L: JLong), change = true))) // clean-up 5000 assert(testHarness.numKeyedStateEntries() > 0) testHarness.setProcessingTime(5000) // does not clean up, because data left. New timer 7000 @@ -865,38 +865,38 @@ class OverWindowHarnessTest extends HarnessTestBase{ val expectedOutput = new ConcurrentLinkedQueue[Object]() expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(801), "aaa", 1L: JLong, 1L: JLong, 1L: JLong), true))) + CRow(Row.of(801L: JLong, "aaa", 1L: JLong, 1L: JLong, 1L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(2501), "bbb", 10L: JLong, 10L: JLong, 10L: JLong), true))) + CRow(Row.of(2501L: JLong, "bbb", 10L: JLong, 10L: JLong, 10L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(4001), "aaa", 2L: JLong, 1L: JLong, 2L: JLong), true))) + CRow(Row.of(4001L: JLong, "aaa", 2L: JLong, 1L: JLong, 2L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(4001), "aaa", 3L: JLong, 1L: JLong, 3L: JLong), true))) + CRow(Row.of(4001L: JLong, "aaa", 3L: JLong, 1L: JLong, 3L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(4001), "bbb", 20L: JLong, 10L: JLong, 20L: JLong), true))) + CRow(Row.of(4001L: JLong, "bbb", 20L: JLong, 10L: JLong, 20L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(4801), "aaa", 4L: JLong, 1L: JLong, 4L: JLong), true))) + CRow(Row.of(4801L: JLong, "aaa", 4L: JLong, 1L: JLong, 4L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(6501), "aaa", 5L: JLong, 1L: JLong, 5L: JLong), true))) + CRow(Row.of(6501L: JLong, "aaa", 5L: JLong, 1L: JLong, 5L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(6501), "aaa", 6L: JLong, 1L: JLong, 6L: JLong), true))) + CRow(Row.of(6501L: JLong, "aaa", 6L: JLong, 1L: JLong, 6L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(6501), "bbb", 30L: JLong, 10L: JLong, 30L: JLong), true))) + CRow(Row.of(6501L: JLong, "bbb", 30L: JLong, 10L: JLong, 30L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(7001), "aaa", 7L: JLong, 1L: JLong, 7L: JLong), true))) + CRow(Row.of(7001L: JLong, "aaa", 7L: JLong, 1L: JLong, 7L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(8001), "aaa", 8L: JLong, 1L: JLong, 8L: JLong), true))) + CRow(Row.of(8001L: JLong, "aaa", 8L: JLong, 1L: JLong, 8L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(12001), "aaa", 9L: JLong, 1L: JLong, 9L: JLong), true))) + CRow(Row.of(12001L: JLong, "aaa", 9L: JLong, 1L: JLong, 9L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(12001), "aaa", 10L: JLong, 1L: JLong, 10L: JLong), true))) + CRow(Row.of(12001L: JLong, "aaa", 10L: JLong, 1L: JLong, 10L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(12001), "bbb", 40L: JLong, 10L: JLong, 40L: JLong), true))) + CRow(Row.of(12001L: JLong, "bbb", 40L: JLong, 10L: JLong, 40L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(20001), "ccc", 1L: JLong, 1L: JLong, 1L: JLong), true))) + CRow(Row.of(20001L: JLong, "ccc", 1L: JLong, 1L: JLong, 1L: JLong), change = true))) expectedOutput.add(new StreamRecord( - CRow(Row.of(toTS(20002), "ccc", 2L: JLong, 1L: JLong, 2L: JLong), true))) + CRow(Row.of(20002L: JLong, "ccc", 2L: JLong, 1L: JLong, 2L: JLong), change = true))) verify(expectedOutput, result, new RowResultSortComparator()) testHarness.close() diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/SortProcessFunctionHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/SortProcessFunctionHarnessTest.scala index 18ba6bbe64e36..9490039137822 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/SortProcessFunctionHarnessTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/SortProcessFunctionHarnessTest.scala @@ -21,7 +21,6 @@ package org.apache.flink.table.runtime.harness import java.lang.{Integer => JInt, Long => JLong} import java.util.concurrent.ConcurrentLinkedQueue -import org.apache.calcite.runtime.SqlFunctions.{internalToTimestamp => toTS} import org.apache.flink.api.common.typeinfo.BasicTypeInfo._ import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer} @@ -189,35 +188,35 @@ class SortProcessFunctionHarnessTest { // timestamp is ignored in processing time testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", toTS(1001)), true))) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1001L: JLong), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 12L: JLong, 1: JInt, "aaa", toTS(2002)), true))) + Row.of(1: JInt, 12L: JLong, 1: JInt, "aaa", 2002L: JLong), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 13L: JLong, 2: JInt, "aaa", toTS(2002)), true))) + Row.of(1: JInt, 13L: JLong, 2: JInt, "aaa", 2002L: JLong), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 12L: JLong, 3: JInt, "aaa", toTS(2002)), true))) + Row.of(1: JInt, 12L: JLong, 3: JInt, "aaa", 2002L: JLong), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 14L: JLong, 0: JInt, "aaa", toTS(2002)), true))) + Row.of(1: JInt, 14L: JLong, 0: JInt, "aaa", 2002L: JLong), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 12L: JLong, 3: JInt, "aaa", toTS(2004)), true))) + Row.of(1: JInt, 12L: JLong, 3: JInt, "aaa", 2004L: JLong), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", toTS(2006)), true))) + Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", 2006L: JLong), true))) // move watermark forward testHarness.processWatermark(2007) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 20L: JLong, 1: JInt, "aaa", toTS(2008)), true))) + Row.of(1: JInt, 20L: JLong, 1: JInt, "aaa", 2008L: JLong), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 14L: JLong, 0: JInt, "aaa", toTS(2002)), true))) // too late + Row.of(1: JInt, 14L: JLong, 0: JInt, "aaa", 2002L: JLong), true))) // too late testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 12L: JLong, 3: JInt, "aaa", toTS(2019)), true))) // too early + Row.of(1: JInt, 12L: JLong, 3: JInt, "aaa", 2019L: JLong), true))) // too early testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 20L: JLong, 2: JInt, "aaa", toTS(2008)), true))) + Row.of(1: JInt, 20L: JLong, 2: JInt, "aaa", 2008L: JLong), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", toTS(2010)), true))) + Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", 2010L: JLong), true))) testHarness.processElement(new StreamRecord(new CRow( - Row.of(1: JInt, 19L: JLong, 0: JInt, "aaa", toTS(2008)), true))) + Row.of(1: JInt, 19L: JLong, 0: JInt, "aaa", 2008L: JLong), true))) // move watermark forward testHarness.processWatermark(2012) @@ -231,29 +230,29 @@ class SortProcessFunctionHarnessTest { // (10,0) (11,1) (12,2) (12,1) (12,0) expectedOutput.add(new Watermark(3)) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", toTS(1001)), true))) + Row.of(1: JInt, 11L: JLong, 1: JInt, "aaa", 1001L: JLong), true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 12L: JLong, 3: JInt, "aaa", toTS(2002)), true))) + Row.of(1: JInt, 12L: JLong, 3: JInt, "aaa", 2002L: JLong), true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 12L: JLong, 1: JInt, "aaa", toTS(2002)), true))) + Row.of(1: JInt, 12L: JLong, 1: JInt, "aaa", 2002L: JLong), true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 13L: JLong, 2: JInt, "aaa", toTS(2002)), true))) + Row.of(1: JInt, 13L: JLong, 2: JInt, "aaa", 2002L: JLong), true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 14L: JLong, 0: JInt, "aaa", toTS(2002)), true))) + Row.of(1: JInt, 14L: JLong, 0: JInt, "aaa", 2002L: JLong), true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 12L: JLong, 3: JInt, "aaa", toTS(2004)), true))) + Row.of(1: JInt, 12L: JLong, 3: JInt, "aaa", 2004L: JLong), true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", toTS(2006)), true))) + Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", 2006L: JLong), true))) expectedOutput.add(new Watermark(2007)) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 19L: JLong, 0: JInt, "aaa", toTS(2008)), true))) + Row.of(1: JInt, 19L: JLong, 0: JInt, "aaa", 2008L: JLong), true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 20L: JLong, 2: JInt, "aaa", toTS(2008)), true))) + Row.of(1: JInt, 20L: JLong, 2: JInt, "aaa", 2008L: JLong), true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 20L: JLong, 1: JInt, "aaa", toTS(2008)), true))) + Row.of(1: JInt, 20L: JLong, 1: JInt, "aaa", 2008L: JLong), true))) expectedOutput.add(new StreamRecord(new CRow( - Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", toTS(2010)), true))) + Row.of(1: JInt, 10L: JLong, 0: JInt, "aaa", 2010L: JLong), true))) expectedOutput.add(new Watermark(2012)) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala index 4c478de1596e8..9b682412090e8 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala @@ -63,9 +63,10 @@ class TimeAttributesITCase extends StreamingMultipleProgramsTestBase { val stream = env .fromCollection(data) .assignTimestampsAndWatermarks(new TimestampWithEqualWatermark()) - val table = stream.toTable(tEnv, 'rowtime.rowtime, 'int, 'double, 'float, 'bigdec, 'string) + val table = stream.toTable( + tEnv, 'rowtime.rowtime, 'int, 'double, 'float, 'bigdec, 'string, 'proctime.proctime) - val t = table.select('rowtime.cast(Types.STRING)) + val t = table.select('rowtime.cast(Types.STRING), 'proctime) val results = t.toAppendStream[Row] results.addSink(new StreamITCase.StringSink[Row]) @@ -123,6 +124,13 @@ class TimeAttributesITCase extends StreamingMultipleProgramsTestBase { tEnv, 'rowtime.rowtime, 'int, 'double, 'float, 'bigdec, 'string, 'proctime.proctime) val func = new TableFunc + // we test if this can be executed with any exceptions + table.join(func('proctime, 'proctime, 'string) as 's).toAppendStream[Row] + + // we test if this can be executed with any exceptions + table.join(func('rowtime, 'rowtime, 'string) as 's).toAppendStream[Row] + + // we can only test rowtime, not proctime val t = table.join(func('rowtime, 'proctime, 'string) as 's).select('rowtime, 's) val results = t.toAppendStream[Row] diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala index 82ed81c3e017c..830359fbea17a 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala @@ -421,8 +421,8 @@ class TableSinkITCase extends StreamingMultipleProgramsTestBase { ctx: ProcessFunction[(Boolean, Row), Row]#Context, out: Collector[Row]): Unit = { - val rowTS: Long = row._2.getField(2).asInstanceOf[Long] - if (ctx.timestamp() == rowTS) { + val rowTs = row._2.getField(2).asInstanceOf[Long] + if (ctx.timestamp() == rowTs) { out.collect(row._2) } } From 285c9cdece50af6f8cebd5c600be2086b6e768db Mon Sep 17 00:00:00 2001 From: twalthr Date: Sat, 12 Aug 2017 15:18:41 +0200 Subject: [PATCH 5/7] Minor fixes --- .../scala/org/apache/flink/table/api/TableEnvironment.scala | 2 +- .../flink/table/runtime/stream/TimeAttributesITCase.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 5fe8e77e93f93..a90d8c7eae657 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 @@ -715,7 +715,7 @@ abstract class TableEnvironment(val config: TableConfig) { throw new TableException("Field name can not be '*'.") } - (fieldNames, fieldIndexes) + (fieldNames.toArray, fieldIndexes.toArray) // build fails if not converted to array } protected def generateRowConverterFunction[OUT]( diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala index 9b682412090e8..24d8695819c22 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala @@ -66,7 +66,7 @@ class TimeAttributesITCase extends StreamingMultipleProgramsTestBase { val table = stream.toTable( tEnv, 'rowtime.rowtime, 'int, 'double, 'float, 'bigdec, 'string, 'proctime.proctime) - val t = table.select('rowtime.cast(Types.STRING), 'proctime) + val t = table.select('rowtime.cast(Types.STRING)) val results = t.toAppendStream[Row] results.addSink(new StreamITCase.StringSink[Row]) From 6999c9bb18076297d10acd860ee8a0c1434659c1 Mon Sep 17 00:00:00 2001 From: twalthr Date: Mon, 14 Aug 2017 16:57:06 +0200 Subject: [PATCH 6/7] Feedback addressed --- .../scala/org/apache/flink/table/api/TableEnvironment.scala | 2 +- .../flink/table/calcite/RelTimeIndicatorConverter.scala | 2 +- .../scala/org/apache/flink/table/codegen/CodeGenerator.scala | 4 ++-- .../flink/table/functions/{ => sql}/ProctimeSqlFunction.scala | 2 +- .../table/runtime/conversion/CRowToJavaTupleMapRunner.scala | 4 ++-- 5 files changed, 7 insertions(+), 7 deletions(-) rename flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/{ => sql}/ProctimeSqlFunction.scala (97%) 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 a90d8c7eae657..2e9e18f91a328 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 @@ -715,7 +715,7 @@ abstract class TableEnvironment(val config: TableConfig) { throw new TableException("Field name can not be '*'.") } - (fieldNames.toArray, fieldIndexes.toArray) // build fails if not converted to array + (fieldNames.toArray, fieldIndexes.toArray) // build fails in Scala 2.10 if not converted } protected def generateRowConverterFunction[OUT]( 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 717a1af46f38c..1f88737247aeb 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 @@ -27,7 +27,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo import org.apache.flink.table.api.{TableException, ValidationException} import org.apache.flink.table.calcite.FlinkTypeFactory.{isRowtimeIndicatorType, _} -import org.apache.flink.table.functions.ProctimeSqlFunction +import org.apache.flink.table.functions.sql.ProctimeSqlFunction import org.apache.flink.table.plan.logical.rel.LogicalWindowAggregate import org.apache.flink.table.plan.schema.TimeIndicatorRelDataType 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 7d1b2013d4178..4fbbee14a41a4 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 @@ -38,9 +38,9 @@ import org.apache.flink.table.codegen.CodeGenUtils._ import org.apache.flink.table.codegen.GeneratedExpression.{NEVER_NULL, NO_CODE} import org.apache.flink.table.codegen.calls.FunctionGenerator import org.apache.flink.table.codegen.calls.ScalarOperators._ -import org.apache.flink.table.functions.sql.ScalarSqlFunctions +import org.apache.flink.table.functions.sql.{ProctimeSqlFunction, ScalarSqlFunctions} import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils -import org.apache.flink.table.functions.{FunctionContext, ProctimeSqlFunction, UserDefinedFunction} +import org.apache.flink.table.functions.{FunctionContext, UserDefinedFunction} import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo import org.apache.flink.table.typeutils.TypeCheckUtils._ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/ProctimeSqlFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/sql/ProctimeSqlFunction.scala similarity index 97% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/ProctimeSqlFunction.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/sql/ProctimeSqlFunction.scala index 2e7dba693fb04..f30ad2fc0b35c 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/ProctimeSqlFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/sql/ProctimeSqlFunction.scala @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.flink.table.functions +package org.apache.flink.table.functions.sql import org.apache.calcite.sql._ import org.apache.calcite.sql.`type`._ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToJavaTupleMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToJavaTupleMapRunner.scala index c6393b1e35e4f..95f304d734b42 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToJavaTupleMapRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToJavaTupleMapRunner.scala @@ -38,8 +38,8 @@ class CRowToJavaTupleMapRunner( code: String, @transient var returnType: TypeInformation[JTuple2[JBool, Any]]) extends RichMapFunction[CRow, Any] - with ResultTypeQueryable[JTuple2[JBool, Any]] - with Compiler[MapFunction[Row, Any]] { + with ResultTypeQueryable[JTuple2[JBool, Any]] + with Compiler[MapFunction[Row, Any]] { val LOG: Logger = LoggerFactory.getLogger(this.getClass) From c48d4f803fed4b7ff42ee6fca47d01be78f7aad6 Mon Sep 17 00:00:00 2001 From: twalthr Date: Mon, 14 Aug 2017 17:06:43 +0200 Subject: [PATCH 7/7] Addressed feedback from Fabian's PR --- .../apache/flink/table/codegen/CodeGenerator.scala | 3 +-- .../table/runtime/OutputRowtimeProcessFunction.scala | 12 ++++-------- 2 files changed, 5 insertions(+), 10 deletions(-) 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 4fbbee14a41a4..5afbe57180492 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 @@ -1301,11 +1301,10 @@ abstract class CodeGenerator( private[flink] def generateProctimeTimestamp(): GeneratedExpression = { val resultTerm = newName("result") - val resultTypeTerm = primitiveTypeTermForTypeInfo(SqlTimeTypeInfo.TIMESTAMP) val resultCode = s""" - |$resultTypeTerm $resultTerm = $contextTerm.timerService().currentProcessingTime(); + |long $resultTerm = $contextTerm.timerService().currentProcessingTime(); |""".stripMargin GeneratedExpression(resultTerm, NEVER_NULL, resultCode, SqlTimeTypeInfo.TIMESTAMP) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/OutputRowtimeProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/OutputRowtimeProcessFunction.scala index e7adbcee64de6..3eaeea308f915 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/OutputRowtimeProcessFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/OutputRowtimeProcessFunction.scala @@ -19,7 +19,8 @@ package org.apache.flink.table.runtime import org.apache.calcite.runtime.SqlFunctions -import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction} +import org.apache.flink.api.common.functions.MapFunction +import org.apache.flink.api.common.functions.util.FunctionUtils import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.streaming.api.operators.TimestampedCollector @@ -36,13 +37,8 @@ class OutputRowtimeProcessFunction[OUT]( extends ProcessFunction[CRow, OUT] { override def open(parameters: Configuration): Unit = { - super.open(parameters) - function match { - case f: RichMapFunction[_, _] => - f.setRuntimeContext(getRuntimeContext) - f.open(parameters) - case _ => - } + FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext) + FunctionUtils.openFunction(function, parameters) } override def processElement(