提交 abbb8905 编写于 作者: T Timo Walther

[FLINK-8866] [table] Merge table source/sink/format factories

- Rename to TableFactory and move it to org.apache.flink.table.factories package
- Unify source/sink/format factories with same logic and exceptions
上级 9597248a
......@@ -23,13 +23,13 @@ import org.apache.flink.streaming.connectors.kafka.config.StartupMode;
import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
import org.apache.flink.table.api.TableException;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.connectors.DiscoverableTableFactory;
import org.apache.flink.table.connectors.TableSourceFactory;
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.formats.DeserializationSchemaFactory;
import org.apache.flink.table.formats.TableFormatFactoryService;
import org.apache.flink.table.factories.DeserializationSchemaFactory;
import org.apache.flink.table.factories.TableFactory;
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.types.Row;
......@@ -72,7 +72,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>, DiscoverableTableFactory {
public abstract class KafkaTableSourceFactory implements TableSourceFactory<Row>, TableFactory {
@Override
public Map<String, String> requiredContext() {
......@@ -129,7 +129,7 @@ public abstract class KafkaTableSourceFactory implements TableSourceFactory<Row>
new KafkaValidator().validate(params);
// deserialization schema using format discovery
final DeserializationSchemaFactory<?> formatFactory = TableFormatFactoryService.find(
final DeserializationSchemaFactory<?> formatFactory = TableFactoryService.find(
DeserializationSchemaFactory.class,
properties,
this.getClass().getClassLoader());
......
......@@ -22,8 +22,8 @@ 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.connectors.TableFactoryService;
import org.apache.flink.table.connectors.TableSourceFactory;
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;
......
......@@ -27,15 +27,17 @@ 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;
import org.apache.flink.table.api.Types;
import org.apache.flink.table.descriptors.DescriptorProperties;
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.formats.utils.TestDeserializationSchema;
import org.apache.flink.table.formats.utils.TestTableFormat;
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;
import org.apache.flink.table.sources.TableSource;
import org.apache.flink.table.sources.TableSourceFactoryService;
import org.apache.flink.table.sources.tsextractors.ExistingField;
import org.apache.flink.table.sources.wmstrategies.AscendingTimestamps;
import org.apache.flink.types.Row;
......@@ -141,8 +143,12 @@ public abstract class KafkaTableSourceFactoryTestBase extends TestLogger {
.field(EVENT_TIME, Types.SQL_TIMESTAMP()).rowtime(
new Rowtime().timestampsFromField(TIME).watermarksPeriodicAscending())
.field(PROC_TIME, Types.SQL_TIMESTAMP()).proctime());
final DescriptorProperties descriptorProperties = new DescriptorProperties(true);
testDesc.addProperties(descriptorProperties);
final Map<String, String> propertiesMap = descriptorProperties.asMap();
final TableSource<?> actualSource = TableSourceFactoryService.findAndCreateTableSource(testDesc);
final TableSource<?> actualSource = TableFactoryService.find(TableSourceFactory.class, testDesc)
.createTableSource(propertiesMap);
assertEquals(expected, actualSource);
......
......@@ -23,8 +23,8 @@ import org.apache.flink.api.common.serialization.SerializationSchema;
import org.apache.flink.table.descriptors.AvroValidator;
import org.apache.flink.table.descriptors.DescriptorProperties;
import org.apache.flink.table.descriptors.FormatDescriptorValidator;
import org.apache.flink.table.formats.DeserializationSchemaFactory;
import org.apache.flink.table.formats.SerializationSchemaFactory;
import org.apache.flink.table.factories.DeserializationSchemaFactory;
import org.apache.flink.table.factories.SerializationSchemaFactory;
import org.apache.flink.types.Row;
import org.apache.avro.specific.SpecificRecord;
......
......@@ -24,9 +24,9 @@ import org.apache.flink.formats.avro.generated.User;
import org.apache.flink.table.descriptors.Avro;
import org.apache.flink.table.descriptors.Descriptor;
import org.apache.flink.table.descriptors.DescriptorProperties;
import org.apache.flink.table.formats.DeserializationSchemaFactory;
import org.apache.flink.table.formats.SerializationSchemaFactory;
import org.apache.flink.table.formats.TableFormatFactoryService;
import org.apache.flink.table.factories.DeserializationSchemaFactory;
import org.apache.flink.table.factories.SerializationSchemaFactory;
import org.apache.flink.table.factories.TableFactoryService;
import org.apache.flink.util.TestLogger;
import org.junit.Test;
......@@ -63,7 +63,7 @@ public class AvroRowFormatFactoryTest extends TestLogger {
}
private void testRecordClassSerializationSchema(Map<String, String> properties) {
final DeserializationSchema<?> actual2 = TableFormatFactoryService
final DeserializationSchema<?> actual2 = TableFactoryService
.find(DeserializationSchemaFactory.class, properties)
.createDeserializationSchema(properties);
final AvroRowDeserializationSchema expected2 = new AvroRowDeserializationSchema(AVRO_SPECIFIC_RECORD);
......@@ -71,7 +71,7 @@ public class AvroRowFormatFactoryTest extends TestLogger {
}
private void testRecordClassDeserializationSchema(Map<String, String> properties) {
final SerializationSchema<?> actual1 = TableFormatFactoryService
final SerializationSchema<?> actual1 = TableFactoryService
.find(SerializationSchemaFactory.class, properties)
.createSerializationSchema(properties);
final SerializationSchema<?> expected1 = new AvroRowSerializationSchema(AVRO_SPECIFIC_RECORD);
......@@ -79,7 +79,7 @@ public class AvroRowFormatFactoryTest extends TestLogger {
}
private void testAvroSchemaDeserializationSchema(Map<String, String> properties) {
final DeserializationSchema<?> actual2 = TableFormatFactoryService
final DeserializationSchema<?> actual2 = TableFactoryService
.find(DeserializationSchemaFactory.class, properties)
.createDeserializationSchema(properties);
final AvroRowDeserializationSchema expected2 = new AvroRowDeserializationSchema(AVRO_SCHEMA);
......@@ -87,7 +87,7 @@ public class AvroRowFormatFactoryTest extends TestLogger {
}
private void testAvroSchemaSerializationSchema(Map<String, String> properties) {
final SerializationSchema<?> actual1 = TableFormatFactoryService
final SerializationSchema<?> actual1 = TableFactoryService
.find(SerializationSchemaFactory.class, properties)
.createSerializationSchema(properties);
final SerializationSchema<?> expected1 = new AvroRowSerializationSchema(AVRO_SCHEMA);
......
......@@ -26,8 +26,8 @@ import org.apache.flink.table.descriptors.DescriptorProperties;
import org.apache.flink.table.descriptors.FormatDescriptorValidator;
import org.apache.flink.table.descriptors.JsonValidator;
import org.apache.flink.table.descriptors.SchemaValidator;
import org.apache.flink.table.formats.DeserializationSchemaFactory;
import org.apache.flink.table.formats.SerializationSchemaFactory;
import org.apache.flink.table.factories.DeserializationSchemaFactory;
import org.apache.flink.table.factories.SerializationSchemaFactory;
import org.apache.flink.types.Row;
import java.util.ArrayList;
......
......@@ -26,9 +26,9 @@ import org.apache.flink.table.descriptors.Descriptor;
import org.apache.flink.table.descriptors.DescriptorProperties;
import org.apache.flink.table.descriptors.Json;
import org.apache.flink.table.descriptors.Schema;
import org.apache.flink.table.formats.DeserializationSchemaFactory;
import org.apache.flink.table.formats.SerializationSchemaFactory;
import org.apache.flink.table.formats.TableFormatFactoryService;
import org.apache.flink.table.factories.DeserializationSchemaFactory;
import org.apache.flink.table.factories.SerializationSchemaFactory;
import org.apache.flink.table.factories.TableFactoryService;
import org.apache.flink.types.Row;
import org.apache.flink.util.TestLogger;
......@@ -107,7 +107,7 @@ public class JsonRowFormatFactoryTest extends TestLogger {
}
private void testSchemaDeserializationSchema(Map<String, String> properties) {
final DeserializationSchema<?> actual2 = TableFormatFactoryService
final DeserializationSchema<?> actual2 = TableFactoryService
.find(DeserializationSchemaFactory.class, properties)
.createDeserializationSchema(properties);
final JsonRowDeserializationSchema expected2 = new JsonRowDeserializationSchema(SCHEMA);
......@@ -116,7 +116,7 @@ public class JsonRowFormatFactoryTest extends TestLogger {
}
private void testSchemaSerializationSchema(Map<String, String> properties) {
final SerializationSchema<?> actual1 = TableFormatFactoryService
final SerializationSchema<?> actual1 = TableFactoryService
.find(SerializationSchemaFactory.class, properties)
.createSerializationSchema(properties);
final SerializationSchema expected1 = new JsonRowSerializationSchema(SCHEMA);
......@@ -124,7 +124,7 @@ public class JsonRowFormatFactoryTest extends TestLogger {
}
private void testJsonSchemaDeserializationSchema(Map<String, String> properties) {
final DeserializationSchema<?> actual2 = TableFormatFactoryService
final DeserializationSchema<?> actual2 = TableFactoryService
.find(DeserializationSchemaFactory.class, properties)
.createDeserializationSchema(properties);
final JsonRowDeserializationSchema expected2 = new JsonRowDeserializationSchema(JSON_SCHEMA);
......@@ -133,7 +133,7 @@ public class JsonRowFormatFactoryTest extends TestLogger {
}
private void testJsonSchemaSerializationSchema(Map<String, String> properties) {
final SerializationSchema<?> actual1 = TableFormatFactoryService
final SerializationSchema<?> actual1 = TableFactoryService
.find(SerializationSchemaFactory.class, properties)
.createSerializationSchema(properties);
final SerializationSchema<?> expected1 = new JsonRowSerializationSchema(JSON_SCHEMA);
......
......@@ -159,7 +159,7 @@ under the License.
<include>org.codehaus.commons.compiler.properties</include>
<include>org/codehaus/janino/**</include>
<include>org/codehaus/commons/**</include>
<include>META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory</include>
<include>META-INF/services/org.apache.flink.table.factories.TableFactory</include>
<!-- flink-sql-client -->
<include>org/jline/**</include>
<include>com/fasterxml/jackson/**</include>
......
......@@ -48,7 +48,8 @@ public class Environment {
private Deployment deployment;
private static final String NAME = "name";
private static final String TABLE_NAME = "name";
private static final String TABLE_TYPE = "type";
public Environment() {
this.tables = Collections.emptyMap();
......@@ -64,16 +65,16 @@ public class Environment {
public void setTables(List<Map<String, Object>> tables) {
this.tables = new HashMap<>(tables.size());
tables.forEach(config -> {
if (!config.containsKey(NAME)) {
if (!config.containsKey(TABLE_NAME)) {
throw new SqlClientException("The 'name' attribute of a table is missing.");
}
final Object nameObject = config.get(NAME);
final Object nameObject = config.get(TABLE_NAME);
if (nameObject == null || !(nameObject instanceof String) || ((String) nameObject).length() <= 0) {
throw new SqlClientException("Invalid table name '" + nameObject + "'.");
}
final String name = (String) nameObject;
final Map<String, Object> properties = new HashMap<>(config);
properties.remove(NAME);
properties.remove(TABLE_NAME);
if (this.tables.containsKey(name)) {
throw new SqlClientException("Duplicate table name '" + name + "'.");
......@@ -209,11 +210,12 @@ public class Environment {
* @return table descriptor describing a source, sink, or both
*/
private static TableDescriptor createTableDescriptor(String name, Map<String, Object> config) {
final Object typeObject = config.get(TableDescriptorValidator.TABLE_TYPE());
final Object typeObject = config.get(TABLE_TYPE);
if (typeObject == null || !(typeObject instanceof String)) {
throw new SqlClientException("Invalid 'type' attribute for table '" + name + "'.");
}
final String type = (String) config.get(TableDescriptorValidator.TABLE_TYPE());
final String type = (String) config.get(TABLE_TYPE);
config.remove(TABLE_TYPE);
final Map<String, String> normalizedConfig = ConfigUtil.normalizeYaml(config);
if (type.equals(TableDescriptorValidator.TABLE_TYPE_VALUE_SOURCE())) {
return new Source(name, normalizedConfig);
......
......@@ -51,9 +51,9 @@ 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.connectors.TableFactoryService;
import org.apache.flink.table.connectors.TableSinkFactory;
import org.apache.flink.table.connectors.TableSourceFactory;
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.functions.AggregateFunction;
import org.apache.flink.table.functions.FunctionService;
......
......@@ -40,7 +40,7 @@ under the License.
<file>
<source>src/test/resources/test-factory-services-file</source>
<outputDirectory>META-INF/services</outputDirectory>
<destName>org.apache.flink.table.connectors.DiscoverableTableFactory</destName>
<destName>org.apache.flink.table.factories.TableFactory</destName>
<fileMode>0755</fileMode>
</file>
</files>
......
......@@ -24,8 +24,8 @@ 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.connectors.DiscoverableTableFactory;
import org.apache.flink.table.connectors.TableSourceFactory;
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.sources.DefinedProctimeAttribute;
......@@ -51,7 +51,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>, DiscoverableTableFactory {
public class TestTableSourceFactory implements TableSourceFactory<Row>, TableFactory {
@Override
public Map<String, String> requiredContext() {
......
......@@ -19,7 +19,7 @@
package org.apache.flink.table.api
import org.apache.flink.table.descriptors.DescriptorProperties
import org.apache.flink.table.formats.TableFormatFactory
import org.apache.flink.table.factories.TableFactory
/**
* Exception for all errors occurring during expression parsing.
......@@ -143,23 +143,22 @@ case class CatalogAlreadyExistException(
}
/**
* Exception for not finding a [[org.apache.flink.table.formats.TableFormatFactory]] for the
* given properties.
* Exception for not finding a [[TableFactory]] for the given properties.
*
* @param message message that indicates the current matching step
* @param factoryClass required factory class
* @param formatFactories all found factories
* @param properties properties that describe the table format
* @param factories all found factories
* @param properties properties that describe the configuration
* @param cause the cause
*/
case class NoMatchingTableFormatException(
case class NoMatchingTableFactoryException(
message: String,
factoryClass: Class[_],
formatFactories: Seq[TableFormatFactory[_]],
factories: Seq[TableFactory],
properties: Map[String, String],
cause: Throwable)
extends RuntimeException(
s"""Could not find a suitable table format factory for '${factoryClass.getName}' in
s"""Could not find a suitable table factory for '${factoryClass.getName}' in
|the classpath.
|
|Reason: $message
......@@ -167,94 +166,59 @@ case class NoMatchingTableFormatException(
|The following properties are requested:
|${DescriptorProperties.toString(properties)}
|
|The following format factories have been considered:
|${formatFactories.map(_.getClass.getName).mkString("\n")}
|The following factories have been considered:
|${factories.map(_.getClass.getName).mkString("\n")}
|""".stripMargin,
cause) {
def this(
message: String,
factoryClass: Class[_],
formatFactories: Seq[TableFormatFactory[_]],
factories: Seq[TableFactory],
properties: Map[String, String]) = {
this(message, factoryClass, formatFactories, properties, null)
this(message, factoryClass, factories, properties, null)
}
}
/**
* Exception for finding more than one [[org.apache.flink.table.formats.TableFormatFactory]] for
* the given properties.
* Exception for finding more than one [[TableFactory]] for the given properties.
*
* @param matchingFormatFactories format factories that match the properties
* @param matchingFactories factories that match the properties
* @param factoryClass required factory class
* @param formatFactories all found factories
* @param properties properties that describe the table format
* @param factories all found factories
* @param properties properties that describe the configuration
* @param cause the cause
*/
case class AmbiguousTableFormatException(
matchingFormatFactories: Seq[TableFormatFactory[_]],
case class AmbiguousTableFactoryException(
matchingFactories: Seq[TableFactory],
factoryClass: Class[_],
formatFactories: Seq[TableFormatFactory[_]],
factories: Seq[TableFactory],
properties: Map[String, String],
cause: Throwable)
extends RuntimeException(
s"""More than one suitable table format factory for '${factoryClass.getName}' could
s"""More than one suitable table factory for '${factoryClass.getName}' could
|be found in the classpath.
|
|The following format factories match:
|${matchingFormatFactories.map(_.getClass.getName).mkString("\n")}
|The following factories match:
|${matchingFactories.map(_.getClass.getName).mkString("\n")}
|
|The following properties are requested:
|${DescriptorProperties.toString(properties)}
|
|The following format factories have been considered:
|${formatFactories.map(_.getClass.getName).mkString("\n")}
|The following factories have been considered:
|${factories.map(_.getClass.getName).mkString("\n")}
|""".stripMargin,
cause) {
def this(
matchingFormatFactories: Seq[TableFormatFactory[_]],
matchingFactories: Seq[TableFactory],
factoryClass: Class[_],
formatFactories: Seq[TableFormatFactory[_]],
factories: Seq[TableFactory],
properties: Map[String, String]) = {
this(matchingFormatFactories, factoryClass, formatFactories, properties, null)
this(matchingFactories, factoryClass, factories, properties, null)
}
}
/**
* Exception for not finding a [[org.apache.flink.table.connectors.DiscoverableTableFactory]] for
* the given properties.
*
* @param properties properties that describe the table connector
* @param cause the cause
*/
case class NoMatchingTableFactoryException(properties: Map[String, String], cause: Throwable)
extends RuntimeException(
s"Could not find a table factory in the classpath satisfying the " +
s"following properties: \n" +
s"${DescriptorProperties.toString(properties)}",
cause) {
def this(properties: Map[String, String]) = this(properties, null)
}
/**
* Exception for finding more than one
* [[org.apache.flink.table.connectors.DiscoverableTableFactory]] for the given properties.
*
* @param properties properties that describe the table factory
* @param cause the cause
*/
case class AmbiguousTableFactoryException(properties: Map[String, String], cause: Throwable)
extends RuntimeException(
s"More than one table factory in the classpath satisfying the " +
s"following properties: \n" +
s"${DescriptorProperties.toString(properties)}",
cause) {
def this(properties: Map[String, String]) = this(properties, null)
}
/**
* Exception for operation on a nonexistent external catalog
*
......
......@@ -19,15 +19,13 @@
package org.apache.flink.table.catalog
import org.apache.flink.table.api._
import org.apache.flink.table.connectors.{TableFactoryService, TableSourceFactory}
import org.apache.flink.table.descriptors.DescriptorProperties
import org.apache.flink.table.factories.{TableFactoryService, TableSourceFactory}
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
import _root_.scala.collection.JavaConverters._
/**
* The utility class is used to convert ExternalCatalogTable to TableSourceTable.
*/
......@@ -45,9 +43,10 @@ object ExternalTableSourceUtil extends Logging {
: TableSourceSinkTable[_, _] = {
val properties = new DescriptorProperties()
externalCatalogTable.addProperties(properties)
val source = TableFactoryService.find(classOf[TableSourceFactory[_]], externalCatalogTable)
val javaMap = properties.asMap
val source = TableFactoryService.find(classOf[TableSourceFactory[_]], javaMap)
.asInstanceOf[TableSourceFactory[_]]
.createTableSource(properties.asMap)
.createTableSource(javaMap)
tableEnv match {
// check for a batch table source in this batch environment
case _: BatchTableEnvironment =>
......
/*
* 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.connectors
import java.util.{ServiceConfigurationError, ServiceLoader}
import org.apache.flink.table.api._
import org.apache.flink.table.descriptors.ConnectorDescriptorValidator._
import org.apache.flink.table.descriptors.FormatDescriptorValidator._
import org.apache.flink.table.descriptors.MetadataValidator._
import org.apache.flink.table.descriptors.StatisticsValidator._
import org.apache.flink.table.descriptors.{DescriptorProperties, TableDescriptor, TableDescriptorValidator}
import org.apache.flink.table.util.Logging
import _root_.scala.collection.JavaConverters._
import _root_.scala.collection.mutable
/**
* Unified interface to search for TableFactoryDiscoverable of provided type and properties.
*/
object TableFactoryService extends Logging {
private lazy val defaultLoader = ServiceLoader.load(classOf[DiscoverableTableFactory])
def find(clz: Class[_], descriptor: TableDescriptor): DiscoverableTableFactory = {
find(clz, descriptor, null)
}
def find(clz: Class[_], descriptor: TableDescriptor, classLoader: ClassLoader)
: DiscoverableTableFactory = {
val properties = new DescriptorProperties()
descriptor.addProperties(properties)
find(clz, properties.asMap.asScala.toMap, classLoader)
}
def find(clz: Class[_], properties: Map[String, String]): DiscoverableTableFactory = {
find(clz: Class[_], properties, null)
}
def find(clz: Class[_], properties: Map[String, String],
classLoader: ClassLoader): DiscoverableTableFactory = {
var matchingFactory: Option[(DiscoverableTableFactory, Seq[String])] = None
try {
val iter = if (classLoader == null) {
defaultLoader.iterator()
} else {
val customLoader = ServiceLoader.load(classOf[DiscoverableTableFactory], classLoader)
customLoader.iterator()
}
while (iter.hasNext) {
val factory = iter.next()
if (clz.isAssignableFrom(factory.getClass)) {
val requiredContextJava = try {
factory.requiredContext()
} catch {
case t: Throwable =>
throw new TableException(
s"Table source factory '${factory.getClass.getCanonicalName}' caused an exception.",
t)
}
val requiredContext = if (requiredContextJava != null) {
// normalize properties
requiredContextJava.asScala.map(e => (e._1.toLowerCase, e._2))
} else {
Map[String, String]()
}
val plainContext = mutable.Map[String, String]()
plainContext ++= requiredContext
// we remove the versions for now until we have the first backwards compatibility case
// with the version we can provide mappings in case the format changes
plainContext.remove(CONNECTOR_PROPERTY_VERSION)
plainContext.remove(FORMAT_PROPERTY_VERSION)
plainContext.remove(METADATA_PROPERTY_VERSION)
plainContext.remove(STATISTICS_PROPERTY_VERSION)
if (plainContext.forall(e => properties.contains(e._1) && properties(e._1) == e._2)) {
matchingFactory match {
case Some(_) => throw new AmbiguousTableFactoryException(properties)
case None => matchingFactory =
Some((factory.asInstanceOf[DiscoverableTableFactory], requiredContext.keys.toSeq))
}
}
}
}
} catch {
case e: ServiceConfigurationError =>
LOG.error("Could not load service provider for table factories.", e)
throw new TableException("Could not load service provider for table factories.", e)
}
val (factory, context) = matchingFactory
.getOrElse(throw new NoMatchingTableFactoryException(properties))
val plainProperties = mutable.ArrayBuffer[String]()
properties.keys.foreach { k =>
// replace arrays with wildcard
val key = k.replaceAll(".\\d+", ".#")
// ignore context properties and duplicates
if (!context.contains(key) && !plainProperties.contains(key)) {
plainProperties += key
}
}
val supportedPropertiesJava = try {
factory.supportedProperties()
} catch {
case t: Throwable =>
throw new TableException(
s"Table source factory '${factory.getClass.getCanonicalName}' caused an exception.",
t)
}
val supportedProperties = if (supportedPropertiesJava != null) {
supportedPropertiesJava.asScala.map(_.toLowerCase)
} else {
Seq[String]()
}
// check for supported properties
plainProperties.foreach { k =>
if (!k.equals(TableDescriptorValidator.TABLE_TYPE) && !supportedProperties.contains(k)) {
throw new ValidationException(
s"Table factory '${factory.getClass.getCanonicalName}' does not support the " +
s"property '$k'. Supported properties are: \n" +
s"${supportedProperties.map(DescriptorProperties.toString).mkString("\n")}")
}
}
// create the table connector
try {
factory
} catch {
case t: Throwable =>
throw new TableException(
s"Table connector factory '${factory.getClass.getCanonicalName}' caused an exception.",
t)
}
}
}
......@@ -19,7 +19,7 @@
package org.apache.flink.table.descriptors
import org.apache.flink.table.api.{BatchTableEnvironment, Table, TableException, ValidationException}
import org.apache.flink.table.connectors.{TableFactoryService, TableSourceFactory}
import org.apache.flink.table.factories.{TableFactoryService, TableSourceFactory}
import org.apache.flink.table.sources.{BatchTableSource, TableSource}
class BatchTableSourceDescriptor(tableEnv: BatchTableEnvironment, connector: ConnectorDescriptor)
......
......@@ -46,7 +46,7 @@ class LiteralValueValidator(keyPrefix: String) extends HierarchyDescriptorValida
val typeKey = s"$keyPrefix${LiteralValueValidator.TYPE}"
properties.validateType(typeKey, isOptional = true)
properties.validateType(typeKey, isOptional = true, requireRow = false)
// explicit type
if (properties.containsKey(typeKey)) {
......
......@@ -19,7 +19,7 @@
package org.apache.flink.table.descriptors
import org.apache.flink.table.api.{StreamTableEnvironment, Table, TableException, ValidationException}
import org.apache.flink.table.connectors.{TableFactoryService, TableSourceFactory}
import org.apache.flink.table.factories.{TableFactoryService, TableSourceFactory}
import org.apache.flink.table.sources.{StreamTableSource, TableSource}
/**
......
......@@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.flink.table.formats
package org.apache.flink.table.factories
import java.util
......
......@@ -16,17 +16,25 @@
* limitations under the License.
*/
package org.apache.flink.table.connectors
package org.apache.flink.table.factories
import java.util
/**
* Common trait for all properties-based discoverable table factories.
* A factory to create different table-related instances from string-based properties. This
* factory is used with Java's Service Provider Interfaces (SPI) for discovering. A factory is
* called with a set of normalized properties that describe the desired configuration. The factory
* allows for matching to the given set of properties.
*
* Classes that implement this interface can be added to the
* "META_INF/services/org.apache.flink.table.factories.TableFactory" file of a JAR file in
* the current classpath to be found.
*/
trait DiscoverableTableFactory {
trait TableFactory {
/**
* Specifies the context that this factory has been implemented for.
* Specifies the context that this factory has been implemented for. The framework guarantees to
* only match for this factory if the specified set of properties and values are met.
*
* Typical properties might be:
* - connector.type
......@@ -47,6 +55,9 @@ trait DiscoverableTableFactory {
* list must not contain the keys that are specified by the context.
*
* Example properties might be:
* - schema.#.type
* - schema.#.name
* - connector.topic
* - format.line-delimiter
* - format.ignore-parse-errors
* - format.fields.#.type
......
......@@ -16,12 +16,15 @@
* limitations under the License.
*/
package org.apache.flink.table.formats
package org.apache.flink.table.factories
import java.util.{ServiceConfigurationError, ServiceLoader, Map => JMap}
import org.apache.flink.table.api._
import org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_PROPERTY_VERSION
import org.apache.flink.table.descriptors.ConnectorDescriptorValidator._
import org.apache.flink.table.descriptors.FormatDescriptorValidator._
import org.apache.flink.table.descriptors.MetadataValidator._
import org.apache.flink.table.descriptors.StatisticsValidator._
import org.apache.flink.table.descriptors._
import org.apache.flink.table.util.Logging
import org.apache.flink.util.Preconditions
......@@ -30,35 +33,66 @@ import _root_.scala.collection.JavaConverters._
import _root_.scala.collection.mutable
/**
* Service provider interface for finding a suitable [[TableFormatFactory]] for the
* given properties.
* Unified interface to search for a [[TableFactory]] of provided type and properties.
*/
object TableFormatFactoryService extends Logging {
object TableFactoryService extends Logging {
private lazy val defaultLoader = ServiceLoader.load(classOf[TableFormatFactory[_]])
private lazy val defaultLoader = ServiceLoader.load(classOf[TableFactory])
/**
* Finds a table format factory of the given class and creates configured instances from the
* given property map.
* Finds a table factory of the given class and descriptor.
*
* @param factoryClass desired format factory
* @param propertyMap properties that describes the format
* @param factoryClass desired factory class
* @param descriptor descriptor describing the factory configuration
* @tparam T factory class type
* @return configured instance from factory
* @return the matching factory
*/
def find[T](factoryClass: Class[T], descriptor: Descriptor): T = {
Preconditions.checkNotNull(descriptor)
val descriptorProperties = new DescriptorProperties()
descriptor.addProperties(descriptorProperties)
findInternal(factoryClass, descriptorProperties.asMap, None)
}
/**
* Finds a table factory of the given class, descriptor, and classloader.
*
* @param factoryClass desired factory class
* @param descriptor descriptor describing the factory configuration
* @param classLoader classloader for service loading
* @tparam T factory class type
* @return the matching factory
*/
def find[T](factoryClass: Class[T], descriptor: Descriptor, classLoader: ClassLoader): T = {
Preconditions.checkNotNull(descriptor)
Preconditions.checkNotNull(classLoader)
val descriptorProperties = new DescriptorProperties()
descriptor.addProperties(descriptorProperties)
findInternal(factoryClass, descriptorProperties.asMap, Some(classLoader))
}
/**
* Finds a table factory of the given class and property map.
*
* @param factoryClass desired factory class
* @param propertyMap properties that describe the factory configuration
* @tparam T factory class type
* @return the matching factory
*/
def find[T](factoryClass: Class[T], propertyMap: JMap[String, String]): T = {
findInternal(factoryClass, propertyMap, None)
}
/**
* Finds a table format factory of the given class and creates configured instances from the
* given property map and classloader.
* Finds a table factory of the given class, property map, and classloader.
*
* @param factoryClass desired format factory
* @param propertyMap properties that describes the format
* @param factoryClass desired factory class
* @param propertyMap properties that describe the factory configuration
* @param classLoader classloader for service loading
* @tparam T factory class type
* @return configured instance from factory
* @return the matching factory
*/
def find[T](
factoryClass: Class[T],
......@@ -66,18 +100,18 @@ object TableFormatFactoryService extends Logging {
classLoader: ClassLoader)
: T = {
Preconditions.checkNotNull(classLoader)
findInternal(factoryClass, propertyMap, Some(classLoader))
}
/**
* Finds a table format factory of the given class and creates configured instances from the
* given property map and classloader.
* Finds a table factory of the given class, property map, and classloader.
*
* @param factoryClass desired format factory
* @param propertyMap properties that describes the format
* @param classLoader optional classloader for service loading
* @param factoryClass desired factory class
* @param propertyMap properties that describe the factory configuration
* @param classLoader classloader for service loading
* @tparam T factory class type
* @return configured instance from factory
* @return the matching factory
*/
private def findInternal[T](
factoryClass: Class[T],
......@@ -90,111 +124,127 @@ object TableFormatFactoryService extends Logging {
val properties = propertyMap.asScala.toMap
// find matching context
val (foundFactories, contextFactories) = findMatchingContext(
val foundFactories = discoverFactories(classLoader)
val classFactories = filterByFactoryClass(
factoryClass,
properties,
classLoader)
foundFactories)
// filter by factory class
val classFactories = filterByFactoryClass(
val contextFactories = filterByContext(
factoryClass,
properties,
foundFactories,
contextFactories)
classFactories)
// filter by supported keys
filterBySupportedProperties(
factoryClass,
properties,
foundFactories,
classFactories)
contextFactories)
}
private def findMatchingContext[T](
factoryClass: Class[T],
properties: Map[String, String],
classLoader: Option[ClassLoader])
: (Seq[TableFormatFactory[_]], Seq[TableFormatFactory[_]]) = {
val foundFactories = mutable.ArrayBuffer[TableFormatFactory[_]]()
val matchingFactories = mutable.ArrayBuffer[TableFormatFactory[_]]()
/**
* Searches for factories using Java service providers.
*
* @return all factories in the classpath
*/
private def discoverFactories[T](classLoader: Option[ClassLoader]): Seq[TableFactory] = {
try {
val iter = classLoader match {
val iterator = classLoader match {
case Some(customClassLoader) =>
val customLoader = ServiceLoader.load(classOf[TableFormatFactory[_]], customClassLoader)
val customLoader = ServiceLoader.load(classOf[TableFactory], customClassLoader)
customLoader.iterator()
case None =>
defaultLoader.iterator()
}
iterator.asScala.toSeq
} catch {
case e: ServiceConfigurationError =>
LOG.error("Could not load service provider for table factories.", e)
throw new TableException("Could not load service provider for table factories.", e)
}
}
/**
* Filters factories with matching context by factory class.
*/
private def filterByFactoryClass[T](
factoryClass: Class[T],
properties: Map[String, String],
foundFactories: Seq[TableFactory])
: Seq[TableFactory] = {
val classFactories = foundFactories.filter(f => factoryClass.isAssignableFrom(f.getClass))
if (classFactories.isEmpty) {
throw new NoMatchingTableFactoryException(
s"No factory implements '${factoryClass.getCanonicalName}'.",
factoryClass,
foundFactories,
properties)
}
classFactories
}
while (iter.hasNext) {
val factory = iter.next()
foundFactories += factory
/**
* Filters for factories with matching context.
*
* @return all matching factories
*/
private def filterByContext[T](
factoryClass: Class[T],
properties: Map[String, String],
foundFactories: Seq[TableFactory],
classFactories: Seq[TableFactory])
: Seq[TableFactory] = {
val requestedContext = normalizeContext(factory)
val matchingFactories = classFactories.filter { factory =>
val requestedContext = normalizeContext(factory)
val plainContext = mutable.Map[String, String]()
plainContext ++= requestedContext
// we remove the version for now until we have the first backwards compatibility case
// with the version we can provide mappings in case the format changes
plainContext.remove(FORMAT_PROPERTY_VERSION)
val plainContext = mutable.Map[String, String]()
plainContext ++= requestedContext
// we remove the version for now until we have the first backwards compatibility case
// with the version we can provide mappings in case the format changes
plainContext.remove(CONNECTOR_PROPERTY_VERSION)
plainContext.remove(FORMAT_PROPERTY_VERSION)
plainContext.remove(METADATA_PROPERTY_VERSION)
plainContext.remove(STATISTICS_PROPERTY_VERSION)
// check if required context is met
if (plainContext.forall(e => properties.contains(e._1) && properties(e._1) == e._2)) {
matchingFactories += factory
}
}
} catch {
case e: ServiceConfigurationError =>
LOG.error("Could not load service provider for table format factories.", e)
throw new TableException("Could not load service provider for table format factories.", e)
// check if required context is met
plainContext.forall(e => properties.contains(e._1) && properties(e._1) == e._2)
}
if (matchingFactories.isEmpty) {
throw new NoMatchingTableFormatException(
throw new NoMatchingTableFactoryException(
"No context matches.",
factoryClass,
foundFactories,
properties)
}
(foundFactories, matchingFactories)
matchingFactories
}
private def normalizeContext(factory: TableFormatFactory[_]): Map[String, String] = {
/**
* Prepares the properties of a context to be used for match operations.
*/
private def normalizeContext(factory: TableFactory): Map[String, String] = {
val requiredContextJava = factory.requiredContext()
if (requiredContextJava == null) {
throw new TableException(
s"Required context of format factory '${factory.getClass.getName}' must not be null.")
s"Required context of factory '${factory.getClass.getName}' must not be null.")
}
requiredContextJava.asScala.map(e => (e._1.toLowerCase, e._2)).toMap
}
private def filterByFactoryClass[T](
factoryClass: Class[T],
properties: Map[String, String],
foundFactories: Seq[TableFormatFactory[_]],
contextFactories: Seq[TableFormatFactory[_]])
: Seq[TableFormatFactory[_]] = {
val classFactories = contextFactories.filter(f => factoryClass.isAssignableFrom(f.getClass))
if (classFactories.isEmpty) {
throw new NoMatchingTableFormatException(
s"No factory implements '${factoryClass.getCanonicalName}'.",
factoryClass,
foundFactories,
properties)
}
classFactories
}
/**
* Filters the matching class factories by supported properties.
*/
private def filterBySupportedProperties[T](
factoryClass: Class[T],
properties: Map[String, String],
foundFactories: Seq[TableFormatFactory[_]],
classFactories: Seq[TableFormatFactory[_]])
foundFactories: Seq[TableFactory],
classFactories: Seq[TableFactory])
: T = {
val plainGivenKeys = mutable.ArrayBuffer[String]()
......@@ -209,23 +259,17 @@ object TableFormatFactoryService extends Logging {
var lastKey: Option[String] = None
val supportedFactories = classFactories.filter { factory =>
val requiredContextKeys = normalizeContext(factory).keySet
val includeSchema = factory.supportsSchemaDerivation()
val supportedKeys = normalizeSupportedProperties(factory)
val givenKeys = plainGivenKeys
// ignore context keys
.filter(!requiredContextKeys.contains(_))
// ignore non-format (or schema) keys
.filter { k =>
if (includeSchema) {
k.startsWith(SchemaValidator.SCHEMA + ".") ||
k.startsWith(FormatDescriptorValidator.FORMAT + ".")
} else {
k.startsWith(FormatDescriptorValidator.FORMAT + ".")
}
}
givenKeys.forall { k =>
val (supportedKeys, wildcards) = normalizeSupportedProperties(factory)
// ignore context keys
val givenContextFreeKeys = plainGivenKeys.filter(!requiredContextKeys.contains(_))
// perform factory specific filtering of keys
val givenFilteredKeys = filterSupportedPropertiesFactorySpecific(
factory,
givenContextFreeKeys)
givenFilteredKeys.forall { k =>
lastKey = Option(k)
supportedKeys.contains(k)
supportedKeys.contains(k) || wildcards.exists(k.startsWith)
}
}
......@@ -233,8 +277,8 @@ object TableFormatFactoryService extends Logging {
// special case: when there is only one matching factory but the last property key
// was incorrect
val factory = classFactories.head
val supportedKeys = normalizeSupportedProperties(factory)
throw new NoMatchingTableFormatException(
val (supportedKeys, _) = normalizeSupportedProperties(factory)
throw new NoMatchingTableFactoryException(
s"""
|The matching factory '${factory.getClass.getName}' doesn't support '${lastKey.get}'.
|
......@@ -244,13 +288,13 @@ object TableFormatFactoryService extends Logging {
foundFactories,
properties)
} else if (supportedFactories.isEmpty) {
throw new NoMatchingTableFormatException(
throw new NoMatchingTableFactoryException(
s"No factory supports all properties.",
factoryClass,
foundFactories,
properties)
} else if (supportedFactories.length > 1) {
throw new AmbiguousTableFormatException(
throw new AmbiguousTableFactoryException(
supportedFactories,
factoryClass,
foundFactories,
......@@ -260,12 +304,53 @@ object TableFormatFactoryService extends Logging {
supportedFactories.head.asInstanceOf[T]
}
private def normalizeSupportedProperties(factory: TableFormatFactory[_]): Seq[String] = {
/**
* Prepares the supported properties of a factory to be used for match operations.
*/
private def normalizeSupportedProperties(factory: TableFactory): (Seq[String], Seq[String]) = {
val supportedPropertiesJava = factory.supportedProperties()
if (supportedPropertiesJava == null) {
throw new TableException(
s"Supported properties of format factory '${factory.getClass.getName}' must not be null.")
s"Supported properties of factory '${factory.getClass.getName}' must not be null.")
}
supportedPropertiesJava.asScala.map(_.toLowerCase)
val supportedKeys = supportedPropertiesJava.asScala.map(_.toLowerCase)
// extract wildcard prefixes
val wildcards = extractWildcardPrefixes(supportedKeys)
(supportedKeys, wildcards)
}
/**
* Converts the prefix of properties with wildcards (e.g., "format.*").
*/
private def extractWildcardPrefixes(propertyKeys: Seq[String]): Seq[String] = {
propertyKeys
.filter(_.endsWith("*"))
.map(s => s.substring(0, s.length - 1))
}
/**
* Performs filtering for special cases (i.e. table format factories with schema derivation).
*/
private def filterSupportedPropertiesFactorySpecific(
factory: TableFactory,
keys: Seq[String])
: Seq[String] = factory match {
case formatFactory: TableFormatFactory[_] =>
val includeSchema = formatFactory.supportsSchemaDerivation()
// ignore non-format (or schema) keys
keys.filter { k =>
if (includeSchema) {
k.startsWith(SchemaValidator.SCHEMA + ".") ||
k.startsWith(FormatDescriptorValidator.FORMAT + ".")
} else {
k.startsWith(FormatDescriptorValidator.FORMAT + ".")
}
}
case _ =>
keys
}
}
......@@ -16,42 +16,17 @@
* limitations under the License.
*/
package org.apache.flink.table.formats
package org.apache.flink.table.factories
import java.util
import org.apache.flink.api.common.serialization.{DeserializationSchema, SerializationSchema}
/**
* A factory to create different table format instances. This factory is used with Java's Service
* Provider Interfaces (SPI) for discovering. A factory is called with a set of normalized
* properties that describe the desired format. The factory allows for matching to the given set of
* properties. See also [[SerializationSchemaFactory]] and [[DeserializationSchemaFactory]] for
* creating configured instances of format classes accordingly.
*
* Classes that implement this interface need to be added to the
* "META_INF/services/org.apache.flink.table.formats.TableFormatFactory' file of a JAR file in
* the current classpath to be found.
* A factory to create configured table format instances based on string-based properties. See
* also [[TableFactory]] for more information.
*
* @tparam T record type that the format produces or consumes
*/
trait TableFormatFactory[T] {
/**
* Specifies the context that this factory has been implemented for. The framework guarantees
* to only use the factory if the specified set of properties and values are met.
*
* Typical properties might be:
* - format.type
* - format.version
*
* Specified property versions allow the framework to provide backwards compatible properties
* in case of string format changes:
* - format.property-version
*
* An empty context means that the factory matches for all requests.
*/
def requiredContext(): util.Map[String, String]
trait TableFormatFactory[T] extends TableFactory {
/**
* Flag to indicate if the given format supports deriving information from a schema. If the
......@@ -75,10 +50,13 @@ trait TableFormatFactory[T] {
* - schema.#.name
* - schema.#.type
*
* Note: Supported format properties must be prefixed with "format.". If schema derivation is
* enabled, also properties with "schema." prefix can be used. Use "#" to denote an array of
* values where "#" represents one or more digits. Property versions like
* "format.property-version" must not be part of the supported properties.
* Note: All supported format properties must be prefixed with "format.". If schema derivation is
* enabled, also properties with "schema." prefix can be used.
*
* Use "#" to denote an array of values where "#" represents one or more digits. Property
* versions like "format.property-version" must not be part of the supported properties.
*
* @see See also [[TableFactory.supportedProperties()]] for more information.
*/
def supportedProperties(): util.List[String]
......
......@@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.flink.table.connectors
package org.apache.flink.table.factories
import org.apache.flink.table.sinks.TableSink
......
......@@ -16,13 +16,14 @@
* limitations under the License.
*/
package org.apache.flink.table.connectors
package org.apache.flink.table.factories
import org.apache.flink.table.sources.TableSource
import java.util
trait TableSourceFactory[T] {
trait TableSourceFactory[T] extends TableFactory {
/**
* Creates and configures a [[org.apache.flink.table.sources.TableSource]]
* using the given properties.
......
......@@ -21,7 +21,7 @@ package org.apache.flink.table.sinks
import java.util
import org.apache.flink.table.api.TableException
import org.apache.flink.table.connectors.{DiscoverableTableFactory, TableSinkFactory}
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._
......@@ -34,7 +34,7 @@ import org.apache.flink.types.Row
/**
* Factory for creating configured instances of [[CsvTableSink]].
*/
class CsvTableSinkFactory extends TableSinkFactory[Row] with DiscoverableTableFactory {
class CsvTableSinkFactory extends TableSinkFactory[Row] with TableFactory {
override def requiredContext(): util.Map[String, String] = {
val context = new util.HashMap[String, String]()
......
......@@ -21,7 +21,7 @@ package org.apache.flink.table.sources
import java.util
import org.apache.flink.table.api.TableException
import org.apache.flink.table.connectors.{DiscoverableTableFactory, TableSourceFactory}
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
......@@ -34,7 +34,7 @@ import org.apache.flink.types.Row
/**
* Factory for creating configured instances of [[CsvTableSource]].
*/
class CsvTableSourceFactory extends TableSourceFactory[Row] with DiscoverableTableFactory {
class CsvTableSourceFactory extends TableSourceFactory[Row] with TableFactory {
override def requiredContext(): util.Map[String, String] = {
val context = new util.HashMap[String, String]()
......
......@@ -13,7 +13,9 @@
# See the License for the specific language governing permissions and
# limitations under the License.
org.apache.flink.table.connectors.TestFixedFormatTableFactory
org.apache.flink.table.connectors.TestWildcardFormatTableSourceFactory
org.apache.flink.table.connectors.TestTableSinkFactory
org.apache.flink.table.connectors.TestTableSourceFactory
org.apache.flink.table.factories.utils.TestFixedFormatTableFactory
org.apache.flink.table.factories.utils.TestWildcardFormatTableSourceFactory
org.apache.flink.table.factories.utils.TestTableSinkFactory
org.apache.flink.table.factories.utils.TestTableSourceFactory
org.apache.flink.table.factories.utils.TestTableFormatFactory
org.apache.flink.table.factories.utils.TestAmbiguousTableFormatFactory
# 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.
org.apache.flink.table.formats.utils.TestTableFormatFactory
org.apache.flink.table.formats.utils.TestAmbiguousTableFormatFactory
......@@ -16,20 +16,22 @@
* limitations under the License.
*/
package org.apache.flink.table.formats
package org.apache.flink.table.factories
import java.util.{HashMap => JHashMap, Map => JMap}
import org.apache.flink.table.api.{AmbiguousTableFormatException, NoMatchingTableFormatException}
import org.apache.flink.table.descriptors.FormatDescriptorValidator
import org.apache.flink.table.formats.TableFormatFactoryServiceTest.{COMMON_PATH, SPECIAL_PATH, TEST_FORMAT_TYPE, UNIQUE_PROPERTY}
import org.apache.flink.table.formats.utils.{TestAmbiguousTableFormatFactory, TestTableFormatFactory}
import org.junit.Assert.{assertNotNull, assertTrue}
import org.apache.flink.table.api.{AmbiguousTableFactoryException, NoMatchingTableFactoryException}
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.TableFormatFactoryServiceTest._
import org.apache.flink.table.factories.utils.{TestAmbiguousTableFormatFactory, TestTableFormatFactory}
import org.junit.Assert.assertTrue
import org.junit.Test
/**
* Tests for [[TableFormatFactoryService]]. The tests assume the two format factories
* [[TestTableFormatFactory]] and [[TestAmbiguousTableFormatFactory]] are registered.
* Tests for testing format discovery using [[TableFactoryService]]. The tests assume the two
* format factories [[TestTableFormatFactory]] and [[TestAmbiguousTableFormatFactory]] are
* registered.
*
* The first format does not support SPECIAL_PATH but supports schema derivation. The
* latter format does not support UNIQUE_PROPERTY nor schema derivation. Both formats
......@@ -40,17 +42,17 @@ class TableFormatFactoryServiceTest {
@Test
def testValidProperties(): Unit = {
val props = properties()
assertNotNull(
TableFormatFactoryService.find(classOf[TableFormatFactory[_]], props))
assertTrue(TableFactoryService.find(classOf[TableFormatFactory[_]], props)
.isInstanceOf[TestTableFormatFactory])
}
@Test
def testDifferentContextVersion(): Unit = {
val props = properties()
props.put(FormatDescriptorValidator.FORMAT_PROPERTY_VERSION, "2")
props.put(FORMAT_PROPERTY_VERSION, "2")
// for now we support any property version, the property version should not affect the
// discovery at the moment and thus the format should still be found
val foundFactory = TableFormatFactoryService.find(classOf[TableFormatFactory[_]], props)
val foundFactory = TableFactoryService.find(classOf[TableFormatFactory[_]], props)
assertTrue(foundFactory.isInstanceOf[TestTableFormatFactory])
}
......@@ -60,7 +62,7 @@ class TableFormatFactoryServiceTest {
props.remove(UNIQUE_PROPERTY) // both formats match now
props.put(SPECIAL_PATH, "/what/ever") // now only TestAmbiguousTableFormatFactory
assertTrue(
TableFormatFactoryService
TableFactoryService
.find(classOf[TableFormatFactory[_]], props)
.isInstanceOf[TestAmbiguousTableFormatFactory])
}
......@@ -70,7 +72,7 @@ class TableFormatFactoryServiceTest {
val props = properties()
props.remove(UNIQUE_PROPERTY) // both formats match now
assertTrue(
TableFormatFactoryService
TableFactoryService
// we are looking for a particular class
.find(classOf[TestAmbiguousTableFormatFactory], props)
.isInstanceOf[TestAmbiguousTableFormatFactory])
......@@ -86,47 +88,47 @@ class TableFormatFactoryServiceTest {
// the format with schema derivation feels not responsible because of this field,
// but since there is another format that feels responsible, no exception is thrown.
assertTrue(
TableFormatFactoryService
TableFactoryService
.find(classOf[TableFormatFactory[_]], props)
.isInstanceOf[TestAmbiguousTableFormatFactory])
}
@Test(expected = classOf[NoMatchingTableFormatException])
@Test(expected = classOf[NoMatchingTableFactoryException])
def testMissingClass(): Unit = {
val props = properties()
// this class is not a valid factory
TableFormatFactoryService.find(classOf[TableFormatFactoryServiceTest], props)
TableFactoryService.find(classOf[TableFormatFactoryServiceTest], props)
}
@Test(expected = classOf[NoMatchingTableFormatException])
@Test(expected = classOf[NoMatchingTableFactoryException])
def testInvalidContext(): Unit = {
val props = properties()
// no context specifies this
props.put(FormatDescriptorValidator.FORMAT_TYPE, "unknown_format_type")
TableFormatFactoryService.find(classOf[TableFormatFactory[_]], props)
props.put(FORMAT_TYPE, "unknown_format_type")
TableFactoryService.find(classOf[TableFormatFactory[_]], props)
}
@Test(expected = classOf[NoMatchingTableFormatException])
@Test(expected = classOf[NoMatchingTableFactoryException])
def testUnsupportedProperty(): Unit = {
val props = properties()
props.put("format.property_not_defined_by_any_factory", "/new/path")
TableFormatFactoryService.find(classOf[TableFormatFactory[_]], props)
TableFactoryService.find(classOf[TableFormatFactory[_]], props)
}
@Test(expected = classOf[AmbiguousTableFormatException])
@Test(expected = classOf[AmbiguousTableFactoryException])
def testAmbiguousFactory(): Unit = {
val props = properties()
props.remove(UNIQUE_PROPERTY) // now both factories match
TableFormatFactoryService.find(classOf[TableFormatFactory[_]], props)
TableFactoryService.find(classOf[TableFormatFactory[_]], props)
}
private def properties(): JMap[String, String] = {
val properties = new JHashMap[String, String]()
properties.put("connector.type", "test")
properties.put("format.type", TEST_FORMAT_TYPE)
properties.put(CONNECTOR_TYPE, "test")
properties.put(FORMAT_TYPE, TEST_FORMAT_TYPE)
properties.put(UNIQUE_PROPERTY, "true")
properties.put("connector.property-version", "1")
properties.put("format.property-version", "1")
properties.put(CONNECTOR_PROPERTY_VERSION, "1")
properties.put(FORMAT_PROPERTY_VERSION, "1")
properties.put(COMMON_PATH, "/path/to/target")
properties.put("schema.0.name", "a")
properties.put("schema.1.name", "b")
......
......@@ -16,30 +16,36 @@
* limitations under the License.
*/
package org.apache.flink.table.connectors
package org.apache.flink.table.factories
import org.apache.flink.table.api.{NoMatchingTableFactoryException, ValidationException}
import java.util.{HashMap => JHashMap, Map => JMap}
import org.apache.flink.table.api.NoMatchingTableFactoryException
import org.apache.flink.table.descriptors.ConnectorDescriptorValidator._
import org.apache.flink.table.descriptors.FormatDescriptorValidator._
import org.apache.flink.table.descriptors.TableDescriptorValidator
import org.apache.flink.table.factories.utils.TestTableSinkFactory
import org.apache.flink.table.factories.utils.TestTableSinkFactory._
import org.junit.Assert._
import org.junit.Test
import scala.collection.JavaConverters._
import scala.collection.mutable
/**
* Tests for testing table sink discovery using [[TableFactoryService]]. The tests assume the
* table sink factory [[TestTableSinkFactory]] is registered.
*/
class TableSinkFactoryServiceTest {
@Test
def testValidProperties(): Unit = {
val props = properties()
assertTrue(TableFactoryService.find(classOf[TableSinkFactory[_]], props.toMap) != null)
assertTrue(TableFactoryService.find(classOf[TableSinkFactory[_]], props)
.isInstanceOf[TestTableSinkFactory])
}
@Test(expected = classOf[NoMatchingTableFactoryException])
def testInvalidContext(): Unit = {
val props = properties()
props.put(CONNECTOR_TYPE, "FAIL")
TableFactoryService.find(classOf[TableSinkFactory[_]], props.toMap)
props.put(CONNECTOR_TYPE, "unknown-connector-type")
TableFactoryService.find(classOf[TableSinkFactory[_]], props)
}
@Test
......@@ -47,29 +53,21 @@ class TableSinkFactoryServiceTest {
val props = properties()
props.put(CONNECTOR_PROPERTY_VERSION, "2")
// the table source should still be found
assertTrue(TableFactoryService.find(classOf[TableSinkFactory[_]], props.toMap) != null)
assertTrue(TableFactoryService.find(classOf[TableSinkFactory[_]], props)
.isInstanceOf[TestTableSinkFactory])
}
@Test(expected = classOf[ValidationException])
@Test(expected = classOf[NoMatchingTableFactoryException])
def testUnsupportedProperty(): Unit = {
val props = properties()
props.put("format.path_new", "/new/path")
TableFactoryService.find(classOf[TableSinkFactory[_]], props.toMap)
}
@Test(expected = classOf[IllegalArgumentException])
def testFailingFactory(): Unit = {
val props = properties()
props.put("failing", "true")
TableFactoryService.find(classOf[TableSinkFactory[_]], props.toMap)
.asInstanceOf[TableSinkFactory[_]].createTableSink(props.asJava)
TableFactoryService.find(classOf[TableSinkFactory[_]], props)
}
private def properties(): mutable.Map[String, String] = {
val properties = mutable.Map[String, String]()
properties.put(TableDescriptorValidator.TABLE_TYPE, "sink")
properties.put(CONNECTOR_TYPE, "test")
properties.put(FORMAT_TYPE, "test")
private def properties(): JMap[String, String] = {
val properties = new JHashMap[String, String]()
properties.put(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_TEST)
properties.put(FORMAT_TYPE, FORMAT_TYPE_VALUE_TEST)
properties.put(CONNECTOR_PROPERTY_VERSION, "1")
properties.put(FORMAT_PROPERTY_VERSION, "1")
properties.put(FORMAT_PATH, "/path/to/target")
......
......@@ -16,85 +16,77 @@
* limitations under the License.
*/
package org.apache.flink.table.connectors
package org.apache.flink.table.factories
import org.apache.flink.table.api.{NoMatchingTableFactoryException, ValidationException}
import java.util.{HashMap => JHashMap, Map => JMap}
import org.apache.flink.table.api.NoMatchingTableFactoryException
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.descriptors.TableDescriptorValidator
import org.apache.flink.table.sources.TestWildcardFormatTableSourceFactory
import org.apache.flink.table.factories.utils.TestFixedFormatTableFactory.{CONNECTOR_TYPE_VALUE_FIXED, FORMAT_TYPE_VALUE_TEST}
import org.apache.flink.table.factories.utils.TestWildcardFormatTableSourceFactory.CONNECTOR_TYPE_VALUE_WILDCARD
import org.apache.flink.table.factories.utils.{TestFixedFormatTableFactory, TestWildcardFormatTableSourceFactory}
import org.junit.Assert.assertTrue
import org.junit.Test
import scala.collection.JavaConverters._
import scala.collection.mutable
/**
* Tests for testing table source discovery using [[TableFactoryService]]. The tests assume the
* two table source factories [[TestFixedFormatTableFactory]] and
* [[TestWildcardFormatTableSourceFactory]] are registered.
*
* The first table source has a [[FORMAT_TYPE_VALUE_TEST]] type where as the second source uses
* a wildcard to match arbitrary formats.
*/
class TableSourceFactoryServiceTest {
@Test
def testValidProperties(): Unit = {
val props = properties()
props.put(CONNECTOR_TYPE, "fixed")
props.put(FORMAT_TYPE, "test")
assertTrue(TableFactoryService.find(classOf[TableSourceFactory[_]], props.toMap) != null)
props.put(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_FIXED)
props.put(FORMAT_TYPE, FORMAT_TYPE_VALUE_TEST)
assertTrue(TableFactoryService.find(classOf[TableSourceFactory[_]], props)
.isInstanceOf[TestFixedFormatTableFactory])
}
@Test(expected = classOf[NoMatchingTableFactoryException])
def testInvalidContext(): Unit = {
val props = properties()
props.put(CONNECTOR_TYPE, "FAIL")
props.put(FORMAT_TYPE, "test")
TableFactoryService.find(classOf[TableSourceFactory[_]], props.toMap)
props.put(CONNECTOR_TYPE, "unknown-connector-type")
props.put(FORMAT_TYPE, FORMAT_TYPE_VALUE_TEST)
TableFactoryService.find(classOf[TableSourceFactory[_]], props)
}
@Test
def testDifferentContextVersion(): Unit = {
val props = properties()
props.put(CONNECTOR_TYPE, "fixed")
props.put(FORMAT_TYPE, "test")
props.put(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_FIXED)
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.toMap) != null)
assertTrue(TableFactoryService.find(classOf[TableSourceFactory[_]], props)
.isInstanceOf[TestFixedFormatTableFactory])
}
@Test(expected = classOf[ValidationException])
@Test(expected = classOf[NoMatchingTableFactoryException])
def testUnsupportedProperty(): Unit = {
val props = properties()
props.put(CONNECTOR_TYPE, "fixed")
props.put(FORMAT_TYPE, "test")
props.put("format.path_new", "/new/path")
TableFactoryService.find(classOf[TableSourceFactory[_]], props.toMap)
}
@Test(expected = classOf[IllegalArgumentException])
def testFailingFactory(): Unit = {
val props = properties()
props.put(CONNECTOR_TYPE, "fixed")
props.put(FORMAT_TYPE, "test")
props.put("failing", "true")
TableFactoryService.find(classOf[TableSourceFactory[_]], props.toMap)
.asInstanceOf[TableSourceFactory[_]]
.createTableSource(props.asJava)
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)
}
@Test
def testWildcardFormat(): Unit = {
val props = properties()
props.put(CONNECTOR_TYPE, "wildcard")
props.put(FORMAT_TYPE, "test")
props.put("format.type", "not-test")
props.put("format.not-test-property", "wildcard-property")
val actualTableSource = TableFactoryService.find(classOf[TableSourceFactory[_]], props.toMap)
props.put(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_WILDCARD)
props.put("format.unknown-format-type-property", "wildcard-property")
val actualTableSource = TableFactoryService.find(classOf[TableSourceFactory[_]], props)
assertTrue(actualTableSource.isInstanceOf[TestWildcardFormatTableSourceFactory])
}
private def properties(): mutable.Map[String, String] = {
val properties = mutable.Map[String, String]()
properties.put(
TableDescriptorValidator.TABLE_TYPE,
TableDescriptorValidator.TABLE_TYPE_VALUE_SOURCE)
properties.put(CONNECTOR_TYPE, "test")
properties.put(FORMAT_TYPE, "test")
private def properties(): JMap[String, String] = {
val properties = new JHashMap[String, String]()
properties.put(CONNECTOR_PROPERTY_VERSION, "1")
properties.put(FORMAT_PROPERTY_VERSION, "1")
properties.put("format.path", "/path/to/target")
......@@ -104,7 +96,6 @@ class TableSourceFactoryServiceTest {
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
}
}
......@@ -16,12 +16,12 @@
* limitations under the License.
*/
package org.apache.flink.table.formats.utils
package org.apache.flink.table.factories.utils
import java.util
import org.apache.flink.table.descriptors.FormatDescriptorValidator
import org.apache.flink.table.formats.{TableFormatFactory, TableFormatFactoryServiceTest}
import org.apache.flink.table.factories.{TableFormatFactory, TableFormatFactoryServiceTest}
import org.apache.flink.types.Row
/**
......
......@@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.flink.table.formats.utils
package org.apache.flink.table.factories.utils
import org.apache.flink.api.common.serialization.DeserializationSchema
import org.apache.flink.api.common.typeinfo.TypeInformation
......
......@@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.flink.table.connectors
package org.apache.flink.table.factories.utils
import java.util
......@@ -24,18 +24,20 @@ 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.utils.TestFixedFormatTableFactory._
import org.apache.flink.table.sources.TableSource
import org.apache.flink.types.Row
/**
* Table source factory for testing with a fixed format.
*/
class TestFixedFormatTableFactory extends TableSourceFactory[Row] with DiscoverableTableFactory {
class TestFixedFormatTableFactory extends TableSourceFactory[Row] {
override def requiredContext(): util.Map[String, String] = {
val context = new util.HashMap[String, String]()
context.put(CONNECTOR_TYPE, "fixed")
context.put(FORMAT_TYPE, "test")
context.put(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_FIXED)
context.put(FORMAT_TYPE, FORMAT_TYPE_VALUE_TEST)
context.put(CONNECTOR_PROPERTY_VERSION, "1")
context.put(FORMAT_PROPERTY_VERSION, "1")
context
......@@ -46,14 +48,10 @@ class TestFixedFormatTableFactory extends TableSourceFactory[Row] with Discovera
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()
......@@ -61,3 +59,8 @@ class TestFixedFormatTableFactory extends TableSourceFactory[Row] with Discovera
}
}
}
object TestFixedFormatTableFactory {
val CONNECTOR_TYPE_VALUE_FIXED = "fixed"
val FORMAT_TYPE_VALUE_TEST = "test"
}
......@@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.flink.table.formats.utils
package org.apache.flink.table.factories.utils
import org.apache.flink.api.common.serialization.SerializationSchema
import org.apache.flink.types.Row
......
......@@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.flink.table.formats.utils
package org.apache.flink.table.factories.utils
import org.apache.flink.table.descriptors.{DescriptorProperties, FormatDescriptor}
......
......@@ -16,13 +16,13 @@
* limitations under the License.
*/
package org.apache.flink.table.formats.utils
package org.apache.flink.table.factories.utils
import java.util
import org.apache.flink.api.common.serialization.DeserializationSchema
import org.apache.flink.table.descriptors.{DescriptorProperties, FormatDescriptorValidator, SchemaValidator}
import org.apache.flink.table.formats.{DeserializationSchemaFactory, TableFormatFactoryServiceTest}
import org.apache.flink.table.factories.{DeserializationSchemaFactory, TableFormatFactoryServiceTest}
import org.apache.flink.types.Row
/**
......
......@@ -16,23 +16,27 @@
* limitations under the License.
*/
package org.apache.flink.table.connectors
package org.apache.flink.table.factories.utils
import java.util
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.table.connectors.TestTableSinkFactory.FORMAT_PATH
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.types.Row
class TestTableSinkFactory extends TableSinkFactory[Row] with DiscoverableTableFactory {
/**
* Test table sink factory.
*/
class TestTableSinkFactory extends TableSinkFactory[Row] with TableFactory {
override def requiredContext(): util.Map[String, String] = {
val context = new util.HashMap[String, String]()
context.put(CONNECTOR_TYPE, "test")
context.put(FORMAT_TYPE, "test")
context.put(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_TEST)
context.put(FORMAT_TYPE, FORMAT_TYPE_VALUE_TEST)
context.put(CONNECTOR_PROPERTY_VERSION, "1")
context.put(FORMAT_PROPERTY_VERSION, "1")
context
......
......@@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.flink.table.connectors
package org.apache.flink.table.factories.utils
import java.util
......@@ -24,18 +24,20 @@ 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.types.Row
/**
* Table source factory for testing.
*/
class TestTableSourceFactory extends TableSourceFactory[Row] with DiscoverableTableFactory {
class TestTableSourceFactory extends TableSourceFactory[Row] with TableFactory {
override def requiredContext(): util.Map[String, String] = {
val context = new util.HashMap[String, String]()
context.put(CONNECTOR_TYPE, "test")
context.put(FORMAT_TYPE, "test")
context.put(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_TEST)
context.put(FORMAT_TYPE, FORMAT_TYPE_VALUE_TEST)
context.put(CONNECTOR_PROPERTY_VERSION, "1")
context.put(FORMAT_PROPERTY_VERSION, "1")
context
......@@ -62,3 +64,8 @@ class TestTableSourceFactory extends TableSourceFactory[Row] with DiscoverableTa
}
}
}
object TestTableSourceFactory {
val CONNECTOR_TYPE_VALUE_TEST = "test"
val FORMAT_TYPE_VALUE_TEST = "test"
}
......@@ -16,11 +16,13 @@
* limitations under the License.
*/
package org.apache.flink.table.connectors
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.utils.TestWildcardFormatTableSourceFactory.CONNECTOR_TYPE_VALUE_WILDCARD
import org.apache.flink.table.sources.TableSource
import org.apache.flink.types.Row
......@@ -28,12 +30,11 @@ import org.apache.flink.types.Row
* Table source factory for testing with a wildcard format ("format.*").
*/
class TestWildcardFormatTableSourceFactory
extends TableSourceFactory[Row]
with DiscoverableTableFactory {
extends TableSourceFactory[Row] {
override def requiredContext(): util.Map[String, String] = {
val context = new util.HashMap[String, String]()
context.put(CONNECTOR_TYPE, "wildcard")
context.put(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_WILDCARD)
context.put(CONNECTOR_PROPERTY_VERSION, "1")
context
}
......@@ -43,7 +44,6 @@ class TestWildcardFormatTableSourceFactory
properties.add("format.*")
properties.add("schema.#.name")
properties.add("schema.#.field.#.name")
properties.add("failing")
properties
}
......@@ -51,3 +51,7 @@ class TestWildcardFormatTableSourceFactory
throw new UnsupportedOperationException()
}
}
object TestWildcardFormatTableSourceFactory {
val CONNECTOR_TYPE_VALUE_WILDCARD = "wildcard"
}
......@@ -21,11 +21,11 @@ package org.apache.flink.table.utils
import java.util
import org.apache.flink.api.java.typeutils.RowTypeInfo
import org.apache.flink.table.connectors.{DiscoverableTableFactory, TableSinkFactory, TableSourceFactory}
import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.{CONNECTOR_PROPERTY_VERSION, CONNECTOR_TYPE}
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.types.Row
......@@ -39,7 +39,7 @@ 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 DiscoverableTableFactory {
with TableSinkFactory[Row] with TableFactory {
override def createTableSink(properties: util.Map[String, String]): TableSink[Row] = {
val params: DescriptorProperties = new DescriptorProperties(true)
params.putProperties(properties)
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册