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

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

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

    https://github.com/apache/flink/pull/5240#discussion_r162967648
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogTable.scala
 ---
    @@ -18,28 +18,282 @@
     
     package org.apache.flink.table.catalog
     
    -import java.util.{HashMap => JHashMap, Map => JMap}
     import java.lang.{Long => JLong}
    +import java.util.{HashMap => JHashMap, Map => JMap}
     
    -import org.apache.flink.table.api.TableSchema
    +import org.apache.flink.table.api.{TableException, TableSchema}
    +import 
org.apache.flink.table.catalog.ExternalCatalogTable.{TableTypeConnector, 
toConnectorDescriptor, toMetadataDescriptor, toStatisticsDescriptor}
    +import org.apache.flink.table.descriptors.DescriptorUtils.{connector, 
metadata}
    +import org.apache.flink.table.descriptors._
     import org.apache.flink.table.plan.stats.TableStats
     
    +import scala.collection.JavaConverters._
    +
     /**
       * Defines a table in an [[ExternalCatalog]].
    -  *
    -  * @param tableType            Table type, e.g csv, hbase, kafka
    -  * @param schema               Schema of the table (column names and 
types)
    -  * @param properties           Properties of the table
    -  * @param stats                Statistics of the table
    -  * @param comment              Comment of the table
    -  * @param createTime           Create timestamp of the table
    -  * @param lastAccessTime       Timestamp of last access of the table
       */
    -case class ExternalCatalogTable(
    +class ExternalCatalogTable(
    --- End diff --
    
    Add descriptions for constructor arguments


> 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
>            Priority: Major
>
> 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
(v7.6.3#76005)

Reply via email to