Github user pnowojski commented on a diff in the pull request: https://github.com/apache/flink/pull/6343#discussion_r202964290 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableUtil.scala --- @@ -0,0 +1,148 @@ +/* + * 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.catalog + +import java.util + +import org.apache.flink.table.api._ +import org.apache.flink.table.descriptors.DescriptorProperties +import org.apache.flink.table.factories._ +import org.apache.flink.table.plan.schema._ +import org.apache.flink.table.plan.stats.FlinkStatistic +import org.apache.flink.table.util.Logging + + +/** + * The utility class is used to convert [[ExternalCatalogTable]] to [[TableSourceSinkTable]]. + * + * It uses [[TableFactoryService]] for discovering. + */ +object ExternalTableUtil extends Logging { + + /** + * Converts an [[ExternalCatalogTable]] instance to a [[TableSourceTable]] instance + * + * @param externalCatalogTable the [[ExternalCatalogTable]] instance which to convert + * @return converted [[TableSourceTable]] instance from the input catalog table + */ + def fromExternalCatalogTable[T1, T2]( + tableEnv: TableEnvironment, + externalCatalogTable: ExternalCatalogTable) + : TableSourceSinkTable[T1, T2] = { + + val properties = new DescriptorProperties() + externalCatalogTable.addProperties(properties) + val javaMap = properties.asMap + val statistics = new FlinkStatistic(externalCatalogTable.getTableStats) + + val source: Option[TableSourceTable[T1]] = tableEnv match { + // check for a batch table source in this batch environment + case _: BatchTableEnvironment if externalCatalogTable.isBatchTable => + createBatchTableSource(externalCatalogTable, javaMap, statistics) + + // check for a stream table source in this stream environment + case _: StreamTableEnvironment if externalCatalogTable.isStreamTable => + createStreamTableSource(externalCatalogTable, javaMap, statistics) + + case _ => + throw new ValidationException( + "External catalog table does not support the current environment for a table source.") + } + + val sink: Option[TableSinkTable[T2]] = tableEnv match { + // check for a batch table sink in this batch environment + case _: BatchTableEnvironment if externalCatalogTable.isBatchTable => + createBatchTableSink(externalCatalogTable, javaMap, statistics) + + // check for a stream table sink in this stream environment + case _: StreamTableEnvironment if externalCatalogTable.isStreamTable => + createStreamTableSink(externalCatalogTable, javaMap, statistics) + + case _ => + throw new ValidationException( + "External catalog table does not support the current environment for a table sink.") + } + + new TableSourceSinkTable[T1, T2](source, sink) + } + + private def createBatchTableSource[T]( --- End diff -- I still do not like this lack of abstraction between batch and streaming in form of `createBatchTableSource`/`createStreamTableSource`. Instead of writing if/elses everywhere in our code there should be some common layer that handles such logic. Here half of the problem boils down to factories with methods createBatchTableSource and createStreamTableSource.
---