[ https://issues.apache.org/jira/browse/FLINK-8240?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16300222#comment-16300222 ]
Fabian Hueske commented on FLINK-8240: -------------------------------------- Thanks for starting this discussion [~twalthr]. I agree, we need a common properties API to specify table sources. This API can be used for several purposes: - defining table sources in the Table API (as in [~twalthr]'s example) - defining table sources in a DDL statement (as mentioned by [~wheat9]). The DDL statement would be parsed and converted into a properties map. - defining table sources in a catalog file for a SQL client - saving and loading table sources / catalog information in the Table API as proposed in FLINK-4088 Together with the properties API, we should also add util classes to parse the a Properties object, i.e., a util class for each system (FileSystem, Kafka, etc.) and encoding (Csv, Avro, Orc, etc.). These utility classes then ease the implementation of the factory classes and ensure a uniform behavior. I don't think that each table source _must_ provide a factory because this would mean that the properties API would need to be extended for each supported system (e.g, HBase, JDBC, etc.). This would be an obstacle for user-defined table sources. However, I agree that we should aim that all Flink built-in / provided table sources can be instantiated via the properties API. > Create unified interfaces to configure and instatiate TableSources > ------------------------------------------------------------------ > > Key: FLINK-8240 > URL: https://issues.apache.org/jira/browse/FLINK-8240 > Project: Flink > Issue Type: New Feature > Components: Table API & SQL > Reporter: Timo Walther > Assignee: Timo Walther > > At the moment every table source has different ways for configuration and > instantiation. Some table source are tailored to a specific encoding (e.g., > {{KafkaAvroTableSource}}, {{KafkaJsonTableSource}}) or only support one > encoding for reading (e.g., {{CsvTableSource}}). Each of them might implement > a builder or support table source converters for external catalogs. > The table sources should have a unified interface for discovery, defining > common properties, and instantiation. The {{TableSourceConverters}} provide a > similar functionality but use an external catalog. We might generialize this > interface. > In general a table source declaration depends on the following parts: > {code} > - Source > - Type (e.g. Kafka, Custom) > - Properties (e.g. topic, connection info) > - Encoding > - Type (e.g. Avro, JSON, CSV) > - Schema (e.g. Avro class, JSON field names/types) > - Rowtime descriptor/Proctime > - Watermark strategy and Watermark properties > - Time attribute info > - Bucketization > {code} > This issue needs a design document before implementation. Any discussion is > very welcome. -- This message was sent by Atlassian JIRA (v6.4.14#64029)