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

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

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

    https://github.com/apache/flink/pull/4532#discussion_r132854814
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/OutputRowtimeProcessFunction.scala
 ---
    @@ -0,0 +1,62 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.table.runtime
    +
    +import org.apache.calcite.runtime.SqlFunctions
    +import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction}
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.streaming.api.functions.ProcessFunction
    +import org.apache.flink.streaming.api.operators.TimestampedCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.util.Collector
    +
    +/**
    +  * Wraps a ProcessFunction and sets a Timestamp field of a CRow as
    +  * [[org.apache.flink.streaming.runtime.streamrecord.StreamRecord]] 
timestamp.
    +  */
    +class OutputRowtimeProcessFunction[OUT](
    +    function: MapFunction[CRow, OUT],
    +    rowtimeIdx: Int)
    --- End diff --
    
    in fact the data type is changed to keep the data type unchanged. A time 
indicator is externally represented and treated as a `TIMESTAMP` field and only 
internally handled as `LONG`. Therefore, we need to convert it into a 
`TIMESTAMP` once the result is converted into a `DataStream`.
    
    You are right, that we need to convert all time indicators to `TIMESTAMP` 
and not only one. This is currently enforced by the exception that you 
observed. Currently users have to cast all but one time indicator attributes to 
`TIMESTAMP`. That will also convert them from `long` to `Timestamp`.


> 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