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

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

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

    https://github.com/apache/flink/pull/4894#discussion_r147582469
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSource.scala
 ---
    @@ -38,23 +38,26 @@ import scala.collection.mutable
       * @param path The path to the CSV file.
       * @param fieldNames The names of the table fields.
       * @param fieldTypes The types of the table fields.
    +  * @param selectedFields The fields which will be read and returned by 
the table source.
    +  *                       If None, all fields are returned.
       * @param fieldDelim The field delimiter, "," by default.
       * @param rowDelim The row delimiter, "\n" by default.
       * @param quoteCharacter An optional quote character for String values, 
null by default.
       * @param ignoreFirstLine Flag to ignore the first line, false by default.
       * @param ignoreComments An optional prefix to indicate comments, null by 
default.
       * @param lenient Flag to skip records with parse error instead to fail, 
false by default.
       */
    -class CsvTableSource(
    +class CsvTableSource private (
         private val path: String,
         private val fieldNames: Array[String],
         private val fieldTypes: Array[TypeInformation[_]],
    -    private val fieldDelim: String = 
CsvInputFormat.DEFAULT_FIELD_DELIMITER,
    -    private val rowDelim: String = CsvInputFormat.DEFAULT_LINE_DELIMITER,
    -    private val quoteCharacter: Character = null,
    -    private val ignoreFirstLine: Boolean = false,
    -    private val ignoreComments: String = null,
    -    private val lenient: Boolean = false)
    +    private val selectedFields: Array[Int],
    +    private val fieldDelim: String,
    +    private val rowDelim: String,
    +    private val quoteCharacter: Character,
    +    private val ignoreFirstLine: Boolean,
    +    private val ignoreComments: String,
    +    private val lenient: Boolean)
       extends BatchTableSource[Row]
    --- End diff --
    
    Maybe we need a base class instead of traits to do something like checking 
the equality of numbers of field names/types.


> 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