[ https://issues.apache.org/jira/browse/FLINK-8240?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16334946#comment-16334946 ]
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_r163012442 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptor.scala --- @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.descriptors + +import org.apache.flink.table.api.{StreamTableEnvironment, Table, TableException} +import org.apache.flink.table.sources.{StreamTableSource, TableSource, TableSourceFactoryService} + +/** + * Descriptor for specifying a table source in a streaming environment. + */ +class StreamTableSourceDescriptor( + tableEnv: StreamTableEnvironment, + schema: Schema) + extends TableSourceDescriptor { + + schemaDescriptor = Some(schema) + + /** + * Searches for the specified table source, configures it accordingly, and returns it. + */ + def toTableSource: TableSource[_] = { + val source = TableSourceFactoryService.findTableSourceFactory(this) + source match { + case _: StreamTableSource[_] => source + case _ => throw new TableException( + s"Found table source '${source.getClass.getCanonicalName}' is not applicable " + + s"in a streaming environment.") + } + } + + /** + * Searches for the specified table source, configures it accordingly, and returns it as a table. + */ + def toTable: Table = { + tableEnv.fromTableSource(toTableSource) + } + + /** + * Searches for the specified table source, configures it accordingly, and registers it as + * a table under the given name. + * + * @param name table name to be registered in the table environment + */ + def register(name: String): Unit = { + tableEnv.registerTableSource(name, toTableSource) + } + + /** + * Specifies an connector for reading data from a connector. + */ + def withConnector(connector: ConnectorDescriptor): StreamTableSourceDescriptor = { + connectorDescriptor = Some(connector) + this + } + + /** + * Specifies an encoding that defines how to read data from a connector. + */ + def withEncoding(encoding: EncodingDescriptor): StreamTableSourceDescriptor = { + encodingDescriptor = Some(encoding) --- End diff -- check if the connector requires an encoding? > 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)