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

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

GitHub user fhueske opened a pull request:

    https://github.com/apache/flink/pull/4894

    [FLINK-7548] [table] Improve rowtime support of TableSources.

    ## What is the purpose of the change
    
    This PR refactors the `TableSource` interface and some related interfaces. 
The goal of the refactoring is to:
    
    1. Move timestamp extraction and watermark assignment into the table scan. 
This way it is under control of the Table API and does not rely extraction and 
assignment in the `TableSource`
    2. Unify time attribute handling in `BatchTableSource` and 
`StreamTableSource`.
    3. Projection push down for table sources with time attributes.
    4. Projection push down into table access.
    5. Support of time attributes for table source with atomic return type.
    
    This commit also resolves:
    - FLINK-6870: Unified handling of time attributes in batch and streaming
    - FLINK-7179: Support for projection push down and watermark assigners
    - FLINK-7696: Support for projection push down and time attributes
    
    ## Brief change log
    
    ### API changes
    
    * Added `getTableSchema()` method to `TableSource` to separate logical 
table schema from physical return type (`getReturnType()`).
    * Replaced `DefinedFieldNames` interface by `DefinedFieldMapping` 
interface. Field names can be defined by table schema. The 
`DefinedFieldMapping` interfaces allows to provide an explicit mapping from 
table schema fields to return type fields.
    * `DefinedRowtimeAttribute` interface was replaced by 
`DefinedRowtimeAttributes` interface. The new interface returns a list of 
`RowtimeAttributeDescriptor` which exposes a `TimestampExtractor` and 
`WatermarkStrategy`. Both are used internally to extract timestamps and assign 
watermarks during the table scan. There are default implementation for common 
timestamp extraction and watermark generation methods.
    
    ## Verifying this change
    
    This PR adds an extensive set of validation, plan, and end-to-end 
integration tests.
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): **no**
      - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: **no**
      - The serializers: **no**
      - The runtime per-record code paths (performance sensitive): **YES**
      - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Yarn/Mesos, ZooKeeper: **no**
    
    ## Documentation
    
      - Does this pull request introduce a new feature? **YES**
      - If yes, how is the feature documented? **not yet**, PR will be updated 
soon.
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/fhueske/flink tableWatermarks

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/4894.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #4894
    
----
commit 5a3b0e02b58b0b5eeece60514b9affaf738c10b5
Author: Fabian Hueske <fhue...@apache.org>
Date:   2017-10-05T15:44:35Z

    [FLINK-7548] [table] Improve rowtime support of TableSources.
    
    This commit also resolves:
    - FLINK-6870: Unified handling of time attributes in batch and streaming
    - FLINK-7179: Support for projection push down and watermark assigners
    - FLINK-7696: Support for projection push down and time attributes
    
    TODO:
    - update documentation

----


> Support watermark generation for TableSource
> --------------------------------------------
>
>                 Key: FLINK-7548
>                 URL: https://issues.apache.org/jira/browse/FLINK-7548
>             Project: Flink
>          Issue Type: Improvement
>          Components: Table API & SQL
>            Reporter: Jark Wu
>            Assignee: Fabian Hueske
>            Priority: Blocker
>             Fix For: 1.4.0
>
>
> As discussed in FLINK-7446, currently the TableSource only support to define 
> rowtime field, but not support to extract watermarks from the rowtime field. 
> We can provide a new interface called {{DefinedWatermark}}, which has two 
> methods {{getRowtimeAttribute}} (can only be an existing field) and 
> {{getWatermarkGenerator}}. The {{DefinedRowtimeAttribute}} will be marked 
> deprecated.
> How to support periodic and punctuated watermarks and support some built-in 
> strategies needs further discussion.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to