From 980499f887d72ddf9a405c4ad200d0cab15d889c Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 27 Jun 2018 13:16:49 +0200 Subject: [PATCH 01/17] [FLINK-8558] [table] Add unified format interfaces and separate formats from connectors This PR introduces a format discovery mechanism based on Java Service Providers. The general `TableFormatFactory` is similar to the existing table source discovery mechanism. However, it allows for arbirary format interfaces that might be introduced in the future. At the moment, a connector can request configured instances of `DeserializationSchema` and `SerializationSchema`. In the future we can add interfaces such as a `Writer` or `KeyedSerializationSchema` without breaking backwards compatibility. This PR deprecates the existing strong coupling of connector and format for the Kafa table sources and table source factories. It introduces descriptor-based alternatives. --- docs/dev/table/sqlClient.md | 2 + .../flink-connector-kafka-0.10/pom.xml | 53 ++++ .../kafka/Kafka010AvroTableSource.java | 31 +- .../kafka/Kafka010JsonTableSource.java | 32 ++- .../connectors/kafka/Kafka010TableSource.java | 71 +++-- .../kafka/Kafka010TableSourceFactory.java | 71 +++++ ...che.flink.table.sources.TableSourceFactory | 3 +- .../Kafka010AvroTableSourceFactoryTest.java | 37 --- .../kafka/Kafka010AvroTableSourceTest.java | 4 + .../Kafka010JsonTableSourceFactoryTest.java | 6 +- .../kafka/Kafka010JsonTableSourceTest.java | 4 + .../kafka/Kafka010TableSourceFactoryTest.java | 73 +++++ .../kafka/Kafka011AvroTableSource.java | 30 +- .../kafka/Kafka011AvroTableSourceFactory.java | 37 --- .../kafka/Kafka011JsonTableSource.java | 33 ++- .../kafka/Kafka011JsonTableSourceFactory.java | 37 --- .../connectors/kafka/Kafka011TableSource.java | 70 +++-- .../kafka/Kafka011TableSourceFactory.java | 71 +++++ ...che.flink.table.sources.TableSourceFactory | 3 +- .../kafka/Kafka011AvroTableSourceTest.java | 4 + .../Kafka011JsonTableSourceFactoryTest.java | 6 +- .../kafka/Kafka011JsonTableSourceTest.java | 4 + .../kafka/Kafka011TableSourceFactoryTest.java | 73 +++++ .../kafka/Kafka08AvroTableSource.java | 31 +- .../kafka/Kafka08AvroTableSourceFactory.java | 37 --- .../kafka/Kafka08JsonTableSource.java | 32 ++- .../kafka/Kafka08JsonTableSourceFactory.java | 37 --- .../connectors/kafka/Kafka08TableSource.java | 69 +++-- .../kafka/Kafka08TableSourceFactory.java | 71 +++++ ...che.flink.table.sources.TableSourceFactory | 3 +- .../kafka/Kafka08AvroTableSourceTest.java | 4 + .../kafka/Kafka08JsonTableSourceTest.java | 4 + .../kafka/Kafka08TableSourceFactoryTest.java | 73 +++++ .../flink-connector-kafka-0.9/pom.xml | 52 ++++ .../kafka/Kafka09AvroTableSource.java | 29 +- .../kafka/Kafka09AvroTableSourceFactory.java | 36 --- .../kafka/Kafka09JsonTableSource.java | 32 ++- .../kafka/Kafka09JsonTableSourceFactory.java | 37 --- .../connectors/kafka/Kafka09TableSource.java | 69 +++-- .../kafka/Kafka09TableSourceFactory.java | 71 +++++ ...che.flink.table.sources.TableSourceFactory | 3 +- .../Kafka09AvroTableSourceFactoryTest.java | 37 --- .../kafka/Kafka09AvroTableSourceTest.java | 4 + .../Kafka09JsonTableSourceFactoryTest.java | 4 + .../kafka/Kafka09JsonTableSourceTest.java | 4 + .../kafka/Kafka09TableSourceFactoryTest.java | 73 +++++ .../kafka/KafkaAvroTableSource.java | 83 ++---- .../kafka/KafkaAvroTableSourceFactory.java | 81 ------ .../kafka/KafkaJsonTableSource.java | 99 +++---- .../kafka/KafkaJsonTableSourceFactory.java | 96 ------- .../connectors/kafka/KafkaTableSource.java | 246 +++++++++++++--- .../kafka/KafkaTableSourceFactory.java | 196 +++++++------ .../KafkaAvroTableSourceFactoryTestBase.java | 123 -------- .../kafka/KafkaAvroTableSourceTestBase.java | 6 +- .../KafkaJsonTableSourceFactoryTestBase.java | 18 +- .../kafka/KafkaJsonTableSourceTestBase.java | 6 +- ...a => KafkaTableSourceBuilderTestBase.java} | 8 +- .../KafkaTableSourceFactoryTestBase.java | 187 ++++++++++++ .../avro/AvroRowDeserializationSchema.java | 19 ++ .../formats/avro/AvroRowFormatFactory.java | 99 +++++++ .../avro/AvroRowSerializationSchema.java | 18 ++ ...che.flink.table.formats.TableFormatFactory | 16 ++ .../avro/AvroRowFormatFactoryTest.java | 104 +++++++ .../json/JsonRowDeserializationSchema.java | 20 +- .../formats/json/JsonRowFormatFactory.java | 108 +++++++ ...erter.java => JsonRowSchemaConverter.java} | 4 +- .../json/JsonRowSerializationSchema.java | 20 +- .../apache/flink/table/descriptors/Json.java | 3 +- .../table/descriptors/JsonValidator.java | 2 +- ...che.flink.table.formats.TableFormatFactory | 16 ++ .../json/JsonRowFormatFactoryTest.java | 150 ++++++++++ ...t.java => JsonRowSchemaConverterTest.java} | 22 +- .../json/JsonRowSerializationSchemaTest.java | 2 +- .../table/api/BatchTableEnvironment.scala | 49 +++- .../table/api/StreamTableEnvironment.scala | 49 +++- .../flink/table/api/TableEnvironment.scala | 38 +++ .../apache/flink/table/api/TableSchema.scala | 10 + .../apache/flink/table/api/exceptions.scala | 80 ++++++ .../descriptors/DescriptorProperties.scala | 17 +- .../FormatDescriptorValidator.scala | 5 + .../table/descriptors/RowtimeValidator.scala | 43 ++- .../table/descriptors/SchemaValidator.scala | 46 ++- .../DeserializationSchemaFactory.scala | 30 +- .../formats/SerializationSchemaFactory.scala | 30 +- .../table/formats/TableFormatFactory.scala | 85 ++++++ .../formats/TableFormatFactoryService.scala | 271 ++++++++++++++++++ .../table/sources/CsvTableSourceFactory.scala | 5 +- .../table/sources/DefinedFieldMapping.scala | 3 +- .../table/sources/TableSourceFactory.scala | 9 + .../sources/TableSourceFactoryService.scala | 35 ++- ...che.flink.table.formats.TableFormatFactory | 17 ++ ...che.flink.table.sources.TableSourceFactory | 3 +- .../flink/table/descriptors/RowtimeTest.scala | 7 +- .../flink/table/descriptors/SchemaTest.scala | 5 +- .../StreamTableSourceDescriptorTest.scala | 79 ----- .../TableSourceDescriptorTest.scala | 111 +++++++ .../TableFormatFactoryServiceTest.scala | 144 ++++++++++ .../TestAmbiguousTableFormatFactory.scala | 52 ++++ .../utils/TestDeserializationSchema.scala | 50 ++++ .../utils/TestSerializationSchema.scala | 21 +- .../table/formats/utils/TestTableFormat.scala | 24 +- .../utils/TestTableFormatFactory.scala | 65 +++++ .../TableSourceFactoryServiceTest.scala | 22 +- ...cala => TestFixedFormatTableFactory.scala} | 7 +- ...TestWildcardFormatTableSourceFactory.scala | 56 ++++ .../table/utils/MockTableEnvironment.scala | 3 + 106 files changed, 3614 insertions(+), 1176 deletions(-) create mode 100644 flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactory.java delete mode 100644 flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSourceFactoryTest.java create mode 100644 flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactoryTest.java delete mode 100644 flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSourceFactory.java delete mode 100644 flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceFactory.java create mode 100644 flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactory.java create mode 100644 flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactoryTest.java delete mode 100644 flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSourceFactory.java delete mode 100644 flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceFactory.java create mode 100644 flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactory.java create mode 100644 flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactoryTest.java delete mode 100644 flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSourceFactory.java delete mode 100644 flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceFactory.java create mode 100644 flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactory.java delete mode 100644 flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSourceFactoryTest.java create mode 100644 flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactoryTest.java delete mode 100644 flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSourceFactory.java delete mode 100644 flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactory.java delete mode 100644 flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSourceFactoryTestBase.java rename flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/{KafkaTableSourceTestBase.java => KafkaTableSourceBuilderTestBase.java} (96%) create mode 100644 flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java create mode 100644 flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowFormatFactory.java create mode 100644 flink-formats/flink-avro/src/main/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory create mode 100644 flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroRowFormatFactoryTest.java create mode 100644 flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java rename flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/{JsonSchemaConverter.java => JsonRowSchemaConverter.java} (99%) create mode 100644 flink-formats/flink-json/src/main/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory create mode 100644 flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java rename flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/{JsonSchemaConverterTest.java => JsonRowSchemaConverterTest.java} (80%) rename flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceFactory.java => flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/DeserializationSchemaFactory.scala (52%) rename flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSourceFactory.java => flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/SerializationSchemaFactory.scala (52%) create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/TableFormatFactory.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/TableFormatFactoryService.scala create mode 100644 flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory delete mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptorTest.scala create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/TableSourceDescriptorTest.scala create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/TableFormatFactoryServiceTest.scala create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestAmbiguousTableFormatFactory.scala create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestDeserializationSchema.scala rename flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSourceFactoryTest.java => flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestSerializationSchema.scala (62%) rename flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSourceFactoryTest.java => flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestTableFormat.scala (61%) create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestTableFormatFactory.scala rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/{TestTableSourceFactory.scala => TestFixedFormatTableFactory.scala} (92%) create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TestWildcardFormatTableSourceFactory.scala diff --git a/docs/dev/table/sqlClient.md b/docs/dev/table/sqlClient.md index 2ba81676e9fcd..24af6557dc662 100644 --- a/docs/dev/table/sqlClient.md +++ b/docs/dev/table/sqlClient.md @@ -229,6 +229,8 @@ The SQL Client does not require to setup a Java project using Maven or SBT. Inst | Name | Version | Download | | :---------------- | :------------ | :--------------------- | | Filesystem | | Built-in | +| Apache Kafka | 0.9 | [Download](http://central.maven.org/maven2/org/apache/flink/flink-connector-kafka-0.9{{site.scala_version_suffix}}/{{site.version}}/flink-connector-kafka-0.9{{site.scala_version_suffix}}-{{site.version}}-sql-jar.jar) | +| Apache Kafka | 0.10 | [Download](http://central.maven.org/maven2/org/apache/flink/flink-connector-kafka-0.10{{site.scala_version_suffix}}/{{site.version}}/flink-connector-kafka-0.10{{site.scala_version_suffix}}-{{site.version}}-sql-jar.jar) | | Apache Kafka | 0.11 | [Download](http://central.maven.org/maven2/org/apache/flink/flink-connector-kafka-0.11{{site.scala_version_suffix}}/{{site.version}}/flink-connector-kafka-0.11{{site.scala_version_suffix}}-{{site.version}}-sql-jar.jar) | #### Formats diff --git a/flink-connectors/flink-connector-kafka-0.10/pom.xml b/flink-connectors/flink-connector-kafka-0.10/pom.xml index 22efc3470d7ab..2fb7a32397646 100644 --- a/flink-connectors/flink-connector-kafka-0.10/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.10/pom.xml @@ -202,6 +202,59 @@ under the License. + + + + release + + + release + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + package + + shade + + + true + sql-jar + + + org.apache.kafka:* + org.apache.flink:flink-connector-kafka-base_${scala.binary.version} + org.apache.flink:flink-connector-kafka-0.9_${scala.binary.version} + + + + + *:* + + kafka/kafka-version.properties + + + + + + org.apache.kafka + org.apache.flink.kafka010.shaded.org.apache.kafka + + + + + + + + + + + diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSource.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSource.java index f759f61f8fba1..ebbadcff1e197 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSource.java @@ -18,9 +18,9 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.descriptors.ConnectorDescriptor; import org.apache.flink.table.sources.RowtimeAttributeDescriptor; import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.types.Row; @@ -35,8 +35,13 @@ /** * Kafka {@link StreamTableSource} for Kafka 0.10. + * + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ -@PublicEvolving +@Deprecated public class Kafka010AvroTableSource extends KafkaAvroTableSource { /** @@ -46,7 +51,9 @@ public class Kafka010AvroTableSource extends KafkaAvroTableSource { * @param properties Properties for the Kafka consumer. * @param schema Schema of the produced table. * @param record Avro specific record. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public Kafka010AvroTableSource( String topic, Properties properties, @@ -69,7 +76,9 @@ public Kafka010AvroTableSource( * the value to the field of the Avro record.

* * @param fieldMapping A mapping from schema fields to Avro fields. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public void setFieldMapping(Map fieldMapping) { super.setFieldMapping(fieldMapping); @@ -79,7 +88,9 @@ public void setFieldMapping(Map fieldMapping) { * Declares a field of the schema to be a processing time attribute. * * @param proctimeAttribute The name of the field that becomes the processing time field. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public void setProctimeAttribute(String proctimeAttribute) { super.setProctimeAttribute(proctimeAttribute); @@ -89,7 +100,9 @@ public void setProctimeAttribute(String proctimeAttribute) { * Declares a field of the schema to be a rowtime attribute. * * @param rowtimeAttributeDescriptor The descriptor of the rowtime attribute. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public void setRowtimeAttributeDescriptor(RowtimeAttributeDescriptor rowtimeAttributeDescriptor) { Preconditions.checkNotNull(rowtimeAttributeDescriptor, "Rowtime attribute descriptor must not be null."); super.setRowtimeAttributeDescriptors(Collections.singletonList(rowtimeAttributeDescriptor)); @@ -102,15 +115,27 @@ protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properti /** * Returns a builder to configure and create a {@link Kafka010AvroTableSource}. + * * @return A builder to configure and create a {@link Kafka010AvroTableSource}. + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ + @Deprecated public static Builder builder() { return new Builder(); } /** * A builder to configure and create a {@link Kafka010AvroTableSource}. + * + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ + @Deprecated public static class Builder extends KafkaAvroTableSource.Builder { @Override @@ -127,7 +152,9 @@ protected Kafka010AvroTableSource.Builder builder() { * Builds and configures a {@link Kafka010AvroTableSource}. * * @return A configured {@link Kafka010AvroTableSource}. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public Kafka010AvroTableSource build() { Kafka010AvroTableSource tableSource = new Kafka010AvroTableSource( diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java index bda236f7445b1..a5e33a12715db 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java @@ -18,9 +18,9 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.descriptors.ConnectorDescriptor; import org.apache.flink.table.sources.RowtimeAttributeDescriptor; import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.types.Row; @@ -32,8 +32,13 @@ /** * Kafka {@link StreamTableSource} for Kafka 0.10. + * + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ -@PublicEvolving +@Deprecated public class Kafka010JsonTableSource extends KafkaJsonTableSource { /** @@ -43,7 +48,9 @@ public class Kafka010JsonTableSource extends KafkaJsonTableSource { * @param properties Properties for the Kafka consumer. * @param tableSchema The schema of the table. * @param jsonSchema The schema of the JSON messages to decode from Kafka. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public Kafka010JsonTableSource( String topic, Properties properties, @@ -58,7 +65,9 @@ public Kafka010JsonTableSource( * TableSource will fail for missing fields if set to true. If set to false, the missing field is set to null. * * @param failOnMissingField Flag that specifies the TableSource behavior in case of missing fields. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public void setFailOnMissingField(boolean failOnMissingField) { super.setFailOnMissingField(failOnMissingField); @@ -68,7 +77,9 @@ public void setFailOnMissingField(boolean failOnMissingField) { * Sets the mapping from table schema fields to JSON schema fields. * * @param fieldMapping The mapping from table schema fields to JSON schema fields. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public void setFieldMapping(Map fieldMapping) { super.setFieldMapping(fieldMapping); @@ -78,7 +89,9 @@ public void setFieldMapping(Map fieldMapping) { * Declares a field of the schema to be a processing time attribute. * * @param proctimeAttribute The name of the field that becomes the processing time field. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public void setProctimeAttribute(String proctimeAttribute) { super.setProctimeAttribute(proctimeAttribute); @@ -88,7 +101,9 @@ public void setProctimeAttribute(String proctimeAttribute) { * Declares a field of the schema to be a rowtime attribute. * * @param rowtimeAttributeDescriptor The descriptor of the rowtime attribute. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public void setRowtimeAttributeDescriptor(RowtimeAttributeDescriptor rowtimeAttributeDescriptor) { Preconditions.checkNotNull(rowtimeAttributeDescriptor, "Rowtime attribute descriptor must not be null."); super.setRowtimeAttributeDescriptors(Collections.singletonList(rowtimeAttributeDescriptor)); @@ -101,15 +116,27 @@ protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properti /** * Returns a builder to configure and create a {@link Kafka010JsonTableSource}. + * * @return A builder to configure and create a {@link Kafka010JsonTableSource}. + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ + @Deprecated public static Kafka010JsonTableSource.Builder builder() { return new Kafka010JsonTableSource.Builder(); } /** * A builder to configure and create a {@link Kafka010JsonTableSource}. + * + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ + @Deprecated public static class Builder extends KafkaJsonTableSource.Builder { @Override @@ -126,6 +153,7 @@ protected Kafka010JsonTableSource.Builder builder() { * Builds and configures a {@link Kafka010JsonTableSource}. * * @return A configured {@link Kafka010JsonTableSource}. + * @deprecated Use table descriptors instead of implementation-specific builders. */ @Override public Kafka010JsonTableSource build() { diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java index 5a0222794fd4c..e4b9881842bde 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java @@ -18,48 +18,79 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.annotation.PublicEvolving; +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; +import org.apache.flink.table.sources.RowtimeAttributeDescriptor; import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.types.Row; +import java.util.List; +import java.util.Map; import java.util.Properties; /** * Kafka {@link StreamTableSource} for Kafka 0.10. */ -@PublicEvolving -public abstract class Kafka010TableSource extends KafkaTableSource { - - // The deserialization schema for the Kafka records - private final DeserializationSchema deserializationSchema; +@Internal +public class Kafka010TableSource extends KafkaTableSource { /** * Creates a Kafka 0.10 {@link StreamTableSource}. * - * @param topic Kafka topic to consume. - * @param properties Properties for the Kafka consumer. - * @param deserializationSchema Deserialization schema to use for Kafka records. - * @param typeInfo Type information describing the result type. The field names are used - * to parse the JSON file and so are the types. + * @param schema Schema of the produced table. + * @param proctimeAttribute Field name of the processing time attribute, null if no + * processing time field is defined. + * @param rowtimeAttributeDescriptors Descriptor for a rowtime attribute + * @param fieldMapping Mapping for the fields of the table schema to + * fields of the physical returned type or null. + * @param topic Kafka topic to consume. + * @param properties Properties for the Kafka consumer. + * @param deserializationSchema Deserialization schema for decoding records from Kafka. + * @param startupMode Startup mode for the contained consumer. + * @param specificStartupOffsets Specific startup offsets; only relevant when startup + * mode is {@link StartupMode#SPECIFIC_OFFSETS}. */ public Kafka010TableSource( + TableSchema schema, + String proctimeAttribute, + List rowtimeAttributeDescriptors, + Map fieldMapping, String topic, Properties properties, DeserializationSchema deserializationSchema, - TableSchema schema, - TypeInformation typeInfo) { + StartupMode startupMode, + Map specificStartupOffsets) { - super(topic, properties, schema, typeInfo); - - this.deserializationSchema = deserializationSchema; + super( + schema, + proctimeAttribute, + rowtimeAttributeDescriptors, + fieldMapping, + topic, + properties, + deserializationSchema, + startupMode, + specificStartupOffsets); } - @Override - public DeserializationSchema getDeserializationSchema() { - return this.deserializationSchema; + /** + * Creates a Kafka 0.10 {@link StreamTableSource}. + * + * @param schema Schema of the produced table. + * @param topic Kafka topic to consume. + * @param properties Properties for the Kafka consumer. + * @param deserializationSchema Deserialization schema for decoding records from Kafka. + */ + public Kafka010TableSource( + TableSchema schema, + String topic, + Properties properties, + DeserializationSchema deserializationSchema) { + + super(schema, topic, properties, deserializationSchema); } @Override diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactory.java new file mode 100644 index 0000000000000..04f17704b0320 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactory.java @@ -0,0 +1,71 @@ +/* + * 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.streaming.connectors.kafka; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +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.descriptors.KafkaValidator; +import org.apache.flink.table.sources.RowtimeAttributeDescriptor; +import org.apache.flink.types.Row; + +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * Factory for creating configured instances of {@link Kafka010TableSource}. + */ +public class Kafka010TableSourceFactory extends KafkaTableSourceFactory { + + @Override + protected String kafkaVersion() { + return KafkaValidator.CONNECTOR_VERSION_VALUE_010; + } + + @Override + protected boolean supportsKafkaTimestamps() { + return true; + } + + @Override + protected KafkaTableSource createKafkaTableSource( + TableSchema schema, + String proctimeAttribute, + List rowtimeAttributeDescriptors, + Map fieldMapping, + String topic, + Properties properties, + DeserializationSchema deserializationSchema, + StartupMode startupMode, + Map specificStartupOffsets) { + + return new Kafka010TableSource( + schema, + proctimeAttribute, + rowtimeAttributeDescriptors, + fieldMapping, + topic, + properties, + deserializationSchema, + startupMode, + specificStartupOffsets); + } +} diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory b/flink-connectors/flink-connector-kafka-0.10/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory index cf10939a08f63..21f57077d6348 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory @@ -13,5 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.streaming.connectors.kafka.Kafka010JsonTableSourceFactory -org.apache.flink.streaming.connectors.kafka.Kafka010AvroTableSourceFactory +org.apache.flink.streaming.connectors.kafka.Kafka010TableSourceFactory diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSourceFactoryTest.java deleted file mode 100644 index d78bfbddc9e61..0000000000000 --- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSourceFactoryTest.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.streaming.connectors.kafka; - -import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_010; - -/** - * Tests for {@link Kafka010AvroTableSourceFactory}. - */ -public class Kafka010AvroTableSourceFactoryTest extends KafkaAvroTableSourceFactoryTestBase { - - @Override - protected String version() { - return CONNECTOR_VERSION_VALUE_010; - } - - @Override - protected KafkaAvroTableSource.Builder builder() { - return Kafka010AvroTableSource.builder(); - } -} diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSourceTest.java index f5f8af138d8fd..bf253c43ecec9 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSourceTest.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSourceTest.java @@ -24,7 +24,11 @@ /** * Tests for the {@link Kafka010AvroTableSource}. + * + * @deprecated Ensures backwards compatibility with Flink 1.5. Can be removed once we + * drop support for format-specific table sources. */ +@Deprecated public class Kafka010AvroTableSourceTest extends KafkaAvroTableSourceTestBase { @Override diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceFactoryTest.java index 22cf659ffeb93..ab83a01c48f8b 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceFactoryTest.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceFactoryTest.java @@ -21,8 +21,12 @@ import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_010; /** - * Tests for {@link Kafka010JsonTableSourceFactory}. + * Tests for legacy Kafka010JsonTableSourceFactory. + * + * @deprecated Ensures backwards compatibility with Flink 1.5. Can be removed once we + * drop support for format-specific table sources. */ +@Deprecated public class Kafka010JsonTableSourceFactoryTest extends KafkaJsonTableSourceFactoryTestBase { @Override diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceTest.java index f9d4d84b3bf0a..087f3edc5130d 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceTest.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceTest.java @@ -24,7 +24,11 @@ /** * Tests for the {@link Kafka010JsonTableSource}. + * + * @deprecated Ensures backwards compatibility with Flink 1.5. Can be removed once we + * drop support for format-specific table sources. */ +@Deprecated public class Kafka010JsonTableSourceTest extends KafkaJsonTableSourceTestBase { @Override diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactoryTest.java new file mode 100644 index 0000000000000..8f7b121641802 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactoryTest.java @@ -0,0 +1,73 @@ +/* + * 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.streaming.connectors.kafka; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +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.descriptors.KafkaValidator; +import org.apache.flink.table.sources.RowtimeAttributeDescriptor; +import org.apache.flink.types.Row; + +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * Test for {@link Kafka010TableSource} created by {@link Kafka010TableSourceFactory}. + */ +public class Kafka010TableSourceFactoryTest extends KafkaTableSourceFactoryTestBase { + + @Override + protected String getKafkaVersion() { + return KafkaValidator.CONNECTOR_VERSION_VALUE_010; + } + + @Override + @SuppressWarnings("unchecked") + protected Class> getExpectedFlinkKafkaConsumer() { + return (Class) FlinkKafkaConsumer010.class; + } + + @Override + protected KafkaTableSource getExpectedKafkaTableSource( + TableSchema schema, + String proctimeAttribute, + List rowtimeAttributeDescriptors, + Map fieldMapping, + String topic, + Properties properties, + DeserializationSchema deserializationSchema, + StartupMode startupMode, + Map specificStartupOffsets) { + + return new Kafka010TableSource( + schema, + proctimeAttribute, + rowtimeAttributeDescriptors, + fieldMapping, + topic, + properties, + deserializationSchema, + startupMode, + specificStartupOffsets + ); + } +} diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSource.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSource.java index f4484f6fabb30..b3f4e0a6e7527 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSource.java @@ -18,9 +18,9 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.descriptors.ConnectorDescriptor; import org.apache.flink.table.sources.RowtimeAttributeDescriptor; import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.types.Row; @@ -35,8 +35,13 @@ /** * Kafka {@link StreamTableSource} for Kafka 0.11. + * + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ -@PublicEvolving +@Deprecated public class Kafka011AvroTableSource extends KafkaAvroTableSource { /** @@ -46,7 +51,9 @@ public class Kafka011AvroTableSource extends KafkaAvroTableSource { * @param properties Properties for the Kafka consumer. * @param schema Schema of the produced table. * @param record Avro specific record. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public Kafka011AvroTableSource( String topic, Properties properties, @@ -69,7 +76,9 @@ public Kafka011AvroTableSource( * the value to the field of the Avro record.

* * @param fieldMapping A mapping from schema fields to Avro fields. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public void setFieldMapping(Map fieldMapping) { super.setFieldMapping(fieldMapping); @@ -79,7 +88,9 @@ public void setFieldMapping(Map fieldMapping) { * Declares a field of the schema to be a processing time attribute. * * @param proctimeAttribute The name of the field that becomes the processing time field. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public void setProctimeAttribute(String proctimeAttribute) { super.setProctimeAttribute(proctimeAttribute); @@ -89,7 +100,9 @@ public void setProctimeAttribute(String proctimeAttribute) { * Declares a field of the schema to be a rowtime attribute. * * @param rowtimeAttributeDescriptor The descriptor of the rowtime attribute. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public void setRowtimeAttributeDescriptor(RowtimeAttributeDescriptor rowtimeAttributeDescriptor) { Preconditions.checkNotNull(rowtimeAttributeDescriptor, "Rowtime attribute descriptor must not be null."); super.setRowtimeAttributeDescriptors(Collections.singletonList(rowtimeAttributeDescriptor)); @@ -103,14 +116,25 @@ protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properti /** * Returns a builder to configure and create a {@link Kafka011AvroTableSource}. * @return A builder to configure and create a {@link Kafka011AvroTableSource}. + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ + @Deprecated public static Builder builder() { return new Builder(); } /** * A builder to configure and create a {@link Kafka011AvroTableSource}. + * + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ + @Deprecated public static class Builder extends KafkaAvroTableSource.Builder { @Override @@ -127,7 +151,9 @@ protected Kafka011AvroTableSource.Builder builder() { * Builds and configures a {@link Kafka011AvroTableSource}. * * @return A configured {@link Kafka011AvroTableSource}. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public Kafka011AvroTableSource build() { Kafka011AvroTableSource tableSource = new Kafka011AvroTableSource( diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSourceFactory.java deleted file mode 100644 index a95998316a321..0000000000000 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSourceFactory.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.streaming.connectors.kafka; - -import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_011; - -/** - * Factory for creating configured instances of {@link Kafka011AvroTableSource}. - */ -public class Kafka011AvroTableSourceFactory extends KafkaAvroTableSourceFactory { - - @Override - protected KafkaAvroTableSource.Builder createKafkaAvroBuilder() { - return new Kafka011AvroTableSource.Builder(); - } - - @Override - protected String kafkaVersion() { - return CONNECTOR_VERSION_VALUE_011; - } -} diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSource.java index a012f5d4edde7..74c5007d80aa9 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSource.java @@ -18,9 +18,9 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.descriptors.ConnectorDescriptor; import org.apache.flink.table.sources.RowtimeAttributeDescriptor; import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.types.Row; @@ -32,8 +32,13 @@ /** * Kafka {@link StreamTableSource} for Kafka 0.11. + * + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ -@PublicEvolving +@Deprecated public class Kafka011JsonTableSource extends KafkaJsonTableSource { /** @@ -43,7 +48,9 @@ public class Kafka011JsonTableSource extends KafkaJsonTableSource { * @param properties Properties for the Kafka consumer. * @param tableSchema The schema of the table. * @param jsonSchema The schema of the JSON messages to decode from Kafka. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public Kafka011JsonTableSource( String topic, Properties properties, @@ -58,7 +65,9 @@ public Kafka011JsonTableSource( * TableSource will fail for missing fields if set to true. If set to false, the missing field is set to null. * * @param failOnMissingField Flag that specifies the TableSource behavior in case of missing fields. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public void setFailOnMissingField(boolean failOnMissingField) { super.setFailOnMissingField(failOnMissingField); @@ -68,7 +77,9 @@ public void setFailOnMissingField(boolean failOnMissingField) { * Sets the mapping from table schema fields to JSON schema fields. * * @param fieldMapping The mapping from table schema fields to JSON schema fields. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public void setFieldMapping(Map fieldMapping) { super.setFieldMapping(fieldMapping); @@ -78,7 +89,9 @@ public void setFieldMapping(Map fieldMapping) { * Declares a field of the schema to be a processing time attribute. * * @param proctimeAttribute The name of the field that becomes the processing time field. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public void setProctimeAttribute(String proctimeAttribute) { super.setProctimeAttribute(proctimeAttribute); @@ -88,7 +101,9 @@ public void setProctimeAttribute(String proctimeAttribute) { * Declares a field of the schema to be a rowtime attribute. * * @param rowtimeAttributeDescriptor The descriptor of the rowtime attribute. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public void setRowtimeAttributeDescriptor(RowtimeAttributeDescriptor rowtimeAttributeDescriptor) { Preconditions.checkNotNull(rowtimeAttributeDescriptor, "Rowtime attribute descriptor must not be null."); super.setRowtimeAttributeDescriptors(Collections.singletonList(rowtimeAttributeDescriptor)); @@ -101,15 +116,27 @@ protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properti /** * Returns a builder to configure and create a {@link Kafka011JsonTableSource}. + * * @return A builder to configure and create a {@link Kafka011JsonTableSource}. + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ + @Deprecated public static Kafka011JsonTableSource.Builder builder() { return new Kafka011JsonTableSource.Builder(); } /** * A builder to configure and create a {@link Kafka011JsonTableSource}. + * + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ + @Deprecated public static class Builder extends KafkaJsonTableSource.Builder { @Override @@ -126,7 +153,9 @@ protected Kafka011JsonTableSource.Builder builder() { * Builds and configures a {@link Kafka011JsonTableSource}. * * @return A configured {@link Kafka011JsonTableSource}. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public Kafka011JsonTableSource build() { Kafka011JsonTableSource tableSource = new Kafka011JsonTableSource( diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceFactory.java deleted file mode 100644 index 53bf7be02da4f..0000000000000 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceFactory.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.streaming.connectors.kafka; - -import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_011; - -/** - * Factory for creating configured instances of {@link Kafka011JsonTableSource}. - */ -public class Kafka011JsonTableSourceFactory extends KafkaJsonTableSourceFactory { - - @Override - protected KafkaJsonTableSource.Builder createKafkaJsonBuilder() { - return new Kafka011JsonTableSource.Builder(); - } - - @Override - protected String kafkaVersion() { - return CONNECTOR_VERSION_VALUE_011; - } -} diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java index 6c9c37d79e3de..15a4b7a2eb01d 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java @@ -18,53 +18,83 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.annotation.PublicEvolving; +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; +import org.apache.flink.table.sources.RowtimeAttributeDescriptor; import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.types.Row; +import java.util.List; +import java.util.Map; import java.util.Properties; /** * Kafka {@link StreamTableSource} for Kafka 0.11. */ -@PublicEvolving -public abstract class Kafka011TableSource extends KafkaTableSource { +@Internal +public class Kafka011TableSource extends KafkaTableSource { - // The deserialization schema for the Kafka records - private final DeserializationSchema deserializationSchema; + /** + * Creates a Kafka 0.11 {@link StreamTableSource}. + * + * @param schema Schema of the produced table. + * @param proctimeAttribute Field name of the processing time attribute, null if no + * processing time field is defined. + * @param rowtimeAttributeDescriptors Descriptor for a rowtime attribute + * @param fieldMapping Mapping for the fields of the table schema to + * fields of the physical returned type or null. + * @param topic Kafka topic to consume. + * @param properties Properties for the Kafka consumer. + * @param deserializationSchema Deserialization schema for decoding records from Kafka. + * @param startupMode Startup mode for the contained consumer. + * @param specificStartupOffsets Specific startup offsets; only relevant when startup + * mode is {@link StartupMode#SPECIFIC_OFFSETS}. + */ + public Kafka011TableSource( + TableSchema schema, + String proctimeAttribute, + List rowtimeAttributeDescriptors, + Map fieldMapping, + String topic, Properties properties, + DeserializationSchema deserializationSchema, + StartupMode startupMode, + Map specificStartupOffsets) { + + super( + schema, + proctimeAttribute, + rowtimeAttributeDescriptors, + fieldMapping, + topic, + properties, + deserializationSchema, + startupMode, + specificStartupOffsets); + } /** * Creates a Kafka 0.11 {@link StreamTableSource}. * + * @param schema Schema of the produced table. * @param topic Kafka topic to consume. * @param properties Properties for the Kafka consumer. - * @param deserializationSchema Deserialization schema to use for Kafka records. - * @param typeInfo Type information describing the result type. The field names are used - * to parse the JSON file and so are the types. + * @param deserializationSchema Deserialization schema for decoding records from Kafka. */ public Kafka011TableSource( + TableSchema schema, String topic, Properties properties, - DeserializationSchema deserializationSchema, - TableSchema schema, - TypeInformation typeInfo) { - - super(topic, properties, schema, typeInfo); + DeserializationSchema deserializationSchema) { - this.deserializationSchema = deserializationSchema; - } - - @Override - public DeserializationSchema getDeserializationSchema() { - return this.deserializationSchema; + super(schema, topic, properties, deserializationSchema); } @Override protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properties properties, DeserializationSchema deserializationSchema) { return new FlinkKafkaConsumer011<>(topic, deserializationSchema, properties); } - } diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactory.java new file mode 100644 index 0000000000000..0d00c73e39eb0 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactory.java @@ -0,0 +1,71 @@ +/* + * 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.streaming.connectors.kafka; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +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.descriptors.KafkaValidator; +import org.apache.flink.table.sources.RowtimeAttributeDescriptor; +import org.apache.flink.types.Row; + +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * Factory for creating configured instances of {@link Kafka011TableSource}. + */ +public class Kafka011TableSourceFactory extends KafkaTableSourceFactory { + + @Override + protected String kafkaVersion() { + return KafkaValidator.CONNECTOR_VERSION_VALUE_011; + } + + @Override + protected boolean supportsKafkaTimestamps() { + return true; + } + + @Override + protected KafkaTableSource createKafkaTableSource( + TableSchema schema, + String proctimeAttribute, + List rowtimeAttributeDescriptors, + Map fieldMapping, + String topic, + Properties properties, + DeserializationSchema deserializationSchema, + StartupMode startupMode, + Map specificStartupOffsets) { + + return new Kafka011TableSource( + schema, + proctimeAttribute, + rowtimeAttributeDescriptors, + fieldMapping, + topic, + properties, + deserializationSchema, + startupMode, + specificStartupOffsets); + } +} diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory b/flink-connectors/flink-connector-kafka-0.11/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory index f6825ad99e068..c056097ef8c12 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory @@ -13,5 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.streaming.connectors.kafka.Kafka011JsonTableSourceFactory -org.apache.flink.streaming.connectors.kafka.Kafka011AvroTableSourceFactory +org.apache.flink.streaming.connectors.kafka.Kafka011TableSourceFactory diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSourceTest.java index e348aa6bfb33e..aa083a20535b4 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSourceTest.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSourceTest.java @@ -24,7 +24,11 @@ /** * Tests for the {@link Kafka011AvroTableSource}. + * + * @deprecated Ensures backwards compatibility with Flink 1.5. Can be removed once we + * drop support for format-specific table sources. */ +@Deprecated public class Kafka011AvroTableSourceTest extends KafkaAvroTableSourceTestBase { @Override diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceFactoryTest.java index ed92863ba6d6f..ae46dac6a6a76 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceFactoryTest.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceFactoryTest.java @@ -21,8 +21,12 @@ import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_011; /** - * Tests for {@link Kafka011JsonTableSourceFactory}. + * Tests for legacy Kafka011JsonTableSourceFactory. + * + * @deprecated Ensures backwards compatibility with Flink 1.5. Can be removed once we + * drop support for format-specific table sources. */ +@Deprecated public class Kafka011JsonTableSourceFactoryTest extends KafkaJsonTableSourceFactoryTestBase { @Override diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceTest.java index b35d1c550724b..451795a5bd571 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceTest.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceTest.java @@ -24,7 +24,11 @@ /** * Tests for the {@link Kafka011JsonTableSource}. + * + * @deprecated Ensures backwards compatibility with Flink 1.5. Can be removed once we + * drop support for format-specific table sources. */ +@Deprecated public class Kafka011JsonTableSourceTest extends KafkaJsonTableSourceTestBase { @Override diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactoryTest.java new file mode 100644 index 0000000000000..171644a014201 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactoryTest.java @@ -0,0 +1,73 @@ +/* + * 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.streaming.connectors.kafka; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +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.descriptors.KafkaValidator; +import org.apache.flink.table.sources.RowtimeAttributeDescriptor; +import org.apache.flink.types.Row; + +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * Test for {@link Kafka011TableSource} created by {@link Kafka011TableSourceFactory}. + */ +public class Kafka011TableSourceFactoryTest extends KafkaTableSourceFactoryTestBase { + + @Override + protected String getKafkaVersion() { + return KafkaValidator.CONNECTOR_VERSION_VALUE_011; + } + + @Override + @SuppressWarnings("unchecked") + protected Class> getExpectedFlinkKafkaConsumer() { + return (Class) FlinkKafkaConsumer011.class; + } + + @Override + protected KafkaTableSource getExpectedKafkaTableSource( + TableSchema schema, + String proctimeAttribute, + List rowtimeAttributeDescriptors, + Map fieldMapping, + String topic, + Properties properties, + DeserializationSchema deserializationSchema, + StartupMode startupMode, + Map specificStartupOffsets) { + + return new Kafka011TableSource( + schema, + proctimeAttribute, + rowtimeAttributeDescriptors, + fieldMapping, + topic, + properties, + deserializationSchema, + startupMode, + specificStartupOffsets + ); + } +} diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSource.java index 3c16722add61d..820628796c5ef 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSource.java @@ -18,9 +18,9 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.descriptors.ConnectorDescriptor; import org.apache.flink.table.sources.RowtimeAttributeDescriptor; import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.types.Row; @@ -35,8 +35,13 @@ /** * Kafka {@link StreamTableSource} for Kafka 0.8. + * + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ -@PublicEvolving +@Deprecated public class Kafka08AvroTableSource extends KafkaAvroTableSource { /** @@ -46,7 +51,9 @@ public class Kafka08AvroTableSource extends KafkaAvroTableSource { * @param properties Properties for the Kafka consumer. * @param schema Schema of the produced table. * @param record Avro specific record. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public Kafka08AvroTableSource( String topic, Properties properties, @@ -69,7 +76,9 @@ public Kafka08AvroTableSource( * the value to the field of the Avro record.

* * @param fieldMapping A mapping from schema fields to Avro fields. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public void setFieldMapping(Map fieldMapping) { super.setFieldMapping(fieldMapping); @@ -79,7 +88,9 @@ public void setFieldMapping(Map fieldMapping) { * Declares a field of the schema to be a processing time attribute. * * @param proctimeAttribute The name of the field that becomes the processing time field. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public void setProctimeAttribute(String proctimeAttribute) { super.setProctimeAttribute(proctimeAttribute); @@ -89,7 +100,9 @@ public void setProctimeAttribute(String proctimeAttribute) { * Declares a field of the schema to be a rowtime attribute. * * @param rowtimeAttributeDescriptor The descriptor of the rowtime attribute. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public void setRowtimeAttributeDescriptor(RowtimeAttributeDescriptor rowtimeAttributeDescriptor) { Preconditions.checkNotNull(rowtimeAttributeDescriptor, "Rowtime attribute descriptor must not be null."); super.setRowtimeAttributeDescriptors(Collections.singletonList(rowtimeAttributeDescriptor)); @@ -102,15 +115,27 @@ protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properti /** * Returns a builder to configure and create a {@link Kafka08AvroTableSource}. + * * @return A builder to configure and create a {@link Kafka08AvroTableSource}. + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ + @Deprecated public static Builder builder() { return new Builder(); } /** * A builder to configure and create a {@link Kafka08AvroTableSource}. + * + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ + @Deprecated public static class Builder extends KafkaAvroTableSource.Builder { @Override @@ -127,7 +152,9 @@ protected Kafka08AvroTableSource.Builder builder() { * Builds and configures a {@link Kafka08AvroTableSource}. * * @return A configured {@link Kafka08AvroTableSource}. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public Kafka08AvroTableSource build() { Kafka08AvroTableSource tableSource = new Kafka08AvroTableSource( diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSourceFactory.java deleted file mode 100644 index aefc4dbc007a4..0000000000000 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSourceFactory.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.streaming.connectors.kafka; - -import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_08; - -/** - * Factory for creating configured instances of {@link Kafka08AvroTableSource}. - */ -public class Kafka08AvroTableSourceFactory extends KafkaAvroTableSourceFactory { - - @Override - protected KafkaAvroTableSource.Builder createKafkaAvroBuilder() { - return new Kafka08AvroTableSource.Builder(); - } - - @Override - protected String kafkaVersion() { - return CONNECTOR_VERSION_VALUE_08; - } -} diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java index 73870560fb3ff..acb5783c262e6 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java @@ -18,9 +18,9 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.descriptors.ConnectorDescriptor; import org.apache.flink.table.sources.RowtimeAttributeDescriptor; import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.types.Row; @@ -32,8 +32,13 @@ /** * Kafka {@link StreamTableSource} for Kafka 0.8. + * + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ -@PublicEvolving +@Deprecated public class Kafka08JsonTableSource extends KafkaJsonTableSource { /** @@ -43,7 +48,9 @@ public class Kafka08JsonTableSource extends KafkaJsonTableSource { * @param properties Properties for the Kafka consumer. * @param tableSchema The schema of the table. * @param jsonSchema The schema of the JSON messages to decode from Kafka. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public Kafka08JsonTableSource( String topic, Properties properties, @@ -58,7 +65,9 @@ public Kafka08JsonTableSource( * TableSource will fail for missing fields if set to true. If set to false, the missing field is set to null. * * @param failOnMissingField Flag that specifies the TableSource behavior in case of missing fields. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public void setFailOnMissingField(boolean failOnMissingField) { super.setFailOnMissingField(failOnMissingField); @@ -68,7 +77,9 @@ public void setFailOnMissingField(boolean failOnMissingField) { * Sets the mapping from table schema fields to JSON schema fields. * * @param fieldMapping The mapping from table schema fields to JSON schema fields. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public void setFieldMapping(Map fieldMapping) { super.setFieldMapping(fieldMapping); @@ -78,7 +89,9 @@ public void setFieldMapping(Map fieldMapping) { * Declares a field of the schema to be a processing time attribute. * * @param proctimeAttribute The name of the field that becomes the processing time field. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public void setProctimeAttribute(String proctimeAttribute) { super.setProctimeAttribute(proctimeAttribute); @@ -88,7 +101,9 @@ public void setProctimeAttribute(String proctimeAttribute) { * Declares a field of the schema to be a rowtime attribute. * * @param rowtimeAttributeDescriptor The descriptor of the rowtime attribute. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public void setRowtimeAttributeDescriptor(RowtimeAttributeDescriptor rowtimeAttributeDescriptor) { Preconditions.checkNotNull(rowtimeAttributeDescriptor, "Rowtime attribute descriptor must not be null."); super.setRowtimeAttributeDescriptors(Collections.singletonList(rowtimeAttributeDescriptor)); @@ -102,14 +117,25 @@ protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properti /** * Returns a builder to configure and create a {@link Kafka08JsonTableSource}. * @return A builder to configure and create a {@link Kafka08JsonTableSource}. + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ + @Deprecated public static Kafka08JsonTableSource.Builder builder() { return new Kafka08JsonTableSource.Builder(); } /** * A builder to configure and create a {@link Kafka08JsonTableSource}. + * + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ + @Deprecated public static class Builder extends KafkaJsonTableSource.Builder { @Override @@ -126,7 +152,9 @@ protected Kafka08JsonTableSource.Builder builder() { * Builds and configures a {@link Kafka08JsonTableSource}. * * @return A configured {@link Kafka08JsonTableSource}. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public Kafka08JsonTableSource build() { Kafka08JsonTableSource tableSource = new Kafka08JsonTableSource( diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceFactory.java deleted file mode 100644 index 13e0d57af1245..0000000000000 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceFactory.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.streaming.connectors.kafka; - -import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_08; - -/** - * Factory for creating configured instances of {@link Kafka08JsonTableSource}. - */ -public class Kafka08JsonTableSourceFactory extends KafkaJsonTableSourceFactory { - - @Override - protected KafkaJsonTableSource.Builder createKafkaJsonBuilder() { - return new Kafka08JsonTableSource.Builder(); - } - - @Override - protected String kafkaVersion() { - return CONNECTOR_VERSION_VALUE_08; - } -} diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java index 918964d67f4d1..6f6c387c0e0ed 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java @@ -18,48 +18,79 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.annotation.PublicEvolving; +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; +import org.apache.flink.table.sources.RowtimeAttributeDescriptor; import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.types.Row; +import java.util.List; +import java.util.Map; import java.util.Properties; /** * Kafka {@link StreamTableSource} for Kafka 0.8. */ -@PublicEvolving -public abstract class Kafka08TableSource extends KafkaTableSource { +@Internal +public class Kafka08TableSource extends KafkaTableSource { - // The deserialization schema for the Kafka records - private final DeserializationSchema deserializationSchema; + /** + * Creates a Kafka 0.8 {@link StreamTableSource}. + * + * @param schema Schema of the produced table. + * @param proctimeAttribute Field name of the processing time attribute, null if no + * processing time field is defined. + * @param rowtimeAttributeDescriptors Descriptor for a rowtime attribute + * @param fieldMapping Mapping for the fields of the table schema to + * fields of the physical returned type or null. + * @param topic Kafka topic to consume. + * @param properties Properties for the Kafka consumer. + * @param deserializationSchema Deserialization schema for decoding records from Kafka. + * @param startupMode Startup mode for the contained consumer. + * @param specificStartupOffsets Specific startup offsets; only relevant when startup + * mode is {@link StartupMode#SPECIFIC_OFFSETS}. + */ + public Kafka08TableSource( + TableSchema schema, + String proctimeAttribute, + List rowtimeAttributeDescriptors, + Map fieldMapping, + String topic, Properties properties, + DeserializationSchema deserializationSchema, + StartupMode startupMode, + Map specificStartupOffsets) { + + super( + schema, + proctimeAttribute, + rowtimeAttributeDescriptors, + fieldMapping, + topic, + properties, + deserializationSchema, + startupMode, + specificStartupOffsets); + } /** * Creates a Kafka 0.8 {@link StreamTableSource}. * + * @param schema Schema of the produced table. * @param topic Kafka topic to consume. * @param properties Properties for the Kafka consumer. - * @param deserializationSchema Deserialization schema to use for Kafka records. - * @param typeInfo Type information describing the result type. The field names are used - * to parse the JSON file and so are the types. + * @param deserializationSchema Deserialization schema for decoding records from Kafka. */ public Kafka08TableSource( + TableSchema schema, String topic, Properties properties, - DeserializationSchema deserializationSchema, - TableSchema schema, - TypeInformation typeInfo) { + DeserializationSchema deserializationSchema) { - super(topic, properties, schema, typeInfo); - - this.deserializationSchema = deserializationSchema; - } - - @Override - public DeserializationSchema getDeserializationSchema() { - return this.deserializationSchema; + super(schema, topic, properties, deserializationSchema); } @Override diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactory.java new file mode 100644 index 0000000000000..5cacd3eea5d57 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactory.java @@ -0,0 +1,71 @@ +/* + * 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.streaming.connectors.kafka; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +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.descriptors.KafkaValidator; +import org.apache.flink.table.sources.RowtimeAttributeDescriptor; +import org.apache.flink.types.Row; + +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * Factory for creating configured instances of {@link Kafka08TableSource}. + */ +public class Kafka08TableSourceFactory extends KafkaTableSourceFactory { + + @Override + protected String kafkaVersion() { + return KafkaValidator.CONNECTOR_VERSION_VALUE_08; + } + + @Override + protected boolean supportsKafkaTimestamps() { + return false; + } + + @Override + protected KafkaTableSource createKafkaTableSource( + TableSchema schema, + String proctimeAttribute, + List rowtimeAttributeDescriptors, + Map fieldMapping, + String topic, + Properties properties, + DeserializationSchema deserializationSchema, + StartupMode startupMode, + Map specificStartupOffsets) { + + return new Kafka08TableSource( + schema, + proctimeAttribute, + rowtimeAttributeDescriptors, + fieldMapping, + topic, + properties, + deserializationSchema, + startupMode, + specificStartupOffsets); + } +} diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory b/flink-connectors/flink-connector-kafka-0.8/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory index 7c45718d405a4..b83bb3ffc5aee 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory @@ -13,5 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.streaming.connectors.kafka.Kafka08JsonTableSourceFactory -org.apache.flink.streaming.connectors.kafka.Kafka08AvroTableSourceFactory +org.apache.flink.streaming.connectors.kafka.Kafka08TableSourceFactory diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSourceTest.java index 22daafe8e1aa7..c7b8e8c91d1a7 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSourceTest.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSourceTest.java @@ -24,7 +24,11 @@ /** * Tests for the {@link Kafka08AvroTableSource}. + * + * @deprecated Ensures backwards compatibility with Flink 1.5. Can be removed once we + * drop support for format-specific table sources. */ +@Deprecated public class Kafka08AvroTableSourceTest extends KafkaAvroTableSourceTestBase { @Override diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java index b66e8ffc9cebb..4ce79ddb8ee25 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java @@ -24,7 +24,11 @@ /** * Tests for the {@link Kafka08JsonTableSource}. + * + * @deprecated Ensures backwards compatibility with Flink 1.5. Can be removed once we + * drop support for format-specific table sources. */ +@Deprecated public class Kafka08JsonTableSourceTest extends KafkaJsonTableSourceTestBase { @Override diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactoryTest.java new file mode 100644 index 0000000000000..0cf0f95f4607f --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactoryTest.java @@ -0,0 +1,73 @@ +/* + * 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.streaming.connectors.kafka; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +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.descriptors.KafkaValidator; +import org.apache.flink.table.sources.RowtimeAttributeDescriptor; +import org.apache.flink.types.Row; + +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * Test for {@link Kafka08TableSource} created by {@link Kafka08TableSourceFactory}. + */ +public class Kafka08TableSourceFactoryTest extends KafkaTableSourceFactoryTestBase { + + @Override + protected String getKafkaVersion() { + return KafkaValidator.CONNECTOR_VERSION_VALUE_08; + } + + @Override + @SuppressWarnings("unchecked") + protected Class> getExpectedFlinkKafkaConsumer() { + return (Class) FlinkKafkaConsumer08.class; + } + + @Override + protected KafkaTableSource getExpectedKafkaTableSource( + TableSchema schema, + String proctimeAttribute, + List rowtimeAttributeDescriptors, + Map fieldMapping, + String topic, + Properties properties, + DeserializationSchema deserializationSchema, + StartupMode startupMode, + Map specificStartupOffsets) { + + return new Kafka08TableSource( + schema, + proctimeAttribute, + rowtimeAttributeDescriptors, + fieldMapping, + topic, + properties, + deserializationSchema, + startupMode, + specificStartupOffsets + ); + } +} diff --git a/flink-connectors/flink-connector-kafka-0.9/pom.xml b/flink-connectors/flink-connector-kafka-0.9/pom.xml index a4b0cfc3c1ef9..20d2991688068 100644 --- a/flink-connectors/flink-connector-kafka-0.9/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.9/pom.xml @@ -190,6 +190,58 @@ under the License. + + + + release + + + release + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + package + + shade + + + true + sql-jar + + + org.apache.kafka:* + org.apache.flink:flink-connector-kafka-base_${scala.binary.version} + + + + + *:* + + kafka/kafka-version.properties + + + + + + org.apache.kafka + org.apache.flink.kafka09.shaded.org.apache.kafka + + + + + + + + + + + diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSource.java index 7edbe651059fe..cd4bad9e7589a 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSource.java @@ -18,9 +18,9 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.descriptors.ConnectorDescriptor; import org.apache.flink.table.sources.RowtimeAttributeDescriptor; import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.types.Row; @@ -35,8 +35,13 @@ /** * Kafka {@link StreamTableSource} for Kafka 0.9. + * + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ -@PublicEvolving +@Deprecated public class Kafka09AvroTableSource extends KafkaAvroTableSource { /** @@ -69,7 +74,9 @@ public Kafka09AvroTableSource( * the value to the field of the Avro record.

* * @param fieldMapping A mapping from schema fields to Avro fields. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public void setFieldMapping(Map fieldMapping) { super.setFieldMapping(fieldMapping); @@ -79,7 +86,9 @@ public void setFieldMapping(Map fieldMapping) { * Declares a field of the schema to be a processing time attribute. * * @param proctimeAttribute The name of the field that becomes the processing time field. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public void setProctimeAttribute(String proctimeAttribute) { super.setProctimeAttribute(proctimeAttribute); @@ -89,7 +98,9 @@ public void setProctimeAttribute(String proctimeAttribute) { * Declares a field of the schema to be a rowtime attribute. * * @param rowtimeAttributeDescriptor The descriptor of the rowtime attribute. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public void setRowtimeAttributeDescriptor(RowtimeAttributeDescriptor rowtimeAttributeDescriptor) { Preconditions.checkNotNull(rowtimeAttributeDescriptor, "Rowtime attribute descriptor must not be null."); super.setRowtimeAttributeDescriptors(Collections.singletonList(rowtimeAttributeDescriptor)); @@ -102,15 +113,27 @@ protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properti /** * Returns a builder to configure and create a {@link Kafka09AvroTableSource}. + * * @return A builder to configure and create a {@link Kafka09AvroTableSource}. + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ + @Deprecated public static Builder builder() { return new Builder(); } /** * A builder to configure and create a {@link Kafka09AvroTableSource}. + * + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ + @Deprecated public static class Builder extends KafkaAvroTableSource.Builder { @Override @@ -127,7 +150,9 @@ protected Kafka09AvroTableSource.Builder builder() { * Builds and configures a {@link Kafka09AvroTableSource}. * * @return A configured {@link Kafka09AvroTableSource}. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public Kafka09AvroTableSource build() { Kafka09AvroTableSource tableSource = new Kafka09AvroTableSource( diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSourceFactory.java deleted file mode 100644 index 0cbe2bf8ff9de..0000000000000 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSourceFactory.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.streaming.connectors.kafka; - -import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_09; - -/** - * Factory for creating configured instances of {@link Kafka09AvroTableSource}. - */ -public class Kafka09AvroTableSourceFactory extends KafkaAvroTableSourceFactory { - @Override - protected KafkaAvroTableSource.Builder createKafkaAvroBuilder() { - return new Kafka09AvroTableSource.Builder(); - } - - @Override - protected String kafkaVersion() { - return CONNECTOR_VERSION_VALUE_09; - } -} diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java index a7e54be2275e0..ad4e0d897684a 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java @@ -18,9 +18,9 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.descriptors.ConnectorDescriptor; import org.apache.flink.table.sources.RowtimeAttributeDescriptor; import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.types.Row; @@ -32,8 +32,13 @@ /** * Kafka {@link StreamTableSource} for Kafka 0.9. + * + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ -@PublicEvolving +@Deprecated public class Kafka09JsonTableSource extends KafkaJsonTableSource { /** @@ -43,7 +48,9 @@ public class Kafka09JsonTableSource extends KafkaJsonTableSource { * @param properties Properties for the Kafka consumer. * @param tableSchema The schema of the table. * @param jsonSchema The schema of the JSON messages to decode from Kafka. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public Kafka09JsonTableSource( String topic, Properties properties, @@ -58,7 +65,9 @@ public Kafka09JsonTableSource( * TableSource will fail for missing fields if set to true. If set to false, the missing field is set to null. * * @param failOnMissingField Flag that specifies the TableSource behavior in case of missing fields. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public void setFailOnMissingField(boolean failOnMissingField) { super.setFailOnMissingField(failOnMissingField); @@ -68,7 +77,9 @@ public void setFailOnMissingField(boolean failOnMissingField) { * Sets the mapping from table schema fields to JSON schema fields. * * @param fieldMapping The mapping from table schema fields to JSON schema fields. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public void setFieldMapping(Map fieldMapping) { super.setFieldMapping(fieldMapping); @@ -78,7 +89,9 @@ public void setFieldMapping(Map fieldMapping) { * Declares a field of the schema to be a processing time attribute. * * @param proctimeAttribute The name of the field that becomes the processing time field. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public void setProctimeAttribute(String proctimeAttribute) { super.setProctimeAttribute(proctimeAttribute); @@ -88,7 +101,9 @@ public void setProctimeAttribute(String proctimeAttribute) { * Declares a field of the schema to be a rowtime attribute. * * @param rowtimeAttributeDescriptor The descriptor of the rowtime attribute. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public void setRowtimeAttributeDescriptor(RowtimeAttributeDescriptor rowtimeAttributeDescriptor) { Preconditions.checkNotNull(rowtimeAttributeDescriptor, "Rowtime attribute descriptor must not be null."); super.setRowtimeAttributeDescriptors(Collections.singletonList(rowtimeAttributeDescriptor)); @@ -102,14 +117,25 @@ protected FlinkKafkaConsumerBase createKafkaConsumer(String topic, Properti /** * Returns a builder to configure and create a {@link Kafka09JsonTableSource}. * @return A builder to configure and create a {@link Kafka09JsonTableSource}. + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ + @Deprecated public static Kafka09JsonTableSource.Builder builder() { return new Kafka09JsonTableSource.Builder(); } /** * A builder to configure and create a {@link Kafka09JsonTableSource}. + * + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ + @Deprecated public static class Builder extends KafkaJsonTableSource.Builder { @Override @@ -126,7 +152,9 @@ protected Kafka09JsonTableSource.Builder builder() { * Builds and configures a {@link Kafka09JsonTableSource}. * * @return A configured {@link Kafka09JsonTableSource}. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated @Override public Kafka09JsonTableSource build() { Kafka09JsonTableSource tableSource = new Kafka09JsonTableSource( diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceFactory.java deleted file mode 100644 index 736328214b83b..0000000000000 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceFactory.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.streaming.connectors.kafka; - -import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_09; - -/** - * Factory for creating configured instances of {@link Kafka09JsonTableSource}. - */ -public class Kafka09JsonTableSourceFactory extends KafkaJsonTableSourceFactory { - - @Override - protected KafkaJsonTableSource.Builder createKafkaJsonBuilder() { - return new Kafka09JsonTableSource.Builder(); - } - - @Override - protected String kafkaVersion() { - return CONNECTOR_VERSION_VALUE_09; - } -} diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java index 31518de5c3215..a598b035994fc 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java @@ -18,48 +18,79 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.annotation.PublicEvolving; +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; +import org.apache.flink.table.sources.RowtimeAttributeDescriptor; import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.types.Row; +import java.util.List; +import java.util.Map; import java.util.Properties; /** * Kafka {@link StreamTableSource} for Kafka 0.9. */ -@PublicEvolving -public abstract class Kafka09TableSource extends KafkaTableSource { +@Internal +public class Kafka09TableSource extends KafkaTableSource { - // The deserialization schema for the Kafka records - private final DeserializationSchema deserializationSchema; + /** + * Creates a Kafka 0.9 {@link StreamTableSource}. + * + * @param schema Schema of the produced table. + * @param proctimeAttribute Field name of the processing time attribute, null if no + * processing time field is defined. + * @param rowtimeAttributeDescriptors Descriptor for a rowtime attribute + * @param fieldMapping Mapping for the fields of the table schema to + * fields of the physical returned type or null. + * @param topic Kafka topic to consume. + * @param properties Properties for the Kafka consumer. + * @param deserializationSchema Deserialization schema for decoding records from Kafka. + * @param startupMode Startup mode for the contained consumer. + * @param specificStartupOffsets Specific startup offsets; only relevant when startup + * mode is {@link StartupMode#SPECIFIC_OFFSETS}. + */ + public Kafka09TableSource( + TableSchema schema, + String proctimeAttribute, + List rowtimeAttributeDescriptors, + Map fieldMapping, + String topic, Properties properties, + DeserializationSchema deserializationSchema, + StartupMode startupMode, + Map specificStartupOffsets) { + + super( + schema, + proctimeAttribute, + rowtimeAttributeDescriptors, + fieldMapping, + topic, + properties, + deserializationSchema, + startupMode, + specificStartupOffsets); + } /** * Creates a Kafka 0.9 {@link StreamTableSource}. * + * @param schema Schema of the produced table. * @param topic Kafka topic to consume. * @param properties Properties for the Kafka consumer. - * @param deserializationSchema Deserialization schema to use for Kafka records. - * @param typeInfo Type information describing the result type. The field names are used - * to parse the JSON file and so are the types. + * @param deserializationSchema Deserialization schema for decoding records from Kafka. */ public Kafka09TableSource( + TableSchema schema, String topic, Properties properties, - DeserializationSchema deserializationSchema, - TableSchema schema, - TypeInformation typeInfo) { + DeserializationSchema deserializationSchema) { - super(topic, properties, schema, typeInfo); - - this.deserializationSchema = deserializationSchema; - } - - @Override - public DeserializationSchema getDeserializationSchema() { - return this.deserializationSchema; + super(schema, topic, properties, deserializationSchema); } @Override diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactory.java new file mode 100644 index 0000000000000..98b7fb5427d83 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactory.java @@ -0,0 +1,71 @@ +/* + * 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.streaming.connectors.kafka; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +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.descriptors.KafkaValidator; +import org.apache.flink.table.sources.RowtimeAttributeDescriptor; +import org.apache.flink.types.Row; + +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * Factory for creating configured instances of {@link Kafka09TableSource}. + */ +public class Kafka09TableSourceFactory extends KafkaTableSourceFactory { + + @Override + protected String kafkaVersion() { + return KafkaValidator.CONNECTOR_VERSION_VALUE_09; + } + + @Override + protected boolean supportsKafkaTimestamps() { + return false; + } + + @Override + protected KafkaTableSource createKafkaTableSource( + TableSchema schema, + String proctimeAttribute, + List rowtimeAttributeDescriptors, + Map fieldMapping, + String topic, + Properties properties, + DeserializationSchema deserializationSchema, + StartupMode startupMode, + Map specificStartupOffsets) { + + return new Kafka09TableSource( + schema, + proctimeAttribute, + rowtimeAttributeDescriptors, + fieldMapping, + topic, + properties, + deserializationSchema, + startupMode, + specificStartupOffsets); + } +} diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory b/flink-connectors/flink-connector-kafka-0.9/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory index 4717da925691d..fb14ddb505507 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory @@ -13,5 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.streaming.connectors.kafka.Kafka09JsonTableSourceFactory -org.apache.flink.streaming.connectors.kafka.Kafka09AvroTableSourceFactory +org.apache.flink.streaming.connectors.kafka.Kafka09TableSourceFactory diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSourceFactoryTest.java deleted file mode 100644 index 3078373dc8aa4..0000000000000 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSourceFactoryTest.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.streaming.connectors.kafka; - -import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_09; - -/** - * Tests for {@link Kafka09AvroTableSourceFactory}. - */ -public class Kafka09AvroTableSourceFactoryTest extends KafkaAvroTableSourceFactoryTestBase { - - @Override - protected String version() { - return CONNECTOR_VERSION_VALUE_09; - } - - @Override - protected KafkaAvroTableSource.Builder builder() { - return Kafka09AvroTableSource.builder(); - } -} diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSourceTest.java index 0ab8a4be61cb4..6f3a56639b310 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSourceTest.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09AvroTableSourceTest.java @@ -24,7 +24,11 @@ /** * Tests for the {@link Kafka09AvroTableSource}. + * + * @deprecated Ensures backwards compatibility with Flink 1.5. Can be removed once we + * drop support for format-specific table sources. */ +@Deprecated public class Kafka09AvroTableSourceTest extends KafkaAvroTableSourceTestBase { @Override diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceFactoryTest.java index dd545e9395138..e0437a10c4efd 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceFactoryTest.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceFactoryTest.java @@ -22,7 +22,11 @@ /** * Factory for creating configured instances of {@link Kafka09JsonTableSource}. + * + * @deprecated Ensures backwards compatibility with Flink 1.5. Can be removed once we + * drop support for format-specific table sources. */ +@Deprecated public class Kafka09JsonTableSourceFactoryTest extends KafkaJsonTableSourceFactoryTestBase { @Override diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java index 65fa0f1796b55..bfff7d20102a9 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java @@ -24,7 +24,11 @@ /** * Tests for the {@link Kafka09JsonTableSource}. + * + * @deprecated Ensures backwards compatibility with Flink 1.5. Can be removed once we + * drop support for format-specific table sources. */ +@Deprecated public class Kafka09JsonTableSourceTest extends KafkaJsonTableSourceTestBase { @Override diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactoryTest.java new file mode 100644 index 0000000000000..85a96dfe94dd7 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactoryTest.java @@ -0,0 +1,73 @@ +/* + * 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.streaming.connectors.kafka; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +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.descriptors.KafkaValidator; +import org.apache.flink.table.sources.RowtimeAttributeDescriptor; +import org.apache.flink.types.Row; + +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * Test for {@link Kafka09TableSource} created by {@link Kafka09TableSourceFactory}. + */ +public class Kafka09TableSourceFactoryTest extends KafkaTableSourceFactoryTestBase { + + @Override + protected String getKafkaVersion() { + return KafkaValidator.CONNECTOR_VERSION_VALUE_09; + } + + @Override + @SuppressWarnings("unchecked") + protected Class> getExpectedFlinkKafkaConsumer() { + return (Class) FlinkKafkaConsumer09.class; + } + + @Override + protected KafkaTableSource getExpectedKafkaTableSource( + TableSchema schema, + String proctimeAttribute, + List rowtimeAttributeDescriptors, + Map fieldMapping, + String topic, + Properties properties, + DeserializationSchema deserializationSchema, + StartupMode startupMode, + Map specificStartupOffsets) { + + return new Kafka09TableSource( + schema, + proctimeAttribute, + rowtimeAttributeDescriptors, + fieldMapping, + topic, + properties, + deserializationSchema, + startupMode, + specificStartupOffsets + ); + } +} diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java index 8c8ce324ba997..3e9f2b03e344f 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSource.java @@ -21,16 +21,14 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.formats.avro.AvroRowDeserializationSchema; -import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.sources.DefinedFieldMapping; +import org.apache.flink.table.descriptors.ConnectorDescriptor; import org.apache.flink.table.sources.StreamTableSource; import org.apache.avro.specific.SpecificRecord; import org.apache.avro.specific.SpecificRecordBase; import java.util.Map; -import java.util.Objects; import java.util.Properties; /** @@ -38,13 +36,15 @@ * *

The version-specific Kafka consumers need to extend this class and * override {@link #createKafkaConsumer(String, Properties, DeserializationSchema)}}. + * + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ +@Deprecated @Internal -public abstract class KafkaAvroTableSource extends KafkaTableSource implements DefinedFieldMapping { - - private final Class avroRecordClass; - - private Map fieldMapping; +public abstract class KafkaAvroTableSource extends KafkaTableSource { /** * Creates a generic Kafka Avro {@link StreamTableSource} using a given {@link SpecificRecord}. @@ -53,7 +53,12 @@ public abstract class KafkaAvroTableSource extends KafkaTableSource implements D * @param properties Properties for the Kafka consumer. * @param schema Schema of the produced table. * @param avroRecordClass Class of the Avro record that is read from the Kafka topic. + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ + @Deprecated protected KafkaAvroTableSource( String topic, Properties properties, @@ -61,59 +66,15 @@ protected KafkaAvroTableSource( Class avroRecordClass) { super( + schema, topic, properties, - schema, - AvroSchemaConverter.convertToTypeInfo(avroRecordClass)); - - this.avroRecordClass = avroRecordClass; - } - - @Override - public Map getFieldMapping() { - return fieldMapping; + new AvroRowDeserializationSchema(avroRecordClass)); } @Override public String explainSource() { - return "KafkaAvroTableSource(" + this.avroRecordClass.getSimpleName() + ")"; - } - - @Override - protected AvroRowDeserializationSchema getDeserializationSchema() { - return new AvroRowDeserializationSchema(avroRecordClass); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof KafkaAvroTableSource)) { - return false; - } - if (!super.equals(o)) { - return false; - } - final KafkaAvroTableSource that = (KafkaAvroTableSource) o; - return Objects.equals(avroRecordClass, that.avroRecordClass) && - Objects.equals(fieldMapping, that.fieldMapping); - } - - @Override - public int hashCode() { - return Objects.hash(super.hashCode(), avroRecordClass, fieldMapping); - } - - //////// SETTERS FOR OPTIONAL PARAMETERS - - /** - * Configures a field mapping for this TableSource. - * - * @param fieldMapping The field mapping. - */ - protected void setFieldMapping(Map fieldMapping) { - this.fieldMapping = fieldMapping; + return "KafkaAvroTableSource"; } //////// HELPER METHODS @@ -124,7 +85,13 @@ protected void setFieldMapping(Map fieldMapping) { * * @param Type of the KafkaAvroTableSource produced by the builder. * @param Type of the KafkaAvroTableSource.Builder subclass. + * + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ + @Deprecated protected abstract static class Builder extends KafkaTableSource.Builder { @@ -137,7 +104,9 @@ protected abstract static class Builder avroClass) { this.avroClass = avroClass; return builder(); @@ -153,7 +122,9 @@ public B forAvroRecordClass(Class avroClass) { * * @param schemaToAvroMapping A mapping from schema fields to Avro fields. * @return The builder. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public B withTableToAvroMapping(Map schemaToAvroMapping) { this.fieldMapping = schemaToAvroMapping; return builder(); @@ -163,7 +134,9 @@ public B withTableToAvroMapping(Map schemaToAvroMapping) { * Returns the configured Avro class. * * @return The configured Avro class. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated protected Class getAvroRecordClass() { return this.avroClass; } diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSourceFactory.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSourceFactory.java deleted file mode 100644 index 8ef7270ccae5a..0000000000000 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSourceFactory.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * 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.streaming.connectors.kafka; - -import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; -import org.apache.flink.table.api.TableSchema; -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.descriptors.SchemaValidator; - -import org.apache.avro.specific.SpecificRecordBase; - -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Optional; - -/** - * Factory for creating configured instances of {@link KafkaAvroTableSource}. - */ -public abstract class KafkaAvroTableSourceFactory extends KafkaTableSourceFactory { - - @Override - protected String formatType() { - return AvroValidator.FORMAT_TYPE_VALUE; - } - - @Override - protected int formatPropertyVersion() { - return 1; - } - - @Override - protected List formatProperties() { - return Collections.singletonList(AvroValidator.FORMAT_RECORD_CLASS); - } - - @Override - protected FormatDescriptorValidator formatValidator() { - return new AvroValidator(); - } - - @Override - protected KafkaTableSource.Builder createBuilderWithFormat(DescriptorProperties params) { - KafkaAvroTableSource.Builder builder = createKafkaAvroBuilder(); - - // Avro format schema - final Class avroRecordClass = - params.getClass(AvroValidator.FORMAT_RECORD_CLASS, SpecificRecordBase.class); - builder.forAvroRecordClass(avroRecordClass); - final TableSchema formatSchema = TableSchema.fromTypeInfo(AvroSchemaConverter.convertToTypeInfo(avroRecordClass)); - - // field mapping - final Map mapping = SchemaValidator.deriveFieldMapping(params, Optional.of(formatSchema)); - builder.withTableToAvroMapping(mapping); - - return builder; - } - - /** - * Creates a version specific {@link KafkaAvroTableSource.Builder}. - */ - protected abstract KafkaAvroTableSource.Builder createKafkaAvroBuilder(); -} diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java index b2bb8ff773bf4..bd0d0dedf22e7 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java @@ -20,14 +20,12 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.formats.json.JsonRowDeserializationSchema; import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.sources.DefinedFieldMapping; +import org.apache.flink.table.descriptors.ConnectorDescriptor; import org.apache.flink.table.sources.StreamTableSource; import java.util.Map; -import java.util.Objects; import java.util.Properties; /** @@ -37,15 +35,15 @@ * override {@link #createKafkaConsumer(String, Properties, DeserializationSchema)}}. * *

The field names are used to parse the JSON file and so are the types. + * + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ +@Deprecated @Internal -public abstract class KafkaJsonTableSource extends KafkaTableSource implements DefinedFieldMapping { - - private TableSchema jsonSchema; - - private Map fieldMapping; - - private boolean failOnMissingField; +public abstract class KafkaJsonTableSource extends KafkaTableSource { /** * Creates a generic Kafka JSON {@link StreamTableSource}. @@ -54,7 +52,9 @@ public abstract class KafkaJsonTableSource extends KafkaTableSource implements D * @param properties Properties for the Kafka consumer. * @param tableSchema The schema of the table. * @param jsonSchema The schema of the JSON messages to decode from Kafka. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated protected KafkaJsonTableSource( String topic, Properties properties, @@ -62,24 +62,10 @@ protected KafkaJsonTableSource( TableSchema jsonSchema) { super( + tableSchema, topic, properties, - tableSchema, - jsonSchemaToReturnType(jsonSchema)); - - this.jsonSchema = jsonSchema; - } - - @Override - public Map getFieldMapping() { - return fieldMapping; - } - - @Override - protected JsonRowDeserializationSchema getDeserializationSchema() { - JsonRowDeserializationSchema deserSchema = new JsonRowDeserializationSchema(jsonSchemaToReturnType(jsonSchema)); - deserSchema.setFailOnMissingField(failOnMissingField); - return deserSchema; + new JsonRowDeserializationSchema(jsonSchema.toRowType())); } @Override @@ -87,28 +73,6 @@ public String explainSource() { return "KafkaJsonTableSource"; } - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof KafkaJsonTableSource)) { - return false; - } - if (!super.equals(o)) { - return false; - } - KafkaJsonTableSource that = (KafkaJsonTableSource) o; - return failOnMissingField == that.failOnMissingField && - Objects.equals(jsonSchema, that.jsonSchema) && - Objects.equals(fieldMapping, that.fieldMapping); - } - - @Override - public int hashCode() { - return Objects.hash(super.hashCode(), jsonSchema, fieldMapping, failOnMissingField); - } - //////// SETTERS FOR OPTIONAL PARAMETERS /** @@ -116,34 +80,27 @@ public int hashCode() { * TableSource will fail for missing fields if set to true. If set to false, the missing field is set to null. * * @param failOnMissingField Flag that specifies the TableSource behavior in case of missing fields. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated protected void setFailOnMissingField(boolean failOnMissingField) { - this.failOnMissingField = failOnMissingField; - } - - /** - * Sets the mapping from table schema fields to JSON schema fields. - * - * @param fieldMapping The mapping from table schema fields to JSON schema fields. - */ - protected void setFieldMapping(Map fieldMapping) { - this.fieldMapping = fieldMapping; + ((JsonRowDeserializationSchema) getDeserializationSchema()).setFailOnMissingField(failOnMissingField); } //////// HELPER METHODS - /** Converts the JSON schema into into the return type. */ - private static RowTypeInfo jsonSchemaToReturnType(TableSchema jsonSchema) { - return new RowTypeInfo(jsonSchema.getTypes(), jsonSchema.getColumnNames()); - } - /** * Abstract builder for a {@link KafkaJsonTableSource} to be extended by builders of subclasses of * KafkaJsonTableSource. * * @param Type of the KafkaJsonTableSource produced by the builder. * @param Type of the KafkaJsonTableSource.Builder subclass. + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ + @Deprecated protected abstract static class Builder extends KafkaTableSource.Builder { @@ -159,7 +116,9 @@ protected abstract static class Builder tableToJsonMapping) { this.fieldMapping = tableToJsonMapping; return builder(); @@ -188,7 +149,9 @@ public B withTableToJsonMapping(Map tableToJsonMapping) { * field. * If set to false, a missing field is set to null. * @return The builder. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public B failOnMissingField(boolean failOnMissingField) { this.failOnMissingField = failOnMissingField; return builder(); @@ -199,7 +162,9 @@ public B failOnMissingField(boolean failOnMissingField) { * is returned. * * @return The JSON schema for the TableSource. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated protected TableSchema getJsonSchema() { if (jsonSchema != null) { return this.jsonSchema; @@ -208,7 +173,13 @@ protected TableSchema getJsonSchema() { } } - @Override + /** + * Configures a TableSource with optional parameters. + * + * @param source The TableSource to configure. + * @deprecated Use table descriptors instead of implementation-specific builders. + */ + @Deprecated protected void configureTableSource(T source) { super.configureTableSource(source); // configure field mapping diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactory.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactory.java deleted file mode 100644 index 98985694ab9a2..0000000000000 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactory.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * 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.streaming.connectors.kafka; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.formats.json.JsonSchemaConverter; -import org.apache.flink.table.api.TableSchema; -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 java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Optional; - -/** - * Factory for creating configured instances of {@link KafkaJsonTableSource}. - */ -public abstract class KafkaJsonTableSourceFactory extends KafkaTableSourceFactory { - - @Override - protected String formatType() { - return JsonValidator.FORMAT_TYPE_VALUE; - } - - @Override - protected int formatPropertyVersion() { - return 1; - } - - @Override - protected List formatProperties() { - return Arrays.asList( - JsonValidator.FORMAT_JSON_SCHEMA, - JsonValidator.FORMAT_SCHEMA, - JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD, - FormatDescriptorValidator.FORMAT_DERIVE_SCHEMA()); - } - - @Override - protected FormatDescriptorValidator formatValidator() { - return new JsonValidator(); - } - - @Override - protected KafkaTableSource.Builder createBuilderWithFormat(DescriptorProperties params) { - final KafkaJsonTableSource.Builder builder = createKafkaJsonBuilder(); - - // missing field - params.getOptionalBoolean(JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD) - .ifPresent(builder::failOnMissingField); - - // json schema - final TableSchema formatSchema; - if (params.containsKey(JsonValidator.FORMAT_SCHEMA)) { - final TypeInformation info = params.getType(JsonValidator.FORMAT_SCHEMA); - formatSchema = TableSchema.fromTypeInfo(info); - } else if (params.containsKey(JsonValidator.FORMAT_JSON_SCHEMA)) { - final TypeInformation info = JsonSchemaConverter.convert(params.getString(JsonValidator.FORMAT_JSON_SCHEMA)); - formatSchema = TableSchema.fromTypeInfo(info); - } else { - formatSchema = SchemaValidator.deriveFormatFields(params); - } - builder.forJsonSchema(formatSchema); - - // field mapping - final Map mapping = SchemaValidator.deriveFieldMapping(params, Optional.of(formatSchema)); - builder.withTableToJsonMapping(mapping); - - return builder; - } - - /** - * Creates a version specific {@link KafkaJsonTableSource.Builder}. - */ - protected abstract KafkaJsonTableSource.Builder createKafkaJsonBuilder(); - -} diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java index 29654b061c790..24cfd9436dbdd 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java @@ -28,6 +28,8 @@ import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.Types; import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.descriptors.ConnectorDescriptor; +import org.apache.flink.table.sources.DefinedFieldMapping; import org.apache.flink.table.sources.DefinedProctimeAttribute; import org.apache.flink.table.sources.DefinedRowtimeAttributes; import org.apache.flink.table.sources.RowtimeAttributeDescriptor; @@ -54,29 +56,40 @@ * override {@link #createKafkaConsumer(String, Properties, DeserializationSchema)}}. */ @Internal -public abstract class KafkaTableSource - implements StreamTableSource, DefinedProctimeAttribute, DefinedRowtimeAttributes { +public abstract class KafkaTableSource implements + StreamTableSource, + DefinedProctimeAttribute, + DefinedRowtimeAttributes, + DefinedFieldMapping { + + // common table source attributes + // TODO make all attributes final once we drop support for format-specific table sources /** The schema of the table. */ private final TableSchema schema; + /** Field name of the processing time attribute, null if no processing time field is defined. */ + private String proctimeAttribute; + + /** Descriptor for a rowtime attribute. */ + private List rowtimeAttributeDescriptors; + + /** Mapping for the fields of the table schema to fields of the physical returned type or null. */ + private Map fieldMapping; + + // Kafka-specific attributes + /** The Kafka topic to consume. */ private final String topic; /** Properties for the Kafka consumer. */ private final Properties properties; - /** Type information describing the result type. */ - private TypeInformation returnType; - - /** Field name of the processing time attribute, null if no processing time field is defined. */ - private String proctimeAttribute; - - /** Descriptor for a rowtime attribute. */ - private List rowtimeAttributeDescriptors; + /** Deserialization schema for decoding records from Kafka. */ + private final DeserializationSchema deserializationSchema; /** The startup mode for the contained consumer (default is {@link StartupMode#GROUP_OFFSETS}). */ - private StartupMode startupMode = StartupMode.GROUP_OFFSETS; + private StartupMode startupMode; /** Specific startup offsets; only relevant when startup mode is {@link StartupMode#SPECIFIC_OFFSETS}. */ private Map specificStartupOffsets; @@ -84,21 +97,64 @@ public abstract class KafkaTableSource /** * Creates a generic Kafka {@link StreamTableSource}. * - * @param topic Kafka topic to consume. - * @param properties Properties for the Kafka consumer. - * @param schema Schema of the produced table. - * @param returnType Type information of the produced physical DataStream. + * @param schema Schema of the produced table. + * @param proctimeAttribute Field name of the processing time attribute, null if no + * processing time field is defined. + * @param rowtimeAttributeDescriptors Descriptor for a rowtime attribute + * @param fieldMapping Mapping for the fields of the table schema to + * fields of the physical returned type or null. + * @param topic Kafka topic to consume. + * @param properties Properties for the Kafka consumer. + * @param deserializationSchema Deserialization schema for decoding records from Kafka. + * @param startupMode Startup mode for the contained consumer. + * @param specificStartupOffsets Specific startup offsets; only relevant when startup + * mode is {@link StartupMode#SPECIFIC_OFFSETS}. */ protected KafkaTableSource( + TableSchema schema, + String proctimeAttribute, + List rowtimeAttributeDescriptors, + Map fieldMapping, String topic, Properties properties, - TableSchema schema, - TypeInformation returnType) { - + DeserializationSchema deserializationSchema, + StartupMode startupMode, + Map specificStartupOffsets) { + this.schema = Preconditions.checkNotNull(schema, "Schema must not be null."); + this.proctimeAttribute = validateProctimeAttribute(proctimeAttribute); + this.rowtimeAttributeDescriptors = validateRowtimeAttributeDescriptors(rowtimeAttributeDescriptors); + this.fieldMapping = fieldMapping; this.topic = Preconditions.checkNotNull(topic, "Topic must not be null."); this.properties = Preconditions.checkNotNull(properties, "Properties must not be null."); - this.schema = Preconditions.checkNotNull(schema, "Schema must not be null."); - this.returnType = Preconditions.checkNotNull(returnType, "Type information must not be null."); + this.deserializationSchema = Preconditions.checkNotNull( + deserializationSchema, "Deserialization schema must not be null."); + this.startupMode = Preconditions.checkNotNull(startupMode, "Startup mode must not be null."); + this.specificStartupOffsets = Preconditions.checkNotNull( + specificStartupOffsets, "Specific offsets must not be null."); + } + + /** + * Creates a generic Kafka {@link StreamTableSource}. + * + * @param schema Schema of the produced table. + * @param topic Kafka topic to consume. + * @param properties Properties for the Kafka consumer. + * @param deserializationSchema Deserialization schema for decoding records from Kafka. + */ + protected KafkaTableSource( + TableSchema schema, + String topic, + Properties properties, + DeserializationSchema deserializationSchema) { + this( + schema, + null, + Collections.emptyList(), + null, + topic, properties, + deserializationSchema, + StartupMode.GROUP_OFFSETS, + Collections.emptyMap()); } /** @@ -116,7 +172,7 @@ public DataStream getDataStream(StreamExecutionEnvironment env) { @Override public TypeInformation getReturnType() { - return returnType; + return deserializationSchema.getProducedType(); } @Override @@ -134,34 +190,60 @@ public List getRowtimeAttributeDescriptors() { return rowtimeAttributeDescriptors; } + @Override + public Map getFieldMapping() { + return fieldMapping; + } + @Override public String explainSource() { return TableConnectorUtil.generateRuntimeName(this.getClass(), schema.getColumnNames()); } + /** + * Returns the properties for the Kafka consumer. + * + * @return properties for the Kafka consumer. + */ + public Properties getProperties() { + return properties; + } + + /** + * Returns the deserialization schema. + * + * @return The deserialization schema + */ + public DeserializationSchema getDeserializationSchema(){ + return deserializationSchema; + } + @Override public boolean equals(Object o) { if (this == o) { return true; } - if (!(o instanceof KafkaTableSource)) { + // TODO force classes to be equal once we drop support for format-specific table sources + // if (o == null || getClass() != o.getClass()) { + if (o == null || !(o instanceof KafkaTableSource)) { return false; } - KafkaTableSource that = (KafkaTableSource) o; + final KafkaTableSource that = (KafkaTableSource) o; return Objects.equals(schema, that.schema) && - Objects.equals(topic, that.topic) && - Objects.equals(properties, that.properties) && - Objects.equals(returnType, that.returnType) && Objects.equals(proctimeAttribute, that.proctimeAttribute) && Objects.equals(rowtimeAttributeDescriptors, that.rowtimeAttributeDescriptors) && + Objects.equals(fieldMapping, that.fieldMapping) && + Objects.equals(topic, that.topic) && + Objects.equals(properties, that.properties) && + Objects.equals(deserializationSchema, that.deserializationSchema) && startupMode == that.startupMode && Objects.equals(specificStartupOffsets, that.specificStartupOffsets); } @Override public int hashCode() { - return Objects.hash(schema, topic, properties, returnType, - proctimeAttribute, rowtimeAttributeDescriptors, startupMode, specificStartupOffsets); + return Objects.hash(schema, proctimeAttribute, rowtimeAttributeDescriptors, fieldMapping, + topic, properties, deserializationSchema, startupMode, specificStartupOffsets); } /** @@ -195,14 +277,14 @@ protected FlinkKafkaConsumerBase getKafkaConsumer( return kafkaConsumer; } - //////// SETTERS FOR OPTIONAL PARAMETERS + //////// VALIDATION FOR PARAMETERS /** - * Declares a field of the schema to be the processing time attribute. + * Validates a field of the schema to be the processing time attribute. * * @param proctimeAttribute The name of the field that becomes the processing time field. */ - protected void setProctimeAttribute(String proctimeAttribute) { + private String validateProctimeAttribute(String proctimeAttribute) { if (proctimeAttribute != null) { // validate that field exists and is of correct type Option> tpe = schema.getType(proctimeAttribute); @@ -212,15 +294,16 @@ protected void setProctimeAttribute(String proctimeAttribute) { throw new ValidationException("Processing time attribute '" + proctimeAttribute + "' is not of type SQL_TIMESTAMP."); } } - this.proctimeAttribute = proctimeAttribute; + return proctimeAttribute; } /** - * Declares a list of fields to be rowtime attributes. + * Validates a list of fields to be rowtime attributes. * * @param rowtimeAttributeDescriptors The descriptors of the rowtime attributes. */ - protected void setRowtimeAttributeDescriptors(List rowtimeAttributeDescriptors) { + private List validateRowtimeAttributeDescriptors(List rowtimeAttributeDescriptors) { + Preconditions.checkNotNull(rowtimeAttributeDescriptors, "List of rowtime attributes must not be null."); // validate that all declared fields exist and are of correct type for (RowtimeAttributeDescriptor desc : rowtimeAttributeDescriptors) { String rowtimeAttribute = desc.getAttributeName(); @@ -231,25 +314,64 @@ protected void setRowtimeAttributeDescriptors(List r throw new ValidationException("Rowtime attribute '" + rowtimeAttribute + "' is not of type SQL_TIMESTAMP."); } } - this.rowtimeAttributeDescriptors = rowtimeAttributeDescriptors; + return rowtimeAttributeDescriptors; + } + + //////// SETTERS FOR OPTIONAL PARAMETERS + + /** + * Declares a field of the schema to be the processing time attribute. + * + * @param proctimeAttribute The name of the field that becomes the processing time field. + * @deprecated Use table descriptors instead of implementation-specific builders. + */ + @Deprecated + protected void setProctimeAttribute(String proctimeAttribute) { + this.proctimeAttribute = validateProctimeAttribute(proctimeAttribute); + } + + /** + * Declares a list of fields to be rowtime attributes. + * + * @param rowtimeAttributeDescriptors The descriptors of the rowtime attributes. + * @deprecated Use table descriptors instead of implementation-specific builders. + */ + @Deprecated + protected void setRowtimeAttributeDescriptors(List rowtimeAttributeDescriptors) { + this.rowtimeAttributeDescriptors = validateRowtimeAttributeDescriptors(rowtimeAttributeDescriptors); } /** * Sets the startup mode of the TableSource. * * @param startupMode The startup mode. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated protected void setStartupMode(StartupMode startupMode) { - this.startupMode = startupMode; + this.startupMode = Preconditions.checkNotNull(startupMode); } /** * Sets the startup offsets of the TableSource; only relevant when the startup mode is {@link StartupMode#SPECIFIC_OFFSETS}. * * @param specificStartupOffsets The startup offsets for different partitions. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated protected void setSpecificStartupOffsets(Map specificStartupOffsets) { - this.specificStartupOffsets = specificStartupOffsets; + this.specificStartupOffsets = Preconditions.checkNotNull(specificStartupOffsets); + } + + /** + * Mapping for the fields of the table schema to fields of the physical returned type or null. + * + * @param fieldMapping The mapping from table schema fields to format schema fields. + * @deprecated Use table descriptors instead of implementation-specific builders. + */ + @Deprecated + protected void setFieldMapping(Map fieldMapping) { + this.fieldMapping = fieldMapping; } //////// ABSTRACT METHODS FOR SUBCLASSES @@ -267,20 +389,18 @@ protected abstract FlinkKafkaConsumerBase createKafkaConsumer( Properties properties, DeserializationSchema deserializationSchema); - /** - * Returns the deserialization schema. - * - * @return The deserialization schema - */ - protected abstract DeserializationSchema getDeserializationSchema(); - /** * Abstract builder for a {@link KafkaTableSource} to be extended by builders of subclasses of * KafkaTableSource. * * @param Type of the KafkaTableSource produced by the builder. * @param Type of the KafkaTableSource.Builder subclass. + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ + @Deprecated protected abstract static class Builder { private String topic; @@ -299,13 +419,14 @@ protected abstract static class Builder specificStartupOffsets = null; - /** * Sets the topic from which the table is read. * * @param topic The topic from which the table is read. * @return The builder. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public B forTopic(String topic) { Preconditions.checkNotNull(topic, "Topic must not be null."); Preconditions.checkArgument(this.topic == null, "Topic has already been set."); @@ -318,7 +439,9 @@ public B forTopic(String topic) { * * @param props The configuration properties for the Kafka consumer. * @return The builder. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public B withKafkaProperties(Properties props) { Preconditions.checkNotNull(props, "Properties must not be null."); Preconditions.checkArgument(this.kafkaProps == null, "Properties have already been set."); @@ -331,7 +454,9 @@ public B withKafkaProperties(Properties props) { * * @param schema The schema of the produced table. * @return The builder. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public B withSchema(TableSchema schema) { Preconditions.checkNotNull(schema, "Schema must not be null."); Preconditions.checkArgument(this.schema == null, "Schema has already been set."); @@ -345,7 +470,9 @@ public B withSchema(TableSchema schema) { * * @param proctimeAttribute The name of the processing time attribute in the table schema. * @return The builder. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public B withProctimeAttribute(String proctimeAttribute) { Preconditions.checkNotNull(proctimeAttribute, "Proctime attribute must not be null."); Preconditions.checkArgument(!proctimeAttribute.isEmpty(), "Proctime attribute must not be empty."); @@ -362,7 +489,9 @@ public B withProctimeAttribute(String proctimeAttribute) { * @param timestampExtractor The {@link TimestampExtractor} to extract the rowtime attribute from the physical type. * @param watermarkStrategy The {@link WatermarkStrategy} to generate watermarks for the rowtime attribute. * @return The builder. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public B withRowtimeAttribute( String rowtimeAttribute, TimestampExtractor timestampExtractor, @@ -389,7 +518,9 @@ public B withRowtimeAttribute( * @param rowtimeAttribute The name of the rowtime attribute in the table schema. * @param watermarkStrategy The {@link WatermarkStrategy} to generate watermarks for the rowtime attribute. * @return The builder. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public B withKafkaTimestampAsRowtimeAttribute( String rowtimeAttribute, WatermarkStrategy watermarkStrategy) { @@ -412,7 +543,9 @@ public B withKafkaTimestampAsRowtimeAttribute( * Configures the TableSource to start reading from the earliest offset for all partitions. * * @see FlinkKafkaConsumerBase#setStartFromEarliest() + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public B fromEarliest() { this.startupMode = StartupMode.EARLIEST; this.specificStartupOffsets = null; @@ -423,7 +556,9 @@ public B fromEarliest() { * Configures the TableSource to start reading from the latest offset for all partitions. * * @see FlinkKafkaConsumerBase#setStartFromLatest() + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public B fromLatest() { this.startupMode = StartupMode.LATEST; this.specificStartupOffsets = null; @@ -434,7 +569,9 @@ public B fromLatest() { * Configures the TableSource to start reading from any committed group offsets found in Zookeeper / Kafka brokers. * * @see FlinkKafkaConsumerBase#setStartFromGroupOffsets() + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public B fromGroupOffsets() { this.startupMode = StartupMode.GROUP_OFFSETS; this.specificStartupOffsets = null; @@ -446,7 +583,9 @@ public B fromGroupOffsets() { * * @param specificStartupOffsets the specified offsets for partitions * @see FlinkKafkaConsumerBase#setStartFromSpecificOffsets(Map) + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated public B fromSpecificOffsets(Map specificStartupOffsets) { this.startupMode = StartupMode.SPECIFIC_OFFSETS; this.specificStartupOffsets = Preconditions.checkNotNull(specificStartupOffsets); @@ -457,7 +596,9 @@ public B fromSpecificOffsets(Map specificStartupOffse * Returns the configured topic. * * @return the configured topic. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated protected String getTopic() { return this.topic; } @@ -466,7 +607,9 @@ protected String getTopic() { * Returns the configured Kafka properties. * * @return the configured Kafka properties. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated protected Properties getKafkaProps() { return this.kafkaProps; } @@ -475,7 +618,9 @@ protected Properties getKafkaProps() { * Returns the configured table schema. * * @return the configured table schema. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated protected TableSchema getTableSchema() { return this.schema; } @@ -484,14 +629,18 @@ protected TableSchema getTableSchema() { * True if the KafkaSource supports Kafka timestamps, false otherwise. * * @return True if the KafkaSource supports Kafka timestamps, false otherwise. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated protected abstract boolean supportsKafkaTimestamps(); /** * Configures a TableSource with optional parameters. * * @param tableSource The TableSource to configure. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated protected void configureTableSource(T tableSource) { // configure processing time attributes tableSource.setProctimeAttribute(proctimeAttribute); @@ -516,13 +665,20 @@ protected void configureTableSource(T tableSource) { /** * Returns the builder. * @return the builder. + * @deprecated Use the {@link org.apache.flink.table.descriptors.Kafka} descriptor together + * with descriptors for schema and format instead. Descriptors allow for + * implementation-agnostic definition of tables. See also + * {@link org.apache.flink.table.api.TableEnvironment#from(ConnectorDescriptor)}. */ + @Deprecated protected abstract B builder(); /** * Builds the configured {@link KafkaTableSource}. * @return The configured {@link KafkaTableSource}. + * @deprecated Use table descriptors instead of implementation-specific builders. */ + @Deprecated protected abstract KafkaTableSource build(); } } diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java index 6aaef9b8d2b45..3a2a81e5fe62c 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java @@ -18,13 +18,16 @@ package org.apache.flink.streaming.connectors.kafka; +import org.apache.flink.api.common.serialization.DeserializationSchema; +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.descriptors.DescriptorProperties; -import org.apache.flink.table.descriptors.FormatDescriptorValidator; 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.sources.RowtimeAttributeDescriptor; import org.apache.flink.table.sources.TableSource; import org.apache.flink.table.sources.TableSourceFactory; @@ -35,13 +38,13 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_PROPERTY_VERSION; import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE; import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_VERSION; -import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_PROPERTY_VERSION; -import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_TYPE; +import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT; import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_PROPERTIES; import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_PROPERTIES_KEY; import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_PROPERTIES_VALUE; @@ -66,21 +69,16 @@ import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_TYPE; /** - * Factory for creating configured instances of {@link KafkaJsonTableSource}. + * Factory for creating configured instances of {@link KafkaTableSource}. */ -abstract class KafkaTableSourceFactory implements TableSourceFactory { +public abstract class KafkaTableSourceFactory implements TableSourceFactory { @Override public Map requiredContext() { Map context = new HashMap<>(); context.put(CONNECTOR_TYPE(), CONNECTOR_TYPE_VALUE_KAFKA); // kafka context.put(CONNECTOR_VERSION(), kafkaVersion()); // version - - context.put(FORMAT_TYPE(), formatType()); // format - context.put(CONNECTOR_PROPERTY_VERSION(), "1"); // backwards compatibility - context.put(FORMAT_PROPERTY_VERSION(), String.valueOf(formatPropertyVersion())); - return context; } @@ -113,7 +111,8 @@ public List supportedProperties() { properties.add(SCHEMA() + ".#." + ROWTIME_WATERMARKS_SERIALIZED()); properties.add(SCHEMA() + ".#." + ROWTIME_WATERMARKS_DELAY()); - properties.addAll(formatProperties()); + // format wildcard + properties.add(FORMAT() + ".*"); return properties; } @@ -124,110 +123,127 @@ public TableSource create(Map properties) { params.putProperties(properties); // validate - new SchemaValidator(true).validate(params); + // allow Kafka timestamps to be used, watermarks can not be received from source + new SchemaValidator(true, supportsKafkaTimestamps(), false).validate(params); new KafkaValidator().validate(params); - formatValidator().validate(params); - // build - final KafkaTableSource.Builder builder = createBuilderWithFormat(params); + // deserialization schema using format discovery + final DeserializationSchemaFactory formatFactory = TableFormatFactoryService.find( + DeserializationSchemaFactory.class, + properties, + this.getClass().getClassLoader()); + @SuppressWarnings("unchecked") + final DeserializationSchema deserializationSchema = (DeserializationSchema) formatFactory + .createDeserializationSchema(properties); - // topic - final String topic = params.getString(CONNECTOR_TOPIC); - builder.forTopic(topic); + // schema + final TableSchema schema = params.getTableSchema(SCHEMA()); + + // proctime + final String proctimeAttribute = SchemaValidator.deriveProctimeAttribute(params).orElse(null); + + // rowtime + final List rowtimeAttributes = SchemaValidator.deriveRowtimeAttributes(params); + + // field mapping + final Map fieldMapping = SchemaValidator.deriveFieldMapping(params, Optional.of(schema)); // properties - final Properties props = new Properties(); + final Properties kafkaProperties = new Properties(); final List> propsList = params.getFixedIndexedProperties( CONNECTOR_PROPERTIES, Arrays.asList(CONNECTOR_PROPERTIES_KEY, CONNECTOR_PROPERTIES_VALUE)); - propsList.forEach(kv -> props.put( + propsList.forEach(kv -> kafkaProperties.put( params.getString(kv.get(CONNECTOR_PROPERTIES_KEY)), params.getString(kv.get(CONNECTOR_PROPERTIES_VALUE)) )); - builder.withKafkaProperties(props); + + // topic + final String topic = params.getString(CONNECTOR_TOPIC); // startup mode - params + final Map specificOffsets = new HashMap<>(); + final StartupMode startupMode = params .getOptionalString(CONNECTOR_STARTUP_MODE) - .ifPresent(startupMode -> { - switch (startupMode) { - - case KafkaValidator.CONNECTOR_STARTUP_MODE_VALUE_EARLIEST: - builder.fromEarliest(); - break; - - case KafkaValidator.CONNECTOR_STARTUP_MODE_VALUE_LATEST: - builder.fromLatest(); - break; - - case KafkaValidator.CONNECTOR_STARTUP_MODE_VALUE_GROUP_OFFSETS: - builder.fromGroupOffsets(); - break; - - case KafkaValidator.CONNECTOR_STARTUP_MODE_VALUE_SPECIFIC_OFFSETS: - final Map offsetMap = new HashMap<>(); - - final List> offsetList = params.getFixedIndexedProperties( - CONNECTOR_SPECIFIC_OFFSETS, - Arrays.asList(CONNECTOR_SPECIFIC_OFFSETS_PARTITION, CONNECTOR_SPECIFIC_OFFSETS_OFFSET)); - offsetList.forEach(kv -> { - final int partition = params.getInt(kv.get(CONNECTOR_SPECIFIC_OFFSETS_PARTITION)); - final long offset = params.getLong(kv.get(CONNECTOR_SPECIFIC_OFFSETS_OFFSET)); - final KafkaTopicPartition topicPartition = new KafkaTopicPartition(topic, partition); - offsetMap.put(topicPartition, offset); - }); - builder.fromSpecificOffsets(offsetMap); - break; + .map(modeString -> { + switch (modeString) { + case KafkaValidator.CONNECTOR_STARTUP_MODE_VALUE_EARLIEST: + return StartupMode.EARLIEST; + + case KafkaValidator.CONNECTOR_STARTUP_MODE_VALUE_LATEST: + return StartupMode.LATEST; + + case KafkaValidator.CONNECTOR_STARTUP_MODE_VALUE_GROUP_OFFSETS: + return StartupMode.GROUP_OFFSETS; + + case KafkaValidator.CONNECTOR_STARTUP_MODE_VALUE_SPECIFIC_OFFSETS: + final List> offsetList = params.getFixedIndexedProperties( + CONNECTOR_SPECIFIC_OFFSETS, + Arrays.asList(CONNECTOR_SPECIFIC_OFFSETS_PARTITION, CONNECTOR_SPECIFIC_OFFSETS_OFFSET)); + offsetList.forEach(kv -> { + final int partition = params.getInt(kv.get(CONNECTOR_SPECIFIC_OFFSETS_PARTITION)); + final long offset = params.getLong(kv.get(CONNECTOR_SPECIFIC_OFFSETS_OFFSET)); + final KafkaTopicPartition topicPartition = new KafkaTopicPartition(topic, partition); + specificOffsets.put(topicPartition, offset); + }); + return StartupMode.SPECIFIC_OFFSETS; + default: + throw new TableException("Unsupported startup mode. Validator should have checked that."); } - }); - - // schema - final TableSchema schema = params.getTableSchema(SCHEMA()); - builder.withSchema(schema); - - // proctime - SchemaValidator.deriveProctimeAttribute(params).ifPresent(builder::withProctimeAttribute); - - // rowtime - final List descriptors = SchemaValidator.deriveRowtimeAttributes(params); - if (descriptors.size() > 1) { - throw new TableException("More than one rowtime attribute is not supported yet."); - } else if (descriptors.size() == 1) { - final RowtimeAttributeDescriptor desc = descriptors.get(0); - builder.withRowtimeAttribute(desc.getAttributeName(), desc.getTimestampExtractor(), desc.getWatermarkStrategy()); - } - - return builder.build(); + }).orElse(StartupMode.GROUP_OFFSETS); + + return createKafkaTableSource( + schema, + proctimeAttribute, + rowtimeAttributes, + fieldMapping, + topic, + kafkaProperties, + deserializationSchema, + startupMode, + specificOffsets); } - /** - * Returns the format type string (e.g., "json"). - */ - protected abstract String formatType(); + // -------------------------------------------------------------------------------------------- + // For version-specific factories + // -------------------------------------------------------------------------------------------- /** - * Returns the format property version. - */ - protected abstract int formatPropertyVersion(); - - /** - * Returns the properties of the format. - */ - protected abstract List formatProperties(); - - /** - * Returns the validator for the format. + * Returns the Kafka version. */ - protected abstract FormatDescriptorValidator formatValidator(); + protected abstract String kafkaVersion(); /** - * Returns the Kafka version. + * True if the Kafka source supports Kafka timestamps, false otherwise. + * + * @return True if the Kafka source supports Kafka timestamps, false otherwise. */ - protected abstract String kafkaVersion(); + protected abstract boolean supportsKafkaTimestamps(); /** - * Creates a builder with all the format-related configurations have been set. + * Constructs the version-specific Kafka table source. + * + * @param schema Schema of the produced table. + * @param proctimeAttribute Field name of the processing time attribute, null if no + * processing time field is defined. + * @param rowtimeAttributeDescriptors Descriptor for a rowtime attribute + * @param fieldMapping Mapping for the fields of the table schema to + * fields of the physical returned type or null. + * @param topic Kafka topic to consume. + * @param properties Properties for the Kafka consumer. + * @param deserializationSchema Deserialization schema for decoding records from Kafka. + * @param startupMode Startup mode for the contained consumer. + * @param specificStartupOffsets Specific startup offsets; only relevant when startup + * mode is {@link StartupMode#SPECIFIC_OFFSETS}. */ - protected abstract KafkaTableSource.Builder createBuilderWithFormat(DescriptorProperties params); + protected abstract KafkaTableSource createKafkaTableSource( + TableSchema schema, + String proctimeAttribute, + List rowtimeAttributeDescriptors, + Map fieldMapping, + String topic, Properties properties, + DeserializationSchema deserializationSchema, + StartupMode startupMode, + Map specificStartupOffsets); } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSourceFactoryTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSourceFactoryTestBase.java deleted file mode 100644 index 95525590777b3..0000000000000 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSourceFactoryTestBase.java +++ /dev/null @@ -1,123 +0,0 @@ -/* - * 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.streaming.connectors.kafka; - -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.formats.avro.generated.Address; -import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.descriptors.Avro; -import org.apache.flink.table.descriptors.FormatDescriptor; -import org.apache.flink.table.descriptors.Kafka; -import org.apache.flink.table.descriptors.Schema; -import org.apache.flink.table.descriptors.TestTableSourceDescriptor; -import org.apache.flink.table.sources.TableSource; -import org.apache.flink.table.sources.TableSourceFactoryService; - -import org.junit.Test; - -import static org.junit.Assert.assertEquals; - -import java.util.HashMap; -import java.util.Map; -import java.util.Properties; - -/** - * Tests for {@link KafkaAvroTableSourceFactory}. - */ -public abstract class KafkaAvroTableSourceFactoryTestBase { - - private static final String TOPIC = "test-topic"; - - protected abstract String version(); - - protected abstract KafkaAvroTableSource.Builder builder(); - - @Test - public void testTableSourceFromAvroSchema() { - testTableSource(new Avro().recordClass(Address.class)); - } - - private void testTableSource(FormatDescriptor format) { - // construct table source using a builder - - final Map tableAvroMapping = new HashMap<>(); - tableAvroMapping.put("a_street", "street"); - tableAvroMapping.put("street", "street"); - tableAvroMapping.put("b_city", "city"); - tableAvroMapping.put("city", "city"); - tableAvroMapping.put("c_state", "state"); - tableAvroMapping.put("state", "state"); - tableAvroMapping.put("zip", "zip"); - tableAvroMapping.put("num", "num"); - - final Properties props = new Properties(); - props.put("group.id", "test-group"); - props.put("bootstrap.servers", "localhost:1234"); - - final Map specificOffsets = new HashMap<>(); - specificOffsets.put(new KafkaTopicPartition(TOPIC, 0), 100L); - specificOffsets.put(new KafkaTopicPartition(TOPIC, 1), 123L); - - KafkaAvroTableSource.Builder builder = builder(); - - builder.forAvroRecordClass(Address.class) - .withTableToAvroMapping(tableAvroMapping); - - final KafkaTableSource builderSource = builder - .withKafkaProperties(props) - .forTopic(TOPIC) - .fromSpecificOffsets(specificOffsets) - .withSchema( - TableSchema.builder() - .field("a_street", Types.STRING) - .field("b_city", Types.STRING) - .field("c_state", Types.STRING) - .field("zip", Types.STRING) - .field("proctime", Types.SQL_TIMESTAMP) - .build()) - .withProctimeAttribute("proctime") - .build(); - - // construct table source using descriptors and table source factory - - final Map offsets = new HashMap<>(); - offsets.put(0, 100L); - offsets.put(1, 123L); - - final TestTableSourceDescriptor testDesc = new TestTableSourceDescriptor( - new Kafka() - .version(version()) - .topic(TOPIC) - .properties(props) - .startFromSpecificOffsets(offsets)) - .addFormat(format) - .addSchema( - new Schema() - .field("a_street", Types.STRING).from("street") - .field("b_city", Types.STRING).from("city") - .field("c_state", Types.STRING).from("state") - .field("zip", Types.STRING) - .field("proctime", Types.SQL_TIMESTAMP).proctime()); - - final TableSource factorySource = TableSourceFactoryService.findAndCreateTableSource(testDesc); - - assertEquals(builderSource, factorySource); - } -} diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSourceTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSourceTestBase.java index f86fc9542e143..140ce21d9421b 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSourceTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaAvroTableSourceTestBase.java @@ -35,8 +35,12 @@ /** * Abstract test base for all Kafka Avro table sources. + * + * @deprecated Ensures backwards compatibility with Flink 1.5. Can be removed once we + * drop support for format-specific table sources. */ -public abstract class KafkaAvroTableSourceTestBase extends KafkaTableSourceTestBase { +@Deprecated +public abstract class KafkaAvroTableSourceTestBase extends KafkaTableSourceBuilderTestBase { @Override protected void configureBuilder(KafkaTableSource.Builder builder) { diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java index e50810762c698..84c5bd30099d5 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java @@ -19,7 +19,7 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.formats.json.JsonSchemaConverter; +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.descriptors.FormatDescriptor; @@ -31,7 +31,7 @@ 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.PreserveWatermarks; +import org.apache.flink.table.sources.wmstrategies.AscendingTimestamps; import org.junit.Test; @@ -42,8 +42,12 @@ import static org.junit.Assert.assertEquals; /** - * Tests for {@link KafkaJsonTableSourceFactory}. + * Tests for legacy KafkaJsonTableSourceFactory. + * + * @deprecated Ensures backwards compatibility with Flink 1.5. Can be removed once we + * drop support for format-specific table sources. */ +@Deprecated public abstract class KafkaJsonTableSourceFactoryTestBase { private static final String JSON_SCHEMA = @@ -94,10 +98,8 @@ private void testTableSource(FormatDescriptor format) { // construct table source using a builder final Map tableJsonMapping = new HashMap<>(); - tableJsonMapping.put("name", "name"); tableJsonMapping.put("fruit-name", "name"); tableJsonMapping.put("count", "count"); - tableJsonMapping.put("time", "time"); final Properties props = new Properties(); props.put("group.id", "test-group"); @@ -108,7 +110,7 @@ private void testTableSource(FormatDescriptor format) { specificOffsets.put(new KafkaTopicPartition(TOPIC, 1), 123L); final KafkaTableSource builderSource = builder() - .forJsonSchema(TableSchema.fromTypeInfo(JsonSchemaConverter.convert(JSON_SCHEMA))) + .forJsonSchema(TableSchema.fromTypeInfo(JsonRowSchemaConverter.convert(JSON_SCHEMA))) .failOnMissingField(true) .withTableToJsonMapping(tableJsonMapping) .withKafkaProperties(props) @@ -122,7 +124,7 @@ private void testTableSource(FormatDescriptor format) { .field("proc-time", Types.SQL_TIMESTAMP) .build()) .withProctimeAttribute("proc-time") - .withRowtimeAttribute("event-time", new ExistingField("time"), PreserveWatermarks.INSTANCE()) + .withRowtimeAttribute("event-time", new ExistingField("time"), new AscendingTimestamps()) .build(); // construct table source using descriptors and table source factory @@ -143,7 +145,7 @@ private void testTableSource(FormatDescriptor format) { .field("fruit-name", Types.STRING).from("name") .field("count", Types.BIG_DEC) // no from so it must match with the input .field("event-time", Types.SQL_TIMESTAMP).rowtime( - new Rowtime().timestampsFromField("time").watermarksFromSource()) + new Rowtime().timestampsFromField("time").watermarksPeriodicAscending()) .field("proc-time", Types.SQL_TIMESTAMP).proctime()); final TableSource factorySource = TableSourceFactoryService.findAndCreateTableSource(testDesc); diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceTestBase.java index 9b867f6afdbae..4f95b96293690 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceTestBase.java @@ -35,8 +35,12 @@ /** * Abstract test base for all Kafka JSON table sources. + * + * @deprecated Ensures backwards compatibility with Flink 1.5. Can be removed once we + * drop support for format-specific table sources. */ -public abstract class KafkaJsonTableSourceTestBase extends KafkaTableSourceTestBase { +@Deprecated +public abstract class KafkaJsonTableSourceTestBase extends KafkaTableSourceBuilderTestBase { @Test public void testJsonEqualsTableSchema() { diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceBuilderTestBase.java similarity index 96% rename from flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java rename to flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceBuilderTestBase.java index 94281dfb430a9..b9ae47b3f5413 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceBuilderTestBase.java @@ -50,9 +50,13 @@ import static org.mockito.Mockito.when; /** - * Abstract test base for all Kafka table sources. + * Abstract test base for all format-specific Kafka table sources with builders. + * + * @deprecated Ensures backwards compatibility with Flink 1.5. Can be removed once we + * drop support for format-specific table sources. */ -public abstract class KafkaTableSourceTestBase { +@Deprecated +public abstract class KafkaTableSourceBuilderTestBase { static final String[] FIELD_NAMES = new String[]{"field1", "field2", "time1", "time2", "field3"}; diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java new file mode 100644 index 0000000000000..9e1b58f9f78fa --- /dev/null +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java @@ -0,0 +1,187 @@ +/* + * 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.streaming.connectors.kafka; + +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +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.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.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; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * Abstract test base for {@link KafkaTableSourceFactory}. + */ +public abstract class KafkaTableSourceFactoryTestBase extends TestLogger { + + private static final String TOPIC = "myTopic"; + private static final int PARTITION_0 = 0; + private static final long OFFSET_0 = 100L; + private static final int PARTITION_1 = 1; + private static final long OFFSET_1 = 123L; + private static final String FRUIT_NAME = "fruit-name"; + private static final String NAME = "name"; + private static final String COUNT = "count"; + private static final String TIME = "time"; + private static final String EVENT_TIME = "event-time"; + private static final String PROC_TIME = "proc-time"; + private static final Properties KAFKA_PROPERTIES = new Properties(); + static { + KAFKA_PROPERTIES.setProperty("zookeeper.connect", "dummy"); + KAFKA_PROPERTIES.setProperty("group.id", "dummy"); + } + + @Test + @SuppressWarnings("unchecked") + public void testTableSource() { + + // prepare parameters for Kafka table source + + final TableSchema schema = TableSchema.builder() + .field(FRUIT_NAME, Types.STRING()) + .field(COUNT, Types.DECIMAL()) + .field(EVENT_TIME, Types.SQL_TIMESTAMP()) + .field(PROC_TIME, Types.SQL_TIMESTAMP()) + .build(); + + final List rowtimeAttributeDescriptors = Collections.singletonList( + new RowtimeAttributeDescriptor(EVENT_TIME, new ExistingField(TIME), new AscendingTimestamps())); + + final Map fieldMapping = new HashMap<>(); + fieldMapping.put(FRUIT_NAME, NAME); + fieldMapping.put(COUNT, COUNT); + + final Map specificOffsets = new HashMap<>(); + specificOffsets.put(new KafkaTopicPartition(TOPIC, PARTITION_0), OFFSET_0); + specificOffsets.put(new KafkaTopicPartition(TOPIC, PARTITION_1), OFFSET_1); + + final TestDeserializationSchema deserializationSchema = new TestDeserializationSchema( + TableSchema.builder() + .field(NAME, Types.STRING()) + .field(COUNT, Types.DECIMAL()) + .field(TIME, Types.SQL_TIMESTAMP()) + .build() + .toRowType() + ); + + final StartupMode startupMode = StartupMode.SPECIFIC_OFFSETS; + + final KafkaTableSource expected = getExpectedKafkaTableSource( + schema, PROC_TIME, + rowtimeAttributeDescriptors, + fieldMapping, + TOPIC, + KAFKA_PROPERTIES, + deserializationSchema, + startupMode, + specificOffsets); + + // construct table source using descriptors and table source factory + + final Map offsets = new HashMap<>(); + offsets.put(PARTITION_0, OFFSET_0); + offsets.put(PARTITION_1, OFFSET_1); + + final TestTableSourceDescriptor testDesc = new TestTableSourceDescriptor( + new Kafka() + .version(getKafkaVersion()) + .topic(TOPIC) + .properties(KAFKA_PROPERTIES) + .startFromSpecificOffsets(offsets)) + .addFormat(new TestTableFormat()) + .addSchema( + new Schema() + .field(FRUIT_NAME, Types.STRING()).from(NAME) + .field(COUNT, Types.DECIMAL()) // no from so it must match with the input + .field(EVENT_TIME, Types.SQL_TIMESTAMP()).rowtime( + new Rowtime().timestampsFromField(TIME).watermarksPeriodicAscending()) + .field(PROC_TIME, Types.SQL_TIMESTAMP()).proctime()); + + final TableSource actualSource = TableSourceFactoryService.findAndCreateTableSource(testDesc); + + assertEquals(expected, actualSource); + + // test Kafka consumer + final KafkaTableSource actualKafkaSource = (KafkaTableSource) actualSource; + final StreamExecutionEnvironmentMock mock = new StreamExecutionEnvironmentMock(); + actualKafkaSource.getDataStream(mock); + assertTrue(getExpectedFlinkKafkaConsumer().isAssignableFrom(mock.function.getClass())); + } + + private static class StreamExecutionEnvironmentMock extends StreamExecutionEnvironment { + + public SourceFunction function; + + @Override + public DataStreamSource addSource(SourceFunction function) { + this.function = function; + return super.addSource(function); + } + + @Override + public JobExecutionResult execute(String jobName) { + throw new UnsupportedOperationException(); + } + } + + // -------------------------------------------------------------------------------------------- + // For version-specific tests + // -------------------------------------------------------------------------------------------- + + protected abstract String getKafkaVersion(); + + protected abstract Class> getExpectedFlinkKafkaConsumer(); + + protected abstract KafkaTableSource getExpectedKafkaTableSource( + TableSchema schema, + String proctimeAttribute, + List rowtimeAttributeDescriptors, + Map fieldMapping, + String topic, Properties properties, + DeserializationSchema deserializationSchema, + StartupMode startupMode, + Map specificStartupOffsets); +} diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDeserializationSchema.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDeserializationSchema.java index c36a4be13b54e..5e3edf77a5e8a 100644 --- a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDeserializationSchema.java +++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDeserializationSchema.java @@ -60,6 +60,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.TimeZone; /** @@ -175,6 +176,24 @@ public TypeInformation getProducedType() { return typeInfo; } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final AvroRowDeserializationSchema that = (AvroRowDeserializationSchema) o; + return Objects.equals(recordClazz, that.recordClazz) && + Objects.equals(schemaString, that.schemaString); + } + + @Override + public int hashCode() { + return Objects.hash(recordClazz, schemaString); + } + // -------------------------------------------------------------------------------------------- private Row convertAvroRecordToRow(Schema schema, RowTypeInfo typeInfo, IndexedRecord record) { diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowFormatFactory.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowFormatFactory.java new file mode 100644 index 0000000000000..c94f09adc242b --- /dev/null +++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowFormatFactory.java @@ -0,0 +1,99 @@ +/* + * 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.formats.avro; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +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.types.Row; + +import org.apache.avro.specific.SpecificRecord; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Table format factory for providing configured instances of Avro-to-row {@link SerializationSchema} + * and {@link DeserializationSchema}. + */ +public class AvroRowFormatFactory implements SerializationSchemaFactory, DeserializationSchemaFactory { + + @Override + public Map requiredContext() { + final Map context = new HashMap<>(); + context.put(FormatDescriptorValidator.FORMAT_TYPE(), AvroValidator.FORMAT_TYPE_VALUE); + context.put(FormatDescriptorValidator.FORMAT_PROPERTY_VERSION(), "1"); + return context; + } + + @Override + public boolean supportsSchemaDerivation() { + return false; + } + + @Override + public List supportedProperties() { + final List properties = new ArrayList<>(); + properties.add(AvroValidator.FORMAT_RECORD_CLASS); + properties.add(AvroValidator.FORMAT_AVRO_SCHEMA); + return properties; + } + + @Override + public DeserializationSchema createDeserializationSchema(Map properties) { + final DescriptorProperties descriptorProperties = validateAndGetProperties(properties); + + // create and configure + if (descriptorProperties.containsKey(AvroValidator.FORMAT_RECORD_CLASS)) { + return new AvroRowDeserializationSchema( + descriptorProperties.getClass(AvroValidator.FORMAT_RECORD_CLASS, SpecificRecord.class)); + } else { + return new AvroRowDeserializationSchema(descriptorProperties.getString(AvroValidator.FORMAT_AVRO_SCHEMA)); + } + } + + @Override + public SerializationSchema createSerializationSchema(Map properties) { + final DescriptorProperties descriptorProperties = validateAndGetProperties(properties); + + // create and configure + if (descriptorProperties.containsKey(AvroValidator.FORMAT_RECORD_CLASS)) { + return new AvroRowSerializationSchema( + descriptorProperties.getClass(AvroValidator.FORMAT_RECORD_CLASS, SpecificRecord.class)); + } else { + return new AvroRowSerializationSchema(descriptorProperties.getString(AvroValidator.FORMAT_AVRO_SCHEMA)); + } + } + + private static DescriptorProperties validateAndGetProperties(Map propertiesMap) { + final DescriptorProperties descriptorProperties = new DescriptorProperties(true); + descriptorProperties.putProperties(propertiesMap); + + // validate + new AvroValidator().validate(descriptorProperties); + + return descriptorProperties; + } +} diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowSerializationSchema.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowSerializationSchema.java index 80f5f1d2da004..d4c731975d8f0 100644 --- a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowSerializationSchema.java +++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowSerializationSchema.java @@ -51,6 +51,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.TimeZone; /** @@ -147,6 +148,23 @@ public byte[] serialize(Row row) { } } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final AvroRowSerializationSchema that = (AvroRowSerializationSchema) o; + return Objects.equals(recordClazz, that.recordClazz) && Objects.equals(schemaString, that.schemaString); + } + + @Override + public int hashCode() { + return Objects.hash(recordClazz, schemaString); + } + // -------------------------------------------------------------------------------------------- private GenericRecord convertRowToAvroRecord(Schema schema, Row row) { diff --git a/flink-formats/flink-avro/src/main/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory b/flink-formats/flink-avro/src/main/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory new file mode 100644 index 0000000000000..e24424a8dc6bd --- /dev/null +++ b/flink-formats/flink-avro/src/main/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory @@ -0,0 +1,16 @@ +# 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.formats.avro.AvroRowFormatFactory diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroRowFormatFactoryTest.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroRowFormatFactoryTest.java new file mode 100644 index 0000000000000..4f5e21890bd41 --- /dev/null +++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroRowFormatFactoryTest.java @@ -0,0 +1,104 @@ +/* + * 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.formats.avro; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +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.util.TestLogger; + +import org.junit.Test; + +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for the {@link AvroRowFormatFactory}. + */ +public class AvroRowFormatFactoryTest extends TestLogger { + + private static final Class AVRO_SPECIFIC_RECORD = User.class; + + private static final String AVRO_SCHEMA = User.getClassSchema().toString(); + + @Test + public void testRecordClass() { + final Map properties = toMap(new Avro().recordClass(AVRO_SPECIFIC_RECORD)); + + testRecordClassDeserializationSchema(properties); + + testRecordClassSerializationSchema(properties); + } + + @Test + public void testAvroSchema() { + final Map properties = toMap(new Avro().avroSchema(AVRO_SCHEMA)); + + testAvroSchemaSerializationSchema(properties); + + testAvroSchemaDeserializationSchema(properties); + } + + private void testRecordClassSerializationSchema(Map properties) { + final DeserializationSchema actual2 = TableFormatFactoryService + .find(DeserializationSchemaFactory.class, properties) + .createDeserializationSchema(properties); + final AvroRowDeserializationSchema expected2 = new AvroRowDeserializationSchema(AVRO_SPECIFIC_RECORD); + assertEquals(expected2, actual2); + } + + private void testRecordClassDeserializationSchema(Map properties) { + final SerializationSchema actual1 = TableFormatFactoryService + .find(SerializationSchemaFactory.class, properties) + .createSerializationSchema(properties); + final SerializationSchema expected1 = new AvroRowSerializationSchema(AVRO_SPECIFIC_RECORD); + assertEquals(expected1, actual1); + } + + private void testAvroSchemaDeserializationSchema(Map properties) { + final DeserializationSchema actual2 = TableFormatFactoryService + .find(DeserializationSchemaFactory.class, properties) + .createDeserializationSchema(properties); + final AvroRowDeserializationSchema expected2 = new AvroRowDeserializationSchema(AVRO_SCHEMA); + assertEquals(expected2, actual2); + } + + private void testAvroSchemaSerializationSchema(Map properties) { + final SerializationSchema actual1 = TableFormatFactoryService + .find(SerializationSchemaFactory.class, properties) + .createSerializationSchema(properties); + final SerializationSchema expected1 = new AvroRowSerializationSchema(AVRO_SCHEMA); + assertEquals(expected1, actual1); + } + + private static Map toMap(Descriptor... desc) { + final DescriptorProperties descriptorProperties = new DescriptorProperties(true); + for (Descriptor d : desc) { + d.addProperties(descriptorProperties); + } + return descriptorProperties.asMap(); + } +} diff --git a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java index df5285190e55d..dc8a116ac62f2 100644 --- a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java +++ b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowDeserializationSchema.java @@ -38,6 +38,7 @@ import java.sql.Date; import java.sql.Time; import java.sql.Timestamp; +import java.util.Objects; /** * Deserialization schema from JSON to Flink types. @@ -84,7 +85,7 @@ public JsonRowDeserializationSchema(TypeInformation typeInfo) { * @see http://json-schema.org/ */ public JsonRowDeserializationSchema(String jsonSchema) { - this(JsonSchemaConverter.convert(jsonSchema)); + this(JsonRowSchemaConverter.convert(jsonSchema)); } @Override @@ -118,6 +119,23 @@ public void setFailOnMissingField(boolean failOnMissingField) { this.failOnMissingField = failOnMissingField; } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final JsonRowDeserializationSchema that = (JsonRowDeserializationSchema) o; + return failOnMissingField == that.failOnMissingField && Objects.equals(typeInfo, that.typeInfo); + } + + @Override + public int hashCode() { + return Objects.hash(typeInfo, failOnMissingField); + } + // -------------------------------------------------------------------------------------------- private Object convert(JsonNode node, TypeInformation info) { diff --git a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java new file mode 100644 index 0000000000000..fd7bda6c7de0c --- /dev/null +++ b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java @@ -0,0 +1,108 @@ +/* + * 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.formats.json; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +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.types.Row; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Table format factory for providing configured instances of JSON-to-row {@link SerializationSchema} + * and {@link DeserializationSchema}. + */ +public class JsonRowFormatFactory implements SerializationSchemaFactory, DeserializationSchemaFactory { + + @Override + public Map requiredContext() { + final Map context = new HashMap<>(); + context.put(FormatDescriptorValidator.FORMAT_TYPE(), JsonValidator.FORMAT_TYPE_VALUE); + context.put(FormatDescriptorValidator.FORMAT_PROPERTY_VERSION(), "1"); + return context; + } + + @Override + public boolean supportsSchemaDerivation() { + return true; + } + + @Override + public List supportedProperties() { + final List properties = new ArrayList<>(); + properties.add(JsonValidator.FORMAT_JSON_SCHEMA); + properties.add(JsonValidator.FORMAT_SCHEMA); + properties.add(JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD); + properties.add(FormatDescriptorValidator.FORMAT_DERIVE_SCHEMA()); + properties.addAll(SchemaValidator.getSchemaDerivationKeys()); + return properties; + } + + @Override + public DeserializationSchema createDeserializationSchema(Map properties) { + final DescriptorProperties descriptorProperties = validateAndGetProperties(properties); + + // create and configure + final JsonRowDeserializationSchema schema = new JsonRowDeserializationSchema(createTypeInformation(descriptorProperties)); + + descriptorProperties.getOptionalBoolean(JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD) + .ifPresent(schema::setFailOnMissingField); + + return schema; + } + + @Override + public SerializationSchema createSerializationSchema(Map properties) { + final DescriptorProperties descriptorProperties = validateAndGetProperties(properties); + + // create and configure + return new JsonRowSerializationSchema(createTypeInformation(descriptorProperties)); + } + + private static DescriptorProperties validateAndGetProperties(Map propertiesMap) { + final DescriptorProperties descriptorProperties = new DescriptorProperties(true); + descriptorProperties.putProperties(propertiesMap); + + // validate + new JsonValidator().validate(descriptorProperties); + + return descriptorProperties; + } + + private static TypeInformation createTypeInformation(DescriptorProperties descriptorProperties) { + if (descriptorProperties.containsKey(JsonValidator.FORMAT_SCHEMA)) { + return (RowTypeInfo) descriptorProperties.getType(JsonValidator.FORMAT_SCHEMA); + } else if (descriptorProperties.containsKey(JsonValidator.FORMAT_JSON_SCHEMA)) { + return JsonRowSchemaConverter.convert(descriptorProperties.getString(JsonValidator.FORMAT_JSON_SCHEMA)); + } else { + return SchemaValidator.deriveFormatFields(descriptorProperties).toRowType(); + } + } +} diff --git a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonSchemaConverter.java b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowSchemaConverter.java similarity index 99% rename from flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonSchemaConverter.java rename to flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowSchemaConverter.java index 7a001f63753e3..320ca1f8c4d5d 100644 --- a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonSchemaConverter.java +++ b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowSchemaConverter.java @@ -49,9 +49,9 @@ *

This converter has been developed for JSON Schema draft-07 but also includes keywords of * older drafts to be as compatible as possible. */ -public final class JsonSchemaConverter { +public final class JsonRowSchemaConverter { - private JsonSchemaConverter() { + private JsonRowSchemaConverter() { // private } diff --git a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java index 8fee6a4d944e3..d9420629fea85 100644 --- a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java +++ b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowSerializationSchema.java @@ -39,6 +39,7 @@ import java.sql.Time; import java.sql.Timestamp; import java.text.SimpleDateFormat; +import java.util.Objects; /** * Serialization schema that serializes an object of Flink types into a JSON bytes. @@ -89,7 +90,7 @@ public JsonRowSerializationSchema(TypeInformation typeInfo) { * @see http://json-schema.org/ */ public JsonRowSerializationSchema(String jsonSchema) { - this(JsonSchemaConverter.convert(jsonSchema)); + this(JsonRowSchemaConverter.convert(jsonSchema)); } @Override @@ -107,6 +108,23 @@ public byte[] serialize(Row row) { } } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final JsonRowSerializationSchema that = (JsonRowSerializationSchema) o; + return Objects.equals(typeInfo, that.typeInfo); + } + + @Override + public int hashCode() { + return Objects.hash(typeInfo); + } + // -------------------------------------------------------------------------------------------- private ObjectNode convertRow(ObjectNode reuse, RowTypeInfo info, Row row) { diff --git a/flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/Json.java b/flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/Json.java index 9c121916d5416..035f05f7cd540 100644 --- a/flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/Json.java +++ b/flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/Json.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.table.typeutils.TypeStringUtils; +import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_DERIVE_SCHEMA; @@ -81,7 +82,7 @@ public Json jsonSchema(String jsonSchema) { * * @param schemaType type information that describes the schema */ - public Json schema(TypeInformation schemaType) { + public Json schema(TypeInformation schemaType) { Preconditions.checkNotNull(schemaType); this.schema = TypeStringUtils.writeTypeInfo(schemaType); this.jsonSchema = null; diff --git a/flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java b/flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java index fea7cf55b5d33..49e1abc8a1995 100644 --- a/flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java +++ b/flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java @@ -45,7 +45,7 @@ public void validate(DescriptorProperties properties) { } else if (!deriveSchema && !hasSchema && !hasSchemaString) { throw new ValidationException("A definition of a schema or JSON schema is required."); } else if (hasSchema) { - properties.validateType(FORMAT_SCHEMA, false); + properties.validateType(FORMAT_SCHEMA, true, false); } else if (hasSchemaString) { properties.validateString(FORMAT_JSON_SCHEMA, false, 1); } diff --git a/flink-formats/flink-json/src/main/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory b/flink-formats/flink-json/src/main/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory new file mode 100644 index 0000000000000..aec584688eb02 --- /dev/null +++ b/flink-formats/flink-json/src/main/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory @@ -0,0 +1,16 @@ +# 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.formats.json.JsonRowFormatFactory diff --git a/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java new file mode 100644 index 0000000000000..d763b9047139c --- /dev/null +++ b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java @@ -0,0 +1,150 @@ +/* + * 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.formats.json; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.api.Types; +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.types.Row; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for the {@link JsonRowFormatFactory}. + */ +public class JsonRowFormatFactoryTest extends TestLogger { + + private static final String JSON_SCHEMA = + "{" + + " 'title': 'Fruit'," + + " 'type': 'object'," + + " 'properties': {" + + " 'name': {" + + " 'type': 'string'" + + " }," + + " 'count': {" + + " 'type': 'integer'" + + " }," + + " 'time': {" + + " 'description': 'row time'," + + " 'type': 'string'," + + " 'format': 'date-time'" + + " }" + + " }," + + " 'required': ['name', 'count', 'time']" + + "}"; + + private static final TypeInformation SCHEMA = Types.ROW( + new String[]{"field1", "field2"}, + new TypeInformation[]{Types.BOOLEAN(), Types.INT()}); + + @Test + public void testSchema() { + final Map properties = toMap( + new Json() + .schema(SCHEMA) + .failOnMissingField(false)); + + testSchemaSerializationSchema(properties); + + testSchemaDeserializationSchema(properties); + } + + @Test + public void testJsonSchema() { + final Map properties = toMap( + new Json() + .jsonSchema(JSON_SCHEMA) + .failOnMissingField(true)); + + testJsonSchemaSerializationSchema(properties); + + testJsonSchemaDeserializationSchema(properties); + } + + @Test + public void testSchemaDerivation() { + final Map properties = toMap( + new Schema() + .field("field1", Types.BOOLEAN()) + .field("field2", Types.INT()) + .field("proctime", Types.SQL_TIMESTAMP()).proctime(), + new Json() + .deriveSchema()); + + testSchemaSerializationSchema(properties); + + testSchemaDeserializationSchema(properties); + } + + private void testSchemaDeserializationSchema(Map properties) { + final DeserializationSchema actual2 = TableFormatFactoryService + .find(DeserializationSchemaFactory.class, properties) + .createDeserializationSchema(properties); + final JsonRowDeserializationSchema expected2 = new JsonRowDeserializationSchema(SCHEMA); + expected2.setFailOnMissingField(false); + assertEquals(expected2, actual2); + } + + private void testSchemaSerializationSchema(Map properties) { + final SerializationSchema actual1 = TableFormatFactoryService + .find(SerializationSchemaFactory.class, properties) + .createSerializationSchema(properties); + final SerializationSchema expected1 = new JsonRowSerializationSchema(SCHEMA); + assertEquals(expected1, actual1); + } + + private void testJsonSchemaDeserializationSchema(Map properties) { + final DeserializationSchema actual2 = TableFormatFactoryService + .find(DeserializationSchemaFactory.class, properties) + .createDeserializationSchema(properties); + final JsonRowDeserializationSchema expected2 = new JsonRowDeserializationSchema(JSON_SCHEMA); + expected2.setFailOnMissingField(true); + assertEquals(expected2, actual2); + } + + private void testJsonSchemaSerializationSchema(Map properties) { + final SerializationSchema actual1 = TableFormatFactoryService + .find(SerializationSchemaFactory.class, properties) + .createSerializationSchema(properties); + final SerializationSchema expected1 = new JsonRowSerializationSchema(JSON_SCHEMA); + assertEquals(expected1, actual1); + } + + private static Map toMap(Descriptor... desc) { + final DescriptorProperties descriptorProperties = new DescriptorProperties(true); + for (Descriptor d : desc) { + d.addProperties(descriptorProperties); + } + return descriptorProperties.asMap(); + } +} diff --git a/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonSchemaConverterTest.java b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowSchemaConverterTest.java similarity index 80% rename from flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonSchemaConverterTest.java rename to flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowSchemaConverterTest.java index 7cf3b7c1ea0c3..1af45f4e89e0f 100644 --- a/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonSchemaConverterTest.java +++ b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowSchemaConverterTest.java @@ -31,16 +31,16 @@ import static org.junit.Assert.assertEquals; /** - * Tests for {@link JsonSchemaConverter}. + * Tests for {@link JsonRowSchemaConverter}. */ -public class JsonSchemaConverterTest { +public class JsonRowSchemaConverterTest { @Test public void testComplexSchema() throws Exception { final URL url = getClass().getClassLoader().getResource("complex-schema.json"); Objects.requireNonNull(url); final String schema = FileUtils.readFileUtf8(new File(url.getFile())); - final TypeInformation result = JsonSchemaConverter.convert(schema); + final TypeInformation result = JsonRowSchemaConverter.convert(schema); final TypeInformation expected = Types.ROW_NAMED( new String[] {"fn", "familyName", "additionalName", "tuples", "honorificPrefix", "url", @@ -58,7 +58,7 @@ public void testReferenceSchema() throws Exception { final URL url = getClass().getClassLoader().getResource("reference-schema.json"); Objects.requireNonNull(url); final String schema = FileUtils.readFileUtf8(new File(url.getFile())); - final TypeInformation result = JsonSchemaConverter.convert(schema); + final TypeInformation result = JsonRowSchemaConverter.convert(schema); final TypeInformation expected = Types.ROW_NAMED( new String[] {"billing_address", "shipping_address", "optional_address"}, @@ -71,43 +71,43 @@ public void testReferenceSchema() throws Exception { @Test public void testAtomicType() { - final TypeInformation result = JsonSchemaConverter.convert("{ type: 'number' }"); + final TypeInformation result = JsonRowSchemaConverter.convert("{ type: 'number' }"); assertEquals(Types.BIG_DEC, result); } @Test(expected = IllegalArgumentException.class) public void testMissingType() { - JsonSchemaConverter.convert("{ }"); + JsonRowSchemaConverter.convert("{ }"); } @Test(expected = IllegalArgumentException.class) public void testWrongType() { - JsonSchemaConverter.convert("{ type: 'whatever' }"); + JsonRowSchemaConverter.convert("{ type: 'whatever' }"); } @Test(expected = IllegalArgumentException.class) public void testArrayWithAdditionalItems() { - JsonSchemaConverter.convert("{ type: 'array', items: [{type: 'integer'}], additionalItems: true }"); + JsonRowSchemaConverter.convert("{ type: 'array', items: [{type: 'integer'}], additionalItems: true }"); } @Test public void testMissingProperties() { - final TypeInformation result = JsonSchemaConverter.convert("{ type: 'object' }"); + final TypeInformation result = JsonRowSchemaConverter.convert("{ type: 'object' }"); assertEquals(Types.ROW(), result); } @Test public void testNullUnionTypes() { - final TypeInformation result = JsonSchemaConverter.convert("{ type: ['string', 'null'] }"); + final TypeInformation result = JsonRowSchemaConverter.convert("{ type: ['string', 'null'] }"); assertEquals(Types.STRING, result); } @Test public void testTimestamp() { - final TypeInformation result = JsonSchemaConverter.convert("{ type: 'string', format: 'date-time' }"); + final TypeInformation result = JsonRowSchemaConverter.convert("{ type: 'string', format: 'date-time' }"); assertEquals(Types.SQL_TIMESTAMP, result); } diff --git a/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java index 94a05b3d7c3cc..e2410d4a7ba08 100644 --- a/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java +++ b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowSerializationSchemaTest.java @@ -111,7 +111,7 @@ public void testSerializeRowWithInvalidNumberOfFields() { @Test public void testSchema() throws IOException { - final TypeInformation rowSchema = JsonSchemaConverter.convert( + final TypeInformation rowSchema = JsonRowSchemaConverter.convert( "{" + " type: 'object'," + " properties: {" + diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala index 05255fd5b568b..043a345257efc 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala @@ -31,6 +31,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.io.DiscardingOutputFormat import org.apache.flink.api.java.typeutils.GenericTypeInfo import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} +import org.apache.flink.table.descriptors.{BatchTableSourceDescriptor, ConnectorDescriptor} import org.apache.flink.table.explain.PlanJsonParser import org.apache.flink.table.expressions.{Expression, TimeAttribute} import org.apache.flink.table.plan.nodes.FlinkConventions @@ -110,16 +111,44 @@ abstract class BatchTableEnvironment( } } -// TODO expose this once we have enough table source factories that can deal with it -// /** -// * Creates a table from a descriptor that describes the source connector, source encoding, -// * the resulting table schema, and other properties. -// * -// * @param connectorDescriptor connector descriptor describing the source of the table -// */ -// def from(connectorDescriptor: ConnectorDescriptor): BatchTableSourceDescriptor = { -// new BatchTableSourceDescriptor(this, connectorDescriptor) -// } + /** + * Creates a table from a descriptor that describes the source connector, the source format, + * the resulting table schema, and other properties. + * + * Descriptors allow for declaring communication to external systems in an + * implementation-agnostic way. The classpath is scanned for connectors and matching connectors + * are configured accordingly. + * + * The following example shows how to read from a Kafka connector using a JSON format and + * creating a table: + * + * {{{ + * + * tableEnv + * .from( + * new Kafka() + * .version("0.11") + * .topic("clicks") + * .property("zookeeper.connect", "localhost") + * .property("group.id", "click-group") + * .startFromEarliest()) + * .withFormat( + * new Json() + * .jsonSchema("{...}") + * .failOnMissingField(false)) + * .withSchema( + * new Schema() + * .field("user-name", "VARCHAR").from("u_name") + * .field("count", "DECIMAL") + * .field("proc-time", "TIMESTAMP").proctime()) + * .toTable() + * }}} + * + * @param connectorDescriptor connector descriptor describing the source of the table + */ + def from(connectorDescriptor: ConnectorDescriptor): BatchTableSourceDescriptor = { + new BatchTableSourceDescriptor(this, connectorDescriptor) + } /** * Registers an external [[TableSink]] with given field names and types in this diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala index 84d724065fa34..510fe0d7cf9d8 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala @@ -37,6 +37,7 @@ import org.apache.flink.streaming.api.TimeCharacteristic import org.apache.flink.streaming.api.datastream.DataStream import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.table.calcite.{FlinkTypeFactory, RelTimeIndicatorConverter} +import org.apache.flink.table.descriptors.{ConnectorDescriptor, StreamTableSourceDescriptor} import org.apache.flink.table.explain.PlanJsonParser import org.apache.flink.table.expressions._ import org.apache.flink.table.plan.nodes.FlinkConventions @@ -128,16 +129,44 @@ abstract class StreamTableEnvironment( } } -// TODO expose this once we have enough table source factories that can deal with it -// /** -// * Creates a table from a descriptor that describes the source connector, source encoding, -// * the resulting table schema, and other properties. -// * -// * @param connectorDescriptor connector descriptor describing the source of the table -// */ -// def from(connectorDescriptor: ConnectorDescriptor): StreamTableSourceDescriptor = { -// new StreamTableSourceDescriptor(this, connectorDescriptor) -// } + /** + * Creates a table from a descriptor that describes the source connector, the source format, + * the resulting table schema, and other properties. + * + * Descriptors allow for declaring communication to external systems in an + * implementation-agnostic way. The classpath is scanned for connectors and matching connectors + * are configured accordingly. + * + * The following example shows how to read from a Kafka connector using a JSON format and + * creating a table: + * + * {{{ + * + * tableEnv + * .from( + * new Kafka() + * .version("0.11") + * .topic("clicks") + * .property("zookeeper.connect", "localhost") + * .property("group.id", "click-group") + * .startFromEarliest()) + * .withFormat( + * new Json() + * .jsonSchema("{...}") + * .failOnMissingField(false)) + * .withSchema( + * new Schema() + * .field("user-name", "VARCHAR").from("u_name") + * .field("count", "DECIMAL") + * .field("proc-time", "TIMESTAMP").proctime()) + * .toTable() + * }}} + * + * @param connectorDescriptor connector descriptor describing the source of the table + */ + def from(connectorDescriptor: ConnectorDescriptor): StreamTableSourceDescriptor = { + new StreamTableSourceDescriptor(this, connectorDescriptor) + } /** * Registers an external [[TableSink]] with given field names and types in this diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala index d6106bec69bd9..88dc1e922b1ec 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala @@ -49,6 +49,7 @@ import org.apache.flink.table.api.scala.{BatchTableEnvironment => ScalaBatchTabl import org.apache.flink.table.calcite.{FlinkPlannerImpl, FlinkRelBuilder, FlinkTypeFactory, FlinkTypeSystem} import org.apache.flink.table.catalog.{ExternalCatalog, ExternalCatalogSchema} import org.apache.flink.table.codegen.{ExpressionReducer, FunctionCodeGenerator, GeneratedFunction} +import org.apache.flink.table.descriptors.{ConnectorDescriptor, TableSourceDescriptor} import org.apache.flink.table.expressions._ import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils._ import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, TableFunction} @@ -511,6 +512,43 @@ abstract class TableEnvironment(val config: TableConfig) { } } + /** + * Creates a table from a descriptor that describes the source connector, the source format, + * the resulting table schema, and other properties. + * + * Descriptors allow for declaring communication to external systems in an + * implementation-agnostic way. The classpath is scanned for connectors and matching connectors + * are configured accordingly. + * + * The following example shows how to read from a Kafka connector using a JSON format and + * creating table: + * + * {{{ + * + * tableEnv + * .from( + * new Kafka() + * .version("0.11") + * .topic("clicks") + * .property("zookeeper.connect", "localhost") + * .property("group.id", "click-group") + * .startFromEarliest()) + * .withFormat( + * new Json() + * .jsonSchema("{...}") + * .failOnMissingField(false)) + * .withSchema( + * new Schema() + * .field("user-name", "VARCHAR").from("u_name") + * .field("count", "DECIMAL") + * .field("proc-time", "TIMESTAMP").proctime()) + * .toTable() + * }}} + * + * @param connectorDescriptor connector descriptor describing the source of the table + */ + def from(connectorDescriptor: ConnectorDescriptor): TableSourceDescriptor + private[flink] def scanInternal(tablePath: Array[String]): Option[Table] = { require(tablePath != null && !tablePath.isEmpty, "tablePath must not be null or empty.") val schemaPaths = tablePath.slice(0, tablePath.length - 1) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableSchema.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableSchema.scala index 6389b55b12509..bcee5acd3a612 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableSchema.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableSchema.scala @@ -24,6 +24,7 @@ import _root_.scala.collection.mutable.ArrayBuffer import _root_.java.util.Objects import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.types.Row /** * A TableSchema represents a Table's structure. @@ -135,6 +136,15 @@ class TableSchema( new TableSchema(columnNames, converted) } + /** + * Converts a table schema into a (nested) type information describing a [[Row]]. + * + * @return type information where columns are fields of a row + */ + def toRowType: TypeInformation[Row] = { + Types.ROW(getColumnNames, getTypes) + } + override def toString: String = { val builder = new StringBuilder builder.append("root\n") diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala index eb87c9dca6b2e..e266a47ef392a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala @@ -19,6 +19,7 @@ package org.apache.flink.table.api import org.apache.flink.table.descriptors.DescriptorProperties +import org.apache.flink.table.formats.TableFormatFactory /** * Exception for all errors occurring during expression parsing. @@ -141,6 +142,85 @@ case class CatalogAlreadyExistException( def this(catalog: String) = this(catalog, null) } +/** + * Exception for not finding a [[org.apache.flink.table.formats.TableFormatFactory]] 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 cause the cause + */ +case class NoMatchingTableFormatException( + message: String, + factoryClass: Class[_], + formatFactories: Seq[TableFormatFactory[_]], + properties: Map[String, String], + cause: Throwable) + extends RuntimeException( + s"""Could not find a suitable table format factory for '${factoryClass.getName}' in + |the classpath. + | + |Reason: $message + | + |The following properties are requested: + |${DescriptorProperties.toString(properties)} + | + |The following format factories have been considered: + |${formatFactories.map(_.getClass.getName).mkString("\n")} + |""".stripMargin, + cause) { + + def this( + message: String, + factoryClass: Class[_], + formatFactories: Seq[TableFormatFactory[_]], + properties: Map[String, String]) = { + this(message, factoryClass, formatFactories, properties, null) + } +} + +/** + * Exception for finding more than one [[org.apache.flink.table.formats.TableFormatFactory]] for + * the given properties. + * + * @param matchingFormatFactories format factories that match the properties + * @param factoryClass required factory class + * @param formatFactories all found factories + * @param properties properties that describe the table format + * @param cause the cause + */ +case class AmbiguousTableFormatException( + matchingFormatFactories: Seq[TableFormatFactory[_]], + factoryClass: Class[_], + formatFactories: Seq[TableFormatFactory[_]], + properties: Map[String, String], + cause: Throwable) + extends RuntimeException( + s"""More than one suitable table format factory for '${factoryClass.getName}' could + |be found in the classpath. + | + |The following format factories match: + |${matchingFormatFactories.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")} + |""".stripMargin, + cause) { + + def this( + matchingFormatFactories: Seq[TableFormatFactory[_]], + factoryClass: Class[_], + formatFactories: Seq[TableFormatFactory[_]], + properties: Map[String, String]) = { + this(matchingFormatFactories, factoryClass, formatFactories, properties, null) + } +} + /** * Exception for not finding a [[org.apache.flink.table.sources.TableSourceFactory]] for the * given properties. diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/DescriptorProperties.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/DescriptorProperties.scala index 4f5bd81d2ef12..812b78a0a239a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/DescriptorProperties.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/DescriptorProperties.scala @@ -30,6 +30,7 @@ import org.apache.commons.codec.binary.Base64 import org.apache.commons.lang.StringEscapeUtils import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} +import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.table.api.{TableException, TableSchema, ValidationException} import org.apache.flink.table.descriptors.DescriptorProperties.{NAME, TYPE, normalizeTableSchema, toJava} import org.apache.flink.table.typeutils.TypeStringUtils @@ -950,7 +951,7 @@ class DescriptorProperties(normalizeKeys: Boolean = true) { validateString(prefix + NAME, isOptional = false, minLen = 1) } val typeValidation = (prefix: String) => { - validateType(prefix + TYPE, isOptional = false) + validateType(prefix + TYPE, requireRow = false, isOptional = false) } validateFixedIndexedProperties( @@ -998,13 +999,19 @@ class DescriptorProperties(normalizeKeys: Boolean = true) { /** * Validates a type property. */ - def validateType(key: String, isOptional: Boolean): Unit = { + def validateType(key: String, requireRow: Boolean, isOptional: Boolean): Unit = { if (!properties.contains(key)) { if (!isOptional) { throw new ValidationException(s"Could not find required property '$key'.") } } else { - TypeStringUtils.readTypeInfo(properties(key)) // throws validation exceptions + // we don't validate the string but let the parser do the work for us + // it throws a validation exception + val info = TypeStringUtils.readTypeInfo(properties(key)) + if (requireRow && !info.isInstanceOf[RowTypeInfo]) { + throw new ValidationException( + s"Row type information expected for '$key' but was: ${properties(key)}") + } } } @@ -1079,7 +1086,7 @@ class DescriptorProperties(normalizeKeys: Boolean = true) { */ private def put(key: String, value: String): Unit = { if (properties.contains(key)) { - throw new IllegalStateException("Property already present.") + throw new IllegalStateException("Property already present:" + key) } if (normalizeKeys) { properties.put(key.toLowerCase, value) @@ -1263,7 +1270,7 @@ object DescriptorProperties { } def toString(keyOrValue: String): String = { - StringEscapeUtils.escapeJava(keyOrValue) + StringEscapeUtils.escapeJava(keyOrValue).replace("\\/", "/") // '/' must not be escaped } def toString(key: String, value: String): String = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FormatDescriptorValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FormatDescriptorValidator.scala index 301189a17966b..3d44c2424c5a1 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FormatDescriptorValidator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FormatDescriptorValidator.scala @@ -33,6 +33,11 @@ class FormatDescriptorValidator extends DescriptorValidator { object FormatDescriptorValidator { + /** + * Prefix for format-related properties. + */ + val FORMAT = "format" + /** * Key for describing the type of the format. Usually used for factory discovery. */ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/RowtimeValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/RowtimeValidator.scala index fdec82008aee9..160347ec420eb 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/RowtimeValidator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/RowtimeValidator.scala @@ -28,7 +28,11 @@ import scala.collection.JavaConverters._ /** * Validator for [[Rowtime]]. */ -class RowtimeValidator(val prefix: String = "") extends DescriptorValidator { +class RowtimeValidator( + supportsSourceTimestamps: Boolean, + supportsSourceWatermarks: Boolean, + prefix: String = "") + extends DescriptorValidator { override def validate(properties: DescriptorProperties): Unit = { val timestampExistingField = (_: String) => { @@ -43,14 +47,21 @@ class RowtimeValidator(val prefix: String = "") extends DescriptorValidator { prefix + ROWTIME_TIMESTAMPS_SERIALIZED, isOptional = false, minLen = 1) } - properties.validateEnum( - prefix + ROWTIME_TIMESTAMPS_TYPE, - isOptional = false, + val timestampsValidation = if (supportsSourceTimestamps) { Map( ROWTIME_TIMESTAMPS_TYPE_VALUE_FROM_FIELD -> toJava(timestampExistingField), ROWTIME_TIMESTAMPS_TYPE_VALUE_FROM_SOURCE -> properties.noValidation(), - ROWTIME_TIMESTAMPS_TYPE_VALUE_CUSTOM -> toJava(timestampCustom) - ).asJava + ROWTIME_TIMESTAMPS_TYPE_VALUE_CUSTOM -> toJava(timestampCustom)) + } else { + Map( + ROWTIME_TIMESTAMPS_TYPE_VALUE_FROM_FIELD -> toJava(timestampExistingField), + ROWTIME_TIMESTAMPS_TYPE_VALUE_CUSTOM -> toJava(timestampCustom)) + } + + properties.validateEnum( + prefix + ROWTIME_TIMESTAMPS_TYPE, + isOptional = false, + timestampsValidation.asJava ) val watermarkPeriodicBounded = (_: String) => { @@ -65,15 +76,23 @@ class RowtimeValidator(val prefix: String = "") extends DescriptorValidator { prefix + ROWTIME_WATERMARKS_SERIALIZED, isOptional = false, minLen = 1) } - properties.validateEnum( - prefix + ROWTIME_WATERMARKS_TYPE, - isOptional = false, + val watermarksValidation = if (supportsSourceWatermarks) { Map( ROWTIME_WATERMARKS_TYPE_VALUE_PERIODIC_ASCENDING -> properties.noValidation(), ROWTIME_WATERMARKS_TYPE_VALUE_PERIODIC_BOUNDED -> toJava(watermarkPeriodicBounded), ROWTIME_WATERMARKS_TYPE_VALUE_FROM_SOURCE -> properties.noValidation(), - ROWTIME_WATERMARKS_TYPE_VALUE_CUSTOM -> toJava(watermarkCustom) - ).asJava + ROWTIME_WATERMARKS_TYPE_VALUE_CUSTOM -> toJava(watermarkCustom)) + } else { + Map( + ROWTIME_WATERMARKS_TYPE_VALUE_PERIODIC_ASCENDING -> properties.noValidation(), + ROWTIME_WATERMARKS_TYPE_VALUE_PERIODIC_BOUNDED -> toJava(watermarkPeriodicBounded), + ROWTIME_WATERMARKS_TYPE_VALUE_CUSTOM -> toJava(watermarkCustom)) + } + + properties.validateEnum( + prefix + ROWTIME_WATERMARKS_TYPE, + isOptional = false, + watermarksValidation.asJava ) } } @@ -154,7 +173,7 @@ object RowtimeValidator { new ExistingField(field) case ROWTIME_TIMESTAMPS_TYPE_VALUE_FROM_SOURCE => - new StreamRecordTimestamp + StreamRecordTimestamp.INSTANCE case ROWTIME_TIMESTAMPS_TYPE_VALUE_CUSTOM => val clazz = properties.getClass( diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala index 9cb3258d68aba..64b7c0be5c8af 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala @@ -23,7 +23,7 @@ import java.util.Optional import org.apache.flink.table.api.{TableSchema, ValidationException} import org.apache.flink.table.descriptors.DescriptorProperties.{toJava, toScala} -import org.apache.flink.table.descriptors.RowtimeValidator.{ROWTIME, ROWTIME_TIMESTAMPS_FROM, ROWTIME_TIMESTAMPS_TYPE, ROWTIME_TIMESTAMPS_TYPE_VALUE_FROM_FIELD} +import org.apache.flink.table.descriptors.RowtimeValidator._ import org.apache.flink.table.descriptors.SchemaValidator._ import org.apache.flink.table.sources.RowtimeAttributeDescriptor @@ -33,7 +33,11 @@ import scala.collection.mutable /** * Validator for [[Schema]]. */ -class SchemaValidator(isStreamEnvironment: Boolean = true) extends DescriptorValidator { +class SchemaValidator( + isStreamEnvironment: Boolean, + supportsSourceTimestamps: Boolean, + supportsSourceWatermarks: Boolean) + extends DescriptorValidator { override def validate(properties: DescriptorProperties): Unit = { val names = properties.getIndexedProperty(SCHEMA, SCHEMA_NAME) @@ -50,7 +54,7 @@ class SchemaValidator(isStreamEnvironment: Boolean = true) extends DescriptorVal properties .validateString(s"$SCHEMA.$i.$SCHEMA_NAME", isOptional = false, minLen = 1) properties - .validateType(s"$SCHEMA.$i.$SCHEMA_TYPE", isOptional = false) + .validateType(s"$SCHEMA.$i.$SCHEMA_TYPE", requireRow = false, isOptional = false) properties .validateString(s"$SCHEMA.$i.$SCHEMA_FROM", isOptional = true, minLen = 1) // either proctime or rowtime @@ -73,7 +77,10 @@ class SchemaValidator(isStreamEnvironment: Boolean = true) extends DescriptorVal properties.validatePrefixExclusion(rowtime) } else if (properties.hasPrefix(rowtime)) { // check rowtime - val rowtimeValidator = new RowtimeValidator(s"$SCHEMA.$i.") + val rowtimeValidator = new RowtimeValidator( + supportsSourceTimestamps, + supportsSourceWatermarks, + s"$SCHEMA.$i.") rowtimeValidator.validate(properties) // no proctime properties.validateExclusion(proctime) @@ -84,12 +91,43 @@ class SchemaValidator(isStreamEnvironment: Boolean = true) extends DescriptorVal object SchemaValidator { + /** + * Prefix for schema-related properties. + */ val SCHEMA = "schema" + val SCHEMA_NAME = "name" val SCHEMA_TYPE = "type" val SCHEMA_PROCTIME = "proctime" val SCHEMA_FROM = "from" + /** + * Returns keys for a [[org.apache.flink.table.sources.TableSourceFactory.supportedProperties()]] + * method that are accepted for schema derivation using + * [[deriveFormatFields(DescriptorProperties)]]. + */ + def getSchemaDerivationKeys: util.List[String] = { + val keys = new util.ArrayList[String]() + + // schema + keys.add(SCHEMA + ".#." + SCHEMA_TYPE) + keys.add(SCHEMA + ".#." + SCHEMA_NAME) + keys.add(SCHEMA + ".#." + SCHEMA_FROM) + + // time attributes + keys.add(SCHEMA + ".#." + SCHEMA_PROCTIME) + keys.add(SCHEMA + ".#." + ROWTIME_TIMESTAMPS_TYPE) + keys.add(SCHEMA + ".#." + ROWTIME_TIMESTAMPS_FROM) + keys.add(SCHEMA + ".#." + ROWTIME_TIMESTAMPS_CLASS) + keys.add(SCHEMA + ".#." + ROWTIME_TIMESTAMPS_SERIALIZED) + keys.add(SCHEMA + ".#." + ROWTIME_WATERMARKS_TYPE) + keys.add(SCHEMA + ".#." + ROWTIME_WATERMARKS_CLASS) + keys.add(SCHEMA + ".#." + ROWTIME_WATERMARKS_SERIALIZED) + keys.add(SCHEMA + ".#." + ROWTIME_WATERMARKS_DELAY) + + keys + } + // utilities /** diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceFactory.java b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/DeserializationSchemaFactory.scala similarity index 52% rename from flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceFactory.java rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/DeserializationSchemaFactory.scala index 6c128b0e0add7..da7918579256f 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceFactory.java +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/DeserializationSchemaFactory.scala @@ -16,22 +16,26 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.kafka; +package org.apache.flink.table.formats -import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_010; +import java.util + +import org.apache.flink.api.common.serialization.DeserializationSchema /** - * Factory for creating configured instances of {@link Kafka010JsonTableSource}. - */ -public class Kafka010JsonTableSourceFactory extends KafkaJsonTableSourceFactory { + * Factory for creating configured instances of [[DeserializationSchema]]. + * + * @tparam T record type that the format produces or consumes + */ +trait DeserializationSchemaFactory[T] extends TableFormatFactory[T] { - @Override - protected KafkaJsonTableSource.Builder createKafkaJsonBuilder() { - return new Kafka010JsonTableSource.Builder(); - } + /** + * Creates and configures a [[DeserializationSchema]] using the given properties. + * + * @param properties normalized properties describing the format + * @return the configured serialization schema or null if the factory cannot provide an + * instance of this class + */ + def createDeserializationSchema(properties: util.Map[String, String]): DeserializationSchema[T] - @Override - protected String kafkaVersion() { - return CONNECTOR_VERSION_VALUE_010; - } } diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSourceFactory.java b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/SerializationSchemaFactory.scala similarity index 52% rename from flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSourceFactory.java rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/SerializationSchemaFactory.scala index 3972c3fcdeded..e4818cdd75bc7 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010AvroTableSourceFactory.java +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/SerializationSchemaFactory.scala @@ -16,22 +16,26 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.kafka; +package org.apache.flink.table.formats -import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_010; +import java.util + +import org.apache.flink.api.common.serialization.SerializationSchema /** - * Factory for creating configured instances of {@link Kafka010AvroTableSource}. - */ -public class Kafka010AvroTableSourceFactory extends KafkaAvroTableSourceFactory { + * Factory for creating configured instances of [[SerializationSchema]]. + * + * @tparam T record type that the format produces or consumes + */ +trait SerializationSchemaFactory[T] extends TableFormatFactory[T] { - @Override - protected KafkaAvroTableSource.Builder createKafkaAvroBuilder() { - return new Kafka010AvroTableSource.Builder(); - } + /** + * Creates and configures a [[SerializationSchema]] using the given properties. + * + * @param properties normalized properties describing the format + * @return the configured serialization schema or null if the factory cannot provide an + * instance of this class + */ + def createSerializationSchema(properties: util.Map[String, String]): SerializationSchema[T] - @Override - protected String kafkaVersion() { - return CONNECTOR_VERSION_VALUE_010; - } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/TableFormatFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/TableFormatFactory.scala new file mode 100644 index 0000000000000..3afef839e54c0 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/TableFormatFactory.scala @@ -0,0 +1,85 @@ +/* + * 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.formats + +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. + * + * @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] + + /** + * Flag to indicate if the given format supports deriving information from a schema. If the + * format can handle schema information, those properties must be added to the list of + * supported properties. + */ + def supportsSchemaDerivation(): Boolean + + /** + * List of format property keys that this factory can handle. This method will be used for + * validation. If a property is passed that this factory cannot handle, an exception will be + * thrown. The list must not contain the keys that are specified by the context. + * + * Example format properties might be: + * - format.line-delimiter + * - format.ignore-parse-errors + * - format.fields.#.type + * - format.fields.#.name + * + * If schema derivation is enabled, the list must include schema properties: + * - 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. + */ + def supportedProperties(): util.List[String] + +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/TableFormatFactoryService.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/TableFormatFactoryService.scala new file mode 100644 index 0000000000000..44911a4389b23 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/TableFormatFactoryService.scala @@ -0,0 +1,271 @@ +/* + * 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.formats + +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._ +import org.apache.flink.table.util.Logging +import org.apache.flink.util.Preconditions + +import _root_.scala.collection.JavaConverters._ +import _root_.scala.collection.mutable + +/** + * Service provider interface for finding a suitable [[TableFormatFactory]] for the + * given properties. + */ +object TableFormatFactoryService extends Logging { + + private lazy val defaultLoader = ServiceLoader.load(classOf[TableFormatFactory[_]]) + + /** + * Finds a table format factory of the given class and creates configured instances from the + * given property map. + * + * @param factoryClass desired format factory + * @param propertyMap properties that describes the format + * @tparam T factory class type + * @return configured instance from 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. + * + * @param factoryClass desired format factory + * @param propertyMap properties that describes the format + * @param classLoader classloader for service loading + * @tparam T factory class type + * @return configured instance from factory + */ + def find[T]( + factoryClass: Class[T], + propertyMap: JMap[String, String], + 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. + * + * @param factoryClass desired format factory + * @param propertyMap properties that describes the format + * @param classLoader optional classloader for service loading + * @tparam T factory class type + * @return configured instance from factory + */ + private def findInternal[T]( + factoryClass: Class[T], + propertyMap: JMap[String, String], + classLoader: Option[ClassLoader]) + : T = { + + Preconditions.checkNotNull(factoryClass) + Preconditions.checkNotNull(propertyMap) + + val properties = propertyMap.asScala.toMap + + // find matching context + val (foundFactories, contextFactories) = findMatchingContext( + factoryClass, + properties, + classLoader) + + // filter by factory class + val classFactories = filterByFactoryClass( + factoryClass, + properties, + foundFactories, + contextFactories) + + // filter by supported keys + filterBySupportedProperties( + factoryClass, + properties, + foundFactories, + classFactories) + } + + 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[_]]() + + try { + val iter = classLoader match { + case Some(customClassLoader) => + val customLoader = ServiceLoader.load(classOf[TableFormatFactory[_]], customClassLoader) + customLoader.iterator() + case None => + defaultLoader.iterator() + } + + while (iter.hasNext) { + val factory = iter.next() + foundFactories += 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) + + // 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) + } + + if (matchingFactories.isEmpty) { + throw new NoMatchingTableFormatException( + "No context matches.", + factoryClass, + foundFactories, + properties) + } + + (foundFactories, matchingFactories) + } + + private def normalizeContext(factory: TableFormatFactory[_]): 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.") + } + 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 + } + + private def filterBySupportedProperties[T]( + factoryClass: Class[T], + properties: Map[String, String], + foundFactories: Seq[TableFormatFactory[_]], + classFactories: Seq[TableFormatFactory[_]]) + : T = { + + val plainGivenKeys = mutable.ArrayBuffer[String]() + properties.keys.foreach { k => + // replace arrays with wildcard + val key = k.replaceAll(".\\d+", ".#") + // ignore duplicates + if (!plainGivenKeys.contains(key)) { + plainGivenKeys += key + } + } + 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 => + lastKey = Option(k) + supportedKeys.contains(k) + } + } + + if (supportedFactories.isEmpty && classFactories.length == 1 && lastKey.isDefined) { + // 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( + s""" + |The matching factory '${factory.getClass.getName}' doesn't support '${lastKey.get}'. + | + |Supported properties of this factory are: + |${supportedKeys.sorted.mkString("\n")}""".stripMargin, + factoryClass, + foundFactories, + properties) + } else if (supportedFactories.isEmpty) { + throw new NoMatchingTableFormatException( + s"No factory supports all properties.", + factoryClass, + foundFactories, + properties) + } else if (supportedFactories.length > 1) { + throw new AmbiguousTableFormatException( + supportedFactories, + factoryClass, + foundFactories, + properties) + } + + supportedFactories.head.asInstanceOf[T] + } + + private def normalizeSupportedProperties(factory: TableFormatFactory[_]): 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.") + } + supportedPropertiesJava.asScala.map(_.toLowerCase) + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala index 06d6bfba0d038..a451afff76d67 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala @@ -71,7 +71,10 @@ class CsvTableSourceFactory extends TableSourceFactory[Row] { // validate new FileSystemValidator().validate(params) new CsvValidator().validate(params) - new SchemaValidator().validate(params) + new SchemaValidator( + isStreamEnvironment = true, + supportsSourceTimestamps = false, + supportsSourceWatermarks = false).validate(params) // build val csvTableSourceBuilder = new CsvTableSource.Builder diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/DefinedFieldMapping.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/DefinedFieldMapping.scala index fcfc2ca30a396..1c43d68c1a95d 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/DefinedFieldMapping.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/DefinedFieldMapping.scala @@ -50,7 +50,8 @@ trait DefinedFieldMapping { * type. It can also provide a mapping for fields which are not in the [[TableSchema]] to make * fields in the physical [[TypeInformation]] accessible for a [[TimestampExtractor]]. * - * @return A mapping from [[TableSchema]] fields to [[TypeInformation]] fields. + * @return A mapping from [[TableSchema]] fields to [[TypeInformation]] fields or + * null if no mapping is necessary. */ def getFieldMapping: JMap[String, String] } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactory.scala index e5f696503279a..c16e4d0d89f2e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactory.scala @@ -63,6 +63,15 @@ trait TableSourceFactory[T] { * * Note: 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. + * + * In some cases it might be useful to declare wildcards "*". Wildcards can only be declared at + * the end of a property key. + * + * For example, if an arbitrary format should be supported: + * - format.* + * + * Note: Wildcards should be used with caution as they might swallow unsupported properties + * and thus might lead to undesired behavior. */ def supportedProperties(): util.List[String] diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactoryService.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactoryService.scala index 0c81335f6b6fb..94aab7202cfae 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactoryService.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactoryService.scala @@ -139,15 +139,11 @@ object TableSourceFactoryService extends Logging { Seq[String]() } + // extract wildcard prefixes + val wildcards = extractWildcardPrefixes(supportedProperties) + // check for supported properties - plainProperties.foreach { k => - if (!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")}") - } - } + validateSupportedProperties(factory, supportedProperties, wildcards, plainProperties) // create the table source try { @@ -159,4 +155,27 @@ object TableSourceFactoryService extends Logging { t) } } + + private def extractWildcardPrefixes(propertyKeys: Seq[String]): Seq[String] = { + propertyKeys + .filter(_.endsWith("*")) + .map(s => s.substring(0, s.length - 1)) + } + + private def validateSupportedProperties[T]( + factory: TableSourceFactory[T], + supportedProperties: Seq[String], + wildcards: Seq[String], + plainProperties: Seq[String]) + : Unit = { + + plainProperties.foreach { k => + if (!supportedProperties.contains(k) && !wildcards.exists(k.startsWith)) { + 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")}") + } + } + } } diff --git a/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory b/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory new file mode 100644 index 0000000000000..b5646a3925d8a --- /dev/null +++ b/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory @@ -0,0 +1,17 @@ +# 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 diff --git a/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory b/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory index 1b2506e829ada..06bf8536c73de 100644 --- a/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory +++ b/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory @@ -13,4 +13,5 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.table.sources.TestTableSourceFactory +org.apache.flink.table.sources.TestFixedFormatTableFactory +org.apache.flink.table.sources.TestWildcardFormatTableSourceFactory diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/RowtimeTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/RowtimeTest.scala index 9e339d02adc83..d5930fae64f6d 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/RowtimeTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/RowtimeTest.scala @@ -41,6 +41,11 @@ class RowtimeTest extends DescriptorTestBase { removePropertyAndVerify(descriptors().get(1), "rowtime.watermarks.class") } + @Test(expected = classOf[ValidationException]) + def testUnsupportedSourceWatermarks(): Unit = { + addPropertyAndVerify(descriptors().get(0), "rowtime.watermarks.type", "from-source") + } + // ---------------------------------------------------------------------------------------------- override def descriptors(): util.List[Descriptor] = { @@ -56,7 +61,7 @@ class RowtimeTest extends DescriptorTestBase { } override def validator(): DescriptorValidator = { - new RowtimeValidator() + new RowtimeValidator(supportsSourceTimestamps = true, supportsSourceWatermarks = false) } override def properties(): util.List[util.Map[String, String]] = { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaTest.scala index 589ec4f582b3d..2eb77a84b8ce2 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaTest.scala @@ -61,7 +61,10 @@ class SchemaTest extends DescriptorTestBase { } override def validator(): DescriptorValidator = { - new SchemaValidator(isStreamEnvironment = true) + new SchemaValidator( + isStreamEnvironment = true, + supportsSourceTimestamps = true, + supportsSourceWatermarks = true) } override def properties(): util.List[util.Map[String, String]] = { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptorTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptorTest.scala deleted file mode 100644 index 2c9a89c47ff65..0000000000000 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptorTest.scala +++ /dev/null @@ -1,79 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.descriptors - -import org.apache.flink.table.utils.TableTestBase - -class StreamTableSourceDescriptorTest extends TableTestBase { - -// TODO enable this test once we expose the feature through the table environment -// @Test -// def testStreamTableSourceDescriptor(): Unit = { -// val util = streamTestUtil() -// val desc = util.tableEnv -// .from( -// FileSystem() -// .path("/path/to/csv")) -// .withFormat( -// Csv() -// .field("myfield", Types.STRING) -// .field("myfield2", Types.INT) -// .quoteCharacter(';') -// .fieldDelimiter("#") -// .lineDelimiter("\r\n") -// .commentPrefix("%%") -// .ignoreFirstLine() -// .ignoreParseErrors()) -// .withSchema( -// Schema() -// .field("myfield", Types.STRING) -// .field("myfield2", Types.INT) -// .field("proctime", Types.SQL_TIMESTAMP).proctime() -// .field("rowtime", Types.SQL_TIMESTAMP).rowtime( -// Rowtime().timestampsFromSource().watermarksFromSource()) -// ) -// val expected = Seq( -// "connector.type" -> "filesystem", -// "connector.path" -> "/path/to/csv", -// "format.type" -> "csv", -// "format.fields.0.name" -> "myfield", -// "format.fields.0.type" -> "VARCHAR", -// "format.fields.1.name" -> "myfield2", -// "format.fields.1.type" -> "INT", -// "format.quote-character" -> ";", -// "format.field-delimiter" -> "#", -// "format.line-delimiter" -> "\r\n", -// "format.comment-prefix" -> "%%", -// "format.ignore-first-line" -> "true", -// "format.ignore-parse-errors" -> "true", -// "schema.0.name" -> "myfield", -// "schema.0.type" -> "VARCHAR", -// "schema.1.name" -> "myfield2", -// "schema.1.type" -> "INT", -// "schema.2.name" -> "proctime", -// "schema.2.type" -> "TIMESTAMP", -// "schema.2.proctime" -> "true", -// "schema.3.name" -> "rowtime", -// "schema.3.type" -> "TIMESTAMP", -// "schema.3.rowtime.0.timestamps.type" -> "from-source", -// "schema.3.rowtime.0.watermarks.type" -> "from-source" -// ) -// verifyProperties(desc, expected) -// } -} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/TableSourceDescriptorTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/TableSourceDescriptorTest.scala new file mode 100644 index 0000000000000..91f3aac0d654c --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/TableSourceDescriptorTest.scala @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.descriptors + +import org.apache.flink.table.api.Types +import org.apache.flink.table.utils.TableTestBase +import org.junit.Assert.assertEquals +import org.junit.Test + +import scala.collection.JavaConverters._ + +class TableSourceDescriptorTest extends TableTestBase { + + @Test + def testStreamTableSourceDescriptor(): Unit = { + testTableSourceDescriptor(true) + } + + @Test + def testBatchTableSourceDescriptor(): Unit = { + testTableSourceDescriptor(false) + } + + private def testTableSourceDescriptor(isStreaming: Boolean): Unit = { + + val schema = Schema() + .field("myfield", Types.STRING) + .field("myfield2", Types.INT) + if (isStreaming) { + schema.field("proctime", Types.SQL_TIMESTAMP).proctime() + } + + val connector = FileSystem() + .path("/path/to/csv") + + val format = Csv() + .field("myfield", Types.STRING) + .field("myfield2", Types.INT) + .quoteCharacter(';') + .fieldDelimiter("#") + .lineDelimiter("\r\n") + .commentPrefix("%%") + .ignoreFirstLine() + .ignoreParseErrors() + + val descriptor = if (isStreaming) { + streamTestUtil().tableEnv + .from(connector) + .withFormat(format) + .withSchema(schema) + } else { + batchTestUtil().tableEnv + .from(connector) + .withFormat(format) + .withSchema(schema) + } + + val expectedCommonProperties = Seq( + "connector.property-version" -> "1", + "connector.type" -> "filesystem", + "connector.path" -> "/path/to/csv", + "format.property-version" -> "1", + "format.type" -> "csv", + "format.fields.0.name" -> "myfield", + "format.fields.0.type" -> "VARCHAR", + "format.fields.1.name" -> "myfield2", + "format.fields.1.type" -> "INT", + "format.quote-character" -> ";", + "format.field-delimiter" -> "#", + "format.line-delimiter" -> "\r\n", + "format.comment-prefix" -> "%%", + "format.ignore-first-line" -> "true", + "format.ignore-parse-errors" -> "true", + "schema.0.name" -> "myfield", + "schema.0.type" -> "VARCHAR", + "schema.1.name" -> "myfield2", + "schema.1.type" -> "INT" + ) + + val expectedProperties = if (isStreaming) { + expectedCommonProperties ++ Seq( + "schema.2.name" -> "proctime", + "schema.2.type" -> "TIMESTAMP", + "schema.2.proctime" -> "true" + ) + } else { + expectedCommonProperties + } + + val actualProperties = new DescriptorProperties(true) + descriptor.addProperties(actualProperties) + + assertEquals(expectedProperties.toMap.asJava, actualProperties.asMap) + } +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/TableFormatFactoryServiceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/TableFormatFactoryServiceTest.scala new file mode 100644 index 0000000000000..7d7d2d2446574 --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/TableFormatFactoryServiceTest.scala @@ -0,0 +1,144 @@ +/* + * 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.formats + +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.junit.Test + +/** + * Tests for [[TableFormatFactoryService]]. 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 + * have the same context and support COMMON_PATH. + */ +class TableFormatFactoryServiceTest { + + @Test + def testValidProperties(): Unit = { + val props = properties() + assertNotNull( + TableFormatFactoryService.find(classOf[TableFormatFactory[_]], props)) + } + + @Test + def testDifferentContextVersion(): Unit = { + val props = properties() + props.put(FormatDescriptorValidator.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) + assertTrue(foundFactory.isInstanceOf[TestTableFormatFactory]) + } + + @Test + def testAmbiguousMoreSupportSelection(): Unit = { + val props = properties() + props.remove(UNIQUE_PROPERTY) // both formats match now + props.put(SPECIAL_PATH, "/what/ever") // now only TestAmbiguousTableFormatFactory + assertTrue( + TableFormatFactoryService + .find(classOf[TableFormatFactory[_]], props) + .isInstanceOf[TestAmbiguousTableFormatFactory]) + } + + @Test + def testAmbiguousClassBasedSelection(): Unit = { + val props = properties() + props.remove(UNIQUE_PROPERTY) // both formats match now + assertTrue( + TableFormatFactoryService + // we are looking for a particular class + .find(classOf[TestAmbiguousTableFormatFactory], props) + .isInstanceOf[TestAmbiguousTableFormatFactory]) + } + + @Test + def testAmbiguousSchemaBasedSelection(): Unit = { + val props = properties() + props.remove(UNIQUE_PROPERTY) // both formats match now + // this is unknown to the schema derivation factory + props.put("schema.unknown-schema-field", "unknown") + + // 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 + .find(classOf[TableFormatFactory[_]], props) + .isInstanceOf[TestAmbiguousTableFormatFactory]) + } + + @Test(expected = classOf[NoMatchingTableFormatException]) + def testMissingClass(): Unit = { + val props = properties() + // this class is not a valid factory + TableFormatFactoryService.find(classOf[TableFormatFactoryServiceTest], props) + } + + @Test(expected = classOf[NoMatchingTableFormatException]) + def testInvalidContext(): Unit = { + val props = properties() + // no context specifies this + props.put(FormatDescriptorValidator.FORMAT_TYPE, "unknown_format_type") + TableFormatFactoryService.find(classOf[TableFormatFactory[_]], props) + } + + @Test(expected = classOf[NoMatchingTableFormatException]) + def testUnsupportedProperty(): Unit = { + val props = properties() + props.put("format.property_not_defined_by_any_factory", "/new/path") + TableFormatFactoryService.find(classOf[TableFormatFactory[_]], props) + } + + @Test(expected = classOf[AmbiguousTableFormatException]) + def testAmbiguousFactory(): Unit = { + val props = properties() + props.remove(UNIQUE_PROPERTY) // now both factories match + TableFormatFactoryService.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(UNIQUE_PROPERTY, "true") + 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") + properties.put("schema.2.name", "c") + properties + } +} + +object TableFormatFactoryServiceTest { + + val TEST_FORMAT_TYPE = "test-format" + val COMMON_PATH = "format.common-path" + val SPECIAL_PATH = "format.special-path" + val UNIQUE_PROPERTY = "format.unique-property" +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestAmbiguousTableFormatFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestAmbiguousTableFormatFactory.scala new file mode 100644 index 0000000000000..1a30ac0bfec41 --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestAmbiguousTableFormatFactory.scala @@ -0,0 +1,52 @@ +/* + * 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.formats.utils + +import java.util + +import org.apache.flink.table.descriptors.FormatDescriptorValidator +import org.apache.flink.table.formats.{TableFormatFactory, TableFormatFactoryServiceTest} +import org.apache.flink.types.Row + +/** + * Table format factory for testing. + * + * It does not support UNIQUE_PROPERTY compared to [[TestTableFormatFactory]] nor + * schema derivation. Both formats have the same context and support COMMON_PATH. + */ +class TestAmbiguousTableFormatFactory extends TableFormatFactory[Row] { + + override def requiredContext(): util.Map[String, String] = { + val context = new util.HashMap[String, String]() + context.put( + FormatDescriptorValidator.FORMAT_TYPE, + TableFormatFactoryServiceTest.TEST_FORMAT_TYPE) + context.put(FormatDescriptorValidator.FORMAT_PROPERTY_VERSION, "1") + context + } + + override def supportsSchemaDerivation(): Boolean = false // no schema derivation + + override def supportedProperties(): util.List[String] = { + val properties = new util.ArrayList[String]() + properties.add(TableFormatFactoryServiceTest.COMMON_PATH) + properties.add(TableFormatFactoryServiceTest.SPECIAL_PATH) + properties + } +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestDeserializationSchema.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestDeserializationSchema.scala new file mode 100644 index 0000000000000..0b519efb2f3bc --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestDeserializationSchema.scala @@ -0,0 +1,50 @@ +/* + * 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.formats.utils + +import org.apache.flink.api.common.serialization.DeserializationSchema +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.types.Row + +/** + * Deserialization schema for testing purposes. + */ +class TestDeserializationSchema(val typeInfo: TypeInformation[Row]) + extends DeserializationSchema[Row] { + + override def deserialize(message: Array[Byte]): Row = throw new UnsupportedOperationException() + + override def isEndOfStream(nextElement: Row): Boolean = throw new UnsupportedOperationException() + + override def getProducedType: TypeInformation[Row] = typeInfo + + def canEqual(other: Any): Boolean = other.isInstanceOf[TestDeserializationSchema] + + override def equals(other: Any): Boolean = other match { + case that: TestDeserializationSchema => + (that canEqual this) && + typeInfo == that.typeInfo + case _ => false + } + + override def hashCode(): Int = { + val state = Seq(typeInfo) + state.map(_.hashCode()).foldLeft(0)((a, b) => 31 * a + b) + } +} diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSourceFactoryTest.java b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestSerializationSchema.scala similarity index 62% rename from flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSourceFactoryTest.java rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestSerializationSchema.scala index f367636e73601..7043ecadd6b9c 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08AvroTableSourceFactoryTest.java +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestSerializationSchema.scala @@ -16,22 +16,15 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.kafka; +package org.apache.flink.table.formats.utils -import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_08; +import org.apache.flink.api.common.serialization.SerializationSchema +import org.apache.flink.types.Row /** - * Tests for {@link Kafka08AvroTableSourceFactory}. - */ -public class Kafka08AvroTableSourceFactoryTest extends KafkaAvroTableSourceFactoryTestBase { - - @Override - protected String version() { - return CONNECTOR_VERSION_VALUE_08; - } + * Serialization schema for testing purposes. + */ +class TestSerializationSchema extends SerializationSchema[Row] { - @Override - protected KafkaAvroTableSource.Builder builder() { - return Kafka08AvroTableSource.builder(); - } + override def serialize(element: Row): Array[Byte] = throw new UnsupportedOperationException() } diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSourceFactoryTest.java b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestTableFormat.scala similarity index 61% rename from flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSourceFactoryTest.java rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestTableFormat.scala index 787a525daa827..38cef0533d6b1 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011AvroTableSourceFactoryTest.java +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestTableFormat.scala @@ -16,22 +16,18 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.kafka; +package org.apache.flink.table.formats.utils -import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_011; +import org.apache.flink.table.descriptors.{DescriptorProperties, FormatDescriptor} /** - * Tests for {@link Kafka011AvroTableSourceFactory}. - */ -public class Kafka011AvroTableSourceFactoryTest extends KafkaAvroTableSourceFactoryTestBase { - - @Override - protected String version() { - return CONNECTOR_VERSION_VALUE_011; - } + * Format descriptor for testing purposes. + */ +class TestTableFormat extends FormatDescriptor("test-format", 1) { - @Override - protected KafkaAvroTableSource.Builder builder() { - return Kafka011AvroTableSource.builder(); - } + 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") + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestTableFormatFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestTableFormatFactory.scala new file mode 100644 index 0000000000000..efd9afea5e784 --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestTableFormatFactory.scala @@ -0,0 +1,65 @@ +/* + * 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.formats.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.types.Row + +/** + * Table format factory for testing. + * + * It has the same context as [[TestAmbiguousTableFormatFactory]] and both support COMMON_PATH. + * This format does not support SPECIAL_PATH but supports schema derivation. + */ +class TestTableFormatFactory extends DeserializationSchemaFactory[Row] { + + override def requiredContext(): util.Map[String, String] = { + val context = new util.HashMap[String, String]() + context.put( + FormatDescriptorValidator.FORMAT_TYPE, + TableFormatFactoryServiceTest.TEST_FORMAT_TYPE) + context.put(FormatDescriptorValidator.FORMAT_PROPERTY_VERSION, "1") + context + } + + override def supportsSchemaDerivation(): Boolean = true + + override def supportedProperties(): util.List[String] = { + val properties = new util.ArrayList[String]() + properties.add(TableFormatFactoryServiceTest.UNIQUE_PROPERTY) + properties.add(TableFormatFactoryServiceTest.COMMON_PATH) + properties.add(FormatDescriptorValidator.FORMAT_DERIVE_SCHEMA) + properties.addAll(SchemaValidator.getSchemaDerivationKeys) + properties + } + + override def createDeserializationSchema( + properties: util.Map[String, String]) + : DeserializationSchema[Row] = { + + val props = new DescriptorProperties(true) + props.putProperties(properties) + val schema = SchemaValidator.deriveFormatFields(props) + new TestDeserializationSchema(schema.toRowType) + } +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TableSourceFactoryServiceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TableSourceFactoryServiceTest.scala index 48db9dab6fca2..e70c4f1932c65 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TableSourceFactoryServiceTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TableSourceFactoryServiceTest.scala @@ -31,6 +31,8 @@ class TableSourceFactoryServiceTest { @Test def testValidProperties(): Unit = { val props = properties() + props.put(CONNECTOR_TYPE, "fixed") + props.put(FORMAT_TYPE, "test") assertTrue(TableSourceFactoryService.findAndCreateTableSource(props.toMap) != null) } @@ -38,12 +40,15 @@ class TableSourceFactoryServiceTest { def testInvalidContext(): Unit = { val props = properties() props.put(CONNECTOR_TYPE, "FAIL") + props.put(FORMAT_TYPE, "test") TableSourceFactoryService.findAndCreateTableSource(props.toMap) } @Test def testDifferentContextVersion(): Unit = { val props = properties() + props.put(CONNECTOR_TYPE, "fixed") + props.put(FORMAT_TYPE, "test") props.put(CONNECTOR_PROPERTY_VERSION, "2") // the table source should still be found assertTrue(TableSourceFactoryService.findAndCreateTableSource(props.toMap) != null) @@ -52,6 +57,8 @@ class TableSourceFactoryServiceTest { @Test(expected = classOf[ValidationException]) def testUnsupportedProperty(): Unit = { val props = properties() + props.put(CONNECTOR_TYPE, "fixed") + props.put(FORMAT_TYPE, "test") props.put("format.path_new", "/new/path") TableSourceFactoryService.findAndCreateTableSource(props.toMap) } @@ -59,14 +66,25 @@ class TableSourceFactoryServiceTest { @Test(expected = classOf[TableException]) def testFailingFactory(): Unit = { val props = properties() + props.put(CONNECTOR_TYPE, "fixed") + props.put(FORMAT_TYPE, "test") props.put("failing", "true") TableSourceFactoryService.findAndCreateTableSource(props.toMap) } + @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 = TableSourceFactoryService.findAndCreateTableSource(props.toMap) + assertTrue(actualTableSource.isInstanceOf[TestWildcardFormatTableSourceFactory]) + } + private def properties(): mutable.Map[String, String] = { val properties = mutable.Map[String, String]() - properties.put(CONNECTOR_TYPE, "test") - properties.put(FORMAT_TYPE, "test") properties.put(CONNECTOR_PROPERTY_VERSION, "1") properties.put(FORMAT_PROPERTY_VERSION, "1") properties.put("format.path", "/path/to/target") diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TestTableSourceFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TestFixedFormatTableFactory.scala similarity index 92% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TestTableSourceFactory.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TestFixedFormatTableFactory.scala index 026ee0a9f62d8..6a8041b03d2eb 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TestTableSourceFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TestFixedFormatTableFactory.scala @@ -27,13 +27,13 @@ import org.apache.flink.table.descriptors.FormatDescriptorValidator.{FORMAT_PROP import org.apache.flink.types.Row /** - * Table source factory for testing. + * Table source factory for testing with a fixed format. */ -class TestTableSourceFactory extends TableSourceFactory[Row] { +class TestFixedFormatTableFactory extends TableSourceFactory[Row] { override def requiredContext(): util.Map[String, String] = { val context = new util.HashMap[String, String]() - context.put(CONNECTOR_TYPE, "test") + context.put(CONNECTOR_TYPE, "fixed") context.put(FORMAT_TYPE, "test") context.put(CONNECTOR_PROPERTY_VERSION, "1") context.put(FORMAT_PROPERTY_VERSION, "1") @@ -42,7 +42,6 @@ class TestTableSourceFactory extends TableSourceFactory[Row] { override def supportedProperties(): util.List[String] = { val properties = new util.ArrayList[String]() - // connector properties.add("format.path") properties.add("schema.#.name") properties.add("schema.#.field.#.name") diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TestWildcardFormatTableSourceFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TestWildcardFormatTableSourceFactory.scala new file mode 100644 index 0000000000000..91598a6fdce0a --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TestWildcardFormatTableSourceFactory.scala @@ -0,0 +1,56 @@ +/* + * 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.sources + +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.types.Row + +/** + * Table source factory for testing with a wildcard format ("format.*"). + */ +class TestWildcardFormatTableSourceFactory extends TableSourceFactory[Row] with TableSource[Row] { + + override def requiredContext(): util.Map[String, String] = { + val context = new util.HashMap[String, String]() + context.put(CONNECTOR_TYPE, "wildcard") + context.put(CONNECTOR_PROPERTY_VERSION, "1") + context + } + + override def supportedProperties(): util.List[String] = { + val properties = new util.ArrayList[String]() + properties.add("format.*") + properties.add("schema.#.name") + properties.add("schema.#.field.#.name") + properties.add("failing") + properties + } + + override def create(properties: util.Map[String, String]): TableSource[Row] = { + this + } + + override def getTableSchema: TableSchema = throw new UnsupportedOperationException() + + override def getReturnType: TypeInformation[Row] = throw new UnsupportedOperationException() +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala index dc09e57337f8c..308ead2ed57a8 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala @@ -21,6 +21,7 @@ package org.apache.flink.table.utils import org.apache.calcite.tools.RuleSet import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.api.{QueryConfig, Table, TableConfig, TableEnvironment} +import org.apache.flink.table.descriptors.{ConnectorDescriptor, TableSourceDescriptor} import org.apache.flink.table.sinks.TableSink import org.apache.flink.table.sources.TableSource @@ -51,4 +52,6 @@ class MockTableEnvironment extends TableEnvironment(new TableConfig) { : Unit = ??? override def explain(table: Table): String = ??? + + override def from(connectorDescriptor: ConnectorDescriptor): TableSourceDescriptor = ??? } From 42a8a156d4e6f8f3d119c458350b6c897306fc48 Mon Sep 17 00:00:00 2001 From: Shuyi Chen Date: Tue, 19 Jun 2018 12:00:34 -0700 Subject: [PATCH 02/17] [FLINK-8866] [table] Create unified interfaces to configure and instatiate TableSinks This closes #6201. --- ...table.connectors.DiscoverableTableFactory} | 0 ...table.connectors.DiscoverableTableFactory} | 0 ...table.connectors.DiscoverableTableFactory} | 0 ...table.connectors.DiscoverableTableFactory} | 0 .../kafka/KafkaTableSourceFactory.java | 7 +- .../KafkaJsonTableSourceFactoryTestBase.java | 10 +- flink-libraries/flink-sql-client/pom.xml | 6 +- .../table/client/config/Environment.java | 54 ++++-- .../flink/table/client/config/Sink.java | 53 +++++ .../flink/table/client/config/Source.java | 22 +-- .../flink/table/client/config/SourceSink.java | 66 +++++++ .../flink/table/client/gateway/Executor.java | 2 +- .../gateway/local/ExecutionContext.java | 34 +++- .../client/gateway/local/LocalExecutor.java | 12 ++ .../client/gateway/local/ProgramDeployer.java | 59 ++++-- .../assembly/test-table-source-factory.xml | 4 +- .../client/gateway/local/DependencyTest.java | 9 +- .../gateway/local/LocalExecutorITCase.java | 3 +- .../gateway/utils/TestTableSinkFactory.java | 123 ++++++++++++ .../gateway/utils/TestTableSourceFactory.java | 13 +- .../test/resources/test-factory-services-file | 1 + .../resources/test-sql-client-factory.yaml | 5 +- ...table.connectors.DiscoverableTableFactory} | 1 + .../resources/tableSourceConverter.properties | 36 ++++ .../table/api/BatchTableEnvironment.scala | 49 ++++- .../table/api/StreamTableEnvironment.scala | 49 ++++- .../flink/table/api/TableEnvironment.scala | 21 +- .../apache/flink/table/api/exceptions.scala | 40 ++-- .../api/java/BatchTableEnvironment.scala | 2 + .../catalog/ExternalTableSourceUtil.scala | 25 ++- .../DiscoverableTableFactory.scala} | 26 +-- .../connectors/TableFactoryService.scala | 160 ++++++++++++++++ .../table/connectors/TableSinkFactory.scala | 34 ++++ .../table/connectors/TableSourceFactory.scala | 34 ++++ .../BatchTableSourceDescriptor.scala | 9 +- .../table/descriptors/CsvValidator.scala | 1 - .../descriptors/DescriptorProperties.scala | 3 +- .../flink/table/descriptors/FileSystem.scala | 27 ++- .../descriptors/FileSystemValidator.scala | 10 +- .../table/descriptors/SchemaValidator.scala | 39 +++- .../StreamTableSourceDescriptor.scala | 9 +- .../TableDescriptorValidator.scala | 5 +- .../descriptors/TableSinkDescriptor.scala | 30 +++ .../descriptors/TableSourceDescriptor.scala | 2 + .../plan/nodes/PhysicalTableSourceScan.scala | 11 +- .../FlinkLogicalTableFunctionScan.scala | 1 - .../logical/FlinkLogicalTableSourceScan.scala | 20 +- .../dataSet/BatchTableSourceScanRule.scala | 10 +- .../StreamTableSourceScanRule.scala | 8 +- .../PushFilterIntoTableSourceScanRule.scala | 6 +- .../plan/schema/StreamTableSourceTable.scala | 2 +- .../table/plan/schema/TableSinkTable.scala | 14 +- .../plan/schema/TableSourceSinkTable.scala | 43 +++++ .../table/plan/schema/TableSourceTable.scala | 17 +- .../flink/table/sinks/CsvTableSink.scala | 92 +++++++++ .../table/sinks/CsvTableSinkFactory.scala | 96 ++++++++++ .../table/sources/CsvTableSourceFactory.scala | 5 +- .../sources/TableSourceFactoryService.scala | 181 ------------------ ...table.connectors.DiscoverableTableFactory} | 5 + .../validation/InsertIntoValidationTest.scala | 8 +- .../validation/InsertIntoValidationTest.scala | 6 +- .../validation/InsertIntoValidationTest.scala | 8 +- .../validation/InsertIntoValidationTest.scala | 6 +- .../validation/TableSinksValidationTest.scala | 2 +- .../catalog/ExternalCatalogSchemaTest.scala | 8 +- .../TableSinkFactoryServiceTest.scala | 85 ++++++++ .../TableSourceFactoryServiceTest.scala | 30 ++- .../TestFixedFormatTableFactory.scala | 0 .../connectors/TestTableSinkFactory.scala | 69 +++++++ .../connectors/TestTableSourceFactory.scala | 64 +++++++ ...TestWildcardFormatTableSourceFactory.scala | 0 .../table/descriptors/FileSystemTest.scala | 15 +- .../descriptors/SchemaValidatorTest.scala | 36 +++- .../batch/sql/TableEnvironmentITCase.scala | 8 +- .../batch/table/TableEnvironmentITCase.scala | 8 +- .../runtime/stream/TimeAttributesITCase.scala | 8 +- .../table/runtime/stream/sql/SqlITCase.scala | 48 ++++- .../stream/table/TableSinkITCase.scala | 8 +- .../table/utils/InMemoryTableFactory.scala | 113 +++++++++++ ....scala => MemoryTableSourceSinkUtil.scala} | 72 ++++++- .../table/utils/MockTableEnvironment.scala | 2 + 81 files changed, 1766 insertions(+), 444 deletions(-) rename flink-connectors/flink-connector-kafka-0.10/src/main/resources/META-INF/services/{org.apache.flink.table.sources.TableSourceFactory => org.apache.flink.table.connectors.DiscoverableTableFactory} (100%) rename flink-connectors/flink-connector-kafka-0.11/src/main/resources/META-INF/services/{org.apache.flink.table.sources.TableSourceFactory => org.apache.flink.table.connectors.DiscoverableTableFactory} (100%) rename flink-connectors/flink-connector-kafka-0.8/src/main/resources/META-INF/services/{org.apache.flink.table.sources.TableSourceFactory => org.apache.flink.table.connectors.DiscoverableTableFactory} (100%) rename flink-connectors/flink-connector-kafka-0.9/src/main/resources/META-INF/services/{org.apache.flink.table.sources.TableSourceFactory => org.apache.flink.table.connectors.DiscoverableTableFactory} (100%) create mode 100644 flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Sink.java create mode 100644 flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/SourceSink.java create mode 100644 flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactory.java rename flink-libraries/flink-table/src/main/resources/META-INF/services/{org.apache.flink.table.sources.TableSourceFactory => org.apache.flink.table.connectors.DiscoverableTableFactory} (94%) create mode 100644 flink-libraries/flink-table/src/main/resources/tableSourceConverter.properties rename flink-libraries/flink-table/src/main/scala/org/apache/flink/table/{sources/TableSourceFactory.scala => connectors/DiscoverableTableFactory.scala} (69%) create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/TableFactoryService.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/TableSinkFactory.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/TableSourceFactory.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSinkDescriptor.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceSinkTable.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala delete mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactoryService.scala rename flink-libraries/flink-table/src/test/resources/META-INF/services/{org.apache.flink.table.sources.TableSourceFactory => org.apache.flink.table.connectors.DiscoverableTableFactory} (65%) create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TableSinkFactoryServiceTest.scala rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/{sources => connectors}/TableSourceFactoryServiceTest.scala (71%) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/{sources => connectors}/TestFixedFormatTableFactory.scala (100%) create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TestTableSinkFactory.scala create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TestTableSourceFactory.scala rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/{sources => connectors}/TestWildcardFormatTableSourceFactory.scala (100%) create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/{MemoryTableSinkUtil.scala => MemoryTableSourceSinkUtil.scala} (53%) diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory b/flink-connectors/flink-connector-kafka-0.10/src/main/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory similarity index 100% rename from flink-connectors/flink-connector-kafka-0.10/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory rename to flink-connectors/flink-connector-kafka-0.10/src/main/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory b/flink-connectors/flink-connector-kafka-0.11/src/main/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory similarity index 100% rename from flink-connectors/flink-connector-kafka-0.11/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory rename to flink-connectors/flink-connector-kafka-0.11/src/main/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory b/flink-connectors/flink-connector-kafka-0.8/src/main/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory similarity index 100% rename from flink-connectors/flink-connector-kafka-0.8/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory rename to flink-connectors/flink-connector-kafka-0.8/src/main/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory b/flink-connectors/flink-connector-kafka-0.9/src/main/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory similarity index 100% rename from flink-connectors/flink-connector-kafka-0.9/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory rename to flink-connectors/flink-connector-kafka-0.9/src/main/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java index 3a2a81e5fe62c..dc4374e48d490 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java @@ -23,6 +23,8 @@ 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; @@ -30,7 +32,6 @@ import org.apache.flink.table.formats.TableFormatFactoryService; import org.apache.flink.table.sources.RowtimeAttributeDescriptor; import org.apache.flink.table.sources.TableSource; -import org.apache.flink.table.sources.TableSourceFactory; import org.apache.flink.types.Row; import java.util.ArrayList; @@ -71,7 +72,7 @@ /** * Factory for creating configured instances of {@link KafkaTableSource}. */ -public abstract class KafkaTableSourceFactory implements TableSourceFactory { +public abstract class KafkaTableSourceFactory implements TableSourceFactory, DiscoverableTableFactory { @Override public Map requiredContext() { @@ -118,7 +119,7 @@ public List supportedProperties() { } @Override - public TableSource create(Map properties) { + public TableSource createTableSource(Map properties) { final DescriptorProperties params = new DescriptorProperties(true); params.putProperties(properties); diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java index 84c5bd30099d5..c8b422fa0e312 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java @@ -22,6 +22,9 @@ 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.descriptors.DescriptorProperties; import org.apache.flink.table.descriptors.FormatDescriptor; import org.apache.flink.table.descriptors.Json; import org.apache.flink.table.descriptors.Kafka; @@ -29,7 +32,6 @@ import org.apache.flink.table.descriptors.Schema; import org.apache.flink.table.descriptors.TestTableSourceDescriptor; 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; @@ -148,7 +150,11 @@ private void testTableSource(FormatDescriptor format) { new Rowtime().timestampsFromField("time").watermarksPeriodicAscending()) .field("proc-time", Types.SQL_TIMESTAMP).proctime()); - final TableSource factorySource = TableSourceFactoryService.findAndCreateTableSource(testDesc); + DescriptorProperties properties = new DescriptorProperties(true); + testDesc.addProperties(properties); + final TableSource factorySource = + ((TableSourceFactory) TableFactoryService.find(TableSourceFactory.class, testDesc)) + .createTableSource(properties.asMap()); assertEquals(builderSource, factorySource); } diff --git a/flink-libraries/flink-sql-client/pom.xml b/flink-libraries/flink-sql-client/pom.xml index 6bcfc137a9203..9d34afa2c68a7 100644 --- a/flink-libraries/flink-sql-client/pom.xml +++ b/flink-libraries/flink-sql-client/pom.xml @@ -159,7 +159,7 @@ under the License. org.codehaus.commons.compiler.properties org/codehaus/janino/** org/codehaus/commons/** - META-INF/services/org.apache.flink.table.sources.TableSourceFactory + META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory org/jline/** com/fasterxml/jackson/** @@ -177,13 +177,13 @@ under the License. 2.4 - create-table-source-factory-jar + create-table-connector-factory-jar process-test-classes single - table-source-factory + table-connector-factory false src/test/assembly/test-table-source-factory.xml diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java index b26c45f5ff047..966a58152d624 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java @@ -48,6 +48,8 @@ public class Environment { private Deployment deployment; + private static final String NAME = "name"; + public Environment() { this.tables = Collections.emptyMap(); this.functions = Collections.emptyMap(); @@ -62,20 +64,21 @@ public Map getTables() { public void setTables(List> tables) { this.tables = new HashMap<>(tables.size()); tables.forEach(config -> { - if (!config.containsKey(TableDescriptorValidator.TABLE_TYPE())) { - throw new SqlClientException("The 'type' attribute of a table is missing."); + if (!config.containsKey(NAME)) { + throw new SqlClientException("The 'name' attribute of a table is missing."); + } + final Object nameObject = config.get(NAME); + if (nameObject == null || !(nameObject instanceof String) || ((String) nameObject).length() <= 0) { + throw new SqlClientException("Invalid table name '" + nameObject + "'."); } - if (config.get(TableDescriptorValidator.TABLE_TYPE()).equals(TableDescriptorValidator.TABLE_TYPE_VALUE_SOURCE())) { - config.remove(TableDescriptorValidator.TABLE_TYPE()); - final Source s = Source.create(config); - if (this.tables.containsKey(s.getName())) { - throw new SqlClientException("Duplicate source name '" + s.getName() + "'."); - } - this.tables.put(s.getName(), s); - } else { - throw new SqlClientException( - "Invalid table 'type' attribute value, only 'source' is supported"); + final String name = (String) nameObject; + final Map properties = new HashMap<>(config); + properties.remove(NAME); + + if (this.tables.containsKey(name)) { + throw new SqlClientException("Duplicate table name '" + name + "'."); } + this.tables.put(name, createTableDescriptor(name, properties)); }); } @@ -195,4 +198,31 @@ public static Environment enrich(Environment env, Map properties return enrichedEnv; } + + // -------------------------------------------------------------------------------------------- + + /** + * Creates a table descriptor from a YAML config map. + * + * @param name name of the table + * @param config YAML config map + * @return table descriptor describing a source, sink, or both + */ + private static TableDescriptor createTableDescriptor(String name, Map config) { + final Object typeObject = config.get(TableDescriptorValidator.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 Map normalizedConfig = ConfigUtil.normalizeYaml(config); + if (type.equals(TableDescriptorValidator.TABLE_TYPE_VALUE_SOURCE())) { + return new Source(name, normalizedConfig); + } else if (type.equals(TableDescriptorValidator.TABLE_TYPE_VALUE_SINK())) { + return new Sink(name, normalizedConfig); + } else if (type.equals(TableDescriptorValidator.TABLE_TYPE_VALUE_SOURCE_SINK())) { + return new SourceSink(name, normalizedConfig); + } + throw new SqlClientException("Invalid 'type' attribute for table '" + name + "'. " + + "Only 'source', 'sink', and 'both' are supported."); + } } diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Sink.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Sink.java new file mode 100644 index 0000000000000..eb8c360f3a717 --- /dev/null +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Sink.java @@ -0,0 +1,53 @@ +/* + * 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.client.config; + +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.descriptors.TableSinkDescriptor; + +import java.util.Map; + +/** + * Configuration of a table sink. Parses an entry in the `tables` list of an environment + * file and translates to table descriptor properties. + */ +public class Sink extends TableSinkDescriptor { + private String name; + private Map properties; + + protected Sink(String name, Map properties) { + this.name = name; + this.properties = properties; + } + + public String getName() { + return name; + } + + public Map getProperties() { + return properties; + } + + // -------------------------------------------------------------------------------------------- + + @Override + public void addProperties(DescriptorProperties properties) { + this.properties.forEach(properties::putString); + } +} diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Source.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Source.java index 2bef257863ffe..08e8cf5c7b4d6 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Source.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Source.java @@ -18,11 +18,9 @@ package org.apache.flink.table.client.config; -import org.apache.flink.table.client.SqlClientException; import org.apache.flink.table.descriptors.DescriptorProperties; import org.apache.flink.table.descriptors.TableSourceDescriptor; -import java.util.HashMap; import java.util.Map; /** @@ -34,9 +32,7 @@ public class Source extends TableSourceDescriptor { private String name; private Map properties; - private static final String NAME = "name"; - - private Source(String name, Map properties) { + protected Source(String name, Map properties) { this.name = name; this.properties = properties; } @@ -49,22 +45,6 @@ public Map getProperties() { return properties; } - /** - * Creates a table source descriptor with the given config. - */ - public static Source create(Map config) { - if (!config.containsKey(NAME)) { - throw new SqlClientException("The 'name' attribute of a table source is missing."); - } - final Object name = config.get(NAME); - if (name == null || !(name instanceof String) || ((String) name).length() <= 0) { - throw new SqlClientException("Invalid table source name '" + name + "'."); - } - final Map properties = new HashMap<>(config); - properties.remove(NAME); - return new Source((String) name, ConfigUtil.normalizeYaml(properties)); - } - // -------------------------------------------------------------------------------------------- @Override diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/SourceSink.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/SourceSink.java new file mode 100644 index 0000000000000..c64887e3535b2 --- /dev/null +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/SourceSink.java @@ -0,0 +1,66 @@ +/* + * 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.client.config; + +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.descriptors.TableDescriptor; +import org.apache.flink.table.descriptors.TableDescriptorValidator; + +import java.util.HashMap; +import java.util.Map; + +/** + * Common class for all descriptors describing a table source and sink together. + */ +public class SourceSink extends TableDescriptor { + private String name; + private Map properties; + + protected SourceSink(String name, Map properties) { + this.name = name; + this.properties = properties; + } + + public String getName() { + return name; + } + + public Map getProperties() { + return properties; + } + + @Override + public void addProperties(DescriptorProperties properties) { + this.properties.forEach(properties::putString); + } + + public Source toSource() { + final Map newProperties = new HashMap<>(properties); + newProperties.replace(TableDescriptorValidator.TABLE_TYPE(), + TableDescriptorValidator.TABLE_TYPE_VALUE_SOURCE()); + return new Source(name, newProperties); + } + + public Sink toSink() { + final Map newProperties = new HashMap<>(properties); + newProperties.replace(TableDescriptorValidator.TABLE_TYPE(), + TableDescriptorValidator.TABLE_TYPE_VALUE_SINK()); + return new Sink(name, newProperties); + } +} diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/Executor.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/Executor.java index f12ff4d80127d..9ace240d50545 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/Executor.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/Executor.java @@ -62,7 +62,7 @@ public interface Executor { String explainStatement(SessionContext session, String statement) throws SqlExecutionException; /** - * Submits a Flink job (detached) and returns the result descriptor. + * Submits a Flink SQL query job (detached) and returns the result descriptor. */ ResultDescriptor executeQuery(SessionContext session, String query) throws SqlExecutionException; diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java index 9cd3d8dc05e15..114d5a6d4df7f 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java @@ -46,16 +46,22 @@ 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.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.descriptors.TableSourceDescriptor; +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.descriptors.DescriptorProperties; import org.apache.flink.table.functions.AggregateFunction; import org.apache.flink.table.functions.FunctionService; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.TableFunction; import org.apache.flink.table.functions.UserDefinedFunction; +import org.apache.flink.table.sinks.TableSink; import org.apache.flink.table.sources.TableSource; -import org.apache.flink.table.sources.TableSourceFactoryService; import org.apache.flink.util.FlinkException; import org.apache.commons.cli.CommandLine; @@ -80,6 +86,7 @@ public class ExecutionContext { private final List dependencies; private final ClassLoader classLoader; private final Map> tableSources; + private final Map> tableSinks; private final Map functions; private final Configuration flinkConfig; private final CommandLine commandLine; @@ -100,14 +107,22 @@ public ExecutionContext(Environment defaultEnvironment, SessionContext sessionCo dependencies.toArray(new URL[dependencies.size()]), this.getClass().getClassLoader()); - // create table sources + // create table sources & sinks. tableSources = new HashMap<>(); + tableSinks = new HashMap<>(); mergedEnv.getTables().forEach((name, descriptor) -> { - if (descriptor instanceof TableSourceDescriptor) { - final TableSource tableSource = TableSourceFactoryService.findAndCreateTableSource( - (TableSourceDescriptor) descriptor, - classLoader); - tableSources.put(name, tableSource); + final DescriptorProperties properties = new DescriptorProperties(true); + descriptor.addProperties(properties); + final Map 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)); + } + if (descriptor instanceof Sink || descriptor instanceof SourceSink) { + final TableSinkFactory factory = (TableSinkFactory) + TableFactoryService.find(TableSinkFactory.class, descriptor, classLoader); + tableSinks.put(name, factory.createTableSink(propertyMap)); } }); @@ -224,6 +239,9 @@ private EnvironmentInstance() { // register table sources tableSources.forEach(tableEnv::registerTableSource); + // register table sinks + tableSinks.forEach(tableEnv::registerTableSink); + // register user-defined functions if (tableEnv instanceof StreamTableEnvironment) { StreamTableEnvironment streamTableEnvironment = (StreamTableEnvironment) tableEnv; diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java index d658ee994d130..a178b7e135727 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java @@ -329,6 +329,18 @@ private void cancelQueryInternal(ExecutionContext context, String resultI } } + private void executeUpdateInternal(ExecutionContext context, String query) { + final ExecutionContext.EnvironmentInstance envInst = context.createEnvironmentInstance(); + + envInst.getTableEnvironment().sqlUpdate(query); + // create job graph with dependencies + final String jobName = context.getSessionContext().getName() + ": " + query; + final JobGraph jobGraph = envInst.createJobGraph(jobName); + + // create execution + new Thread(new ProgramDeployer<>(context, jobName, jobGraph, null)).start(); + } + private ResultDescriptor executeQueryInternal(ExecutionContext context, String query) { final ExecutionContext.EnvironmentInstance envInst = context.createEnvironmentInstance(); diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ProgramDeployer.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ProgramDeployer.java index 5c5cf982e478a..b89af321c04ef 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ProgramDeployer.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ProgramDeployer.java @@ -19,6 +19,7 @@ package org.apache.flink.table.client.gateway.local; import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.JobSubmissionResult; import org.apache.flink.client.deployment.ClusterDescriptor; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.rest.RestClusterClient; @@ -62,45 +63,65 @@ public void run() { LOG.debug("Submitting job {} with the following environment: \n{}", jobGraph.getJobID(), context.getMergedEnvironment()); } - executionResultBucket.add(deployJob(context, jobGraph, result)); + if (result != null) { + executionResultBucket.add(deployJob(context, jobGraph, result)); + } else { + deployJob(context, jobGraph, result); + } } public JobExecutionResult fetchExecutionResult() { - return executionResultBucket.poll(); + if (result != null) { + return executionResultBucket.poll(); + } else { + return null; + } } /** - * Deploys a job. Depending on the deployment creates a new job cluster. It saves the cluster id in - * the result and blocks until job completion. + * Deploys a job. Depending on the deployment creates a new job cluster. If result is requested, + * it saves the cluster id in the result and blocks until job completion. */ private JobExecutionResult deployJob(ExecutionContext context, JobGraph jobGraph, DynamicResult result) { + final boolean retrieveResults = result != null; // create or retrieve cluster and deploy job try (final ClusterDescriptor clusterDescriptor = context.createClusterDescriptor()) { ClusterClient clusterClient = null; try { // new cluster if (context.getClusterId() == null) { - // deploy job cluster with job attached - clusterClient = clusterDescriptor.deployJobCluster(context.getClusterSpec(), jobGraph, false); - // save the new cluster id - result.setClusterId(clusterClient.getClusterId()); - // we need to hard cast for now - return ((RestClusterClient) clusterClient) - .requestJobResult(jobGraph.getJobID()) - .get() - .toJobExecutionResult(context.getClassLoader()); // throws exception if job fails + // deploy job cluster, attach the job if result is requested + clusterClient = clusterDescriptor.deployJobCluster( + context.getClusterSpec(), jobGraph, !retrieveResults); + if (retrieveResults) { + // save the new cluster id + result.setClusterId(clusterClient.getClusterId()); + // we need to hard cast for now + return ((RestClusterClient) clusterClient) + .requestJobResult(jobGraph.getJobID()) + .get() + .toJobExecutionResult(context.getClassLoader()); // throws exception if job fails + } else { + return null; + } } // reuse existing cluster else { // retrieve existing cluster clusterClient = clusterDescriptor.retrieve(context.getClusterId()); - // save the cluster id - result.setClusterId(clusterClient.getClusterId()); + if (retrieveResults) { + // save the cluster id + result.setClusterId(clusterClient.getClusterId()); + } // submit the job - clusterClient.setDetached(false); - return clusterClient - .submitJob(jobGraph, context.getClassLoader()) - .getJobExecutionResult(); // throws exception if job fails + clusterClient.setDetached(!retrieveResults); + JobSubmissionResult submissionResult = + clusterClient.submitJob(jobGraph, context.getClassLoader()); + if (retrieveResults) { + return submissionResult.getJobExecutionResult(); + } else { + return null; + } } } catch (Exception e) { throw new SqlExecutionException("Could not retrieve or create a cluster.", e); diff --git a/flink-libraries/flink-sql-client/src/test/assembly/test-table-source-factory.xml b/flink-libraries/flink-sql-client/src/test/assembly/test-table-source-factory.xml index fb9673c593ed3..137d03293a485 100644 --- a/flink-libraries/flink-sql-client/src/test/assembly/test-table-source-factory.xml +++ b/flink-libraries/flink-sql-client/src/test/assembly/test-table-source-factory.xml @@ -32,6 +32,8 @@ under the License. org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.class org/apache/flink/table/client/gateway/utils/TestTableSourceFactory$*.class + org/apache/flink/table/client/gateway/utils/TestTableSinkFactory.class + org/apache/flink/table/client/gateway/utils/TestTableSinkFactory$*.class @@ -40,7 +42,7 @@ under the License. src/test/resources/test-factory-services-file META-INF/services - org.apache.flink.table.sources.TableSourceFactory + org.apache.flink.table.connectors.DiscoverableTableFactory 0755 diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java index 6760583dcfdea..0f892303b9792 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java @@ -42,19 +42,20 @@ public class DependencyTest { private static final String FACTORY_ENVIRONMENT_FILE = "test-sql-client-factory.yaml"; - private static final String TABLE_SOURCE_FACTORY_JAR_FILE = "table-source-factory-test-jar.jar"; + private static final String TABLE_CONNECTOR_FACTORY_JAR_FILE = + "table-connector-factory-test-jar.jar"; @Test - public void testTableSourceFactoryDiscovery() throws Exception { + public void testTableFactoryDiscovery() throws Exception { // create environment final Map replaceVars = new HashMap<>(); - replaceVars.put("$VAR_0", "test-table-source-factory"); + replaceVars.put("$VAR_0", "test-connector"); replaceVars.put("$VAR_1", "test-property"); replaceVars.put("$VAR_2", "test-value"); final Environment env = EnvironmentFileUtil.parseModified(FACTORY_ENVIRONMENT_FILE, replaceVars); // create executor with dependencies - final URL dependency = Paths.get("target", TABLE_SOURCE_FACTORY_JAR_FILE).toUri().toURL(); + final URL dependency = Paths.get("target", TABLE_CONNECTOR_FACTORY_JAR_FILE).toUri().toURL(); final LocalExecutor executor = new LocalExecutor( env, Collections.singletonList(dependency), diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java index 8f62be448e767..b4253d9215c01 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java @@ -69,7 +69,6 @@ public class LocalExecutorITCase extends TestLogger { private static final int NUM_TMS = 2; private static final int NUM_SLOTS_PER_TM = 2; - @ClassRule public static final MiniClusterResource MINI_CLUSTER_RESOURCE = new MiniClusterResource( new MiniClusterResourceConfiguration.Builder() @@ -160,7 +159,7 @@ public void testTableSchema() throws Exception { } @Test(timeout = 30_000L) - public void testStreamQueryExecutionChangelog() throws Exception { + public void testQueryExecutionChangelog() throws Exception { final URL url = getClass().getClassLoader().getResource("test-data.csv"); Objects.requireNonNull(url); final Map replaceVars = new HashMap<>(); diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactory.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactory.java new file mode 100644 index 0000000000000..69c510852ce7c --- /dev/null +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactory.java @@ -0,0 +1,123 @@ +/* + * 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.client.gateway.utils; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.datastream.DataStream; +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.TableSinkFactory; +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.descriptors.SchemaValidator; +import org.apache.flink.table.sinks.AppendStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE; +import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_FROM; +import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_TYPE; +import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_WATERMARKS_TYPE; +import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA; +import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_NAME; +import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_TYPE; + +/** + * Table sink factory for testing the classloading in {@link DependencyTest}. + */ +public class TestTableSinkFactory implements TableSinkFactory, DiscoverableTableFactory { + + @Override + public Map requiredContext() { + final Map context = new HashMap<>(); + context.put(CONNECTOR_TYPE(), "test-connector"); + return context; + } + + @Override + public List supportedProperties() { + final List properties = new ArrayList<>(); + properties.add("connector.test-property"); + properties.add(SCHEMA() + ".#." + SCHEMA_TYPE()); + properties.add(SCHEMA() + ".#." + SCHEMA_NAME()); + properties.add(SCHEMA() + ".#." + ROWTIME_TIMESTAMPS_TYPE()); + properties.add(SCHEMA() + ".#." + ROWTIME_TIMESTAMPS_FROM()); + properties.add(SCHEMA() + ".#." + ROWTIME_WATERMARKS_TYPE()); + return properties; + } + + @Override + public TableSink createTableSink(Map properties) { + final DescriptorProperties params = new DescriptorProperties(true); + params.putProperties(properties); + return new TestTableSink( + SchemaValidator.deriveTableSinkSchema(params), + properties.get("connector.test-property")); + } + + // -------------------------------------------------------------------------------------------- + + /** + * Test table sink. + */ + public static class TestTableSink implements TableSink, AppendStreamTableSink { + + private final TableSchema schema; + private final String property; + + public TestTableSink(TableSchema schema, String property) { + this.schema = schema; + this.property = property; + } + + public String getProperty() { + return property; + } + + @Override + public TypeInformation getOutputType() { + return Types.ROW(schema.getColumnNames(), schema.getTypes()); + } + + @Override + public String[] getFieldNames() { + return schema.getColumnNames(); + } + + @Override + public TypeInformation[] getFieldTypes() { + return schema.getTypes(); + } + + @Override + public TableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { + return new TestTableSink(new TableSchema(fieldNames, fieldTypes), property); + } + + @Override + public void emitDataStream(DataStream dataStream) { + } + } +} diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java index 1b0a30e35615e..34af3f7bc4d2b 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java @@ -24,6 +24,8 @@ 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.descriptors.DescriptorProperties; import org.apache.flink.table.descriptors.SchemaValidator; import org.apache.flink.table.sources.DefinedProctimeAttribute; @@ -31,7 +33,6 @@ import org.apache.flink.table.sources.RowtimeAttributeDescriptor; import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.table.sources.TableSource; -import org.apache.flink.table.sources.TableSourceFactory; import org.apache.flink.types.Row; import java.util.ArrayList; @@ -41,6 +42,7 @@ import java.util.Optional; import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE; +import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_FROM; import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_TYPE; import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_WATERMARKS_TYPE; import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA; @@ -50,12 +52,12 @@ /** * Table source factory for testing the classloading in {@link DependencyTest}. */ -public class TestTableSourceFactory implements TableSourceFactory { +public class TestTableSourceFactory implements TableSourceFactory, DiscoverableTableFactory { @Override public Map requiredContext() { final Map context = new HashMap<>(); - context.put(CONNECTOR_TYPE(), "test-table-source-factory"); + context.put(CONNECTOR_TYPE(), "test-connector"); return context; } @@ -66,18 +68,19 @@ public List supportedProperties() { properties.add(SCHEMA() + ".#." + SCHEMA_TYPE()); properties.add(SCHEMA() + ".#." + SCHEMA_NAME()); properties.add(SCHEMA() + ".#." + ROWTIME_TIMESTAMPS_TYPE()); + properties.add(SCHEMA() + ".#." + ROWTIME_TIMESTAMPS_FROM()); properties.add(SCHEMA() + ".#." + ROWTIME_WATERMARKS_TYPE()); return properties; } @Override - public TableSource create(Map properties) { + public TableSource createTableSource(Map properties) { final DescriptorProperties params = new DescriptorProperties(true); params.putProperties(properties); final Optional proctime = SchemaValidator.deriveProctimeAttribute(params); final List rowtime = SchemaValidator.deriveRowtimeAttributes(params); return new TestTableSource( - params.getTableSchema(SCHEMA()), + SchemaValidator.deriveTableSourceSchema(params), properties.get("connector.test-property"), proctime.orElse(null), rowtime); diff --git a/flink-libraries/flink-sql-client/src/test/resources/test-factory-services-file b/flink-libraries/flink-sql-client/src/test/resources/test-factory-services-file index 41e7fb2cbd561..cf76016dcdc76 100644 --- a/flink-libraries/flink-sql-client/src/test/resources/test-factory-services-file +++ b/flink-libraries/flink-sql-client/src/test/resources/test-factory-services-file @@ -18,3 +18,4 @@ #============================================================================== org.apache.flink.table.client.gateway.utils.TestTableSourceFactory +org.apache.flink.table.client.gateway.utils.TestTableSinkFactory diff --git a/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml b/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml index c7b60978d1916..01ad63d9db0ae 100644 --- a/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml +++ b/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml @@ -25,7 +25,7 @@ tables: - name: TableNumber1 - type: source + type: both schema: - name: IntegerField1 type: INT @@ -35,7 +35,8 @@ tables: type: TIMESTAMP rowtime: timestamps: - type: from-source + type: from-field + from: rowtimeField watermarks: type: from-source connector: diff --git a/flink-libraries/flink-table/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory b/flink-libraries/flink-table/src/main/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory similarity index 94% rename from flink-libraries/flink-table/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory rename to flink-libraries/flink-table/src/main/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory index ff43eed077209..4cda0ad2f40d9 100644 --- a/flink-libraries/flink-table/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory +++ b/flink-libraries/flink-table/src/main/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory @@ -14,3 +14,4 @@ # limitations under the License. org.apache.flink.table.sources.CsvTableSourceFactory +org.apache.flink.table.sinks.CsvTableSinkFactory diff --git a/flink-libraries/flink-table/src/main/resources/tableSourceConverter.properties b/flink-libraries/flink-table/src/main/resources/tableSourceConverter.properties new file mode 100644 index 0000000000000..9a54834ef6fa9 --- /dev/null +++ b/flink-libraries/flink-table/src/main/resources/tableSourceConverter.properties @@ -0,0 +1,36 @@ +################################################################################ +# 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. +################################################################################ + +################################################################################ +# NOTE: THIS APPROACH IS DEPRECATED AND WILL BE REMOVED IN FUTURE VERSIONS! +# +# We recommend to use a org.apache.flink.table.connector.TableConnectorFactory +# instead. They allow to define new factories by using Java Service Providers. +################################################################################ + +################################################################################ +# The config file is used to specify the packages of current module where +# to find TableSourceConverter implementation class annotated with TableType. +# If there are multiple packages to scan, put those packages together into a +# string separated with ',', for example, org.package1,org.package2. +# Please notice: +# It's better to have a tableSourceConverter.properties in each connector Module +# which offers converters instead of put all information into the +# tableSourceConverter.properties of flink-table module. +################################################################################ +scan.packages=org.apache.flink.table.sources diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala index 043a345257efc..fe0bc9e64770d 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala @@ -37,9 +37,9 @@ import org.apache.flink.table.expressions.{Expression, TimeAttribute} import org.apache.flink.table.plan.nodes.FlinkConventions import org.apache.flink.table.plan.nodes.dataset.DataSetRel import org.apache.flink.table.plan.rules.FlinkRuleSets -import org.apache.flink.table.plan.schema.{BatchTableSourceTable, DataSetTable, RowSchema, TableSinkTable} +import org.apache.flink.table.plan.schema._ import org.apache.flink.table.runtime.MapRunner -import org.apache.flink.table.sinks.{BatchTableSink, TableSink} +import org.apache.flink.table.sinks._ import org.apache.flink.table.sources.{BatchTableSource, TableSource} import org.apache.flink.types.Row @@ -104,7 +104,20 @@ abstract class BatchTableEnvironment( tableSource match { case batchTableSource: BatchTableSource[_] => - registerTableInternal(name, new BatchTableSourceTable(batchTableSource)) + Option(getTable(name)) match { + case Some(table: TableSourceSinkTable[_, _]) => table.tableSourceTableOpt match { + case Some(_: TableSourceTable[_]) => + throw new TableException(s"Table \'$name\' already exists. " + + s"Please, choose a different name.") + case _ => replaceRegisteredTable(name, + new TableSourceSinkTable(Some(new BatchTableSourceTable(batchTableSource)), + table.tableSinkTableOpt)) + } + case None => registerTableInternal(name, + new TableSourceSinkTable(Some(new BatchTableSourceTable(batchTableSource)), None)) + case _ => throw new TableException(s"Table \'$name\' already exists. " + + s"Please, choose a different name.") + } case _ => throw new TableException("Only BatchTableSource can be registered in " + "BatchTableEnvironment") @@ -189,10 +202,34 @@ abstract class BatchTableEnvironment( throw new TableException("Same number of field names and types required.") } - tableSink match { + val configuredSink = tableSink.configure(fieldNames, fieldTypes) + registerTableSinkInternal(name, configuredSink) + } + + def registerTableSink(name: String, configuredSink: TableSink[_]): Unit = { + checkValidTableName(name) + if (configuredSink.getFieldNames == null || configuredSink.getFieldTypes == null) { + throw TableException("TableSink is not configured.") + } + registerTableSinkInternal(name, configuredSink) + } + + private def registerTableSinkInternal(name: String, configuredSink: TableSink[_]): Unit = { + configuredSink match { case batchTableSink: BatchTableSink[_] => - val configuredSink = batchTableSink.configure(fieldNames, fieldTypes) - registerTableInternal(name, new TableSinkTable(configuredSink)) + Option(getTable(name)) match { + case Some(table: TableSourceSinkTable[_, _]) => table.tableSinkTableOpt match { + case Some(_: TableSinkTable[_]) => + throw new TableException(s"Table \'$name\' already exists. " + + s"Please, choose a different name.") + case _ => replaceRegisteredTable (name, new TableSourceSinkTable( + table.tableSourceTableOpt, Some(new TableSinkTable(configuredSink)))) + } + case None => registerTableInternal(name, + new TableSourceSinkTable(None, Some(new TableSinkTable(configuredSink)))) + case _ => throw new TableException(s"Table \'$name\' already exists. " + + s"Please, choose a different name.") + } case _ => throw new TableException("Only BatchTableSink can be registered in BatchTableEnvironment.") } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala index 510fe0d7cf9d8..f6ec07555062e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala @@ -43,7 +43,7 @@ import org.apache.flink.table.expressions._ import org.apache.flink.table.plan.nodes.FlinkConventions import org.apache.flink.table.plan.nodes.datastream.{DataStreamRel, UpdateAsRetractionTrait} import org.apache.flink.table.plan.rules.FlinkRuleSets -import org.apache.flink.table.plan.schema.{DataStreamTable, RowSchema, StreamTableSourceTable, TableSinkTable} +import org.apache.flink.table.plan.schema._ import org.apache.flink.table.plan.util.UpdatingPlanChecker import org.apache.flink.table.runtime.conversion._ import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} @@ -122,7 +122,21 @@ abstract class StreamTableEnvironment( s"A rowtime attribute requires an EventTime time characteristic in stream " + s"environment. But is: ${execEnv.getStreamTimeCharacteristic}") } - registerTableInternal(name, new StreamTableSourceTable(streamTableSource)) + Option(getTable(name)) match { + case Some(table: TableSourceSinkTable[_, _]) => table.tableSourceTableOpt match { + case Some(_: TableSourceTable[_]) => + throw new TableException(s"Table \'$name\' already exists. " + + s"Please, choose a different name.") + case _ => replaceRegisteredTable(name, + new TableSourceSinkTable(Some(new StreamTableSourceTable(streamTableSource)), + table.tableSinkTableOpt)) + } + case None => registerTableInternal(name, + new TableSourceSinkTable(Some(new StreamTableSourceTable(streamTableSource)), + None)) + case _ => throw new TableException(s"Table \'$name\' already exists. " + + s"Please, choose a different name.") + } case _ => throw new TableException("Only StreamTableSource can be registered in " + "StreamTableEnvironment") @@ -207,14 +221,39 @@ abstract class StreamTableEnvironment( throw new TableException("Same number of field names and types required.") } - tableSink match { + val configuredSink = tableSink.configure(fieldNames, fieldTypes) + registerTableSinkInternal(name, configuredSink) + } + + def registerTableSink(name: String, configuredSink: TableSink[_]): Unit = { + checkValidTableName(name) + if (configuredSink.getFieldNames == null || configuredSink.getFieldTypes == null) { + throw TableException("TableSink is not configured.") + } + registerTableSinkInternal(name, configuredSink) + } + + private def registerTableSinkInternal(name: String, configuredSink: TableSink[_]): Unit = { + configuredSink match { case streamTableSink@( _: AppendStreamTableSink[_] | _: UpsertStreamTableSink[_] | _: RetractStreamTableSink[_]) => - val configuredSink = streamTableSink.configure(fieldNames, fieldTypes) - registerTableInternal(name, new TableSinkTable(configuredSink)) + Option(getTable(name)) match { + case Some(table: TableSourceSinkTable[_, _]) => table.tableSinkTableOpt match { + case Some(_: TableSinkTable[_]) => + throw new TableException(s"Table \'$name\' already exists. " + + s"Please, choose a different name.") + case _ => replaceRegisteredTable(name, new TableSourceSinkTable( + table.tableSourceTableOpt, + Some(new TableSinkTable(configuredSink)))) + } + case None => registerTableInternal(name, + new TableSourceSinkTable(None, Some(new TableSinkTable(configuredSink)))) + case _ => throw new TableException(s"Table \'$name\' already exists. " + + s"Please, choose a different name.") + } case _ => throw new TableException( "Only AppendStreamTableSink, UpsertStreamTableSink, and RetractStreamTableSink can be " + diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala index 88dc1e922b1ec..66a746e06b5af 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala @@ -56,7 +56,7 @@ import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, Tabl import org.apache.flink.table.plan.cost.DataSetCostFactory import org.apache.flink.table.plan.logical.{CatalogNode, LogicalRelNode} import org.apache.flink.table.plan.rules.FlinkRuleSets -import org.apache.flink.table.plan.schema.{RelTable, RowSchema, TableSinkTable} +import org.apache.flink.table.plan.schema.{RelTable, RowSchema, TableSinkTable, TableSourceSinkTable} import org.apache.flink.table.sinks.TableSink import org.apache.flink.table.sources.TableSource import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo @@ -464,6 +464,16 @@ abstract class TableEnvironment(val config: TableConfig) { fieldTypes: Array[TypeInformation[_]], tableSink: TableSink[_]): Unit + /** + * Registers an external [[TableSink]] which is already configured in this + * [[TableEnvironment]]'s catalog. + * Registered sink tables can be referenced in SQL DML statements. + * + * @param name The name under which the [[TableSink]] is registered. + * @param tableSink The [[TableSink]] to register. + */ + def registerTableSink(name: String, tableSink: TableSink[_]): Unit + /** * Replaces a registered Table with another Table under the same name. * We use this method to replace a [[org.apache.flink.table.plan.schema.DataStreamTable]] @@ -750,8 +760,8 @@ abstract class TableEnvironment(val config: TableConfig) { } getTable(sinkTableName) match { - case s: TableSinkTable[_] => - val tableSink = s.tableSink + case s: TableSourceSinkTable[_, _] if s.tableSinkTableOpt != None => + val tableSink = s.tableSinkTableOpt.get.tableSink // validate schema of source table and table sink val srcFieldTypes = table.getSchema.getTypes val sinkFieldTypes = tableSink.getFieldTypes @@ -771,10 +781,11 @@ abstract class TableEnvironment(val config: TableConfig) { .mkString("[", ", ", "]") throw ValidationException( - s"Field types of query result and registered TableSink $sinkTableName do not match.\n" + + s"Field types of query result and registered TableSink $sinkTableName do not match." + s"Query result schema: $srcSchema\n" + s"TableSink schema: $sinkSchema") } + // emit the table to the configured table sink writeToSink(table, tableSink, conf) case _ => @@ -821,7 +832,7 @@ abstract class TableEnvironment(val config: TableConfig) { rootSchema.getTableNames.contains(name) } - private def getTable(name: String): org.apache.calcite.schema.Table = { + protected def getTable(name: String): org.apache.calcite.schema.Table = { rootSchema.getTable(name) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala index e266a47ef392a..e311727c433db 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala @@ -222,39 +222,35 @@ case class AmbiguousTableFormatException( } /** - * Exception for not finding a [[org.apache.flink.table.sources.TableSourceFactory]] for the - * given properties. + * Exception for not finding a [[org.apache.flink.table.connectors.DiscoverableTableFactory]] for + * the given properties. * - * @param properties properties that describe the table source + * @param properties properties that describe the table connector * @param cause the cause */ -case class NoMatchingTableSourceException( - properties: Map[String, String], - cause: Throwable) - extends RuntimeException( - s"Could not find a table source factory in the classpath satisfying the " + - s"following properties: \n" + - s"${DescriptorProperties.toString(properties)}", - 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.sources.TableSourceFactory]] for - * the given properties. + * Exception for finding more than one + * [[org.apache.flink.table.connectors.DiscoverableTableFactory]] for the given properties. * - * @param properties properties that describe the table source + * @param properties properties that describe the table factory * @param cause the cause */ -case class AmbiguousTableSourceException( - properties: Map[String, String], - cause: Throwable) - extends RuntimeException( - s"More than one table source factory in the classpath satisfying the " + - s"following properties: \n" + - s"${DescriptorProperties.toString(properties)}", - 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) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/BatchTableEnvironment.scala index f8f35eb5c745a..a3d232c647bdf 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/BatchTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/BatchTableEnvironment.scala @@ -23,6 +23,8 @@ import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} import org.apache.flink.table.api._ import org.apache.flink.table.expressions.ExpressionParser import org.apache.flink.table.functions.{AggregateFunction, TableFunction} +import org.apache.flink.table.sinks.TableSink +import org.apache.flink.table.sources.TableSource /** * The [[TableEnvironment]] for a Java batch [[DataSet]] diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableSourceUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableSourceUtil.scala index 1182fc985d3ef..7e9ac21f0d519 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableSourceUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableSourceUtil.scala @@ -19,11 +19,15 @@ package org.apache.flink.table.catalog import org.apache.flink.table.api._ -import org.apache.flink.table.plan.schema.{BatchTableSourceTable, StreamTableSourceTable, TableSourceTable} +import org.apache.flink.table.connectors.{TableFactoryService, TableSourceFactory} +import org.apache.flink.table.descriptors.DescriptorProperties +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, TableSourceFactoryService} +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. */ @@ -38,16 +42,20 @@ object ExternalTableSourceUtil extends Logging { def fromExternalCatalogTable( tableEnv: TableEnvironment, externalCatalogTable: ExternalCatalogTable) - : TableSourceTable[_] = { - val source = TableSourceFactoryService.findAndCreateTableSource(externalCatalogTable) + : TableSourceSinkTable[_, _] = { + val properties = new DescriptorProperties() + externalCatalogTable.addProperties(properties) + val source = TableFactoryService.find(classOf[TableSourceFactory[_]], externalCatalogTable) + .asInstanceOf[TableSourceFactory[_]] + .createTableSource(properties.asMap) tableEnv match { // check for a batch table source in this batch environment case _: BatchTableEnvironment => source match { case bts: BatchTableSource[_] => - new BatchTableSourceTable( + new TableSourceSinkTable(Some(new BatchTableSourceTable( bts, - new FlinkStatistic(externalCatalogTable.getTableStats)) + new FlinkStatistic(externalCatalogTable.getTableStats))), None) case _ => throw new TableException( s"Found table source '${source.getClass.getCanonicalName}' is not applicable " + s"in a batch environment.") @@ -56,9 +64,9 @@ object ExternalTableSourceUtil extends Logging { case _: StreamTableEnvironment => source match { case sts: StreamTableSource[_] => - new StreamTableSourceTable( + new TableSourceSinkTable(Some(new StreamTableSourceTable( sts, - new FlinkStatistic(externalCatalogTable.getTableStats)) + new FlinkStatistic(externalCatalogTable.getTableStats))), None) case _ => throw new TableException( s"Found table source '${source.getClass.getCanonicalName}' is not applicable " + s"in a streaming environment.") @@ -66,5 +74,4 @@ object ExternalTableSourceUtil extends Logging { case _ => throw new TableException("Unsupported table environment.") } } - } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/DiscoverableTableFactory.scala similarity index 69% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactory.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/DiscoverableTableFactory.scala index c16e4d0d89f2e..db5a8860e25e7 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/DiscoverableTableFactory.scala @@ -16,26 +16,17 @@ * limitations under the License. */ -package org.apache.flink.table.sources +package org.apache.flink.table.connectors import java.util /** - * A factory to create a [[TableSource]]. 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 table source. The factory allows for matching to the given set of - * properties and creating a configured [[TableSource]] accordingly. - * - * Classes that implement this interface need to be added to the - * "META_INF/services/org.apache.flink.table.sources.TableSourceFactory' file of a JAR file in - * the current classpath to be found. + * Common trait for all properties-based discoverable table factories. */ -trait TableSourceFactory[T] { +trait DiscoverableTableFactory { /** - * Specifies the context that this factory has been implemented for. The framework guarantees - * to only call the [[create()]] method of the factory if the specified set of properties and - * values are met. + * Specifies the context that this factory has been implemented for. * * Typical properties might be: * - connector.type @@ -74,13 +65,4 @@ trait TableSourceFactory[T] { * and thus might lead to undesired behavior. */ def supportedProperties(): util.List[String] - - /** - * Creates and configures a [[TableSource]] using the given properties. - * - * @param properties normalized properties describing a table source - * @return the configured table source - */ - def create(properties: util.Map[String, String]): TableSource[T] - } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/TableFactoryService.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/TableFactoryService.scala new file mode 100644 index 0000000000000..5ad6e70ed772f --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/TableFactoryService.scala @@ -0,0 +1,160 @@ +/* + * 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) + } + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/TableSinkFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/TableSinkFactory.scala new file mode 100644 index 0000000000000..6346e389dfd44 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/TableSinkFactory.scala @@ -0,0 +1,34 @@ +/* + * 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 org.apache.flink.table.sinks.TableSink + +import java.util + +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] +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/TableSourceFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/TableSourceFactory.scala new file mode 100644 index 0000000000000..bd3130a3f49bb --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/TableSourceFactory.scala @@ -0,0 +1,34 @@ +/* + * 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 org.apache.flink.table.sources.TableSource + +import java.util + +trait TableSourceFactory[T] { + /** + * Creates and configures a [[org.apache.flink.table.sources.TableSource]] + * using the given properties. + * + * @param properties normalized properties describing a table source. + * @return the configured table source. + */ + def createTableSource(properties: util.Map[String, String]): TableSource[T] +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableSourceDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableSourceDescriptor.scala index afdd84c78a9c2..155153fd9ffd7 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableSourceDescriptor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableSourceDescriptor.scala @@ -19,7 +19,8 @@ package org.apache.flink.table.descriptors import org.apache.flink.table.api.{BatchTableEnvironment, Table, TableException, ValidationException} -import org.apache.flink.table.sources.{BatchTableSource, TableSource, TableSourceFactoryService} +import org.apache.flink.table.connectors.{TableFactoryService, TableSourceFactory} +import org.apache.flink.table.sources.{BatchTableSource, TableSource} class BatchTableSourceDescriptor(tableEnv: BatchTableEnvironment, connector: ConnectorDescriptor) extends TableSourceDescriptor { @@ -43,7 +44,11 @@ class BatchTableSourceDescriptor(tableEnv: BatchTableEnvironment, connector: Con * Searches for the specified table source, configures it accordingly, and returns it. */ def toTableSource: TableSource[_] = { - val source = TableSourceFactoryService.findAndCreateTableSource(this) + 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( diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/CsvValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/CsvValidator.scala index d49314ee8a8b6..3329b9f544008 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/CsvValidator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/CsvValidator.scala @@ -49,5 +49,4 @@ object CsvValidator { val FORMAT_IGNORE_FIRST_LINE = "format.ignore-first-line" val FORMAT_IGNORE_PARSE_ERRORS = "format.ignore-parse-errors" val FORMAT_FIELDS = "format.fields" - } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/DescriptorProperties.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/DescriptorProperties.scala index 812b78a0a239a..3ad3eac83d710 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/DescriptorProperties.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/DescriptorProperties.scala @@ -386,7 +386,6 @@ class DescriptorProperties(normalizeKeys: Boolean = true) { if (fieldCount == 0) { return toJava(None) } - // validate fields and build schema val schemaBuilder = TableSchema.builder() for (i <- 0 until fieldCount) { @@ -398,7 +397,7 @@ class DescriptorProperties(normalizeKeys: Boolean = true) { ), TypeStringUtils.readTypeInfo( properties.getOrElse(tpe, throw new ValidationException(s"Invalid table schema. " + - s"Could not find type for field '$key.$i'.")) + s"Could not find type for field '$key.$i'.")) ) ) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FileSystem.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FileSystem.scala index f306b5aa72a50..9e38b3835b1d0 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FileSystem.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FileSystem.scala @@ -18,7 +18,8 @@ package org.apache.flink.table.descriptors -import org.apache.flink.table.descriptors.FileSystemValidator.{CONNECTOR_PATH, CONNECTOR_TYPE_VALUE} +import org.apache.flink.core.fs.FileSystem.WriteMode +import org.apache.flink.table.descriptors.FileSystemValidator._ /** * Connector descriptor for a file system. @@ -27,6 +28,8 @@ class FileSystem extends ConnectorDescriptor( CONNECTOR_TYPE_VALUE, version = 1, formatNeeded = true) { private var path: Option[String] = None + private var numFiles: Option[Int] = None + private var writeMode: Option[String] = None /** * Sets the path to a file or directory in a file system. @@ -38,11 +41,33 @@ class FileSystem extends ConnectorDescriptor( this } + /** + * Set the number of files to write. + * + * @param n number of files + */ + def numFiles(n: Int): FileSystem = { + this.numFiles = Some(n) + this + } + + /** + * Set the write mode. + * + * @param mode write mode. + */ + def writeMode(mode: String): FileSystem = { + this.writeMode = Some(mode) + this + } + /** * Internal method for properties conversion. */ override protected def addConnectorProperties(properties: DescriptorProperties): Unit = { path.foreach(properties.putString(CONNECTOR_PATH, _)) + writeMode.foreach(mode => properties.putString(WRITE_MODE, mode)) + numFiles.foreach(properties.putInt(NUM_FILES, _)) } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FileSystemValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FileSystemValidator.scala index b065b5f127be9..0a4b80f408820 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FileSystemValidator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FileSystemValidator.scala @@ -18,8 +18,11 @@ package org.apache.flink.table.descriptors +import org.apache.flink.core.fs.FileSystem.WriteMode import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE -import org.apache.flink.table.descriptors.FileSystemValidator.{CONNECTOR_PATH, CONNECTOR_TYPE_VALUE} +import org.apache.flink.table.descriptors.FileSystemValidator._ + +import scala.collection.JavaConversions._ /** * Validator for [[FileSystem]]. @@ -30,6 +33,8 @@ class FileSystemValidator extends ConnectorDescriptorValidator { super.validate(properties) properties.validateValue(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE, isOptional = false) properties.validateString(CONNECTOR_PATH, isOptional = false, minLen = 1) + properties.validateInt(NUM_FILES, isOptional = true, min = 1) + properties.validateEnumValues(WRITE_MODE, true, WriteMode.values().map(_.toString).toList) } } @@ -37,5 +42,6 @@ object FileSystemValidator { val CONNECTOR_TYPE_VALUE = "filesystem" val CONNECTOR_PATH = "connector.path" - + val NUM_FILES = "connector.num-files" + val WRITE_MODE = "connector.write-mode" } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala index 64b7c0be5c8af..9643337a3b696 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala @@ -21,7 +21,7 @@ package org.apache.flink.table.descriptors import java.util import java.util.Optional -import org.apache.flink.table.api.{TableSchema, ValidationException} +import org.apache.flink.table.api.{TableException, TableSchema, ValidationException} import org.apache.flink.table.descriptors.DescriptorProperties.{toJava, toScala} import org.apache.flink.table.descriptors.RowtimeValidator._ import org.apache.flink.table.descriptors.SchemaValidator._ @@ -174,6 +174,43 @@ object SchemaValidator { attributes.asJava } + def deriveTableSourceSchema(properties: DescriptorProperties): TableSchema = { + properties.getTableSchema(SCHEMA) + } + + def deriveTableSinkSchema(properties: DescriptorProperties): TableSchema = { + val builder = TableSchema.builder() + + val schema = properties.getTableSchema(SCHEMA) + + schema.getColumnNames.zip(schema.getTypes).zipWithIndex.foreach { case ((n, t), i) => + val isProctime = properties + .getOptionalBoolean(s"$SCHEMA.$i.$SCHEMA_PROCTIME") + .orElse(false) + val tsType = s"$SCHEMA.$i.$ROWTIME_TIMESTAMPS_TYPE" + val isRowtime = properties.containsKey(tsType) + if (!isProctime && !isRowtime) { + // check for a aliasing + val fieldName = properties.getOptionalString(s"$SCHEMA.$i.$SCHEMA_FROM") + .orElse(n) + builder.field(fieldName, t) + } + // only use the rowtime attribute if it references a field + else if (isRowtime) { + properties.getString(tsType) match { + case ROWTIME_TIMESTAMPS_TYPE_VALUE_FROM_FIELD => { + val field = properties.getString(s"$SCHEMA.$i.$ROWTIME_TIMESTAMPS_FROM") + builder.field(field, t) + } + case _ => throw new TableException(s"Unsupported rowtime type for sink table schema: " + + s"${properties.getString(tsType)}") + } + } + } + + builder.build() + } + /** * Finds a table source field mapping. */ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptor.scala index 8f2e4736f166f..3e7c900befcff 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptor.scala @@ -19,7 +19,8 @@ package org.apache.flink.table.descriptors import org.apache.flink.table.api.{StreamTableEnvironment, Table, TableException, ValidationException} -import org.apache.flink.table.sources.{StreamTableSource, TableSource, TableSourceFactoryService} +import org.apache.flink.table.connectors.{TableFactoryService, TableSourceFactory} +import org.apache.flink.table.sources.{StreamTableSource, TableSource} /** * Descriptor for specifying a table source in a streaming environment. @@ -46,7 +47,11 @@ class StreamTableSourceDescriptor(tableEnv: StreamTableEnvironment, connector: C * Searches for the specified table source, configures it accordingly, and returns it. */ def toTableSource: TableSource[_] = { - val source = TableSourceFactoryService.findAndCreateTableSource(this) + 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( diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableDescriptorValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableDescriptorValidator.scala index b868a8abe043f..c712e72602799 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableDescriptorValidator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableDescriptorValidator.scala @@ -31,7 +31,7 @@ class TableDescriptorValidator extends DescriptorValidator { object TableDescriptorValidator { /** - * Key for describing the type of this table, valid values are ('source'). + * Key for describing the type of this table, valid values are ('source', 'sink', 'both'). */ val TABLE_TYPE = "type" @@ -39,5 +39,6 @@ object TableDescriptorValidator { * Valid TABLE_TYPE value. */ val TABLE_TYPE_VALUE_SOURCE = "source" - + val TABLE_TYPE_VALUE_SINK = "sink" + val TABLE_TYPE_VALUE_SOURCE_SINK = "both" } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSinkDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSinkDescriptor.scala new file mode 100644 index 0000000000000..97ed47d65ea54 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSinkDescriptor.scala @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.descriptors + +/** + * Common class for all descriptors describing a table sink. + */ +abstract class TableSinkDescriptor extends TableDescriptor { + override private[flink] def addProperties(properties: DescriptorProperties): Unit = { + super.addProperties(properties) + properties.putString(TableDescriptorValidator.TABLE_TYPE, + TableDescriptorValidator.TABLE_TYPE_VALUE_SOURCE) + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSourceDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSourceDescriptor.scala index 3ca39c2bbfca2..2a0b67ce4f78e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSourceDescriptor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSourceDescriptor.scala @@ -36,6 +36,8 @@ abstract class TableSourceDescriptor extends TableDescriptor { */ override private[flink] def addProperties(properties: DescriptorProperties): Unit = { super.addProperties(properties) + properties.putString(TableDescriptorValidator.TABLE_TYPE, + TableDescriptorValidator.TABLE_TYPE_VALUE_SOURCE) statisticsDescriptor.foreach(_.addProperties(properties)) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala index c9b42438ff12c..f652274a57083 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala @@ -24,7 +24,7 @@ import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.TableScan import org.apache.flink.table.api.TableException import org.apache.flink.table.calcite.FlinkTypeFactory -import org.apache.flink.table.plan.schema.{BatchTableSourceTable, StreamTableSourceTable, TableSourceTable} +import org.apache.flink.table.plan.schema.{BatchTableSourceTable, StreamTableSourceTable, TableSourceSinkTable, TableSourceTable} import org.apache.flink.table.sources.{TableSource, TableSourceUtil} import scala.collection.JavaConverters._ @@ -39,9 +39,12 @@ abstract class PhysicalTableSourceScan( override def deriveRowType(): RelDataType = { val flinkTypeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory] - val streamingTable = table.unwrap(classOf[TableSourceTable[_]]) match { - case _: StreamTableSourceTable[_] => true - case _: BatchTableSourceTable[_] => false + val streamingTable = table.unwrap(classOf[TableSourceSinkTable[_, _]]) match { + case t: TableSourceSinkTable[_, _] => t.tableSourceTableOpt match { + case Some(_: StreamTableSourceTable[_]) => true + case Some(_: BatchTableSourceTable[_]) => false + case _ => throw TableException(s"Unknown Table type ${t.getClass}.") + } case t => throw TableException(s"Unknown Table type ${t.getClass}.") } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableFunctionScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableFunctionScan.scala index 07b48a31643d5..4bf1ca1f9f774 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableFunctionScan.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableFunctionScan.scala @@ -30,7 +30,6 @@ import org.apache.calcite.rel.logical.{LogicalTableFunctionScan, LogicalTableSca import org.apache.calcite.rel.metadata.RelColumnMapping import org.apache.calcite.rex.RexNode import org.apache.flink.table.plan.nodes.FlinkConventions -import org.apache.flink.table.plan.schema.TableSourceTable class FlinkLogicalTableFunctionScan( cluster: RelOptCluster, diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala index ca8e56921390c..1d3818864a084 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala @@ -28,7 +28,7 @@ import org.apache.calcite.rel.{RelNode, RelWriter} import org.apache.flink.table.api.TableException import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.plan.nodes.FlinkConventions -import org.apache.flink.table.plan.schema.{BatchTableSourceTable, StreamTableSourceTable, TableSourceTable} +import org.apache.flink.table.plan.schema.{BatchTableSourceTable, StreamTableSourceTable, TableSourceSinkTable, TableSourceTable} import org.apache.flink.table.sources.{FilterableTableSource, TableSource, TableSourceUtil} import scala.collection.JavaConverters._ @@ -51,9 +51,12 @@ class FlinkLogicalTableSourceScan( override def deriveRowType(): RelDataType = { val flinkTypeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory] - val streamingTable = table.unwrap(classOf[TableSourceTable[_]]) match { - case _: StreamTableSourceTable[_] => true - case _: BatchTableSourceTable[_] => false + val streamingTable = table.unwrap(classOf[TableSourceSinkTable[_, _]]) match { + case t: TableSourceSinkTable[_, _] => t.tableSourceTableOpt match { + case Some(_: StreamTableSourceTable[_]) => true + case Some(_: BatchTableSourceTable[_]) => false + case _ => throw TableException(s"Unknown Table type ${t.getClass}.") + } case t => throw TableException(s"Unknown Table type ${t.getClass}.") } @@ -111,9 +114,9 @@ class FlinkLogicalTableSourceScanConverter override def matches(call: RelOptRuleCall): Boolean = { val scan = call.rel[TableScan](0) - val tableSourceTable = scan.getTable.unwrap(classOf[TableSourceTable[_]]) - tableSourceTable match { - case _: TableSourceTable[_] => true + val tableSourceSinkTable = scan.getTable.unwrap(classOf[TableSourceSinkTable[_, _]]) + tableSourceSinkTable match { + case t: TableSourceSinkTable[_, _] if t.tableSourceTableOpt != None => true case _ => false } } @@ -121,7 +124,8 @@ class FlinkLogicalTableSourceScanConverter def convert(rel: RelNode): RelNode = { val scan = rel.asInstanceOf[TableScan] val traitSet = rel.getTraitSet.replace(FlinkConventions.LOGICAL) - val tableSource = scan.getTable.unwrap(classOf[TableSourceTable[_]]).tableSource + val tableSource = scan.getTable.unwrap(classOf[TableSourceSinkTable[_, _]]) + .tableSourceTableOpt.map(_.tableSource).orNull new FlinkLogicalTableSourceScan( rel.getCluster, diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/BatchTableSourceScanRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/BatchTableSourceScanRule.scala index 32ca12785f5ad..a4fd151f690d0 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/BatchTableSourceScanRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/BatchTableSourceScanRule.scala @@ -24,7 +24,7 @@ import org.apache.calcite.rel.convert.ConverterRule import org.apache.calcite.rel.core.TableScan import org.apache.flink.table.plan.nodes.FlinkConventions import org.apache.flink.table.plan.nodes.dataset.BatchTableSourceScan -import org.apache.flink.table.plan.schema.TableSourceTable +import org.apache.flink.table.plan.schema.{TableSourceSinkTable, TableSourceTable} import org.apache.flink.table.plan.nodes.logical.FlinkLogicalTableSourceScan import org.apache.flink.table.sources.BatchTableSource @@ -38,10 +38,10 @@ class BatchTableSourceScanRule /** Rule must only match if TableScan targets a [[BatchTableSource]] */ override def matches(call: RelOptRuleCall): Boolean = { val scan: TableScan = call.rel(0).asInstanceOf[TableScan] - val dataSetTable = scan.getTable.unwrap(classOf[TableSourceTable[_]]) - dataSetTable match { - case tst: TableSourceTable[_] => - tst.tableSource match { + val dataSetTable = scan.getTable.unwrap(classOf[TableSourceSinkTable[_, _]]) + dataSetTable.tableSourceTableOpt match { + case Some(tt: TableSourceTable[_]) => + tt.tableSource match { case _: BatchTableSource[_] => true case _ => diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/StreamTableSourceScanRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/StreamTableSourceScanRule.scala index fca1557e3097f..62fef2bf25cc9 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/StreamTableSourceScanRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/StreamTableSourceScanRule.scala @@ -24,7 +24,7 @@ import org.apache.calcite.rel.convert.ConverterRule import org.apache.calcite.rel.core.TableScan import org.apache.flink.table.plan.nodes.FlinkConventions import org.apache.flink.table.plan.nodes.datastream.StreamTableSourceScan -import org.apache.flink.table.plan.schema.TableSourceTable +import org.apache.flink.table.plan.schema.{TableSourceSinkTable, TableSourceTable} import org.apache.flink.table.plan.nodes.logical.FlinkLogicalTableSourceScan import org.apache.flink.table.sources.StreamTableSource @@ -39,9 +39,9 @@ class StreamTableSourceScanRule /** Rule must only match if TableScan targets a [[StreamTableSource]] */ override def matches(call: RelOptRuleCall): Boolean = { val scan: TableScan = call.rel(0).asInstanceOf[TableScan] - val dataSetTable = scan.getTable.unwrap(classOf[TableSourceTable[_]]) - dataSetTable match { - case tst: TableSourceTable[_] => + val dataSetTable = scan.getTable.unwrap(classOf[TableSourceSinkTable[_, _]]) + dataSetTable.tableSourceTableOpt match { + case Some(tst: TableSourceTable[_]) => tst.tableSource match { case _: StreamTableSource[_] => true diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala index e87fa35d97656..6754db2a07fa3 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala @@ -24,7 +24,7 @@ import org.apache.calcite.plan.RelOptRule.{none, operand} import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} import org.apache.calcite.rex.RexProgram import org.apache.flink.table.expressions.Expression -import org.apache.flink.table.plan.schema.TableSourceTable +import org.apache.flink.table.plan.schema.{TableSourceSinkTable, TableSourceTable} import org.apache.flink.table.plan.util.RexProgramExtractor import org.apache.flink.table.plan.nodes.logical.{FlinkLogicalCalc, FlinkLogicalTableSourceScan} import org.apache.flink.table.sources.FilterableTableSource @@ -51,16 +51,14 @@ class PushFilterIntoTableSourceScanRule extends RelOptRule( override def onMatch(call: RelOptRuleCall): Unit = { val calc: FlinkLogicalCalc = call.rel(0).asInstanceOf[FlinkLogicalCalc] val scan: FlinkLogicalTableSourceScan = call.rel(1).asInstanceOf[FlinkLogicalTableSourceScan] - val tableSourceTable = scan.getTable.unwrap(classOf[TableSourceTable[_]]) val filterableSource = scan.tableSource.asInstanceOf[FilterableTableSource[_]] - pushFilterIntoScan(call, calc, scan, tableSourceTable, filterableSource, description) + pushFilterIntoScan(call, calc, scan, filterableSource, description) } private def pushFilterIntoScan( call: RelOptRuleCall, calc: FlinkLogicalCalc, scan: FlinkLogicalTableSourceScan, - tableSourceTable: TableSourceTable[_], filterableSource: FilterableTableSource[_], description: String): Unit = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/StreamTableSourceTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/StreamTableSourceTable.scala index 9e82313643620..38b7df851e542 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/StreamTableSourceTable.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/StreamTableSourceTable.scala @@ -32,7 +32,7 @@ class StreamTableSourceTable[T]( TableSourceUtil.validateTableSource(tableSource) - override def getRowType(typeFactory: RelDataTypeFactory): RelDataType = { + def getRowType(typeFactory: RelDataTypeFactory): RelDataType = { TableSourceUtil.getRelDataType( tableSource, None, diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSinkTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSinkTable.scala index f5e80d579f930..75ce3dad16606 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSinkTable.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSinkTable.scala @@ -25,13 +25,17 @@ import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.plan.stats.FlinkStatistic import org.apache.flink.table.sinks.TableSink -/** Table which defines an external table via a [[TableSink]] */ +/** Class which implements the logic to convert a [[TableSink]] to Calcite Table */ class TableSinkTable[T]( val tableSink: TableSink[T], - val statistic: FlinkStatistic = FlinkStatistic.UNKNOWN) - extends AbstractTable { + val statistic: FlinkStatistic = FlinkStatistic.UNKNOWN) { - override def getRowType(typeFactory: RelDataTypeFactory): RelDataType = { + /** Returns the row type of the table with this tableSink. + * + * @param typeFactory Type factory with which to create the type + * @return Row type + */ + def getRowType(typeFactory: RelDataTypeFactory): RelDataType = { val flinkTypeFactory = typeFactory.asInstanceOf[FlinkTypeFactory] flinkTypeFactory.buildLogicalRowType(tableSink.getFieldNames, tableSink.getFieldTypes) } @@ -41,5 +45,5 @@ class TableSinkTable[T]( * * @return statistics of current table */ - override def getStatistic: Statistic = statistic + def getStatistic: Statistic = statistic } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceSinkTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceSinkTable.scala new file mode 100644 index 0000000000000..042293507bca0 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceSinkTable.scala @@ -0,0 +1,43 @@ +/* + * 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.plan.schema + +import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory} +import org.apache.calcite.schema.Statistic +import org.apache.calcite.schema.impl.AbstractTable + +class TableSourceSinkTable[T1, T2](val tableSourceTableOpt: Option[TableSourceTable[T1]], + val tableSinkTableOpt: Option[TableSinkTable[T2]]) + extends AbstractTable { + + // In streaming case, the table schema as source and sink can differ because of extra + // rowtime/proctime fields. We will always return the source table schema if tableSourceTable + // is not None, otherwise return the sink table schema. We move the Calcite validation logic of + // the sink table schema into Flink. This allows us to have different schemas as source and sink + // of the same table. + override def getRowType(typeFactory: RelDataTypeFactory): RelDataType = { + tableSourceTableOpt.map(_.getRowType(typeFactory)) + .orElse(tableSinkTableOpt.map(_.getRowType(typeFactory))).orNull + } + + override def getStatistic: Statistic = { + tableSourceTableOpt.map(_.getStatistic) + .orElse(tableSinkTableOpt.map(_.getStatistic)).orNull + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceTable.scala index 048e862649dbc..26224b61c03de 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceTable.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceTable.scala @@ -18,22 +18,29 @@ package org.apache.flink.table.plan.schema +import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory} import org.apache.calcite.schema.Statistic -import org.apache.calcite.schema.impl.AbstractTable import org.apache.flink.table.plan.stats.FlinkStatistic import org.apache.flink.table.sources.TableSource -/** Table which defines an external table via a [[TableSource]] */ +/** Abstract class which define the interfaces required to convert a [[TableSource]] to + * a Calcite Table */ abstract class TableSourceTable[T]( val tableSource: TableSource[T], - val statistic: FlinkStatistic) - extends AbstractTable { + val statistic: FlinkStatistic) { + + /** Returns the row type of the table with this tableSource. + * + * @param typeFactory Type factory with which to create the type + * @return Row type + */ + def getRowType(typeFactory: RelDataTypeFactory): RelDataType /** * Returns statistics of current table * * @return statistics of current table */ - override def getStatistic: Statistic = statistic + def getStatistic: Statistic = statistic } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala index f98d03a4d4a16..b2ad5b2d3ed5a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala @@ -21,6 +21,7 @@ package org.apache.flink.table.sinks import org.apache.flink.api.common.functions.MapFunction import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.DataSet +import org.apache.flink.api.java.io.CsvInputFormat import org.apache.flink.types.Row import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.core.fs.FileSystem.WriteMode @@ -138,3 +139,94 @@ class CsvFormatter(fieldDelim: String) extends MapFunction[Row, String] { builder.mkString } } + +object CsvTableSink { + + /** + * A builder for creating [[CsvTableSink]] instances. + * + * For example: + * + * {{{ + * val sink: CsvTableSink = new CsvTableSink.builder() + * .path("/path/to/your/file.csv") + * .build() + * }}} + * + */ + class Builder { + + private var path: String = _ + private var fieldDelimOpt: Option[String] = Some(CsvInputFormat.DEFAULT_FIELD_DELIMITER) + private var numFilesOpt: Option[Int] = None + private var writeModeOpt: Option[WriteMode] = None + + /** + * Sets the path to the CSV file. Required. + * + * @param path the path to the CSV file + */ + def path(path: String): Builder = { + this.path = path + this + } + + /** + * Sets the field delimiter, "," by default. + * + * @param delim the field delimiter + */ + def fieldDelimiter(delim: String): Builder = { + this.fieldDelimOpt = Some(delim) + this + } + + /** + * Sets the number of files to write to + * + * @param numFiles the number of files + */ + def numFiles(numFiles: Int): Builder = { + this.numFilesOpt = Some(numFiles) + this + } + + /** + * Sets the write mode + * + * @param mode the write mode + */ + def writeMode(mode: String): Builder = { + this.writeModeOpt = Some(WriteMode.valueOf(mode)) + this + } + + /** + * Apply the current values and constructs a newly-created [[CsvTableSink]]. + * + * @return a newly-created [[CsvTableSink]]. + */ + def build(): CsvTableSink = { + if (path == null) { + throw new IllegalArgumentException("Path must be defined.") + } + new CsvTableSink(path, fieldDelimOpt, numFilesOpt, writeModeOpt) + } + + } + + /** + * Return a new builder that builds a [[CsvTableSink]]. + * + * For example: + * + * {{{ + * val sink: CsvTableSink = CsvTableSink + * .builder() + * .path("/path/to/your/file.csv") + * .build() + * }}} + * @return a new builder to build a [[CsvTableSink]] + */ + def builder(): Builder = new Builder +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala new file mode 100644 index 0000000000000..c333778e0b98b --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala @@ -0,0 +1,96 @@ +/* + * 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.sinks + +import java.util + +import org.apache.flink.table.api.TableException +import org.apache.flink.table.connectors.{DiscoverableTableFactory, TableSinkFactory} +import org.apache.flink.table.descriptors.ConnectorDescriptorValidator._ +import org.apache.flink.table.descriptors.CsvValidator._ +import org.apache.flink.table.descriptors.DescriptorProperties._ +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.types.Row + +/** + * Factory for creating configured instances of [[CsvTableSink]]. + */ +class CsvTableSinkFactory extends TableSinkFactory[Row] with DiscoverableTableFactory { + + override def requiredContext(): util.Map[String, String] = { + val context = new util.HashMap[String, String]() + context.put(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE) + context.put(FORMAT_TYPE, FORMAT_TYPE_VALUE) + context.put(CONNECTOR_PROPERTY_VERSION, "1") + context.put(FORMAT_PROPERTY_VERSION, "1") + context + } + + override def supportedProperties(): util.List[String] = { + val properties = new util.ArrayList[String]() + // connector + properties.add(CONNECTOR_PATH) + // format + properties.add(s"$FORMAT_FIELDS.#.${DescriptorProperties.TYPE}") + properties.add(s"$FORMAT_FIELDS.#.${DescriptorProperties.NAME}") + properties.add(FORMAT_FIELD_DELIMITER) + properties.add(CONNECTOR_PATH) + properties.add(NUM_FILES) + properties.add(WRITE_MODE) + // schema + properties.add(s"$SCHEMA.#.${DescriptorProperties.TYPE}") + properties.add(s"$SCHEMA.#.${DescriptorProperties.NAME}") + properties + } + + override def createTableSink(properties: util.Map[String, String]): TableSink[Row] = { + val params = new DescriptorProperties() + params.putProperties(properties) + + // validate + new FileSystemValidator().validate(params) + new CsvValidator().validate(params) + new SchemaValidator().validate(params) + + // build + val csvTableSinkBuilder = new CsvTableSink.Builder + + val formatSchema = params.getTableSchema(FORMAT_FIELDS) + val tableSchema = SchemaValidator.deriveTableSinkSchema(params) + + if (!formatSchema.equals(tableSchema)) { + throw new TableException( + "Encodings that differ from the schema are not supported yet for CsvTableSink.") + } + + toScala(params.getOptionalString(CONNECTOR_PATH)) + .foreach(csvTableSinkBuilder.path) + toScala(params.getOptionalInt(NUM_FILES)) + .foreach(n => csvTableSinkBuilder.numFiles(n)) + toScala(params.getOptionalString(WRITE_MODE)) + .foreach(csvTableSinkBuilder.writeMode) + toScala(params.getOptionalString(FORMAT_FIELD_DELIMITER)) + .foreach(csvTableSinkBuilder.fieldDelimiter) + + csvTableSinkBuilder.build().configure(formatSchema.getColumnNames, formatSchema.getTypes) + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala index a451afff76d67..c8e450380185a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala @@ -21,6 +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.descriptors.ConnectorDescriptorValidator.{CONNECTOR_PROPERTY_VERSION, CONNECTOR_TYPE} import org.apache.flink.table.descriptors.CsvValidator._ import org.apache.flink.table.descriptors.DescriptorProperties.toScala @@ -33,7 +34,7 @@ import org.apache.flink.types.Row /** * Factory for creating configured instances of [[CsvTableSource]]. */ -class CsvTableSourceFactory extends TableSourceFactory[Row] { +class CsvTableSourceFactory extends TableSourceFactory[Row] with DiscoverableTableFactory { override def requiredContext(): util.Map[String, String] = { val context = new util.HashMap[String, String]() @@ -64,7 +65,7 @@ class CsvTableSourceFactory extends TableSourceFactory[Row] { properties } - override def create(properties: util.Map[String, String]): TableSource[Row] = { + override def createTableSource(properties: util.Map[String, String]): TableSource[Row] = { val params = new DescriptorProperties() params.putProperties(properties) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactoryService.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactoryService.scala deleted file mode 100644 index 94aab7202cfae..0000000000000 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactoryService.scala +++ /dev/null @@ -1,181 +0,0 @@ -/* - * 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.sources - -import java.util.{ServiceConfigurationError, ServiceLoader} - -import org.apache.flink.table.api.{AmbiguousTableSourceException, NoMatchingTableSourceException, TableException, ValidationException} -import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_PROPERTY_VERSION -import org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_PROPERTY_VERSION -import org.apache.flink.table.descriptors.MetadataValidator.METADATA_PROPERTY_VERSION -import org.apache.flink.table.descriptors.StatisticsValidator.STATISTICS_PROPERTY_VERSION -import org.apache.flink.table.descriptors._ -import org.apache.flink.table.util.Logging - -import scala.collection.JavaConverters._ -import scala.collection.mutable - -/** - * Service provider interface for finding suitable table source factories for the given properties. - */ -object TableSourceFactoryService extends Logging { - - private lazy val defaultLoader = ServiceLoader.load(classOf[TableSourceFactory[_]]) - - def findAndCreateTableSource(descriptor: TableSourceDescriptor): TableSource[_] = { - findAndCreateTableSource(descriptor, null) - } - - def findAndCreateTableSource( - descriptor: TableSourceDescriptor, - classLoader: ClassLoader) - : TableSource[_] = { - - val properties = new DescriptorProperties() - descriptor.addProperties(properties) - findAndCreateTableSource(properties.asMap.asScala.toMap, classLoader) - } - - def findAndCreateTableSource(properties: Map[String, String]): TableSource[_] = { - findAndCreateTableSource(properties, null) - } - - def findAndCreateTableSource( - properties: Map[String, String], - classLoader: ClassLoader) - : TableSource[_] = { - - var matchingFactory: Option[(TableSourceFactory[_], Seq[String])] = None - try { - val iter = if (classLoader == null) { - defaultLoader.iterator() - } else { - val customLoader = ServiceLoader.load(classOf[TableSourceFactory[_]], classLoader) - customLoader.iterator() - } - while (iter.hasNext) { - val factory = iter.next() - - 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) - - // check if required context is met - if (plainContext.forall(e => properties.contains(e._1) && properties(e._1) == e._2)) { - matchingFactory match { - case Some(_) => throw new AmbiguousTableSourceException(properties) - case None => matchingFactory = Some((factory, requiredContext.keys.toSeq)) - } - } - } - } catch { - case e: ServiceConfigurationError => - LOG.error("Could not load service provider for table source factories.", e) - throw new TableException("Could not load service provider for table source factories.", e) - } - - val (factory, context) = matchingFactory - .getOrElse(throw new NoMatchingTableSourceException(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]() - } - - // extract wildcard prefixes - val wildcards = extractWildcardPrefixes(supportedProperties) - - // check for supported properties - validateSupportedProperties(factory, supportedProperties, wildcards, plainProperties) - - // create the table source - try { - factory.create(properties.asJava) - } catch { - case t: Throwable => - throw new TableException( - s"Table source factory '${factory.getClass.getCanonicalName}' caused an exception.", - t) - } - } - - private def extractWildcardPrefixes(propertyKeys: Seq[String]): Seq[String] = { - propertyKeys - .filter(_.endsWith("*")) - .map(s => s.substring(0, s.length - 1)) - } - - private def validateSupportedProperties[T]( - factory: TableSourceFactory[T], - supportedProperties: Seq[String], - wildcards: Seq[String], - plainProperties: Seq[String]) - : Unit = { - - plainProperties.foreach { k => - if (!supportedProperties.contains(k) && !wildcards.exists(k.startsWith)) { - 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")}") - } - } - } -} diff --git a/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory b/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory similarity index 65% rename from flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory rename to flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory index 06bf8536c73de..45f1cf6a73c95 100644 --- a/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory +++ b/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory @@ -13,5 +13,10 @@ # See the License for the specific language governing permissions and # limitations under the License. +<<<<<<< HEAD:flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory org.apache.flink.table.sources.TestFixedFormatTableFactory org.apache.flink.table.sources.TestWildcardFormatTableSourceFactory +======= +org.apache.flink.table.connectors.TestTableSinkFactory +org.apache.flink.table.connectors.TestTableSourceFactory +>>>>>>> 2fce6f6765... [FLINK-8866] [table] Create unified interfaces to configure and instatiate TableSinks:flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/validation/InsertIntoValidationTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/validation/InsertIntoValidationTest.scala index ef9e6a35de6a1..ff510d3593beb 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/validation/InsertIntoValidationTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/validation/InsertIntoValidationTest.scala @@ -22,7 +22,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala._ import org.apache.flink.table.api.scala._ import org.apache.flink.table.api.{Types, ValidationException} -import org.apache.flink.table.utils.{MemoryTableSinkUtil, TableTestBase} +import org.apache.flink.table.utils.{MemoryTableSourceSinkUtil, TableTestBase} import org.junit._ class InsertIntoValidationTest extends TableTestBase { @@ -34,7 +34,7 @@ class InsertIntoValidationTest extends TableTestBase { val fieldNames = Array("d", "e") val fieldTypes: Array[TypeInformation[_]] = Array(Types.INT, Types.LONG) - val sink = new MemoryTableSinkUtil.UnsafeMemoryAppendTableSink + val sink = new MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink util.tableEnv.registerTableSink("targetTable", fieldNames, fieldTypes, sink) val sql = "INSERT INTO targetTable SELECT a, b, c FROM sourceTable" @@ -50,7 +50,7 @@ class InsertIntoValidationTest extends TableTestBase { val fieldNames = Array("d", "e", "f") val fieldTypes: Array[TypeInformation[_]] = Array(Types.STRING, Types.INT, Types.LONG) - val sink = new MemoryTableSinkUtil.UnsafeMemoryAppendTableSink + val sink = new MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink util.tableEnv.registerTableSink("targetTable", fieldNames, fieldTypes, sink) val sql = "INSERT INTO targetTable SELECT a, b, c FROM sourceTable" @@ -66,7 +66,7 @@ class InsertIntoValidationTest extends TableTestBase { val fieldNames = Array("d", "e", "f") val fieldTypes = util.tableEnv.scan("sourceTable").getSchema.getTypes - val sink = new MemoryTableSinkUtil.UnsafeMemoryAppendTableSink + val sink = new MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink util.tableEnv.registerTableSink("targetTable", fieldNames, fieldTypes, sink) val sql = "INSERT INTO targetTable (d, f) SELECT a, c FROM sourceTable" diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/validation/InsertIntoValidationTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/validation/InsertIntoValidationTest.scala index 2cfe9312ce84c..27cf854dc4d9a 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/validation/InsertIntoValidationTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/validation/InsertIntoValidationTest.scala @@ -22,7 +22,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala._ import org.apache.flink.table.api.{Types, ValidationException} import org.apache.flink.table.api.scala._ -import org.apache.flink.table.utils.{MemoryTableSinkUtil, TableTestBase} +import org.apache.flink.table.utils.{MemoryTableSourceSinkUtil, TableTestBase} import org.junit._ class InsertIntoValidationTest extends TableTestBase { @@ -34,7 +34,7 @@ class InsertIntoValidationTest extends TableTestBase { val fieldNames = Array("d", "e") val fieldTypes: Array[TypeInformation[_]] = Array(Types.INT, Types.LONG) - val sink = new MemoryTableSinkUtil.UnsafeMemoryAppendTableSink + val sink = new MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink util.tableEnv.registerTableSink("targetTable", fieldNames, fieldTypes, sink) // must fail because TableSink accepts fewer fields. @@ -50,7 +50,7 @@ class InsertIntoValidationTest extends TableTestBase { val fieldNames = Array("d", "e", "f") val fieldTypes: Array[TypeInformation[_]] = Array(Types.STRING, Types.INT, Types.LONG) - val sink = new MemoryTableSinkUtil.UnsafeMemoryAppendTableSink + val sink = new MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink util.tableEnv.registerTableSink("targetTable", fieldNames, fieldTypes, sink) // must fail because types of result and TableSink do not match. diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/validation/InsertIntoValidationTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/validation/InsertIntoValidationTest.scala index 3045100bea084..1343473fbeed8 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/validation/InsertIntoValidationTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/validation/InsertIntoValidationTest.scala @@ -23,7 +23,7 @@ import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.table.api.scala._ import org.apache.flink.table.api.{TableEnvironment, Types, ValidationException} import org.apache.flink.table.runtime.utils.StreamTestData -import org.apache.flink.table.utils.MemoryTableSinkUtil +import org.apache.flink.table.utils.MemoryTableSourceSinkUtil import org.junit.Test class InsertIntoValidationTest { @@ -38,7 +38,7 @@ class InsertIntoValidationTest { val fieldNames = Array("d", "e") val fieldTypes: Array[TypeInformation[_]] = Array(Types.INT, Types.LONG) - val sink = new MemoryTableSinkUtil.UnsafeMemoryAppendTableSink + val sink = new MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink tEnv.registerTableSink("targetTable", fieldNames, fieldTypes, sink) val sql = "INSERT INTO targetTable SELECT a, b, c FROM sourceTable" @@ -57,7 +57,7 @@ class InsertIntoValidationTest { val fieldNames = Array("d", "e", "f") val fieldTypes: Array[TypeInformation[_]] = Array(Types.STRING, Types.INT, Types.LONG) - val sink = new MemoryTableSinkUtil.UnsafeMemoryAppendTableSink + val sink = new MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink tEnv.registerTableSink("targetTable", fieldNames, fieldTypes, sink) val sql = "INSERT INTO targetTable SELECT a, b, c FROM sourceTable" @@ -76,7 +76,7 @@ class InsertIntoValidationTest { val fieldNames = Array("d", "e", "f") val fieldTypes = tEnv.scan("sourceTable").getSchema.getTypes - val sink = new MemoryTableSinkUtil.UnsafeMemoryAppendTableSink + val sink = new MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink tEnv.registerTableSink("targetTable", fieldNames, fieldTypes, sink) val sql = "INSERT INTO targetTable (d, f) SELECT a, c FROM sourceTable" diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/InsertIntoValidationTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/InsertIntoValidationTest.scala index 2fcfd6cd3a188..557dc8ef97dc4 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/InsertIntoValidationTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/validation/InsertIntoValidationTest.scala @@ -23,7 +23,7 @@ import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.table.api.scala._ import org.apache.flink.table.api.{TableEnvironment, Types, ValidationException} import org.apache.flink.table.runtime.utils.StreamTestData -import org.apache.flink.table.utils.MemoryTableSinkUtil +import org.apache.flink.table.utils.MemoryTableSourceSinkUtil import org.junit.Test class InsertIntoValidationTest { @@ -38,7 +38,7 @@ class InsertIntoValidationTest { val fieldNames = Array("d", "f") val fieldTypes: Array[TypeInformation[_]] = Array(Types.INT, Types.LONG) - val sink = new MemoryTableSinkUtil.UnsafeMemoryAppendTableSink + val sink = new MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink tEnv.registerTableSink("targetTable", fieldNames, fieldTypes, sink) // must fail because table sink has too few fields. @@ -57,7 +57,7 @@ class InsertIntoValidationTest { val fieldNames = Array("d", "e", "f") val fieldTypes: Array[TypeInformation[_]] = Array(Types.STRING, Types.INT, Types.LONG) - val sink = new MemoryTableSinkUtil.UnsafeMemoryAppendTableSink + val sink = new MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink tEnv.registerTableSink("targetTable", fieldNames, fieldTypes, sink) // must fail because field types of table sink are incompatible. diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/validation/TableSinksValidationTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/validation/TableSinksValidationTest.scala index 628925b632ad2..90a12ee2f70c1 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/validation/TableSinksValidationTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/validation/TableSinksValidationTest.scala @@ -23,7 +23,7 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api.{TableException, Types} import org.apache.flink.table.api.scala._ import org.apache.flink.table.runtime.stream.table.TestAppendSink -import org.apache.flink.table.utils.MemoryTableSinkUtil.UnsafeMemoryAppendTableSink +import org.apache.flink.table.utils.MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink import org.apache.flink.table.utils.TableTestBase import org.junit.Test diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/catalog/ExternalCatalogSchemaTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/catalog/ExternalCatalogSchemaTest.scala index 4f7b90aa1e7da..434c0cdf92521 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/catalog/ExternalCatalogSchemaTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/catalog/ExternalCatalogSchemaTest.scala @@ -27,7 +27,7 @@ import org.apache.calcite.prepare.CalciteCatalogReader import org.apache.calcite.schema.SchemaPlus import org.apache.calcite.sql.validate.SqlMonikerType import org.apache.flink.table.calcite.{FlinkTypeFactory, FlinkTypeSystem} -import org.apache.flink.table.plan.schema.TableSourceTable +import org.apache.flink.table.plan.schema.{TableSourceSinkTable, TableSourceTable} import org.apache.flink.table.runtime.utils.CommonTestData import org.apache.flink.table.sources.CsvTableSource import org.apache.flink.table.utils.TableTestBase @@ -78,9 +78,9 @@ class ExternalCatalogSchemaTest extends TableTestBase { def testGetTable(): Unit = { val relOptTable = calciteCatalogReader.getTable(Lists.newArrayList(schemaName, db, tb)) assertNotNull(relOptTable) - val tableSourceTable = relOptTable.unwrap(classOf[TableSourceTable[_]]) - tableSourceTable match { - case tst: TableSourceTable[_] => + val tableSourceSinkTable = relOptTable.unwrap(classOf[TableSourceSinkTable[_, _]]) + tableSourceSinkTable.tableSourceTableOpt match { + case Some(tst: TableSourceTable[_]) => assertTrue(tst.tableSource.isInstanceOf[CsvTableSource]) case _ => fail("unexpected table type!") diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TableSinkFactoryServiceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TableSinkFactoryServiceTest.scala new file mode 100644 index 0000000000000..4617f2a19ad15 --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TableSinkFactoryServiceTest.scala @@ -0,0 +1,85 @@ +/* + * 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 org.apache.flink.table.api.{NoMatchingTableFactoryException, ValidationException} +import org.apache.flink.table.descriptors.ConnectorDescriptorValidator._ +import org.apache.flink.table.descriptors.FormatDescriptorValidator._ +import org.apache.flink.table.descriptors.TableDescriptorValidator +import org.junit.Assert._ +import org.junit.Test + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +class TableSinkFactoryServiceTest { + @Test + def testValidProperties(): Unit = { + val props = properties() + assertTrue(TableFactoryService.find(classOf[TableSinkFactory[_]], props.toMap) != null) + } + + @Test(expected = classOf[NoMatchingTableFactoryException]) + def testInvalidContext(): Unit = { + val props = properties() + props.put(CONNECTOR_TYPE, "FAIL") + TableFactoryService.find(classOf[TableSinkFactory[_]], props.toMap) + } + + @Test + def testDifferentContextVersion(): Unit = { + val props = properties() + props.put(CONNECTOR_PROPERTY_VERSION, "2") + // the table source should still be found + assertTrue(TableFactoryService.find(classOf[TableSinkFactory[_]], props.toMap) != null) + } + + @Test(expected = classOf[ValidationException]) + 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) + } + + 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") + properties.put(CONNECTOR_PROPERTY_VERSION, "1") + properties.put(FORMAT_PROPERTY_VERSION, "1") + properties.put("format.path", "/path/to/target") + properties.put("schema.0.name", "a") + properties.put("schema.1.name", "b") + properties.put("schema.2.name", "c") + 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 + } +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TableSourceFactoryServiceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TableSourceFactoryServiceTest.scala similarity index 71% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TableSourceFactoryServiceTest.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TableSourceFactoryServiceTest.scala index e70c4f1932c65..b32d70af1d05d 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TableSourceFactoryServiceTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TableSourceFactoryServiceTest.scala @@ -16,14 +16,17 @@ * limitations under the License. */ -package org.apache.flink.table.sources +package org.apache.flink.table.connectors -import org.apache.flink.table.api.{NoMatchingTableSourceException, TableException, ValidationException} +import org.apache.flink.table.api.{NoMatchingTableFactoryException, ValidationException} 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.junit.Assert.assertTrue import org.junit.Test +import scala.collection.JavaConverters._ import scala.collection.mutable class TableSourceFactoryServiceTest { @@ -33,15 +36,15 @@ class TableSourceFactoryServiceTest { val props = properties() props.put(CONNECTOR_TYPE, "fixed") props.put(FORMAT_TYPE, "test") - assertTrue(TableSourceFactoryService.findAndCreateTableSource(props.toMap) != null) + assertTrue(TableFactoryService.find(classOf[TableSourceFactory[_]], props.toMap) != null) } - @Test(expected = classOf[NoMatchingTableSourceException]) + @Test(expected = classOf[NoMatchingTableFactoryException]) def testInvalidContext(): Unit = { val props = properties() props.put(CONNECTOR_TYPE, "FAIL") props.put(FORMAT_TYPE, "test") - TableSourceFactoryService.findAndCreateTableSource(props.toMap) + TableFactoryService.find(classOf[TableSourceFactory[_]], props.toMap) } @Test @@ -51,7 +54,7 @@ class TableSourceFactoryServiceTest { props.put(FORMAT_TYPE, "test") props.put(CONNECTOR_PROPERTY_VERSION, "2") // the table source should still be found - assertTrue(TableSourceFactoryService.findAndCreateTableSource(props.toMap) != null) + assertTrue(TableFactoryService.find(classOf[TableSourceFactory[_]], props.toMap) != null) } @Test(expected = classOf[ValidationException]) @@ -60,16 +63,18 @@ class TableSourceFactoryServiceTest { props.put(CONNECTOR_TYPE, "fixed") props.put(FORMAT_TYPE, "test") props.put("format.path_new", "/new/path") - TableSourceFactoryService.findAndCreateTableSource(props.toMap) + TableFactoryService.find(classOf[TableSourceFactory[_]], props.toMap) } - @Test(expected = classOf[TableException]) + @Test(expected = classOf[IllegalArgumentException]) def testFailingFactory(): Unit = { val props = properties() props.put(CONNECTOR_TYPE, "fixed") props.put(FORMAT_TYPE, "test") props.put("failing", "true") - TableSourceFactoryService.findAndCreateTableSource(props.toMap) + TableFactoryService.find(classOf[TableSourceFactory[_]], props.toMap) + .asInstanceOf[TableSourceFactory[_]] + .createTableSource(props.asJava) } @Test @@ -79,12 +84,17 @@ class TableSourceFactoryServiceTest { props.put(FORMAT_TYPE, "test") props.put("format.type", "not-test") props.put("format.not-test-property", "wildcard-property") - val actualTableSource = TableSourceFactoryService.findAndCreateTableSource(props.toMap) + val actualTableSource = TableFactoryService.find(classOf[TableSourceFactory[_]], props.toMap) 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") properties.put(CONNECTOR_PROPERTY_VERSION, "1") properties.put(FORMAT_PROPERTY_VERSION, "1") properties.put("format.path", "/path/to/target") diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TestFixedFormatTableFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TestFixedFormatTableFactory.scala similarity index 100% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TestFixedFormatTableFactory.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TestFixedFormatTableFactory.scala diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TestTableSinkFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TestTableSinkFactory.scala new file mode 100644 index 0000000000000..7f583a55f4b30 --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TestTableSinkFactory.scala @@ -0,0 +1,69 @@ +/* + * 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 + +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.descriptors.TableDescriptorValidator +import org.apache.flink.table.sinks.TableSink +import org.apache.flink.types.Row + +class TestTableSinkFactory extends TableSinkFactory[Row] with DiscoverableTableFactory { + + 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_PROPERTY_VERSION, "1") + context.put(FORMAT_PROPERTY_VERSION, "1") + context + } + + override def supportedProperties(): util.List[String] = { + val properties = new util.ArrayList[String]() + // connector + 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() + } + } +} + diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TestTableSourceFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TestTableSourceFactory.scala new file mode 100644 index 0000000000000..345f47e24599b --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TestTableSourceFactory.scala @@ -0,0 +1,64 @@ +/* + * 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 + +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.sources.TableSource +import org.apache.flink.types.Row + +/** + * Table source factory for testing. + */ +class TestTableSourceFactory extends TableSourceFactory[Row] with DiscoverableTableFactory { + + 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_PROPERTY_VERSION, "1") + context.put(FORMAT_PROPERTY_VERSION, "1") + context + } + + override def supportedProperties(): util.List[String] = { + val properties = new util.ArrayList[String]() + // connector + 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() + } + } +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TestWildcardFormatTableSourceFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TestWildcardFormatTableSourceFactory.scala similarity index 100% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TestWildcardFormatTableSourceFactory.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TestWildcardFormatTableSourceFactory.scala diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/FileSystemTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/FileSystemTest.scala index 1162694a01dc7..02d37de26eb2e 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/FileSystemTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/FileSystemTest.scala @@ -32,6 +32,17 @@ class FileSystemTest extends DescriptorTestBase { addPropertyAndVerify(descriptors().get(0), "connector.path", "") } + @Test(expected = classOf[ValidationException]) + def testInvalidWriteMode(): Unit = { + addPropertyAndVerify(descriptors().get(0), "connector.write-mode", "") + } + + + @Test(expected = classOf[ValidationException]) + def testInvalidNumFiles(): Unit = { + addPropertyAndVerify(descriptors().get(0), "connector.num-files", "0") + } + @Test(expected = classOf[ValidationException]) def testMissingPath(): Unit = { removePropertyAndVerify(descriptors().get(0), "connector.path") @@ -40,7 +51,7 @@ class FileSystemTest extends DescriptorTestBase { // ---------------------------------------------------------------------------------------------- override def descriptors(): util.List[Descriptor] = { - util.Arrays.asList(FileSystem().path("/myfile")) + util.Arrays.asList(FileSystem().path("/myfile").writeMode("OVERWRITE").numFiles(1)) } override def validator(): DescriptorValidator = { @@ -51,6 +62,8 @@ class FileSystemTest extends DescriptorTestBase { val desc = Map( "connector.type" -> "filesystem", "connector.property-version" -> "1", + "connector.num-files" -> "1", + "connector.write-mode" -> "OVERWRITE", "connector.path" -> "/myfile") util.Arrays.asList(desc.asJava) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaValidatorTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaValidatorTest.scala index bf7b84b8e769d..f40c19ca4f7d1 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaValidatorTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaValidatorTest.scala @@ -20,7 +20,7 @@ package org.apache.flink.table.descriptors import java.util.Optional -import org.apache.flink.table.api.{TableSchema, Types} +import org.apache.flink.table.api.{TableException, TableSchema, Types} import org.apache.flink.table.sources.tsextractors.{ExistingField, StreamRecordTimestamp} import org.apache.flink.table.sources.wmstrategies.PreserveWatermarks import org.junit.Assert.{assertEquals, assertTrue} @@ -78,6 +78,40 @@ class SchemaValidatorTest { assertEquals(expectedFormatSchema, formatSchema) } + @Test(expected = classOf[TableException]) + def testDeriveTableSinkSchemaWithRowtimeFromSource(): Unit = { + val desc1 = Schema() + .field("otherField", Types.STRING).from("csvField") + .field("abcField", Types.STRING) + .field("p", Types.SQL_TIMESTAMP).proctime() + .field("r", Types.SQL_TIMESTAMP).rowtime( + Rowtime().timestampsFromSource().watermarksFromSource()) + val props = new DescriptorProperties() + desc1.addProperties(props) + + SchemaValidator.deriveTableSinkSchema(props) + } + + @Test + def testDeriveTableSinkSchemaWithRowtimeFromField(): Unit = { + val desc1 = Schema() + .field("otherField", Types.STRING).from("csvField") + .field("abcField", Types.STRING) + .field("p", Types.SQL_TIMESTAMP).proctime() + .field("r", Types.SQL_TIMESTAMP).rowtime( + Rowtime().timestampsFromField("myTime").watermarksFromSource()) + val props = new DescriptorProperties() + desc1.addProperties(props) + + val expectedTableSinkSchema = TableSchema.builder() + .field("csvField", Types.STRING) // aliased + .field("abcField", Types.STRING) + .field("myTime", Types.SQL_TIMESTAMP) + .build() + + assertEquals(expectedTableSinkSchema, SchemaValidator.deriveTableSinkSchema(props)) + } + @Test def testSchemaWithRowtimeFromField(): Unit = { val desc1 = Schema() diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/TableEnvironmentITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/TableEnvironmentITCase.scala index be8278ff6a4a3..978e65c1dff4e 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/TableEnvironmentITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/TableEnvironmentITCase.scala @@ -24,7 +24,7 @@ import org.apache.flink.table.api.TableEnvironment import org.apache.flink.table.api.scala._ import org.apache.flink.table.runtime.utils.TableProgramsCollectionTestBase import org.apache.flink.table.runtime.utils.TableProgramsTestBase.TableConfigMode -import org.apache.flink.table.utils.MemoryTableSinkUtil +import org.apache.flink.table.utils.MemoryTableSourceSinkUtil import org.apache.flink.test.util.TestBaseUtils import org.apache.flink.types.Row import org.junit.Assert.assertEquals @@ -120,14 +120,14 @@ class TableEnvironmentITCase( def testInsertIntoMemoryTable(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env) - MemoryTableSinkUtil.clear + MemoryTableSourceSinkUtil.clear val t = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c) tEnv.registerTable("sourceTable", t) val fieldNames = Array("d", "e", "f") val fieldTypes = tEnv.scan("sourceTable").getSchema.getTypes - val sink = new MemoryTableSinkUtil.UnsafeMemoryAppendTableSink + val sink = new MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink tEnv.registerTableSink("targetTable", fieldNames, fieldTypes, sink) val sql = "INSERT INTO targetTable SELECT a, b, c FROM sourceTable" @@ -135,7 +135,7 @@ class TableEnvironmentITCase( env.execute() val expected = List("1,1,Hi", "2,2,Hello", "3,2,Hello world") - assertEquals(expected.sorted, MemoryTableSinkUtil.results.sorted) + assertEquals(expected.sorted, MemoryTableSourceSinkUtil.tableData.map(_.toString).sorted) } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/TableEnvironmentITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/TableEnvironmentITCase.scala index 5295e7c02eea6..a833e004835c2 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/TableEnvironmentITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/TableEnvironmentITCase.scala @@ -26,7 +26,7 @@ import org.apache.flink.table.api.TableEnvironment import org.apache.flink.table.api.scala._ import org.apache.flink.table.runtime.utils.TableProgramsTestBase.TableConfigMode import org.apache.flink.table.runtime.utils.{TableProgramsCollectionTestBase, TableProgramsTestBase} -import org.apache.flink.table.utils.MemoryTableSinkUtil +import org.apache.flink.table.utils.MemoryTableSourceSinkUtil import org.apache.flink.test.util.TestBaseUtils import org.apache.flink.types.Row import org.junit.Assert.assertEquals @@ -186,14 +186,14 @@ class TableEnvironmentITCase( def testInsertIntoMemoryTable(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env) - MemoryTableSinkUtil.clear + MemoryTableSourceSinkUtil.clear val t = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c) tEnv.registerTable("sourceTable", t) val fieldNames = Array("d", "e", "f") val fieldTypes = tEnv.scan("sourceTable").getSchema.getTypes - val sink = new MemoryTableSinkUtil.UnsafeMemoryAppendTableSink + val sink = new MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink tEnv.registerTableSink("targetTable", fieldNames, fieldTypes, sink) tEnv.scan("sourceTable") @@ -202,7 +202,7 @@ class TableEnvironmentITCase( env.execute() val expected = List("1,1,Hi", "2,2,Hello", "3,2,Hello world") - assertEquals(expected.sorted, MemoryTableSinkUtil.results.sorted) + assertEquals(expected.sorted, MemoryTableSourceSinkUtil.tableData.map(_.toString).sorted) } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala index c553ee6828d93..2554d143ac9aa 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala @@ -35,7 +35,7 @@ import org.apache.flink.table.plan.TimeIndicatorConversionTest.TableFunc import org.apache.flink.table.runtime.stream.TimeAttributesITCase.{AtomicTimestampWithEqualWatermark, TestPojo, TimestampWithEqualWatermark, TimestampWithEqualWatermarkPojo} import org.apache.flink.table.runtime.utils.JavaPojos.Pojo1 import org.apache.flink.table.runtime.utils.StreamITCase -import org.apache.flink.table.utils.{MemoryTableSinkUtil, TestTableSourceWithTime} +import org.apache.flink.table.utils.{MemoryTableSourceSinkUtil, TestTableSourceWithTime} import org.apache.flink.test.util.AbstractTestBase import org.apache.flink.types.Row import org.junit.Assert._ @@ -183,7 +183,7 @@ class TimeAttributesITCase extends AbstractTestBase { val env = StreamExecutionEnvironment.getExecutionEnvironment env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) val tEnv = TableEnvironment.getTableEnvironment(env) - MemoryTableSinkUtil.clear + MemoryTableSourceSinkUtil.clear val stream = env .fromCollection(data) @@ -191,7 +191,7 @@ class TimeAttributesITCase extends AbstractTestBase { stream.toTable(tEnv, 'rowtime.rowtime, 'int, 'double, 'float, 'bigdec, 'string) .filter('rowtime.cast(Types.LONG) > 4) .select('rowtime, 'rowtime.floor(TimeIntervalUnit.DAY), 'rowtime.ceil(TimeIntervalUnit.DAY)) - .writeToSink(new MemoryTableSinkUtil.UnsafeMemoryAppendTableSink) + .writeToSink(new MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink) env.execute() @@ -199,7 +199,7 @@ class TimeAttributesITCase extends AbstractTestBase { "1970-01-01 00:00:00.007,1970-01-01 00:00:00.0,1970-01-02 00:00:00.0", "1970-01-01 00:00:00.008,1970-01-01 00:00:00.0,1970-01-02 00:00:00.0", "1970-01-01 00:00:00.016,1970-01-01 00:00:00.0,1970-01-02 00:00:00.0") - assertEquals(expected.sorted, MemoryTableSinkUtil.results.sorted) + assertEquals(expected.sorted, MemoryTableSourceSinkUtil.tableData.map(_.toString).sorted) } @Test diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala index e13234981dc99..3ab3cb8ba38c9 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala @@ -27,13 +27,14 @@ import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.streaming.api.watermark.Watermark import org.apache.flink.table.api.{TableEnvironment, Types} import org.apache.flink.table.api.scala._ +import org.apache.flink.table.descriptors.{DescriptorProperties, Rowtime, Schema} import org.apache.flink.table.expressions.utils.SplitUDF import org.apache.flink.table.expressions.utils.Func15 import org.apache.flink.table.runtime.stream.sql.SqlITCase.TimestampAndWatermarkWithOffset import org.apache.flink.table.runtime.utils.TimeTestUtil.EventTimeSourceFunction import org.apache.flink.table.runtime.utils.{JavaUserDefinedTableFunctions, StreamITCase, StreamTestData, StreamingWithStateTestBase} import org.apache.flink.types.Row -import org.apache.flink.table.utils.MemoryTableSinkUtil +import org.apache.flink.table.utils.{InMemoryTableFactory, MemoryTableSourceSinkUtil} import org.junit.Assert._ import org.junit._ @@ -691,7 +692,7 @@ class SqlITCase extends StreamingWithStateTestBase { val env = StreamExecutionEnvironment.getExecutionEnvironment env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) val tEnv = TableEnvironment.getTableEnvironment(env) - MemoryTableSinkUtil.clear + MemoryTableSourceSinkUtil.clear val t = StreamTestData.getSmall3TupleDataStream(env) .assignAscendingTimestamps(x => x._2) @@ -701,7 +702,7 @@ class SqlITCase extends StreamingWithStateTestBase { val fieldNames = Array("d", "e", "f", "t") val fieldTypes = Array(Types.INT, Types.LONG, Types.STRING, Types.SQL_TIMESTAMP) .asInstanceOf[Array[TypeInformation[_]]] - val sink = new MemoryTableSinkUtil.UnsafeMemoryAppendTableSink + val sink = new MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink tEnv.registerTableSink("targetTable", fieldNames, fieldTypes, sink) val sql = "INSERT INTO targetTable SELECT a, b, c, rowtime FROM sourceTable" @@ -712,7 +713,46 @@ class SqlITCase extends StreamingWithStateTestBase { "1,1,Hi,1970-01-01 00:00:00.001", "2,2,Hello,1970-01-01 00:00:00.002", "3,2,Hello world,1970-01-01 00:00:00.002") - assertEquals(expected.sorted, MemoryTableSinkUtil.results.sorted) + assertEquals(expected.sorted, MemoryTableSourceSinkUtil.tableData.map(_.toString).sorted) + } + + @Test + def testWriteReadTableSourceSink(): Unit = { + var env = StreamExecutionEnvironment.getExecutionEnvironment + env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) + var tEnv = TableEnvironment.getTableEnvironment(env) + MemoryTableSourceSinkUtil.clear + + val desc = Schema() + .field("a", Types.INT) + .field("e", Types.LONG) + .field("f", Types.STRING) + .field("t", Types.SQL_TIMESTAMP) + .rowtime(Rowtime().timestampsFromField("t").watermarksPeriodicAscending()) + .field("proctime", Types.SQL_TIMESTAMP).proctime() + val props = new DescriptorProperties() + desc.addProperties(props) + + val t = StreamTestData.getSmall3TupleDataStream(env) + .assignAscendingTimestamps(x => x._2) + .toTable(tEnv, 'a, 'b, 'c, 'rowtime.rowtime) + tEnv.registerTable("sourceTable", t) + + tEnv.registerTableSource("targetTable", + new InMemoryTableFactory().createTableSource(props.asMap)) + tEnv.registerTableSink("targetTable", + new InMemoryTableFactory().createTableSink(props.asMap)) + + tEnv.sqlUpdate("INSERT INTO targetTable SELECT a, b, c, rowtime FROM sourceTable") + tEnv.sqlQuery("SELECT a, e, f, t from targetTable") + .addSink(new StreamITCase.StringSink[Row]) + env.execute() + + val expected = List( + "1,1,Hi,1970-01-01 00:00:00.001", + "2,2,Hello,1970-01-01 00:00:00.002", + "3,2,Hello world,1970-01-01 00:00:00.002") + assertEquals(expected.sorted, StreamITCase.testResults.sorted) } @Test diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala index 467d9d3d2e367..70e59f3d24d8f 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/TableSinkITCase.scala @@ -35,7 +35,7 @@ import org.apache.flink.table.api.scala._ import org.apache.flink.table.api.{TableEnvironment, TableException, Types} import org.apache.flink.table.runtime.utils.{StreamITCase, StreamTestData} import org.apache.flink.table.sinks._ -import org.apache.flink.table.utils.MemoryTableSinkUtil +import org.apache.flink.table.utils.MemoryTableSourceSinkUtil import org.apache.flink.test.util.{AbstractTestBase, TestBaseUtils} import org.apache.flink.types.Row import org.apache.flink.util.Collector @@ -54,13 +54,13 @@ class TableSinkITCase extends AbstractTestBase { env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) val tEnv = TableEnvironment.getTableEnvironment(env) - MemoryTableSinkUtil.clear + MemoryTableSourceSinkUtil.clear val input = StreamTestData.get3TupleDataStream(env) .assignAscendingTimestamps(r => r._2) val fieldNames = Array("d", "e", "t") val fieldTypes: Array[TypeInformation[_]] = Array(Types.STRING, Types.SQL_TIMESTAMP, Types.LONG) - val sink = new MemoryTableSinkUtil.UnsafeMemoryAppendTableSink + val sink = new MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink tEnv.registerTableSink("targetTable", fieldNames, fieldTypes, sink) input.toTable(tEnv, 'a, 'b, 'c, 't.rowtime) @@ -75,7 +75,7 @@ class TableSinkITCase extends AbstractTestBase { "Comment#14,1970-01-01 00:00:00.006,6", "Comment#15,1970-01-01 00:00:00.006,6").mkString("\n") - TestBaseUtils.compareResultAsText(MemoryTableSinkUtil.results.asJava, expected) + TestBaseUtils.compareResultAsText(MemoryTableSourceSinkUtil.tableData.asJava, expected) } @Test diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala new file mode 100644 index 0000000000000..a778a3f44cdd8 --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala @@ -0,0 +1,113 @@ +/* + * 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.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.sources.TableSource +import org.apache.flink.types.Row +import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_PROPERTY_VERSION +import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE +import org.apache.flink.table.descriptors.{DescriptorProperties, SchemaValidator} +import org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_CLASS +import org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_FROM +import org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_SERIALIZED +import org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_TYPE +import org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_WATERMARKS_CLASS +import org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_WATERMARKS_DELAY +import org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_WATERMARKS_SERIALIZED +import org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_WATERMARKS_TYPE +import org.apache.flink.table.descriptors.SchemaValidator.SCHEMA +import org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_FROM +import org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_NAME +import org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_PROCTIME +import org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_TYPE +import org.apache.flink.table.sinks.TableSink + +class InMemoryTableFactory extends TableSourceFactory[Row] + with TableSinkFactory[Row] with DiscoverableTableFactory { + override def createTableSink(properties: util.Map[String, String]): TableSink[Row] = { + val params: DescriptorProperties = new DescriptorProperties(true) + params.putProperties(properties) + + // validate + new SchemaValidator(true).validate(params) + + val tableSchema = SchemaValidator.deriveTableSinkSchema(params); + + new MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink() + .configure(tableSchema.getColumnNames, tableSchema.getTypes) + } + + override def createTableSource(properties: util.Map[String, String]): TableSource[Row] = { + val params: DescriptorProperties = new DescriptorProperties(true) + params.putProperties(properties) + + // validate + new SchemaValidator(true).validate(params) + + val tableSchema = SchemaValidator.deriveTableSourceSchema(params); + + // proctime + val proctimeAttributeOpt = SchemaValidator.deriveProctimeAttribute(params) + + val (names, types) = tableSchema.getColumnNames.zip(tableSchema.getTypes) + .filter(_._1 != proctimeAttributeOpt.get()).unzip + // rowtime + val rowtimeDescriptors = SchemaValidator.deriveRowtimeAttributes(params) + new MemoryTableSourceSinkUtil.UnsafeMemoryTableSource( + tableSchema, + new RowTypeInfo(types, names), + rowtimeDescriptors, + proctimeAttributeOpt.get(), + 3) + } + + override def requiredContext(): util.Map[String, String] = { + val context: util.Map[String, String] = new util.HashMap[String, String] + context.put(CONNECTOR_TYPE, "memory") + context.put(CONNECTOR_PROPERTY_VERSION, "1") // backwards compatibility + + context + } + + override def supportedProperties(): util.List[String] = { + val properties = new util.ArrayList[String]() + + // schema + properties.add(SCHEMA + ".#." + SCHEMA_TYPE) + properties.add(SCHEMA + ".#." + SCHEMA_NAME) + properties.add(SCHEMA + ".#." + SCHEMA_FROM) + + // time attributes + properties.add(SCHEMA + ".#." + SCHEMA_PROCTIME) + properties.add(SCHEMA + ".#." + ROWTIME_TIMESTAMPS_TYPE) + properties.add(SCHEMA + ".#." + ROWTIME_TIMESTAMPS_FROM) + properties.add(SCHEMA + ".#." + ROWTIME_TIMESTAMPS_CLASS) + properties.add(SCHEMA + ".#." + ROWTIME_TIMESTAMPS_SERIALIZED) + properties.add(SCHEMA + ".#." + ROWTIME_WATERMARKS_TYPE) + properties.add(SCHEMA + ".#." + ROWTIME_WATERMARKS_CLASS) + properties.add(SCHEMA + ".#." + ROWTIME_WATERMARKS_SERIALIZED) + properties.add(SCHEMA + ".#." + ROWTIME_WATERMARKS_DELAY) + + properties + } +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MemoryTableSinkUtil.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala similarity index 53% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MemoryTableSinkUtil.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala index 6d6307b6f6581..511fedf868de7 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MemoryTableSinkUtil.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala @@ -18,24 +18,80 @@ package org.apache.flink.table.utils +import java.sql.Timestamp +import java.util +import java.util.Collections + import org.apache.flink.api.common.io.RichOutputFormat import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.java.DataSet +import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.datastream.DataStream +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.streaming.api.functions.sink.RichSinkFunction +import org.apache.flink.streaming.api.functions.source.SourceFunction +import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext +import org.apache.flink.table.api.TableSchema import org.apache.flink.table.sinks.{AppendStreamTableSink, BatchTableSink, TableSinkBase} +import org.apache.flink.table.sources.tsextractors.StreamRecordTimestamp +import org.apache.flink.table.sources.wmstrategies.AscendingTimestamps +import org.apache.flink.table.sources._ import org.apache.flink.table.util.TableConnectorUtil import org.apache.flink.types.Row import scala.collection.mutable +import scala.collection.JavaConverters._ -object MemoryTableSinkUtil { - var results: mutable.MutableList[String] = mutable.MutableList.empty[String] +object MemoryTableSourceSinkUtil { + var tableData: mutable.ListBuffer[Row] = mutable.ListBuffer[Row]() def clear = { - MemoryTableSinkUtil.results.clear() + MemoryTableSourceSinkUtil.tableData.clear() + } + + class UnsafeMemoryTableSource( + tableSchema: TableSchema, + returnType: TypeInformation[Row], + rowtimeAttributeDescriptor: util.List[RowtimeAttributeDescriptor], + proctime: String, + val rowCount: Integer) + extends BatchTableSource[Row] + with StreamTableSource[Row] with DefinedProctimeAttribute with DefinedRowtimeAttributes { + + override def getReturnType: TypeInformation[Row] = returnType + + override def getTableSchema: TableSchema = tableSchema + + override def getDataSet(execEnv: ExecutionEnvironment): DataSet[Row] = { + execEnv.fromCollection(tableData.asJava, returnType) + } + + final class InMemorySourceFunction(var count: Int = rowCount) extends SourceFunction[Row] { + override def cancel(): Unit = ??? + + override def run(ctx: SourceContext[Row]): Unit = { + while (count > 0) { + tableData.synchronized { + if (tableData.size > 0) { + val r = tableData.remove(0) + ctx.collect(r) + count -= 1 + } + } + } + } + } + + override def getDataStream(execEnv: StreamExecutionEnvironment): DataStream[Row] = { + execEnv.addSource(new InMemorySourceFunction, returnType) + } + + override def getProctimeAttribute: String = proctime + + override def getRowtimeAttributeDescriptors: util.List[RowtimeAttributeDescriptor] = { + rowtimeAttributeDescriptor + } } final class UnsafeMemoryAppendTableSink @@ -66,8 +122,8 @@ object MemoryTableSinkUtil { private class MemoryAppendSink extends RichSinkFunction[Row]() { override def invoke(value: Row): Unit = { - results.synchronized { - results += value.toString + tableData.synchronized { + tableData += Row.copy(value) } } } @@ -79,8 +135,8 @@ object MemoryTableSinkUtil { override def open(taskNumber: Int, numTasks: Int): Unit = {} override def writeRecord(record: Row): Unit = { - results.synchronized { - results += record.toString + tableData.synchronized { + tableData += Row.copy(record) } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala index 308ead2ed57a8..c25f30f37fc8d 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MockTableEnvironment.scala @@ -46,6 +46,8 @@ class MockTableEnvironment extends TableEnvironment(new TableConfig) { fieldTypes: Array[TypeInformation[_]], tableSink: TableSink[_]): Unit = ??? + override def registerTableSink(name: String, tableSink: TableSink[_]): Unit = ??? + override protected def createUniqueTableName(): String = ??? override protected def registerTableSourceInternal(name: String, tableSource: TableSource[_]) From 311dc62e59c0e4146c094b73c21b979f31b2e1d9 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 11 Jul 2018 13:29:03 +0200 Subject: [PATCH 03/17] Rename to TableFactory and move it to factories package --- ....apache.flink.table.factories.TableFactory} | 0 ....apache.flink.table.factories.TableFactory} | 0 ....apache.flink.table.factories.TableFactory} | 0 ....apache.flink.table.factories.TableFactory} | 0 .../kafka/KafkaTableSourceFactory.java | 6 +++--- .../KafkaJsonTableSourceFactoryTestBase.java | 4 ++-- flink-libraries/flink-sql-client/pom.xml | 2 +- .../client/gateway/local/ExecutionContext.java | 6 +++--- .../assembly/test-table-source-factory.xml | 2 +- .../gateway/utils/TestTableSinkFactory.java | 6 +++--- .../gateway/utils/TestTableSourceFactory.java | 6 +++--- ....apache.flink.table.factories.TableFactory} | 0 .../apache/flink/table/api/exceptions.scala | 4 ++-- .../catalog/ExternalTableSourceUtil.scala | 2 +- .../BatchTableSourceDescriptor.scala | 2 +- .../StreamTableSourceDescriptor.scala | 2 +- .../TableFactory.scala} | 4 ++-- .../TableFactoryService.scala | 18 +++++++++--------- .../TableSinkFactory.scala | 2 +- .../TableSourceFactory.scala | 2 +- .../table/sinks/CsvTableSinkFactory.scala | 4 ++-- .../table/sources/CsvTableSourceFactory.scala | 4 ++-- ....apache.flink.table.factories.TableFactory} | 6 +++--- .../TableSinkFactoryServiceTest.scala | 2 +- .../TableSourceFactoryServiceTest.scala | 2 +- .../TestFixedFormatTableFactory.scala | 0 .../TestTableSinkFactory.scala | 4 ++-- .../TestTableSourceFactory.scala | 4 ++-- .../TestWildcardFormatTableSourceFactory.scala | 0 .../table/utils/InMemoryTableFactory.scala | 4 ++-- 30 files changed, 49 insertions(+), 49 deletions(-) rename flink-connectors/flink-connector-kafka-0.10/src/main/resources/META-INF/services/{org.apache.flink.table.connectors.DiscoverableTableFactory => org.apache.flink.table.factories.TableFactory} (100%) rename flink-connectors/flink-connector-kafka-0.11/src/main/resources/META-INF/services/{org.apache.flink.table.connectors.DiscoverableTableFactory => org.apache.flink.table.factories.TableFactory} (100%) rename flink-connectors/flink-connector-kafka-0.8/src/main/resources/META-INF/services/{org.apache.flink.table.connectors.DiscoverableTableFactory => org.apache.flink.table.factories.TableFactory} (100%) rename flink-connectors/flink-connector-kafka-0.9/src/main/resources/META-INF/services/{org.apache.flink.table.connectors.DiscoverableTableFactory => org.apache.flink.table.factories.TableFactory} (100%) rename flink-libraries/flink-table/src/main/resources/META-INF/services/{org.apache.flink.table.connectors.DiscoverableTableFactory => org.apache.flink.table.factories.TableFactory} (100%) rename flink-libraries/flink-table/src/main/scala/org/apache/flink/table/{connectors/DiscoverableTableFactory.scala => factories/TableFactory.scala} (97%) rename flink-libraries/flink-table/src/main/scala/org/apache/flink/table/{connectors => factories}/TableFactoryService.scala (88%) rename flink-libraries/flink-table/src/main/scala/org/apache/flink/table/{connectors => factories}/TableSinkFactory.scala (96%) rename flink-libraries/flink-table/src/main/scala/org/apache/flink/table/{connectors => factories}/TableSourceFactory.scala (96%) rename flink-libraries/flink-table/src/test/resources/META-INF/services/{org.apache.flink.table.connectors.DiscoverableTableFactory => org.apache.flink.table.factories.TableFactory} (89%) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/{connectors => factories}/TableSinkFactoryServiceTest.scala (98%) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/{connectors => factories}/TableSourceFactoryServiceTest.scala (99%) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/{connectors => factories}/TestFixedFormatTableFactory.scala (100%) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/{connectors => factories}/TestTableSinkFactory.scala (95%) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/{connectors => factories}/TestTableSourceFactory.scala (97%) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/{connectors => factories}/TestWildcardFormatTableSourceFactory.scala (100%) diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory b/flink-connectors/flink-connector-kafka-0.10/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory similarity index 100% rename from flink-connectors/flink-connector-kafka-0.10/src/main/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory rename to flink-connectors/flink-connector-kafka-0.10/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory b/flink-connectors/flink-connector-kafka-0.11/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory similarity index 100% rename from flink-connectors/flink-connector-kafka-0.11/src/main/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory rename to flink-connectors/flink-connector-kafka-0.11/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory b/flink-connectors/flink-connector-kafka-0.8/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory similarity index 100% rename from flink-connectors/flink-connector-kafka-0.8/src/main/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory rename to flink-connectors/flink-connector-kafka-0.8/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory b/flink-connectors/flink-connector-kafka-0.9/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory similarity index 100% rename from flink-connectors/flink-connector-kafka-0.9/src/main/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory rename to flink-connectors/flink-connector-kafka-0.9/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java index dc4374e48d490..7a05302a44314 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java @@ -23,8 +23,8 @@ 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.factories.TableFactory; +import org.apache.flink.table.factories.TableSourceFactory; import org.apache.flink.table.descriptors.DescriptorProperties; import org.apache.flink.table.descriptors.KafkaValidator; import org.apache.flink.table.descriptors.SchemaValidator; @@ -72,7 +72,7 @@ /** * Factory for creating configured instances of {@link KafkaTableSource}. */ -public abstract class KafkaTableSourceFactory implements TableSourceFactory, DiscoverableTableFactory { +public abstract class KafkaTableSourceFactory implements TableSourceFactory, TableFactory { @Override public Map requiredContext() { diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java index c8b422fa0e312..fbe983d1146bd 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java @@ -22,8 +22,8 @@ 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; diff --git a/flink-libraries/flink-sql-client/pom.xml b/flink-libraries/flink-sql-client/pom.xml index 9d34afa2c68a7..924a95854e4ae 100644 --- a/flink-libraries/flink-sql-client/pom.xml +++ b/flink-libraries/flink-sql-client/pom.xml @@ -159,7 +159,7 @@ under the License. org.codehaus.commons.compiler.properties org/codehaus/janino/** org/codehaus/commons/** - META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory + META-INF/services/org.apache.flink.table.factories.TableFactory org/jline/** com/fasterxml/jackson/** diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java index 114d5a6d4df7f..c6b54ce683ce2 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java @@ -51,9 +51,9 @@ 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; diff --git a/flink-libraries/flink-sql-client/src/test/assembly/test-table-source-factory.xml b/flink-libraries/flink-sql-client/src/test/assembly/test-table-source-factory.xml index 137d03293a485..fafffae4a2d2b 100644 --- a/flink-libraries/flink-sql-client/src/test/assembly/test-table-source-factory.xml +++ b/flink-libraries/flink-sql-client/src/test/assembly/test-table-source-factory.xml @@ -42,7 +42,7 @@ under the License. src/test/resources/test-factory-services-file META-INF/services - org.apache.flink.table.connectors.DiscoverableTableFactory + org.apache.flink.table.factories.TableFactory 0755 diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactory.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactory.java index 69c510852ce7c..34060756f8ca1 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactory.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactory.java @@ -23,8 +23,8 @@ 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.TableSinkFactory; +import org.apache.flink.table.factories.TableFactory; +import org.apache.flink.table.factories.TableSinkFactory; import org.apache.flink.table.descriptors.DescriptorProperties; import org.apache.flink.table.descriptors.SchemaValidator; import org.apache.flink.table.sinks.AppendStreamTableSink; @@ -47,7 +47,7 @@ /** * Table sink factory for testing the classloading in {@link DependencyTest}. */ -public class TestTableSinkFactory implements TableSinkFactory, DiscoverableTableFactory { +public class TestTableSinkFactory implements TableSinkFactory, TableFactory { @Override public Map requiredContext() { diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java index 34af3f7bc4d2b..cc8c4d40c4fd4 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java @@ -24,8 +24,8 @@ 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; @@ -52,7 +52,7 @@ /** * Table source factory for testing the classloading in {@link DependencyTest}. */ -public class TestTableSourceFactory implements TableSourceFactory, DiscoverableTableFactory { +public class TestTableSourceFactory implements TableSourceFactory, TableFactory { @Override public Map requiredContext() { diff --git a/flink-libraries/flink-table/src/main/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory b/flink-libraries/flink-table/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory similarity index 100% rename from flink-libraries/flink-table/src/main/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory rename to flink-libraries/flink-table/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala index e311727c433db..c0cf22cefb0f0 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala @@ -222,7 +222,7 @@ case class AmbiguousTableFormatException( } /** - * Exception for not finding a [[org.apache.flink.table.connectors.DiscoverableTableFactory]] for + * Exception for not finding a [[org.apache.flink.table.factories.TableFactory]] for * the given properties. * * @param properties properties that describe the table connector @@ -240,7 +240,7 @@ case class NoMatchingTableFactoryException(properties: Map[String, String], caus /** * Exception for finding more than one - * [[org.apache.flink.table.connectors.DiscoverableTableFactory]] for the given properties. + * [[org.apache.flink.table.factories.TableFactory]] for the given properties. * * @param properties properties that describe the table factory * @param cause the cause diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableSourceUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableSourceUtil.scala index 7e9ac21f0d519..b1fc58a66f5a7 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableSourceUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableSourceUtil.scala @@ -19,7 +19,7 @@ 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.factories.{TableFactoryService, TableSourceFactory} import org.apache.flink.table.descriptors.DescriptorProperties import org.apache.flink.table.plan.schema.{BatchTableSourceTable, StreamTableSourceTable, TableSourceSinkTable, TableSourceTable} import org.apache.flink.table.plan.stats.FlinkStatistic diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableSourceDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableSourceDescriptor.scala index 155153fd9ffd7..3fbb00c3bedc6 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableSourceDescriptor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableSourceDescriptor.scala @@ -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) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptor.scala index 3e7c900befcff..6c90f07eb710d 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptor.scala @@ -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} /** diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/DiscoverableTableFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala similarity index 97% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/DiscoverableTableFactory.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala index db5a8860e25e7..bd391e3a2e4f4 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/DiscoverableTableFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala @@ -16,14 +16,14 @@ * 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. */ -trait DiscoverableTableFactory { +trait TableFactory { /** * Specifies the context that this factory has been implemented for. diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/TableFactoryService.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryService.scala similarity index 88% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/TableFactoryService.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryService.scala index 5ad6e70ed772f..4c49ffc6c0490 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/TableFactoryService.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryService.scala @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.connectors +package org.apache.flink.table.factories import java.util.{ServiceConfigurationError, ServiceLoader} @@ -36,33 +36,33 @@ import _root_.scala.collection.mutable */ object TableFactoryService extends Logging { - private lazy val defaultLoader = ServiceLoader.load(classOf[DiscoverableTableFactory]) + private lazy val defaultLoader = ServiceLoader.load(classOf[TableFactory]) - def find(clz: Class[_], descriptor: TableDescriptor): DiscoverableTableFactory = { + def find(clz: Class[_], descriptor: TableDescriptor): TableFactory = { find(clz, descriptor, null) } def find(clz: Class[_], descriptor: TableDescriptor, classLoader: ClassLoader) - : DiscoverableTableFactory = { + : TableFactory = { val properties = new DescriptorProperties() descriptor.addProperties(properties) find(clz, properties.asMap.asScala.toMap, classLoader) } - def find(clz: Class[_], properties: Map[String, String]): DiscoverableTableFactory = { + def find(clz: Class[_], properties: Map[String, String]): TableFactory = { find(clz: Class[_], properties, null) } def find(clz: Class[_], properties: Map[String, String], - classLoader: ClassLoader): DiscoverableTableFactory = { + classLoader: ClassLoader): TableFactory = { - var matchingFactory: Option[(DiscoverableTableFactory, Seq[String])] = None + var matchingFactory: Option[(TableFactory, Seq[String])] = None try { val iter = if (classLoader == null) { defaultLoader.iterator() } else { - val customLoader = ServiceLoader.load(classOf[DiscoverableTableFactory], classLoader) + val customLoader = ServiceLoader.load(classOf[TableFactory], classLoader) customLoader.iterator() } while (iter.hasNext) { @@ -98,7 +98,7 @@ object TableFactoryService extends Logging { matchingFactory match { case Some(_) => throw new AmbiguousTableFactoryException(properties) case None => matchingFactory = - Some((factory.asInstanceOf[DiscoverableTableFactory], requiredContext.keys.toSeq)) + Some((factory.asInstanceOf[TableFactory], requiredContext.keys.toSeq)) } } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/TableSinkFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableSinkFactory.scala similarity index 96% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/TableSinkFactory.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableSinkFactory.scala index 6346e389dfd44..fc7b3652b4d7e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/TableSinkFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableSinkFactory.scala @@ -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 diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/TableSourceFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableSourceFactory.scala similarity index 96% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/TableSourceFactory.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableSourceFactory.scala index bd3130a3f49bb..cbfacc1002b3c 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/connectors/TableSourceFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableSourceFactory.scala @@ -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.sources.TableSource diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala index c333778e0b98b..9ea6b0e4e142c 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala @@ -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]() diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala index c8e450380185a..c5136694460bf 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala @@ -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]() diff --git a/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory b/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory similarity index 89% rename from flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory rename to flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory index 45f1cf6a73c95..93b290952932c 100644 --- a/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory +++ b/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -17,6 +17,6 @@ org.apache.flink.table.sources.TestFixedFormatTableFactory org.apache.flink.table.sources.TestWildcardFormatTableSourceFactory ======= -org.apache.flink.table.connectors.TestTableSinkFactory -org.apache.flink.table.connectors.TestTableSourceFactory ->>>>>>> 2fce6f6765... [FLINK-8866] [table] Create unified interfaces to configure and instatiate TableSinks:flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.connectors.DiscoverableTableFactory +org.apache.flink.table.factories.TestTableSinkFactory +org.apache.flink.table.factories.TestTableSourceFactory +>>>>>>> 2fce6f6765... [FLINK-8866] [table] Create unified interfaces to configure and instatiate TableSinks:flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TableSinkFactoryServiceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSinkFactoryServiceTest.scala similarity index 98% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TableSinkFactoryServiceTest.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSinkFactoryServiceTest.scala index 4617f2a19ad15..7e6cf490c9c01 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TableSinkFactoryServiceTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSinkFactoryServiceTest.scala @@ -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.api.{NoMatchingTableFactoryException, ValidationException} import org.apache.flink.table.descriptors.ConnectorDescriptorValidator._ diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TableSourceFactoryServiceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSourceFactoryServiceTest.scala similarity index 99% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TableSourceFactoryServiceTest.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSourceFactoryServiceTest.scala index b32d70af1d05d..2664c59b987ea 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TableSourceFactoryServiceTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSourceFactoryServiceTest.scala @@ -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.api.{NoMatchingTableFactoryException, ValidationException} import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.{CONNECTOR_PROPERTY_VERSION, CONNECTOR_TYPE} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TestFixedFormatTableFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TestFixedFormatTableFactory.scala similarity index 100% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TestFixedFormatTableFactory.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TestFixedFormatTableFactory.scala diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TestTableSinkFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TestTableSinkFactory.scala similarity index 95% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TestTableSinkFactory.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TestTableSinkFactory.scala index 7f583a55f4b30..e3fd1ffb60540 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TestTableSinkFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TestTableSinkFactory.scala @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.connectors +package org.apache.flink.table.factories import java.util @@ -27,7 +27,7 @@ import org.apache.flink.table.descriptors.TableDescriptorValidator import org.apache.flink.table.sinks.TableSink import org.apache.flink.types.Row -class TestTableSinkFactory extends TableSinkFactory[Row] with DiscoverableTableFactory { +class TestTableSinkFactory extends TableSinkFactory[Row] with TableFactory { override def requiredContext(): util.Map[String, String] = { val context = new util.HashMap[String, String]() diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TestTableSourceFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TestTableSourceFactory.scala similarity index 97% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TestTableSourceFactory.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TestTableSourceFactory.scala index 345f47e24599b..5b511ccd0fb9b 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TestTableSourceFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TestTableSourceFactory.scala @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.connectors +package org.apache.flink.table.factories import java.util @@ -30,7 +30,7 @@ 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]() diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TestWildcardFormatTableSourceFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TestWildcardFormatTableSourceFactory.scala similarity index 100% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/connectors/TestWildcardFormatTableSourceFactory.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TestWildcardFormatTableSourceFactory.scala diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala index a778a3f44cdd8..bac31f673bbc2 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala @@ -21,7 +21,7 @@ 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.factories.{TableFactory, TableSinkFactory, TableSourceFactory} import org.apache.flink.table.sources.TableSource import org.apache.flink.types.Row import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_PROPERTY_VERSION @@ -43,7 +43,7 @@ import org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_TYPE import org.apache.flink.table.sinks.TableSink class InMemoryTableFactory 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) From 1c581cba61ba321bb6de6a4d298a881840d11cfe Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 11 Jul 2018 13:46:31 +0200 Subject: [PATCH 04/17] Refactor format factories --- .../kafka/KafkaTableSourceFactory.java | 2 +- .../formats/avro/AvroRowFormatFactory.java | 4 +-- ....flink.table.factories.TableFormatFactory} | 0 .../avro/AvroRowFormatFactoryTest.java | 4 +-- .../formats/json/JsonRowFormatFactory.java | 4 +-- ....flink.table.factories.TableFormatFactory} | 0 .../json/JsonRowFormatFactoryTest.java | 4 +-- .../apache/flink/table/api/exceptions.scala | 6 ++-- .../DeserializationSchemaFactory.scala | 2 +- .../SerializationSchemaFactory.scala | 2 +- .../flink/table/factories/TableFactory.scala | 9 ++++- .../TableFormatFactory.scala | 33 +++---------------- .../formats/TableFormatFactoryService.scala | 1 + ....flink.table.factories.TableFormatFactory} | 0 .../TableFormatFactoryServiceTest.scala | 1 + .../TestAmbiguousTableFormatFactory.scala | 3 +- .../utils/TestTableFormatFactory.scala | 3 +- 17 files changed, 32 insertions(+), 46 deletions(-) rename flink-formats/flink-avro/src/main/resources/META-INF/services/{org.apache.flink.table.formats.TableFormatFactory => org.apache.flink.table.factories.TableFormatFactory} (100%) rename flink-formats/flink-json/src/main/resources/META-INF/services/{org.apache.flink.table.formats.TableFormatFactory => org.apache.flink.table.factories.TableFormatFactory} (100%) rename flink-libraries/flink-table/src/main/scala/org/apache/flink/table/{formats => factories}/DeserializationSchemaFactory.scala (97%) rename flink-libraries/flink-table/src/main/scala/org/apache/flink/table/{formats => factories}/SerializationSchemaFactory.scala (97%) rename flink-libraries/flink-table/src/main/scala/org/apache/flink/table/{formats => factories}/TableFormatFactory.scala (60%) rename flink-libraries/flink-table/src/test/resources/META-INF/services/{org.apache.flink.table.formats.TableFormatFactory => org.apache.flink.table.factories.TableFormatFactory} (100%) diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java index 7a05302a44314..621b92b4acd2c 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java @@ -28,7 +28,7 @@ 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.factories.DeserializationSchemaFactory; import org.apache.flink.table.formats.TableFormatFactoryService; import org.apache.flink.table.sources.RowtimeAttributeDescriptor; import org.apache.flink.table.sources.TableSource; diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowFormatFactory.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowFormatFactory.java index c94f09adc242b..416dd8299700b 100644 --- a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowFormatFactory.java +++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowFormatFactory.java @@ -23,8 +23,8 @@ 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; diff --git a/flink-formats/flink-avro/src/main/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory b/flink-formats/flink-avro/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFormatFactory similarity index 100% rename from flink-formats/flink-avro/src/main/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory rename to flink-formats/flink-avro/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFormatFactory diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroRowFormatFactoryTest.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroRowFormatFactoryTest.java index 4f5e21890bd41..b8a05939ae71d 100644 --- a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroRowFormatFactoryTest.java +++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroRowFormatFactoryTest.java @@ -24,8 +24,8 @@ 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.factories.DeserializationSchemaFactory; +import org.apache.flink.table.factories.SerializationSchemaFactory; import org.apache.flink.table.formats.TableFormatFactoryService; import org.apache.flink.util.TestLogger; diff --git a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java index fd7bda6c7de0c..458b94a5df7f1 100644 --- a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java +++ b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java @@ -26,8 +26,8 @@ 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; diff --git a/flink-formats/flink-json/src/main/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory b/flink-formats/flink-json/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFormatFactory similarity index 100% rename from flink-formats/flink-json/src/main/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory rename to flink-formats/flink-json/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFormatFactory diff --git a/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java index d763b9047139c..a246593ad5479 100644 --- a/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java +++ b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java @@ -26,8 +26,8 @@ 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.factories.DeserializationSchemaFactory; +import org.apache.flink.table.factories.SerializationSchemaFactory; import org.apache.flink.table.formats.TableFormatFactoryService; import org.apache.flink.types.Row; import org.apache.flink.util.TestLogger; diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala index c0cf22cefb0f0..8bd5f963849b6 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala @@ -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.TableFormatFactory /** * Exception for all errors occurring during expression parsing. @@ -143,7 +143,7 @@ case class CatalogAlreadyExistException( } /** - * Exception for not finding a [[org.apache.flink.table.formats.TableFormatFactory]] for the + * Exception for not finding a [[TableFormatFactory]] for the * given properties. * * @param message message that indicates the current matching step @@ -182,7 +182,7 @@ case class NoMatchingTableFormatException( } /** - * Exception for finding more than one [[org.apache.flink.table.formats.TableFormatFactory]] for + * Exception for finding more than one [[TableFormatFactory]] for * the given properties. * * @param matchingFormatFactories format factories that match the properties diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/DeserializationSchemaFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/DeserializationSchemaFactory.scala similarity index 97% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/DeserializationSchemaFactory.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/DeserializationSchemaFactory.scala index da7918579256f..92aa8c956291b 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/DeserializationSchemaFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/DeserializationSchemaFactory.scala @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.formats +package org.apache.flink.table.factories import java.util diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/SerializationSchemaFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/SerializationSchemaFactory.scala similarity index 97% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/SerializationSchemaFactory.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/SerializationSchemaFactory.scala index e4818cdd75bc7..1f6b6bf6cbc37 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/SerializationSchemaFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/SerializationSchemaFactory.scala @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.formats +package org.apache.flink.table.factories import java.util diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala index bd391e3a2e4f4..119de08ca8224 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala @@ -21,7 +21,14 @@ 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 format. 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 TableFactory { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/TableFormatFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFormatFactory.scala similarity index 60% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/TableFormatFactory.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFormatFactory.scala index 3afef839e54c0..63d788be74cc0 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/TableFormatFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFormatFactory.scala @@ -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 diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/TableFormatFactoryService.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/TableFormatFactoryService.scala index 44911a4389b23..8e8a30263f91e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/TableFormatFactoryService.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/TableFormatFactoryService.scala @@ -23,6 +23,7 @@ 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._ +import org.apache.flink.table.factories.TableFormatFactory import org.apache.flink.table.util.Logging import org.apache.flink.util.Preconditions diff --git a/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory b/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFormatFactory similarity index 100% rename from flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.formats.TableFormatFactory rename to flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFormatFactory diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/TableFormatFactoryServiceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/TableFormatFactoryServiceTest.scala index 7d7d2d2446574..3d23f030c044b 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/TableFormatFactoryServiceTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/TableFormatFactoryServiceTest.scala @@ -22,6 +22,7 @@ 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.factories.TableFormatFactory 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} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestAmbiguousTableFormatFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestAmbiguousTableFormatFactory.scala index 1a30ac0bfec41..ca9808650b009 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestAmbiguousTableFormatFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestAmbiguousTableFormatFactory.scala @@ -21,7 +21,8 @@ package org.apache.flink.table.formats.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 +import org.apache.flink.table.formats.TableFormatFactoryServiceTest import org.apache.flink.types.Row /** diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestTableFormatFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestTableFormatFactory.scala index efd9afea5e784..edc189a567b0f 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestTableFormatFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestTableFormatFactory.scala @@ -22,7 +22,8 @@ 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 +import org.apache.flink.table.formats.TableFormatFactoryServiceTest import org.apache.flink.types.Row /** From 5c6df7598d1f1c3c698ae9b6b35eb37d7fff8295 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Thu, 12 Jul 2018 08:35:00 +0200 Subject: [PATCH 05/17] Unify table factories --- .../kafka/KafkaTableSourceFactory.java | 8 +- .../KafkaTableSourceFactoryTestBase.java | 14 +- .../avro/AvroRowFormatFactoryTest.java | 10 +- .../json/JsonRowFormatFactoryTest.java | 10 +- .../table/client/config/Environment.java | 14 +- .../apache/flink/table/api/exceptions.scala | 88 ++-- .../catalog/ExternalTableSourceUtil.scala | 9 +- .../descriptors/LiteralValueValidator.scala | 2 +- .../table/factories/TableFactoryService.scala | 397 ++++++++++++++---- .../table/factories/TableSourceFactory.scala | 3 +- .../formats/TableFormatFactoryService.scala | 272 ------------ .../table/sinks/CsvTableSinkFactory.scala | 2 +- ....apache.flink.table.factories.TableFactory | 13 +- ...e.flink.table.factories.TableFormatFactory | 17 - .../TableFormatFactoryServiceTest.scala | 59 +-- .../TableSinkFactoryServiceTest.scala | 46 +- .../TableSourceFactoryServiceTest.scala | 79 ++-- .../TestAmbiguousTableFormatFactory.scala | 5 +- .../utils/TestDeserializationSchema.scala | 2 +- .../TestFixedFormatTableFactory.scala | 20 +- .../utils/TestSerializationSchema.scala | 2 +- .../utils/TestTableFormat.scala | 2 +- .../utils/TestTableFormatFactory.scala | 5 +- .../{ => utils}/TestTableSinkFactory.scala | 17 +- .../{ => utils}/TestTableSourceFactory.scala | 13 +- ...TestWildcardFormatTableSourceFactory.scala | 16 +- .../table/utils/InMemoryTableFactory.scala | 8 +- 27 files changed, 521 insertions(+), 612 deletions(-) delete mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/TableFormatFactoryService.scala delete mode 100644 flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFormatFactory rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/{formats => factories}/TableFormatFactoryServiceTest.scala (68%) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/{formats => factories}/utils/TestAmbiguousTableFormatFactory.scala (91%) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/{formats => factories}/utils/TestDeserializationSchema.scala (97%) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/{ => utils}/TestFixedFormatTableFactory.scala (78%) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/{formats => factories}/utils/TestSerializationSchema.scala (96%) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/{formats => factories}/utils/TestTableFormat.scala (96%) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/{formats => factories}/utils/TestTableFormatFactory.scala (93%) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/{ => utils}/TestTableSinkFactory.scala (84%) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/{ => utils}/TestTableSourceFactory.scala (83%) rename flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/{ => utils}/TestWildcardFormatTableSourceFactory.scala (78%) diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java index 621b92b4acd2c..25faa88c7eca6 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java @@ -23,13 +23,13 @@ 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.factories.TableFactory; -import org.apache.flink.table.factories.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.factories.DeserializationSchemaFactory; -import org.apache.flink.table.formats.TableFormatFactoryService; +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; @@ -129,7 +129,7 @@ public TableSource createTableSource(Map properties) { 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()); diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java index 9e1b58f9f78fa..6e40c83e384f9 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java @@ -27,15 +27,17 @@ 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; @@ -139,8 +141,12 @@ public void testTableSource() { .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 propertiesMap = descriptorProperties.asMap(); - final TableSource actualSource = TableSourceFactoryService.findAndCreateTableSource(testDesc); + final TableSource actualSource = TableFactoryService.find(TableSourceFactory.class, testDesc) + .createTableSource(propertiesMap); assertEquals(expected, actualSource); diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroRowFormatFactoryTest.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroRowFormatFactoryTest.java index b8a05939ae71d..26fdd53e2984e 100644 --- a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroRowFormatFactoryTest.java +++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroRowFormatFactoryTest.java @@ -26,7 +26,7 @@ import org.apache.flink.table.descriptors.DescriptorProperties; import org.apache.flink.table.factories.DeserializationSchemaFactory; import org.apache.flink.table.factories.SerializationSchemaFactory; -import org.apache.flink.table.formats.TableFormatFactoryService; +import org.apache.flink.table.factories.TableFactoryService; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -63,7 +63,7 @@ public void testAvroSchema() { } private void testRecordClassSerializationSchema(Map 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 @@ private void testRecordClassSerializationSchema(Map properties) } private void testRecordClassDeserializationSchema(Map 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 @@ private void testRecordClassDeserializationSchema(Map properties } private void testAvroSchemaDeserializationSchema(Map 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 @@ private void testAvroSchemaDeserializationSchema(Map properties) } private void testAvroSchemaSerializationSchema(Map properties) { - final SerializationSchema actual1 = TableFormatFactoryService + final SerializationSchema actual1 = TableFactoryService .find(SerializationSchemaFactory.class, properties) .createSerializationSchema(properties); final SerializationSchema expected1 = new AvroRowSerializationSchema(AVRO_SCHEMA); diff --git a/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java index a246593ad5479..f43234ddeb196 100644 --- a/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java +++ b/flink-formats/flink-json/src/test/java/org/apache/flink/formats/json/JsonRowFormatFactoryTest.java @@ -28,7 +28,7 @@ import org.apache.flink.table.descriptors.Schema; import org.apache.flink.table.factories.DeserializationSchemaFactory; import org.apache.flink.table.factories.SerializationSchemaFactory; -import org.apache.flink.table.formats.TableFormatFactoryService; +import org.apache.flink.table.factories.TableFactoryService; import org.apache.flink.types.Row; import org.apache.flink.util.TestLogger; @@ -107,7 +107,7 @@ public void testSchemaDerivation() { } private void testSchemaDeserializationSchema(Map 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 @@ private void testSchemaDeserializationSchema(Map properties) { } private void testSchemaSerializationSchema(Map 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 @@ private void testSchemaSerializationSchema(Map properties) { } private void testJsonSchemaDeserializationSchema(Map 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 @@ private void testJsonSchemaDeserializationSchema(Map properties) } private void testJsonSchemaSerializationSchema(Map properties) { - final SerializationSchema actual1 = TableFormatFactoryService + final SerializationSchema actual1 = TableFactoryService .find(SerializationSchemaFactory.class, properties) .createSerializationSchema(properties); final SerializationSchema expected1 = new JsonRowSerializationSchema(JSON_SCHEMA); diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java index 966a58152d624..2b59e06591956 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java @@ -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 Map getTables() { public void setTables(List> 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 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 static Environment enrich(Environment env, Map properties * @return table descriptor describing a source, sink, or both */ private static TableDescriptor createTableDescriptor(String name, Map 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 normalizedConfig = ConfigUtil.normalizeYaml(config); if (type.equals(TableDescriptorValidator.TABLE_TYPE_VALUE_SOURCE())) { return new Source(name, normalizedConfig); diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala index 8bd5f963849b6..3e3bfac58ed93 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala @@ -19,7 +19,7 @@ package org.apache.flink.table.api import org.apache.flink.table.descriptors.DescriptorProperties -import org.apache.flink.table.factories.TableFormatFactory +import org.apache.flink.table.factories.{TableFactory, TableFormatFactory} /** * Exception for all errors occurring during expression parsing. @@ -143,23 +143,22 @@ case class CatalogAlreadyExistException( } /** - * Exception for not finding a [[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 [[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.factories.TableFactory]] 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.factories.TableFactory]] 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 * diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableSourceUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableSourceUtil.scala index b1fc58a66f5a7..abf27b3115ea4 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableSourceUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableSourceUtil.scala @@ -19,15 +19,13 @@ package org.apache.flink.table.catalog import org.apache.flink.table.api._ -import org.apache.flink.table.factories.{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 => diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/LiteralValueValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/LiteralValueValidator.scala index d9c081f92bc63..943b01abc2021 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/LiteralValueValidator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/LiteralValueValidator.scala @@ -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)) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryService.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryService.scala index 4c49ffc6c0490..fc260d8e01850 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryService.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryService.scala @@ -18,143 +18,358 @@ package org.apache.flink.table.factories -import java.util.{ServiceConfigurationError, ServiceLoader} +import java.util.{ServiceConfigurationError, ServiceLoader, Map => JMap} 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.descriptors._ import org.apache.flink.table.util.Logging +import org.apache.flink.util.Preconditions import _root_.scala.collection.JavaConverters._ import _root_.scala.collection.mutable /** - * Unified interface to search for TableFactoryDiscoverable of provided type and properties. + * Unified interface to search for a [[TableFactory]] of provided type and properties. */ object TableFactoryService extends Logging { private lazy val defaultLoader = ServiceLoader.load(classOf[TableFactory]) - def find(clz: Class[_], descriptor: TableDescriptor): TableFactory = { - find(clz, descriptor, null) + /** + * Finds a table factory of the given class and descriptor. + * + * @param factoryClass desired factory class + * @param descriptor descriptor describing the factory configuration + * @tparam T factory class type + * @return the matching factory + */ + def find[T](factoryClass: Class[T], descriptor: Descriptor): T = { + Preconditions.checkNotNull(factoryClass) + Preconditions.checkNotNull(descriptor) + + val descriptorProperties = new DescriptorProperties() + descriptor.addProperties(descriptorProperties) + findInternal(factoryClass, descriptorProperties.asMap, None) } - def find(clz: Class[_], descriptor: TableDescriptor, classLoader: ClassLoader) - : TableFactory = { + /** + * 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(factoryClass) + Preconditions.checkNotNull(descriptor) + Preconditions.checkNotNull(classLoader) - val properties = new DescriptorProperties() - descriptor.addProperties(properties) - find(clz, properties.asMap.asScala.toMap, classLoader) + val descriptorProperties = new DescriptorProperties() + descriptor.addProperties(descriptorProperties) + findInternal(factoryClass, descriptorProperties.asMap, None) } - def find(clz: Class[_], properties: Map[String, String]): TableFactory = { - find(clz: Class[_], properties, null) + /** + * 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 = { + Preconditions.checkNotNull(factoryClass) + Preconditions.checkNotNull(propertyMap) + + findInternal(factoryClass, propertyMap, None) } - def find(clz: Class[_], properties: Map[String, String], - classLoader: ClassLoader): TableFactory = { + /** + * Finds a table factory of the given class, property map, and classloader. + * + * @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 the matching factory + */ + def find[T]( + factoryClass: Class[T], + propertyMap: JMap[String, String], + classLoader: ClassLoader) + : T = { + Preconditions.checkNotNull(factoryClass) + Preconditions.checkNotNull(propertyMap) + Preconditions.checkNotNull(classLoader) + + findInternal(factoryClass, propertyMap, Some(classLoader)) + } + + /** + * Finds a table factory of the given class, property map, and classloader. + * + * @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 the matching factory + */ + private def findInternal[T]( + factoryClass: Class[T], + propertyMap: JMap[String, String], + classLoader: Option[ClassLoader]) + : T = { + + val properties = propertyMap.asScala.toMap + + // discover table factories + val foundFactories = discoverFactories(classLoader) - var matchingFactory: Option[(TableFactory, Seq[String])] = None + // filter by factory class + val classFactories = filterByFactoryClass( + factoryClass, + properties, + foundFactories) + + // find matching context + val contextFactories = filterByContext( + factoryClass, + properties, + foundFactories, + classFactories) + + // filter by supported keys + filterBySupportedProperties( + factoryClass, + properties, + foundFactories, + contextFactories) + } + + /** + * Searches for factories using Java service providers. + * + * @return all factories in the classpath + */ + private def discoverFactories[T](classLoader: Option[ClassLoader]): Seq[TableFactory] = { + val foundFactories = mutable.ArrayBuffer[TableFactory]() try { - val iter = if (classLoader == null) { - defaultLoader.iterator() - } else { - val customLoader = ServiceLoader.load(classOf[TableFactory], classLoader) - customLoader.iterator() + val iterator = classLoader match { + case Some(customClassLoader) => + val customLoader = ServiceLoader.load(classOf[TableFactory], customClassLoader) + customLoader.iterator() + case None => + defaultLoader.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[TableFactory], requiredContext.keys.toSeq)) - } - } - } + + while (iterator.hasNext) { + val factory = iterator.next() + foundFactories += factory } + + foundFactories } 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 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 matchingFactories = mutable.ArrayBuffer[TableFactory]() + + classFactories.foreach { 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(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 + } + } + + if (matchingFactories.isEmpty) { + throw new NoMatchingTableFactoryException( + "No context matches.", + factoryClass, + foundFactories, + properties) + } + + matchingFactories + } + + /** + * 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 + } - val (factory, context) = matchingFactory - .getOrElse(throw new NoMatchingTableFactoryException(properties)) + /** + * 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 factory '${factory.getClass.getName}' must not be null.") + } + requiredContextJava.asScala.map(e => (e._1.toLowerCase, e._2)).toMap + } - val plainProperties = mutable.ArrayBuffer[String]() + /** + * Filters the matching class factories by supported properties. + */ + private def filterBySupportedProperties[T]( + factoryClass: Class[T], + properties: Map[String, String], + foundFactories: Seq[TableFactory], + classFactories: Seq[TableFactory]) + : T = { + + val plainGivenKeys = 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 + // ignore duplicates + if (!plainGivenKeys.contains(key)) { + plainGivenKeys += key } } + var lastKey: Option[String] = None + val supportedFactories = classFactories.filter { factory => + val requiredContextKeys = normalizeContext(factory).keySet + 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) - val supportedPropertiesJava = try { - factory.supportedProperties() - } catch { - case t: Throwable => - throw new TableException( - s"Table source factory '${factory.getClass.getCanonicalName}' caused an exception.", - t) + givenFilteredKeys.forall { k => + lastKey = Option(k) + supportedKeys.contains(k) || wildcards.exists(k.startsWith) + } } - val supportedProperties = if (supportedPropertiesJava != null) { - supportedPropertiesJava.asScala.map(_.toLowerCase) - } else { - Seq[String]() + if (supportedFactories.isEmpty && classFactories.length == 1 && lastKey.isDefined) { + // 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 NoMatchingTableFactoryException( + s""" + |The matching factory '${factory.getClass.getName}' doesn't support '${lastKey.get}'. + | + |Supported properties of this factory are: + |${supportedKeys.sorted.mkString("\n")}""".stripMargin, + factoryClass, + foundFactories, + properties) + } else if (supportedFactories.isEmpty) { + throw new NoMatchingTableFactoryException( + s"No factory supports all properties.", + factoryClass, + foundFactories, + properties) + } else if (supportedFactories.length > 1) { + throw new AmbiguousTableFactoryException( + supportedFactories, + factoryClass, + foundFactories, + properties) } - // 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")}") - } - } + supportedFactories.head.asInstanceOf[T] + } - // create the table connector - try { - factory - } catch { - case t: Throwable => - throw new TableException( - s"Table connector factory '${factory.getClass.getCanonicalName}' caused an exception.", - t) + /** + * 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 factory '${factory.getClass.getName}' must not be null.") } + 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 } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableSourceFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableSourceFactory.scala index cbfacc1002b3c..06bab6c621c0f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableSourceFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableSourceFactory.scala @@ -22,7 +22,8 @@ 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. diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/TableFormatFactoryService.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/TableFormatFactoryService.scala deleted file mode 100644 index 8e8a30263f91e..0000000000000 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/formats/TableFormatFactoryService.scala +++ /dev/null @@ -1,272 +0,0 @@ -/* - * 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.formats - -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._ -import org.apache.flink.table.factories.TableFormatFactory -import org.apache.flink.table.util.Logging -import org.apache.flink.util.Preconditions - -import _root_.scala.collection.JavaConverters._ -import _root_.scala.collection.mutable - -/** - * Service provider interface for finding a suitable [[TableFormatFactory]] for the - * given properties. - */ -object TableFormatFactoryService extends Logging { - - private lazy val defaultLoader = ServiceLoader.load(classOf[TableFormatFactory[_]]) - - /** - * Finds a table format factory of the given class and creates configured instances from the - * given property map. - * - * @param factoryClass desired format factory - * @param propertyMap properties that describes the format - * @tparam T factory class type - * @return configured instance from 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. - * - * @param factoryClass desired format factory - * @param propertyMap properties that describes the format - * @param classLoader classloader for service loading - * @tparam T factory class type - * @return configured instance from factory - */ - def find[T]( - factoryClass: Class[T], - propertyMap: JMap[String, String], - 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. - * - * @param factoryClass desired format factory - * @param propertyMap properties that describes the format - * @param classLoader optional classloader for service loading - * @tparam T factory class type - * @return configured instance from factory - */ - private def findInternal[T]( - factoryClass: Class[T], - propertyMap: JMap[String, String], - classLoader: Option[ClassLoader]) - : T = { - - Preconditions.checkNotNull(factoryClass) - Preconditions.checkNotNull(propertyMap) - - val properties = propertyMap.asScala.toMap - - // find matching context - val (foundFactories, contextFactories) = findMatchingContext( - factoryClass, - properties, - classLoader) - - // filter by factory class - val classFactories = filterByFactoryClass( - factoryClass, - properties, - foundFactories, - contextFactories) - - // filter by supported keys - filterBySupportedProperties( - factoryClass, - properties, - foundFactories, - classFactories) - } - - 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[_]]() - - try { - val iter = classLoader match { - case Some(customClassLoader) => - val customLoader = ServiceLoader.load(classOf[TableFormatFactory[_]], customClassLoader) - customLoader.iterator() - case None => - defaultLoader.iterator() - } - - while (iter.hasNext) { - val factory = iter.next() - foundFactories += 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) - - // 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) - } - - if (matchingFactories.isEmpty) { - throw new NoMatchingTableFormatException( - "No context matches.", - factoryClass, - foundFactories, - properties) - } - - (foundFactories, matchingFactories) - } - - private def normalizeContext(factory: TableFormatFactory[_]): 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.") - } - 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 - } - - private def filterBySupportedProperties[T]( - factoryClass: Class[T], - properties: Map[String, String], - foundFactories: Seq[TableFormatFactory[_]], - classFactories: Seq[TableFormatFactory[_]]) - : T = { - - val plainGivenKeys = mutable.ArrayBuffer[String]() - properties.keys.foreach { k => - // replace arrays with wildcard - val key = k.replaceAll(".\\d+", ".#") - // ignore duplicates - if (!plainGivenKeys.contains(key)) { - plainGivenKeys += key - } - } - 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 => - lastKey = Option(k) - supportedKeys.contains(k) - } - } - - if (supportedFactories.isEmpty && classFactories.length == 1 && lastKey.isDefined) { - // 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( - s""" - |The matching factory '${factory.getClass.getName}' doesn't support '${lastKey.get}'. - | - |Supported properties of this factory are: - |${supportedKeys.sorted.mkString("\n")}""".stripMargin, - factoryClass, - foundFactories, - properties) - } else if (supportedFactories.isEmpty) { - throw new NoMatchingTableFormatException( - s"No factory supports all properties.", - factoryClass, - foundFactories, - properties) - } else if (supportedFactories.length > 1) { - throw new AmbiguousTableFormatException( - supportedFactories, - factoryClass, - foundFactories, - properties) - } - - supportedFactories.head.asInstanceOf[T] - } - - private def normalizeSupportedProperties(factory: TableFormatFactory[_]): 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.") - } - supportedPropertiesJava.asScala.map(_.toLowerCase) - } -} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala index 9ea6b0e4e142c..0e4f546fb08bf 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala @@ -69,7 +69,7 @@ class CsvTableSinkFactory extends TableSinkFactory[Row] with TableFactory { // validate new FileSystemValidator().validate(params) new CsvValidator().validate(params) - new SchemaValidator().validate(params) + new SchemaValidator(true, false, false).validate(params) // build val csvTableSinkBuilder = new CsvTableSink.Builder diff --git a/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory index 93b290952932c..c97fe8e994552 100644 --- a/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ b/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -13,10 +13,9 @@ # See the License for the specific language governing permissions and # limitations under the License. -<<<<<<< HEAD:flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory -org.apache.flink.table.sources.TestFixedFormatTableFactory -org.apache.flink.table.sources.TestWildcardFormatTableSourceFactory -======= -org.apache.flink.table.factories.TestTableSinkFactory -org.apache.flink.table.factories.TestTableSourceFactory ->>>>>>> 2fce6f6765... [FLINK-8866] [table] Create unified interfaces to configure and instatiate TableSinks:flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +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 diff --git a/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFormatFactory b/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFormatFactory deleted file mode 100644 index b5646a3925d8a..0000000000000 --- a/flink-libraries/flink-table/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFormatFactory +++ /dev/null @@ -1,17 +0,0 @@ -# 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 diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/TableFormatFactoryServiceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableFormatFactoryServiceTest.scala similarity index 68% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/TableFormatFactoryServiceTest.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableFormatFactoryServiceTest.scala index 3d23f030c044b..ab5a0512b43f4 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/TableFormatFactoryServiceTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableFormatFactoryServiceTest.scala @@ -16,21 +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.factories.TableFormatFactory -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 @@ -41,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]) } @@ -61,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]) } @@ -71,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]) @@ -87,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") diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSinkFactoryServiceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSinkFactoryServiceTest.scala index 7e6cf490c9c01..113ad5cc4b419 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSinkFactoryServiceTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSinkFactoryServiceTest.scala @@ -18,28 +18,34 @@ 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") diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSourceFactoryServiceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSourceFactoryServiceTest.scala index 2664c59b987ea..16f185371fbf4 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSourceFactoryServiceTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSourceFactoryServiceTest.scala @@ -18,83 +18,75 @@ 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 } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestAmbiguousTableFormatFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestAmbiguousTableFormatFactory.scala similarity index 91% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestAmbiguousTableFormatFactory.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestAmbiguousTableFormatFactory.scala index ca9808650b009..f8c90788f4704 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestAmbiguousTableFormatFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestAmbiguousTableFormatFactory.scala @@ -16,13 +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.factories.TableFormatFactory -import org.apache.flink.table.formats.TableFormatFactoryServiceTest +import org.apache.flink.table.factories.{TableFormatFactory, TableFormatFactoryServiceTest} import org.apache.flink.types.Row /** diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestDeserializationSchema.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestDeserializationSchema.scala similarity index 97% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestDeserializationSchema.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestDeserializationSchema.scala index 0b519efb2f3bc..dd723e213c536 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestDeserializationSchema.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestDeserializationSchema.scala @@ -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 diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TestFixedFormatTableFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestFixedFormatTableFactory.scala similarity index 78% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TestFixedFormatTableFactory.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestFixedFormatTableFactory.scala index 6a8041b03d2eb..07692be4e3a15 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TestFixedFormatTableFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestFixedFormatTableFactory.scala @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.sources +package org.apache.flink.table.factories.utils import java.util @@ -24,6 +24,9 @@ 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 /** @@ -33,8 +36,8 @@ 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 @@ -45,14 +48,10 @@ class TestFixedFormatTableFactory extends TableSourceFactory[Row] { properties.add("format.path") properties.add("schema.#.name") properties.add("schema.#.field.#.name") - properties.add("failing") properties } - override def create(properties: util.Map[String, String]): TableSource[Row] = { - if (properties.get("failing") == "true") { - throw new IllegalArgumentException("Error in this factory.") - } + override def createTableSource(properties: util.Map[String, String]): TableSource[Row] = { new TableSource[Row] { override def getTableSchema: TableSchema = throw new UnsupportedOperationException() @@ -60,3 +59,8 @@ class TestFixedFormatTableFactory extends TableSourceFactory[Row] { } } } + +object TestFixedFormatTableFactory { + val CONNECTOR_TYPE_VALUE_FIXED = "fixed" + val FORMAT_TYPE_VALUE_TEST = "test" +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestSerializationSchema.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestSerializationSchema.scala similarity index 96% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestSerializationSchema.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestSerializationSchema.scala index 7043ecadd6b9c..ab613a9187e3a 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestSerializationSchema.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestSerializationSchema.scala @@ -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 diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestTableFormat.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableFormat.scala similarity index 96% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestTableFormat.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableFormat.scala index 38cef0533d6b1..5e26995882adf 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestTableFormat.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableFormat.scala @@ -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} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestTableFormatFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableFormatFactory.scala similarity index 93% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestTableFormatFactory.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableFormatFactory.scala index edc189a567b0f..475cff951bd5d 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/formats/utils/TestTableFormatFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableFormatFactory.scala @@ -16,14 +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.factories.DeserializationSchemaFactory -import org.apache.flink.table.formats.TableFormatFactoryServiceTest +import org.apache.flink.table.factories.{DeserializationSchemaFactory, TableFormatFactoryServiceTest} import org.apache.flink.types.Row /** diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TestTableSinkFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableSinkFactory.scala similarity index 84% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TestTableSinkFactory.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableSinkFactory.scala index e3fd1ffb60540..d919889c47a88 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TestTableSinkFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableSinkFactory.scala @@ -16,23 +16,27 @@ * limitations under the License. */ -package org.apache.flink.table.factories +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.descriptors.TableDescriptorValidator +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 +/** + * 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 @@ -67,3 +71,8 @@ class TestTableSinkFactory extends TableSinkFactory[Row] with TableFactory { } } +object TestTableSinkFactory { + val CONNECTOR_TYPE_VALUE_TEST = "test" + val FORMAT_TYPE_VALUE_TEST = "test" +} + diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TestTableSourceFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableSourceFactory.scala similarity index 83% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TestTableSourceFactory.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableSourceFactory.scala index 5b511ccd0fb9b..ff3b24a50d3dd 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TestTableSourceFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableSourceFactory.scala @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.factories +package org.apache.flink.table.factories.utils import java.util @@ -24,6 +24,8 @@ 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 @@ -34,8 +36,8 @@ 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 TableFactory { } } } + +object TestTableSourceFactory { + val CONNECTOR_TYPE_VALUE_TEST = "test" + val FORMAT_TYPE_VALUE_TEST = "test" +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TestWildcardFormatTableSourceFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestWildcardFormatTableSourceFactory.scala similarity index 78% rename from flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TestWildcardFormatTableSourceFactory.scala rename to flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestWildcardFormatTableSourceFactory.scala index 91598a6fdce0a..6c691253df42c 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TestWildcardFormatTableSourceFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestWildcardFormatTableSourceFactory.scala @@ -16,13 +16,16 @@ * limitations under the License. */ -package org.apache.flink.table.sources +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.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 /** @@ -32,7 +35,7 @@ class TestWildcardFormatTableSourceFactory extends TableSourceFactory[Row] with 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 } @@ -42,15 +45,16 @@ class TestWildcardFormatTableSourceFactory extends TableSourceFactory[Row] with properties.add("format.*") properties.add("schema.#.name") properties.add("schema.#.field.#.name") - properties.add("failing") properties } - override def create(properties: util.Map[String, String]): TableSource[Row] = { - this - } + override def createTableSource(properties: util.Map[String, String]): TableSource[Row] = this override def getTableSchema: TableSchema = throw new UnsupportedOperationException() override def getReturnType: TypeInformation[Row] = throw new UnsupportedOperationException() } + +object TestWildcardFormatTableSourceFactory { + val CONNECTOR_TYPE_VALUE_WILDCARD = "wildcard" +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala index bac31f673bbc2..e4ffe2171fad9 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala @@ -49,9 +49,9 @@ class InMemoryTableFactory extends TableSourceFactory[Row] params.putProperties(properties) // validate - new SchemaValidator(true).validate(params) + new SchemaValidator(true, true, true).validate(params) - val tableSchema = SchemaValidator.deriveTableSinkSchema(params); + val tableSchema = SchemaValidator.deriveTableSinkSchema(params) new MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink() .configure(tableSchema.getColumnNames, tableSchema.getTypes) @@ -62,9 +62,9 @@ class InMemoryTableFactory extends TableSourceFactory[Row] params.putProperties(properties) // validate - new SchemaValidator(true).validate(params) + new SchemaValidator(true, true, true).validate(params) - val tableSchema = SchemaValidator.deriveTableSourceSchema(params); + val tableSchema = SchemaValidator.deriveTableSourceSchema(params) // proctime val proctimeAttributeOpt = SchemaValidator.deriveProctimeAttribute(params) From 0cd7c44c006aba21c32d8785d17bfc3dbee03916 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Thu, 12 Jul 2018 09:05:50 +0200 Subject: [PATCH 06/17] Move table type out of descriptors --- .../table/client/config/Environment.java | 23 +++++++++++-------- .../flink/table/client/config/SourceSink.java | 10 ++------ .../TableDescriptorValidator.scala | 15 ------------ .../descriptors/TableSinkDescriptor.scala | 6 +++-- .../descriptors/TableSourceDescriptor.scala | 2 -- 5 files changed, 20 insertions(+), 36 deletions(-) diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java index 2b59e06591956..bc97906630ae3 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java @@ -21,7 +21,6 @@ import org.apache.flink.table.client.SqlClientException; import org.apache.flink.table.descriptors.DescriptorProperties; import org.apache.flink.table.descriptors.TableDescriptor; -import org.apache.flink.table.descriptors.TableDescriptorValidator; import java.io.IOException; import java.net.URL; @@ -50,6 +49,9 @@ public class Environment { private static final String TABLE_NAME = "name"; private static final String TABLE_TYPE = "type"; + private static final String TABLE_TYPE_VALUE_SOURCE = "type"; + private static final String TABLE_TYPE_VALUE_SINK = "sink"; + private static final String TABLE_TYPE_VALUE_BOTH = "both"; public Environment() { this.tables = Collections.emptyMap(); @@ -214,15 +216,18 @@ private static TableDescriptor createTableDescriptor(String name, Map properties = new HashMap<>(config); config.remove(TABLE_TYPE); - final Map normalizedConfig = ConfigUtil.normalizeYaml(config); - if (type.equals(TableDescriptorValidator.TABLE_TYPE_VALUE_SOURCE())) { - return new Source(name, normalizedConfig); - } else if (type.equals(TableDescriptorValidator.TABLE_TYPE_VALUE_SINK())) { - return new Sink(name, normalizedConfig); - } else if (type.equals(TableDescriptorValidator.TABLE_TYPE_VALUE_SOURCE_SINK())) { - return new SourceSink(name, normalizedConfig); + + final Map normalizedProperties = ConfigUtil.normalizeYaml(properties); + switch (type) { + case TABLE_TYPE_VALUE_SOURCE: + return new Source(name, normalizedProperties); + case TABLE_TYPE_VALUE_SINK: + return new Sink(name, normalizedProperties); + case TABLE_TYPE_VALUE_BOTH: + return new SourceSink(name, normalizedProperties); } throw new SqlClientException("Invalid 'type' attribute for table '" + name + "'. " + "Only 'source', 'sink', and 'both' are supported."); diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/SourceSink.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/SourceSink.java index c64887e3535b2..bd5a194cbc085 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/SourceSink.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/SourceSink.java @@ -51,16 +51,10 @@ public void addProperties(DescriptorProperties properties) { } public Source toSource() { - final Map newProperties = new HashMap<>(properties); - newProperties.replace(TableDescriptorValidator.TABLE_TYPE(), - TableDescriptorValidator.TABLE_TYPE_VALUE_SOURCE()); - return new Source(name, newProperties); + return new Source(name, properties); } public Sink toSink() { - final Map newProperties = new HashMap<>(properties); - newProperties.replace(TableDescriptorValidator.TABLE_TYPE(), - TableDescriptorValidator.TABLE_TYPE_VALUE_SINK()); - return new Sink(name, newProperties); + return new Sink(name, properties); } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableDescriptorValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableDescriptorValidator.scala index c712e72602799..e0fa6025811a3 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableDescriptorValidator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableDescriptorValidator.scala @@ -27,18 +27,3 @@ class TableDescriptorValidator extends DescriptorValidator { // nothing to do } } - -object TableDescriptorValidator { - - /** - * Key for describing the type of this table, valid values are ('source', 'sink', 'both'). - */ - val TABLE_TYPE = "type" - - /** - * Valid TABLE_TYPE value. - */ - val TABLE_TYPE_VALUE_SOURCE = "source" - val TABLE_TYPE_VALUE_SINK = "sink" - val TABLE_TYPE_VALUE_SOURCE_SINK = "both" -} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSinkDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSinkDescriptor.scala index 97ed47d65ea54..0a4d5044ff8a5 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSinkDescriptor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSinkDescriptor.scala @@ -22,9 +22,11 @@ package org.apache.flink.table.descriptors * Common class for all descriptors describing a table sink. */ abstract class TableSinkDescriptor extends TableDescriptor { + + /** + * Internal method for properties conversion. + */ override private[flink] def addProperties(properties: DescriptorProperties): Unit = { super.addProperties(properties) - properties.putString(TableDescriptorValidator.TABLE_TYPE, - TableDescriptorValidator.TABLE_TYPE_VALUE_SOURCE) } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSourceDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSourceDescriptor.scala index 2a0b67ce4f78e..3ca39c2bbfca2 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSourceDescriptor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSourceDescriptor.scala @@ -36,8 +36,6 @@ abstract class TableSourceDescriptor extends TableDescriptor { */ override private[flink] def addProperties(properties: DescriptorProperties): Unit = { super.addProperties(properties) - properties.putString(TableDescriptorValidator.TABLE_TYPE, - TableDescriptorValidator.TABLE_TYPE_VALUE_SOURCE) statisticsDescriptor.foreach(_.addProperties(properties)) } From 6b83f2e1c0e63147f049dc5389c5633077b789a4 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Thu, 12 Jul 2018 10:50:09 +0200 Subject: [PATCH 07/17] Make source/sink factories environment-dependent --- .../connectors/kafka/Kafka011TableSource.java | 1 - .../connectors/kafka/Kafka08TableSource.java | 1 - .../Kafka08JsonTableSourceFactoryTest.java | 6 +- .../connectors/kafka/Kafka09TableSource.java | 1 - .../kafka/KafkaTableSourceFactory.java | 9 ++- .../KafkaJsonTableSourceFactoryTestBase.java | 8 +-- .../KafkaTableSourceFactoryTestBase.java | 6 +- ...apache.flink.table.factories.TableFactory} | 0 ...apache.flink.table.factories.TableFactory} | 0 .../gateway/local/ExecutionContext.java | 44 ++++++++++++--- .../gateway/utils/TestTableSinkFactory.java | 8 +-- .../gateway/utils/TestTableSourceFactory.java | 7 +-- .../catalog/ExternalTableSourceUtil.scala | 40 ++++++------- .../BatchTableSourceDescriptor.scala | 19 +++---- .../StreamTableSourceDescriptor.scala | 19 +++---- ...tory.scala => BatchTableSinkFactory.scala} | 20 ++++--- .../factories/BatchTableSourceFactory.scala | 41 ++++++++++++++ .../factories/StreamTableSinkFactory.scala | 41 ++++++++++++++ .../factories/StreamTableSourceFactory.scala | 41 ++++++++++++++ .../table/sinks/AppendStreamTableSink.scala | 2 +- .../table/sinks/CsvTableSinkFactory.scala | 35 ++++++++++-- .../table/sinks/RetractStreamTableSink.scala | 2 +- .../StreamTableSink.scala} | 23 ++++---- .../table/sinks/UpsertStreamTableSink.scala | 2 +- .../table/sources/CsvTableSourceFactory.scala | 27 +++++++-- .../table/sources/StreamTableSource.scala | 2 +- .../TableSinkFactoryServiceTest.scala | 9 ++- .../TableSourceFactoryServiceTest.scala | 10 ++-- .../utils/TestFixedFormatTableFactory.scala | 18 +++--- .../factories/utils/TestTableFormat.scala | 9 +-- .../utils/TestTableSinkFactory.scala | 28 +++------- .../utils/TestTableSourceFactory.scala | 22 +++----- ...TestWildcardFormatTableSourceFactory.scala | 18 +++--- .../table/runtime/stream/sql/SqlITCase.scala | 8 +-- .../table/utils/InMemoryTableFactory.scala | 56 ++++++++++--------- 35 files changed, 370 insertions(+), 213 deletions(-) rename flink-formats/flink-avro/src/main/resources/META-INF/services/{org.apache.flink.table.factories.TableFormatFactory => org.apache.flink.table.factories.TableFactory} (100%) rename flink-formats/flink-json/src/main/resources/META-INF/services/{org.apache.flink.table.factories.TableFormatFactory => org.apache.flink.table.factories.TableFactory} (100%) rename flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/{TableSourceFactory.scala => BatchTableSinkFactory.scala} (64%) create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/BatchTableSourceFactory.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/StreamTableSinkFactory.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/StreamTableSourceFactory.scala rename flink-libraries/flink-table/src/main/scala/org/apache/flink/table/{factories/TableSinkFactory.scala => sinks/StreamTableSink.scala} (64%) diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java index 15a4b7a2eb01d..195663d1a78dd 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java @@ -20,7 +20,6 @@ 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; diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java index 6f6c387c0e0ed..4ead94caaee08 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java @@ -20,7 +20,6 @@ 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; diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceFactoryTest.java index 0238b2bf4f935..915ce42d3132c 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceFactoryTest.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceFactoryTest.java @@ -21,8 +21,12 @@ 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 diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java index a598b035994fc..b770056c1aa11 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java @@ -20,7 +20,6 @@ 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; diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java index 25faa88c7eca6..2c65976c62e3c 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java @@ -27,11 +27,10 @@ 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 @@ /** * Factory for creating configured instances of {@link KafkaTableSource}. */ -public abstract class KafkaTableSourceFactory implements TableSourceFactory, TableFactory { +public abstract class KafkaTableSourceFactory implements StreamTableSourceFactory { @Override public Map requiredContext() { @@ -119,7 +118,7 @@ public List supportedProperties() { } @Override - public TableSource createTableSource(Map properties) { + public StreamTableSource createStreamTableSource(Map properties) { final DescriptorProperties params = new DescriptorProperties(true); params.putProperties(properties); diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java index fbe983d1146bd..51017f415d8ec 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java @@ -22,8 +22,6 @@ 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.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 @@ private void testTableSource(FormatDescriptor format) { 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); } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java index 6e40c83e384f9..8cd058f5bcfbc 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java @@ -32,8 +32,8 @@ 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; @@ -145,8 +145,8 @@ public void testTableSource() { testDesc.addProperties(descriptorProperties); final Map 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); diff --git a/flink-formats/flink-avro/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFormatFactory b/flink-formats/flink-avro/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory similarity index 100% rename from flink-formats/flink-avro/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFormatFactory rename to flink-formats/flink-avro/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory diff --git a/flink-formats/flink-json/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFormatFactory b/flink-formats/flink-json/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory similarity index 100% rename from flink-formats/flink-json/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFormatFactory rename to flink-formats/flink-json/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java index c6b54ce683ce2..13e282a594d6d 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java @@ -46,15 +46,18 @@ 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 ExecutionContext(Environment defaultEnvironment, SessionContext sessionCo final DescriptorProperties properties = new DescriptorProperties(true); descriptor.addProperties(properties); final Map 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 @@ private static ClusterSpecification createClusterSpecification(CustomCommandLine } } + private static TableSource createTableSource(Execution execution, Map 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 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."); + } + // -------------------------------------------------------------------------------------------- /** diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactory.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactory.java index 34060756f8ca1..b75c780b0fba5 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactory.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactory.java @@ -23,11 +23,11 @@ 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.TableSinkFactory; import org.apache.flink.table.descriptors.DescriptorProperties; import org.apache.flink.table.descriptors.SchemaValidator; +import org.apache.flink.table.factories.StreamTableSinkFactory; import org.apache.flink.table.sinks.AppendStreamTableSink; +import org.apache.flink.table.sinks.StreamTableSink; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; @@ -47,7 +47,7 @@ /** * Table sink factory for testing the classloading in {@link DependencyTest}. */ -public class TestTableSinkFactory implements TableSinkFactory, TableFactory { +public class TestTableSinkFactory implements StreamTableSinkFactory { @Override public Map requiredContext() { @@ -69,7 +69,7 @@ public List supportedProperties() { } @Override - public TableSink createTableSink(Map properties) { + public StreamTableSink createStreamTableSink(Map properties) { final DescriptorProperties params = new DescriptorProperties(true); params.putProperties(properties); return new TestTableSink( diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java index cc8c4d40c4fd4..9b623ccc481e6 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java @@ -24,10 +24,9 @@ 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; @@ -52,7 +51,7 @@ /** * Table source factory for testing the classloading in {@link DependencyTest}. */ -public class TestTableSourceFactory implements TableSourceFactory, TableFactory { +public class TestTableSourceFactory implements StreamTableSourceFactory { @Override public Map requiredContext() { @@ -74,7 +73,7 @@ public List supportedProperties() { } @Override - public TableSource createTableSource(Map properties) { + public StreamTableSource createStreamTableSource(Map properties) { final DescriptorProperties params = new DescriptorProperties(true); params.putProperties(properties); final Optional proctime = SchemaValidator.deriveProctimeAttribute(params); diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableSourceUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableSourceUtil.scala index abf27b3115ea4..011cbecb454b1 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableSourceUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableSourceUtil.scala @@ -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.") } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableSourceDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableSourceDescriptor.scala index 3fbb00c3bedc6..c967291d25489 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableSourceDescriptor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableSourceDescriptor.scala @@ -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) } /** diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptor.scala index 6c90f07eb710d..6ade2d6699a42 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptor.scala @@ -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) } /** diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableSourceFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/BatchTableSinkFactory.scala similarity index 64% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableSourceFactory.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/BatchTableSinkFactory.scala index 06bab6c621c0f..6fd1f7afb16b5 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableSourceFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/BatchTableSinkFactory.scala @@ -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] } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/BatchTableSourceFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/BatchTableSourceFactory.scala new file mode 100644 index 0000000000000..b60861eefedc9 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/BatchTableSourceFactory.scala @@ -0,0 +1,41 @@ +/* + * 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] +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/StreamTableSinkFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/StreamTableSinkFactory.scala new file mode 100644 index 0000000000000..b0c220ca1311c --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/StreamTableSinkFactory.scala @@ -0,0 +1,41 @@ +/* + * 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] +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/StreamTableSourceFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/StreamTableSourceFactory.scala new file mode 100644 index 0000000000000..d4e50506aa702 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/StreamTableSourceFactory.scala @@ -0,0 +1,41 @@ +/* + * 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] +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/AppendStreamTableSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/AppendStreamTableSink.scala index abdca172ad23f..d73380eef5497 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/AppendStreamTableSink.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/AppendStreamTableSink.scala @@ -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 diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala index 0e4f546fb08bf..1c37be2761cc5 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala @@ -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]() @@ -62,14 +65,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 csvTableSinkBuilder = new CsvTableSink.Builder @@ -91,6 +113,9 @@ class CsvTableSinkFactory extends TableSinkFactory[Row] with TableFactory { toScala(params.getOptionalString(FORMAT_FIELD_DELIMITER)) .foreach(csvTableSinkBuilder.fieldDelimiter) - csvTableSinkBuilder.build().configure(formatSchema.getColumnNames, formatSchema.getTypes) + csvTableSinkBuilder + .build() + .configure(formatSchema.getColumnNames, formatSchema.getTypes) + .asInstanceOf[CsvTableSink] } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/RetractStreamTableSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/RetractStreamTableSink.scala index 3ab997ed4a2d5..2d7c46fcfd313 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/RetractStreamTableSink.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/RetractStreamTableSink.scala @@ -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] diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableSinkFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/StreamTableSink.scala similarity index 64% rename from flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableSinkFactory.scala rename to flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/StreamTableSink.scala index fc7b3652b4d7e..55fbfac03d6fb 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableSinkFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/StreamTableSink.scala @@ -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] } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/UpsertStreamTableSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/UpsertStreamTableSink.scala index 9c735d0658230..c171573eed135 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/UpsertStreamTableSink.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/UpsertStreamTableSink.scala @@ -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. diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala index c5136694460bf..96751ec650f16 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala @@ -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) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/StreamTableSource.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/StreamTableSource.scala index 7a2737c5dfd73..0e687a4c96f9e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/StreamTableSource.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/StreamTableSource.scala @@ -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]]. */ diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSinkFactoryServiceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSinkFactoryServiceTest.scala index 113ad5cc4b419..90762ce8097e2 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSinkFactoryServiceTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSinkFactoryServiceTest.scala @@ -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 } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSourceFactoryServiceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSourceFactoryServiceTest.scala index 16f185371fbf4..663d2e5a41229 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSourceFactoryServiceTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSourceFactoryServiceTest.scala @@ -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]) } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestFixedFormatTableFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestFixedFormatTableFactory.scala index 07692be4e3a15..19f7282d44ea0 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestFixedFormatTableFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestFixedFormatTableFactory.scala @@ -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() } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableFormat.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableFormat.scala index 5e26995882adf..f00b6a1fd8966 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableFormat.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableFormat.scala @@ -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") } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableSinkFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableSinkFactory.scala index d919889c47a88..0a40530a68668 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableSinkFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableSinkFactory.scala @@ -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() } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableSourceFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableSourceFactory.scala index ff3b24a50d3dd..b5584a906ad0a 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableSourceFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableSourceFactory.scala @@ -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() } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestWildcardFormatTableSourceFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestWildcardFormatTableSourceFactory.scala index 6c691253df42c..e241b99096de6 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestWildcardFormatTableSourceFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestWildcardFormatTableSourceFactory.scala @@ -20,18 +20,16 @@ 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.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] with TableSource[Row] { +class TestWildcardFormatTableSourceFactory extends StreamTableSourceFactory[Row] { override def requiredContext(): util.Map[String, String] = { val context = new util.HashMap[String, String]() @@ -48,11 +46,11 @@ class TestWildcardFormatTableSourceFactory extends TableSourceFactory[Row] with properties } - override def createTableSource(properties: util.Map[String, String]): TableSource[Row] = this - - 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() + } } object TestWildcardFormatTableSourceFactory { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala index 3ab3cb8ba38c9..8ef99a83e80b1 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala @@ -718,9 +718,9 @@ class SqlITCase extends StreamingWithStateTestBase { @Test def testWriteReadTableSourceSink(): Unit = { - var env = StreamExecutionEnvironment.getExecutionEnvironment + val env = StreamExecutionEnvironment.getExecutionEnvironment env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) - var tEnv = TableEnvironment.getTableEnvironment(env) + val tEnv = TableEnvironment.getTableEnvironment(env) MemoryTableSourceSinkUtil.clear val desc = Schema() @@ -739,9 +739,9 @@ class SqlITCase extends StreamingWithStateTestBase { tEnv.registerTable("sourceTable", t) tEnv.registerTableSource("targetTable", - new InMemoryTableFactory().createTableSource(props.asMap)) + new InMemoryTableFactory().createStreamTableSource(props.asMap)) tEnv.registerTableSink("targetTable", - new InMemoryTableFactory().createTableSink(props.asMap)) + new InMemoryTableFactory().createStreamTableSink(props.asMap)) tEnv.sqlUpdate("INSERT INTO targetTable SELECT a, b, c, rowtime FROM sourceTable") tEnv.sqlQuery("SELECT a, e, f, t from targetTable") diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala index e4ffe2171fad9..36b3c16f50ecc 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala @@ -21,48 +21,52 @@ package org.apache.flink.table.utils import java.util import org.apache.flink.api.java.typeutils.RowTypeInfo -import org.apache.flink.table.factories.{TableFactory, TableSinkFactory, TableSourceFactory} -import org.apache.flink.table.sources.TableSource -import org.apache.flink.types.Row -import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_PROPERTY_VERSION -import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE +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.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_CLASS -import org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_FROM -import org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_SERIALIZED -import org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_TYPE -import org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_WATERMARKS_CLASS -import org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_WATERMARKS_DELAY -import org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_WATERMARKS_SERIALIZED -import org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_WATERMARKS_TYPE -import org.apache.flink.table.descriptors.SchemaValidator.SCHEMA -import org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_FROM -import org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_NAME -import org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_PROCTIME -import org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_TYPE -import org.apache.flink.table.sinks.TableSink - -class InMemoryTableFactory extends TableSourceFactory[Row] - with TableSinkFactory[Row] with TableFactory { - override def createTableSink(properties: util.Map[String, String]): TableSink[Row] = { +import org.apache.flink.table.factories._ +import org.apache.flink.table.sinks.StreamTableSink +import org.apache.flink.table.sources.StreamTableSource +import org.apache.flink.types.Row + +class InMemoryTableFactory + 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) From 4f1255fd003080f078afe6ef67ffa58f40ffec36 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Thu, 12 Jul 2018 20:48:45 +0200 Subject: [PATCH 08/17] Clean up and simplify changes --- flink-libraries/flink-sql-client/pom.xml | 4 +- .../table/client/config/Environment.java | 4 +- .../flink/table/client/config/Sink.java | 4 +- .../flink/table/client/config/Source.java | 3 +- .../flink/table/client/config/SourceSink.java | 3 +- .../client/gateway/local/LocalExecutor.java | 12 -- .../client/gateway/local/ProgramDeployer.java | 59 +++------ .../assembly/test-table-source-factory.xml | 2 - .../client/gateway/local/DependencyTest.java | 9 +- .../gateway/local/LocalExecutorITCase.java | 3 +- .../gateway/utils/TestTableSinkFactory.java | 123 ------------------ .../gateway/utils/TestTableSourceFactory.java | 3 +- .../test/resources/test-factory-services-file | 1 - .../resources/test-sql-client-factory.yaml | 2 +- .../resources/tableSourceConverter.properties | 36 ----- .../table/api/BatchTableEnvironment.scala | 104 +++++++++++---- .../table/api/StreamTableEnvironment.scala | 104 ++++++++++----- .../flink/table/api/TableEnvironment.scala | 18 +-- .../api/java/BatchTableEnvironment.scala | 2 - .../flink/table/descriptors/FileSystem.scala | 27 +--- .../descriptors/FileSystemValidator.scala | 10 +- .../table/descriptors/SchemaValidator.scala | 26 +++- .../flink/table/factories/TableFactory.scala | 3 +- .../plan/nodes/PhysicalTableSourceScan.scala | 9 +- .../logical/FlinkLogicalTableSourceScan.scala | 21 ++- .../dataSet/BatchTableSourceScanRule.scala | 17 +-- .../StreamTableSourceScanRule.scala | 17 +-- .../PushFilterIntoTableSourceScanRule.scala | 3 +- .../plan/schema/TableSourceSinkTable.scala | 38 +++++- .../flink/table/sinks/CsvTableSink.scala | 92 ------------- .../table/sinks/CsvTableSinkFactory.scala | 21 +-- .../catalog/ExternalCatalogSchemaTest.scala | 2 +- .../table/descriptors/FileSystemTest.scala | 15 +-- .../descriptors/SchemaValidatorTest.scala | 4 +- .../TableSourceDescriptorTest.scala | 3 + .../TableSinkFactoryServiceTest.scala | 2 +- .../utils/TestTableSinkFactory.scala | 3 +- .../batch/sql/TableEnvironmentITCase.scala | 4 +- .../batch/table/TableEnvironmentITCase.scala | 4 +- .../runtime/stream/TimeAttributesITCase.scala | 4 +- .../table/runtime/stream/sql/SqlITCase.scala | 10 +- .../table/utils/InMemoryTableFactory.scala | 12 +- .../utils/MemoryTableSourceSinkUtil.scala | 32 +++-- 43 files changed, 333 insertions(+), 542 deletions(-) delete mode 100644 flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactory.java delete mode 100644 flink-libraries/flink-table/src/main/resources/tableSourceConverter.properties diff --git a/flink-libraries/flink-sql-client/pom.xml b/flink-libraries/flink-sql-client/pom.xml index 924a95854e4ae..a9fc00e7b417a 100644 --- a/flink-libraries/flink-sql-client/pom.xml +++ b/flink-libraries/flink-sql-client/pom.xml @@ -177,13 +177,13 @@ under the License. 2.4 - create-table-connector-factory-jar + create-table-source-factory-jar process-test-classes single - table-connector-factory + table-source-factory false src/test/assembly/test-table-source-factory.xml diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java index bc97906630ae3..edf43253612b1 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java @@ -49,7 +49,7 @@ public class Environment { private static final String TABLE_NAME = "name"; private static final String TABLE_TYPE = "type"; - private static final String TABLE_TYPE_VALUE_SOURCE = "type"; + private static final String TABLE_TYPE_VALUE_SOURCE = "source"; private static final String TABLE_TYPE_VALUE_SINK = "sink"; private static final String TABLE_TYPE_VALUE_BOTH = "both"; @@ -218,7 +218,7 @@ private static TableDescriptor createTableDescriptor(String name, Map properties = new HashMap<>(config); - config.remove(TABLE_TYPE); + properties.remove(TABLE_TYPE); final Map normalizedProperties = ConfigUtil.normalizeYaml(properties); switch (type) { diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Sink.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Sink.java index eb8c360f3a717..0de65fb92e5db 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Sink.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Sink.java @@ -24,10 +24,10 @@ import java.util.Map; /** - * Configuration of a table sink. Parses an entry in the `tables` list of an environment - * file and translates to table descriptor properties. + * Configuration of a table sink. */ public class Sink extends TableSinkDescriptor { + private String name; private Map properties; diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Source.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Source.java index 08e8cf5c7b4d6..ef80596323dc9 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Source.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Source.java @@ -24,8 +24,7 @@ import java.util.Map; /** - * Configuration of a table source. Parses an entry in the `tables` list of an environment - * file and translates to table descriptor properties. + * Configuration of a table source. */ public class Source extends TableSourceDescriptor { diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/SourceSink.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/SourceSink.java index bd5a194cbc085..bfa3c4444629f 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/SourceSink.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/SourceSink.java @@ -20,15 +20,14 @@ import org.apache.flink.table.descriptors.DescriptorProperties; import org.apache.flink.table.descriptors.TableDescriptor; -import org.apache.flink.table.descriptors.TableDescriptorValidator; -import java.util.HashMap; import java.util.Map; /** * Common class for all descriptors describing a table source and sink together. */ public class SourceSink extends TableDescriptor { + private String name; private Map properties; diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java index a178b7e135727..d658ee994d130 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java @@ -329,18 +329,6 @@ private void cancelQueryInternal(ExecutionContext context, String resultI } } - private void executeUpdateInternal(ExecutionContext context, String query) { - final ExecutionContext.EnvironmentInstance envInst = context.createEnvironmentInstance(); - - envInst.getTableEnvironment().sqlUpdate(query); - // create job graph with dependencies - final String jobName = context.getSessionContext().getName() + ": " + query; - final JobGraph jobGraph = envInst.createJobGraph(jobName); - - // create execution - new Thread(new ProgramDeployer<>(context, jobName, jobGraph, null)).start(); - } - private ResultDescriptor executeQueryInternal(ExecutionContext context, String query) { final ExecutionContext.EnvironmentInstance envInst = context.createEnvironmentInstance(); diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ProgramDeployer.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ProgramDeployer.java index b89af321c04ef..5c5cf982e478a 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ProgramDeployer.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ProgramDeployer.java @@ -19,7 +19,6 @@ package org.apache.flink.table.client.gateway.local; import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.JobSubmissionResult; import org.apache.flink.client.deployment.ClusterDescriptor; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.rest.RestClusterClient; @@ -63,65 +62,45 @@ public void run() { LOG.debug("Submitting job {} with the following environment: \n{}", jobGraph.getJobID(), context.getMergedEnvironment()); } - if (result != null) { - executionResultBucket.add(deployJob(context, jobGraph, result)); - } else { - deployJob(context, jobGraph, result); - } + executionResultBucket.add(deployJob(context, jobGraph, result)); } public JobExecutionResult fetchExecutionResult() { - if (result != null) { - return executionResultBucket.poll(); - } else { - return null; - } + return executionResultBucket.poll(); } /** - * Deploys a job. Depending on the deployment creates a new job cluster. If result is requested, - * it saves the cluster id in the result and blocks until job completion. + * Deploys a job. Depending on the deployment creates a new job cluster. It saves the cluster id in + * the result and blocks until job completion. */ private JobExecutionResult deployJob(ExecutionContext context, JobGraph jobGraph, DynamicResult result) { - final boolean retrieveResults = result != null; // create or retrieve cluster and deploy job try (final ClusterDescriptor clusterDescriptor = context.createClusterDescriptor()) { ClusterClient clusterClient = null; try { // new cluster if (context.getClusterId() == null) { - // deploy job cluster, attach the job if result is requested - clusterClient = clusterDescriptor.deployJobCluster( - context.getClusterSpec(), jobGraph, !retrieveResults); - if (retrieveResults) { - // save the new cluster id - result.setClusterId(clusterClient.getClusterId()); - // we need to hard cast for now - return ((RestClusterClient) clusterClient) - .requestJobResult(jobGraph.getJobID()) - .get() - .toJobExecutionResult(context.getClassLoader()); // throws exception if job fails - } else { - return null; - } + // deploy job cluster with job attached + clusterClient = clusterDescriptor.deployJobCluster(context.getClusterSpec(), jobGraph, false); + // save the new cluster id + result.setClusterId(clusterClient.getClusterId()); + // we need to hard cast for now + return ((RestClusterClient) clusterClient) + .requestJobResult(jobGraph.getJobID()) + .get() + .toJobExecutionResult(context.getClassLoader()); // throws exception if job fails } // reuse existing cluster else { // retrieve existing cluster clusterClient = clusterDescriptor.retrieve(context.getClusterId()); - if (retrieveResults) { - // save the cluster id - result.setClusterId(clusterClient.getClusterId()); - } + // save the cluster id + result.setClusterId(clusterClient.getClusterId()); // submit the job - clusterClient.setDetached(!retrieveResults); - JobSubmissionResult submissionResult = - clusterClient.submitJob(jobGraph, context.getClassLoader()); - if (retrieveResults) { - return submissionResult.getJobExecutionResult(); - } else { - return null; - } + clusterClient.setDetached(false); + return clusterClient + .submitJob(jobGraph, context.getClassLoader()) + .getJobExecutionResult(); // throws exception if job fails } } catch (Exception e) { throw new SqlExecutionException("Could not retrieve or create a cluster.", e); diff --git a/flink-libraries/flink-sql-client/src/test/assembly/test-table-source-factory.xml b/flink-libraries/flink-sql-client/src/test/assembly/test-table-source-factory.xml index fafffae4a2d2b..482781435a720 100644 --- a/flink-libraries/flink-sql-client/src/test/assembly/test-table-source-factory.xml +++ b/flink-libraries/flink-sql-client/src/test/assembly/test-table-source-factory.xml @@ -32,8 +32,6 @@ under the License. org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.class org/apache/flink/table/client/gateway/utils/TestTableSourceFactory$*.class - org/apache/flink/table/client/gateway/utils/TestTableSinkFactory.class - org/apache/flink/table/client/gateway/utils/TestTableSinkFactory$*.class diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java index 0f892303b9792..6760583dcfdea 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java @@ -42,20 +42,19 @@ public class DependencyTest { private static final String FACTORY_ENVIRONMENT_FILE = "test-sql-client-factory.yaml"; - private static final String TABLE_CONNECTOR_FACTORY_JAR_FILE = - "table-connector-factory-test-jar.jar"; + private static final String TABLE_SOURCE_FACTORY_JAR_FILE = "table-source-factory-test-jar.jar"; @Test - public void testTableFactoryDiscovery() throws Exception { + public void testTableSourceFactoryDiscovery() throws Exception { // create environment final Map replaceVars = new HashMap<>(); - replaceVars.put("$VAR_0", "test-connector"); + replaceVars.put("$VAR_0", "test-table-source-factory"); replaceVars.put("$VAR_1", "test-property"); replaceVars.put("$VAR_2", "test-value"); final Environment env = EnvironmentFileUtil.parseModified(FACTORY_ENVIRONMENT_FILE, replaceVars); // create executor with dependencies - final URL dependency = Paths.get("target", TABLE_CONNECTOR_FACTORY_JAR_FILE).toUri().toURL(); + final URL dependency = Paths.get("target", TABLE_SOURCE_FACTORY_JAR_FILE).toUri().toURL(); final LocalExecutor executor = new LocalExecutor( env, Collections.singletonList(dependency), diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java index b4253d9215c01..8f62be448e767 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java @@ -69,6 +69,7 @@ public class LocalExecutorITCase extends TestLogger { private static final int NUM_TMS = 2; private static final int NUM_SLOTS_PER_TM = 2; + @ClassRule public static final MiniClusterResource MINI_CLUSTER_RESOURCE = new MiniClusterResource( new MiniClusterResourceConfiguration.Builder() @@ -159,7 +160,7 @@ public void testTableSchema() throws Exception { } @Test(timeout = 30_000L) - public void testQueryExecutionChangelog() throws Exception { + public void testStreamQueryExecutionChangelog() throws Exception { final URL url = getClass().getClassLoader().getResource("test-data.csv"); Objects.requireNonNull(url); final Map replaceVars = new HashMap<>(); diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactory.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactory.java deleted file mode 100644 index b75c780b0fba5..0000000000000 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSinkFactory.java +++ /dev/null @@ -1,123 +0,0 @@ -/* - * 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.client.gateway.utils; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.streaming.api.datastream.DataStream; -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.descriptors.DescriptorProperties; -import org.apache.flink.table.descriptors.SchemaValidator; -import org.apache.flink.table.factories.StreamTableSinkFactory; -import org.apache.flink.table.sinks.AppendStreamTableSink; -import org.apache.flink.table.sinks.StreamTableSink; -import org.apache.flink.table.sinks.TableSink; -import org.apache.flink.types.Row; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE; -import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_FROM; -import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_TYPE; -import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_WATERMARKS_TYPE; -import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA; -import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_NAME; -import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_TYPE; - -/** - * Table sink factory for testing the classloading in {@link DependencyTest}. - */ -public class TestTableSinkFactory implements StreamTableSinkFactory { - - @Override - public Map requiredContext() { - final Map context = new HashMap<>(); - context.put(CONNECTOR_TYPE(), "test-connector"); - return context; - } - - @Override - public List supportedProperties() { - final List properties = new ArrayList<>(); - properties.add("connector.test-property"); - properties.add(SCHEMA() + ".#." + SCHEMA_TYPE()); - properties.add(SCHEMA() + ".#." + SCHEMA_NAME()); - properties.add(SCHEMA() + ".#." + ROWTIME_TIMESTAMPS_TYPE()); - properties.add(SCHEMA() + ".#." + ROWTIME_TIMESTAMPS_FROM()); - properties.add(SCHEMA() + ".#." + ROWTIME_WATERMARKS_TYPE()); - return properties; - } - - @Override - public StreamTableSink createStreamTableSink(Map properties) { - final DescriptorProperties params = new DescriptorProperties(true); - params.putProperties(properties); - return new TestTableSink( - SchemaValidator.deriveTableSinkSchema(params), - properties.get("connector.test-property")); - } - - // -------------------------------------------------------------------------------------------- - - /** - * Test table sink. - */ - public static class TestTableSink implements TableSink, AppendStreamTableSink { - - private final TableSchema schema; - private final String property; - - public TestTableSink(TableSchema schema, String property) { - this.schema = schema; - this.property = property; - } - - public String getProperty() { - return property; - } - - @Override - public TypeInformation getOutputType() { - return Types.ROW(schema.getColumnNames(), schema.getTypes()); - } - - @Override - public String[] getFieldNames() { - return schema.getColumnNames(); - } - - @Override - public TypeInformation[] getFieldTypes() { - return schema.getTypes(); - } - - @Override - public TableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { - return new TestTableSink(new TableSchema(fieldNames, fieldTypes), property); - } - - @Override - public void emitDataStream(DataStream dataStream) { - } - } -} diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java index 9b623ccc481e6..9911176db6d22 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java @@ -31,7 +31,6 @@ import org.apache.flink.table.sources.DefinedRowtimeAttributes; import org.apache.flink.table.sources.RowtimeAttributeDescriptor; import org.apache.flink.table.sources.StreamTableSource; -import org.apache.flink.table.sources.TableSource; import org.apache.flink.types.Row; import java.util.ArrayList; @@ -56,7 +55,7 @@ public class TestTableSourceFactory implements StreamTableSourceFactory { @Override public Map requiredContext() { final Map context = new HashMap<>(); - context.put(CONNECTOR_TYPE(), "test-connector"); + context.put(CONNECTOR_TYPE(), "test-table-source-factory"); return context; } diff --git a/flink-libraries/flink-sql-client/src/test/resources/test-factory-services-file b/flink-libraries/flink-sql-client/src/test/resources/test-factory-services-file index cf76016dcdc76..41e7fb2cbd561 100644 --- a/flink-libraries/flink-sql-client/src/test/resources/test-factory-services-file +++ b/flink-libraries/flink-sql-client/src/test/resources/test-factory-services-file @@ -18,4 +18,3 @@ #============================================================================== org.apache.flink.table.client.gateway.utils.TestTableSourceFactory -org.apache.flink.table.client.gateway.utils.TestTableSinkFactory diff --git a/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml b/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml index 01ad63d9db0ae..b1924797ff173 100644 --- a/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml +++ b/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml @@ -25,7 +25,7 @@ tables: - name: TableNumber1 - type: both + type: source schema: - name: IntegerField1 type: INT diff --git a/flink-libraries/flink-table/src/main/resources/tableSourceConverter.properties b/flink-libraries/flink-table/src/main/resources/tableSourceConverter.properties deleted file mode 100644 index 9a54834ef6fa9..0000000000000 --- a/flink-libraries/flink-table/src/main/resources/tableSourceConverter.properties +++ /dev/null @@ -1,36 +0,0 @@ -################################################################################ -# 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. -################################################################################ - -################################################################################ -# NOTE: THIS APPROACH IS DEPRECATED AND WILL BE REMOVED IN FUTURE VERSIONS! -# -# We recommend to use a org.apache.flink.table.connector.TableConnectorFactory -# instead. They allow to define new factories by using Java Service Providers. -################################################################################ - -################################################################################ -# The config file is used to specify the packages of current module where -# to find TableSourceConverter implementation class annotated with TableType. -# If there are multiple packages to scan, put those packages together into a -# string separated with ',', for example, org.package1,org.package2. -# Please notice: -# It's better to have a tableSourceConverter.properties in each connector Module -# which offers converters instead of put all information into the -# tableSourceConverter.properties of flink-table module. -################################################################################ -scan.packages=org.apache.flink.table.sources diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala index fe0bc9e64770d..a239ad56e1ed4 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala @@ -103,24 +103,40 @@ abstract class BatchTableEnvironment( : Unit = { tableSource match { + + // check for proper batch table source case batchTableSource: BatchTableSource[_] => + // check if a table (source or sink) is registered Option(getTable(name)) match { - case Some(table: TableSourceSinkTable[_, _]) => table.tableSourceTableOpt match { + + // table source and/or sink is registered + case Some(table: TableSourceSinkTable[_, _]) => table.tableSourceTable match { + + // wrapper contains source case Some(_: TableSourceTable[_]) => - throw new TableException(s"Table \'$name\' already exists. " + - s"Please, choose a different name.") - case _ => replaceRegisteredTable(name, - new TableSourceSinkTable(Some(new BatchTableSourceTable(batchTableSource)), - table.tableSinkTableOpt)) + throw new TableException(s"Table '$name' already exists. " + + s"Please choose a different name.") + + // wrapper contains only sink (not source) + case _ => + val enrichedTable = new TableSourceSinkTable( + Some(new BatchTableSourceTable(batchTableSource)), + table.tableSinkTable) + replaceRegisteredTable(name, enrichedTable) } - case None => registerTableInternal(name, - new TableSourceSinkTable(Some(new BatchTableSourceTable(batchTableSource)), None)) - case _ => throw new TableException(s"Table \'$name\' already exists. " + - s"Please, choose a different name.") + + // no table is registered + case None => + val newTable = new TableSourceSinkTable( + Some(new BatchTableSourceTable(batchTableSource)), + None) + registerTableInternal(name, newTable) } + + // not a batch table source case _ => throw new TableException("Only BatchTableSource can be registered in " + - "BatchTableEnvironment") + "BatchTableEnvironment.") } } @@ -193,7 +209,7 @@ abstract class BatchTableEnvironment( fieldNames: Array[String], fieldTypes: Array[TypeInformation[_]], tableSink: TableSink[_]): Unit = { - + // validate checkValidTableName(name) if (fieldNames == null) throw TableException("fieldNames must not be null.") if (fieldTypes == null) throw TableException("fieldTypes must not be null.") @@ -202,34 +218,70 @@ abstract class BatchTableEnvironment( throw new TableException("Same number of field names and types required.") } + // configure and register val configuredSink = tableSink.configure(fieldNames, fieldTypes) registerTableSinkInternal(name, configuredSink) } + /** + * Registers an external [[TableSink]] with already configured field names and field types in + * this [[TableEnvironment]]'s catalog. + * Registered sink tables can be referenced in SQL DML statements. + * + * @param name The name under which the [[TableSink]] is registered. + * @param configuredSink The configured [[TableSink]] to register. + */ def registerTableSink(name: String, configuredSink: TableSink[_]): Unit = { - checkValidTableName(name) - if (configuredSink.getFieldNames == null || configuredSink.getFieldTypes == null) { - throw TableException("TableSink is not configured.") - } registerTableSinkInternal(name, configuredSink) } private def registerTableSinkInternal(name: String, configuredSink: TableSink[_]): Unit = { + // validate + checkValidTableName(name) + if (configuredSink.getFieldNames == null || configuredSink.getFieldTypes == null) { + throw new TableException("Table sink is not configured.") + } + if (configuredSink.getFieldNames.length == 0) { + throw new TableException("Field names must not be empty.") + } + if (configuredSink.getFieldNames.length != configuredSink.getFieldTypes.length) { + throw new TableException("Same number of field names and types required.") + } + + // register configuredSink match { - case batchTableSink: BatchTableSink[_] => + + // check for proper batch table sink + case _: BatchTableSink[_] => + + // check if a table (source or sink) is registered Option(getTable(name)) match { - case Some(table: TableSourceSinkTable[_, _]) => table.tableSinkTableOpt match { + + // table source and/or sink is registered + case Some(table: TableSourceSinkTable[_, _]) => table.tableSinkTable match { + + // wrapper contains sink case Some(_: TableSinkTable[_]) => - throw new TableException(s"Table \'$name\' already exists. " + - s"Please, choose a different name.") - case _ => replaceRegisteredTable (name, new TableSourceSinkTable( - table.tableSourceTableOpt, Some(new TableSinkTable(configuredSink)))) + throw new TableException(s"Table '$name' already exists. " + + s"Please choose a different name.") + + // wrapper contains only source (not sink) + case _ => + val enrichedTable = new TableSourceSinkTable( + table.tableSourceTable, + Some(new TableSinkTable(configuredSink))) + replaceRegisteredTable(name, enrichedTable) } - case None => registerTableInternal(name, - new TableSourceSinkTable(None, Some(new TableSinkTable(configuredSink)))) - case _ => throw new TableException(s"Table \'$name\' already exists. " + - s"Please, choose a different name.") + + // no table is registered + case _ => + val newTable = new TableSourceSinkTable( + None, + Some(new TableSinkTable(configuredSink))) + registerTableInternal(name, newTable) } + + // not a batch table sink case _ => throw new TableException("Only BatchTableSink can be registered in BatchTableEnvironment.") } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala index f6ec07555062e..33b984dd8eb43 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala @@ -114,6 +114,8 @@ abstract class StreamTableEnvironment( : Unit = { tableSource match { + + // check for proper stream table source case streamTableSource: StreamTableSource[_] => // check that event-time is enabled if table source includes rowtime attributes if (TableSourceUtil.hasRowtimeAttribute(streamTableSource) && @@ -122,21 +124,35 @@ abstract class StreamTableEnvironment( s"A rowtime attribute requires an EventTime time characteristic in stream " + s"environment. But is: ${execEnv.getStreamTimeCharacteristic}") } + + // register Option(getTable(name)) match { - case Some(table: TableSourceSinkTable[_, _]) => table.tableSourceTableOpt match { + + // check if a table (source or sink) is registered + case Some(table: TableSourceSinkTable[_, _]) => table.tableSourceTable match { + + // wrapper contains source case Some(_: TableSourceTable[_]) => - throw new TableException(s"Table \'$name\' already exists. " + - s"Please, choose a different name.") - case _ => replaceRegisteredTable(name, - new TableSourceSinkTable(Some(new StreamTableSourceTable(streamTableSource)), - table.tableSinkTableOpt)) + throw new TableException(s"Table '$name' already exists. " + + s"Please choose a different name.") + + // wrapper contains only sink (not source) + case _ => + val enrichedTable = new TableSourceSinkTable( + Some(new StreamTableSourceTable(streamTableSource)), + table.tableSinkTable) + replaceRegisteredTable(name, enrichedTable) } - case None => registerTableInternal(name, - new TableSourceSinkTable(Some(new StreamTableSourceTable(streamTableSource)), - None)) - case _ => throw new TableException(s"Table \'$name\' already exists. " + - s"Please, choose a different name.") + + // no table is registered + case None => + val newTable = new TableSourceSinkTable( + Some(new StreamTableSourceTable(streamTableSource)), + None) + registerTableInternal(name, newTable) } + + // not a stream table source case _ => throw new TableException("Only StreamTableSource can be registered in " + "StreamTableEnvironment") @@ -225,35 +241,65 @@ abstract class StreamTableEnvironment( registerTableSinkInternal(name, configuredSink) } + /** + * Registers an external [[TableSink]] with already configured field names and field types in + * this [[TableEnvironment]]'s catalog. + * Registered sink tables can be referenced in SQL DML statements. + * + * @param name The name under which the [[TableSink]] is registered. + * @param configuredSink The configured [[TableSink]] to register. + */ def registerTableSink(name: String, configuredSink: TableSink[_]): Unit = { - checkValidTableName(name) - if (configuredSink.getFieldNames == null || configuredSink.getFieldTypes == null) { - throw TableException("TableSink is not configured.") - } registerTableSinkInternal(name, configuredSink) } private def registerTableSinkInternal(name: String, configuredSink: TableSink[_]): Unit = { + // validate + checkValidTableName(name) + if (configuredSink.getFieldNames == null || configuredSink.getFieldTypes == null) { + throw new TableException("Table sink is not configured.") + } + if (configuredSink.getFieldNames.length == 0) { + throw new TableException("Field names must not be empty.") + } + if (configuredSink.getFieldNames.length != configuredSink.getFieldTypes.length) { + throw new TableException("Same number of field names and types required.") + } + + // register configuredSink match { - case streamTableSink@( - _: AppendStreamTableSink[_] | - _: UpsertStreamTableSink[_] | - _: RetractStreamTableSink[_]) => + // check for proper batch table sink + case _: StreamTableSink[_] => + + // check if a table (source or sink) is registered Option(getTable(name)) match { - case Some(table: TableSourceSinkTable[_, _]) => table.tableSinkTableOpt match { + + // table source and/or sink is registered + case Some(table: TableSourceSinkTable[_, _]) => table.tableSinkTable match { + + // wrapper contains sink case Some(_: TableSinkTable[_]) => - throw new TableException(s"Table \'$name\' already exists. " + - s"Please, choose a different name.") - case _ => replaceRegisteredTable(name, new TableSourceSinkTable( - table.tableSourceTableOpt, - Some(new TableSinkTable(configuredSink)))) + throw new TableException(s"Table '$name' already exists. " + + s"Please choose a different name.") + + // wrapper contains only source (not sink) + case _ => + val enrichedTable = new TableSourceSinkTable( + table.tableSourceTable, + Some(new TableSinkTable(configuredSink))) + replaceRegisteredTable(name, enrichedTable) } - case None => registerTableInternal(name, - new TableSourceSinkTable(None, Some(new TableSinkTable(configuredSink)))) - case _ => throw new TableException(s"Table \'$name\' already exists. " + - s"Please, choose a different name.") + + // no table is registered + case _ => + val newTable = new TableSourceSinkTable( + None, + Some(new TableSinkTable(configuredSink))) + registerTableInternal(name, newTable) } + + // not a stream table sink case _ => throw new TableException( "Only AppendStreamTableSink, UpsertStreamTableSink, and RetractStreamTableSink can be " + diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala index 66a746e06b5af..6a299dde95cc9 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala @@ -56,7 +56,7 @@ import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, Tabl import org.apache.flink.table.plan.cost.DataSetCostFactory import org.apache.flink.table.plan.logical.{CatalogNode, LogicalRelNode} import org.apache.flink.table.plan.rules.FlinkRuleSets -import org.apache.flink.table.plan.schema.{RelTable, RowSchema, TableSinkTable, TableSourceSinkTable} +import org.apache.flink.table.plan.schema.{RelTable, RowSchema, TableSourceSinkTable} import org.apache.flink.table.sinks.TableSink import org.apache.flink.table.sources.TableSource import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo @@ -465,14 +465,14 @@ abstract class TableEnvironment(val config: TableConfig) { tableSink: TableSink[_]): Unit /** - * Registers an external [[TableSink]] which is already configured in this - * [[TableEnvironment]]'s catalog. + * Registers an external [[TableSink]] with already configured field names and field types in + * this [[TableEnvironment]]'s catalog. * Registered sink tables can be referenced in SQL DML statements. * * @param name The name under which the [[TableSink]] is registered. - * @param tableSink The [[TableSink]] to register. + * @param configuredSink The configured [[TableSink]] to register. */ - def registerTableSink(name: String, tableSink: TableSink[_]): Unit + def registerTableSink(name: String, configuredSink: TableSink[_]): Unit /** * Replaces a registered Table with another Table under the same name. @@ -760,8 +760,10 @@ abstract class TableEnvironment(val config: TableConfig) { } getTable(sinkTableName) match { - case s: TableSourceSinkTable[_, _] if s.tableSinkTableOpt != None => - val tableSink = s.tableSinkTableOpt.get.tableSink + + // check for registered table that wraps a sink + case s: TableSourceSinkTable[_, _] if s.tableSinkTable.isDefined => + val tableSink = s.tableSinkTable.get.tableSink // validate schema of source table and table sink val srcFieldTypes = table.getSchema.getTypes val sinkFieldTypes = tableSink.getFieldTypes @@ -781,7 +783,7 @@ abstract class TableEnvironment(val config: TableConfig) { .mkString("[", ", ", "]") throw ValidationException( - s"Field types of query result and registered TableSink $sinkTableName do not match." + + s"Field types of query result and registered TableSink $sinkTableName do not match.\n" + s"Query result schema: $srcSchema\n" + s"TableSink schema: $sinkSchema") } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/BatchTableEnvironment.scala index a3d232c647bdf..f8f35eb5c745a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/BatchTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/BatchTableEnvironment.scala @@ -23,8 +23,6 @@ import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} import org.apache.flink.table.api._ import org.apache.flink.table.expressions.ExpressionParser import org.apache.flink.table.functions.{AggregateFunction, TableFunction} -import org.apache.flink.table.sinks.TableSink -import org.apache.flink.table.sources.TableSource /** * The [[TableEnvironment]] for a Java batch [[DataSet]] diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FileSystem.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FileSystem.scala index 9e38b3835b1d0..f306b5aa72a50 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FileSystem.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FileSystem.scala @@ -18,8 +18,7 @@ package org.apache.flink.table.descriptors -import org.apache.flink.core.fs.FileSystem.WriteMode -import org.apache.flink.table.descriptors.FileSystemValidator._ +import org.apache.flink.table.descriptors.FileSystemValidator.{CONNECTOR_PATH, CONNECTOR_TYPE_VALUE} /** * Connector descriptor for a file system. @@ -28,8 +27,6 @@ class FileSystem extends ConnectorDescriptor( CONNECTOR_TYPE_VALUE, version = 1, formatNeeded = true) { private var path: Option[String] = None - private var numFiles: Option[Int] = None - private var writeMode: Option[String] = None /** * Sets the path to a file or directory in a file system. @@ -41,33 +38,11 @@ class FileSystem extends ConnectorDescriptor( this } - /** - * Set the number of files to write. - * - * @param n number of files - */ - def numFiles(n: Int): FileSystem = { - this.numFiles = Some(n) - this - } - - /** - * Set the write mode. - * - * @param mode write mode. - */ - def writeMode(mode: String): FileSystem = { - this.writeMode = Some(mode) - this - } - /** * Internal method for properties conversion. */ override protected def addConnectorProperties(properties: DescriptorProperties): Unit = { path.foreach(properties.putString(CONNECTOR_PATH, _)) - writeMode.foreach(mode => properties.putString(WRITE_MODE, mode)) - numFiles.foreach(properties.putInt(NUM_FILES, _)) } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FileSystemValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FileSystemValidator.scala index 0a4b80f408820..b065b5f127be9 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FileSystemValidator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FileSystemValidator.scala @@ -18,11 +18,8 @@ package org.apache.flink.table.descriptors -import org.apache.flink.core.fs.FileSystem.WriteMode import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE -import org.apache.flink.table.descriptors.FileSystemValidator._ - -import scala.collection.JavaConversions._ +import org.apache.flink.table.descriptors.FileSystemValidator.{CONNECTOR_PATH, CONNECTOR_TYPE_VALUE} /** * Validator for [[FileSystem]]. @@ -33,8 +30,6 @@ class FileSystemValidator extends ConnectorDescriptorValidator { super.validate(properties) properties.validateValue(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE, isOptional = false) properties.validateString(CONNECTOR_PATH, isOptional = false, minLen = 1) - properties.validateInt(NUM_FILES, isOptional = true, min = 1) - properties.validateEnumValues(WRITE_MODE, true, WriteMode.values().map(_.toString).toList) } } @@ -42,6 +37,5 @@ object FileSystemValidator { val CONNECTOR_TYPE_VALUE = "filesystem" val CONNECTOR_PATH = "connector.path" - val NUM_FILES = "connector.num-files" - val WRITE_MODE = "connector.write-mode" + } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala index 9643337a3b696..ec83b3c62a1ca 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala @@ -102,9 +102,9 @@ object SchemaValidator { val SCHEMA_FROM = "from" /** - * Returns keys for a [[org.apache.flink.table.sources.TableSourceFactory.supportedProperties()]] - * method that are accepted for schema derivation using - * [[deriveFormatFields(DescriptorProperties)]]. + * Returns keys for a + * [[org.apache.flink.table.factories.TableFormatFactory.supportedProperties()]] method that + * are accepted for schema derivation using [[deriveFormatFields(DescriptorProperties)]]. */ def getSchemaDerivationKeys: util.List[String] = { val keys = new util.ArrayList[String]() @@ -174,10 +174,18 @@ object SchemaValidator { attributes.asJava } + /** + * Derives the table schema for a table source. A table source can directly use "name" and + * "type" and needs no special handling for time attributes or aliasing. + */ def deriveTableSourceSchema(properties: DescriptorProperties): TableSchema = { properties.getTableSchema(SCHEMA) } + /** + * Derives the table schema for a table sink. A sink ignores a proctime attribute and + * needs to track the origin of a rowtime field. + */ def deriveTableSinkSchema(properties: DescriptorProperties): TableSchema = { val builder = TableSchema.builder() @@ -198,12 +206,16 @@ object SchemaValidator { // only use the rowtime attribute if it references a field else if (isRowtime) { properties.getString(tsType) match { - case ROWTIME_TIMESTAMPS_TYPE_VALUE_FROM_FIELD => { + case ROWTIME_TIMESTAMPS_TYPE_VALUE_FROM_FIELD => val field = properties.getString(s"$SCHEMA.$i.$ROWTIME_TIMESTAMPS_FROM") builder.field(field, t) - } - case _ => throw new TableException(s"Unsupported rowtime type for sink table schema: " + - s"${properties.getString(tsType)}") + + // other timestamp strategies require a reverse timestamp extractor to + // insert the timestamp into the output + case t@_ => + throw new TableException( + s"Unsupported rowtime type '$t' for sink table schema. Currently " + + s"only '$ROWTIME_TIMESTAMPS_TYPE_VALUE_FROM_FIELD' is supported for table sinks.") } } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala index 119de08ca8224..608332dec97a6 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala @@ -33,7 +33,8 @@ import java.util 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 diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala index f652274a57083..0a3735351f62b 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/PhysicalTableSourceScan.scala @@ -40,12 +40,9 @@ abstract class PhysicalTableSourceScan( override def deriveRowType(): RelDataType = { val flinkTypeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory] val streamingTable = table.unwrap(classOf[TableSourceSinkTable[_, _]]) match { - case t: TableSourceSinkTable[_, _] => t.tableSourceTableOpt match { - case Some(_: StreamTableSourceTable[_]) => true - case Some(_: BatchTableSourceTable[_]) => false - case _ => throw TableException(s"Unknown Table type ${t.getClass}.") - } - case t => throw TableException(s"Unknown Table type ${t.getClass}.") + case t: TableSourceSinkTable[_, _] if t.isStreamSourceTable => true + // null + case _ => false } TableSourceUtil.getRelDataType(tableSource, selectedFields, streamingTable, flinkTypeFactory) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala index 1d3818864a084..e9195f6fbe592 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalTableSourceScan.scala @@ -28,7 +28,7 @@ import org.apache.calcite.rel.{RelNode, RelWriter} import org.apache.flink.table.api.TableException import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.plan.nodes.FlinkConventions -import org.apache.flink.table.plan.schema.{BatchTableSourceTable, StreamTableSourceTable, TableSourceSinkTable, TableSourceTable} +import org.apache.flink.table.plan.schema.TableSourceSinkTable import org.apache.flink.table.sources.{FilterableTableSource, TableSource, TableSourceUtil} import scala.collection.JavaConverters._ @@ -52,12 +52,9 @@ class FlinkLogicalTableSourceScan( override def deriveRowType(): RelDataType = { val flinkTypeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory] val streamingTable = table.unwrap(classOf[TableSourceSinkTable[_, _]]) match { - case t: TableSourceSinkTable[_, _] => t.tableSourceTableOpt match { - case Some(_: StreamTableSourceTable[_]) => true - case Some(_: BatchTableSourceTable[_]) => false - case _ => throw TableException(s"Unknown Table type ${t.getClass}.") - } - case t => throw TableException(s"Unknown Table type ${t.getClass}.") + case t: TableSourceSinkTable[_, _] if t.isStreamSourceTable => true + // null + case _ => false } TableSourceUtil.getRelDataType(tableSource, selectedFields, streamingTable, flinkTypeFactory) @@ -114,9 +111,9 @@ class FlinkLogicalTableSourceScanConverter override def matches(call: RelOptRuleCall): Boolean = { val scan = call.rel[TableScan](0) - val tableSourceSinkTable = scan.getTable.unwrap(classOf[TableSourceSinkTable[_, _]]) - tableSourceSinkTable match { - case t: TableSourceSinkTable[_, _] if t.tableSourceTableOpt != None => true + scan.getTable.unwrap(classOf[TableSourceSinkTable[_, _]]) match { + case t: TableSourceSinkTable[_, _] if t.isSourceTable => true + // null case _ => false } } @@ -125,7 +122,9 @@ class FlinkLogicalTableSourceScanConverter val scan = rel.asInstanceOf[TableScan] val traitSet = rel.getTraitSet.replace(FlinkConventions.LOGICAL) val tableSource = scan.getTable.unwrap(classOf[TableSourceSinkTable[_, _]]) - .tableSourceTableOpt.map(_.tableSource).orNull + .tableSourceTable + .map(_.tableSource) + .getOrElse(throw new TableException("Table source expected.")) new FlinkLogicalTableSourceScan( rel.getCluster, diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/BatchTableSourceScanRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/BatchTableSourceScanRule.scala index a4fd151f690d0..cee6eef769f67 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/BatchTableSourceScanRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/BatchTableSourceScanRule.scala @@ -24,8 +24,8 @@ import org.apache.calcite.rel.convert.ConverterRule import org.apache.calcite.rel.core.TableScan import org.apache.flink.table.plan.nodes.FlinkConventions import org.apache.flink.table.plan.nodes.dataset.BatchTableSourceScan -import org.apache.flink.table.plan.schema.{TableSourceSinkTable, TableSourceTable} import org.apache.flink.table.plan.nodes.logical.FlinkLogicalTableSourceScan +import org.apache.flink.table.plan.schema.TableSourceSinkTable import org.apache.flink.table.sources.BatchTableSource class BatchTableSourceScanRule @@ -38,17 +38,10 @@ class BatchTableSourceScanRule /** Rule must only match if TableScan targets a [[BatchTableSource]] */ override def matches(call: RelOptRuleCall): Boolean = { val scan: TableScan = call.rel(0).asInstanceOf[TableScan] - val dataSetTable = scan.getTable.unwrap(classOf[TableSourceSinkTable[_, _]]) - dataSetTable.tableSourceTableOpt match { - case Some(tt: TableSourceTable[_]) => - tt.tableSource match { - case _: BatchTableSource[_] => - true - case _ => - false - } - case _ => - false + scan.getTable.unwrap(classOf[TableSourceSinkTable[_, _]]) match { + case t: TableSourceSinkTable[_, _] if t.isBatchSourceTable => true + // null + case _ => false } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/StreamTableSourceScanRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/StreamTableSourceScanRule.scala index 62fef2bf25cc9..e99118f7a9b3a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/StreamTableSourceScanRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/StreamTableSourceScanRule.scala @@ -24,8 +24,8 @@ import org.apache.calcite.rel.convert.ConverterRule import org.apache.calcite.rel.core.TableScan import org.apache.flink.table.plan.nodes.FlinkConventions import org.apache.flink.table.plan.nodes.datastream.StreamTableSourceScan -import org.apache.flink.table.plan.schema.{TableSourceSinkTable, TableSourceTable} import org.apache.flink.table.plan.nodes.logical.FlinkLogicalTableSourceScan +import org.apache.flink.table.plan.schema.TableSourceSinkTable import org.apache.flink.table.sources.StreamTableSource class StreamTableSourceScanRule @@ -39,17 +39,10 @@ class StreamTableSourceScanRule /** Rule must only match if TableScan targets a [[StreamTableSource]] */ override def matches(call: RelOptRuleCall): Boolean = { val scan: TableScan = call.rel(0).asInstanceOf[TableScan] - val dataSetTable = scan.getTable.unwrap(classOf[TableSourceSinkTable[_, _]]) - dataSetTable.tableSourceTableOpt match { - case Some(tst: TableSourceTable[_]) => - tst.tableSource match { - case _: StreamTableSource[_] => - true - case _ => - false - } - case _ => - false + scan.getTable.unwrap(classOf[TableSourceSinkTable[_, _]]) match { + case t: TableSourceSinkTable[_, _] if t.isStreamSourceTable => true + // null + case _ => false } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala index 6754db2a07fa3..be9fda2e5f36e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala @@ -24,9 +24,8 @@ import org.apache.calcite.plan.RelOptRule.{none, operand} import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} import org.apache.calcite.rex.RexProgram import org.apache.flink.table.expressions.Expression -import org.apache.flink.table.plan.schema.{TableSourceSinkTable, TableSourceTable} -import org.apache.flink.table.plan.util.RexProgramExtractor import org.apache.flink.table.plan.nodes.logical.{FlinkLogicalCalc, FlinkLogicalTableSourceScan} +import org.apache.flink.table.plan.util.RexProgramExtractor import org.apache.flink.table.sources.FilterableTableSource import org.apache.flink.table.validate.FunctionCatalog import org.apache.flink.util.Preconditions diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceSinkTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceSinkTable.scala index 042293507bca0..923d82ce84e55 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceSinkTable.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceSinkTable.scala @@ -21,23 +21,47 @@ package org.apache.flink.table.plan.schema import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory} import org.apache.calcite.schema.Statistic import org.apache.calcite.schema.impl.AbstractTable +import org.apache.flink.table.api.TableException -class TableSourceSinkTable[T1, T2](val tableSourceTableOpt: Option[TableSourceTable[T1]], - val tableSinkTableOpt: Option[TableSinkTable[T2]]) +/** + * Wrapper for both a [[TableSourceTable]] and [[TableSinkTable]] under a common name. + * + * @param tableSourceTable table source table (if available) + * @param tableSinkTable table sink table (if available) + * @tparam T1 type of the table source table + * @tparam T2 type of the table sink table + */ +class TableSourceSinkTable[T1, T2]( + val tableSourceTable: Option[TableSourceTable[T1]], + val tableSinkTable: Option[TableSinkTable[T2]]) extends AbstractTable { - // In streaming case, the table schema as source and sink can differ because of extra + // In the streaming case, the table schema of source and sink can differ because of extra // rowtime/proctime fields. We will always return the source table schema if tableSourceTable // is not None, otherwise return the sink table schema. We move the Calcite validation logic of // the sink table schema into Flink. This allows us to have different schemas as source and sink // of the same table. override def getRowType(typeFactory: RelDataTypeFactory): RelDataType = { - tableSourceTableOpt.map(_.getRowType(typeFactory)) - .orElse(tableSinkTableOpt.map(_.getRowType(typeFactory))).orNull + tableSourceTable.map(_.getRowType(typeFactory)) + .orElse(tableSinkTable.map(_.getRowType(typeFactory))) + .getOrElse(throw new TableException("Unable to get row type of table source sink table.")) } override def getStatistic: Statistic = { - tableSourceTableOpt.map(_.getStatistic) - .orElse(tableSinkTableOpt.map(_.getStatistic)).orNull + tableSourceTable.map(_.getStatistic) + .orElse(tableSinkTable.map(_.getStatistic)) + .getOrElse(throw new TableException("Unable to get statistics of table source sink table.")) + } + + def isSourceTable: Boolean = tableSourceTable.isDefined + + def isStreamSourceTable: Boolean = tableSourceTable match { + case Some(_: StreamTableSourceTable[_]) => true + case _ => false + } + + def isBatchSourceTable: Boolean = tableSourceTable match { + case Some(_: BatchTableSourceTable[_]) => true + case _ => false } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala index b2ad5b2d3ed5a..f98d03a4d4a16 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala @@ -21,7 +21,6 @@ package org.apache.flink.table.sinks import org.apache.flink.api.common.functions.MapFunction import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.DataSet -import org.apache.flink.api.java.io.CsvInputFormat import org.apache.flink.types.Row import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.core.fs.FileSystem.WriteMode @@ -139,94 +138,3 @@ class CsvFormatter(fieldDelim: String) extends MapFunction[Row, String] { builder.mkString } } - -object CsvTableSink { - - /** - * A builder for creating [[CsvTableSink]] instances. - * - * For example: - * - * {{{ - * val sink: CsvTableSink = new CsvTableSink.builder() - * .path("/path/to/your/file.csv") - * .build() - * }}} - * - */ - class Builder { - - private var path: String = _ - private var fieldDelimOpt: Option[String] = Some(CsvInputFormat.DEFAULT_FIELD_DELIMITER) - private var numFilesOpt: Option[Int] = None - private var writeModeOpt: Option[WriteMode] = None - - /** - * Sets the path to the CSV file. Required. - * - * @param path the path to the CSV file - */ - def path(path: String): Builder = { - this.path = path - this - } - - /** - * Sets the field delimiter, "," by default. - * - * @param delim the field delimiter - */ - def fieldDelimiter(delim: String): Builder = { - this.fieldDelimOpt = Some(delim) - this - } - - /** - * Sets the number of files to write to - * - * @param numFiles the number of files - */ - def numFiles(numFiles: Int): Builder = { - this.numFilesOpt = Some(numFiles) - this - } - - /** - * Sets the write mode - * - * @param mode the write mode - */ - def writeMode(mode: String): Builder = { - this.writeModeOpt = Some(WriteMode.valueOf(mode)) - this - } - - /** - * Apply the current values and constructs a newly-created [[CsvTableSink]]. - * - * @return a newly-created [[CsvTableSink]]. - */ - def build(): CsvTableSink = { - if (path == null) { - throw new IllegalArgumentException("Path must be defined.") - } - new CsvTableSink(path, fieldDelimOpt, numFilesOpt, writeModeOpt) - } - - } - - /** - * Return a new builder that builds a [[CsvTableSink]]. - * - * For example: - * - * {{{ - * val sink: CsvTableSink = CsvTableSink - * .builder() - * .path("/path/to/your/file.csv") - * .build() - * }}} - * @return a new builder to build a [[CsvTableSink]] - */ - def builder(): Builder = new Builder -} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala index 1c37be2761cc5..eb99f028edec5 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSinkFactory.scala @@ -57,8 +57,6 @@ class CsvTableSinkFactory properties.add(s"$FORMAT_FIELDS.#.${DescriptorProperties.NAME}") properties.add(FORMAT_FIELD_DELIMITER) properties.add(CONNECTOR_PATH) - properties.add(NUM_FILES) - properties.add(WRITE_MODE) // schema properties.add(s"$SCHEMA.#.${DescriptorProperties.TYPE}") properties.add(s"$SCHEMA.#.${DescriptorProperties.NAME}") @@ -94,8 +92,6 @@ class CsvTableSinkFactory supportsSourceWatermarks = false).validate(params) // build - val csvTableSinkBuilder = new CsvTableSink.Builder - val formatSchema = params.getTableSchema(FORMAT_FIELDS) val tableSchema = SchemaValidator.deriveTableSinkSchema(params) @@ -104,17 +100,12 @@ class CsvTableSinkFactory "Encodings that differ from the schema are not supported yet for CsvTableSink.") } - toScala(params.getOptionalString(CONNECTOR_PATH)) - .foreach(csvTableSinkBuilder.path) - toScala(params.getOptionalInt(NUM_FILES)) - .foreach(n => csvTableSinkBuilder.numFiles(n)) - toScala(params.getOptionalString(WRITE_MODE)) - .foreach(csvTableSinkBuilder.writeMode) - toScala(params.getOptionalString(FORMAT_FIELD_DELIMITER)) - .foreach(csvTableSinkBuilder.fieldDelimiter) - - csvTableSinkBuilder - .build() + val path = params.getString(CONNECTOR_PATH) + val fieldDelimiter = toScala(params.getOptionalString(FORMAT_FIELD_DELIMITER)).getOrElse(",") + + val csvTableSink = new CsvTableSink(path, fieldDelimiter) + + csvTableSink .configure(formatSchema.getColumnNames, formatSchema.getTypes) .asInstanceOf[CsvTableSink] } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/catalog/ExternalCatalogSchemaTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/catalog/ExternalCatalogSchemaTest.scala index 434c0cdf92521..58f51f91c446c 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/catalog/ExternalCatalogSchemaTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/catalog/ExternalCatalogSchemaTest.scala @@ -79,7 +79,7 @@ class ExternalCatalogSchemaTest extends TableTestBase { val relOptTable = calciteCatalogReader.getTable(Lists.newArrayList(schemaName, db, tb)) assertNotNull(relOptTable) val tableSourceSinkTable = relOptTable.unwrap(classOf[TableSourceSinkTable[_, _]]) - tableSourceSinkTable.tableSourceTableOpt match { + tableSourceSinkTable.tableSourceTable match { case Some(tst: TableSourceTable[_]) => assertTrue(tst.tableSource.isInstanceOf[CsvTableSource]) case _ => diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/FileSystemTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/FileSystemTest.scala index 02d37de26eb2e..1162694a01dc7 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/FileSystemTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/FileSystemTest.scala @@ -32,17 +32,6 @@ class FileSystemTest extends DescriptorTestBase { addPropertyAndVerify(descriptors().get(0), "connector.path", "") } - @Test(expected = classOf[ValidationException]) - def testInvalidWriteMode(): Unit = { - addPropertyAndVerify(descriptors().get(0), "connector.write-mode", "") - } - - - @Test(expected = classOf[ValidationException]) - def testInvalidNumFiles(): Unit = { - addPropertyAndVerify(descriptors().get(0), "connector.num-files", "0") - } - @Test(expected = classOf[ValidationException]) def testMissingPath(): Unit = { removePropertyAndVerify(descriptors().get(0), "connector.path") @@ -51,7 +40,7 @@ class FileSystemTest extends DescriptorTestBase { // ---------------------------------------------------------------------------------------------- override def descriptors(): util.List[Descriptor] = { - util.Arrays.asList(FileSystem().path("/myfile").writeMode("OVERWRITE").numFiles(1)) + util.Arrays.asList(FileSystem().path("/myfile")) } override def validator(): DescriptorValidator = { @@ -62,8 +51,6 @@ class FileSystemTest extends DescriptorTestBase { val desc = Map( "connector.type" -> "filesystem", "connector.property-version" -> "1", - "connector.num-files" -> "1", - "connector.write-mode" -> "OVERWRITE", "connector.path" -> "/myfile") util.Arrays.asList(desc.asJava) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaValidatorTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaValidatorTest.scala index f40c19ca4f7d1..c55805705868a 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaValidatorTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaValidatorTest.scala @@ -85,7 +85,7 @@ class SchemaValidatorTest { .field("abcField", Types.STRING) .field("p", Types.SQL_TIMESTAMP).proctime() .field("r", Types.SQL_TIMESTAMP).rowtime( - Rowtime().timestampsFromSource().watermarksFromSource()) + Rowtime().timestampsFromSource().watermarksFromSource()) val props = new DescriptorProperties() desc1.addProperties(props) @@ -99,7 +99,7 @@ class SchemaValidatorTest { .field("abcField", Types.STRING) .field("p", Types.SQL_TIMESTAMP).proctime() .field("r", Types.SQL_TIMESTAMP).rowtime( - Rowtime().timestampsFromField("myTime").watermarksFromSource()) + Rowtime().timestampsFromField("myTime").watermarksFromSource()) val props = new DescriptorProperties() desc1.addProperties(props) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/TableSourceDescriptorTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/TableSourceDescriptorTest.scala index 91f3aac0d654c..a7dd644f3aa25 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/TableSourceDescriptorTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/TableSourceDescriptorTest.scala @@ -25,6 +25,9 @@ import org.junit.Test import scala.collection.JavaConverters._ +/** + * Tests for [[TableSourceDescriptor]]. + */ class TableSourceDescriptorTest extends TableTestBase { @Test diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSinkFactoryServiceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSinkFactoryServiceTest.scala index 90762ce8097e2..ebc9a8a630ec0 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSinkFactoryServiceTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/TableSinkFactoryServiceTest.scala @@ -70,7 +70,7 @@ class TableSinkFactoryServiceTest { 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") + properties.put(FORMAT_PATH, "/path/to/target") properties.put("schema.0.name", "a") properties.put("schema.1.name", "b") properties.put("schema.2.name", "c") diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableSinkFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableSinkFactory.scala index 0a40530a68668..ed4fec5130558 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableSinkFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/factories/utils/TestTableSinkFactory.scala @@ -44,7 +44,7 @@ class TestTableSinkFactory extends StreamTableSinkFactory[Row] with TableFactory override def supportedProperties(): util.List[String] = { val properties = new util.ArrayList[String]() // connector - properties.add("format.path") + properties.add(FORMAT_PATH) properties.add("schema.#.name") properties.add("schema.#.field.#.name") properties @@ -60,5 +60,6 @@ class TestTableSinkFactory extends StreamTableSinkFactory[Row] with TableFactory object TestTableSinkFactory { val CONNECTOR_TYPE_VALUE_TEST = "test" val FORMAT_TYPE_VALUE_TEST = "test" + val FORMAT_PATH = "format.path" } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/TableEnvironmentITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/TableEnvironmentITCase.scala index 978e65c1dff4e..07839e6970b55 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/TableEnvironmentITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/sql/TableEnvironmentITCase.scala @@ -120,7 +120,7 @@ class TableEnvironmentITCase( def testInsertIntoMemoryTable(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env) - MemoryTableSourceSinkUtil.clear + MemoryTableSourceSinkUtil.clear() val t = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c) tEnv.registerTable("sourceTable", t) @@ -135,7 +135,7 @@ class TableEnvironmentITCase( env.execute() val expected = List("1,1,Hi", "2,2,Hello", "3,2,Hello world") - assertEquals(expected.sorted, MemoryTableSourceSinkUtil.tableData.map(_.toString).sorted) + assertEquals(expected.sorted, MemoryTableSourceSinkUtil.tableDataStrings.sorted) } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/TableEnvironmentITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/TableEnvironmentITCase.scala index a833e004835c2..67fde382c29bd 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/TableEnvironmentITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/TableEnvironmentITCase.scala @@ -186,7 +186,7 @@ class TableEnvironmentITCase( def testInsertIntoMemoryTable(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env) - MemoryTableSourceSinkUtil.clear + MemoryTableSourceSinkUtil.clear() val t = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c) tEnv.registerTable("sourceTable", t) @@ -202,7 +202,7 @@ class TableEnvironmentITCase( env.execute() val expected = List("1,1,Hi", "2,2,Hello", "3,2,Hello world") - assertEquals(expected.sorted, MemoryTableSourceSinkUtil.tableData.map(_.toString).sorted) + assertEquals(expected.sorted, MemoryTableSourceSinkUtil.tableDataStrings.sorted) } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala index 2554d143ac9aa..85a96475a9a93 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/TimeAttributesITCase.scala @@ -183,7 +183,7 @@ class TimeAttributesITCase extends AbstractTestBase { val env = StreamExecutionEnvironment.getExecutionEnvironment env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) val tEnv = TableEnvironment.getTableEnvironment(env) - MemoryTableSourceSinkUtil.clear + MemoryTableSourceSinkUtil.clear() val stream = env .fromCollection(data) @@ -199,7 +199,7 @@ class TimeAttributesITCase extends AbstractTestBase { "1970-01-01 00:00:00.007,1970-01-01 00:00:00.0,1970-01-02 00:00:00.0", "1970-01-01 00:00:00.008,1970-01-01 00:00:00.0,1970-01-02 00:00:00.0", "1970-01-01 00:00:00.016,1970-01-01 00:00:00.0,1970-01-02 00:00:00.0") - assertEquals(expected.sorted, MemoryTableSourceSinkUtil.tableData.map(_.toString).sorted) + assertEquals(expected.sorted, MemoryTableSourceSinkUtil.tableDataStrings.sorted) } @Test diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala index 8ef99a83e80b1..15b8699aeed86 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala @@ -692,7 +692,7 @@ class SqlITCase extends StreamingWithStateTestBase { val env = StreamExecutionEnvironment.getExecutionEnvironment env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) val tEnv = TableEnvironment.getTableEnvironment(env) - MemoryTableSourceSinkUtil.clear + MemoryTableSourceSinkUtil.clear() val t = StreamTestData.getSmall3TupleDataStream(env) .assignAscendingTimestamps(x => x._2) @@ -713,7 +713,7 @@ class SqlITCase extends StreamingWithStateTestBase { "1,1,Hi,1970-01-01 00:00:00.001", "2,2,Hello,1970-01-01 00:00:00.002", "3,2,Hello world,1970-01-01 00:00:00.002") - assertEquals(expected.sorted, MemoryTableSourceSinkUtil.tableData.map(_.toString).sorted) + assertEquals(expected.sorted, MemoryTableSourceSinkUtil.tableDataStrings.sorted) } @Test @@ -721,7 +721,7 @@ class SqlITCase extends StreamingWithStateTestBase { val env = StreamExecutionEnvironment.getExecutionEnvironment env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime) val tEnv = TableEnvironment.getTableEnvironment(env) - MemoryTableSourceSinkUtil.clear + MemoryTableSourceSinkUtil.clear() val desc = Schema() .field("a", Types.INT) @@ -739,9 +739,9 @@ class SqlITCase extends StreamingWithStateTestBase { tEnv.registerTable("sourceTable", t) tEnv.registerTableSource("targetTable", - new InMemoryTableFactory().createStreamTableSource(props.asMap)) + new InMemoryTableFactory(3).createStreamTableSource(props.asMap)) tEnv.registerTableSink("targetTable", - new InMemoryTableFactory().createStreamTableSink(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") diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala index 36b3c16f50ecc..02662282822e2 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/InMemoryTableFactory.scala @@ -30,7 +30,15 @@ import org.apache.flink.table.sinks.StreamTableSink import org.apache.flink.table.sources.StreamTableSource import org.apache.flink.types.Row -class InMemoryTableFactory +/** + * Factory for creating stream table sources and sinks. + * + * See [[MemoryTableSourceSinkUtil.UnsafeMemoryTableSource]] and + * [[MemoryTableSourceSinkUtil.UnsafeMemoryAppendTableSink]]. + * + * @param terminationCount determines when to shutdown the streaming source function + */ +class InMemoryTableFactory(terminationCount: Int) extends TableFactory with StreamTableSourceFactory[Row] with StreamTableSinkFactory[Row] { @@ -82,7 +90,7 @@ class InMemoryTableFactory new RowTypeInfo(types, names), rowtimeDescriptors, proctimeAttributeOpt.get(), - 3) + terminationCount) } override def requiredContext(): util.Map[String, String] = { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala index 511fedf868de7..cb0ad436a1839 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/MemoryTableSourceSinkUtil.scala @@ -18,14 +18,12 @@ package org.apache.flink.table.utils -import java.sql.Timestamp import java.util -import java.util.Collections import org.apache.flink.api.common.io.RichOutputFormat import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.datastream.DataStream import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment @@ -34,19 +32,23 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext import org.apache.flink.table.api.TableSchema import org.apache.flink.table.sinks.{AppendStreamTableSink, BatchTableSink, TableSinkBase} -import org.apache.flink.table.sources.tsextractors.StreamRecordTimestamp -import org.apache.flink.table.sources.wmstrategies.AscendingTimestamps import org.apache.flink.table.sources._ import org.apache.flink.table.util.TableConnectorUtil import org.apache.flink.types.Row -import scala.collection.mutable import scala.collection.JavaConverters._ +import scala.collection.mutable +/** + * Utilities to ingest and retrieve results into and from a table program. + */ object MemoryTableSourceSinkUtil { - var tableData: mutable.ListBuffer[Row] = mutable.ListBuffer[Row]() - def clear = { + val tableData: mutable.ListBuffer[Row] = mutable.ListBuffer[Row]() + + def tableDataStrings: Seq[String] = tableData.map(_.toString) + + def clear(): Unit = { MemoryTableSourceSinkUtil.tableData.clear() } @@ -55,9 +57,11 @@ object MemoryTableSourceSinkUtil { returnType: TypeInformation[Row], rowtimeAttributeDescriptor: util.List[RowtimeAttributeDescriptor], proctime: String, - val rowCount: Integer) + val terminationCount: Int) extends BatchTableSource[Row] - with StreamTableSource[Row] with DefinedProctimeAttribute with DefinedRowtimeAttributes { + with StreamTableSource[Row] + with DefinedProctimeAttribute + with DefinedRowtimeAttributes { override def getReturnType: TypeInformation[Row] = returnType @@ -67,13 +71,15 @@ object MemoryTableSourceSinkUtil { execEnv.fromCollection(tableData.asJava, returnType) } - final class InMemorySourceFunction(var count: Int = rowCount) extends SourceFunction[Row] { - override def cancel(): Unit = ??? + final class InMemorySourceFunction(var count: Int = terminationCount) + extends SourceFunction[Row] { + + override def cancel(): Unit = throw new UnsupportedOperationException() override def run(ctx: SourceContext[Row]): Unit = { while (count > 0) { tableData.synchronized { - if (tableData.size > 0) { + if (tableData.nonEmpty) { val r = tableData.remove(0) ctx.collect(r) count -= 1 From 256bad5f91e2fdf6656fc1222460cc45a1b1d19d Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Sat, 14 Jul 2018 06:59:00 +0200 Subject: [PATCH 09/17] Replace nulls with Optional --- .../connectors/kafka/Kafka010TableSource.java | 10 ++--- .../kafka/Kafka010TableSourceFactory.java | 5 ++- .../kafka/Kafka010TableSourceFactoryTest.java | 5 ++- .../connectors/kafka/Kafka011TableSource.java | 10 ++--- .../kafka/Kafka011TableSourceFactory.java | 5 ++- .../kafka/Kafka011TableSourceFactoryTest.java | 5 ++- .../connectors/kafka/Kafka08TableSource.java | 10 ++--- .../kafka/Kafka08TableSourceFactory.java | 5 ++- .../kafka/Kafka08TableSourceFactoryTest.java | 5 ++- .../connectors/kafka/Kafka09TableSource.java | 10 ++--- .../kafka/Kafka09TableSourceFactory.java | 5 ++- .../kafka/Kafka09TableSourceFactoryTest.java | 5 ++- .../connectors/kafka/KafkaTableSource.java | 44 +++++++++---------- .../kafka/KafkaTableSourceFactory.java | 9 ++-- .../KafkaTableSourceFactoryTestBase.java | 6 ++- 15 files changed, 74 insertions(+), 65 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java index e4b9881842bde..f6574623b2a17 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java @@ -29,6 +29,7 @@ import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; /** @@ -41,11 +42,10 @@ public class Kafka010TableSource extends KafkaTableSource { * Creates a Kafka 0.10 {@link StreamTableSource}. * * @param schema Schema of the produced table. - * @param proctimeAttribute Field name of the processing time attribute, null if no - * processing time field is defined. + * @param proctimeAttribute Field name of the processing time attribute. * @param rowtimeAttributeDescriptors Descriptor for a rowtime attribute * @param fieldMapping Mapping for the fields of the table schema to - * fields of the physical returned type or null. + * fields of the physical returned type. * @param topic Kafka topic to consume. * @param properties Properties for the Kafka consumer. * @param deserializationSchema Deserialization schema for decoding records from Kafka. @@ -55,9 +55,9 @@ public class Kafka010TableSource extends KafkaTableSource { */ public Kafka010TableSource( TableSchema schema, - String proctimeAttribute, + Optional proctimeAttribute, List rowtimeAttributeDescriptors, - Map fieldMapping, + Optional> fieldMapping, String topic, Properties properties, DeserializationSchema deserializationSchema, diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactory.java index 04f17704b0320..4a860162222a7 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactory.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactory.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; /** @@ -48,7 +49,7 @@ protected boolean supportsKafkaTimestamps() { @Override protected KafkaTableSource createKafkaTableSource( TableSchema schema, - String proctimeAttribute, + Optional proctimeAttribute, List rowtimeAttributeDescriptors, Map fieldMapping, String topic, @@ -61,7 +62,7 @@ protected KafkaTableSource createKafkaTableSource( schema, proctimeAttribute, rowtimeAttributeDescriptors, - fieldMapping, + Optional.of(fieldMapping), topic, properties, deserializationSchema, diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactoryTest.java index 8f7b121641802..ff3b0b0001f98 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactoryTest.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactoryTest.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; /** @@ -49,7 +50,7 @@ protected Class> getExpectedFlinkKafkaConsumer() { @Override protected KafkaTableSource getExpectedKafkaTableSource( TableSchema schema, - String proctimeAttribute, + Optional proctimeAttribute, List rowtimeAttributeDescriptors, Map fieldMapping, String topic, @@ -62,7 +63,7 @@ protected KafkaTableSource getExpectedKafkaTableSource( schema, proctimeAttribute, rowtimeAttributeDescriptors, - fieldMapping, + Optional.of(fieldMapping), topic, properties, deserializationSchema, diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java index 195663d1a78dd..85f566925bb60 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSource.java @@ -29,6 +29,7 @@ import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; /** @@ -41,11 +42,10 @@ public class Kafka011TableSource extends KafkaTableSource { * Creates a Kafka 0.11 {@link StreamTableSource}. * * @param schema Schema of the produced table. - * @param proctimeAttribute Field name of the processing time attribute, null if no - * processing time field is defined. + * @param proctimeAttribute Field name of the processing time attribute. * @param rowtimeAttributeDescriptors Descriptor for a rowtime attribute * @param fieldMapping Mapping for the fields of the table schema to - * fields of the physical returned type or null. + * fields of the physical returned type. * @param topic Kafka topic to consume. * @param properties Properties for the Kafka consumer. * @param deserializationSchema Deserialization schema for decoding records from Kafka. @@ -55,9 +55,9 @@ public class Kafka011TableSource extends KafkaTableSource { */ public Kafka011TableSource( TableSchema schema, - String proctimeAttribute, + Optional proctimeAttribute, List rowtimeAttributeDescriptors, - Map fieldMapping, + Optional> fieldMapping, String topic, Properties properties, DeserializationSchema deserializationSchema, StartupMode startupMode, diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactory.java index 0d00c73e39eb0..b1e392921a3f9 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactory.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactory.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; /** @@ -48,7 +49,7 @@ protected boolean supportsKafkaTimestamps() { @Override protected KafkaTableSource createKafkaTableSource( TableSchema schema, - String proctimeAttribute, + Optional proctimeAttribute, List rowtimeAttributeDescriptors, Map fieldMapping, String topic, @@ -61,7 +62,7 @@ protected KafkaTableSource createKafkaTableSource( schema, proctimeAttribute, rowtimeAttributeDescriptors, - fieldMapping, + Optional.of(fieldMapping), topic, properties, deserializationSchema, diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactoryTest.java index 171644a014201..abaa49081f069 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactoryTest.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactoryTest.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; /** @@ -49,7 +50,7 @@ protected Class> getExpectedFlinkKafkaConsumer() { @Override protected KafkaTableSource getExpectedKafkaTableSource( TableSchema schema, - String proctimeAttribute, + Optional proctimeAttribute, List rowtimeAttributeDescriptors, Map fieldMapping, String topic, @@ -62,7 +63,7 @@ protected KafkaTableSource getExpectedKafkaTableSource( schema, proctimeAttribute, rowtimeAttributeDescriptors, - fieldMapping, + Optional.of(fieldMapping), topic, properties, deserializationSchema, diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java index 4ead94caaee08..1a025b8a24457 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java @@ -29,6 +29,7 @@ import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; /** @@ -41,11 +42,10 @@ public class Kafka08TableSource extends KafkaTableSource { * Creates a Kafka 0.8 {@link StreamTableSource}. * * @param schema Schema of the produced table. - * @param proctimeAttribute Field name of the processing time attribute, null if no - * processing time field is defined. + * @param proctimeAttribute Field name of the processing time attribute. * @param rowtimeAttributeDescriptors Descriptor for a rowtime attribute * @param fieldMapping Mapping for the fields of the table schema to - * fields of the physical returned type or null. + * fields of the physical returned type. * @param topic Kafka topic to consume. * @param properties Properties for the Kafka consumer. * @param deserializationSchema Deserialization schema for decoding records from Kafka. @@ -55,9 +55,9 @@ public class Kafka08TableSource extends KafkaTableSource { */ public Kafka08TableSource( TableSchema schema, - String proctimeAttribute, + Optional proctimeAttribute, List rowtimeAttributeDescriptors, - Map fieldMapping, + Optional> fieldMapping, String topic, Properties properties, DeserializationSchema deserializationSchema, StartupMode startupMode, diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactory.java index 5cacd3eea5d57..cd33751561463 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactory.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactory.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; /** @@ -48,7 +49,7 @@ protected boolean supportsKafkaTimestamps() { @Override protected KafkaTableSource createKafkaTableSource( TableSchema schema, - String proctimeAttribute, + Optional proctimeAttribute, List rowtimeAttributeDescriptors, Map fieldMapping, String topic, @@ -61,7 +62,7 @@ protected KafkaTableSource createKafkaTableSource( schema, proctimeAttribute, rowtimeAttributeDescriptors, - fieldMapping, + Optional.of(fieldMapping), topic, properties, deserializationSchema, diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactoryTest.java index 0cf0f95f4607f..d939d885242fa 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactoryTest.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactoryTest.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; /** @@ -49,7 +50,7 @@ protected Class> getExpectedFlinkKafkaConsumer() { @Override protected KafkaTableSource getExpectedKafkaTableSource( TableSchema schema, - String proctimeAttribute, + Optional proctimeAttribute, List rowtimeAttributeDescriptors, Map fieldMapping, String topic, @@ -62,7 +63,7 @@ protected KafkaTableSource getExpectedKafkaTableSource( schema, proctimeAttribute, rowtimeAttributeDescriptors, - fieldMapping, + Optional.of(fieldMapping), topic, properties, deserializationSchema, diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java index b770056c1aa11..18bc1c4351363 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java @@ -29,6 +29,7 @@ import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; /** @@ -41,11 +42,10 @@ public class Kafka09TableSource extends KafkaTableSource { * Creates a Kafka 0.9 {@link StreamTableSource}. * * @param schema Schema of the produced table. - * @param proctimeAttribute Field name of the processing time attribute, null if no - * processing time field is defined. + * @param proctimeAttribute Field name of the processing time attribute. * @param rowtimeAttributeDescriptors Descriptor for a rowtime attribute * @param fieldMapping Mapping for the fields of the table schema to - * fields of the physical returned type or null. + * fields of the physical returned type. * @param topic Kafka topic to consume. * @param properties Properties for the Kafka consumer. * @param deserializationSchema Deserialization schema for decoding records from Kafka. @@ -55,9 +55,9 @@ public class Kafka09TableSource extends KafkaTableSource { */ public Kafka09TableSource( TableSchema schema, - String proctimeAttribute, + Optional proctimeAttribute, List rowtimeAttributeDescriptors, - Map fieldMapping, + Optional> fieldMapping, String topic, Properties properties, DeserializationSchema deserializationSchema, StartupMode startupMode, diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactory.java index 98b7fb5427d83..14c52fd0f8300 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactory.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactory.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; /** @@ -48,7 +49,7 @@ protected boolean supportsKafkaTimestamps() { @Override protected KafkaTableSource createKafkaTableSource( TableSchema schema, - String proctimeAttribute, + Optional proctimeAttribute, List rowtimeAttributeDescriptors, Map fieldMapping, String topic, @@ -61,7 +62,7 @@ protected KafkaTableSource createKafkaTableSource( schema, proctimeAttribute, rowtimeAttributeDescriptors, - fieldMapping, + Optional.of(fieldMapping), topic, properties, deserializationSchema, diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactoryTest.java index 85a96dfe94dd7..b976e14e8cdce 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactoryTest.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactoryTest.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; /** @@ -49,7 +50,7 @@ protected Class> getExpectedFlinkKafkaConsumer() { @Override protected KafkaTableSource getExpectedKafkaTableSource( TableSchema schema, - String proctimeAttribute, + Optional proctimeAttribute, List rowtimeAttributeDescriptors, Map fieldMapping, String topic, @@ -62,7 +63,7 @@ protected KafkaTableSource getExpectedKafkaTableSource( schema, proctimeAttribute, rowtimeAttributeDescriptors, - fieldMapping, + Optional.of(fieldMapping), topic, properties, deserializationSchema, diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java index 24cfd9436dbdd..cf801a7f4b307 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java @@ -45,6 +45,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.Properties; import scala.Option; @@ -69,13 +70,13 @@ public abstract class KafkaTableSource implements private final TableSchema schema; /** Field name of the processing time attribute, null if no processing time field is defined. */ - private String proctimeAttribute; + private Optional proctimeAttribute; /** Descriptor for a rowtime attribute. */ private List rowtimeAttributeDescriptors; - /** Mapping for the fields of the table schema to fields of the physical returned type or null. */ - private Map fieldMapping; + /** Mapping for the fields of the table schema to fields of the physical returned type. */ + private Optional> fieldMapping; // Kafka-specific attributes @@ -98,11 +99,10 @@ public abstract class KafkaTableSource implements * Creates a generic Kafka {@link StreamTableSource}. * * @param schema Schema of the produced table. - * @param proctimeAttribute Field name of the processing time attribute, null if no - * processing time field is defined. + * @param proctimeAttribute Field name of the processing time attribute. * @param rowtimeAttributeDescriptors Descriptor for a rowtime attribute * @param fieldMapping Mapping for the fields of the table schema to - * fields of the physical returned type or null. + * fields of the physical returned type. * @param topic Kafka topic to consume. * @param properties Properties for the Kafka consumer. * @param deserializationSchema Deserialization schema for decoding records from Kafka. @@ -112,9 +112,9 @@ public abstract class KafkaTableSource implements */ protected KafkaTableSource( TableSchema schema, - String proctimeAttribute, + Optional proctimeAttribute, List rowtimeAttributeDescriptors, - Map fieldMapping, + Optional> fieldMapping, String topic, Properties properties, DeserializationSchema deserializationSchema, @@ -148,9 +148,9 @@ protected KafkaTableSource( DeserializationSchema deserializationSchema) { this( schema, - null, + Optional.empty(), Collections.emptyList(), - null, + Optional.empty(), topic, properties, deserializationSchema, StartupMode.GROUP_OFFSETS, @@ -182,7 +182,7 @@ public TableSchema getTableSchema() { @Override public String getProctimeAttribute() { - return proctimeAttribute; + return proctimeAttribute.orElse(null); } @Override @@ -192,7 +192,7 @@ public List getRowtimeAttributeDescriptors() { @Override public Map getFieldMapping() { - return fieldMapping; + return fieldMapping.orElse(null); } @Override @@ -284,17 +284,17 @@ protected FlinkKafkaConsumerBase getKafkaConsumer( * * @param proctimeAttribute The name of the field that becomes the processing time field. */ - private String validateProctimeAttribute(String proctimeAttribute) { - if (proctimeAttribute != null) { + private Optional validateProctimeAttribute(Optional proctimeAttribute) { + return proctimeAttribute.map((attribute) -> { // validate that field exists and is of correct type - Option> tpe = schema.getType(proctimeAttribute); + Option> tpe = schema.getType(attribute); if (tpe.isEmpty()) { - throw new ValidationException("Processing time attribute '" + proctimeAttribute + "' is not present in TableSchema."); + throw new ValidationException("Processing time attribute '" + attribute + "' is not present in TableSchema."); } else if (tpe.get() != Types.SQL_TIMESTAMP()) { - throw new ValidationException("Processing time attribute '" + proctimeAttribute + "' is not of type SQL_TIMESTAMP."); + throw new ValidationException("Processing time attribute '" + attribute + "' is not of type SQL_TIMESTAMP."); } - } - return proctimeAttribute; + return attribute; + }); } /** @@ -327,7 +327,7 @@ private List validateRowtimeAttributeDescriptors(Lis */ @Deprecated protected void setProctimeAttribute(String proctimeAttribute) { - this.proctimeAttribute = validateProctimeAttribute(proctimeAttribute); + this.proctimeAttribute = validateProctimeAttribute(Optional.ofNullable(proctimeAttribute)); } /** @@ -364,14 +364,14 @@ protected void setSpecificStartupOffsets(Map specific } /** - * Mapping for the fields of the table schema to fields of the physical returned type or null. + * Mapping for the fields of the table schema to fields of the physical returned type. * * @param fieldMapping The mapping from table schema fields to format schema fields. * @deprecated Use table descriptors instead of implementation-specific builders. */ @Deprecated protected void setFieldMapping(Map fieldMapping) { - this.fieldMapping = fieldMapping; + this.fieldMapping = Optional.ofNullable(fieldMapping); } //////// ABSTRACT METHODS FOR SUBCLASSES diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java index 2c65976c62e3c..380d657e0f8ca 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactory.java @@ -140,7 +140,7 @@ public StreamTableSource createStreamTableSource(Map proper final TableSchema schema = params.getTableSchema(SCHEMA()); // proctime - final String proctimeAttribute = SchemaValidator.deriveProctimeAttribute(params).orElse(null); + final Optional proctimeAttribute = SchemaValidator.deriveProctimeAttribute(params); // rowtime final List rowtimeAttributes = SchemaValidator.deriveRowtimeAttributes(params); @@ -224,11 +224,10 @@ public StreamTableSource createStreamTableSource(Map proper * Constructs the version-specific Kafka table source. * * @param schema Schema of the produced table. - * @param proctimeAttribute Field name of the processing time attribute, null if no - * processing time field is defined. + * @param proctimeAttribute Field name of the processing time attribute. * @param rowtimeAttributeDescriptors Descriptor for a rowtime attribute * @param fieldMapping Mapping for the fields of the table schema to - * fields of the physical returned type or null. + * fields of the physical returned type. * @param topic Kafka topic to consume. * @param properties Properties for the Kafka consumer. * @param deserializationSchema Deserialization schema for decoding records from Kafka. @@ -238,7 +237,7 @@ public StreamTableSource createStreamTableSource(Map proper */ protected abstract KafkaTableSource createKafkaTableSource( TableSchema schema, - String proctimeAttribute, + Optional proctimeAttribute, List rowtimeAttributeDescriptors, Map fieldMapping, String topic, Properties properties, diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java index 8cd058f5bcfbc..1e8266d9025dc 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceFactoryTestBase.java @@ -49,6 +49,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; import static org.junit.Assert.assertEquals; @@ -112,7 +113,8 @@ public void testTableSource() { final StartupMode startupMode = StartupMode.SPECIFIC_OFFSETS; final KafkaTableSource expected = getExpectedKafkaTableSource( - schema, PROC_TIME, + schema, + Optional.of(PROC_TIME), rowtimeAttributeDescriptors, fieldMapping, TOPIC, @@ -183,7 +185,7 @@ public JobExecutionResult execute(String jobName) { protected abstract KafkaTableSource getExpectedKafkaTableSource( TableSchema schema, - String proctimeAttribute, + Optional proctimeAttribute, List rowtimeAttributeDescriptors, Map fieldMapping, String topic, Properties properties, From 99a3f6985def1bdfa1e8b6e947b3aa64ee675ae5 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Sat, 14 Jul 2018 07:00:57 +0200 Subject: [PATCH 10/17] Add nullabel annotation --- .../org/apache/flink/table/sources/DefinedFieldMapping.scala | 2 ++ .../org/apache/flink/table/sources/definedTimeAttributes.scala | 2 ++ 2 files changed, 4 insertions(+) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/DefinedFieldMapping.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/DefinedFieldMapping.scala index 1c43d68c1a95d..7fbcd37aedcb3 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/DefinedFieldMapping.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/DefinedFieldMapping.scala @@ -19,6 +19,7 @@ package org.apache.flink.table.sources import java.util.{Map => JMap} +import javax.annotation.Nullable import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.api.TableSchema @@ -53,5 +54,6 @@ trait DefinedFieldMapping { * @return A mapping from [[TableSchema]] fields to [[TypeInformation]] fields or * null if no mapping is necessary. */ + @Nullable def getFieldMapping: JMap[String, String] } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/definedTimeAttributes.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/definedTimeAttributes.scala index 73b76a58584eb..b144312caa5bd 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/definedTimeAttributes.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/definedTimeAttributes.scala @@ -20,6 +20,7 @@ package org.apache.flink.table.sources import java.util import java.util.Objects +import javax.annotation.Nullable import org.apache.flink.table.api.TableSchema import org.apache.flink.table.api.Types @@ -38,6 +39,7 @@ trait DefinedProctimeAttribute { * The referenced attribute must be present in the [[TableSchema]] of the [[TableSource]] and of * type [[Types.SQL_TIMESTAMP]]. */ + @Nullable def getProctimeAttribute: String } From 4d8aba0624dfa2a4b2757fa84adab142cafa2740 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Sat, 14 Jul 2018 07:02:51 +0200 Subject: [PATCH 11/17] One entry per line --- .../streaming/connectors/kafka/KafkaTableSource.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java index cf801a7f4b307..78b373bc2c486 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java @@ -242,8 +242,16 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(schema, proctimeAttribute, rowtimeAttributeDescriptors, fieldMapping, - topic, properties, deserializationSchema, startupMode, specificStartupOffsets); + return Objects.hash( + schema, + proctimeAttribute, + rowtimeAttributeDescriptors, + fieldMapping, + topic, + properties, + deserializationSchema, + startupMode, + specificStartupOffsets); } /** From 312eb1bb8bd741faab056d6580c16877b7ed3e44 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Sat, 14 Jul 2018 07:06:45 +0200 Subject: [PATCH 12/17] Improve comment --- .../scala/org/apache/flink/table/factories/TableFactory.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala index 608332dec97a6..ed16e8a48acdf 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala @@ -23,8 +23,8 @@ import java.util /** * 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 format. The factory allows - * for matching to the given set of properties. + * 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 From 834d55337f5bac179ec077af4c5597dafcac3ee5 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Sat, 14 Jul 2018 07:13:32 +0200 Subject: [PATCH 13/17] Improve variable naming --- .../flink/table/client/config/Environment.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java index edf43253612b1..9d4a7ceffaf4f 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java @@ -217,17 +217,17 @@ private static TableDescriptor createTableDescriptor(String name, Map properties = new HashMap<>(config); - properties.remove(TABLE_TYPE); + final Map configCopy = new HashMap<>(config); + configCopy.remove(TABLE_TYPE); - final Map normalizedProperties = ConfigUtil.normalizeYaml(properties); + final Map normalizedConfig = ConfigUtil.normalizeYaml(configCopy); switch (type) { case TABLE_TYPE_VALUE_SOURCE: - return new Source(name, normalizedProperties); + return new Source(name, normalizedConfig); case TABLE_TYPE_VALUE_SINK: - return new Sink(name, normalizedProperties); + return new Sink(name, normalizedConfig); case TABLE_TYPE_VALUE_BOTH: - return new SourceSink(name, normalizedProperties); + return new SourceSink(name, normalizedConfig); } throw new SqlClientException("Invalid 'type' attribute for table '" + name + "'. " + "Only 'source', 'sink', and 'both' are supported."); From 5d4a6aee24e6fc10b525b36d8d7a851ecf3f740f Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Sat, 14 Jul 2018 07:21:19 +0200 Subject: [PATCH 14/17] Move exception --- .../org/apache/flink/table/client/config/Environment.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java index 9d4a7ceffaf4f..0853afbb45efc 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Environment.java @@ -228,8 +228,9 @@ private static TableDescriptor createTableDescriptor(String name, Map Date: Sat, 14 Jul 2018 09:18:04 +0200 Subject: [PATCH 15/17] Improve TableFactoryService --- .../table/factories/TableFactoryService.scala | 61 +++++++------------ 1 file changed, 23 insertions(+), 38 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryService.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryService.scala index fc260d8e01850..e147eeffb9bda 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryService.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryService.scala @@ -128,23 +128,19 @@ object TableFactoryService extends Logging { val properties = propertyMap.asScala.toMap - // discover table factories val foundFactories = discoverFactories(classLoader) - // filter by factory class val classFactories = filterByFactoryClass( factoryClass, properties, foundFactories) - // find matching context val contextFactories = filterByContext( factoryClass, properties, foundFactories, classFactories) - // filter by supported keys filterBySupportedProperties( factoryClass, properties, @@ -158,7 +154,6 @@ object TableFactoryService extends Logging { * @return all factories in the classpath */ private def discoverFactories[T](classLoader: Option[ClassLoader]): Seq[TableFactory] = { - val foundFactories = mutable.ArrayBuffer[TableFactory]() try { val iterator = classLoader match { case Some(customClassLoader) => @@ -167,13 +162,7 @@ object TableFactoryService extends Logging { case None => defaultLoader.iterator() } - - while (iterator.hasNext) { - val factory = iterator.next() - foundFactories += factory - } - - foundFactories + iterator.asScala.toSeq } catch { case e: ServiceConfigurationError => LOG.error("Could not load service provider for table factories.", e) @@ -181,6 +170,26 @@ object TableFactoryService extends Logging { } } + /** + * 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 + } + /** * Filters for factories with matching context. * @@ -193,9 +202,7 @@ object TableFactoryService extends Logging { classFactories: Seq[TableFactory]) : Seq[TableFactory] = { - val matchingFactories = mutable.ArrayBuffer[TableFactory]() - - classFactories.foreach { factory => + val matchingFactories = classFactories.filter { factory => val requestedContext = normalizeContext(factory) val plainContext = mutable.Map[String, String]() @@ -208,9 +215,7 @@ object TableFactoryService extends Logging { 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 - } + plainContext.forall(e => properties.contains(e._1) && properties(e._1) == e._2) } if (matchingFactories.isEmpty) { @@ -224,26 +229,6 @@ object TableFactoryService extends Logging { matchingFactories } - /** - * 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 - } - /** * Prepares the properties of a context to be used for match operations. */ From 9dc2cad2be9426f2185635f542f049918550ac61 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Sat, 14 Jul 2018 09:25:05 +0200 Subject: [PATCH 16/17] Remove unused import --- .../src/main/scala/org/apache/flink/table/api/exceptions.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala index 3e3bfac58ed93..81dad3e584ca7 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala @@ -19,7 +19,7 @@ package org.apache.flink.table.api import org.apache.flink.table.descriptors.DescriptorProperties -import org.apache.flink.table.factories.{TableFactory, TableFormatFactory} +import org.apache.flink.table.factories.TableFactory /** * Exception for all errors occurring during expression parsing. From 72585a1e3b19c63a941a3e954a028b7d44f0da51 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Sun, 15 Jul 2018 07:32:09 +0200 Subject: [PATCH 17/17] Last feedback addressed --- .../apache/flink/table/factories/TableFactory.scala | 3 +++ .../flink/table/factories/TableFactoryService.scala | 12 ++++-------- .../flink/table/factories/TableFormatFactory.scala | 11 +++++++---- 3 files changed, 14 insertions(+), 12 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala index ed16e8a48acdf..cc99eccfe243b 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactory.scala @@ -55,6 +55,9 @@ trait TableFactory { * 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 diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryService.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryService.scala index e147eeffb9bda..3baff8e87a80b 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryService.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFactoryService.scala @@ -48,7 +48,6 @@ object TableFactoryService extends Logging { * @return the matching factory */ def find[T](factoryClass: Class[T], descriptor: Descriptor): T = { - Preconditions.checkNotNull(factoryClass) Preconditions.checkNotNull(descriptor) val descriptorProperties = new DescriptorProperties() @@ -66,13 +65,12 @@ object TableFactoryService extends Logging { * @return the matching factory */ def find[T](factoryClass: Class[T], descriptor: Descriptor, classLoader: ClassLoader): T = { - Preconditions.checkNotNull(factoryClass) Preconditions.checkNotNull(descriptor) Preconditions.checkNotNull(classLoader) val descriptorProperties = new DescriptorProperties() descriptor.addProperties(descriptorProperties) - findInternal(factoryClass, descriptorProperties.asMap, None) + findInternal(factoryClass, descriptorProperties.asMap, Some(classLoader)) } /** @@ -84,9 +82,6 @@ object TableFactoryService extends Logging { * @return the matching factory */ def find[T](factoryClass: Class[T], propertyMap: JMap[String, String]): T = { - Preconditions.checkNotNull(factoryClass) - Preconditions.checkNotNull(propertyMap) - findInternal(factoryClass, propertyMap, None) } @@ -104,8 +99,6 @@ object TableFactoryService extends Logging { propertyMap: JMap[String, String], classLoader: ClassLoader) : T = { - Preconditions.checkNotNull(factoryClass) - Preconditions.checkNotNull(propertyMap) Preconditions.checkNotNull(classLoader) findInternal(factoryClass, propertyMap, Some(classLoader)) @@ -126,6 +119,9 @@ object TableFactoryService extends Logging { classLoader: Option[ClassLoader]) : T = { + Preconditions.checkNotNull(factoryClass) + Preconditions.checkNotNull(propertyMap) + val properties = propertyMap.asScala.toMap val foundFactories = discoverFactories(classLoader) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFormatFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFormatFactory.scala index 63d788be74cc0..8fa6fadabe767 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFormatFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/factories/TableFormatFactory.scala @@ -50,10 +50,13 @@ trait TableFormatFactory[T] extends TableFactory { * - 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]