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

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

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_r236196242
 
 

 ##########
 File path: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala
 ##########
 @@ -165,7 +165,30 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) 
extends RelShuttle {
   override def visit(exchange: LogicalExchange): RelNode =
     throw new TableException("Logical exchange in a stream environment is not 
supported yet.")
 
-  override def visit(scan: TableScan): RelNode = scan
+  override def visit(scan: TableScan): RelNode = {
+    val upsertStreamTable = scan.getTable.unwrap(classOf[UpsertStreamTable[_]])
+    if (upsertStreamTable != null) {
+      val relTypes = scan.getRowType.getFieldList.map(_.getType)
+      val timeIndicatorIndexes = relTypes.zipWithIndex
+        .filter(e => FlinkTypeFactory.isTimeIndicatorType(e._1))
+        .map(_._2)
+      val input = if (timeIndicatorIndexes.nonEmpty) {
+        // materialize time indicator
+        val rewrittenScan = scan.copy(scan.getTraitSet, scan.getInputs)
+        materializerUtils.projectAndMaterializeFields(rewrittenScan, 
timeIndicatorIndexes.toSet)
+      } else {
+        scan
+      }
+
+      LogicalLastRow.create(
 
 Review comment:
   Do we ALWAYS convert upserts to retractions? Even for pipelines `upsert 
source -> filter -> upsert sink` ?

----------------------------------------------------------------
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:
us...@infra.apache.org


> 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