[ https://issues.apache.org/jira/browse/FLINK-8577?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16698758#comment-16698758 ]
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_r236187717 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/DataStreamConversions.scala ########## @@ -59,5 +59,38 @@ class DataStreamConversions[T](dataStream: DataStream[T], inputType: TypeInforma } } + /** + * Converts the [[DataStream]] with upsert messages into a [[Table]] with keys. + * + * The message will be encoded as [[Tuple2]]. The first field is a [[Boolean]] flag, the second + * field holds the record. A true [[Boolean]] flag indicates an update message, a false flag + * indicates a delete message. + * + * The field name and key of the new [[Table]] can be specified like this: + * + * {{{ + * val env = StreamExecutionEnvironment.getExecutionEnvironment + * val tEnv = TableEnvironment.getTableEnvironment(env) + * + * val stream: DataStream[(Boolean, (String, Int))] = ... + * val table = stream.toKeyedTable(tEnv, 'name.key, 'amount) + * }}} + * + * If field names are not explicitly specified, names are automatically extracted from the type + * of the [[DataStream]]. + * If keys are not explicitly specified, an empty key will be used and the table will be a + * single row table. + * + * @param tableEnv The [[StreamTableEnvironment]] in which the new [[Table]] is created. + * @param fields The field names of the new [[Table]] (optional). + * @return The resulting [[Table]]. + */ + def toKeyedTable(tableEnv: StreamTableEnvironment, fields: Expression*): Table = { Review comment: This is some naming inconsistency: `toKeyedTable` vs `fromUpsertStream`. Rename `keyed` to `upsert`? ---------------------------------------------------------------- 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)