提交 0e5ac4d7 编写于 作者: T Timo Walther

[FLINK-8866] [table] Make source/sink factories environment-dependent

Usually it is very uncommon to define both a batch and streaming source in the same factory. Separating by environment is a concept that can be find throughout the entire flink-table module because both sources and sinks behave quite different per environment.

This closes #6323.
上级 09fbfdfa
......@@ -20,7 +20,6 @@ package org.apache.flink.streaming.connectors.kafka;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.serialization.DeserializationSchema;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.streaming.connectors.kafka.config.StartupMode;
import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
import org.apache.flink.table.api.TableSchema;
......
......@@ -20,7 +20,6 @@ package org.apache.flink.streaming.connectors.kafka;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.serialization.DeserializationSchema;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.streaming.connectors.kafka.config.StartupMode;
import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
import org.apache.flink.table.api.TableSchema;
......
......@@ -21,8 +21,12 @@ package org.apache.flink.streaming.connectors.kafka;
import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_08;
/**
* Tests for {@link Kafka08JsonTableSourceFactory}.
* Tests for legacy Kafka08JsonTableSourceFactory.
*
* @deprecated Ensures backwards compatibility with Flink 1.5. Can be removed once we
* drop support for format-specific table sources.
*/
@Deprecated
public class Kafka08JsonTableSourceFactoryTest extends KafkaJsonTableSourceFactoryTestBase {
@Override
......
......@@ -20,7 +20,6 @@ package org.apache.flink.streaming.connectors.kafka;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.serialization.DeserializationSchema;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.streaming.connectors.kafka.config.StartupMode;
import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
import org.apache.flink.table.api.TableSchema;
......
......@@ -27,11 +27,10 @@ import org.apache.flink.table.descriptors.DescriptorProperties;
import org.apache.flink.table.descriptors.KafkaValidator;
import org.apache.flink.table.descriptors.SchemaValidator;
import org.apache.flink.table.factories.DeserializationSchemaFactory;
import org.apache.flink.table.factories.TableFactory;
import org.apache.flink.table.factories.StreamTableSourceFactory;
import org.apache.flink.table.factories.TableFactoryService;
import org.apache.flink.table.factories.TableSourceFactory;
import org.apache.flink.table.sources.RowtimeAttributeDescriptor;
import org.apache.flink.table.sources.TableSource;
import org.apache.flink.table.sources.StreamTableSource;
import org.apache.flink.types.Row;
import java.util.ArrayList;
......@@ -72,7 +71,7 @@ import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_TYPE;
/**
* Factory for creating configured instances of {@link KafkaTableSource}.
*/
public abstract class KafkaTableSourceFactory implements TableSourceFactory<Row>, TableFactory {
public abstract class KafkaTableSourceFactory implements StreamTableSourceFactory<Row> {
@Override
public Map<String, String> requiredContext() {
......@@ -119,7 +118,7 @@ public abstract class KafkaTableSourceFactory implements TableSourceFactory<Row>
}
@Override
public TableSource<Row> createTableSource(Map<String, String> properties) {
public StreamTableSource<Row> createStreamTableSource(Map<String, String> properties) {
final DescriptorProperties params = new DescriptorProperties(true);
params.putProperties(properties);
......
......@@ -22,8 +22,6 @@ import org.apache.flink.api.common.typeinfo.Types;
import org.apache.flink.formats.json.JsonRowSchemaConverter;
import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.factories.TableFactoryService;
import org.apache.flink.table.factories.TableSourceFactory;
import org.apache.flink.table.descriptors.DescriptorProperties;
import org.apache.flink.table.descriptors.FormatDescriptor;
import org.apache.flink.table.descriptors.Json;
......@@ -31,6 +29,8 @@ import org.apache.flink.table.descriptors.Kafka;
import org.apache.flink.table.descriptors.Rowtime;
import org.apache.flink.table.descriptors.Schema;
import org.apache.flink.table.descriptors.TestTableSourceDescriptor;
import org.apache.flink.table.factories.StreamTableSourceFactory;
import org.apache.flink.table.factories.TableFactoryService;
import org.apache.flink.table.sources.TableSource;
import org.apache.flink.table.sources.tsextractors.ExistingField;
import org.apache.flink.table.sources.wmstrategies.AscendingTimestamps;
......@@ -153,8 +153,8 @@ public abstract class KafkaJsonTableSourceFactoryTestBase {
DescriptorProperties properties = new DescriptorProperties(true);
testDesc.addProperties(properties);
final TableSource<?> factorySource =
((TableSourceFactory) TableFactoryService.find(TableSourceFactory.class, testDesc))
.createTableSource(properties.asMap());
TableFactoryService.find(StreamTableSourceFactory.class, testDesc)
.createStreamTableSource(properties.asMap());
assertEquals(builderSource, factorySource);
}
......
......@@ -32,8 +32,8 @@ import org.apache.flink.table.descriptors.Kafka;
import org.apache.flink.table.descriptors.Rowtime;
import org.apache.flink.table.descriptors.Schema;
import org.apache.flink.table.descriptors.TestTableSourceDescriptor;
import org.apache.flink.table.factories.StreamTableSourceFactory;
import org.apache.flink.table.factories.TableFactoryService;
import org.apache.flink.table.factories.TableSourceFactory;
import org.apache.flink.table.factories.utils.TestDeserializationSchema;
import org.apache.flink.table.factories.utils.TestTableFormat;
import org.apache.flink.table.sources.RowtimeAttributeDescriptor;
......@@ -147,8 +147,8 @@ public abstract class KafkaTableSourceFactoryTestBase extends TestLogger {
testDesc.addProperties(descriptorProperties);
final Map<String, String> propertiesMap = descriptorProperties.asMap();
final TableSource<?> actualSource = TableFactoryService.find(TableSourceFactory.class, testDesc)
.createTableSource(propertiesMap);
final TableSource<?> actualSource = TableFactoryService.find(StreamTableSourceFactory.class, testDesc)
.createStreamTableSource(propertiesMap);
assertEquals(expected, actualSource);
......
......@@ -46,15 +46,18 @@ import org.apache.flink.table.api.java.BatchTableEnvironment;
import org.apache.flink.table.api.java.StreamTableEnvironment;
import org.apache.flink.table.client.config.Deployment;
import org.apache.flink.table.client.config.Environment;
import org.apache.flink.table.client.config.Execution;
import org.apache.flink.table.client.config.Sink;
import org.apache.flink.table.client.config.Source;
import org.apache.flink.table.client.config.SourceSink;
import org.apache.flink.table.client.gateway.SessionContext;
import org.apache.flink.table.client.gateway.SqlExecutionException;
import org.apache.flink.table.factories.TableFactoryService;
import org.apache.flink.table.factories.TableSinkFactory;
import org.apache.flink.table.factories.TableSourceFactory;
import org.apache.flink.table.descriptors.DescriptorProperties;
import org.apache.flink.table.factories.BatchTableSinkFactory;
import org.apache.flink.table.factories.BatchTableSourceFactory;
import org.apache.flink.table.factories.StreamTableSinkFactory;
import org.apache.flink.table.factories.StreamTableSourceFactory;
import org.apache.flink.table.factories.TableFactoryService;
import org.apache.flink.table.functions.AggregateFunction;
import org.apache.flink.table.functions.FunctionService;
import org.apache.flink.table.functions.ScalarFunction;
......@@ -114,15 +117,12 @@ public class ExecutionContext<T> {
final DescriptorProperties properties = new DescriptorProperties(true);
descriptor.addProperties(properties);
final Map<String, String> propertyMap = properties.asMap();
if (descriptor instanceof Source || descriptor instanceof SourceSink) {
final TableSourceFactory<?> factory = (TableSourceFactory<?>)
TableFactoryService.find(TableSourceFactory.class, descriptor, classLoader);
tableSources.put(name, factory.createTableSource(propertyMap));
tableSources.put(name, createTableSource(mergedEnv.getExecution(), propertyMap, classLoader));
}
if (descriptor instanceof Sink || descriptor instanceof SourceSink) {
final TableSinkFactory<?> factory = (TableSinkFactory<?>)
TableFactoryService.find(TableSinkFactory.class, descriptor, classLoader);
tableSinks.put(name, factory.createTableSink(propertyMap));
tableSinks.put(name, createTableSink(mergedEnv.getExecution(), propertyMap, classLoader));
}
});
......@@ -205,6 +205,32 @@ public class ExecutionContext<T> {
}
}
private static TableSource<?> createTableSource(Execution execution, Map<String, String> sourceProperties, ClassLoader classLoader) {
if (execution.isStreamingExecution()) {
final StreamTableSourceFactory<?> factory = (StreamTableSourceFactory<?>)
TableFactoryService.find(StreamTableSourceFactory.class, sourceProperties, classLoader);
return factory.createStreamTableSource(sourceProperties);
} else if (execution.isBatchExecution()) {
final BatchTableSourceFactory<?> factory = (BatchTableSourceFactory<?>)
TableFactoryService.find(BatchTableSourceFactory.class, sourceProperties, classLoader);
return factory.createBatchTableSource(sourceProperties);
}
throw new SqlExecutionException("Unsupported execution type for sources.");
}
private static TableSink<?> createTableSink(Execution execution, Map<String, String> sinkProperties, ClassLoader classLoader) {
if (execution.isStreamingExecution()) {
final StreamTableSinkFactory<?> factory = (StreamTableSinkFactory<?>)
TableFactoryService.find(StreamTableSinkFactory.class, sinkProperties, classLoader);
return factory.createStreamTableSink(sinkProperties);
} else if (execution.isBatchExecution()) {
final BatchTableSinkFactory<?> factory = (BatchTableSinkFactory<?>)
TableFactoryService.find(BatchTableSinkFactory.class, sinkProperties, classLoader);
return factory.createBatchTableSink(sinkProperties);
}
throw new SqlExecutionException("Unsupported execution type for sources.");
}
// --------------------------------------------------------------------------------------------
/**
......
......@@ -24,10 +24,9 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.api.Types;
import org.apache.flink.table.client.gateway.local.DependencyTest;
import org.apache.flink.table.factories.TableFactory;
import org.apache.flink.table.factories.TableSourceFactory;
import org.apache.flink.table.descriptors.DescriptorProperties;
import org.apache.flink.table.descriptors.SchemaValidator;
import org.apache.flink.table.factories.StreamTableSourceFactory;
import org.apache.flink.table.sources.DefinedProctimeAttribute;
import org.apache.flink.table.sources.DefinedRowtimeAttributes;
import org.apache.flink.table.sources.RowtimeAttributeDescriptor;
......@@ -51,7 +50,7 @@ import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_TYPE;
/**
* Table source factory for testing the classloading in {@link DependencyTest}.
*/
public class TestTableSourceFactory implements TableSourceFactory<Row>, TableFactory {
public class TestTableSourceFactory implements StreamTableSourceFactory<Row> {
@Override
public Map<String, String> requiredContext() {
......@@ -73,7 +72,7 @@ public class TestTableSourceFactory implements TableSourceFactory<Row>, TableFac
}
@Override
public TableSource<Row> createTableSource(Map<String, String> properties) {
public StreamTableSource<Row> createStreamTableSource(Map<String, String> properties) {
final DescriptorProperties params = new DescriptorProperties(true);
params.putProperties(properties);
final Optional<String> proctime = SchemaValidator.deriveProctimeAttribute(params);
......
......@@ -20,10 +20,9 @@ package org.apache.flink.table.catalog
import org.apache.flink.table.api._
import org.apache.flink.table.descriptors.DescriptorProperties
import org.apache.flink.table.factories.{TableFactoryService, TableSourceFactory}
import org.apache.flink.table.factories.{BatchTableSourceFactory, StreamTableSourceFactory, TableFactoryService}
import org.apache.flink.table.plan.schema.{BatchTableSourceTable, StreamTableSourceTable, TableSourceSinkTable, TableSourceTable}
import org.apache.flink.table.plan.stats.FlinkStatistic
import org.apache.flink.table.sources.{BatchTableSource, StreamTableSource}
import org.apache.flink.table.util.Logging
/**
......@@ -44,32 +43,27 @@ object ExternalTableSourceUtil extends Logging {
val properties = new DescriptorProperties()
externalCatalogTable.addProperties(properties)
val javaMap = properties.asMap
val source = TableFactoryService.find(classOf[TableSourceFactory[_]], javaMap)
.asInstanceOf[TableSourceFactory[_]]
.createTableSource(javaMap)
tableEnv match {
// check for a batch table source in this batch environment
case _: BatchTableEnvironment =>
source match {
case bts: BatchTableSource[_] =>
new TableSourceSinkTable(Some(new BatchTableSourceTable(
bts,
new FlinkStatistic(externalCatalogTable.getTableStats))), None)
case _ => throw new TableException(
s"Found table source '${source.getClass.getCanonicalName}' is not applicable " +
s"in a batch environment.")
}
val source = TableFactoryService
.find(classOf[BatchTableSourceFactory[_]], javaMap)
.createBatchTableSource(javaMap)
val sourceTable = new BatchTableSourceTable(
source,
new FlinkStatistic(externalCatalogTable.getTableStats))
new TableSourceSinkTable(Some(sourceTable), None)
// check for a stream table source in this streaming environment
case _: StreamTableEnvironment =>
source match {
case sts: StreamTableSource[_] =>
new TableSourceSinkTable(Some(new StreamTableSourceTable(
sts,
new FlinkStatistic(externalCatalogTable.getTableStats))), None)
case _ => throw new TableException(
s"Found table source '${source.getClass.getCanonicalName}' is not applicable " +
s"in a streaming environment.")
}
val source = TableFactoryService
.find(classOf[StreamTableSourceFactory[_]], javaMap)
.createStreamTableSource(javaMap)
val sourceTable = new StreamTableSourceTable(
source,
new FlinkStatistic(externalCatalogTable.getTableStats))
new TableSourceSinkTable(Some(sourceTable), None)
case _ => throw new TableException("Unsupported table environment.")
}
}
......
......@@ -18,9 +18,9 @@
package org.apache.flink.table.descriptors
import org.apache.flink.table.api.{BatchTableEnvironment, Table, TableException, ValidationException}
import org.apache.flink.table.factories.{TableFactoryService, TableSourceFactory}
import org.apache.flink.table.sources.{BatchTableSource, TableSource}
import org.apache.flink.table.api.{BatchTableEnvironment, Table, ValidationException}
import org.apache.flink.table.factories.{BatchTableSourceFactory, TableFactoryService}
import org.apache.flink.table.sources.TableSource
class BatchTableSourceDescriptor(tableEnv: BatchTableEnvironment, connector: ConnectorDescriptor)
extends TableSourceDescriptor {
......@@ -46,15 +46,10 @@ class BatchTableSourceDescriptor(tableEnv: BatchTableEnvironment, connector: Con
def toTableSource: TableSource[_] = {
val properties = new DescriptorProperties()
addProperties(properties)
val source = TableFactoryService.find(classOf[TableSourceFactory[_]], this)
.asInstanceOf[TableSourceFactory[_]]
.createTableSource(properties.asMap)
source match {
case _: BatchTableSource[_] => source
case _ => throw new TableException(
s"Found table source '${source.getClass.getCanonicalName}' is not applicable " +
s"in a batch environment.")
}
val javaMap = properties.asMap
TableFactoryService
.find(classOf[BatchTableSourceFactory[_]], javaMap)
.createBatchTableSource(javaMap)
}
/**
......
......@@ -18,9 +18,9 @@
package org.apache.flink.table.descriptors
import org.apache.flink.table.api.{StreamTableEnvironment, Table, TableException, ValidationException}
import org.apache.flink.table.factories.{TableFactoryService, TableSourceFactory}
import org.apache.flink.table.sources.{StreamTableSource, TableSource}
import org.apache.flink.table.api.{StreamTableEnvironment, Table, ValidationException}
import org.apache.flink.table.factories.{StreamTableSourceFactory, TableFactoryService}
import org.apache.flink.table.sources.TableSource
/**
* Descriptor for specifying a table source in a streaming environment.
......@@ -49,15 +49,10 @@ class StreamTableSourceDescriptor(tableEnv: StreamTableEnvironment, connector: C
def toTableSource: TableSource[_] = {
val properties = new DescriptorProperties()
addProperties(properties)
val source = TableFactoryService.find(classOf[TableSourceFactory[_]], this)
.asInstanceOf[TableSourceFactory[_]]
.createTableSource(properties.asMap)
source match {
case _: StreamTableSource[_] => source
case _ => throw new TableException(
s"Found table source '${source.getClass.getCanonicalName}' is not applicable " +
s"in a streaming environment.")
}
val javaMap = properties.asMap
TableFactoryService
.find(classOf[StreamTableSourceFactory[_]], javaMap)
.createStreamTableSource(javaMap)
}
/**
......
......@@ -18,18 +18,24 @@
package org.apache.flink.table.factories
import org.apache.flink.table.sources.TableSource
import java.util
trait TableSourceFactory[T] extends TableFactory {
import org.apache.flink.table.sinks.BatchTableSink
/**
* A factory to create configured table sink instances in a streaming environment based on
* string-based properties. See also [[TableFactory]] for more information.
*
* @tparam T type of records that the factory consumes
*/
trait BatchTableSinkFactory[T] extends TableFactory {
/**
* Creates and configures a [[org.apache.flink.table.sources.TableSource]]
* Creates and configures a [[org.apache.flink.table.sinks.BatchTableSink]]
* using the given properties.
*
* @param properties normalized properties describing a table source.
* @return the configured table source.
* @param properties normalized properties describing a table sink.
* @return the configured table sink.
*/
def createTableSource(properties: util.Map[String, String]): TableSource[T]
def createBatchTableSink(properties: util.Map[String, String]): BatchTableSink[T]
}
/*
* 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.factories
import java.util
import org.apache.flink.table.sources.BatchTableSource
/**
* A factory to create configured table source instances in a batch environment based on
* string-based properties. See also [[TableFactory]] for more information.
*
* @tparam T type of records that the factory produces
*/
trait BatchTableSourceFactory[T] extends TableFactory {
/**
* Creates and configures a [[org.apache.flink.table.sources.BatchTableSource]]
* using the given properties.
*
* @param properties normalized properties describing a batch table source.
* @return the configured batch table source.
*/
def createBatchTableSource(properties: util.Map[String, String]): BatchTableSource[T]
}
/*
* 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.factories
import java.util
import org.apache.flink.table.sinks.StreamTableSink
/**
* A factory to create configured table sink instances in a streaming environment based on
* string-based properties. See also [[TableFactory]] for more information.
*
* @tparam T type of records that the factory consumes
*/
trait StreamTableSinkFactory[T] extends TableFactory {
/**
* Creates and configures a [[org.apache.flink.table.sinks.StreamTableSink]]
* using the given properties.
*
* @param properties normalized properties describing a table sink.
* @return the configured table sink.
*/
def createStreamTableSink(properties: util.Map[String, String]): StreamTableSink[T]
}
/*
* 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.factories
import java.util
import org.apache.flink.table.sources.StreamTableSource
/**
* A factory to create configured table source instances in a streaming environment based on
* string-based properties. See also [[TableFactory]] for more information.
*
* @tparam T type of records that the factory produces
*/
trait StreamTableSourceFactory[T] extends TableFactory {
/**
* Creates and configures a [[org.apache.flink.table.sources.StreamTableSource]]
* using the given properties.
*
* @param properties normalized properties describing a stream table source.
* @return the configured stream table source.
*/
def createStreamTableSource(properties: util.Map[String, String]): StreamTableSource[T]
}
......@@ -29,7 +29,7 @@ import org.apache.flink.table.api.Table
*
* @tparam T Type of [[DataStream]] that this [[TableSink]] expects and supports.
*/
trait AppendStreamTableSink[T] extends TableSink[T] {
trait AppendStreamTableSink[T] extends StreamTableSink[T] {
/** Emits the DataStream. */
def emitDataStream(dataStream: DataStream[T]): Unit
......
......@@ -21,7 +21,6 @@ package org.apache.flink.table.sinks
import java.util
import org.apache.flink.table.api.TableException
import org.apache.flink.table.factories.{TableFactory, TableSinkFactory}
import org.apache.flink.table.descriptors.ConnectorDescriptorValidator._
import org.apache.flink.table.descriptors.CsvValidator._
import org.apache.flink.table.descriptors.DescriptorProperties._
......@@ -29,12 +28,16 @@ import org.apache.flink.table.descriptors.FileSystemValidator._
import org.apache.flink.table.descriptors.FormatDescriptorValidator._
import org.apache.flink.table.descriptors.SchemaValidator._
import org.apache.flink.table.descriptors._
import org.apache.flink.table.factories.{BatchTableSinkFactory, StreamTableSinkFactory, TableFactory}
import org.apache.flink.types.Row
/**
* Factory for creating configured instances of [[CsvTableSink]].
*/
class CsvTableSinkFactory extends TableSinkFactory[Row] with TableFactory {
class CsvTableSinkFactory
extends TableFactory
with StreamTableSinkFactory[Row]
with BatchTableSinkFactory[Row] {
override def requiredContext(): util.Map[String, String] = {
val context = new util.HashMap[String, String]()
......@@ -60,14 +63,33 @@ class CsvTableSinkFactory extends TableSinkFactory[Row] with TableFactory {
properties
}
override def createTableSink(properties: util.Map[String, String]): TableSink[Row] = {
override def createStreamTableSink(
properties: util.Map[String, String])
: StreamTableSink[Row] = {
createTableSink(isStreaming = true, properties)
}
override def createBatchTableSink(
properties: util.Map[String, String])
: BatchTableSink[Row] = {
createTableSink(isStreaming = false, properties)
}
private def createTableSink(
isStreaming: Boolean,
properties: util.Map[String, String])
: CsvTableSink = {
val params = new DescriptorProperties()
params.putProperties(properties)
// validate
new FileSystemValidator().validate(params)
new CsvValidator().validate(params)
new SchemaValidator(true, false, false).validate(params)
new SchemaValidator(
isStreaming,
supportsSourceTimestamps = false,
supportsSourceWatermarks = false).validate(params)
// build
val formatSchema = params.getTableSchema(FORMAT_FIELDS)
......
......@@ -42,7 +42,7 @@ import org.apache.flink.table.api.Table
*
* @tparam T Type of records that this [[TableSink]] expects and supports.
*/
trait RetractStreamTableSink[T] extends TableSink[JTuple2[JBool, T]] {
trait RetractStreamTableSink[T] extends StreamTableSink[JTuple2[JBool, T]] {
/** Returns the requested record type */
def getRecordType: TypeInformation[T]
......
......@@ -16,19 +16,18 @@
* limitations under the License.
*/
package org.apache.flink.table.factories
package org.apache.flink.table.sinks
import org.apache.flink.table.sinks.TableSink
import org.apache.flink.streaming.api.datastream.DataStream
import java.util
/**
* Defines an external stream table and provides write access to its data.
*
* @tparam T Type of the [[DataStream]] created by this [[TableSink]].
*/
trait StreamTableSink[T] extends TableSink[T] {
/** Emits the DataStream. */
def emitDataStream(dataStream: DataStream[T]): Unit
trait TableSinkFactory[T] {
/**
* Creates and configures a [[org.apache.flink.table.sinks.TableSink]]
* using the given properties.
*
* @param properties normalized properties describing a table source.
* @return the configured table source.
*/
def createTableSink(properties: util.Map[String, String]): TableSink[T]
}
......@@ -48,7 +48,7 @@ import org.apache.flink.table.api.{Table, Types}
*
* @tparam T Type of records that this [[TableSink]] expects and supports.
*/
trait UpsertStreamTableSink[T] extends TableSink[JTuple2[JBool, T]] {
trait UpsertStreamTableSink[T] extends StreamTableSink[JTuple2[JBool, T]] {
/**
* Configures the unique key fields of the [[Table]] to write.
......
......@@ -21,7 +21,6 @@ package org.apache.flink.table.sources
import java.util
import org.apache.flink.table.api.TableException
import org.apache.flink.table.factories.{TableFactory, TableSourceFactory}
import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.{CONNECTOR_PROPERTY_VERSION, CONNECTOR_TYPE}
import org.apache.flink.table.descriptors.CsvValidator._
import org.apache.flink.table.descriptors.DescriptorProperties.toScala
......@@ -29,12 +28,16 @@ import org.apache.flink.table.descriptors.FileSystemValidator.{CONNECTOR_PATH, C
import org.apache.flink.table.descriptors.FormatDescriptorValidator.{FORMAT_PROPERTY_VERSION, FORMAT_TYPE}
import org.apache.flink.table.descriptors.SchemaValidator.SCHEMA
import org.apache.flink.table.descriptors._
import org.apache.flink.table.factories.{BatchTableSourceFactory, StreamTableSourceFactory, TableFactory}
import org.apache.flink.types.Row
/**
* Factory for creating configured instances of [[CsvTableSource]].
*/
class CsvTableSourceFactory extends TableSourceFactory[Row] with TableFactory {
class CsvTableSourceFactory
extends TableFactory
with StreamTableSourceFactory[Row]
with BatchTableSourceFactory[Row] {
override def requiredContext(): util.Map[String, String] = {
val context = new util.HashMap[String, String]()
......@@ -65,7 +68,23 @@ class CsvTableSourceFactory extends TableSourceFactory[Row] with TableFactory {
properties
}
override def createTableSource(properties: util.Map[String, String]): TableSource[Row] = {
override def createStreamTableSource(
properties: util.Map[String, String])
: StreamTableSource[Row] = {
createTableSource(isStreaming = true, properties)
}
override def createBatchTableSource(
properties: util.Map[String, String])
: BatchTableSource[Row] = {
createTableSource(isStreaming = false, properties)
}
private def createTableSource(
isStreaming: Boolean,
properties: util.Map[String, String])
: CsvTableSource = {
val params = new DescriptorProperties()
params.putProperties(properties)
......@@ -73,7 +92,7 @@ class CsvTableSourceFactory extends TableSourceFactory[Row] with TableFactory {
new FileSystemValidator().validate(params)
new CsvValidator().validate(params)
new SchemaValidator(
isStreamEnvironment = true,
isStreaming,
supportsSourceTimestamps = false,
supportsSourceWatermarks = false).validate(params)
......
......@@ -21,7 +21,7 @@ package org.apache.flink.table.sources
import org.apache.flink.streaming.api.datastream.DataStream
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
/** Defines an external stream table and provides access to its data.
/** Defines an external stream table and provides read access to its data.
*
* @tparam T Type of the [[DataStream]] created by this [[TableSource]].
*/
......
......@@ -37,7 +37,7 @@ class TableSinkFactoryServiceTest {
@Test
def testValidProperties(): Unit = {
val props = properties()
assertTrue(TableFactoryService.find(classOf[TableSinkFactory[_]], props)
assertTrue(TableFactoryService.find(classOf[StreamTableSinkFactory[_]], props)
.isInstanceOf[TestTableSinkFactory])
}
......@@ -45,7 +45,7 @@ class TableSinkFactoryServiceTest {
def testInvalidContext(): Unit = {
val props = properties()
props.put(CONNECTOR_TYPE, "unknown-connector-type")
TableFactoryService.find(classOf[TableSinkFactory[_]], props)
TableFactoryService.find(classOf[StreamTableSinkFactory[_]], props)
}
@Test
......@@ -53,7 +53,7 @@ class TableSinkFactoryServiceTest {
val props = properties()
props.put(CONNECTOR_PROPERTY_VERSION, "2")
// the table source should still be found
assertTrue(TableFactoryService.find(classOf[TableSinkFactory[_]], props)
assertTrue(TableFactoryService.find(classOf[StreamTableSinkFactory[_]], props)
.isInstanceOf[TestTableSinkFactory])
}
......@@ -61,7 +61,7 @@ class TableSinkFactoryServiceTest {
def testUnsupportedProperty(): Unit = {
val props = properties()
props.put("format.path_new", "/new/path")
TableFactoryService.find(classOf[TableSinkFactory[_]], props)
TableFactoryService.find(classOf[StreamTableSinkFactory[_]], props)
}
private def properties(): JMap[String, String] = {
......@@ -77,7 +77,6 @@ class TableSinkFactoryServiceTest {
properties.put("schema.0.field.0.name", "a")
properties.put("schema.0.field.1.name", "b")
properties.put("schema.0.field.2.name", "c")
properties.put("failing", "false")
properties
}
}
......@@ -44,7 +44,7 @@ class TableSourceFactoryServiceTest {
val props = properties()
props.put(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_FIXED)
props.put(FORMAT_TYPE, FORMAT_TYPE_VALUE_TEST)
assertTrue(TableFactoryService.find(classOf[TableSourceFactory[_]], props)
assertTrue(TableFactoryService.find(classOf[StreamTableSourceFactory[_]], props)
.isInstanceOf[TestFixedFormatTableFactory])
}
......@@ -53,7 +53,7 @@ class TableSourceFactoryServiceTest {
val props = properties()
props.put(CONNECTOR_TYPE, "unknown-connector-type")
props.put(FORMAT_TYPE, FORMAT_TYPE_VALUE_TEST)
TableFactoryService.find(classOf[TableSourceFactory[_]], props)
TableFactoryService.find(classOf[StreamTableSourceFactory[_]], props)
}
@Test
......@@ -63,7 +63,7 @@ class TableSourceFactoryServiceTest {
props.put(FORMAT_TYPE, FORMAT_TYPE_VALUE_TEST)
props.put(CONNECTOR_PROPERTY_VERSION, "2")
// the table source should still be found
assertTrue(TableFactoryService.find(classOf[TableSourceFactory[_]], props)
assertTrue(TableFactoryService.find(classOf[StreamTableSourceFactory[_]], props)
.isInstanceOf[TestFixedFormatTableFactory])
}
......@@ -73,7 +73,7 @@ class TableSourceFactoryServiceTest {
props.put(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_FIXED)
props.put(FORMAT_TYPE, FORMAT_TYPE_VALUE_TEST)
props.put("format.unknown-format-type-property", "/new/path")
TableFactoryService.find(classOf[TableSourceFactory[_]], props)
TableFactoryService.find(classOf[StreamTableSourceFactory[_]], props)
}
@Test
......@@ -81,7 +81,7 @@ class TableSourceFactoryServiceTest {
val props = properties()
props.put(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_WILDCARD)
props.put("format.unknown-format-type-property", "wildcard-property")
val actualTableSource = TableFactoryService.find(classOf[TableSourceFactory[_]], props)
val actualTableSource = TableFactoryService.find(classOf[StreamTableSourceFactory[_]], props)
assertTrue(actualTableSource.isInstanceOf[TestWildcardFormatTableSourceFactory])
}
......
......@@ -20,19 +20,17 @@ package org.apache.flink.table.factories.utils
import java.util
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.table.api.TableSchema
import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.{CONNECTOR_PROPERTY_VERSION, CONNECTOR_TYPE}
import org.apache.flink.table.descriptors.FormatDescriptorValidator.{FORMAT_PROPERTY_VERSION, FORMAT_TYPE}
import org.apache.flink.table.factories.TableSourceFactory
import org.apache.flink.table.factories.StreamTableSourceFactory
import org.apache.flink.table.factories.utils.TestFixedFormatTableFactory._
import org.apache.flink.table.sources.TableSource
import org.apache.flink.table.sources.StreamTableSource
import org.apache.flink.types.Row
/**
* Table source factory for testing with a fixed format.
*/
class TestFixedFormatTableFactory extends TableSourceFactory[Row] {
class TestFixedFormatTableFactory extends StreamTableSourceFactory[Row] {
override def requiredContext(): util.Map[String, String] = {
val context = new util.HashMap[String, String]()
......@@ -51,12 +49,10 @@ class TestFixedFormatTableFactory extends TableSourceFactory[Row] {
properties
}
override def createTableSource(properties: util.Map[String, String]): TableSource[Row] = {
new TableSource[Row] {
override def getTableSchema: TableSchema = throw new UnsupportedOperationException()
override def getReturnType: TypeInformation[Row] = throw new UnsupportedOperationException()
}
override def createStreamTableSource(
properties: util.Map[String, String])
: StreamTableSource[Row] = {
throw new UnsupportedOperationException()
}
}
......
......@@ -18,16 +18,17 @@
package org.apache.flink.table.factories.utils
import org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_DERIVE_SCHEMA
import org.apache.flink.table.descriptors.{DescriptorProperties, FormatDescriptor}
import org.apache.flink.table.factories.TableFormatFactoryServiceTest._
/**
* Format descriptor for testing purposes.
*/
class TestTableFormat extends FormatDescriptor("test-format", 1) {
class TestTableFormat extends FormatDescriptor(TEST_FORMAT_TYPE, 1) {
override protected def addFormatProperties(properties: DescriptorProperties): Unit = {
properties.putString("format.important", "this is important")
properties.putString("format.path", "/path/to/sth")
properties.putString("format.derive-schema", "true")
properties.putString(COMMON_PATH, "/path/to/sth")
properties.putString(FORMAT_DERIVE_SCHEMA, "true")
}
}
......@@ -20,18 +20,17 @@ package org.apache.flink.table.factories.utils
import java.util
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.table.descriptors.ConnectorDescriptorValidator._
import org.apache.flink.table.descriptors.FormatDescriptorValidator._
import org.apache.flink.table.factories.utils.TestTableSinkFactory._
import org.apache.flink.table.factories.{TableFactory, TableSinkFactory}
import org.apache.flink.table.sinks.TableSink
import org.apache.flink.table.factories.{StreamTableSinkFactory, TableFactory}
import org.apache.flink.table.sinks.StreamTableSink
import org.apache.flink.types.Row
/**
* Test table sink factory.
*/
class TestTableSinkFactory extends TableSinkFactory[Row] with TableFactory {
class TestTableSinkFactory extends StreamTableSinkFactory[Row] with TableFactory {
override def requiredContext(): util.Map[String, String] = {
val context = new util.HashMap[String, String]()
......@@ -48,26 +47,13 @@ class TestTableSinkFactory extends TableSinkFactory[Row] with TableFactory {
properties.add(FORMAT_PATH)
properties.add("schema.#.name")
properties.add("schema.#.field.#.name")
properties.add("failing")
properties
}
override def createTableSink(properties: util.Map[String, String]): TableSink[Row] = {
if (properties.get("failing") == "true") {
throw new IllegalArgumentException("Error in this factory.")
}
new TableSink[Row] {
override def getOutputType: TypeInformation[Row] = throw new UnsupportedOperationException()
override def getFieldNames: Array[String] = throw new UnsupportedOperationException()
override def getFieldTypes: Array[TypeInformation[_]] =
throw new UnsupportedOperationException()
override def configure(fieldNames: Array[String],
fieldTypes: Array[TypeInformation[_]]): TableSink[Row] =
throw new UnsupportedOperationException()
}
override def createStreamTableSink(
properties: util.Map[String, String])
: StreamTableSink[Row] = {
throw new UnsupportedOperationException()
}
}
......
......@@ -20,19 +20,17 @@ package org.apache.flink.table.factories.utils
import java.util
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.table.api.TableSchema
import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.{CONNECTOR_PROPERTY_VERSION, CONNECTOR_TYPE}
import org.apache.flink.table.descriptors.FormatDescriptorValidator.{FORMAT_PROPERTY_VERSION, FORMAT_TYPE}
import org.apache.flink.table.factories.utils.TestTableSinkFactory.{CONNECTOR_TYPE_VALUE_TEST, FORMAT_TYPE_VALUE_TEST}
import org.apache.flink.table.factories.{TableFactory, TableSourceFactory}
import org.apache.flink.table.sources.TableSource
import org.apache.flink.table.factories.{StreamTableSourceFactory, TableFactory}
import org.apache.flink.table.sources.StreamTableSource
import org.apache.flink.types.Row
/**
* Table source factory for testing.
*/
class TestTableSourceFactory extends TableSourceFactory[Row] with TableFactory {
class TestTableSourceFactory extends StreamTableSourceFactory[Row] with TableFactory {
override def requiredContext(): util.Map[String, String] = {
val context = new util.HashMap[String, String]()
......@@ -49,19 +47,13 @@ class TestTableSourceFactory extends TableSourceFactory[Row] with TableFactory {
properties.add("format.path")
properties.add("schema.#.name")
properties.add("schema.#.field.#.name")
properties.add("failing")
properties
}
override def createTableSource(properties: util.Map[String, String]): TableSource[Row] = {
if (properties.get("failing") == "true") {
throw new IllegalArgumentException("Error in this factory.")
}
new TableSource[Row] {
override def getTableSchema: TableSchema = throw new UnsupportedOperationException()
override def getReturnType: TypeInformation[Row] = throw new UnsupportedOperationException()
}
override def createStreamTableSource(
properties: util.Map[String, String])
: StreamTableSource[Row] = {
throw new UnsupportedOperationException()
}
}
......
......@@ -21,16 +21,15 @@ package org.apache.flink.table.factories.utils
import java.util
import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.{CONNECTOR_PROPERTY_VERSION, CONNECTOR_TYPE}
import org.apache.flink.table.factories.TableSourceFactory
import org.apache.flink.table.factories.StreamTableSourceFactory
import org.apache.flink.table.factories.utils.TestWildcardFormatTableSourceFactory.CONNECTOR_TYPE_VALUE_WILDCARD
import org.apache.flink.table.sources.TableSource
import org.apache.flink.table.sources.StreamTableSource
import org.apache.flink.types.Row
/**
* Table source factory for testing with a wildcard format ("format.*").
*/
class TestWildcardFormatTableSourceFactory
extends TableSourceFactory[Row] {
class TestWildcardFormatTableSourceFactory extends StreamTableSourceFactory[Row] {
override def requiredContext(): util.Map[String, String] = {
val context = new util.HashMap[String, String]()
......@@ -47,7 +46,9 @@ class TestWildcardFormatTableSourceFactory
properties
}
override def createTableSource(properties: util.Map[String, String]): TableSource[Row] = {
override def createStreamTableSource(
properties: util.Map[String, String])
: StreamTableSource[Row] = {
throw new UnsupportedOperationException()
}
}
......
......@@ -739,9 +739,9 @@ class SqlITCase extends StreamingWithStateTestBase {
tEnv.registerTable("sourceTable", t)
tEnv.registerTableSource("targetTable",
new InMemoryTableFactory(3).createTableSource(props.asMap))
new InMemoryTableFactory(3).createStreamTableSource(props.asMap))
tEnv.registerTableSink("targetTable",
new InMemoryTableFactory(3).createTableSink(props.asMap))
new InMemoryTableFactory(3).createStreamTableSink(props.asMap))
tEnv.sqlUpdate("INSERT INTO targetTable SELECT a, b, c, rowtime FROM sourceTable")
tEnv.sqlQuery("SELECT a, e, f, t from targetTable")
......
......@@ -25,9 +25,9 @@ import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.{CONNECTO
import org.apache.flink.table.descriptors.RowtimeValidator._
import org.apache.flink.table.descriptors.SchemaValidator._
import org.apache.flink.table.descriptors.{DescriptorProperties, SchemaValidator}
import org.apache.flink.table.factories.{TableFactory, TableSinkFactory, TableSourceFactory}
import org.apache.flink.table.sinks.TableSink
import org.apache.flink.table.sources.TableSource
import org.apache.flink.table.factories.{StreamTableSinkFactory, StreamTableSourceFactory, TableFactory}
import org.apache.flink.table.sinks.StreamTableSink
import org.apache.flink.table.sources.StreamTableSource
import org.apache.flink.types.Row
/**
......@@ -38,27 +38,43 @@ import org.apache.flink.types.Row
*
* @param terminationCount determines when to shutdown the streaming source function
*/
class InMemoryTableFactory(terminationCount: Int) extends TableSourceFactory[Row]
with TableSinkFactory[Row] with TableFactory {
override def createTableSink(properties: util.Map[String, String]): TableSink[Row] = {
class InMemoryTableFactory(terminationCount: Int)
extends TableFactory
with StreamTableSourceFactory[Row]
with StreamTableSinkFactory[Row] {
override def createStreamTableSink(
properties: util.Map[String, String])
: StreamTableSink[Row] = {
val params: DescriptorProperties = new DescriptorProperties(true)
params.putProperties(properties)
// validate
new SchemaValidator(true, true, true).validate(params)
new SchemaValidator(
isStreamEnvironment = true,
supportsSourceTimestamps = true,
supportsSourceWatermarks = true).validate(params)
val tableSchema = SchemaValidator.deriveTableSinkSchema(params)
new MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink()
.configure(tableSchema.getColumnNames, tableSchema.getTypes)
.asInstanceOf[StreamTableSink[Row]]
}
override def createTableSource(properties: util.Map[String, String]): TableSource[Row] = {
override def createStreamTableSource(
properties: util.Map[String, String])
: StreamTableSource[Row] = {
val params: DescriptorProperties = new DescriptorProperties(true)
params.putProperties(properties)
// validate
new SchemaValidator(true, true, true).validate(params)
new SchemaValidator(
isStreamEnvironment = true,
supportsSourceTimestamps = true,
supportsSourceWatermarks = true).validate(params)
val tableSchema = SchemaValidator.deriveTableSourceSchema(params)
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册