[ https://issues.apache.org/jira/browse/FLINK-8558?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16536999#comment-16536999 ]
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_r200987026 --- Diff: flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java --- @@ -18,48 +18,99 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.connectors.kafka.config.StartupMode; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.sources.RowtimeAttributeDescriptor; import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.types.Row; +import java.util.List; +import java.util.Map; import java.util.Properties; /** * Kafka {@link StreamTableSource} for Kafka 0.10. */ -@PublicEvolving -public abstract class Kafka010TableSource extends KafkaTableSource { +@Internal +public class Kafka010TableSource extends KafkaTableSource { - // The deserialization schema for the Kafka records - private final DeserializationSchema<Row> deserializationSchema; + /** + * Creates a Kafka 0.10 {@link StreamTableSource}. + * + * @param schema Schema of the produced table. + * @param proctimeAttribute Field name of the processing time attribute, null if no --- End diff -- This is kind of controversial topic. Generally speaking I suspect that Java discourage to use `Optional` beside return values because we should use `@Nullable` or not use any of them. However in projects that ignored `@Nullable` annotation (such as Flink), it's virtually impossible to start using them and thus using `Optional` is the only way to have a compiler control over optional/nullable fields. In this particular use case of "optional" arguments my preference hierarchy is: 1. provide a builder for this class 2. provide alternative constructor without this argument 3. use `@Nullable` with enabled compile errors on incorrectly handled `@Nullable` annotations 4. use `Optional` ... 1336. use `@Nullable` WITHOUT compile errors on incorrectly handled `@Nullable` annotations 1337. use nullable argument without `@Nullable` annotation two last options are for me out of the question, since `1337` is evil and `1336` doesn't improve situation. Option `3` is sadly impossible for Flink. The same logic applies for me to other use cases (like fields, return values etc): 1. avoid nulls/optionals (for example via builders or named parameters with default values) 2. use `@Nullable` with compiler errors 3. use `Optional` > 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)