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

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

dianfu 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_r235346023
 
 

 ##########
 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 = null
 
 Review comment:
   null -> _

----------------------------------------------------------------
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]


> Implement proctime DataStream to Table upsert conversion.
> ---------------------------------------------------------
>
>                 Key: FLINK-8577
>                 URL: https://issues.apache.org/jira/browse/FLINK-8577
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API & SQL
>            Reporter: Hequn Cheng
>            Assignee: Hequn Cheng
>            Priority: Major
>              Labels: pull-request-available
>
> Api will looks like:
> {code:java}
> DataStream[(String, Long, Int)] input = ???
> // upsert with keyTable 
> table = tEnv.upsertFromStream(input, 'a, 'b, 'c.key)
> // upsert without key -> single row tableTable 
> table = tEnv.upsertFromStream(input, 'a, 'b, 'c){code}
> A simple design 
> [doc|https://docs.google.com/document/d/1yMEi9xkm3pNYNmmZykbLzqtKjFVezaWe0y7Xqd0c1zE/edit?usp=sharing]
>  about this subtask.



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

Reply via email to