diff --git a/docs/zh/08-develop/07-tmq.md b/docs/zh/08-develop/07-tmq.md index 7eb8628e56..96f34c6d5d 100644 --- a/docs/zh/08-develop/07-tmq.md +++ b/docs/zh/08-develop/07-tmq.md @@ -34,13 +34,15 @@ TDengine 消费者的概念跟 Kafka 类似,消费者通过订阅主题来接 | `td.connect.user` | string | 用户名 | | | `td.connect.pass` | string | 密码 | | | `td.connect.port` | integer | 服务端的端口号 | | -| `group.id` | string | 消费组 ID,同一消费组共享消费进度 |
**必填项**。最大长度:192。
每个topic最多可建立100个 consumer group | -| `client.id` | string | 客户端 ID | 最大长度:192。 | +| `group.id` | string | 消费组 ID,同一消费组共享消费进度 |
**必填项**。最大长度:192。
每个topic最多可建立 100 个 consumer group | +| `client.id` | string | 客户端 ID | 最大长度:192 | | `auto.offset.reset` | enum | 消费组订阅的初始位置 |
`earliest`: default(version < 3.2.0.0);从头开始订阅;
`latest`: default(version >= 3.2.0.0);仅从最新数据开始订阅;
`none`: 没有提交的 offset 无法订阅 | | `enable.auto.commit` | boolean | 是否启用消费位点自动提交,true: 自动提交,客户端应用无需commit;false:客户端应用需要自行commit | 默认值为 true | | `auto.commit.interval.ms` | integer | 消费记录自动提交消费位点时间间隔,单位为毫秒 | 默认值为 5000 | | `msg.with.table.name` | boolean | 是否允许从消息中解析表名, 不适用于列订阅(列订阅时可将 tbname 作为列写入 subquery 语句)(从3.2.0.0版本该参数废弃,恒为true) | 默认关闭 | | `enable.replay` | boolean | 是否开启数据回放功能 | 默认关闭 | +| `session.timeout.ms` | integer | consumer 心跳丢失后超时时间,超时后会触发 rebalance 逻辑,成功后该 consumer 会被删除(从3.3.3.0版本开始支持) | 默认值为 12000,取值范围 [6000, 1800000] | +| `max.poll.interval.ms` | integer | consumer poll 拉取数据间隔的最长时间,超过该时间,会认为该 consumer 离线,触发rebalance 逻辑,成功后该 consumer 会被删除(从3.3.3.0版本开始支持) | 默认值为 300000,[1000,INT32_MAX] | 下面是各语言连接器创建参数: diff --git a/docs/zh/26-tdinternal/07-topic.md b/docs/zh/26-tdinternal/07-topic.md index 3fe12c943d..4299b3d503 100644 --- a/docs/zh/26-tdinternal/07-topic.md +++ b/docs/zh/26-tdinternal/07-topic.md @@ -61,37 +61,37 @@ TDengine 会为 WAL 文件自动创建索引以支持快速随机访问。通过 ![数据订阅架构图](./topicarch.png) -在客户端成功建立与服务器的连接之后,用户须首先指定消费组和主题,以创建相应的消费者实例。随后,客户端便会向服务器提交订阅请求。此刻,消费者的状态被标记为 modify,表示正处于配置阶段。消费者随后会定期向服务器发送请求,以检索并获取待消费的 vnode 列表,直至服务器为其分配相应的 vnode。一旦分配完成,消费者的状态便更新为 ready,标志着订阅流程的成功完成。此刻,客户端便可正式启动向 vnode 发送消费数据请求的过程。 +在客户端成功建立与服务器的连接之后,用户须首先指定消费组和主题,以创建相应的消费者实例。随后,客户端便会向服务器提交订阅请求。此刻,消费者的状态被标记为 rebalancing,表示正处于 rebalance 阶段。消费者随后会定期向服务器发送请求,以检索并获取待消费的 vnode 列表,直至服务器为其分配相应的 vnode。一旦分配完成,消费者的状态便更新为 ready,标志着订阅流程的成功完成。此刻,客户端便可正式启动向 vnode 发送消费数据请求的过程。 在消费数据的过程中,消费者会不断地向每个分配到的 vnode 发送请求,以尝试获取新的数据。一旦收到数据,消费者在完成消费后会继续向该 vnode 发送请求,以便持续消费。若在预设时间内未收到任何数据,消费者便会在 vnode 上注册一个消费 handle。这样一来,一旦 vnode 上有新数据产生,便会立即推送给消费者,从而确保数据消费的即时性,并有效减少消费者频繁主动拉取数据所导致的性能损耗。因此,可以看出,消费者从 vnode 获取数据的方式是一种拉取(pull)与推送(push)相结合的高效模式。 -消费者在收到数据时,会同时收到数据的版本号,并将其记录为当前在每个 vnode上的消费进度。这一进度仅在消费者内部以内存形式存储,确保仅对该消费者有效。这种设计保证了消费者在每次启动时能够从上次的消费中断处继续,避免了数据的重复处理。然而,若消费者需要退出并希望在之后恢复上次的消费进度,就必须在退出前将消费进度提交至服务器,执行所谓的 commit 操作。这一操作会将消费进度在服务器进行持久化存储,支持自动提交或手动提交两种方式。 +消费者在收到数据时,会同时收到数据的版本号,并将其记录为当前在每个 vnode 上的消费进度。这一进度仅在消费者内部以内存形式存储,确保仅对该消费者有效。这种设计保证了消费者在每次启动时能够从上次的消费中断处继续,避免了数据的重复处理。然而,若消费者需要退出并希望在之后恢复上次的消费进度,就必须在退出前将消费进度提交至服务器,执行所谓的 commit 操作。这一操作会将消费进度在服务器进行持久化存储,支持自动提交或手动提交两种方式。 -此外,为了维持消费者的活跃状态,客户端还实施了心跳保活机制。通过定期向服务器发送心跳信号,消费者能够向服务器证明自己仍然在线。若服务器在一定时间内未收到消费者的心跳,便会将其标记为 lost 状态,即认为消费者已离线。服务器依赖心跳机制来监控所有消费者的状态,进而有效地管理整个消费者群体。 +此外,为了维持消费者的活跃状态,客户端还实施了心跳保活机制。通过定期向服务器发送心跳信号,消费者能够向服务器证明自己仍然在线。若服务器在一定时间内未收到消费者的心跳,便会认为消费者已离线。对于一定时间(可以通过参数控制)不拉取数据的 consumer,服务器也会标记为离线,并从消费组中删除该消费者。服务器依赖心跳机制来监控所有消费者的状态,进而有效地管理整个消费者群体。 -mnode 主要负责处理订阅过程中的控制消息,包括创建和删除主题、订阅消息、查询 endpoint 消息以及心跳消息等。vnode 则专注于处理消费消息和 commit 消息。当mnode 收到消费者的订阅消息时,如果该消费者尚未订阅过,其状态将被设置为 modify。如果消费者已经订阅过,但订阅的主题发生变更,消费者的状态同样会被设置为 modify。等待再平衡的计时器到来时,mnode 会对 modify 状态的消费者执行再平衡操作,将心跳超过固定时间的消费者设置为 lost 状态,并对关闭或 lost 状态超过一定时间的消费者进行删除操作。 +mnode 主要负责处理订阅过程中的控制消息,包括创建和删除主题、订阅消息、查询 endpoint 消息以及心跳消息等。vnode 则专注于处理消费消息和 commit 消息。当 mnode 收到消费者的订阅消息时,如果该消费者尚未订阅过,其状态将被设置为 rebalancing。如果消费者已经订阅过,但订阅的主题发生变更,消费者的状态同样会被设置为 rebalancing。然后 mnode 会对 rebalancing 状态的消费者执行 rebalance 操作。心跳超过固定时间的消费者或主动关闭的消费者将被删除。 消费者会定期向 mnode 发送查询 endpoint 消息,以获取再平衡后的最新 vnode 分配结果。同时,消费者还会定期发送心跳消息,通知 mnode 自身处于活跃状态。此外,消费者的一些信息也会通过心跳消息上报至 mnode,用户可以查询 mnode 上的这些信息以监测各个消费者的状态。这种设计有助于实现对消费者的有效管理和监控。 -## 再平衡过程 +## rebalance 过程 -每个主题的数据可能分散在多个 vnode 上。服务器通过执行再平衡过程,将这些vnode 合理地分配给各个消费者,确保数据的均匀分布和高效消费。 +每个主题的数据可能分散在多个 vnode 上。服务器通过执行 rebalance 过程,将这些 vnode 合理地分配给各个消费者,确保数据的均匀分布和高效消费。 -如下图所示,c1 表示消费者 1,c2 表示消费者 2,g1 表示消费组 1。起初 g1 中只有 c1 消费数据,c1 发送订阅信息到 mnode,mnode 把数据所在的所有 4 个vnode 分配给 c1。当 c2 增加到 g1 后,c2 将订阅信息发送给 mnode,mnode 将在下一次再平衡计时器到来时检测到这个 g1 需要重新分配,就会启动再平衡过程,随后 c2 分得其中两个 vnode 消费。分配信息还会通过 mnode 发送给 vnode,同时 c1 和 c2 也会获取自己消费的 vnode 信息并开始消费。 +如下图所示,c1 表示消费者 1,c2 表示消费者 2,g1 表示消费组 1。起初 g1 中只有 c1 消费数据,c1 发送订阅信息到 mnode,mnode 把数据所在的所有 4 个vnode 分配给 c1。当 c2 增加到 g1 后,c2 将订阅信息发送给 mnode,mnode 检测到这个 g1 需要重新分配,就会启动 rebalance 过程,随后 c2 分得其中两个 vnode 消费。分配信息还会通过 mnode 发送给 vnode,同时 c1 和 c2 也会获取自己消费的 vnode 信息并开始消费。 -![再平衡过程](./rebalance.png) +![rebalance 过程](./rebalance.png) 再平衡计时器每 2s 检测一次是否需要再平衡。在再平衡过程中,如果消费者获取的状态是 not ready,则不能进行消费。只有再平衡正常结束后,消费者获取分配 vnode 的offset 后才可正常消费,否则消费者会重试指定次数后报错。 ## 消费者状态处理 -消费者的状态转换过程如下图所示。初始状态下,刚发起订阅的消费者处于modify 状态,此时客户端获取的消费者状态为 not ready,表明消费者尚未准备好进行数据消费。一旦 mnode 在再平衡计时器中检测到处于 modify 状态的消费者,便会启动再平衡过程。再平衡成功后,消费者的状态将转变为 ready,表示消费者已准备就绪。随后,当消费者通过定时的查询 endpoint 消息获取自身的 ready 状态以及分配的 vnode 列表后,便能正式开始消费数据。 +消费者的状态转换过程如下图所示。初始状态下,刚发起订阅的消费者处于 rebalancing 状态,表明消费者尚未准备好进行数据消费。一旦 mnode 检测到处于 rebalancing 状态的消费者,便会启动 rebalance 过程,成功后,消费者的状态将转变为 ready,表示消费者已准备就绪。随后,当消费者通过定时的查询 endpoint 消息获取自身的 ready 状态以及分配的 vnode 列表后,便能正式开始消费数据。 ![消费者状态转换](./statetransition.png) -若消费者的心跳丢失时间超过 12s,经过再平衡过程,其状态将被更新为 lost,表明消费者被认为已离线。如果心跳丢失时间超过 1 天,消费者的状态将进一步转变为 clear,此时消费者将被系统删除。然而,如果在上述过程中消费者重新发送心跳信号,其状态将恢复为 modify,并重新进入下一轮的再平衡过程。 +若消费者的心跳丢失时间超过 12s,经过 rebalance 过程,其状态将被更新为 clear,然后消费者将被系统删除。 -当消费者主动退出时,会发送 unsubscribe 消息。该消息将清除消费者订阅的所有主题,并将消费者的状态设置为 modify。随后,在再平衡过程中,一旦消费者的状态变为ready 且无订阅的主题,mnode 将在再平衡的计时器中检测到此状态变化,并据此删除该消费者。这一系列措施确保了消费者退出的有序性和系统的稳定性。 +当消费者主动退出时,会发送 unsubscribe 消息。该消息将清除消费者订阅的所有主题,并将消费者的状态设置为 rebalancing。随后,检测到处于 rebalancing 状态的消费者,便会启动 rebalance 过程,成功后,其状态将被更新为 clear,然后消费者将被系统删除。这一系列措施确保了消费者退出的有序性和系统的稳定性。 ## 消费数据 diff --git a/docs/zh/26-tdinternal/consumer.png b/docs/zh/26-tdinternal/consumer.png index cc996a431a..0c9a061107 100644 Binary files a/docs/zh/26-tdinternal/consumer.png and b/docs/zh/26-tdinternal/consumer.png differ diff --git a/docs/zh/26-tdinternal/offset.png b/docs/zh/26-tdinternal/offset.png index d8146b8d50..0ddb005d66 100644 Binary files a/docs/zh/26-tdinternal/offset.png and b/docs/zh/26-tdinternal/offset.png differ diff --git a/docs/zh/26-tdinternal/rebalance.png b/docs/zh/26-tdinternal/rebalance.png index 216ba5ec3d..fbc2147c91 100644 Binary files a/docs/zh/26-tdinternal/rebalance.png and b/docs/zh/26-tdinternal/rebalance.png differ diff --git a/docs/zh/26-tdinternal/statetransition.png b/docs/zh/26-tdinternal/statetransition.png index 1ec8ae22f7..dd2b472890 100644 Binary files a/docs/zh/26-tdinternal/statetransition.png and b/docs/zh/26-tdinternal/statetransition.png differ diff --git a/docs/zh/26-tdinternal/topic.png b/docs/zh/26-tdinternal/topic.png index 82f3df6b73..d9306cdb74 100644 Binary files a/docs/zh/26-tdinternal/topic.png and b/docs/zh/26-tdinternal/topic.png differ diff --git a/docs/zh/26-tdinternal/topicarch.png b/docs/zh/26-tdinternal/topicarch.png index 2442158ebb..1bb4bb3014 100644 Binary files a/docs/zh/26-tdinternal/topicarch.png and b/docs/zh/26-tdinternal/topicarch.png differ