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

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

Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4488#discussion_r131706276
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala
 ---
    @@ -172,45 +172,20 @@ class FlinkTypeFactory(typeSystem: RelDataTypeSystem) 
extends JavaTypeFactoryImp
         *
         * @param fieldNames field names
         * @param fieldTypes field types, every element is Flink's 
[[TypeInformation]]
    -    * @param rowtime optional system field to indicate event-time; the 
index determines the index
    -    *                in the final record. If the index is smaller than the 
number of specified
    -    *                fields, it shifts all following fields.
    -    * @param proctime optional system field to indicate processing-time; 
the index determines the
    -    *                 index in the final record. If the index is smaller 
than the number of
    -    *                 specified fields, it shifts all following fields.
         * @return a struct type with the input fieldNames, input fieldTypes, 
and system fields
         */
       def buildLogicalRowType(
    --- End diff --
    
    Is the name of this method still correct?


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

Reply via email to