diff --git a/docs/dev/table/sqlClient.md b/docs/dev/table/sqlClient.md index f8500826bc156..a64e78b27d311 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..1f46af2c782fa --- /dev/null +++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowFormatFactory.java @@ -0,0 +1,97 @@ +/* + * 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 = new DescriptorProperties(true); + descriptorProperties.putProperties(properties); + + // validate + new AvroValidator().validate(descriptorProperties); + + // 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 = new DescriptorProperties(true); + descriptorProperties.putProperties(properties); + + // validate + new AvroValidator().validate(descriptorProperties); + + // 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)); + } + } +} 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..42e13a2b05327 --- /dev/null +++ b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonRowFormatFactory.java @@ -0,0 +1,106 @@ +/* + * 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 = new DescriptorProperties(true); + descriptorProperties.putProperties(properties); + + // validate + new JsonValidator().validate(descriptorProperties); + + // 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 = new DescriptorProperties(true); + descriptorProperties.putProperties(properties); + + // validate + new JsonValidator().validate(descriptorProperties); + + // create and configure + return new JsonRowSerializationSchema(createTypeInformation(descriptorProperties)); + } + + private 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 4b0b60d85d8ed..030f629efe2f4 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 @@ -29,6 +29,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 @@ -810,7 +811,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( @@ -858,13 +859,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)}") + } } } @@ -939,7 +946,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) @@ -1091,7 +1098,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 = ??? }