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/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 03d5ba3e1339e..1f26e834749fa 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.connector.pulsar.source.enumerator.cursor.MessageIdUtils.nextMessageId;
import static org.apache.flink.connector.pulsar.source.enumerator.cursor.MessageIdUtils.unwrapMessageId;
@@ -61,4 +63,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 da51a58e9433c..ee1d43723a414 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, boolean inclusive) {
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/cursor/stop/EventTimestampStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java
index d2a44ea362d80..df5dee80d0c9c 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java
@@ -22,6 +22,8 @@
import org.apache.pulsar.client.api.Message;
+import java.util.Objects;
+
/** Stop consuming message at the given event time. */
public class EventTimestampStopCursor implements StopCursor {
private static final long serialVersionUID = 2391576769339369027L;
@@ -39,4 +41,21 @@ public StopCondition shouldStop(Message> message) {
long eventTime = message.getEventTime();
return StopCondition.compare(timestamp, eventTime, inclusive);
}
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ EventTimestampStopCursor that = (EventTimestampStopCursor) 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/cursor/stop/LatestMessageStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java
index 0de963ed4fb53..0311e4a8fb362 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java
@@ -25,6 +25,8 @@
import org.apache.pulsar.client.api.Message;
import org.apache.pulsar.client.api.MessageId;
+import java.util.Objects;
+
import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyAdmin;
/**
@@ -55,4 +57,21 @@ public void open(PulsarAdmin admin, TopicPartition partition) {
this.messageId = sneakyAdmin(() -> admin.topics().getLastMessageId(topic));
}
}
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ LatestMessageStopCursor that = (LatestMessageStopCursor) o;
+ return inclusive == that.inclusive && Objects.equals(messageId, that.messageId);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(messageId, inclusive);
+ }
}
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java
index 03d83aa4495d1..4532e45df6e3d 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java
@@ -24,6 +24,8 @@
import org.apache.pulsar.client.api.MessageId;
import org.apache.pulsar.client.impl.MessageIdImpl;
+import java.util.Objects;
+
import static org.apache.flink.connector.pulsar.source.enumerator.cursor.MessageIdUtils.unwrapMessageId;
import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.pulsar.client.api.MessageId.earliest;
@@ -56,4 +58,21 @@ public StopCondition shouldStop(Message> message) {
MessageId current = message.getMessageId();
return StopCondition.compare(messageId, current, inclusive);
}
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ MessageIdStopCursor that = (MessageIdStopCursor) o;
+ return inclusive == that.inclusive && messageId.equals(that.messageId);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(messageId, inclusive);
+ }
}
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java
index 3eb035634ae0a..27dfc6e30eff8 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java
@@ -30,4 +30,15 @@ public class NeverStopCursor implements StopCursor {
public StopCondition shouldStop(Message> message) {
return StopCondition.CONTINUE;
}
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ return true;
+ }
}
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java
index 2dfdd7658428a..cca722dd83d9c 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java
@@ -22,6 +22,8 @@
import org.apache.pulsar.client.api.Message;
+import java.util.Objects;
+
/** Stop consuming message at the given publish time. */
public class PublishTimestampStopCursor implements StopCursor {
private static final long serialVersionUID = 4386276745339324527L;
@@ -39,4 +41,21 @@ public StopCondition shouldStop(Message> message) {
long publishTime = message.getPublishTime();
return StopCondition.compare(timestamp, publishTime, inclusive);
}
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ PublishTimestampStopCursor that = (PublishTimestampStopCursor) 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/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..867025de457e4
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java
@@ -0,0 +1,305 @@
+/*
+ * 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.source.enumerator.cursor.StopCursor;
+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.getStopCursor;
+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.EXPLICIT;
+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_STOP_AFTER_MESSAGE_ID;
+import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_MESSAGE_ID;
+import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_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.PulsarTableOptions.VALUE_FORMAT;
+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.SINK_PARALLELISM;
+import static org.apache.pulsar.shade.org.apache.commons.lang3.RandomStringUtils.randomAlphabetic;
+
+/**
+ * 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";
+
+ public static final String DEFAULT_SUBSCRIPTION_NAME_PREFIX = "flink-sql-connector-pulsar-";
+
+ public static final boolean UPSERT_DISABLED = false;
+
+ @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(), helper);
+
+ validateTableSourceOptions(tableOptions);
+
+ // Retrieve configs
+ final List topics = getTopicListFromOptions(tableOptions);
+ final StartCursor startCursor = getStartCursor(tableOptions);
+ final StopCursor stopCursor = getStopCursor(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));
+ // Set random subscriptionName if not provided
+ properties.setProperty(
+ PULSAR_SUBSCRIPTION_NAME.key(),
+ tableOptions
+ .getOptional(SOURCE_SUBSCRIPTION_NAME)
+ .orElse(DEFAULT_SUBSCRIPTION_NAME_PREFIX + randomAlphabetic(5)));
+ // 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,
+ UPSERT_DISABLED);
+
+ // Set default values for configuration not exposed to user.
+ final DecodingFormat> decodingFormatForMetadataPushdown =
+ valueDecodingFormat;
+ final ChangelogMode changelogMode = decodingFormatForMetadataPushdown.getChangelogMode();
+
+ return new PulsarTableSource(
+ deserializationSchemaFactory,
+ decodingFormatForMetadataPushdown,
+ changelogMode,
+ topics,
+ properties,
+ startCursor,
+ stopCursor,
+ 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(), 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,
+ UPSERT_DISABLED);
+
+ // 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;
+ }
+
+ @Override
+ public Set> requiredOptions() {
+ return Stream.of(TOPICS, ADMIN_URL, SERVICE_URL).collect(Collectors.toSet());
+ }
+
+ @Override
+ public Set> optionalOptions() {
+ return Stream.of(
+ FactoryUtil.FORMAT,
+ VALUE_FORMAT,
+ SOURCE_SUBSCRIPTION_NAME,
+ SOURCE_SUBSCRIPTION_TYPE,
+ SOURCE_START_FROM_MESSAGE_ID,
+ SOURCE_START_FROM_PUBLISH_TIME,
+ SOURCE_STOP_AT_MESSAGE_ID,
+ SOURCE_STOP_AFTER_MESSAGE_ID,
+ SOURCE_STOP_AT_PUBLISH_TIME,
+ SINK_CUSTOM_TOPIC_ROUTER,
+ SINK_TOPIC_ROUTING_MODE,
+ SINK_MESSAGE_DELAY_INTERVAL,
+ SINK_PARALLELISM,
+ KEY_FORMAT,
+ KEY_FIELDS,
+ EXPLICIT)
+ .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,
+ SOURCE_STOP_AT_MESSAGE_ID,
+ SOURCE_STOP_AFTER_MESSAGE_ID,
+ SOURCE_STOP_AT_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..76f0f509d6b21
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java
@@ -0,0 +1,344 @@
+/*
+ * 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.connector.pulsar.source.enumerator.cursor.StopCursor;
+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.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_STOP_AFTER_MESSAGE_ID;
+import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_MESSAGE_ID;
+import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_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.PulsarTableOptions.VALUE_FORMAT;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * 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() {}
+
+ public static final String TOPIC_LIST_DELIMITER = ";";
+
+ // --------------------------------------------------------------------------------------------
+ // 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, FactoryUtil.FORMAT)
+ .orElseGet(
+ () ->
+ helper.discoverDecodingFormat(
+ DeserializationFormatFactory.class, VALUE_FORMAT));
+ }
+
+ public static EncodingFormat> getValueEncodingFormat(
+ FactoryUtil.TableFactoryHelper helper) {
+ return helper.discoverOptionalEncodingFormat(
+ SerializationFormatFactory.class, FactoryUtil.FORMAT)
+ .orElseGet(
+ () ->
+ helper.discoverEncodingFormat(
+ SerializationFormatFactory.class, VALUE_FORMAT));
+ }
+
+ /**
+ * 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();
+ 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();
+ 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 Map configs = ((Configuration) tableOptions).toMap();
+ return getPulsarProperties(configs);
+ }
+
+ public static Properties getPulsarProperties(Map configs) {
+ return getPulsarPropertiesWithPrefix(configs, "pulsar");
+ }
+
+ public static Properties getPulsarPropertiesWithPrefix(
+ ReadableConfig tableOptions, String prefix) {
+ final Map configs = ((Configuration) tableOptions).toMap();
+ return getPulsarPropertiesWithPrefix(configs, prefix);
+ }
+
+ public static Properties getPulsarPropertiesWithPrefix(
+ Map configs, String prefix) {
+ final Properties pulsarProperties = new Properties();
+ configs.keySet().stream()
+ .filter(key -> key.startsWith(prefix))
+ .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 StopCursor getStopCursor(ReadableConfig tableOptions) {
+ if (tableOptions.getOptional(SOURCE_STOP_AT_MESSAGE_ID).isPresent()) {
+ return parseAtMessageIdStopCursor(tableOptions.get(SOURCE_STOP_AT_MESSAGE_ID));
+ } else if (tableOptions.getOptional(SOURCE_STOP_AFTER_MESSAGE_ID).isPresent()) {
+ return parseAfterMessageIdStopCursor(tableOptions.get(SOURCE_STOP_AFTER_MESSAGE_ID));
+ } else if (tableOptions.getOptional(SOURCE_STOP_AT_PUBLISH_TIME).isPresent()) {
+ return parseAtPublishTimeStopCursor(tableOptions.get(SOURCE_STOP_AT_PUBLISH_TIME));
+ } else {
+ return StopCursor.never();
+ }
+ }
+
+ 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 StartCursor.fromMessageId(parseMessageIdString(config));
+ }
+ }
+
+ protected static StartCursor parsePublishTimeStartCursor(Long config) {
+ return StartCursor.fromPublishTime(config);
+ }
+
+ protected static StopCursor parseAtMessageIdStopCursor(String config) {
+ if (Objects.equals(config, "never")) {
+ return StopCursor.never();
+ } else if (Objects.equals(config, "latest")) {
+ return StopCursor.latest();
+ } else {
+ return StopCursor.atMessageId(parseMessageIdString(config));
+ }
+ }
+
+ protected static StopCursor parseAfterMessageIdStopCursor(String config) {
+ return StopCursor.afterMessageId(parseMessageIdString(config));
+ }
+
+ protected static StopCursor parseAtPublishTimeStopCursor(Long config) {
+ return StopCursor.atPublishTime(config);
+ }
+
+ protected static MessageIdImpl parseMessageIdString(String config) {
+ String[] tokens = config.split(":", 3);
+ checkArgument(tokens.length == 3, "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 messageId;
+ } catch (NumberFormatException e) {
+ throw new IllegalArgumentException(
+ "MessageId format must be ledgerId:entryId:partitionId. "
+ + "Each id should be able to parsed to long type.");
+ }
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // 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);
+ }
+ }
+
+ 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();
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Table Topic Name Utils
+ // --------------------------------------------------------------------------------------------
+
+ public static boolean hasMultipleTopics(String topicsConfigString) {
+ checkNotNull(topicsConfigString);
+ String[] topics = topicsConfigString.split(TOPIC_LIST_DELIMITER);
+ return topics.length > 1;
+ }
+
+ public static String getFirstTopic(String topicsConfigString) {
+ checkNotNull(topicsConfigString);
+ String[] topics = topicsConfigString.split(TOPIC_LIST_DELIMITER);
+ checkArgument(topics.length > 0);
+ return topics[0];
+ }
+}
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..2f3d9d602a3db
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.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.annotation.PublicEvolving;
+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}.
+ */
+@PublicEvolving
+public final class PulsarTableOptions {
+
+ private PulsarTableOptions() {}
+
+ public static final ConfigOption> TOPICS =
+ ConfigOptions.key("topics")
+ .stringType()
+ .asList()
+ .noDefaultValue()
+ .withDescription(
+ Description.builder()
+ .text(
+ "Topic name(s) the table reads data from. It can be a single topic name or a list of topic names separated by a semicolon symbol (%s) like %s.",
+ code(";"), code("topic-1;topic-2"))
+ .build());
+
+ // --------------------------------------------------------------------------------------------
+ // Table Source Options
+ // --------------------------------------------------------------------------------------------
+
+ public static final ConfigOption SOURCE_SUBSCRIPTION_TYPE =
+ ConfigOptions.key("source.subscription-type")
+ .enumType(SubscriptionType.class)
+ .defaultValue(SubscriptionType.Exclusive)
+ .withDescription(
+ Description.builder()
+ .text(
+ "The [subscription type](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-subscriptions) that is supported by the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). Currently, only %s and %s subscription types are supported.",
+ code("Exclusive"), code("Shared"))
+ .build());
+
+ /**
+ * 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()
+ .noDefaultValue()
+ .withDescription(
+ Description.builder()
+ .text(
+ "The subscription name of the consumer that is used by the runtime [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). This argument is required for constructing the consumer.")
+ .build());
+
+ public static final ConfigOption SOURCE_START_FROM_MESSAGE_ID =
+ ConfigOptions.key("source.start.message-id")
+ .stringType()
+ .noDefaultValue()
+ .withDescription(
+ Description.builder()
+ .text(
+ "Optional message id used to specify a consuming starting point for "
+ + "source. Use %s, %s or pass in a message id "
+ + "representation in %s, "
+ + "such as %s",
+ code("earliest"),
+ code("latest"),
+ code("ledgerId:entryId:partitionId"),
+ code("12:2:-1"))
+ .build());
+
+ public static final ConfigOption SOURCE_START_FROM_PUBLISH_TIME =
+ ConfigOptions.key("source.start.publish-time")
+ .longType()
+ .noDefaultValue()
+ .withDescription(
+ Description.builder()
+ .text(
+ "(Optional) the publish timestamp that is used to specify a starting point for the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source) to consume data.")
+ .build());
+
+ public static final ConfigOption SOURCE_STOP_AT_MESSAGE_ID =
+ ConfigOptions.key("source.stop.at-message-id")
+ .stringType()
+ .noDefaultValue()
+ .withDescription(
+ "Optional message id used to specify a stop cursor for the unbounded sql "
+ + "source. Use \"never\", \"latest\" or pass in a message id "
+ + "representation in \"ledgerId:entryId:partitionId\", "
+ + "such as \"12:2:-1\"");
+
+ public static final ConfigOption SOURCE_STOP_AFTER_MESSAGE_ID =
+ ConfigOptions.key("source.stop.after-message-id")
+ .stringType()
+ .noDefaultValue()
+ .withDescription(
+ "Optional message id used to specify a stop position but include the "
+ + "given message in the consuming result for the unbounded sql "
+ + "source. Pass in a message id "
+ + "representation in \"ledgerId:entryId:partitionId\", "
+ + "such as \"12:2:-1\". ");
+
+ public static final ConfigOption SOURCE_STOP_AT_PUBLISH_TIME =
+ ConfigOptions.key("source.stop.at-publish-time")
+ .longType()
+ .noDefaultValue()
+ .withDescription(
+ "Optional publish timestamp used to specify a stop cursor"
+ + " for the unbounded sql source.");
+
+ // --------------------------------------------------------------------------------------------
+ // Table Sink Options
+ // --------------------------------------------------------------------------------------------
+
+ public static final ConfigOption SINK_CUSTOM_TOPIC_ROUTER =
+ ConfigOptions.key("sink.custom-topic-router")
+ .stringType()
+ .noDefaultValue()
+ .withDescription(
+ Description.builder()
+ .text(
+ "(Optional) the custom topic router class URL that is used in the [Pulsar DataStream sink connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-sink). If this option is provided, the %s option will be ignored.",
+ code("sink.topic-routing-mode"))
+ .build());
+
+ public static final ConfigOption SINK_TOPIC_ROUTING_MODE =
+ ConfigOptions.key("sink.topic-routing-mode")
+ .enumType(TopicRoutingMode.class)
+ .defaultValue(TopicRoutingMode.ROUND_ROBIN)
+ .withDescription(
+ Description.builder()
+ .text(
+ "(Optional) the topic routing mode. Available options are %s and %s. By default, it is set to %s. If you want to use a custom topic router, use the %s option to determine the partition for a particular message.",
+ code("round-robin"),
+ code("message-key-hash"),
+ code("round-robin"),
+ code("sink.custom-topic-router"))
+ .build());
+
+ public static final ConfigOption SINK_MESSAGE_DELAY_INTERVAL =
+ ConfigOptions.key("sink.message-delay-interval")
+ .durationType()
+ .defaultValue(Duration.ZERO)
+ .withDescription(
+ "(Optional) the message delay delivery interval that is used in the [Pulsar DataStream sink connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-sink).");
+
+ // --------------------------------------------------------------------------------------------
+ // 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(
+ "An explicit list of physical columns from the table schema that are decoded/encoded from the key bytes of a Pulsar message. By default, this list is empty and thus a key is undefined.");
+
+ public static final ConfigOption VALUE_FORMAT =
+ ConfigOptions.key("value" + FORMAT_SUFFIX)
+ .stringType()
+ .noDefaultValue()
+ .withDescription(
+ "Defines the format identifier for decoding/encoding value data. "
+ + "The identifier is used to discover a suitable format factory.");
+
+ // --------------------------------------------------------------------------------------------
+ // 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());
+
+ public static final ConfigOption EXPLICIT =
+ ConfigOptions.key("explicit")
+ .booleanType()
+ .defaultValue(true)
+ .withDescription("Indicate if the table is an explicit Flink table.");
+}
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..42f8855d5082c
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java
@@ -0,0 +1,198 @@
+/*
+ * 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.ConfigOption;
+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.flink.shaded.guava30.com.google.common.collect.Sets;
+
+import org.apache.pulsar.client.api.SubscriptionType;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+
+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_STOP_AFTER_MESSAGE_ID;
+import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_MESSAGE_ID;
+import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_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. */
+public class PulsarTableValidationUtils {
+
+ private PulsarTableValidationUtils() {}
+
+ public static void validatePrimaryKeyConstraints(
+ ObjectIdentifier tableName,
+ int[] primaryKeyIndexes,
+ FactoryUtil.TableFactoryHelper helper) {
+ final DecodingFormat> format =
+ getValueDecodingFormat(helper);
+ if (primaryKeyIndexes.length > 0
+ && format.getChangelogMode().containsOnly(RowKind.INSERT)) {
+ 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(), format));
+ }
+ }
+
+ public static void validateTableSourceOptions(ReadableConfig tableOptions) {
+ validateTopicsConfigs(tableOptions);
+ validateStartCursorConfigs(tableOptions);
+ validateStopCursorConfigs(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 validateStopCursorConfigs(ReadableConfig tableOptions) {
+ Set> conflictConfigOptions =
+ Sets.newHashSet(
+ SOURCE_STOP_AT_MESSAGE_ID,
+ SOURCE_STOP_AFTER_MESSAGE_ID,
+ SOURCE_STOP_AT_PUBLISH_TIME);
+
+ long configsNums =
+ conflictConfigOptions.stream()
+ .map(tableOptions::getOptional)
+ .filter(Optional::isPresent)
+ .count();
+
+ if (configsNums > 1) {
+ throw new ValidationException(
+ String.format(
+ "Only one of %s, %s and %s can be specified. Detected more than 1 of them",
+ SOURCE_STOP_AT_MESSAGE_ID,
+ SOURCE_STOP_AFTER_MESSAGE_ID,
+ SOURCE_STOP_AT_PUBLISH_TIME));
+ }
+ }
+
+ protected static void validateSubscriptionTypeConfigs(ReadableConfig tableOptions) {
+ SubscriptionType subscriptionType = tableOptions.get(SOURCE_SUBSCRIPTION_TYPE);
+
+ if (subscriptionType == SubscriptionType.Failover) {
+ throw new ValidationException(
+ String.format(
+ "%s SubscriptionType is not supported. ", SubscriptionType.Failover));
+ }
+ }
+
+ 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));
+ }
+ }
+
+ protected static void validateUpsertModeKeyConstraints(
+ ReadableConfig tableOptions, int[] primaryKeyIndexes) {
+ if (!tableOptions.getOptional(KEY_FIELDS).isPresent()) {
+ throw new ValidationException(
+ String.format(
+ "Upsert mode requires key.fields set to the primary key fields, should be set"));
+ }
+
+ if (tableOptions.getOptional(KEY_FIELDS).get().size() == 0
+ || primaryKeyIndexes.length == 0) {
+ throw new ValidationException(
+ "'upsert-pulsar' require to define a PRIMARY KEY constraint. "
+ + "The PRIMARY KEY specifies which columns should be read from or write to the Pulsar message key. "
+ + "The PRIMARY KEY also defines records in the 'upsert-pulsar' table should update or delete on which keys.");
+ }
+ }
+}
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..68271438247b7
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java
@@ -0,0 +1,131 @@
+/*
+ * 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;
+
+ private final boolean upsertMode;
+
+ public PulsarTableSerializationSchema(
+ @Nullable SerializationSchema keySerialization,
+ RowData.FieldGetter[] keyFieldGetters,
+ SerializationSchema valueSerialization,
+ RowData.FieldGetter[] valueFieldGetters,
+ PulsarWritableMetadata writableMetadata,
+ boolean upsertMode) {
+ this.keySerialization = keySerialization;
+ this.keyFieldGetters = checkNotNull(keyFieldGetters);
+ this.valueSerialization = checkNotNull(valueSerialization);
+ this.valueFieldGetters = checkNotNull(valueFieldGetters);
+ this.writableMetadata = checkNotNull(writableMetadata);
+ this.upsertMode = upsertMode;
+ }
+
+ @Override
+ public void open(
+ SerializationSchema.InitializationContext initializationContext,
+ PulsarSinkContext sinkContext,
+ SinkConfiguration sinkConfiguration)
+ throws Exception {
+ if (keySerialization != null) {
+ keySerialization.open(initializationContext);
+ }
+ valueSerialization.open(initializationContext);
+ }
+
+ @Override
+ public PulsarMessage> serialize(RowData consumedRow, PulsarSinkContext sinkContext) {
+
+ PulsarMessageBuilder messageBuilder = new PulsarMessageBuilder<>();
+
+ final RowKind kind = consumedRow.getRowKind();
+ final byte[] serializedData;
+ if (upsertMode) {
+ if (kind == RowKind.DELETE || kind == RowKind.UPDATE_BEFORE) {
+ // use null message as the tombstone message
+ serializedData = null;
+ } else {
+ final RowData valueRow = createProjectedRow(consumedRow, kind, valueFieldGetters);
+ valueRow.setRowKind(RowKind.INSERT);
+ serializedData = valueSerialization.serialize(valueRow);
+ }
+ } else {
+ final RowData valueRow = createProjectedRow(consumedRow, kind, valueFieldGetters);
+ serializedData = valueSerialization.serialize(valueRow);
+ }
+
+ // apply metadata
+ writableMetadata.applyWritableMetadataInMessage(consumedRow, messageBuilder);
+
+ // get key row data
+ if (keySerialization != null) {
+ final RowData keyRow = createProjectedRow(consumedRow, RowKind.INSERT, keyFieldGetters);
+ messageBuilder.keyBytes(keySerialization.serialize(keyRow));
+ }
+
+ 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..292c30b7e8a4d
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchemaFactory.java
@@ -0,0 +1,160 @@
+/*
+ * 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;
+
+/**
+ * A factory class which contains required fields mapping and encoding format information to
+ * construct a {@link 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;
+
+ private final boolean upsertMode;
+
+ public PulsarTableSerializationSchemaFactory(
+ DataType physicalDataType,
+ @Nullable EncodingFormat> keyEncodingFormat,
+ int[] keyProjection,
+ EncodingFormat> valueEncodingFormat,
+ int[] valueProjection,
+ boolean upsertMode) {
+ this.physicalDataType = checkNotNull(physicalDataType);
+ this.keyEncodingFormat = keyEncodingFormat;
+ this.keyProjection = checkNotNull(keyProjection);
+ this.valueEncodingFormat = checkNotNull(valueEncodingFormat);
+ this.valueProjection = checkNotNull(valueProjection);
+ this.writableMetadataKeys = Collections.emptyList();
+ this.upsertMode = upsertMode;
+ }
+
+ 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,
+ upsertMode);
+ }
+
+ 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)
+ && Objects.equals(upsertMode, that.upsertMode);
+ }
+
+ @Override
+ public int hashCode() {
+ int result =
+ Objects.hash(
+ physicalDataType,
+ keyEncodingFormat,
+ valueEncodingFormat,
+ writableMetadataKeys,
+ upsertMode);
+ 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..909b04095f925
--- /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 metadata that is non virtual 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(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..b769db3488bd7
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarReadableMetadata.java
@@ -0,0 +1,142 @@
+/*
+ * 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();
+ }
+
+ 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..603da1e950b6f
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarRowDataConverter.java
@@ -0,0 +1,134 @@
+/*
+ * 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;
+
+ private final boolean upsertMode;
+
+ public PulsarRowDataConverter(
+ int physicalArity,
+ int[] keyProjection,
+ int[] valueProjection,
+ PulsarReadableMetadata readableMetadata,
+ boolean upsertMode) {
+ this.physicalArity = physicalArity;
+ this.keyProjection = keyProjection;
+ this.valueProjection = valueProjection;
+ this.readableMetadata = readableMetadata;
+ this.upsertMode = upsertMode;
+ }
+
+ 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)));
+ }
+ }
+
+ public void projectToRowWithNullValueRow(
+ Message> message, List keyRowDataList, Collector collector) {
+ for (RowData keyRow : keyRowDataList) {
+ emitRow((GenericRowData) keyRow, null, collector, message);
+ }
+ }
+
+ private void emitRow(
+ @Nullable GenericRowData physicalKeyRow,
+ @Nullable GenericRowData physicalValueRow,
+ Collector collector,
+ Message> message) {
+
+ final RowKind rowKind;
+ if (physicalValueRow == null) {
+ if (upsertMode) {
+ rowKind = RowKind.DELETE;
+ } else {
+ 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..4edf89f91a440
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchema.java
@@ -0,0 +1,115 @@
+/*
+ * 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 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;
+
+ private final boolean upsertMode;
+
+ public PulsarTableDeserializationSchema(
+ @Nullable DeserializationSchema keyDeserialization,
+ DeserializationSchema valueDeserialization,
+ TypeInformation producedTypeInfo,
+ PulsarRowDataConverter rowDataConverter,
+ boolean upsertMode) {
+ this.keyDeserialization = keyDeserialization;
+ this.valueDeserialization = checkNotNull(valueDeserialization);
+ this.rowDataConverter = checkNotNull(rowDataConverter);
+ this.producedTypeInfo = checkNotNull(producedTypeInfo);
+ this.upsertMode = upsertMode;
+ }
+
+ @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 key row data
+ List keyRowData = new ArrayList<>();
+ if (keyDeserialization != null) {
+ keyDeserialization.deserialize(message.getKeyBytes(), new ListCollector<>(keyRowData));
+ }
+
+ // Get the value row data
+ List valueRowData = new ArrayList<>();
+
+ if (upsertMode && message.getData().length == 0) {
+ checkNotNull(keyDeserialization, "upsert mode must specify a key format");
+ rowDataConverter.projectToRowWithNullValueRow(message, keyRowData, collector);
+ return;
+ }
+
+ valueDeserialization.deserialize(message.getData(), new ListCollector<>(valueRowData));
+
+ rowDataConverter.projectToProducedRowAndCollect(
+ message, keyRowData, valueRowData, collector);
+ }
+
+ @Override
+ public TypeInformation getProducedType() {
+ return producedTypeInfo;
+ }
+}
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..70b7678d457ce
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchemaFactory.java
@@ -0,0 +1,215 @@
+/*
+ * 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 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;
+
+ private final boolean upsertMode;
+
+ public PulsarTableDeserializationSchemaFactory(
+ DataType physicalDataType,
+ @Nullable DecodingFormat> keyDecodingFormat,
+ int[] keyProjection,
+ DecodingFormat> valueDecodingFormat,
+ int[] valueProjection,
+ boolean upsertMode) {
+ 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();
+ this.upsertMode = upsertMode;
+ }
+
+ 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,
+ upsertMode);
+
+ return new PulsarTableDeserializationSchema(
+ keyDeserialization,
+ valueDeserialization,
+ producedTypeInfo,
+ rowDataConverter,
+ upsertMode);
+ }
+
+ 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)
+ && Objects.equals(upsertMode, that.upsertMode);
+ }
+
+ @Override
+ public int hashCode() {
+ int result =
+ Objects.hash(
+ physicalDataType,
+ keyDecodingFormat,
+ valueDecodingFormat,
+ producedDataType,
+ connectorMetadataKeys,
+ upsertMode);
+ 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..92ee64b009ffd
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableSource.java
@@ -0,0 +1,229 @@
+/*
+ * 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.enumerator.cursor.StopCursor;
+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;
+
+ private final ChangelogMode changelogMode;
+
+ // --------------------------------------------------------------------------------------------
+ // PulsarSource needed attributes
+ // --------------------------------------------------------------------------------------------
+
+ private final List topics;
+
+ private final Properties properties;
+
+ private final StartCursor startCursor;
+
+ private final StopCursor stopCursor;
+
+ private final SubscriptionType subscriptionType;
+
+ public PulsarTableSource(
+ PulsarTableDeserializationSchemaFactory deserializationSchemaFactory,
+ DecodingFormat> decodingFormatForReadingMetadata,
+ ChangelogMode changelogMode,
+ List topics,
+ Properties properties,
+ StartCursor startCursor,
+ StopCursor stopCursor,
+ SubscriptionType subscriptionType) {
+ // Format attributes
+ this.deserializationSchemaFactory = checkNotNull(deserializationSchemaFactory);
+ this.decodingFormatForReadingMetadata = checkNotNull(decodingFormatForReadingMetadata);
+ this.changelogMode = changelogMode;
+ // DataStream connector attributes
+ this.topics = topics;
+ this.properties = checkNotNull(properties);
+ this.startCursor = checkNotNull(startCursor);
+ this.stopCursor = checkNotNull(stopCursor);
+ this.subscriptionType = checkNotNull(subscriptionType);
+ }
+
+ @Override
+ public ChangelogMode getChangelogMode() {
+ return changelogMode;
+ }
+
+ @Override
+ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext context) {
+ PulsarDeserializationSchema deserializationSchema =
+ deserializationSchemaFactory.createPulsarDeserialization(context);
+ PulsarSource source =
+ PulsarSource.builder()
+ .setTopics(topics)
+ .setStartCursor(startCursor)
+ .setUnboundedStopCursor(stopCursor)
+ .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,
+ changelogMode,
+ topics,
+ properties,
+ startCursor,
+ stopCursor,
+ 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(changelogMode, that.changelogMode)
+ && Objects.equals(topics, that.topics)
+ && Objects.equals(properties, that.properties)
+ && Objects.equals(startCursor, that.startCursor)
+ && Objects.equals(stopCursor, that.stopCursor)
+ && subscriptionType == that.subscriptionType;
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(
+ deserializationSchemaFactory,
+ decodingFormatForReadingMetadata,
+ changelogMode,
+ topics,
+ properties,
+ startCursor,
+ stopCursor,
+ 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/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/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..eb637bb7ec654
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java
@@ -0,0 +1,441 @@
+/*
+ * 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.source.enumerator.cursor.StopCursor;
+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.PulsarTableFactory.UPSERT_DISABLED;
+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 org.apache.flink.connector.pulsar.table.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
+ 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},
+ UPSERT_DISABLED);
+
+ final PulsarTableSource expectedPulsarSource =
+ new PulsarTableSource(
+ deserializationSchemaFactory,
+ valueDecodingFormat,
+ valueDecodingFormat.getChangelogMode(),
+ Lists.list(TEST_TOPIC),
+ EXPECTED_PULSAR_SOURCE_PROPERTIES,
+ StartCursor.earliest(),
+ StopCursor.never(),
+ SubscriptionType.Exclusive);
+ assertThat(actualPulsarSource).isEqualTo(expectedPulsarSource);
+
+ ScanTableSource.ScanRuntimeProvider provider =
+ actualPulsarSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE);
+ assertPulsarSourceIsSameAsExpected(provider);
+ }
+
+ @Test
+ 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},
+ UPSERT_DISABLED);
+
+ final PulsarTableSource expectedPulsarSource =
+ new PulsarTableSource(
+ deserializationSchemaFactory,
+ valueDecodingFormat,
+ valueDecodingFormat.getChangelogMode(),
+ Lists.list(TEST_TOPIC),
+ EXPECTED_PULSAR_SOURCE_PROPERTIES,
+ StartCursor.earliest(),
+ StopCursor.never(),
+ SubscriptionType.Exclusive);
+
+ assertThat(actualPulsarSource).isEqualTo(expectedPulsarSource);
+ }
+
+ @Test
+ 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},
+ UPSERT_DISABLED);
+
+ final PulsarTableSource expectedPulsarSource =
+ new PulsarTableSource(
+ deserializationSchemaFactory,
+ expectedValueFormat,
+ expectedValueFormat.getChangelogMode(),
+ Lists.list(TEST_TOPIC),
+ EXPECTED_PULSAR_SOURCE_PROPERTIES,
+ StartCursor.earliest(),
+ StopCursor.never(),
+ SubscriptionType.Exclusive);
+
+ deserializationSchemaFactory.setProducedDataType(
+ SCHEMA_WITH_METADATA.toSourceRowDataType());
+ deserializationSchemaFactory.setConnectorMetadataKeys(
+ Collections.singletonList("publish_time"));
+
+ assertThat(actualPulsarSource).isEqualTo(expectedPulsarSource);
+ }
+
+ @Test
+ 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},
+ UPSERT_DISABLED);
+
+ 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
+ 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},
+ UPSERT_DISABLED);
+
+ 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..4ee36a6876167
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java
@@ -0,0 +1,616 @@
+/*
+ * 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.JobExecutionResult;
+import org.apache.flink.connector.pulsar.table.testutils.TestingUser;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.test.util.SuccessException;
+import org.apache.flink.types.Row;
+
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Stream;
+
+import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic;
+import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric;
+import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyThrow;
+import static org.apache.flink.connector.pulsar.table.testutils.PulsarTableTestUtils.collectRows;
+import static org.apache.flink.connector.pulsar.table.testutils.TestingUser.createRandomUser;
+import static org.apache.flink.util.CollectionUtil.entry;
+import static org.apache.flink.util.CollectionUtil.map;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
+import static org.assertj.core.api.Assertions.assertThatNoException;
+
+/**
+ * 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.
+ */
+public class PulsarTableITCase extends PulsarTableTestBase {
+ protected static final String JSON_FORMAT = "json";
+ protected static final String AVRO_FORMAT = "avro";
+ protected static final String CSV_FORMAT = "csv";
+ protected static final String RAW_FORMAT = "raw";
+
+ @ParameterizedTest
+ @ValueSource(strings = {JSON_FORMAT, AVRO_FORMAT, CSV_FORMAT})
+ 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 -------------------
+ String randomTableName = randomAlphabetic(5);
+ final String createTable =
+ String.format(
+ "create table %s (\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"
+ + " 'format' = '%s'\n"
+ + ")",
+ randomTableName,
+ PulsarTableFactory.IDENTIFIER,
+ topic,
+ pulsar.operator().serviceUrl(),
+ pulsar.operator().adminUrl(),
+ format);
+
+ tableEnv.executeSql(createTable);
+
+ String initialValues =
+ String.format(
+ "INSERT INTO %s\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)",
+ randomTableName);
+ tableEnv.executeSql(initialValues).await();
+
+ String query =
+ String.format(
+ "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 %s\n"
+ + "GROUP BY TUMBLE(ts, INTERVAL '5' SECOND)",
+ randomTableName);
+
+ DataStream result = tableEnv.toDataStream(tableEnv.sqlQuery(query));
+ 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, AVRO_FORMAT, CSV_FORMAT})
+ 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
+ String randomTableName = randomAlphabetic(5);
+ final String createTable =
+ String.format(
+ "CREATE TABLE %s (\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"
+ + ")",
+ randomTableName,
+ PulsarTableFactory.IDENTIFIER,
+ topic,
+ pulsar.operator().serviceUrl(),
+ pulsar.operator().adminUrl(),
+ format,
+ format);
+
+ tableEnv.executeSql(createTable);
+
+ String initialValues =
+ String.format(
+ "INSERT INTO %s\n"
+ + "VALUES\n"
+ + " (1, 'name 1', 100, 'payload 1'),\n"
+ + " (2, 'name 2', 101, 'payload 2'),\n"
+ + " (3, 'name 3', 102, 'payload 3')",
+ randomTableName);
+ tableEnv.executeSql(initialValues).await();
+
+ final List result =
+ collectRows(
+ tableEnv.sqlQuery(String.format("SELECT * FROM %s", randomTableName)), 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, AVRO_FORMAT, CSV_FORMAT})
+ 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);
+
+ String randomTableName = randomAlphabetic(5);
+ final String createTable =
+ String.format(
+ "CREATE TABLE %s (\n"
+ + " `physical_1` STRING,\n"
+ + " `physical_2` INT,\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"
+ + ")",
+ randomTableName,
+ PulsarTableFactory.IDENTIFIER,
+ topic,
+ pulsar.operator().serviceUrl(),
+ pulsar.operator().adminUrl(),
+ format);
+ tableEnv.executeSql(createTable);
+
+ String initialValues =
+ String.format(
+ "INSERT INTO %s\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)",
+ randomTableName);
+ tableEnv.executeSql(initialValues).await();
+
+ // ---------- Consume stream from Pulsar -------------------
+
+ final List result =
+ collectRows(
+ tableEnv.sqlQuery(String.format("SELECT * FROM %s", randomTableName)), 3);
+ assertThat(result)
+ .containsExactlyInAnyOrder(
+ Row.of(
+ "data 1",
+ 1,
+ LocalDateTime.parse("2022-03-24T13:12:11.123"),
+ map(entry("k1", "C0FFEE"), entry("k2", "BABE01")),
+ true),
+ Row.of(
+ "data 2",
+ 2,
+ LocalDateTime.parse("2022-03-25T13:12:11.123"),
+ Collections.emptyMap(),
+ false),
+ Row.of(
+ "data 3",
+ 3,
+ LocalDateTime.parse("2022-03-26T13:12:11.123"),
+ map(entry("k1", "C0FFEE"), entry("k2", "BABE01")),
+ true));
+ }
+
+ @Test
+ void sendMessageWithPropertiesAndReadPropertiesMetadata() throws Exception {
+ final String sourceTopic = "source_topic_" + randomAlphanumeric(3);
+ createTestTopic(sourceTopic, 1);
+
+ // create producer and send one message
+ String value = randomAlphabetic(5);
+ Map properties = new HashMap<>();
+ properties.put("key1", "value1");
+ properties.put("key2", "value2");
+ try (Producer producer =
+ pulsar.operator().createProducer(sourceTopic, Schema.STRING)) {
+ producer.newMessage().value(value).properties(properties).send();
+ } catch (PulsarClientException e) {
+ sneakyThrow(e);
+ }
+
+ String sourceTableName = randomAlphabetic(5);
+ final String createSourceTable =
+ String.format(
+ "create table %s (\n"
+ + " name STRING\n,"
+ + " `properties` MAP METADATA\n"
+ + ") with (\n"
+ + " 'connector' = '%s',\n"
+ + " 'topics' = '%s',\n"
+ + " 'service-url' = '%s',\n"
+ + " 'admin-url' = '%s',\n"
+ + " 'format' = '%s'\n"
+ + ")",
+ sourceTableName,
+ PulsarTableFactory.IDENTIFIER,
+ sourceTopic,
+ pulsar.operator().serviceUrl(),
+ pulsar.operator().adminUrl(),
+ RAW_FORMAT);
+
+ tableEnv.executeSql(createSourceTable);
+ final List result =
+ collectRows(
+ tableEnv.sqlQuery(String.format("SELECT * FROM %s", sourceTableName)), 1);
+ assertThat(result)
+ .containsExactlyInAnyOrder(
+ Row.of(value, map(entry("key1", "value1"), entry("key2", "value2"))));
+ }
+
+ @ParameterizedTest
+ @MethodSource("provideSchemaData")
+ void readAndSelectIntoTableUsingSimpleSchema(
+ String format, Schema schema, T value, String flinkTableDataType) throws Exception {
+ final String sourceTopic = "source_topic_" + randomAlphanumeric(3);
+ createTestTopic(sourceTopic, 1);
+ pulsar.operator().sendMessage(sourceTopic, schema, value);
+
+ String sourceTableName = randomAlphabetic(5);
+ final String createSourceTable =
+ String.format(
+ "create table %s (\n"
+ + " `field_1` %s\n"
+ + ") with (\n"
+ + " 'connector' = '%s',\n"
+ + " 'topics' = '%s',\n"
+ + " 'service-url' = '%s',\n"
+ + " 'admin-url' = '%s',\n"
+ + " 'format' = '%s'\n"
+ + ")",
+ sourceTableName,
+ flinkTableDataType,
+ PulsarTableFactory.IDENTIFIER,
+ sourceTopic,
+ pulsar.operator().serviceUrl(),
+ pulsar.operator().adminUrl(),
+ format);
+
+ tableEnv.executeSql(createSourceTable);
+ final List result =
+ collectRows(
+ tableEnv.sqlQuery(String.format("SELECT * FROM %s", sourceTableName)), 1);
+ assertThat(result).containsExactlyInAnyOrder(Row.of(value));
+
+ // insert into ... select from
+
+ final String sinkTopic = "sink_topic_" + randomAlphanumeric(3);
+ createTestTopic(sinkTopic, 1);
+
+ String sinkTableName = randomAlphabetic(5);
+ pulsar.operator().sendMessage(sourceTopic, schema, value);
+ final String createSinkTable =
+ String.format(
+ "create table %s (\n"
+ + " `field_1` %s\n"
+ + ") with (\n"
+ + " 'connector' = '%s',\n"
+ + " 'topics' = '%s',\n"
+ + " 'service-url' = '%s',\n"
+ + " 'admin-url' = '%s',\n"
+ + " 'format' = '%s'\n"
+ + ")",
+ sinkTableName,
+ flinkTableDataType,
+ PulsarTableFactory.IDENTIFIER,
+ sinkTopic,
+ pulsar.operator().serviceUrl(),
+ pulsar.operator().adminUrl(),
+ format);
+
+ tableEnv.executeSql(createSinkTable);
+ tableEnv.executeSql(
+ String.format("INSERT INTO %s SELECT * FROM %s", sinkTableName, sourceTableName));
+ Message sinkResult = pulsar.operator().receiveMessage(sinkTopic, schema);
+ assertThat(sinkResult.getValue()).isEqualTo(value);
+ }
+
+ @ParameterizedTest
+ @MethodSource("provideAvroBasedSchemaData")
+ void sendMessageToTopicAndReadUsingAvroBasedSchema(
+ String format, Schema schema, TestingUser value) throws Exception {
+ final String sourceTopic = "source_topic_" + randomAlphanumeric(3);
+ createTestTopic(sourceTopic, 1);
+ pulsar.operator().sendMessage(sourceTopic, schema, value);
+
+ String sourceTableName = randomAlphabetic(5);
+ final String createSourceTable =
+ String.format(
+ "create table %s (\n"
+ + " age INT,\n"
+ + " name STRING\n"
+ + ") with (\n"
+ + " 'connector' = '%s',\n"
+ + " 'topics' = '%s',\n"
+ + " 'service-url' = '%s',\n"
+ + " 'admin-url' = '%s',\n"
+ + " 'format' = '%s'\n"
+ + ")",
+ sourceTableName,
+ PulsarTableFactory.IDENTIFIER,
+ sourceTopic,
+ pulsar.operator().serviceUrl(),
+ pulsar.operator().adminUrl(),
+ format);
+
+ tableEnv.executeSql(createSourceTable);
+ final List result =
+ collectRows(
+ tableEnv.sqlQuery(String.format("SELECT * FROM %s", sourceTableName)), 1);
+ assertThat(result).containsExactlyInAnyOrder(Row.of(value.getAge(), value.getName()));
+ }
+
+ @ParameterizedTest
+ @MethodSource("provideAvroBasedSchemaData")
+ void writeAndReadUsingAvroBasedSchema(
+ String format, Schema schema, TestingUser value) throws Exception {
+ final String sourceTopic = "source_topic_" + randomAlphanumeric(3);
+ createTestTopic(sourceTopic, 1);
+
+ String sourceTableName = randomAlphabetic(5);
+ final String createSourceTable =
+ String.format(
+ "create table %s (\n"
+ + " name STRING,\n"
+ + " age INT\n"
+ + ") with (\n"
+ + " 'connector' = '%s',\n"
+ + " 'topics' = '%s',\n"
+ + " 'service-url' = '%s',\n"
+ + " 'admin-url' = '%s',\n"
+ + " 'format' = '%s'\n"
+ + ")",
+ sourceTableName,
+ PulsarTableFactory.IDENTIFIER,
+ sourceTopic,
+ pulsar.operator().serviceUrl(),
+ pulsar.operator().adminUrl(),
+ format);
+
+ tableEnv.executeSql(createSourceTable);
+ String initialValues =
+ String.format(
+ "INSERT INTO %s\n" + "VALUES\n" + " ('%s', %s)",
+ sourceTableName, value.getName(), value.getAge());
+ tableEnv.executeSql(initialValues).await();
+
+ final List result =
+ collectRows(
+ tableEnv.sqlQuery(String.format("SELECT * FROM %s", sourceTableName)), 1);
+ assertThat(result).containsExactlyInAnyOrder(Row.of(value.getName(), value.getAge()));
+ }
+
+ @Test
+ void sendMessageToTopicAndReadUntilBoundedStopCursor() {
+ final String sourceTopic = "source_topic_" + randomAlphanumeric(3);
+ createTestTopic(sourceTopic, 1);
+ pulsar.operator().sendMessage(sourceTopic, Schema.STRING, randomAlphabetic(5));
+
+ String sourceTableName = randomAlphabetic(5);
+ final String createSourceTable =
+ String.format(
+ "create table %s (\n"
+ + " name STRING\n"
+ + ") with (\n"
+ + " 'connector' = '%s',\n"
+ + " 'topics' = '%s',\n"
+ + " 'service-url' = '%s',\n"
+ + " 'admin-url' = '%s',\n"
+ + " 'pulsar.source.partitionDiscoveryIntervalMs' = '-1',\n"
+ + " 'source.stop.at-message-id' = 'latest',\n"
+ + " 'format' = '%s'\n"
+ + ")",
+ sourceTableName,
+ PulsarTableFactory.IDENTIFIER,
+ sourceTopic,
+ pulsar.operator().serviceUrl(),
+ pulsar.operator().adminUrl(),
+ RAW_FORMAT);
+
+ tableEnv.executeSql(createSourceTable);
+ JobClient jobClient =
+ tableEnv.sqlQuery(String.format("SELECT * FROM %s", sourceTableName))
+ .execute()
+ .getJobClient()
+ .get();
+ assertThatNoException()
+ .isThrownBy(
+ () -> {
+ JobExecutionResult result =
+ jobClient.getJobExecutionResult().get(1, TimeUnit.MINUTES);
+ });
+ }
+
+ @Test
+ void sendMessageToTopicAndReadUntilBoundedStopCursorButHasPartitionDiscovery() {
+ final String sourceTopic = "source_topic_" + randomAlphanumeric(3);
+ createTestTopic(sourceTopic, 1);
+ pulsar.operator().sendMessage(sourceTopic, Schema.STRING, randomAlphabetic(5));
+
+ String sourceTableName = randomAlphabetic(5);
+ final String createSourceTable =
+ String.format(
+ "create table %s (\n"
+ + " name STRING\n"
+ + ") with (\n"
+ + " 'connector' = '%s',\n"
+ + " 'topics' = '%s',\n"
+ + " 'service-url' = '%s',\n"
+ + " 'admin-url' = '%s',\n"
+ + " 'source.stop.at-message-id' = 'latest',\n"
+ + " 'format' = '%s'\n"
+ + ")",
+ sourceTableName,
+ PulsarTableFactory.IDENTIFIER,
+ sourceTopic,
+ pulsar.operator().serviceUrl(),
+ pulsar.operator().adminUrl(),
+ RAW_FORMAT);
+
+ tableEnv.executeSql(createSourceTable);
+ JobClient jobClient =
+ tableEnv.sqlQuery(String.format("SELECT * FROM %s", sourceTableName))
+ .execute()
+ .getJobClient()
+ .get();
+ assertThatExceptionOfType(TimeoutException.class)
+ .isThrownBy(
+ () -> {
+ JobExecutionResult result =
+ jobClient.getJobExecutionResult().get(1, TimeUnit.MINUTES);
+ });
+ }
+
+ 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(Row 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 static Stream provideSchemaData() {
+ return Stream.of(
+ Arguments.of(RAW_FORMAT, Schema.INT8, (byte) 0xa, DataTypes.TINYINT().toString()),
+ Arguments.of(
+ RAW_FORMAT, Schema.INT16, Short.MAX_VALUE, DataTypes.SMALLINT().toString()),
+ Arguments.of(
+ RAW_FORMAT, Schema.INT32, Integer.MAX_VALUE, DataTypes.INT().toString()),
+ Arguments.of(
+ RAW_FORMAT, Schema.INT64, Long.MAX_VALUE, DataTypes.BIGINT().toString()),
+ Arguments.of(
+ RAW_FORMAT, Schema.FLOAT, Float.MAX_VALUE, DataTypes.FLOAT().toString()),
+ Arguments.of(
+ RAW_FORMAT, Schema.DOUBLE, Double.MAX_VALUE, DataTypes.DOUBLE().toString()),
+ Arguments.of(RAW_FORMAT, Schema.BOOL, Boolean.TRUE, DataTypes.BOOLEAN().toString()),
+ Arguments.of(RAW_FORMAT, Schema.BYTES, new byte[1], DataTypes.BYTES().toString()),
+ Arguments.of(
+ RAW_FORMAT,
+ Schema.STRING,
+ "this is a string",
+ DataTypes.STRING().toString()));
+ }
+
+ private static Stream provideAvroBasedSchemaData() {
+ return Stream.of(
+ Arguments.of(JSON_FORMAT, Schema.JSON(TestingUser.class), createRandomUser()),
+ Arguments.of(AVRO_FORMAT, Schema.AVRO(TestingUser.class), createRandomUser()));
+ }
+}
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..53ce6bd02a2a5
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java
@@ -0,0 +1,416 @@
+/*
+ * 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.StopCursor;
+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.source.enumerator.cursor.stop.PublishTimestampStopCursor;
+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.getStopCursor;
+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.parseAfterMessageIdStopCursor;
+import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.parseAtMessageIdStopCursor;
+import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.parseMessageIdStartCursor;
+import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.parseMessageIdString;
+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_STOP_AT_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;
+import static org.assertj.core.api.Assertions.assertThatNoException;
+
+/**
+ * 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 canParseMessageIdEarliestOrLatestStartCursor() {
+ 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 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'.");
+ }
+
+ @Test
+ void canParseMessageIdNeverOrLatestStopCursor() {
+ String never = "never";
+ StopCursor stopCursor = parseAtMessageIdStopCursor(never);
+ assertThat(stopCursor).isEqualTo(StopCursor.never());
+
+ String latest = "latest";
+ stopCursor = parseAtMessageIdStopCursor(latest);
+ assertThat(stopCursor).isEqualTo(StopCursor.latest());
+
+ String precise = "0:0:100";
+ stopCursor = parseAtMessageIdStopCursor(precise);
+ assertThat(stopCursor).isEqualTo(StopCursor.atMessageId(new MessageIdImpl(0, 0, 100)));
+
+ stopCursor = parseAfterMessageIdStopCursor(precise);
+ assertThat(stopCursor).isEqualTo(StopCursor.afterMessageId(new MessageIdImpl(0, 0, 100)));
+ }
+
+ @Test
+ void publishTimeStopCursor() {
+ final Map options = createDefaultOptions();
+ options.put(SOURCE_STOP_AT_PUBLISH_TIME.key(), "12345");
+ StopCursor stopCursor = getStopCursor(Configuration.fromMap(options));
+ assertThat(stopCursor).isInstanceOf(PublishTimestampStopCursor.class);
+
+ options.put(SOURCE_STOP_AT_PUBLISH_TIME.key(), "12345L");
+ assertThatExceptionOfType(IllegalArgumentException.class)
+ .isThrownBy(() -> getStopCursor(Configuration.fromMap(options)))
+ .withMessage(
+ "Could not parse value '12345L' for key 'source.stop.at-publish-time'.");
+ }
+
+ @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";
+ assertThatNoException().isThrownBy(() -> parseMessageIdString(precise));
+
+ String empty = "";
+ assertThatExceptionOfType(IllegalArgumentException.class)
+ .isThrownBy(() -> parseMessageIdString(empty))
+ .withMessage(invalidFormatMessage);
+
+ String noSemicolon = "0";
+ assertThatExceptionOfType(IllegalArgumentException.class)
+ .isThrownBy(() -> parseMessageIdString(noSemicolon))
+ .withMessage(invalidFormatMessage);
+
+ String oneSemiColon = "0:";
+ assertThatExceptionOfType(IllegalArgumentException.class)
+ .isThrownBy(() -> parseMessageIdString(oneSemiColon))
+ .withMessage(invalidFormatMessage);
+
+ String oneSemiColonComplete = "0:0";
+ assertThatExceptionOfType(IllegalArgumentException.class)
+ .isThrownBy(() -> parseMessageIdString(oneSemiColonComplete))
+ .withMessage(invalidFormatMessage);
+
+ String twoSemiColon = "0:0:";
+ assertThatExceptionOfType(IllegalArgumentException.class)
+ .isThrownBy(() -> parseMessageIdString(twoSemiColon))
+ .withMessage(invalidNumberMessage);
+
+ String twoSemiColonComplete = "0:0:0";
+ assertThatNoException().isThrownBy(() -> parseMessageIdString(twoSemiColonComplete));
+
+ String threeSemicolon = "0:0:0:";
+ assertThatExceptionOfType(IllegalArgumentException.class)
+ .isThrownBy(() -> parseMessageIdString(threeSemicolon))
+ .withMessage(invalidNumberMessage);
+
+ String threeSemicolonComplete = "0:0:0:0";
+ assertThatExceptionOfType(IllegalArgumentException.class)
+ .isThrownBy(() -> parseMessageIdString(threeSemicolonComplete))
+ .withMessage(invalidNumberMessage);
+
+ String invalidNumber = "0:0:adf";
+ assertThatExceptionOfType(IllegalArgumentException.class)
+ .isThrownBy(() -> parseMessageIdString(invalidNumber))
+ .withMessage(invalidNumberMessage);
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // 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..cebd6eafb8658
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java
@@ -0,0 +1,496 @@
+/*
+ * 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.apache.flink.table.factories.FactoryUtil;
+
+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.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_STOP_AT_MESSAGE_ID;
+import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_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.TestDynamicTableFactory.VALUE_FORMAT;
+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 = testConfigWithFormat();
+ 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 = testConfigWithFormat();
+ 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 = testConfigWithFormat();
+ testConfigs.put(TOPICS.key(), topicName + ";");
+ runSql(topicName, createTestConfig(testConfigs));
+ runSinkAndExpectSucceed(topicName);
+ runSourceAndExpectSucceed(topicName);
+ }
+
+ @Test
+ void invalidTopicName() {
+ final String topicName = randomTopicName();
+ Map testConfigs = testConfigWithFormat();
+ 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 = testConfigWithFormat();
+ testConfigs.put(
+ TOPICS.key(),
+ topicNameWithPartition(topicName, 0) + ";" + topicNameWithPartition(topicName, 1));
+ runSql(topicName, createTestConfig(testConfigs));
+ runSinkAndExpectSucceed(topicName);
+ runSourceAndExpectSucceed(topicName);
+ }
+
+ @Test
+ void usingFormat() {
+ final String topicName = randomTopicName();
+ Map testConfigs = testConfigWithTopic(topicName);
+ testConfigs.put(FactoryUtil.FORMAT.key(), "json");
+ runSql(topicName, createTestConfig(testConfigs));
+ runSinkAndExpectSucceed(topicName);
+ runSourceAndExpectSucceed(topicName);
+ }
+
+ @Test
+ void usingValueFormat() {
+ final String topicName = randomTopicName();
+ Map testConfigs = testConfigWithTopic(topicName);
+ testConfigs.put(VALUE_FORMAT.key(), "json");
+ runSql(topicName, createTestConfig(testConfigs));
+ runSinkAndExpectSucceed(topicName);
+ runSourceAndExpectSucceed(topicName);
+ }
+
+ @Test
+ void usingValueFormatAndFormatOptions() {
+ final String topicName = randomTopicName();
+ Map testConfigs = testConfigWithTopic(topicName);
+ testConfigs.put(VALUE_FORMAT.key(), "json");
+ testConfigs.put("value.json.fail-on-missing-field", "false");
+ runSql(topicName, createTestConfig(testConfigs));
+ runSinkAndExpectSucceed(topicName);
+ runSourceAndExpectSucceed(topicName);
+ }
+
+ @Test
+ void subscriptionType() {
+ final String topicName = randomTopicName();
+ Map testConfigs = testConfigWithTopicAndFormat(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 = testConfigWithTopicAndFormat(topicName);
+ testConfigs.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Failover");
+ runSql(topicName, createTestConfig(testConfigs));
+ runSinkAndExpectSucceed(topicName);
+ runSourceAndExpectException(
+ topicName, new ValidationException("Failover SubscriptionType is not supported. "));
+ }
+
+ @Test
+ void invalidNonExistSubscriptionType() {
+ final String topicName = randomTopicName();
+ Map testConfigs = testConfigWithTopicAndFormat(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 = testConfigWithTopicAndFormat(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 = testConfigWithTopicAndFormat(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 = testConfigWithTopicAndFormat(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 = testConfigWithTopicAndFormat(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 = testConfigWithTopicAndFormat(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 = testConfigWithTopicAndFormat(topicName);
+
+ testConfigs.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "233010230");
+
+ runSql(topicName, createTestConfig(testConfigs));
+ runSinkAndExpectSucceed(topicName);
+ runSourceAndExpectSucceed(topicName);
+ }
+
+ @Test
+ void messageIdStopCursorNever() {
+ final String topicName = randomTopicName();
+ Map testConfigs = testConfigWithTopicAndFormat(topicName);
+
+ testConfigs.put(SOURCE_STOP_AT_MESSAGE_ID.key(), "never");
+
+ runSql(topicName, createTestConfig(testConfigs));
+ runSinkAndExpectSucceed(topicName);
+ runSourceAndExpectSucceed(topicName);
+ }
+
+ @Test
+ void messageIdStopCursorLatest() {
+ final String topicName = randomTopicName();
+ Map testConfigs = testConfigWithTopicAndFormat(topicName);
+
+ testConfigs.put(SOURCE_STOP_AT_MESSAGE_ID.key(), "latest");
+
+ runSql(topicName, createTestConfig(testConfigs));
+ runSinkAndExpectSucceed(topicName);
+ runSourceAndExpectSucceed(topicName);
+ }
+
+ @Test
+ void messageIdStopCursorExact() {
+ final String topicName = randomTopicName();
+ Map testConfigs = testConfigWithTopicAndFormat(topicName);
+
+ testConfigs.put(SOURCE_STOP_AT_MESSAGE_ID.key(), "0:0:-1");
+
+ runSql(topicName, createTestConfig(testConfigs));
+ runSinkAndExpectSucceed(topicName);
+ runSourceAndExpectSucceed(topicName);
+ }
+
+ @Test
+ void timestampStopCursor() {
+ final String topicName = randomTopicName();
+ Map testConfigs = testConfigWithTopicAndFormat(topicName);
+
+ testConfigs.put(SOURCE_STOP_AT_PUBLISH_TIME.key(), "233010230");
+
+ runSql(topicName, createTestConfig(testConfigs));
+ runSinkAndExpectSucceed(topicName);
+ runSourceAndExpectSucceed(topicName);
+ }
+
+ @Test
+ void topicRoutingMode() {
+ final String topicName = randomTopicName();
+ Map testConfigs = testConfigWithTopicAndFormat(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 = testConfigWithTopicAndFormat(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 = testConfigWithTopicAndFormat(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 = testConfigWithTopicAndFormat(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'."));
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // PulsarOptions, PulsarSourceOptions, PulsarSinkOptions Test
+ // --------------------------------------------------------------------------------------------
+
+ @Test
+ void pulsarOptionsAuthParamMap() {
+ final String topicName = randomTopicName();
+ Map testConfigs = testConfigWithTopicAndFormat(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 = testConfigWithTopicAndFormat(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"
+ + " 'service-url' = '%s',\n"
+ + " 'admin-url' = '%s',\n"
+ + " %s\n"
+ + " 'connector' = 'pulsar'"
+ + ")",
+ 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 extends Throwable> 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