You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ti...@apache.org on 2022/09/30 04:29:17 UTC

[flink] branch release-1.16 updated (729168f0edd -> 64c284f5f95)

This is an automated email from the ASF dual-hosted git repository.

tison pushed a change to branch release-1.16
in repository https://gitbox.apache.org/repos/asf/flink.git


    from 729168f0edd [FLINK-29460] fix the unstable test HsResultPartitionTest.testRelease
     new dd33ad5f790 [FLINK-26182][Connector/Pulsar] Drop the embedded Pulsar runtime for in favor of mock Pulsar runtime with local metastore support.
     new e310a1be2fd [FLINK-26182][Connector/Pulsar] Enable the transaction in the Pulsar TestContainers in a better way.
     new 908405fc835 [FLINK-26182][Connector/pulsar] Extract common logic from Pulsar source testing tools.
     new 632f42dbe0e [FLINK-29381][Connector/Pulsar] Fixes the split assignment for Key Shared subscription.
     new 0dbcca0b871 [FLINK-29381][Connector/Pulsar] Add a document on how to use Key_Shared subscription.
     new 64c284f5f95 [FLINK-26182][Connector/pulsar] Create e2e tests for the Pulsar source and sink based on the connector testing framework.

The 6 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../docs/connectors/datastream/pulsar.md           |   16 +-
 docs/content/docs/connectors/datastream/pulsar.md  |   39 +-
 .../generated/pulsar_consumer_configuration.html   |    2 +-
 .../generated/pulsar_source_configuration.html     |    6 +
 .../f4d91193-72ba-4ce4-ad83-98f780dce581           |    6 +
 flink-connectors/flink-connector-pulsar/pom.xml    |    2 -
 .../pulsar/common/utils/PulsarSerdeUtils.java      |   28 +
 .../pulsar/source/PulsarSourceBuilder.java         |    8 +-
 .../pulsar/source/PulsarSourceOptions.java         |   21 +-
 .../source/config/PulsarConsumerBuilder.java       |   89 ++
 .../source/config/PulsarSourceConfigUtils.java     |   16 +-
 .../pulsar/source/config/SourceConfiguration.java  |   14 +-
 .../PulsarSourceEnumStateSerializer.java           |   16 +-
 ...itAssigner.java => KeySharedSplitAssigner.java} |   65 +-
 .../assigner/NonSharedSplitAssigner.java           |   35 +-
 .../enumerator/assigner/SplitAssignerBase.java     |   33 +
 .../enumerator/assigner/SplitAssignerFactory.java  |    4 +-
 .../subscriber/impl/BasePulsarSubscriber.java      |   34 +-
 .../subscriber/impl/TopicListSubscriber.java       |   10 +-
 .../subscriber/impl/TopicPatternSubscriber.java    |    7 +-
 .../source/enumerator/topic/TopicPartition.java    |   70 +-
 .../pulsar/source/enumerator/topic/TopicRange.java |    2 +-
 .../topic/range/FixedKeysRangeGenerator.java       |  185 ++++
 .../topic/range/FixedRangeGenerator.java           |   17 +
 .../enumerator/topic/range/FullRangeGenerator.java |   12 +-
 .../enumerator/topic/range/RangeGenerator.java     |   41 +-
 ...angeGenerator.java => SplitRangeGenerator.java} |   45 +-
 .../enumerator/topic/range/TopicRangeUtils.java    |  136 +++
 .../split/PulsarPartitionSplitReaderBase.java      |   14 +-
 .../split/PulsarPartitionSplitSerializer.java      |   47 +-
 .../connector/pulsar/sink/PulsarSinkITCase.java    |  151 +--
 .../pulsar/source/PulsarSourceBuilderTest.java     |    4 +-
 .../pulsar/source/PulsarSourceITCase.java          |    6 +-
 .../pulsar/source/PulsarUnorderedSourceITCase.java |   62 +-
 .../PulsarSourceEnumStateSerializerTest.java       |   95 +-
 .../enumerator/PulsarSourceEnumeratorTest.java     |    3 +-
 .../assigner/NonSharedSplitAssignerTest.java       |    2 +-
 .../enumerator/assigner/SplitAssignerTestBase.java |    3 +-
 .../source/enumerator/cursor/StopCursorTest.java   |    3 +-
 .../subscriber/PulsarSubscriberTest.java           |   17 +-
 .../enumerator/topic/TopicPartitionTest.java       |    5 +-
 .../topic/range/SplitRangeGeneratorTest.java       |   82 ++
 .../topic/range/TopicRangeUtilsTest.java           |   84 ++
 .../split/PulsarPartitionSplitReaderTestBase.java  |    5 +-
 .../split/PulsarPartitionSplitSerializerTest.java  |   50 +-
 .../pulsar/testutils/PulsarTestCommonUtils.java    |   22 +-
 .../pulsar/testutils/PulsarTestContext.java        |   47 +-
 .../pulsar/testutils/PulsarTestContextFactory.java |    5 +-
 .../pulsar/testutils/PulsarTestEnvironment.java    |    2 +-
 .../pulsar/testutils/PulsarTestSuiteBase.java      |    5 +-
 .../cases/MultipleTopicTemplateContext.java        |  134 ---
 .../cases/SingleTopicConsumingContext.java         |  132 ---
 .../pulsar/testutils/runtime/PulsarRuntime.java    |   17 +-
 .../testutils/runtime/PulsarRuntimeOperator.java   |   15 +-
 .../runtime/container/PulsarContainerRuntime.java  |   26 +-
 .../runtime/embedded/PulsarEmbeddedRuntime.java    |  184 ----
 .../testutils/runtime/mock/PulsarMockRuntime.java  |    5 +-
 .../testutils/sink/PulsarPartitionDataReader.java  |   93 ++
 .../testutils/sink/PulsarSinkTestContext.java      |  124 +++
 .../testutils/sink/PulsarSinkTestSuiteBase.java}   |   29 +-
 .../source}/KeyedPulsarPartitionDataWriter.java    |   23 +-
 .../{ => source}/PulsarPartitionDataWriter.java    |   18 +-
 .../testutils/source/PulsarSourceTestContext.java  |  138 +++
 .../source/UnorderedSourceTestSuiteBase.java}      |   45 +-
 .../source/cases/KeySharedSubscriptionContext.java |   85 ++
 .../cases/MultipleTopicConsumingContext.java       |   37 +-
 .../cases/SharedSubscriptionContext.java}          |   26 +-
 .../source/cases/SingleTopicConsumingContext.java  |   73 ++
 .../test/resources/containers/txnStandalone.conf   | 1042 --------------------
 .../src/test/resources/docker/bootstrap.sh         |   28 -
 .../flink-end-to-end-tests-pulsar/pom.xml          |   24 +-
 ...eOrderedE2ECase.java => PulsarSinkE2ECase.java} |   30 +-
 .../util/pulsar/PulsarSourceOrderedE2ECase.java    |    5 +-
 .../util/pulsar/PulsarSourceUnorderedE2ECase.java  |   13 +-
 .../pulsar/cases/KeySharedSubscriptionContext.java |  144 ---
 .../pulsar/cases/SharedSubscriptionContext.java    |  116 ---
 .../FlinkContainerWithPulsarEnvironment.java       |    8 +-
 .../common/UnorderedSourceTestSuiteBase.java       |   86 --
 .../ExclusiveSubscriptionContext.java              |   19 +-
 .../FailoverSubscriptionContext.java               |   19 +-
 .../source/enumerator/NoOpEnumStateSerializer.java |    2 +-
 81 files changed, 2009 insertions(+), 2404 deletions(-)
 create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/PulsarConsumerBuilder.java
 copy flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/{NonSharedSplitAssigner.java => KeySharedSplitAssigner.java} (57%)
 create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/FixedKeysRangeGenerator.java
 rename flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/{UniformRangeGenerator.java => SplitRangeGenerator.java} (55%)
 create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/TopicRangeUtils.java
 create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/SplitRangeGeneratorTest.java
 create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/TopicRangeUtilsTest.java
 delete mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicTemplateContext.java
 delete mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/SingleTopicConsumingContext.java
 delete mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/embedded/PulsarEmbeddedRuntime.java
 create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarPartitionDataReader.java
 create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarSinkTestContext.java
 copy flink-connectors/flink-connector-pulsar/src/{main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/FixedRangeGenerator.java => test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarSinkTestSuiteBase.java} (50%)
 rename {flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common => flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source}/KeyedPulsarPartitionDataWriter.java (72%)
 rename flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/{ => source}/PulsarPartitionDataWriter.java (69%)
 create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/PulsarSourceTestContext.java
 copy flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/{source/PulsarUnorderedSourceITCase.java => testutils/source/UnorderedSourceTestSuiteBase.java} (55%)
 create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/KeySharedSubscriptionContext.java
 rename flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/{ => source}/cases/MultipleTopicConsumingContext.java (58%)
 rename flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/{cases/SharedSubscriptionConsumingContext.java => source/cases/SharedSubscriptionContext.java} (60%)
 create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/SingleTopicConsumingContext.java
 delete mode 100644 flink-connectors/flink-connector-pulsar/src/test/resources/containers/txnStandalone.conf
 delete mode 100755 flink-connectors/flink-connector-pulsar/src/test/resources/docker/bootstrap.sh
 copy flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/{PulsarSourceOrderedE2ECase.java => PulsarSinkE2ECase.java} (65%)
 delete mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/KeySharedSubscriptionContext.java
 delete mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/SharedSubscriptionContext.java
 delete mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/UnorderedSourceTestSuiteBase.java
 rename flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/{cases => source}/ExclusiveSubscriptionContext.java (71%)
 rename flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/{cases => source}/FailoverSubscriptionContext.java (71%)


[flink] 05/06: [FLINK-29381][Connector/Pulsar] Add a document on how to use Key_Shared subscription.

Posted by ti...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tison pushed a commit to branch release-1.16
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 0dbcca0b87117b73491b4cf6a8a54da094621f55
Author: Yufan Sheng <yu...@streamnative.io>
AuthorDate: Wed Sep 28 15:27:53 2022 +0800

    [FLINK-29381][Connector/Pulsar] Add a document on how to use Key_Shared subscription.
---
 .../docs/connectors/datastream/pulsar.md           | 16 +++++++--
 docs/content/docs/connectors/datastream/pulsar.md  | 39 ++++++++++++++++++++--
 2 files changed, 50 insertions(+), 5 deletions(-)

diff --git a/docs/content.zh/docs/connectors/datastream/pulsar.md b/docs/content.zh/docs/connectors/datastream/pulsar.md
index 8dab844e14b..0778eb8b9eb 100644
--- a/docs/content.zh/docs/connectors/datastream/pulsar.md
+++ b/docs/content.zh/docs/connectors/datastream/pulsar.md
@@ -273,9 +273,21 @@ PulsarSource.builder().set_subscription_name("my-exclusive").set_subscription_ty
 {{< /tab >}}
 {{< /tabs >}}
 
-如果想在 Pulsar Source 里面使用 `key 共享` 订阅,需要提供 `RangeGenerator` 实例。`RangeGenerator` 会生成一组消息 key 的 hash 范围,Pulsar Source 会基于给定的范围来消费数据。
+#### Key_Shared 订阅
 
-Pulsar Source 也提供了一个名为 `UniformRangeGenerator` 的默认实现,它会基于 flink 数据源的并行度将 hash 范围均分。
+当时用 Key_Shared 订阅时,Pulsar 将会基于 Message 的 key 去计算对应的 Hash 值,Hash 取值范围为(0~65535)。我们首先会使用 `Message.getOrderingKey()` 计算 Hash,如果没有则会依次使用 `Message.getKey()` 和 `Message.getKeyBytes()`。对于上述 key 都找不到的消息,我们会使用字符串 `"NO_KEY"` 来计算消息的 Hash 值。
+
+在 Flink Connector 中针对 Key_Shared 订阅提供了两种消费模式,分别是 `KeySharedMode.SPLIT` 和 `KeySharedMode.JOIN`,它们的实际消费行为并不相同。`KeySharedMode.JOIN` 会把所有的给定的 Hash 范围放于一个 Reader 中进行消费,而 `KeySharedMode.SPLIT` 会打散给定的 Hash 范围于不同的 Reader 中消费。
+
+之所以这么设计的主要原因是因为,在 Key_Shared 的订阅模式中,如果一条消息找不到对应的消费者,所有的消息都不会继续往下发送。所以我们提供了 `KeySharedMode.JOIN` 模式,允许用户只消费部分 Hash 范围的消息。
+
+##### 定义 RangeGenerator
+
+如果想在 Pulsar Source 里面使用 `Key_Shared` 订阅,需要提供 `RangeGenerator` 实例。`RangeGenerator` 会生成一组消息 key 的 hash 范围,Pulsar Source 会基于给定的范围来消费数据。
+
+Pulsar Source 也提供了一个名为 `SplitRangeGenerator` 的默认实现,它会基于 flink 数据源的并行度将 hash 范围均分。
+
+由于 Pulsar 并未提供 Key 的 Hash 计算方法,所以我们在 Flink 中提供了名为 `FixedKeysRangeGenerator` 的实现,你可以在 builder 中依次提供需要消费的 Key 内容即可。但需要注意的是,Pulsar 的 Key Hash 值并不对应唯一的一个 Key,所以如果你只想消费某几个 Key 的消息,还需要在后面的代码中使用 `DataStream.filter()` 方法来过滤出对应的消息。
 
 ### 起始消费位置
 
diff --git a/docs/content/docs/connectors/datastream/pulsar.md b/docs/content/docs/connectors/datastream/pulsar.md
index b3a8ec6300b..34a3bbcd762 100644
--- a/docs/content/docs/connectors/datastream/pulsar.md
+++ b/docs/content/docs/connectors/datastream/pulsar.md
@@ -301,12 +301,45 @@ PulsarSource.builder().set_subscription_name("my-exclusive").set_subscription_ty
 {{< /tab >}}
 {{< /tabs >}}
 
-Ensure that you provide a `RangeGenerator` implementation if you want to use the `Key_Shared` subscription type on the Pulsar connector.
-The `RangeGenerator` generates a set of key hash ranges so that a respective reader subtask only dispatches messages where the hash of the message key is contained in the specified range.
+#### Key_Shared subscriptions
 
-The Pulsar connector uses `UniformRangeGenerator` that divides the range by the Flink source
+All the Pulsar's messages will be calculated with a key hash in Key_Shared subscription.
+The hash range must be 0 to 65535. We try to compute the key hash in the order of `Message.getOrderingKey()`,
+`Message.getKey()` or `Message.getKeyBytes()`. We will use `"NO_KEY"` str as the message key if none of these keys has been provided.
+
+Pulsar's Key_Shared subscription comes in two forms in Connector, the `KeySharedMode.SPLIT` and `KeySharedMode.JOIN`.
+Different `KeySharedMode` means different split assignment behaviors. If you only consume a subset of Pulsar's key hash range,
+remember to use the `KeySharedMode.JOIN` which will subscribe all the range in only one reader.
+Otherwise, when the ranges can join into a full Pulsar key hash range (0~65535) you should use `KeySharedMode.SPLIT`
+mode for sharing the splits among all the backend readers.
+
+In the `KeySharedMode.SPLIT` mode. The topic will be subscribed by multiple readers.
+But Pulsar has one limit in this situation. That is if a Message can't find the corresponding reader by the key hash range.
+No messages will be delivered to the current readers, until there is a reader which can subscribe to such messages.
+
+##### Define a RangeGenerator
+
+Ensure that you have provided a `RangeGenerator` implementation if you want to use the `Key_Shared` subscription type on the Pulsar connector.
+The `RangeGenerator` generates a set of key hash ranges so that a respective reader subtask only dispatches
+messages where the hash of the message key is contained in the specified range.
+
+The Pulsar connector uses `SplitRangeGenerator` that divides the range by the Flink source
 parallelism if no `RangeGenerator` is provided in the `Key_Shared` subscription type.
 
+Since the Pulsar didn't expose the key hash range method. We have to provide an `FixedKeysRangeGenerator` for end-user.
+You can add the keys you want to consume, no need to calculate any hash ranges.
+The key's hash isn't specified to only one key, so the consuming results may contain the messages with
+different keys comparing the keys you have defined in this range generator.
+Remember to use flink's `DataStream.filter()` method after the Pulsar source.
+
+```java
+FixedKeysRangeGenerator.builder()
+    .supportNullKey()
+    .key("someKey")
+    .keys(Arrays.asList("key1", "key2"))
+    .build()
+```
+
 ### Starting Position
 
 The Pulsar source is able to consume messages starting from different positions by setting the `setStartCursor(StartCursor)` option.


[flink] 01/06: [FLINK-26182][Connector/Pulsar] Drop the embedded Pulsar runtime for in favor of mock Pulsar runtime with local metastore support.

Posted by ti...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tison pushed a commit to branch release-1.16
in repository https://gitbox.apache.org/repos/asf/flink.git

commit dd33ad5f7901f4cb04c101df2ca74ce84cadb817
Author: Yufan Sheng <yu...@streamnative.io>
AuthorDate: Tue Sep 27 18:04:23 2022 +0800

    [FLINK-26182][Connector/Pulsar] Drop the embedded Pulsar runtime for in favor of mock Pulsar runtime with local metastore support.
---
 .../pulsar/testutils/runtime/PulsarRuntime.java    |  17 +-
 .../runtime/embedded/PulsarEmbeddedRuntime.java    | 184 ---------------------
 2 files changed, 2 insertions(+), 199 deletions(-)

diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntime.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntime.java
index 92dd94c6657..6beb94a68c4 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntime.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntime.java
@@ -20,7 +20,6 @@ package org.apache.flink.connector.pulsar.testutils.runtime;
 
 import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment;
 import org.apache.flink.connector.pulsar.testutils.runtime.container.PulsarContainerRuntime;
-import org.apache.flink.connector.pulsar.testutils.runtime.embedded.PulsarEmbeddedRuntime;
 import org.apache.flink.connector.pulsar.testutils.runtime.mock.PulsarMockRuntime;
 
 import org.testcontainers.containers.GenericContainer;
@@ -50,22 +49,10 @@ public interface PulsarRuntime {
         return new PulsarMockRuntime();
     }
 
-    /**
-     * Create a standalone Pulsar instance in test thread. We would start an embedded zookeeper and
-     * bookkeeper. The stream storage for bookkeeper is disabled. The function worker is disabled on
-     * Pulsar broker.
-     *
-     * <p>This runtime would be faster than {@link #container()} and behaves the same as the {@link
-     * #container()}.
-     */
-    static PulsarRuntime embedded() {
-        return new PulsarEmbeddedRuntime();
-    }
-
     /**
      * Create a Pulsar instance in docker. We would start a standalone Pulsar in TestContainers.
      * This runtime is often used in end-to-end tests. The performance may be a bit of slower than
-     * {@link #embedded()}. The stream storage for bookkeeper is disabled. The function worker is
+     * {@link #mock()}. The stream storage for bookkeeper is disabled. The function worker is
      * disabled on Pulsar broker.
      */
     static PulsarRuntime container() {
@@ -75,7 +62,7 @@ public interface PulsarRuntime {
     /**
      * Create a Pulsar instance in docker. We would start a standalone Pulsar in TestContainers.
      * This runtime is often used in end-to-end tests. The performance may be a bit of slower than
-     * {@link #embedded()}. The stream storage for bookkeeper is disabled. The function worker is
+     * {@link #mock()}. The stream storage for bookkeeper is disabled. The function worker is
      * disabled on Pulsar broker.
      *
      * <p>We would link the created Pulsar docker instance with the given flink instance. This would
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/embedded/PulsarEmbeddedRuntime.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/embedded/PulsarEmbeddedRuntime.java
deleted file mode 100644
index 2ca9a51f3c5..00000000000
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/embedded/PulsarEmbeddedRuntime.java
+++ /dev/null
@@ -1,184 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.connector.pulsar.testutils.runtime.embedded;
-
-import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime;
-import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator;
-import org.apache.flink.util.FileUtils;
-
-import org.apache.bookkeeper.conf.ServerConfiguration;
-import org.apache.pulsar.broker.PulsarService;
-import org.apache.pulsar.broker.ServiceConfiguration;
-import org.apache.pulsar.common.configuration.PulsarConfigurationLoader;
-import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.net.URL;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.Optional;
-
-import static org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeUtils.initializePulsarEnvironment;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.pulsar.broker.ServiceConfigurationUtils.brokerUrl;
-import static org.apache.pulsar.broker.ServiceConfigurationUtils.webServiceUrl;
-
-/** Providing a embedded pulsar server. We use this runtime for transaction related tests. */
-public class PulsarEmbeddedRuntime implements PulsarRuntime {
-    private static final Logger LOG = LoggerFactory.getLogger(PulsarEmbeddedRuntime.class);
-
-    private static final String CONFIG_FILE_PATH;
-
-    static {
-        // Find the absolute path for containers/txnStandalone.conf
-        ClassLoader classLoader = PulsarEmbeddedRuntime.class.getClassLoader();
-        URL resource = classLoader.getResource("containers/txnStandalone.conf");
-        File file = new File(checkNotNull(resource).getFile());
-        CONFIG_FILE_PATH = file.getAbsolutePath();
-    }
-
-    private final Path tempDir;
-
-    private LocalBookkeeperEnsemble bookkeeper;
-    private PulsarService pulsarService;
-    private PulsarRuntimeOperator operator;
-
-    public PulsarEmbeddedRuntime() {
-        this.tempDir = createTempDir();
-    }
-
-    @Override
-    public void startUp() {
-        try {
-            startBookkeeper();
-            startPulsarService();
-
-            // Create the operator.
-            this.operator = new PulsarRuntimeOperator(serviceUrl(), adminUrl());
-        } catch (Exception e) {
-            throw new IllegalStateException(e);
-        }
-    }
-
-    @Override
-    public void tearDown() {
-        try {
-            if (operator != null) {
-                operator.close();
-            }
-            if (pulsarService != null) {
-                pulsarService.close();
-            }
-            if (bookkeeper != null) {
-                bookkeeper.stop();
-            }
-        } catch (Exception e) {
-            throw new IllegalStateException(e);
-        } finally {
-            removeTempDir(tempDir);
-        }
-    }
-
-    @Override
-    public PulsarRuntimeOperator operator() {
-        return checkNotNull(operator, "You should start this embedded Pulsar first.");
-    }
-
-    private Path createTempDir() {
-        try {
-            return Files.createTempDirectory("pulsar");
-        } catch (IOException e) {
-            throw new IllegalStateException(e);
-        }
-    }
-
-    private void removeTempDir(Path tempDir) {
-        try {
-            FileUtils.deleteDirectory(tempDir.normalize().toFile());
-        } catch (IOException e) {
-            throw new IllegalStateException(e);
-        }
-    }
-
-    public void startBookkeeper() throws Exception {
-        Path zkPath = Paths.get("data", "standalone", "zookeeper");
-        Path bkPath = Paths.get("data", "standalone", "bookkeeper");
-
-        String zkDir = tempDir.resolve(zkPath).normalize().toString();
-        String bkDir = tempDir.resolve(bkPath).normalize().toString();
-
-        ServerConfiguration bkServerConf = new ServerConfiguration();
-        bkServerConf.loadConf(new File(CONFIG_FILE_PATH).toURI().toURL());
-        this.bookkeeper = new LocalBookkeeperEnsemble(1, 0, 0, zkDir, bkDir, true, "127.0.0.1");
-
-        // Start Bookkeeper & zookeeper.
-        bookkeeper.startStandalone(bkServerConf, false);
-    }
-
-    private void startPulsarService() throws Exception {
-        ServiceConfiguration config;
-        try (FileInputStream inputStream = new FileInputStream(CONFIG_FILE_PATH)) {
-            config = PulsarConfigurationLoader.create(inputStream, ServiceConfiguration.class);
-        } catch (IOException e) {
-            throw new IllegalStateException(e);
-        }
-
-        // Use runtime dynamic ports for broker.
-        config.setAdvertisedAddress("127.0.0.1");
-        config.setClusterName("standalone");
-
-        // Use random port.
-        config.setBrokerServicePort(Optional.of(0));
-        config.setWebServicePort(Optional.of(0));
-
-        // Select available port for bookkeeper and zookeeper.
-        int zkPort = getZkPort();
-        String zkConnect = "127.0.0.1" + ":" + zkPort;
-        config.setZookeeperServers(zkConnect);
-        config.setConfigurationStoreServers(zkConnect);
-        config.setRunningStandalone(true);
-
-        this.pulsarService = new PulsarService(config);
-
-        // Start Pulsar Broker.
-        pulsarService.start();
-
-        // Create sample data environment.
-        initializePulsarEnvironment(config, serviceUrl(), adminUrl());
-    }
-
-    private int getZkPort() {
-        return checkNotNull(bookkeeper).getZookeeperPort();
-    }
-
-    private String serviceUrl() {
-        Integer port = pulsarService.getBrokerListenPort().orElseThrow(IllegalStateException::new);
-        return brokerUrl("127.0.0.1", port);
-    }
-
-    private String adminUrl() {
-        Integer port = pulsarService.getListenPortHTTP().orElseThrow(IllegalArgumentException::new);
-        return webServiceUrl("127.0.0.1", port);
-    }
-}


[flink] 02/06: [FLINK-26182][Connector/Pulsar] Enable the transaction in the Pulsar TestContainers in a better way.

Posted by ti...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tison pushed a commit to branch release-1.16
in repository https://gitbox.apache.org/repos/asf/flink.git

commit e310a1be2fd3b8baa7308c538efc22731dd36744
Author: Yufan Sheng <yu...@streamnative.io>
AuthorDate: Tue Sep 27 18:04:24 2022 +0800

    [FLINK-26182][Connector/Pulsar] Enable the transaction in the Pulsar TestContainers in a better way.
---
 .../runtime/container/PulsarContainerRuntime.java  |    7 +-
 .../test/resources/containers/txnStandalone.conf   | 1042 --------------------
 2 files changed, 4 insertions(+), 1045 deletions(-)

diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java
index 10bd5120a46..910985338b9 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java
@@ -90,15 +90,16 @@ public class PulsarContainerRuntime implements PulsarRuntime {
 
         // Override the default configuration in container for enabling the Pulsar transaction.
         container.withClasspathResourceMapping(
-                "containers/txnStandalone.conf",
-                "/pulsar/conf/standalone.conf",
-                BindMode.READ_ONLY);
+                "docker/bootstrap.sh", "/pulsar/bin/bootstrap.sh", BindMode.READ_ONLY);
         // Waiting for the Pulsar border is ready.
         container.waitingFor(
                 forHttp("/admin/v2/namespaces/public/default")
                         .forPort(BROKER_HTTP_PORT)
                         .forStatusCode(200)
                         .withStartupTimeout(Duration.ofMinutes(5)));
+        // Set custom startup script.
+        container.withCommand("sh /pulsar/bin/bootstrap.sh");
+
         // Start the Pulsar Container.
         container.start();
         // Append the output to this runtime logger. Used for local debug purpose.
diff --git a/flink-connectors/flink-connector-pulsar/src/test/resources/containers/txnStandalone.conf b/flink-connectors/flink-connector-pulsar/src/test/resources/containers/txnStandalone.conf
deleted file mode 100644
index 10437a71740..00000000000
--- a/flink-connectors/flink-connector-pulsar/src/test/resources/containers/txnStandalone.conf
+++ /dev/null
@@ -1,1042 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#   http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing,
-# software distributed under the License is distributed on an
-# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-# KIND, either express or implied.  See the License for the
-# specific language governing permissions and limitations
-# under the License.
-#
-
-### --- General broker settings --- ###
-
-# Zookeeper quorum connection string
-zookeeperServers=
-
-# Configuration Store connection string
-configurationStoreServers=
-
-brokerServicePort=6650
-
-# Port to use to server HTTP request
-webServicePort=8080
-
-# Hostname or IP address the service binds on, default is 0.0.0.0.
-bindAddress=0.0.0.0
-
-# Extra bind addresses for the service: <listener_name>:<scheme>://<host>:<port>,[...]
-bindAddresses=
-
-# Hostname or IP address the service advertises to the outside world. If not set, the value of InetAddress.getLocalHost().getHostName() is used.
-advertisedAddress=
-
-# Enable or disable the HAProxy protocol.
-haProxyProtocolEnabled=false
-
-# Number of threads to use for Netty IO. Default is set to 2 * Runtime.getRuntime().availableProcessors()
-numIOThreads=
-
-# Number of threads to use for ordered executor. The ordered executor is used to operate with zookeeper,
-# such as init zookeeper client, get namespace policies from zookeeper etc. It also used to split bundle. Default is 8
-numOrderedExecutorThreads=8
-
-# Number of threads to use for HTTP requests processing. Default is set to 2 * Runtime.getRuntime().availableProcessors()
-numHttpServerThreads=
-
-# Number of thread pool size to use for pulsar broker service.
-# The executor in thread pool will do basic broker operation like load/unload bundle, update managedLedgerConfig,
-# update topic/subscription/replicator message dispatch rate, do leader election etc.
-# Default is Runtime.getRuntime().availableProcessors()
-numExecutorThreadPoolSize=
-
-# Number of thread pool size to use for pulsar zookeeper callback service
-# The cache executor thread pool is used for restarting global zookeeper session.
-# Default is 10
-numCacheExecutorThreadPoolSize=10
-
-# Max concurrent web requests
-maxConcurrentHttpRequests=1024
-
-# Name of the cluster to which this broker belongs to
-clusterName=standalone
-
-# Enable cluster's failure-domain which can distribute brokers into logical region
-failureDomainsEnabled=false
-
-# Zookeeper session timeout in milliseconds
-zooKeeperSessionTimeoutMillis=30000
-
-# ZooKeeper operation timeout in seconds
-zooKeeperOperationTimeoutSeconds=30
-
-# ZooKeeper cache expiry time in seconds
-zooKeeperCacheExpirySeconds=300
-
-# Time to wait for broker graceful shutdown. After this time elapses, the process will be killed
-brokerShutdownTimeoutMs=60000
-
-# Flag to skip broker shutdown when broker handles Out of memory error
-skipBrokerShutdownOnOOM=false
-
-# Enable backlog quota check. Enforces action on topic when the quota is reached
-backlogQuotaCheckEnabled=true
-
-# How often to check for topics that have reached the quota
-backlogQuotaCheckIntervalInSeconds=60
-
-# Default per-topic backlog quota limit
-backlogQuotaDefaultLimitGB=10
-
-# Default per-topic backlog quota time limit in second, less than 0 means no limitation. default is -1.
-backlogQuotaDefaultLimitSecond=-1
-
-# Default ttl for namespaces if ttl is not already configured at namespace policies. (disable default-ttl with value 0)
-ttlDurationDefaultInSeconds=0
-
-# Enable the deletion of inactive topics. This parameter need to cooperate with the allowAutoTopicCreation parameter.
-# If brokerDeleteInactiveTopicsEnabled is set to true, we should ensure that allowAutoTopicCreation is also set to true.
-brokerDeleteInactiveTopicsEnabled=true
-
-# How often to check for inactive topics
-brokerDeleteInactiveTopicsFrequencySeconds=60
-
-# Allow you to delete a tenant forcefully.
-forceDeleteTenantAllowed=false
-
-# Allow you to delete a namespace forcefully.
-forceDeleteNamespaceAllowed=false
-
-# Max pending publish requests per connection to avoid keeping large number of pending
-# requests in memory. Default: 1000
-maxPendingPublishRequestsPerConnection=1000
-
-# How frequently to proactively check and purge expired messages
-messageExpiryCheckIntervalInMinutes=5
-
-# Check between intervals to see if max message size in topic policies has been updated.
-# Default is 60s
-maxMessageSizeCheckIntervalInSeconds=60
-
-# How long to delay rewinding cursor and dispatching messages when active consumer is changed
-activeConsumerFailoverDelayTimeMillis=1000
-
-# How long to delete inactive subscriptions from last consuming
-# When it is 0, inactive subscriptions are not deleted automatically
-subscriptionExpirationTimeMinutes=0
-
-# Enable subscription message redelivery tracker to send redelivery count to consumer (default is enabled)
-subscriptionRedeliveryTrackerEnabled=true
-
-# On KeyShared subscriptions, with default AUTO_SPLIT mode, use splitting ranges or
-# consistent hashing to reassign keys to new consumers
-subscriptionKeySharedUseConsistentHashing=false
-
-# On KeyShared subscriptions, number of points in the consistent-hashing ring.
-# The higher the number, the more equal the assignment of keys to consumers
-subscriptionKeySharedConsistentHashingReplicaPoints=100
-
-# How frequently to proactively check and purge expired subscription
-subscriptionExpiryCheckIntervalInMinutes=5
-
-# Set the default behavior for message deduplication in the broker
-# This can be overridden per-namespace. If enabled, broker will reject
-# messages that were already stored in the topic
-brokerDeduplicationEnabled=false
-
-# Maximum number of producer information that it's going to be
-# persisted for deduplication purposes
-brokerDeduplicationMaxNumberOfProducers=10000
-
-# Number of entries after which a dedup info snapshot is taken.
-# A bigger interval will lead to less snapshots being taken though it would
-# increase the topic recovery time, when the entries published after the
-# snapshot need to be replayed
-brokerDeduplicationEntriesInterval=1000
-
-# Time of inactivity after which the broker will discard the deduplication information
-# relative to a disconnected producer. Default is 6 hours.
-brokerDeduplicationProducerInactivityTimeoutMinutes=360
-
-# When a namespace is created without specifying the number of bundle, this
-# value will be used as the default
-defaultNumberOfNamespaceBundles=4
-
-# Max number of topics allowed to be created in the namespace. When the topics reach the max topics of the namespace,
-# the broker should reject the new topic request(include topic auto-created by the producer or consumer)
-# until the number of connected consumers decrease.
-# Using a value of 0, is disabling maxTopicsPerNamespace-limit check.
-maxTopicsPerNamespace=0
-
-# Allow schema to be auto updated at broker level. User can override this by
-# 'is_allow_auto_update_schema' of namespace policy.
-isAllowAutoUpdateSchemaEnabled=true
-
-# Enable check for minimum allowed client library version
-clientLibraryVersionCheckEnabled=false
-
-# Path for the file used to determine the rotation status for the broker when responding
-# to service discovery health checks
-statusFilePath=/usr/local/apache/htdocs
-
-# Max number of unacknowledged messages allowed to receive messages by a consumer on a shared subscription. Broker will stop sending
-# messages to consumer once, this limit reaches until consumer starts acknowledging messages back
-# Using a value of 0, is disabling unackeMessage limit check and consumer can receive messages without any restriction
-maxUnackedMessagesPerConsumer=50000
-
-# Max number of unacknowledged messages allowed per shared subscription. Broker will stop dispatching messages to
-# all consumers of the subscription once this limit reaches until consumer starts acknowledging messages back and
-# unack count reaches to limit/2. Using a value of 0, is disabling unackedMessage-limit
-# check and dispatcher can dispatch messages without any restriction
-maxUnackedMessagesPerSubscription=200000
-
-# Max number of unacknowledged messages allowed per broker. Once this limit reaches, broker will stop dispatching
-# messages to all shared subscription which has higher number of unack messages until subscriptions start
-# acknowledging messages back and unack count reaches to limit/2. Using a value of 0, is disabling
-# unackedMessage-limit check and broker doesn't block dispatchers
-maxUnackedMessagesPerBroker=0
-
-# Once broker reaches maxUnackedMessagesPerBroker limit, it blocks subscriptions which has higher unacked messages
-# than this percentage limit and subscription will not receive any new messages until that subscription acks back
-# limit/2 messages
-maxUnackedMessagesPerSubscriptionOnBrokerBlocked=0.16
-
-# Tick time to schedule task that checks topic publish rate limiting across all topics
-# Reducing to lower value can give more accuracy while throttling publish but
-# it uses more CPU to perform frequent check. (Disable publish throttling with value 0)
-topicPublisherThrottlingTickTimeMillis=2
-
-# Enable precise rate limit for topic publish
-preciseTopicPublishRateLimiterEnable=false
-
-# Tick time to schedule task that checks broker publish rate limiting across all topics
-# Reducing to lower value can give more accuracy while throttling publish but
-# it uses more CPU to perform frequent check. (Disable publish throttling with value 0)
-brokerPublisherThrottlingTickTimeMillis=50
-
-# Max Rate(in 1 seconds) of Message allowed to publish for a broker if broker publish rate limiting enabled
-# (Disable message rate limit with value 0)
-brokerPublisherThrottlingMaxMessageRate=0
-
-# Max Rate(in 1 seconds) of Byte allowed to publish for a broker if broker publish rate limiting enabled
-# (Disable byte rate limit with value 0)
-brokerPublisherThrottlingMaxByteRate=0
-
-# Default messages per second dispatch throttling-limit for every topic. Using a value of 0, is disabling default
-# message dispatch-throttling
-dispatchThrottlingRatePerTopicInMsg=0
-
-# Default bytes per second dispatch throttling-limit for every topic. Using a value of 0, is disabling
-# default message-byte dispatch-throttling
-dispatchThrottlingRatePerTopicInByte=0
-
-# Apply dispatch rate limiting on batch message instead individual
-# messages with in batch message. (Default is disabled)
-dispatchThrottlingOnBatchMessageEnabled=false
-
-# Dispatch rate-limiting relative to publish rate.
-# (Enabling flag will make broker to dynamically update dispatch-rate relatively to publish-rate:
-# throttle-dispatch-rate = (publish-rate + configured dispatch-rate).
-dispatchThrottlingRateRelativeToPublishRate=false
-
-# By default we enable dispatch-throttling for both caught up consumers as well as consumers who have
-# backlog.
-dispatchThrottlingOnNonBacklogConsumerEnabled=true
-
-# The read failure backoff initial time in milliseconds. By default it is 15s.
-dispatcherReadFailureBackoffInitialTimeInMs=15000
-
-# The read failure backoff max time in milliseconds. By default it is 60s.
-dispatcherReadFailureBackoffMaxTimeInMs=60000
-
-# The read failure backoff mandatory stop time in milliseconds. By default it is 0s.
-dispatcherReadFailureBackoffMandatoryStopTimeInMs=0
-
-# Precise dispathcer flow control according to history message number of each entry
-preciseDispatcherFlowControl=false
-
-# Max number of concurrent lookup request broker allows to throttle heavy incoming lookup traffic
-maxConcurrentLookupRequest=50000
-
-# Max number of concurrent topic loading request broker allows to control number of zk-operations
-maxConcurrentTopicLoadRequest=5000
-
-# Max concurrent non-persistent message can be processed per connection
-maxConcurrentNonPersistentMessagePerConnection=1000
-
-# Number of worker threads to serve non-persistent topic
-numWorkerThreadsForNonPersistentTopic=8
-
-# Enable broker to load persistent topics
-enablePersistentTopics=true
-
-# Enable broker to load non-persistent topics
-enableNonPersistentTopics=true
-
-# Max number of producers allowed to connect to topic. Once this limit reaches, Broker will reject new producers
-# until the number of connected producers decrease.
-# Using a value of 0, is disabling maxProducersPerTopic-limit check.
-maxProducersPerTopic=0
-
-# Max number of producers with the same IP address allowed to connect to topic.
-# Once this limit reaches, Broker will reject new producers until the number of
-# connected producers with the same IP address decrease.
-# Using a value of 0, is disabling maxSameAddressProducersPerTopic-limit check.
-maxSameAddressProducersPerTopic=0
-
-# Enforce producer to publish encrypted messages.(default disable).
-encryptionRequireOnProducer=false
-
-# Max number of consumers allowed to connect to topic. Once this limit reaches, Broker will reject new consumers
-# until the number of connected consumers decrease.
-# Using a value of 0, is disabling maxConsumersPerTopic-limit check.
-maxConsumersPerTopic=0
-
-# Max number of consumers with the same IP address allowed to connect to topic.
-# Once this limit reaches, Broker will reject new consumers until the number of
-# connected consumers with the same IP address decrease.
-# Using a value of 0, is disabling maxSameAddressConsumersPerTopic-limit check.
-maxSameAddressConsumersPerTopic=0
-
-# Max number of subscriptions allowed to subscribe to topic. Once this limit reaches, broker will reject
-# new subscription until the number of subscribed subscriptions decrease.
-# Using a value of 0, is disabling maxSubscriptionsPerTopic limit check.
-maxSubscriptionsPerTopic=0
-
-# Max number of consumers allowed to connect to subscription. Once this limit reaches, Broker will reject new consumers
-# until the number of connected consumers decrease.
-# Using a value of 0, is disabling maxConsumersPerSubscription-limit check.
-maxConsumersPerSubscription=0
-
-# Max number of partitions per partitioned topic
-# Use 0 or negative number to disable the check
-maxNumPartitionsPerPartitionedTopic=0
-
-### --- Metadata Store --- ###
-
-# Whether we should enable metadata operations batching
-metadataStoreBatchingEnabled=true
-
-# Maximum delay to impose on batching grouping
-metadataStoreBatchingMaxDelayMillis=5
-
-# Maximum number of operations to include in a singular batch
-metadataStoreBatchingMaxOperations=1000
-
-# Maximum size of a batch
-metadataStoreBatchingMaxSizeKb=128
-
-### --- TLS --- ###
-# Deprecated - Use webServicePortTls and brokerServicePortTls instead
-tlsEnabled=false
-
-# Tls cert refresh duration in seconds (set 0 to check on every new connection)
-tlsCertRefreshCheckDurationSec=300
-
-# Path for the TLS certificate file
-tlsCertificateFilePath=
-
-# Path for the TLS private key file
-tlsKeyFilePath=
-
-# Path for the trusted TLS certificate file.
-# This cert is used to verify that any certs presented by connecting clients
-# are signed by a certificate authority. If this verification
-# fails, then the certs are untrusted and the connections are dropped.
-tlsTrustCertsFilePath=
-
-# Accept untrusted TLS certificate from client.
-# If true, a client with a cert which cannot be verified with the
-# 'tlsTrustCertsFilePath' cert will allowed to connect to the server,
-# though the cert will not be used for client authentication.
-tlsAllowInsecureConnection=false
-
-# Specify the tls protocols the broker will use to negotiate during TLS handshake
-# (a comma-separated list of protocol names).
-# Examples:- [TLSv1.3, TLSv1.2]
-tlsProtocols=
-
-# Specify the tls cipher the broker will use to negotiate during TLS Handshake
-# (a comma-separated list of ciphers).
-# Examples:- [TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256]
-tlsCiphers=
-
-# Trusted client certificates are required for to connect TLS
-# Reject the Connection if the Client Certificate is not trusted.
-# In effect, this requires that all connecting clients perform TLS client
-# authentication.
-tlsRequireTrustedClientCertOnConnect=false
-
-### --- KeyStore TLS config variables --- ###
-# Enable TLS with KeyStore type configuration in broker.
-tlsEnabledWithKeyStore=false
-
-# TLS Provider for KeyStore type
-tlsProvider=
-
-# TLS KeyStore type configuration in broker: JKS, PKCS12
-tlsKeyStoreType=JKS
-
-# TLS KeyStore path in broker
-tlsKeyStore=
-
-# TLS KeyStore password for broker
-tlsKeyStorePassword=
-
-# TLS TrustStore type configuration in broker: JKS, PKCS12
-tlsTrustStoreType=JKS
-
-# TLS TrustStore path in broker
-tlsTrustStore=
-
-# TLS TrustStore password for broker
-tlsTrustStorePassword=
-
-# Whether internal client use KeyStore type to authenticate with Pulsar brokers
-brokerClientTlsEnabledWithKeyStore=false
-
-# The TLS Provider used by internal client to authenticate with other Pulsar brokers
-brokerClientSslProvider=
-
-# TLS TrustStore type configuration for internal client: JKS, PKCS12
-# used by the internal client to authenticate with Pulsar brokers
-brokerClientTlsTrustStoreType=JKS
-
-# TLS TrustStore path for internal client
-# used by the internal client to authenticate with Pulsar brokers
-brokerClientTlsTrustStore=
-
-# TLS TrustStore password for internal client,
-# used by the internal client to authenticate with Pulsar brokers
-brokerClientTlsTrustStorePassword=
-
-# Specify the tls cipher the internal client will use to negotiate during TLS Handshake
-# (a comma-separated list of ciphers)
-# e.g.  [TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256].
-# used by the internal client to authenticate with Pulsar brokers
-brokerClientTlsCiphers=
-
-# Specify the tls protocols the broker will use to negotiate during TLS handshake
-# (a comma-separated list of protocol names).
-# e.g.  [TLSv1.3, TLSv1.2]
-# used by the internal client to authenticate with Pulsar brokers
-brokerClientTlsProtocols=
-
-# Enable or disable system topic
-systemTopicEnabled=true
-
-# Enable or disable topic level policies, topic level policies depends on the system topic
-# Please enable the system topic first.
-topicLevelPoliciesEnabled=false
-
-# If a topic remains fenced for this number of seconds, it will be closed forcefully.
-# If it is set to 0 or a negative number, the fenced topic will not be closed.
-topicFencingTimeoutSeconds=0
-
-### --- Authentication --- ###
-# Role names that are treated as "proxy roles". If the broker sees a request with
-#role as proxyRoles - it will demand to see a valid original principal.
-proxyRoles=
-
-# If this flag is set then the broker authenticates the original Auth data
-# else it just accepts the originalPrincipal and authorizes it (if required).
-authenticateOriginalAuthData=false
-
-# Enable authentication
-authenticationEnabled=false
-
-# Authentication provider name list, which is comma separated list of class names
-authenticationProviders=
-
-# Enforce authorization
-authorizationEnabled=false
-
-# Authorization provider fully qualified class-name
-authorizationProvider=org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider
-
-# Allow wildcard matching in authorization
-# (wildcard matching only applicable if wildcard-char:
-# * presents at first or last position eg: *.pulsar.service, pulsar.service.*)
-authorizationAllowWildcardsMatching=false
-
-# Role names that are treated as "super-user", meaning they will be able to do all admin
-# operations and publish/consume from all topics
-superUserRoles=
-
-# Authentication settings of the broker itself. Used when the broker connects to other brokers,
-# either in same or other clusters
-brokerClientAuthenticationPlugin=
-brokerClientAuthenticationParameters=
-
-# Supported Athenz provider domain names(comma separated) for authentication
-athenzDomainNames=
-
-# When this parameter is not empty, unauthenticated users perform as anonymousUserRole
-anonymousUserRole=
-
-
-### --- Token Authentication Provider --- ###
-
-## Symmetric key
-# Configure the secret key to be used to validate auth tokens
-# The key can be specified like:
-# tokenSecretKey=data:;base64,xxxxxxxxx
-# tokenSecretKey=file:///my/secret.key  ( Note: key file must be DER-encoded )
-tokenSecretKey=
-
-## Asymmetric public/private key pair
-# Configure the public key to be used to validate auth tokens
-# The key can be specified like:
-# tokenPublicKey=data:;base64,xxxxxxxxx
-# tokenPublicKey=file:///my/public.key    ( Note: key file must be DER-encoded )
-tokenPublicKey=
-
-
-# The token "claim" that will be interpreted as the authentication "role" or "principal" by AuthenticationProviderToken (defaults to "sub" if blank)
-tokenAuthClaim=
-
-# The token audience "claim" name, e.g. "aud", that will be used to get the audience from token.
-# If not set, audience will not be verified.
-tokenAudienceClaim=
-
-# The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this.
-tokenAudience=
-
-### --- BookKeeper Client --- ###
-
-# Authentication plugin to use when connecting to bookies
-bookkeeperClientAuthenticationPlugin=
-
-# BookKeeper auth plugin implementation specifics parameters name and values
-bookkeeperClientAuthenticationParametersName=
-bookkeeperClientAuthenticationParameters=
-
-# Timeout for BK add / read operations
-bookkeeperClientTimeoutInSeconds=30
-
-# Number of BookKeeper client worker threads
-# Default is Runtime.getRuntime().availableProcessors()
-bookkeeperClientNumWorkerThreads=
-
-# Speculative reads are initiated if a read request doesn't complete within a certain time
-# Using a value of 0, is disabling the speculative reads
-bookkeeperClientSpeculativeReadTimeoutInMillis=0
-
-# Number of channels per bookie
-bookkeeperNumberOfChannelsPerBookie=16
-
-# Enable bookies health check. Bookies that have more than the configured number of failure within
-# the interval will be quarantined for some time. During this period, new ledgers won't be created
-# on these bookies
-bookkeeperClientHealthCheckEnabled=true
-bookkeeperClientHealthCheckIntervalSeconds=60
-bookkeeperClientHealthCheckErrorThresholdPerInterval=5
-bookkeeperClientHealthCheckQuarantineTimeInSeconds=1800
-
-#bookie quarantine ratio to avoid all clients quarantine the high pressure bookie servers at the same time
-bookkeeperClientQuarantineRatio=1.0
-
-# Enable rack-aware bookie selection policy. BK will chose bookies from different racks when
-# forming a new bookie ensemble
-# This parameter related to ensemblePlacementPolicy in conf/bookkeeper.conf, if enabled, ensemblePlacementPolicy
-# should be set to org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy
-bookkeeperClientRackawarePolicyEnabled=true
-
-# Enable region-aware bookie selection policy. BK will chose bookies from
-# different regions and racks when forming a new bookie ensemble.
-# If enabled, the value of bookkeeperClientRackawarePolicyEnabled is ignored
-# This parameter related to ensemblePlacementPolicy in conf/bookkeeper.conf, if enabled, ensemblePlacementPolicy
-# should be set to org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy
-bookkeeperClientRegionawarePolicyEnabled=false
-
-# Minimum number of racks per write quorum. BK rack-aware bookie selection policy will try to
-# get bookies from at least 'bookkeeperClientMinNumRacksPerWriteQuorum' racks for a write quorum.
-bookkeeperClientMinNumRacksPerWriteQuorum=1
-
-# Enforces rack-aware bookie selection policy to pick bookies from 'bookkeeperClientMinNumRacksPerWriteQuorum'
-# racks for a writeQuorum.
-# If BK can't find bookie then it would throw BKNotEnoughBookiesException instead of picking random one.
-bookkeeperClientEnforceMinNumRacksPerWriteQuorum=false
-
-# Enable/disable reordering read sequence on reading entries.
-bookkeeperClientReorderReadSequenceEnabled=false
-
-# Enable bookie isolation by specifying a list of bookie groups to choose from. Any bookie
-# outside the specified groups will not be used by the broker
-bookkeeperClientIsolationGroups=
-
-# Enable bookie secondary-isolation group if bookkeeperClientIsolationGroups doesn't
-# have enough bookie available.
-bookkeeperClientSecondaryIsolationGroups=
-
-# Minimum bookies that should be available as part of bookkeeperClientIsolationGroups
-# else broker will include bookkeeperClientSecondaryIsolationGroups bookies in isolated list.
-bookkeeperClientMinAvailableBookiesInIsolationGroups=
-
-# Set the client security provider factory class name.
-# Default: org.apache.bookkeeper.tls.TLSContextFactory
-bookkeeperTLSProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory
-
-# Enable tls authentication with bookie
-bookkeeperTLSClientAuthentication=false
-
-# Supported type: PEM, JKS, PKCS12. Default value: PEM
-bookkeeperTLSKeyFileType=PEM
-
-#Supported type: PEM, JKS, PKCS12. Default value: PEM
-bookkeeperTLSTrustCertTypes=PEM
-
-# Path to file containing keystore password, if the client keystore is password protected.
-bookkeeperTLSKeyStorePasswordPath=
-
-# Path to file containing truststore password, if the client truststore is password protected.
-bookkeeperTLSTrustStorePasswordPath=
-
-# Path for the TLS private key file
-bookkeeperTLSKeyFilePath=
-
-# Path for the TLS certificate file
-bookkeeperTLSCertificateFilePath=
-
-# Path for the trusted TLS certificate file
-bookkeeperTLSTrustCertsFilePath=
-
-# Enable/disable disk weight based placement. Default is false
-bookkeeperDiskWeightBasedPlacementEnabled=false
-
-# Set the interval to check the need for sending an explicit LAC
-# A value of '0' disables sending any explicit LACs. Default is 0.
-bookkeeperExplicitLacIntervalInMills=0
-
-# Use older Bookkeeper wire protocol with bookie
-bookkeeperUseV2WireProtocol=true
-
-# Expose bookkeeper client managed ledger stats to prometheus. default is false
-# bookkeeperClientExposeStatsToPrometheus=false
-
-### --- Managed Ledger --- ###
-
-# Number of bookies to use when creating a ledger
-managedLedgerDefaultEnsembleSize=1
-
-# Number of copies to store for each message
-managedLedgerDefaultWriteQuorum=1
-
-# Number of guaranteed copies (acks to wait before write is complete)
-managedLedgerDefaultAckQuorum=1
-
-# How frequently to flush the cursor positions that were accumulated due to rate limiting. (seconds).
-# Default is 60 seconds
-managedLedgerCursorPositionFlushSeconds=60
-
-# Default type of checksum to use when writing to BookKeeper. Default is "CRC32C"
-# Other possible options are "CRC32", "MAC" or "DUMMY" (no checksum).
-managedLedgerDigestType=CRC32C
-
-# Number of threads to be used for managed ledger tasks dispatching
-managedLedgerNumWorkerThreads=4
-
-# Number of threads to be used for managed ledger scheduled tasks
-managedLedgerNumSchedulerThreads=4
-
-# Amount of memory to use for caching data payload in managed ledger. This memory
-# is allocated from JVM direct memory and it's shared across all the topics
-# running  in the same broker. By default, uses 1/5th of available direct memory
-managedLedgerCacheSizeMB=
-
-# Whether we should make a copy of the entry payloads when inserting in cache
-managedLedgerCacheCopyEntries=false
-
-# Threshold to which bring down the cache level when eviction is triggered
-managedLedgerCacheEvictionWatermark=0.9
-
-# Configure the cache eviction frequency for the managed ledger cache (evictions/sec)
-managedLedgerCacheEvictionFrequency=100.0
-
-# All entries that have stayed in cache for more than the configured time, will be evicted
-managedLedgerCacheEvictionTimeThresholdMillis=1000
-
-# Configure the threshold (in number of entries) from where a cursor should be considered 'backlogged'
-# and thus should be set as inactive.
-managedLedgerCursorBackloggedThreshold=1000
-
-# Rate limit the amount of writes generated by consumer acking the messages
-managedLedgerDefaultMarkDeleteRateLimit=0.1
-
-# Max number of entries to append to a ledger before triggering a rollover
-# A ledger rollover is triggered after the min rollover time has passed
-# and one of the following conditions is true:
-#  * The max rollover time has been reached
-#  * The max entries have been written to the ledger
-#  * The max ledger size has been written to the ledger
-managedLedgerMaxEntriesPerLedger=50000
-
-# Minimum time between ledger rollover for a topic
-managedLedgerMinLedgerRolloverTimeMinutes=10
-
-# Maximum time before forcing a ledger rollover for a topic
-managedLedgerMaxLedgerRolloverTimeMinutes=240
-
-# Max number of entries to append to a cursor ledger
-managedLedgerCursorMaxEntriesPerLedger=50000
-
-# Max time before triggering a rollover on a cursor ledger
-managedLedgerCursorRolloverTimeInSeconds=14400
-
-# Maximum ledger size before triggering a rollover for a topic (MB)
-managedLedgerMaxSizePerLedgerMbytes=2048
-
-# Max number of "acknowledgment holes" that are going to be persistently stored.
-# When acknowledging out of order, a consumer will leave holes that are supposed
-# to be quickly filled by acking all the messages. The information of which
-# messages are acknowledged is persisted by compressing in "ranges" of messages
-# that were acknowledged. After the max number of ranges is reached, the information
-# will only be tracked in memory and messages will be redelivered in case of
-# crashes.
-managedLedgerMaxUnackedRangesToPersist=10000
-
-# Max number of "acknowledgment holes" that can be stored in Zookeeper. If number of unack message range is higher
-# than this limit then broker will persist unacked ranges into bookkeeper to avoid additional data overhead into
-# zookeeper.
-managedLedgerMaxUnackedRangesToPersistInZooKeeper=1000
-
-# Skip reading non-recoverable/unreadable data-ledger under managed-ledger's list. It helps when data-ledgers gets
-# corrupted at bookkeeper and managed-cursor is stuck at that ledger.
-autoSkipNonRecoverableData=false
-
-# operation timeout while updating managed-ledger metadata.
-managedLedgerMetadataOperationsTimeoutSeconds=60
-
-# Read entries timeout when broker tries to read messages from bookkeeper.
-managedLedgerReadEntryTimeoutSeconds=0
-
-# Add entry timeout when broker tries to publish message to bookkeeper (0 to disable it).
-managedLedgerAddEntryTimeoutSeconds=0
-
-# New entries check delay for the cursor under the managed ledger.
-# If no new messages in the topic, the cursor will try to check again after the delay time.
-# For consumption latency sensitive scenario, can set to a smaller value or set to 0.
-# Of course, use a smaller value may degrade consumption throughput. Default is 10ms.
-managedLedgerNewEntriesCheckDelayInMillis=10
-
-# Use Open Range-Set to cache unacked messages
-managedLedgerUnackedRangesOpenCacheSetEnabled=true
-
-# Managed ledger prometheus stats latency rollover seconds (default: 60s)
-managedLedgerPrometheusStatsLatencyRolloverSeconds=60
-
-# Whether trace managed ledger task execution time
-managedLedgerTraceTaskExecution=true
-
-# If you want to custom bookie ID or use a dynamic network address for the bookie,
-# you can set this option.
-# Bookie advertises itself using bookieId rather than
-# BookieSocketAddress (hostname:port or IP:port).
-# bookieId is a non empty string that can contain ASCII digits and letters ([a-zA-Z9-0]),
-# colons, dashes, and dots.
-# For more information about bookieId, see http://bookkeeper.apache.org/bps/BP-41-bookieid/.
-# bookieId=
-
-### --- Load balancer --- ###
-
-loadManagerClassName=org.apache.pulsar.broker.loadbalance.NoopLoadManager
-
-# Enable load balancer
-loadBalancerEnabled=false
-
-# Percentage of change to trigger load report update
-loadBalancerReportUpdateThresholdPercentage=10
-
-# maximum interval to update load report
-loadBalancerReportUpdateMaxIntervalMinutes=15
-
-# Frequency of report to collect
-loadBalancerHostUsageCheckIntervalMinutes=1
-
-# Load shedding interval. Broker periodically checks whether some traffic should be offload from
-# some over-loaded broker to other under-loaded brokers
-loadBalancerSheddingIntervalMinutes=1
-
-# Prevent the same topics to be shed and moved to other broker more than once within this timeframe
-loadBalancerSheddingGracePeriodMinutes=30
-
-# Usage threshold to allocate max number of topics to broker
-loadBalancerBrokerMaxTopics=50000
-
-# Interval to flush dynamic resource quota to ZooKeeper
-loadBalancerResourceQuotaUpdateIntervalMinutes=15
-
-# enable/disable namespace bundle auto split
-loadBalancerAutoBundleSplitEnabled=true
-
-# enable/disable automatic unloading of split bundles
-loadBalancerAutoUnloadSplitBundlesEnabled=true
-
-# maximum topics in a bundle, otherwise bundle split will be triggered
-loadBalancerNamespaceBundleMaxTopics=1000
-
-# maximum sessions (producers + consumers) in a bundle, otherwise bundle split will be triggered
-loadBalancerNamespaceBundleMaxSessions=1000
-
-# maximum msgRate (in + out) in a bundle, otherwise bundle split will be triggered
-loadBalancerNamespaceBundleMaxMsgRate=30000
-
-# maximum bandwidth (in + out) in a bundle, otherwise bundle split will be triggered
-loadBalancerNamespaceBundleMaxBandwidthMbytes=100
-
-# maximum number of bundles in a namespace
-loadBalancerNamespaceMaximumBundles=128
-
-# The broker resource usage threshold.
-# When the broker resource usage is greater than the pulsar cluster average resource usage,
-# the threshold shedder will be triggered to offload bundles from the broker.
-# It only takes effect in the ThresholdShedder strategy.
-loadBalancerBrokerThresholdShedderPercentage=10
-
-# When calculating new resource usage, the history usage accounts for.
-# It only takes effect in the ThresholdShedder strategy.
-loadBalancerHistoryResourcePercentage=0.9
-
-# The BandWithIn usage weight when calculating new resource usage.
-# It only takes effect in the ThresholdShedder strategy.
-loadBalancerBandwithInResourceWeight=1.0
-
-# The BandWithOut usage weight when calculating new resource usage.
-# It only takes effect in the ThresholdShedder strategy.
-loadBalancerBandwithOutResourceWeight=1.0
-
-# The CPU usage weight when calculating new resource usage.
-# It only takes effect in the ThresholdShedder strategy.
-loadBalancerCPUResourceWeight=1.0
-
-# The heap memory usage weight when calculating new resource usage.
-# It only takes effect in the ThresholdShedder strategy.
-loadBalancerMemoryResourceWeight=1.0
-
-# The direct memory usage weight when calculating new resource usage.
-# It only takes effect in the ThresholdShedder strategy.
-loadBalancerDirectMemoryResourceWeight=1.0
-
-# Bundle unload minimum throughput threshold (MB), avoiding bundle unload frequently.
-# It only takes effect in the ThresholdShedder strategy.
-loadBalancerBundleUnloadMinThroughputThreshold=10
-
-# Time to wait for the unloading of a namespace bundle
-namespaceBundleUnloadingTimeoutMs=60000
-
-### --- Replication --- ###
-
-# Enable replication metrics
-replicationMetricsEnabled=true
-
-# Max number of connections to open for each broker in a remote cluster
-# More connections host-to-host lead to better throughput over high-latency
-# links.
-replicationConnectionsPerBroker=16
-
-# Replicator producer queue size
-replicationProducerQueueSize=1000
-
-# Duration to check replication policy to avoid replicator inconsistency
-# due to missing ZooKeeper watch (disable with value 0)
-replicationPolicyCheckDurationSeconds=600
-
-# Default message retention time
-defaultRetentionTimeInMinutes=0
-
-# Default retention size
-defaultRetentionSizeInMB=0
-
-# How often to check whether the connections are still alive
-keepAliveIntervalSeconds=30
-
-### --- WebSocket --- ###
-
-# Enable the WebSocket API service in broker
-webSocketServiceEnabled=true
-
-# Number of IO threads in Pulsar Client used in WebSocket proxy
-webSocketNumIoThreads=8
-
-# Number of connections per Broker in Pulsar Client used in WebSocket proxy
-webSocketConnectionsPerBroker=8
-
-# Time in milliseconds that idle WebSocket session times out
-webSocketSessionIdleTimeoutMillis=300000
-
-# The maximum size of a text message during parsing in WebSocket proxy
-webSocketMaxTextFrameSize=1048576
-
-### --- Metrics --- ###
-
-# Enable topic level metrics
-exposeTopicLevelMetricsInPrometheus=true
-
-# Time in milliseconds that metrics endpoint would time out. Default is 30s.
-# Increase it if there are a lot of topics to expose topic-level metrics.
-# Set it to 0 to disable timeout.
-metricsServletTimeoutMs=30000
-
-# Classname of Pluggable JVM GC metrics logger that can log GC specific metrics
-# jvmGCMetricsLoggerClassName=
-
-### --- Broker Web Stats --- ###
-
-# Enable topic level metrics
-exposePublisherStats=true
-
-# Enable expose the precise backlog stats.
-# Set false to use published counter and consumed counter to calculate, this would be more efficient but may be inaccurate.
-# Default is false.
-exposePreciseBacklogInPrometheus=false
-
-# Enable splitting topic and partition label in Prometheus.
-# If enabled, a topic name will split into 2 parts, one is topic name without partition index,
-# another one is partition index, e.g. (topic=xxx, partition=0).
-# If the topic is a non-partitioned topic, -1 will be used for the partition index.
-# If disabled, one label to represent the topic and partition, e.g. (topic=xxx-partition-0)
-# Default is false.
-
-splitTopicAndPartitionLabelInPrometheus=false
-
-### --- Deprecated config variables --- ###
-
-# Deprecated. Use configurationStoreServers
-globalZookeeperServers=
-
-# Deprecated. Use brokerDeleteInactiveTopicsFrequencySeconds
-brokerServicePurgeInactiveFrequencyInSeconds=60
-
-### --- BookKeeper Configuration --- #####
-
-ledgerStorageClass=org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage
-
-# The maximum netty frame size in bytes. Any message received larger than this will be rejected. The default value is 5MB.
-nettyMaxFrameSizeBytes=5253120
-
-# Size of Write Cache. Memory is allocated from JVM direct memory.
-# Write cache is used to buffer entries before flushing into the entry log
-# For good performance, it should be big enough to hold a substantial amount
-# of entries in the flush interval
-# By default it will be allocated to 1/4th of the available direct memory
-dbStorage_writeCacheMaxSizeMb=
-
-# Size of Read cache. Memory is allocated from JVM direct memory.
-# This read cache is pre-filled doing read-ahead whenever a cache miss happens
-# By default it will be allocated to 1/4th of the available direct memory
-dbStorage_readAheadCacheMaxSizeMb=
-
-# How many entries to pre-fill in cache after a read cache miss
-dbStorage_readAheadCacheBatchSize=1000
-
-flushInterval=60000
-
-## RocksDB specific configurations
-## DbLedgerStorage uses RocksDB to store the indexes from
-## (ledgerId, entryId) -> (entryLog, offset)
-
-# Size of RocksDB block-cache. For best performance, this cache
-# should be big enough to hold a significant portion of the index
-# database which can reach ~2GB in some cases
-# Default is to use 10% of the direct memory size
-dbStorage_rocksDB_blockCacheSize=
-
-# Other RocksDB specific tunables
-dbStorage_rocksDB_writeBufferSizeMB=4
-dbStorage_rocksDB_sstSizeInMB=4
-dbStorage_rocksDB_blockSize=4096
-dbStorage_rocksDB_bloomFilterBitsPerKey=10
-dbStorage_rocksDB_numLevels=-1
-dbStorage_rocksDB_numFilesInLevel0=4
-dbStorage_rocksDB_maxSizeInLevel1MB=256
-
-# Maximum latency to impose on a journal write to achieve grouping
-journalMaxGroupWaitMSec=1
-
-# Should the data be fsynced on journal before acknowledgment.
-journalSyncData=false
-
-
-# For each ledger dir, maximum disk space which can be used.
-# Default is 0.95f. i.e. 95% of disk can be used at most after which nothing will
-# be written to that partition. If all ledger dir partions are full, then bookie
-# will turn to readonly mode if 'readOnlyModeEnabled=true' is set, else it will
-# shutdown.
-# Valid values should be in between 0 and 1 (exclusive).
-diskUsageThreshold=0.99
-
-# The disk free space low water mark threshold.
-# Disk is considered full when usage threshold is exceeded.
-# Disk returns back to non-full state when usage is below low water mark threshold.
-# This prevents it from going back and forth between these states frequently
-# when concurrent writes and compaction are happening. This also prevent bookie from
-# switching frequently between read-only and read-writes states in the same cases.
-diskUsageWarnThreshold=0.99
-
-# Whether the bookie allowed to use a loopback interface as its primary
-# interface(i.e. the interface it uses to establish its identity)?
-# By default, loopback interfaces are not allowed as the primary
-# interface.
-# Using a loopback interface as the primary interface usually indicates
-# a configuration error. For example, its fairly common in some VPS setups
-# to not configure a hostname, or to have the hostname resolve to
-# 127.0.0.1. If this is the case, then all bookies in the cluster will
-# establish their identities as 127.0.0.1:3181, and only one will be able
-# to join the cluster. For VPSs configured like this, you should explicitly
-# set the listening interface.
-allowLoopback=true
-
-# How long the interval to trigger next garbage collection, in milliseconds
-# Since garbage collection is running in background, too frequent gc
-# will heart performance. It is better to give a higher number of gc
-# interval if there is enough disk capacity.
-gcWaitTime=300000
-
-# Enable topic auto creation if new producer or consumer connected (disable auto creation with value false)
-allowAutoTopicCreation=true
-
-# The type of topic that is allowed to be automatically created.(partitioned/non-partitioned)
-allowAutoTopicCreationType=non-partitioned
-
-# Enable subscription auto creation if new consumer connected (disable auto creation with value false)
-allowAutoSubscriptionCreation=true
-
-# The number of partitioned topics that is allowed to be automatically created if allowAutoTopicCreationType is partitioned.
-defaultNumPartitions=1
-
-### --- Transaction config variables --- ###
-# Enable transaction coordinator in broker
-transactionCoordinatorEnabled=true
-transactionMetadataStoreProviderClassName=org.apache.pulsar.transaction.coordinator.impl.InMemTransactionMetadataStoreProvider
-
-# Transaction buffer take snapshot transaction count
-transactionBufferSnapshotMaxTransactionCount=1000
-
-# Transaction buffer take snapshot interval time
-# Unit : millisecond
-transactionBufferSnapshotMinTimeInMillis=5000
-
-### --- Packages management service configuration variables (begin) --- ###
-
-# Enable the packages management service or not
-enablePackagesManagement=false
-
-# The packages management service storage service provide
-packagesManagementStorageProvider=org.apache.pulsar.packages.management.storage.bookkeeper.BookKeeperPackagesStorageProvider
-
-# When the packages storage provider is bookkeeper, you can use this configuration to
-# control the number of replicas for storing the package
-packagesReplicas=1
-
-# The bookkeeper ledger root path
-packagesManagementLedgerRootPath=/ledgers
-
-### --- Packages management service configuration variables (end) --- ###


[flink] 04/06: [FLINK-29381][Connector/Pulsar] Fixes the split assignment for Key Shared subscription.

Posted by ti...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tison pushed a commit to branch release-1.16
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 632f42dbe0e5d254dedcdbfdcb858e5f766d6db9
Author: Yufan Sheng <yu...@streamnative.io>
AuthorDate: Wed Sep 28 15:27:26 2022 +0800

    [FLINK-29381][Connector/Pulsar] Fixes the split assignment for Key Shared subscription.
---
 .../generated/pulsar_consumer_configuration.html   |   2 +-
 .../generated/pulsar_source_configuration.html     |   6 +
 .../pulsar/common/utils/PulsarSerdeUtils.java      |  28 ++++
 .../pulsar/source/PulsarSourceBuilder.java         |   8 +-
 .../pulsar/source/PulsarSourceOptions.java         |  21 ++-
 .../source/config/PulsarConsumerBuilder.java       |  89 ++++++++++
 .../source/config/PulsarSourceConfigUtils.java     |  16 +-
 .../pulsar/source/config/SourceConfiguration.java  |  14 +-
 .../PulsarSourceEnumStateSerializer.java           |  16 +-
 ...itAssigner.java => KeySharedSplitAssigner.java} |  65 +++-----
 .../assigner/NonSharedSplitAssigner.java           |  35 +---
 .../enumerator/assigner/SplitAssignerBase.java     |  33 ++++
 .../enumerator/assigner/SplitAssignerFactory.java  |   4 +-
 .../subscriber/impl/BasePulsarSubscriber.java      |  34 ++--
 .../subscriber/impl/TopicListSubscriber.java       |  10 +-
 .../subscriber/impl/TopicPatternSubscriber.java    |   7 +-
 .../source/enumerator/topic/TopicPartition.java    |  70 ++++++--
 .../topic/range/FixedKeysRangeGenerator.java       | 185 +++++++++++++++++++++
 .../topic/range/FixedRangeGenerator.java           |  17 ++
 .../enumerator/topic/range/FullRangeGenerator.java |  12 +-
 .../enumerator/topic/range/RangeGenerator.java     |  41 ++++-
 ...angeGenerator.java => SplitRangeGenerator.java} |  45 ++++-
 .../enumerator/topic/range/TopicRangeUtils.java    | 136 +++++++++++++++
 .../split/PulsarPartitionSplitReaderBase.java      |  14 +-
 .../split/PulsarPartitionSplitSerializer.java      |  47 ++++--
 .../pulsar/source/PulsarSourceBuilderTest.java     |   4 +-
 .../PulsarSourceEnumStateSerializerTest.java       |  95 ++++++++++-
 .../enumerator/PulsarSourceEnumeratorTest.java     |   3 +-
 .../assigner/NonSharedSplitAssignerTest.java       |   2 +-
 .../enumerator/assigner/SplitAssignerTestBase.java |   3 +-
 .../source/enumerator/cursor/StopCursorTest.java   |   3 +-
 .../subscriber/PulsarSubscriberTest.java           |  17 +-
 .../enumerator/topic/TopicPartitionTest.java       |   5 +-
 .../topic/range/SplitRangeGeneratorTest.java       |  82 +++++++++
 .../topic/range/TopicRangeUtilsTest.java           |  84 ++++++++++
 .../split/PulsarPartitionSplitReaderTestBase.java  |   5 +-
 .../split/PulsarPartitionSplitSerializerTest.java  |  50 +++++-
 37 files changed, 1124 insertions(+), 184 deletions(-)

diff --git a/docs/layouts/shortcodes/generated/pulsar_consumer_configuration.html b/docs/layouts/shortcodes/generated/pulsar_consumer_configuration.html
index 4e05b270de4..e206ff4f20f 100644
--- a/docs/layouts/shortcodes/generated/pulsar_consumer_configuration.html
+++ b/docs/layouts/shortcodes/generated/pulsar_consumer_configuration.html
@@ -58,7 +58,7 @@
         </tr>
         <tr>
             <td><h5>pulsar.consumer.deadLetterPolicy.maxRedeliverCount</h5></td>
-            <td style="word-wrap: break-word;">0</td>
+            <td style="word-wrap: break-word;">(none)</td>
             <td>Integer</td>
             <td>The maximum number of times that a message are redelivered before being sent to the dead letter queue.</td>
         </tr>
diff --git a/docs/layouts/shortcodes/generated/pulsar_source_configuration.html b/docs/layouts/shortcodes/generated/pulsar_source_configuration.html
index 3bcdad85e1a..65053e4b0ef 100644
--- a/docs/layouts/shortcodes/generated/pulsar_source_configuration.html
+++ b/docs/layouts/shortcodes/generated/pulsar_source_configuration.html
@@ -8,6 +8,12 @@
         </tr>
     </thead>
     <tbody>
+        <tr>
+            <td><h5>pulsar.source.allowKeySharedOutOfOrderDelivery</h5></td>
+            <td style="word-wrap: break-word;">false</td>
+            <td>Boolean</td>
+            <td>If enabled, it will relax the ordering requirement, allowing the broker to send out-of-order messages in case of failures. This will make it faster for new consumers to join without being stalled by an existing slow consumer.<br />In this case, a single consumer will still receive all the keys, but they may be coming in different orders.</td>
+        </tr>
         <tr>
             <td><h5>pulsar.source.autoCommitCursorInterval</h5></td>
             <td style="word-wrap: break-word;">5000</td>
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/utils/PulsarSerdeUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/utils/PulsarSerdeUtils.java
index 93609bc720d..ffe041b9bd7 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/utils/PulsarSerdeUtils.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/utils/PulsarSerdeUtils.java
@@ -27,8 +27,10 @@ import org.apache.flink.util.function.FunctionWithException;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
@@ -75,6 +77,32 @@ public final class PulsarSerdeUtils {
         }
     }
 
+    // Common List serialization.
+
+    public static <T> void serializeList(
+            DataOutputStream out,
+            List<T> list,
+            BiConsumerWithException<DataOutputStream, T, IOException> serializer)
+            throws IOException {
+        out.writeInt(list.size());
+        for (T t : list) {
+            serializer.accept(out, t);
+        }
+    }
+
+    public static <T> List<T> deserializeList(
+            DataInputStream in, FunctionWithException<DataInputStream, T, IOException> deserializer)
+            throws IOException {
+        int size = in.readInt();
+        List<T> set = new ArrayList<>(size);
+        for (int i = 0; i < size; i++) {
+            T t = deserializer.apply(in);
+            set.add(t);
+        }
+
+        return set;
+    }
+
     // Common Set serialization.
 
     public static <T> void serializeSet(
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java
index 1a5d6ea9758..5309dd0eea0 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java
@@ -33,7 +33,7 @@ import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.range.FullRangeGenerator;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.range.RangeGenerator;
-import org.apache.flink.connector.pulsar.source.enumerator.topic.range.UniformRangeGenerator;
+import org.apache.flink.connector.pulsar.source.enumerator.topic.range.SplitRangeGenerator;
 import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema;
 
 import org.apache.pulsar.client.api.RegexSubscriptionMode;
@@ -426,8 +426,8 @@ public final class PulsarSourceBuilder<OUT> {
             if (rangeGenerator == null) {
                 LOG.warn(
                         "No range generator provided for key_shared subscription,"
-                                + " we would use the UniformRangeGenerator as the default range generator.");
-                this.rangeGenerator = new UniformRangeGenerator();
+                                + " we would use the SplitRangeGenerator as the default range generator.");
+                this.rangeGenerator = new SplitRangeGenerator();
             }
         } else {
             // Override the range generator.
@@ -501,7 +501,7 @@ public final class PulsarSourceBuilder<OUT> {
 
     // ------------- private helpers  --------------
 
-    /** Helper method for java compiler recognize the generic type. */
+    /** Helper method for java compiler recognizes the generic type. */
     @SuppressWarnings("unchecked")
     private <T extends OUT> PulsarSourceBuilder<T> specialized() {
         return (PulsarSourceBuilder<T>) this;
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java
index c80ddd3b1fa..440e96b4f71 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java
@@ -71,7 +71,7 @@ public final class PulsarSourceOptions {
     ///////////////////////////////////////////////////////////////////////////////
     //
     // The configuration for pulsar source part.
-    // All the configuration listed below should have the pulsar.source prefix.
+    // All the configurations listed below should have the pulsar.source prefix.
     //
     ///////////////////////////////////////////////////////////////////////////////
 
@@ -189,10 +189,25 @@ public final class PulsarSourceOptions {
                                             " A possible solution is to adjust the retention settings in Pulsar or ignoring the check result.")
                                     .build());
 
+    public static final ConfigOption<Boolean> PULSAR_ALLOW_KEY_SHARED_OUT_OF_ORDER_DELIVERY =
+            ConfigOptions.key(SOURCE_CONFIG_PREFIX + "allowKeySharedOutOfOrderDelivery")
+                    .booleanType()
+                    .defaultValue(false)
+                    .withDescription(
+                            Description.builder()
+                                    .text(
+                                            "If enabled, it will relax the ordering requirement, allowing the broker to send out-of-order messages in case of failures.")
+                                    .text(
+                                            " This will make it faster for new consumers to join without being stalled by an existing slow consumer.")
+                                    .linebreak()
+                                    .text(
+                                            "In this case, a single consumer will still receive all the keys, but they may be coming in different orders.")
+                                    .build());
+
     ///////////////////////////////////////////////////////////////////////////////
     //
     // The configuration for ConsumerConfigurationData part.
-    // All the configuration listed below should have the pulsar.consumer prefix.
+    // All the configurations listed below should have the pulsar.consumer prefix.
     //
     ///////////////////////////////////////////////////////////////////////////////
 
@@ -548,7 +563,7 @@ public final class PulsarSourceOptions {
     public static final ConfigOption<Integer> PULSAR_MAX_REDELIVER_COUNT =
             ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "deadLetterPolicy.maxRedeliverCount")
                     .intType()
-                    .defaultValue(0)
+                    .noDefaultValue()
                     .withDescription(
                             "The maximum number of times that a message are redelivered before being sent to the dead letter queue.");
 
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/PulsarConsumerBuilder.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/PulsarConsumerBuilder.java
new file mode 100644
index 00000000000..09e4322af45
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/PulsarConsumerBuilder.java
@@ -0,0 +1,89 @@
+/*
+ * 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.source.config;
+
+import org.apache.flink.shaded.guava30.com.google.common.base.Strings;
+
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerInterceptor;
+import org.apache.pulsar.client.api.DeadLetterPolicy;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.impl.ConsumerBuilderImpl;
+import org.apache.pulsar.client.impl.ConsumerInterceptors;
+import org.apache.pulsar.client.impl.PulsarClientImpl;
+import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData;
+import org.apache.pulsar.common.naming.TopicName;
+
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import static org.apache.pulsar.client.util.RetryMessageUtil.DLQ_GROUP_TOPIC_SUFFIX;
+import static org.apache.pulsar.client.util.RetryMessageUtil.MAX_RECONSUMETIMES;
+import static org.apache.pulsar.client.util.RetryMessageUtil.RETRY_GROUP_TOPIC_SUFFIX;
+
+/** Override the default consumer builder for supporting build the custom Key_Shared consumer. */
+public class PulsarConsumerBuilder<T> extends ConsumerBuilderImpl<T> {
+
+    public PulsarConsumerBuilder(PulsarClient client, Schema<T> schema) {
+        super((PulsarClientImpl) client, schema);
+    }
+
+    @Override
+    public CompletableFuture<Consumer<T>> subscribeAsync() {
+        PulsarClientImpl client = super.getClient();
+        ConsumerConfigurationData<T> conf = super.getConf();
+        Schema<T> schema = super.getSchema();
+        List<ConsumerInterceptor<T>> interceptorList = super.getInterceptorList();
+
+        // Override the default subscribeAsync for skipping the subscription validation.
+        if (conf.isRetryEnable()) {
+            TopicName topicFirst = TopicName.get(conf.getTopicNames().iterator().next());
+            String retryLetterTopic =
+                    topicFirst + "-" + conf.getSubscriptionName() + RETRY_GROUP_TOPIC_SUFFIX;
+            String deadLetterTopic =
+                    topicFirst + "-" + conf.getSubscriptionName() + DLQ_GROUP_TOPIC_SUFFIX;
+
+            DeadLetterPolicy deadLetterPolicy = conf.getDeadLetterPolicy();
+            if (deadLetterPolicy == null) {
+                conf.setDeadLetterPolicy(
+                        DeadLetterPolicy.builder()
+                                .maxRedeliverCount(MAX_RECONSUMETIMES)
+                                .retryLetterTopic(retryLetterTopic)
+                                .deadLetterTopic(deadLetterTopic)
+                                .build());
+            } else {
+                if (Strings.isNullOrEmpty(deadLetterPolicy.getRetryLetterTopic())) {
+                    deadLetterPolicy.setRetryLetterTopic(retryLetterTopic);
+                }
+                if (Strings.isNullOrEmpty(deadLetterPolicy.getDeadLetterTopic())) {
+                    deadLetterPolicy.setDeadLetterTopic(deadLetterTopic);
+                }
+            }
+
+            conf.getTopicNames().add(conf.getDeadLetterPolicy().getRetryLetterTopic());
+        }
+
+        if (interceptorList == null || interceptorList.isEmpty()) {
+            return client.subscribeAsync(conf, schema, null);
+        } else {
+            return client.subscribeAsync(conf, schema, new ConsumerInterceptors<>(interceptorList));
+        }
+    }
+}
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/PulsarSourceConfigUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/PulsarSourceConfigUtils.java
index 0a4dc31e8d3..063d588decc 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/PulsarSourceConfigUtils.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/PulsarSourceConfigUtils.java
@@ -21,6 +21,7 @@ package org.apache.flink.connector.pulsar.source.config;
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.connector.pulsar.common.config.PulsarConfigValidator;
 
+import org.apache.pulsar.client.api.BatchReceivePolicy;
 import org.apache.pulsar.client.api.Consumer;
 import org.apache.pulsar.client.api.ConsumerBuilder;
 import org.apache.pulsar.client.api.DeadLetterPolicy;
@@ -30,6 +31,7 @@ import org.apache.pulsar.client.api.Schema;
 import java.util.Map;
 import java.util.Optional;
 import java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
 import static java.util.concurrent.TimeUnit.MICROSECONDS;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
@@ -68,8 +70,14 @@ import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSA
 @Internal
 public final class PulsarSourceConfigUtils {
 
+    private static final BatchReceivePolicy DISABLED_BATCH_RECEIVE_POLICY =
+            BatchReceivePolicy.builder()
+                    .timeout(0, TimeUnit.MILLISECONDS)
+                    .maxNumMessages(1)
+                    .build();
+
     private PulsarSourceConfigUtils() {
-        // No need to create instance.
+        // No need to create an instance.
     }
 
     public static final PulsarConfigValidator SOURCE_CONFIG_VALIDATOR =
@@ -83,7 +91,7 @@ public final class PulsarSourceConfigUtils {
     /** Create a pulsar consumer builder by using the given Configuration. */
     public static <T> ConsumerBuilder<T> createConsumerBuilder(
             PulsarClient client, Schema<T> schema, SourceConfiguration configuration) {
-        ConsumerBuilder<T> builder = client.newConsumer(schema);
+        ConsumerBuilder<T> builder = new PulsarConsumerBuilder<>(client, schema);
 
         configuration.useOption(PULSAR_SUBSCRIPTION_NAME, builder::subscriptionName);
         configuration.useOption(
@@ -132,6 +140,10 @@ public final class PulsarSourceConfigUtils {
             builder.properties(properties);
         }
 
+        // Flink connector doesn't need any batch receiving behaviours.
+        // Disable the batch-receive timer for the Consumer instance.
+        builder.batchReceivePolicy(DISABLED_BATCH_RECEIVE_POLICY);
+
         return builder;
     }
 
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java
index f957e1a5775..f9e8a65f0fc 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java
@@ -34,6 +34,7 @@ import java.time.Duration;
 import java.util.Objects;
 
 import static org.apache.flink.connector.base.source.reader.SourceReaderOptions.ELEMENT_QUEUE_CAPACITY;
+import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_ALLOW_KEY_SHARED_OUT_OF_ORDER_DELIVERY;
 import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_AUTO_COMMIT_CURSOR_INTERVAL;
 import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE;
 import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_RECORDS;
@@ -61,6 +62,7 @@ public class SourceConfiguration extends PulsarConfiguration {
     private final String subscriptionName;
     private final SubscriptionType subscriptionType;
     private final SubscriptionMode subscriptionMode;
+    private final boolean allowKeySharedOutOfOrderDelivery;
 
     public SourceConfiguration(Configuration configuration) {
         super(configuration);
@@ -76,6 +78,7 @@ public class SourceConfiguration extends PulsarConfiguration {
         this.subscriptionName = get(PULSAR_SUBSCRIPTION_NAME);
         this.subscriptionType = get(PULSAR_SUBSCRIPTION_TYPE);
         this.subscriptionMode = get(PULSAR_SUBSCRIPTION_MODE);
+        this.allowKeySharedOutOfOrderDelivery = get(PULSAR_ALLOW_KEY_SHARED_OUT_OF_ORDER_DELIVERY);
     }
 
     /** The capacity of the element queue in the source reader. */
@@ -183,6 +186,11 @@ public class SourceConfiguration extends PulsarConfiguration {
         return subscriptionMode;
     }
 
+    /** Whether to enable the out-of-order delivery in Key Shared subscription. */
+    public boolean isAllowKeySharedOutOfOrderDelivery() {
+        return allowKeySharedOutOfOrderDelivery;
+    }
+
     /** Convert the subscription into a readable str. */
     public String getSubscriptionDesc() {
         return getSubscriptionName()
@@ -214,7 +222,8 @@ public class SourceConfiguration extends PulsarConfiguration {
                 && verifyInitialOffsets == that.verifyInitialOffsets
                 && Objects.equals(subscriptionName, that.subscriptionName)
                 && subscriptionType == that.subscriptionType
-                && subscriptionMode == that.subscriptionMode;
+                && subscriptionMode == that.subscriptionMode
+                && allowKeySharedOutOfOrderDelivery == that.allowKeySharedOutOfOrderDelivery;
     }
 
     @Override
@@ -230,6 +239,7 @@ public class SourceConfiguration extends PulsarConfiguration {
                 verifyInitialOffsets,
                 subscriptionName,
                 subscriptionType,
-                subscriptionMode);
+                subscriptionMode,
+                allowKeySharedOutOfOrderDelivery);
     }
 }
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumStateSerializer.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumStateSerializer.java
index af5e85c1b3b..30b374b45e4 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumStateSerializer.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumStateSerializer.java
@@ -41,7 +41,7 @@ public class PulsarSourceEnumStateSerializer
         implements SimpleVersionedSerializer<PulsarSourceEnumState> {
 
     // This version should be bumped after modifying the PulsarSourceEnumState.
-    public static final int CURRENT_VERSION = 1;
+    public static final int CURRENT_VERSION = 2;
 
     public static final PulsarSourceEnumStateSerializer INSTANCE =
             new PulsarSourceEnumStateSerializer();
@@ -71,16 +71,20 @@ public class PulsarSourceEnumStateSerializer
 
     @Override
     public PulsarSourceEnumState deserialize(int version, byte[] serialized) throws IOException {
-        // VERSION 1 deserialization, support VERSION 0 deserialization in the meantime.
+        // VERSION 2 deserialization, support VERSION 0 and 1 deserialization in the meantime.
         try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized);
                 DataInputStream in = new DataInputStream(bais)) {
-            Set<TopicPartition> partitions = deserializeSet(in, deserializePartition(version));
+            Set<TopicPartition> partitions = null;
+            if (version == 2) {
+                partitions = deserializeSet(in, deserializePartition(1));
+            } else {
+                partitions = deserializeSet(in, deserializePartition(0));
+            }
 
             // Only deserialize these fields for backward compatibility.
             if (version == 0) {
-                deserializeSet(in, deserializeSplit(version));
-                deserializeMap(
-                        in, DataInput::readInt, i -> deserializeSet(i, deserializeSplit(version)));
+                deserializeSet(in, deserializeSplit(0));
+                deserializeMap(in, DataInput::readInt, i -> deserializeSet(i, deserializeSplit(0)));
                 deserializeMap(in, DataInput::readInt, i -> deserializeSet(i, DataInput::readUTF));
                 in.readBoolean();
             }
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssigner.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/KeySharedSplitAssigner.java
similarity index 57%
copy from flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssigner.java
copy to flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/KeySharedSplitAssigner.java
index 1b7b4a6f446..9ef43476f71 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssigner.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/KeySharedSplitAssigner.java
@@ -19,7 +19,6 @@
 package org.apache.flink.connector.pulsar.source.enumerator.assigner;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.connector.source.SplitEnumeratorContext;
 import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState;
 import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor;
@@ -32,14 +31,11 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
 
-/**
- * This assigner is used for {@link SubscriptionType#Failover}, {@link SubscriptionType#Exclusive}
- * and {@link SubscriptionType#Key_Shared} subscriptions.
- */
+/** This assigner is used for {@link SubscriptionType#Key_Shared} subscription. */
 @Internal
-class NonSharedSplitAssigner extends SplitAssignerBase {
+public class KeySharedSplitAssigner extends SplitAssignerBase {
 
-    public NonSharedSplitAssigner(
+    public KeySharedSplitAssigner(
             StopCursor stopCursor,
             boolean enablePartitionDiscovery,
             SplitEnumeratorContext<PulsarPartitionSplit> context,
@@ -52,10 +48,15 @@ class NonSharedSplitAssigner extends SplitAssignerBase {
         List<TopicPartition> newPartitions = new ArrayList<>();
 
         for (TopicPartition partition : fetchedPartitions) {
+            boolean shouldAssign = false;
             if (!appendedPartitions.contains(partition)) {
                 appendedPartitions.add(partition);
                 newPartitions.add(partition);
+                shouldAssign = true;
+            }
 
+            // Reassign the incoming splits when restarting from state.
+            if (shouldAssign || !initialized) {
                 // Calculate the reader id by the current parallelism.
                 int readerId = partitionOwner(partition);
                 PulsarPartitionSplit split = new PulsarPartitionSplit(partition, stopCursor);
@@ -72,41 +73,21 @@ class NonSharedSplitAssigner extends SplitAssignerBase {
 
     @Override
     public void addSplitsBack(List<PulsarPartitionSplit> splits, int subtaskId) {
-        for (PulsarPartitionSplit split : splits) {
-            int readerId = partitionOwner(split.getPartition());
-            addSplitToPendingList(readerId, split);
+        if (splits.isEmpty()) {
+            // In case of the task failure. No splits will be put back to the enumerator.
+            for (TopicPartition partition : appendedPartitions) {
+                int readId = partitionOwner(partition);
+                if (readId == subtaskId) {
+                    PulsarPartitionSplit split = new PulsarPartitionSplit(partition, stopCursor);
+                    addSplitToPendingList(subtaskId, split);
+                }
+            }
+        } else {
+            // Manually put all the splits back to the enumerator.
+            for (PulsarPartitionSplit split : splits) {
+                int readerId = partitionOwner(split.getPartition());
+                addSplitToPendingList(readerId, split);
+            }
         }
     }
-
-    /**
-     * Returns the index of the target subtask that a specific partition should be assigned to. It's
-     * inspired by the {@code KafkaSourceEnumerator.getSplitOwner()}
-     *
-     * <p>The resulting distribution of partition has the following contract:
-     *
-     * <ul>
-     *   <li>1. Uniformly distributed across subtasks.
-     *   <li>2. Partitions are round-robin distributed (strictly clockwise w.r.t. ascending subtask
-     *       indices) by using the partition id as the offset from a starting index (i.e., the index
-     *       of the subtask which partition 0 of the topic will be assigned to, determined using the
-     *       topic name).
-     * </ul>
-     *
-     * @param partition The Pulsar partition to assign.
-     * @return The id of the reader that owns this partition.
-     */
-    private int partitionOwner(TopicPartition partition) {
-        return calculatePartitionOwner(
-                partition.getTopic(), partition.getPartitionId(), context.currentParallelism());
-    }
-
-    @VisibleForTesting
-    static int calculatePartitionOwner(String topic, int partitionId, int parallelism) {
-        int startIndex = ((topic.hashCode() * 31) & 0x7FFFFFFF) % parallelism;
-        /*
-         * Here, the assumption is that the id of Pulsar partitions are always ascending starting from
-         * 0. Therefore, can be used directly as the offset clockwise from the start index.
-         */
-        return (startIndex + partitionId) % parallelism;
-    }
 }
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssigner.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssigner.java
index 1b7b4a6f446..1cc061e2ae6 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssigner.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssigner.java
@@ -19,7 +19,6 @@
 package org.apache.flink.connector.pulsar.source.enumerator.assigner;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.connector.source.SplitEnumeratorContext;
 import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState;
 import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor;
@@ -34,7 +33,7 @@ import java.util.Set;
 
 /**
  * This assigner is used for {@link SubscriptionType#Failover}, {@link SubscriptionType#Exclusive}
- * and {@link SubscriptionType#Key_Shared} subscriptions.
+ * subscriptions.
  */
 @Internal
 class NonSharedSplitAssigner extends SplitAssignerBase {
@@ -77,36 +76,4 @@ class NonSharedSplitAssigner extends SplitAssignerBase {
             addSplitToPendingList(readerId, split);
         }
     }
-
-    /**
-     * Returns the index of the target subtask that a specific partition should be assigned to. It's
-     * inspired by the {@code KafkaSourceEnumerator.getSplitOwner()}
-     *
-     * <p>The resulting distribution of partition has the following contract:
-     *
-     * <ul>
-     *   <li>1. Uniformly distributed across subtasks.
-     *   <li>2. Partitions are round-robin distributed (strictly clockwise w.r.t. ascending subtask
-     *       indices) by using the partition id as the offset from a starting index (i.e., the index
-     *       of the subtask which partition 0 of the topic will be assigned to, determined using the
-     *       topic name).
-     * </ul>
-     *
-     * @param partition The Pulsar partition to assign.
-     * @return The id of the reader that owns this partition.
-     */
-    private int partitionOwner(TopicPartition partition) {
-        return calculatePartitionOwner(
-                partition.getTopic(), partition.getPartitionId(), context.currentParallelism());
-    }
-
-    @VisibleForTesting
-    static int calculatePartitionOwner(String topic, int partitionId, int parallelism) {
-        int startIndex = ((topic.hashCode() * 31) & 0x7FFFFFFF) % parallelism;
-        /*
-         * Here, the assumption is that the id of Pulsar partitions are always ascending starting from
-         * 0. Therefore, can be used directly as the offset clockwise from the start index.
-         */
-        return (startIndex + partitionId) % parallelism;
-    }
 }
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerBase.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerBase.java
index 32af433acbb..733072ca7e4 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerBase.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerBase.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.connector.pulsar.source.enumerator.assigner;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.connector.source.SplitEnumeratorContext;
 import org.apache.flink.api.connector.source.SplitsAssignment;
 import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState;
@@ -99,4 +100,36 @@ abstract class SplitAssignerBase implements SplitAssigner {
                 pendingPartitionSplits.computeIfAbsent(readerId, i -> new HashSet<>());
         splits.add(split);
     }
+
+    /**
+     * Returns the index of the target subtask that a specific partition should be assigned to. It's
+     * inspired by the {@code KafkaSourceEnumerator.getSplitOwner()}
+     *
+     * <p>The resulting distribution of partition has the following contract:
+     *
+     * <ul>
+     *   <li>1. Uniformly distributed across subtasks.
+     *   <li>2. Partitions are round-robin distributed (strictly clockwise w.r.t. ascending subtask
+     *       indices) by using the partition id as the offset from a starting index (i.e., the index
+     *       of the subtask which partition 0 of the topic will be assigned to, determined using the
+     *       topic name).
+     * </ul>
+     *
+     * @param partition The Pulsar partition to assign.
+     * @return The id of the reader that owns this partition.
+     */
+    protected int partitionOwner(TopicPartition partition) {
+        return calculatePartitionOwner(
+                partition.getTopic(), partition.getPartitionId(), context.currentParallelism());
+    }
+
+    @VisibleForTesting
+    static int calculatePartitionOwner(String topic, int partitionId, int parallelism) {
+        int startIndex = ((topic.hashCode() * 31) & 0x7FFFFFFF) % parallelism;
+        /*
+         * Here, the assumption is that the id of Pulsar partitions are always ascending starting from
+         * 0. Therefore, can be used directly as the offset clockwise from the start index.
+         */
+        return (startIndex + partitionId) % parallelism;
+    }
 }
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerFactory.java
index 02f8934a6bf..4ade1e561a7 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerFactory.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerFactory.java
@@ -46,12 +46,14 @@ public final class SplitAssignerFactory {
         switch (subscriptionType) {
             case Failover:
             case Exclusive:
-            case Key_Shared:
                 return new NonSharedSplitAssigner(
                         stopCursor, enablePartitionDiscovery, context, enumState);
             case Shared:
                 return new SharedSplitAssigner(
                         stopCursor, enablePartitionDiscovery, context, enumState);
+            case Key_Shared:
+                return new KeySharedSplitAssigner(
+                        stopCursor, enablePartitionDiscovery, context, enumState);
             default:
                 throw new IllegalArgumentException(
                         "We don't support this subscription type: " + subscriptionType);
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java
index d29f7a0d548..0e11996f9a5 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java
@@ -23,6 +23,7 @@ import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange;
+import org.apache.flink.connector.pulsar.source.enumerator.topic.range.RangeGenerator.KeySharedMode;
 
 import org.apache.pulsar.client.admin.PulsarAdmin;
 import org.apache.pulsar.client.admin.PulsarAdminException;
@@ -31,7 +32,7 @@ import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
 import java.util.ArrayList;
 import java.util.List;
 
-import static java.util.stream.Collectors.toList;
+import static java.util.Collections.singletonList;
 
 /** PulsarSubscriber abstract class to simplify Pulsar admin related operations. */
 public abstract class BasePulsarSubscriber implements PulsarSubscriber {
@@ -49,29 +50,42 @@ public abstract class BasePulsarSubscriber implements PulsarSubscriber {
                 // Return null for skipping the topic metadata query.
                 return null;
             } else {
-                // This method would cause the failure for subscriber.
+                // This method would cause failure for subscribers.
                 throw new IllegalStateException(e);
             }
         }
     }
 
     protected List<TopicPartition> toTopicPartitions(
-            TopicMetadata metadata, List<TopicRange> ranges) {
+            TopicMetadata metadata, List<TopicRange> ranges, KeySharedMode mode) {
         if (!metadata.isPartitioned()) {
             // For non-partitioned topic.
-            return ranges.stream()
-                    .map(range -> new TopicPartition(metadata.getName(), -1, range))
-                    .collect(toList());
+            return toTopicPartitions(metadata.getName(), -1, ranges, mode);
         } else {
+            // For partitioned topic.
             List<TopicPartition> partitions = new ArrayList<>();
             for (int i = 0; i < metadata.getPartitionSize(); i++) {
+                partitions.addAll(toTopicPartitions(metadata.getName(), i, ranges, mode));
+            }
+            return partitions;
+        }
+    }
+
+    protected List<TopicPartition> toTopicPartitions(
+            String topic, int partitionId, List<TopicRange> ranges, KeySharedMode mode) {
+        switch (mode) {
+            case JOIN:
+                return singletonList(new TopicPartition(topic, partitionId, ranges, mode));
+            case SPLIT:
+                List<TopicPartition> partitions = new ArrayList<>(ranges.size());
                 for (TopicRange range : ranges) {
-                    TopicPartition partition = new TopicPartition(metadata.getName(), i, range);
+                    TopicPartition partition =
+                            new TopicPartition(topic, partitionId, singletonList(range), mode);
                     partitions.add(partition);
                 }
-            }
-
-            return partitions;
+                return partitions;
+            default:
+                throw new UnsupportedOperationException(mode + " isn't supported.");
         }
     }
 }
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/TopicListSubscriber.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/TopicListSubscriber.java
index 28f525b2d07..127adddc845 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/TopicListSubscriber.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/TopicListSubscriber.java
@@ -22,6 +22,7 @@ import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.range.RangeGenerator;
+import org.apache.flink.connector.pulsar.source.enumerator.topic.range.RangeGenerator.KeySharedMode;
 
 import org.apache.pulsar.client.admin.PulsarAdmin;
 import org.apache.pulsar.common.naming.TopicName;
@@ -62,9 +63,9 @@ public class TopicListSubscriber extends BasePulsarSubscriber {
         for (String topic : fullTopicNames) {
             TopicMetadata metadata = queryTopicMetadata(pulsarAdmin, topic);
             List<TopicRange> ranges = rangeGenerator.range(metadata, parallelism);
-            List<TopicPartition> list = toTopicPartitions(metadata, ranges);
+            KeySharedMode mode = rangeGenerator.keyShareMode(metadata, parallelism);
 
-            results.addAll(list);
+            results.addAll(toTopicPartitions(metadata, ranges, mode));
         }
 
         for (String partition : partitions) {
@@ -74,10 +75,9 @@ public class TopicListSubscriber extends BasePulsarSubscriber {
 
             TopicMetadata metadata = queryTopicMetadata(pulsarAdmin, name);
             List<TopicRange> ranges = rangeGenerator.range(metadata, parallelism);
+            KeySharedMode mode = rangeGenerator.keyShareMode(metadata, parallelism);
 
-            for (TopicRange range : ranges) {
-                results.add(new TopicPartition(name, index, range));
-            }
+            results.addAll(toTopicPartitions(name, index, ranges, mode));
         }
 
         return results;
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/TopicPatternSubscriber.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/TopicPatternSubscriber.java
index fae2bac7b3e..6173fcaba71 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/TopicPatternSubscriber.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/TopicPatternSubscriber.java
@@ -22,6 +22,7 @@ import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.range.RangeGenerator;
+import org.apache.flink.connector.pulsar.source.enumerator.topic.range.RangeGenerator.KeySharedMode;
 
 import org.apache.pulsar.client.admin.PulsarAdmin;
 import org.apache.pulsar.client.admin.PulsarAdminException;
@@ -74,7 +75,9 @@ public class TopicPatternSubscriber extends BasePulsarSubscriber {
                             metadata -> {
                                 List<TopicRange> ranges =
                                         rangeGenerator.range(metadata, parallelism);
-                                return toTopicPartitions(metadata, ranges).stream();
+                                KeySharedMode mode =
+                                        rangeGenerator.keyShareMode(metadata, parallelism);
+                                return toTopicPartitions(metadata, ranges, mode).stream();
                             })
                     .collect(toSet());
         } catch (PulsarAdminException e) {
@@ -82,7 +85,7 @@ public class TopicPatternSubscriber extends BasePulsarSubscriber {
                 // Skip the topic metadata query.
                 return Collections.emptySet();
             } else {
-                // This method would cause the failure for subscriber.
+                // This method would cause failure for subscribers.
                 throw new IllegalStateException(e);
             }
         }
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicPartition.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicPartition.java
index b3035cde848..b25851027a9 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicPartition.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicPartition.java
@@ -18,17 +18,25 @@
 
 package org.apache.flink.connector.pulsar.source.enumerator.topic;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.pulsar.source.enumerator.topic.range.RangeGenerator.KeySharedMode;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList;
 
 import org.apache.pulsar.client.api.Range;
 import org.apache.pulsar.client.api.SubscriptionType;
 
 import java.io.Serializable;
+import java.util.List;
 import java.util.Objects;
 
+import static java.util.stream.Collectors.toList;
 import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicName;
 import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition;
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange;
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.range.RangeGenerator.KeySharedMode.SPLIT;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
@@ -39,15 +47,17 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
 public class TopicPartition implements Serializable {
     private static final long serialVersionUID = -1474354741550810953L;
 
+    private static final List<TopicRange> FULL_RANGES = ImmutableList.of(createFullRange());
+
     /**
-     * The topic name of the pulsar. It would be a full topic name, if your don't provide the tenant
+     * The topic name of the pulsar. It would be a full topic name. If you don't provide the tenant
      * and namespace, we would add them automatically.
      */
     private final String topic;
 
     /**
-     * Index of partition for the topic. It would be natural number for partitioned topic with a
-     * non-key_shared subscription.
+     * Index of partition for the topic. It would be a natural number for the partitioned topic with
+     * a non-key_shared subscription.
      */
     private final int partitionId;
 
@@ -56,12 +66,28 @@ public class TopicPartition implements Serializable {
      * TopicRange#createFullRange()} full range for all the subscription type except {@link
      * SubscriptionType#Key_Shared}.
      */
-    private final TopicRange range;
+    private final List<TopicRange> ranges;
+
+    /**
+     * The key share mode for the {@link SubscriptionType#Key_Shared}. It will be {@link
+     * KeySharedMode#JOIN} for other subscriptions.
+     */
+    private final KeySharedMode mode;
+
+    public TopicPartition(String topic, int partitionId) {
+        this(topic, partitionId, FULL_RANGES, SPLIT);
+    }
+
+    public TopicPartition(String topic, int partitionId, List<TopicRange> ranges) {
+        this(topic, partitionId, ranges, SPLIT);
+    }
 
-    public TopicPartition(String topic, int partitionId, TopicRange range) {
+    public TopicPartition(
+            String topic, int partitionId, List<TopicRange> ranges, KeySharedMode mode) {
         this.topic = topicName(checkNotNull(topic));
         this.partitionId = partitionId;
-        this.range = checkNotNull(range);
+        this.ranges = checkNotNull(ranges);
+        this.mode = mode;
     }
 
     public String getTopic() {
@@ -84,12 +110,22 @@ public class TopicPartition implements Serializable {
         }
     }
 
-    public TopicRange getRange() {
-        return range;
+    /** This method is internal used for serialization. */
+    @Internal
+    public List<TopicRange> getRanges() {
+        return ranges;
     }
 
-    public Range getPulsarRange() {
-        return range.toPulsarRange();
+    /** This method is internal used for define key shared subscription. */
+    @Internal
+    public List<Range> getPulsarRanges() {
+        return ranges.stream().map(TopicRange::toPulsarRange).collect(toList());
+    }
+
+    /** This method is internal used for key shared mode. */
+    @Internal
+    public KeySharedMode getMode() {
+        return mode;
     }
 
     @Override
@@ -100,19 +136,21 @@ public class TopicPartition implements Serializable {
         if (o == null || getClass() != o.getClass()) {
             return false;
         }
-        TopicPartition that = (TopicPartition) o;
-        return partitionId == that.partitionId
-                && topic.equals(that.topic)
-                && range.equals(that.range);
+        TopicPartition partition = (TopicPartition) o;
+
+        return partitionId == partition.partitionId
+                && topic.equals(partition.topic)
+                && ranges.equals(partition.ranges)
+                && mode == partition.mode;
     }
 
     @Override
     public int hashCode() {
-        return Objects.hash(topic, partitionId, range);
+        return Objects.hash(topic, partitionId, ranges, mode);
     }
 
     @Override
     public String toString() {
-        return getFullTopicName() + "|" + range;
+        return getFullTopicName() + "|" + ranges;
     }
 }
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/FixedKeysRangeGenerator.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/FixedKeysRangeGenerator.java
new file mode 100644
index 00000000000..e21147d54b8
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/FixedKeysRangeGenerator.java
@@ -0,0 +1,185 @@
+/*
+ * 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.source.enumerator.topic.range;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessageBuilder;
+import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata;
+import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange;
+
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.TypedMessageBuilder;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.range.TopicRangeUtils.NONE_KEY;
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.range.TopicRangeUtils.keyBytesHash;
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.range.TopicRangeUtils.keyHash;
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.range.TopicRangeUtils.validateTopicRanges;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Pulsar didn't expose the key hash range method. We have to provide an implementation for
+ * end-user. You can add the keys you want to consume, no need to provide any hash ranges.
+ *
+ * <p>Since the key's hash isn't specified to only one key. The consuming results may contain the
+ * messages with different keys comparing the keys you have defined in this range generator.
+ * Remember to use flink's <code>DataStream.filter()</code> method.
+ *
+ * <p>Usage: <code><pre>
+ * FixedKeysRangeGenerator.builder()
+ *     .supportNullKey()
+ *     .key("someKey")
+ *     .keys(Arrays.asList("key1", "key2"))
+ *     .build()
+ * </pre></code>
+ */
+@PublicEvolving
+public class FixedKeysRangeGenerator implements RangeGenerator {
+    private static final long serialVersionUID = 2372969466289052100L;
+
+    private final List<TopicRange> ranges;
+    private final KeySharedMode sharedMode;
+
+    private FixedKeysRangeGenerator(List<TopicRange> ranges, KeySharedMode sharedMode) {
+        this.ranges = ranges;
+        this.sharedMode = sharedMode;
+    }
+
+    @Override
+    public List<TopicRange> range(TopicMetadata metadata, int parallelism) {
+        return ranges;
+    }
+
+    @Override
+    public KeySharedMode keyShareMode(TopicMetadata metadata, int parallelism) {
+        return sharedMode;
+    }
+
+    public static FixedKeysRangeGeneratorBuilder builder() {
+        return new FixedKeysRangeGeneratorBuilder();
+    }
+
+    /** The builder for {@link FixedKeysRangeGenerator}. */
+    @PublicEvolving
+    public static class FixedKeysRangeGeneratorBuilder {
+
+        private final SortedSet<Integer> keyHashes = new TreeSet<>();
+        private KeySharedMode sharedMode = KeySharedMode.JOIN;
+
+        private FixedKeysRangeGeneratorBuilder() {
+            // No public for builder
+        }
+
+        /**
+         * Some {@link Message} in Pulsar may not have {@link Message#getOrderingKey()} or {@link
+         * Message#getKey()}, use this method for supporting consuming such messages.
+         */
+        public FixedKeysRangeGeneratorBuilder supportNullKey() {
+            keyHashes.add(keyHash(NONE_KEY));
+            return this;
+        }
+
+        /**
+         * If you set the message key by using {@link PulsarMessageBuilder#key(String)} or {@link
+         * TypedMessageBuilder#key(String)}, use this method for supporting consuming such messages.
+         */
+        public FixedKeysRangeGeneratorBuilder key(String key) {
+            keyHashes.add(keyHash(key));
+            return this;
+        }
+
+        /** Same as the {@link #key(String)}, support setting multiple keys in the same time. */
+        public FixedKeysRangeGeneratorBuilder keys(Collection<String> someKeys) {
+            checkNotNull(someKeys);
+            for (String someKey : someKeys) {
+                keyHashes.add(keyHash(someKey));
+            }
+            return this;
+        }
+
+        /**
+         * If you set the message key by using {@link TypedMessageBuilder#keyBytes(byte[])}, use
+         * this method for supporting consuming such messages.
+         */
+        public FixedKeysRangeGeneratorBuilder keyBytes(byte[] keyBytes) {
+            keyHashes.add(keyBytesHash(keyBytes));
+            return this;
+        }
+
+        /**
+         * Pulsar's ordering key is prior to the message key. If you set the ordering key by using
+         * {@link PulsarMessageBuilder#orderingKey(byte[])} or {@link
+         * TypedMessageBuilder#orderingKey(byte[])}, use this method for supporting consuming such
+         * messages.
+         */
+        public FixedKeysRangeGeneratorBuilder orderingKey(byte[] keyBytes) {
+            keyHashes.add(keyHash(keyBytes));
+            return this;
+        }
+
+        /** Override the default {@link KeySharedMode#JOIN} to the mode your have provided. */
+        public FixedKeysRangeGeneratorBuilder keySharedMode(KeySharedMode sharedMode) {
+            this.sharedMode = sharedMode;
+            return this;
+        }
+
+        /** Create the FixedKeysRangeGenerator by the given keys. */
+        public FixedKeysRangeGenerator build() {
+            List<TopicRange> ranges = new ArrayList<>();
+            // Calculate the topic ranges.
+            Integer start = null;
+            Integer next = null;
+            for (Integer hash : keyHashes) {
+                // Start
+                if (start == null) {
+                    start = hash;
+                    next = hash;
+                    continue;
+                }
+
+                // Continue range.
+                if (hash - next == 1) {
+                    next = hash;
+                    continue;
+                }
+
+                // Found one continues topic range.
+                TopicRange range = new TopicRange(start, next);
+                ranges.add(range);
+
+                start = hash;
+                next = hash;
+            }
+
+            // Support the last range.
+            if (start != null) {
+                TopicRange range = new TopicRange(start, next);
+                ranges.add(range);
+            }
+
+            validateTopicRanges(ranges, sharedMode);
+            return new FixedKeysRangeGenerator(ranges, sharedMode);
+        }
+    }
+}
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/FixedRangeGenerator.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/FixedRangeGenerator.java
index 6f82725d8d5..84c41e96e01 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/FixedRangeGenerator.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/FixedRangeGenerator.java
@@ -18,23 +18,40 @@
 
 package org.apache.flink.connector.pulsar.source.enumerator.topic.range;
 
+import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange;
 
 import java.util.List;
 
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.range.TopicRangeUtils.validateTopicRanges;
+
 /** Always return the same range set for all topics. */
+@PublicEvolving
 public class FixedRangeGenerator implements RangeGenerator {
     private static final long serialVersionUID = -3895203007855538734L;
 
     private final List<TopicRange> ranges;
+    private final KeySharedMode sharedMode;
 
     public FixedRangeGenerator(List<TopicRange> ranges) {
+        this(ranges, KeySharedMode.JOIN);
+    }
+
+    public FixedRangeGenerator(List<TopicRange> ranges, KeySharedMode sharedMode) {
+        validateTopicRanges(ranges, sharedMode);
+
         this.ranges = ranges;
+        this.sharedMode = sharedMode;
     }
 
     @Override
     public List<TopicRange> range(TopicMetadata metadata, int parallelism) {
         return ranges;
     }
+
+    @Override
+    public KeySharedMode keyShareMode(TopicMetadata metadata, int parallelism) {
+        return sharedMode;
+    }
 }
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/FullRangeGenerator.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/FullRangeGenerator.java
index 31d5c73d9f6..1ac69e00af9 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/FullRangeGenerator.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/FullRangeGenerator.java
@@ -18,23 +18,31 @@
 
 package org.apache.flink.connector.pulsar.source.enumerator.topic.range;
 
+import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange;
 
 import org.apache.pulsar.client.api.SubscriptionType;
 
-import java.util.Collections;
 import java.util.List;
 
+import static java.util.Collections.singletonList;
+
 /**
  * Default implementation for {@link SubscriptionType#Shared}, {@link SubscriptionType#Failover} and
  * {@link SubscriptionType#Exclusive} subscription.
  */
+@PublicEvolving
 public class FullRangeGenerator implements RangeGenerator {
     private static final long serialVersionUID = -4571731955155036216L;
 
     @Override
     public List<TopicRange> range(TopicMetadata metadata, int parallelism) {
-        return Collections.singletonList(TopicRange.createFullRange());
+        return singletonList(TopicRange.createFullRange());
+    }
+
+    @Override
+    public KeySharedMode keyShareMode(TopicMetadata metadata, int parallelism) {
+        return KeySharedMode.SPLIT;
     }
 }
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/RangeGenerator.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/RangeGenerator.java
index 825ebdb1d6d..8bfdcbafb8c 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/RangeGenerator.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/RangeGenerator.java
@@ -51,7 +51,18 @@ public interface RangeGenerator extends Serializable {
      */
     List<TopicRange> range(TopicMetadata metadata, int parallelism);
 
-    /** Initialize some extra resource when bootstrap the source. */
+    /**
+     * Defines the default behavior for Key_Shared subscription in Flink. See {@link KeySharedMode}
+     * for the detailed usage of the key share mode.
+     *
+     * @param metadata The metadata of the topic.
+     * @param parallelism The reader size for this topic.
+     */
+    default KeySharedMode keyShareMode(TopicMetadata metadata, int parallelism) {
+        return KeySharedMode.SPLIT;
+    }
+
+    /** Initialize some extra resources when bootstrap the source. */
     default void open(SourceConfiguration sourceConfiguration) {
         // This method is used for user implementation.
         open(sourceConfiguration, sourceConfiguration);
@@ -62,4 +73,32 @@ public interface RangeGenerator extends Serializable {
     default void open(Configuration configuration, SourceConfiguration sourceConfiguration) {
         // This method is used for user implementation.
     }
+
+    /**
+     * Different Key_Shared mode means different split assignment behaviors. If you only consume a
+     * subset of Pulsar's key hash range, remember to use the {@link KeySharedMode#JOIN} mode which
+     * will subscribe all the range in only one reader. Otherwise, when the ranges can join into a
+     * full Pulsar key hash range (0 ~ 65535) you should use {@link KeySharedMode#SPLIT} for sharing
+     * the splits among all the backend readers.
+     *
+     * <p>In the {@link KeySharedMode#SPLIT} mode. The topic will be subscribed by multiple readers.
+     * But Pulsar has one limit in this situation. That is if a Message can't find the corresponding
+     * reader by the key hash range. No messages will be delivered to the current readers, until
+     * there is a reader which can subscribe to such messages.
+     */
+    @PublicEvolving
+    enum KeySharedMode {
+
+        /**
+         * The topic ranges that the {@link RangeGenerator} generated will be split among the
+         * readers.
+         */
+        SPLIT,
+
+        /**
+         * Assign all the topic ranges to only one reader instance. This is used for partial key
+         * hash range subscription.
+         */
+        JOIN
+    }
 }
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/UniformRangeGenerator.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/SplitRangeGenerator.java
similarity index 55%
rename from flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/UniformRangeGenerator.java
rename to flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/SplitRangeGenerator.java
index 814cf9eee1f..17653269bac 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/UniformRangeGenerator.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/SplitRangeGenerator.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.connector.pulsar.source.enumerator.topic.range;
 
+import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange;
 
@@ -27,27 +28,55 @@ import java.util.ArrayList;
 import java.util.List;
 
 import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.MAX_RANGE;
-import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.RANGE_SIZE;
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.MIN_RANGE;
+import static org.apache.flink.util.Preconditions.checkArgument;
 
 /**
  * This range generator would divide the range by the flink source parallelism. It would be the
  * default implementation for {@link SubscriptionType#Key_Shared} subscription.
  */
-public class UniformRangeGenerator implements RangeGenerator {
-    private static final long serialVersionUID = -7292650922683609268L;
+@PublicEvolving
+public class SplitRangeGenerator implements RangeGenerator {
+    private static final long serialVersionUID = -8682286436352905249L;
+
+    private final int start;
+    private final int end;
+
+    public SplitRangeGenerator() {
+        this(MIN_RANGE, MAX_RANGE);
+    }
+
+    public SplitRangeGenerator(int start, int end) {
+        checkArgument(
+                start >= MIN_RANGE,
+                "Start range should be equal to or great than the min range " + MIN_RANGE);
+        checkArgument(
+                end <= MAX_RANGE, "End range should below or less than the max range " + MAX_RANGE);
+        checkArgument(start <= end, "Start range should be equal to or less than the end range");
+
+        this.start = start;
+        this.end = end;
+    }
 
     @Override
     public List<TopicRange> range(TopicMetadata metadata, int parallelism) {
-        List<TopicRange> results = new ArrayList<>(parallelism);
+        final int range = end - start + 1;
+        final int size = Math.min(range, parallelism);
+        int startRange = start;
 
-        int startRange = 0;
-        for (int i = 1; i < parallelism; i++) {
-            int nextStartRange = i * RANGE_SIZE / parallelism;
+        List<TopicRange> results = new ArrayList<>(size);
+        for (int i = 1; i < size; i++) {
+            int nextStartRange = i * range / size + start;
             results.add(new TopicRange(startRange, nextStartRange - 1));
             startRange = nextStartRange;
         }
-        results.add(new TopicRange(startRange, MAX_RANGE));
+        results.add(new TopicRange(startRange, end));
 
         return results;
     }
+
+    @Override
+    public KeySharedMode keyShareMode(TopicMetadata metadata, int parallelism) {
+        return KeySharedMode.SPLIT;
+    }
 }
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/TopicRangeUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/TopicRangeUtils.java
new file mode 100644
index 00000000000..8728bdeee3f
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/TopicRangeUtils.java
@@ -0,0 +1,136 @@
+/*
+ * 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.source.enumerator.topic.range;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessageBuilder;
+import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange;
+import org.apache.flink.connector.pulsar.source.enumerator.topic.range.RangeGenerator.KeySharedMode;
+
+import org.apache.pulsar.client.api.KeySharedPolicy;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Range;
+import org.apache.pulsar.client.api.TypedMessageBuilder;
+import org.apache.pulsar.common.util.Murmur3_32Hash;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Base64;
+import java.util.List;
+
+import static java.util.Comparator.comparingLong;
+import static java.util.stream.Collectors.toList;
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.MAX_RANGE;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This is the helper class for Pulsar's {@link Range}, used in Key Shared mode. It will be exposed
+ * to the end users for simplifying the implementation of the {@link RangeGenerator}.
+ */
+@PublicEvolving
+public final class TopicRangeUtils {
+    private static final Logger LOG = LoggerFactory.getLogger(TopicRangeUtils.class);
+
+    /** Pulsar would use this as default key if no key was provided. */
+    public static final String NONE_KEY = "NONE_KEY";
+
+    private TopicRangeUtils() {
+        // No public constructor.
+    }
+
+    /** Make sure all the ranges should be valid in Pulsar Key Shared Policy. */
+    public static void validateTopicRanges(List<TopicRange> ranges, KeySharedMode sharedMode) {
+        List<Range> pulsarRanges = ranges.stream().map(TopicRange::toPulsarRange).collect(toList());
+        KeySharedPolicy.stickyHashRange().ranges(pulsarRanges).validate();
+
+        if (!isFullTopicRanges(ranges) && KeySharedMode.SPLIT == sharedMode) {
+            LOG.warn(
+                    "You have provided a partial key hash range with KeySharedMode.SPLIT. "
+                            + "You can't consume any message if there are any messages with keys that are out of the given ranges.");
+        }
+    }
+
+    /** Check if the given topic ranges are full Pulsar range. */
+    public static boolean isFullTopicRanges(List<TopicRange> ranges) {
+        List<TopicRange> sorted =
+                ranges.stream().sorted(comparingLong(TopicRange::getStart)).collect(toList());
+        int start = 0;
+        for (TopicRange range : sorted) {
+            if (start == 0) {
+                if (range.getStart() == 0) {
+                    start = range.getEnd();
+                    continue;
+                } else {
+                    return false;
+                }
+            }
+
+            if (range.getStart() - start != 1) {
+                return false;
+            }
+            start = range.getEnd();
+        }
+
+        return start == MAX_RANGE;
+    }
+
+    /**
+     * Pulsar didn't expose the key hash range method. We have to manually define it here.
+     *
+     * @param key The key of Pulsar's {@link Message}. Pulsar would try to use {@link
+     *     Message#getOrderingKey()} first. If it doesn't exist Pulsar will use {@link
+     *     Message#getKey()} instead. Remember that the {@link Message#getOrderingKey()} could be
+     *     configured by {@link PulsarMessageBuilder#orderingKey(byte[])} and the {@link
+     *     Message#getKey()} could be configured by {@link PulsarMessageBuilder#key(String)}.
+     */
+    public static int keyHash(String key) {
+        checkNotNull(key);
+        return keyHash(key.getBytes(StandardCharsets.UTF_8));
+    }
+
+    /**
+     * This method is a bit of different compared to the {@link #keyHash(byte[])}. We only define
+     * this method when you set the message key by using {@link
+     * TypedMessageBuilder#keyBytes(byte[])}. Because the Pulsar would calculate the message key
+     * hash in a different way.
+     *
+     * <p>It should be <strong>a bug on Pulsar</strong>, but we can't fix it for backward
+     * compatibility.
+     */
+    public static int keyBytesHash(byte[] keyBytes) {
+        String encodedKey = Base64.getEncoder().encodeToString(checkNotNull(keyBytes));
+        byte[] encodedKeyBytes = encodedKey.getBytes(StandardCharsets.UTF_8);
+        return keyHash(encodedKeyBytes);
+    }
+
+    /**
+     * Pulsar didn't expose the key hash range method. We have to manually define it here.
+     *
+     * @param keyBytes The key bytes of Pulsar's {@link Message}. Pulsar would try to use {@link
+     *     Message#getOrderingKey()} first. If it doesn't exist Pulsar will use {@link
+     *     Message#getKey()} instead. Remember that the {@link Message#getOrderingKey()} could be
+     *     configured by {@link PulsarMessageBuilder#orderingKey(byte[])} and the {@link
+     *     Message#getKey()} could be configured by {@link PulsarMessageBuilder#key(String)}.
+     */
+    public static int keyHash(byte[] keyBytes) {
+        int stickyKeyHash = Murmur3_32Hash.getInstance().makeHash(checkNotNull(keyBytes));
+        return stickyKeyHash % TopicRange.RANGE_SIZE;
+    }
+}
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java
index d9ecb32bd56..95379697e0e 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java
@@ -56,6 +56,8 @@ import java.util.concurrent.TimeoutException;
 
 import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient;
 import static org.apache.flink.connector.pulsar.source.config.PulsarSourceConfigUtils.createConsumerBuilder;
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.range.RangeGenerator.KeySharedMode.JOIN;
+import static org.apache.pulsar.client.api.KeySharedPolicy.stickyHashRange;
 
 /**
  * The common partition split reader.
@@ -219,9 +221,17 @@ abstract class PulsarPartitionSplitReaderBase<OUT>
 
         // Add KeySharedPolicy for Key_Shared subscription.
         if (sourceConfiguration.getSubscriptionType() == SubscriptionType.Key_Shared) {
-            KeySharedPolicy policy =
-                    KeySharedPolicy.stickyHashRange().ranges(partition.getPulsarRange());
+            KeySharedPolicy policy = stickyHashRange().ranges(partition.getPulsarRanges());
+            // We may enable out of order delivery for speeding up. It was turned off by default.
+            policy.setAllowOutOfOrderDelivery(
+                    sourceConfiguration.isAllowKeySharedOutOfOrderDelivery());
             consumerBuilder.keySharedPolicy(policy);
+
+            if (partition.getMode() == JOIN) {
+                // Override the key shared subscription into exclusive for making it behaviors like
+                // a Pulsar Reader which supports partial key hash ranges.
+                consumerBuilder.subscriptionType(SubscriptionType.Exclusive);
+            }
         }
 
         // Create the consumer configuration by using common utils.
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplitSerializer.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplitSerializer.java
index 15f126f66f7..fb88d3d910d 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplitSerializer.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplitSerializer.java
@@ -21,6 +21,7 @@ package org.apache.flink.connector.pulsar.source.split;
 import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange;
+import org.apache.flink.connector.pulsar.source.enumerator.topic.range.RangeGenerator.KeySharedMode;
 import org.apache.flink.core.io.SimpleVersionedSerializer;
 
 import org.apache.pulsar.client.api.MessageId;
@@ -31,10 +32,14 @@ import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.util.List;
 
+import static java.util.Collections.singletonList;
 import static org.apache.flink.connector.pulsar.common.utils.PulsarSerdeUtils.deserializeBytes;
+import static org.apache.flink.connector.pulsar.common.utils.PulsarSerdeUtils.deserializeList;
 import static org.apache.flink.connector.pulsar.common.utils.PulsarSerdeUtils.deserializeObject;
 import static org.apache.flink.connector.pulsar.common.utils.PulsarSerdeUtils.serializeBytes;
+import static org.apache.flink.connector.pulsar.common.utils.PulsarSerdeUtils.serializeList;
 import static org.apache.flink.connector.pulsar.common.utils.PulsarSerdeUtils.serializeObject;
 
 /** The {@link SimpleVersionedSerializer serializer} for {@link PulsarPartitionSplit}. */
@@ -45,7 +50,7 @@ public class PulsarPartitionSplitSerializer
             new PulsarPartitionSplitSerializer();
 
     // This version should be bumped after modifying the PulsarPartitionSplit.
-    public static final int CURRENT_VERSION = 0;
+    public static final int CURRENT_VERSION = 1;
 
     private PulsarPartitionSplitSerializer() {
         // Singleton instance.
@@ -136,23 +141,45 @@ public class PulsarPartitionSplitSerializer
 
     public void serializeTopicPartition(DataOutputStream out, TopicPartition partition)
             throws IOException {
-        // VERSION 0 serialization
-        TopicRange range = partition.getRange();
+        // VERSION 1 serialization
         out.writeUTF(partition.getTopic());
         out.writeInt(partition.getPartitionId());
-        out.writeInt(range.getStart());
-        out.writeInt(range.getEnd());
+        serializeList(
+                out,
+                partition.getRanges(),
+                (o, r) -> {
+                    o.writeInt(r.getStart());
+                    o.writeInt(r.getEnd());
+                });
+        out.writeInt(partition.getMode().ordinal());
     }
 
     public TopicPartition deserializeTopicPartition(int version, DataInputStream in)
             throws IOException {
-        // VERSION 0 deserialization
         String topic = in.readUTF();
         int partitionId = in.readInt();
-        int start = in.readInt();
-        int end = in.readInt();
+        List<TopicRange> ranges;
+        KeySharedMode keySharedMode;
+        if (version == 0) {
+            // VERSION 0 deserialization
+            int start = in.readInt();
+            int end = in.readInt();
+            TopicRange range = new TopicRange(start, end);
+            ranges = singletonList(range);
+            keySharedMode = KeySharedMode.SPLIT;
+        } else {
+            // VERSION 1 deserialization
+            ranges =
+                    deserializeList(
+                            in,
+                            i -> {
+                                int start = i.readInt();
+                                int end = i.readInt();
+                                return new TopicRange(start, end);
+                            });
+            keySharedMode = KeySharedMode.values()[in.readInt()];
+        }
 
-        TopicRange range = new TopicRange(start, end);
-        return new TopicPartition(topic, partitionId, range);
+        return new TopicPartition(topic, partitionId, ranges, keySharedMode);
     }
 }
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilderTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilderTest.java
index 5825e4e7020..644523a33bc 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilderTest.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilderTest.java
@@ -19,7 +19,7 @@
 package org.apache.flink.connector.pulsar.source;
 
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.pulsar.source.enumerator.topic.range.UniformRangeGenerator;
+import org.apache.flink.connector.pulsar.source.enumerator.topic.range.SplitRangeGenerator;
 
 import org.apache.pulsar.client.api.Schema;
 import org.apache.pulsar.client.api.SubscriptionType;
@@ -73,7 +73,7 @@ class PulsarSourceBuilderTest {
     void rangeGeneratorRequiresKeyShared() {
         PulsarSourceBuilder<String> builder = new PulsarSourceBuilder<>();
         builder.setSubscriptionType(SubscriptionType.Shared);
-        UniformRangeGenerator rangeGenerator = new UniformRangeGenerator();
+        SplitRangeGenerator rangeGenerator = new SplitRangeGenerator();
 
         assertThatThrownBy(() -> builder.setRangeGenerator(rangeGenerator))
                 .isInstanceOf(IllegalArgumentException.class);
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumStateSerializerTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumStateSerializerTest.java
index 54687a3277e..a584d67d169 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumStateSerializerTest.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumStateSerializerTest.java
@@ -18,16 +18,23 @@
 
 package org.apache.flink.connector.pulsar.source.enumerator;
 
+import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange;
+import org.apache.flink.core.memory.DataOutputSerializer;
+import org.apache.flink.util.InstantiationUtil;
 
 import org.apache.flink.shaded.guava30.com.google.common.collect.Sets;
 
+import org.apache.pulsar.client.api.MessageId;
 import org.junit.jupiter.api.Test;
 
+import java.io.IOException;
 import java.util.Set;
 
+import static java.util.Collections.singletonList;
 import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic;
+import static org.apache.commons.lang3.RandomStringUtils.randomNumeric;
 import static org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumStateSerializer.INSTANCE;
 import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange;
 import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -37,19 +44,97 @@ import static org.junit.jupiter.api.Assertions.assertNotSame;
 class PulsarSourceEnumStateSerializerTest {
 
     @Test
-    void serializeAndDeserializePulsarSourceEnumState() throws Exception {
+    void version2SerializeAndDeserialize() throws Exception {
         Set<TopicPartition> partitions =
                 Sets.newHashSet(
-                        new TopicPartition(randomAlphabetic(10), 2, new TopicRange(1, 30)),
-                        new TopicPartition(randomAlphabetic(10), 1, createFullRange()));
+                        new TopicPartition(
+                                randomAlphabetic(10), 2, singletonList(new TopicRange(1, 30))),
+                        new TopicPartition(
+                                randomAlphabetic(10), 1, singletonList(createFullRange())));
 
         PulsarSourceEnumState state = new PulsarSourceEnumState(partitions);
 
         byte[] bytes = INSTANCE.serialize(state);
-        PulsarSourceEnumState state1 = INSTANCE.deserialize(INSTANCE.getVersion(), bytes);
+        PulsarSourceEnumState state1 = INSTANCE.deserialize(2, bytes);
 
         assertEquals(state.getAppendedPartitions(), state1.getAppendedPartitions());
-
         assertNotSame(state, state1);
     }
+
+    @Test
+    void version1Deserialize() throws Exception {
+        // Serialize in version 1 logic.
+        DataOutputSerializer serializer = new DataOutputSerializer(4096);
+        serializer.writeInt(2);
+        serializer.writeUTF("topic1");
+        serializer.writeInt(0);
+        serializer.writeInt(300);
+        serializer.writeInt(4000);
+        serializer.writeUTF("topic2");
+        serializer.writeInt(4);
+        serializer.writeInt(600);
+        serializer.writeInt(8000);
+        byte[] bytes = serializer.getSharedBuffer();
+
+        PulsarSourceEnumState state = INSTANCE.deserialize(1, bytes);
+        Set<TopicPartition> partitions = state.getAppendedPartitions();
+        Set<TopicPartition> expectedPartitions =
+                Sets.newHashSet(
+                        new TopicPartition("topic1", 0, singletonList(new TopicRange(300, 4000))),
+                        new TopicPartition("topic2", 4, singletonList(new TopicRange(600, 8000))));
+
+        assertEquals(partitions, expectedPartitions);
+    }
+
+    @Test
+    void version0Deserialize() throws Exception {
+        DataOutputSerializer serializer = new DataOutputSerializer(4096);
+        // Serialize in version 0 logic.
+        serializer.writeInt(2);
+        serializer.writeUTF("topic3");
+        serializer.writeInt(5);
+        serializer.writeInt(600);
+        serializer.writeInt(2000);
+        serializer.writeUTF("topic4");
+        serializer.writeInt(8);
+        serializer.writeInt(300);
+        serializer.writeInt(1000);
+        serializeVersion0SplitSet(serializer);
+        serializer.writeInt(1);
+        serializer.writeInt(3);
+        serializeVersion0SplitSet(serializer);
+        serializer.writeInt(1);
+        serializer.writeInt(1);
+        serializer.writeInt(1);
+        serializer.writeUTF("some-topic");
+        serializer.writeBoolean(true);
+        byte[] bytes = serializer.getSharedBuffer();
+
+        PulsarSourceEnumState state = INSTANCE.deserialize(0, bytes);
+        Set<TopicPartition> partitions = state.getAppendedPartitions();
+        Set<TopicPartition> expectedPartitions =
+                Sets.newHashSet(
+                        new TopicPartition("topic3", 5, singletonList(new TopicRange(600, 2000))),
+                        new TopicPartition("topic4", 8, singletonList(new TopicRange(300, 1000))));
+
+        assertEquals(partitions, expectedPartitions);
+    }
+
+    private void serializeVersion0SplitSet(DataOutputSerializer serializer) throws IOException {
+        serializer.writeInt(1);
+        serializer.writeUTF("topic" + randomNumeric(2));
+        serializer.writeInt(2);
+        serializer.writeInt(400);
+        serializer.writeInt(5000);
+        byte[] stopCursorBytes = InstantiationUtil.serializeObject(StopCursor.latest());
+        serializer.writeInt(stopCursorBytes.length);
+        serializer.write(stopCursorBytes);
+        serializer.writeBoolean(true);
+        byte[] messageIdBytes = MessageId.latest.toByteArray();
+        serializer.writeInt(messageIdBytes.length);
+        serializer.write(messageIdBytes);
+        serializer.writeBoolean(true);
+        serializer.writeLong(1000);
+        serializer.writeLong(2000);
+    }
 }
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumeratorTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumeratorTest.java
index 9c81438d02c..702a567408f 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumeratorTest.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumeratorTest.java
@@ -53,7 +53,6 @@ import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSA
 import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_TYPE;
 import static org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState.initialState;
 import static org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber.getTopicPatternSubscriber;
-import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange;
 import static org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator.DEFAULT_PARTITIONS;
 import static org.apache.flink.shaded.guava30.com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import static org.assertj.core.api.Assertions.assertThat;
@@ -414,7 +413,7 @@ class PulsarSourceEnumeratorTest extends PulsarTestSuiteBase {
         Set<TopicPartition> allPartitions = new HashSet<>();
         for (String topicName : topics) {
             for (int i = 0; i < DEFAULT_PARTITIONS; i++) {
-                allPartitions.add(new TopicPartition(topicName, i, createFullRange()));
+                allPartitions.add(new TopicPartition(topicName, i));
             }
         }
         return allPartitions;
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssignerTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssignerTest.java
index 58f8d8fc51c..bc2d669aeb9 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssignerTest.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssignerTest.java
@@ -36,7 +36,7 @@ import java.util.Optional;
 import java.util.Set;
 
 import static java.util.Collections.singletonList;
-import static org.apache.flink.connector.pulsar.source.enumerator.assigner.NonSharedSplitAssigner.calculatePartitionOwner;
+import static org.apache.flink.connector.pulsar.source.enumerator.assigner.SplitAssignerBase.calculatePartitionOwner;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertTrue;
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerTestBase.java
index e9b7be52def..712ab509e65 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerTestBase.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerTestBase.java
@@ -40,7 +40,6 @@ import static java.util.Collections.singleton;
 import static java.util.Collections.singletonList;
 import static org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState.initialState;
 import static org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor.defaultStopCursor;
-import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange;
 import static org.assertj.core.api.Assertions.assertThat;
 
 /** Test utils for split assigners. */
@@ -93,7 +92,7 @@ abstract class SplitAssignerTestBase extends TestLogger {
     }
 
     protected Set<TopicPartition> createPartitions(String topic, int partitionId) {
-        TopicPartition p1 = new TopicPartition(topic, partitionId, createFullRange());
+        TopicPartition p1 = new TopicPartition(topic, partitionId);
         return singleton(p1);
     }
 
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java
index 301368b6f4a..831d9e0eba3 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java
@@ -43,7 +43,6 @@ import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSA
 import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_TIME;
 import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME;
 import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition;
-import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange;
 import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.flinkSchema;
 import static org.assertj.core.api.Assertions.assertThat;
 
@@ -68,7 +67,7 @@ class StopCursorTest extends PulsarTestSuiteBase {
                         Schema.STRING,
                         randomAlphanumeric(10));
         long currentTimeStamp = System.currentTimeMillis();
-        TopicPartition partition = new TopicPartition(topicName, 0, createFullRange());
+        TopicPartition partition = new TopicPartition(topicName, 0);
         PulsarPartitionSplit split =
                 new PulsarPartitionSplit(
                         partition,
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/PulsarSubscriberTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/PulsarSubscriberTest.java
index f57980e3481..cb8819d2dc9 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/PulsarSubscriberTest.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/PulsarSubscriberTest.java
@@ -37,7 +37,6 @@ import static org.apache.flink.connector.pulsar.source.enumerator.subscriber.Pul
 import static org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber.getTopicPatternSubscriber;
 import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicName;
 import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition;
-import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange;
 import static org.apache.pulsar.client.api.RegexSubscriptionMode.AllTopics;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -81,8 +80,8 @@ class PulsarSubscriberTest extends PulsarTestSuiteBase {
         Set<TopicPartition> expectedPartitions = new HashSet<>();
 
         for (int i = 0; i < NUM_PARTITIONS_PER_TOPIC; i++) {
-            expectedPartitions.add(new TopicPartition(topic1, i, createFullRange()));
-            expectedPartitions.add(new TopicPartition(topic2, i, createFullRange()));
+            expectedPartitions.add(new TopicPartition(topic1, i));
+            expectedPartitions.add(new TopicPartition(topic2, i));
         }
 
         assertEquals(expectedPartitions, topicPartitions);
@@ -97,7 +96,7 @@ class PulsarSubscriberTest extends PulsarTestSuiteBase {
                 subscriber.getSubscribedTopicPartitions(
                         operator().admin(), new FullRangeGenerator(), NUM_PARALLELISM);
 
-        TopicPartition desiredPartition = new TopicPartition(topic1, 2, createFullRange());
+        TopicPartition desiredPartition = new TopicPartition(topic1, 2);
         assertThat(partitions).hasSize(1).containsExactly(desiredPartition);
     }
 
@@ -108,7 +107,7 @@ class PulsarSubscriberTest extends PulsarTestSuiteBase {
                 subscriber.getSubscribedTopicPartitions(
                         operator().admin(), new FullRangeGenerator(), NUM_PARALLELISM);
 
-        TopicPartition desiredPartition = new TopicPartition(topic4, -1, createFullRange());
+        TopicPartition desiredPartition = new TopicPartition(topic4, -1);
         assertThat(partitions).hasSize(1).containsExactly(desiredPartition);
     }
 
@@ -125,8 +124,8 @@ class PulsarSubscriberTest extends PulsarTestSuiteBase {
 
         Set<TopicPartition> expectedPartitions = new HashSet<>();
 
-        expectedPartitions.add(new TopicPartition(topic4, -1, createFullRange()));
-        expectedPartitions.add(new TopicPartition(topic5, -1, createFullRange()));
+        expectedPartitions.add(new TopicPartition(topic4, -1));
+        expectedPartitions.add(new TopicPartition(topic5, -1));
 
         assertEquals(expectedPartitions, topicPartitions);
     }
@@ -144,8 +143,8 @@ class PulsarSubscriberTest extends PulsarTestSuiteBase {
         Set<TopicPartition> expectedPartitions = new HashSet<>();
 
         for (int i = 0; i < NUM_PARTITIONS_PER_TOPIC; i++) {
-            expectedPartitions.add(new TopicPartition(topic1, i, createFullRange()));
-            expectedPartitions.add(new TopicPartition(topic3, i, createFullRange()));
+            expectedPartitions.add(new TopicPartition(topic1, i));
+            expectedPartitions.add(new TopicPartition(topic3, i));
         }
 
         assertEquals(expectedPartitions, topicPartitions);
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicPartitionTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicPartitionTest.java
index 89e7c1f4add..7e5f171116f 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicPartitionTest.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicPartitionTest.java
@@ -20,7 +20,6 @@ package org.apache.flink.connector.pulsar.source.enumerator.topic;
 
 import org.junit.jupiter.api.Test;
 
-import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 /** Unit tests for {@link TopicPartition}. */
@@ -29,12 +28,12 @@ class TopicPartitionTest {
     @Test
     void topicNameForPartitionedAndNonPartitionedTopic() {
         // For partitioned topic
-        TopicPartition partition = new TopicPartition("test-name", 12, createFullRange());
+        TopicPartition partition = new TopicPartition("test-name", 12);
         assertEquals(
                 partition.getFullTopicName(), "persistent://public/default/test-name-partition-12");
 
         // For non-partitioned topic
-        TopicPartition partition1 = new TopicPartition("test-topic", -1, createFullRange());
+        TopicPartition partition1 = new TopicPartition("test-topic", -1);
         assertEquals(partition1.getFullTopicName(), "persistent://public/default/test-topic");
     }
 }
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/SplitRangeGeneratorTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/SplitRangeGeneratorTest.java
new file mode 100644
index 00000000000..ef182bf9eac
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/SplitRangeGeneratorTest.java
@@ -0,0 +1,82 @@
+/*
+ * 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.source.enumerator.topic.range;
+
+import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata;
+import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** The unit test for {@link SplitRangeGenerator}. */
+class SplitRangeGeneratorTest {
+
+    private static final TopicMetadata METADATA = new TopicMetadata("fake", 10);
+
+    @Test
+    void rangeWithParallelismOne() {
+        SplitRangeGenerator generator = new SplitRangeGenerator(6, 65534);
+        List<TopicRange> ranges = generator.range(METADATA, 1);
+        List<TopicRange> desired = Collections.singletonList(new TopicRange(6, 65534));
+
+        assertThat(ranges).hasSize(1).containsExactlyElementsOf(desired);
+    }
+
+    @Test
+    void rangeBelowTheParallelism() {
+        SplitRangeGenerator generator = new SplitRangeGenerator(3, 10);
+        List<TopicRange> ranges = generator.range(METADATA, 12);
+        List<TopicRange> desired =
+                Arrays.asList(
+                        new TopicRange(3, 3),
+                        new TopicRange(4, 4),
+                        new TopicRange(5, 5),
+                        new TopicRange(6, 6),
+                        new TopicRange(7, 7),
+                        new TopicRange(8, 8),
+                        new TopicRange(9, 9),
+                        new TopicRange(10, 10));
+
+        assertThat(ranges).hasSize(8).containsExactlyElementsOf(desired);
+    }
+
+    @Test
+    void rangeWasDivideWithLastBiggerSize() {
+        SplitRangeGenerator generator = new SplitRangeGenerator(0, 100);
+        List<TopicRange> ranges = generator.range(METADATA, 9);
+        List<TopicRange> desired =
+                Arrays.asList(
+                        new TopicRange(0, 10),
+                        new TopicRange(11, 21),
+                        new TopicRange(22, 32),
+                        new TopicRange(33, 43),
+                        new TopicRange(44, 55),
+                        new TopicRange(56, 66),
+                        new TopicRange(67, 77),
+                        new TopicRange(78, 88),
+                        new TopicRange(89, 100));
+
+        assertThat(ranges).hasSize(9).containsExactlyElementsOf(desired);
+    }
+}
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/TopicRangeUtilsTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/TopicRangeUtilsTest.java
new file mode 100644
index 00000000000..f844aae8f84
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/topic/range/TopicRangeUtilsTest.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.pulsar.source.enumerator.topic.range;
+
+import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.range.RangeGenerator.KeySharedMode.JOIN;
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.range.TopicRangeUtils.isFullTopicRanges;
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.range.TopicRangeUtils.validateTopicRanges;
+import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/** Test class for {@link TopicRangeUtils}. */
+class TopicRangeUtilsTest {
+
+    @Test
+    void testValidateTopicRanges() {
+        List<TopicRange> ranges1 = Arrays.asList(new TopicRange(1, 2), new TopicRange(2, 3));
+        assertThrows(IllegalArgumentException.class, () -> validateTopicRanges(ranges1, JOIN));
+
+        List<TopicRange> ranges2 = Arrays.asList(new TopicRange(1, 14), new TopicRange(2, 5));
+        assertThrows(IllegalArgumentException.class, () -> validateTopicRanges(ranges2, JOIN));
+
+        List<TopicRange> ranges3 = Arrays.asList(new TopicRange(1, 14), new TopicRange(5, 30));
+        assertThrows(IllegalArgumentException.class, () -> validateTopicRanges(ranges3, JOIN));
+
+        List<TopicRange> ranges4 = Arrays.asList(new TopicRange(1, 14), new TopicRange(15, 30));
+        assertDoesNotThrow(() -> validateTopicRanges(ranges4, JOIN));
+    }
+
+    @Test
+    void testIsFullTopicRanges() {
+        List<TopicRange> ranges1 =
+                Arrays.asList(
+                        new TopicRange(16384, 32767),
+                        new TopicRange(0, 16383),
+                        new TopicRange(32768, 49151),
+                        new TopicRange(49152, 65535));
+        assertTrue(isFullTopicRanges(ranges1));
+
+        List<TopicRange> ranges2 =
+                Arrays.asList(
+                        new TopicRange(32768, 49151),
+                        new TopicRange(0, 16383),
+                        new TopicRange(16384, 32767),
+                        new TopicRange(49152, 65531));
+        assertFalse(isFullTopicRanges(ranges2));
+
+        List<TopicRange> ranges3 =
+                Arrays.asList(
+                        new TopicRange(33, 16383),
+                        new TopicRange(32768, 49151),
+                        new TopicRange(16384, 32767),
+                        new TopicRange(49152, 65535));
+        assertFalse(isFullTopicRanges(ranges3));
+
+        List<TopicRange> ranges4 = Collections.singletonList(TopicRange.createFullRange());
+        assertTrue(isFullTopicRanges(ranges4));
+    }
+}
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java
index 008cee7d22f..e63424a787d 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java
@@ -65,7 +65,6 @@ import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSA
 import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_TIME;
 import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME;
 import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition;
-import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange;
 import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.flinkSchema;
 import static org.apache.flink.connector.pulsar.testutils.PulsarTestCommonUtils.isAssignableFromParameterContext;
 import static org.apache.flink.connector.pulsar.testutils.extension.TestOrderlinessExtension.PULSAR_SOURCE_READER_SUBSCRIPTION_TYPE_STORE_KEY;
@@ -110,7 +109,7 @@ abstract class PulsarPartitionSplitReaderTestBase extends PulsarTestSuiteBase {
             String topicName,
             int partitionId,
             MessageId startPosition) {
-        TopicPartition partition = new TopicPartition(topicName, partitionId, createFullRange());
+        TopicPartition partition = new TopicPartition(topicName, partitionId);
         PulsarPartitionSplit split =
                 new PulsarPartitionSplit(partition, StopCursor.never(), startPosition, null);
         SplitsAddition<PulsarPartitionSplit> addition = new SplitsAddition<>(singletonList(split));
@@ -127,7 +126,7 @@ abstract class PulsarPartitionSplitReaderTestBase extends PulsarTestSuiteBase {
             String topicName,
             int partitionId,
             MessageId startPosition) {
-        TopicPartition partition = new TopicPartition(topicName, partitionId, createFullRange());
+        TopicPartition partition = new TopicPartition(topicName, partitionId);
         PulsarPartitionSplit split =
                 new PulsarPartitionSplit(partition, StopCursor.never(), null, null);
         SplitsAddition<PulsarPartitionSplit> addition = new SplitsAddition<>(singletonList(split));
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplitSerializerTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplitSerializerTest.java
index 63350af1f91..3d4ee8860d5 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplitSerializerTest.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplitSerializerTest.java
@@ -20,11 +20,17 @@ package org.apache.flink.connector.pulsar.source.split;
 
 import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition;
+import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange;
+import org.apache.flink.core.memory.DataOutputSerializer;
+import org.apache.flink.util.InstantiationUtil;
 
+import org.apache.pulsar.client.api.MessageId;
+import org.apache.pulsar.client.api.transaction.TxnID;
 import org.junit.jupiter.api.Test;
 
+import static java.util.Collections.singletonList;
 import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic;
-import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange;
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.range.RangeGenerator.KeySharedMode.JOIN;
 import static org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplitSerializer.INSTANCE;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotSame;
@@ -33,16 +39,52 @@ import static org.junit.jupiter.api.Assertions.assertNotSame;
 class PulsarPartitionSplitSerializerTest {
 
     @Test
-    void serializeAndDeserializePulsarPartitionSplit() throws Exception {
+    void version1SerializeAndDeserialize() throws Exception {
         PulsarPartitionSplit split =
                 new PulsarPartitionSplit(
-                        new TopicPartition(randomAlphabetic(10), 10, createFullRange()),
+                        new TopicPartition(
+                                randomAlphabetic(10),
+                                10,
+                                singletonList(new TopicRange(400, 5000)),
+                                JOIN),
                         StopCursor.defaultStopCursor());
 
         byte[] bytes = INSTANCE.serialize(split);
-        PulsarPartitionSplit split1 = INSTANCE.deserialize(INSTANCE.getVersion(), bytes);
+        PulsarPartitionSplit split1 = INSTANCE.deserialize(1, bytes);
 
         assertEquals(split, split1);
         assertNotSame(split, split1);
     }
+
+    @Test
+    void version0Deserialize() throws Exception {
+        DataOutputSerializer serializer = new DataOutputSerializer(4096);
+        // Serialize in version 0 logic.
+        serializer.writeUTF("topic44");
+        serializer.writeInt(2);
+        serializer.writeInt(400);
+        serializer.writeInt(5000);
+        byte[] stopCursorBytes = InstantiationUtil.serializeObject(StopCursor.latest());
+        serializer.writeInt(stopCursorBytes.length);
+        serializer.write(stopCursorBytes);
+        serializer.writeBoolean(true);
+        byte[] messageIdBytes = MessageId.latest.toByteArray();
+        serializer.writeInt(messageIdBytes.length);
+        serializer.write(messageIdBytes);
+        serializer.writeBoolean(true);
+        serializer.writeLong(1000);
+        serializer.writeLong(2000);
+        byte[] bytes = serializer.getSharedBuffer();
+
+        PulsarPartitionSplit split = INSTANCE.deserialize(0, bytes);
+        PulsarPartitionSplit expectedSplit =
+                new PulsarPartitionSplit(
+                        new TopicPartition("topic44", 2, singletonList(new TopicRange(400, 5000))),
+                        StopCursor.latest(),
+                        MessageId.latest,
+                        new TxnID(1000, 2000));
+
+        assertEquals(split, expectedSplit);
+        assertNotSame(split, expectedSplit);
+    }
 }


[flink] 03/06: [FLINK-26182][Connector/pulsar] Extract common logic from Pulsar source testing tools.

Posted by ti...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tison pushed a commit to branch release-1.16
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 908405fc8354856dc741350eadb71e2a892830a9
Author: Yufan Sheng <yu...@streamnative.io>
AuthorDate: Tue Sep 27 18:04:24 2022 +0800

    [FLINK-26182][Connector/pulsar] Extract common logic from Pulsar source testing tools.
---
 .../pulsar/source/enumerator/topic/TopicRange.java |   2 +-
 .../pulsar/source/PulsarSourceITCase.java          |   6 +-
 .../pulsar/source/PulsarUnorderedSourceITCase.java |  60 ++-------
 .../pulsar/testutils/PulsarTestCommonUtils.java    |  17 ++-
 .../pulsar/testutils/PulsarTestContext.java        |  45 +++----
 .../pulsar/testutils/PulsarTestContextFactory.java |   5 +-
 .../pulsar/testutils/PulsarTestEnvironment.java    |   2 +-
 .../pulsar/testutils/PulsarTestSuiteBase.java      |   5 +-
 .../cases/MultipleTopicTemplateContext.java        | 134 -------------------
 .../cases/SharedSubscriptionConsumingContext.java  |  58 ---------
 .../cases/SingleTopicConsumingContext.java         | 132 -------------------
 .../{ => source}/PulsarPartitionDataWriter.java    |  18 ++-
 .../testutils/source/PulsarSourceTestContext.java  | 138 ++++++++++++++++++++
 .../source/UnorderedSourceTestSuiteBase.java}      |  40 ++----
 .../cases/MultipleTopicConsumingContext.java       |  42 ++++--
 .../source/cases/SingleTopicConsumingContext.java  |  73 +++++++++++
 .../flink-end-to-end-tests-pulsar/pom.xml          |  24 +++-
 .../util/pulsar/PulsarSourceOrderedE2ECase.java    |   5 +-
 .../util/pulsar/PulsarSourceUnorderedE2ECase.java  |  13 +-
 .../pulsar/cases/KeySharedSubscriptionContext.java | 144 ---------------------
 .../pulsar/cases/SharedSubscriptionContext.java    | 116 -----------------
 .../FlinkContainerWithPulsarEnvironment.java       |   3 +-
 .../common/KeyedPulsarPartitionDataWriter.java     |  19 +--
 .../common/UnorderedSourceTestSuiteBase.java       |  86 ------------
 .../ExclusiveSubscriptionContext.java              |  19 +--
 .../FailoverSubscriptionContext.java               |  19 +--
 .../source/KeySharedSubscriptionContext.java       |  87 +++++++++++++
 .../pulsar/source/SharedSubscriptionContext.java   |  32 ++---
 .../source/enumerator/NoOpEnumStateSerializer.java |   2 +-
 29 files changed, 466 insertions(+), 880 deletions(-)

diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicRange.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicRange.java
index 5b779224b87..1508b8732a3 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicRange.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicRange.java
@@ -68,7 +68,7 @@ public class TopicRange implements Serializable {
         return new Range(start, end);
     }
 
-    /** Create a topic range which contains the fully hash range. */
+    /** Create a topic range which contains the full hash range. */
     public static TopicRange createFullRange() {
         return new TopicRange(MIN_RANGE, MAX_RANGE);
     }
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java
index 54fc4d71dd5..c2c95f255b5 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java
@@ -20,9 +20,9 @@ package org.apache.flink.connector.pulsar.source;
 
 import org.apache.flink.connector.pulsar.testutils.PulsarTestContextFactory;
 import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment;
-import org.apache.flink.connector.pulsar.testutils.cases.MultipleTopicConsumingContext;
-import org.apache.flink.connector.pulsar.testutils.cases.SingleTopicConsumingContext;
 import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime;
+import org.apache.flink.connector.pulsar.testutils.source.cases.MultipleTopicConsumingContext;
+import org.apache.flink.connector.pulsar.testutils.source.cases.SingleTopicConsumingContext;
 import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
 import org.apache.flink.connector.testframe.junit.annotations.TestContext;
 import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
@@ -41,6 +41,7 @@ import org.junit.experimental.categories.Category;
  */
 @Category(value = {FailsOnJava11.class})
 class PulsarSourceITCase extends SourceTestSuiteBase<String> {
+
     // Defines test environment on Flink MiniCluster
     @TestEnv MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment();
 
@@ -48,6 +49,7 @@ class PulsarSourceITCase extends SourceTestSuiteBase<String> {
     @TestExternalSystem
     PulsarTestEnvironment pulsar = new PulsarTestEnvironment(PulsarRuntime.mock());
 
+    // This field is preserved, we don't support the semantics in source currently.
     @TestSemantics
     CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
 
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarUnorderedSourceITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarUnorderedSourceITCase.java
index 6bad7220886..d18c6c2633b 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarUnorderedSourceITCase.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarUnorderedSourceITCase.java
@@ -20,36 +20,28 @@ package org.apache.flink.connector.pulsar.source;
 
 import org.apache.flink.connector.pulsar.testutils.PulsarTestContextFactory;
 import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment;
-import org.apache.flink.connector.pulsar.testutils.cases.SharedSubscriptionConsumingContext;
 import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime;
+import org.apache.flink.connector.pulsar.testutils.source.UnorderedSourceTestSuiteBase;
+import org.apache.flink.connector.pulsar.testutils.source.cases.MultipleTopicConsumingContext;
 import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
-import org.apache.flink.connector.testframe.environment.TestEnvironment;
-import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
 import org.apache.flink.connector.testframe.junit.annotations.TestContext;
 import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
 import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
 import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
-import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
 import org.apache.flink.streaming.api.CheckpointingMode;
 import org.apache.flink.testutils.junit.FailsOnJava11;
-import org.apache.flink.util.CloseableIterator;
 
 import org.apache.pulsar.client.api.SubscriptionType;
 import org.junit.experimental.categories.Category;
-import org.junit.jupiter.api.Disabled;
 
-import java.util.List;
-
-import static java.util.concurrent.CompletableFuture.runAsync;
-import static org.apache.flink.connector.testframe.utils.CollectIteratorAssertions.assertUnordered;
-import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT;
-import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+import static org.apache.pulsar.client.api.SubscriptionType.Shared;
 
 /**
  * Unit test class for {@link PulsarSource}. Used for {@link SubscriptionType#Shared} subscription.
  */
 @Category(value = {FailsOnJava11.class})
-public class PulsarUnorderedSourceITCase extends SourceTestSuiteBase<String> {
+public class PulsarUnorderedSourceITCase extends UnorderedSourceTestSuiteBase<String> {
+
     // Defines test environment on Flink MiniCluster
     @TestEnv MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment();
 
@@ -61,43 +53,7 @@ public class PulsarUnorderedSourceITCase extends SourceTestSuiteBase<String> {
     CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
 
     @TestContext
-    PulsarTestContextFactory<String, SharedSubscriptionConsumingContext> singleTopic =
-            new PulsarTestContextFactory<>(pulsar, SharedSubscriptionConsumingContext::new);
-
-    @Override
-    protected void checkResultWithSemantic(
-            CloseableIterator<String> resultIterator,
-            List<List<String>> testData,
-            CheckpointingMode semantic,
-            Integer limit) {
-        Runnable runnable =
-                () ->
-                        assertUnordered(resultIterator)
-                                .withNumRecordsLimit(getExpectedSize(testData, limit))
-                                .matchesRecordsFromSource(testData, semantic);
-
-        assertThat(runAsync(runnable)).succeedsWithin(DEFAULT_COLLECT_DATA_TIMEOUT);
-    }
-
-    /**
-     * Shared subscription will have multiple readers on same partition, this would make hard to
-     * automatically stop like a bounded source.
-     */
-    private static int getExpectedSize(List<List<String>> testData, Integer limit) {
-        if (limit == null) {
-            return testData.stream().mapToInt(List::size).sum();
-        } else {
-            return limit;
-        }
-    }
-
-    @Override
-    @Disabled("We don't have any idle readers in Pulsar's shared subscription.")
-    public void testIdleReader(
-            TestEnvironment testEnv,
-            DataStreamSourceExternalContext<String> externalContext,
-            CheckpointingMode semantic)
-            throws Exception {
-        super.testIdleReader(testEnv, externalContext, semantic);
-    }
+    PulsarTestContextFactory<String, MultipleTopicConsumingContext> multipleTopic =
+            new PulsarTestContextFactory<>(
+                    pulsar, env -> new MultipleTopicConsumingContext(env, Shared));
 }
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestCommonUtils.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestCommonUtils.java
index 87f3976c6a7..96b1ca62f2a 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestCommonUtils.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestCommonUtils.java
@@ -19,10 +19,12 @@
 package org.apache.flink.connector.pulsar.testutils;
 
 import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.connector.base.DeliveryGuarantee;
 import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange;
 import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit;
+import org.apache.flink.streaming.api.CheckpointingMode;
 
 import org.apache.pulsar.client.api.MessageId;
 import org.junit.jupiter.api.extension.ParameterContext;
@@ -33,7 +35,18 @@ import java.util.List;
 /** Put static methods that can be used by multiple test classes. */
 public class PulsarTestCommonUtils {
 
-    // ------- CreateSplits
+    /** Convert the CheckpointingMode to a connector related DeliveryGuarantee. */
+    public static DeliveryGuarantee toDeliveryGuarantee(CheckpointingMode checkpointingMode) {
+        if (checkpointingMode == CheckpointingMode.AT_LEAST_ONCE) {
+            return DeliveryGuarantee.AT_LEAST_ONCE;
+        } else if (checkpointingMode == CheckpointingMode.EXACTLY_ONCE) {
+            return DeliveryGuarantee.EXACTLY_ONCE;
+        } else {
+            throw new IllegalArgumentException(
+                    "Only exactly-once and al-least-once checkpointing mode are supported.");
+        }
+    }
+
     /** creates a fullRange() partitionSplit. */
     public static PulsarPartitionSplit createPartitionSplit(String topic, int partitionId) {
         return createPartitionSplit(topic, partitionId, Boundedness.CONTINUOUS_UNBOUNDED);
@@ -63,8 +76,6 @@ public class PulsarTestCommonUtils {
         return splits;
     }
 
-    // -------- InvocationContext Utils
-
     public static boolean isAssignableFromParameterContext(
             Class<?> requiredType, ParameterContext context) {
         return requiredType.isAssignableFrom(context.getParameter().getType());
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContext.java
index 9eaa24041c7..a4e44441bfd 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContext.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContext.java
@@ -19,41 +19,30 @@
 package org.apache.flink.connector.pulsar.testutils;
 
 import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator;
-import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.external.ExternalContext;
+
+import org.apache.pulsar.client.api.Schema;
 
 import java.net.URL;
-import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
-import java.util.Random;
-
-import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric;
 
-/** Common test context for pulsar based test. */
-public abstract class PulsarTestContext<T> implements DataStreamSourceExternalContext<T> {
+/**
+ * The implementation for Flink connector test tools. Providing the common test case writing
+ * constraint for both source, sink and table API.
+ */
+public abstract class PulsarTestContext<T> implements ExternalContext {
 
     protected final PulsarRuntimeOperator operator;
-    protected final List<URL> connectorJarPaths;
+    // The schema used for consuming and producing messages between Pulsar and tests.
+    protected final Schema<T> schema;
 
-    protected PulsarTestContext(PulsarTestEnvironment environment, List<URL> connectorJarPaths) {
+    protected PulsarTestContext(PulsarTestEnvironment environment, Schema<T> schema) {
         this.operator = environment.operator();
-        this.connectorJarPaths = connectorJarPaths;
-    }
-
-    // Helper methods for generating data.
-
-    protected List<String> generateStringTestData(int splitIndex, long seed) {
-        Random random = new Random(seed);
-        int recordNum = 300 + random.nextInt(200);
-        List<String> records = new ArrayList<>(recordNum);
-
-        for (int i = 0; i < recordNum; i++) {
-            int length = random.nextInt(40) + 10;
-            records.add(splitIndex + "-" + i + "-" + randomAlphanumeric(length));
-        }
-
-        return records;
+        this.schema = schema;
     }
 
+    /** Implement this method for providing a more friendly test name in IDE. */
     protected abstract String displayName();
 
     @Override
@@ -63,6 +52,10 @@ public abstract class PulsarTestContext<T> implements DataStreamSourceExternalCo
 
     @Override
     public List<URL> getConnectorJarPaths() {
-        return connectorJarPaths;
+        // We don't need any tests jar definition. They are provided in docker-related environments.
+        return Collections.emptyList();
     }
+
+    @Override
+    public void close() throws Exception {}
 }
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContextFactory.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContextFactory.java
index c634efffc44..100c2a67d43 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContextFactory.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContextFactory.java
@@ -18,13 +18,14 @@
 
 package org.apache.flink.connector.pulsar.testutils;
 
+import org.apache.flink.connector.pulsar.testutils.source.PulsarSourceTestContext;
 import org.apache.flink.connector.testframe.external.ExternalContextFactory;
 
 import java.util.function.Function;
 
 /**
- * Factory for creating all the test context that extends {@link PulsarTestContext}. Test context
- * class should have a constructor with {@link PulsarTestEnvironment} arg.
+ * Factory for creating all the test context that extends {@link PulsarSourceTestContext}. Test
+ * context class should have a constructor with single {@link PulsarTestEnvironment} arg.
  */
 public class PulsarTestContextFactory<F, T extends PulsarTestContext<F>>
         implements ExternalContextFactory<T> {
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestEnvironment.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestEnvironment.java
index 0f3fb9e15f9..f921e4b92f1 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestEnvironment.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestEnvironment.java
@@ -48,7 +48,7 @@ import java.util.List;
  * }</pre>
  *
  * <p>If you want to use this class in JUnit 5, just simply extends {@link PulsarTestSuiteBase}, all
- * the helper methods in {@code PulsarContainerOperator} is also exposed there.
+ * the helper methods in {@link PulsarRuntimeOperator} is also exposed there.
  */
 public class PulsarTestEnvironment
         implements BeforeAllCallback, AfterAllCallback, TestResource, TestRule {
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestSuiteBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestSuiteBase.java
index b55fdc553b0..6f64ec008b3 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestSuiteBase.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestSuiteBase.java
@@ -30,9 +30,8 @@ import org.junit.jupiter.api.extension.RegisterExtension;
  * The base class for the all Pulsar related test sites. It brings up:
  *
  * <ul>
- *   <li>A Zookeeper cluster.
- *   <li>Pulsar Broker.
- *   <li>A Bookkeeper cluster.
+ *   <li>A Pulsar Broker with memory based local metastore.
+ *   <li>A standalone Bookkeeper with memory based local metastore.
  * </ul>
  *
  * <p>You just need to write a JUnit 5 test class and extends this suite class. All the helper
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicTemplateContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicTemplateContext.java
deleted file mode 100644
index 3eca9e7f926..00000000000
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicTemplateContext.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.connector.pulsar.testutils.cases;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.connector.source.Boundedness;
-import org.apache.flink.api.connector.source.Source;
-import org.apache.flink.connector.pulsar.source.PulsarSource;
-import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder;
-import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor;
-import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils;
-import org.apache.flink.connector.pulsar.testutils.PulsarPartitionDataWriter;
-import org.apache.flink.connector.pulsar.testutils.PulsarTestContext;
-import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment;
-import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter;
-import org.apache.flink.connector.testframe.external.source.TestingSourceSettings;
-
-import org.apache.pulsar.client.api.RegexSubscriptionMode;
-import org.apache.pulsar.client.api.SubscriptionType;
-
-import java.net.URL;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic;
-import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.pulsarSchema;
-import static org.apache.pulsar.client.api.Schema.STRING;
-
-/**
- * Pulsar external context template that will create multiple topics with only one partitions as
- * source splits.
- */
-public abstract class MultipleTopicTemplateContext extends PulsarTestContext<String> {
-
-    private int numTopics = 0;
-
-    private final String topicPattern = "pulsar-multiple-topic-[0-9]+-" + randomAlphabetic(8);
-
-    private final Map<String, ExternalSystemSplitDataWriter<String>> topicNameToSplitWriters =
-            new HashMap<>();
-
-    public MultipleTopicTemplateContext(PulsarTestEnvironment environment) {
-        this(environment, Collections.emptyList());
-    }
-
-    public MultipleTopicTemplateContext(
-            PulsarTestEnvironment environment, List<URL> connectorJarPaths) {
-        super(environment, connectorJarPaths);
-    }
-
-    @Override
-    public Source<String, ?, ?> createSource(TestingSourceSettings sourceSettings) {
-        PulsarSourceBuilder<String> builder =
-                PulsarSource.builder()
-                        .setDeserializationSchema(pulsarSchema(STRING))
-                        .setServiceUrl(serviceUrl())
-                        .setAdminUrl(adminUrl())
-                        .setTopicPattern(topicPattern, RegexSubscriptionMode.AllTopics)
-                        .setSubscriptionType(subscriptionType())
-                        .setSubscriptionName(subscriptionName());
-        if (sourceSettings.getBoundedness() == Boundedness.BOUNDED) {
-            // Using latest stop cursor for making sure the source could be stopped.
-            // This is required for SourceTestSuiteBase.
-            builder.setBoundedStopCursor(StopCursor.latest());
-        }
-
-        return builder.build();
-    }
-
-    @Override
-    public ExternalSystemSplitDataWriter<String> createSourceSplitDataWriter(
-            TestingSourceSettings sourceSettings) {
-        String topicName = topicPattern.replace("[0-9]+", String.valueOf(numTopics));
-        operator.createTopic(topicName, 1);
-
-        String partitionName = TopicNameUtils.topicNameWithPartition(topicName, 0);
-        PulsarPartitionDataWriter writer = new PulsarPartitionDataWriter(operator, partitionName);
-
-        topicNameToSplitWriters.put(partitionName, writer);
-        numTopics++;
-
-        return writer;
-    }
-
-    @Override
-    public List<String> generateTestData(
-            TestingSourceSettings sourceSettings, int splitIndex, long seed) {
-        return generateStringTestData(splitIndex, seed);
-    }
-
-    @Override
-    public TypeInformation<String> getProducedType() {
-        return TypeInformation.of(String.class);
-    }
-
-    @Override
-    public void close() throws Exception {
-        for (ExternalSystemSplitDataWriter<String> writer : topicNameToSplitWriters.values()) {
-            writer.close();
-        }
-
-        topicNameToSplitWriters.clear();
-    }
-
-    protected abstract String subscriptionName();
-
-    protected abstract SubscriptionType subscriptionType();
-
-    protected String serviceUrl() {
-        return operator.serviceUrl();
-    }
-
-    protected String adminUrl() {
-        return operator.adminUrl();
-    }
-}
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/SharedSubscriptionConsumingContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/SharedSubscriptionConsumingContext.java
deleted file mode 100644
index 8001b6e7300..00000000000
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/SharedSubscriptionConsumingContext.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.connector.pulsar.testutils.cases;
-
-import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment;
-
-import org.apache.pulsar.client.api.SubscriptionType;
-
-import java.net.URL;
-import java.util.Collections;
-import java.util.List;
-
-/**
- * A consuming context with {@link SubscriptionType#Shared}, it's almost the same as {@link
- * MultipleTopicConsumingContext}.
- */
-public class SharedSubscriptionConsumingContext extends MultipleTopicTemplateContext {
-
-    public SharedSubscriptionConsumingContext(PulsarTestEnvironment environment) {
-        this(environment, Collections.emptyList());
-    }
-
-    public SharedSubscriptionConsumingContext(
-            PulsarTestEnvironment environment, List<URL> connectorJarPaths) {
-        super(environment, connectorJarPaths);
-    }
-
-    @Override
-    protected String displayName() {
-        return "consuming message with shared subscription";
-    }
-
-    @Override
-    protected String subscriptionName() {
-        return "flink-shared-subscription-test";
-    }
-
-    @Override
-    protected SubscriptionType subscriptionType() {
-        return SubscriptionType.Shared;
-    }
-}
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/SingleTopicConsumingContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/SingleTopicConsumingContext.java
deleted file mode 100644
index f5bfa45f32b..00000000000
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/SingleTopicConsumingContext.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.connector.pulsar.testutils.cases;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.connector.source.Boundedness;
-import org.apache.flink.api.connector.source.Source;
-import org.apache.flink.connector.pulsar.source.PulsarSource;
-import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder;
-import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor;
-import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils;
-import org.apache.flink.connector.pulsar.testutils.PulsarPartitionDataWriter;
-import org.apache.flink.connector.pulsar.testutils.PulsarTestContext;
-import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment;
-import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter;
-import org.apache.flink.connector.testframe.external.source.TestingSourceSettings;
-
-import java.net.URL;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ThreadLocalRandom;
-
-import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.pulsarSchema;
-import static org.apache.pulsar.client.api.Schema.STRING;
-import static org.apache.pulsar.client.api.SubscriptionType.Exclusive;
-
-/**
- * A Pulsar external context that will create only one topic and use partitions in that topic as
- * source splits.
- */
-public class SingleTopicConsumingContext extends PulsarTestContext<String> {
-
-    private static final String TOPIC_NAME_PREFIX = "pulsar-single-topic";
-    private final String topicName;
-    private final Map<Integer, ExternalSystemSplitDataWriter<String>> partitionToSplitWriter =
-            new HashMap<>();
-
-    private int numSplits = 0;
-
-    public SingleTopicConsumingContext(PulsarTestEnvironment environment) {
-        this(environment, Collections.emptyList());
-    }
-
-    public SingleTopicConsumingContext(
-            PulsarTestEnvironment environment, List<URL> connectorJarPaths) {
-        super(environment, connectorJarPaths);
-        this.topicName =
-                TOPIC_NAME_PREFIX + "-" + ThreadLocalRandom.current().nextLong(Long.MAX_VALUE);
-    }
-
-    @Override
-    protected String displayName() {
-        return "consuming message on single topic";
-    }
-
-    @Override
-    public Source<String, ?, ?> createSource(TestingSourceSettings sourceSettings) {
-        PulsarSourceBuilder<String> builder =
-                PulsarSource.builder()
-                        .setDeserializationSchema(pulsarSchema(STRING))
-                        .setServiceUrl(operator.serviceUrl())
-                        .setAdminUrl(operator.adminUrl())
-                        .setTopics(topicName)
-                        .setSubscriptionType(Exclusive)
-                        .setSubscriptionName("pulsar-single-topic");
-        if (sourceSettings.getBoundedness() == Boundedness.BOUNDED) {
-            // Using latest stop cursor for making sure the source could be stopped.
-            // This is required for SourceTestSuiteBase.
-            builder.setBoundedStopCursor(StopCursor.latest());
-        }
-
-        return builder.build();
-    }
-
-    @Override
-    public ExternalSystemSplitDataWriter<String> createSourceSplitDataWriter(
-            TestingSourceSettings sourceSettings) {
-        if (numSplits == 0) {
-            // Create the topic first.
-            operator.createTopic(topicName, 1);
-            numSplits++;
-        } else {
-            numSplits++;
-            operator.increaseTopicPartitions(topicName, numSplits);
-        }
-
-        String partitionName = TopicNameUtils.topicNameWithPartition(topicName, numSplits - 1);
-        PulsarPartitionDataWriter writer = new PulsarPartitionDataWriter(operator, partitionName);
-        partitionToSplitWriter.put(numSplits - 1, writer);
-
-        return writer;
-    }
-
-    @Override
-    public List<String> generateTestData(
-            TestingSourceSettings sourceSettings, int splitIndex, long seed) {
-        return generateStringTestData(splitIndex, seed);
-    }
-
-    @Override
-    public TypeInformation<String> getProducedType() {
-        return TypeInformation.of(String.class);
-    }
-
-    @Override
-    public void close() throws Exception {
-        // Close writer.
-        for (ExternalSystemSplitDataWriter<String> writer : partitionToSplitWriter.values()) {
-            writer.close();
-        }
-
-        partitionToSplitWriter.clear();
-    }
-}
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarPartitionDataWriter.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/PulsarPartitionDataWriter.java
similarity index 69%
rename from flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarPartitionDataWriter.java
rename to flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/PulsarPartitionDataWriter.java
index da48e26cd25..1ceb2922bc3 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarPartitionDataWriter.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/PulsarPartitionDataWriter.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.connector.pulsar.testutils;
+package org.apache.flink.connector.pulsar.testutils.source;
 
 import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator;
 import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter;
@@ -25,20 +25,26 @@ import org.apache.pulsar.client.api.Schema;
 
 import java.util.List;
 
-/** Source split data writer for writing test data into a Pulsar topic partition. */
-public class PulsarPartitionDataWriter implements ExternalSystemSplitDataWriter<String> {
+/**
+ * Source split data writer for writing test data into a Pulsar topic partition. This writer doesn't
+ * need to be closed.
+ */
+public class PulsarPartitionDataWriter<T> implements ExternalSystemSplitDataWriter<T> {
 
     private final PulsarRuntimeOperator operator;
     private final String fullTopicName;
+    private final Schema<T> schema;
 
-    public PulsarPartitionDataWriter(PulsarRuntimeOperator operator, String fullTopicName) {
+    public PulsarPartitionDataWriter(
+            PulsarRuntimeOperator operator, String fullTopicName, Schema<T> schema) {
         this.operator = operator;
         this.fullTopicName = fullTopicName;
+        this.schema = schema;
     }
 
     @Override
-    public void writeRecords(List<String> records) {
-        operator.sendMessages(fullTopicName, Schema.STRING, records);
+    public void writeRecords(List<T> records) {
+        operator.sendMessages(fullTopicName, schema, records);
     }
 
     @Override
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/PulsarSourceTestContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/PulsarSourceTestContext.java
new file mode 100644
index 00000000000..8089f8c58e7
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/PulsarSourceTestContext.java
@@ -0,0 +1,138 @@
+/*
+ * 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.testutils.source;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.connector.pulsar.source.PulsarSource;
+import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder;
+import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor;
+import org.apache.flink.connector.pulsar.testutils.PulsarTestContext;
+import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment;
+import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter;
+import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
+import org.apache.flink.connector.testframe.external.source.TestingSourceSettings;
+
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionType;
+
+import java.util.List;
+import java.util.Random;
+import java.util.stream.IntStream;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric;
+import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS;
+import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.pulsarSchema;
+import static org.apache.pulsar.client.api.RegexSubscriptionMode.AllTopics;
+
+/**
+ * Common source test context for pulsar based test. We use the string text as the basic send
+ * content.
+ */
+public abstract class PulsarSourceTestContext extends PulsarTestContext<String>
+        implements DataStreamSourceExternalContext<String> {
+
+    private static final long DISCOVERY_INTERVAL = 1000L;
+    private static final int BATCH_DATA_SIZE = 300;
+
+    protected PulsarSourceTestContext(PulsarTestEnvironment environment) {
+        super(environment, Schema.STRING);
+    }
+
+    @Override
+    public Source<String, ?, ?> createSource(TestingSourceSettings sourceSettings) {
+        PulsarSourceBuilder<String> builder =
+                PulsarSource.builder()
+                        .setDeserializationSchema(pulsarSchema(schema))
+                        .setServiceUrl(operator.serviceUrl())
+                        .setAdminUrl(operator.adminUrl())
+                        .setTopicPattern(topicPattern(), AllTopics)
+                        .setSubscriptionType(subscriptionType())
+                        .setSubscriptionName(subscriptionName())
+                        .setConfig(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, DISCOVERY_INTERVAL);
+
+        // Set extra configuration for source builder.
+        setSourceBuilder(builder);
+
+        if (sourceSettings.getBoundedness() == Boundedness.BOUNDED) {
+            // Using the latest stop cursor for making sure the source could be stopped.
+            // This is required for SourceTestSuiteBase.
+            builder.setBoundedStopCursor(StopCursor.latest());
+        }
+
+        return builder.build();
+    }
+
+    @Override
+    public ExternalSystemSplitDataWriter<String> createSourceSplitDataWriter(
+            TestingSourceSettings sourceSettings) {
+        String partitionName = generatePartitionName();
+        return new PulsarPartitionDataWriter<>(operator, partitionName, schema);
+    }
+
+    @Override
+    public List<String> generateTestData(
+            TestingSourceSettings sourceSettings, int splitIndex, long seed) {
+        Random random = new Random(seed);
+        return IntStream.range(0, BATCH_DATA_SIZE)
+                .boxed()
+                .map(
+                        index -> {
+                            int length = random.nextInt(20) + 1;
+                            return "split:"
+                                    + splitIndex
+                                    + "-index:"
+                                    + index
+                                    + "-content:"
+                                    + randomAlphanumeric(length);
+                        })
+                .collect(toList());
+    }
+
+    @Override
+    public TypeInformation<String> getProducedType() {
+        return Types.STRING;
+    }
+
+    /** Override this method for creating builder. */
+    protected void setSourceBuilder(PulsarSourceBuilder<String> builder) {
+        // Nothing to do by default.
+    }
+
+    /**
+     * The topic pattern which is used in Pulsar topic auto discovery. It was discovered every
+     * {@link #DISCOVERY_INTERVAL} ms;
+     */
+    protected abstract String topicPattern();
+
+    /** The subscription name used in Pulsar consumer. */
+    protected abstract String subscriptionName();
+
+    /** The subscription type used in Pulsar consumer. */
+    protected abstract SubscriptionType subscriptionType();
+
+    /**
+     * Dynamic generate a partition related topic in Pulsar. This topic should be pre-created in
+     * Pulsar. Everytime we call this method, we may get a new partition name.
+     */
+    protected abstract String generatePartitionName();
+}
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarUnorderedSourceITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/UnorderedSourceTestSuiteBase.java
similarity index 58%
copy from flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarUnorderedSourceITCase.java
copy to flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/UnorderedSourceTestSuiteBase.java
index 6bad7220886..35496ccab21 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarUnorderedSourceITCase.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/UnorderedSourceTestSuiteBase.java
@@ -16,26 +16,15 @@
  * limitations under the License.
  */
 
-package org.apache.flink.connector.pulsar.source;
+package org.apache.flink.connector.pulsar.testutils.source;
 
-import org.apache.flink.connector.pulsar.testutils.PulsarTestContextFactory;
-import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment;
-import org.apache.flink.connector.pulsar.testutils.cases.SharedSubscriptionConsumingContext;
-import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime;
-import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
 import org.apache.flink.connector.testframe.environment.TestEnvironment;
 import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
-import org.apache.flink.connector.testframe.junit.annotations.TestContext;
-import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
-import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
-import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
 import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
 import org.apache.flink.streaming.api.CheckpointingMode;
-import org.apache.flink.testutils.junit.FailsOnJava11;
 import org.apache.flink.util.CloseableIterator;
 
 import org.apache.pulsar.client.api.SubscriptionType;
-import org.junit.experimental.categories.Category;
 import org.junit.jupiter.api.Disabled;
 
 import java.util.List;
@@ -46,28 +35,15 @@ import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.
 import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
 
 /**
- * Unit test class for {@link PulsarSource}. Used for {@link SubscriptionType#Shared} subscription.
+ * Pulsar with {@link SubscriptionType#Key_Shared} and {@link SubscriptionType#Shared} consumes the
+ * message out of order. So we have to override the default connector test tool.
  */
-@Category(value = {FailsOnJava11.class})
-public class PulsarUnorderedSourceITCase extends SourceTestSuiteBase<String> {
-    // Defines test environment on Flink MiniCluster
-    @TestEnv MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment();
-
-    // Defines pulsar running environment
-    @TestExternalSystem
-    PulsarTestEnvironment pulsar = new PulsarTestEnvironment(PulsarRuntime.mock());
-
-    @TestSemantics
-    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
-
-    @TestContext
-    PulsarTestContextFactory<String, SharedSubscriptionConsumingContext> singleTopic =
-            new PulsarTestContextFactory<>(pulsar, SharedSubscriptionConsumingContext::new);
+public abstract class UnorderedSourceTestSuiteBase<T> extends SourceTestSuiteBase<T> {
 
     @Override
     protected void checkResultWithSemantic(
-            CloseableIterator<String> resultIterator,
-            List<List<String>> testData,
+            CloseableIterator<T> resultIterator,
+            List<List<T>> testData,
             CheckpointingMode semantic,
             Integer limit) {
         Runnable runnable =
@@ -83,7 +59,7 @@ public class PulsarUnorderedSourceITCase extends SourceTestSuiteBase<String> {
      * Shared subscription will have multiple readers on same partition, this would make hard to
      * automatically stop like a bounded source.
      */
-    private static int getExpectedSize(List<List<String>> testData, Integer limit) {
+    private static <T> int getExpectedSize(List<List<T>> testData, Integer limit) {
         if (limit == null) {
             return testData.stream().mapToInt(List::size).sum();
         } else {
@@ -95,7 +71,7 @@ public class PulsarUnorderedSourceITCase extends SourceTestSuiteBase<String> {
     @Disabled("We don't have any idle readers in Pulsar's shared subscription.")
     public void testIdleReader(
             TestEnvironment testEnv,
-            DataStreamSourceExternalContext<String> externalContext,
+            DataStreamSourceExternalContext<T> externalContext,
             CheckpointingMode semantic)
             throws Exception {
         super.testIdleReader(testEnv, externalContext, semantic);
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicConsumingContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/MultipleTopicConsumingContext.java
similarity index 52%
copy from flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicConsumingContext.java
copy to flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/MultipleTopicConsumingContext.java
index 57027f33e1b..ca6410fbc43 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicConsumingContext.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/MultipleTopicConsumingContext.java
@@ -16,43 +16,63 @@
  * limitations under the License.
  */
 
-package org.apache.flink.connector.pulsar.testutils.cases;
+package org.apache.flink.connector.pulsar.testutils.source.cases;
 
 import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment;
+import org.apache.flink.connector.pulsar.testutils.source.PulsarSourceTestContext;
 
 import org.apache.pulsar.client.api.SubscriptionType;
 
-import java.net.URL;
-import java.util.Collections;
-import java.util.List;
+import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic;
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition;
 
 /**
  * Pulsar external context that will create multiple topics with only one partitions as source
  * splits.
  */
-public class MultipleTopicConsumingContext extends MultipleTopicTemplateContext {
+public class MultipleTopicConsumingContext extends PulsarSourceTestContext {
+
+    private final String topicPrefix = "flink-multiple-topic-" + randomAlphabetic(8) + "-";
+    private final SubscriptionType subscriptionType;
+
+    private int index = 0;
 
     public MultipleTopicConsumingContext(PulsarTestEnvironment environment) {
-        this(environment, Collections.emptyList());
+        this(environment, SubscriptionType.Exclusive);
     }
 
     public MultipleTopicConsumingContext(
-            PulsarTestEnvironment environment, List<URL> connectorJarPaths) {
-        super(environment, connectorJarPaths);
+            PulsarTestEnvironment environment, SubscriptionType subscriptionType) {
+        super(environment);
+        this.subscriptionType = subscriptionType;
     }
 
     @Override
     protected String displayName() {
-        return "consuming message on multiple topic";
+        return "consume message on multiple topic";
+    }
+
+    @Override
+    protected String topicPattern() {
+        return topicPrefix + ".+";
     }
 
     @Override
     protected String subscriptionName() {
-        return "flink-pulsar-multiple-topic-test";
+        return "flink-multiple-topic-test";
     }
 
     @Override
     protected SubscriptionType subscriptionType() {
-        return SubscriptionType.Exclusive;
+        return subscriptionType;
+    }
+
+    @Override
+    protected String generatePartitionName() {
+        String topic = topicPrefix + index;
+        operator.createTopic(topic, 1);
+        index++;
+
+        return topicNameWithPartition(topic, 0);
     }
 }
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/SingleTopicConsumingContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/SingleTopicConsumingContext.java
new file mode 100644
index 00000000000..df5166926e0
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/SingleTopicConsumingContext.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.pulsar.testutils.source.cases;
+
+import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment;
+import org.apache.flink.connector.pulsar.testutils.source.PulsarSourceTestContext;
+
+import org.apache.pulsar.client.api.SubscriptionType;
+
+import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric;
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition;
+
+/**
+ * A Pulsar external context that will create only one topic and use partitions in that topic as
+ * source splits.
+ */
+public class SingleTopicConsumingContext extends PulsarSourceTestContext {
+
+    private final String topicName = "pulsar-single-topic-" + randomAlphanumeric(8);
+
+    private int index = 0;
+
+    public SingleTopicConsumingContext(PulsarTestEnvironment environment) {
+        super(environment);
+    }
+
+    @Override
+    protected String displayName() {
+        return "consume message on single topic";
+    }
+
+    @Override
+    protected String topicPattern() {
+        return topicName + ".+";
+    }
+
+    @Override
+    protected String subscriptionName() {
+        return "pulsar-single-topic-test";
+    }
+
+    @Override
+    protected SubscriptionType subscriptionType() {
+        return SubscriptionType.Exclusive;
+    }
+
+    @Override
+    protected String generatePartitionName() {
+        if (index == 0) {
+            operator.createTopic(topicName, index + 1);
+        } else {
+            operator.increaseTopicPartitions(topicName, index + 1);
+        }
+
+        return topicNameWithPartition(topicName, index++);
+    }
+}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/pom.xml
index b9bc5d4c2b1..7f7ab46ddda 100644
--- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/pom.xml
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/pom.xml
@@ -33,6 +33,7 @@ under the License.
 	<properties>
 		<pulsar.version>2.10.0</pulsar.version>
 		<bouncycastle.version>1.69</bouncycastle.version>
+		<jaxb-api.version>2.3.1</jaxb-api.version>
 	</properties>
 
 	<dependencies>
@@ -41,6 +42,17 @@ under the License.
 			<artifactId>flink-end-to-end-tests-common</artifactId>
 			<version>${project.version}</version>
 		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-test-utils</artifactId>
+			<version>${project.version}</version>
+			<exclusions>
+				<exclusion>
+					<groupId>com.google.guava</groupId>
+					<artifactId>guava</artifactId>
+				</exclusion>
+			</exclusions>
+		</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-connector-pulsar</artifactId>
@@ -70,9 +82,6 @@ under the License.
 			<plugin>
 				<groupId>org.apache.maven.plugins</groupId>
 				<artifactId>maven-surefire-plugin</artifactId>
-				<configuration>
-					<skip>true</skip>
-				</configuration>
 			</plugin>
 			<plugin>
 				<groupId>org.apache.maven.plugins</groupId>
@@ -88,6 +97,15 @@ under the License.
 				</executions>
 				<configuration>
 					<artifactItems>
+						<artifactItem>
+							<groupId>org.apache.flink</groupId>
+							<artifactId>flink-connector-test-utils</artifactId>
+							<version>${project.version}</version>
+							<destFileName>flink-connector-testing.jar</destFileName>
+							<type>jar</type>
+							<outputDirectory>${project.build.directory}/dependencies
+							</outputDirectory>
+						</artifactItem>
 						<artifactItem>
 							<groupId>org.apache.flink</groupId>
 							<artifactId>flink-connector-pulsar</artifactId>
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceOrderedE2ECase.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceOrderedE2ECase.java
index 234c1a01cd6..ea6a982044b 100644
--- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceOrderedE2ECase.java
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceOrderedE2ECase.java
@@ -25,10 +25,10 @@ import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem
 import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
 import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase;
 import org.apache.flink.streaming.api.CheckpointingMode;
-import org.apache.flink.tests.util.pulsar.cases.ExclusiveSubscriptionContext;
-import org.apache.flink.tests.util.pulsar.cases.FailoverSubscriptionContext;
 import org.apache.flink.tests.util.pulsar.common.FlinkContainerWithPulsarEnvironment;
 import org.apache.flink.tests.util.pulsar.common.PulsarContainerTestEnvironment;
+import org.apache.flink.tests.util.pulsar.source.ExclusiveSubscriptionContext;
+import org.apache.flink.tests.util.pulsar.source.FailoverSubscriptionContext;
 import org.apache.flink.testutils.junit.FailsOnJava11;
 
 import org.junit.experimental.categories.Category;
@@ -37,6 +37,7 @@ import org.junit.experimental.categories.Category;
  * Pulsar E2E test based on connector testing framework. It's used for Failover & Exclusive
  * subscription.
  */
+@SuppressWarnings("unused")
 @Category(value = {FailsOnJava11.class})
 public class PulsarSourceOrderedE2ECase extends SourceTestSuiteBase<String> {
 
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceUnorderedE2ECase.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceUnorderedE2ECase.java
index 50390486dd6..15333a1d538 100644
--- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceUnorderedE2ECase.java
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceUnorderedE2ECase.java
@@ -19,21 +19,26 @@
 package org.apache.flink.tests.util.pulsar;
 
 import org.apache.flink.connector.pulsar.testutils.PulsarTestContextFactory;
+import org.apache.flink.connector.pulsar.testutils.source.UnorderedSourceTestSuiteBase;
 import org.apache.flink.connector.testframe.junit.annotations.TestContext;
 import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
 import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
 import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
 import org.apache.flink.streaming.api.CheckpointingMode;
-import org.apache.flink.tests.util.pulsar.cases.KeySharedSubscriptionContext;
-import org.apache.flink.tests.util.pulsar.cases.SharedSubscriptionContext;
 import org.apache.flink.tests.util.pulsar.common.FlinkContainerWithPulsarEnvironment;
 import org.apache.flink.tests.util.pulsar.common.PulsarContainerTestEnvironment;
-import org.apache.flink.tests.util.pulsar.common.UnorderedSourceTestSuiteBase;
+import org.apache.flink.tests.util.pulsar.source.KeySharedSubscriptionContext;
+import org.apache.flink.tests.util.pulsar.source.SharedSubscriptionContext;
+import org.apache.flink.testutils.junit.FailsOnJava11;
+
+import org.junit.experimental.categories.Category;
 
 /**
  * Pulsar E2E test based on connector testing framework. It's used for Shared & Key_Shared
  * subscription.
  */
+@SuppressWarnings("unused")
+@Category(value = {FailsOnJava11.class})
 public class PulsarSourceUnorderedE2ECase extends UnorderedSourceTestSuiteBase<String> {
 
     // Defines the Semantic.
@@ -49,12 +54,10 @@ public class PulsarSourceUnorderedE2ECase extends UnorderedSourceTestSuiteBase<S
     PulsarContainerTestEnvironment pulsar = new PulsarContainerTestEnvironment(flink);
 
     // Defines a set of external context Factories for different test cases.
-    @SuppressWarnings("unused")
     @TestContext
     PulsarTestContextFactory<String, SharedSubscriptionContext> shared =
             new PulsarTestContextFactory<>(pulsar, SharedSubscriptionContext::new);
 
-    @SuppressWarnings("unused")
     @TestContext
     PulsarTestContextFactory<String, KeySharedSubscriptionContext> keyShared =
             new PulsarTestContextFactory<>(pulsar, KeySharedSubscriptionContext::new);
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/KeySharedSubscriptionContext.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/KeySharedSubscriptionContext.java
deleted file mode 100644
index 5ad369bcf03..00000000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/KeySharedSubscriptionContext.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.tests.util.pulsar.cases;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.connector.source.Boundedness;
-import org.apache.flink.api.connector.source.Source;
-import org.apache.flink.connector.pulsar.source.PulsarSource;
-import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder;
-import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor;
-import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils;
-import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange;
-import org.apache.flink.connector.pulsar.source.enumerator.topic.range.FixedRangeGenerator;
-import org.apache.flink.connector.pulsar.testutils.PulsarTestContext;
-import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment;
-import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter;
-import org.apache.flink.connector.testframe.external.source.TestingSourceSettings;
-import org.apache.flink.tests.util.pulsar.common.KeyedPulsarPartitionDataWriter;
-
-import org.apache.pulsar.client.api.RegexSubscriptionMode;
-import org.apache.pulsar.client.api.SubscriptionType;
-import org.apache.pulsar.common.util.Murmur3_32Hash;
-
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import static java.util.Collections.singletonList;
-import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic;
-import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.RANGE_SIZE;
-import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.pulsarSchema;
-import static org.apache.pulsar.client.api.Schema.STRING;
-
-/** We would consume from test splits by using {@link SubscriptionType#Key_Shared} subscription. */
-public class KeySharedSubscriptionContext extends PulsarTestContext<String> {
-
-    private int index = 0;
-
-    private final List<KeyedPulsarPartitionDataWriter> writers = new ArrayList<>();
-
-    // Message keys.
-    private final String key1;
-    private final String key2;
-
-    public KeySharedSubscriptionContext(PulsarTestEnvironment environment) {
-        this(environment, Collections.emptyList());
-    }
-
-    public KeySharedSubscriptionContext(
-            PulsarTestEnvironment environment, List<URL> connectorJarPaths) {
-        super(environment, connectorJarPaths);
-
-        // Init message keys.
-        this.key1 = randomAlphabetic(8);
-        String newKey2;
-        do {
-            newKey2 = randomAlphabetic(8);
-        } while (keyHash(key1) == keyHash(newKey2));
-        this.key2 = newKey2;
-    }
-
-    @Override
-    protected String displayName() {
-        return "consuming message by Key_Shared";
-    }
-
-    @Override
-    public Source<String, ?, ?> createSource(TestingSourceSettings sourceSettings) {
-        int keyHash = keyHash(key1);
-        TopicRange range = new TopicRange(keyHash, keyHash);
-
-        PulsarSourceBuilder<String> builder =
-                PulsarSource.builder()
-                        .setDeserializationSchema(pulsarSchema(STRING))
-                        .setServiceUrl(operator.serviceUrl())
-                        .setAdminUrl(operator.adminUrl())
-                        .setTopicPattern(
-                                "pulsar-[0-9]+-key-shared", RegexSubscriptionMode.AllTopics)
-                        .setSubscriptionType(SubscriptionType.Key_Shared)
-                        .setSubscriptionName("pulsar-key-shared")
-                        .setRangeGenerator(new FixedRangeGenerator(singletonList(range)));
-        if (sourceSettings.getBoundedness() == Boundedness.BOUNDED) {
-            // Using latest stop cursor for making sure the source could be stopped.
-            builder.setBoundedStopCursor(StopCursor.latest());
-        }
-
-        return builder.build();
-    }
-
-    @Override
-    public ExternalSystemSplitDataWriter<String> createSourceSplitDataWriter(
-            TestingSourceSettings sourceSettings) {
-        String topicName = "pulsar-" + index + "-key-shared";
-        operator.createTopic(topicName, 1);
-        index++;
-
-        String partitionName = TopicNameUtils.topicNameWithPartition(topicName, 0);
-        KeyedPulsarPartitionDataWriter writer =
-                new KeyedPulsarPartitionDataWriter(operator, partitionName, key1, key2);
-        writers.add(writer);
-
-        return writer;
-    }
-
-    @Override
-    public List<String> generateTestData(
-            TestingSourceSettings sourceSettings, int splitIndex, long seed) {
-        return generateStringTestData(splitIndex, seed);
-    }
-
-    @Override
-    public TypeInformation<String> getProducedType() {
-        return TypeInformation.of(String.class);
-    }
-
-    @Override
-    public void close() {
-        for (KeyedPulsarPartitionDataWriter writer : writers) {
-            writer.close();
-        }
-        writers.clear();
-    }
-
-    private int keyHash(String key) {
-        return Murmur3_32Hash.getInstance().makeHash(key.getBytes()) % RANGE_SIZE;
-    }
-}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/SharedSubscriptionContext.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/SharedSubscriptionContext.java
deleted file mode 100644
index 1a2db6694d1..00000000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/SharedSubscriptionContext.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.tests.util.pulsar.cases;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.connector.source.Boundedness;
-import org.apache.flink.api.connector.source.Source;
-import org.apache.flink.connector.pulsar.source.PulsarSource;
-import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder;
-import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor;
-import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils;
-import org.apache.flink.connector.pulsar.testutils.PulsarPartitionDataWriter;
-import org.apache.flink.connector.pulsar.testutils.PulsarTestContext;
-import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment;
-import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter;
-import org.apache.flink.connector.testframe.external.source.TestingSourceSettings;
-
-import org.apache.pulsar.client.api.RegexSubscriptionMode;
-import org.apache.pulsar.client.api.SubscriptionType;
-
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.pulsarSchema;
-import static org.apache.pulsar.client.api.Schema.STRING;
-
-/** We would consuming from test splits by using {@link SubscriptionType#Shared} subscription. */
-public class SharedSubscriptionContext extends PulsarTestContext<String> {
-
-    private int index = 0;
-
-    private final List<PulsarPartitionDataWriter> writers = new ArrayList<>();
-
-    public SharedSubscriptionContext(PulsarTestEnvironment environment) {
-        this(environment, Collections.emptyList());
-    }
-
-    public SharedSubscriptionContext(
-            PulsarTestEnvironment environment, List<URL> connectorJarPaths) {
-        super(environment, connectorJarPaths);
-    }
-
-    @Override
-    protected String displayName() {
-        return "consuming message by Shared";
-    }
-
-    @Override
-    public Source<String, ?, ?> createSource(TestingSourceSettings sourceSettings) {
-        PulsarSourceBuilder<String> builder =
-                PulsarSource.builder()
-                        .setDeserializationSchema(pulsarSchema(STRING))
-                        .setServiceUrl(operator.serviceUrl())
-                        .setAdminUrl(operator.adminUrl())
-                        .setTopicPattern("pulsar-[0-9]+-shared", RegexSubscriptionMode.AllTopics)
-                        .setSubscriptionType(SubscriptionType.Shared)
-                        .setSubscriptionName("pulsar-shared");
-        if (sourceSettings.getBoundedness() == Boundedness.BOUNDED) {
-            // Using latest stop cursor for making sure the source could be stopped.
-            builder.setBoundedStopCursor(StopCursor.latest());
-        }
-
-        return builder.build();
-    }
-
-    @Override
-    public ExternalSystemSplitDataWriter<String> createSourceSplitDataWriter(
-            TestingSourceSettings sourceSettings) {
-        String topicName = "pulsar-" + index + "-shared";
-        operator.createTopic(topicName, 1);
-        index++;
-
-        String partitionName = TopicNameUtils.topicNameWithPartition(topicName, 0);
-        PulsarPartitionDataWriter writer = new PulsarPartitionDataWriter(operator, partitionName);
-        writers.add(writer);
-
-        return writer;
-    }
-
-    @Override
-    public List<String> generateTestData(
-            TestingSourceSettings sourceSettings, int splitIndex, long seed) {
-        return generateStringTestData(splitIndex, seed);
-    }
-
-    @Override
-    public TypeInformation<String> getProducedType() {
-        return TypeInformation.of(String.class);
-    }
-
-    @Override
-    public void close() {
-        for (PulsarPartitionDataWriter writer : writers) {
-            writer.close();
-        }
-        writers.clear();
-    }
-}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/FlinkContainerWithPulsarEnvironment.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/FlinkContainerWithPulsarEnvironment.java
index 9f34554607e..f5e862f17c2 100644
--- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/FlinkContainerWithPulsarEnvironment.java
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/FlinkContainerWithPulsarEnvironment.java
@@ -43,7 +43,8 @@ public class FlinkContainerWithPulsarEnvironment extends FlinkContainerTestEnvir
                 resourcePath("bcutil-jdk15on.jar"),
                 resourcePath("bcprov-ext-jdk15on.jar"),
                 resourcePath("jaxb-api.jar"),
-                resourcePath("jul-to-slf4j.jar"));
+                resourcePath("jul-to-slf4j.jar"),
+                resourcePath("flink-connector-testing.jar"));
     }
 
     private static String resourcePath(String jarName) {
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/KeyedPulsarPartitionDataWriter.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/KeyedPulsarPartitionDataWriter.java
index e431e4c89d0..d5f6e11c00a 100644
--- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/KeyedPulsarPartitionDataWriter.java
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/KeyedPulsarPartitionDataWriter.java
@@ -35,23 +35,26 @@ public class KeyedPulsarPartitionDataWriter implements ExternalSystemSplitDataWr
 
     private final PulsarRuntimeOperator operator;
     private final String fullTopicName;
-    private final String key1;
-    private final String key2;
+    private final String keyToRead;
+    private final String keyToExclude;
 
     public KeyedPulsarPartitionDataWriter(
-            PulsarRuntimeOperator operator, String fullTopicName, String key1, String key2) {
+            PulsarRuntimeOperator operator,
+            String fullTopicName,
+            String keyToRead,
+            String keyToExclude) {
         this.operator = operator;
         this.fullTopicName = fullTopicName;
-        this.key1 = key1;
-        this.key2 = key2;
+        this.keyToRead = keyToRead;
+        this.keyToExclude = keyToExclude;
     }
 
     @Override
     public void writeRecords(List<String> records) {
-        operator.sendMessages(fullTopicName, Schema.STRING, key1, records);
+        List<String> newRecords = records.stream().map(a -> a + keyToRead).collect(toList());
 
-        List<String> newRecords = records.stream().map(a -> a + key1).collect(toList());
-        operator.sendMessages(fullTopicName, Schema.STRING, key2, newRecords);
+        operator.sendMessages(fullTopicName, Schema.STRING, keyToExclude, newRecords);
+        operator.sendMessages(fullTopicName, Schema.STRING, keyToRead, records);
     }
 
     @Override
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/UnorderedSourceTestSuiteBase.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/UnorderedSourceTestSuiteBase.java
deleted file mode 100644
index 01527ea5482..00000000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/UnorderedSourceTestSuiteBase.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.tests.util.pulsar.common;
-
-import org.apache.flink.api.common.eventtime.WatermarkStrategy;
-import org.apache.flink.api.connector.source.Boundedness;
-import org.apache.flink.api.connector.source.Source;
-import org.apache.flink.connector.testframe.environment.TestEnvironment;
-import org.apache.flink.connector.testframe.environment.TestEnvironmentSettings;
-import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter;
-import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext;
-import org.apache.flink.connector.testframe.external.source.TestingSourceSettings;
-import org.apache.flink.connector.testframe.junit.extensions.ConnectorTestingExtension;
-import org.apache.flink.connector.testframe.junit.extensions.TestCaseInvocationContextProvider;
-import org.apache.flink.streaming.api.CheckpointingMode;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.util.TestLoggerExtension;
-
-import org.junit.jupiter.api.DisplayName;
-import org.junit.jupiter.api.TestInstance;
-import org.junit.jupiter.api.TestTemplate;
-import org.junit.jupiter.api.extension.ExtendWith;
-
-import java.util.List;
-import java.util.concurrent.ThreadLocalRandom;
-
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-
-/** A source test template for testing the messages which could be consumed in a unordered way. */
-@ExtendWith({
-    ConnectorTestingExtension.class,
-    TestLoggerExtension.class,
-    TestCaseInvocationContextProvider.class
-})
-@TestInstance(TestInstance.Lifecycle.PER_CLASS)
-public abstract class UnorderedSourceTestSuiteBase<T> {
-
-    @TestTemplate
-    @DisplayName("Test source with one split and four consumers")
-    public void testOneSplitWithMultipleConsumers(
-            TestEnvironment testEnv, DataStreamSourceExternalContext<T> externalContext)
-            throws Exception {
-        TestingSourceSettings sourceSettings =
-                TestingSourceSettings.builder()
-                        .setBoundedness(Boundedness.BOUNDED)
-                        .setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE)
-                        .build();
-        TestEnvironmentSettings envOptions =
-                TestEnvironmentSettings.builder()
-                        .setConnectorJarPaths(externalContext.getConnectorJarPaths())
-                        .build();
-        List<T> testData =
-                externalContext.generateTestData(
-                        sourceSettings, 0, ThreadLocalRandom.current().nextLong());
-        ExternalSystemSplitDataWriter<T> writer =
-                externalContext.createSourceSplitDataWriter(sourceSettings);
-        writer.writeRecords(testData);
-
-        Source<T, ?, ?> source = externalContext.createSource(sourceSettings);
-        StreamExecutionEnvironment execEnv = testEnv.createExecutionEnvironment(envOptions);
-        List<T> results =
-                execEnv.fromSource(source, WatermarkStrategy.noWatermarks(), "Pulsar source")
-                        .setParallelism(4)
-                        .executeAndCollect(
-                                "Source single split with four readers.", testData.size());
-
-        assertThat(results, containsInAnyOrder(testData.toArray()));
-    }
-}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/ExclusiveSubscriptionContext.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/ExclusiveSubscriptionContext.java
similarity index 71%
rename from flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/ExclusiveSubscriptionContext.java
rename to flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/ExclusiveSubscriptionContext.java
index 6fea0c9c9e5..4906ad6cc7e 100644
--- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/ExclusiveSubscriptionContext.java
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/ExclusiveSubscriptionContext.java
@@ -16,32 +16,23 @@
  * limitations under the License.
  */
 
-package org.apache.flink.tests.util.pulsar.cases;
+package org.apache.flink.tests.util.pulsar.source;
 
 import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment;
-import org.apache.flink.connector.pulsar.testutils.cases.MultipleTopicTemplateContext;
+import org.apache.flink.connector.pulsar.testutils.source.cases.MultipleTopicConsumingContext;
 
 import org.apache.pulsar.client.api.SubscriptionType;
 
-import java.net.URL;
-import java.util.Collections;
-import java.util.List;
-
 /** We would consume from test splits by using {@link SubscriptionType#Exclusive} subscription. */
-public class ExclusiveSubscriptionContext extends MultipleTopicTemplateContext {
+public class ExclusiveSubscriptionContext extends MultipleTopicConsumingContext {
 
     public ExclusiveSubscriptionContext(PulsarTestEnvironment environment) {
-        this(environment, Collections.emptyList());
-    }
-
-    public ExclusiveSubscriptionContext(
-            PulsarTestEnvironment environment, List<URL> connectorJarPaths) {
-        super(environment, connectorJarPaths);
+        super(environment);
     }
 
     @Override
     protected String displayName() {
-        return "consuming message by Exclusive";
+        return "consume message by Exclusive";
     }
 
     @Override
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/FailoverSubscriptionContext.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/FailoverSubscriptionContext.java
similarity index 71%
rename from flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/FailoverSubscriptionContext.java
rename to flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/FailoverSubscriptionContext.java
index c47348861c1..3134db4bdda 100644
--- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/FailoverSubscriptionContext.java
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/FailoverSubscriptionContext.java
@@ -16,32 +16,23 @@
  * limitations under the License.
  */
 
-package org.apache.flink.tests.util.pulsar.cases;
+package org.apache.flink.tests.util.pulsar.source;
 
 import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment;
-import org.apache.flink.connector.pulsar.testutils.cases.MultipleTopicTemplateContext;
+import org.apache.flink.connector.pulsar.testutils.source.cases.MultipleTopicConsumingContext;
 
 import org.apache.pulsar.client.api.SubscriptionType;
 
-import java.net.URL;
-import java.util.Collections;
-import java.util.List;
-
 /** We would consume from test splits by using {@link SubscriptionType#Failover} subscription. */
-public class FailoverSubscriptionContext extends MultipleTopicTemplateContext {
+public class FailoverSubscriptionContext extends MultipleTopicConsumingContext {
 
     public FailoverSubscriptionContext(PulsarTestEnvironment environment) {
-        this(environment, Collections.emptyList());
-    }
-
-    public FailoverSubscriptionContext(
-            PulsarTestEnvironment environment, List<URL> connectorJarPaths) {
-        super(environment, connectorJarPaths);
+        super(environment);
     }
 
     @Override
     protected String displayName() {
-        return "consuming message by Failover";
+        return "consume message by Failover";
     }
 
     @Override
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/KeySharedSubscriptionContext.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/KeySharedSubscriptionContext.java
new file mode 100644
index 00000000000..0cae6e58405
--- /dev/null
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/KeySharedSubscriptionContext.java
@@ -0,0 +1,87 @@
+/*
+ * 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.tests.util.pulsar.source;
+
+import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder;
+import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange;
+import org.apache.flink.connector.pulsar.source.enumerator.topic.range.FixedRangeGenerator;
+import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment;
+import org.apache.flink.connector.pulsar.testutils.source.cases.MultipleTopicConsumingContext;
+import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter;
+import org.apache.flink.connector.testframe.external.source.TestingSourceSettings;
+import org.apache.flink.tests.util.pulsar.common.KeyedPulsarPartitionDataWriter;
+
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.common.util.Murmur3_32Hash;
+
+import static java.util.Collections.singletonList;
+import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic;
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.RANGE_SIZE;
+import static org.apache.pulsar.client.api.SubscriptionType.Key_Shared;
+
+/** We would consume from test splits by using {@link SubscriptionType#Key_Shared} subscription. */
+public class KeySharedSubscriptionContext extends MultipleTopicConsumingContext {
+
+    private final String keyToRead;
+    private final String keyToExclude;
+
+    public KeySharedSubscriptionContext(PulsarTestEnvironment environment) {
+        super(environment, Key_Shared);
+
+        this.keyToRead = randomAlphabetic(8);
+
+        // Make sure they have different hash code.
+        int readHash = keyHash(keyToRead);
+        String randomKey;
+        do {
+            randomKey = randomAlphabetic(8);
+        } while (keyHash(randomKey) == readHash);
+        this.keyToExclude = randomKey;
+    }
+
+    @Override
+    public ExternalSystemSplitDataWriter<String> createSourceSplitDataWriter(
+            TestingSourceSettings sourceSettings) {
+        String partitionName = generatePartitionName();
+        return new KeyedPulsarPartitionDataWriter(operator, partitionName, keyToRead, keyToExclude);
+    }
+
+    @Override
+    protected String displayName() {
+        return "consume message by Key_Shared";
+    }
+
+    @Override
+    protected void setSourceBuilder(PulsarSourceBuilder<String> builder) {
+        int keyHash = keyHash(keyToRead);
+        TopicRange range = new TopicRange(keyHash, keyHash);
+
+        builder.setRangeGenerator(new FixedRangeGenerator(singletonList(range)));
+    }
+
+    @Override
+    protected String subscriptionName() {
+        return "pulsar-key-shared-subscription";
+    }
+
+    // This method is copied from Pulsar for calculating message key hash.
+    private int keyHash(String key) {
+        return Murmur3_32Hash.getInstance().makeHash(key.getBytes()) % RANGE_SIZE;
+    }
+}
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicConsumingContext.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/SharedSubscriptionContext.java
similarity index 55%
rename from flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicConsumingContext.java
rename to flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/SharedSubscriptionContext.java
index 57027f33e1b..fe9f07832af 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicConsumingContext.java
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/SharedSubscriptionContext.java
@@ -16,43 +16,29 @@
  * limitations under the License.
  */
 
-package org.apache.flink.connector.pulsar.testutils.cases;
+package org.apache.flink.tests.util.pulsar.source;
 
 import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment;
+import org.apache.flink.connector.pulsar.testutils.source.cases.MultipleTopicConsumingContext;
 
 import org.apache.pulsar.client.api.SubscriptionType;
 
-import java.net.URL;
-import java.util.Collections;
-import java.util.List;
+import static org.apache.pulsar.client.api.SubscriptionType.Shared;
 
-/**
- * Pulsar external context that will create multiple topics with only one partitions as source
- * splits.
- */
-public class MultipleTopicConsumingContext extends MultipleTopicTemplateContext {
-
-    public MultipleTopicConsumingContext(PulsarTestEnvironment environment) {
-        this(environment, Collections.emptyList());
-    }
+/** We would consume from test splits by using {@link SubscriptionType#Shared} subscription. */
+public class SharedSubscriptionContext extends MultipleTopicConsumingContext {
 
-    public MultipleTopicConsumingContext(
-            PulsarTestEnvironment environment, List<URL> connectorJarPaths) {
-        super(environment, connectorJarPaths);
+    public SharedSubscriptionContext(PulsarTestEnvironment environment) {
+        super(environment, Shared);
     }
 
     @Override
     protected String displayName() {
-        return "consuming message on multiple topic";
+        return "consume message by Shared";
     }
 
     @Override
     protected String subscriptionName() {
-        return "flink-pulsar-multiple-topic-test";
-    }
-
-    @Override
-    protected SubscriptionType subscriptionType() {
-        return SubscriptionType.Exclusive;
+        return "pulsar-shared-subscription";
     }
 }
diff --git a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/source/enumerator/NoOpEnumStateSerializer.java b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/source/enumerator/NoOpEnumStateSerializer.java
index 7be0e8d2d88..0628e632f19 100644
--- a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/source/enumerator/NoOpEnumStateSerializer.java
+++ b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/source/enumerator/NoOpEnumStateSerializer.java
@@ -22,7 +22,7 @@ import org.apache.flink.core.io.SimpleVersionedSerializer;
 
 import java.io.IOException;
 
-/** Mock enumerator state seializer. */
+/** Mock enumerator state serializer. */
 public class NoOpEnumStateSerializer implements SimpleVersionedSerializer<NoOpEnumState> {
     @Override
     public int getVersion() {


[flink] 06/06: [FLINK-26182][Connector/pulsar] Create e2e tests for the Pulsar source and sink based on the connector testing framework.

Posted by ti...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

tison pushed a commit to branch release-1.16
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 64c284f5f9526517459f115f7008b3dd8df9d705
Author: Yufan Sheng <yu...@streamnative.io>
AuthorDate: Wed Sep 28 15:28:14 2022 +0800

    [FLINK-26182][Connector/pulsar] Create e2e tests for the Pulsar source and sink based on the connector testing framework.
---
 .../f4d91193-72ba-4ce4-ad83-98f780dce581           |   6 +
 flink-connectors/flink-connector-pulsar/pom.xml    |   2 -
 .../connector/pulsar/sink/PulsarSinkITCase.java    | 151 +++++++++++++--------
 .../pulsar/source/PulsarUnorderedSourceITCase.java |  14 +-
 .../pulsar/testutils/PulsarTestCommonUtils.java    |   5 +-
 .../pulsar/testutils/PulsarTestContext.java        |   6 +-
 .../testutils/runtime/PulsarRuntimeOperator.java   |  15 +-
 .../runtime/container/PulsarContainerRuntime.java  |  25 ++--
 .../testutils/runtime/mock/PulsarMockRuntime.java  |   5 +-
 .../testutils/sink/PulsarPartitionDataReader.java  |  93 +++++++++++++
 .../testutils/sink/PulsarSinkTestContext.java      | 124 +++++++++++++++++
 .../testutils/sink/PulsarSinkTestSuiteBase.java    |  37 +++++
 .../source}/KeyedPulsarPartitionDataWriter.java    |   6 +-
 .../source/UnorderedSourceTestSuiteBase.java       |   5 +-
 .../cases}/KeySharedSubscriptionContext.java       |  32 ++---
 .../cases/MultipleTopicConsumingContext.java       |   9 +-
 .../source/cases}/SharedSubscriptionContext.java   |  12 +-
 .../src/test/resources/docker/bootstrap.sh         |  28 ----
 ...norderedE2ECase.java => PulsarSinkE2ECase.java} |  31 ++---
 .../util/pulsar/PulsarSourceUnorderedE2ECase.java  |   4 +-
 .../FlinkContainerWithPulsarEnvironment.java       |   5 +-
 21 files changed, 434 insertions(+), 181 deletions(-)

diff --git a/flink-connectors/flink-connector-pulsar/archunit-violations/f4d91193-72ba-4ce4-ad83-98f780dce581 b/flink-connectors/flink-connector-pulsar/archunit-violations/f4d91193-72ba-4ce4-ad83-98f780dce581
index 03d0ae583af..40e7dc9686c 100644
--- a/flink-connectors/flink-connector-pulsar/archunit-violations/f4d91193-72ba-4ce4-ad83-98f780dce581
+++ b/flink-connectors/flink-connector-pulsar/archunit-violations/f4d91193-72ba-4ce4-ad83-98f780dce581
@@ -1,3 +1,9 @@
+org.apache.flink.connector.pulsar.sink.PulsarSinkITCase does not satisfy: only one of the following predicates match:\
+* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
+* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\
+* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\
+* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\
+ or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule
 org.apache.flink.connector.pulsar.source.PulsarSourceITCase does not satisfy: only one of the following predicates match:\
 * reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\
 * reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\
diff --git a/flink-connectors/flink-connector-pulsar/pom.xml b/flink-connectors/flink-connector-pulsar/pom.xml
index bfba4068ee8..a9f4169a3c2 100644
--- a/flink-connectors/flink-connector-pulsar/pom.xml
+++ b/flink-connectors/flink-connector-pulsar/pom.xml
@@ -300,7 +300,6 @@ under the License.
 								<include>**/testutils/**</include>
 								<include>META-INF/LICENSE</include>
 								<include>META-INF/NOTICE</include>
-								<include>containers/txnStandalone.conf</include>
 							</includes>
 						</configuration>
 					</execution>
@@ -324,7 +323,6 @@ under the License.
 								<include>**/testutils/**</include>
 								<include>META-INF/LICENSE</include>
 								<include>META-INF/NOTICE</include>
-								<include>containers/txnStandalone.conf</include>
 							</includes>
 						</configuration>
 					</execution>
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/PulsarSinkITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/PulsarSinkITCase.java
index 3fccc28c706..b0ef390b728 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/PulsarSinkITCase.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/PulsarSinkITCase.java
@@ -20,16 +20,28 @@ package org.apache.flink.connector.pulsar.sink;
 
 import org.apache.flink.api.common.serialization.SimpleStringSchema;
 import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.pulsar.testutils.PulsarTestContextFactory;
+import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment;
 import org.apache.flink.connector.pulsar.testutils.PulsarTestSuiteBase;
 import org.apache.flink.connector.pulsar.testutils.function.ControlSource;
+import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime;
+import org.apache.flink.connector.pulsar.testutils.sink.PulsarSinkTestContext;
+import org.apache.flink.connector.pulsar.testutils.sink.PulsarSinkTestSuiteBase;
+import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
 import org.apache.flink.runtime.minicluster.RpcServiceSharing;
 import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.CheckpointingMode;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.test.junit5.MiniClusterExtension;
 import org.apache.flink.testutils.junit.FailsOnJava11;
 import org.apache.flink.testutils.junit.SharedObjectsExtension;
 
 import org.junit.experimental.categories.Category;
+import org.junit.jupiter.api.Nested;
 import org.junit.jupiter.api.extension.RegisterExtension;
 import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.EnumSource;
@@ -44,63 +56,90 @@ import static org.assertj.core.api.Assertions.assertThat;
 
 /** Tests for using PulsarSink writing to a Pulsar cluster. */
 @Category(value = {FailsOnJava11.class})
-class PulsarSinkITCase extends PulsarTestSuiteBase {
-
-    private static final int PARALLELISM = 1;
-
-    @RegisterExtension
-    private static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
-            new MiniClusterExtension(
-                    new MiniClusterResourceConfiguration.Builder()
-                            .setNumberTaskManagers(1)
-                            .setNumberSlotsPerTaskManager(PARALLELISM)
-                            .setRpcServiceSharing(RpcServiceSharing.DEDICATED)
-                            .withHaLeadershipControl()
-                            .build());
-
-    // Using this extension for creating shared reference which would be used in source function.
-    @RegisterExtension final SharedObjectsExtension sharedObjects = SharedObjectsExtension.create();
-
-    @ParameterizedTest
-    @EnumSource(DeliveryGuarantee.class)
-    void writeRecordsToPulsar(DeliveryGuarantee guarantee) throws Exception {
-        // A random topic with partition 4.
-        String topic = randomAlphabetic(8);
-        operator().createTopic(topic, 4);
-        int counts = ThreadLocalRandom.current().nextInt(100, 200);
-
-        ControlSource source =
-                new ControlSource(
-                        sharedObjects,
-                        operator(),
-                        topic,
-                        guarantee,
-                        counts,
-                        Duration.ofMillis(50),
-                        Duration.ofMinutes(5));
-        PulsarSink<String> sink =
-                PulsarSink.builder()
-                        .setServiceUrl(operator().serviceUrl())
-                        .setAdminUrl(operator().adminUrl())
-                        .setDeliveryGuarantee(guarantee)
-                        .setTopics(topic)
-                        .setSerializationSchema(flinkSchema(new SimpleStringSchema()))
-                        .build();
-
-        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
-        env.setParallelism(PARALLELISM);
-        if (guarantee != DeliveryGuarantee.NONE) {
-            env.enableCheckpointing(500L);
-        }
-        env.addSource(source).sinkTo(sink);
-        env.execute();
+class PulsarSinkITCase {
+
+    /** Integration test based on connector testing framework. */
+    @Nested
+    class IntegrationTest extends PulsarSinkTestSuiteBase {
+
+        @TestEnv MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment();
+
+        @TestExternalSystem
+        PulsarTestEnvironment pulsar = new PulsarTestEnvironment(PulsarRuntime.mock());
+
+        @TestSemantics
+        CheckpointingMode[] semantics =
+                new CheckpointingMode[] {
+                    CheckpointingMode.EXACTLY_ONCE, CheckpointingMode.AT_LEAST_ONCE
+                };
+
+        @TestContext
+        PulsarTestContextFactory<String, PulsarSinkTestContext> sinkContext =
+                new PulsarTestContextFactory<>(pulsar, PulsarSinkTestContext::new);
+    }
+
+    /** Tests for using PulsarSink writing to a Pulsar cluster. */
+    @Nested
+    class DeliveryGuaranteeTest extends PulsarTestSuiteBase {
 
-        List<String> expectedRecords = source.getExpectedRecords();
-        List<String> consumedRecords = source.getConsumedRecords();
+        private static final int PARALLELISM = 1;
 
-        assertThat(consumedRecords)
-                .hasSameSizeAs(expectedRecords)
-                .containsExactlyInAnyOrderElementsOf(expectedRecords);
+        @RegisterExtension
+        private final MiniClusterExtension clusterExtension =
+                new MiniClusterExtension(
+                        new MiniClusterResourceConfiguration.Builder()
+                                .setNumberTaskManagers(1)
+                                .setNumberSlotsPerTaskManager(PARALLELISM)
+                                .setRpcServiceSharing(RpcServiceSharing.DEDICATED)
+                                .withHaLeadershipControl()
+                                .build());
+
+        // Using this extension for creating shared reference which would be used in source
+        // function.
+        @RegisterExtension
+        final SharedObjectsExtension sharedObjects = SharedObjectsExtension.create();
+
+        @ParameterizedTest
+        @EnumSource(DeliveryGuarantee.class)
+        void writeRecordsToPulsar(DeliveryGuarantee guarantee) throws Exception {
+            // A random topic with partition 4.
+            String topic = randomAlphabetic(8);
+            operator().createTopic(topic, 4);
+            int counts = ThreadLocalRandom.current().nextInt(100, 200);
+
+            ControlSource source =
+                    new ControlSource(
+                            sharedObjects,
+                            operator(),
+                            topic,
+                            guarantee,
+                            counts,
+                            Duration.ofMillis(50),
+                            Duration.ofMinutes(5));
+            PulsarSink<String> sink =
+                    PulsarSink.builder()
+                            .setServiceUrl(operator().serviceUrl())
+                            .setAdminUrl(operator().adminUrl())
+                            .setDeliveryGuarantee(guarantee)
+                            .setTopics(topic)
+                            .setSerializationSchema(flinkSchema(new SimpleStringSchema()))
+                            .build();
+
+            StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+
+            env.setParallelism(PARALLELISM);
+            if (guarantee != DeliveryGuarantee.NONE) {
+                env.enableCheckpointing(500L);
+            }
+            env.addSource(source).sinkTo(sink);
+            env.execute();
+
+            List<String> expectedRecords = source.getExpectedRecords();
+            List<String> consumedRecords = source.getConsumedRecords();
+
+            assertThat(consumedRecords)
+                    .hasSameSizeAs(expectedRecords)
+                    .containsExactlyInAnyOrderElementsOf(expectedRecords);
+        }
     }
 }
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarUnorderedSourceITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarUnorderedSourceITCase.java
index d18c6c2633b..070a16126a7 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarUnorderedSourceITCase.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarUnorderedSourceITCase.java
@@ -22,7 +22,8 @@ import org.apache.flink.connector.pulsar.testutils.PulsarTestContextFactory;
 import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment;
 import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime;
 import org.apache.flink.connector.pulsar.testutils.source.UnorderedSourceTestSuiteBase;
-import org.apache.flink.connector.pulsar.testutils.source.cases.MultipleTopicConsumingContext;
+import org.apache.flink.connector.pulsar.testutils.source.cases.KeySharedSubscriptionContext;
+import org.apache.flink.connector.pulsar.testutils.source.cases.SharedSubscriptionContext;
 import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment;
 import org.apache.flink.connector.testframe.junit.annotations.TestContext;
 import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
@@ -34,8 +35,6 @@ import org.apache.flink.testutils.junit.FailsOnJava11;
 import org.apache.pulsar.client.api.SubscriptionType;
 import org.junit.experimental.categories.Category;
 
-import static org.apache.pulsar.client.api.SubscriptionType.Shared;
-
 /**
  * Unit test class for {@link PulsarSource}. Used for {@link SubscriptionType#Shared} subscription.
  */
@@ -53,7 +52,10 @@ public class PulsarUnorderedSourceITCase extends UnorderedSourceTestSuiteBase<St
     CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
 
     @TestContext
-    PulsarTestContextFactory<String, MultipleTopicConsumingContext> multipleTopic =
-            new PulsarTestContextFactory<>(
-                    pulsar, env -> new MultipleTopicConsumingContext(env, Shared));
+    PulsarTestContextFactory<String, SharedSubscriptionContext> sharedSubscription =
+            new PulsarTestContextFactory<>(pulsar, SharedSubscriptionContext::new);
+
+    @TestContext
+    PulsarTestContextFactory<String, KeySharedSubscriptionContext> keySharedSubscription =
+            new PulsarTestContextFactory<>(pulsar, KeySharedSubscriptionContext::new);
 }
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestCommonUtils.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestCommonUtils.java
index 96b1ca62f2a..994601329d6 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestCommonUtils.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestCommonUtils.java
@@ -22,7 +22,6 @@ import org.apache.flink.api.connector.source.Boundedness;
 import org.apache.flink.connector.base.DeliveryGuarantee;
 import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition;
-import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange;
 import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit;
 import org.apache.flink.streaming.api.CheckpointingMode;
 
@@ -59,9 +58,7 @@ public class PulsarTestCommonUtils {
 
     public static PulsarPartitionSplit createPartitionSplit(
             String topic, int partitionId, Boundedness boundedness, MessageId latestConsumedId) {
-        TopicPartition topicPartition =
-                new TopicPartition(topic, partitionId, TopicRange.createFullRange());
-
+        TopicPartition topicPartition = new TopicPartition(topic, partitionId);
         StopCursor stopCursor =
                 boundedness == Boundedness.BOUNDED ? StopCursor.latest() : StopCursor.never();
         return new PulsarPartitionSplit(topicPartition, stopCursor, latestConsumedId, null);
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContext.java
index a4e44441bfd..1408656f986 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContext.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContext.java
@@ -52,10 +52,12 @@ public abstract class PulsarTestContext<T> implements ExternalContext {
 
     @Override
     public List<URL> getConnectorJarPaths() {
-        // We don't need any tests jar definition. They are provided in docker-related environments.
+        // We don't need any test jars definition. They are provided in docker-related environments.
         return Collections.emptyList();
     }
 
     @Override
-    public void close() throws Exception {}
+    public void close() throws Exception {
+        // All the topics would be deleted in the PulsarRuntime. No need to manually close them.
+    }
 }
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java
index 9787a8ceeb9..5d14d7ee081 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java
@@ -22,7 +22,6 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.connector.base.DeliveryGuarantee;
 import org.apache.flink.connector.pulsar.common.config.PulsarConfiguration;
 import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition;
-import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange;
 import org.apache.flink.connector.testframe.external.ExternalContext;
 
 import org.apache.flink.shaded.guava30.com.google.common.base.Strings;
@@ -48,7 +47,6 @@ import org.apache.pulsar.common.naming.TopicName;
 import org.apache.pulsar.common.partition.PartitionedTopicMetadata;
 
 import java.io.Closeable;
-import java.io.IOException;
 import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -228,12 +226,7 @@ public class PulsarRuntimeOperator implements Closeable {
     public List<TopicPartition> topicInfo(String topic) {
         try {
             return client().getPartitionsForTopic(topic).get().stream()
-                    .map(
-                            p ->
-                                    new TopicPartition(
-                                            topic,
-                                            TopicName.getPartitionIndex(p),
-                                            TopicRange.createFullRange()))
+                    .map(p -> new TopicPartition(topic, TopicName.getPartitionIndex(p)))
                     .collect(toList());
         } catch (InterruptedException | ExecutionException e) {
             throw new IllegalStateException(e);
@@ -452,7 +445,7 @@ public class PulsarRuntimeOperator implements Closeable {
 
     /** This method is used for test framework. You can't close this operator manually. */
     @Override
-    public void close() throws IOException {
+    public void close() throws PulsarClientException {
         if (admin != null) {
             admin.close();
         }
@@ -485,7 +478,7 @@ public class PulsarRuntimeOperator implements Closeable {
         }
     }
 
-    private synchronized <T> Producer<T> createProducer(String topic, Schema<T> schema) {
+    private <T> Producer<T> createProducer(String topic, Schema<T> schema) {
         ProducerBuilder<T> builder =
                 client().newProducer(schema)
                         .topic(topic)
@@ -496,7 +489,7 @@ public class PulsarRuntimeOperator implements Closeable {
         return sneakyClient(builder::create);
     }
 
-    private synchronized <T> Consumer<T> createConsumer(String topic, Schema<T> schema) {
+    private <T> Consumer<T> createConsumer(String topic, Schema<T> schema) {
         // Create the earliest subscription if it's not existed.
         List<String> subscriptions = sneakyAdmin(() -> admin().topics().getSubscriptions(topic));
         if (!subscriptions.contains(SUBSCRIPTION_NAME)) {
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java
index 910985338b9..eac42f063bc 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java
@@ -24,7 +24,6 @@ import org.apache.flink.util.DockerImageVersions;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.testcontainers.containers.BindMode;
 import org.testcontainers.containers.GenericContainer;
 import org.testcontainers.containers.PulsarContainer;
 import org.testcontainers.containers.output.Slf4jLogConsumer;
@@ -88,22 +87,30 @@ public class PulsarContainerRuntime implements PulsarRuntime {
             return;
         }
 
-        // Override the default configuration in container for enabling the Pulsar transaction.
-        container.withClasspathResourceMapping(
-                "docker/bootstrap.sh", "/pulsar/bin/bootstrap.sh", BindMode.READ_ONLY);
-        // Waiting for the Pulsar border is ready.
+        // Override the default standalone configuration by system environments.
+        container.withEnv("PULSAR_PREFIX_transactionCoordinatorEnabled", "true");
+        container.withEnv("PULSAR_PREFIX_acknowledgmentAtBatchIndexLevelEnabled", "true");
+        container.withEnv("PULSAR_PREFIX_systemTopicEnabled", "true");
+        container.withEnv("PULSAR_PREFIX_brokerDeduplicationEnabled", "true");
+        container.withEnv("PULSAR_PREFIX_defaultNumberOfNamespaceBundles", "1");
+        // Change the default bootstrap script, it will override the default configuration
+        // and start a standalone Pulsar without streaming storage and function worker.
+        container.withCommand(
+                "sh",
+                "-c",
+                "/pulsar/bin/apply-config-from-env.py /pulsar/conf/standalone.conf && /pulsar/bin/pulsar standalone --no-functions-worker -nss");
+        // Waiting for the Pulsar broker and the transaction is ready after the container started.
         container.waitingFor(
-                forHttp("/admin/v2/namespaces/public/default")
+                forHttp(
+                                "/admin/v2/persistent/pulsar/system/transaction_coordinator_assign/partitions")
                         .forPort(BROKER_HTTP_PORT)
                         .forStatusCode(200)
                         .withStartupTimeout(Duration.ofMinutes(5)));
-        // Set custom startup script.
-        container.withCommand("sh /pulsar/bin/bootstrap.sh");
 
         // Start the Pulsar Container.
         container.start();
         // Append the output to this runtime logger. Used for local debug purpose.
-        container.followOutput(new Slf4jLogConsumer(LOG).withSeparateOutputStreams());
+        container.followOutput(new Slf4jLogConsumer(LOG, true));
 
         // Create the operator.
         if (boundFlink) {
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/PulsarMockRuntime.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/PulsarMockRuntime.java
index 326c3ff6a89..b0df4c5d1ff 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/PulsarMockRuntime.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/PulsarMockRuntime.java
@@ -64,10 +64,10 @@ public class PulsarMockRuntime implements PulsarRuntime {
     @Override
     public void tearDown() {
         try {
-            pulsarService.close();
             if (operator != null) {
                 operator.close();
             }
+            pulsarService.close();
         } catch (Exception e) {
             throw new IllegalStateException(e);
         }
@@ -95,7 +95,7 @@ public class PulsarMockRuntime implements PulsarRuntime {
         configuration.setAuthorizationEnabled(false);
         configuration.setAllowAutoTopicCreation(true);
         configuration.setBrokerDeleteInactiveTopicsEnabled(false);
-
+        configuration.setTopicLevelPoliciesEnabled(true);
         configuration.setWebSocketServiceEnabled(false);
         // Use runtime dynamic ports
         configuration.setBrokerServicePort(Optional.of(0));
@@ -107,6 +107,7 @@ public class PulsarMockRuntime implements PulsarRuntime {
         configuration.setTransactionCoordinatorEnabled(true);
         configuration.setTransactionMetadataStoreProviderClassName(
                 "org.apache.pulsar.transaction.coordinator.impl.MLTransactionMetadataStoreProvider");
+        configuration.setDefaultNumberOfNamespaceBundles(1);
 
         return configuration;
     }
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarPartitionDataReader.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarPartitionDataReader.java
new file mode 100644
index 00000000000..6ef9510d116
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarPartitionDataReader.java
@@ -0,0 +1,93 @@
+/*
+ * 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.testutils.sink;
+
+import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator;
+import org.apache.flink.connector.testframe.external.ExternalSystemDataReader;
+
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionMode;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric;
+
+/** The data reader for a specified topic partition from Pulsar. */
+public class PulsarPartitionDataReader<T> implements ExternalSystemDataReader<T>, Closeable {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PulsarPartitionDataReader.class);
+
+    private final Consumer<T> consumer;
+
+    public PulsarPartitionDataReader(
+            PulsarRuntimeOperator operator, String fullTopicName, Schema<T> schema)
+            throws PulsarClientException {
+        // Create client for supporting the use in E2E test.
+        String subscriptionName = randomAlphanumeric(12);
+        this.consumer =
+                operator.client()
+                        .newConsumer(schema)
+                        .topic(fullTopicName)
+                        .subscriptionName(subscriptionName)
+                        .subscriptionType(SubscriptionType.Exclusive)
+                        .subscriptionMode(SubscriptionMode.Durable)
+                        .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                        .subscribe();
+    }
+
+    @Override
+    public List<T> poll(Duration timeout) {
+        List<T> results = new ArrayList<>();
+
+        while (true) {
+            try {
+                int millis = Math.toIntExact(timeout.toMillis());
+                Message<T> message = consumer.receive(millis, MILLISECONDS);
+
+                if (message != null) {
+                    consumer.acknowledgeCumulative(message);
+                    results.add(message.getValue());
+                } else {
+                    break;
+                }
+            } catch (Exception e) {
+                LOG.error("", e);
+                break;
+            }
+        }
+
+        return results;
+    }
+
+    @Override
+    public void close() throws PulsarClientException {
+        consumer.close();
+    }
+}
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarSinkTestContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarSinkTestContext.java
new file mode 100644
index 00000000000..6b9cf2d1041
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarSinkTestContext.java
@@ -0,0 +1,124 @@
+/*
+ * 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.testutils.sink;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+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.testutils.PulsarTestContext;
+import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment;
+import org.apache.flink.connector.testframe.external.ExternalSystemDataReader;
+import org.apache.flink.connector.testframe.external.sink.DataStreamSinkV2ExternalContext;
+import org.apache.flink.connector.testframe.external.sink.TestingSinkSettings;
+
+import org.apache.flink.shaded.guava30.com.google.common.io.Closer;
+
+import org.apache.pulsar.client.api.Schema;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric;
+import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient;
+import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_BATCHING_MAX_MESSAGES;
+import static org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema.pulsarSchema;
+import static org.apache.flink.connector.pulsar.testutils.PulsarTestCommonUtils.toDeliveryGuarantee;
+
+/** Common sink test context for pulsar based test. */
+public class PulsarSinkTestContext extends PulsarTestContext<String>
+        implements DataStreamSinkV2ExternalContext<String> {
+
+    private static final String TOPIC_NAME_PREFIX = "flink-sink-topic-";
+    private static final int RECORD_SIZE_UPPER_BOUND = 300;
+    private static final int RECORD_SIZE_LOWER_BOUND = 100;
+    private static final int RECORD_STRING_SIZE = 20;
+
+    private String topicName = topicName();
+    private final Closer closer = Closer.create();
+
+    public PulsarSinkTestContext(PulsarTestEnvironment environment) {
+        super(environment, Schema.STRING);
+    }
+
+    @Override
+    protected String displayName() {
+        return "write messages into one topic in Pulsar";
+    }
+
+    @Override
+    public Sink<String> createSink(TestingSinkSettings sinkSettings) {
+        operator.createTopic(topicName, 4);
+        DeliveryGuarantee guarantee = toDeliveryGuarantee(sinkSettings.getCheckpointingMode());
+
+        return PulsarSink.builder()
+                .setServiceUrl(operator.serviceUrl())
+                .setAdminUrl(operator.adminUrl())
+                .setTopics(topicName)
+                .setDeliveryGuarantee(guarantee)
+                .setSerializationSchema(pulsarSchema(schema))
+                .enableSchemaEvolution()
+                .setConfig(PULSAR_BATCHING_MAX_MESSAGES, 4)
+                .build();
+    }
+
+    @Override
+    public ExternalSystemDataReader<String> createSinkDataReader(TestingSinkSettings sinkSettings) {
+        PulsarPartitionDataReader<String> reader =
+                sneakyClient(
+                        () -> new PulsarPartitionDataReader<>(operator, topicName, Schema.STRING));
+        closer.register(reader);
+
+        return reader;
+    }
+
+    @Override
+    public List<String> generateTestData(TestingSinkSettings sinkSettings, long seed) {
+        Random random = new Random(seed);
+        int recordSize =
+                random.nextInt(RECORD_SIZE_UPPER_BOUND - RECORD_SIZE_LOWER_BOUND)
+                        + RECORD_SIZE_LOWER_BOUND;
+        List<String> records = new ArrayList<>(recordSize);
+        for (int i = 0; i < recordSize; i++) {
+            int size = random.nextInt(RECORD_STRING_SIZE) + RECORD_STRING_SIZE;
+            String record = "index:" + i + "-data:" + randomAlphanumeric(size);
+            records.add(record);
+        }
+
+        return records;
+    }
+
+    @Override
+    public TypeInformation<String> getProducedType() {
+        return Types.STRING;
+    }
+
+    @Override
+    public void close() throws Exception {
+        // Change the topic name after finishing a test case.
+        closer.register(() -> topicName = topicName());
+        closer.close();
+    }
+
+    private String topicName() {
+        return TOPIC_NAME_PREFIX + randomAlphanumeric(8);
+    }
+}
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarSinkTestSuiteBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarSinkTestSuiteBase.java
new file mode 100644
index 00000000000..0695a43569b
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarSinkTestSuiteBase.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.pulsar.testutils.sink;
+
+import org.apache.flink.connector.testframe.environment.TestEnvironment;
+import org.apache.flink.connector.testframe.external.sink.DataStreamSinkExternalContext;
+import org.apache.flink.connector.testframe.testsuites.SinkTestSuiteBase;
+import org.apache.flink.streaming.api.CheckpointingMode;
+
+import org.junit.jupiter.api.Disabled;
+
+/** Pulsar sink don't expose the monitor metrics now. We have to disable this test. */
+public abstract class PulsarSinkTestSuiteBase extends SinkTestSuiteBase<String> {
+
+    @Override
+    @Disabled("Enable this test after FLINK-26027 being merged.")
+    public void testMetrics(
+            TestEnvironment testEnv,
+            DataStreamSinkExternalContext<String> externalContext,
+            CheckpointingMode semantic) {}
+}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/KeyedPulsarPartitionDataWriter.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/KeyedPulsarPartitionDataWriter.java
similarity index 93%
rename from flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/KeyedPulsarPartitionDataWriter.java
rename to flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/KeyedPulsarPartitionDataWriter.java
index d5f6e11c00a..23d65f04a81 100644
--- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/KeyedPulsarPartitionDataWriter.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/KeyedPulsarPartitionDataWriter.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.tests.util.pulsar.common;
+package org.apache.flink.connector.pulsar.testutils.source;
 
 import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator;
 import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter;
@@ -51,9 +51,11 @@ public class KeyedPulsarPartitionDataWriter implements ExternalSystemSplitDataWr
 
     @Override
     public void writeRecords(List<String> records) {
+        // Send messages with the key we don't need.
         List<String> newRecords = records.stream().map(a -> a + keyToRead).collect(toList());
-
         operator.sendMessages(fullTopicName, Schema.STRING, keyToExclude, newRecords);
+
+        // Send messages with the given key.
         operator.sendMessages(fullTopicName, Schema.STRING, keyToRead, records);
     }
 
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/UnorderedSourceTestSuiteBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/UnorderedSourceTestSuiteBase.java
index 35496ccab21..af41d8c3093 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/UnorderedSourceTestSuiteBase.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/UnorderedSourceTestSuiteBase.java
@@ -72,8 +72,5 @@ public abstract class UnorderedSourceTestSuiteBase<T> extends SourceTestSuiteBas
     public void testIdleReader(
             TestEnvironment testEnv,
             DataStreamSourceExternalContext<T> externalContext,
-            CheckpointingMode semantic)
-            throws Exception {
-        super.testIdleReader(testEnv, externalContext, semantic);
-    }
+            CheckpointingMode semantic) {}
 }
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/KeySharedSubscriptionContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/KeySharedSubscriptionContext.java
similarity index 73%
rename from flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/KeySharedSubscriptionContext.java
rename to flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/KeySharedSubscriptionContext.java
index 0cae6e58405..c348853872d 100644
--- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/KeySharedSubscriptionContext.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/KeySharedSubscriptionContext.java
@@ -16,24 +16,21 @@
  * limitations under the License.
  */
 
-package org.apache.flink.tests.util.pulsar.source;
+package org.apache.flink.connector.pulsar.testutils.source.cases;
 
 import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder;
-import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange;
-import org.apache.flink.connector.pulsar.source.enumerator.topic.range.FixedRangeGenerator;
+import org.apache.flink.connector.pulsar.source.enumerator.topic.range.FixedKeysRangeGenerator;
 import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment;
-import org.apache.flink.connector.pulsar.testutils.source.cases.MultipleTopicConsumingContext;
+import org.apache.flink.connector.pulsar.testutils.source.KeyedPulsarPartitionDataWriter;
 import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter;
 import org.apache.flink.connector.testframe.external.source.TestingSourceSettings;
-import org.apache.flink.tests.util.pulsar.common.KeyedPulsarPartitionDataWriter;
 
 import org.apache.pulsar.client.api.SubscriptionType;
-import org.apache.pulsar.common.util.Murmur3_32Hash;
 
-import static java.util.Collections.singletonList;
 import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic;
-import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.RANGE_SIZE;
-import static org.apache.pulsar.client.api.SubscriptionType.Key_Shared;
+import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_ALLOW_KEY_SHARED_OUT_OF_ORDER_DELIVERY;
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.range.RangeGenerator.KeySharedMode.JOIN;
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.range.TopicRangeUtils.keyHash;
 
 /** We would consume from test splits by using {@link SubscriptionType#Key_Shared} subscription. */
 public class KeySharedSubscriptionContext extends MultipleTopicConsumingContext {
@@ -42,7 +39,7 @@ public class KeySharedSubscriptionContext extends MultipleTopicConsumingContext
     private final String keyToExclude;
 
     public KeySharedSubscriptionContext(PulsarTestEnvironment environment) {
-        super(environment, Key_Shared);
+        super(environment);
 
         this.keyToRead = randomAlphabetic(8);
 
@@ -69,10 +66,11 @@ public class KeySharedSubscriptionContext extends MultipleTopicConsumingContext
 
     @Override
     protected void setSourceBuilder(PulsarSourceBuilder<String> builder) {
-        int keyHash = keyHash(keyToRead);
-        TopicRange range = new TopicRange(keyHash, keyHash);
-
-        builder.setRangeGenerator(new FixedRangeGenerator(singletonList(range)));
+        // Make sure we only consume the messages with keyToRead.
+        FixedKeysRangeGenerator generator =
+                FixedKeysRangeGenerator.builder().key(keyToRead).keySharedMode(JOIN).build();
+        builder.setRangeGenerator(generator);
+        builder.setConfig(PULSAR_ALLOW_KEY_SHARED_OUT_OF_ORDER_DELIVERY, true);
     }
 
     @Override
@@ -80,8 +78,8 @@ public class KeySharedSubscriptionContext extends MultipleTopicConsumingContext
         return "pulsar-key-shared-subscription";
     }
 
-    // This method is copied from Pulsar for calculating message key hash.
-    private int keyHash(String key) {
-        return Murmur3_32Hash.getInstance().makeHash(key.getBytes()) % RANGE_SIZE;
+    @Override
+    protected SubscriptionType subscriptionType() {
+        return SubscriptionType.Key_Shared;
     }
 }
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/MultipleTopicConsumingContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/MultipleTopicConsumingContext.java
index ca6410fbc43..422c954a4e7 100644
--- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/MultipleTopicConsumingContext.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/MultipleTopicConsumingContext.java
@@ -33,18 +33,11 @@ import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNam
 public class MultipleTopicConsumingContext extends PulsarSourceTestContext {
 
     private final String topicPrefix = "flink-multiple-topic-" + randomAlphabetic(8) + "-";
-    private final SubscriptionType subscriptionType;
 
     private int index = 0;
 
     public MultipleTopicConsumingContext(PulsarTestEnvironment environment) {
-        this(environment, SubscriptionType.Exclusive);
-    }
-
-    public MultipleTopicConsumingContext(
-            PulsarTestEnvironment environment, SubscriptionType subscriptionType) {
         super(environment);
-        this.subscriptionType = subscriptionType;
     }
 
     @Override
@@ -64,7 +57,7 @@ public class MultipleTopicConsumingContext extends PulsarSourceTestContext {
 
     @Override
     protected SubscriptionType subscriptionType() {
-        return subscriptionType;
+        return SubscriptionType.Exclusive;
     }
 
     @Override
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/SharedSubscriptionContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/SharedSubscriptionContext.java
similarity index 84%
rename from flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/SharedSubscriptionContext.java
rename to flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/SharedSubscriptionContext.java
index fe9f07832af..cd649fef040 100644
--- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/SharedSubscriptionContext.java
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/SharedSubscriptionContext.java
@@ -16,20 +16,17 @@
  * limitations under the License.
  */
 
-package org.apache.flink.tests.util.pulsar.source;
+package org.apache.flink.connector.pulsar.testutils.source.cases;
 
 import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment;
-import org.apache.flink.connector.pulsar.testutils.source.cases.MultipleTopicConsumingContext;
 
 import org.apache.pulsar.client.api.SubscriptionType;
 
-import static org.apache.pulsar.client.api.SubscriptionType.Shared;
-
 /** We would consume from test splits by using {@link SubscriptionType#Shared} subscription. */
 public class SharedSubscriptionContext extends MultipleTopicConsumingContext {
 
     public SharedSubscriptionContext(PulsarTestEnvironment environment) {
-        super(environment, Shared);
+        super(environment);
     }
 
     @Override
@@ -41,4 +38,9 @@ public class SharedSubscriptionContext extends MultipleTopicConsumingContext {
     protected String subscriptionName() {
         return "pulsar-shared-subscription";
     }
+
+    @Override
+    protected SubscriptionType subscriptionType() {
+        return SubscriptionType.Shared;
+    }
 }
diff --git a/flink-connectors/flink-connector-pulsar/src/test/resources/docker/bootstrap.sh b/flink-connectors/flink-connector-pulsar/src/test/resources/docker/bootstrap.sh
deleted file mode 100755
index fa4e2921a75..00000000000
--- a/flink-connectors/flink-connector-pulsar/src/test/resources/docker/bootstrap.sh
+++ /dev/null
@@ -1,28 +0,0 @@
-#!/usr/bin/env bash
-# ----------------------------------------------------------------------------
-# 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.
-# ----------------------------------------------------------------------------
-
-# Enable the transaction in standalone config.
-sed -i 's/transactionCoordinatorEnabled=false/transactionCoordinatorEnabled=true/g' /pulsar/conf/standalone.conf
-sed -i 's/acknowledgmentAtBatchIndexLevelEnabled=false/acknowledgmentAtBatchIndexLevelEnabled=true/g' /pulsar/conf/standalone.conf
-sed -i 's/systemTopicEnabled=false/systemTopicEnabled=true/g' /pulsar/conf/standalone.conf
-sed -i 's/brokerDeduplicationEnabled=false/brokerDeduplicationEnabled=true/g' /pulsar/conf/standalone.conf
-
-# Start Pulsar standalone without function worker and streaming storage.
-/pulsar/bin/pulsar standalone --no-functions-worker -nss
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceUnorderedE2ECase.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSinkE2ECase.java
similarity index 64%
copy from flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceUnorderedE2ECase.java
copy to flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSinkE2ECase.java
index 15333a1d538..a19f5931b58 100644
--- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceUnorderedE2ECase.java
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSinkE2ECase.java
@@ -19,7 +19,8 @@
 package org.apache.flink.tests.util.pulsar;
 
 import org.apache.flink.connector.pulsar.testutils.PulsarTestContextFactory;
-import org.apache.flink.connector.pulsar.testutils.source.UnorderedSourceTestSuiteBase;
+import org.apache.flink.connector.pulsar.testutils.sink.PulsarSinkTestContext;
+import org.apache.flink.connector.pulsar.testutils.sink.PulsarSinkTestSuiteBase;
 import org.apache.flink.connector.testframe.junit.annotations.TestContext;
 import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
 import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
@@ -27,38 +28,30 @@ import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
 import org.apache.flink.streaming.api.CheckpointingMode;
 import org.apache.flink.tests.util.pulsar.common.FlinkContainerWithPulsarEnvironment;
 import org.apache.flink.tests.util.pulsar.common.PulsarContainerTestEnvironment;
-import org.apache.flink.tests.util.pulsar.source.KeySharedSubscriptionContext;
-import org.apache.flink.tests.util.pulsar.source.SharedSubscriptionContext;
 import org.apache.flink.testutils.junit.FailsOnJava11;
 
 import org.junit.experimental.categories.Category;
 
-/**
- * Pulsar E2E test based on connector testing framework. It's used for Shared & Key_Shared
- * subscription.
- */
+/** Pulsar sink E2E test based on connector testing framework. */
 @SuppressWarnings("unused")
 @Category(value = {FailsOnJava11.class})
-public class PulsarSourceUnorderedE2ECase extends UnorderedSourceTestSuiteBase<String> {
+public class PulsarSinkE2ECase extends PulsarSinkTestSuiteBase {
 
-    // Defines the Semantic.
     @TestSemantics
-    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+    CheckpointingMode[] semantics =
+            new CheckpointingMode[] {
+                CheckpointingMode.EXACTLY_ONCE, CheckpointingMode.AT_LEAST_ONCE
+            };
 
-    // Defines TestEnvironment.
+    // Defines TestEnvironment
     @TestEnv
-    FlinkContainerWithPulsarEnvironment flink = new FlinkContainerWithPulsarEnvironment(1, 8);
+    FlinkContainerWithPulsarEnvironment flink = new FlinkContainerWithPulsarEnvironment(1, 6);
 
     // Defines ConnectorExternalSystem.
     @TestExternalSystem
     PulsarContainerTestEnvironment pulsar = new PulsarContainerTestEnvironment(flink);
 
-    // Defines a set of external context Factories for different test cases.
-    @TestContext
-    PulsarTestContextFactory<String, SharedSubscriptionContext> shared =
-            new PulsarTestContextFactory<>(pulsar, SharedSubscriptionContext::new);
-
     @TestContext
-    PulsarTestContextFactory<String, KeySharedSubscriptionContext> keyShared =
-            new PulsarTestContextFactory<>(pulsar, KeySharedSubscriptionContext::new);
+    PulsarTestContextFactory<String, PulsarSinkTestContext> sinkContext =
+            new PulsarTestContextFactory<>(pulsar, PulsarSinkTestContext::new);
 }
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceUnorderedE2ECase.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceUnorderedE2ECase.java
index 15333a1d538..89692d1c449 100644
--- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceUnorderedE2ECase.java
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceUnorderedE2ECase.java
@@ -20,6 +20,8 @@ package org.apache.flink.tests.util.pulsar;
 
 import org.apache.flink.connector.pulsar.testutils.PulsarTestContextFactory;
 import org.apache.flink.connector.pulsar.testutils.source.UnorderedSourceTestSuiteBase;
+import org.apache.flink.connector.pulsar.testutils.source.cases.KeySharedSubscriptionContext;
+import org.apache.flink.connector.pulsar.testutils.source.cases.SharedSubscriptionContext;
 import org.apache.flink.connector.testframe.junit.annotations.TestContext;
 import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
 import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
@@ -27,8 +29,6 @@ import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
 import org.apache.flink.streaming.api.CheckpointingMode;
 import org.apache.flink.tests.util.pulsar.common.FlinkContainerWithPulsarEnvironment;
 import org.apache.flink.tests.util.pulsar.common.PulsarContainerTestEnvironment;
-import org.apache.flink.tests.util.pulsar.source.KeySharedSubscriptionContext;
-import org.apache.flink.tests.util.pulsar.source.SharedSubscriptionContext;
 import org.apache.flink.testutils.junit.FailsOnJava11;
 
 import org.junit.experimental.categories.Category;
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/FlinkContainerWithPulsarEnvironment.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/FlinkContainerWithPulsarEnvironment.java
index f5e862f17c2..65e99a81456 100644
--- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/FlinkContainerWithPulsarEnvironment.java
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/FlinkContainerWithPulsarEnvironment.java
@@ -51,11 +51,8 @@ public class FlinkContainerWithPulsarEnvironment extends FlinkContainerTestEnvir
         return ResourceTestUtils.getResource(jarName).toAbsolutePath().toString();
     }
 
-    protected static Configuration flinkConfiguration() {
+    private static Configuration flinkConfiguration() {
         Configuration configuration = new Configuration();
-        // Increase the off heap memory of TaskManager to avoid direct buffer memory error in Pulsar
-        // e2e tests.
-        configuration.set(TaskManagerOptions.TASK_OFF_HEAP_MEMORY, MemorySize.ofMebiBytes(100));
 
         // Increase the jvm metaspace memory to avoid java.lang.OutOfMemoryError: Metaspace
         configuration.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.ofMebiBytes(2048));