diff --git a/.github/workflows/run-pr-check.yml b/.github/workflows/run-pr-check.yml index 3c765bb17a8db..54c7ef67b2a11 100644 --- a/.github/workflows/run-pr-check.yml +++ b/.github/workflows/run-pr-check.yml @@ -12,9 +12,12 @@ jobs: uses: actions/checkout@v2 with: ref: ${{ github.event.pull_request.head.sha }} - + - name: Set up Maven + uses: apache/pulsar-test-infra/setup-maven@master + with: + maven-version: 3.6.2 - name: Run Unit Test and Install run: | - ./mvnw -pl 'flink-connectors/flink-connector-pulsar,flink-connectors/flink-sql-connector-pulsar' \ + mvn -ntp -pl 'flink-connectors/flink-connector-pulsar,flink-connectors/flink-sql-connector-pulsar' \ clean install diff --git a/docs/content.zh/docs/connectors/datastream/pulsar.md b/docs/content.zh/docs/connectors/datastream/pulsar.md index 8d44be88c71f9..68bf281510a43 100644 --- a/docs/content.zh/docs/connectors/datastream/pulsar.md +++ b/docs/content.zh/docs/connectors/datastream/pulsar.md @@ -155,6 +155,19 @@ Pulsar Source 提供了两种订阅 Topic 或 Topic 分区的方式。 PulsarDeserializationSchema.flinkTypeInfo(TypeInformation, ExecutionConfig); ``` +如果使用 KeyValue 或者 Struct 类型的Schema, 那么 pulsar `Schema` 讲不会含有类型类信息, 但 `PulsarSchemaTypeInformation` 需要通过传入类型类信息来构造。因此我们提供的API支持用户传入类型类信息。 + +例子如下: + +```java + // Primitive 类型: 不需要提供类型信息 + PulsarDeserializationSchema.pulsarSchema(Schema.INT32); + // Struct 类型 (JSON, Protobuf, Avro, 等等.) + PulsarDeserializationSchema.pulsarSchema(Schema.AVRO(SomeClass), SomeClass.class); + // KeyValue 类型 + PulsarDeserializationSchema.pulsarSchema(Schema.KeyValue(Schema.INT32, Schema.AVRO(SomeClass)), Integer.class, SomeClass.class); +``` + Pulsar 的 `Message` 包含了很多 [额外的属性](https://pulsar.apache.org/docs/zh-CN/concepts-messaging/#%E6%B6%88%E6%81%AF)。例如,消息的 key、消息发送时间、消息生产时间、用户在消息上自定义的键值对属性等。可以使用 `Message` 接口来获取这些属性。 如果用户需要基于这些额外的属性来解析一条消息,可以实现 `PulsarDeserializationSchema` 接口。并一定要确保 `PulsarDeserializationSchema.getProducedType()` 方法返回的 `TypeInformation` 是正确的结果。Flink 使用 `TypeInformation` 将解析出来的结果序列化传递到下游算子。 diff --git a/docs/content/docs/connectors/datastream/pulsar.md b/docs/content/docs/connectors/datastream/pulsar.md index 0953753ee33fb..a8c7057cdddf2 100644 --- a/docs/content/docs/connectors/datastream/pulsar.md +++ b/docs/content/docs/connectors/datastream/pulsar.md @@ -176,6 +176,21 @@ you can use the predefined `PulsarDeserializationSchema`. Pulsar connector provi ```java PulsarDeserializationSchema.flinkTypeInfo(TypeInformation, ExecutionConfig); ``` +If using KeyValue type or Struct types, the pulsar `Schema` does not contain type class info which is +needed by `PulsarSchemaTypeInformation`. So the two APIs provides 2 parameter to pass the type info. + +A example would be: + +```java + // Primitive types: do not need to provide type class info + PulsarDeserializationSchema.pulsarSchema(Schema.INT32); + + // Struct types (JSON, Protobuf, Avro, etc.) + PulsarDeserializationSchema.pulsarSchema(Schema.AVRO(SomeClass), SomeClass.class); + + // KeyValue type + PulsarDeserializationSchema.pulsarSchema(Schema.KeyValue(Schema.INT32, Schema.AVRO(SomeClass)), Integer.class, SomeClass.class); +``` Pulsar `Message` contains some [extra properties](https://pulsar.apache.org/docs/en/concepts-messaging/#messages), such as message key, message publish time, message time, and application-defined key/value pairs etc. diff --git a/docs/layouts/shortcodes/generated/pulsar_source_configuration.html b/docs/layouts/shortcodes/generated/pulsar_source_configuration.html index 3bcdad85e1a04..c79495029b45d 100644 --- a/docs/layouts/shortcodes/generated/pulsar_source_configuration.html +++ b/docs/layouts/shortcodes/generated/pulsar_source_configuration.html @@ -20,6 +20,12 @@ Boolean Flink commits the consuming position with pulsar transactions on checkpoint. However, if you have disabled the Flink checkpoint or disabled transaction for your Pulsar cluster, ensure that you have set this option to true.
The source would use pulsar client's internal mechanism and commit cursor in two ways.
  • For Key_Shared and Shared subscription, the cursor would be committed once the message is consumed.
  • For Exclusive and Failover subscription, the cursor would be committed in a given interval.
+ +
pulsar.source.enableSchemaEvolution
+ false + Boolean + If you enable this option, we would consume and deserialize the message by using Pulsar's Schema. +
pulsar.source.maxFetchRecords
100 diff --git a/flink-connectors/flink-connector-pulsar/archunit-violations/48c7dd05-c840-4ac4-a3ba-919e07191450 b/flink-connectors/flink-connector-pulsar/archunit-violations/48c7dd05-c840-4ac4-a3ba-919e07191450 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/flink-connectors/flink-connector-pulsar/archunit-violations/a2ce237e-b050-4ba0-8748-d83637a207a8 b/flink-connectors/flink-connector-pulsar/archunit-violations/a2ce237e-b050-4ba0-8748-d83637a207a8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/flink-connectors/flink-connector-pulsar/archunit-violations/stored.rules b/flink-connectors/flink-connector-pulsar/archunit-violations/stored.rules new file mode 100644 index 0000000000000..ecdc39b0f1907 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/archunit-violations/stored.rules @@ -0,0 +1,4 @@ +# +#Thu May 12 11:35:25 CST 2022 +Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=48c7dd05-c840-4ac4-a3ba-919e07191450 +ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=a2ce237e-b050-4ba0-8748-d83637a207a8 diff --git a/flink-connectors/flink-connector-pulsar/pom.xml b/flink-connectors/flink-connector-pulsar/pom.xml index 7529e533c82df..0d5bb8740a747 100644 --- a/flink-connectors/flink-connector-pulsar/pom.xml +++ b/flink-connectors/flink-connector-pulsar/pom.xml @@ -56,6 +56,34 @@ under the License. ${project.version} + + + org.apache.flink + flink-table-common + ${project.version} + provided + + + + org.apache.flink + flink-table-api-java-bridge + ${project.version} + provided + + + + org.apache.flink + flink-table-runtime + ${project.version} + provided + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${project.version} + provided + @@ -108,6 +136,62 @@ under the License. test + + + org.apache.flink + flink-table-common + ${project.version} + test-jar + test + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${project.version} + test + + + + org.apache.flink + flink-table-api-scala-bridge_${scala.binary.version} + ${project.version} + test + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${project.version} + test-jar + test + + + + + org.apache.flink + flink-json + ${project.version} + test + + + org.apache.flink + flink-avro + ${project.version} + test + + + org.apache.flink + flink-avro-confluent-registry + ${project.version} + test + + + org.apache.flink + flink-csv + ${project.version} + test + + @@ -236,7 +320,6 @@ under the License. org.apache.maven.plugins maven-surefire-plugin - true 1 -Xms256m -Xmx2048m -Dmvn.forkNumber=${surefire.forkNumber} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/AvroSchemaFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/AvroSchemaFactory.java index ab97b4102cc31..8288136288641 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/AvroSchemaFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/AvroSchemaFactory.java @@ -18,6 +18,9 @@ package org.apache.flink.connector.pulsar.common.schema.factories; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.AvroUtils; + import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.schema.SchemaDefinition; import org.apache.pulsar.client.impl.schema.AvroSchema; @@ -45,4 +48,14 @@ public Schema createSchema(SchemaInfo info) { return AvroSchema.of(definition); } + + @Override + public TypeInformation createTypeInfo(SchemaInfo info) { + try { + Class decodeClassInfo = decodeClassInfo(info); + return AvroUtils.getAvroUtils().createAvroTypeInfo(decodeClassInfo); + } catch (Exception e) { + return super.createTypeInfo(info); + } + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactory.java index 8d1a4944d0c5c..2280d353d95af 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactory.java @@ -18,6 +18,9 @@ package org.apache.flink.connector.pulsar.common.schema.factories; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.AvroUtils; + import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.schema.JSONSchema; import org.apache.pulsar.common.schema.SchemaInfo; @@ -38,4 +41,14 @@ public Schema createSchema(SchemaInfo info) { Class typeClass = decodeClassInfo(info); return JSONSchema.of(typeClass, info.getProperties()); } + + @Override + public TypeInformation createTypeInfo(SchemaInfo info) { + try { + Class decodeClassInfo = decodeClassInfo(info); + return AvroUtils.getAvroUtils().createAvroTypeInfo(decodeClassInfo); + } catch (Exception e) { + return super.createTypeInfo(info); + } + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/KeyValueSchemaFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/KeyValueSchemaFactory.java index 893e7e6eeb1db..d60f06d2a7643 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/KeyValueSchemaFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/KeyValueSchemaFactory.java @@ -69,11 +69,8 @@ public Schema> createSchema(SchemaInfo info) { public TypeInformation> createTypeInfo(SchemaInfo info) { KeyValue kvSchemaInfo = decodeKeyValueSchemaInfo(info); - Schema keySchema = PulsarSchemaUtils.createSchema(kvSchemaInfo.getKey()); - Class keyClass = decodeClassInfo(keySchema.getSchemaInfo()); - - Schema valueSchema = PulsarSchemaUtils.createSchema(kvSchemaInfo.getValue()); - Class valueClass = decodeClassInfo(valueSchema.getSchemaInfo()); + Class keyClass = decodeClassInfo(kvSchemaInfo.getKey()); + Class valueClass = decodeClassInfo(kvSchemaInfo.getValue()); Schema> schema = createSchema(info); PulsarSchema> pulsarSchema = diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java index 1e4113a8d03b1..30c454455ca9f 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java @@ -45,6 +45,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.Base64; import java.util.Collection; import java.util.List; import java.util.Map; @@ -133,9 +134,16 @@ public void write(IN element, Context context) throws IOException, InterruptedEx PulsarMessage message = serializationSchema.serialize(element, sinkContext); // Choose the right topic to send. - String key = message.getKey(); + List availableTopics = metadataListener.availableTopics(); - String topic = topicRouter.route(element, key, availableTopics, sinkContext); + String keyString; + // TODO if both keyBytes and key are set, use keyBytes. This is a temporary solution. + if (message.getKeyBytes() == null) { + keyString = message.getKey(); + } else { + keyString = Base64.getEncoder().encodeToString(message.getKeyBytes()); + } + String topic = topicRouter.route(element, keyString, availableTopics, sinkContext); // Create message builder for sending message. TypedMessageBuilder builder = createMessageBuilder(topic, context, message); @@ -206,6 +214,11 @@ private TypedMessageBuilder createMessageBuilder( builder.key(key); } + byte[] keyBytes = message.getKeyBytes(); + if (keyBytes != null) { + builder.keyBytes(keyBytes); + } + long eventTime = message.getEventTime(); if (eventTime > 0) { builder.eventTime(eventTime); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessage.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessage.java index 0c45763cdb1ef..0dd52655f9040 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessage.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessage.java @@ -37,6 +37,7 @@ public class PulsarMessage { @Nullable private final byte[] orderingKey; @Nullable private final String key; + @Nullable private final byte[] keyBytes; private final long eventTime; private final Schema schema; @Nullable private final T value; @@ -49,6 +50,7 @@ public class PulsarMessage { PulsarMessage( @Nullable byte[] orderingKey, @Nullable String key, + @Nullable byte[] keyBytes, long eventTime, Schema schema, @Nullable T value, @@ -58,6 +60,7 @@ public class PulsarMessage { boolean disableReplication) { this.orderingKey = orderingKey; this.key = key; + this.keyBytes = keyBytes; this.eventTime = eventTime; this.schema = schema; this.value = value; @@ -77,6 +80,11 @@ public String getKey() { return key; } + @Nullable + public byte[] getKeyBytes() { + return keyBytes; + } + public long getEventTime() { return eventTime; } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessageBuilder.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessageBuilder.java index 9330d09a3d03b..f894f2410778c 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessageBuilder.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessageBuilder.java @@ -36,6 +36,7 @@ public class PulsarMessageBuilder { private byte[] orderingKey; private String key; + private byte[] keyBytes; private long eventTime; Schema schema; private T value; @@ -59,6 +60,15 @@ public PulsarMessageBuilder key(String key) { return null; } + /** + * Property {@link TypedMessageBuilder#keyBytes(byte[])}. This property would also be used in + * {@link KeyHashTopicRouter}. + */ + public PulsarMessageBuilder keyBytes(byte[] keyBytes) { + this.keyBytes = checkNotNull(keyBytes); + return null; + } + /** Method wrapper of {@link TypedMessageBuilder#eventTime(long)}. */ public PulsarMessageBuilder eventTime(long eventTime) { this.eventTime = eventTime; @@ -116,6 +126,7 @@ public PulsarMessage build() { return new PulsarMessage<>( orderingKey, key, + keyBytes, eventTime, schema, value, diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java index a6c48d14bc87e..560b32162f2b3 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java @@ -94,7 +94,7 @@ public final class PulsarSource * The constructor for PulsarSource, it's package protected for forcing using {@link * PulsarSourceBuilder}. */ - PulsarSource( + public PulsarSource( SourceConfiguration sourceConfiguration, PulsarSubscriber subscriber, RangeGenerator rangeGenerator, diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java index 517286d8341a3..b6e0f408d5bbf 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java @@ -25,6 +25,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.pulsar.common.config.PulsarConfigBuilder; import org.apache.flink.connector.pulsar.common.config.PulsarOptions; +import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSchemaWrapper; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; @@ -54,6 +55,7 @@ import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_CONSUMER_NAME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_READ_SCHEMA_EVOLUTION; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_READ_TRANSACTION_TIMEOUT; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_TYPE; @@ -273,6 +275,15 @@ public PulsarSourceBuilder setConsumerName(String consumerName) { return setConfig(PULSAR_CONSUMER_NAME, consumerName); } + /** + * If you enable this option, we would consume and deserialize the message by using Pulsar + * {@link Schema}. + */ + public PulsarSourceBuilder enableSchemaEvolution() { + configBuilder.set(PULSAR_READ_SCHEMA_EVOLUTION, true); + return this; + } + /** * Set a topic range generator for Key_Shared subscription. * @@ -470,6 +481,21 @@ public PulsarSource build() { } } + // Schema evolution check. + if (deserializationSchema instanceof PulsarSchemaWrapper + && !Boolean.TRUE.equals(configBuilder.get(PULSAR_READ_SCHEMA_EVOLUTION))) { + LOG.info( + "It seems like you want to read message using Pulsar Schema." + + " You can enableSchemaEvolution for using this feature." + + " We would use Schema.BYTES as the default schema if you don't enable this option."); + } + + if (Boolean.TRUE.equals(configBuilder.get(PULSAR_READ_SCHEMA_EVOLUTION))) { + checkState( + deserializationSchema instanceof PulsarSchemaWrapper, + "When enabling schema evolution, you must use a Pulsar schema."); + } + if (!configBuilder.contains(PULSAR_CONSUMER_NAME)) { LOG.warn( "We recommend set a readable consumer name through setConsumerName(String) in production mode."); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java index 39a73974f5c0a..70ddb6793dfd8 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java @@ -188,6 +188,17 @@ private PulsarSourceOptions() { " A possible solution is to adjust the retention settings in Pulsar or ignoring the check result.") .build()); + public static final ConfigOption PULSAR_READ_SCHEMA_EVOLUTION = + ConfigOptions.key(SOURCE_CONFIG_PREFIX + "enableSchemaEvolution") + .booleanType() + .defaultValue(false) + .withDescription( + Description.builder() + .text( + "If you enable this option, we would consume and deserialize the message by using Pulsar's %s.", + code("Schema")) + .build()); + /////////////////////////////////////////////////////////////////////////////// // // The configuration for ConsumerConfigurationData part. diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java index 806fe4a418b23..bd6c61216acd7 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java @@ -23,10 +23,12 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.pulsar.common.config.PulsarConfiguration; +import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSchemaWrapper; import org.apache.flink.connector.pulsar.source.enumerator.cursor.CursorPosition; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.pulsar.client.api.ConsumerBuilder; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionMode; import org.apache.pulsar.client.api.SubscriptionType; @@ -39,6 +41,7 @@ import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_RECORDS; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_TIME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_READ_SCHEMA_EVOLUTION; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_READ_TRANSACTION_TIMEOUT; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_MODE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; @@ -53,6 +56,7 @@ public class SourceConfiguration extends PulsarConfiguration { private final int messageQueueCapacity; private final long partitionDiscoveryIntervalMs; private final boolean enableAutoAcknowledgeMessage; + private final boolean enableSchemaEvolution; private final long autoCommitCursorInterval; private final long transactionTimeoutMillis; private final Duration maxFetchTime; @@ -68,6 +72,7 @@ public SourceConfiguration(Configuration configuration) { this.messageQueueCapacity = getInteger(ELEMENT_QUEUE_CAPACITY); this.partitionDiscoveryIntervalMs = get(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS); this.enableAutoAcknowledgeMessage = get(PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE); + this.enableSchemaEvolution = get(PULSAR_READ_SCHEMA_EVOLUTION); this.autoCommitCursorInterval = get(PULSAR_AUTO_COMMIT_CURSOR_INTERVAL); this.transactionTimeoutMillis = get(PULSAR_READ_TRANSACTION_TIMEOUT); this.maxFetchTime = get(PULSAR_MAX_FETCH_TIME, Duration::ofMillis); @@ -107,6 +112,14 @@ public boolean isEnableAutoAcknowledgeMessage() { return enableAutoAcknowledgeMessage; } + /** + * If we should deserialize the message with a specified Pulsar {@link Schema} instead the + * default {@link Schema#BYTES}. This switch is only used for {@link PulsarSchemaWrapper}. + */ + public boolean isEnableSchemaEvolution() { + return enableSchemaEvolution; + } + /** * The interval in millis for acknowledge message when you enable {@link * #isEnableAutoAcknowledgeMessage} and use {@link SubscriptionType#Failover} or {@link diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java index f8079608cc65a..3705f865ebc01 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java @@ -25,6 +25,8 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.impl.MessageIdImpl; +import java.util.Objects; + import static org.apache.flink.util.Preconditions.checkState; /** This cursor would left pulsar start consuming from a specific message id. */ @@ -67,4 +69,21 @@ public MessageIdStartCursor(MessageId messageId, boolean inclusive) { public CursorPosition position(String topic, int partitionId) { return new CursorPosition(messageId); } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MessageIdStartCursor that = (MessageIdStartCursor) o; + return Objects.equals(messageId, that.messageId); + } + + @Override + public int hashCode() { + return Objects.hash(messageId); + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java index eb4ea32ebb6b9..e03a773281b83 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java @@ -21,7 +21,9 @@ import org.apache.flink.connector.pulsar.source.enumerator.cursor.CursorPosition; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; -/** This cursor would left pulsar start consuming from a specific publish timestamp. */ +import java.util.Objects; + +/** A cursor starts consuming from a specific publish timestamp. */ public class TimestampStartCursor implements StartCursor { private static final long serialVersionUID = 5170578885838095320L; @@ -35,4 +37,21 @@ public TimestampStartCursor(long timestamp) { public CursorPosition position(String topic, int partitionId) { return new CursorPosition(timestamp); } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TimestampStartCursor that = (TimestampStartCursor) o; + return timestamp == that.timestamp; + } + + @Override + public int hashCode() { + return Objects.hash(timestamp); + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java index d266c98df4253..57d5f02ed24c0 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java @@ -20,7 +20,6 @@ import org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange; @@ -32,6 +31,7 @@ import java.util.stream.IntStream; import static java.util.stream.Collectors.toList; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicName; /** PulsarSubscriber abstract class to simplify Pulsar admin related operations. */ public abstract class BasePulsarSubscriber implements PulsarSubscriber { @@ -39,7 +39,7 @@ public abstract class BasePulsarSubscriber implements PulsarSubscriber { protected TopicMetadata queryTopicMetadata(PulsarAdmin pulsarAdmin, String topicName) { // Drop the complete topic name for a clean partitioned topic name. - String completeTopicName = TopicNameUtils.topicName(topicName); + String completeTopicName = topicName(topicName); try { PartitionedTopicMetadata metadata = pulsarAdmin.topics().getPartitionedTopicMetadata(completeTopicName); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchema.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchema.java index 4a116cdc64dc1..3e3bc4217bf70 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchema.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchema.java @@ -76,7 +76,16 @@ default void open(InitializationContext context) throws Exception { * @param message The message decoded by pulsar. * @param out The collector to put the resulting messages. */ - void deserialize(Message message, Collector out) throws Exception; + void deserialize(Message message, Collector out) throws Exception; + + /** + * By default, deserializers will decode bytes array message. + * + * @return The related Pulsar Schema for this serializer. + */ + default Schema schema() { + return Schema.BYTES; + } /** * Create a PulsarDeserializationSchema by using the flink's {@link DeserializationSchema}. It diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaWrapper.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaWrapper.java index e9b2779d9a99c..e3f0a7770d2ff 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaWrapper.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaWrapper.java @@ -52,10 +52,9 @@ public void open(InitializationContext context, SourceConfiguration configuratio } @Override - public void deserialize(Message message, Collector out) throws Exception { + public void deserialize(Message message, Collector out) throws Exception { byte[] bytes = message.getData(); T instance = deserializationSchema.deserialize(bytes); - out.collect(instance); } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarSchemaWrapper.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarSchemaWrapper.java index 7926d803d4a11..2bcc163d198c1 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarSchemaWrapper.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarSchemaWrapper.java @@ -18,8 +18,10 @@ package org.apache.flink.connector.pulsar.source.reader.deserializer; 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.connector.pulsar.common.schema.PulsarSchema; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.util.Collector; import org.apache.pulsar.client.api.Message; @@ -41,17 +43,31 @@ class PulsarSchemaWrapper implements PulsarDeserializationSchema { /** The serializable pulsar schema, it wrap the schema with type class. */ private final PulsarSchema pulsarSchema; + private boolean isSchemaEvolutionEnabled; + public PulsarSchemaWrapper(PulsarSchema pulsarSchema) { this.pulsarSchema = pulsarSchema; } @Override - public void deserialize(Message message, Collector out) throws Exception { - Schema schema = this.pulsarSchema.getPulsarSchema(); - byte[] bytes = message.getData(); - T instance = schema.decode(bytes); + public void open( + DeserializationSchema.InitializationContext context, SourceConfiguration configuration) + throws Exception { + this.isSchemaEvolutionEnabled = configuration.isEnableSchemaEvolution(); + } - out.collect(instance); + @Override + public void deserialize(Message message, Collector out) throws Exception { + if (isSchemaEvolutionEnabled) { + @SuppressWarnings("unchecked") + T value = (T) message.getValue(); + out.collect(value); + } else { + Schema schema = this.pulsarSchema.getPulsarSchema(); + byte[] bytes = message.getData(); + T instance = schema.decode(bytes); + out.collect(instance); + } } @Override @@ -59,4 +75,13 @@ public TypeInformation getProducedType() { SchemaInfo info = pulsarSchema.getSchemaInfo(); return createTypeInformation(info); } + + @Override + public Schema schema() { + if (isSchemaEvolutionEnabled) { + return pulsarSchema.getPulsarSchema(); + } else { + return Schema.BYTES; + } + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarTypeInformationWrapper.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarTypeInformationWrapper.java index 0118079f82fa4..5ea9de99dff81 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarTypeInformationWrapper.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarTypeInformationWrapper.java @@ -51,7 +51,7 @@ public PulsarTypeInformationWrapper(TypeInformation information, ExecutionCon } @Override - public void deserialize(Message message, Collector out) throws Exception { + public void deserialize(Message message, Collector out) throws Exception { DataInputDeserializer dis = DESERIALIZER.get(); dis.setBuffer(message.getData()); T instance = serializer.deserialize(dis); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java index bb6d79641f503..a6c713cf6577c 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java @@ -61,12 +61,12 @@ public PulsarOrderedPartitionSplitReader( } @Override - protected Message pollMessage(Duration timeout) throws PulsarClientException { + protected Message pollMessage(Duration timeout) throws PulsarClientException { return pulsarConsumer.receive(Math.toIntExact(timeout.toMillis()), TimeUnit.MILLISECONDS); } @Override - protected void finishedPollMessage(Message message) { + protected void finishedPollMessage(Message message) { // Nothing to do here. LOG.debug("Finished polling message {}", message); @@ -75,7 +75,7 @@ protected void finishedPollMessage(Message message) { } @Override - protected void startConsumer(PulsarPartitionSplit split, Consumer consumer) { + protected void startConsumer(PulsarPartitionSplit split, Consumer consumer) { MessageId latestConsumedId = split.getLatestConsumedId(); // Reset the start position for ordered pulsar consumer. diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java index 37b5630a8d113..9778bd183c8bd 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java @@ -72,7 +72,7 @@ abstract class PulsarPartitionSplitReaderBase protected final PulsarDeserializationSchema deserializationSchema; protected final AtomicBoolean wakeup; - protected Consumer pulsarConsumer; + protected Consumer pulsarConsumer; protected PulsarPartitionSplit registeredSplit; protected PulsarPartitionSplitReaderBase( @@ -104,7 +104,7 @@ public RecordsWithSplitIds> fetch() throws IOException { PulsarMessageCollector collector = new PulsarMessageCollector<>(splitId, builder); Deadline deadline = Deadline.fromNow(sourceConfiguration.getMaxFetchTime()); - // Consume message from pulsar until it was woke up by flink reader. + // Consume message from pulsar until it was woken up by flink reader. for (int messageNum = 0; messageNum < sourceConfiguration.getMaxFetchRecords() && deadline.hasTimeLeft() @@ -112,16 +112,17 @@ public RecordsWithSplitIds> fetch() throws IOException { messageNum++) { try { Duration timeout = deadline.timeLeftIfAny(); - Message message = pollMessage(timeout); + Message message = pollMessage(timeout); if (message == null) { break; } - // Deserialize message. collector.setMessage(message); + + // Deserialize message by DeserializationSchema or Pulsar Schema. deserializationSchema.deserialize(message, collector); - // Acknowledge message if need. + // Acknowledge message if needed. finishedPollMessage(message); if (stopCursor.shouldStop(message)) { @@ -166,7 +167,7 @@ public void handleSplitsChanges(SplitsChange splitsChanges PulsarPartitionSplit newSplit = newSplits.get(0); // Create pulsar consumer. - Consumer consumer = createPulsarConsumer(newSplit); + Consumer consumer = createPulsarConsumer(newSplit); // Open start & stop cursor. newSplit.open(pulsarAdmin); @@ -192,12 +193,12 @@ public void close() { } @Nullable - protected abstract Message pollMessage(Duration timeout) + protected abstract Message pollMessage(Duration timeout) throws ExecutionException, InterruptedException, PulsarClientException; - protected abstract void finishedPollMessage(Message message); + protected abstract void finishedPollMessage(Message message); - protected abstract void startConsumer(PulsarPartitionSplit split, Consumer consumer); + protected abstract void startConsumer(PulsarPartitionSplit split, Consumer consumer); // --------------------------- Helper Methods ----------------------------- @@ -205,15 +206,19 @@ protected boolean isNotWakeup() { return !wakeup.get(); } - /** Create a specified {@link Consumer} by the given split information. */ - protected Consumer createPulsarConsumer(PulsarPartitionSplit split) { + /** + * Create a specified {@link Consumer} by the given split information. If using pulsar schema, + * then use the pulsar schema, if using flink schema, then use a Schema.BYTES + */ + protected Consumer createPulsarConsumer(PulsarPartitionSplit split) { return createPulsarConsumer(split.getPartition()); } - /** Create a specified {@link Consumer} by the given topic partition. */ - protected Consumer createPulsarConsumer(TopicPartition partition) { - ConsumerBuilder consumerBuilder = - createConsumerBuilder(pulsarClient, Schema.BYTES, sourceConfiguration); + protected Consumer createPulsarConsumer(TopicPartition partition) { + Schema schema = deserializationSchema.schema(); + + ConsumerBuilder consumerBuilder = + createConsumerBuilder(pulsarClient, schema, sourceConfiguration); consumerBuilder.topic(partition.getFullTopicName()); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java index 5940cc9ac19be..d8dc49c4de2aa 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java @@ -75,9 +75,9 @@ public PulsarUnorderedPartitionSplitReader( } @Override - protected Message pollMessage(Duration timeout) + protected Message pollMessage(Duration timeout) throws ExecutionException, InterruptedException, PulsarClientException { - Message message = + Message message = pulsarConsumer.receive(Math.toIntExact(timeout.toMillis()), TimeUnit.MILLISECONDS); // Skip the message when receive timeout @@ -116,7 +116,7 @@ protected Message pollMessage(Duration timeout) } @Override - protected void finishedPollMessage(Message message) { + protected void finishedPollMessage(Message message) { if (sourceConfiguration.isEnableAutoAcknowledgeMessage()) { sneakyClient(() -> pulsarConsumer.acknowledge(message)); } @@ -126,7 +126,7 @@ protected void finishedPollMessage(Message message) { } @Override - protected void startConsumer(PulsarPartitionSplit split, Consumer consumer) { + protected void startConsumer(PulsarPartitionSplit split, Consumer consumer) { TxnID uncommittedTransactionId = split.getUncommittedTransactionId(); // Abort the uncommitted pulsar transaction. diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java new file mode 100644 index 0000000000000..171836b063214 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java @@ -0,0 +1,282 @@ +/* + * 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.connector.pulsar.table; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.pulsar.common.config.PulsarOptions; +import org.apache.flink.connector.pulsar.sink.PulsarSinkOptions; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.connector.pulsar.source.PulsarSourceOptions; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.table.sink.PulsarTableSerializationSchemaFactory; +import org.apache.flink.connector.pulsar.table.sink.PulsarTableSink; +import org.apache.flink.connector.pulsar.table.source.PulsarTableDeserializationSchemaFactory; +import org.apache.flink.connector.pulsar.table.source.PulsarTableSource; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.DynamicTableSourceFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.types.DataType; + +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.List; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL; +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.createKeyFormatProjection; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.createValueFormatProjection; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getKeyDecodingFormat; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getKeyEncodingFormat; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getMessageDelayMillis; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getPulsarProperties; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getStartCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getSubscriptionType; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicListFromOptions; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicRouter; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicRoutingMode; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getValueDecodingFormat; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getValueEncodingFormat; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.ADMIN_URL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FORMAT; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SERVICE_URL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_MESSAGE_DELAY_INTERVAL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_NAME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validatePrimaryKeyConstraints; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateTableSinkOptions; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateTableSourceOptions; +import static org.apache.flink.table.factories.FactoryUtil.FORMAT; +import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM; + +/** + * Factory for creating {@link DynamicTableSource} and {@link DynamicTableSink}. + * + *

The main role of this class is to retrieve config options and validate options from config and + * the table schema. It also sets default values if a config option is not present. + */ +public class PulsarTableFactory implements DynamicTableSourceFactory, DynamicTableSinkFactory { + + public static final String IDENTIFIER = "pulsar"; + + @Override + public DynamicTableSource createDynamicTableSource(Context context) { + FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); + // Format options should be retrieved before validation. + final DecodingFormat> keyDecodingFormat = + getKeyDecodingFormat(helper); + final DecodingFormat> valueDecodingFormat = + getValueDecodingFormat(helper); + ReadableConfig tableOptions = helper.getOptions(); + + // Validate configs are not conflict; each options is consumed; no unwanted configs + // PulsarOptions, PulsarSourceOptions and PulsarSinkOptions is not part of the validation. + helper.validateExcept( + PulsarOptions.CLIENT_CONFIG_PREFIX, + PulsarOptions.ADMIN_CONFIG_PREFIX, + PulsarSourceOptions.SOURCE_CONFIG_PREFIX, + PulsarSourceOptions.CONSUMER_CONFIG_PREFIX, + PulsarSinkOptions.PRODUCER_CONFIG_PREFIX, + PulsarSinkOptions.SINK_CONFIG_PREFIX); + + validatePrimaryKeyConstraints( + context.getObjectIdentifier(), + context.getPrimaryKeyIndexes(), + context.getCatalogTable().getOptions(), + helper); + + validateTableSourceOptions(tableOptions); + + // Retrieve configs + final List topics = getTopicListFromOptions(tableOptions); + final StartCursor startCursor = getStartCursor(tableOptions); + final SubscriptionType subscriptionType = getSubscriptionType(tableOptions); + + // Forward source configs + final Properties properties = getPulsarProperties(tableOptions); + properties.setProperty(PULSAR_ADMIN_URL.key(), tableOptions.get(ADMIN_URL)); + properties.setProperty(PULSAR_SERVICE_URL.key(), tableOptions.get(SERVICE_URL)); + properties.setProperty( + PULSAR_SUBSCRIPTION_NAME.key(), tableOptions.get(SOURCE_SUBSCRIPTION_NAME)); + // Retrieve physical fields (not including computed or metadata fields), + // and projections and create a schema factory based on such information. + final DataType physicalDataType = context.getPhysicalRowDataType(); + + final int[] valueProjection = createValueFormatProjection(tableOptions, physicalDataType); + final int[] keyProjection = createKeyFormatProjection(tableOptions, physicalDataType); + + final PulsarTableDeserializationSchemaFactory deserializationSchemaFactory = + new PulsarTableDeserializationSchemaFactory( + physicalDataType, + keyDecodingFormat, + keyProjection, + valueDecodingFormat, + valueProjection); + + // Set default values for configuration not exposed to user. + final DecodingFormat> decodingFormatForMetadataPushdown = + valueDecodingFormat; + + return new PulsarTableSource( + deserializationSchemaFactory, + decodingFormatForMetadataPushdown, + topics, + properties, + startCursor, + subscriptionType); + } + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); + // Format options should be retrieved before validation. + final EncodingFormat> keyEncodingFormat = + getKeyEncodingFormat(helper); + final EncodingFormat> valueEncodingFormat = + getValueEncodingFormat(helper); + ReadableConfig tableOptions = helper.getOptions(); + + // Validate configs are not conflict; each options is consumed; no unwanted configs + // PulsarOptions, PulsarSourceOptions and PulsarSinkOptions is not part of the validation. + helper.validateExcept( + PulsarOptions.CLIENT_CONFIG_PREFIX, + PulsarOptions.ADMIN_CONFIG_PREFIX, + PulsarSourceOptions.SOURCE_CONFIG_PREFIX, + PulsarSourceOptions.CONSUMER_CONFIG_PREFIX, + PulsarSinkOptions.PRODUCER_CONFIG_PREFIX, + PulsarSinkOptions.SINK_CONFIG_PREFIX); + + validatePrimaryKeyConstraints( + context.getObjectIdentifier(), + context.getPrimaryKeyIndexes(), + context.getCatalogTable().getOptions(), + helper); + + validateTableSinkOptions(tableOptions); + + // Retrieve configs + final TopicRouter topicRouter = + getTopicRouter(tableOptions, context.getClassLoader()); + final TopicRoutingMode topicRoutingMode = getTopicRoutingMode(tableOptions); + final long messageDelayMillis = getMessageDelayMillis(tableOptions); + + final List topics = getTopicListFromOptions(tableOptions); + + // Forward sink configs + final Properties properties = getPulsarProperties(tableOptions); + properties.setProperty(PULSAR_ADMIN_URL.key(), tableOptions.get(ADMIN_URL)); + properties.setProperty(PULSAR_SERVICE_URL.key(), tableOptions.get(SERVICE_URL)); + + // Retrieve physical DataType (not including computed or metadata fields) + final DataType physicalDataType = context.getPhysicalRowDataType(); + final int[] keyProjection = createKeyFormatProjection(tableOptions, physicalDataType); + final int[] valueProjection = createValueFormatProjection(tableOptions, physicalDataType); + + final PulsarTableSerializationSchemaFactory serializationSchemaFactory = + new PulsarTableSerializationSchemaFactory( + physicalDataType, + keyEncodingFormat, + keyProjection, + valueEncodingFormat, + valueProjection); + + // Set default values for configuration not exposed to user. + final DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE; + final ChangelogMode changelogMode = valueEncodingFormat.getChangelogMode(); + + return new PulsarTableSink( + serializationSchemaFactory, + changelogMode, + topics, + properties, + deliveryGuarantee, + topicRouter, + topicRoutingMode, + messageDelayMillis); + } + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + // TODO source_subscription_name has default value so it should be optional ? + @Override + public Set> requiredOptions() { + return Stream.of(TOPICS, ADMIN_URL, SERVICE_URL, FORMAT, SOURCE_SUBSCRIPTION_NAME) + .collect(Collectors.toSet()); + } + + @Override + public Set> optionalOptions() { + return Stream.of( + SOURCE_SUBSCRIPTION_TYPE, + SOURCE_START_FROM_MESSAGE_ID, + SOURCE_START_FROM_PUBLISH_TIME, + SINK_CUSTOM_TOPIC_ROUTER, + SINK_TOPIC_ROUTING_MODE, + SINK_MESSAGE_DELAY_INTERVAL, + SINK_PARALLELISM, + KEY_FORMAT, + KEY_FIELDS) + .collect(Collectors.toSet()); + } + + /** + * Format and Delivery guarantee related options are not forward options. + * + * @return + */ + @Override + public Set> forwardOptions() { + return Stream.of( + TOPICS, + ADMIN_URL, + SERVICE_URL, + SOURCE_SUBSCRIPTION_TYPE, + SOURCE_SUBSCRIPTION_NAME, + SOURCE_START_FROM_MESSAGE_ID, + SOURCE_START_FROM_PUBLISH_TIME, + SINK_CUSTOM_TOPIC_ROUTER, + SINK_TOPIC_ROUTING_MODE, + SINK_MESSAGE_DELAY_INTERVAL) + .collect(Collectors.toSet()); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java new file mode 100644 index 0000000000000..611bccc75e2d7 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java @@ -0,0 +1,272 @@ +/* + * 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.connector.pulsar.table; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DeserializationFormatFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.Preconditions; + +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.impl.MessageIdImpl; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Properties; +import java.util.stream.IntStream; + +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FORMAT; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_MESSAGE_DELAY_INTERVAL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.table.factories.FactoryUtil.FORMAT; + +/** + * A util class for getting fields from config options, getting formats and other useful + * information. + * + *

It contains the following functionalities. + * + *

    + *
  • Get Topics from configurations. + *
  • Get source StartCursor from configurations. + *
  • Get source SubscriptionType from configurations. + *
  • Get sink messageDelayMillis from configurations. + *
  • Get sink TopicRouter/TopicRoutingMode from configurations. + *
  • Create key and value encoding/decoding format. + *
  • Create key and value projection. + *
+ */ +public class PulsarTableOptionUtils { + + private PulsarTableOptionUtils() {} + + // -------------------------------------------------------------------------------------------- + // Decoding / Encoding and Projection + // -------------------------------------------------------------------------------------------- + + @Nullable + public static DecodingFormat> getKeyDecodingFormat( + FactoryUtil.TableFactoryHelper helper) { + return helper.discoverOptionalDecodingFormat(DeserializationFormatFactory.class, KEY_FORMAT) + .orElse(null); + } + + @Nullable + public static EncodingFormat> getKeyEncodingFormat( + FactoryUtil.TableFactoryHelper helper) { + return helper.discoverOptionalEncodingFormat(SerializationFormatFactory.class, KEY_FORMAT) + .orElse(null); + } + + public static DecodingFormat> getValueDecodingFormat( + FactoryUtil.TableFactoryHelper helper) { + return helper.discoverOptionalDecodingFormat(DeserializationFormatFactory.class, FORMAT) + .get(); + } + + public static EncodingFormat> getValueEncodingFormat( + FactoryUtil.TableFactoryHelper helper) { + return helper.discoverOptionalEncodingFormat(SerializationFormatFactory.class, FORMAT) + .get(); + } + + /** + * Creates an array of indices that determine which physical fields of the table schema to + * include in the key format and the order that those fields have in the key format. + */ + public static int[] createKeyFormatProjection( + ReadableConfig options, DataType physicalDataType) { + final LogicalType physicalType = physicalDataType.getLogicalType(); + Preconditions.checkArgument( + physicalType.is(LogicalTypeRoot.ROW), "Row data type expected."); + final Optional optionalKeyFormat = options.getOptional(KEY_FORMAT); + final Optional> optionalKeyFields = options.getOptional(KEY_FIELDS); + + if (!optionalKeyFormat.isPresent()) { + return new int[0]; + } + + final List keyFields = optionalKeyFields.get(); + final List physicalFields = LogicalTypeChecks.getFieldNames(physicalType); + return keyFields.stream() + .mapToInt( + keyField -> { + final int pos = physicalFields.indexOf(keyField); + // check that field name exists + if (pos < 0) { + throw new ValidationException( + String.format( + "Could not find the field '%s' in the table schema for usage in the key format. " + + "A key field must be a regular, physical column. " + + "The following columns can be selected in the '%s' option: %s", + keyField, KEY_FIELDS.key(), physicalFields)); + } + // check that field name is prefixed correctly + return pos; + }) + .toArray(); + } + + public static int[] createValueFormatProjection( + ReadableConfig options, DataType physicalDataType) { + final LogicalType physicalType = physicalDataType.getLogicalType(); + Preconditions.checkArgument( + physicalType.is(LogicalTypeRoot.ROW), "Row data type expected."); + + final int physicalFieldCount = LogicalTypeChecks.getFieldCount(physicalType); + final IntStream physicalFields = IntStream.range(0, physicalFieldCount); + final int[] keyProjection = createKeyFormatProjection(options, physicalDataType); + return physicalFields + .filter(pos -> IntStream.of(keyProjection).noneMatch(k -> k == pos)) + .toArray(); + } + + // -------------------------------------------------------------------------------------------- + // Table Source Option Utils + // -------------------------------------------------------------------------------------------- + + public static List getTopicListFromOptions(ReadableConfig tableOptions) { + List topics = tableOptions.get(TOPICS); + return topics; + } + + public static Properties getPulsarProperties(ReadableConfig tableOptions) { + final Properties pulsarProperties = new Properties(); + final Map configs = ((Configuration) tableOptions).toMap(); + configs.keySet().stream() + .filter(key -> key.startsWith("pulsar")) + .forEach(key -> pulsarProperties.put(key, configs.get(key))); + return pulsarProperties; + } + + public static StartCursor getStartCursor(ReadableConfig tableOptions) { + if (tableOptions.getOptional(SOURCE_START_FROM_MESSAGE_ID).isPresent()) { + return parseMessageIdStartCursor(tableOptions.get(SOURCE_START_FROM_MESSAGE_ID)); + } else if (tableOptions.getOptional(SOURCE_START_FROM_PUBLISH_TIME).isPresent()) { + return parsePublishTimeStartCursor(tableOptions.get(SOURCE_START_FROM_PUBLISH_TIME)); + } else { + return StartCursor.earliest(); + } + } + + public static SubscriptionType getSubscriptionType(ReadableConfig tableOptions) { + return tableOptions.get(SOURCE_SUBSCRIPTION_TYPE); + } + + protected static StartCursor parseMessageIdStartCursor(String config) { + if (Objects.equals(config, "earliest")) { + return StartCursor.earliest(); + } else if (Objects.equals(config, "latest")) { + return StartCursor.latest(); + } else { + return parseMessageIdString(config); + } + } + + protected static StartCursor parseMessageIdString(String config) { + String[] tokens = config.split(":", 3); + if (tokens.length != 3) { + throw new IllegalArgumentException( + "MessageId format must be ledgerId:entryId:partitionId."); + } + try { + long ledgerId = Long.parseLong(tokens[0]); + long entryId = Long.parseLong(tokens[1]); + int partitionId = Integer.parseInt(tokens[2]); + MessageIdImpl messageId = new MessageIdImpl(ledgerId, entryId, partitionId); + return StartCursor.fromMessageId(messageId); + } catch (NumberFormatException e) { + throw new IllegalArgumentException( + "MessageId format must be ledgerId:entryId:partitionId. " + + "Each id should be able to parsed to long type."); + } + } + + protected static StartCursor parsePublishTimeStartCursor(Long config) { + return StartCursor.fromMessageTime(config); + } + + // -------------------------------------------------------------------------------------------- + // Table Sink Option Utils + // -------------------------------------------------------------------------------------------- + + public static TopicRouter getTopicRouter( + ReadableConfig readableConfig, ClassLoader classLoader) { + if (!readableConfig.getOptional(SINK_CUSTOM_TOPIC_ROUTER).isPresent()) { + return null; + } + + String className = readableConfig.get(SINK_CUSTOM_TOPIC_ROUTER); + try { + Class clazz = Class.forName(className, true, classLoader); + if (!TopicRouter.class.isAssignableFrom(clazz)) { + throw new ValidationException( + String.format( + "Sink TopicRouter class '%s' should extend from the required class %s", + className, TopicRouter.class.getName())); + } + @SuppressWarnings("unchecked") + final TopicRouter topicRouter = + InstantiationUtil.instantiate(className, TopicRouter.class, classLoader); + + return topicRouter; + } catch (ClassNotFoundException | FlinkException e) { + throw new ValidationException( + String.format( + "Could not find and instantiate TopicRouter class '%s'", className), + e); + } + } + + // TODO what if we use a topicRouter and set TopicRoutingMode to CUSTOM ? + public static TopicRoutingMode getTopicRoutingMode(ReadableConfig readableConfig) { + return readableConfig.get(SINK_TOPIC_ROUTING_MODE); + } + + public static long getMessageDelayMillis(ReadableConfig readableConfig) { + return readableConfig.get(SINK_MESSAGE_DELAY_INTERVAL).toMillis(); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java new file mode 100644 index 0000000000000..91b0b0a1e7fba --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java @@ -0,0 +1,216 @@ +/* + * 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.connector.pulsar.table; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.description.Description; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; + +import org.apache.pulsar.client.api.SubscriptionType; + +import java.time.Duration; +import java.util.List; + +import static org.apache.flink.configuration.description.TextElement.code; +import static org.apache.flink.configuration.description.TextElement.text; +import static org.apache.flink.table.factories.FactoryUtil.FORMAT_SUFFIX; + +/** + * Config options that is used to configure a Pulsar SQL Connector. These config options are + * specific to SQL Connectors only. Other runtime configurations can be found in {@link + * org.apache.flink.connector.pulsar.common.config.PulsarOptions}, {@link + * org.apache.flink.connector.pulsar.source.PulsarSourceOptions}, and {@link + * org.apache.flink.connector.pulsar.sink.PulsarSinkOptions}. + */ +public class PulsarTableOptions { + + private PulsarTableOptions() {} + + public static final ConfigOption> TOPICS = + ConfigOptions.key("topics") + .stringType() + .asList() + .noDefaultValue() + .withDescription( + "Topic names from which the table is read. It is required for both source and sink"); + + // -------------------------------------------------------------------------------------------- + // Table Source Options + // -------------------------------------------------------------------------------------------- + + public static final ConfigOption SOURCE_SUBSCRIPTION_TYPE = + ConfigOptions.key("source.subscription-type") + .enumType(SubscriptionType.class) + .defaultValue(SubscriptionType.Exclusive) + .withDescription( + "Subscription type for Pulsar source to use. Only \"Exclusive\" and \"Shared\" are allowed."); + + /** + * Exactly same as {@link + * org.apache.flink.connector.pulsar.source.PulsarSourceOptions#PULSAR_SUBSCRIPTION_NAME}. + * Copied because we want to have a default value for it. + */ + public static final ConfigOption SOURCE_SUBSCRIPTION_NAME = + ConfigOptions.key("source.subscription-name") + .stringType() + .defaultValue("flink-sql-connector-pulsar") + .withDescription( + Description.builder() + .text( + "Specify the subscription name consumer used by runtime PulsarSource.") + .text( + " This argument is required when constructing the consumer.") + .build()); + + public static final ConfigOption SOURCE_START_FROM_MESSAGE_ID = + ConfigOptions.key("source.start.message-id") + .stringType() + .noDefaultValue() + .withDescription( + "Optional message id used to specify a consuming starting point for " + + "source. Use \"earliest\", \"latest\" or pass in a message id " + + "representation in \"ledgerId:entryId:partitionId\", " + + "such as \"12:2:-1\""); + + public static final ConfigOption SOURCE_START_FROM_PUBLISH_TIME = + ConfigOptions.key("source.start.publish-time") + .longType() + .noDefaultValue() + .withDescription( + "Optional publish timestamp used to specify a consuming starting point for source."); + + // -------------------------------------------------------------------------------------------- + // Table Sink Options + // -------------------------------------------------------------------------------------------- + + public static final ConfigOption SINK_CUSTOM_TOPIC_ROUTER = + ConfigOptions.key("sink.custom-topic-router") + .stringType() + .noDefaultValue() + .withDescription( + "Optional custom TopicRouter implementation class URL to use in sink. If this option" + + "is provided, \"sink.topic-routing-mode\" will be ignored."); + + public static final ConfigOption SINK_TOPIC_ROUTING_MODE = + ConfigOptions.key("sink.topic-routing-mode") + .enumType(TopicRoutingMode.class) + .defaultValue(TopicRoutingMode.ROUND_ROBIN) + .withDescription( + "Optional TopicRoutingMode. There are \"round-robin\" and " + + "\"message-key-hash\" two options. Default use" + + "\"round-robin\", if you want to use a custom" + + "TopicRouter implementation, use \"sink.custom-topic-router\""); + + public static final ConfigOption SINK_MESSAGE_DELAY_INTERVAL = + ConfigOptions.key("sink.message-delay-interval") + .durationType() + .defaultValue(Duration.ZERO) + .withDescription("Optional sink message delay delivery interval."); + + // -------------------------------------------------------------------------------------------- + // Format Options + // -------------------------------------------------------------------------------------------- + + public static final ConfigOption KEY_FORMAT = + ConfigOptions.key("key" + FORMAT_SUFFIX) + .stringType() + .noDefaultValue() + .withDescription( + "Defines the format identifier for decoding/encoding key bytes in " + + "Pulsar message. The identifier is used to discover a suitable format factory."); + + public static final ConfigOption> KEY_FIELDS = + ConfigOptions.key("key.fields") + .stringType() + .asList() + .defaultValues() + .withDescription( + "Defines an explicit list of physical columns from the " + + "table schema which should be decoded/encoded " + + "from the key bytes of a Pulsar message. By default, " + + "this list is empty and thus a key is undefined."); + + // -------------------------------------------------------------------------------------------- + // Pulsar Options + // -------------------------------------------------------------------------------------------- + + /** + * Exactly same as {@link + * org.apache.flink.connector.pulsar.common.config.PulsarOptions#PULSAR_ADMIN_URL}. Copied here + * because it is a required config option and should not be included in the {@link + * org.apache.flink.table.factories.FactoryUtil.FactoryHelper#validateExcept(String...)} method. + * + *

By default all {@link org.apache.flink.connector.pulsar.common.config.PulsarOptions} are + * included in the validateExcept() method./p> + */ + public static final ConfigOption ADMIN_URL = + ConfigOptions.key("admin-url") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "The Pulsar service HTTP URL for the admin endpoint. For example, %s, or %s for TLS.", + code("http://my-broker.example.com:8080"), + code("https://my-broker.example.com:8443")) + .build()); + + /** + * Exactly same as {@link + * org.apache.flink.connector.pulsar.common.config.PulsarOptions#PULSAR_SERVICE_URL}. Copied + * here because it is a required config option and should not be included in the {@link + * org.apache.flink.table.factories.FactoryUtil.FactoryHelper#validateExcept(String...)} method. + * + *

By default all {@link org.apache.flink.connector.pulsar.common.config.PulsarOptions} are + * included in the validateExcept() method./p> + */ + public static final ConfigOption SERVICE_URL = + ConfigOptions.key("service-url") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text("Service URL provider for Pulsar service.") + .linebreak() + .text( + "To connect to Pulsar using client libraries, you need to specify a Pulsar protocol URL.") + .linebreak() + .text( + "You can assign Pulsar protocol URLs to specific clusters and use the Pulsar scheme.") + .linebreak() + .list( + text( + "This is an example of %s: %s.", + code("localhost"), + code("pulsar://localhost:6650")), + text( + "If you have multiple brokers, the URL is as: %s", + code( + "pulsar://localhost:6550,localhost:6651,localhost:6652")), + text( + "A URL for a production Pulsar cluster is as: %s", + code( + "pulsar://pulsar.us-west.example.com:6650")), + text( + "If you use TLS authentication, the URL is as %s", + code( + "pulsar+ssl://pulsar.us-west.example.com:6651"))) + .build()); +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java new file mode 100644 index 0000000000000..12c5ff72e47bc --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java @@ -0,0 +1,158 @@ +/* + * 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.connector.pulsar.table; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.types.RowKind; + +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getValueDecodingFormat; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FORMAT; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.pulsar.common.naming.TopicName.isValid; + +/** + * Util class for source and sink validation rules. TODO refactor to use the PulsarConfigValidator + */ +public class PulsarTableValidationUtils { + + private PulsarTableValidationUtils() {} + + public static void validatePrimaryKeyConstraints( + ObjectIdentifier tableName, + int[] primaryKeyIndexes, + Map options, + FactoryUtil.TableFactoryHelper helper) { + final DecodingFormat> format = + getValueDecodingFormat(helper); + if (primaryKeyIndexes.length > 0 + && format.getChangelogMode().containsOnly(RowKind.INSERT)) { + Configuration configuration = Configuration.fromMap(options); + String formatName = configuration.getOptional(FactoryUtil.FORMAT).get(); + throw new ValidationException( + String.format( + "The Pulsar table '%s' with '%s' format doesn't support defining PRIMARY KEY constraint" + + " on the table, because it can't guarantee the semantic of primary key.", + tableName.asSummaryString(), formatName)); + } + } + + public static void validateTableSourceOptions(ReadableConfig tableOptions) { + validateTopicsConfigs(tableOptions); + validateStartCursorConfigs(tableOptions); + validateSubscriptionTypeConfigs(tableOptions); + validateKeyFormatConfigs(tableOptions); + } + + public static void validateTableSinkOptions(ReadableConfig tableOptions) { + validateTopicsConfigs(tableOptions); + validateKeyFormatConfigs(tableOptions); + validateSinkRoutingConfigs(tableOptions); + } + + protected static void validateTopicsConfigs(ReadableConfig tableOptions) { + if (tableOptions.get(TOPICS).isEmpty()) { + throw new ValidationException("The topics list should not be empty."); + } + + for (String topic : tableOptions.get(TOPICS)) { + if (!isValid(topic)) { + throw new ValidationException( + String.format("The topics name %s is not a valid topic name.", topic)); + } + } + } + + protected static void validateStartCursorConfigs(ReadableConfig tableOptions) { + if (tableOptions.getOptional(SOURCE_START_FROM_MESSAGE_ID).isPresent() + && tableOptions.getOptional(SOURCE_START_FROM_PUBLISH_TIME).isPresent()) { + throw new ValidationException( + String.format( + "Only one of %s and %s can be specified. Detected both of them", + SOURCE_START_FROM_MESSAGE_ID, SOURCE_START_FROM_PUBLISH_TIME)); + } + } + + protected static void validateSubscriptionTypeConfigs(ReadableConfig tableOptions) { + SubscriptionType subscriptionType = tableOptions.get(SOURCE_SUBSCRIPTION_TYPE); + if (subscriptionType == SubscriptionType.Failover + || subscriptionType == SubscriptionType.Key_Shared) { + throw new ValidationException( + String.format( + "Only %s and %s SubscriptionType is supported. ", + SubscriptionType.Exclusive, SubscriptionType.Shared)); + } + } + + protected static void validateKeyFormatConfigs(ReadableConfig tableOptions) { + final Optional optionalKeyFormat = tableOptions.getOptional(KEY_FORMAT); + final Optional> optionalKeyFields = tableOptions.getOptional(KEY_FIELDS); + if (!optionalKeyFormat.isPresent() && optionalKeyFields.isPresent()) { + throw new ValidationException( + String.format( + "The option '%s' can only be declared if a key format is defined using '%s'.", + KEY_FIELDS.key(), KEY_FORMAT.key())); + } else if (optionalKeyFormat.isPresent() + && (!optionalKeyFields.isPresent() || optionalKeyFields.get().size() == 0)) { + throw new ValidationException( + String.format( + "A key format '%s' requires the declaration of one or more of key fields using '%s'.", + KEY_FORMAT.key(), KEY_FIELDS.key())); + } + } + + protected static void validateSinkRoutingConfigs(ReadableConfig tableOptions) { + if (tableOptions.getOptional(SINK_TOPIC_ROUTING_MODE).orElse(TopicRoutingMode.ROUND_ROBIN) + == TopicRoutingMode.CUSTOM) { + throw new ValidationException( + String.format( + "Only %s and %s can be used. For %s, please use sink.custom-topic-router for" + + "custom topic router and do not set this config.", + TopicRoutingMode.ROUND_ROBIN, + TopicRoutingMode.MESSAGE_KEY_HASH, + TopicRoutingMode.CUSTOM)); + } + if (tableOptions.getOptional(SINK_CUSTOM_TOPIC_ROUTER).isPresent() + && tableOptions.getOptional(SINK_TOPIC_ROUTING_MODE).isPresent()) { + throw new ValidationException( + String.format( + "Only one of %s and %s can be specified. Detected both of them", + SINK_CUSTOM_TOPIC_ROUTER, SINK_TOPIC_ROUTING_MODE)); + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java new file mode 100644 index 0000000000000..0f996193f160f --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java @@ -0,0 +1,110 @@ +/* + * Licensed 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.connector.pulsar.table.sink; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; +import org.apache.flink.connector.pulsar.sink.writer.context.PulsarSinkContext; +import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessage; +import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessageBuilder; +import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.RowKind; + +import org.apache.pulsar.client.api.Schema; + +import javax.annotation.Nullable; + +import java.io.Serializable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link PulsarSerializationSchema} implementation for Pulsar SQL sink connector. It is + * responsible for retrieving fields from Flink row and serialize into Pulsar message key or body, + * and set necessary metadata fields as required. + */ +public class PulsarTableSerializationSchema implements PulsarSerializationSchema { + + private static final long serialVersionUID = 7314442107082067836L; + + @Nullable private final SerializationSchema keySerialization; + + private final RowData.FieldGetter[] keyFieldGetters; + + private final SerializationSchema valueSerialization; + + private final RowData.FieldGetter[] valueFieldGetters; + + private final PulsarWritableMetadata writableMetadata; + + public PulsarTableSerializationSchema( + @Nullable SerializationSchema keySerialization, + RowData.FieldGetter[] keyFieldGetters, + SerializationSchema valueSerialization, + RowData.FieldGetter[] valueFieldGetters, + PulsarWritableMetadata writableMetadata) { + this.keySerialization = keySerialization; + this.keyFieldGetters = checkNotNull(keyFieldGetters); + this.valueSerialization = checkNotNull(valueSerialization); + this.valueFieldGetters = checkNotNull(valueFieldGetters); + this.writableMetadata = checkNotNull(writableMetadata); + } + + @Override + public void open( + SerializationSchema.InitializationContext initializationContext, + PulsarSinkContext sinkContext, + SinkConfiguration sinkConfiguration) + throws Exception { + valueSerialization.open(initializationContext); + } + + @Override + public PulsarMessage serialize(RowData consumedRow, PulsarSinkContext sinkContext) { + + PulsarMessageBuilder messageBuilder = new PulsarMessageBuilder<>(); + + final RowKind kind = consumedRow.getRowKind(); + final RowData valueRow = createProjectedRow(consumedRow, kind, valueFieldGetters); + + writableMetadata.applyWritableMetadataInMessage(consumedRow, messageBuilder); + + if (keySerialization != null) { + final RowData keyRow = createProjectedRow(consumedRow, RowKind.INSERT, keyFieldGetters); + messageBuilder.keyBytes(keySerialization.serialize(keyRow)); + } + + byte[] serializedData = valueSerialization.serialize(valueRow); + messageBuilder.value(Schema.BYTES, serializedData); + return messageBuilder.build(); + } + + private static RowData createProjectedRow( + RowData consumedRow, RowKind kind, RowData.FieldGetter[] fieldGetters) { + final int arity = fieldGetters.length; + final GenericRowData genericRowData = new GenericRowData(kind, arity); + for (int fieldPos = 0; fieldPos < arity; fieldPos++) { + genericRowData.setField(fieldPos, fieldGetters[fieldPos].getFieldOrNull(consumedRow)); + } + return genericRowData; + } + + /** A class to read fields from Flink row and map to a Pulsar metadata. */ + public interface MetadataConverter extends Serializable { + Object read(RowData consumedRow, int pos); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchemaFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchemaFactory.java new file mode 100644 index 0000000000000..fd3f7c252489f --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchemaFactory.java @@ -0,0 +1,153 @@ +/* + * Licensed 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.connector.pulsar.table.sink; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; +import org.apache.flink.table.connector.Projection; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.utils.DataTypeUtils; + +import javax.annotation.Nullable; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Contains needed field mapping and encoding format information to construct a {@link + * org.apache.flink.connector.pulsar.table.sink.PulsarTableSerializationSchema} instance. + */ +public class PulsarTableSerializationSchemaFactory { + + private final DataType physicalDataType; + + @Nullable private final EncodingFormat> keyEncodingFormat; + + private final int[] keyProjection; + + private final EncodingFormat> valueEncodingFormat; + + private final int[] valueProjection; + + /** Metadata that is appended at the end of a physical sink row. */ + private List writableMetadataKeys; + + public PulsarTableSerializationSchemaFactory( + DataType physicalDataType, + @Nullable EncodingFormat> keyEncodingFormat, + int[] keyProjection, + EncodingFormat> valueEncodingFormat, + int[] valueProjection) { + this.physicalDataType = checkNotNull(physicalDataType); + this.keyEncodingFormat = keyEncodingFormat; + this.keyProjection = checkNotNull(keyProjection); + this.valueEncodingFormat = checkNotNull(valueEncodingFormat); + this.valueProjection = checkNotNull(valueProjection); + this.writableMetadataKeys = Collections.emptyList(); + } + + public PulsarSerializationSchema createPulsarSerializationSchema( + DynamicTableSink.Context context) { + + final SerializationSchema keySerialization = + createSerialization(context, keyEncodingFormat, keyProjection, null); + + final SerializationSchema valueSerialization = + createSerialization(context, valueEncodingFormat, valueProjection, null); + final List physicalChildren = physicalDataType.getLogicalType().getChildren(); + + final RowData.FieldGetter[] keyFieldGetters = + getFieldGetters(physicalChildren, keyProjection); + final RowData.FieldGetter[] valueFieldGetters = + getFieldGetters(physicalChildren, valueProjection); + + final PulsarWritableMetadata writableMetadata = + new PulsarWritableMetadata(writableMetadataKeys, physicalChildren.size()); + + return new PulsarTableSerializationSchema( + keySerialization, + keyFieldGetters, + valueSerialization, + valueFieldGetters, + writableMetadata); + } + + private @Nullable SerializationSchema createSerialization( + DynamicTableSink.Context context, + @Nullable EncodingFormat> format, + int[] projection, + @Nullable String prefix) { + if (format == null) { + return null; + } + DataType physicalFormatDataType = Projection.of(projection).project(this.physicalDataType); + if (prefix != null) { + physicalFormatDataType = DataTypeUtils.stripRowPrefix(physicalFormatDataType, prefix); + } + return format.createRuntimeEncoder(context, physicalFormatDataType); + } + + private RowData.FieldGetter[] getFieldGetters( + List physicalChildren, int[] projection) { + return Arrays.stream(projection) + .mapToObj( + targetField -> + RowData.createFieldGetter( + physicalChildren.get(targetField), targetField)) + .toArray(RowData.FieldGetter[]::new); + } + + public void setWritableMetadataKeys(List writableMetadataKeys) { + this.writableMetadataKeys = writableMetadataKeys; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PulsarTableSerializationSchemaFactory that = (PulsarTableSerializationSchemaFactory) o; + return Objects.equals(physicalDataType, that.physicalDataType) + && Objects.equals(keyEncodingFormat, that.keyEncodingFormat) + && Arrays.equals(keyProjection, that.keyProjection) + && Objects.equals(valueEncodingFormat, that.valueEncodingFormat) + && Arrays.equals(valueProjection, that.valueProjection) + && Objects.equals(writableMetadataKeys, that.writableMetadataKeys); + } + + @Override + public int hashCode() { + int result = + Objects.hash( + physicalDataType, + keyEncodingFormat, + valueEncodingFormat, + writableMetadataKeys); + result = 31 * result + Arrays.hashCode(keyProjection); + result = 31 * result + Arrays.hashCode(valueProjection); + return result; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSink.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSink.java new file mode 100644 index 0000000000000..b290967873e5f --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSink.java @@ -0,0 +1,171 @@ +/* + * Licensed 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.connector.pulsar.table.sink; + +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.pulsar.sink.PulsarSink; +import org.apache.flink.connector.pulsar.sink.PulsarSinkBuilder; +import org.apache.flink.connector.pulsar.sink.writer.delayer.FixedMessageDelayer; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkV2Provider; +import org.apache.flink.table.connector.sink.abilities.SupportsWritingMetadata; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; + +import javax.annotation.Nullable; + +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.stream.Stream; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Pulsar SQL Connector sink. It supports {@link SupportsWritingMetadata}. */ +public class PulsarTableSink implements DynamicTableSink, SupportsWritingMetadata { + + private final PulsarTableSerializationSchemaFactory serializationSchemaFactory; + + private final ChangelogMode changelogMode; + + private final List topics; + + private final Properties properties; + + private final DeliveryGuarantee deliveryGuarantee; + + @Nullable private final TopicRouter topicRouter; + + private final TopicRoutingMode topicRoutingMode; + + private final long messageDelayMillis; + + public PulsarTableSink( + PulsarTableSerializationSchemaFactory serializationSchemaFactory, + ChangelogMode changelogMode, + List topics, + Properties properties, + DeliveryGuarantee deliveryGuarantee, + @Nullable TopicRouter topicRouter, + TopicRoutingMode topicRoutingMode, + long messageDelayMillis) { + this.serializationSchemaFactory = checkNotNull(serializationSchemaFactory); + this.changelogMode = checkNotNull(changelogMode); + this.topics = checkNotNull(topics); + // Mutable attributes + this.properties = checkNotNull(properties); + this.deliveryGuarantee = checkNotNull(deliveryGuarantee); + this.topicRouter = topicRouter; + this.topicRoutingMode = checkNotNull(topicRoutingMode); + this.messageDelayMillis = messageDelayMillis; + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + return this.changelogMode; + } + + @Override + public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { + + final PulsarSerializationSchema pulsarSerializationSchema = + serializationSchemaFactory.createPulsarSerializationSchema(context); + + final PulsarSinkBuilder pulsarSinkBuilder = + PulsarSink.builder() + .setSerializationSchema(pulsarSerializationSchema) + .setProperties(properties) + .setDeliveryGuarantee(deliveryGuarantee) + .setTopics(topics) + .setTopicRoutingMode(topicRoutingMode) + .delaySendingMessage(new FixedMessageDelayer<>(messageDelayMillis)); + + if (topicRouter != null) { + pulsarSinkBuilder.setTopicRouter(topicRouter); + } + return SinkV2Provider.of(pulsarSinkBuilder.build()); + } + + @Override + public String asSummaryString() { + return "Pulsar dynamic table sink"; + } + + @Override + public Map listWritableMetadata() { + final Map metadataMap = new LinkedHashMap<>(); + Stream.of(PulsarWritableMetadata.WritableMetadata.values()) + .forEachOrdered(m -> metadataMap.put(m.key, m.dataType)); + return metadataMap; + } + + @Override + public void applyWritableMetadata(List metadataKeys, DataType consumedDataType) { + this.serializationSchemaFactory.setWritableMetadataKeys(metadataKeys); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PulsarTableSink that = (PulsarTableSink) o; + return Objects.equals(serializationSchemaFactory, that.serializationSchemaFactory) + && Objects.equals(changelogMode, that.changelogMode) + && Objects.equals(topics, that.topics) + && Objects.equals(properties, that.properties) + && deliveryGuarantee == that.deliveryGuarantee + && Objects.equals(topicRouter, that.topicRouter) + && topicRoutingMode == that.topicRoutingMode + && messageDelayMillis == that.messageDelayMillis; + } + + @Override + public int hashCode() { + return Objects.hash( + serializationSchemaFactory, + changelogMode, + topics, + properties, + deliveryGuarantee, + topicRouter, + topicRoutingMode, + messageDelayMillis); + } + + @Override + public DynamicTableSink copy() { + final PulsarTableSink copy = + new PulsarTableSink( + serializationSchemaFactory, + changelogMode, + topics, + properties, + deliveryGuarantee, + topicRouter, + topicRoutingMode, + messageDelayMillis); + return copy; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarWritableMetadata.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarWritableMetadata.java new file mode 100644 index 0000000000000..8f25db73ec442 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarWritableMetadata.java @@ -0,0 +1,140 @@ +/* + * Licensed 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.connector.pulsar.table.sink; + +import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessageBuilder; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; + +import org.bouncycastle.util.Arrays; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.stream.Stream; + +/** A class used to manage WritableMetadata for Pulsar SQL sink connector. */ +public class PulsarWritableMetadata implements Serializable { + + private static final long serialVersionUID = 8117156158379846715L; + + private final List writableMetadataKeys; + + private final int physicalChildrenSize; + + /** + * Contains the position for each value of {@link WritableMetadata} in the consumed row or -1 if + * this metadata key is not used. + */ + private int[] metadataPositions; + + public PulsarWritableMetadata(List writableMetadataKeys, int physicalChildrenSize) { + this.writableMetadataKeys = writableMetadataKeys; + this.physicalChildrenSize = physicalChildrenSize; + this.metadataPositions = getMetadataPositions(); + } + + public void applyWritableMetadataInMessage( + RowData consumedRow, PulsarMessageBuilder messageBuilder) { + Map properties = readMetadata(consumedRow, WritableMetadata.PROPERTIES); + if (properties != null) { + messageBuilder.properties(properties); + } + final Long eventTime = readMetadata(consumedRow, WritableMetadata.EVENT_TIME); + if (eventTime != null && eventTime >= 0) { + messageBuilder.eventTime(eventTime); + } + } + + @SuppressWarnings("unchecked") + private T readMetadata(RowData consumedRow, WritableMetadata metadata) { + if (Arrays.isNullOrEmpty(metadataPositions)) { + return null; + } + final int pos = metadataPositions[metadata.ordinal()]; + if (pos < 0) { + return null; + } + return (T) metadata.converter.read(consumedRow, pos); + } + + private int[] getMetadataPositions() { + return Stream.of(PulsarWritableMetadata.WritableMetadata.values()) + .mapToInt( + m -> { + final int pos = writableMetadataKeys.indexOf(m.key); + if (pos < 0) { + return -1; + } + return physicalChildrenSize + pos; + }) + .toArray(); + } + + /** A list of writable metadata used by Pulsar SQL sink connector. */ + public enum WritableMetadata { + PROPERTIES( + "properties", + // key and value of the map are nullable to make handling easier in queries + DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.STRING().nullable()) + .nullable(), + (row, pos) -> { + if (row.isNullAt(pos)) { + return null; + } + final MapData map = row.getMap(pos); + final ArrayData keyArray = map.keyArray(); + final ArrayData valueArray = map.valueArray(); + + final Properties properties = new Properties(); + for (int i = 0; i < keyArray.size(); i++) { + if (!keyArray.isNullAt(i) && !valueArray.isNullAt(i)) { + final String key = keyArray.getString(i).toString(); + final String value = valueArray.getString(i).toString(); + properties.put(key, value); + } + } + return properties; + }), + + EVENT_TIME( + "event_time", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), + (row, pos) -> { + if (row.isNullAt(pos)) { + return null; + } + return row.getTimestamp(pos, 3).getMillisecond(); + }); + public final String key; + + public final DataType dataType; + + public final PulsarTableSerializationSchema.MetadataConverter converter; + + WritableMetadata( + String key, + DataType dataType, + PulsarTableSerializationSchema.MetadataConverter converter) { + this.key = key; + this.dataType = dataType; + this.converter = converter; + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarReadableMetadata.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarReadableMetadata.java new file mode 100644 index 0000000000000..e0a7f9dc77557 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarReadableMetadata.java @@ -0,0 +1,145 @@ +/* + * 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.connector.pulsar.table.source; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; + +import org.apache.pulsar.client.api.Message; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Class for reading metadata fields from a Pulsar message and put in corresponding Flink row + * fields. + * + *

Contains list of readable metadata and provide util methods for metadata manipulation. + */ +public class PulsarReadableMetadata implements Serializable { + + private static final long serialVersionUID = -4409932324481235973L; + + private final List connectorMetadataKeys; + + private final List metadataConverters; + + public PulsarReadableMetadata(List connectorMetadataKeys) { + this.connectorMetadataKeys = connectorMetadataKeys; + this.metadataConverters = initializeMetadataConverters(); + } + + private List initializeMetadataConverters() { + return connectorMetadataKeys.stream() + .map( + k -> + Stream.of(ReadableMetadata.values()) + .filter(rm -> rm.key.equals(k)) + .findFirst() + .orElseThrow(IllegalStateException::new)) + .map(m -> m.converter) + .collect(Collectors.toList()); + } + + public void appendProducedRowWithMetadata( + GenericRowData producedRowData, int physicalArity, Message message) { + for (int metadataPos = 0; metadataPos < metadataConverters.size(); metadataPos++) { + producedRowData.setField( + physicalArity + metadataPos, metadataConverters.get(metadataPos).read(message)); + } + } + + public int getConnectorMetadataArity() { + return metadataConverters.size(); + } + + // -------------------------------------------------------------------------------------------- + // Metadata handling + // -------------------------------------------------------------------------------------------- + interface MetadataConverter extends Serializable { + Object read(Message message); + } + + /** Lists the metadata that is readable from a Pulsar message. Used in SQL source connector. */ + public enum ReadableMetadata { + TOPIC( + "topic", + DataTypes.STRING().notNull(), + message -> StringData.fromString(message.getTopicName())), + + MESSAGE_SIZE("message_size", DataTypes.INT().notNull(), message -> message.size()), + + PRODUCER_NAME( + "producer_name", + DataTypes.STRING().notNull(), + message -> StringData.fromString(message.getProducerName())), + + MESSAGE_ID( + "message_id", + DataTypes.BYTES().notNull(), + message -> message.getMessageId().toByteArray()), + + SEQUENCE_ID("sequenceId", DataTypes.BIGINT().notNull(), Message::getSequenceId), + + PUBLISH_TIME( + "publish_time", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull(), + message -> TimestampData.fromEpochMillis(message.getPublishTime())), + + EVENT_TIME( + "event_time", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull(), + message -> TimestampData.fromEpochMillis(message.getEventTime())), + + PROPERTIES( + "properties", + // key and value of the map are nullable to make handling easier in queries + DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.STRING().nullable()) + .notNull(), + message -> { + final Map map = new HashMap<>(); + for (Map.Entry e : message.getProperties().entrySet()) { + map.put( + StringData.fromString(e.getKey()), + StringData.fromString(e.getValue())); + } + return new GenericMapData(map); + }); + + public final String key; + + public final DataType dataType; + + public final MetadataConverter converter; + + ReadableMetadata(String key, DataType dataType, MetadataConverter converter) { + this.key = key; + this.dataType = dataType; + this.converter = converter; + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarRowDataConverter.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarRowDataConverter.java new file mode 100644 index 0000000000000..5f8063669d6e7 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarRowDataConverter.java @@ -0,0 +1,118 @@ +/* + * 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.connector.pulsar.table.source; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.DeserializationException; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.Collector; + +import org.apache.pulsar.client.api.Message; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.List; + +/** + * Contains the projection information needed to map a Pulsar message to proper key fields, value + * fields and metadata fields. + */ +public class PulsarRowDataConverter implements Serializable { + private static final long serialVersionUID = -3399264407634977459L; + + private final int physicalArity; + + private final int[] keyProjection; + + private final int[] valueProjection; + + private final PulsarReadableMetadata readableMetadata; + + public PulsarRowDataConverter( + int physicalArity, + int[] keyProjection, + int[] valueProjection, + PulsarReadableMetadata readableMetadata) { + this.physicalArity = physicalArity; + this.keyProjection = keyProjection; + this.valueProjection = valueProjection; + this.readableMetadata = readableMetadata; + } + + public void projectToProducedRowAndCollect( + Message message, + List keyRowDataList, + List valueRowDataList, + Collector collector) { + // no key defined + if (hasNoKeyProjection()) { + valueRowDataList.forEach( + valueRow -> emitRow(null, (GenericRowData) valueRow, collector, message)); + } else { + // otherwise emit a value for each key + valueRowDataList.forEach( + valueRow -> + keyRowDataList.forEach( + keyRow -> + emitRow( + (GenericRowData) keyRow, + (GenericRowData) valueRow, + collector, + message))); + } + } + + private void emitRow( + @Nullable GenericRowData physicalKeyRow, + @Nullable GenericRowData physicalValueRow, + Collector collector, + Message message) { + + final RowKind rowKind; + if (physicalValueRow == null) { + throw new DeserializationException( + "Invalid null value received in non-upsert mode. Could not to set row kind for output record." + + "upsert mode is not supported yet."); + } else { + rowKind = physicalValueRow.getRowKind(); + } + + final GenericRowData producedRow = + new GenericRowData( + rowKind, physicalArity + readableMetadata.getConnectorMetadataArity()); + + for (int valuePos = 0; valuePos < valueProjection.length; valuePos++) { + producedRow.setField(valueProjection[valuePos], physicalValueRow.getField(valuePos)); + } + + for (int keyPos = 0; keyPos < keyProjection.length; keyPos++) { + assert physicalKeyRow != null; + producedRow.setField(keyProjection[keyPos], physicalKeyRow.getField(keyPos)); + } + + readableMetadata.appendProducedRowWithMetadata(producedRow, physicalArity, message); + collector.collect(producedRow); + } + + private boolean hasNoKeyProjection() { + return keyProjection.length == 0; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchema.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchema.java new file mode 100644 index 0000000000000..6c4d4caa1e3f8 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchema.java @@ -0,0 +1,109 @@ +/* + * 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.connector.pulsar.table.source; + +import org.apache.flink.api.common.functions.util.ListCollector; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.Collector; + +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.Schema; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A specific {@link PulsarDeserializationSchema} for {@link PulsarTableSource}. + * + *

Both Flink's key decoding format and value decoding format are wrapped in this class. It is + * responsible for getting metadata fields from a physical pulsar message body, and the final + * projection mapping from Pulsar message fields to Flink row. + * + *

After retrieving key and value bytes and convert them into a list of {@link RowData}, it then + * delegates metadata appending, key and value {@link RowData} combining to a {@link + * PulsarRowDataConverter} instance. + */ +public class PulsarTableDeserializationSchema implements PulsarDeserializationSchema { + + private static final long serialVersionUID = -3298784447432136216L; + + private final TypeInformation producedTypeInfo; + + @Nullable private final DeserializationSchema keyDeserialization; + + private final DeserializationSchema valueDeserialization; + + private final PulsarRowDataConverter rowDataConverter; + + public PulsarTableDeserializationSchema( + @Nullable DeserializationSchema keyDeserialization, + DeserializationSchema valueDeserialization, + TypeInformation producedTypeInfo, + PulsarRowDataConverter rowDataConverter) { + this.keyDeserialization = keyDeserialization; + this.valueDeserialization = checkNotNull(valueDeserialization); + this.rowDataConverter = checkNotNull(rowDataConverter); + this.producedTypeInfo = checkNotNull(producedTypeInfo); + } + + @Override + public void open( + DeserializationSchema.InitializationContext context, SourceConfiguration configuration) + throws Exception { + if (keyDeserialization != null) { + keyDeserialization.open(context); + } + valueDeserialization.open(context); + } + + @Override + public void deserialize(Message message, Collector collector) throws IOException { + // Get the value row data + List valueRowData = new ArrayList<>(); + valueDeserialization.deserialize(message.getData(), new ListCollector<>(valueRowData)); + + // Get the key row data + List keyRowData = new ArrayList<>(); + if (keyDeserialization != null) { + keyDeserialization.deserialize(message.getKeyBytes(), new ListCollector<>(keyRowData)); + } + + rowDataConverter.projectToProducedRowAndCollect( + message, keyRowData, valueRowData, collector); + } + + @Override + public TypeInformation getProducedType() { + return producedTypeInfo; + } + + @Override + public Schema schema() { + return Schema.BYTES; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchemaFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchemaFactory.java new file mode 100644 index 0000000000000..b583cf667df9a --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchemaFactory.java @@ -0,0 +1,204 @@ +/* + * 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.connector.pulsar.table.source; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; +import org.apache.flink.table.connector.Projection; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.utils.DataTypeUtils; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.stream.IntStream; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Contains key, value projection and format information, and use such information to create a + * {@link org.apache.flink.connector.pulsar.table.source.PulsarTableDeserializationSchema} instance + * used by runtime {@link org.apache.flink.connector.pulsar.source.PulsarSource} instance. + * + *

A Flink row fields has a strict order: Physical Fields (Key + value) + Format Metadata Fields + * Connector Metadata Fields. Physical Fields are fields come directly from Pulsar message body; + * Format Metadata Fields are from the extra information from the decoding format. Connector + * metadata fields are the ones most Pulsar messages have, such as publish time, message size and + * producer name. + * + *

In general, Physical fields + Format Metadata fields are contained in the RowData decoded + * using valueDecodingFormat. Only Connector Metadata fields needs to be appended to the decoded + * RowData. The tricky part is to put format metadata and connector metadata in the right location. + * This requires an explicit adjustment process. + * + *

For example, suppose Physical Fields (Key + value) + Format Metadata Fields + Connector + * Metadata Fields. has arity of 11, key projection is [0, 6], and physical value projection is [1, + * 2, 3, 4, 5], Then after the adjustment, key projection should be [0, 6], physical value + * projection should be [1, 2, 3, 4, 5] and format metadata projection should be [7], connector + * metadata projection should be [8, 9, 10]. + */ +public class PulsarTableDeserializationSchemaFactory implements Serializable { + + private static final long serialVersionUID = 6091562041940740434L; + + private final DataType physicalDataType; + + @Nullable private final DecodingFormat> keyDecodingFormat; + + private final int[] keyProjection; + + private final DecodingFormat> valueDecodingFormat; + + private final int[] valueProjection; + + // -------------------------------------------------------------------------------------------- + // Mutable attributes. Will be updated after the applyReadableMetadata() + // -------------------------------------------------------------------------------------------- + private DataType producedDataType; + + private List connectorMetadataKeys; + + public PulsarTableDeserializationSchemaFactory( + DataType physicalDataType, + @Nullable DecodingFormat> keyDecodingFormat, + int[] keyProjection, + DecodingFormat> valueDecodingFormat, + int[] valueProjection) { + this.physicalDataType = + checkNotNull(physicalDataType, "field physicalDataType must not be null."); + this.keyDecodingFormat = keyDecodingFormat; + this.keyProjection = checkNotNull(keyProjection); + this.valueDecodingFormat = + checkNotNull(valueDecodingFormat, "field valueDecodingFormat must not be null."); + this.valueProjection = + checkNotNull(valueProjection, "field valueProjection must not be null."); + + this.producedDataType = physicalDataType; + this.connectorMetadataKeys = Collections.emptyList(); + } + + private @Nullable DeserializationSchema createDeserialization( + DynamicTableSource.Context context, + @Nullable DecodingFormat> format, + int[] projection, + @Nullable String prefix) { + if (format == null) { + return null; + } + + DataType physicalFormatDataType = Projection.of(projection).project(this.physicalDataType); + if (prefix != null) { + physicalFormatDataType = DataTypeUtils.stripRowPrefix(physicalFormatDataType, prefix); + } + return format.createRuntimeDecoder(context, physicalFormatDataType); + } + + public PulsarDeserializationSchema createPulsarDeserialization( + ScanTableSource.ScanContext context) { + final DeserializationSchema keyDeserialization = + createDeserialization(context, keyDecodingFormat, keyProjection, ""); + final DeserializationSchema valueDeserialization = + createDeserialization(context, valueDecodingFormat, valueProjection, ""); + + final TypeInformation producedTypeInfo = + context.createTypeInformation(producedDataType); + + final PulsarReadableMetadata readableMetadata = + new PulsarReadableMetadata(connectorMetadataKeys); + + // Get Physical Fields (key + value) + Format Metadata arity + final int physicalPlusFormatMetadataArity = + DataType.getFieldDataTypes(producedDataType).size() + - readableMetadata.getConnectorMetadataArity(); + final int[] physicalValuePlusFormatMetadataProjection = + adjustValueProjectionByAppendConnectorMetadata(physicalPlusFormatMetadataArity); + + final PulsarRowDataConverter rowDataConverter = + new PulsarRowDataConverter( + physicalPlusFormatMetadataArity, + keyProjection, + physicalValuePlusFormatMetadataProjection, + readableMetadata); + + return new PulsarTableDeserializationSchema( + keyDeserialization, valueDeserialization, producedTypeInfo, rowDataConverter); + } + + public void setProducedDataType(DataType producedDataType) { + this.producedDataType = producedDataType; + } + + public void setConnectorMetadataKeys(List metadataKeys) { + this.connectorMetadataKeys = metadataKeys; + } + + private int[] adjustValueProjectionByAppendConnectorMetadata( + int physicalValuePlusFormatMetadataArity) { + // Concat the Physical Fields (value only) with Format metadata projection. + final int[] physicalValuePlusFormatMetadataProjection = + IntStream.concat( + IntStream.of(valueProjection), + IntStream.range( + keyProjection.length + valueProjection.length, + physicalValuePlusFormatMetadataArity)) + .toArray(); + return physicalValuePlusFormatMetadataProjection; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PulsarTableDeserializationSchemaFactory that = (PulsarTableDeserializationSchemaFactory) o; + return Objects.equals(physicalDataType, that.physicalDataType) + && Objects.equals(keyDecodingFormat, that.keyDecodingFormat) + && Arrays.equals(keyProjection, that.keyProjection) + && Objects.equals(valueDecodingFormat, that.valueDecodingFormat) + && Arrays.equals(valueProjection, that.valueProjection) + && Objects.equals(producedDataType, that.producedDataType) + && Objects.equals(connectorMetadataKeys, that.connectorMetadataKeys); + } + + @Override + public int hashCode() { + int result = + Objects.hash( + physicalDataType, + keyDecodingFormat, + valueDecodingFormat, + producedDataType, + connectorMetadataKeys); + result = 31 * result + Arrays.hashCode(keyProjection); + result = 31 * result + Arrays.hashCode(valueProjection); + return result; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableSource.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableSource.java new file mode 100644 index 0000000000000..b8ba6d401f9b4 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableSource.java @@ -0,0 +1,213 @@ +/* + * 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.connector.pulsar.table.source; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.connector.pulsar.source.PulsarSource; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.SourceProvider; +import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; + +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link ScanTableSource} implementation for Pulsar SQL Connector. It uses a {@link + * SourceProvider} so it doesn't need to support {@link + * org.apache.flink.table.connector.source.abilities.SupportsWatermarkPushDown} interface. + * + *

{@link PulsarTableSource} + */ +public class PulsarTableSource implements ScanTableSource, SupportsReadingMetadata { + // -------------------------------------------------------------------------------------------- + // Format attributes + // -------------------------------------------------------------------------------------------- + + private static final String FORMAT_METADATA_PREFIX = "value."; + + private final PulsarTableDeserializationSchemaFactory deserializationSchemaFactory; + + /** + * Usually it is the same as the valueDecodingFormat, but use a different naming to show that it + * is used to list all the format metadata keys. + */ + private final DecodingFormat> decodingFormatForReadingMetadata; + + // -------------------------------------------------------------------------------------------- + // PulsarSource needed attributes + // -------------------------------------------------------------------------------------------- + + private final List topics; + + private final Properties properties; + + private final StartCursor startCursor; + + private final SubscriptionType subscriptionType; + + public PulsarTableSource( + PulsarTableDeserializationSchemaFactory deserializationSchemaFactory, + DecodingFormat> decodingFormatForReadingMetadata, + List topics, + Properties properties, + StartCursor startCursor, + SubscriptionType subscriptionType) { + // Format attributes + this.deserializationSchemaFactory = checkNotNull(deserializationSchemaFactory); + this.decodingFormatForReadingMetadata = checkNotNull(decodingFormatForReadingMetadata); + // DataStream connector attributes + this.topics = topics; + this.properties = checkNotNull(properties); + this.startCursor = checkNotNull(startCursor); + this.subscriptionType = checkNotNull(subscriptionType); + } + + @Override + public ChangelogMode getChangelogMode() { + return decodingFormatForReadingMetadata.getChangelogMode(); + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext context) { + PulsarDeserializationSchema deserializationSchema = + deserializationSchemaFactory.createPulsarDeserialization(context); + PulsarSource source = + PulsarSource.builder() + .setTopics(topics) + .setStartCursor(startCursor) + .setDeserializationSchema(deserializationSchema) + .setSubscriptionType(subscriptionType) + .setProperties(properties) + .build(); + return SourceProvider.of(source); + } + + /** + * According to convention, the order of the final row must be PHYSICAL + FORMAT METADATA + + * CONNECTOR METADATA where the format metadata has the highest precedence. + * + * @return + */ + @Override + public Map listReadableMetadata() { + final Map allMetadataMap = new LinkedHashMap<>(); + + // add value format metadata with prefix + decodingFormatForReadingMetadata + .listReadableMetadata() + .forEach((key, value) -> allMetadataMap.put(FORMAT_METADATA_PREFIX + key, value)); + // add connector metadata + Stream.of(PulsarReadableMetadata.ReadableMetadata.values()) + .forEachOrdered(m -> allMetadataMap.putIfAbsent(m.key, m.dataType)); + + return allMetadataMap; + } + + @Override + public void applyReadableMetadata(List allMetadataKeys, DataType producedDataType) { + // separate connector and format metadata + final List formatMetadataKeys = + allMetadataKeys.stream() + .filter(k -> k.startsWith(FORMAT_METADATA_PREFIX)) + .collect(Collectors.toList()); + + final List connectorMetadataKeys = new ArrayList<>(allMetadataKeys); + connectorMetadataKeys.removeAll(formatMetadataKeys); + + // push down format metadata + final Map formatMetadata = + decodingFormatForReadingMetadata.listReadableMetadata(); + if (formatMetadata.size() > 0) { + final List requestedFormatMetadataKeys = + formatMetadataKeys.stream() + .map(k -> k.substring(FORMAT_METADATA_PREFIX.length())) + .collect(Collectors.toList()); + decodingFormatForReadingMetadata.applyReadableMetadata(requestedFormatMetadataKeys); + } + + // update the factory attributes. + deserializationSchemaFactory.setConnectorMetadataKeys(connectorMetadataKeys); + deserializationSchemaFactory.setProducedDataType(producedDataType); + } + + @Override + public String asSummaryString() { + return "Pulsar table source"; + } + + @Override + public DynamicTableSource copy() { + final PulsarTableSource copy = + new PulsarTableSource( + deserializationSchemaFactory, + decodingFormatForReadingMetadata, + topics, + properties, + startCursor, + subscriptionType); + return copy; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + PulsarTableSource that = (PulsarTableSource) o; + return Objects.equals(deserializationSchemaFactory, that.deserializationSchemaFactory) + && Objects.equals( + decodingFormatForReadingMetadata, that.decodingFormatForReadingMetadata) + && Objects.equals(topics, that.topics) + && Objects.equals(properties, that.properties) + && Objects.equals(startCursor, that.startCursor) + && subscriptionType == that.subscriptionType; + } + + @Override + public int hashCode() { + return Objects.hash( + deserializationSchemaFactory, + decodingFormatForReadingMetadata, + topics, + properties, + startCursor, + subscriptionType); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connectors/flink-connector-pulsar/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 0000000000000..b153b56175670 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -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.connector.pulsar.table.PulsarTableFactory diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java index 7565193ca9a42..b99a631d92548 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java @@ -34,7 +34,6 @@ ImportOptions.ExcludeShadedImportOption.class }) public class TestCodeArchitectureTest { - @ArchTest public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class); } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactoryTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactoryTest.java index e4b2dc1b9e9cf..d67b64ceac9e7 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactoryTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactoryTest.java @@ -20,9 +20,9 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.connector.pulsar.common.schema.PulsarSchema; -import org.apache.flink.connector.pulsar.common.schema.PulsarSchemaTypeInformation; import org.apache.flink.connector.pulsar.testutils.SampleData.FL; import org.apache.flink.connector.pulsar.testutils.SampleData.Foo; +import org.apache.flink.formats.avro.typeutils.AvroTypeInfo; import org.apache.flink.util.InstantiationUtil; import org.apache.pulsar.client.api.Schema; @@ -62,7 +62,7 @@ void createJSONTypeInformationFromSchemaInfo() { TypeInformation typeInfo = factory.createTypeInfo(pulsarSchema.getSchemaInfo()); assertThat(typeInfo) - .isInstanceOf(PulsarSchemaTypeInformation.class) + .isInstanceOf(AvroTypeInfo.class) .hasFieldOrPropertyWithValue("typeClass", FL.class); // TypeInformation serialization. diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java index 9a72c8ace2a19..0259773445ee1 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java @@ -32,14 +32,15 @@ import org.apache.flink.connector.testframe.junit.annotations.TestSemantics; import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase; import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.test.junit5.MiniClusterExtension; -import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.extension.ExtendWith; /** Unite test class for {@link PulsarSource}. */ @SuppressWarnings("unused") +@ExtendWith(MiniClusterExtension.class) class PulsarSourceITCase extends SourceTestSuiteBase { - @Disabled // TODO: remove override after FLINK-26177 is fixed @Override public void testScaleUp( TestEnvironment testEnv, @@ -49,7 +50,6 @@ public void testScaleUp( super.testScaleUp(testEnv, externalContext, semantic); } - @Disabled // TODO: remove override after FLINK-26177 is fixed @Override public void testScaleDown( TestEnvironment testEnv, diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaTest.java index 18888df64b52a..3478b5c235c0f 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaTest.java @@ -18,13 +18,21 @@ package org.apache.flink.connector.pulsar.source.reader.deserializer; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.serialization.SimpleStringSchema; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.connector.pulsar.SampleMessage.TestMessage; +import org.apache.flink.connector.pulsar.source.PulsarSource; +import org.apache.flink.connector.pulsar.source.PulsarSourceOptions; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; +import org.apache.flink.connector.pulsar.testutils.PulsarTestSuiteBase; import org.apache.flink.connector.testutils.source.deserialization.TestingDeserializationContext; import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.StringValue; +import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.Collector; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.function.FunctionWithException; @@ -33,9 +41,12 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.common.api.proto.MessageMetadata; +import org.apache.pulsar.common.schema.KeyValue; import org.junit.jupiter.api.Test; +import java.io.Serializable; import java.nio.ByteBuffer; +import java.util.Objects; import java.util.concurrent.ThreadLocalRandom; import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; @@ -44,13 +55,16 @@ import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.pulsarSchema; import static org.apache.flink.util.Preconditions.checkState; import static org.apache.pulsar.client.api.Schema.PROTOBUF_NATIVE; +import static org.apache.pulsar.client.api.SubscriptionType.Exclusive; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.mockito.Mockito.mock; /** Unit tests for {@link PulsarDeserializationSchema}. */ -class PulsarDeserializationSchemaTest { +class PulsarDeserializationSchemaTest extends PulsarTestSuiteBase { @Test void createFromFlinkDeserializationSchema() throws Exception { @@ -108,6 +122,243 @@ void createFromFlinkTypeInformation() throws Exception { assertEquals(collector.result, "test-content"); } + @Test + void primitiveStringPulsarSchema() { + final String topicName = + "primitiveString-" + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + String expectedMessage = randomAlphabetic(10); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.STRING, + expectedMessage); + PulsarSource source = createSource(topicName, pulsarSchema(Schema.STRING)); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + @Test + void unversionedJsonStructPulsarSchema() { + final String topicName = + "unversionedJsonStruct-" + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + TestingUser expectedMessage = createRandomUser(); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.JSON(TestingUser.class), + expectedMessage); + PulsarSource source = + createSource( + topicName, pulsarSchema(Schema.JSON(TestingUser.class), TestingUser.class)); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + @Test + void keyValueJsonStructPulsarSchema() { + final String topicName = + "keyValueJsonStruct-" + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + KeyValue expectedMessage = + new KeyValue<>(createRandomUser(), createRandomUser()); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.KeyValue( + Schema.JSON(TestingUser.class), Schema.JSON(TestingUser.class)), + expectedMessage); + PulsarSource> source = + createSource( + topicName, + pulsarSchema( + Schema.KeyValue( + Schema.JSON(TestingUser.class), + Schema.JSON(TestingUser.class)), + TestingUser.class, + TestingUser.class)); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + @Test + void keyValueAvroStructPulsarSchema() { + final String topicName = + "keyValueAvroStruct-" + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + KeyValue expectedMessage = + new KeyValue<>(createRandomUser(), createRandomUser()); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.KeyValue( + Schema.AVRO(TestingUser.class), Schema.AVRO(TestingUser.class)), + expectedMessage); + PulsarSource> source = + createSource( + topicName, + pulsarSchema( + Schema.KeyValue( + Schema.AVRO(TestingUser.class), + Schema.AVRO(TestingUser.class)), + TestingUser.class, + TestingUser.class)); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + @Test + void keyValuePrimitivePulsarSchema() { + final String topicName = + "keyValuePrimitive-" + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + KeyValue expectedMessage = new KeyValue<>(randomAlphabetic(5), 5); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.KeyValue(Schema.STRING, Schema.INT32), + expectedMessage); + PulsarSource> source = + createSource( + topicName, + pulsarSchema( + Schema.KeyValue(Schema.STRING, Schema.INT32), + String.class, + Integer.class)); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + @Test + void keyValuePrimitiveKeyStructValuePulsarSchema() { + final String topicName = + "primitiveKeyStructValue-" + + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + KeyValue expectedMessage = + new KeyValue<>(randomAlphabetic(5), createRandomUser()); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.KeyValue(Schema.STRING, Schema.JSON(TestingUser.class)), + expectedMessage); + PulsarSource> source = + createSource( + topicName, + pulsarSchema( + Schema.KeyValue(Schema.STRING, Schema.JSON(TestingUser.class)), + String.class, + TestingUser.class)); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + @Test + void keyValueStructKeyPrimitiveValuePulsarSchema() { + final String topicName = + "structKeyPrimitiveValue-" + + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + KeyValue expectedMessage = + new KeyValue<>(createRandomUser(), randomAlphabetic(5)); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.KeyValue(Schema.JSON(TestingUser.class), Schema.STRING), + expectedMessage); + PulsarSource> source = + createSource( + topicName, + pulsarSchema( + Schema.KeyValue(Schema.JSON(TestingUser.class), Schema.STRING), + TestingUser.class, + String.class)); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + @Test + void simpleFlinkSchema() { + final String topicName = + "simpleFlinkSchema-" + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + String expectedMessage = randomAlphabetic(5); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.STRING, + expectedMessage); + PulsarSource source = + createSource(topicName, flinkSchema(new SimpleStringSchema())); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + private PulsarSource createSource( + String topicName, PulsarDeserializationSchema deserializationSchema) { + return PulsarSource.builder() + .setDeserializationSchema(deserializationSchema) + .setServiceUrl(operator().serviceUrl()) + .setAdminUrl(operator().adminUrl()) + .setTopics(topicName) + .setSubscriptionType(Exclusive) + .setSubscriptionName(topicName + "-subscription") + .setBoundedStopCursor(StopCursor.latest()) + .setConfig(PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, -1L) + .build(); + } + + private void runPipeline(PulsarSource source, T expected) throws Exception { + try (CloseableIterator iterator = + StreamExecutionEnvironment.getExecutionEnvironment() + .setParallelism(1) + .fromSource(source, WatermarkStrategy.noWatermarks(), "testSource") + .executeAndCollect()) { + assertThat(iterator).hasNext(); + assertThat(iterator.next()).isEqualTo(expected); + } + } + + /** A test POJO class. */ + public static class TestingUser implements Serializable { + private static final long serialVersionUID = -1123545861004770003L; + public String name; + public Integer age; + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public Integer getAge() { + return age; + } + + public void setAge(Integer age) { + this.age = age; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TestingUser that = (TestingUser) o; + return Objects.equals(name, that.name) && Objects.equals(age, that.age); + } + + @Override + public int hashCode() { + return Objects.hash(name, age); + } + } + + private TestingUser createRandomUser() { + TestingUser user = new TestingUser(); + user.setName(randomAlphabetic(5)); + user.setAge(ThreadLocalRandom.current().nextInt(0, Integer.MAX_VALUE)); + return user; + } + /** Create a test message by given bytes. The message don't contains any meta data. */ private Message getMessage( T message, FunctionWithException decoder) throws Exception { diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java index 538e45826d7fc..e1e31b6de41fd 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java @@ -136,7 +136,8 @@ private void seekStartPositionAndHandleSplit( SplitsAddition addition = new SplitsAddition<>(singletonList(split)); // create consumer and seek before split changes - try (Consumer consumer = reader.createPulsarConsumer(partition)) { + try (Consumer consumer = + (Consumer) reader.createPulsarConsumer(partition)) { // inclusive messageId StartCursor startCursor = StartCursor.fromMessageId(startPosition); startCursor.seekPosition(partition.getTopic(), partition.getPartitionId(), consumer); diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java new file mode 100644 index 0000000000000..440abfa6f4e3e --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java @@ -0,0 +1,427 @@ +/* + * 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.connector.pulsar.table; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.pulsar.sink.PulsarSink; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.connector.pulsar.source.PulsarSource; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.table.sink.PulsarTableSerializationSchemaFactory; +import org.apache.flink.connector.pulsar.table.sink.PulsarTableSink; +import org.apache.flink.connector.pulsar.table.source.PulsarTableDeserializationSchemaFactory; +import org.apache.flink.connector.pulsar.table.source.PulsarTableSource; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.WatermarkSpec; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkV2Provider; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.SourceProvider; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.expressions.utils.ResolvedExpressionMock; +import org.apache.flink.table.factories.TestFormatFactory; +import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; +import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; +import org.apache.flink.table.types.DataType; + +import org.apache.pulsar.client.api.SubscriptionType; +import org.assertj.core.util.Lists; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL; +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.ADMIN_URL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FORMAT; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SERVICE_URL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_NAME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; +import static org.apache.flink.table.factories.FactoryUtil.FORMAT; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * This test aims to verify that {@link PulsarTableFactory} can consume proper config options and + * produce expected {@link PulsarTableSource} and {@link PulsarTableSink}. It guarantees that config + * options is used internally by the implementation classes. + */ +public class PulsarTableFactoryTest { + private static final String TEST_TOPIC = "test-topic"; + private static final String TEST_ADMIN_URL = "http://my-broker.example.com:8080"; + private static final String TEST_SERVICE_URL = "pulsar://localhost:6650"; + private static final String TEST_SUBSCRIPTION_NAME = "default-subscription"; + + private static final String NAME = "name"; + private static final String COUNT = "count"; + private static final String TIME = "time"; + private static final String METADATA = "metadata"; + private static final String WATERMARK_EXPRESSION = TIME + " - INTERVAL '5' SECOND"; + private static final DataType WATERMARK_DATATYPE = DataTypes.TIMESTAMP(3); + private static final String COMPUTED_COLUMN_NAME = "computed-column"; + private static final String COMPUTED_COLUMN_EXPRESSION = COUNT + " + 1.0"; + private static final DataType COMPUTED_COLUMN_DATATYPE = DataTypes.DECIMAL(10, 3); + + private static final Properties EXPECTED_PULSAR_SOURCE_PROPERTIES = new Properties(); + private static final Properties EXPECTED_PULSAR_SINK_PROPERTIES = new Properties(); + + private static final String FORMAT_DELIMITER_KEY = + String.format("%s.%s", TestFormatFactory.IDENTIFIER, TestFormatFactory.DELIMITER.key()); + + private static final String FORMAT_FAIL_ON_MISSING_KEY = + String.format( + "%s.%s", TestFormatFactory.IDENTIFIER, TestFormatFactory.FAIL_ON_MISSING.key()); + + static { + EXPECTED_PULSAR_SOURCE_PROPERTIES.setProperty(PULSAR_ADMIN_URL.key(), TEST_ADMIN_URL); + EXPECTED_PULSAR_SOURCE_PROPERTIES.setProperty(PULSAR_SERVICE_URL.key(), TEST_SERVICE_URL); + EXPECTED_PULSAR_SOURCE_PROPERTIES.setProperty( + PULSAR_SUBSCRIPTION_NAME.key(), TEST_SUBSCRIPTION_NAME); + + EXPECTED_PULSAR_SINK_PROPERTIES.setProperty(PULSAR_ADMIN_URL.key(), TEST_ADMIN_URL); + EXPECTED_PULSAR_SINK_PROPERTIES.setProperty(PULSAR_SERVICE_URL.key(), TEST_SERVICE_URL); + } + + private static final ResolvedSchema SCHEMA = + new ResolvedSchema( + Arrays.asList( + Column.physical(NAME, DataTypes.STRING().notNull()), + Column.physical(COUNT, DataTypes.DECIMAL(38, 18)), + Column.physical(TIME, DataTypes.TIMESTAMP(3)), + Column.computed( + COMPUTED_COLUMN_NAME, + ResolvedExpressionMock.of( + COMPUTED_COLUMN_DATATYPE, COMPUTED_COLUMN_EXPRESSION))), + Collections.singletonList( + WatermarkSpec.of( + TIME, + ResolvedExpressionMock.of( + WATERMARK_DATATYPE, WATERMARK_EXPRESSION))), + null); + + private static final ResolvedSchema SCHEMA_WITH_METADATA = + new ResolvedSchema( + Arrays.asList( + Column.physical(NAME, DataTypes.STRING()), + Column.physical(COUNT, DataTypes.DECIMAL(38, 18)), + Column.metadata(TIME, DataTypes.TIMESTAMP(3), "publish_time", false), + Column.metadata( + METADATA, DataTypes.STRING(), "value.metadata_2", false)), + Collections.emptyList(), + null); + + private static final DataType SCHEMA_DATA_TYPE = SCHEMA.toPhysicalRowDataType(); + + @Test + public void testTableSource() { + final Map configuration = getBasicSourceOptions(); + final PulsarTableSource actualPulsarSource = + (PulsarTableSource) createTableSource(SCHEMA, configuration); + + final DecodingFormat> valueDecodingFormat = + new TestFormatFactory.DecodingFormatMock(",", true); + + final PulsarTableDeserializationSchemaFactory deserializationSchemaFactory = + new PulsarTableDeserializationSchemaFactory( + SCHEMA_DATA_TYPE, + null, + new int[0], + valueDecodingFormat, + new int[] {0, 1, 2}); + + final PulsarTableSource expectedPulsarSource = + new PulsarTableSource( + deserializationSchemaFactory, + valueDecodingFormat, + Lists.list(TEST_TOPIC), + EXPECTED_PULSAR_SOURCE_PROPERTIES, + StartCursor.earliest(), + SubscriptionType.Exclusive); + assertThat(actualPulsarSource).isEqualTo(expectedPulsarSource); + + ScanTableSource.ScanRuntimeProvider provider = + actualPulsarSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + assertPulsarSourceIsSameAsExpected(provider); + } + + @Test + public void testTableSourceWithKeyValue() { + final Map configuration = getSourceKeyValueOptions(); + + final PulsarTableSource actualPulsarSource = + (PulsarTableSource) createTableSource(SCHEMA, configuration); + // initialize stateful testing formats + actualPulsarSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + + final TestFormatFactory.DecodingFormatMock keyDecodingFormat = + new TestFormatFactory.DecodingFormatMock("#", false); + keyDecodingFormat.producedDataType = + DataTypes.ROW(DataTypes.FIELD(NAME, DataTypes.STRING().notNull())).notNull(); + + final TestFormatFactory.DecodingFormatMock valueDecodingFormat = + new TestFormatFactory.DecodingFormatMock("|", false); + valueDecodingFormat.producedDataType = + DataTypes.ROW( + DataTypes.FIELD(COUNT, DataTypes.DECIMAL(38, 18)), + DataTypes.FIELD(TIME, DataTypes.TIMESTAMP(3))) + .notNull(); + + final PulsarTableDeserializationSchemaFactory deserializationSchemaFactory = + new PulsarTableDeserializationSchemaFactory( + SCHEMA_DATA_TYPE, + keyDecodingFormat, + new int[] {0}, + valueDecodingFormat, + new int[] {1, 2}); + + final PulsarTableSource expectedPulsarSource = + new PulsarTableSource( + deserializationSchemaFactory, + valueDecodingFormat, + Lists.list(TEST_TOPIC), + EXPECTED_PULSAR_SOURCE_PROPERTIES, + StartCursor.earliest(), + SubscriptionType.Exclusive); + + assertThat(actualPulsarSource).isEqualTo(expectedPulsarSource); + } + + @Test + public void testTableSourceWithKeyValueAndMetadata() { + final Map options = getSourceKeyValueOptions(); + options.put("test-format.readable-metadata", "metadata_1:INT, metadata_2:STRING"); + + final PulsarTableSource actualPulsarSource = + (PulsarTableSource) createTableSource(SCHEMA_WITH_METADATA, options); + // initialize stateful testing formats + actualPulsarSource.applyReadableMetadata( + Arrays.asList("publish_time", "value.metadata_2"), + SCHEMA_WITH_METADATA.toSourceRowDataType()); + actualPulsarSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + + final TestFormatFactory.DecodingFormatMock expectedKeyFormat = + new TestFormatFactory.DecodingFormatMock( + "#", false, ChangelogMode.insertOnly(), Collections.emptyMap()); + expectedKeyFormat.producedDataType = + DataTypes.ROW(DataTypes.FIELD(NAME, DataTypes.STRING())).notNull(); + + final Map expectedReadableMetadata = new HashMap<>(); + expectedReadableMetadata.put("metadata_1", DataTypes.INT()); + expectedReadableMetadata.put("metadata_2", DataTypes.STRING()); + + final TestFormatFactory.DecodingFormatMock expectedValueFormat = + new TestFormatFactory.DecodingFormatMock( + "|", false, ChangelogMode.insertOnly(), expectedReadableMetadata); + expectedValueFormat.producedDataType = + DataTypes.ROW( + DataTypes.FIELD(COUNT, DataTypes.DECIMAL(38, 18)), + DataTypes.FIELD("metadata_2", DataTypes.STRING())) + .notNull(); + expectedValueFormat.metadataKeys = Collections.singletonList("metadata_2"); + + final PulsarTableDeserializationSchemaFactory deserializationSchemaFactory = + new PulsarTableDeserializationSchemaFactory( + SCHEMA_WITH_METADATA.toPhysicalRowDataType(), + expectedKeyFormat, + new int[] {0}, + expectedValueFormat, + new int[] {1}); + + final PulsarTableSource expectedPulsarSource = + new PulsarTableSource( + deserializationSchemaFactory, + expectedValueFormat, + Lists.list(TEST_TOPIC), + EXPECTED_PULSAR_SOURCE_PROPERTIES, + StartCursor.earliest(), + SubscriptionType.Exclusive); + + deserializationSchemaFactory.setProducedDataType( + SCHEMA_WITH_METADATA.toSourceRowDataType()); + deserializationSchemaFactory.setConnectorMetadataKeys( + Collections.singletonList("publish_time")); + + assertThat(actualPulsarSource).isEqualTo(expectedPulsarSource); + } + + @Test + public void testTableSink() { + final Map modifiedOptions = getBasicSinkOptions(); + final DynamicTableSink actualPulsarTableSink = createTableSink(SCHEMA, modifiedOptions); + + final EncodingFormat> valueEncodingFormat = + new TestFormatFactory.EncodingFormatMock(","); + + final PulsarTableSerializationSchemaFactory serializationSchemaFactory = + new PulsarTableSerializationSchemaFactory( + SCHEMA_DATA_TYPE, + null, + new int[0], + valueEncodingFormat, + new int[] {0, 1, 2}); + + final PulsarTableSink expectedPulsarTableSink = + new PulsarTableSink( + serializationSchemaFactory, + valueEncodingFormat.getChangelogMode(), + Lists.list(TEST_TOPIC), + EXPECTED_PULSAR_SINK_PROPERTIES, + DeliveryGuarantee.AT_LEAST_ONCE, + null, + TopicRoutingMode.ROUND_ROBIN, + 0); + assertThat(actualPulsarTableSink).isEqualTo(expectedPulsarTableSink); + + DynamicTableSink.SinkRuntimeProvider provider = + actualPulsarTableSink.getSinkRuntimeProvider(new SinkRuntimeProviderContext(false)); + assertThat(provider).isInstanceOf(SinkV2Provider.class); + final SinkV2Provider sinkProvider = (SinkV2Provider) provider; + final Sink sinkFunction = sinkProvider.createSink(); + assertThat(sinkFunction).isInstanceOf(PulsarSink.class); + } + + @Test + public void testTableSinkWithKeyValue() { + final Map modifiedOptions = getSinkKeyValueOptions(); + final PulsarTableSink actualPulsarTableSink = + (PulsarTableSink) createTableSink(SCHEMA, modifiedOptions); + // initialize stateful testing formats + actualPulsarTableSink.getSinkRuntimeProvider(new SinkRuntimeProviderContext(false)); + + final TestFormatFactory.EncodingFormatMock keyEncodingFormat = + new TestFormatFactory.EncodingFormatMock("#"); + keyEncodingFormat.consumedDataType = + DataTypes.ROW(DataTypes.FIELD(NAME, DataTypes.STRING().notNull())).notNull(); + + final TestFormatFactory.EncodingFormatMock valueEncodingFormat = + new TestFormatFactory.EncodingFormatMock("|"); + valueEncodingFormat.consumedDataType = + DataTypes.ROW( + DataTypes.FIELD(COUNT, DataTypes.DECIMAL(38, 18)), + DataTypes.FIELD(TIME, DataTypes.TIMESTAMP(3))) + .notNull(); + + final PulsarTableSerializationSchemaFactory serializationSchemaFactory = + new PulsarTableSerializationSchemaFactory( + SCHEMA_DATA_TYPE, + keyEncodingFormat, + new int[] {0}, + valueEncodingFormat, + new int[] {1, 2}); + + final PulsarTableSink expectedPulsarTableSink = + new PulsarTableSink( + serializationSchemaFactory, + valueEncodingFormat.getChangelogMode(), + Lists.list(TEST_TOPIC), + EXPECTED_PULSAR_SINK_PROPERTIES, + DeliveryGuarantee.AT_LEAST_ONCE, + null, + TopicRoutingMode.ROUND_ROBIN, + 0); + assertThat(actualPulsarTableSink).isEqualTo(expectedPulsarTableSink); + } + + private static Map getBasicSourceOptions() { + Map tableOptions = new HashMap<>(); + tableOptions.put(CONNECTOR.key(), PulsarTableFactory.IDENTIFIER); + tableOptions.put(TOPICS.key(), TEST_TOPIC); + tableOptions.put(ADMIN_URL.key(), TEST_ADMIN_URL); + tableOptions.put(SERVICE_URL.key(), TEST_SERVICE_URL); + tableOptions.put(SOURCE_SUBSCRIPTION_NAME.key(), TEST_SUBSCRIPTION_NAME); + // Format options. + tableOptions.put(FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put(FORMAT_DELIMITER_KEY, ","); + tableOptions.put(FORMAT_FAIL_ON_MISSING_KEY, "true"); + return tableOptions; + } + + private static Map getSourceKeyValueOptions() { + Map tableOptions = new HashMap<>(); + tableOptions.put(CONNECTOR.key(), PulsarTableFactory.IDENTIFIER); + tableOptions.put(TOPICS.key(), TEST_TOPIC); + tableOptions.put(ADMIN_URL.key(), TEST_ADMIN_URL); + tableOptions.put(SERVICE_URL.key(), TEST_SERVICE_URL); + tableOptions.put(SOURCE_SUBSCRIPTION_NAME.key(), TEST_SUBSCRIPTION_NAME); + // Format options. + tableOptions.put(FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put(FORMAT_DELIMITER_KEY, "|"); + tableOptions.put(FORMAT_FAIL_ON_MISSING_KEY, "false"); + + tableOptions.put(KEY_FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put("key." + FORMAT_DELIMITER_KEY, "#"); + tableOptions.put("key." + FORMAT_FAIL_ON_MISSING_KEY, "false"); + tableOptions.put(KEY_FIELDS.key(), NAME); + return tableOptions; + } + + private static Map getBasicSinkOptions() { + Map tableOptions = new HashMap<>(); + tableOptions.put(CONNECTOR.key(), PulsarTableFactory.IDENTIFIER); + tableOptions.put(TOPICS.key(), TEST_TOPIC); + tableOptions.put(ADMIN_URL.key(), TEST_ADMIN_URL); + tableOptions.put(SERVICE_URL.key(), TEST_SERVICE_URL); + // Format options. + tableOptions.put(FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put(FORMAT_DELIMITER_KEY, ","); + tableOptions.put(FORMAT_FAIL_ON_MISSING_KEY, "true"); + return tableOptions; + } + + private static Map getSinkKeyValueOptions() { + Map tableOptions = new HashMap<>(); + tableOptions.put(CONNECTOR.key(), PulsarTableFactory.IDENTIFIER); + tableOptions.put(TOPICS.key(), TEST_TOPIC); + tableOptions.put(ADMIN_URL.key(), TEST_ADMIN_URL); + tableOptions.put(SERVICE_URL.key(), TEST_SERVICE_URL); + // Format options. + tableOptions.put(FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put(FORMAT_DELIMITER_KEY, "|"); + tableOptions.put(FORMAT_FAIL_ON_MISSING_KEY, "false"); + + tableOptions.put(KEY_FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put("key." + FORMAT_DELIMITER_KEY, "#"); + tableOptions.put("key." + FORMAT_FAIL_ON_MISSING_KEY, "false"); + tableOptions.put(KEY_FIELDS.key(), NAME); + return tableOptions; + } + + private PulsarSource assertPulsarSourceIsSameAsExpected( + ScanTableSource.ScanRuntimeProvider provider) { + assertThat(provider).isInstanceOf(SourceProvider.class); + final SourceProvider sourceProvider = (SourceProvider) provider; + return (PulsarSource) sourceProvider.createSource(); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java new file mode 100644 index 0000000000000..08cf89c014ff4 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java @@ -0,0 +1,295 @@ +/* + * 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.connector.pulsar.table; + +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.test.util.SuccessException; +import org.apache.flink.types.Row; + +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; +import static org.apache.flink.connector.pulsar.table.testutils.PulsarTableTestUtils.collectRows; +import static org.apache.flink.util.CollectionUtil.entry; +import static org.apache.flink.util.CollectionUtil.map; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * IT cases for the Pulsar table source and sink. It aims to verify runtime behaviour and certain + * use cases are correct and can produce/consume the desired records as user specifies. + */ +@ExtendWith(MiniClusterExtension.class) +public class PulsarTableITCase extends PulsarTableTestBase { + + private static final String JSON_FORMAT = "json"; + private static final String AVRO_FORMAT = "avro"; + private static final String CSV_FORMAT = "csv"; + + @ParameterizedTest + @ValueSource(strings = {JSON_FORMAT}) + public void pulsarSourceSink(String format) throws Exception { + // we always use a different topic name for each parameterized topic, + // in order to make sure the topic can be created. + final String topic = "test_topic_" + format + randomAlphanumeric(3); + createTestTopic(topic, 1); + + // ---------- Produce an event time stream into Pulsar ------------------- + + final String createTable = + String.format( + "create table pulsar_source_sink (\n" + + " `computed-price` as price + 1.0,\n" + + " price decimal(38, 18),\n" + + " currency string,\n" + + " log_date date,\n" + + " log_time time(3),\n" + + " log_ts timestamp(3),\n" + + " ts as log_ts + INTERVAL '1' SECOND,\n" + + " watermark for ts as ts\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " %s\n" + + ")", + PulsarTableFactory.IDENTIFIER, + topic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + formatOptions(format)); + + tableEnv.executeSql(createTable); + + String initialValues = + "INSERT INTO pulsar_source_sink\n" + + "SELECT CAST(price AS DECIMAL(10, 2)), currency, " + + " CAST(d AS DATE), CAST(t AS TIME(0)), CAST(ts AS TIMESTAMP(3))\n" + + "FROM (VALUES (2.02,'Euro','2019-12-12', '00:00:01', '2019-12-12 00:00:01.001001'), \n" + + " (1.11,'US Dollar','2019-12-12', '00:00:02', '2019-12-12 00:00:02.002001'), \n" + + " (50,'Yen','2019-12-12', '00:00:03', '2019-12-12 00:00:03.004001'), \n" + + " (3.1,'Euro','2019-12-12', '00:00:04', '2019-12-12 00:00:04.005001'), \n" + + " (5.33,'US Dollar','2019-12-12', '00:00:05', '2019-12-12 00:00:05.006001'), \n" + + " (0,'DUMMY','2019-12-12', '00:00:10', '2019-12-12 00:00:10'))\n" + + " AS orders (price, currency, d, t, ts)"; + tableEnv.executeSql(initialValues).await(); + + String query = + "SELECT\n" + + " CAST(TUMBLE_END(ts, INTERVAL '5' SECOND) AS VARCHAR),\n" + + " CAST(MAX(log_date) AS VARCHAR),\n" + + " CAST(MAX(log_time) AS VARCHAR),\n" + + " CAST(MAX(ts) AS VARCHAR),\n" + + " COUNT(*),\n" + + " CAST(MAX(price) AS DECIMAL(10, 2))\n" + + "FROM pulsar_source_sink\n" + + "GROUP BY TUMBLE(ts, INTERVAL '5' SECOND)"; + + DataStream result = + tableEnv.toAppendStream(tableEnv.sqlQuery(query), RowData.class); + TestingSinkFunction sink = new TestingSinkFunction(2); + result.addSink(sink).setParallelism(1); + + try { + env.execute("Job_2"); + } catch (Throwable e) { + if (!isCausedByJobFinished(e)) { + // re-throw + throw e; + } + } + + List expected = + Arrays.asList( + "+I(2019-12-12 00:00:05.000,2019-12-12,00:00:03,2019-12-12 00:00:04.004,3,50.00)", + "+I(2019-12-12 00:00:10.000,2019-12-12,00:00:05,2019-12-12 00:00:06.006,2,5.33)"); + + assertThat(TestingSinkFunction.rows).isEqualTo(expected); + } + + @ParameterizedTest + @ValueSource(strings = {JSON_FORMAT}) + public void pulsarSourceSinkWithKeyAndPartialValue(String format) throws Exception { + // we always use a different topic name for each parameterized topic, + // in order to make sure the topic can be created. + final String topic = "key_partial_value_topic_" + format + randomAlphanumeric(3); + createTestTopic(topic, 1); + + // ---------- Produce an event time stream into Pulsar ------------------- + + // k_user_id and user_id have different data types to verify the correct mapping, + // fields are reordered on purpose + final String createTable = + String.format( + "CREATE TABLE pulsar_key_value (\n" + + " `user_id` BIGINT,\n" + + " `name` STRING,\n" + + " `event_id` BIGINT,\n" + + " `payload` STRING\n" + + ") WITH (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s',\n" + + " 'key.format' = '%s',\n" + + " 'key.fields' = 'user_id; event_id'\n" + + ")", + PulsarTableFactory.IDENTIFIER, + topic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format, + format); + + tableEnv.executeSql(createTable); + + String initialValues = + "INSERT INTO pulsar_key_value\n" + + "VALUES\n" + + " (1, 'name 1', 100, 'payload 1'),\n" + + " (2, 'name 2', 101, 'payload 2'),\n" + + " (3, 'name 3', 102, 'payload 3')"; + tableEnv.executeSql(initialValues).await(); + + final List result = + collectRows(tableEnv.sqlQuery("SELECT * FROM pulsar_key_value"), 3); + + assertThat(result) + .containsExactlyInAnyOrder( + Row.of(1L, "name 1", 100L, "payload 1"), + Row.of(2L, "name 2", 101L, "payload 2"), + Row.of(3L, "name 3", 102L, "payload 3")); + } + + @ParameterizedTest + @ValueSource(strings = {JSON_FORMAT}) + public void pulsarSourceSinkWithMetadata(String format) throws Exception { + // we always use a different topic name for each parameterized topic, + // in order to make sure the topic can be created. + final String topic = "metadata_topic_" + format + randomAlphanumeric(3); + createTestTopic(topic, 1); + + final String createTable = + String.format( + "CREATE TABLE pulsar_metadata (\n" + + " `physical_1` STRING,\n" + + " `physical_2` INT,\n" + + " `message_size` INT METADATA VIRTUAL,\n" + + " `event_time` TIMESTAMP(3) METADATA,\n" + + " `properties` MAP METADATA,\n" + + " `physical_3` BOOLEAN\n" + + ") WITH (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'pulsar.producer.producerName' = 'pulsar-table-test',\n" + + " 'format' = '%s'\n" + + ")", + PulsarTableFactory.IDENTIFIER, + topic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format); + tableEnv.executeSql(createTable); + + String initialValues = + "INSERT INTO pulsar_metadata\n" + + "VALUES\n" + + " ('data 1', 1, TIMESTAMP '2022-03-24 13:12:11.123', MAP['k1', 'C0FFEE', 'k2', 'BABE01'], TRUE),\n" + + " ('data 2', 2, TIMESTAMP '2022-03-25 13:12:11.123', CAST(NULL AS MAP), FALSE),\n" + + " ('data 3', 3, TIMESTAMP '2022-03-26 13:12:11.123', MAP['k1', 'C0FFEE', 'k2', 'BABE01'], TRUE)"; + tableEnv.executeSql(initialValues).await(); + + // ---------- Consume stream from Pulsar ------------------- + + final List result = collectRows(tableEnv.sqlQuery("SELECT * FROM pulsar_metadata"), 3); + assertThat(result) + .containsExactlyInAnyOrder( + Row.of( + "data 1", + 1, + 56, + LocalDateTime.parse("2022-03-24T13:12:11.123"), + map(entry("k1", "C0FFEE"), entry("k2", "BABE01")), + true), + Row.of( + "data 2", + 2, + 57, + LocalDateTime.parse("2022-03-25T13:12:11.123"), + Collections.emptyMap(), + false), + Row.of( + "data 3", + 3, + 56, + LocalDateTime.parse("2022-03-26T13:12:11.123"), + map(entry("k1", "C0FFEE"), entry("k2", "BABE01")), + true)); + } + + private static final class TestingSinkFunction implements SinkFunction { + + private static final long serialVersionUID = 455430015321124493L; + private static List rows = new ArrayList<>(); + + private final int expectedSize; + + private TestingSinkFunction(int expectedSize) { + this.expectedSize = expectedSize; + rows.clear(); + } + + @Override + public void invoke(RowData value, Context context) { + rows.add(value.toString()); + if (rows.size() >= expectedSize) { + // job finish + throw new SuccessException(); + } + } + } + + private static boolean isCausedByJobFinished(Throwable e) { + if (e instanceof SuccessException) { + return true; + } else if (e.getCause() != null) { + return isCausedByJobFinished(e.getCause()); + } else { + return false; + } + } + + private String formatOptions(String format) { + return String.format("'format' = '%s'", format); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java new file mode 100644 index 0000000000000..08e250b7a86db --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java @@ -0,0 +1,376 @@ +/* + * 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.connector.pulsar.table; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.MessageIdStartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.TimestampStartCursor; +import org.apache.flink.connector.pulsar.table.testutils.MockTopicRouter; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.utils.FactoryMocks; +import org.apache.flink.table.types.DataType; + +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.impl.MessageIdImpl; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_STATS_INTERVAL_SECONDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.createKeyFormatProjection; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.createValueFormatProjection; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getMessageDelayMillis; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getPulsarProperties; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getStartCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getSubscriptionType; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicListFromOptions; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicRouter; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicRoutingMode; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.parseMessageIdStartCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_MESSAGE_DELAY_INTERVAL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.table.api.DataTypes.FIELD; +import static org.apache.flink.table.api.DataTypes.INT; +import static org.apache.flink.table.api.DataTypes.ROW; +import static org.apache.flink.table.api.DataTypes.STRING; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; + +/** + * Unit test for {@link PulsarTableOptionUtils}. Tests each method and different inputs. Some tests + * have overlapping semantics with {@link PulsarTableOptionsTest} and {@link + * PulsarTableValidationUtilsTest}, but they cover different aspects of the validation, so all of + * them should be kept. + */ +public class PulsarTableOptionUtilsTest { + // -------------------------------------------------------------------------------------------- + // Format and Projection Test + // -------------------------------------------------------------------------------------------- + @Test + void formatProjection() { + final DataType dataType = + DataTypes.ROW( + FIELD("id", INT()), + FIELD("name", STRING()), + FIELD("age", INT()), + FIELD("address", STRING())); + + final Map options = createTestOptions(); + options.put("key.fields", "address; name"); + + final Configuration config = Configuration.fromMap(options); + + assertThat(createKeyFormatProjection(config, dataType)).containsExactly(3, 1); + assertThat(createValueFormatProjection(config, dataType)).containsExactly(0, 2); + } + + @Test + void invalidKeyFormatFieldProjection() { + final DataType dataType = ROW(FIELD("id", INT()), FIELD("name", STRING())); + final Map options = createTestOptions(); + options.put("key.fields", "non_existing"); + + final Configuration config = Configuration.fromMap(options); + + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> createKeyFormatProjection(config, dataType)) + .withMessage( + String.format( + "Could not find the field '%s' in the table schema for usage in the key format. " + + "A key field must be a regular, physical column. " + + "The following columns can be selected in the '%s' option: [id, name]", + "non_existing", KEY_FIELDS.key())); + } + + static Map createTestOptions() { + final Map options = new HashMap<>(); + options.put("key.format", "test-format"); + options.put("key.test-format.delimiter", ","); + options.put("value.format", "test-format"); + options.put("value.test-format.delimiter", "|"); + options.put("value.test-format.fail-on-missing", "true"); + return options; + } + + // -------------------------------------------------------------------------------------------- + // Table Source Option Utils Test + // -------------------------------------------------------------------------------------------- + + @Test + void topicsList() { + final Map options = createDefaultOptions(); + options.put(TOPICS.key(), ";"); + List topicList = getTopicListFromOptions(Configuration.fromMap(options)); + assertThat(topicList).isEmpty(); + + options.put(TOPICS.key(), "topic1;"); + topicList = getTopicListFromOptions(Configuration.fromMap(options)); + assertThat(topicList).hasSize(1); + + options.put(TOPICS.key(), "topic1;topic2"); + topicList = getTopicListFromOptions(Configuration.fromMap(options)); + assertThat(topicList).hasSize(2); + + options.put(TOPICS.key(), ""); + topicList = getTopicListFromOptions(Configuration.fromMap(options)); + assertThat(topicList).isEmpty(); + } + + @Test + void pulsarProperties() { + final Map options = createDefaultOptions(); + options.put(PULSAR_STATS_INTERVAL_SECONDS.key(), "30"); + Properties properties = getPulsarProperties(Configuration.fromMap(options)); + assertThat(properties.getProperty(PULSAR_STATS_INTERVAL_SECONDS.key())).isEqualTo("30"); + } + + @Test + void startCursor() { + // TDOO Use isEqualTo() to assert; need equals() method + final Map options = createDefaultOptions(); + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "earliest"); + StartCursor startCursor = getStartCursor(Configuration.fromMap(options)); + assertThat(startCursor).isInstanceOf(MessageIdStartCursor.class); + + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "latest"); + startCursor = getStartCursor(Configuration.fromMap(options)); + assertThat(startCursor).isInstanceOf(MessageIdStartCursor.class); + + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "0:0:-1"); + startCursor = getStartCursor(Configuration.fromMap(options)); + assertThat(startCursor).isInstanceOf(MessageIdStartCursor.class); + + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "other"); + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> getStartCursor(Configuration.fromMap(options))) + .withMessage("MessageId format must be ledgerId:entryId:partitionId."); + + options.remove(SOURCE_START_FROM_MESSAGE_ID.key()); + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "123545"); + startCursor = getStartCursor(Configuration.fromMap(options)); + assertThat(startCursor).isInstanceOf(TimestampStartCursor.class); + + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "123545L"); + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> getStartCursor(Configuration.fromMap(options))) + .withMessage( + "Could not parse value '123545L' for key 'source.start.publish-time'."); + } + + @Test + void subscriptionType() { + final Map options = createDefaultOptions(); + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Shared"); + SubscriptionType subscriptionType = getSubscriptionType(Configuration.fromMap(options)); + assertThat(subscriptionType).isEqualTo(SubscriptionType.Shared); + + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Exclusive"); + subscriptionType = getSubscriptionType(Configuration.fromMap(options)); + assertThat(subscriptionType).isEqualTo(SubscriptionType.Exclusive); + } + + @Test + void canParseMessageIdEarliestOrLatest() { + String earliest = "earliest"; + StartCursor startCursor = parseMessageIdStartCursor(earliest); + assertThat(startCursor).isEqualTo(StartCursor.earliest()); + + String latest = "latest"; + startCursor = parseMessageIdStartCursor(latest); + assertThat(startCursor).isEqualTo(StartCursor.latest()); + + String precise = "0:0:100"; + startCursor = parseMessageIdStartCursor(precise); + assertThat(startCursor).isEqualTo(StartCursor.fromMessageId(new MessageIdImpl(0, 0, 100))); + } + + @Test + void canParseMessageIdUsingMessageIdImpl() { + final String invalidFormatMessage = + "MessageId format must be ledgerId:entryId:partitionId."; + final String invalidNumberMessage = + "MessageId format must be ledgerId:entryId:partitionId. Each id should be able to parsed to long type."; + String precise = "0:0:100"; + StartCursor startCursor = parseMessageIdStartCursor(precise); + + String empty = ""; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdStartCursor(empty)) + .withMessage(invalidFormatMessage); + + String noSemicolon = "0"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdStartCursor(noSemicolon)) + .withMessage(invalidFormatMessage); + + String oneSemiColon = "0:"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdStartCursor(oneSemiColon)) + .withMessage(invalidFormatMessage); + + String oneSemiColonComplete = "0:0"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdStartCursor(oneSemiColonComplete)) + .withMessage(invalidFormatMessage); + + String twoSemiColon = "0:0:"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdStartCursor(twoSemiColon)) + .withMessage(invalidNumberMessage); + + String twoSemiColonComplete = "0:0:0"; + startCursor = parseMessageIdStartCursor(twoSemiColonComplete); + + String threeSemicolon = "0:0:0:"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdStartCursor(threeSemicolon)) + .withMessage(invalidNumberMessage); + + String threeSemicolonComplete = "0:0:0:0"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdStartCursor(threeSemicolonComplete)) + .withMessage(invalidNumberMessage); + + String invalidNumber = "0:0:adf"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdStartCursor(invalidNumber)) + .withMessage(invalidNumberMessage); + } + + @Test + void publishTimeStartCursor() { + final Map options = createDefaultOptions(); + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "12345"); + StartCursor startCursor = getStartCursor(Configuration.fromMap(options)); + assertThat(startCursor).isInstanceOf(TimestampStartCursor.class); + + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "12345L"); + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> getStartCursor(Configuration.fromMap(options))) + .withMessage("Could not parse value '12345L' for key 'source.start.publish-time'."); + } + + // -------------------------------------------------------------------------------------------- + // Table Sink Option Utils Test + // -------------------------------------------------------------------------------------------- + + @Test + void topicRouter() { + final Map options = createDefaultOptions(); + options.put( + SINK_CUSTOM_TOPIC_ROUTER.key(), + "org.apache.flink.connector.pulsar.table.testutils.MockTopicRouter"); + TopicRouter topicRouter = + getTopicRouter(Configuration.fromMap(options), FactoryMocks.class.getClassLoader()); + assertThat(topicRouter).isInstanceOf(MockTopicRouter.class); + + options.put( + SINK_CUSTOM_TOPIC_ROUTER.key(), + "org.apache.flink.connector.pulsar.table.PulsarTableOptionsTest"); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy( + () -> + getTopicRouter( + Configuration.fromMap(options), + FactoryMocks.class.getClassLoader())) + .withMessage( + String.format( + "Sink TopicRouter class '%s' should extend from the required class %s", + PulsarTableOptionsTest.class.getName(), + TopicRouter.class.getName())); + + options.put( + SINK_CUSTOM_TOPIC_ROUTER.key(), + "org.apache.flink.connector.pulsar.table.testutils.NonExistMockTopicRouter"); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy( + () -> + getTopicRouter( + Configuration.fromMap(options), + FactoryMocks.class.getClassLoader())) + .withMessage( + String.format( + "Could not find and instantiate TopicRouter class '%s'", + "org.apache.flink.connector.pulsar.table.testutils.NonExistMockTopicRouter")); + } + + @Test + void topicRoutingMode() { + final Map options = createDefaultOptions(); + options.put(SINK_TOPIC_ROUTING_MODE.key(), "round-robin"); + TopicRoutingMode topicRoutingMode = getTopicRoutingMode(Configuration.fromMap(options)); + assertThat(topicRoutingMode).isEqualTo(TopicRoutingMode.ROUND_ROBIN); + + options.put(SINK_TOPIC_ROUTING_MODE.key(), "message-key-hash"); + topicRoutingMode = getTopicRoutingMode(Configuration.fromMap(options)); + assertThat(topicRoutingMode).isEqualTo(TopicRoutingMode.MESSAGE_KEY_HASH); + } + + @Test + void messageDelayMillis() { + final Map options = createDefaultOptions(); + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "10 s"); + long messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofSeconds(10).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "10s"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofSeconds(10).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "1000ms"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofMillis(1000).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "1 d"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofDays(1).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "1 H"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofHours(1).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "1 min"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofMinutes(1).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "1m"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofMinutes(1).toMillis()); + } + + private Map createDefaultOptions() { + Map optionMap = new HashMap<>(); + return optionMap; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java new file mode 100644 index 0000000000000..500dd0f49716e --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java @@ -0,0 +1,420 @@ +/* + * 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.connector.pulsar.table; + +import org.apache.flink.connector.pulsar.table.testutils.MockPulsarAuthentication; +import org.apache.flink.table.api.ValidationException; + +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_AUTH_PARAM_MAP; +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_AUTH_PLUGIN_CLASS_NAME; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_INITIAL_POSITION; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_MESSAGE_DELAY_INTERVAL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; +import static org.assertj.core.api.Assertions.assertThatNoException; + +/** + * Test config options for Pulsar SQL connector. This test aims to verify legal combination of + * config options will be accepted and do not cause runtime exceptions (but cannot guarantee they + * are taking effect), and illegal combinations of config options will be rejected early. + */ +public class PulsarTableOptionsTest extends PulsarTableTestBase { + @Test + void noTopicsSpecified() { + final String topicName = randomTopicName(); + Map testConfigs = new HashMap<>(); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, + new ValidationException( + "One or more required options are missing.\n\n" + + "Missing required options are:\n\n" + + "topics")); + } + + @Test + void invalidEmptyTopics() { + final String topicName = randomTopicName(); + Map testConfigs = new HashMap<>(); + + testConfigs.put(TOPICS.key(), ""); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, new ValidationException("The topics list should not be empty.")); + } + + @Test + void topicsWithSemicolon() { + final String topicName = randomTopicName(); + Map testConfigs = new HashMap<>(); + testConfigs.put(TOPICS.key(), topicName + ";"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void invalidTopicName() { + final String topicName = randomTopicName(); + Map testConfigs = new HashMap<>(); + String invalidTopicName = "persistent://tenant/no-topic"; + testConfigs.put(TOPICS.key(), invalidTopicName); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, + new ValidationException( + String.format( + "The topics name %s is not a valid topic name.", + invalidTopicName))); + } + + @Test + void topicsList() { + final String topicName = randomTopicName(); + Map testConfigs = new HashMap<>(); + testConfigs.put( + TOPICS.key(), + topicNameWithPartition(topicName, 0) + ";" + topicNameWithPartition(topicName, 1)); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void subscriptionType() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + testConfigs.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Exclusive"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void invalidUnsupportedSubscriptionType() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + testConfigs.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Key_Shared"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectException( + topicName, + new ValidationException( + "Only Exclusive and Shared SubscriptionType is supported. ")); + } + + @Test + void invalidNonExistSubscriptionType() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + testConfigs.put(SOURCE_SUBSCRIPTION_TYPE.key(), "random-subscription-type"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, + new ValidationException("Invalid value for option 'source.subscription-type'.")); + } + + @Test + void messageIdStartCursorEarliest() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "earliest"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void messageIdStartCursorLatest() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "latest"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void messageIdStartCursorExact() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "0:0:-1"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void invalidMessageIdStartCursorEmptyId() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "0:0:"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectException( + topicName, + new IllegalArgumentException( + "MessageId format must be ledgerId:entryId:partitionId. " + + "Each id should be able to parsed to long type.")); + } + + @Test + void invalidMessageIdStartCursorIncomplete() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "0:0"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectException( + topicName, + new IllegalArgumentException( + "MessageId format must be ledgerId:entryId:partitionId.")); + } + + @Test + void timestampStartCursor() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "233010230"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void topicRoutingMode() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put(SINK_TOPIC_ROUTING_MODE.key(), "message-key-hash"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void invalidTopicRouter() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + String invalidClassName = "invalid class name"; + testConfigs.put(SINK_CUSTOM_TOPIC_ROUTER.key(), invalidClassName); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, + new ValidationException( + String.format( + "Could not find and instantiate TopicRouter class '%s'", + invalidClassName))); + } + + @Test + void messageDelay() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "10s"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void invalidMessageDelay() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "invalid-duration"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, + new ValidationException("Invalid value for option 'sink.message-delay-interval'.")); + } + + // -------------------------------------------------------------------------------------------- + // PulsarSourceOptions Test + // -------------------------------------------------------------------------------------------- + @Test + void subscriptionInitialPosition() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put(PULSAR_SUBSCRIPTION_INITIAL_POSITION.key(), "Earliest"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + // -------------------------------------------------------------------------------------------- + // PulsarOptions, PulsarSourceOptions, PulsarSinkOptions Test + // -------------------------------------------------------------------------------------------- + + @Test + void pulsarOptionsAuthParamMap() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put(PULSAR_AUTH_PARAM_MAP.key(), "key1:value1,key2:value2"); + testConfigs.put( + PULSAR_AUTH_PLUGIN_CLASS_NAME.key(), MockPulsarAuthentication.class.getName()); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + // -------------------------------------------------------------------------------------------- + // requiredOptions(), optionalOptions() Test + // -------------------------------------------------------------------------------------------- + + @Test + void unusedConfigOptions() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put("random_config", "random_value"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException(topicName, ValidationException.class); + } + + // -------------------------------------------------------------------------------------------- + // Utils methods + // -------------------------------------------------------------------------------------------- + + private String createTestConfig(Map configMap) { + StringBuilder sb = new StringBuilder(); + for (Map.Entry entry : configMap.entrySet()) { + sb.append(String.format(" '%s' = '%s' ,\n", entry.getKey(), entry.getValue())); + } + return sb.toString(); + } + + private void runSql(String topicName, String testConfigString) { + createTestTopic(topicName, 2); + final String createTable = + String.format( + "CREATE TABLE %s (\n" + + " `physical_1` STRING,\n" + + " `physical_2` INT,\n" + + " `physical_3` BOOLEAN\n" + + ") WITH (\n" + + " 'connector' = 'pulsar',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " %s" + + " 'format' = 'json'\n" + + ")", + topicName, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + testConfigString); + tableEnv.executeSql(createTable); + } + + private void runSinkAndExpectSucceed(String topicName) { + String initialValues = + String.format( + "INSERT INTO %s\n" + + "VALUES\n" + + " ('data 1', 1, TRUE),\n" + + " ('data 2', 2, FALSE),\n" + + " ('data 3', 3, TRUE)", + topicName); + assertThatNoException().isThrownBy(() -> tableEnv.executeSql(initialValues).await()); + } + + private void runSinkAndExpectException( + String topicName, final Class exceptionType) { + String initialValues = + String.format( + "INSERT INTO %s\n" + + "VALUES\n" + + " ('data 1', 1, TRUE),\n" + + " ('data 2', 2, FALSE),\n" + + " ('data 3', 3, TRUE)", + topicName); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> tableEnv.executeSql(initialValues).await()); + } + + private void runSinkAndExpectException(String topicName, Throwable cause) { + String initialValues = + String.format( + "INSERT INTO %s\n" + + "VALUES\n" + + " ('data 1', 1, TRUE),\n" + + " ('data 2', 2, FALSE),\n" + + " ('data 3', 3, TRUE)", + topicName); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> tableEnv.executeSql(initialValues).await()) + .withCause(cause); + } + + private void runSourceAndExpectSucceed(String topicName) { + assertThatNoException() + .isThrownBy(() -> tableEnv.sqlQuery(String.format("SELECT * FROM %s", topicName))); + } + + private void runSourceAndExpectException(String topicName, Throwable cause) { + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> tableEnv.sqlQuery(String.format("SELECT * FROM %s", topicName))) + .withCause(cause); + } + + private String randomTopicName() { + final String testTopicPrefix = "test_config_topic"; + return testTopicPrefix + randomAlphabetic(5); + } + + private Map testConfigWithTopic(String tableName) { + Map testConfigs = new HashMap<>(); + testConfigs.put(TOPICS.key(), tableName); + return testConfigs; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java new file mode 100644 index 0000000000000..90ba67a129261 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java @@ -0,0 +1,75 @@ +/* + * 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.connector.pulsar.table; + +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; +import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime; +import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment; +import org.apache.flink.connector.testframe.junit.annotations.TestEnv; +import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem; +import org.apache.flink.connector.testframe.junit.annotations.TestSemantics; +import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestInstance; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Base class for Pulsar table integration test. */ +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public abstract class PulsarTableTestBase { + // TODO fix the archunit violations + private static final Logger LOG = LoggerFactory.getLogger(PulsarTableTestBase.class); + + @TestEnv MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment(); + + // Defines pulsar running environment + @TestExternalSystem + PulsarTestEnvironment pulsar = new PulsarTestEnvironment(PulsarRuntime.container()); + + @TestSemantics + CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE}; + + protected StreamExecutionEnvironment env; + + protected StreamTableEnvironment tableEnv; + + @BeforeAll + public void beforeAll() { + pulsar.startUp(); + // run env + env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + env.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + tableEnv = StreamTableEnvironment.create(env); + } + + public void createTestTopic(String topic, int numPartitions) { + pulsar.operator().createTopic(topic, numPartitions); + } + + @AfterAll + public void afterAll() { + pulsar.tearDown(); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtilsTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtilsTest.java new file mode 100644 index 0000000000000..27c1133068aa5 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtilsTest.java @@ -0,0 +1,193 @@ +/* + * 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.connector.pulsar.table; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.ValidationException; + +import org.apache.pulsar.client.api.SubscriptionType; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FORMAT; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateKeyFormatConfigs; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateSinkRoutingConfigs; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateStartCursorConfigs; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateSubscriptionTypeConfigs; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateTopicsConfigs; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; +import static org.assertj.core.api.Assertions.assertThatNoException; + +/** Unit test for {@link PulsarTableValidationUtils}. */ +public class PulsarTableValidationUtilsTest extends PulsarTableTestBase { + @Test + void topicsConfigs() { + final Map options = createDefaultOptions(); + options.put(TOPICS.key(), ""); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateTopicsConfigs(Configuration.fromMap(options))) + .withMessage("The topics list should not be empty."); + + String invalidTopicName = "persistent://tenant/topic"; + String validTopicName = "valid-topic"; + + options.put(TOPICS.key(), invalidTopicName); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateTopicsConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "The topics name %s is not a valid topic name.", invalidTopicName)); + + options.put(TOPICS.key(), validTopicName + ";" + invalidTopicName); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateTopicsConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "The topics name %s is not a valid topic name.", invalidTopicName)); + + options.put(TOPICS.key(), validTopicName + ";" + validTopicName); + assertThatNoException() + .isThrownBy(() -> validateTopicsConfigs(Configuration.fromMap(options))); + + options.put(TOPICS.key(), validTopicName + ";"); + assertThatNoException() + .isThrownBy(() -> validateTopicsConfigs(Configuration.fromMap(options))); + } + + @Test + void startCursorConfigs() { + final Map options = createDefaultOptions(); + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "latest"); + assertThatNoException() + .isThrownBy(() -> validateStartCursorConfigs(Configuration.fromMap(options))); + + options.remove(SOURCE_START_FROM_MESSAGE_ID.key()); + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "2345123234"); + assertThatNoException() + .isThrownBy(() -> validateStartCursorConfigs(Configuration.fromMap(options))); + + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "latest"); + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "2345123234"); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateStartCursorConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "Only one of %s and %s can be specified. Detected both of them", + SOURCE_START_FROM_MESSAGE_ID, SOURCE_START_FROM_PUBLISH_TIME)); + } + + @Test + void subscriptionTypeConfigs() { + final Map options = createDefaultOptions(); + + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Exclusive"); + assertThatNoException() + .isThrownBy(() -> validateSubscriptionTypeConfigs(Configuration.fromMap(options))); + + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Shared"); + assertThatNoException() + .isThrownBy(() -> validateSubscriptionTypeConfigs(Configuration.fromMap(options))); + + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Key_Shared"); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateSubscriptionTypeConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "Only %s and %s SubscriptionType is supported. ", + SubscriptionType.Exclusive, SubscriptionType.Shared)); + + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "invalid-subscription"); + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> validateSubscriptionTypeConfigs(Configuration.fromMap(options))) + .withMessage( + "Could not parse value 'invalid-subscription' for key 'source.subscription-type'."); + } + + @Test + void sinkRoutingConfigs() { + final Map options = createDefaultOptions(); + options.put(SINK_TOPIC_ROUTING_MODE.key(), "round-robin"); + assertThatNoException() + .isThrownBy(() -> validateSinkRoutingConfigs(Configuration.fromMap(options))); + + // validation does not try to create the class + options.remove(SINK_TOPIC_ROUTING_MODE.key()); + options.put(SINK_CUSTOM_TOPIC_ROUTER.key(), "invalid-class-name"); + assertThatNoException() + .isThrownBy(() -> validateSinkRoutingConfigs(Configuration.fromMap(options))); + + options.put(SINK_TOPIC_ROUTING_MODE.key(), "round-robin"); + options.put(SINK_CUSTOM_TOPIC_ROUTER.key(), "invalid-class-name"); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateSinkRoutingConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "Only one of %s and %s can be specified. Detected both of them", + SINK_CUSTOM_TOPIC_ROUTER, SINK_TOPIC_ROUTING_MODE)); + } + + @Test + void keyFormatConfigs() { + final Map options = createDefaultOptions(); + options.put(KEY_FIELDS.key(), ""); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateKeyFormatConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "The option '%s' can only be declared if a key format is defined using '%s'.", + KEY_FIELDS.key(), KEY_FORMAT.key())); + + options.put(KEY_FORMAT.key(), "json"); + options.remove(KEY_FIELDS.key()); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateKeyFormatConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "A key format '%s' requires the declaration of one or more of key fields using '%s'.", + KEY_FORMAT.key(), KEY_FIELDS.key())); + + options.put(KEY_FORMAT.key(), "json"); + options.put(KEY_FIELDS.key(), ""); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateKeyFormatConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "A key format '%s' requires the declaration of one or more of key fields using '%s'.", + KEY_FORMAT.key(), KEY_FIELDS.key())); + + options.put(KEY_FORMAT.key(), "json"); + options.put(KEY_FIELDS.key(), "k_field1"); + assertThatNoException() + .isThrownBy(() -> validateKeyFormatConfigs(Configuration.fromMap(options))); + } + + private Map createDefaultOptions() { + Map optionMap = new HashMap<>(); + return optionMap; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockPulsarAuthentication.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockPulsarAuthentication.java new file mode 100644 index 0000000000000..cbb2a867f19b3 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockPulsarAuthentication.java @@ -0,0 +1,84 @@ +/* + * 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.connector.pulsar.table.testutils; + +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationDataProvider; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.impl.auth.AuthenticationDataNull; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.CompletableFuture; + +/** A class to verify Pulsar authentication auth params map is created properly. way to do this */ +public class MockPulsarAuthentication implements Authentication { + public static String key1 = "key1"; + public static String key2 = "key2"; + public static String value1 = "value1"; + public static String value2 = "value2"; + + @Override + public String getAuthMethodName() { + return "custom authentication"; + } + + @Override + public AuthenticationDataProvider getAuthData() { + return new AuthenticationDataNull(); + } + + @Override + public AuthenticationDataProvider getAuthData(String brokerHostName) { + return new AuthenticationDataNull(); + } + + @Override + public void authenticationStage( + String requestUrl, + AuthenticationDataProvider authData, + Map previousResHeaders, + CompletableFuture> authFuture) { + Authentication.super.authenticationStage( + requestUrl, authData, previousResHeaders, authFuture); + } + + @Override + public Set> newRequestHeader( + String hostName, + AuthenticationDataProvider authData, + Map previousResHeaders) { + return new HashSet<>(); + } + + @Override + public void configure(Map authParams) { + assert Objects.equals(authParams.get(key1), value1); + assert Objects.equals(authParams.get(key2), value2); + } + + @Override + public void start() throws PulsarClientException {} + + @Override + public void close() throws IOException {} +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockTopicRouter.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockTopicRouter.java new file mode 100644 index 0000000000000..b5ef53841aaee --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockTopicRouter.java @@ -0,0 +1,37 @@ +/* + * 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.connector.pulsar.table.testutils; + +import org.apache.flink.connector.pulsar.sink.writer.context.PulsarSinkContext; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; +import org.apache.flink.table.data.RowData; + +import java.util.List; + +/** A mock topic Router for testing purposes only. */ +public class MockTopicRouter implements TopicRouter { + + private static final long serialVersionUID = 1316133122715449818L; + + @Override + public String route( + RowData rowData, String key, List partitions, PulsarSinkContext context) { + return "never-exist-topic"; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/PulsarTableTestUtils.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/PulsarTableTestUtils.java new file mode 100644 index 0000000000000..f43be2d63eebf --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/PulsarTableTestUtils.java @@ -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.connector.pulsar.table.testutils; + +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + +/** Util class for verify testing results. */ +public class PulsarTableTestUtils { + public static List collectRows(Table table, int expectedSize) throws Exception { + final TableResult result = table.execute(); + final List collectedRows = new ArrayList<>(); + try (CloseableIterator iterator = result.collect()) { + while (collectedRows.size() < expectedSize && iterator.hasNext()) { + collectedRows.add(iterator.next()); + } + } + result.getJobClient() + .ifPresent( + jc -> { + try { + jc.cancel().get(5, TimeUnit.SECONDS); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + + return collectedRows; + } +}