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

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

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

    https://github.com/apache/flink/pull/6323#discussion_r202271871
  
    --- Diff: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java
 ---
    @@ -124,110 +123,127 @@
                params.putProperties(properties);
     
                // validate
    -           new SchemaValidator(true).validate(params);
    +           // allow Kafka timestamps to be used, watermarks can not be 
received from source
    +           new SchemaValidator(true, supportsKafkaTimestamps(), 
false).validate(params);
                new KafkaValidator().validate(params);
    -           formatValidator().validate(params);
     
    -           // build
    -           final KafkaTableSource.Builder builder = 
createBuilderWithFormat(params);
    +           // deserialization schema using format discovery
    +           final DeserializationSchemaFactory<?> formatFactory = 
TableFormatFactoryService.find(
    +                   DeserializationSchemaFactory.class,
    +                   properties,
    +                   this.getClass().getClassLoader());
    +           @SuppressWarnings("unchecked")
    +           final DeserializationSchema<Row> deserializationSchema = 
(DeserializationSchema<Row>) formatFactory
    +                   .createDeserializationSchema(properties);
     
    -           // topic
    -           final String topic = params.getString(CONNECTOR_TOPIC);
    -           builder.forTopic(topic);
    +           // schema
    +           final TableSchema schema = params.getTableSchema(SCHEMA());
    +
    +           // proctime
    +           final String proctimeAttribute = 
SchemaValidator.deriveProctimeAttribute(params).orElse(null);
    --- End diff --
    
    this is kind of ridiculous :/ Deep inside we work on `Optional` and then we 
switch to null...
    
    Please drag this `Optional` until the very end and do this `orElse(null)` 
conversion in 
`org.apache.flink.table.sources.DefinedProctimeAttribute#getProctimeAttribute` 
implementation.


> 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