[
https://issues.apache.org/jira/browse/FLINK-7337?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16124586#comment-16124586
]
ASF GitHub Bot commented on FLINK-7337:
---------------------------------------
Github user twalthr commented on a diff in the pull request:
https://github.com/apache/flink/pull/4532#discussion_r132819174
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
---
@@ -667,30 +719,62 @@ 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(
--- End diff --
This is a very good question. I don't know how we want to solve the `SELECT
*` problem. My initial idea was to just the attribute of the left table as a
time indicator. Another idea would be to remove all time indicators and
explicitly choose one attribute `SELECT TIME(OrderA.rowtime)`. Still not
perfect. We need to discuss this. @fhueske what do you think?
> Refactor handling of time indicator attributes
> ----------------------------------------------
>
> Key: FLINK-7337
> URL: https://issues.apache.org/jira/browse/FLINK-7337
> Project: Flink
> Issue Type: Improvement
> Components: Table API & SQL
> Affects Versions: 1.4.0
> Reporter: Fabian Hueske
> Assignee: Fabian Hueske
>
> After a [discussion on the dev mailing
> list|https://lists.apache.org/thread.html/735d55f9022df8ff73566a9f1553e14be94f8443986ad46559b35869@%3Cdev.flink.apache.org%3E]
> I propose the following changes to the current handling of time indicator
> attributes:
> * Remove the separation of logical and physical row type.
> ** Hold the event-time timestamp as regular Long field in Row
> ** Represent the processing-time indicator type as a null-valued field in Row
> (1 bit overhead)
> * Remove materialization of event-time timestamps because timestamp is
> already accessible in Row.
> * Add {{ProcessFunction}} to set timestamp into the timestamp field of a
> {{StreamRecord}}.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)