导语

Apache Pulsar 是一个多租户、高性能的服务间消息传输解决方案,支持多租户、低延时、读写分离、跨地域复制(GEO replication)、快速扩容、灵活容错等特性。在很多场景下,用户需要使用到延迟消息,本文是 Pulsar 技术系列中的一篇,主要介绍 Pulsar 3.x 大规模延迟消息投递的实现。

背景

之前有文章介绍过延迟消息的使用场景、使用方式以及实现原理,同时也提出了当时版本的局限性,完全基于内存构建延迟消息索引,导致无法支持大规模延迟消息场景。Pulsar 在 3.x 版本支持了基于磁盘的延迟消息索引方案,使得内存不再是延迟消息规模的瓶颈。

大规模延迟消息方案

Pulsar 消费流程

在讲解延迟消息之前,先简单描述一下 Pulsar 服务端消费处理流程。Broker 为每个订阅维护单独的 Dispatcher 对象,Dispatcher 负责管理整个订阅的消费。大致流程如下:

  1. Dispatcher 启动后初始化 Markdelete(已 Ack 最小位置)和 ReadPosition(当前读位置)(起始时 ReadPosition 为 MarkDelete 的下一个位置)。
  2. 如果 RedeliveryTracker 中有数据,优先推送 RedeliveryTracker 中数据, 否则从 Bookie 中读取数据。
  3. 普通数据: 推送数据给合适的 Consumer,Consumer 未就绪时,拉取到的数据把索引信息存储到 RedeliveryTracker 中。
  4. 延迟消息: 如果时间到了,直接推送给用户,否则添加到 DelayMesageTracker 里。
  5. DelayMesageTracker 会定时把到期数据倒入 RedeliveryTracker。

可以简单理解为,Dispatcher 持续往后读取数据,已过期延迟消息就和普通消息一样推送给客户端。其余延迟消息会被添加到 DelayedDeliveryTracker,数据到期后再推送给客户端。

DelayedDeliveryTracker

延迟消息是由 DelayedDeliveryTracker 管理,从上图中可以看到,DelayedDeliveryTracker 主要功能为添加消息以及读取已经到期的消息,代码片段如下:

public interface DelayedDeliveryTracker extends AutoCloseable {
    # 添加消息
    boolean addMessage(long ledgerId, long entryId, long deliveryAt);
    # 获取到期消息
    NavigableSet<PositionImpl> getScheduledMessages(int maxMessages);
    # 其余为辅助方法
    boolean hasMessageAvailable();
    long getNumberOfDelayedMessages();
    long getBufferMemoryUsage();
    boolean shouldPauseAllDeliveries();
    void resetTickTime(long tickTime);
    CompletableFuture<Void> clear();
    void close();}

可以看到,DeliveryTracker 只添加 Position 信息,Data 信息是不需要保存的。目前具体的实现 Tracker 有两个,InMemoryDelayedDeliveryTracker 和 BucketDelayedDeliveryTracker,前者为完全基于内存的老版本实现,后者为基于 BK 存储支持超大规模延迟消息的实现,下文主要分析 BucketDelayedDeliveryTracker 的实现原理。

BucketDelayedDeliveryTracker

如何存储

Broker 设计为无状态服务,所以基于磁盘的 DelayedDeliveryTracker 是 Bookie 来负责。BucketDelayedDeliveryTracker 为了降低磁盘的写入次数(写入 Bookie),会积累到一定量延迟消息索引后再触发写入。可以看到 Bucket 生命周期和 Ledgers 是很类似的,只有最后一个 Bucket 支持写入(LastMutableBucket),前面的 Bucket 只支持读取(ImmutableBuckets),读取完成后会删除。由于 Ledger 是递增的,所以可以看到每个 Bucket 中 Ledger 也是递增的,每个 Bucket 负责存储一定范围的 Ledger 延迟消息。

数据加入 DeliveryTracker 具体流程如下:

  1. 判断是否存在,如果存在了,直接返回(Bucket 内的 Metadata 有 Bitmap 快速标识哪些 Entry 是已存在的)。
  2. 如果需要加入的消息 LedgerId 不包含于 LastMutableBucket 中的话,说明之前的数据没有存储到(不应该出现)。直接放到 Share 池子里面。
  3. 其余正常情况会直接存储在 LastMutableBucket 中,LastMutableBucket 中的延迟消息累计到一定数量后会生成新的 ImmutableBucket 并刷入磁盘。
  4. 刷盘成功后,会清空 LastMutableBucket 数据重新接收新的延迟消息写入。

当 LastMutableBucket 累计到一定量的延迟消息后(默认 5w,会存储完最后一个 Ledger 的全部延迟消息再切换),会触发刷盘, 具体步骤如下:

  1. LastMutableBucket 内存中的延迟消息,按照时间(5min)、条数(5000)维度分成多个 Segment(简单理解第一个 Segment 是最新5分钟的消息,第二个是最近5~10分钟消息)。
  2. 生成好每个 Segement 的元数据(包含每个 Segement 中消息的最大和最小时间,以及用来快速判断某个消息是否是延迟消息的 Bitmap)。
  3. 创建一个新的 Ledger,并把元数据存储在对应 Cursor Property 中。格式为 <#pulsar.internal.delayed.bucket-${startLedgerId}-${endLedgerId},${bucketLedgerId>。
  4. 将元数据写入 Entry0 位置,其余 Segment 分别作为一个 Entry 顺序写入,并关闭 Ledger。

可以看到,整个 Bucket 通过几次 Append 写入就能把全部的延迟信息落盘,已写入的数据不能修改,只能删除。

如何读取

BucketDelayedDeliveryTracker 在 Addmsg 时已经把延迟消息写入到磁盘,内存中只会存储部分的延迟消息。在 BucketDelayedDeliveryTracker 中内存中的延迟消息都存储在 SharedBucketPriorityQueue(小堆实现的优先级队列)中。所有的延迟消息都通过 SharedBucketPriorityQueue 来获取。

读取相对来说比较简单, 大致流程如下:

  1. 每次拉取延迟消息时会先把 LastMutableBucket 中到期的延迟消息转移到 SharedBucketPriorityQueue 中。
  2. 在每个 ImmutableBucket 刷磁盘时,已经把第一个Segment 加载到 SharedBucketPriorityQueue 中。
  3. 当 ImmutableBucket 中的当前 Segement 中的最后一条消息被获取后,会触发从 Bookie 中加载下一个 Segement 存入 SharedBucketPriorityQueue。

何时删除

ImmutableBucket 被使用完后,在以下几个时机会被删除。

  1. Bucket 中的最后一个 Segement 的最后一条数据被读取后(实现上为加载下一个 Segment 发现未空时)。
  2. 订阅重新加载时(分区 Leadership 发生变化)时,如果 Bucket 中的延迟消息都已到期。
  3. Bucket 触发 Merger 后(可控制内存中的 Bucket 个数进而控制内存消耗),会删除原先的 Bucket。用户设置了 MaxNumBuckets,已存在 Bucket 个数大于这个配置值时,挑选若干个 Bucket 合并成一个 Bucket,并删除掉原有的 Bucket。
  4. 用户调用 

    org.apache.pulsar.client.admin.Topics#skipAllMessages。

  5. ImmutableBucket 删除时会先清空 ZK 中的元数据,再删除对应的 Ledger。

Bucket 数据丢失

如果延迟消息在 LastMutableBucket 中还没有刷盘到 Bookie,此时发生故障,LastMutableBucket 中的数据将会丢失。但是这并不会有什么影响,在重新启动后,依然会从 MackDelete 位置往后读取消息,重建 Bucket。这也就是为什么 Bucket 中的数据只需要被读走(不需要客户端 Ack)就可以被删除。

重建 Bucket

重建 Bucket 其余与第一次构建 Bucket 是一致的,都是往后读取消息,未过期的延迟消息重新加入 Bucket 中,已过期和延迟消息会被当做普通消息直接推送给客户端。重建 Bucket 过程资源消耗可控,也不会阻塞消费,暂时不必担心重建 Bucket 对消费造成额外压力。

总结

Pulsar 3.x 版本大规模延迟消息方案整体比较简单,采用先分桶,再分段的策略,只在内存中保存最近的延迟消息,延迟消息规模将不再受到内存的限制。新版本实际压测下来与设计基本一致,内存占用达到稳定后将不会上涨,目前验证单节点数十亿延迟消息稳定运行。

未来规划

当前的实现版本 DelayedDeliveryTracker 是基于订阅维度,如果 Topic 下有很多订阅,占用的内存和磁盘存储会随着订阅数量等比例放大。目前这里有一定的优化空间,可以 Topic 下的多订阅共享 DeliveryTracker 存储,类似每个订阅有单独的 ReadPosition 即可。


腾讯云中间件
0 声望7 粉丝

关注云原生,分享腾讯云中间件技术