pnowojski commented on a change in pull request #6787: [FLINK-8577][table]
Implement proctime DataStream to Table upsert conversion
URL: https://github.com/apache/flink/pull/6787#discussion_r253465887
##########
File path:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala
##########
@@ -19,22 +19,138 @@
package org.apache.flink.table.plan.nodes.datastream
import org.apache.calcite.rex.RexNode
-import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction}
+import org.apache.flink.api.common.functions.RichMapFunction
import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
import org.apache.flink.configuration.Configuration
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.api.{TableConfig, Types}
import org.apache.flink.table.codegen.{FunctionCodeGenerator,
GeneratedFunction}
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.runtime.CRowOutputProcessRunner
import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo
+import java.lang.{Boolean => JBool}
+
+import org.apache.flink.api.java.typeutils.TupleTypeInfo
+import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo
+import
org.apache.flink.table.runtime.conversion.{ExternalTypeToCRowProcessRunner,
JavaTupleToCRowProcessRunner, ScalaTupleToCRowProcessRunner}
trait StreamScan extends CommonScan[CRow] with DataStreamRel {
+ protected def convertUpsertToInternalRow(
+ schema: RowSchema,
+ input: DataStream[Any],
+ fieldIdxs: Array[Int],
+ config: TableConfig,
+ rowtimeExpression: Option[RexNode]): DataStream[CRow] = {
+
+ val internalType = schema.typeInfo
+ val cRowType = CRowTypeInfo(internalType)
+
+ val hasTimeIndicator = fieldIdxs.exists(f =>
+ f == TimeIndicatorTypeInfo.ROWTIME_STREAM_MARKER ||
+ f == TimeIndicatorTypeInfo.PROCTIME_STREAM_MARKER)
+
+ val dsType = input.getType
+
+ dsType match {
+ // Scala tuple
+ case t: CaseClassTypeInfo[_]
+ if t.getTypeClass == classOf[(_, _)] && t.getTypeAt(0) ==
Types.BOOLEAN =>
+
+ val inputType = t.getTypeAt[Any](1)
+ if (inputType == internalType && !hasTimeIndicator) {
+ // input is already of correct type. Only need to wrap it as CRow
+ input.asInstanceOf[DataStream[(Boolean, Row)]]
+ .map(new RichMapFunction[(Boolean, Row), CRow] {
+ @transient private var outCRow: CRow = _
+ override def open(parameters: Configuration): Unit = {
+ outCRow = new CRow(null, change = true)
+ }
+
+ override def map(v: (Boolean, Row)): CRow = {
+ outCRow.row = v._2
+ outCRow.change = v._1
+ outCRow
+ }
+ }).returns(cRowType)
+
+ } else {
+ // input needs to be converted and wrapped as CRow or time
indicators need to be generated
+
+ val function = generateConversionProcessFunction(
+ config,
+ inputType.asInstanceOf[TypeInformation[Any]],
+ internalType,
+ "UpsertStreamSourceConversion",
+ schema.fieldNames,
+ fieldIdxs,
+ rowtimeExpression
+ )
+
+ val processFunc = new ScalaTupleToCRowProcessRunner(
+ function.name,
+ function.code,
+ cRowType)
+
+ val opName = s"from: (${schema.fieldNames.mkString(", ")})"
+
+ input
+ .asInstanceOf[DataStream[(Boolean, Any)]]
+ .process(processFunc).name(opName).returns(cRowType)
+ }
+
+ // Java tuple
+ case t: TupleTypeInfo[_]
+ if t.getTypeClass == classOf[JTuple2[_, _]] && t.getTypeAt(0) ==
Types.BOOLEAN =>
+
+ val inputType = t.getTypeAt[Any](1)
+ if (inputType == internalType && !hasTimeIndicator) {
+ // input is already of correct type. Only need to wrap it as CRow
+ input.asInstanceOf[DataStream[JTuple2[JBool, Row]]]
+ .map(new RichMapFunction[JTuple2[JBool, Row], CRow] {
+ @transient private var outCRow: CRow = _
+ override def open(parameters: Configuration): Unit = {
+ outCRow = new CRow(null, change = true)
+ }
+
+ override def map(v: JTuple2[JBool, Row]): CRow = {
+ outCRow.row = v.f1
+ outCRow.change = v.f0
+ outCRow
+ }
+ }).returns(cRowType)
+
+ } else {
+ // input needs to be converted and wrapped as CRow or time
indicators need to be generated
+
+ val function = generateConversionProcessFunction(
+ config,
+ inputType.asInstanceOf[TypeInformation[Any]],
+ internalType,
+ "UpsertStreamSourceConversion",
+ schema.fieldNames,
+ fieldIdxs,
+ rowtimeExpression
+ )
+
+ val processFunc = new JavaTupleToCRowProcessRunner(
+ function.name,
+ function.code,
+ cRowType)
+
+ val opName = s"from: (${schema.fieldNames.mkString(", ")})"
+
+ input
+ .asInstanceOf[DataStream[JTuple2[JBool, Any]]]
+ .process(processFunc).name(opName).returns(cRowType)
+ }
+ }
+ }
+
protected def convertToInternalRow(
Review comment:
rename `convertAppendToInternalRow`, `convertAppendStreamToInternalRow`,
`convertAppendStream` or `convertAppendStreamToCRow`? (if you pick something
else beside the first option, make sure it's in sync with
`convertUpsertToInternalRow`
btw, this rename is a good example of a rename that is actually tightly
connected to the functional change - renaming this method without adding a new
one in the same commit probably wouldn't have much sense.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services