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

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

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

    https://github.com/apache/flink/pull/6264#discussion_r200988428
  
    --- Diff: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java
 ---
    @@ -55,50 +57,101 @@
      */
     @Internal
     public abstract class KafkaTableSource
    -   implements StreamTableSource<Row>, DefinedProctimeAttribute, 
DefinedRowtimeAttributes {
    +   implements StreamTableSource<Row>, DefinedProctimeAttribute, 
DefinedRowtimeAttributes, DefinedFieldMapping {
    +
    +   // common table source attributes
    +   // TODO make all attributes final once we drop support for 
format-specific table sources
     
        /** The schema of the table. */
        private final TableSchema schema;
     
    +   /** Field name of the processing time attribute, null if no processing 
time field is defined. */
    +   private String proctimeAttribute;
    +
    +   /** Descriptor for a rowtime attribute. */
    +   private List<RowtimeAttributeDescriptor> rowtimeAttributeDescriptors;
    +
    +   /** Mapping for the fields of the table schema to fields of the 
physical returned type or null. */
    +   private Map<String, String> fieldMapping;
    --- End diff --
    
    please check my other comment about that. `@Nullable` without compiler 
errors is not in any way better :(


> Add unified format interfaces and format discovery
> --------------------------------------------------
>
>                 Key: FLINK-8558
>                 URL: https://issues.apache.org/jira/browse/FLINK-8558
>             Project: Flink
>          Issue Type: New Feature
>          Components: Streaming Connectors
>            Reporter: Timo Walther
>            Assignee: Timo Walther
>            Priority: Major
>              Labels: pull-request-available
>
> In the last release, we introduced a new module {{flink-formats}}. Currently 
> only {{flink-avro}} is located there but we will add more formats such as 
> {{flink-json}}, {{flink-protobuf}}, and so on. For better separation of 
> concerns we want decouple connectors from formats: e.g., remove 
> {{KafkaAvroTableSource}} and {{KafkaJsonTableSource}}.
> A newly introduced {{FormatFactory}} will use Java service loaders to 
> discovery available formats in the classpath (similar to how file systems are 
> discovered now). A {{Format}} will provide a method for converting {{byte[]}} 
> to target record type.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to