[ https://issues.apache.org/jira/browse/FLINK-8558?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16537022#comment-16537022 ]
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_r201021369 --- Diff: flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestAmbiguousTableFormatFactory.scala --- @@ -16,22 +16,32 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.kafka; +package org.apache.flink.table.formats.utils -import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_010; +import java.util + +import org.apache.flink.table.formats.TableFormatFactory +import org.apache.flink.types.Row /** - * Tests for {@link Kafka010AvroTableSourceFactory}. - */ -public class Kafka010AvroTableSourceFactoryTest extends KafkaAvroTableSourceFactoryTestBase { + * Table format factory for testing. + */ +class TestAmbiguousTableFormatFactory extends TableFormatFactory[Row] { --- End diff -- shouldn't this extend or reuse `TestTableFormatFactory`? They are now interconnected implicitly - you assume some dependencies between those two classes - and this can veerryy easily brake (silently) if someone modifies only one of those classes forgetting about the other. (silently - tests might still pass while they will not be testing what you intended anymore). > 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)