From 95d14edbc8dc16fe420ec12fbd5d7f61dc873699 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Sun, 14 Aug 2022 22:49:13 +0800 Subject: [PATCH] [FLINK-27399][Connector/Pulsar] Change initial consuming position setting logic for better handle the checkpoint. (#19972) (#20565) * Change the initial start cursor and stop cursor to better handle the consuming behaviors. * Create the initial subscription instead seek every time. This should fix the wrong position setting. * Fix the wrong stop cursor, make sure it stops at the correct space * Drop Consumer.seek() for https://github.com/apache/pulsar/pull/16171 --- .../docs/connectors/datastream/pulsar.md | 227 ++++----- .../docs/connectors/datastream/pulsar.md | 268 ++++++----- .../pulsar_consumer_configuration.html | 6 - .../flink-connector-pulsar/pom.xml | 75 ++- .../common/config/PulsarConfigUtils.java | 32 ++ .../common/utils/PulsarTransactionUtils.java | 118 ----- .../connector/pulsar/source/PulsarSource.java | 14 +- .../pulsar/source/PulsarSourceBuilder.java | 39 +- .../pulsar/source/PulsarSourceOptions.java | 7 + .../config/PulsarSourceConfigUtils.java | 20 +- .../source/config/SourceConfiguration.java | 170 ++++--- .../enumerator/PulsarSourceEnumState.java | 16 +- .../enumerator/PulsarSourceEnumerator.java | 150 +++--- .../enumerator/SplitsAssignmentState.java | 239 ---------- .../assigner/NonSharedSplitAssigner.java | 126 +++++ .../assigner/SharedSplitAssigner.java | 148 ++++++ .../enumerator/assigner/SplitAssigner.java | 64 +++ .../assigner/SplitAssignerFactory.java | 65 +++ .../enumerator/cursor/CursorPosition.java | 36 +- .../enumerator/cursor/MessageIdUtils.java | 71 +++ .../source/enumerator/cursor/StartCursor.java | 32 +- .../source/enumerator/cursor/StopCursor.java | 96 +++- .../cursor/start/MessageIdStartCursor.java | 20 +- .../cursor/start/TimestampStartCursor.java | 4 +- .../cursor/stop/EventTimestampStopCursor.java | 9 +- .../cursor/stop/LatestMessageStopCursor.java | 21 +- .../cursor/stop/MessageIdStopCursor.java | 32 +- .../cursor/stop/NeverStopCursor.java | 4 +- .../stop/PublishTimestampStopCursor.java | 9 +- .../subscriber/PulsarSubscriber.java | 2 +- .../reader/PulsarSourceReaderFactory.java | 2 +- .../PulsarOrderedPartitionSplitReader.java | 42 +- .../split/PulsarPartitionSplitReaderBase.java | 55 ++- .../PulsarUnorderedPartitionSplitReader.java | 5 +- .../source/split/PulsarPartitionSplit.java | 4 +- .../PulsarSourceEnumStateSerializerTest.java | 5 +- .../PulsarSourceEnumeratorTest.java | 447 +++++++++++++++--- .../enumerator/SplitsAssignmentStateTest.java | 119 ----- .../assigner/NonSharedSplitAssignerTest.java | 95 ++++ .../assigner/SharedSplitAssignerTest.java | 98 ++++ .../assigner/SplitAssignerTestBase.java | 113 +++++ .../enumerator/cursor/StopCursorTest.java | 20 +- .../subscriber/PulsarSubscriberTest.java | 10 +- .../PulsarDeserializationSchemaTest.java | 2 +- ...PulsarOrderedPartitionSplitReaderTest.java | 74 ++- .../PulsarPartitionSplitReaderTestBase.java | 270 ++++++++++- ...lsarUnorderedPartitionSplitReaderTest.java | 16 +- .../pulsar/testutils/PulsarTestContext.java | 13 +- .../pulsar/testutils/extension/SubType.java | 32 ++ .../extension/TestOrderlinessExtension.java | 65 +++ .../runtime/PulsarRuntimeOperator.java | 380 +++++++++++++-- .../runtime/mock/PulsarMockRuntime.java | 138 ++++-- .../src/main/resources/META-INF/NOTICE | 17 + .../flink-end-to-end-tests-pulsar/pom.xml | 43 +- .../FlinkContainerWithPulsarEnvironment.java | 5 + .../flink/util/DockerImageVersions.java | 2 +- 56 files changed, 2927 insertions(+), 1265 deletions(-) delete mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/utils/PulsarTransactionUtils.java delete mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/SplitsAssignmentState.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssigner.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssigner.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssigner.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerFactory.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/MessageIdUtils.java delete mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/SplitsAssignmentStateTest.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssignerTest.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssignerTest.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerTestBase.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/extension/SubType.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/extension/TestOrderlinessExtension.java create mode 100644 flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/NOTICE diff --git a/docs/content.zh/docs/connectors/datastream/pulsar.md b/docs/content.zh/docs/connectors/datastream/pulsar.md index 14c4d89c96230..2ba0392bf1a49 100644 --- a/docs/content.zh/docs/connectors/datastream/pulsar.md +++ b/docs/content.zh/docs/connectors/datastream/pulsar.md @@ -28,8 +28,7 @@ Flink 当前只提供 [Apache Pulsar](https://pulsar.apache.org) 数据源,用 ## 添加依赖 -连接器当前支持 Pulsar 2.7.0 之后的版本,但是连接器使用到了 Pulsar 的[事务机制](https://pulsar.apache.org/docs/en/txn-what/),建议在 Pulsar 2.8.0 -及其之后的版本上使用连接器进行数据读取。 +Pulsar Source 当前支持 Pulsar 2.9.0 之后的版本,但是 Pulsar Source 使用到了 Pulsar 的[事务机制](https://pulsar.apache.org/docs/zh-CN/txn-what/),建议在 Pulsar 2.10.0 及其之后的版本上使用 Pulsar Source 进行数据读取。 如果想要了解更多关于 Pulsar API 兼容性设计,可以阅读文档 [PIP-72](https://github.com/apache/pulsar/wiki/PIP-72%3A-Introduce-Pulsar-Interface-Taxonomy%3A-Audience-and-Stability-Classification)。 @@ -37,18 +36,15 @@ Flink 当前只提供 [Apache Pulsar](https://pulsar.apache.org) 数据源,用 Flink 的流连接器并不会放到发行文件里面一同发布,阅读[此文档]({{< ref "docs/dev/datastream/project-configuration" >}}),了解如何将连接器添加到集群实例内。 -## Pulsar 数据源 +## Pulsar Source {{< hint info >}} -Pulsar 数据源基于 Flink 最新的[批流一体 API]({{< ref "docs/dev/datastream/sources.md" >}}) 进行开发。 - -如果要想使用基于旧版的 `SourceFunction` 实现的 Pulsar 数据源,或者是项目的 Flink 版本低于 1.14,可以使用 StreamNative 单独维护的 [pulsar-flink](https://github.com/streamnative/pulsar-flink)。 +Pulsar Source 基于 Flink 最新的[批流一体 API]({{< ref "docs/dev/datastream/sources.md" >}}) 进行开发。 {{< /hint >}} ### 使用示例 -Pulsar 数据源提供了 builder 类来构造数据源实例。下面的代码实例使用 builder 类创建的数据源会从 topic "persistent://public/default/my-topic" 的数据开始端进行消费。 -连接器使用了 **Exclusive**(独占)的订阅方式消费消息,订阅名称为 `my-subscription`,并把消息体的二进制字节流以 UTF-8 的方式编码为字符串。 +Pulsar Source 提供了 builder 类来构造 `PulsarSource` 实例。下面的代码实例使用 builder 类创建的实例会从 “persistent://public/default/my-topic” 的数据开始端进行消费。对应的 Pulsar Source 使用了 **Exclusive**(独占)的订阅方式消费消息,订阅名称为 `my-subscription`,并把消息体的二进制字节流以 UTF-8 的方式编码为字符串。 ```java PulsarSource pulsarSource = PulsarSource.builder() @@ -64,226 +60,229 @@ PulsarSource pulsarSource = PulsarSource.builder() env.fromSource(source, WatermarkStrategy.noWatermarks(), "Pulsar Source"); ``` -如果使用构造类构造 Pulsar 数据源,一定要提供下面几个属性: +如果使用构造类构造 `PulsarSource`,一定要提供下面几个属性: -- Pulsar 数据消费的地址,使用 `setServiceUrl(String)` 方法提供 -- Pulsar HTTP 管理地址,使用 `setAdminUrl(String)` 方法提供 -- Pulsar 订阅名称,使用 `setSubscriptionName(String)` 方法提供 -- 需要消费的 topic 或者是 topic 下面的分区,详见[指定消费的 Topic 或者 Topic 分区](#指定消费的-topic-或者-topic-分区) -- 解码 Pulsar 消息的反序列化器,详见[反序列化器](#反序列化器) +- Pulsar 数据消费的地址,使用 `setServiceUrl(String)` 方法提供。 +- Pulsar HTTP 管理地址,使用 `setAdminUrl(String)` 方法提供。 +- Pulsar 订阅名称,使用 `setSubscriptionName(String)` 方法提供。 +- 需要消费的 Topic 或者是 Topic 下面的分区,详见[指定消费的 Topic 或者 Topic 分区](#指定消费的-topic-或者-topic-分区)。 +- 解码 Pulsar 消息的反序列化器,详见[反序列化器](#反序列化器)。 ### 指定消费的 Topic 或者 Topic 分区 -Pulsar 数据源提供了两种订阅 topic 或 topic 分区的方式。 +Pulsar Source 提供了两种订阅 Topic 或 Topic 分区的方式。 - Topic 列表,从这个 Topic 的所有分区上消费消息,例如: ```java - PulsarSource.builder().setTopics("some-topic1", "some-topic2") + PulsarSource.builder().setTopics("some-topic1", "some-topic2"); - // 从 topic "topic-a" 的 0 和 1 分区上消费 - PulsarSource.builder().setTopics("topic-a-partition-0", "topic-a-partition-2") + // 从 topic "topic-a" 的 0 和 2 分区上消费 + PulsarSource.builder().setTopics("topic-a-partition-0", "topic-a-partition-2"); ``` -- Topic 正则,连接器使用给定的正则表达式匹配出所有合规的 topic,例如: +- Topic 正则,Pulsar Source 使用给定的正则表达式匹配出所有合规的 Topic,例如: ```java - PulsarSource.builder().setTopicPattern("topic-*") + PulsarSource.builder().setTopicPattern("topic-*"); ``` #### Topic 名称简写 -从 Pulsar 2.0 之后,完整的 topic 名称格式为 `{persistent|non-persistent}://租户/命名空间/topic`。 -但是连接器不需要提供 topic 名称的完整定义,因为 topic 类型、租户、命名空间都设置了默认值。 +从 Pulsar 2.0 之后,完整的 Topic 名称格式为 `{persistent|non-persistent}://租户/命名空间/topic`。但是 Pulsar Source 不需要提供 Topic 名称的完整定义,因为 Topic 类型、租户、命名空间都设置了默认值。 -Topic 属性 | 默认值 -:------------|:------- -topic 类型 | `persistent` -租户 | `public` -命名空间 | `default` +| Topic 属性 | 默认值 | +|:---------|:-------------| +| Topic 类型 | `persistent` | +| 租户 | `public` | +| 命名空间 | `default` | -下面的表格提供了当前 Pulsar 支持的简写方式: +下面的表格提供了当前 Pulsar Topic 支持的简写方式: -topic 名称简写 | 翻译后的 topic 名称 -:----------------|:--------------------- -`my-topic` | `persistent://public/default/my-topic` -`my-tenant/my-namespace/my-topic` | `persistent://my-tenant/my-namespace/my-topic` +| Topic 名称简写 | 翻译后的 Topic 名称 | +|:----------------------------------|:-----------------------------------------------| +| `my-topic` | `persistent://public/default/my-topic` | +| `my-tenant/my-namespace/my-topic` | `persistent://my-tenant/my-namespace/my-topic` | {{< hint warning >}} -对于 non-persistent(非持久化) topic,连接器不支持简写名称。所以无法将 `non-persistent://public/default/my-topic` 简写成 `non-persistent://my-topic`。 +对于 Non-persistent(非持久化)Topic,Pulsar Source 不支持简写名称。所以无法将 `non-persistent://public/default/my-topic` 简写成 `non-persistent://my-topic`。 {{< /hint >}} #### Pulsar Topic 层次结构 对于 Pulsar 而言,Topic 分区也是一种 Topic。Pulsar 会将一个有分区的 Topic 在内部按照分区的大小拆分成等量的无分区 Topic。 -例如,在 Pulsar 的 `sample` 租户下面的 `flink` 命名空间里面创建了一个有 3 个分区的 topic,给它起名为 `simple-string`。 -可以在 Pulsar 上看到如下的 topic 列表: +由于 Pulsar 内部的分区实际实现为一个 Topic,我们将用“分区”来指代“仅有一个分区的 Topic(Non-partitioned Topic)”和“具有多个分区的 Topic 下属的分区”。 + +例如,在 Pulsar 的 `sample` 租户下面的 `flink` 命名空间里面创建了一个有 3 个分区的 Topic,给它起名为 `simple-string`。可以在 Pulsar 上看到如下的 Topic 列表: -Topic 名称 | 是否分区 -:--------- | :---------- -`persistent://sample/flink/simple-string` | 是 -`persistent://sample/flink/simple-string-partition-0` | 否 -`persistent://sample/flink/simple-string-partition-1` | 否 -`persistent://sample/flink/simple-string-partition-2` | 否 +| Topic 名称 | 是否分区 | +|:------------------------------------------------------|:-----| +| `persistent://sample/flink/simple-string` | 是 | +| `persistent://sample/flink/simple-string-partition-0` | 否 | +| `persistent://sample/flink/simple-string-partition-1` | 否 | +| `persistent://sample/flink/simple-string-partition-2` | 否 | -这意味着,用户可以用上面的子 topic 去直接消费分区里面的数据,不需要再去基于上层的父 topic 去消费全部分区的数据。 -例如:使用 `PulsarSource.builder().setTopics("sample/flink/simple-string-partition-1", "sample/flink/simple-string-partition-2")` 将会只消费 topic `sample/flink/simple-string` 上面的分区 1 和 2 里面的消息。 +这意味着,用户可以用上面的子 Topic 去直接消费分区里面的数据,不需要再去基于上层的父 Topic 去消费全部分区的数据。例如:使用 `PulsarSource.builder().setTopics("sample/flink/simple-string-partition-1", "sample/flink/simple-string-partition-2")` 将会只消费 Topic `sample/flink/simple-string` 分区 1 和 2 里面的消息。 #### 配置 Topic 正则表达式 -前面提到了 Pulsar topic 有 `persistent`、`non-persistent` 两种类型,使用正则表达式消费数据的时候,连接器会尝试从正则表达式里面解析出消息的类型。 -例如:`PulsarSource.builder().setTopicPattern("non-persistent://my-topic*")` 会解析出 `non-persistent` 这个 topic 类型。 -如果用户使用 topic 名称简写的方式,连接器会使用默认的消息类型 `persistent`。 +前面提到了 Pulsar Topic 有 `persistent`、`non-persistent` 两种类型,使用正则表达式消费数据的时候,Pulsar Source 会尝试从正则表达式里面解析出消息的类型。例如:`PulsarSource.builder().setTopicPattern("non-persistent://my-topic*")` 会解析出 `non-persistent` 这个 Topic 类型。如果用户使用 Topic 名称简写的方式,Pulsar Source 会使用默认的消息类型 `persistent`。 -如果想用正则去消费 `persistent` 和 `non-persistent` 类型的 topic,需要使用 `RegexSubscriptionMode` 定义 topic 类型,例如:`setTopicPattern("topic-*", RegexSubscriptionMode.AllTopics)`。 +如果想用正则去消费 `persistent` 和 `non-persistent` 类型的 Topic,需要使用 `RegexSubscriptionMode` 定义 Topic 类型,例如:`setTopicPattern("topic-*", RegexSubscriptionMode.AllTopics)`。 ### 反序列化器 -反序列化器用于解析 Pulsar 消息,连接器使用 `PulsarDeserializationSchema` 来定义反序列化器。 -用户可以在 builder 类中使用 `setDeserializationSchema(PulsarDeserializationSchema)` 方法配置反序列化器,它会解析 Pulsar 的 `Message` 实例。 +反序列化器用于解析 Pulsar 消息,Pulsar Source 使用 `PulsarDeserializationSchema` 来定义反序列化器。用户可以在 builder 类中使用 `setDeserializationSchema(PulsarDeserializationSchema)` 方法配置反序列化器。 -如果用户只关心消息体的二进制字节流,并不需要其他属性来解析数据。可以直接使用预定义的 `PulsarDeserializationSchema`。Pulsar 连接器里面提供了 3 种预定义好的反序列化器。 +如果用户只关心消息体的二进制字节流,并不需要其他属性来解析数据。可以直接使用预定义的 `PulsarDeserializationSchema`。Pulsar Source里面提供了 3 种预定义的反序列化器。 -- 使用 Pulsar 的 [Schema](https://pulsar.apache.org/docs/en/schema-understand/) 解析消息。 +- 使用 Pulsar 的 [Schema](https://pulsar.apache.org/docs/zh-CN/schema-understand/) 解析消息。 ```java // 基础数据类型 - PulsarDeserializationSchema.pulsarSchema(Schema) + PulsarDeserializationSchema.pulsarSchema(Schema); // 结构类型 (JSON, Protobuf, Avro, etc.) - PulsarDeserializationSchema.pulsarSchema(Schema, Class) + PulsarDeserializationSchema.pulsarSchema(Schema, Class); // 键值对类型 - PulsarDeserializationSchema.pulsarSchema(Schema, Class, Class) + PulsarDeserializationSchema.pulsarSchema(Schema, Class, Class); ``` - 使用 Flink 的 `DeserializationSchema` 解析消息。 ```java - PulsarDeserializationSchema.flinkSchema(DeserializationSchema) + PulsarDeserializationSchema.flinkSchema(DeserializationSchema); ``` - 使用 Flink 的 `TypeInformation` 解析消息。 ```java - PulsarDeserializationSchema.flinkTypeInfo(TypeInformation, ExecutionConfig) + PulsarDeserializationSchema.flinkTypeInfo(TypeInformation, ExecutionConfig); ``` -Pulsar 的 `Message` 包含了很多 [额外的属性](https://pulsar.apache.org/docs/zh-CN/concepts-messaging/#%E6%B6%88%E6%81%AF)。 -例如,消息的 key,消息发送时间,消息生产时间,用户在消息上自定义的键值对属性等。可以使用 `Message` 接口来获取这些属性。 +Pulsar 的 `Message` 包含了很多 [额外的属性](https://pulsar.apache.org/docs/zh-CN/concepts-messaging/#%E6%B6%88%E6%81%AF)。例如,消息的 key、消息发送时间、消息生产时间、用户在消息上自定义的键值对属性等。可以使用 `Message` 接口来获取这些属性。 -如果用户需要基于这些额外的属性来解析一条消息,可以实现 `PulsarDeserializationSchema` 接口。 -并一定要确保 `PulsarDeserializationSchema.getProducedType()` 方法返回的 `TypeInformation` 是正确的结果。 -Flink 使用 `TypeInformation` 将解析出来的结果序列化传递到下游算子。 +如果用户需要基于这些额外的属性来解析一条消息,可以实现 `PulsarDeserializationSchema` 接口。并一定要确保 `PulsarDeserializationSchema.getProducedType()` 方法返回的 `TypeInformation` 是正确的结果。Flink 使用 `TypeInformation` 将解析出来的结果序列化传递到下游算子。 ### Pulsar 订阅 -订阅是命名好的配置规则,指导消息如何投递给消费者。连接器需要提供一个独立的订阅名称,支持 Pulsar 的四种订阅模式: +订阅是命名好的配置规则,指导消息如何投递给消费者。Pulsar Source 需要提供一个独立的订阅名称,支持 Pulsar 的四种订阅模式: - [exclusive(独占)](https://pulsar.apache.org/docs/zh-CN/concepts-messaging/#exclusive) - [shared(共享)](https://pulsar.apache.org/docs/zh-CN/concepts-messaging/#shared%E5%85%B1%E4%BA%AB) - [failover(灾备)](https://pulsar.apache.org/docs/zh-CN/concepts-messaging/#failover%E7%81%BE%E5%A4%87) - [key_shared(key 共享)](https://pulsar.apache.org/docs/zh-CN/concepts-messaging/#key_shared) -当前 Pulsar 连接器里面,`独占` 和 `灾备` 的实现没有区别,如果 Flink 的一个 reader 挂了,连接器会把所有未消费的数据交给其他的 reader 来消费数据。 +当前 Pulsar Source 里,`独占` 和 `灾备` 的实现没有区别,如果 Flink 的一个 reader 挂了,Pulsar Source 会把所有未消费的数据交给其他的 reader 来消费数据。 -默认情况下,如果没有指定订阅类型,连接器使用共享订阅类型(`SubscriptionType.Shared`)。 +默认情况下,如果没有指定订阅类型,Pulsar Source 使用共享订阅类型(`SubscriptionType.Shared`)。 ```java // 名为 "my-shared" 的共享订阅 -PulsarSource.builder().setSubscriptionName("my-shared") +PulsarSource.builder().setSubscriptionName("my-shared"); // 名为 "my-exclusive" 的独占订阅 -PulsarSource.builder().setSubscriptionName("my-exclusive").setSubscriptionType(SubscriptionType.Exclusive) +PulsarSource.builder().setSubscriptionName("my-exclusive").setSubscriptionType(SubscriptionType.Exclusive); ``` -如果想在 Pulsar 连接器里面使用 `key 共享` 订阅,需要提供 `RangeGenerator` 实例。`RangeGenerator` 会生成一组消息 key 的 hash 范围,连接器会基于给定的范围来消费数据。 +如果想在 Pulsar Source 里面使用 `key 共享` 订阅,需要提供 `RangeGenerator` 实例。`RangeGenerator` 会生成一组消息 key 的 hash 范围,Pulsar Source 会基于给定的范围来消费数据。 -Pulsar 连接器也提供了一个名为 `UniformRangeGenerator` 的默认实现,它会基于 flink 数据源的并行度将 hash 范围均分。 +Pulsar Source 也提供了一个名为 `UniformRangeGenerator` 的默认实现,它会基于 flink 数据源的并行度将 hash 范围均分。 ### 起始消费位置 -连接器使用 `setStartCursor(StartCursor)` 方法给定开始消费的位置。内置的消费位置有: +Pulsar Source 使用 `setStartCursor(StartCursor)` 方法给定开始消费的位置。内置的开始消费位置有: -- 从 topic 里面最早的一条消息开始消费。 +- 从 Topic 里面最早的一条消息开始消费。 ```java - StartCursor.earliest() + StartCursor.earliest(); ``` -- 从 topic 里面最新的一条消息开始消费。 +- 从 Topic 里面最新的一条消息开始消费。 ```java - StartCursor.latest() + StartCursor.latest(); ``` - 从给定的消息开始消费。 ```java - StartCursor.fromMessageId(MessageId) + StartCursor.fromMessageId(MessageId); ``` - 与前者不同的是,给定的消息可以跳过,再进行消费。 ```java - StartCursor.fromMessageId(MessageId, boolean) + StartCursor.fromMessageId(MessageId, boolean); + ``` +- 从给定的消息发布时间开始消费,这个方法因为名称容易导致误解现在已经不建议使用。你可以使用方法 `StartCursor.fromPublishTime(long)`。 + ```java + StartCursor.fromMessageTime(long); ``` -- 从给定的消息时间开始消费。 +- 从给定的消息发布时间开始消费。 ```java - StartCursor.fromMessageTime(long) + StartCursor.fromPublishTime(long); ``` {{< hint info >}} 每条消息都有一个固定的序列号,这个序列号在 Pulsar 上有序排列,其包含了 ledger、entry、partition 等原始信息,用于在 Pulsar 底层存储上查找到具体的消息。 + Pulsar 称这个序列号为 `MessageId`,用户可以使用 `DefaultImplementation.newMessageId(long ledgerId, long entryId, int partitionIndex)` 创建它。 {{< /hint >}} ### 边界 -Pulsar 连接器同时支持流式和批的消费方式,默认情况下,连接器使用流的方式消费数据。除非任务失败或者被取消,否则连接器将持续消费数据。 -用户可以使用 `setBoundedStopCursor(StopCursor)` 给定停止消费的位置,这种情况下连接器会使用批的方式进行消费。当所有 topic 分区都消费到了停止位置,Flink 任务就会结束。 +Pulsar Source 默认情况下使用流的方式消费数据。除非任务失败或者被取消,否则将持续消费数据。用户可以使用 `setBoundedStopCursor(StopCursor)` 给定停止消费的位置,这种情况下会使用批的方式进行消费。使用流的方式一样可以给定停止位置,使用 `setUnboundedStopCursor(StopCursor)` 方法即可。 -使用流的方式一样可以给定停止位置,使用 `setUnboundedStopCursor(StopCursor)` 方法即可。 +在批模式下,使用 `setBoundedStopCursor(StopCursor)` 来指定一个消费停止位置。 -内置的停止位置如下: +内置的停止消费位置如下: - 永不停止。 ```java - StopCursor.never() + StopCursor.never(); ``` -- 停止于 Pulsar 启动时 topic 里面最新的那条数据。 +- 停止于 Pulsar 启动时 Topic 里面最新的那条数据。 ```java - StopCursor.latest() + StopCursor.latest(); ``` - 停止于某条消息,结果里不包含此消息。 ```java - StopCursor.atMessageId(MessageId) + StopCursor.atMessageId(MessageId); ``` - 停止于某条消息之后,结果里包含此消息。 ```java - StopCursor.afterMessageId(MessageId) + StopCursor.afterMessageId(MessageId); + ``` +- 停止于某个给定的消息事件时间戳,比如 `Message.getEventTime()`,消费结果里不包含此时间戳的消息。 + ```java + StopCursor.atEventTime(long); + ``` +- 停止于某个给定的消息事件时间戳,比如 `Message.getEventTime()`,消费结果里包含此时间戳的消息。 + ```java + StopCursor.afterEventTime(long); ``` -- 停止于某个给定的消息发布时间戳,比如 `Message.getPublishTime()`。 +- 停止于某个给定的消息发布时间戳,比如 `Message.getPublishTime()`,消费结果里不包含此时间戳的消息。 ```java StopCursor.atPublishTime(long); ``` - -{{< hint warning >}} -StopCursor.atEventTime(long) 目前已经处于弃用状态。 -{{< /hint >}} +- 停止于某个给定的消息发布时间戳,比如 `Message.getPublishTime()`,消费结果里包含此时间戳的消息。 + ```java + StopCursor.afterPublishTime(long); + ``` ### Source 配置项 -除了前面提到的配置选项,连接器还提供了丰富的选项供 Pulsar 专家使用,在 builder 类里通过 `setConfig(ConfigOption, T)` 和 `setConfig(Configuration)` 方法给定下述的全部配置。 +除了前面提到的配置选项,Pulsar Source 还提供了丰富的选项供 Pulsar 专家使用,在 builder 类里通过 `setConfig(ConfigOption, T)` 和 `setConfig(Configuration)` 方法给定下述的全部配置。 #### Pulsar Java 客户端配置项 -Pulsar 连接器使用[Java 客户端](https://pulsar.apache.org/docs/en/client-libraries-java/)来创建消费实例,相关的配置定义于 Pulsar 的 `ClientConfigurationData` 内。连接器在 `PulsarOptions` 选项中,定义大部分的可供用户定义的配置。 +Pulsar Source 使用 [Java 客户端](https://pulsar.apache.org/docs/zh-CN/client-libraries-java/)来创建消费实例,相关的配置定义于 Pulsar 的 `ClientConfigurationData` 内。在 `PulsarOptions` 选项中,定义大部分的可供用户定义的配置。 {{< generated/pulsar_client_configuration >}} #### Pulsar 管理 API 配置项 -[管理 API](https://pulsar.apache.org/docs/en/admin-api-overview/) 用于查询 topic 的元数据和用正则订阅的时候的 topic 查找,它与 -Java 客户端共享大部分配置。下面列举的配置只供管理 API 使用,连接器也在 `PulsarOptions` 里予以定义。 +[管理 API](https://pulsar.apache.org/docs/zh-CN/admin-api-overview/) 用于查询 Topic 的元数据和用正则订阅的时候的 Topic 查找,它与 Java 客户端共享大部分配置。下面列举的配置只供管理 API 使用,`PulsarOptions` 包含了这些配置 。 {{< generated/pulsar_admin_configuration >}} #### Pulsar 消费者 API 配置项 -Pulsar 提供了消费者 API 和读者 API 两套 API 来进行数据消费,它们可用于不同的业务场景。 -Flink 上的 Pulsar 连接器使用消费者 API 进行消费,它的配置定义于 Pulsar 的 `ConsumerConfigurationData` 内。连接器将其中大部分的可供用户定义的配置定义于 `PulsarSourceOptions` 内。 +Pulsar 提供了消费者 API 和读者 API 两套 API 来进行数据消费,它们可用于不同的业务场景。Flink 上的 Pulsar Source 使用消费者 API 进行消费,它的配置定义于 Pulsar 的 `ConsumerConfigurationData` 内。Pulsar Source 将其中大部分的可供用户定义的配置定义于 `PulsarSourceOptions` 内。 {{< generated/pulsar_consumer_configuration >}} -#### Pulsar 数据源配置项 +#### Pulsar Source配置项 下述配置主要用于性能调优或者是控制消息确认的行为。如非必要,可以不用强制配置。 @@ -291,8 +290,7 @@ Flink 上的 Pulsar 连接器使用消费者 API 进行消费,它的配置定 ### 动态分区发现 -为了能在启动 Flink 任务之后还能发现在 Pulsar 上扩容的分区或者是新创建的 topic,连接器提供了动态分区发现机制。该机制不需要重启 Flink 任务。 -对选项 `PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS` 设置一个正整数即可启用。 +为了能在启动 Flink 任务之后还能发现在 Pulsar 上扩容的分区或者是新创建的 Topic,Pulsar Source 提供了动态分区发现机制。该机制不需要重启 Flink 任务。对选项 `PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS` 设置一个正整数即可启用。 ```java // 10 秒查询一次分区信息 @@ -306,29 +304,29 @@ PulsarSource.builder() ### 事件时间和水位线 -默认情况下,连接器使用 Pulsar 的 `Message` 里面的时间作为解析结果的时间戳。用户可以使用 `WatermarkStrategy` 来自行解析出想要的消息时间,并向下游传递对应的水位线。 +默认情况下,Pulsar Source 使用 Pulsar 的 `Message` 里面的时间作为解析结果的时间戳。用户可以使用 `WatermarkStrategy` 来自行解析出想要的消息时间,并向下游传递对应的水位线。 ```java -env.fromSource(pulsarSource, new CustomWatermarkStrategy(), "Pulsar Source With Custom Watermark Strategy") +env.fromSource(pulsarSource, new CustomWatermarkStrategy(), "Pulsar Source With Custom Watermark Strategy"); ``` -[这篇文档]({{< ref "docs/dev/datastream/event-time/generating_watermarks.md" >}}) 详细讲解了如何定义 `WatermarkStrategy`。 +[这篇文档]({{< ref "docs/dev/datastream/event-time/generating_watermarks.md" >}})详细讲解了如何定义 `WatermarkStrategy`。 ### 消息确认 -一旦在 topic 上创建了订阅,消息便会[存储](https://pulsar.apache.org/docs/zh-CN/concepts-architecture-overview/#%E6%8C%81%E4%B9%85%E5%8C%96%E5%AD%98%E5%82%A8)在 Pulsar 里。即使没有消费者,消息也不会被丢弃。只有当连接器同 Pulsar 确认此条消息已经被消费,该消息才以某种机制会被移除。连接器支持四种订阅方式,它们的消息确认方式也大不相同。 +一旦在 Topic 上创建了订阅,消息便会[存储](https://pulsar.apache.org/docs/zh-CN/concepts-architecture-overview/#%E6%8C%81%E4%B9%85%E5%8C%96%E5%AD%98%E5%82%A8)在 Pulsar 里。即使没有消费者,消息也不会被丢弃。只有当 Pulsar Source 同 Pulsar 确认此条消息已经被消费,该消息才以某种机制会被移除。Pulsar Source 支持四种订阅方式,它们的消息确认方式也大不相同。 #### 独占和灾备订阅下的消息确认 -`独占` 和 `灾备` 订阅下,连接器使用累进式确认方式。确认某条消息已经被处理时,其前面被消费的消息会自动被置为已读。Pulsar 连接器会在 Flink 完成检查点时将对应时刻消费的消息置为已读,以此来保证 Pulsar 状态与 Flink 状态一致。 +`独占` 和 `灾备` 订阅下,Pulsar Source 使用累进式确认方式。确认某条消息已经被处理时,其前面消息会自动被置为已读。Pulsar Source 会在 Flink 完成检查点时将对应时刻消费的消息置为已读,以此来保证 Pulsar 状态与 Flink 状态一致。 -如果用户没有在 Flink 上启用检查点,连接器可以使用周期性提交来将消费状态提交给 Pulsar,使用配置 `PulsarSourceOptions.PULSAR_AUTO_COMMIT_CURSOR_INTERVAL` 来进行定义。 +如果用户没有在 Flink 上启用检查点,Pulsar Source 可以使用周期性提交来将消费状态提交给 Pulsar,使用配置 `PulsarSourceOptions.PULSAR_AUTO_COMMIT_CURSOR_INTERVAL` 来进行定义。 -需要注意的是,此种场景下,Pulsar 连接器并不依赖于提交到 Pulsar 的状态来做容错。消息确认只是为了能在 Pulsar 端看到对应的消费处理情况。 +需要注意的是,此种场景下,Pulsar Source 并不依赖于提交到 Pulsar 的状态来做容错。消息确认只是为了能在 Pulsar 端看到对应的消费处理情况。 #### 共享和 key 共享订阅下的消息确认 -`共享` 和 `key 共享` 需要依次确认每一条消息,所以连接器在 Pulsar 事务里面进行消息确认,然后将事务提交到 Pulsar。 +`共享` 和 `key 共享` 需要依次确认每一条消息,所以 Pulsar Source 在 Pulsar 事务里面进行消息确认,然后将事务提交到 Pulsar。 首先需要在 Pulsar 的 `borker.conf` 文件里面启用事务: @@ -336,11 +334,11 @@ env.fromSource(pulsarSource, new CustomWatermarkStrategy(), "Pulsar Source With transactionCoordinatorEnabled=true ``` -连接器创建的事务的默认超时时间为 3 小时,请确保这个时间大于 Flink 检查点的间隔。用户可以使用 `PulsarSourceOptions.PULSAR_TRANSACTION_TIMEOUT_MILLIS` 来设置事务的超时时间。 +Pulsar Source 创建的事务的默认超时时间为 3 小时,请确保这个时间大于 Flink 检查点的间隔。用户可以使用 `PulsarSourceOptions.PULSAR_TRANSACTION_TIMEOUT_MILLIS` 来设置事务的超时时间。 -如果用户无法启用 Pulsar 的事务,或者是因为项目禁用了检查点,需要将 `PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE` 选项设置为 `true`,消息从 Pulsar 消费后会被立刻置为已读。连接器无法保证此种场景下的消息一致性。 +如果用户无法启用 Pulsar 的事务,或者是因为项目禁用了检查点,需要将 `PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE` 选项设置为 `true`,消息从 Pulsar 消费后会被立刻置为已读。Pulsar Source 无法保证此种场景下的消息一致性。 -连接器在 Pulsar 上使用日志的形式记录某个事务下的消息确认,为了更好的性能,请缩短 Flink 做检查点的间隔。 +Pulsar Source 在 Pulsar 上使用日志的形式记录某个事务下的消息确认,为了更好的性能,请缩短 Flink 做检查点的间隔。 ## 升级至最新的连接器 @@ -351,6 +349,11 @@ transactionCoordinatorEnabled=true ## 问题诊断 -使用 Flink 和 Pulsar 交互时如果遇到问题,一定要牢记 Flink 只使用了 Pulsar 的[Java 客户端](https://pulsar.apache.org/docs/zh-CN/client-libraries-java/) 和[管理 API](https://pulsar.apache.org/docs/zh-CN/admin-api-overview/)。用户遇到的问题很有可能与 Flink 无关,请先升级 Pulsar 的版本、Pulsar 客户端的版本、或者修改 Pulsar 的配置,Pulsar 连接器的配置来尝试解决问题。 +使用 Flink 和 Pulsar 交互时如果遇到问题,由于 Flink 内部实现只是基于 Pulsar 的 [Java 客户端](https://pulsar.apache.org/docs/zh-CN/client-libraries-java/)和[管理 API](https://pulsar.apache.org/docs/zh-CN/admin-api-overview/) 而开发的。 + +用户遇到的问题可能与 Flink 无关,请先升级 Pulsar 的版本、Pulsar 客户端的版本,或者修改 Pulsar 的配置、Pulsar 连接器的配置来尝试解决问题。 + +### 在 Java 11 上使用不稳定 +Pulsar connector 在 Java 11 中有一些尚未修复的问题。我们当前推荐在 Java 8 环境中运行Pulsar connector. {{< top >}} diff --git a/docs/content/docs/connectors/datastream/pulsar.md b/docs/content/docs/connectors/datastream/pulsar.md index 307170129dfe6..6f48170a657c5 100644 --- a/docs/content/docs/connectors/datastream/pulsar.md +++ b/docs/content/docs/connectors/datastream/pulsar.md @@ -24,18 +24,17 @@ under the License. # Apache Pulsar Connector -Flink provides an [Apache Pulsar](https://pulsar.apache.org) source connector for reading data from Pulsar topics with exactly-once guarantees. +Flink provides an [Apache Pulsar](https://pulsar.apache.org) connector for reading and writing data from and to Pulsar topics with exactly-once guarantees. ## Dependency -You can use the connector with Pulsar 2.7.0 or higher. However, the Pulsar source connector supports -Pulsar [transactions](https://pulsar.apache.org/docs/en/txn-what/), -it is recommended to use Pulsar 2.8.0 or higher releases. -For details on Pulsar compatibility, please refer to the [PIP-72](https://github.com/apache/pulsar/wiki/PIP-72%3A-Introduce-Pulsar-Interface-Taxonomy%3A-Audience-and-Stability-Classification). +You can use the connector with the Pulsar 2.9.0 or higher. Because the Pulsar connector supports +Pulsar [transactions](https://pulsar.apache.org/docs/en/txn-what/), it is recommended to use the Pulsar 2.10.0 or higher. +Details on Pulsar compatibility can be found in [PIP-72](https://github.com/apache/pulsar/wiki/PIP-72%3A-Introduce-Pulsar-Interface-Taxonomy%3A-Audience-and-Stability-Classification). {{< artifact flink-connector-pulsar withScalaVersion >}} -Flink's streaming connectors are not currently part of the binary distribution. +Flink's streaming connectors are not part of the binary distribution. See how to link with them for cluster execution [here]({{< ref "docs/dev/datastream/project-configuration" >}}). ## Pulsar Source @@ -49,12 +48,12 @@ If you want to use the legacy `SourceFunction` or on Flink 1.13 or lower release ### Usage -Pulsar source provides a builder class for constructing an instance of PulsarSource. The code snippet below shows -how to build a PulsarSource to consume messages from the earliest cursor of topic "persistent://public/default/my-topic", -with **Exclusive** subscription `my-subscription` and deserialize the raw payload of the messages as strings. +The Pulsar source provides a builder class for constructing a PulsarSource instance. The code snippet below builds a PulsarSource instance. It consumes messages from the earliest cursor of the topic +"persistent://public/default/my-topic" in **Exclusive** subscription type (`my-subscription`) +and deserializes the raw payload of the messages as strings. ```java -PulsarSource pulsarSource = PulsarSource.builder() +PulsarSource source = PulsarSource.builder() .setServiceUrl(serviceUrl) .setAdminUrl(adminUrl) .setStartCursor(StartCursor.earliest()) @@ -69,13 +68,17 @@ env.fromSource(source, WatermarkStrategy.noWatermarks(), "Pulsar Source"); The following properties are **required** for building a PulsarSource: -- Pulsar service url, configured by `setServiceUrl(String)` -- Pulsar service http url (aka. admin url), configured by `setAdminUrl(String)` +- Pulsar service URL, configured by `setServiceUrl(String)` +- Pulsar service HTTP URL (also known as admin URL), configured by `setAdminUrl(String)` - Pulsar subscription name, configured by `setSubscriptionName(String)` - Topics / partitions to subscribe, see the following - [Topic-partition subscription](#topic-partition-subscription) for more details. + [topic-partition subscription](#topic-partition-subscription) for more details. - Deserializer to parse Pulsar messages, see the following - [Deserializer](#deserializer) for more details. + [deserializer](#deserializer) for more details. + +It is recommended to set the consumer name in Pulsar Source by `setConsumerName(String)`. +This sets a unique name for the Flink connector in the Pulsar statistic dashboard. +You can use it to monitor the performance of your Flink connector and applications. ### Topic-partition Subscription @@ -83,105 +86,106 @@ Pulsar source provide two ways of topic-partition subscription: - Topic list, subscribing messages from all partitions in a list of topics. For example: ```java - PulsarSource.builder().setTopics("some-topic1", "some-topic2") + PulsarSource.builder().setTopics("some-topic1", "some-topic2"); // Partition 0 and 2 of topic "topic-a" - PulsarSource.builder().setTopics("topic-a-partition-0", "topic-a-partition-2") + PulsarSource.builder().setTopics("topic-a-partition-0", "topic-a-partition-2"); ``` - Topic pattern, subscribing messages from all topics whose name matches the provided regular expression. For example: ```java - PulsarSource.builder().setTopicPattern("topic-*") + PulsarSource.builder().setTopicPattern("topic-*"); ``` #### Flexible Topic Naming -Since Pulsar 2.0, all topic names internally have the form `{persistent|non-persistent}://tenant/namespace/topic`. +Since Pulsar 2.0, all topic names internally are in a form of `{persistent|non-persistent}://tenant/namespace/topic`. Now, for partitioned topics, you can use short names in many cases (for the sake of simplicity). The flexible naming system stems from the fact that there is now a default topic type, tenant, and namespace in a Pulsar cluster. -Topic property | Default -:------------|:------- -topic type | `persistent` -tenant | `public` -namespace | `default` +| Topic property | Default | +|:---------------|:-------------| +| topic type | `persistent` | +| tenant | `public` | +| namespace | `default` | -This table lists a mapping relationship between your input topic name and translated topic name: +This table lists a mapping relationship between your input topic name and the translated topic name: -Input topic name | Translated topic name -:----------------|:--------------------- -`my-topic` | `persistent://public/default/my-topic` -`my-tenant/my-namespace/my-topic` | `persistent://my-tenant/my-namespace/my-topic` +| Input topic name | Translated topic name | +|:----------------------------------|:-----------------------------------------------| +| `my-topic` | `persistent://public/default/my-topic` | +| `my-tenant/my-namespace/my-topic` | `persistent://my-tenant/my-namespace/my-topic` | {{< hint warning >}} -For non-persistent topics, you need to continue to specify the entire topic name, +For non-persistent topics, you need to specify the entire topic name, as the default-based rules do not apply for non-partitioned topics. -Thus, you cannot use a short name like `non-persistent://my-topic` and would need to use `non-persistent://public/default/my-topic` instead. +Thus, you cannot use a short name like `non-persistent://my-topic` and need to use `non-persistent://public/default/my-topic` instead. {{< /hint >}} #### Subscribing Pulsar Topic Partition Internally, Pulsar divides a partitioned topic as a set of non-partitioned topics according to the partition size. -For example, if a `simple-string` topic with 3 partitions is created under the `sample` tenant with `flink` namespace. +For example, if a `simple-string` topic with 3 partitions is created under the `sample` tenant with the `flink` namespace. The topics on Pulsar would be: -Topic name | Partitioned -:--------- | :---------- -`persistent://sample/flink/simple-string` | Y -`persistent://sample/flink/simple-string-partition-0` | N -`persistent://sample/flink/simple-string-partition-1` | N -`persistent://sample/flink/simple-string-partition-2` | N +| Topic name | Partitioned | +|:------------------------------------------------------|:------------| +| `persistent://sample/flink/simple-string` | Y | +| `persistent://sample/flink/simple-string-partition-0` | N | +| `persistent://sample/flink/simple-string-partition-1` | N | +| `persistent://sample/flink/simple-string-partition-2` | N | You can directly consume messages from the topic partitions by using the non-partitioned topic names above. -For example, use `PulsarSource.builder().setTopics("sample/flink/simple-string-partition-1", "sample/flink/simple-string-partition-2")` would consume the partitions 1 and 2 of the `sample/flink/simple-string` topic. +For example, use `PulsarSource.builder().setTopics("sample/flink/simple-string-partition-1", "sample/flink/simple-string-partition-2")` +would consume the partitions 1 and 2 of the `sample/flink/simple-string` topic. -#### RegexSubscriptionMode for Topic Pattern +#### Setting Topic Patterns -Pulsar connector extracts the topic type (`persistent` or `non-persistent`) from the given topic pattern. -For example, `PulsarSource.builder().setTopicPattern("non-persistent://my-topic*")` would be `non-persistent`. -The topic type would be `persistent` if you do not provide the topic type in the regular expression. +The Pulsar source extracts the topic type (`persistent` or `non-persistent`) from the provided topic pattern. +For example, you can use the `PulsarSource.builder().setTopicPattern("non-persistent://my-topic*")` to specify a `non-persistent` topic. +By default, a `persistent` topic is created if you do not specify the topic type in the regular expression. -To consume both `persistent` and `non-persistent` topics based on the topic pattern, -you can use `setTopicPattern("topic-*", RegexSubscriptionMode.AllTopics)`. -Pulsar connector would filter the available topics by the `RegexSubscriptionMode`. +You can use `setTopicPattern("topic-*", RegexSubscriptionMode.AllTopics)` to consume +both `persistent` and `non-persistent` topics based on the topic pattern. +The Pulsar source would filter the available topics by the `RegexSubscriptionMode`. ### Deserializer -A deserializer (Deserialization schema) is required for parsing Pulsar messages. The deserializer is -configured by `setDeserializationSchema(PulsarDeserializationSchema)`. +A deserializer (`PulsarDeserializationSchema`) is for decoding Pulsar messages from bytes. +You can configure the deserializer using `setDeserializationSchema(PulsarDeserializationSchema)`. The `PulsarDeserializationSchema` defines how to deserialize a Pulsar `Message`. If only the raw payload of a message (message data in bytes) is needed, -you can use the predefined `PulsarDeserializationSchema`. Pulsar connector provides three types of implementation. +you can use the predefined `PulsarDeserializationSchema`. Pulsar connector provides three implementation methods. - Decode the message by using Pulsar's [Schema](https://pulsar.apache.org/docs/en/schema-understand/). ```java // Primitive types - PulsarDeserializationSchema.pulsarSchema(Schema) + PulsarDeserializationSchema.pulsarSchema(Schema); // Struct types (JSON, Protobuf, Avro, etc.) - PulsarDeserializationSchema.pulsarSchema(Schema, Class) + PulsarDeserializationSchema.pulsarSchema(Schema, Class); // KeyValue type - PulsarDeserializationSchema.pulsarSchema(Schema, Class, Class) + PulsarDeserializationSchema.pulsarSchema(Schema, Class, Class); ``` - Decode the message by using Flink's `DeserializationSchema` ```java - PulsarDeserializationSchema.flinkSchema(DeserializationSchema) + PulsarDeserializationSchema.flinkSchema(DeserializationSchema); ``` - Decode the message by using Flink's `TypeInformation` ```java - PulsarDeserializationSchema.flinkTypeInfo(TypeInformation, ExecutionConfig) + PulsarDeserializationSchema.flinkTypeInfo(TypeInformation, ExecutionConfig); ``` Pulsar `Message` contains some [extra properties](https://pulsar.apache.org/docs/en/concepts-messaging/#messages), -such as message key, message publish time, message time, application defined key/value pairs that will be attached to the message, etc. -These properties could be acquired by the `Message` interface. +such as message key, message publish time, message time, and application-defined key/value pairs etc. +These properties could be defined in the `Message` interface. If you want to deserialize the Pulsar message by these properties, you need to implement `PulsarDeserializationSchema`. -And ensure that the `TypeInformation` from the `PulsarDeserializationSchema.getProducedType()` must be correct. -Flink would use this `TypeInformation` for passing the messages to downstream operators. +Ensure that the `TypeInformation` from the `PulsarDeserializationSchema.getProducedType()` is correct. +Flink uses this `TypeInformation` to pass the messages to downstream operators. ### Pulsar Subscriptions @@ -196,110 +200,126 @@ The subscription name is required for consuming messages. Pulsar connector suppo There is no difference between `Exclusive` and `Failover` in the Pulsar connector. When a Flink reader crashes, all (non-acknowledged and subsequent) messages are redelivered to the available Flink readers. -By default, if no subscription type is defined, Pulsar source uses `Shared` subscription. +By default, if no subscription type is defined, Pulsar source uses the `Shared` subscription type. ```java // Shared subscription with name "my-shared" -PulsarSource.builder().setSubscriptionName("my-shared") +PulsarSource.builder().setSubscriptionName("my-shared"); // Exclusive subscription with name "my-exclusive" -PulsarSource.builder().setSubscriptionName("my-exclusive").setSubscriptionType(SubscriptionType.Exclusive) +PulsarSource.builder().setSubscriptionName("my-exclusive").setSubscriptionType(SubscriptionType.Exclusive); ``` -If you want to use `Key_Shared` subscription type on the Pulsar connector. Ensure that you provide a `RangeGenerator` implementation. -The `RangeGenerator` generates a set of key hash ranges so that -a respective reader subtask will only dispatch messages where the hash of the message key is contained in the specified range. +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. -Pulsar connector would use a `UniformRangeGenerator` which would divides the range by the Flink source parallelism -if no `RangeGenerator` is provided in the `Key_Shared` subscription type. +The Pulsar connector uses `UniformRangeGenerator` that divides the range by the Flink source +parallelism if no `RangeGenerator` is provided in the `Key_Shared` subscription type. ### Starting Position -Pulsar source is able to consume messages starting from different positions by `setStartCursor(StartCursor)`. +The Pulsar source is able to consume messages starting from different positions by setting the `setStartCursor(StartCursor)` option. Built-in start cursors include: - Start from the earliest available message in the topic. ```java - StartCursor.earliest() + StartCursor.earliest(); ``` - Start from the latest available message in the topic. ```java - StartCursor.latest() + StartCursor.latest(); ``` - Start from a specified message between the earliest and the latest. - Pulsar connector would consume from the latest available message if the message id doesn't exist. +The Pulsar connector consumes from the latest available message if the message ID does not exist. The start message is included in consuming result. ```java - StartCursor.fromMessageId(MessageId) + StartCursor.fromMessageId(MessageId); ``` - Start from a specified message between the earliest and the latest. - Pulsar connector would consume from the latest available message if the message id doesn't exist. +The Pulsar connector consumes from the latest available message if the message ID doesn't exist. Include or exclude the start message by using the second boolean parameter. ```java - StartCursor.fromMessageId(MessageId, boolean) + StartCursor.fromMessageId(MessageId, boolean); + ``` +- Start from the specified message publish time by `Message.getPublishTime()`. +This method is deprecated because the name is totally wrong which may cause confuse. +You can use `StartCursor.fromPublishTime(long)` instead. + ```java + StartCursor.fromMessageTime(long); ``` -- Start from the specified message time by `Message.getPublishTime()`. +- Start from the specified message publish time by `Message.getPublishTime()`. ```java - StartCursor.fromMessageTime(long) + StartCursor.fromPublishTime(long); ``` {{< hint info >}} Each Pulsar message belongs to an ordered sequence on its topic. The sequence ID (`MessageId`) of the message is ordered in that sequence. -`MessageId` contains some extra information (the ledger, entry, partition) on how the message is stored, +The `MessageId` contains some extra information (the ledger, entry, partition) about how the message is stored, you can create a `MessageId` by using `DefaultImplementation.newMessageId(long ledgerId, long entryId, int partitionIndex)`. {{< /hint >}} ### Boundedness -Pulsar source supports streaming and batch running modes. -By default, the `PulsarSource` is set to run in the streaming mode. +The Pulsar source supports streaming and batch execution mode. +By default, the `PulsarSource` is configured for unbounded data. -In streaming mode, Pulsar source never stops until a Flink job fails or is cancelled. However, -you can set Pulsar source stopping at a stop position by using ```setUnboundedStopCursor(StopCursor)```. -The Pulsar source will finish when all partitions reach their specified stop positions. +For unbounded data the Pulsar source never stops until a Flink job is stopped or failed. +You can use the `setUnboundedStopCursor(StopCursor)` to set the Pulsar source to stop at a specific stop position. -You can use ```setBoundedStopCursor(StopCursor)``` to specify a stop position so that the Pulsar source can run in the batch mode. -When all partitions have reached their stop positions, the source will finish. +You can use `setBoundedStopCursor(StopCursor)` to specify a stop position for bounded data. Built-in stop cursors include: -- Connector will never stop consuming. +- The Pulsar source never stops consuming messages. ```java - StopCursor.never() + StopCursor.never(); ``` -- Stop at the latest available message in Pulsar when the connector starts consuming. +- Stop at the latest available message when the Pulsar source starts consuming messages. ```java - StopCursor.latest() + StopCursor.latest(); ``` -- Stop when connector meet a given message, or stop at a message which is produced after this given message. +- Stop when the connector meets a given message, or stop at a message which is produced after this given message. ```java - StopCursor.atMessageId(MessageId) + StopCursor.atMessageId(MessageId); ``` -- Stop but include the given message in consuming result. +- Stop but include the given message in the consuming result. ```java - StopCursor.afterMessageId(MessageId) + StopCursor.afterMessageId(MessageId); ``` -- Stop at the specified message time by `Message.getPublishTime()`. +- Stop at the specified event time by `Message.getEventTime()`. The message with the +given event time won't be included in the consuming result. + ```java + StopCursor.atEventTime(long); + ``` +- Stop after the specified event time by `Message.getEventTime()`. The message with the +given event time will be included in the consuming result. + ```java + StopCursor.afterEventTime(long); + ``` +- Stop at the specified publish time by `Message.getPublishTime()`. The message with the +given publish time won't be included in the consuming result. ```java StopCursor.atPublishTime(long); ``` - -{{< hint warning >}} -StopCursor.atEventTime(long) is now deprecated. - {{< /hint >}} +- Stop after the specified publish time by `Message.getPublishTime()`. The message with the + given publish time will be included in the consuming result. + ```java + StopCursor.afterPublishTime(long); + ``` ### Source Configurable Options In addition to configuration options described above, you can set arbitrary options for `PulsarClient`, -`PulsarAdmin`, Pulsar `Consumer` and `PulsarSource` by using `setConfig(ConfigOption, T)` and `setConfig(Configuration)`. +`PulsarAdmin`, Pulsar `Consumer` and `PulsarSource` by using `setConfig(ConfigOption, T)`, +`setConfig(Configuration)` and `setConfig(Properties)`. #### PulsarClient Options -Pulsar connector use the [client API](https://pulsar.apache.org/docs/en/client-libraries-java/) -to create the `Consumer` instance. Pulsar connector extracts most parts of Pulsar's `ClientConfigurationData`, +The Pulsar connector uses the [client API](https://pulsar.apache.org/docs/en/client-libraries-java/) +to create the `Consumer` instance. The Pulsar connector extracts most parts of Pulsar's `ClientConfigurationData`, which is required for creating a `PulsarClient`, as Flink configuration options in `PulsarOptions`. {{< generated/pulsar_client_configuration >}} @@ -307,8 +327,9 @@ which is required for creating a `PulsarClient`, as Flink configuration options #### PulsarAdmin Options The [admin API](https://pulsar.apache.org/docs/en/admin-api-overview/) is used for querying topic metadata -and for discovering the desired topics when Pulsar connector uses topic pattern subscription. It would share most part of the -configuration options with the client API. The configuration options listed here are only used in the admin API. +and for discovering the desired topics when the Pulsar connector uses topic-pattern subscription. +It shares most part of the configuration options with the client API. +The configuration options listed here are only used in the admin API. They are also defined in `PulsarOptions`. {{< generated/pulsar_admin_configuration >}} @@ -316,34 +337,34 @@ They are also defined in `PulsarOptions`. #### Pulsar Consumer Options In general, Pulsar provides the Reader API and Consumer API for consuming messages in different scenarios. -Flink's Pulsar connector uses the Consumer API. It extracts most parts of Pulsar's `ConsumerConfigurationData` as Flink configuration options in `PulsarSourceOptions`. +The Pulsar connector uses the Consumer API. It extracts most parts of Pulsar's `ConsumerConfigurationData` as Flink configuration options in `PulsarSourceOptions`. {{< generated/pulsar_consumer_configuration >}} #### PulsarSource Options The configuration options below are mainly used for customizing the performance and message acknowledgement behavior. -You can just leave them alone if you do not meet any performance issues. +You can ignore them if you do not have any performance issues. {{< generated/pulsar_source_configuration >}} ### Dynamic Partition Discovery To handle scenarios like topic scaling-out or topic creation without restarting the Flink -job, Pulsar source can be configured to periodically discover new partitions under provided -topic-partition subscribing pattern. To enable partition discovery, set a non-negative value for -the option `PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS`: +job, the Pulsar source periodically discover new partitions under a provided +topic-partition subscription pattern. To enable partition discovery, you can set a non-negative value for +the `PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS` option: ```java // discover new partitions per 10 seconds PulsarSource.builder() - .setConfig(PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, 10000); + .setConfig(PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, 10000); ``` {{< hint warning >}} -- Partition discovery is **enabled** by default. Pulsar connector would query the topic metadata every 30 seconds. -- You need to set the partition discovery interval to a negative value to disable this feature. -- The partition discovery would be disabled in batch mode even if you set this option with a non-negative value. +- Partition discovery is **enabled** by default. The Pulsar connector queries the topic metadata every 30 seconds. +- To disable partition discovery, you need to set a negative partition discovery interval. +- Partition discovery is disabled for bounded data even if you set this option with a non-negative value. {{< /hint >}} ### Event Time and Watermarks @@ -353,7 +374,7 @@ You can define your own `WatermarkStrategy` to extract the event time from the m and emit the watermark downstream: ```java -env.fromSource(pulsarSource, new CustomWatermarkStrategy(), "Pulsar Source With Custom Watermark Strategy") +env.fromSource(pulsarSource, new CustomWatermarkStrategy(), "Pulsar Source With Custom Watermark Strategy"); ``` [This documentation]({{< ref "docs/dev/datastream/event-time/generating_watermarks.md" >}}) describes @@ -363,7 +384,7 @@ details about how to define a `WatermarkStrategy`. When a subscription is created, Pulsar [retains](https://pulsar.apache.org/docs/en/concepts-architecture-overview/#persistent-storage) all messages, even if the consumer is disconnected. The retained messages are discarded only when the connector acknowledges that all these messages are processed successfully. -Pulsar connector supports four subscription types, which makes the acknowledgement behaviors variety among different subscriptions. +The Pulsar connector supports four subscription types, which makes the acknowledgement behaviors vary among different subscriptions. #### Acknowledgement on Exclusive and Failover Subscription Types @@ -371,18 +392,18 @@ Pulsar connector supports four subscription types, which makes the acknowledgeme the latest successfully consumed message. All the message before the given message are marked with a consumed status. -Pulsar source acknowledges the current consuming message when checkpoints are **completed**, -to ensure the consistency between Flink's checkpoint state and committed position on Pulsar brokers. +The Pulsar source acknowledges the current consuming message when checkpoints are **completed**, +to ensure the consistency between Flink's checkpoint state and committed position on the Pulsar brokers. If checkpointing is disabled, Pulsar source periodically acknowledges messages. -You can set the acknowledgement period by using the `PulsarSourceOptions.PULSAR_AUTO_COMMIT_CURSOR_INTERVAL` option. +You can use the `PulsarSourceOptions.PULSAR_AUTO_COMMIT_CURSOR_INTERVAL` option to set the acknowledgement period. Pulsar source does **NOT** rely on committed positions for fault tolerance. -Acknowledging messages is only for exposing the progress of consumer and monitoring on these two subscription types. +Acknowledging messages is only for exposing the progress of consumers and monitoring on these two subscription types. #### Acknowledgement on Shared and Key_Shared Subscription Types -`Shared` and `Key_Shared` subscription types need to acknowledge messages one by one. You can acknowledge +In `Shared` and `Key_Shared` subscription types, messages are acknowledged one by one. You can acknowledge a message in a transaction and commit it to Pulsar. You should enable transaction in the Pulsar `borker.conf` file when using these two subscription types in connector: @@ -391,14 +412,15 @@ You should enable transaction in the Pulsar `borker.conf` file when using these transactionCoordinatorEnabled=true ``` -Pulsar transaction would be created with 3 hours as the timeout by default. Make sure that timeout > checkpoint interval + maximum recovery time. -A shorter checkpoint interval would increase the consuming performance. -You can change the transaction timeout by using the `PulsarSourceOptions.PULSAR_TRANSACTION_TIMEOUT_MILLIS` option. +The default timeout for Pulsar transactions is 3 hours. +Make sure that that timeout is greater than checkpoint interval + maximum recovery time. +A shorter checkpoint interval indicates a better consuming performance. +You can use the `PulsarSourceOptions.PULSAR_TRANSACTION_TIMEOUT_MILLIS` option to change the transaction timeout. If checkpointing is disabled or you can not enable the transaction on Pulsar broker, you should set `PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE` to `true`. -The message would be immediately acknowledged after consuming. -We can not promise consistency in this scenario. +The message is immediately acknowledged after consuming. +No consistency guarantees can be made in this scenario. {{< hint info >}} All acknowledgements in a transaction are recorded in the Pulsar broker side. @@ -411,7 +433,7 @@ The Pulsar connector does not store any state on the Flink side. The Pulsar conn For Pulsar, you additionally need to know these limitations: * Do not upgrade the Pulsar connector and Pulsar broker version at the same time. -* Always use a newer Pulsar client with Pulsar connector for consuming message from Pulsar. +* Always use a newer Pulsar client with Pulsar connector to consume messages from Pulsar. ## Troubleshooting @@ -426,7 +448,7 @@ reconfiguring Pulsar brokers or reconfiguring Pulsar connector in Flink. When the Pulsar source connector reads from a low volume topic, users might observe a 10 seconds delay between messages. Pulsar buffers messages from topics by default. Before emitting to downstream operators, the number of buffered records must be equal or larger than `PulsarSourceOptions.PULSAR_MAX_FETCH_RECORDS`. If the data volume is low, it could be that filling up the number of buffered records takes longer than `PULSAR_MAX_FETCH_TIME` (default to 10 seconds). If that's the case, it means that only after this time has passed the messages will be emitted. -To avoid this behaviour, you need to change either the buffered records or the waiting time. +To avoid this behaviour, you need to change either the buffered records or the waiting time. diff --git a/docs/layouts/shortcodes/generated/pulsar_consumer_configuration.html b/docs/layouts/shortcodes/generated/pulsar_consumer_configuration.html index bc8b6df40608b..4e05b270de4b0 100644 --- a/docs/layouts/shortcodes/generated/pulsar_consumer_configuration.html +++ b/docs/layouts/shortcodes/generated/pulsar_consumer_configuration.html @@ -140,12 +140,6 @@ Boolean If enabled, the consumer will automatically retry messages. - -
pulsar.consumer.subscriptionInitialPosition
- Latest -

Enum

- Initial position at which to set cursor when subscribing to a topic at first time.

Possible values:
  • "Latest"
  • "Earliest"
-
pulsar.consumer.subscriptionMode
Durable diff --git a/flink-connectors/flink-connector-pulsar/pom.xml b/flink-connectors/flink-connector-pulsar/pom.xml index 21482fcd2d2b9..193b3f8be63ab 100644 --- a/flink-connectors/flink-connector-pulsar/pom.xml +++ b/flink-connectors/flink-connector-pulsar/pom.xml @@ -36,13 +36,15 @@ under the License. jar - 2.8.0 + 2.9.1 0.6.1 3.20.2 - 3.11 - 1.33.0 + 3.11 + 3.6.3 + 4.1.72.Final + 1.33.0 @@ -153,12 +155,22 @@ under the License. ${pulsar.version} test + org.apache.commons commons-lang3 - ${commons-lang3.version} + ${pulsar-commons-lang3.version} + test + + + + + + org.apache.zookeeper + zookeeper + ${pulsar-zookeeper.version} test @@ -170,21 +182,63 @@ under the License. pulsar-client-all ${pulsar.version} + + com.sun.activation + javax.activation + + + jakarta.activation + jakarta.activation-api + + + jakarta.ws.rs + jakarta.ws.rs-api + + + jakarta.xml.bind + jakarta.xml.bind-api + + + javax.validation + validation-api + + + javax.xml.bind + jaxb-api + + + net.jcip + jcip-annotations + org.apache.pulsar pulsar-package-core + + com.beust + jcommander + - + + io.grpc grpc-bom - ${grpc.version} + ${pulsar-grpc.version} + pom + import + + + + + io.netty + netty-bom + ${pulsar-netty.version} pom import @@ -207,7 +261,9 @@ under the License. 1 - -Xms256m -Xmx2048m -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit -Duser.country=US -Duser.language=en + -Xms256m -Xmx2048m -Dmvn.forkNumber=${surefire.forkNumber} + -XX:-UseGCOverheadLimit -Duser.country=US -Duser.language=en + @@ -229,7 +285,8 @@ under the License. ${project.build.directory}/generated-test-sources/protobuf/java - com.google.protobuf:protoc:3.5.1:exe:${os.detected.classifier} + + com.google.protobuf:protoc:${protoc.version}:exe:${os.detected.classifier} @@ -275,6 +332,7 @@ under the License. **/testutils/** META-INF/LICENSE META-INF/NOTICE + containers/txnStandalone.conf @@ -298,6 +356,7 @@ under the License. **/testutils/** META-INF/LICENSE META-INF/NOTICE + containers/txnStandalone.conf diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarConfigUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarConfigUtils.java index ae5d784ccdf73..7e978d890a36f 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarConfigUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarConfigUtils.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; import org.apache.pulsar.client.admin.PulsarAdmin; @@ -31,6 +32,8 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.impl.auth.AuthenticationDisabled; +import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.TreeSet; import java.util.function.Consumer; @@ -40,6 +43,7 @@ import static java.util.concurrent.TimeUnit.NANOSECONDS; import static java.util.concurrent.TimeUnit.SECONDS; import static java.util.function.Function.identity; +import static java.util.stream.Collectors.toList; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_AUTH_PARAMS; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_AUTH_PARAM_MAP; @@ -273,4 +277,32 @@ public static void setOptionValue( setter.accept(value); } } + + /** + * Get the option value by a prefix. We would return an empty map if the option doesn't exist. + */ + public static Map getProperties( + Configuration configuration, ConfigOption> option) { + Map properties = new HashMap<>(); + if (configuration.contains(option)) { + Map map = configuration.get(option); + properties.putAll(map); + } + + // Filter the sub config option. These options could be provided by SQL. + String prefix = option.key() + "."; + List keys = + configuration.keySet().stream() + .filter(key -> key.startsWith(prefix) && key.length() > prefix.length()) + .collect(toList()); + + // Put these config options' value into return result. + for (String key : keys) { + ConfigOption o = ConfigOptions.key(key).stringType().noDefaultValue(); + String value = configuration.get(o); + properties.put(key.substring(prefix.length()), value); + } + + return properties; + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/utils/PulsarTransactionUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/utils/PulsarTransactionUtils.java deleted file mode 100644 index ef54779ae6464..0000000000000 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/utils/PulsarTransactionUtils.java +++ /dev/null @@ -1,118 +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.common.utils; - -import org.apache.flink.annotation.Internal; - -import org.apache.pulsar.client.api.transaction.Transaction; -import org.apache.pulsar.client.api.transaction.TxnID; -import org.apache.pulsar.client.impl.transaction.TransactionImpl; - -import java.lang.reflect.Field; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Transaction was introduced into pulsar since 2.7.0, but the interface {@link Transaction} didn't - * provide a id method until 2.8.1. We have to add this util for acquiring the {@link TxnID} for - * compatible consideration. - * - *

TODO Remove this hack after pulsar 2.8.1 release. - */ -@Internal -@SuppressWarnings("java:S3011") -public final class PulsarTransactionUtils { - - private static volatile Field mostBitsField; - private static volatile Field leastBitsField; - - private PulsarTransactionUtils() { - // No public constructor - } - - public static TxnID getId(Transaction transaction) { - // 2.8.1 and after. - try { - Method getId = Transaction.class.getDeclaredMethod("getTxnID"); - return (TxnID) getId.invoke(transaction); - } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) { - // 2.8.0 and before. - TransactionImpl impl = (TransactionImpl) transaction; - Long txnIdMostBits = getTxnIdMostBits(impl); - Long txnIdLeastBits = getTxnIdLeastBits(impl); - - checkNotNull(txnIdMostBits, "Failed to get txnIdMostBits"); - checkNotNull(txnIdLeastBits, "Failed to get txnIdLeastBits"); - - return new TxnID(txnIdMostBits, txnIdLeastBits); - } - } - - private static Long getTxnIdMostBits(TransactionImpl transaction) { - if (mostBitsField == null) { - synchronized (PulsarTransactionUtils.class) { - if (mostBitsField == null) { - try { - mostBitsField = TransactionImpl.class.getDeclaredField("txnIdMostBits"); - mostBitsField.setAccessible(true); - } catch (NoSuchFieldException e) { - // Nothing to do for this exception. - } - } - } - } - - if (mostBitsField != null) { - try { - return (Long) mostBitsField.get(transaction); - } catch (IllegalAccessException e) { - // Nothing to do for this exception. - } - } - - return null; - } - - private static Long getTxnIdLeastBits(TransactionImpl transaction) { - if (leastBitsField == null) { - synchronized (PulsarTransactionUtils.class) { - if (leastBitsField == null) { - try { - leastBitsField = TransactionImpl.class.getDeclaredField("txnIdLeastBits"); - leastBitsField.setAccessible(true); - } catch (NoSuchFieldException e) { - // Nothing to do for this exception. - } - } - } - } - - if (leastBitsField != null) { - try { - return (Long) leastBitsField.get(transaction); - } catch (IllegalAccessException e) { - // Nothing to do for this exception. - } - } - - return null; - } -} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java index f2b2e39eac627..4ada27cecdc37 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java @@ -32,7 +32,8 @@ import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumStateSerializer; import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumerator; -import org.apache.flink.connector.pulsar.source.enumerator.SplitsAssignmentState; +import org.apache.flink.connector.pulsar.source.enumerator.assigner.SplitAssigner; +import org.apache.flink.connector.pulsar.source.enumerator.assigner.SplitAssignerFactory; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; import org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; @@ -144,8 +145,7 @@ public SourceReader createReader(SourceReaderContext @Override public SplitEnumerator createEnumerator( SplitEnumeratorContext enumContext) { - SplitsAssignmentState assignmentState = - new SplitsAssignmentState(stopCursor, sourceConfiguration); + SplitAssigner splitAssigner = SplitAssignerFactory.create(stopCursor, sourceConfiguration); return new PulsarSourceEnumerator( subscriber, startCursor, @@ -153,15 +153,15 @@ public SplitEnumerator createEnumer configuration, sourceConfiguration, enumContext, - assignmentState); + splitAssigner); } @Override public SplitEnumerator restoreEnumerator( SplitEnumeratorContext enumContext, PulsarSourceEnumState checkpoint) { - SplitsAssignmentState assignmentState = - new SplitsAssignmentState(stopCursor, sourceConfiguration, checkpoint); + SplitAssigner splitAssigner = + SplitAssignerFactory.create(stopCursor, sourceConfiguration, checkpoint); return new PulsarSourceEnumerator( subscriber, startCursor, @@ -169,7 +169,7 @@ public SplitEnumerator restoreEnume configuration, sourceConfiguration, enumContext, - assignmentState); + splitAssigner); } @Override 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 19c93f4014d24..432b213713a56 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 @@ -28,6 +28,7 @@ import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; import org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; +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; @@ -51,13 +52,13 @@ import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ENABLE_TRANSACTION; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_CONSUMER_NAME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_TYPE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_TRANSACTION_TIMEOUT_MILLIS; import static org.apache.flink.connector.pulsar.source.config.PulsarSourceConfigUtils.checkConfigurations; -import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.distinctTopics; import static org.apache.flink.util.InstantiationUtil.isSerializable; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -97,7 +98,7 @@ *

To stop the connector user has to disable the auto partition discovery. As auto partition * discovery always expected new splits to come and not exiting. To disable auto partition * discovery, use builder.setConfig({@link - * PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS}, -1). + * PulsarSourceOptions#PULSAR_PARTITION_DISCOVERY_INTERVAL_MS}, -1). * *

{@code
  * PulsarSource source = PulsarSource
@@ -130,6 +131,7 @@ public final class PulsarSourceBuilder {
         this.configuration = new Configuration();
         this.startCursor = StartCursor.defaultStartCursor();
         this.stopCursor = StopCursor.defaultStopCursor();
+        this.boundedness = Boundedness.CONTINUOUS_UNBOUNDED;
     }
 
     /**
@@ -210,7 +212,8 @@ public PulsarSourceBuilder setTopics(String... topics) {
      */
     public PulsarSourceBuilder setTopics(List topics) {
         ensureSubscriberIsNull("topics");
-        this.subscriber = PulsarSubscriber.getTopicListSubscriber(distinctTopics(topics));
+        List distinctTopics = TopicNameUtils.distinctTopics(topics);
+        this.subscriber = PulsarSubscriber.getTopicListSubscriber(distinctTopics);
         return this;
     }
 
@@ -272,6 +275,14 @@ public PulsarSourceBuilder setTopicPattern(
         return this;
     }
 
+    /**
+     * The consumer name is informative, and it can be used to identify a particular consumer
+     * instance from the topic stats.
+     */
+    public PulsarSourceBuilder setConsumerName(String consumerName) {
+        return setConfig(PULSAR_CONSUMER_NAME, consumerName);
+    }
+
     /**
      * Set a topic range generator for Key_Shared subscription.
      *
@@ -320,7 +331,7 @@ public PulsarSourceBuilder setStartCursor(StartCursor startCursor) {
      * 

To stop the connector user has to disable the auto partition discovery. As auto partition * discovery always expected new splits to come and not exiting. To disable auto partition * discovery, use builder.setConfig({@link - * PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS}, -1). + * PulsarSourceOptions#PULSAR_PARTITION_DISCOVERY_INTERVAL_MS}, -1). * * @param stopCursor The {@link StopCursor} to specify the stopping offset. * @return this PulsarSourceBuilder. @@ -333,7 +344,7 @@ public PulsarSourceBuilder setUnboundedStopCursor(StopCursor stopCursor) { } /** - * By default the PulsarSource is set to run in {@link Boundedness#CONTINUOUS_UNBOUNDED} manner + * By default, the PulsarSource is set to run in {@link Boundedness#CONTINUOUS_UNBOUNDED} manner * and thus never stops until the Flink job fails or is canceled. To let the PulsarSource run in * {@link Boundedness#BOUNDED} manner and stops at some point, one can set an {@link StopCursor} * to specify the stopping offsets for each partition. When all the partitions have reached @@ -369,7 +380,7 @@ public PulsarSourceBuilder setDeserializationSchema( } /** - * Set an arbitrary property for the PulsarSource and PulsarConsumer. The valid keys can be + * Set an arbitrary property for the PulsarSource and Pulsar Consumer. The valid keys can be * found in {@link PulsarSourceOptions} and {@link PulsarOptions}. * *

Make sure the option could be set only once or with same value. @@ -395,8 +406,8 @@ public PulsarSourceBuilder setConfig(ConfigOption key, T value) { } /** - * Set arbitrary properties for the PulsarSource and PulsarConsumer. The valid keys can be found - * in {@link PulsarSourceOptions} and {@link PulsarOptions}. + * Set arbitrary properties for the PulsarSource and Pulsar Consumer. The valid keys can be + * found in {@link PulsarSourceOptions} and {@link PulsarOptions}. * * @param config the config to set for the PulsarSource. * @return this PulsarSourceBuilder. @@ -437,7 +448,7 @@ public PulsarSource build() { if (rangeGenerator == null) { LOG.warn( "No range generator provided for key_shared subscription," - + " we would use the DivideRangeGenerator as the default range generator."); + + " we would use the UniformRangeGenerator as the default range generator."); this.rangeGenerator = new UniformRangeGenerator(); } } else { @@ -481,6 +492,16 @@ public PulsarSource build() { } } + if (!configuration.contains(PULSAR_CONSUMER_NAME)) { + LOG.warn( + "We recommend set a readable consumer name through setConsumerName(String) in production mode."); + } else { + String consumerName = configuration.get(PULSAR_CONSUMER_NAME); + if (!consumerName.contains("%s")) { + configuration.set(PULSAR_CONSUMER_NAME, consumerName + " - %s"); + } + } + // Since these implementation could be a lambda, make sure they are serializable. checkState(isSerializable(startCursor), "StartCursor isn't serializable"); checkState(isSerializable(stopCursor), "StopCursor isn't serializable"); 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 c3199156be0cd..b65e709019b98 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 @@ -26,6 +26,7 @@ import org.apache.flink.configuration.description.Description; import org.apache.flink.connector.pulsar.common.config.PulsarOptions; import org.apache.flink.connector.pulsar.source.config.CursorVerification; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.pulsar.client.api.ConsumerCryptoFailureAction; import org.apache.pulsar.client.api.SubscriptionInitialPosition; @@ -495,6 +496,12 @@ private PulsarSourceOptions() { code("PulsarClientException")) .build()); + /** + * @deprecated This option would be reset by {@link StartCursor}, no need to use it anymore. + * Pulsar didn't support this config option before 1.10.1, so we have to remove this config + * option. + */ + @Deprecated public static final ConfigOption PULSAR_SUBSCRIPTION_INITIAL_POSITION = ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "subscriptionInitialPosition") 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 7ac480a39da14..596b03e76dae7 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 @@ -33,12 +33,15 @@ import org.apache.pulsar.client.api.Schema; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.UUID; import static java.util.concurrent.TimeUnit.MICROSECONDS; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.flink.connector.pulsar.common.config.PulsarConfigUtils.getProperties; import static org.apache.flink.connector.pulsar.common.config.PulsarConfigUtils.setOptionValue; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_AUTH_PARAMS; @@ -65,7 +68,6 @@ import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_REPLICATE_SUBSCRIPTION_STATE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_RETRY_ENABLE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_RETRY_LETTER_TOPIC; -import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_INITIAL_POSITION; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_MODE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_TYPE; @@ -150,14 +152,13 @@ public static ConsumerBuilder createConsumerBuilder( configuration, PULSAR_MAX_TOTAL_RECEIVER_QUEUE_SIZE_ACROSS_PARTITIONS, builder::maxTotalReceiverQueueSizeAcrossPartitions); - setOptionValue(configuration, PULSAR_CONSUMER_NAME, builder::consumerName); - setOptionValue(configuration, PULSAR_READ_COMPACTED, builder::readCompacted); - setOptionValue(configuration, PULSAR_PRIORITY_LEVEL, builder::priorityLevel); - setOptionValue(configuration, PULSAR_CONSUMER_PROPERTIES, builder::properties); setOptionValue( configuration, - PULSAR_SUBSCRIPTION_INITIAL_POSITION, - builder::subscriptionInitialPosition); + PULSAR_CONSUMER_NAME, + consumerName -> String.format(consumerName, UUID.randomUUID()), + builder::consumerName); + setOptionValue(configuration, PULSAR_READ_COMPACTED, builder::readCompacted); + setOptionValue(configuration, PULSAR_PRIORITY_LEVEL, builder::priorityLevel); createDeadLetterPolicy(configuration).ifPresent(builder::deadLetterPolicy); setOptionValue( configuration, @@ -178,6 +179,11 @@ public static ConsumerBuilder createConsumerBuilder( v -> builder.expireTimeOfIncompleteChunkedMessage(v, MILLISECONDS)); setOptionValue(configuration, PULSAR_POOL_MESSAGES, builder::poolMessages); + Map properties = getProperties(configuration, PULSAR_CONSUMER_PROPERTIES); + if (!properties.isEmpty()) { + builder.properties(properties); + } + 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 f13fbf2ab6514..8c4163d989007 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 @@ -31,7 +31,9 @@ import java.io.Serializable; 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.common.config.PulsarConfigUtils.getOptionValue; 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; @@ -49,79 +51,20 @@ public class SourceConfiguration implements Serializable { private static final long serialVersionUID = 8488507275800787580L; - /** The interval in millis for flink querying topic partition information. */ + private final int messageQueueCapacity; private final long partitionDiscoveryIntervalMs; - - /** - * This is used for all subscription type. But the behavior may not be the same among them. If - * you don't enable the flink checkpoint, make sure this option is set to true. - * - *

    - *
  • {@link SubscriptionType#Shared} and {@link SubscriptionType#Key_Shared} would - * immediately acknowledge the message after consuming it. - *
  • {@link SubscriptionType#Failover} and {@link SubscriptionType#Exclusive} would perform - * a incremental acknowledge in a fixed {@link #autoCommitCursorInterval}. - *
- */ private final boolean enableAutoAcknowledgeMessage; - - /** - * The interval in millis for acknowledge message when you enable {@link - * #enableAutoAcknowledgeMessage} and use {@link SubscriptionType#Failover} or {@link - * SubscriptionType#Exclusive} as your consuming subscription type. - */ private final long autoCommitCursorInterval; - - /** - * Pulsar's transaction have a timeout mechanism for uncommitted transaction. We use transaction - * for {@link SubscriptionType#Shared} and {@link SubscriptionType#Key_Shared} when user disable - * {@link #enableAutoAcknowledgeMessage} and enable flink checkpoint. Since the checkpoint - * interval couldn't be acquired from {@link SourceReaderContext#getConfiguration()}, we have to - * expose this option. Make sure this value is greater than the checkpoint interval. - */ private final long transactionTimeoutMillis; - - /** - * The fetch time for flink split reader polling message. We would stop polling message and - * return the message in {@link RecordsWithSplitIds} when timeout or exceed the {@link - * #maxFetchRecords}. - */ private final Duration maxFetchTime; - - /** - * The fetch counts for a split reader. We would stop polling message and return the message in - * {@link RecordsWithSplitIds} when timeout {@link #maxFetchTime} or exceed this value. - */ private final int maxFetchRecords; - - /** Validate the {@link CursorPosition} generated by {@link StartCursor}. */ private final CursorVerification verifyInitialOffsets; - - /** - * The pulsar's subscription name for this flink source. All the readers would share this - * subscription name. - * - * @see ConsumerBuilder#subscriptionName - */ private final String subscriptionName; - - /** - * The pulsar's subscription type for this flink source. All the readers would share this - * subscription type. - * - * @see SubscriptionType - */ private final SubscriptionType subscriptionType; - - /** - * The pulsar's subscription mode for this flink source. All the readers would share this - * subscription mode. - * - * @see SubscriptionMode - */ private final SubscriptionMode subscriptionMode; public SourceConfiguration(Configuration configuration) { + this.messageQueueCapacity = configuration.getInteger(ELEMENT_QUEUE_CAPACITY); this.partitionDiscoveryIntervalMs = configuration.get(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS); this.enableAutoAcknowledgeMessage = @@ -137,46 +80,107 @@ public SourceConfiguration(Configuration configuration) { this.subscriptionMode = configuration.get(PULSAR_SUBSCRIPTION_MODE); } - public boolean enablePartitionDiscovery() { - return partitionDiscoveryIntervalMs > 0; + /** The capacity of the element queue in the source reader. */ + public int getMessageQueueCapacity() { + return messageQueueCapacity; } + /** + * We would override the interval into a negative number when we set the connector with bounded + * stop cursor. + */ + public boolean isEnablePartitionDiscovery() { + return getPartitionDiscoveryIntervalMs() > 0; + } + + /** The interval in millis for flink querying topic partition information. */ public long getPartitionDiscoveryIntervalMs() { return partitionDiscoveryIntervalMs; } + /** + * This is used for all subscription type. But the behavior may not be the same among them. If + * you don't enable the flink checkpoint, make sure this option is set to true. + * + *
    + *
  • {@link SubscriptionType#Shared} and {@link SubscriptionType#Key_Shared} would + * immediately acknowledge the message after consuming it. + *
  • {@link SubscriptionType#Failover} and {@link SubscriptionType#Exclusive} would perform + * a incremental acknowledge in a fixed {@link #getAutoCommitCursorInterval}. + *
+ */ public boolean isEnableAutoAcknowledgeMessage() { return enableAutoAcknowledgeMessage; } + /** + * The interval in millis for acknowledge message when you enable {@link + * #isEnableAutoAcknowledgeMessage} and use {@link SubscriptionType#Failover} or {@link + * SubscriptionType#Exclusive} as your consuming subscription type. + */ public long getAutoCommitCursorInterval() { return autoCommitCursorInterval; } + /** + * Pulsar's transaction have a timeout mechanism for uncommitted transaction. We use transaction + * for {@link SubscriptionType#Shared} and {@link SubscriptionType#Key_Shared} when user disable + * {@link #isEnableAutoAcknowledgeMessage} and enable flink checkpoint. Since the checkpoint + * interval couldn't be acquired from {@link SourceReaderContext#getConfiguration()}, we have to + * expose this option. Make sure this value is greater than the checkpoint interval. + */ public long getTransactionTimeoutMillis() { return transactionTimeoutMillis; } + /** + * The fetch time for flink split reader polling message. We would stop polling message and + * return the message in {@link RecordsWithSplitIds} when timeout or exceed the {@link + * #getMaxFetchRecords}. + */ public Duration getMaxFetchTime() { return maxFetchTime; } + /** + * The fetch counts for a split reader. We would stop polling message and return the message in + * {@link RecordsWithSplitIds} when timeout {@link #getMaxFetchTime} or exceed this value. + */ public int getMaxFetchRecords() { return maxFetchRecords; } + /** Validate the {@link CursorPosition} generated by {@link StartCursor}. */ public CursorVerification getVerifyInitialOffsets() { return verifyInitialOffsets; } + /** + * The pulsar's subscription name for this flink source. All the readers would share this + * subscription name. + * + * @see ConsumerBuilder#subscriptionName + */ public String getSubscriptionName() { return subscriptionName; } + /** + * The pulsar's subscription type for this flink source. All the readers would share this + * subscription type. + * + * @see SubscriptionType + */ public SubscriptionType getSubscriptionType() { return subscriptionType; } + /** + * The pulsar's subscription mode for this flink source. All the readers would share this + * subscription mode. + * + * @see SubscriptionMode + */ public SubscriptionMode getSubscriptionMode() { return subscriptionMode; } @@ -190,4 +194,44 @@ public String getSubscriptionDesc() { + getSubscriptionMode() + ")"; } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + SourceConfiguration that = (SourceConfiguration) o; + return partitionDiscoveryIntervalMs == that.partitionDiscoveryIntervalMs + && enableAutoAcknowledgeMessage == that.enableAutoAcknowledgeMessage + && autoCommitCursorInterval == that.autoCommitCursorInterval + && transactionTimeoutMillis == that.transactionTimeoutMillis + && maxFetchRecords == that.maxFetchRecords + && Objects.equals(maxFetchTime, that.maxFetchTime) + && verifyInitialOffsets == that.verifyInitialOffsets + && Objects.equals(subscriptionName, that.subscriptionName) + && subscriptionType == that.subscriptionType + && subscriptionMode == that.subscriptionMode; + } + + @Override + public int hashCode() { + return Objects.hash( + super.hashCode(), + partitionDiscoveryIntervalMs, + enableAutoAcknowledgeMessage, + autoCommitCursorInterval, + transactionTimeoutMillis, + maxFetchTime, + maxFetchRecords, + verifyInitialOffsets, + subscriptionName, + subscriptionType, + subscriptionMode); + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumState.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumState.java index dbab9e2178123..56bbbd20a32a2 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumState.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumState.java @@ -18,15 +18,18 @@ package org.apache.flink.connector.pulsar.source.enumerator; +import org.apache.flink.connector.pulsar.source.enumerator.assigner.SplitAssigner; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; +import java.util.HashMap; +import java.util.HashSet; import java.util.Map; import java.util.Set; /** * The state class for pulsar source enumerator, used for storing the split state. This class is - * managed and controlled by {@link SplitsAssignmentState}. + * managed and controlled by {@link SplitAssigner}. */ public class PulsarSourceEnumState { @@ -46,11 +49,12 @@ public class PulsarSourceEnumState { private final Map> sharedPendingPartitionSplits; /** - * A {@link PulsarPartitionSplit} should be assigned for all flink readers. Using this map for - * recording assign status. + * It is used for Shared subscription. Every {@link PulsarPartitionSplit} should be assigned for + * all flink readers. Using this map for recording assign status. */ private final Map> readerAssignedSplits; + /** The pipeline has been triggered and topic partitions have been assigned to readers. */ private final boolean initialized; public PulsarSourceEnumState( @@ -85,4 +89,10 @@ public Map> getReaderAssignedSplits() { public boolean isInitialized() { return initialized; } + + /** The initial assignment state for Pulsar. */ + public static PulsarSourceEnumState initialState() { + return new PulsarSourceEnumState( + new HashSet<>(), new HashSet<>(), new HashMap<>(), new HashMap<>(), false); + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumerator.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumerator.java index 67cc3c77d5af5..8d65888283a2d 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumerator.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumerator.java @@ -23,40 +23,29 @@ import org.apache.flink.api.connector.source.SplitEnumeratorContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.assigner.SplitAssigner; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.CursorPosition; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; 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.split.PulsarPartitionSplit; import org.apache.flink.util.FlinkRuntimeException; import org.apache.pulsar.client.admin.PulsarAdmin; -import org.apache.pulsar.client.api.Consumer; -import org.apache.pulsar.client.api.ConsumerBuilder; -import org.apache.pulsar.client.api.KeySharedPolicy; -import org.apache.pulsar.client.api.KeySharedPolicy.KeySharedPolicySticky; -import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.PulsarClientException; -import org.apache.pulsar.client.api.Range; -import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.MessageId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.HashSet; import java.util.List; import java.util.Set; import static java.util.Collections.singletonList; import static org.apache.flink.connector.pulsar.common.config.PulsarConfigUtils.createAdmin; -import static org.apache.flink.connector.pulsar.common.config.PulsarConfigUtils.createClient; -import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; -import static org.apache.flink.connector.pulsar.source.config.CursorVerification.FAIL_ON_MISMATCH; -import static org.apache.flink.connector.pulsar.source.config.PulsarSourceConfigUtils.createConsumerBuilder; +import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyAdmin; /** The enumerator class for pulsar source. */ @Internal @@ -66,14 +55,13 @@ public class PulsarSourceEnumerator private static final Logger LOG = LoggerFactory.getLogger(PulsarSourceEnumerator.class); private final PulsarAdmin pulsarAdmin; - private final PulsarClient pulsarClient; private final PulsarSubscriber subscriber; private final StartCursor startCursor; private final RangeGenerator rangeGenerator; private final Configuration configuration; private final SourceConfiguration sourceConfiguration; private final SplitEnumeratorContext context; - private final SplitsAssignmentState assignmentState; + private final SplitAssigner splitAssigner; public PulsarSourceEnumerator( PulsarSubscriber subscriber, @@ -82,16 +70,15 @@ public PulsarSourceEnumerator( Configuration configuration, SourceConfiguration sourceConfiguration, SplitEnumeratorContext context, - SplitsAssignmentState assignmentState) { + SplitAssigner splitAssigner) { this.pulsarAdmin = createAdmin(configuration); - this.pulsarClient = createClient(configuration); this.subscriber = subscriber; this.startCursor = startCursor; this.rangeGenerator = rangeGenerator; this.configuration = configuration; this.sourceConfiguration = sourceConfiguration; this.context = context; - this.assignmentState = assignmentState; + this.splitAssigner = splitAssigner; } @Override @@ -99,7 +86,7 @@ public void start() { rangeGenerator.open(configuration, sourceConfiguration); // Check the pulsar topic information and convert it into source split. - if (sourceConfiguration.enablePartitionDiscovery()) { + if (sourceConfiguration.isEnablePartitionDiscovery()) { LOG.info( "Starting the PulsarSourceEnumerator for subscription {} " + "with partition discovery interval of {} ms.", @@ -127,9 +114,9 @@ public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname @Override public void addSplitsBack(List splits, int subtaskId) { // Put the split back to current pending splits. - assignmentState.putSplitsBackToPendingList(splits, subtaskId); + splitAssigner.addSplitsBack(splits, subtaskId); - // If the failed subtask has already restarted, we need to assign pending splits to it + // If the failed subtask has already restarted, we need to assign pending splits to it. if (context.registeredReaders().containsKey(subtaskId)) { assignPendingPartitionSplits(singletonList(subtaskId)); } @@ -146,7 +133,7 @@ public void addReader(int subtaskId) { @Override public PulsarSourceEnumState snapshotState(long checkpointId) { - return assignmentState.snapshotState(); + return splitAssigner.snapshotState(); } @Override @@ -168,54 +155,7 @@ public void close() { */ private Set getSubscribedTopicPartitions() { int parallelism = context.currentParallelism(); - Set partitions = - subscriber.getSubscribedTopicPartitions(pulsarAdmin, rangeGenerator, parallelism); - - // Seek start position for given partitions. - seekStartPosition(partitions); - - return partitions; - } - - private void seekStartPosition(Set partitions) { - ConsumerBuilder consumerBuilder = consumerBuilder(); - Set seekedTopics = new HashSet<>(); - - for (TopicPartition partition : partitions) { - String topicName = partition.getFullTopicName(); - if (!assignmentState.containsTopic(topicName) && seekedTopics.add(topicName)) { - try (Consumer consumer = - sneakyClient(() -> consumerBuilder.clone().topic(topicName).subscribe())) { - startCursor.seekPosition( - partition.getTopic(), partition.getPartitionId(), consumer); - } catch (PulsarClientException e) { - if (sourceConfiguration.getVerifyInitialOffsets() == FAIL_ON_MISMATCH) { - throw new IllegalArgumentException(e); - } else { - // WARN_ON_MISMATCH would just print this warning message. - // No need to print the stacktrace. - LOG.warn( - "Failed to set initial consuming position for partition {}", - partition, - e); - } - } - } - } - } - - private ConsumerBuilder consumerBuilder() { - ConsumerBuilder builder = - createConsumerBuilder(pulsarClient, Schema.BYTES, configuration); - if (sourceConfiguration.getSubscriptionType() == SubscriptionType.Key_Shared) { - Range range = TopicRange.createFullRange().toPulsarRange(); - KeySharedPolicySticky keySharedPolicy = KeySharedPolicy.stickyHashRange().ranges(range); - // Force this consume use sticky hash range in Key_Shared subscription. - // Pulsar won't remove old message dispatcher before 2.8.2 release. - builder.keySharedPolicy(keySharedPolicy); - } - - return builder; + return subscriber.getSubscribedTopicPartitions(pulsarAdmin, rangeGenerator, parallelism); } /** @@ -234,13 +174,55 @@ private void checkPartitionChanges(Set fetchedPartitions, Throwa } // Append the partitions into current assignment state. - assignmentState.appendTopicPartitions(fetchedPartitions); - List registeredReaders = new ArrayList<>(context.registeredReaders().keySet()); + List newPartitions = + splitAssigner.registerTopicPartitions(fetchedPartitions); + createSubscription(newPartitions); // Assign the new readers. + List registeredReaders = new ArrayList<>(context.registeredReaders().keySet()); assignPendingPartitionSplits(registeredReaders); } + /** Create subscription on topic partition if it doesn't exist. */ + private void createSubscription(List newPartitions) { + for (TopicPartition partition : newPartitions) { + String topicName = partition.getFullTopicName(); + String subscriptionName = sourceConfiguration.getSubscriptionName(); + + List subscriptions = + sneakyAdmin(() -> pulsarAdmin.topics().getSubscriptions(topicName)); + if (!subscriptions.contains(subscriptionName)) { + CursorPosition position = + startCursor.position(partition.getTopic(), partition.getPartitionId()); + MessageId initialPosition = queryInitialPosition(topicName, position); + + sneakyAdmin( + () -> + pulsarAdmin + .topics() + .createSubscription( + topicName, subscriptionName, initialPosition)); + } + } + } + + /** Query the available message id from Pulsar. */ + private MessageId queryInitialPosition(String topicName, CursorPosition position) { + CursorPosition.Type type = position.getType(); + if (type == CursorPosition.Type.TIMESTAMP) { + return sneakyAdmin( + () -> + pulsarAdmin + .topics() + .getMessageIdByTimestamp(topicName, position.getTimestamp())); + } else if (type == CursorPosition.Type.MESSAGE_ID) { + return position.getMessageId(); + } else { + throw new UnsupportedOperationException("We don't support this seek type " + type); + } + } + + /** Query the unassigned splits and assign them to the available readers. */ private void assignPendingPartitionSplits(List pendingReaders) { // Validate the reader. pendingReaders.forEach( @@ -252,17 +234,19 @@ private void assignPendingPartitionSplits(List pendingReaders) { }); // Assign splits to downstream readers. - assignmentState.assignSplits(pendingReaders).ifPresent(context::assignSplits); + splitAssigner.createAssignment(pendingReaders).ifPresent(context::assignSplits); // If periodically partition discovery is disabled and the initializing discovery has done, - // signal NoMoreSplitsEvent to pending readers - if (assignmentState.noMoreNewPartitionSplits()) { - LOG.debug( - "No more PulsarPartitionSplits to assign." - + " Sending NoMoreSplitsEvent to reader {} in subscription {}.", - pendingReaders, - sourceConfiguration.getSubscriptionDesc()); - pendingReaders.forEach(this.context::signalNoMoreSplits); + // signal NoMoreSplitsEvent to pending readers. + for (Integer reader : pendingReaders) { + if (splitAssigner.noMoreSplits(reader)) { + LOG.debug( + "No more PulsarPartitionSplits to assign." + + " Sending NoMoreSplitsEvent to reader {} in subscription {}.", + reader, + sourceConfiguration.getSubscriptionDesc()); + context.signalNoMoreSplits(reader); + } } } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/SplitsAssignmentState.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/SplitsAssignmentState.java deleted file mode 100644 index a8460d437b814..0000000000000 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/SplitsAssignmentState.java +++ /dev/null @@ -1,239 +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.source.enumerator; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.flink.api.connector.source.SplitsAssignment; -import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; -import org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; -import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; -import org.apache.flink.util.InstantiationUtil; - -import org.apache.pulsar.client.api.SubscriptionType; - -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.Objects; -import java.util.Optional; -import java.util.Set; - -/** The state class for recording the split assignment. */ -@Internal -public class SplitsAssignmentState { - - private final StopCursor stopCursor; - private final SourceConfiguration sourceConfiguration; - - // The dynamic states for checkpoint. - private final Set appendedPartitions; - // This pending splits is used for Key_Shared, Failover, Exclusive subscription. - private final Set pendingPartitionSplits; - // These two fields are used for Shared subscription. - private final Map> sharedPendingPartitionSplits; - private final Map> readerAssignedSplits; - private boolean initialized; - - public SplitsAssignmentState(StopCursor stopCursor, SourceConfiguration sourceConfiguration) { - this.stopCursor = stopCursor; - this.sourceConfiguration = sourceConfiguration; - this.appendedPartitions = new HashSet<>(); - this.pendingPartitionSplits = new HashSet<>(); - this.sharedPendingPartitionSplits = new HashMap<>(); - this.readerAssignedSplits = new HashMap<>(); - this.initialized = false; - } - - public SplitsAssignmentState( - StopCursor stopCursor, - SourceConfiguration sourceConfiguration, - PulsarSourceEnumState sourceEnumState) { - this.stopCursor = stopCursor; - this.sourceConfiguration = sourceConfiguration; - this.appendedPartitions = sourceEnumState.getAppendedPartitions(); - this.pendingPartitionSplits = sourceEnumState.getPendingPartitionSplits(); - this.sharedPendingPartitionSplits = sourceEnumState.getSharedPendingPartitionSplits(); - this.readerAssignedSplits = sourceEnumState.getReaderAssignedSplits(); - this.initialized = sourceEnumState.isInitialized(); - } - - public PulsarSourceEnumState snapshotState() { - return new PulsarSourceEnumState( - appendedPartitions, - pendingPartitionSplits, - sharedPendingPartitionSplits, - readerAssignedSplits, - initialized); - } - - /** - * Append the new fetched partitions to current state. We would generate pending source split - * for downstream pulsar readers. Since the {@link SplitEnumeratorContext} don't support put the - * split back to enumerator, we don't support partition deletion. - * - * @param fetchedPartitions The partitions from the {@link PulsarSubscriber}. - */ - public void appendTopicPartitions(Set fetchedPartitions) { - for (TopicPartition partition : fetchedPartitions) { - // If this partition is a new partition. - if (!appendedPartitions.contains(partition)) { - if (!sharePartition()) { - // Create a split and add it to pending list. - pendingPartitionSplits.add(createSplit(partition)); - } - - // Shared subscription don't create splits, we just register partitions. - appendedPartitions.add(partition); - } - } - - // Update this initialize flag. - if (!initialized) { - this.initialized = true; - } - } - - public boolean containsTopic(String topicName) { - return appendedPartitions.stream() - .anyMatch(partition -> Objects.equals(partition.getFullTopicName(), topicName)); - } - - /** Put these splits back to pending list. */ - public void putSplitsBackToPendingList(List splits, int readerId) { - if (!sharePartition()) { - // Put these splits back to normal pending list. - pendingPartitionSplits.addAll(splits); - } else { - // Put the splits back to shared pending list. - Set pending = - sharedPendingPartitionSplits.computeIfAbsent(readerId, id -> new HashSet<>()); - pending.addAll(splits); - } - } - - public Optional> assignSplits( - List pendingReaders) { - // Avoid empty readers assign. - if (pendingReaders.isEmpty()) { - return Optional.empty(); - } - - Map> assignMap; - - // We extract the assign logic into two method for better readability. - if (!sharePartition()) { - assignMap = assignNormalSplits(pendingReaders); - } else { - assignMap = assignSharedSplits(pendingReaders); - } - - if (assignMap.isEmpty()) { - return Optional.empty(); - } else { - return Optional.of(new SplitsAssignment<>(assignMap)); - } - } - - /** - * @return It would return true only if periodically partition discovery is disabled, the - * initializing partition discovery has finished AND there is no pending splits for - * assignment. - */ - public boolean noMoreNewPartitionSplits() { - return !sourceConfiguration.enablePartitionDiscovery() - && initialized - && pendingPartitionSplits.isEmpty(); - } - - // ----------------- private methods ------------------- - - /** The splits don't shared for all the readers. */ - private Map> assignNormalSplits( - List pendingReaders) { - Map> assignMap = new HashMap<>(); - - // Drain a list of splits. - List pendingSplits = drainPendingPartitionsSplits(); - for (int i = 0; i < pendingSplits.size(); i++) { - PulsarPartitionSplit split = pendingSplits.get(i); - int readerId = pendingReaders.get(i % pendingReaders.size()); - assignMap.computeIfAbsent(readerId, id -> new ArrayList<>()).add(split); - } - - return assignMap; - } - - /** Every split would be shared among available readers. */ - private Map> assignSharedSplits( - List pendingReaders) { - Map> assignMap = new HashMap<>(); - - // Drain the splits from share pending list. - for (Integer reader : pendingReaders) { - Set pendingSplits = sharedPendingPartitionSplits.remove(reader); - if (pendingSplits == null) { - pendingSplits = new HashSet<>(); - } - - Set assignedSplits = - readerAssignedSplits.computeIfAbsent(reader, r -> new HashSet<>()); - - for (TopicPartition partition : appendedPartitions) { - String partitionName = partition.toString(); - if (!assignedSplits.contains(partitionName)) { - pendingSplits.add(createSplit(partition)); - assignedSplits.add(partitionName); - } - } - - if (!pendingSplits.isEmpty()) { - assignMap.put(reader, new ArrayList<>(pendingSplits)); - } - } - - return assignMap; - } - - private PulsarPartitionSplit createSplit(TopicPartition partition) { - try { - StopCursor stop = InstantiationUtil.clone(stopCursor); - return new PulsarPartitionSplit(partition, stop); - } catch (IOException | ClassNotFoundException e) { - throw new IllegalStateException(e); - } - } - - private List drainPendingPartitionsSplits() { - List splits = new ArrayList<>(pendingPartitionSplits); - pendingPartitionSplits.clear(); - - return splits; - } - - /** {@link SubscriptionType#Shared} mode should share a same split for all the readers. */ - private boolean sharePartition() { - return sourceConfiguration.getSubscriptionType() == SubscriptionType.Shared; - } -} 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 new file mode 100644 index 0000000000000..087e96157d655 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssigner.java @@ -0,0 +1,126 @@ +/* + * 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.assigner; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +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.split.PulsarPartitionSplit; + +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +/** + * This assigner is used for {@link SubscriptionType#Failover}, {@link SubscriptionType#Exclusive} + * and {@link SubscriptionType#Key_Shared} subscriptions. + */ +@Internal +public class NonSharedSplitAssigner implements SplitAssigner { + private static final long serialVersionUID = 8412586087991597092L; + + private final StopCursor stopCursor; + private final boolean enablePartitionDiscovery; + + // These fields would be saved into checkpoint. + + private final Set appendedPartitions; + private final Set pendingPartitionSplits; + private boolean initialized; + + public NonSharedSplitAssigner( + StopCursor stopCursor, + SourceConfiguration sourceConfiguration, + PulsarSourceEnumState sourceEnumState) { + this.stopCursor = stopCursor; + this.enablePartitionDiscovery = sourceConfiguration.isEnablePartitionDiscovery(); + this.appendedPartitions = sourceEnumState.getAppendedPartitions(); + this.pendingPartitionSplits = sourceEnumState.getPendingPartitionSplits(); + this.initialized = sourceEnumState.isInitialized(); + } + + @Override + public List registerTopicPartitions(Set fetchedPartitions) { + List newPartitions = new ArrayList<>(); + + for (TopicPartition partition : fetchedPartitions) { + if (!appendedPartitions.contains(partition)) { + pendingPartitionSplits.add(new PulsarPartitionSplit(partition, stopCursor)); + appendedPartitions.add(partition); + newPartitions.add(partition); + } + } + + if (!initialized) { + initialized = true; + } + + return newPartitions; + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + pendingPartitionSplits.addAll(splits); + } + + @Override + public Optional> createAssignment( + List readers) { + if (pendingPartitionSplits.isEmpty() || readers.isEmpty()) { + return Optional.empty(); + } + + Map> assignMap = new HashMap<>(); + + List partitionSplits = new ArrayList<>(pendingPartitionSplits); + int readerCount = readers.size(); + for (int i = 0; i < partitionSplits.size(); i++) { + int index = i % readerCount; + Integer readerId = readers.get(index); + PulsarPartitionSplit split = partitionSplits.get(i); + assignMap.computeIfAbsent(readerId, id -> new ArrayList<>()).add(split); + } + pendingPartitionSplits.clear(); + + return Optional.of(new SplitsAssignment<>(assignMap)); + } + + @Override + public boolean noMoreSplits(Integer reader) { + return !enablePartitionDiscovery && initialized && pendingPartitionSplits.isEmpty(); + } + + @Override + public PulsarSourceEnumState snapshotState() { + return new PulsarSourceEnumState( + appendedPartitions, + pendingPartitionSplits, + new HashMap<>(), + new HashMap<>(), + initialized); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssigner.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssigner.java new file mode 100644 index 0000000000000..48d75c8dee30d --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssigner.java @@ -0,0 +1,148 @@ +/* + * 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.assigner; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +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.split.PulsarPartitionSplit; + +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +/** This assigner is used for {@link SubscriptionType#Shared} subscriptions. */ +@Internal +public class SharedSplitAssigner implements SplitAssigner { + private static final long serialVersionUID = 8468503133499402491L; + + private final StopCursor stopCursor; + private final boolean enablePartitionDiscovery; + + // These fields would be saved into checkpoint. + + private final Set appendedPartitions; + private final Map> sharedPendingPartitionSplits; + private final Map> readerAssignedSplits; + private boolean initialized; + + public SharedSplitAssigner( + StopCursor stopCursor, + SourceConfiguration sourceConfiguration, + PulsarSourceEnumState sourceEnumState) { + this.stopCursor = stopCursor; + this.enablePartitionDiscovery = sourceConfiguration.isEnablePartitionDiscovery(); + this.appendedPartitions = sourceEnumState.getAppendedPartitions(); + this.sharedPendingPartitionSplits = sourceEnumState.getSharedPendingPartitionSplits(); + this.readerAssignedSplits = sourceEnumState.getReaderAssignedSplits(); + this.initialized = sourceEnumState.isInitialized(); + } + + @Override + public List registerTopicPartitions(Set fetchedPartitions) { + List newPartitions = new ArrayList<>(); + + for (TopicPartition partition : fetchedPartitions) { + if (!appendedPartitions.contains(partition)) { + appendedPartitions.add(partition); + newPartitions.add(partition); + } + } + + if (!initialized) { + initialized = true; + } + + return newPartitions; + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + Set pendingPartitionSplits = + sharedPendingPartitionSplits.computeIfAbsent(subtaskId, id -> new HashSet<>()); + pendingPartitionSplits.addAll(splits); + } + + @Override + public Optional> createAssignment( + List readers) { + if (readers.isEmpty()) { + return Optional.empty(); + } + + Map> assignMap = new HashMap<>(); + for (Integer reader : readers) { + Set pendingSplits = sharedPendingPartitionSplits.remove(reader); + if (pendingSplits == null) { + pendingSplits = new HashSet<>(); + } + + Set assignedSplits = + readerAssignedSplits.computeIfAbsent(reader, r -> new HashSet<>()); + + for (TopicPartition partition : appendedPartitions) { + String partitionName = partition.toString(); + if (!assignedSplits.contains(partitionName)) { + pendingSplits.add(new PulsarPartitionSplit(partition, stopCursor)); + assignedSplits.add(partitionName); + } + } + + if (!pendingSplits.isEmpty()) { + assignMap.put(reader, new ArrayList<>(pendingSplits)); + } + } + + if (assignMap.isEmpty()) { + return Optional.empty(); + } else { + return Optional.of(new SplitsAssignment<>(assignMap)); + } + } + + @Override + public boolean noMoreSplits(Integer reader) { + Set pendingSplits = sharedPendingPartitionSplits.get(reader); + Set assignedSplits = readerAssignedSplits.get(reader); + + return !enablePartitionDiscovery + && initialized + && (pendingSplits == null || pendingSplits.isEmpty()) + && (assignedSplits != null && assignedSplits.size() == appendedPartitions.size()); + } + + @Override + public PulsarSourceEnumState snapshotState() { + return new PulsarSourceEnumState( + appendedPartitions, + new HashSet<>(), + sharedPendingPartitionSplits, + readerAssignedSplits, + initialized); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssigner.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssigner.java new file mode 100644 index 0000000000000..bc03f5103fd88 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssigner.java @@ -0,0 +1,64 @@ +/* + * 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.assigner; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; +import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; + +import java.io.Serializable; +import java.util.List; +import java.util.Optional; +import java.util.Set; + +/** + * The split assigner for different subscription. We would spread all the splits to different + * readers and store all the state into checkpoint. + */ +@Internal +public interface SplitAssigner extends Serializable { + + /** + * Add the current available partitions into assigner. + * + * @param fetchedPartitions The available partitions queried from Pulsar broker. + * @return New topic partitions compare to previous registered partitions. + */ + List registerTopicPartitions(Set fetchedPartitions); + + /** + * Add a split back to the split assigner if the reader fails. We would try to reassign the + * split or add it to the pending list. + */ + void addSplitsBack(List splits, int subtaskId); + + /** Create a split assignment from the current readers. */ + Optional> createAssignment(List readers); + + /** + * It would return true only if periodically partition discovery is disabled, the initializing + * partition discovery has finished AND there is no pending splits for assignment. + */ + boolean noMoreSplits(Integer reader); + + /** Snapshot the current assign state into checkpoint. */ + PulsarSourceEnumState snapshotState(); +} 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 new file mode 100644 index 0000000000000..3e6ebccb49b4c --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerFactory.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.source.enumerator.assigner; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; + +import org.apache.pulsar.client.api.SubscriptionType; + +import static org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState.initialState; +import static org.apache.pulsar.client.api.SubscriptionType.Exclusive; +import static org.apache.pulsar.client.api.SubscriptionType.Failover; +import static org.apache.pulsar.client.api.SubscriptionType.Key_Shared; +import static org.apache.pulsar.client.api.SubscriptionType.Shared; + +/** The factory for creating split assigner. */ +@Internal +public final class SplitAssignerFactory { + + private SplitAssignerFactory() { + // No public constructor. + } + + /** Create blank assigner. */ + public static SplitAssigner create( + StopCursor stopCursor, SourceConfiguration sourceConfiguration) { + return create(stopCursor, sourceConfiguration, initialState()); + } + + /** Create assigner from checkpoint state. */ + public static SplitAssigner create( + StopCursor stopCursor, + SourceConfiguration sourceConfiguration, + PulsarSourceEnumState sourceEnumState) { + SubscriptionType subscriptionType = sourceConfiguration.getSubscriptionType(); + if (subscriptionType == Exclusive + || subscriptionType == Failover + || subscriptionType == Key_Shared) { + return new NonSharedSplitAssigner(stopCursor, sourceConfiguration, sourceEnumState); + } else if (subscriptionType == Shared) { + return new SharedSplitAssigner(stopCursor, sourceConfiguration, sourceEnumState); + } else { + 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/cursor/CursorPosition.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/CursorPosition.java index a2aaff629066d..c965ff962f807 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/CursorPosition.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/CursorPosition.java @@ -18,18 +18,16 @@ package org.apache.flink.connector.pulsar.source.enumerator.cursor; +import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.PulsarClientException; - -import javax.annotation.Nullable; import java.io.Serializable; -/** The class for defining the start or stop position. */ +/** + * The class for defining the start or stop position. We only expose the constructor for end user. + */ @PublicEvolving public final class CursorPosition implements Serializable { private static final long serialVersionUID = -802405183307684549L; @@ -40,34 +38,31 @@ public final class CursorPosition implements Serializable { private final Long timestamp; - public CursorPosition(@Nullable MessageId messageId) { + public CursorPosition(MessageId messageId) { this.type = Type.MESSAGE_ID; this.messageId = messageId; this.timestamp = null; } - public CursorPosition(@Nullable Long timestamp) { + public CursorPosition(Long timestamp) { this.type = Type.TIMESTAMP; this.messageId = null; this.timestamp = timestamp; } - @VisibleForTesting + @Internal + public Type getType() { + return type; + } + + @Internal public MessageId getMessageId() { return messageId; } - /** Pulsar consumer could be subscribed by the position. */ - public void seekPosition(Consumer consumer) throws PulsarClientException { - if (type == Type.MESSAGE_ID) { - consumer.seek(messageId); - } else { - if (timestamp != null) { - consumer.seek(timestamp); - } else { - consumer.seek(System.currentTimeMillis()); - } - } + @Internal + public Long getTimestamp() { + return timestamp; } @Override @@ -82,6 +77,7 @@ public String toString() { /** * The position type for reader to choose whether timestamp or message id as the start position. */ + @Internal public enum Type { TIMESTAMP, diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/MessageIdUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/MessageIdUtils.java new file mode 100644 index 0000000000000..a8c3a6b2ef284 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/MessageIdUtils.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.source.enumerator.cursor; + +import org.apache.flink.annotation.Internal; + +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.impl.BatchMessageIdImpl; +import org.apache.pulsar.client.impl.MessageIdImpl; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** The helper class for Pulsar's message id. */ +@Internal +public final class MessageIdUtils { + + private MessageIdUtils() { + // No public constructor. + } + + /** + * The implementation from this + * code snippet to get next message id. + */ + public static MessageId nextMessageId(MessageId messageId) { + MessageIdImpl idImpl = unwrapMessageId(messageId); + + if (idImpl.getEntryId() < 0) { + return newMessageId(idImpl.getLedgerId(), 0, idImpl.getPartitionIndex()); + } else { + return newMessageId( + idImpl.getLedgerId(), idImpl.getEntryId() + 1, idImpl.getPartitionIndex()); + } + } + + /** + * Convert the message id interface to its backend implementation. And check if it's a batch + * message id. We don't support the batch message for its low performance now. + */ + public static MessageIdImpl unwrapMessageId(MessageId messageId) { + MessageIdImpl idImpl = MessageIdImpl.convertToMessageIdImpl(messageId); + if (idImpl instanceof BatchMessageIdImpl) { + int batchSize = ((BatchMessageIdImpl) idImpl).getBatchSize(); + checkArgument(batchSize == 1, "We only support normal message id currently."); + } + + return idImpl; + } + + /** Hide the message id implementation. */ + public static MessageId newMessageId(long ledgerId, long entryId, int partitionIndex) { + return new MessageIdImpl(ledgerId, entryId, partitionIndex); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StartCursor.java index e99222fd8c722..9c1d699a269f7 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StartCursor.java @@ -22,9 +22,7 @@ import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.MessageIdStartCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.TimestampStartCursor; -import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SubscriptionType; import java.io.Serializable; @@ -43,13 +41,6 @@ public interface StartCursor extends Serializable { CursorPosition position(String topic, int partitionId); - /** Helper method for seek the right position for given pulsar consumer. */ - default void seekPosition(String topic, int partitionId, Consumer consumer) - throws PulsarClientException { - CursorPosition position = position(topic, partitionId); - position.seekPosition(consumer); - } - // --------------------------- Static Factory Methods ----------------------------- static StartCursor defaultStartCursor() { @@ -64,19 +55,38 @@ static StartCursor latest() { return fromMessageId(MessageId.latest); } + /** + * Find the available message id and start consuming from it. The given message is included in + * the consuming result by default if you provide a specified message id instead of {@link + * MessageId#earliest} or {@link MessageId#latest}. + */ static StartCursor fromMessageId(MessageId messageId) { return fromMessageId(messageId, true); } /** * @param messageId Find the available message id and start consuming from it. - * @param inclusive {@code ture} would include the given message id. + * @param inclusive {@code true} would include the given message id if it's not the {@link + * MessageId#earliest} or {@link MessageId#latest}. */ static StartCursor fromMessageId(MessageId messageId, boolean inclusive) { return new MessageIdStartCursor(messageId, inclusive); } + /** + * This method is designed for seeking message from event time. But Pulsar didn't support + * seeking from message time, instead, it would seek the position from publish time. We only + * keep this method for backward compatible. + * + * @deprecated Use {@link #fromPublishTime(long)} instead. + */ + @Deprecated static StartCursor fromMessageTime(long timestamp) { - return new TimestampStartCursor(timestamp); + return new TimestampStartCursor(timestamp, true); + } + + /** Seek the start position by using message publish time. */ + static StartCursor fromPublishTime(long timestamp) { + return new TimestampStartCursor(timestamp, true); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursor.java index 0bf46ce128274..d44c78fcf1a44 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursor.java @@ -44,11 +44,55 @@ public interface StopCursor extends Serializable { /** The open method for the cursor initializer. This method could be executed multiple times. */ default void open(PulsarAdmin admin, TopicPartition partition) {} - /** - * Determine whether to pause consumption on the current message by the returned boolean value. - * The message presented in method argument wouldn't be consumed if the return result is true. - */ - boolean shouldStop(Message message); + /** Determine whether to pause consumption on the current message by the returned enum. */ + StopCondition shouldStop(Message message); + + /** The conditional for control the stop behavior of the pulsar source. */ + @PublicEvolving + enum StopCondition { + + /** This message should be included in the result. */ + CONTINUE, + /** This message should be included in the result and stop consuming. */ + EXACTLY, + /** Stop consuming, the given message wouldn't be included in the result. */ + TERMINATE; + + /** + * Common methods for comparing the message id. + * + * @param desired The stop goal of the message id. + * @param current The upcoming message id. + * @param inclusive Should the desired message be included in the consuming result. + */ + public static StopCondition compare( + MessageId desired, MessageId current, boolean inclusive) { + if (current.compareTo(desired) < 0) { + return StopCondition.CONTINUE; + } else if (current.compareTo(desired) == 0) { + return inclusive ? StopCondition.EXACTLY : StopCondition.TERMINATE; + } else { + return StopCondition.TERMINATE; + } + } + + /** + * Common methods for comparing the message time. + * + * @param desired The stop goal of the message time. + * @param current The upcoming message time. + * @param inclusive Should the desired message be included in the consuming result. + */ + public static StopCondition compare(long desired, long current, boolean inclusive) { + if (current < desired) { + return StopCondition.CONTINUE; + } else if (current == desired) { + return inclusive ? StopCondition.EXACTLY : StopCondition.TERMINATE; + } else { + return StopCondition.TERMINATE; + } + } + } // --------------------------- Static Factory Methods ----------------------------- @@ -61,32 +105,52 @@ static StopCursor never() { } static StopCursor latest() { - return new LatestMessageStopCursor(); + return new LatestMessageStopCursor(true); } /** - * Stop when the messageId is equal or greater than the specified messageId. Message that is - * equal to the specified messageId will not be consumed. + * Stop consuming when the messageId is equal or greater than the specified messageId. Message + * that is equal to the specified messageId will not be consumed. */ static StopCursor atMessageId(MessageId messageId) { - return new MessageIdStopCursor(messageId); + if (MessageId.latest.equals(messageId)) { + return new LatestMessageStopCursor(false); + } else { + return new MessageIdStopCursor(messageId, false); + } } /** - * Stop when the messageId is greater than the specified messageId. Message that is equal to the - * specified messageId will be consumed. + * Stop consuming when the messageId is greater than the specified messageId. Message that is + * equal to the specified messageId will be consumed. */ static StopCursor afterMessageId(MessageId messageId) { - return new MessageIdStopCursor(messageId, false); + if (MessageId.latest.equals(messageId)) { + return new LatestMessageStopCursor(true); + } else { + return new MessageIdStopCursor(messageId, true); + } } - @Deprecated + /** Stop consuming when message eventTime is greater than or equals the specified timestamp. */ static StopCursor atEventTime(long timestamp) { - return new EventTimestampStopCursor(timestamp); + return new EventTimestampStopCursor(timestamp, false); } - /** Stop when message publishTime is greater than the specified timestamp. */ + /** Stop consuming when message eventTime is greater than the specified timestamp. */ + static StopCursor afterEventTime(long timestamp) { + return new EventTimestampStopCursor(timestamp, true); + } + + /** + * Stop consuming when message publishTime is greater than or equals the specified timestamp. + */ static StopCursor atPublishTime(long timestamp) { - return new PublishTimestampStopCursor(timestamp); + return new PublishTimestampStopCursor(timestamp, false); + } + + /** Stop consuming when message publishTime is greater than the specified timestamp. */ + static StopCursor afterPublishTime(long timestamp) { + return new PublishTimestampStopCursor(timestamp, true); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java index 0185bb3820fbf..33591c583f6ca 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java @@ -25,7 +25,8 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.impl.MessageIdImpl; -import static org.apache.flink.util.Preconditions.checkState; +import static org.apache.flink.connector.pulsar.source.enumerator.cursor.MessageIdUtils.nextMessageId; +import static org.apache.flink.connector.pulsar.source.enumerator.cursor.MessageIdUtils.unwrapMessageId; /** This cursor would left pulsar start consuming from a specific message id. */ public class MessageIdStartCursor implements StartCursor { @@ -43,19 +44,16 @@ public class MessageIdStartCursor implements StartCursor { * code for understanding pulsar internal logic. * * @param messageId The message id for start position. - * @param inclusive Should we include the start message id in consuming result. + * @param inclusive Whether we include the start message id in consuming result. This works only + * if we provide a specified message id instead of {@link MessageId#earliest} or {@link + * MessageId#latest}. */ public MessageIdStartCursor(MessageId messageId, boolean inclusive) { - if (inclusive) { - this.messageId = messageId; + MessageIdImpl idImpl = unwrapMessageId(messageId); + if (MessageId.earliest.equals(idImpl) || MessageId.latest.equals(idImpl) || inclusive) { + this.messageId = idImpl; } else { - checkState( - messageId instanceof MessageIdImpl, - "We only support normal message id and batch message id."); - MessageIdImpl id = (MessageIdImpl) messageId; - this.messageId = - new MessageIdImpl( - id.getLedgerId(), id.getEntryId() + 1, id.getPartitionIndex()); + this.messageId = nextMessageId(idImpl); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java index eb4ea32ebb6b9..da51a58e9433c 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java @@ -27,8 +27,8 @@ public class TimestampStartCursor implements StartCursor { private final long timestamp; - public TimestampStartCursor(long timestamp) { - this.timestamp = timestamp; + public TimestampStartCursor(long timestamp, boolean inclusive) { + this.timestamp = inclusive ? timestamp : timestamp + 1; } @Override diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java index e425545de4412..d2a44ea362d80 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java @@ -27,13 +27,16 @@ public class EventTimestampStopCursor implements StopCursor { private static final long serialVersionUID = 2391576769339369027L; private final long timestamp; + private final boolean inclusive; - public EventTimestampStopCursor(long timestamp) { + public EventTimestampStopCursor(long timestamp, boolean inclusive) { this.timestamp = timestamp; + this.inclusive = inclusive; } @Override - public boolean shouldStop(Message message) { - return message.getEventTime() >= timestamp; + public StopCondition shouldStop(Message message) { + long eventTime = message.getEventTime(); + return StopCondition.compare(timestamp, eventTime, inclusive); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java index e42064c8f2e3f..58d8e2ed42b1e 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java @@ -30,23 +30,28 @@ /** * A stop cursor that initialize the position to the latest message id. The offsets initialization * are taken care of by the {@code PulsarPartitionSplitReaderBase} instead of by the {@code - * PulsarSourceEnumerator}. + * PulsarSourceEnumerator}. We would include the latest message available in Pulsar by default. */ public class LatestMessageStopCursor implements StopCursor { private MessageId messageId; + private final boolean inclusive; + + public LatestMessageStopCursor(boolean inclusive) { + this.inclusive = inclusive; + } + + @Override + public StopCondition shouldStop(Message message) { + MessageId current = message.getMessageId(); + return StopCondition.compare(messageId, current, inclusive); + } @Override public void open(PulsarAdmin admin, TopicPartition partition) { if (messageId == null) { String topic = partition.getFullTopicName(); - messageId = sneakyAdmin(() -> admin.topics().getLastMessageId(topic)); + this.messageId = sneakyAdmin(() -> admin.topics().getLastMessageId(topic)); } } - - @Override - public boolean shouldStop(Message message) { - MessageId id = message.getMessageId(); - return id.compareTo(messageId) >= 0; - } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java index 7af55a00cc09f..03d83aa4495d1 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java @@ -22,6 +22,12 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.impl.MessageIdImpl; + +import static org.apache.flink.connector.pulsar.source.enumerator.cursor.MessageIdUtils.unwrapMessageId; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.pulsar.client.api.MessageId.earliest; +import static org.apache.pulsar.client.api.MessageId.latest; /** * Stop consuming message at a given message id. We use the {@link MessageId#compareTo(Object)} for @@ -32,24 +38,22 @@ public class MessageIdStopCursor implements StopCursor { private final MessageId messageId; - private final boolean exclusive; + private final boolean inclusive; - public MessageIdStopCursor(MessageId messageId) { - this(messageId, true); - } + public MessageIdStopCursor(MessageId messageId, boolean inclusive) { + MessageIdImpl idImpl = unwrapMessageId(messageId); + checkArgument(!earliest.equals(idImpl), "MessageId.earliest is not supported."); + checkArgument( + !latest.equals(idImpl), + "MessageId.latest is not supported, use LatestMessageStopCursor instead."); - public MessageIdStopCursor(MessageId messageId, boolean exclusive) { - this.messageId = messageId; - this.exclusive = exclusive; + this.messageId = idImpl; + this.inclusive = inclusive; } @Override - public boolean shouldStop(Message message) { - MessageId id = message.getMessageId(); - if (exclusive) { - return id.compareTo(messageId) > 0; - } else { - return id.compareTo(messageId) >= 0; - } + public StopCondition shouldStop(Message message) { + MessageId current = message.getMessageId(); + return StopCondition.compare(messageId, current, inclusive); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java index ff2c619afb8b0..3eb035634ae0a 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java @@ -27,7 +27,7 @@ public class NeverStopCursor implements StopCursor { private static final long serialVersionUID = -3113601090292771786L; @Override - public boolean shouldStop(Message message) { - return false; + public StopCondition shouldStop(Message message) { + return StopCondition.CONTINUE; } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java index b598e7addd422..2dfdd7658428a 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java @@ -27,13 +27,16 @@ public class PublishTimestampStopCursor implements StopCursor { private static final long serialVersionUID = 4386276745339324527L; private final long timestamp; + private final boolean inclusive; - public PublishTimestampStopCursor(long timestamp) { + public PublishTimestampStopCursor(long timestamp, boolean inclusive) { this.timestamp = timestamp; + this.inclusive = inclusive; } @Override - public boolean shouldStop(Message message) { - return message.getPublishTime() >= timestamp; + public StopCondition shouldStop(Message message) { + long publishTime = message.getPublishTime(); + return StopCondition.compare(timestamp, publishTime, inclusive); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/PulsarSubscriber.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/PulsarSubscriber.java index 08ba1faa44214..b8a55bf8a34b8 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/PulsarSubscriber.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/PulsarSubscriber.java @@ -48,7 +48,7 @@ public interface PulsarSubscriber extends Serializable { /** * Get a set of subscribed {@link TopicPartition}s. The method could throw {@link - * IllegalStateException}, a extra try catch is required. + * IllegalStateException}, an extra try catch is required. * * @param pulsarAdmin The admin interface used to retrieve subscribed topic partitions. * @param rangeGenerator The range for different partitions. diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java index e60ef99e15ad7..32c6180cbc16e 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java @@ -80,7 +80,7 @@ public static SourceReader create( SubscriptionType subscriptionType = sourceConfiguration.getSubscriptionType(); if (subscriptionType == SubscriptionType.Failover || subscriptionType == SubscriptionType.Exclusive) { - // Create a ordered split reader supplier. + // Create an ordered split reader supplier. Supplier> splitReaderSupplier = () -> new PulsarOrderedPartitionSplitReader<>( diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java index 8643891250513..950fa9b8156b7 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java @@ -21,14 +21,13 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; import org.apache.flink.connector.pulsar.source.reader.source.PulsarOrderedSourceReader; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.pulsar.client.admin.PulsarAdmin; -import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClient; @@ -37,10 +36,12 @@ import org.slf4j.LoggerFactory; import java.time.Duration; +import java.util.List; import java.util.concurrent.TimeUnit; import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; import static org.apache.flink.connector.pulsar.source.config.CursorVerification.FAIL_ON_MISMATCH; +import static org.apache.flink.connector.pulsar.source.enumerator.cursor.MessageIdUtils.nextMessageId; /** * The split reader a given {@link PulsarPartitionSplit}, it would be closed once the {@link @@ -77,18 +78,39 @@ protected void finishedPollMessage(Message message) { } @Override - protected void startConsumer(PulsarPartitionSplit split, Consumer consumer) { + protected void beforeCreatingConsumer(PulsarPartitionSplit split) { MessageId latestConsumedId = split.getLatestConsumedId(); // Reset the start position for ordered pulsar consumer. if (latestConsumedId != null) { - StartCursor startCursor = StartCursor.fromMessageId(latestConsumedId, false); - TopicPartition partition = split.getPartition(); - + LOG.info("Reset subscription position by the checkpoint {}", latestConsumedId); try { - startCursor.seekPosition( - partition.getTopic(), partition.getPartitionId(), consumer); - } catch (PulsarClientException e) { + MessageId initialPosition; + if (latestConsumedId == MessageId.latest + || latestConsumedId == MessageId.earliest) { + // for compatibility + initialPosition = latestConsumedId; + } else { + initialPosition = nextMessageId(latestConsumedId); + } + + // Remove Consumer.seek() here for waiting for pulsar-client-all 2.12.0 + // See https://github.com/apache/pulsar/issues/16757 for more details. + + String topicName = split.getPartition().getFullTopicName(); + List subscriptions = pulsarAdmin.topics().getSubscriptions(topicName); + String subscriptionName = sourceConfiguration.getSubscriptionName(); + + if (!subscriptions.contains(subscriptionName)) { + // If this subscription is not available. Just create it. + pulsarAdmin + .topics() + .createSubscription(topicName, subscriptionName, initialPosition); + } else { + // Reset the subscription if this is existed. + pulsarAdmin.topics().resetCursor(topicName, subscriptionName, initialPosition); + } + } catch (PulsarAdminException e) { if (sourceConfiguration.getVerifyInitialOffsets() == FAIL_ON_MISMATCH) { throw new IllegalArgumentException(e); } else { @@ -97,7 +119,7 @@ protected void startConsumer(PulsarPartitionSplit split, Consumer consum LOG.warn( "Failed to reset cursor to {} on partition {}", latestConsumedId, - partition, + split.getPartition(), e); } } 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 69c7b5ebbd2cf..d6548dc70702a 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 @@ -27,6 +27,7 @@ import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor.StopCondition; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; @@ -53,7 +54,6 @@ import java.util.List; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; import static org.apache.flink.connector.pulsar.source.config.PulsarSourceConfigUtils.createConsumerBuilder; @@ -72,7 +72,6 @@ abstract class PulsarPartitionSplitReaderBase protected final Configuration configuration; protected final SourceConfiguration sourceConfiguration; protected final PulsarDeserializationSchema deserializationSchema; - protected final AtomicBoolean wakeup; protected Consumer pulsarConsumer; protected PulsarPartitionSplit registeredSplit; @@ -88,7 +87,6 @@ protected PulsarPartitionSplitReaderBase( this.configuration = configuration; this.sourceConfiguration = sourceConfiguration; this.deserializationSchema = deserializationSchema; - this.wakeup = new AtomicBoolean(false); } @Override @@ -100,9 +98,6 @@ public RecordsWithSplitIds> fetch() throws IOException { return builder.build(); } - // Set wakeup to false for start consuming. - wakeup.compareAndSet(true, false); - StopCursor stopCursor = registeredSplit.getStopCursor(); String splitId = registeredSplit.splitId(); PulsarMessageCollector collector = new PulsarMessageCollector<>(splitId, builder); @@ -110,9 +105,7 @@ public RecordsWithSplitIds> fetch() throws IOException { // Consume message from pulsar until it was woke up by flink reader. for (int messageNum = 0; - messageNum < sourceConfiguration.getMaxFetchRecords() - && deadline.hasTimeLeft() - && isNotWakeup(); + messageNum < sourceConfiguration.getMaxFetchRecords() && deadline.hasTimeLeft(); messageNum++) { try { Duration timeout = deadline.timeLeftIfAny(); @@ -121,14 +114,18 @@ public RecordsWithSplitIds> fetch() throws IOException { break; } - // Deserialize message. - collector.setMessage(message); - deserializationSchema.deserialize(message, collector); + StopCondition condition = stopCursor.shouldStop(message); + + if (condition == StopCondition.CONTINUE || condition == StopCondition.EXACTLY) { + // Deserialize message. + collector.setMessage(message); + deserializationSchema.deserialize(message, collector); - // Acknowledge message if need. - finishedPollMessage(message); + // Acknowledge message if need. + finishedPollMessage(message); + } - if (stopCursor.shouldStop(message)) { + if (condition == StopCondition.EXACTLY || condition == StopCondition.TERMINATE) { builder.addFinishedSplit(splitId); break; } @@ -169,23 +166,27 @@ public void handleSplitsChanges(SplitsChange splitsChanges newSplits.size() == 1, "This pulsar split reader only support one split."); PulsarPartitionSplit newSplit = newSplits.get(0); + // Open stop cursor. + newSplit.open(pulsarAdmin); + + // Before creating the consumer. + beforeCreatingConsumer(newSplit); + // Create pulsar consumer. Consumer consumer = createPulsarConsumer(newSplit); - // Open start & stop cursor. - newSplit.open(pulsarAdmin); - - // Start Consumer. - startConsumer(newSplit, consumer); + // After creating the consumer. + afterCreatingConsumer(newSplit, consumer); LOG.info("Register split {} consumer for current reader.", newSplit); + this.registeredSplit = newSplit; this.pulsarConsumer = consumer; } @Override public void wakeUp() { - wakeup.compareAndSet(false, true); + // Nothing to do on this method. } @Override @@ -201,14 +202,16 @@ protected abstract Message pollMessage(Duration timeout) protected abstract void finishedPollMessage(Message message); - protected abstract void startConsumer(PulsarPartitionSplit split, Consumer consumer); - - // --------------------------- Helper Methods ----------------------------- + protected void beforeCreatingConsumer(PulsarPartitionSplit split) { + // Nothing to do by default. + } - protected boolean isNotWakeup() { - return !wakeup.get(); + protected void afterCreatingConsumer(PulsarPartitionSplit split, Consumer consumer) { + // Nothing to do by default. } + // --------------------------- Helper Methods ----------------------------- + /** Create a specified {@link Consumer} by the given split information. */ protected Consumer createPulsarConsumer(PulsarPartitionSplit split) { return createPulsarConsumer(split.getPartition()); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java index 846101dcb9f52..630e2567dbc1b 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java @@ -20,7 +20,6 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.configuration.Configuration; -import org.apache.flink.connector.pulsar.common.utils.PulsarTransactionUtils; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; import org.apache.flink.connector.pulsar.source.reader.source.PulsarUnorderedSourceReader; @@ -129,7 +128,7 @@ protected void finishedPollMessage(Message message) { } @Override - protected void startConsumer(PulsarPartitionSplit split, Consumer consumer) { + protected void afterCreatingConsumer(PulsarPartitionSplit split, Consumer consumer) { TxnID uncommittedTransactionId = split.getUncommittedTransactionId(); // Abort the uncommitted pulsar transaction. @@ -155,7 +154,7 @@ public PulsarPartitionSplitState snapshotState(long checkpointId) { // Avoiding NP problem when Pulsar don't get the message before Flink checkpoint. if (uncommittedTransaction != null) { - TxnID txnID = PulsarTransactionUtils.getId(uncommittedTransaction); + TxnID txnID = uncommittedTransaction.getTxnID(); this.uncommittedTransaction = newTransaction(); state.setUncommittedTransactionId(txnID); } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplit.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplit.java index 0ec693a2b26d2..8eb8f59931899 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplit.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplit.java @@ -30,13 +30,15 @@ import javax.annotation.Nullable; +import java.io.Serializable; import java.util.Objects; import static org.apache.flink.util.Preconditions.checkNotNull; /** A {@link SourceSplit} implementation for a Pulsar's partition. */ @Internal -public class PulsarPartitionSplit implements SourceSplit { +public class PulsarPartitionSplit implements SourceSplit, Serializable { + private static final long serialVersionUID = -6857317360756062625L; private final TopicPartition partition; 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 38c40ed88ef7b..5f18e8f51316c 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 @@ -34,6 +34,7 @@ import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; import static org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumStateSerializer.INSTANCE; +import static org.apache.flink.connector.pulsar.source.enumerator.cursor.MessageIdUtils.newMessageId; import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotSame; @@ -51,7 +52,9 @@ void serializeAndDeserializePulsarSourceEnumState() throws Exception { Collections.singleton( new PulsarPartitionSplit( new TopicPartition(randomAlphabetic(10), 10, createFullRange()), - StopCursor.defaultStopCursor())); + StopCursor.defaultStopCursor(), + newMessageId(100L, 23L, 44), + null)); Map> shared = Collections.singletonMap(5, splits); Map> mapping = ImmutableMap.of( 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 a4cc0a4988655..44bd9ae52f1bc 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 @@ -18,117 +18,329 @@ package org.apache.flink.connector.pulsar.source.enumerator; +import org.apache.flink.api.connector.source.ReaderInfo; import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.assigner.SplitAssigner; +import org.apache.flink.connector.pulsar.source.enumerator.assigner.SplitAssignerFactory; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; +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.FullRangeGenerator; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.flink.connector.pulsar.testutils.PulsarTestSuiteBase; +import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator; import org.apache.flink.shaded.guava30.com.google.common.collect.Maps; import org.apache.flink.shaded.guava30.com.google.common.collect.Sets; import org.apache.pulsar.client.api.RegexSubscriptionMode; import org.apache.pulsar.client.api.SubscriptionType; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; -import java.util.ArrayList; -import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; +import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; +import java.util.stream.Collectors; +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_TYPE; import static org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor.latest; import static org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber.getTopicPatternSubscriber; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.apache.flink.shaded.guava30.com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly; +import static org.assertj.core.api.Assertions.assertThat; /** Unit tests for {@link PulsarSourceEnumerator}. */ class PulsarSourceEnumeratorTest extends PulsarTestSuiteBase { private static final int NUM_SUBTASKS = 3; - private static final String DYNAMIC_TOPIC_NAME = "dynamic_topic"; - private static final String TOPIC1 = "topic"; - private static final String TOPIC2 = "pattern-topic"; - private static final Set PRE_EXISTING_TOPICS = Sets.newHashSet(TOPIC1, TOPIC2); + private static final int READER0 = 0; + private static final int READER1 = 1; + private static final int PARTITION_DISCOVERY_CALLABLE_INDEX = 0; private static final boolean ENABLE_PERIODIC_PARTITION_DISCOVERY = true; private static final boolean DISABLE_PERIODIC_PARTITION_DISCOVERY = false; - private static final boolean INCLUDE_DYNAMIC_TOPIC = true; - private static final boolean EXCLUDE_DYNAMIC_TOPIC = false; - // @TestInstance(TestInstance.Lifecycle.PER_CLASS) is annotated in PulsarTestSuitBase, so this - // method could be non-static. - @BeforeAll - void beforeAll() { - operator().setupTopic(TOPIC1); - operator().setupTopic(TOPIC2); + @ParameterizedTest + @EnumSource( + value = SubscriptionType.class, + names = {"Failover", "Shared"}) + void startWithDiscoverPartitionsOnce(SubscriptionType subscriptionType) throws Exception { + Set prexistingTopics = setupPreexistingTopics(); + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + PulsarSourceEnumerator enumerator = + createEnumerator( + subscriptionType, + prexistingTopics, + context, + DISABLE_PERIODIC_PARTITION_DISCOVERY)) { + + // Start the enumerator and it should schedule a one time task to discover and assign + // partitions. + enumerator.start(); + assertThat(context.getPeriodicCallables()).isEmpty(); + assertThat(context.getOneTimeCallables()) + .as("A one time partition discovery callable should have been scheduled") + .hasSize(1); + } } - @AfterAll - void afterAll() { - operator().deleteTopic(TOPIC1, true); - operator().deleteTopic(TOPIC2, true); + @ParameterizedTest + @EnumSource( + value = SubscriptionType.class, + names = {"Failover", "Shared"}) + void startWithPeriodicPartitionDiscovery(SubscriptionType subscriptionType) throws Exception { + Set prexistingTopics = setupPreexistingTopics(); + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + PulsarSourceEnumerator enumerator = + createEnumerator( + subscriptionType, + prexistingTopics, + context, + ENABLE_PERIODIC_PARTITION_DISCOVERY)) { + + enumerator.start(); + assertThat(context.getOneTimeCallables()).isEmpty(); + assertThat((context.getPeriodicCallables())) + .as("A periodic partition discovery callable should have been scheduled") + .hasSize(1); + } } - @Test - void startWithDiscoverPartitionsOnce() throws Exception { + @ParameterizedTest + @EnumSource( + value = SubscriptionType.class, + names = {"Failover", "Shared"}) + void discoverPartitionsTriggersAssignments(SubscriptionType subscriptionType) throws Throwable { + Set prexistingTopics = setupPreexistingTopics(); try (MockSplitEnumeratorContext context = new MockSplitEnumeratorContext<>(NUM_SUBTASKS); PulsarSourceEnumerator enumerator = - createEnumerator(context, DISABLE_PERIODIC_PARTITION_DISCOVERY)) { + createEnumerator( + subscriptionType, + prexistingTopics, + context, + DISABLE_PERIODIC_PARTITION_DISCOVERY)) { - // Start the enumerator and it should schedule a one time task to discover and assign - // partitions. enumerator.start(); - assertTrue(context.getPeriodicCallables().isEmpty()); - assertEquals( - 1, - context.getOneTimeCallables().size(), - "A one time partition discovery callable should have been scheduled"); + + // register reader 0, 1 + registerReader(context, enumerator, READER0); + registerReader(context, enumerator, READER1); + assertThat(context.getSplitsAssignmentSequence()).isEmpty(); + + // Run the partition discover callable and check the partition assignment. + runOneTimePartitionDiscovery(context); + verifyLastReadersAssignments(subscriptionType, context, prexistingTopics, 1); } } - @Test - void startWithPeriodicPartitionDiscovery() throws Exception { + @ParameterizedTest + @EnumSource( + value = SubscriptionType.class, + names = {"Failover", "Shared"}) + void discoverPartitionsPeriodically(SubscriptionType subscriptionType) throws Throwable { + String dynamicTopic = randomAlphabetic(10); + Set prexistingTopics = setupPreexistingTopics(); + Set topicsToSubscribe = new HashSet<>(prexistingTopics); + topicsToSubscribe.add(dynamicTopic); try (MockSplitEnumeratorContext context = new MockSplitEnumeratorContext<>(NUM_SUBTASKS); PulsarSourceEnumerator enumerator = - createEnumerator(context, ENABLE_PERIODIC_PARTITION_DISCOVERY)) { + createEnumerator( + subscriptionType, + topicsToSubscribe, + context, + ENABLE_PERIODIC_PARTITION_DISCOVERY)) { - // Start the enumerator and it should schedule a one time task to discover and assign - // partitions. + testRegisterReadersForPreexistingTopics( + subscriptionType, prexistingTopics, context, enumerator); + + // invoke partition discovery callable again and there should be no new assignments. + runPeriodicPartitionDiscovery(context); + + int expectedSplitsAssignmentSequenceSize = + subscriptionType == SubscriptionType.Failover ? 1 : 2; + + assertThat(context.getSplitsAssignmentSequence()) + .as("No new assignments should be made because there is no partition change") + .hasSize(expectedSplitsAssignmentSequenceSize); + + // create the dynamic topic. + operator().createTopic(dynamicTopic, PulsarRuntimeOperator.DEFAULT_PARTITIONS); + + // invoke partition discovery callable again. + while (true) { + runPeriodicPartitionDiscovery(context); + if (context.getSplitsAssignmentSequence().size() < 2) { + sleepUninterruptibly(10, TimeUnit.MILLISECONDS); + } else { + break; + } + } + verifyLastReadersAssignments( + subscriptionType, + context, + Collections.singleton(dynamicTopic), + expectedSplitsAssignmentSequenceSize + 1); + } + } + + @ParameterizedTest + @EnumSource( + value = SubscriptionType.class, + names = {"Failover", "Shared"}) + void addSplitsBack(SubscriptionType subscriptionType) throws Throwable { + Set prexistingTopics = setupPreexistingTopics(); + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + PulsarSourceEnumerator enumerator = + createEnumerator( + subscriptionType, + prexistingTopics, + context, + ENABLE_PERIODIC_PARTITION_DISCOVERY)) { + + testRegisterReadersForPreexistingTopics( + subscriptionType, prexistingTopics, context, enumerator); + + // Simulate a reader failure. + context.unregisterReader(READER0); + enumerator.addSplitsBack( + context.getSplitsAssignmentSequence().get(0).assignment().get(READER0), + READER0); + int expectedSplitsAssignmentSequenceSize = + subscriptionType == SubscriptionType.Failover ? 1 : 2; + assertThat(context.getSplitsAssignmentSequence()) + .as("The added back splits should have not been assigned") + .hasSize(expectedSplitsAssignmentSequenceSize); + + // Simulate a reader recovery. + registerReader(context, enumerator, READER0); + verifyLastReadersAssignments( + subscriptionType, + context, + prexistingTopics, + expectedSplitsAssignmentSequenceSize + 1); + } + } + + @ParameterizedTest + @EnumSource( + value = SubscriptionType.class, + names = {"Failover"}) + void workWithPreexistingAssignments(SubscriptionType subscriptionType) throws Throwable { + Set prexistingTopics = setupPreexistingTopics(); + PulsarSourceEnumState preexistingAssignments; + try (MockSplitEnumeratorContext context1 = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + PulsarSourceEnumerator enumerator = + createEnumerator( + subscriptionType, + prexistingTopics, + context1, + ENABLE_PERIODIC_PARTITION_DISCOVERY)) { + testRegisterReadersForPreexistingTopics( + subscriptionType, prexistingTopics, context1, enumerator); + preexistingAssignments = + asEnumState(context1.getSplitsAssignmentSequence().get(0).assignment()); + } + + try (MockSplitEnumeratorContext context2 = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + PulsarSourceEnumerator enumerator = + createEnumerator( + subscriptionType, + prexistingTopics, + context2, + ENABLE_PERIODIC_PARTITION_DISCOVERY, + preexistingAssignments)) { enumerator.start(); - assertTrue(context.getOneTimeCallables().isEmpty()); - assertEquals( - 1, - context.getPeriodicCallables().size(), - "A periodic partition discovery callable should have been scheduled"); + runPeriodicPartitionDiscovery(context2); + + registerReader(context2, enumerator, READER0); + verifyLastReadersAssignments(subscriptionType, context2, prexistingTopics, 1); } } - private PulsarSourceEnumerator createEnumerator( + @ParameterizedTest + @EnumSource( + value = SubscriptionType.class, + names = {"Failover", "Shared"}) + void snapshotState(SubscriptionType subscriptionType) throws Throwable { + Set prexistingTopics = setupPreexistingTopics(); + try (MockSplitEnumeratorContext context = + new MockSplitEnumeratorContext<>(NUM_SUBTASKS); + PulsarSourceEnumerator enumerator = + createEnumerator(subscriptionType, prexistingTopics, context, false)) { + enumerator.start(); + + // No reader is registered, so the state should be empty + final PulsarSourceEnumState state1 = enumerator.snapshotState(1L); + assertThat(state1.getAppendedPartitions()).isEmpty(); + + registerReader(context, enumerator, READER0); + registerReader(context, enumerator, READER1); + runOneTimePartitionDiscovery(context); + + // The state should contain splits assigned to READER0 and READER1 + final PulsarSourceEnumState state2 = enumerator.snapshotState(1L); + verifySplitAssignmentWithPartitions( + getExpectedTopicPartitions(prexistingTopics), state2.getAppendedPartitions()); + } + } + + private Set setupPreexistingTopics() { + String topic1 = randomAlphabetic(10); + String topic2 = randomAlphabetic(10); + operator().setupTopic(topic1); + operator().setupTopic(topic2); + Set preexistingTopics = new HashSet<>(); + preexistingTopics.add(topic1); + preexistingTopics.add(topic2); + return preexistingTopics; + } + + private void testRegisterReadersForPreexistingTopics( + SubscriptionType subscriptionType, + Set topics, MockSplitEnumeratorContext context, - boolean enablePeriodicPartitionDiscovery) { - return createEnumerator(context, enablePeriodicPartitionDiscovery, EXCLUDE_DYNAMIC_TOPIC); + PulsarSourceEnumerator enumerator) + throws Throwable { + enumerator.start(); + + // register reader 0 before the partition discovery. + registerReader(context, enumerator, READER0); + assertThat(context.getSplitsAssignmentSequence()).isEmpty(); + + // Run the partition discover callable and check the partition assignment. + runPeriodicPartitionDiscovery(context); + verifyLastReadersAssignments(subscriptionType, context, topics, 1); + + registerReader(context, enumerator, READER1); + + int expectedSplitsAssignmentSequenceSize = + subscriptionType == SubscriptionType.Failover ? 1 : 2; + verifyLastReadersAssignments( + subscriptionType, context, topics, expectedSplitsAssignmentSequenceSize); } private PulsarSourceEnumerator createEnumerator( + SubscriptionType subscriptionType, + Set topics, MockSplitEnumeratorContext enumContext, - boolean enablePeriodicPartitionDiscovery, - boolean includeDynamicTopic) { - List topics = new ArrayList<>(PRE_EXISTING_TOPICS); - if (includeDynamicTopic) { - topics.add(DYNAMIC_TOPIC_NAME); - } - Configuration configuration = operator().config(); - configuration.set(PULSAR_SUBSCRIPTION_TYPE, SubscriptionType.Failover); - + boolean enablePeriodicPartitionDiscovery) { PulsarSourceEnumState sourceEnumState = new PulsarSourceEnumState( Sets.newHashSet(), @@ -136,40 +348,39 @@ private PulsarSourceEnumerator createEnumerator( Maps.newHashMap(), Maps.newHashMap(), false); - return createEnumerator( + subscriptionType, + topics, enumContext, enablePeriodicPartitionDiscovery, - topics, - sourceEnumState, - configuration); + sourceEnumState); } - /** - * Create the enumerator. For the purpose of the tests in this class we don't care about the - * subscriber and offsets initializer, so just use arbitrary settings. - */ private PulsarSourceEnumerator createEnumerator( + SubscriptionType subscriptionType, + Set topicsToSubscribe, MockSplitEnumeratorContext enumContext, boolean enablePeriodicPartitionDiscovery, - Collection topicsToSubscribe, - PulsarSourceEnumState sourceEnumState, - Configuration configuration) { + PulsarSourceEnumState sourceEnumState) { // Use a TopicPatternSubscriber so that no exception if a subscribed topic hasn't been // created yet. String topicRegex = String.join("|", topicsToSubscribe); Pattern topicPattern = Pattern.compile(topicRegex); PulsarSubscriber subscriber = getTopicPatternSubscriber(topicPattern, RegexSubscriptionMode.AllTopics); + + Configuration configuration = operator().config(); + configuration.set(PULSAR_SUBSCRIPTION_TYPE, subscriptionType); + configuration.set(PULSAR_SUBSCRIPTION_NAME, randomAlphabetic(10)); if (enablePeriodicPartitionDiscovery) { configuration.set(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, 60L); } else { configuration.set(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, -1L); } SourceConfiguration sourceConfiguration = new SourceConfiguration(configuration); - SplitsAssignmentState assignmentState = - new SplitsAssignmentState(latest(), sourceConfiguration, sourceEnumState); + SplitAssigner assigner = + SplitAssignerFactory.create(latest(), sourceConfiguration, sourceEnumState); return new PulsarSourceEnumerator( subscriber, StartCursor.earliest(), @@ -177,6 +388,104 @@ private PulsarSourceEnumerator createEnumerator( configuration, sourceConfiguration, enumContext, - assignmentState); + assigner); + } + + private void registerReader( + MockSplitEnumeratorContext context, + PulsarSourceEnumerator enumerator, + int reader) { + context.registerReader(new ReaderInfo(reader, "testing location ")); + enumerator.addReader(reader); + } + + private void verifyLastReadersAssignments( + SubscriptionType subscriptionType, + MockSplitEnumeratorContext context, + Set topics, + int expectedAssignmentSeqSize) { + assertThat(context.getSplitsAssignmentSequence()).hasSize(expectedAssignmentSeqSize); + verifyAssignments( + subscriptionType, + getExpectedTopicPartitions(topics), + context.getSplitsAssignmentSequence() + .get(expectedAssignmentSeqSize - 1) + .assignment()); + } + + private void verifyAssignments( + SubscriptionType subscriptionType, + Set expectedTopicPartitions, + Map> actualAssignments) { + if (subscriptionType == SubscriptionType.Failover) { + int actualSize = actualAssignments.values().stream().mapToInt(List::size).sum(); + assertThat(actualSize).isEqualTo(expectedTopicPartitions.size()); + } else if (subscriptionType == SubscriptionType.Shared) { + actualAssignments + .values() + .forEach( + (splits) -> assertThat(splits).hasSize(expectedTopicPartitions.size())); + } + } + + private Set getExpectedTopicPartitions(Set topics) { + Set allPartitions = new HashSet<>(); + for (String topicName : topics) { + for (int i = 0; i < PulsarRuntimeOperator.DEFAULT_PARTITIONS; i++) { + allPartitions.add(new TopicPartition(topicName, i, TopicRange.createFullRange())); + } + } + return allPartitions; + } + + private void verifySplitAssignmentWithPartitions( + Set expectedAssignment, Set actualTopicPartitions) { + assertThat(actualTopicPartitions).isEqualTo(expectedAssignment); + } + + // this method only works for non Shared Mode + private PulsarSourceEnumState asEnumState( + Map> assignments) { + Set appendedPartitions = new HashSet<>(); + Set pendingPartitionSplits = new HashSet<>(); + Map> sharedPendingPartitionSplits = new HashMap<>(); + Map> readerAssignedSplits = new HashMap<>(); + boolean initialized = false; + + assignments + .values() + .forEach( + splits -> { + appendedPartitions.addAll( + splits.stream() + .map(PulsarPartitionSplit::getPartition) + .collect(Collectors.toList())); + pendingPartitionSplits.addAll(splits); + }); + + return new PulsarSourceEnumState( + appendedPartitions, + pendingPartitionSplits, + sharedPendingPartitionSplits, + readerAssignedSplits, + initialized); + } + + private void runOneTimePartitionDiscovery( + MockSplitEnumeratorContext context) throws Throwable { + // Fetch potential topic descriptions + context.runNextOneTimeCallable(); + if (!context.getOneTimeCallables().isEmpty()) { + context.runNextOneTimeCallable(); + } + } + + private void runPeriodicPartitionDiscovery( + MockSplitEnumeratorContext context) throws Throwable { + // Fetch potential topic descriptions + context.runPeriodicCallable(PARTITION_DISCOVERY_CALLABLE_INDEX); + if (!context.getOneTimeCallables().isEmpty()) { + context.runNextOneTimeCallable(); + } } } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/SplitsAssignmentStateTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/SplitsAssignmentStateTest.java deleted file mode 100644 index ac811c3dddbfe..0000000000000 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/SplitsAssignmentStateTest.java +++ /dev/null @@ -1,119 +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.source.enumerator; - -import org.apache.flink.api.connector.source.SplitsAssignment; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange; -import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; - -import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; -import org.apache.flink.shaded.guava30.com.google.common.collect.Sets; - -import org.apache.pulsar.client.api.SubscriptionType; -import org.junit.jupiter.api.Test; - -import java.util.List; -import java.util.Optional; -import java.util.Set; - -import static java.util.Collections.singletonList; -import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; -import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_TYPE; -import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.MAX_RANGE; -import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.InstanceOfAssertFactories.map; - -/** Unit tests for {@link SplitsAssignmentState}. */ -class SplitsAssignmentStateTest { - - private final Set partitions = - Sets.newHashSet( - new TopicPartition("some-topic", 1, new TopicRange(1, 30)), - new TopicPartition("some-topic", 2, new TopicRange(31, 60)), - new TopicPartition("some-topic", 3, new TopicRange(61, MAX_RANGE)), - new TopicPartition(randomAlphabetic(10), -1, createFullRange())); - - @Test - void assignSplitsForSharedSubscription() { - SplitsAssignmentState state1 = - new SplitsAssignmentState( - StopCursor.defaultStopCursor(), createConfig(SubscriptionType.Shared)); - state1.appendTopicPartitions(partitions); - Optional> assignment1 = - state1.assignSplits(Lists.newArrayList(0, 1, 2, 3, 4)); - - assertThat(assignment1) - .isPresent() - .get() - .extracting(SplitsAssignment::assignment) - .asInstanceOf(map(Integer.class, List.class)) - .hasSize(5) - .allSatisfy((idx, list) -> assertThat(list).hasSize(4)); - - Optional> assignment2 = - state1.assignSplits(Lists.newArrayList(0, 1, 2, 3, 4)); - assertThat(assignment2).isNotPresent(); - - // Reassign reader 3. - state1.putSplitsBackToPendingList(assignment1.get().assignment().get(3), 3); - Optional> assignment3 = - state1.assignSplits(Lists.newArrayList(0, 1, 2, 4)); - assertThat(assignment3).isNotPresent(); - - Optional> assignment4 = - state1.assignSplits(singletonList(3)); - assertThat(assignment4) - .isPresent() - .get() - .extracting(SplitsAssignment::assignment) - .asInstanceOf(map(Integer.class, List.class)) - .hasSize(1); - } - - @Test - void assignSplitsForExclusiveSubscription() { - SplitsAssignmentState state1 = - new SplitsAssignmentState( - StopCursor.defaultStopCursor(), createConfig(SubscriptionType.Exclusive)); - state1.appendTopicPartitions(partitions); - Optional> assignment1 = - state1.assignSplits(Lists.newArrayList(0, 1, 2, 3, 4)); - - assertThat(assignment1).isPresent(); - assertThat(assignment1.get().assignment()) - .hasSize(4) - .allSatisfy((idx, list) -> assertThat(list).hasSize(1)); - - Optional> assignment2 = - state1.assignSplits(Lists.newArrayList(0, 1, 2, 3, 4)); - assertThat(assignment2).isNotPresent(); - } - - private SourceConfiguration createConfig(SubscriptionType type) { - Configuration configuration = new Configuration(); - configuration.set(PULSAR_SUBSCRIPTION_TYPE, type); - - return new SourceConfiguration(configuration); - } -} 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 new file mode 100644 index 0000000000000..2e9ada3b74135 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssignerTest.java @@ -0,0 +1,95 @@ +/* + * 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.assigner; + +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +import static java.util.Collections.singletonList; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** Unit tests for {@link NonSharedSplitAssigner}. */ +class NonSharedSplitAssignerTest extends SplitAssignerTestBase { + + @Test + void noMoreSplits() { + NonSharedSplitAssigner assigner = splitAssigner(true); + assertFalse(assigner.noMoreSplits(3)); + + assigner = splitAssigner(false); + assertFalse(assigner.noMoreSplits(3)); + + assigner.registerTopicPartitions(createPartitions("f", 8)); + assertFalse(assigner.noMoreSplits(3)); + + assigner.createAssignment(singletonList(1)); + assertTrue(assigner.noMoreSplits(1)); + assertTrue(assigner.noMoreSplits(3)); + } + + @Test + void partitionsAssignment() { + NonSharedSplitAssigner assigner = splitAssigner(true); + assigner.registerTopicPartitions(createPartitions("d", 4)); + List readers = Arrays.asList(1, 3, 5, 7); + + // Assignment with initial states. + Optional> assignment = + assigner.createAssignment(readers); + assertThat(assignment).isPresent(); + assertThat(assignment.get().assignment()).hasSize(1); + + // Reassignment with same readers. + assignment = assigner.createAssignment(readers); + assertThat(assignment).isNotPresent(); + + // Register new partition and assign. + assigner.registerTopicPartitions(createPartitions("e", 5)); + assigner.registerTopicPartitions(createPartitions("f", 1)); + assigner.registerTopicPartitions(createPartitions("g", 3)); + assigner.registerTopicPartitions(createPartitions("h", 4)); + assignment = assigner.createAssignment(readers); + assertThat(assignment).isPresent(); + assertThat(assignment.get().assignment()).hasSize(4); + + // Assign to new readers. + readers = Arrays.asList(2, 4, 6, 8); + assignment = assigner.createAssignment(readers); + assertThat(assignment).isNotPresent(); + } + + @Override + protected NonSharedSplitAssigner createAssigner( + StopCursor stopCursor, + SourceConfiguration sourceConfiguration, + PulsarSourceEnumState sourceEnumState) { + return new NonSharedSplitAssigner(stopCursor, sourceConfiguration, sourceEnumState); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssignerTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssignerTest.java new file mode 100644 index 0000000000000..91584b8768821 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssignerTest.java @@ -0,0 +1,98 @@ +/* + * 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.assigner; + +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +import static java.util.Collections.singletonList; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** Unit tests for {@link SharedSplitAssigner}. */ +class SharedSplitAssignerTest extends SplitAssignerTestBase { + + @Test + void noMoreSplits() { + SharedSplitAssigner assigner = splitAssigner(true); + assertFalse(assigner.noMoreSplits(3)); + + assigner = splitAssigner(false); + assertFalse(assigner.noMoreSplits(3)); + + assigner.registerTopicPartitions(createPartitions("f", 8)); + assertFalse(assigner.noMoreSplits(3)); + + assigner.createAssignment(singletonList(1)); + assertTrue(assigner.noMoreSplits(1)); + assertFalse(assigner.noMoreSplits(3)); + + assigner.createAssignment(singletonList(3)); + assertTrue(assigner.noMoreSplits(3)); + } + + @Test + void partitionsAssignment() { + SharedSplitAssigner assigner = splitAssigner(true); + assigner.registerTopicPartitions(createPartitions("d", 4)); + List readers = Arrays.asList(1, 3, 5, 7); + + // Assignment with initial states. + Optional> assignment = + assigner.createAssignment(readers); + assertThat(assignment).isPresent(); + assertThat(assignment.get().assignment()).hasSize(4); + + // Reassignment with same readers. + assignment = assigner.createAssignment(readers); + assertThat(assignment).isNotPresent(); + + // Register new partition and assign. + assigner.registerTopicPartitions(createPartitions("e", 5)); + assignment = assigner.createAssignment(readers); + assertThat(assignment).isPresent(); + assertThat(assignment.get().assignment()).hasSize(4); + + // Assign to new readers. + readers = Arrays.asList(2, 4, 6, 8); + assignment = assigner.createAssignment(readers); + assertThat(assignment).isPresent(); + assertThat(assignment.get().assignment()) + .hasSize(4) + .allSatisfy((k, v) -> assertThat(v).hasSize(2)); + } + + @Override + protected SharedSplitAssigner createAssigner( + StopCursor stopCursor, + SourceConfiguration sourceConfiguration, + PulsarSourceEnumState sourceEnumState) { + return new SharedSplitAssigner(stopCursor, sourceConfiguration, sourceEnumState); + } +} 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 new file mode 100644 index 0000000000000..6509401472068 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerTestBase.java @@ -0,0 +1,113 @@ +/* + * 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.assigner; + +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +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.split.PulsarPartitionSplit; +import org.apache.flink.util.TestLogger; + +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.Optional; +import java.util.Set; + +import static java.util.Collections.emptyList; +import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS; +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. */ +abstract class SplitAssignerTestBase extends TestLogger { + + @Test + void registerTopicPartitionsWillOnlyReturnNewPartitions() { + T assigner = splitAssigner(true); + + Set partitions = createPartitions("persistent://public/default/a", 1); + List newPartitions = assigner.registerTopicPartitions(partitions); + assertThat(newPartitions) + .hasSize(1) + .first() + .hasFieldOrPropertyWithValue("topic", "persistent://public/default/a") + .hasFieldOrPropertyWithValue("partitionId", 1); + + newPartitions = assigner.registerTopicPartitions(partitions); + assertThat(newPartitions).isEmpty(); + + partitions = createPartitions("persistent://public/default/b", 2); + newPartitions = assigner.registerTopicPartitions(partitions); + assertThat(newPartitions) + .hasSize(1) + .hasSize(1) + .first() + .hasFieldOrPropertyWithValue("topic", "persistent://public/default/b") + .hasFieldOrPropertyWithValue("partitionId", 2); + } + + @Test + void noReadersProvideForAssignment() { + T assigner = splitAssigner(false); + assigner.registerTopicPartitions(createPartitions("c", 5)); + + Optional> assignment = + assigner.createAssignment(emptyList()); + assertThat(assignment).isNotPresent(); + } + + @Test + void noPartitionsProvideForAssignment() { + T assigner = splitAssigner(true); + Optional> assignment = + assigner.createAssignment(singletonList(4)); + assertThat(assignment).isNotPresent(); + } + + protected Set createPartitions(String topic, int partitionId) { + TopicPartition p1 = new TopicPartition(topic, partitionId, createFullRange()); + return singleton(p1); + } + + protected T splitAssigner(boolean discovery) { + Configuration configuration = new Configuration(); + + if (discovery) { + configuration.set(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, 1000L); + } else { + configuration.set(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, -1L); + } + + SourceConfiguration sourceConfiguration = new SourceConfiguration(configuration); + return createAssigner(defaultStopCursor(), sourceConfiguration, initialState()); + } + + protected abstract T createAssigner( + StopCursor stopCursor, + SourceConfiguration sourceConfiguration, + PulsarSourceEnumState sourceEnumState); +} 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 cce21be9e5e28..0c69bf301b244 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 @@ -23,7 +23,6 @@ import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; -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.reader.message.PulsarMessage; import org.apache.flink.connector.pulsar.source.reader.split.PulsarOrderedPartitionSplitReader; @@ -43,29 +42,31 @@ import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_RECORDS; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_TIME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_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; /** Test different implementation of StopCursor. */ -public class StopCursorTest extends PulsarTestSuiteBase { +class StopCursorTest extends PulsarTestSuiteBase { @Test void publishTimeStopCursor() throws IOException { String topicName = randomAlphanumeric(5); operator().createTopic(topicName, 2); + Configuration configuration = config(); PulsarOrderedPartitionSplitReader splitReader = new PulsarOrderedPartitionSplitReader<>( operator().client(), operator().admin(), - config(), - sourceConfig(), + configuration, + sourceConfig(configuration), flinkSchema(new SimpleStringSchema())); // send the first message and set the stopCursor to filter any late stopCursor operator() .sendMessage( - TopicNameUtils.topicNameWithPartition(topicName, 0), + topicNameWithPartition(topicName, 0), Schema.STRING, randomAlphanumeric(10)); long currentTimeStamp = System.currentTimeMillis(); @@ -86,12 +87,11 @@ void publishTimeStopCursor() throws IOException { // send the second message and expect it will not be received operator() .sendMessage( - TopicNameUtils.topicNameWithPartition(topicName, 0), + topicNameWithPartition(topicName, 0), Schema.STRING, randomAlphanumeric(10)); RecordsWithSplitIds> secondResult = splitReader.fetch(); - assertThat(secondResult.nextSplit()).isNotNull(); - assertThat(firstResult.nextRecordFromSplit()).isNull(); + assertThat(secondResult.nextSplit()).isNull(); assertThat(secondResult.finishedSplits()).isNotEmpty(); } @@ -104,7 +104,7 @@ private Configuration config() { return config; } - private SourceConfiguration sourceConfig() { - return new SourceConfiguration(config()); + private SourceConfiguration sourceConfig(Configuration config) { + return new SourceConfiguration(config); } } 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 eaae6ba6b33f5..f57980e348192 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 @@ -65,11 +65,11 @@ void setUp() { @AfterAll void tearDown() { - operator().deleteTopic(topic1, true); - operator().deleteTopic(topic2, true); - operator().deleteTopic(topic3, true); - operator().deleteTopic(topic4, false); - operator().deleteTopic(topic5, false); + operator().deleteTopic(topic1); + operator().deleteTopic(topic2); + operator().deleteTopic(topic3); + operator().deleteTopic(topic4); + operator().deleteTopic(topic5); } @Test diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaTest.java index aa4bcee6320a7..48e6e7a4c21b3 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaTest.java @@ -113,7 +113,7 @@ private Message getMessage( MessageMetadata metadata = new MessageMetadata(); ByteBuffer payload = ByteBuffer.wrap(bytes); - return MessageImpl.create(metadata, payload, Schema.BYTES); + return MessageImpl.create(metadata, payload, Schema.BYTES, ""); } /** This collector is used for collecting only one message. Used for test purpose. */ diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReaderTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReaderTest.java index a94504a9400ae..3d58d5efeb4c8 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReaderTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReaderTest.java @@ -18,20 +18,74 @@ package org.apache.flink.connector.pulsar.source.reader.split; -import org.apache.flink.api.common.serialization.SimpleStringSchema; +import org.apache.flink.connector.pulsar.testutils.extension.SubType; -import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.flinkSchema; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.impl.MessageIdImpl; +import org.junit.jupiter.api.TestTemplate; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; +import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyAdmin; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; +import static org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator.NUM_RECORDS_PER_PARTITION; +import static org.apache.pulsar.client.api.Schema.STRING; /** Unit tests for {@link PulsarOrderedPartitionSplitReaderTest}. */ class PulsarOrderedPartitionSplitReaderTest extends PulsarPartitionSplitReaderTestBase { - @Override - protected PulsarPartitionSplitReaderBase splitReader() { - return new PulsarOrderedPartitionSplitReader<>( - operator().client(), - operator().admin(), - readerConfig(), - sourceConfig(), - flinkSchema(new SimpleStringSchema())); + @SubType SubscriptionType subscriptionType = SubscriptionType.Failover; + + @TestTemplate + void consumeMessageCreatedBeforeHandleSplitsChangesWithoutSeek( + PulsarPartitionSplitReaderBase splitReader) { + String topicName = randomAlphabetic(10); + operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); + handleSplit(splitReader, topicName, 0); + fetchedMessages(splitReader, 0, true); + } + + @TestTemplate + void consumeMessageCreatedBeforeHandleSplitsChangesAndUseLatestStartCursorWithoutSeek( + PulsarPartitionSplitReaderBase splitReader) { + String topicName = randomAlphabetic(10); + operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); + handleSplit(splitReader, topicName, 0, MessageId.latest); + fetchedMessages(splitReader, 0, true); + } + + @TestTemplate + void consumeMessageCreatedBeforeHandleSplitsChangesAndUseEarliestStartCursorWithoutSeek( + PulsarPartitionSplitReaderBase splitReader) { + String topicName = randomAlphabetic(10); + operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); + handleSplit(splitReader, topicName, 0, MessageId.earliest); + fetchedMessages(splitReader, NUM_RECORDS_PER_PARTITION, true); + } + + @TestTemplate + void consumeMessageCreatedBeforeHandleSplitsChangesAndUseSecondLastMessageWithoutSeek( + PulsarPartitionSplitReaderBase splitReader) { + String topicName = randomAlphabetic(10); + operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); + MessageIdImpl lastMessageId = + (MessageIdImpl) + sneakyAdmin( + () -> + operator() + .admin() + .topics() + .getLastMessageId( + topicNameWithPartition(topicName, 0))); + // when recover, use exclusive startCursor + handleSplit( + splitReader, + topicName, + 0, + new MessageIdImpl( + lastMessageId.getLedgerId(), + lastMessageId.getEntryId() - 1, + lastMessageId.getPartitionIndex())); + fetchedMessages(splitReader, 1, true); } } 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 fb9fa01e4896e..e481d1fd793b7 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 @@ -18,17 +18,27 @@ package org.apache.flink.connector.pulsar.source.reader.split; +import org.apache.flink.api.common.serialization.SimpleStringSchema; +import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.flink.connector.pulsar.testutils.PulsarTestSuiteBase; +import org.apache.flink.connector.pulsar.testutils.extension.TestOrderlinessExtension; +import org.apache.flink.connectors.test.common.junit.extensions.TestLoggerExtension; -import org.junit.jupiter.api.DisplayName; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.impl.MessageIdImpl; import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.extension.Extension; import org.junit.jupiter.api.extension.ExtensionContext; import org.junit.jupiter.api.extension.ParameterContext; @@ -39,27 +49,41 @@ import org.junit.jupiter.api.extension.TestTemplateInvocationContextProvider; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Stream; import static java.time.Duration.ofSeconds; import static java.util.Collections.singletonList; import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; +import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyAdmin; +import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyThrow; 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; 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.cursor.StopCursor.never; 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.extension.TestOrderlinessExtension.PULSAR_SOURCE_READER_SUBSCRIPTION_TYPE_STORE_KEY; +import static org.apache.flink.connector.pulsar.testutils.extension.TestOrderlinessExtension.PULSAR_TEST_RESOURCE_NAMESPACE; +import static org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator.DEFAULT_PARTITIONS; +import static org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator.NUM_RECORDS_PER_PARTITION; import static org.apache.flink.core.testutils.CommonTestUtils.waitUtil; +import static org.apache.flink.shaded.guava30.com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.pulsar.client.api.Schema.STRING; -import static org.junit.jupiter.api.Assertions.assertNull; +import static org.assertj.core.api.Assertions.assertThat; /** Test utils for split readers. */ +@ExtendWith({ + TestOrderlinessExtension.class, + TestLoggerExtension.class, +}) public abstract class PulsarPartitionSplitReaderTestBase extends PulsarTestSuiteBase { @RegisterExtension @@ -76,41 +100,111 @@ protected Configuration readerConfig() { return config; } - protected SourceConfiguration sourceConfig() { - return new SourceConfiguration(readerConfig()); + protected SourceConfiguration sourceConfig(Configuration config) { + return new SourceConfiguration(config); } - protected SplitsAddition createSplit(String topicName, int partitionId) { + protected void handleSplit( + PulsarPartitionSplitReaderBase reader, String topicName, int partitionId) { + handleSplit(reader, topicName, partitionId, null); + } + + protected void handleSplit( + PulsarPartitionSplitReaderBase reader, + String topicName, + int partitionId, + MessageId startPosition) { + TopicPartition partition = new TopicPartition(topicName, partitionId, createFullRange()); + PulsarPartitionSplit split = + new PulsarPartitionSplit(partition, StopCursor.never(), startPosition, null); + SplitsAddition addition = new SplitsAddition<>(singletonList(split)); + reader.handleSplitsChanges(addition); + } + + private void seekStartPositionAndHandleSplit( + PulsarPartitionSplitReaderBase reader, String topicName, int partitionId) { + seekStartPositionAndHandleSplit(reader, topicName, partitionId, MessageId.latest); + } + + private void seekStartPositionAndHandleSplit( + PulsarPartitionSplitReaderBase reader, + String topicName, + int partitionId, + MessageId startPosition) { TopicPartition partition = new TopicPartition(topicName, partitionId, createFullRange()); - PulsarPartitionSplit split = new PulsarPartitionSplit(partition, never()); - return new SplitsAddition<>(singletonList(split)); + PulsarPartitionSplit split = + new PulsarPartitionSplit(partition, StopCursor.never(), null, null); + SplitsAddition addition = new SplitsAddition<>(singletonList(split)); + + // create consumer and seek before split changes + try (Consumer consumer = reader.createPulsarConsumer(partition)) { + // inclusive messageId + consumer.seek(startPosition); + } catch (PulsarClientException e) { + sneakyThrow(e); + } + + reader.handleSplitsChanges(addition); + } + + private PulsarMessage fetchedMessage(PulsarPartitionSplitReaderBase splitReader) { + return fetchedMessages(splitReader, 1, false).stream().findFirst().orElse(null); + } + + protected List> fetchedMessages( + PulsarPartitionSplitReaderBase splitReader, int expectedCount, boolean verify) { + return fetchedMessages( + splitReader, expectedCount, verify, Boundedness.CONTINUOUS_UNBOUNDED); } - protected PulsarMessage fetchedMessage(PulsarPartitionSplitReaderBase splitReader) { - try { - RecordsWithSplitIds> records = splitReader.fetch(); - if (records.nextSplit() != null) { - return records.nextRecordFromSplit(); + private List> fetchedMessages( + PulsarPartitionSplitReaderBase splitReader, + int expectedCount, + boolean verify, + Boundedness boundedness) { + List> messages = new ArrayList<>(expectedCount); + List finishedSplits = new ArrayList<>(); + for (int i = 0; i < 3; ) { + try { + RecordsWithSplitIds> recordsBySplitIds = splitReader.fetch(); + if (recordsBySplitIds.nextSplit() != null) { + // Collect the records in this split. + PulsarMessage record; + while ((record = recordsBySplitIds.nextRecordFromSplit()) != null) { + messages.add(record); + } + finishedSplits.addAll(recordsBySplitIds.finishedSplits()); + } else { + i++; + } + } catch (IOException e) { + i++; + } + sleepUninterruptibly(1, TimeUnit.SECONDS); + } + if (verify) { + assertThat(messages).as("We should fetch the expected size").hasSize(expectedCount); + if (boundedness == Boundedness.CONTINUOUS_UNBOUNDED) { + assertThat(finishedSplits).as("Split should not be marked as finished").isEmpty(); } else { - return null; + assertThat(finishedSplits).as("Split should be marked as finished").hasSize(1); } - } catch (IOException e) { - return null; } + + return messages; } @TestTemplate - @DisplayName("Retrieve message after timeout by using given split reader") void pollMessageAfterTimeout(PulsarPartitionSplitReaderBase splitReader) throws InterruptedException, TimeoutException { String topicName = randomAlphabetic(10); // Add a split - splitReader.handleSplitsChanges(createSplit(topicName, 0)); + seekStartPositionAndHandleSplit(splitReader, topicName, 0); // Poll once with a null message PulsarMessage message1 = fetchedMessage(splitReader); - assertNull(message1); + assertThat(message1).isNull(); // Send a message to pulsar String topic = topicNameWithPartition(topicName, 0); @@ -122,14 +216,140 @@ void pollMessageAfterTimeout(PulsarPartitionSplitReaderBase splitReader) PulsarMessage message2 = fetchedMessage(splitReader); return message2 != null; }, - ofSeconds(10), + ofSeconds(Integer.MAX_VALUE), "Couldn't poll message from Pulsar."); } + @TestTemplate + void consumeMessageCreatedAfterHandleSplitChangesAndFetch( + PulsarPartitionSplitReaderBase splitReader) { + String topicName = randomAlphabetic(10); + seekStartPositionAndHandleSplit(splitReader, topicName, 0); + operator().sendMessage(topicNameWithPartition(topicName, 0), STRING, randomAlphabetic(10)); + fetchedMessages(splitReader, 1, true); + } + + @TestTemplate + void consumeMessageCreatedBeforeHandleSplitsChanges( + PulsarPartitionSplitReaderBase splitReader) { + String topicName = randomAlphabetic(10); + operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); + seekStartPositionAndHandleSplit(splitReader, topicName, 0); + fetchedMessages(splitReader, 0, true); + } + + @TestTemplate + void consumeMessageCreatedBeforeHandleSplitsChangesAndResetToEarliestPosition( + PulsarPartitionSplitReaderBase splitReader) { + String topicName = randomAlphabetic(10); + operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); + seekStartPositionAndHandleSplit(splitReader, topicName, 0, MessageId.earliest); + fetchedMessages(splitReader, NUM_RECORDS_PER_PARTITION, true); + } + + @TestTemplate + void consumeMessageCreatedBeforeHandleSplitsChangesAndResetToLatestPosition( + PulsarPartitionSplitReaderBase splitReader) { + String topicName = randomAlphabetic(10); + operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); + seekStartPositionAndHandleSplit(splitReader, topicName, 0, MessageId.latest); + fetchedMessages(splitReader, 0, true); + } + + @TestTemplate + void consumeMessageCreatedBeforeHandleSplitsChangesAndUseSecondLastMessageIdCursor( + PulsarPartitionSplitReaderBase splitReader) { + + String topicName = randomAlphabetic(10); + operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); + MessageIdImpl lastMessageId = + (MessageIdImpl) + sneakyAdmin( + () -> + operator() + .admin() + .topics() + .getLastMessageId( + topicNameWithPartition(topicName, 0))); + // when doing seek directly on consumer, by default it includes the specified messageId + seekStartPositionAndHandleSplit( + splitReader, + topicName, + 0, + new MessageIdImpl( + lastMessageId.getLedgerId(), + lastMessageId.getEntryId() - 1, + lastMessageId.getPartitionIndex())); + fetchedMessages(splitReader, 2, true); + } + + @TestTemplate + void emptyTopic(PulsarPartitionSplitReaderBase splitReader) { + String topicName = randomAlphabetic(10); + operator().createTopic(topicName, DEFAULT_PARTITIONS); + seekStartPositionAndHandleSplit(splitReader, topicName, 0); + fetchedMessages(splitReader, 0, true); + } + + @TestTemplate + void emptyTopicWithoutSeek(PulsarPartitionSplitReaderBase splitReader) { + String topicName = randomAlphabetic(10); + operator().createTopic(topicName, DEFAULT_PARTITIONS); + handleSplit(splitReader, topicName, 0); + fetchedMessages(splitReader, 0, true); + } + + @TestTemplate + void wakeupSplitReaderShouldNotCauseException( + PulsarPartitionSplitReaderBase splitReader) { + handleSplit(splitReader, "non-exist", 0); + AtomicReference error = new AtomicReference<>(); + Thread t = + new Thread( + () -> { + try { + splitReader.fetch(); + } catch (Throwable e) { + error.set(e); + } + }, + "testWakeUp-thread"); + t.start(); + long deadline = System.currentTimeMillis() + 5000L; + while (t.isAlive() && System.currentTimeMillis() < deadline) { + splitReader.wakeUp(); + sleepUninterruptibly(10, TimeUnit.MILLISECONDS); + } + assertThat(error.get()).isNull(); + } + + @TestTemplate + void assignNoSplits(PulsarPartitionSplitReaderBase splitReader) { + assertThat(fetchedMessage(splitReader)).isNull(); + } + /** Create a split reader with max message 1, fetch timeout 1s. */ - protected abstract PulsarPartitionSplitReaderBase splitReader(); + private PulsarPartitionSplitReaderBase splitReader(SubscriptionType subscriptionType) { + Configuration configuration = readerConfig(); + if (subscriptionType == SubscriptionType.Failover) { + return new PulsarOrderedPartitionSplitReader<>( + operator().client(), + operator().admin(), + configuration, + sourceConfig(configuration), + flinkSchema(new SimpleStringSchema())); + } else { + return new PulsarUnorderedPartitionSplitReader<>( + operator().client(), + operator().admin(), + configuration, + sourceConfig(configuration), + flinkSchema(new SimpleStringSchema()), + null); + } + } - /** JUnit5 extension for all the TestTemplate methods in this class. */ + /** Context Provider for PulsarSplitReaderTestBase. */ public class PulsarSplitReaderInvocationContextProvider implements TestTemplateInvocationContextProvider { @@ -141,7 +361,11 @@ public boolean supportsTestTemplate(ExtensionContext context) { @Override public Stream provideTestTemplateInvocationContexts( ExtensionContext context) { - return Stream.of(new PulsarSplitReaderInvocationContext(splitReader())); + SubscriptionType subscriptionType = + (SubscriptionType) + context.getStore(PULSAR_TEST_RESOURCE_NAMESPACE) + .get(PULSAR_SOURCE_READER_SUBSCRIPTION_TYPE_STORE_KEY); + return Stream.of(new PulsarSplitReaderInvocationContext(splitReader(subscriptionType))); } } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReaderTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReaderTest.java index 917cacd67e9c8..2cb3cb9c84744 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReaderTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReaderTest.java @@ -18,21 +18,11 @@ package org.apache.flink.connector.pulsar.source.reader.split; -import org.apache.flink.api.common.serialization.SimpleStringSchema; +import org.apache.flink.connector.pulsar.testutils.extension.SubType; -import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.flinkSchema; +import org.apache.pulsar.client.api.SubscriptionType; /** Unit tests for {@link PulsarUnorderedPartitionSplitReaderTest}. */ class PulsarUnorderedPartitionSplitReaderTest extends PulsarPartitionSplitReaderTestBase { - - @Override - protected PulsarPartitionSplitReaderBase splitReader() { - return new PulsarUnorderedPartitionSplitReader<>( - operator().client(), - operator().admin(), - readerConfig(), - sourceConfig(), - flinkSchema(new SimpleStringSchema()), - null); - } + @SubType SubscriptionType subscriptionType = SubscriptionType.Shared; } 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 2ad4c2f8bc76e..e562703461e31 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 @@ -29,10 +29,7 @@ /** Common test context for pulsar based test. */ public abstract class PulsarTestContext implements ExternalContext { - private static final long serialVersionUID = 4717940854368532130L; - - private static final int NUM_RECORDS_UPPER_BOUND = 500; - private static final int NUM_RECORDS_LOWER_BOUND = 100; + private static final long serialVersionUID = 8109719617929996743L; protected final PulsarRuntimeOperator operator; @@ -44,14 +41,12 @@ protected PulsarTestContext(PulsarTestEnvironment environment) { protected List generateStringTestData(int splitIndex, long seed) { Random random = new Random(seed); - int recordNum = - random.nextInt(NUM_RECORDS_UPPER_BOUND - NUM_RECORDS_LOWER_BOUND) - + NUM_RECORDS_LOWER_BOUND; + int recordNum = 300 + random.nextInt(200); List records = new ArrayList<>(recordNum); for (int i = 0; i < recordNum; i++) { - int stringLength = random.nextInt(50) + 1; - records.add(splitIndex + "-" + randomAlphanumeric(stringLength)); + int length = random.nextInt(40) + 10; + records.add(splitIndex + "-" + i + "-" + randomAlphanumeric(length)); } return records; diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/extension/SubType.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/extension/SubType.java new file mode 100644 index 0000000000000..b1a83624158b3 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/extension/SubType.java @@ -0,0 +1,32 @@ +/* + * 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.extension; + +import org.apache.flink.annotation.Experimental; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** Marks the field in test class defining {@link org.apache.pulsar.client.api.SubscriptionType}. */ +@Target(ElementType.FIELD) +@Retention(RetentionPolicy.RUNTIME) +@Experimental +public @interface SubType {} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/extension/TestOrderlinessExtension.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/extension/TestOrderlinessExtension.java new file mode 100644 index 0000000000000..07c9287ff05f4 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/extension/TestOrderlinessExtension.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.extension; + +import org.apache.pulsar.client.api.SubscriptionType; +import org.junit.jupiter.api.extension.BeforeAllCallback; +import org.junit.jupiter.api.extension.ExtensionContext; +import org.junit.platform.commons.support.AnnotationSupport; + +import java.lang.annotation.Annotation; +import java.util.Collection; +import java.util.List; + +/** An extension for subclasses to specify {@link org.apache.pulsar.client.api.SubscriptionType}. */ +public class TestOrderlinessExtension implements BeforeAllCallback { + + public static final ExtensionContext.Namespace PULSAR_TEST_RESOURCE_NAMESPACE = + ExtensionContext.Namespace.create("pulsarTestResourceNamespace"); + public static final String PULSAR_SOURCE_READER_SUBSCRIPTION_TYPE_STORE_KEY = + "pulsarSourceReaderSubscriptionTypeStoreKey"; + + private SubscriptionType subscriptionType; + + @Override + public void beforeAll(ExtensionContext context) throws Exception { + final List subscriptionTypes = + AnnotationSupport.findAnnotatedFieldValues( + context.getRequiredTestInstance(), SubType.class, SubscriptionType.class); + checkExactlyOneAnnotatedField(subscriptionTypes, SubType.class); + subscriptionType = subscriptionTypes.get(0); + context.getStore(PULSAR_TEST_RESOURCE_NAMESPACE) + .put(PULSAR_SOURCE_READER_SUBSCRIPTION_TYPE_STORE_KEY, subscriptionType); + } + + private void checkExactlyOneAnnotatedField( + Collection fields, Class annotation) { + if (fields.size() > 1) { + throw new IllegalStateException( + String.format( + "Multiple fields are annotated with '@%s'", + annotation.getSimpleName())); + } + if (fields.isEmpty()) { + throw new IllegalStateException( + String.format( + "No fields are annotated with '@%s'", annotation.getSimpleName())); + } + } +} 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 2d26925b66cf5..ada1bc58f4194 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 @@ -19,7 +19,6 @@ package org.apache.flink.connector.pulsar.testutils.runtime; import org.apache.flink.configuration.Configuration; -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.connectors.test.common.external.ExternalContext; @@ -28,28 +27,38 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.admin.PulsarAdminException.NotFoundException; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.ConsumerBuilder; +import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.TypedMessageBuilder; +import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient; +import org.apache.pulsar.client.api.transaction.TxnID; +import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import java.io.Closeable; import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.io.Serializable; +import java.time.Duration; import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Random; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.function.Supplier; import java.util.stream.Stream; import static java.util.Collections.emptyList; import static java.util.Collections.singletonList; +import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.stream.Collectors.toList; import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL; @@ -57,52 +66,108 @@ import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyAdmin; import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyThrow; +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.util.Preconditions.checkArgument; +import static org.apache.pulsar.client.api.SubscriptionInitialPosition.Earliest; +import static org.apache.pulsar.client.api.SubscriptionMode.Durable; +import static org.apache.pulsar.client.api.SubscriptionType.Exclusive; +import static org.apache.pulsar.common.partition.PartitionedTopicMetadata.NON_PARTITIONED; /** - * A pulsar cluster operator is used for operating pulsar instance. It's serializable for using in + * A pulsar cluster operator used for operating pulsar instance. It's serializable for using in * {@link ExternalContext}. */ -public class PulsarRuntimeOperator implements Serializable, Closeable { - private static final long serialVersionUID = -630646912412751301L; +public class PulsarRuntimeOperator implements Closeable { public static final int DEFAULT_PARTITIONS = 10; public static final int NUM_RECORDS_PER_PARTITION = 20; + public static final String SUBSCRIPTION_NAME = "PulsarRuntimeOperator"; - private String serviceUrl; - private String adminUrl; - - private transient PulsarClient client; - private transient PulsarAdmin admin; + private final String serviceUrl; + private final String adminUrl; + private final PulsarClient client; + private final PulsarAdmin admin; + private final ConcurrentHashMap>> producers; + private final ConcurrentHashMap>> consumers; public PulsarRuntimeOperator(String serviceUrl, String adminUrl) { - this.serviceUrl = serviceUrl; - this.adminUrl = adminUrl; - initializeClients(); + this(serviceUrl, serviceUrl, adminUrl, adminUrl); + } + + public PulsarRuntimeOperator( + String serviceUrl, + String containerServiceUrl, + String adminUrl, + String containerAdminUrl) { + this.serviceUrl = containerServiceUrl; + this.adminUrl = containerAdminUrl; + this.client = + sneakyClient( + () -> + PulsarClient.builder() + .serviceUrl(serviceUrl) + .enableTransaction(true) + .build()); + this.admin = sneakyClient(() -> PulsarAdmin.builder().serviceHttpUrl(adminUrl).build()); + this.producers = new ConcurrentHashMap<>(); + this.consumers = new ConcurrentHashMap<>(); } /** * Create a topic with default {@link #DEFAULT_PARTITIONS} partitions and send a fixed number * {@link #NUM_RECORDS_PER_PARTITION} of records to this topic. + * + * @param topic Pulsar topic name, it couldn't be a name with partition index. */ public void setupTopic(String topic) { Random random = new Random(System.currentTimeMillis()); setupTopic(topic, Schema.STRING, () -> randomAlphanumeric(10 + random.nextInt(20))); } + /** + * Create a topic with default {@link #DEFAULT_PARTITIONS} partitions and send a fixed number + * {@link #NUM_RECORDS_PER_PARTITION} of records to this topic. + * + * @param topic Pulsar topic name, it couldn't be a name with partition index. + * @param schema The Pulsar schema for serializing records into bytes. + * @param supplier The supplier for providing the records which would be sent to Pulsar. + */ public void setupTopic(String topic, Schema schema, Supplier supplier) { - createTopic(topic, DEFAULT_PARTITIONS); + setupTopic(topic, schema, supplier, NUM_RECORDS_PER_PARTITION); + } - // Make sure every topic partition has message. + /** + * Create a topic with default {@link #DEFAULT_PARTITIONS} partitions and send a fixed number of + * records to this topic. + * + * @param topic Pulsar topic name, it couldn't be a name with partition index. + * @param schema The Pulsar schema for serializing records into bytes. + * @param supplier The supplier for providing the records which would be sent to Pulsar. + * @param numRecordsPerSplit The number of records for a partition. + */ + public void setupTopic( + String topic, Schema schema, Supplier supplier, int numRecordsPerSplit) { + String topicName = topicName(topic); + createTopic(topicName, DEFAULT_PARTITIONS); + + // Make sure every topic partition has messages. for (int i = 0; i < DEFAULT_PARTITIONS; i++) { - String partitionName = TopicNameUtils.topicNameWithPartition(topic, i); + String partitionName = topicNameWithPartition(topic, i); List messages = - Stream.generate(supplier).limit(NUM_RECORDS_PER_PARTITION).collect(toList()); + Stream.generate(supplier).limit(numRecordsPerSplit).collect(toList()); sendMessages(partitionName, schema, messages); } } + /** + * Create a pulsar topic with given partition number. + * + * @param topic The name of the topic. + * @param numberOfPartitions The number of partitions. We would create a non-partitioned topic + * if this number is zero. + */ public void createTopic(String topic, int numberOfPartitions) { checkArgument(numberOfPartitions >= 0); if (numberOfPartitions == 0) { @@ -112,18 +177,53 @@ public void createTopic(String topic, int numberOfPartitions) { } } + /** + * Increase the partition number of the topic. + * + * @param topic The topic name. + * @param newPartitionsNum The new partition size which should exceed previous size. + */ public void increaseTopicPartitions(String topic, int newPartitionsNum) { + PartitionedTopicMetadata metadata = + sneakyAdmin(() -> admin().topics().getPartitionedTopicMetadata(topic)); + checkArgument( + metadata.partitions < newPartitionsNum, + "The new partition size which should greater than previous size."); + sneakyAdmin(() -> admin().topics().updatePartitionedTopic(topic, newPartitionsNum)); } - public void deleteTopic(String topic, boolean isPartitioned) { - if (isPartitioned) { - sneakyAdmin(() -> admin().topics().deletePartitionedTopic(topic)); + /** + * Delete a Pulsar topic. + * + * @param topic The topic name. + */ + public void deleteTopic(String topic) { + String topicName = topicName(topic); + PartitionedTopicMetadata metadata; + + try { + metadata = admin().topics().getPartitionedTopicMetadata(topicName); + } catch (NotFoundException e) { + // This topic doesn't exist. Just skip deletion. + return; + } catch (PulsarAdminException e) { + sneakyThrow(e); + return; + } + + // Close all the available consumers and producers. + removeConsumers(topic); + removeProducers(topic); + + if (metadata.partitions == NON_PARTITIONED) { + sneakyAdmin(() -> admin().topics().delete(topicName)); } else { - sneakyAdmin(() -> admin().topics().delete(topic)); + sneakyAdmin(() -> admin().topics().deletePartitionedTopic(topicName)); } } + /** Convert the topic metadata into a list of topic partitions. */ public List topicInfo(String topic) { try { return client().getPartitionsForTopic(topic).get().stream() @@ -139,10 +239,15 @@ public List topicInfo(String topic) { } } - protected List topicsInfo(Collection topics) { - return topics.stream().flatMap(s -> topicInfo(s).stream()).collect(toList()); - } - + /** + * Send a single message to Pulsar, return the message id after the ack from Pulsar. + * + * @param topic The name of the topic. + * @param schema The schema for serialization. + * @param message The record need to be sent. + * @param The type of the record. + * @return message id. + */ public MessageId sendMessage(String topic, Schema schema, T message) { List messageIds = sendMessages(topic, schema, singletonList(message)); checkArgument(messageIds.size() == 1); @@ -150,6 +255,16 @@ public MessageId sendMessage(String topic, Schema schema, T message) { return messageIds.get(0); } + /** + * Send a single message to Pulsar, return the message id after the ack from Pulsar. + * + * @param topic The name of the topic. + * @param schema The schema for serialization. + * @param key The message key. + * @param message The record need to be sent. + * @param The type of the record. + * @return message id. + */ public MessageId sendMessage(String topic, Schema schema, String key, T message) { List messageIds = sendMessages(topic, schema, key, singletonList(message)); checkArgument(messageIds.size() == 1); @@ -157,23 +272,42 @@ public MessageId sendMessage(String topic, Schema schema, String key, T m return messageIds.get(0); } + /** + * Send a list of messages to Pulsar, return the message id set after the ack from Pulsar. + * + * @param topic The name of the topic. + * @param schema The schema for serialization. + * @param messages The records need to be sent. + * @param The type of the record. + * @return message id. + */ public List sendMessages( String topic, Schema schema, Collection messages) { return sendMessages(topic, schema, null, messages); } + /** + * Send a list messages to Pulsar, return the message id set after the ack from Pulsar. + * + * @param topic The name of the topic. + * @param schema The schema for serialization. + * @param key The message key. + * @param messages The records need to be sent. + * @param The type of the record. + * @return message id. + */ public List sendMessages( String topic, Schema schema, String key, Collection messages) { - try (Producer producer = client().newProducer(schema).topic(topic).create()) { + try { + Producer producer = createProducer(topic, schema); List messageIds = new ArrayList<>(messages.size()); for (T message : messages) { - MessageId messageId; - if (Strings.isNullOrEmpty(key)) { - messageId = producer.newMessage().value(message).send(); - } else { - messageId = producer.newMessage().key(key).value(message).send(); + TypedMessageBuilder builder = producer.newMessage().value(message); + if (!Strings.isNullOrEmpty(key)) { + builder.key(key); } + MessageId messageId = builder.send(); messageIds.add(messageId); } @@ -184,32 +318,130 @@ public List sendMessages( } } + /** + * Consume a message from the given Pulsar topic, this method would be blocked until we get a + * message from this topic. + */ + public Message receiveMessage(String topic, Schema schema) { + try { + Consumer consumer = createConsumer(topic, schema); + return drainOneMessage(consumer); + } catch (PulsarClientException e) { + sneakyThrow(e); + return null; + } + } + + /** + * Consume a message from the given Pulsar topic, this method would be blocked until we meet + * timeout. A null message would be returned if no message has been consumed from Pulsar. + */ + public Message receiveMessage(String topic, Schema schema, Duration timeout) { + try { + Consumer consumer = createConsumer(topic, schema); + Message message = consumer.receiveAsync().get(timeout.toMillis(), MILLISECONDS); + consumer.acknowledgeCumulative(message.getMessageId()); + + return message; + } catch (Exception e) { + return null; + } + } + + /** + * Consume a fixed number of messages from the given Pulsar topic, this method would be blocked + * until we get the exactly number of messages from this topic. + */ + public List> receiveMessages(String topic, Schema schema, int counts) { + if (counts == 0) { + return emptyList(); + } else if (counts < 0) { + // Drain all messages. + return receiveAllMessages(topic, schema, Duration.ofMinutes(1)); + } else if (counts == 1) { + // Drain one message. + Message message = receiveMessage(topic, schema); + return singletonList(message); + } else { + // Drain a fixed number of messages. + try { + Consumer consumer = createConsumer(topic, schema); + List> messages = new ArrayList<>(counts); + for (int i = 0; i < counts; i++) { + Message message = drainOneMessage(consumer); + messages.add(message); + } + return messages; + } catch (PulsarClientException e) { + sneakyThrow(e); + return emptyList(); + } + } + } + + /** + * Drain all the messages from current topic. We will wait for all the messages has been + * consumed until the timeout. + */ + public List> receiveAllMessages( + String topic, Schema schema, Duration timeout) { + List> messages = new ArrayList<>(); + + Message message = receiveMessage(topic, schema, timeout); + while (message != null) { + messages.add(message); + message = receiveMessage(topic, schema, timeout); + } + + return messages; + } + + /** Return the transaction coordinator client for operating {@link TxnID}. */ + public TransactionCoordinatorClient coordinatorClient() { + return ((PulsarClientImpl) client()).getTcClient(); + } + + /** + * Return the broker url for this Pulsar runtime. It's only used in flink environment. You can't + * create the {@link PulsarClient} by this broker url, use the {@link #client()} instead. + */ public String serviceUrl() { return serviceUrl; } + /** + * Return the broker http url for this Pulsar runtime. It's only used in flink environment. You + * can't create the {@link PulsarAdmin} by this broker http url, use the {@link #admin()} + * instead. + */ public String adminUrl() { return adminUrl; } + /** The client for creating producer and consumer. It's used in tests. */ public PulsarClient client() { return client; } + /** The client for creating topics and query other metadata, etc. It's used in tests. */ public PulsarAdmin admin() { return admin; } + /** The configuration for constructing {@link Configuration}. */ public Configuration config() { Configuration configuration = new Configuration(); configuration.set(PULSAR_SERVICE_URL, serviceUrl()); configuration.set(PULSAR_ADMIN_URL, adminUrl()); - return configuration; } + /** This method is used for test framework. You can't close this operator manually. */ @Override public void close() throws IOException { + producers.clear(); + consumers.clear(); + if (admin != null) { admin.close(); } @@ -232,27 +464,83 @@ private void createNonPartitionedTopic(String topic) { private void createPartitionedTopic(String topic, int numberOfPartitions) { try { admin().lookups().lookupPartitionedTopic(topic); - sneakyAdmin(() -> admin().topics().expireMessagesForAllSubscriptionsAsync(topic, 0)); + sneakyAdmin(() -> admin().topics().expireMessagesForAllSubscriptions(topic, 0)); } catch (PulsarAdminException e) { sneakyAdmin(() -> admin().topics().createPartitionedTopic(topic, numberOfPartitions)); } } - private void initializeClients() { - this.client = sneakyClient(() -> PulsarClient.builder().serviceUrl(serviceUrl).build()); - this.admin = sneakyClient(() -> PulsarAdmin.builder().serviceHttpUrl(adminUrl).build()); + @SuppressWarnings("unchecked") + private Producer createProducer(String topic, Schema schema) + throws PulsarClientException { + TopicName topicName = TopicName.get(topic); + String name = topicName.getPartitionedTopicName(); + int index = topicName.getPartitionIndex(); + ConcurrentHashMap> topicProducers = + producers.computeIfAbsent(name, d -> new ConcurrentHashMap<>()); + + return (Producer) + topicProducers.computeIfAbsent( + index, + i -> { + ProducerBuilder builder = + client().newProducer(schema) + .topic(topic) + .enableBatching(false) + .enableMultiSchema(true); + + return sneakyClient(builder::create); + }); + } + + @SuppressWarnings("unchecked") + private Consumer createConsumer(String topic, Schema schema) + throws PulsarClientException { + TopicName topicName = TopicName.get(topic); + String name = topicName.getPartitionedTopicName(); + int index = topicName.getPartitionIndex(); + ConcurrentHashMap> topicConsumers = + consumers.computeIfAbsent(name, d -> new ConcurrentHashMap<>()); + + return (Consumer) + topicConsumers.computeIfAbsent( + index, + i -> { + ConsumerBuilder builder = + client().newConsumer(schema) + .topic(topic) + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionMode(Durable) + .subscriptionType(Exclusive) + .subscriptionInitialPosition(Earliest); + + return sneakyClient(builder::subscribe); + }); } - // --------------------------- Serialization Logic ----------------------------- + private void removeProducers(String topic) { + String topicName = topicName(topic); + ConcurrentHashMap> integerProducers = producers.remove(topicName); + if (integerProducers != null) { + for (Producer producer : integerProducers.values()) { + sneakyClient(producer::close); + } + } + } - private void writeObject(ObjectOutputStream oos) throws IOException { - oos.writeUTF(serviceUrl); - oos.writeUTF(adminUrl); + private void removeConsumers(String topic) { + String topicName = topicName(topic); + ConcurrentHashMap> integerConsumers = consumers.remove(topicName); + if (integerConsumers != null) { + for (Consumer consumer : integerConsumers.values()) { + sneakyClient(consumer::close); + } + } } - private void readObject(ObjectInputStream ois) throws ClassNotFoundException, IOException { - this.serviceUrl = ois.readUTF(); - this.adminUrl = ois.readUTF(); - initializeClients(); + private Message drainOneMessage(Consumer consumer) throws PulsarClientException { + Message message = consumer.receive(); + consumer.acknowledgeCumulative(message.getMessageId()); + return message; } } 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 b6f8aa4e769a9..bd5dd997c565d 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 @@ -21,26 +21,29 @@ import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime; import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator; -import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableSet; - -import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ClusterData; -import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import java.util.Collections; +import java.util.List; import java.util.Optional; import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; -import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyAdmin; import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.pulsar.common.naming.NamespaceName.SYSTEM_NAMESPACE; +import static org.apache.pulsar.common.naming.TopicName.TRANSACTION_COORDINATOR_ASSIGN; /** Providing a mocked pulsar server. */ public class PulsarMockRuntime implements PulsarRuntime { private static final String CLUSTER_NAME = "mock-pulsar-" + randomAlphanumeric(6); + private final ServiceConfiguration configuration; private final MockPulsarService pulsarService; private PulsarRuntimeOperator operator; @@ -49,6 +52,7 @@ public PulsarMockRuntime() { } public PulsarMockRuntime(ServiceConfiguration configuration) { + this.configuration = configuration; this.pulsarService = new MockPulsarService(configuration); } @@ -56,15 +60,15 @@ public PulsarMockRuntime(ServiceConfiguration configuration) { public void startUp() { try { pulsarService.start(); - } catch (PulsarServerException e) { + + String serviceUrl = pulsarService.getBrokerServiceUrl(); + String adminUrl = pulsarService.getWebServiceAddress(); + initializePulsarEnvironment(configuration, serviceUrl, adminUrl); + + this.operator = new PulsarRuntimeOperator(serviceUrl, adminUrl); + } catch (Exception e) { throw new IllegalStateException(e); } - this.operator = - new PulsarRuntimeOperator( - pulsarService.getBrokerServiceUrl(), pulsarService.getWebServiceAddress()); - - // Successfully start a pulsar broker, we have to create the required resources. - sneakyAdmin(this::createTestResource); } @Override @@ -83,45 +87,85 @@ public PulsarRuntimeOperator operator() { return checkNotNull(operator, "You should start this mock pulsar first."); } - private void createTestResource() throws PulsarAdminException { - PulsarAdmin admin = operator().admin(); - if (!admin.clusters().getClusters().contains(CLUSTER_NAME)) { - // Make clients can test short names - ClusterData data = - ClusterData.builder() - .serviceUrl("http://127.0.0.1:" + pulsarService.getBrokerServicePort()) - .build(); - admin.clusters().createCluster(CLUSTER_NAME, data); + /** Create the system topics. */ + private static void initializePulsarEnvironment( + ServiceConfiguration config, String serviceUrl, String adminUrl) + throws PulsarAdminException, PulsarClientException { + try (PulsarAdmin admin = PulsarAdmin.builder().serviceHttpUrl(adminUrl).build()) { + ClusterData clusterData = + ClusterData.builder().serviceUrl(adminUrl).brokerServiceUrl(serviceUrl).build(); + String cluster = config.getClusterName(); + createSampleNameSpace(admin, clusterData, cluster); + + // Create default namespace + createNameSpace( + admin, + cluster, + TopicName.PUBLIC_TENANT, + TopicName.PUBLIC_TENANT + "/" + TopicName.DEFAULT_NAMESPACE); + + // Create Pulsar system namespace + createNameSpace( + admin, cluster, SYSTEM_NAMESPACE.getTenant(), SYSTEM_NAMESPACE.toString()); + // Enable transaction + if (config.isTransactionCoordinatorEnabled() + && !admin.namespaces() + .getTopics(SYSTEM_NAMESPACE.toString()) + .contains(TRANSACTION_COORDINATOR_ASSIGN.getPartition(0).toString())) { + admin.topics().createPartitionedTopic(TRANSACTION_COORDINATOR_ASSIGN.toString(), 1); + } } - - createOrUpdateTenant("public"); - createOrUpdateNamespace("public", "default"); - - createOrUpdateTenant("pulsar"); - createOrUpdateNamespace("pulsar", "system"); } - private void createOrUpdateTenant(String tenant) throws PulsarAdminException { - PulsarAdmin admin = operator().admin(); - TenantInfo info = - TenantInfo.builder() - .adminRoles(ImmutableSet.of("appid1", "appid2")) - .allowedClusters(ImmutableSet.of(CLUSTER_NAME)) - .build(); - if (!admin.tenants().getTenants().contains(tenant)) { - admin.tenants().createTenant(tenant, info); + private static void createSampleNameSpace( + PulsarAdmin admin, ClusterData clusterData, String cluster) + throws PulsarAdminException { + // Create a sample namespace + String tenant = "sample"; + String globalCluster = "global"; + String namespace = tenant + "/ns1"; + + List clusters = admin.clusters().getClusters(); + if (!clusters.contains(cluster)) { + admin.clusters().createCluster(cluster, clusterData); } else { - admin.tenants().updateTenant(tenant, info); + admin.clusters().updateCluster(cluster, clusterData); + } + // Create marker for "global" cluster + if (!clusters.contains(globalCluster)) { + admin.clusters().createCluster(globalCluster, ClusterData.builder().build()); + } + + if (!admin.tenants().getTenants().contains(tenant)) { + admin.tenants() + .createTenant( + tenant, + new TenantInfoImpl( + Collections.emptySet(), Collections.singleton(cluster))); + } + + if (!admin.namespaces().getNamespaces(tenant).contains(namespace)) { + admin.namespaces().createNamespace(namespace); } } - public void createOrUpdateNamespace(String tenant, String namespace) + private static void createNameSpace( + PulsarAdmin admin, String cluster, String publicTenant, String defaultNamespace) throws PulsarAdminException { - PulsarAdmin admin = operator().admin(); - String namespaceValue = tenant + "/" + namespace; - if (!admin.namespaces().getNamespaces(tenant).contains(namespaceValue)) { - admin.namespaces().createNamespace(namespaceValue); - admin.namespaces().setRetention(namespaceValue, new RetentionPolicies(60, 1000)); + if (!admin.tenants().getTenants().contains(publicTenant)) { + admin.tenants() + .createTenant( + publicTenant, + TenantInfo.builder() + .adminRoles(Collections.emptySet()) + .allowedClusters(Collections.singleton(cluster)) + .build()); + } + if (!admin.namespaces().getNamespaces(publicTenant).contains(defaultNamespace)) { + admin.namespaces().createNamespace(defaultNamespace); + admin.namespaces() + .setNamespaceReplicationClusters( + defaultNamespace, Collections.singleton(cluster)); } } @@ -148,12 +192,12 @@ private static ServiceConfiguration createConfig() { configuration.setBrokerServicePort(Optional.of(0)); configuration.setWebServicePort(Optional.of(0)); - // Enable transaction with in memory. + // Enable transactions. + configuration.setSystemTopicEnabled(true); + configuration.setBrokerDeduplicationEnabled(true); configuration.setTransactionCoordinatorEnabled(true); configuration.setTransactionMetadataStoreProviderClassName( - "org.apache.pulsar.transaction.coordinator.impl.InMemTransactionMetadataStoreProvider"); - configuration.setTransactionBufferProviderClassName( - "org.apache.pulsar.broker.transaction.buffer.impl.InMemTransactionBufferProvider"); + "org.apache.pulsar.transaction.coordinator.impl.MLTransactionMetadataStoreProvider"); return configuration; } diff --git a/flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..56ad187e06b3c --- /dev/null +++ b/flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/NOTICE @@ -0,0 +1,17 @@ +flink-sql-connector-pulsar +Copyright 2014-2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- org.apache.pulsar:bouncy-castle-bc:pkg:2.9.1 +- org.apache.pulsar:pulsar-client-admin-api:2.9.1 +- org.apache.pulsar:pulsar-client-all:2.9.1 +- org.apache.pulsar:pulsar-client-api:2.9.1 +- org.bouncycastle:bcpkix-jdk15on:1.69 +- org.bouncycastle:bcprov-ext-jdk15on:1.69 +- org.bouncycastle:bcprov-jdk15on:1.69 +- org.bouncycastle:bcutil-jdk15on:1.69 +- org.slf4j:jul-to-slf4j:1.7.25 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 dddef46a4ea11..a3d7a4a59148a 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 @@ -31,7 +31,8 @@ under the License. Flink : E2E Tests : Pulsar - 2.8.0 + 2.9.1 + 1.69 @@ -105,6 +106,46 @@ under the License. jar ${project.build.directory}/dependencies + + org.apache.pulsar + bouncy-castle-bc + ${pulsar.version} + bouncy-castle-bc.jar + jar + ${project.build.directory}/dependencies + + + org.bouncycastle + bcpkix-jdk15on + ${bouncycastle.version} + bcpkix-jdk15on.jar + jar + ${project.build.directory}/dependencies + + + org.bouncycastle + bcprov-jdk15on + ${bouncycastle.version} + bcprov-jdk15on.jar + jar + ${project.build.directory}/dependencies + + + org.bouncycastle + bcutil-jdk15on + ${bouncycastle.version} + bcutil-jdk15on.jar + jar + ${project.build.directory}/dependencies + + + org.bouncycastle + bcprov-ext-jdk15on + ${bouncycastle.version} + bcprov-ext-jdk15on.jar + jar + ${project.build.directory}/dependencies + org.slf4j jul-to-slf4j 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 890d09e1834ba..930a1cb7ac591 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 @@ -36,6 +36,11 @@ public FlinkContainerWithPulsarEnvironment(int numTaskManagers, int numSlotsPerT resourcePath("pulsar-client-all.jar"), resourcePath("pulsar-client-api.jar"), resourcePath("pulsar-admin-api.jar"), + resourcePath("bouncy-castle-bc.jar"), + resourcePath("bcpkix-jdk15on.jar"), + resourcePath("bcprov-jdk15on.jar"), + resourcePath("bcutil-jdk15on.jar"), + resourcePath("bcprov-ext-jdk15on.jar"), resourcePath("jul-to-slf4j.jar")); } diff --git a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/util/DockerImageVersions.java b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/util/DockerImageVersions.java index 466d941215159..99bc5c2618636 100644 --- a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/util/DockerImageVersions.java +++ b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/util/DockerImageVersions.java @@ -36,7 +36,7 @@ public class DockerImageVersions { public static final String KINESALITE = "instructure/kinesalite:latest"; - public static final String PULSAR = "apachepulsar/pulsar:2.8.0"; + public static final String PULSAR = "apachepulsar/pulsar:2.9.1"; public static final String CASSANDRA_3 = "cassandra:3.0"; }