leonardBang commented on code in PR #56:
URL: 
https://github.com/apache/flink-connector-pulsar/pull/56#discussion_r1303026417


##########
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<List<String>> TOPICS =
+            ConfigOptions.key("topics")

Review Comment:
    we use `topic` and `topic-pattern `  in kafka connector, I wonder could we 
unified this concept?



##########
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<List<String>> 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<SubscriptionType> 
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<String> 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<String> 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<Long> 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<String> 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<String> 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<Long> 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.");

Review Comment:
   bounded



##########
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.
+ *
+ * <p>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.
+ *
+ * <p>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.
+ *
+ * <p>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;

Review Comment:
   1L



##########
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<List<String>> 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<SubscriptionType> 
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<String> 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<String> 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"),

Review Comment:
   The `earliest` and `latest`  worth a public configuration for users from my 
side, these startup modes are similar to specific-offset in kafka connector. I 
don't think the configuration in Kafka connector is good one, I just want to 
align them for decreasing the users' understanding cost. 



##########
flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarWritableMetadata.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.pulsar.table.sink;
+
+import 
org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessageBuilder;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+
+import org.bouncycastle.util.Arrays;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.stream.Stream;
+
+/** A class used to manage WritableMetadata for Pulsar SQL sink connector. */
+public class PulsarWritableMetadata implements Serializable {
+
+    private static final long serialVersionUID = 8117156158379846715L;

Review Comment:
   1L



##########
flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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<RowData> {
+
+    private static final long serialVersionUID = 7314442107082067836L;

Review Comment:
   IIRC, Flink uses serialVersionUID  from `1L` by default?



##########
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<List<String>> 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<SubscriptionType> 
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<String> 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<String> 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<Long> 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<String> 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 "

Review Comment:
   unbounded?



##########
flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSink.java:
##########
@@ -0,0 +1,169 @@
+/*
+ * 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<String> topics;
+
+    private final Properties properties;
+
+    private final DeliveryGuarantee deliveryGuarantee;
+
+    @Nullable private final TopicRouter<RowData> topicRouter;
+
+    private final TopicRoutingMode topicRoutingMode;
+
+    private final long messageDelayMillis;
+
+    public PulsarTableSink(
+            PulsarTableSerializationSchemaFactory serializationSchemaFactory,
+            ChangelogMode changelogMode,
+            List<String> topics,
+            Properties properties,
+            DeliveryGuarantee deliveryGuarantee,
+            @Nullable TopicRouter<RowData> 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<RowData> pulsarSerializationSchema =
+                
serializationSchemaFactory.createPulsarSerializationSchema(context);
+
+        final PulsarSinkBuilder<RowData> pulsarSinkBuilder =
+                PulsarSink.builder()
+                        .setSerializationSchema(pulsarSerializationSchema)
+                        .setProperties(properties)
+                        .setDeliveryGuarantee(deliveryGuarantee)
+                        .setTopics(topics)
+                        .setTopicRoutingMode(topicRoutingMode)
+                        .delaySendingMessage(new 
FixedMessageDelayer<>(messageDelayMillis));

Review Comment:
   Here looks like we set a different default value.
   In table: default 0 will map to  FixedMessageDelayer 
   In DS:default -1 map to never() FixedMessageDelayer
   Duration of 0 is meaningless for most cases as publish/schedule with a delay 
0 mills is meaningless, I'd like to use -1 as default value.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to