From be883ec5cc87a2263409b16e765221fd550c8113 Mon Sep 17 00:00:00 2001 From: aptend Date: Sat, 28 Mar 2026 22:12:33 +0800 Subject: [PATCH 01/11] base --- docs/design/lazy_catalog_load_for_cn.flow.md | 224 ++++ docs/design/lazy_catalog_load_for_cn.plan.md | 372 ++++++ pkg/catalog/lazy_catalog.go | 158 +++ pkg/catalog/types.go | 45 + pkg/catalog/types_test.go | 49 + pkg/cnservice/distributed_tae.go | 1 + pkg/frontend/session.go | 8 +- pkg/pb/api/api.pb.go | 543 +++++---- pkg/pb/logtail/logtail.pb.go | 1000 +++++++++++++++-- .../issues/lazy_catalog_reconnect_test.go | 279 +++++ pkg/util/status/server.go | 304 ++++- pkg/util/status/server_test.go | 133 +++ pkg/vm/engine/disttae/cache/debug.go | 171 +++ pkg/vm/engine/disttae/cache/debug_test.go | 123 ++ pkg/vm/engine/disttae/debug_state.go | 526 +++++++++ pkg/vm/engine/disttae/debug_state_test.go | 135 +++ pkg/vm/engine/disttae/engine.go | 11 +- pkg/vm/engine/disttae/lazy_catalog_cn.go | 361 ++++++ pkg/vm/engine/disttae/lazy_catalog_cn_test.go | 420 +++++++ pkg/vm/engine/disttae/logtail.go | 19 +- pkg/vm/engine/disttae/logtail_consumer.go | 489 +++++++- .../engine/disttae/logtail_consumer_test.go | 18 + pkg/vm/engine/disttae/logtailreplay/debug.go | 53 + pkg/vm/engine/disttae/mo_table_stats.go | 10 +- pkg/vm/engine/disttae/stats.go | 24 + pkg/vm/engine/disttae/txn_database.go | 3 +- pkg/vm/engine/entire_engine.go | 7 + pkg/vm/engine/entire_engine_test.go | 16 +- .../tae/logtail/service/catalog_filter.go | 265 +++++ .../logtail/service/catalog_filter_test.go | 300 +++++ pkg/vm/engine/tae/logtail/service/client.go | 90 +- .../engine/tae/logtail/service/client_test.go | 160 +++ .../logtail/service/lazy_catalog_session.go | 255 +++++ .../service/lazy_catalog_session_test.go | 110 ++ pkg/vm/engine/tae/logtail/service/server.go | 401 ++++++- .../engine/tae/logtail/service/server_test.go | 191 ++++ pkg/vm/engine/tae/logtail/service/session.go | 29 + .../tae/logtail/service/session_test.go | 22 +- pkg/vm/engine/tae/logtail/txn_handle.go | 99 ++ pkg/vm/engine/tae/logtail/txn_handle_test.go | 51 + pkg/vm/engine/types.go | 8 + proto/api.proto | 4 + proto/logtail.proto | 28 +- 43 files changed, 7054 insertions(+), 461 deletions(-) create mode 100644 docs/design/lazy_catalog_load_for_cn.flow.md create mode 100644 docs/design/lazy_catalog_load_for_cn.plan.md create mode 100644 pkg/catalog/lazy_catalog.go create mode 100644 pkg/catalog/types_test.go create mode 100644 pkg/tests/issues/lazy_catalog_reconnect_test.go create mode 100644 pkg/util/status/server_test.go create mode 100644 pkg/vm/engine/disttae/cache/debug.go create mode 100644 pkg/vm/engine/disttae/cache/debug_test.go create mode 100644 pkg/vm/engine/disttae/debug_state.go create mode 100644 pkg/vm/engine/disttae/debug_state_test.go create mode 100644 pkg/vm/engine/disttae/lazy_catalog_cn.go create mode 100644 pkg/vm/engine/disttae/lazy_catalog_cn_test.go create mode 100644 pkg/vm/engine/disttae/logtailreplay/debug.go create mode 100644 pkg/vm/engine/tae/logtail/service/catalog_filter.go create mode 100644 pkg/vm/engine/tae/logtail/service/catalog_filter_test.go create mode 100644 pkg/vm/engine/tae/logtail/service/client_test.go create mode 100644 pkg/vm/engine/tae/logtail/service/lazy_catalog_session.go create mode 100644 pkg/vm/engine/tae/logtail/service/lazy_catalog_session_test.go create mode 100644 pkg/vm/engine/tae/logtail/txn_handle_test.go diff --git a/docs/design/lazy_catalog_load_for_cn.flow.md b/docs/design/lazy_catalog_load_for_cn.flow.md new file mode 100644 index 0000000000000..3a29e6eeccd98 --- /dev/null +++ b/docs/design/lazy_catalog_load_for_cn.flow.md @@ -0,0 +1,224 @@ +# Lazy Catalog Flow(coding-ready) + +## 关键规则 + +1. **lazy subscribe / activation 不再透传 raw checkpoint** + - 三张系统表的 `CkpLocation` 不是 account-filtered; + - 所以 startup / reconnect subscribe 和 activation response 只转发过滤后的 row-level data,不直接把 raw checkpoint 送到 CN。 + +2. **共享 `PartitionState` 只保存全局基线** + - object / metadata 持续全局推进; + - 只有 row-level in-memory delta 按 account 过滤。 + +3. **account 的 ready 点是 `replayTS`,不是 `targetTS`** + - `targetTS` 是 TN 返回的 barrier; + - `replayTS` 是 CN 在 barrier 真正落地后拿到的确定 snapshot ts。 + +4. **activation 期间普通 push 仍然推进 `PartitionState`** + - 只是 cache 侧不能直接对外可见; + - cache apply 要先进入 `accountDCA[account]`,full replay 后再 flush。 + +5. **row delete 也必须按 account 过滤** + - delete 不看 delete batch 里的 `account_id`; + - 统一按 `cpkey` 解出的 account 判断。 + +6. **push 与 pull 的 account 粒度不一样** + - steady-state push 的三表 in-memory entry 视为只属于一个 account; + - 所以 TN 的 publish 过滤和 CN 的 `accountDCA` 路由都按整条 entry keep/drop,不做 row-splitting; + - 但 subscribe / activation pull 回来的 entry 仍可能混合多个 account,TN 需要在发送前复制出目标行。 + +--- + +## Flow 1:首次 startup + +### CN 发起 + +1. 建立 logtail stream。 +2. `subSysTables()` 对三张系统表发送 subscribe request: + +```text +lazy_catalog = true +initial_active_accounts = [0] +``` + +### TN 返回 + +3. startup subscribe response 返回: + - object / metadata; + - 过滤后的 sys account row-level in-memory delta; + - 不转发 raw `CkpLocation`。 + +### CN 落地 + +4. 把 subscribe response 应用到共享 `PartitionState`。 +5. 继续走当前 startup 骨架: + - `waitTimestamp()` + - `replayCatalogCache()` +6. startup replay 只加载 sys account。 +7. replay 期间普通 cache apply 继续走现有 global DCA 语义。 + - global DCA 只负责 startup / reconnect 的共享基线; + - `accountDCA` 只负责运行中单 account activation,不会阻塞其他 ready account 的正常 cache apply。 +8. replay 完成后: + - flush startup DCA; + - `sys.readyTS = startupReplayTS`; + - `sys.state = ready`。 + +最终结果: + +- `PartitionState`:包含全局 object / metadata,以及 sys 已放行的 row-level delta; +- `CatalogCache`:只有 sys account。 + +说明:startup 仍沿用现有 `waitTimestamp()` / `replayCatalogCache()` 路径,不新增单独的 startup `targetTS`。 + +--- + +## Flow 2:运行中新增 account X + +### A. 触发 activation + +1. `Session.AuthenticateUser()` 完成租户识别。 +2. 在 `tenant.SetTenantID(X)` 之后、第一条 tenant SQL 之前调用: + +```go +ActivateTenantCatalog(ctx, X) +``` + +### B. TN 执行 catch-up + +3. 若 X 已 ready,CN 直接返回。 +4. 否则 CN: + - `singleflight(key = X)`; + - `state[X] = catching_up`; + - 初始化 `accountDCA[X]`; + - 分配本地 `seq`; + - 发送 activation request。 + +5. TN 收到 request 后: + - 记录 `activatingAccounts[X] = seq`; + - phase1 worker 从 `0` 开始按表拉历史 row-level delta,实际下界由 checkpoint 逻辑处理; + - 三张表 catch-up 可以并发执行,直到各自补到 `phase1To`; + - `logtailSender()` 串行完成 phase2,并取 `targetTS`; + - response 成功进入当前 session FIFO `sendChan` 后,才把 X 加入 `activeCatalogAccounts`。 + +这里锁死的边界是: + +- `targetTS` 之前的数据属于 activation response; +- `targetTS` 之后的数据属于 steady-state push。 + +### C. CN 先补 `PartitionState` + +6. CN 收到 `ActivateAccountForCatalogResponse{account_id, seq, target_ts, tails}`。 +7. 用 `(X, seq)` 命中当前 pending activation。 +8. 先把 `tails` 应用到共享 `PartitionState`。 + +这里不要求 `tails` 固定顺序;CN 必须按表身份消费,而不是按位置消费。 + +此时语义: + +- X 在 `targetTS` 之前缺失的 row-level delta 已补齐; +- X 仍然不能 serve cache。 + +### D. CN 取确定的 `replayTS` + +9. 执行: + +```go +replayTS = WaitLogTailAppliedAt(targetTS) +``` + +按当前 waiter 语义: + +- 会先等到 `latestApplied >= targetTS`; +- 返回值通常是 `latestApplied.Next()`; +- 所以 `replayTS` 一般严格大于 `targetTS`。 + +### E. activation 期间普通 push 的处理 + +10. 如果 X 的实时 update push 在 replay 完成前到达: + - `PartitionState` 正常更新; + - cache 不直接 apply; + - 写入 `accountDCA[X]`。 + +这里依赖的是 push-entry 单 account 假设:CN 只对运行中收到的 pushed entry 按 entry 的 account 决定是立刻 apply 还是进入 `accountDCA[X]`。 + +### F. full replay + ready + +11. 在 `catalogCacheMu` 下执行: + +```go +replayCatalogCacheForAccount(ctx, e, X, replayTS) +``` + +12. full replay 完成后,在同一个 `catalogCacheMu` 临界区里: + - drain `accountDCA[X]`; + - `X.readyTS = replayTS`; + - `state[X] = ready`。 + +13. `ActivateTenantCatalog(ctx, X)` 返回。 +14. auth 继续执行 tenant SQL。 + +--- + +## Flow 3:X 激活后的 steady state + +1. TN `publishEvent()`: + - object / metadata 继续全局推进; + - row-level delta 只对 `sys ∪ activeCatalogAccounts` 放行。 + +2. CN 对 ready 的 X: + - `PartitionState` 正常更新; + - cache 正常更新。 + +3. `CanServeAccount(X, ts)` 必须同时满足: + - 全局 `CatalogCache.CanServe(ts)` 为 true; + - `ts >= X.readyTS`。 + +--- + +## Flow 4:reconnect + +1. CN 保存断线前已 ready 的账户集合: + +```text +wantedAccounts +``` + +2. reconnect 后第一次 `subSysTables()` 直接发送: + +```text +initial_active_accounts = wantedAccounts ∪ {0} +``` + +3. TN 返回: + - object / metadata; + - sys + wanted accounts 的 row-level in-memory delta; + - 不转发 raw `CkpLocation`。 + +4. CN 继续走 startup 骨架: + - `waitTimestamp()` + - 拿到 `reconnectReplayTS` + - replay sys + wanted accounts + +5. replay 完成后,对 `wantedAccounts ∪ {0}` 统一设置: + - `readyTS = reconnectReplayTS` + - `state = ready` + +这里的建模是: + +> reconnect 是“带着 wanted accounts 的重新 startup”,不是“运行中新增一个 account”。 + +--- + +## 一句话版 + +### startup + +`PS 先到位 -> startupReplayTS -> sys full replay -> sys ready` + +### 新增 account + +`PS 先补到 targetTS -> WaitLogTailAppliedAt 得到 replayTS -> replayTS 上 full replay -> flush accountDCA -> account ready` + +### reconnect + +`wanted accounts 一次性带给 TN -> reconnectReplayTS -> sys + wanted 全量 replay -> 批量 ready` diff --git a/docs/design/lazy_catalog_load_for_cn.plan.md b/docs/design/lazy_catalog_load_for_cn.plan.md new file mode 100644 index 0000000000000..e71d549ca3a30 --- /dev/null +++ b/docs/design/lazy_catalog_load_for_cn.plan.md @@ -0,0 +1,372 @@ +--- +name: Lazy Catalog Load for CN +overview: "Coding-facing design reference for lazy catalog load: keep shared catalog PartitionState semantics unchanged, add account-scoped cache readiness, and make runtime activation follow the same PS-first then exact-replay model as startup." +todos: + - id: proto-contract + content: "proto/logtail.proto + pkg/pb/logtail: add `SubscribeRequest.lazy_catalog`, `initial_active_accounts`, and dedicated `ActivateAccountForCatalogRequest/Response{account_id, seq, target_ts, tails}`; keep request_id/response_id as stream ids and use `seq` for activation correlation" + status: done + - id: stream-plumbing + content: "pkg/vm/engine/tae/logtail/service/{client.go,server.go,request.go,response.go}: plumb activation messages over the existing segmented logtail stream and dispatch them separately from subscribe/update" + status: done + - id: tn-session-filter + content: "pkg/vm/engine/tae/logtail/service/session.go + helpers: track lazyCatalogMode, activeCatalogAccounts, activatingAccounts, and reuse one catalog entry-account filter helper for subscribe/publish/activation (delete still decodes account from cpkey)" + status: done + - id: cn-replay-refactor + content: "pkg/catalog/types.go + pkg/vm/engine/disttae/logtail_consumer.go: split `replayCatalogCache()` into reusable startup/reconnect/account replay helpers, and add account-filtered SQL templates so replay no longer assumes a global all-account load" + status: done + - id: startup-sys-baseline + content: "pkg/vm/engine/disttae/logtail_consumer.go: make `subSysTables()` send `lazy_catalog=true` with `initial_active_accounts=[0]`, and change startup replay/cache ready so only sys account becomes ready after startup" + status: done + - id: tn-activation-sender + content: "pkg/vm/engine/tae/logtail/service/server.go: implement activation phase1 worker plus sender-serialized phase2/targetTS/response enqueue, and only add an account to `activeCatalogAccounts` after the activation response enters `sendChan`" + status: done + - id: cn-account-state + content: "pkg/vm/engine/disttae/{logtail_consumer.go,engine.go,types.go}: add pending(account, seq), inactive/catching_up/ready, per-account readyTS, per-account accountDCA, and wantedAccounts that survives reconnect" + status: done + - id: cn-activation-flow + content: "pkg/vm/engine/disttae/logtail_consumer.go + pkg/vm/engine/tae/logtail/service/client.go: send activation, apply activation tails to shared PartitionState, call `WaitLogTailAppliedAt(targetTS)`, replay one account at replayTS, flush accountDCA, and remember the account in wantedAccounts" + status: done + - id: serve-gate-audit + content: "pkg/vm/engine/disttae/{engine.go,txn_database.go,txn.go,mo_table_stats.go} + cache/catalog.go: replace global-only `CatalogCache.CanServe(ts)` decisions with account-aware readyTS checks and storage fallback" + status: done + - id: reconnect-restore + content: "pkg/vm/engine/disttae/logtail_consumer.go: preserve wantedAccounts across reconnect, send them in the first system subscribe, replay sys + wanted accounts at one reconnectReplayTS, and batch-mark the whole set ready" + status: done + - id: frontend-auth-hook + content: "pkg/frontend/session.go: trigger activation immediately after `tenant.SetTenantID(...)` and before the first tenant-context SQL in `Session.AuthenticateUser`; return activation errors directly" + status: done + - id: tests-audit + content: "Add coverage for startup sys-only baseline, activation seq/ordering, global/per-account delayed cache apply, subscribe-time mixed-entry row copying, readyTS serve gate, reconnect batch restore, auth-path failure handling, and lazy-off compatibility" + status: done +isProject: false +--- + +# Lazy Catalog Load for CN(主设计文档) + +> 这份文档只保留做代码前必须锁死的设计约束、协议语义、状态机和落点。更细的执行时序见 `lazy_flow.md`。 + +## 1. 目标与边界 + +### 目标 + +1. **startup** 保持现有系统表启动骨架,但 `CatalogCache` 在 startup 阶段只加载 sys account。 +2. **运行中新增 account** 复用 startup 语义:先把数据补进共享 `PartitionState`,再在一个确定的 snapshot ts 上做 full replay,最后再开放 cache serve。 +3. **reconnect** 按“批量恢复已 ready account”处理,而不是逐个 post-reconnect activation。 + +### 非目标 + +- 不优化 system-table checkpoint 内存。 +- 不把 checkpoint 改成按账户懒化。 +- 不把 catalog `PartitionState` 改成 per-account 隔离。 +- 不改普通用户表的订阅 / 推送 / 反订阅流程;lazy catalog 只针对 `mo_database` / `mo_tables` / `mo_columns`。 + +一句话原则: + +> **共享 `PartitionState` 保留全局基线;account 是否可见,由 per-account replay 和 per-account `readyTS` 决定。** + +--- + +## 2. 必须保留的基线语义 + +当前 startup 的真实语义是: + +```text +PS first -> exact replay ts -> full replay -> flush delayed cache apply -> ready +``` + +本轮实现只是在这个基线上扩展: + +- startup 继续走现有 `waitTimestamp()` / `replayCatalogCache()` 骨架; +- runtime activation 改成显式拿 `targetTS`,再通过 `WaitLogTailAppliedAt(targetTS)` 取得确定的 `replayTS`; +- reconnect 沿 startup 骨架一次性恢复 sys + wanted accounts。 + +--- + +## 3. 核心不变量 + +1. **lazy subscribe / activation 不直接透传 raw checkpoint** + - 三张系统表的 `CkpLocation` 不是按 account 过滤的; + - startup / reconnect subscribe 和 activation response 都只转发过滤后的 row-level data,不把 raw checkpoint 直接送到 CN。 +2. **account 只有在 full replay 完成后才能 serve** + - activation response 只负责把缺失 row-level delta 补进 `PartitionState`; + - account 真正 ready 的时点是 full replay 完成后的 `readyTS`。 +3. `**targetTS` 必须在 TN sender 路径上确定** + - 只有这样,`targetTS` 才是和 steady-state push 无 gap 的真实边界。 +4. **account 变 active 必须晚于 activation response 成功入 FIFO send queue** + - 先 response enqueue,再打开 `activeCatalogAccounts`; + - 不能让 post-activation update 先于 activation response 到达 CN。 +5. **object / metadata 全局推进,只有 row-level in-memory delta 按 account 过滤** + - insert / update 按行内容里的 account 过滤; + - row delete 按 `cpkey` 解出的 account 过滤; + - startup / reconnect subscribe、activation response、steady-state push 共用同一套过滤契约。 +6. **reconnect 视为批量恢复,不视为单账户新增** + - reconnect 时第一轮 subscribe 直接带 `wantedAccounts ∪ {0}`; + - 用一个统一 replay ts 恢复 sys + wanted accounts。 +7. **lazy catalog 逻辑必须只落在三张 catalog 系统表** + - `SubscribeRequest.lazy_catalog`、activation、row-level catalog filter、account readyTS 都只服务 `mo_database` / `mo_tables` / `mo_columns`; + - 普通表的订阅 / steady-state push / 反订阅语义不能被这轮改动污染。 + +--- + +## 4. 关键术语与状态 + +### 时间戳语义 + +- `**targetTS`** + - TN 在 activation response 中返回的 barrier; + - 只表示“目标 account 缺失的 row-level delta 已补到这个点”。 +- `**replayTS`** + - CN 在确认 `targetTS` 已应用后拿到的确定 snapshot ts; + - 用于 full replay,也作为该 account 的 `readyTS`; + - 按当前 waiter 语义,通常满足 `replayTS > targetTS`。 +- `**readyTS`** + - 某个 account 在某个 CN 上开始可以安全使用 cache serve 的时间点; + - sys account、新增 account、reconnect 恢复的一批 account 都各自依赖这个语义。 + +### 相关键 + +- `**seq**` + - activation 的相关键; + - 不能依赖 `RequestId/ResponseId`,因为它们在当前协议里承担 stream ID 角色。 + +### CN account 状态 + +```go +type accountCatalogState struct { + state catalogReadyState + readyTS timestamp.Timestamp +} + +const ( + accountInactive catalogReadyState = iota + accountCatchingUp + accountReady +) +``` + +语义固定为: + +- `inactive`:不能 serve,也不能直接把该 account 的增量写入 cache。 +- `catching_up`:activation 中,`PartitionState` 正常推进,但 cache 更新进入 `accountDCA`。 +- `ready`:full replay 完成,`readyTS` 已确定,cache 可正常 serve / apply。 + +--- + +## 5. 协议语义 + +### `SubscribeRequest` + +```protobuf +message SubscribeRequest { + api.TableID table = 1; + bool lazy_catalog = 2; + repeated uint32 initial_active_accounts = 3; +} +``` + +用途: + +- startup:`initial_active_accounts = [0]` +- reconnect:`initial_active_accounts = wantedAccounts ∪ {0}` + +### `ActivateAccountForCatalogRequest/Response` + +```protobuf +message ActivateAccountForCatalogRequest { + uint32 account_id = 1; + uint64 seq = 2; +} + +message ActivateAccountForCatalogResponse { + uint32 account_id = 1; + uint64 seq = 2; + timestamp.Timestamp target_ts = 3; + repeated TableLogtail tails = 4; +} +``` + +锁死两条语义: + +1. activation 是 **account 级** 协议,不是 table 级协议; +2. `tails` 只承载三张 catalog 表的 **row-level in-memory delta**,但**不要求固定顺序**;TN 内部可以并发拉取三表,CN 必须按 table identity 消费,而不是按位置消费。 + +--- + +## 6. TN 侧职责 + +### session 侧状态 + +```go +type Session struct { + // existing fields... + lazyCatalogMode bool + activeCatalogAccounts map[uint32]struct{} + activatingAccounts map[uint32]uint64 +} +``` + +### startup / reconnect + +- checkpoint 继续正常返回并进入共享 `PartitionState`; +- object / metadata 继续全局推进; +- row-level in-memory delta 只对 `initial_active_accounts` 放行。 + +### runtime activation + +activation 保持“两阶段、sender 收口”的模型: + +1. `onMessage()` 只做 request 分发和 session 定位; +2. worker 执行 phase1,按表拉历史 row-level delta; +3. `logtailSender()` 串行完成: + - 校验 `activatingAccounts[account] == seq`; + - 取 `targetTS`; + - 拉 `phase1To -> targetTS` 的 phase2; + - 合并 phase1 + phase2; + - 发送 response; + - **只有 response 成功进入 session FIFO `sendChan` 后** 才把 account 加入 `activeCatalogAccounts`。 + +phase1 的下界固定为从 `0` 开始;实际有效下限由现有 checkpoint 处理逻辑决定,不在 activation 协议里额外维护 per-table catch-up 起点。 + +三张 catalog 表的 catch-up 可以并发执行;response 里的 `tails` 只要求完整覆盖三表,不要求固定顺序。 + +### steady state + +- `publishEvent()` 继续全局推 object / metadata; +- row-level catalog delta 只对 `sys ∪ activeCatalogAccounts` 放行。 + +### 过滤契约 + +TN 侧必须复用一套 catalog filter helper,用在: + +- startup / reconnect subscribe response(过滤 row,且去掉 raw `CkpLocation`) +- activation response(过滤 row,且去掉 raw `CkpLocation`) +- steady-state `publishEvent()` + +过滤规则固定为: + +- steady-state push 的三张 catalog 表 in-memory entry batch 仍假定各自只属于一个 account,因此 TN publish 过滤和 CN `accountDCA` 路由都可以按 entry 整体 keep/drop; +- startup / reconnect subscribe response 和 activation response 里的 pulled entry batch 仍可能混合多个 account,因此 TN 在发送前必须复制出目标行; +- steady-state push 的 insert / update:按 entry 首行里的 `account_id` 判断整个 entry; +- steady-state push 的 row delete:按 entry 首行 `cpkey` 解出的 account 判断整个 entry。 + +--- + +## 7. CN 侧职责 + +### 需要维护的状态 + +- `pending(account, seq)`:request / response 关联; +- `state[account]`:`inactive / catching_up / ready`; +- `readyTS[account]`; +- `accountDCA[account]`:activation 期间普通 push 的暂存区; +- `wantedAccounts`:reconnect 时要恢复的账户集合。 + +### startup + +- 继续走当前 startup 骨架; +- `replayCatalogCache()` 仍只加载 sys account; +- startup replay 完成后设置 `sys.readyTS = startupReplayTS`。 + +### runtime activation + +同步路径固定为: + +1. 如果 account 已 ready,直接返回; +2. `singleflight(key=accountID)`; +3. 置 `state[account] = catching_up` 并初始化 `accountDCA`; +4. 发送 `ActivateAccountForCatalogRequest{account, seq}`; +5. 收到 response 后,先把 `tails` 应用到 `PartitionState`; +6. 调 `WaitLogTailAppliedAt(targetTS)` 得到确定的 `replayTS`; +7. 执行 `replayCatalogCacheForAccount(account, replayTS)`; +8. 在 `catalogCacheMu` 下先 drain `accountDCA[account]`,再发布 `readyTS[account] = replayTS`; +9. 最后把状态切到 `ready`。 + +### cache apply 规则 + +- `inactive`:不直接写 cache; +- `catching_up`:普通 push 继续更新 `PartitionState`,cache apply 写入 `accountDCA`; +- `ready`:正常更新 cache。 + +这里依赖的是 steady-state push-entry 单 account 假设:CN 只对运行中收到的 pushed entry 按 entry 归属 account 决定是立即 apply 还是进入 `accountDCA`,不再按 row 拆分 batch。 + +### serve gate + +最终读路径必须按 account 判断: + +```go +func CanServeAccount(account uint32, ts timestamp.Timestamp) bool { + return globalCatalogCacheCanServe(ts) && + !accountReadyTS[account].IsEmpty() && + ts.GreaterEq(accountReadyTS[account]) +} +``` + +不能只依赖全局 `CatalogCache.CanServe(ts)`。 + +### 失败清理 + +如果 activation 失败或超时,且没有新的 seq 接管,必须: + +- 清理 `pending(account, seq)`; +- 丢弃 `accountDCA[account]`; +- 把 `state[account]` 从 `catching_up` 回退到 `inactive`。 + +--- + +## 8. Frontend 与 reconnect 钩子 + +### Frontend + +`pkg/frontend/session.go::(*Session).AuthenticateUser` 中: + +- 在 `tenant.SetTenantID(...)` 之后触发 activation; +- 在第一个 tenant-context SQL 之前完成 activation; +- 出错时直接返回,不能吞错后继续 tenant SQL。 + +### reconnect + +- CN 记住断线前哪些 account 已 ready,记为 `wantedAccounts`; +- reconnect 后第一次 `subSysTables()` 直接携带 `wantedAccounts ∪ {0}`; +- 用一个 `reconnectReplayTS` 重建 sys + wanted accounts cache; +- 为这一批 account 统一设置 `readyTS` 和 `state=ready`。 + +--- + +## 9. 主要实现落点 + +### 协议 + +- `proto/logtail.proto` +- `pkg/pb/logtail/logtail.pb.go` + +### TN + +- `pkg/vm/engine/tae/logtail/service/session.go` +- `pkg/vm/engine/tae/logtail/service/server.go` +- `pkg/vm/engine/tae/logtail/service/client.go` +- `pkg/vm/engine/tae/logtail/service/` 下新增 catalog row-level filter helper + +### CN + +- `pkg/vm/engine/disttae/logtail_consumer.go` +- `pkg/vm/engine/disttae/logtail.go` +- `pkg/vm/engine/disttae/engine.go` +- `pkg/vm/engine/disttae/cache/catalog.go` +- `pkg/catalog/types.go` + +### Frontend + +- `pkg/frontend/session.go` + +--- + +## 10. 代码前最后核对项 + +1. startup 后 `CatalogCache` 只有 sys account; +2. activation response 只补 `PartitionState`,不直接宣告 cache ready; +3. `replayTS` 是确定值,并且来自 `WaitLogTailAppliedAt(targetTS)`; +4. activation 期间实时 cache 更新会暂存,replay 后再 flush; +5. `CanServeAccount(account, ts)` 依赖 per-account `readyTS`; +6. reconnect 一次性恢复 wanted accounts,而不是逐个 activation; +7. auth 路径必须在 tenant SQL 前完成 activation; +8. row delete 和 insert / update 一样,保持严格 account-scoped 过滤。 diff --git a/pkg/catalog/lazy_catalog.go b/pkg/catalog/lazy_catalog.go new file mode 100644 index 0000000000000..8b9bbf102e8c2 --- /dev/null +++ b/pkg/catalog/lazy_catalog.go @@ -0,0 +1,158 @@ +// Copyright 2024 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package catalog + +import ( + "github.com/matrixorigin/matrixone/pkg/common/moerr" + "github.com/matrixorigin/matrixone/pkg/container/batch" + "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/container/vector" + "github.com/matrixorigin/matrixone/pkg/objectio" + "github.com/matrixorigin/matrixone/pkg/pb/api" +) + +func IsLazyCatalogTableID(tableID uint64) bool { + return tableID == MO_DATABASE_ID || + tableID == MO_TABLES_ID || + tableID == MO_COLUMNS_ID +} + +func SetLazyCatalogEntryAccountSummary(entry *api.Entry, accountID uint32) { + if entry == nil { + return + } + entry.LazyCatalogAccountId = accountID + entry.HasLazyCatalogAccountId = true +} + +func LazyCatalogEntryAccountSummary(entry api.Entry) (uint32, bool) { + if !entry.GetHasLazyCatalogAccountId() { + return 0, false + } + return entry.GetLazyCatalogAccountId(), true +} + +func LazyCatalogEntryAccountID(entry api.Entry) (uint32, bool, error) { + if accountID, ok := LazyCatalogEntryAccountSummary(entry); ok { + return accountID, true, nil + } + if !IsLazyCatalogTableID(entry.GetTableId()) { + return 0, false, nil + } + + bat, err := mustLazyCatalogProtoBatch(entry) + if err != nil { + return 0, false, err + } + if bat.RowCount() == 0 { + return 0, false, nil + } + + switch entry.GetEntryType() { + case api.Entry_Insert, api.Entry_Update: + return lazyCatalogInsertOrUpdateAccountID(entry, bat) + case api.Entry_Delete: + return lazyCatalogDeleteAccountID(bat) + default: + return 0, false, nil + } +} + +func mustLazyCatalogProtoBatch(entry api.Entry) (*batch.Batch, error) { + if entry.Bat == nil { + return nil, moerr.NewInternalErrorNoCtxf( + "catalog logtail entry %s missing batch", + entry.GetEntryType().String(), + ) + } + return batch.ProtoBatchToBatch(entry.Bat) +} + +func lazyCatalogInsertOrUpdateAccountID( + entry api.Entry, + bat *batch.Batch, +) (uint32, bool, error) { + accountIdx := FindBatchAttrIndex(bat.Attrs, SystemDBAttr_AccID) + if accountIdx < 0 { + return 0, false, moerr.NewInternalErrorNoCtxf( + "catalog logtail entry %s missing account_id column, attrs=%v", + entry.GetEntryType().String(), + bat.Attrs, + ) + } + + accounts := vector.MustFixedColWithTypeCheck[uint32](bat.GetVector(int32(accountIdx))) + if len(accounts) == 0 { + return 0, false, nil + } + return accounts[0], true, nil +} + +func lazyCatalogDeleteAccountID(bat *batch.Batch) (uint32, bool, error) { + // Insert/update entries use __mo_cpkey_col; tombstone/delete entries + // use __mo_%1_pk_val. Both contain the same compound-key bytes. + cpkeyIdx := FindCatalogDeletePKIndex(bat.Attrs) + if cpkeyIdx < 0 { + return 0, false, moerr.NewInternalErrorNoCtxf( + "catalog delete logtail entry missing cpkey/pk column, attrs=%v", + bat.Attrs, + ) + } + + accountID, err := DecodeLazyCatalogAccountFromCPKey(bat.GetVector(int32(cpkeyIdx)).GetBytesAt(0)) + if err != nil { + return 0, false, err + } + return accountID, true, nil +} + +// FindCatalogDeletePKIndex returns the batch attribute index that carries +// the compound primary key bytes. Insert/update entries name it +// CPrimaryKeyColName (__mo_cpkey_col); tombstone entries name it +// TombstoneAttr_PK_Attr (__mo_%1_pk_val). Both hold the same encoded tuple. +func FindCatalogDeletePKIndex(attrs []string) int { + if idx := FindBatchAttrIndex(attrs, CPrimaryKeyColName); idx >= 0 { + return idx + } + return FindBatchAttrIndex(attrs, objectio.TombstoneAttr_PK_Attr) +} + +func FindBatchAttrIndex(attrs []string, target string) int { + for i, attr := range attrs { + if attr == target { + return i + } + } + return -1 +} + +func DecodeLazyCatalogAccountFromCPKey(cpkey []byte) (uint32, error) { + tuple, err := types.Unpack(cpkey) + if err != nil { + return 0, err + } + if len(tuple) == 0 { + return 0, moerr.NewInternalErrorNoCtx("empty catalog cpkey") + } + + accountID, ok := tuple[0].(uint32) + if !ok { + return 0, moerr.NewInternalErrorNoCtxf( + "unexpected catalog cpkey account type %T", + tuple[0], + ) + } + return accountID, nil +} diff --git a/pkg/catalog/types.go b/pkg/catalog/types.go index d5d4cc08e3f87..9626673e3f6d4 100644 --- a/pkg/catalog/types.go +++ b/pkg/catalog/types.go @@ -16,6 +16,7 @@ package catalog import ( "fmt" + "strconv" "strings" "github.com/matrixorigin/matrixone/pkg/container/batch" @@ -842,6 +843,50 @@ var ( } ) +func BuildMoDatabaseBatchQuery(accountIDs []uint32) string { + return buildAccountScopedCatalogBatchQuery(MoDatabaseBatchQuery, SystemDBAttr_AccID, accountIDs) +} + +func BuildMoTablesBatchQuery(accountIDs []uint32) string { + return buildAccountScopedCatalogBatchQuery(MoTablesBatchQuery, SystemRelAttr_AccID, accountIDs) +} + +func BuildMoColumnsBatchQuery(accountIDs []uint32) string { + return buildAccountScopedCatalogBatchQuery(MoColumnsBatchQuery, SystemColAttr_AccID, accountIDs) +} + +func buildAccountScopedCatalogBatchQuery(baseQuery string, accountAttr string, accountIDs []uint32) string { + if len(accountIDs) == 0 { + return baseQuery + } + + // accountIDs come from internal tenant identifiers and are rendered as + // decimal uint32 values by joinAccountIDs, so this builder never interpolates + // free-form SQL text. + orderBy := "" + query := baseQuery + const orderByClause = " order by " + if idx := strings.Index(query, orderByClause); idx >= 0 { + orderBy = query[idx:] + query = query[:idx] + } + + return fmt.Sprintf("%s and %s in (%s)%s", + query, + accountAttr, + joinAccountIDs(accountIDs), + orderBy, + ) +} + +func joinAccountIDs(accountIDs []uint32) string { + parts := make([]string, 0, len(accountIDs)) + for _, accountID := range accountIDs { + parts = append(parts, strconv.FormatUint(uint64(accountID), 10)) + } + return strings.Join(parts, ",") +} + var ( QueryResultPath string QueryResultMetaPath string diff --git a/pkg/catalog/types_test.go b/pkg/catalog/types_test.go new file mode 100644 index 0000000000000..fd50c46a6acea --- /dev/null +++ b/pkg/catalog/types_test.go @@ -0,0 +1,49 @@ +// Copyright 2021 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package catalog + +import ( + "strings" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestBuildMoDatabaseBatchQuery(t *testing.T) { + require.Equal(t, MoDatabaseBatchQuery, BuildMoDatabaseBatchQuery(nil)) + require.Equal( + t, + MoDatabaseBatchQuery+" and "+SystemDBAttr_AccID+" in (0,10)", + BuildMoDatabaseBatchQuery([]uint32{0, 10}), + ) +} + +func TestBuildMoTablesBatchQuery(t *testing.T) { + require.Equal(t, MoTablesBatchQuery, BuildMoTablesBatchQuery(nil)) + require.Equal( + t, + MoTablesBatchQuery+" and "+SystemRelAttr_AccID+" in (1)", + BuildMoTablesBatchQuery([]uint32{1}), + ) +} + +func TestBuildMoColumnsBatchQuery(t *testing.T) { + require.Equal(t, MoColumnsBatchQuery, BuildMoColumnsBatchQuery(nil)) + require.Equal( + t, + strings.Replace(MoColumnsBatchQuery, " order by ", " and "+SystemColAttr_AccID+" in (1,2) order by ", 1), + BuildMoColumnsBatchQuery([]uint32{1, 2}), + ) +} diff --git a/pkg/cnservice/distributed_tae.go b/pkg/cnservice/distributed_tae.go index ecd88be66a2aa..952499721cb9d 100644 --- a/pkg/cnservice/distributed_tae.go +++ b/pkg/cnservice/distributed_tae.go @@ -112,6 +112,7 @@ func (s *service) initDistributedTAE( statusServer := ss.(*status.Server) statusServer.SetTxnClient(s.cfg.UUID, client) statusServer.SetLogTailClient(s.cfg.UUID, cnEngine.PushClient()) + statusServer.SetEngine(s.cfg.UUID, cnEngine) } s.initProcessCodecService() diff --git a/pkg/frontend/session.go b/pkg/frontend/session.go index 5ef3d7b2c2988..918b6c8b317d6 100644 --- a/pkg/frontend/session.go +++ b/pkg/frontend/session.go @@ -50,6 +50,7 @@ import ( v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2" "github.com/matrixorigin/matrixone/pkg/util/trace" "github.com/matrixorigin/matrixone/pkg/util/trace/impl/motrace" + "github.com/matrixorigin/matrixone/pkg/vm/engine" "github.com/matrixorigin/matrixone/pkg/vm/process" ) @@ -1390,8 +1391,11 @@ func (ses *Session) AuthenticateUser(ctx context.Context, userInput string, dbNa ses.timestampMap[TSCheckTenantEnd] = time.Now() v2.CheckTenantDurationHistogram.Observe(ses.timestampMap[TSCheckTenantEnd].Sub(ses.timestampMap[TSCheckTenantStart]).Seconds()) - //step2 : check user exists or not in general tenant. - //step3 : get the password of the user + if activator, ok := ses.GetTxnHandler().GetStorage().(engine.TenantCatalogActivator); ok { + if err = activator.ActivateTenantCatalog(ctx, uint32(tenantID)); err != nil { + return nil, err + } + } ses.timestampMap[TSCheckUserStart] = time.Now() tenantCtx := defines.AttachAccountId(ctx, uint32(tenantID)) diff --git a/pkg/pb/api/api.pb.go b/pkg/pb/api/api.pb.go index c9c811b399ad1..98e10670ead0d 100644 --- a/pkg/pb/api/api.pb.go +++ b/pkg/pb/api/api.pb.go @@ -773,10 +773,7 @@ func (m *SyncLogTailResp) GetCommands() []*Entry { // How to parse and handle PrecommiWriteCmd , pls ref to // tae/rpc/handle.go/HandlePreCommit function type PrecommitWriteCmd struct { - EntryList []*Entry `protobuf:"bytes,1,rep,name=entry_list,json=entryList,proto3" json:"entry_list,omitempty"` - // Job ID for sync protection validation during CCPR commits. - // When non-empty, TN validates the sync protection is still valid at prepareTS. - SyncProtectionJobId string `protobuf:"bytes,2,opt,name=sync_protection_job_id,json=syncProtectionJobId,proto3" json:"sync_protection_job_id,omitempty"` + EntryList []*Entry `protobuf:"bytes,1,rep,name=entry_list,json=entryList,proto3" json:"entry_list,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -822,13 +819,6 @@ func (m *PrecommitWriteCmd) GetEntryList() []*Entry { return nil } -func (m *PrecommitWriteCmd) GetSyncProtectionJobId() string { - if m != nil { - return m.SyncProtectionJobId - } - return "" -} - type Entry struct { EntryType Entry_EntryType `protobuf:"varint,1,opt,name=entry_type,json=entryType,proto3,enum=api.Entry_EntryType" json:"entry_type,omitempty"` TableId uint64 `protobuf:"varint,2,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"` @@ -838,10 +828,14 @@ type Entry struct { FileName string `protobuf:"bytes,6,opt,name=file_name,json=fileName,proto3" json:"file_name,omitempty"` Bat *Batch `protobuf:"bytes,7,opt,name=bat,proto3" json:"bat,omitempty"` // whether TN do the PK uniqueness check against txn's workspace or not. - PkCheckByTn int32 `protobuf:"varint,8,opt,name=pk_check_by_tn,json=pkCheckByTn,proto3" json:"pk_check_by_tn,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + PkCheckByTn int32 `protobuf:"varint,8,opt,name=pk_check_by_tn,json=pkCheckByTn,proto3" json:"pk_check_by_tn,omitempty"` + // Presence-safe account summary for lazy-catalog push entries. Account 0 is + // real, so the boolean is required to distinguish "absent" from "sys". + LazyCatalogAccountId uint32 `protobuf:"varint,9,opt,name=lazy_catalog_account_id,json=lazyCatalogAccountId,proto3" json:"lazy_catalog_account_id,omitempty"` + HasLazyCatalogAccountId bool `protobuf:"varint,10,opt,name=has_lazy_catalog_account_id,json=hasLazyCatalogAccountId,proto3" json:"has_lazy_catalog_account_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *Entry) Reset() { *m = Entry{} } @@ -933,6 +927,20 @@ func (m *Entry) GetPkCheckByTn() int32 { return 0 } +func (m *Entry) GetLazyCatalogAccountId() uint32 { + if m != nil { + return m.LazyCatalogAccountId + } + return 0 +} + +func (m *Entry) GetHasLazyCatalogAccountId() bool { + if m != nil { + return m.HasLazyCatalogAccountId + } + return false +} + // CatalogCkp contains information about database and tables in the system,and // MetadataCkp contains information about blocks. type Checkpoint struct { @@ -1783,22 +1791,20 @@ type SchemaExtra struct { DroppedAttrs []string `protobuf:"bytes,2,rep,name=dropped_attrs,json=droppedAttrs,proto3" json:"dropped_attrs,omitempty"` ColumnChanged bool `protobuf:"varint,3,opt,name=column_changed,json=columnChanged,proto3" json:"column_changed,omitempty"` // sending mo_tables deletes by this. - OldName string `protobuf:"bytes,4,opt,name=old_name,json=oldName,proto3" json:"old_name,omitempty"` - MinOsizeQuailifed uint32 `protobuf:"varint,5,opt,name=min_osize_quailifed,json=minOsizeQuailifed,proto3" json:"min_osize_quailifed,omitempty"` - MaxObjOnerun uint32 `protobuf:"varint,6,opt,name=max_obj_onerun,json=maxObjOnerun,proto3" json:"max_obj_onerun,omitempty"` - MaxOsizeMergedObj uint32 `protobuf:"varint,7,opt,name=max_osize_merged_obj,json=maxOsizeMergedObj,proto3" json:"max_osize_merged_obj,omitempty"` - Hints []MergeHint `protobuf:"varint,8,rep,packed,name=hints,proto3,enum=api.MergeHint" json:"hints,omitempty"` - MinCnMergeSize uint64 `protobuf:"varint,9,opt,name=min_cn_merge_size,json=minCnMergeSize,proto3" json:"min_cn_merge_size,omitempty"` - BlockMaxRows uint32 `protobuf:"varint,10,opt,name=block_max_rows,json=blockMaxRows,proto3" json:"block_max_rows,omitempty"` - ObjectMaxBlocks uint32 `protobuf:"varint,11,opt,name=object_max_blocks,json=objectMaxBlocks,proto3" json:"object_max_blocks,omitempty"` - FeatureFlag uint64 `protobuf:"varint,12,opt,name=FeatureFlag,proto3" json:"FeatureFlag,omitempty"` - IndexTables []uint64 `protobuf:"varint,13,rep,packed,name=IndexTables,proto3" json:"IndexTables,omitempty"` - ParentTableID uint64 `protobuf:"varint,14,opt,name=ParentTableID,proto3" json:"ParentTableID,omitempty"` - // mark if table is created by publication (CCPR), should skip merge - FromPublication bool `protobuf:"varint,15,opt,name=FromPublication,proto3" json:"FromPublication,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + OldName string `protobuf:"bytes,4,opt,name=old_name,json=oldName,proto3" json:"old_name,omitempty"` + MinOsizeQuailifed uint32 `protobuf:"varint,5,opt,name=min_osize_quailifed,json=minOsizeQuailifed,proto3" json:"min_osize_quailifed,omitempty"` + MaxObjOnerun uint32 `protobuf:"varint,6,opt,name=max_obj_onerun,json=maxObjOnerun,proto3" json:"max_obj_onerun,omitempty"` + MaxOsizeMergedObj uint32 `protobuf:"varint,7,opt,name=max_osize_merged_obj,json=maxOsizeMergedObj,proto3" json:"max_osize_merged_obj,omitempty"` + Hints []MergeHint `protobuf:"varint,8,rep,packed,name=hints,proto3,enum=api.MergeHint" json:"hints,omitempty"` + MinCnMergeSize uint64 `protobuf:"varint,9,opt,name=min_cn_merge_size,json=minCnMergeSize,proto3" json:"min_cn_merge_size,omitempty"` + BlockMaxRows uint32 `protobuf:"varint,10,opt,name=block_max_rows,json=blockMaxRows,proto3" json:"block_max_rows,omitempty"` + ObjectMaxBlocks uint32 `protobuf:"varint,11,opt,name=object_max_blocks,json=objectMaxBlocks,proto3" json:"object_max_blocks,omitempty"` + FeatureFlag uint64 `protobuf:"varint,12,opt,name=FeatureFlag,proto3" json:"FeatureFlag,omitempty"` + IndexTables []uint64 `protobuf:"varint,13,rep,packed,name=IndexTables,proto3" json:"IndexTables,omitempty"` + ParentTableID uint64 `protobuf:"varint,14,opt,name=ParentTableID,proto3" json:"ParentTableID,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *SchemaExtra) Reset() { *m = SchemaExtra{} } @@ -1932,13 +1938,6 @@ func (m *SchemaExtra) GetParentTableID() uint64 { return 0 } -func (m *SchemaExtra) GetFromPublication() bool { - if m != nil { - return m.FromPublication - } - return false -} - // Int64Map mainly used in unit test type Int64Map struct { M map[int64]int64 `protobuf:"bytes,1,rep,name=m,proto3" json:"m,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` @@ -2472,172 +2471,172 @@ func init() { func init() { proto.RegisterFile("api.proto", fileDescriptor_00212fb1f9d3bf1c) } var fileDescriptor_00212fb1f9d3bf1c = []byte{ - // 2630 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x58, 0x5f, 0x6f, 0x5c, 0x47, - 0x15, 0xf7, 0x7a, 0xff, 0x9f, 0xfd, 0x77, 0x3d, 0x4e, 0xd2, 0xad, 0x29, 0x89, 0xd9, 0xa6, 0xad, - 0x9b, 0x52, 0x47, 0xb8, 0x05, 0xda, 0x2a, 0xa2, 0x8a, 0xed, 0x26, 0xde, 0x12, 0xc7, 0xe6, 0x7a, - 0xd3, 0x4a, 0x15, 0xd2, 0xd5, 0xec, 0xbd, 0x93, 0xf5, 0xcd, 0xde, 0x3b, 0x73, 0x33, 0x33, 0xeb, - 0xd8, 0x7d, 0x05, 0x1e, 0x91, 0x10, 0x6f, 0xbc, 0xb5, 0x1f, 0x01, 0x09, 0x89, 0x6f, 0x80, 0xfa, - 0x58, 0x44, 0xf9, 0x57, 0xa0, 0x84, 0xf2, 0x02, 0x7c, 0x0a, 0x34, 0x67, 0xe6, 0xee, 0xae, 0x9d, - 0xb4, 0x50, 0x84, 0xd4, 0x07, 0x5b, 0x33, 0xbf, 0x73, 0xce, 0xdc, 0xdf, 0x39, 0x73, 0xe6, 0xcc, - 0x99, 0x85, 0x3a, 0xcd, 0xe2, 0xf5, 0x4c, 0x0a, 0x2d, 0x48, 0x91, 0x66, 0xf1, 0xca, 0x8b, 0xa3, - 0x58, 0x1f, 0x4e, 0x86, 0xeb, 0xa1, 0x48, 0xaf, 0x8e, 0xc4, 0x48, 0x5c, 0x45, 0xd9, 0x70, 0x72, - 0x17, 0x67, 0x38, 0xc1, 0x91, 0xb5, 0x59, 0xe9, 0xe8, 0x38, 0x65, 0x4a, 0xd3, 0x34, 0x73, 0x00, - 0x64, 0x09, 0xe5, 0x76, 0xdc, 0xfb, 0x36, 0xb4, 0x06, 0xb7, 0xf7, 0x63, 0x3e, 0xf2, 0xd9, 0xfd, - 0x09, 0x53, 0x9a, 0x3c, 0x05, 0xf5, 0x8c, 0x4a, 0x9a, 0x32, 0xcd, 0x64, 0xb7, 0xb0, 0x5a, 0x58, - 0xab, 0xfb, 0x33, 0xe0, 0xb5, 0xda, 0x7b, 0xef, 0x5f, 0x2a, 0x3c, 0x7c, 0xff, 0xd2, 0x42, 0xef, - 0x17, 0x05, 0x68, 0xe7, 0x96, 0x2a, 0x13, 0x5c, 0x31, 0xd2, 0x85, 0xaa, 0xd2, 0x42, 0xb2, 0xfe, - 0xb6, 0x33, 0xcc, 0xa7, 0xe4, 0x59, 0x68, 0x2b, 0x26, 0x8f, 0xe2, 0x90, 0x5d, 0x8f, 0x22, 0xc9, - 0x94, 0xea, 0x2e, 0xa2, 0xc2, 0x19, 0x14, 0x57, 0x38, 0xa4, 0x32, 0xea, 0x6f, 0x77, 0x8b, 0xab, - 0x85, 0xb5, 0x92, 0x9f, 0x4f, 0x0d, 0x2d, 0xc9, 0xb2, 0x24, 0x0e, 0x69, 0x7f, 0xbb, 0x5b, 0x42, - 0xd9, 0x0c, 0x20, 0x17, 0x01, 0x12, 0x31, 0x3a, 0x70, 0xa6, 0x65, 0x14, 0xcf, 0x21, 0x73, 0xb4, - 0x5f, 0x03, 0x6f, 0x70, 0xfb, 0x40, 0xcb, 0x79, 0xde, 0xb8, 0xb6, 0x9e, 0x48, 0x7e, 0xa0, 0xa7, - 0x2e, 0x4f, 0x81, 0x39, 0xdb, 0x9f, 0x17, 0xa0, 0xf2, 0x16, 0x0b, 0xb5, 0x90, 0x84, 0x40, 0x29, - 0xa2, 0x9a, 0xa2, 0x76, 0xd3, 0xc7, 0x31, 0xb9, 0x0c, 0x25, 0x7d, 0x92, 0x31, 0x74, 0xad, 0xb1, - 0x01, 0xeb, 0x18, 0xe5, 0xc1, 0x49, 0xc6, 0x36, 0x4b, 0x1f, 0x7c, 0x72, 0x69, 0xc1, 0x47, 0x29, - 0x59, 0x81, 0x1a, 0x9f, 0x24, 0x09, 0x1d, 0x26, 0x0c, 0x7d, 0xac, 0xf9, 0xd3, 0x39, 0xf1, 0xa0, - 0xc8, 0x55, 0x86, 0xee, 0x35, 0x7d, 0x33, 0x24, 0x4f, 0x42, 0x2d, 0x56, 0x41, 0x28, 0xb8, 0xd2, - 0xe8, 0x56, 0xcd, 0xaf, 0xc6, 0x6a, 0xcb, 0x4c, 0x8d, 0x72, 0xc2, 0x78, 0xb7, 0xb2, 0x5a, 0x58, - 0x6b, 0xf9, 0x66, 0x68, 0x48, 0x51, 0xc9, 0x68, 0xb7, 0x6a, 0x49, 0x99, 0x71, 0xef, 0x4d, 0x28, - 0x6f, 0x52, 0x1d, 0x1e, 0x92, 0x15, 0x28, 0x53, 0xad, 0xa5, 0xea, 0x16, 0x56, 0x8b, 0x6b, 0x75, - 0x47, 0xc9, 0x42, 0xe4, 0x19, 0x28, 0x1d, 0xb1, 0xd0, 0x6c, 0x4a, 0x71, 0xad, 0xb1, 0xd1, 0x58, - 0x37, 0xf9, 0x66, 0x1d, 0xcd, 0xa9, 0x1b, 0x71, 0xef, 0x57, 0x05, 0xa8, 0x0e, 0x0c, 0xd1, 0xfe, - 0x36, 0x59, 0x86, 0x72, 0x34, 0x0c, 0xe2, 0x08, 0x23, 0x50, 0xf2, 0x4b, 0xd1, 0xb0, 0x1f, 0x19, - 0x50, 0x23, 0xb8, 0x68, 0x41, 0x6d, 0xc0, 0xaf, 0x41, 0x33, 0xa3, 0x52, 0xc7, 0x3a, 0x16, 0xdc, - 0xc8, 0xec, 0xc6, 0x36, 0xa6, 0x58, 0x3f, 0x22, 0xe7, 0xa1, 0x42, 0xc3, 0xd0, 0x08, 0x4b, 0xe8, - 0x4d, 0x99, 0x86, 0x61, 0x3f, 0x22, 0x4f, 0x40, 0x35, 0x1a, 0x06, 0x9c, 0xa6, 0x0c, 0x7d, 0xaf, - 0xfb, 0x95, 0x68, 0x78, 0x9b, 0xa6, 0xcc, 0x08, 0xb4, 0x13, 0x54, 0xac, 0x40, 0x5b, 0xc1, 0x33, - 0xd0, 0xce, 0x64, 0x9c, 0x52, 0x79, 0x12, 0x28, 0x76, 0x9f, 0x4f, 0x52, 0x8c, 0x45, 0xcb, 0x6f, - 0x39, 0xf4, 0x00, 0xc1, 0xde, 0x4f, 0x0b, 0xd0, 0x3e, 0x38, 0xe1, 0xe1, 0x2d, 0x31, 0x1a, 0xd0, - 0x38, 0xf1, 0xd9, 0x7d, 0xf2, 0x22, 0x54, 0x43, 0x1e, 0x1c, 0xd2, 0x23, 0x86, 0x1e, 0x35, 0x36, - 0xce, 0xad, 0xcf, 0x8e, 0xcd, 0x20, 0x1f, 0xf9, 0x95, 0x90, 0xef, 0xd0, 0x23, 0xe6, 0xd4, 0x1f, - 0x50, 0xae, 0xdd, 0x76, 0x7f, 0xa6, 0xfa, 0xdb, 0x94, 0x6b, 0xd2, 0x83, 0xb2, 0x9e, 0xee, 0x78, - 0x63, 0xa3, 0x89, 0x11, 0x76, 0xa1, 0xf4, 0xad, 0xa8, 0xf7, 0x7d, 0xe8, 0x9c, 0xe2, 0xa4, 0x32, - 0x13, 0xba, 0x70, 0x9c, 0x05, 0x89, 0x08, 0xa9, 0x89, 0x94, 0xcb, 0xcd, 0x46, 0x38, 0xce, 0x6e, - 0x39, 0x88, 0x3c, 0x0b, 0xb5, 0x50, 0xa4, 0x29, 0xe5, 0x51, 0xbe, 0x7d, 0x80, 0x8b, 0xbf, 0xc1, - 0xb5, 0x3c, 0xf1, 0xa7, 0xb2, 0x9e, 0x82, 0xa5, 0x7d, 0xc9, 0xcc, 0x34, 0xd6, 0x6f, 0xcb, 0x58, - 0xb3, 0xad, 0x34, 0x22, 0xcf, 0x03, 0x30, 0xa3, 0x17, 0x24, 0xb1, 0xd2, 0x98, 0x18, 0xa7, 0xcd, - 0xeb, 0x28, 0xbd, 0x15, 0x2b, 0x4d, 0x5e, 0x82, 0x0b, 0xea, 0x84, 0x87, 0x81, 0xa9, 0x1a, 0x2c, - 0xc4, 0xbd, 0xbc, 0x27, 0xa6, 0x7b, 0x5d, 0xf7, 0x97, 0x8d, 0x74, 0x7f, 0x2a, 0x7c, 0x53, 0x0c, - 0xfb, 0x51, 0xef, 0x27, 0x45, 0x28, 0xe3, 0x4a, 0xe4, 0xa5, 0xfc, 0x4b, 0x78, 0x42, 0x8c, 0x1f, - 0xed, 0x8d, 0x73, 0xb3, 0x2f, 0xd9, 0xff, 0xe6, 0xac, 0xb8, 0x6f, 0x9a, 0xa1, 0x49, 0x7e, 0x0c, - 0xcd, 0x2c, 0xa3, 0xaa, 0x38, 0xef, 0x47, 0xe4, 0x12, 0x34, 0xcc, 0x99, 0x1b, 0x52, 0xc5, 0x66, - 0x39, 0x05, 0x39, 0xd4, 0x8f, 0xc8, 0x57, 0x01, 0xac, 0x2d, 0x66, 0x49, 0xc9, 0x1e, 0x6a, 0x44, - 0x30, 0x51, 0x9e, 0x86, 0xd6, 0xd4, 0x7e, 0x2e, 0xc1, 0x9a, 0x39, 0x88, 0x4a, 0x5f, 0x81, 0xfa, - 0xdd, 0x38, 0x5f, 0xc2, 0x26, 0x5a, 0xcd, 0x00, 0x28, 0x7c, 0x0a, 0x8a, 0x43, 0xaa, 0x31, 0xbf, - 0xf2, 0xa0, 0xe1, 0x41, 0xf3, 0x0d, 0x4c, 0x9e, 0x86, 0x76, 0x36, 0x0e, 0xc2, 0x43, 0x16, 0x8e, - 0x83, 0xe1, 0x49, 0xa0, 0x79, 0xb7, 0xb6, 0x5a, 0x58, 0x2b, 0xfb, 0x8d, 0x6c, 0xbc, 0x65, 0xc0, - 0xcd, 0x93, 0x01, 0xef, 0x49, 0xa8, 0x4f, 0xfd, 0x26, 0x00, 0x95, 0x3e, 0x57, 0x4c, 0x6a, 0x6f, - 0xc1, 0x8c, 0xb7, 0x59, 0xc2, 0x34, 0xf3, 0x0a, 0x66, 0x7c, 0x27, 0x8b, 0xa8, 0x66, 0xde, 0x22, - 0xa9, 0x43, 0xf9, 0x7a, 0xa2, 0x99, 0xf4, 0x8a, 0x64, 0x09, 0x5a, 0x07, 0x19, 0x0b, 0x63, 0x9a, - 0x38, 0xcd, 0x12, 0x69, 0x03, 0x6c, 0x53, 0x4d, 0xf7, 0x86, 0xf7, 0x58, 0xa8, 0xbd, 0x32, 0x59, - 0x86, 0xce, 0x40, 0xa4, 0x43, 0xa5, 0x05, 0x67, 0x0e, 0xac, 0xf4, 0x7e, 0x58, 0x00, 0x40, 0x06, - 0x99, 0x88, 0xb9, 0x26, 0x2f, 0x40, 0x25, 0x8d, 0x79, 0xa0, 0xd5, 0xe7, 0x66, 0x7d, 0x39, 0x8d, - 0xf9, 0x40, 0xa1, 0x32, 0x3d, 0x36, 0xca, 0x8b, 0x9f, 0xab, 0x4c, 0x8f, 0x07, 0x2a, 0x8f, 0x4f, - 0xf1, 0xb1, 0xf1, 0xb1, 0x34, 0xa8, 0xa6, 0x89, 0x18, 0x6d, 0x8d, 0xb3, 0x2f, 0x8d, 0xc6, 0x8f, - 0x0a, 0xd0, 0xd8, 0x65, 0x9a, 0x9a, 0x6d, 0xff, 0x32, 0x79, 0xfc, 0xab, 0x00, 0x1e, 0xee, 0x2c, - 0xd6, 0x84, 0x7d, 0x91, 0xc4, 0xe1, 0x09, 0x59, 0x87, 0x65, 0x43, 0x46, 0xa8, 0xf8, 0x5d, 0x16, - 0xdc, 0x9f, 0xd0, 0x38, 0x89, 0xef, 0x32, 0x5b, 0x70, 0x5b, 0xfe, 0x52, 0x1a, 0xf3, 0x3d, 0x23, - 0xf9, 0x5e, 0x2e, 0x20, 0x97, 0xa1, 0x6d, 0xf8, 0x88, 0xe1, 0xbd, 0x40, 0x70, 0x26, 0x27, 0x1c, - 0x79, 0xb5, 0xfc, 0x66, 0x4a, 0x8f, 0xf7, 0x86, 0xf7, 0xf6, 0x10, 0x23, 0x57, 0xe1, 0x1c, 0x6a, - 0xe1, 0xaa, 0x29, 0x93, 0x23, 0x16, 0x19, 0x13, 0x64, 0x66, 0x96, 0xa5, 0xc7, 0xb8, 0xec, 0x2e, - 0x4a, 0xf6, 0x86, 0xf7, 0xc8, 0x65, 0x28, 0x1f, 0xc6, 0x5c, 0xab, 0x6e, 0x69, 0xb5, 0xb8, 0xd6, - 0xde, 0x68, 0x23, 0x77, 0x14, 0xef, 0xc4, 0x5c, 0xfb, 0x56, 0x48, 0x9e, 0x07, 0xc3, 0x28, 0x08, - 0xb9, 0x5d, 0x33, 0x30, 0x6b, 0xb8, 0x8b, 0xb8, 0x9d, 0xc6, 0x7c, 0x8b, 0xa3, 0xc5, 0x41, 0xfc, - 0x2e, 0xeb, 0xbd, 0x02, 0xe7, 0x66, 0xbe, 0xe2, 0x5d, 0x26, 0xa9, 0xc9, 0xc5, 0x55, 0x68, 0x84, - 0xd3, 0x99, 0x72, 0x57, 0xeb, 0x3c, 0xd4, 0x7b, 0x11, 0x96, 0xe6, 0x2d, 0xd3, 0x94, 0x71, 0x6d, - 0x7a, 0x86, 0xd0, 0x0e, 0xf3, 0xae, 0xc3, 0x4d, 0x7b, 0xbb, 0x70, 0x7e, 0xa6, 0xee, 0x33, 0x73, - 0x8c, 0x71, 0x68, 0x0a, 0x8b, 0x48, 0x22, 0x7b, 0xae, 0x9d, 0x8d, 0x48, 0x22, 0x3c, 0xd6, 0x4f, - 0x42, 0x8d, 0xb3, 0x07, 0x56, 0x64, 0x2b, 0x5b, 0x95, 0xb3, 0x07, 0x46, 0xd4, 0xe3, 0xb0, 0x7c, - 0x76, 0xb9, 0x2d, 0x91, 0xfc, 0x6f, 0x8b, 0x99, 0xd2, 0xae, 0x4c, 0xc7, 0xc5, 0x43, 0x16, 0x98, - 0x7b, 0xca, 0x86, 0xbf, 0x91, 0x63, 0xb7, 0x27, 0x69, 0x2f, 0x9a, 0xff, 0xde, 0xf5, 0x28, 0xda, - 0x12, 0xc9, 0x24, 0xe5, 0xe4, 0x32, 0x54, 0x42, 0x1c, 0xb9, 0x1c, 0x6d, 0xda, 0x46, 0x63, 0x4b, - 0x24, 0xdb, 0xec, 0xae, 0xef, 0x64, 0xe4, 0x39, 0xe8, 0xc4, 0x58, 0x4e, 0x82, 0x4c, 0x28, 0xbc, - 0x67, 0x91, 0x41, 0xd9, 0x6f, 0x5b, 0x78, 0xdf, 0xa1, 0xa7, 0x77, 0xc3, 0x67, 0x59, 0x42, 0x43, - 0xb6, 0xcd, 0xee, 0x92, 0x55, 0x28, 0x46, 0xec, 0xae, 0xfb, 0x46, 0xdb, 0x35, 0x33, 0x46, 0xc7, - 0x7c, 0xc5, 0x88, 0x7a, 0xef, 0xcc, 0x5b, 0x6e, 0x4b, 0x91, 0x39, 0x82, 0x97, 0xa0, 0x91, 0x88, - 0x51, 0x1c, 0xd2, 0x24, 0x88, 0xa3, 0x63, 0x97, 0xaf, 0xe0, 0xa0, 0x7e, 0x74, 0xfc, 0x88, 0xef, - 0x8b, 0x8f, 0xfa, 0xfe, 0xb0, 0x04, 0xad, 0x79, 0x5a, 0xf7, 0x4f, 0x5d, 0x06, 0x85, 0xd3, 0x97, - 0xc1, 0xb4, 0x17, 0x59, 0x9c, 0xeb, 0x45, 0x7a, 0x50, 0x1a, 0xc7, 0xdc, 0x5e, 0x0d, 0x79, 0xd6, - 0xe2, 0x8a, 0xdf, 0x8d, 0x79, 0xe4, 0xa3, 0x8c, 0xbc, 0x0a, 0x40, 0xa3, 0x28, 0x70, 0xe1, 0x2c, - 0xa1, 0xab, 0xdd, 0x99, 0xe6, 0xe9, 0xc0, 0xef, 0x2c, 0xf8, 0x75, 0x3a, 0xdd, 0x85, 0x6b, 0xd0, - 0x88, 0xa4, 0xc8, 0x72, 0xdb, 0x32, 0xda, 0x3e, 0x79, 0xc6, 0x76, 0x16, 0x94, 0x9d, 0x05, 0x1f, - 0xa2, 0x59, 0x88, 0x5e, 0x87, 0xa6, 0xc4, 0x04, 0x0a, 0x6c, 0x5b, 0x50, 0x41, 0xf3, 0x95, 0x33, - 0xe6, 0x73, 0x29, 0xbb, 0xb3, 0xe0, 0x37, 0xe4, 0x5c, 0x06, 0xbf, 0x0e, 0xed, 0x09, 0xde, 0x0a, - 0x41, 0x9e, 0xfb, 0xf6, 0x22, 0xba, 0x70, 0x66, 0x09, 0x77, 0x48, 0x76, 0x16, 0xfc, 0x96, 0xd5, - 0xcf, 0x4f, 0xcd, 0x35, 0x68, 0xe4, 0x0b, 0x28, 0x2d, 0xf1, 0x76, 0x7a, 0x94, 0xff, 0xec, 0x70, - 0x1a, 0xfe, 0x6e, 0x01, 0xa5, 0x25, 0xb9, 0x06, 0x6e, 0xb9, 0x20, 0xc3, 0x5a, 0xd5, 0xad, 0xa3, - 0xfd, 0xf9, 0x33, 0xf6, 0xb6, 0x90, 0xed, 0x2c, 0xf8, 0x4d, 0xab, 0xed, 0x0a, 0xdb, 0xab, 0x00, - 0xce, 0xfb, 0x50, 0x24, 0xdd, 0xc6, 0x63, 0xc3, 0x3e, 0x3d, 0x5f, 0x26, 0xec, 0x72, 0x7a, 0xd8, - 0xae, 0x41, 0x43, 0xda, 0x1c, 0x0d, 0x4c, 0x76, 0x36, 0x1f, 0x4b, 0x7b, 0x96, 0xc5, 0x86, 0xb6, - 0x9c, 0xce, 0x36, 0x1b, 0x50, 0x17, 0x19, 0x93, 0xd8, 0x39, 0xf5, 0x3e, 0x2e, 0x41, 0xe3, 0x20, - 0x3c, 0x64, 0x29, 0x7d, 0xe3, 0x58, 0x4b, 0x4a, 0x9e, 0x85, 0x0e, 0x67, 0xc7, 0xda, 0x70, 0xca, - 0x9b, 0x47, 0x9b, 0xba, 0x2d, 0x03, 0x6f, 0x89, 0xc4, 0x36, 0x8f, 0xd8, 0x3a, 0x48, 0x91, 0x65, - 0x2c, 0x0a, 0x6c, 0x43, 0x6d, 0xda, 0x2e, 0xd3, 0x3a, 0x58, 0xf0, 0xba, 0xeb, 0xa8, 0xdb, 0x36, - 0x33, 0x82, 0xf0, 0x90, 0xf2, 0x11, 0x8b, 0x5c, 0xaf, 0xdf, 0xb2, 0xe8, 0x96, 0x05, 0x4f, 0xd5, - 0x8e, 0xd2, 0xe9, 0xda, 0xf1, 0x19, 0xd5, 0xbf, 0xfc, 0xdf, 0x57, 0xff, 0xca, 0x17, 0xa8, 0xfe, - 0xd5, 0xff, 0x58, 0xfd, 0x6b, 0x5f, 0xb8, 0xfa, 0xd7, 0x1f, 0x57, 0xfd, 0x0d, 0xcf, 0x61, 0x22, - 0xc2, 0x71, 0x60, 0x78, 0x48, 0xf1, 0x40, 0x75, 0xc1, 0xf2, 0x44, 0x74, 0x97, 0x1e, 0xfb, 0xe2, - 0x81, 0x22, 0x57, 0x60, 0x49, 0x60, 0xcb, 0x82, 0x6a, 0x28, 0x52, 0x98, 0x29, 0x2d, 0xbf, 0x63, - 0x05, 0xbb, 0xf4, 0x78, 0x13, 0x61, 0x73, 0x6f, 0xdc, 0x60, 0x54, 0x4f, 0x24, 0xbb, 0x91, 0xd0, - 0x11, 0xe6, 0x44, 0xc9, 0x9f, 0x87, 0x8c, 0x46, 0x9f, 0x47, 0xec, 0x18, 0xb3, 0x43, 0x75, 0x5b, - 0xab, 0x45, 0xa3, 0x31, 0x07, 0x91, 0xcb, 0xd0, 0xda, 0xa7, 0x92, 0x71, 0xed, 0x9a, 0xf2, 0x6e, - 0x1b, 0x57, 0x39, 0x0d, 0x92, 0x35, 0xe8, 0xdc, 0x90, 0x22, 0xdd, 0x9f, 0x0c, 0xcd, 0xbb, 0x13, - 0x8b, 0x6a, 0x07, 0xb7, 0xf5, 0x2c, 0xdc, 0x8b, 0xa0, 0xd6, 0xe7, 0xfa, 0x5b, 0x2f, 0xef, 0xd2, - 0x8c, 0xf4, 0xa0, 0x90, 0xba, 0xe6, 0xda, 0xb6, 0xbc, 0xb9, 0x64, 0x7d, 0xd7, 0xb6, 0xd9, 0x85, - 0x74, 0xe5, 0x65, 0xa8, 0xd8, 0x89, 0x79, 0xd6, 0x8d, 0xd9, 0x09, 0xa6, 0x5e, 0xd1, 0x37, 0x43, - 0x72, 0x0e, 0xca, 0x47, 0x34, 0x99, 0xd8, 0x2b, 0xa4, 0xe8, 0xdb, 0xc9, 0x6b, 0x8b, 0xaf, 0x14, - 0x7a, 0x6f, 0x41, 0x73, 0x20, 0x29, 0x57, 0xdb, 0x4c, 0x99, 0x82, 0x4e, 0x2e, 0x40, 0x45, 0x0c, - 0xef, 0xf5, 0x5d, 0xd1, 0x2d, 0xfb, 0x6e, 0x66, 0xf0, 0x61, 0x32, 0x36, 0xb8, 0xbd, 0x03, 0xdc, - 0xcc, 0xe0, 0x52, 0x3c, 0x30, 0x78, 0xd1, 0xe2, 0x76, 0xd6, 0xfb, 0x41, 0x01, 0x1a, 0x9b, 0xc9, - 0x18, 0xd7, 0x36, 0x1e, 0xbc, 0x30, 0xf3, 0xe0, 0x09, 0xdb, 0xba, 0xcc, 0x84, 0xce, 0x09, 0xf7, - 0x50, 0x2c, 0xa4, 0x2b, 0x37, 0x1f, 0xe7, 0x4a, 0xd9, 0xba, 0xf2, 0xdc, 0xbc, 0x2b, 0x8d, 0x8d, - 0x25, 0xfb, 0x0e, 0x9a, 0x73, 0x61, 0xde, 0xbb, 0x1d, 0x20, 0xf9, 0x77, 0xee, 0x32, 0xb9, 0x29, - 0xc4, 0x38, 0xe6, 0x23, 0xb2, 0x01, 0xb5, 0x94, 0x66, 0x59, 0xcc, 0x47, 0xca, 0x51, 0xf2, 0xce, - 0x52, 0x72, 0x5c, 0xa6, 0x7a, 0xbd, 0x8f, 0x16, 0xc1, 0xc3, 0x0c, 0xdc, 0xc2, 0xf7, 0x8f, 0x65, - 0xf7, 0xd8, 0x17, 0xec, 0x79, 0xa8, 0xe8, 0x61, 0x32, 0xbb, 0x4b, 0xca, 0x7a, 0x98, 0x3c, 0xf2, - 0x9a, 0x28, 0x9e, 0x7d, 0x4d, 0x7c, 0x13, 0x6a, 0x4a, 0x53, 0xa9, 0x03, 0xec, 0x92, 0x3e, 0xb3, - 0x17, 0x74, 0xbc, 0xaa, 0xa8, 0x3b, 0x50, 0xe6, 0xa2, 0x9c, 0x1d, 0x41, 0xd5, 0x2d, 0xaf, 0x16, - 0xd7, 0x9a, 0x3e, 0xa4, 0xf9, 0xd9, 0x53, 0xf8, 0xfe, 0x93, 0x8c, 0xea, 0x5c, 0xa3, 0x82, 0x1a, - 0x0d, 0x87, 0xa1, 0xca, 0x37, 0xa0, 0x3a, 0xb4, 0x91, 0x71, 0x37, 0xc0, 0xe9, 0x0d, 0x9a, 0x05, - 0xce, 0xcf, 0xf5, 0xcc, 0x67, 0xdd, 0xd0, 0xbc, 0x2c, 0xf1, 0x60, 0xd7, 0x7d, 0x70, 0xd0, 0x2d, - 0x11, 0x9a, 0x7d, 0x63, 0x52, 0xe2, 0xf9, 0xad, 0xfb, 0x66, 0x68, 0x52, 0x30, 0x61, 0x47, 0x2c, - 0xc1, 0xb3, 0x5a, 0xf6, 0xed, 0xa4, 0xf7, 0xb3, 0x45, 0x68, 0x63, 0x58, 0x07, 0x54, 0x8d, 0xff, - 0xef, 0x41, 0x9d, 0x7b, 0xfd, 0x97, 0x4e, 0xbd, 0xfe, 0x7b, 0xd0, 0xd2, 0xc2, 0x15, 0x9a, 0xb9, - 0xc0, 0x35, 0xb4, 0x40, 0x32, 0x18, 0x96, 0x75, 0x58, 0x66, 0x4a, 0xc7, 0x29, 0xc6, 0x2e, 0x65, - 0x69, 0x30, 0x51, 0x74, 0x64, 0xef, 0xd9, 0x92, 0xbf, 0x34, 0x15, 0xed, 0xb2, 0xf4, 0x8e, 0x11, - 0x18, 0x2e, 0x34, 0x0c, 0xc5, 0x84, 0x6b, 0x43, 0xd3, 0x56, 0xc3, 0xba, 0x43, 0xec, 0x2f, 0x11, - 0x13, 0xc5, 0xa4, 0x91, 0xd5, 0x50, 0x56, 0x31, 0x53, 0x2b, 0x90, 0xc2, 0x36, 0x25, 0x75, 0x2b, - 0x30, 0xd3, 0x7e, 0xd4, 0xbb, 0x0d, 0xed, 0xd9, 0x33, 0x0b, 0x1f, 0xf3, 0x2b, 0x50, 0xbb, 0x75, - 0xfa, 0x21, 0x3f, 0x9d, 0x9b, 0x02, 0xa5, 0xe5, 0x84, 0x87, 0x54, 0xb3, 0x5b, 0x8a, 0xbb, 0x30, - 0xcd, 0x43, 0x57, 0x7e, 0x5c, 0x84, 0xca, 0x5e, 0xb6, 0x25, 0x22, 0x46, 0xaa, 0x50, 0xbc, 0x2d, - 0x32, 0x6f, 0x81, 0x2c, 0x41, 0x73, 0x2f, 0xbb, 0xc9, 0xb4, 0xfb, 0xc9, 0xc0, 0xfb, 0x47, 0x95, - 0x78, 0xd0, 0xd8, 0xcb, 0xf6, 0xa5, 0x4b, 0x74, 0xef, 0x9f, 0x55, 0xd2, 0x30, 0x76, 0xfb, 0x31, - 0x1f, 0x79, 0x1f, 0x76, 0x48, 0x13, 0xaa, 0x7b, 0xd9, 0x8d, 0x64, 0xa2, 0x0e, 0xbd, 0x5f, 0x77, - 0xac, 0xfd, 0x8c, 0xa5, 0xf7, 0x9b, 0x0e, 0x69, 0x43, 0x7d, 0x2f, 0xeb, 0x73, 0x95, 0x99, 0xd7, - 0xe2, 0x47, 0x1d, 0x72, 0x0e, 0x3a, 0x7b, 0xd9, 0xf5, 0x28, 0xba, 0x41, 0x27, 0x89, 0xde, 0x47, - 0xad, 0xdf, 0x76, 0x48, 0x0b, 0x6a, 0x7b, 0xd9, 0x26, 0x0d, 0xc7, 0x93, 0xcc, 0xfb, 0x5d, 0xc7, - 0x7e, 0x74, 0x20, 0x69, 0xc8, 0x0e, 0x32, 0xca, 0xbd, 0xdf, 0x77, 0xc8, 0x32, 0xb4, 0xf7, 0xb2, - 0x03, 0x2d, 0x24, 0x1d, 0x31, 0x0c, 0xb0, 0xf7, 0x87, 0x0e, 0x79, 0x02, 0xc8, 0x5e, 0x76, 0x33, - 0x11, 0x43, 0x9a, 0xcc, 0x7d, 0xf4, 0x8f, 0x1d, 0x72, 0x01, 0x96, 0xcc, 0x47, 0x35, 0x93, 0x21, - 0xcb, 0xb4, 0xa3, 0xfe, 0x71, 0x87, 0x10, 0x68, 0x19, 0x97, 0xcd, 0x14, 0x77, 0xd6, 0xfb, 0x93, - 0xd3, 0xdd, 0x8e, 0xd5, 0xd8, 0xfc, 0x6d, 0x25, 0x8c, 0x72, 0x26, 0xbd, 0x3f, 0x3b, 0x4a, 0x3e, - 0xa3, 0x11, 0x93, 0xde, 0x5f, 0x3a, 0x64, 0x05, 0xce, 0xdb, 0xd0, 0x50, 0xcd, 0x94, 0x9e, 0xfb, - 0xdc, 0x27, 0x39, 0x39, 0x4e, 0x33, 0x75, 0x28, 0xb4, 0x31, 0xf1, 0xfe, 0x3a, 0x33, 0x70, 0x37, - 0x33, 0x56, 0xfc, 0x5b, 0xb1, 0xd2, 0xde, 0x43, 0xc7, 0x03, 0x23, 0xd0, 0xe7, 0xf8, 0x8c, 0xfe, - 0x5b, 0xe7, 0xca, 0x2f, 0x0b, 0x50, 0x9f, 0xb6, 0x93, 0xa4, 0x01, 0xd5, 0x3e, 0x3f, 0xa2, 0x49, - 0x1c, 0x79, 0x0b, 0xa4, 0x05, 0xf5, 0x69, 0xd3, 0xe8, 0x15, 0xf0, 0x59, 0x3e, 0xed, 0xfc, 0xbc, - 0x45, 0xd2, 0x81, 0xc6, 0x5c, 0x63, 0x67, 0x9f, 0xf2, 0x77, 0xe6, 0x7b, 0x33, 0xaf, 0x44, 0xce, - 0x81, 0x97, 0x43, 0x79, 0x07, 0xe6, 0x95, 0x89, 0x07, 0xcd, 0x3b, 0x73, 0x7d, 0x94, 0x57, 0x31, - 0xc8, 0xf5, 0x28, 0xda, 0xcf, 0x7f, 0x4a, 0xf3, 0x4c, 0x02, 0x34, 0xa7, 0xad, 0x93, 0xf9, 0x5e, - 0xcd, 0x7c, 0x7f, 0xd6, 0x10, 0x79, 0xf5, 0x2b, 0x37, 0xa1, 0x3e, 0xbd, 0xbf, 0x49, 0x0d, 0x4a, - 0xd7, 0x27, 0x5a, 0x58, 0xd6, 0xb7, 0x85, 0xfd, 0x2d, 0x41, 0x79, 0x05, 0xd2, 0x84, 0xda, 0x66, - 0x3c, 0xb2, 0x14, 0x17, 0xc9, 0x32, 0x74, 0xb6, 0x04, 0xd7, 0x31, 0x9f, 0x88, 0x89, 0xc2, 0x9f, - 0x8f, 0xbc, 0xe2, 0xe6, 0x77, 0x3e, 0xf8, 0xf4, 0x62, 0xe1, 0xc3, 0x4f, 0x2f, 0x16, 0x1e, 0x7e, - 0x7a, 0x71, 0xe1, 0xbd, 0xbf, 0x5f, 0x2c, 0xbc, 0xf3, 0xf5, 0xb9, 0x1f, 0xa6, 0x53, 0xaa, 0x65, - 0x7c, 0x2c, 0x64, 0x3c, 0x8a, 0x79, 0x3e, 0xe1, 0xec, 0x6a, 0x36, 0x1e, 0x5d, 0xcd, 0x86, 0x57, - 0x69, 0x16, 0x0f, 0x2b, 0xf8, 0x0b, 0xf4, 0x4b, 0xff, 0x0e, 0x00, 0x00, 0xff, 0xff, 0x84, 0x64, - 0x0a, 0xc7, 0xdf, 0x16, 0x00, 0x00, + // 2633 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x58, 0x4b, 0x6f, 0x1c, 0xc7, + 0x11, 0xe6, 0x72, 0xdf, 0x35, 0xfb, 0x18, 0xb6, 0x28, 0x6b, 0x4d, 0x3b, 0x12, 0xb3, 0x96, 0x6d, + 0x5a, 0x8e, 0x29, 0x84, 0xb6, 0x13, 0xdb, 0x10, 0x6c, 0x90, 0x4b, 0x4b, 0xdc, 0x84, 0xd4, 0x32, + 0xc3, 0x95, 0x0d, 0x18, 0x01, 0x06, 0xbd, 0x33, 0xad, 0xe5, 0x68, 0x67, 0xba, 0x47, 0xdd, 0xbd, + 0x14, 0xe9, 0x6b, 0xe2, 0x63, 0x2e, 0xb9, 0xe5, 0x66, 0xff, 0x83, 0x04, 0x08, 0x90, 0x7f, 0x10, + 0xf8, 0xe8, 0x20, 0xce, 0x3b, 0x71, 0x14, 0xe7, 0x92, 0xe4, 0x57, 0x04, 0xfd, 0x98, 0xdd, 0x21, + 0x45, 0x39, 0x71, 0x10, 0xc0, 0x07, 0x12, 0xdd, 0x5f, 0x55, 0xf5, 0x7c, 0x55, 0x5d, 0x5d, 0x5d, + 0xbd, 0x50, 0xc7, 0x69, 0xb4, 0x9e, 0x72, 0x26, 0x19, 0x2a, 0xe2, 0x34, 0x5a, 0x79, 0x69, 0x1c, + 0xc9, 0xc3, 0xe9, 0x68, 0x3d, 0x60, 0xc9, 0xf5, 0x31, 0x1b, 0xb3, 0xeb, 0x5a, 0x36, 0x9a, 0xde, + 0xd5, 0x33, 0x3d, 0xd1, 0x23, 0x63, 0xb3, 0xd2, 0x96, 0x51, 0x42, 0x84, 0xc4, 0x49, 0x6a, 0x01, + 0x48, 0x63, 0x4c, 0xcd, 0xb8, 0xfb, 0x6d, 0x68, 0x0e, 0x6f, 0xef, 0x47, 0x74, 0xec, 0x91, 0xfb, + 0x53, 0x22, 0x24, 0x7a, 0x1a, 0xea, 0x29, 0xe6, 0x38, 0x21, 0x92, 0xf0, 0x4e, 0x61, 0xb5, 0xb0, + 0x56, 0xf7, 0xe6, 0xc0, 0x1b, 0xb5, 0x0f, 0x3f, 0xba, 0x52, 0x78, 0xf8, 0xd1, 0x95, 0x85, 0xee, + 0xcf, 0x0b, 0xd0, 0xca, 0x2c, 0x45, 0xca, 0xa8, 0x20, 0xa8, 0x03, 0x55, 0x21, 0x19, 0x27, 0xfd, + 0x6d, 0x6b, 0x98, 0x4d, 0xd1, 0x73, 0xd0, 0x12, 0x84, 0x1f, 0x45, 0x01, 0xd9, 0x0c, 0x43, 0x4e, + 0x84, 0xe8, 0x2c, 0x6a, 0x85, 0x33, 0xa8, 0x5e, 0xe1, 0x10, 0xf3, 0xb0, 0xbf, 0xdd, 0x29, 0xae, + 0x16, 0xd6, 0x4a, 0x5e, 0x36, 0x55, 0xb4, 0x38, 0x49, 0xe3, 0x28, 0xc0, 0xfd, 0xed, 0x4e, 0x49, + 0xcb, 0xe6, 0x00, 0xba, 0x0c, 0x10, 0xb3, 0xf1, 0x81, 0x35, 0x2d, 0x6b, 0x71, 0x0e, 0xc9, 0xd1, + 0x7e, 0x03, 0xdc, 0xe1, 0xed, 0x03, 0xc9, 0xf3, 0xbc, 0xf5, 0xda, 0x72, 0xca, 0xe9, 0x81, 0x9c, + 0xb9, 0x3c, 0x03, 0x72, 0xb6, 0x3f, 0x2b, 0x40, 0xe5, 0x1d, 0x12, 0x48, 0xc6, 0x11, 0x82, 0x52, + 0x88, 0x25, 0xd6, 0xda, 0x0d, 0x4f, 0x8f, 0xd1, 0x55, 0x28, 0xc9, 0x93, 0x94, 0x68, 0xd7, 0x9c, + 0x0d, 0x58, 0xd7, 0x51, 0x1e, 0x9e, 0xa4, 0x64, 0xab, 0xf4, 0xf1, 0x67, 0x57, 0x16, 0x3c, 0x2d, + 0x45, 0x2b, 0x50, 0xa3, 0xd3, 0x38, 0xc6, 0xa3, 0x98, 0x68, 0x1f, 0x6b, 0xde, 0x6c, 0x8e, 0x5c, + 0x28, 0x52, 0x91, 0x6a, 0xf7, 0x1a, 0x9e, 0x1a, 0xa2, 0x27, 0xa1, 0x16, 0x09, 0x3f, 0x60, 0x54, + 0x48, 0xed, 0x56, 0xcd, 0xab, 0x46, 0xa2, 0xa7, 0xa6, 0x4a, 0x39, 0x26, 0xb4, 0x53, 0x59, 0x2d, + 0xac, 0x35, 0x3d, 0x35, 0x54, 0xa4, 0x30, 0x27, 0xb8, 0x53, 0x35, 0xa4, 0xd4, 0xb8, 0xfb, 0x1d, + 0x28, 0x6f, 0x61, 0x19, 0x1c, 0xa2, 0x15, 0x28, 0x63, 0x29, 0xb9, 0xe8, 0x14, 0x56, 0x8b, 0x6b, + 0x75, 0x4b, 0xc9, 0x40, 0xe8, 0x59, 0x28, 0x1d, 0x91, 0x40, 0x6d, 0x4a, 0x71, 0xcd, 0xd9, 0x70, + 0xd6, 0x55, 0xbe, 0x19, 0x47, 0x33, 0xea, 0x4a, 0xdc, 0xfd, 0x65, 0x01, 0xaa, 0x43, 0x45, 0xb4, + 0xbf, 0x8d, 0x2e, 0x40, 0x39, 0x1c, 0xf9, 0x51, 0xa8, 0x23, 0x50, 0xf2, 0x4a, 0xe1, 0xa8, 0x1f, + 0x2a, 0x50, 0x6a, 0x70, 0xd1, 0x80, 0x52, 0x81, 0x5f, 0x87, 0x46, 0x8a, 0xb9, 0x8c, 0x64, 0xc4, + 0xa8, 0x92, 0x99, 0x8d, 0x75, 0x66, 0x58, 0x3f, 0x44, 0x17, 0xa1, 0x82, 0x83, 0x40, 0x09, 0x4b, + 0xda, 0x9b, 0x32, 0x0e, 0x82, 0x7e, 0x88, 0x2e, 0x41, 0x35, 0x1c, 0xf9, 0x14, 0x27, 0x44, 0xfb, + 0x5e, 0xf7, 0x2a, 0xe1, 0xe8, 0x36, 0x4e, 0x88, 0x12, 0x48, 0x2b, 0xa8, 0x18, 0x81, 0x34, 0x82, + 0x67, 0xa1, 0x95, 0xf2, 0x28, 0xc1, 0xfc, 0xc4, 0x17, 0xe4, 0x3e, 0x9d, 0x26, 0x3a, 0x16, 0x4d, + 0xaf, 0x69, 0xd1, 0x03, 0x0d, 0x76, 0x7f, 0x5c, 0x80, 0xd6, 0xc1, 0x09, 0x0d, 0x76, 0xd9, 0x78, + 0x88, 0xa3, 0xd8, 0x23, 0xf7, 0xd1, 0x4b, 0x50, 0x0d, 0xa8, 0x7f, 0x88, 0x8f, 0x88, 0xf6, 0xc8, + 0xd9, 0x58, 0x5e, 0x9f, 0x1f, 0x9b, 0x61, 0x36, 0xf2, 0x2a, 0x01, 0xdd, 0xc1, 0x47, 0xc4, 0xaa, + 0x3f, 0xc0, 0x54, 0xda, 0xed, 0x7e, 0xac, 0xfa, 0xbb, 0x98, 0x4a, 0xd4, 0x85, 0xb2, 0x9c, 0xed, + 0xb8, 0xb3, 0xd1, 0xd0, 0x11, 0xb6, 0xa1, 0xf4, 0x8c, 0xa8, 0xfb, 0x7d, 0x68, 0x9f, 0xe2, 0x24, + 0x52, 0x15, 0xba, 0x60, 0x92, 0xfa, 0x31, 0x0b, 0xb0, 0x8a, 0x94, 0xcd, 0x4d, 0x27, 0x98, 0xa4, + 0xbb, 0x16, 0x42, 0xcf, 0x41, 0x2d, 0x60, 0x49, 0x82, 0x69, 0x98, 0x6d, 0x1f, 0xe8, 0xc5, 0xdf, + 0xa6, 0x92, 0x9f, 0x78, 0x33, 0x59, 0xf7, 0x4d, 0x58, 0xda, 0xe7, 0x44, 0x4d, 0x23, 0xf9, 0x2e, + 0x8f, 0x24, 0xe9, 0x25, 0x21, 0x7a, 0x01, 0x80, 0x28, 0x3d, 0x3f, 0x8e, 0x84, 0xd4, 0x89, 0x71, + 0xda, 0xbc, 0xae, 0xa5, 0xbb, 0x91, 0x90, 0xdd, 0x0f, 0x4a, 0x50, 0xd6, 0x20, 0x7a, 0x39, 0x33, + 0xd2, 0xc9, 0xae, 0x28, 0xb5, 0x36, 0x96, 0xe7, 0x46, 0xe6, 0xbf, 0x4a, 0x7b, 0x6b, 0xae, 0x86, + 0x2a, 0x8f, 0xb5, 0x97, 0xf3, 0xe4, 0xa8, 0xea, 0x79, 0x3f, 0x44, 0x57, 0xc0, 0x51, 0xc7, 0x67, + 0x84, 0x05, 0x99, 0xa7, 0x07, 0x64, 0x50, 0x3f, 0x44, 0x5f, 0x03, 0x30, 0xb6, 0x7a, 0xc3, 0x4b, + 0xe6, 0x7c, 0x6a, 0x44, 0xef, 0xf9, 0x33, 0xd0, 0x9c, 0xd9, 0xe7, 0x72, 0xa5, 0x91, 0x81, 0x5a, + 0xe9, 0x29, 0xa8, 0xdf, 0x8d, 0xb2, 0x25, 0x4c, 0xce, 0xd4, 0x14, 0xa0, 0x85, 0x4f, 0x43, 0x71, + 0x84, 0xa5, 0x4e, 0x95, 0xcc, 0x7f, 0x7d, 0x66, 0x3c, 0x05, 0xa3, 0x67, 0xa0, 0x95, 0x4e, 0xfc, + 0xe0, 0x90, 0x04, 0x13, 0x7f, 0x74, 0xe2, 0x4b, 0xda, 0xa9, 0xad, 0x16, 0xd6, 0xca, 0x9e, 0x93, + 0x4e, 0x7a, 0x0a, 0xdc, 0x3a, 0x19, 0x52, 0xf4, 0x2a, 0x5c, 0x8a, 0xf1, 0xfb, 0x27, 0x7e, 0x80, + 0x25, 0x8e, 0xd9, 0xd8, 0xc7, 0x41, 0xc0, 0xa6, 0x54, 0x2a, 0x87, 0xea, 0x3a, 0x03, 0x97, 0x95, + 0xb8, 0x67, 0xa4, 0x9b, 0x46, 0xd8, 0x0f, 0xd1, 0x0d, 0x78, 0xea, 0x10, 0x0b, 0xff, 0x71, 0xa6, + 0xa0, 0x4f, 0xfc, 0xa5, 0x43, 0x2c, 0x76, 0xcf, 0xb1, 0xee, 0x72, 0xa8, 0xcf, 0x82, 0x8d, 0x00, + 0x2a, 0x7d, 0x2a, 0x08, 0x97, 0xee, 0x82, 0x1a, 0x6f, 0x93, 0x98, 0x48, 0xe2, 0x16, 0xd4, 0xf8, + 0x4e, 0x1a, 0x62, 0x49, 0xdc, 0x45, 0x54, 0x87, 0xf2, 0x66, 0x2c, 0x09, 0x77, 0x8b, 0x68, 0x09, + 0x9a, 0x07, 0x29, 0x09, 0x22, 0x1c, 0x5b, 0xcd, 0x12, 0x6a, 0x01, 0x6c, 0x63, 0x89, 0x07, 0xa3, + 0x7b, 0x24, 0x90, 0x6e, 0x19, 0x5d, 0x80, 0xf6, 0x90, 0x25, 0x23, 0x21, 0x19, 0x25, 0x16, 0xac, + 0x74, 0x7f, 0x58, 0x00, 0xd0, 0x6e, 0xa7, 0x2c, 0xa2, 0x12, 0xbd, 0x08, 0x95, 0x24, 0xa2, 0xbe, + 0x14, 0x5f, 0x78, 0x6a, 0xca, 0x49, 0x44, 0x87, 0x42, 0x2b, 0xe3, 0x63, 0xa5, 0xbc, 0xf8, 0x85, + 0xca, 0xf8, 0x78, 0x28, 0xb2, 0x4d, 0x29, 0x9e, 0xbb, 0x29, 0x86, 0x86, 0x89, 0x47, 0x6f, 0x92, + 0x7e, 0x65, 0x34, 0x3e, 0x28, 0x80, 0xb3, 0x47, 0x24, 0x56, 0xb9, 0xf6, 0x55, 0xf2, 0xf8, 0x57, + 0x01, 0x5c, 0xbd, 0xb3, 0xba, 0xa6, 0xec, 0xb3, 0x38, 0x0a, 0x4e, 0xd0, 0x3a, 0x5c, 0x50, 0x64, + 0x98, 0x88, 0xde, 0x27, 0xfe, 0xfd, 0x29, 0x8e, 0xe2, 0xe8, 0x2e, 0x31, 0x05, 0xbb, 0xe9, 0x2d, + 0x25, 0x11, 0x1d, 0x28, 0xc9, 0xf7, 0x32, 0x01, 0xba, 0x0a, 0x2d, 0xc5, 0x87, 0x8d, 0xee, 0xf9, + 0x8c, 0x12, 0x3e, 0xa5, 0x9a, 0x57, 0xd3, 0x6b, 0x24, 0xf8, 0x78, 0x30, 0xba, 0x37, 0xd0, 0x18, + 0xba, 0x0e, 0xcb, 0x5a, 0x4b, 0xaf, 0x9a, 0x10, 0x3e, 0x26, 0xa1, 0x32, 0xd1, 0xcc, 0xd4, 0xb2, + 0xf8, 0x58, 0x2f, 0xbb, 0xa7, 0x25, 0x83, 0xd1, 0x3d, 0x74, 0x15, 0xca, 0x87, 0x11, 0x95, 0xa2, + 0x53, 0x5a, 0x2d, 0xae, 0xb5, 0x36, 0x5a, 0x9a, 0xbb, 0x16, 0xef, 0x44, 0x54, 0x7a, 0x46, 0x88, + 0x5e, 0x00, 0xc5, 0xc8, 0x0f, 0xa8, 0x59, 0xd3, 0x57, 0x6b, 0xd8, 0x8b, 0xbc, 0x95, 0x44, 0xb4, + 0x47, 0xb5, 0xc5, 0x41, 0xf4, 0x3e, 0xe9, 0xbe, 0x06, 0xcb, 0x73, 0x5f, 0xf5, 0x5d, 0xc8, 0xb1, + 0xca, 0xc5, 0x55, 0x70, 0x82, 0xd9, 0x4c, 0xd8, 0xab, 0x39, 0x0f, 0x75, 0x5f, 0x82, 0xa5, 0xbc, + 0x65, 0x92, 0x10, 0x2a, 0x55, 0xcf, 0x11, 0x98, 0x61, 0xd6, 0xb5, 0xd8, 0x69, 0x77, 0x0f, 0x2e, + 0xce, 0xd5, 0x3d, 0xa2, 0x6a, 0x87, 0x1e, 0xaa, 0x6a, 0xc6, 0xe2, 0xd0, 0x14, 0x13, 0x6b, 0xc3, + 0xe2, 0x50, 0xd7, 0x92, 0x27, 0xa1, 0x46, 0xc9, 0x03, 0x23, 0x32, 0x3d, 0x4e, 0x95, 0x92, 0x07, + 0x4a, 0xd4, 0xa5, 0x70, 0xe1, 0xec, 0x72, 0x3d, 0x16, 0xff, 0x6f, 0x8b, 0xa9, 0xab, 0x41, 0xa8, + 0x8e, 0x8d, 0x06, 0xc4, 0x57, 0xf7, 0x9c, 0x09, 0xbf, 0x93, 0x61, 0xb7, 0xa7, 0x49, 0x37, 0xcc, + 0x7f, 0x6f, 0x33, 0x0c, 0x7b, 0x2c, 0x9e, 0x26, 0x14, 0x5d, 0x85, 0x4a, 0xa0, 0x47, 0x36, 0x47, + 0x1b, 0xa6, 0x51, 0xe9, 0xb1, 0x78, 0x9b, 0xdc, 0xf5, 0xac, 0x0c, 0x3d, 0x0f, 0xed, 0x48, 0x97, + 0x13, 0x3f, 0x65, 0x42, 0xdf, 0xd3, 0x9a, 0x41, 0xd9, 0x6b, 0x19, 0x78, 0xdf, 0xa2, 0xa7, 0x77, + 0xc3, 0x23, 0x69, 0x8c, 0x03, 0xb2, 0x4d, 0xee, 0xa2, 0x55, 0x28, 0x86, 0xe4, 0xae, 0xfd, 0x46, + 0xcb, 0x36, 0x43, 0x4a, 0x47, 0x7d, 0x45, 0x89, 0xba, 0xef, 0xe5, 0x2d, 0xb7, 0x39, 0x4b, 0x2d, + 0xc1, 0x2b, 0xe0, 0xc4, 0x6c, 0x1c, 0x05, 0x38, 0xf6, 0xa3, 0xf0, 0xd8, 0xe6, 0x2b, 0x58, 0xa8, + 0x1f, 0x1e, 0x3f, 0xe2, 0xfb, 0xe2, 0xa3, 0xbe, 0x3f, 0x2c, 0x41, 0x33, 0x4f, 0xeb, 0xfe, 0xa9, + 0x1b, 0xa8, 0x70, 0xfa, 0x06, 0x9a, 0xf5, 0x32, 0x8b, 0xb9, 0x5e, 0xa6, 0x0b, 0xa5, 0x49, 0x44, + 0xcd, 0x7d, 0x94, 0x65, 0xad, 0x5e, 0xf1, 0xbb, 0x11, 0x0d, 0x3d, 0x2d, 0x43, 0xaf, 0x03, 0xe0, + 0x30, 0xf4, 0x6d, 0x38, 0x4b, 0xda, 0xd5, 0xce, 0x5c, 0xf3, 0x74, 0xe0, 0x77, 0x16, 0xbc, 0x3a, + 0x9e, 0xed, 0xc2, 0x0d, 0x70, 0x42, 0xce, 0xd2, 0xcc, 0xb6, 0xac, 0x6d, 0x9f, 0x3c, 0x63, 0x3b, + 0x0f, 0xca, 0xce, 0x82, 0x07, 0xe1, 0x3c, 0x44, 0x6f, 0x41, 0x83, 0xeb, 0x04, 0xf2, 0x4d, 0x5b, + 0x51, 0xd1, 0xe6, 0x2b, 0x67, 0xcc, 0x73, 0x29, 0xbb, 0xb3, 0xe0, 0x39, 0x3c, 0x97, 0xc1, 0x6f, + 0x41, 0x6b, 0xaa, 0x6f, 0x05, 0x3f, 0xcb, 0x7d, 0x73, 0xfb, 0x3d, 0x71, 0x66, 0x09, 0x7b, 0x48, + 0x76, 0x16, 0xbc, 0xa6, 0xd1, 0xcf, 0x4e, 0xcd, 0x0d, 0x70, 0xb2, 0x05, 0x84, 0xe4, 0xfa, 0x4a, + 0x7c, 0x94, 0xff, 0xfc, 0x70, 0x2a, 0xfe, 0x76, 0x01, 0x21, 0x39, 0xba, 0x01, 0x76, 0x39, 0x3f, + 0xd5, 0xb5, 0x4a, 0x5f, 0x92, 0xce, 0xc6, 0xc5, 0x33, 0xf6, 0xa6, 0x90, 0xed, 0x2c, 0x78, 0x0d, + 0xa3, 0x6d, 0x0b, 0xdb, 0xeb, 0x00, 0xd6, 0xfb, 0x80, 0xc5, 0x1d, 0xe7, 0xdc, 0xb0, 0xcf, 0xce, + 0x97, 0x0a, 0x3b, 0x9f, 0x1d, 0xb6, 0x1b, 0xe0, 0x70, 0x93, 0xa3, 0xbe, 0xca, 0xce, 0xc6, 0xb9, + 0xb4, 0xe7, 0x59, 0xac, 0x68, 0xf3, 0xd9, 0x6c, 0xcb, 0x81, 0x3a, 0x4b, 0x09, 0xd7, 0x9d, 0x57, + 0xf7, 0xa7, 0x25, 0x70, 0x0e, 0x82, 0x43, 0x92, 0xe0, 0xb7, 0x8f, 0x25, 0xc7, 0xe8, 0x39, 0x68, + 0x53, 0x72, 0x2c, 0x15, 0xa7, 0xac, 0xf9, 0x34, 0xa9, 0xdb, 0x54, 0x70, 0x8f, 0xc5, 0xa6, 0xf9, + 0xd4, 0xfd, 0x0a, 0x67, 0x69, 0x4a, 0x42, 0xdf, 0x34, 0xe4, 0xaa, 0x6d, 0x53, 0xfd, 0x8a, 0x01, + 0x37, 0x6d, 0x47, 0xde, 0x32, 0x99, 0xe1, 0x07, 0x87, 0x98, 0x8e, 0x49, 0x68, 0xdf, 0x0a, 0x4d, + 0x83, 0xf6, 0x0c, 0x78, 0xaa, 0x76, 0x94, 0x4e, 0xd7, 0x8e, 0xc7, 0x54, 0xff, 0xf2, 0x7f, 0x5f, + 0xfd, 0x2b, 0x5f, 0xa2, 0xfa, 0x57, 0xff, 0x63, 0xf5, 0xaf, 0x7d, 0xe9, 0xea, 0x5f, 0x3f, 0xaf, + 0xfa, 0x2b, 0x9e, 0xa3, 0x98, 0x05, 0x13, 0x5f, 0xf1, 0xe0, 0xec, 0x81, 0xd0, 0x5d, 0x52, 0xd3, + 0x6b, 0x68, 0x74, 0x0f, 0x1f, 0x7b, 0xec, 0x81, 0x40, 0xd7, 0x60, 0x89, 0xe9, 0x96, 0x45, 0xab, + 0x69, 0x91, 0xd0, 0x99, 0xd2, 0xf4, 0xda, 0x46, 0xb0, 0x87, 0x8f, 0xb7, 0x34, 0xac, 0xee, 0x8d, + 0x9b, 0x04, 0xcb, 0x29, 0x27, 0x37, 0x63, 0x3c, 0xd6, 0x39, 0x51, 0xf2, 0xf2, 0x90, 0xd2, 0xe8, + 0xd3, 0x90, 0x1c, 0xeb, 0xec, 0x10, 0x9d, 0xe6, 0x6a, 0x51, 0x69, 0xe4, 0x20, 0x74, 0x15, 0x9a, + 0xfb, 0x98, 0x13, 0x2a, 0x6d, 0x53, 0xdf, 0x69, 0xe9, 0x55, 0x4e, 0x83, 0xdd, 0x10, 0x6a, 0x7d, + 0x2a, 0xbf, 0xf5, 0xca, 0x1e, 0x4e, 0x51, 0x17, 0x0a, 0x89, 0x6d, 0xb9, 0x4d, 0xf7, 0x9c, 0x49, + 0xd6, 0xf7, 0x4c, 0xf3, 0x5d, 0x48, 0x56, 0x5e, 0x81, 0x8a, 0x99, 0xa8, 0xc7, 0xde, 0x84, 0x9c, + 0xe8, 0x84, 0x2a, 0x7a, 0x6a, 0x88, 0x96, 0xa1, 0x7c, 0x84, 0xe3, 0xa9, 0xb9, 0x18, 0x8a, 0x9e, + 0x99, 0xbc, 0xb1, 0xf8, 0x5a, 0xa1, 0xfb, 0x0e, 0x34, 0x86, 0x1c, 0x53, 0xb1, 0x4d, 0x84, 0x2a, + 0xd3, 0xe8, 0x09, 0xa8, 0xb0, 0xd1, 0xbd, 0xbe, 0x2d, 0xa5, 0x65, 0xcf, 0xce, 0x14, 0x3e, 0x8a, + 0x27, 0x0a, 0x37, 0x95, 0xdd, 0xce, 0x14, 0xce, 0xd9, 0x03, 0x85, 0x17, 0x0d, 0x6e, 0x66, 0xdd, + 0x1f, 0x14, 0xc0, 0xd9, 0x8a, 0x27, 0x7a, 0x6d, 0xe5, 0xc1, 0x8b, 0x73, 0x0f, 0x2e, 0x99, 0x86, + 0x64, 0x2e, 0xb4, 0x4e, 0xd8, 0xe7, 0x63, 0x21, 0x59, 0xb9, 0x75, 0x9e, 0x2b, 0x65, 0xe3, 0xca, + 0xf3, 0x79, 0x57, 0x9c, 0x8d, 0x25, 0xf3, 0x3a, 0xca, 0xb9, 0x90, 0xf7, 0x6e, 0x07, 0x50, 0xf6, + 0x9d, 0xbb, 0x84, 0x6f, 0x31, 0x36, 0x89, 0xe8, 0x18, 0x6d, 0x40, 0x2d, 0xc1, 0x69, 0x1a, 0xd1, + 0xb1, 0xb0, 0x94, 0xdc, 0xb3, 0x94, 0x2c, 0x97, 0x99, 0x5e, 0xf7, 0xd3, 0x45, 0x70, 0x75, 0x5e, + 0xf5, 0xf4, 0xab, 0xc8, 0xb0, 0x3b, 0xf7, 0x5d, 0x7b, 0x11, 0x2a, 0x72, 0x14, 0xcf, 0x6f, 0x88, + 0xb2, 0x1c, 0xc5, 0x8f, 0x3c, 0x4c, 0x8a, 0x67, 0x1f, 0x26, 0xaf, 0x42, 0x4d, 0x48, 0xcc, 0xa5, + 0xaf, 0x7b, 0x9f, 0xc7, 0x76, 0x78, 0x96, 0x57, 0x55, 0xeb, 0x0e, 0x85, 0xba, 0xfe, 0xe6, 0x07, + 0x4b, 0x74, 0xca, 0xab, 0xc5, 0xb5, 0x86, 0x07, 0x49, 0x76, 0xa2, 0x84, 0x7e, 0x15, 0x72, 0x82, + 0x65, 0xa6, 0x51, 0xd1, 0x1a, 0x8e, 0xc5, 0xb4, 0xca, 0x37, 0xa1, 0x3a, 0x32, 0x91, 0xb1, 0x75, + 0xfd, 0xf4, 0x06, 0xcd, 0x03, 0xe7, 0x65, 0x7a, 0xea, 0xb3, 0x76, 0xa8, 0xde, 0x9b, 0xfa, 0xb8, + 0xd6, 0x3d, 0xb0, 0xd0, 0x2e, 0x0b, 0xd4, 0xbe, 0x11, 0xce, 0xf5, 0xa9, 0xac, 0x7b, 0x6a, 0xa8, + 0x52, 0x30, 0x26, 0x47, 0x24, 0xd6, 0x27, 0xb0, 0xec, 0x99, 0x49, 0xf7, 0x27, 0x8b, 0xd0, 0xd2, + 0x61, 0x1d, 0x62, 0x31, 0xf9, 0xbf, 0x07, 0x35, 0xf7, 0x9b, 0x40, 0xe9, 0xd4, 0x6f, 0x02, 0x5d, + 0x68, 0x4a, 0x66, 0xcb, 0x47, 0x2e, 0x70, 0x8e, 0x64, 0x9a, 0x8c, 0x0e, 0xcb, 0x3a, 0x5c, 0x20, + 0x42, 0x46, 0x89, 0x8e, 0x5d, 0x42, 0x12, 0x7f, 0x2a, 0xf0, 0xd8, 0xdc, 0x9e, 0x25, 0x6f, 0x69, + 0x26, 0xda, 0x23, 0xc9, 0x1d, 0x25, 0x50, 0x5c, 0x72, 0xaf, 0x31, 0x53, 0xe3, 0xea, 0x78, 0xf6, + 0x7a, 0xbb, 0x04, 0xd5, 0xa9, 0x20, 0x5c, 0xc9, 0x6a, 0x5a, 0x56, 0x51, 0x53, 0x23, 0xe0, 0xcc, + 0xb4, 0x1a, 0xe6, 0xf5, 0x57, 0x51, 0xd3, 0x7e, 0xd8, 0xbd, 0x0d, 0xad, 0xf9, 0xe3, 0x49, 0x3f, + 0xf1, 0x57, 0xa0, 0xb6, 0x7b, 0xfa, 0x79, 0x3f, 0x9b, 0xab, 0xb2, 0x23, 0xf9, 0x94, 0x06, 0x58, + 0x92, 0x5d, 0x41, 0x6d, 0x98, 0xf2, 0xd0, 0xb5, 0x1f, 0x15, 0xa1, 0x32, 0x48, 0x7b, 0x2c, 0x24, + 0xa8, 0x0a, 0xc5, 0xdb, 0x2c, 0x75, 0x17, 0xd0, 0x12, 0x34, 0x06, 0xe9, 0x2d, 0x22, 0xed, 0x0f, + 0x09, 0xee, 0x3f, 0xaa, 0xc8, 0x05, 0x67, 0x90, 0xee, 0x73, 0x9b, 0xe8, 0xee, 0x3f, 0xab, 0xc8, + 0x51, 0x76, 0xfb, 0x11, 0x1d, 0xbb, 0x9f, 0xb4, 0x51, 0x03, 0xaa, 0x83, 0xf4, 0x66, 0x3c, 0x15, + 0x87, 0xee, 0xaf, 0xda, 0xc6, 0x7e, 0xce, 0xd2, 0xfd, 0x75, 0x1b, 0xb5, 0xa0, 0x3e, 0x48, 0xfb, + 0x54, 0xa4, 0xea, 0x0d, 0xf8, 0x69, 0x1b, 0x2d, 0x43, 0x7b, 0x90, 0x6e, 0x86, 0xe1, 0x4d, 0x3c, + 0x8d, 0xe5, 0xbe, 0xd6, 0xfa, 0x4d, 0x1b, 0x35, 0xa1, 0x36, 0x48, 0xb7, 0x70, 0x30, 0x99, 0xa6, + 0xee, 0x6f, 0xdb, 0xe6, 0xa3, 0x43, 0x8e, 0x03, 0x72, 0x90, 0x62, 0xea, 0xfe, 0xae, 0x8d, 0x2e, + 0x40, 0x6b, 0x90, 0x1e, 0x48, 0xc6, 0xf1, 0x98, 0xe8, 0x00, 0xbb, 0xbf, 0x6f, 0xa3, 0x4b, 0x80, + 0x06, 0xe9, 0xad, 0x98, 0x8d, 0x70, 0x9c, 0xfb, 0xe8, 0x1f, 0xda, 0xe8, 0x09, 0x58, 0x52, 0x1f, + 0x95, 0x84, 0x07, 0x24, 0x95, 0x96, 0xfa, 0x1f, 0xdb, 0x08, 0x41, 0x53, 0xb9, 0xac, 0xa6, 0x7a, + 0x67, 0xdd, 0x3f, 0x59, 0xdd, 0xed, 0x48, 0x4c, 0xd4, 0x5f, 0x2f, 0x26, 0x98, 0x12, 0xee, 0xfe, + 0xd9, 0x52, 0xf2, 0x08, 0x0e, 0x09, 0x77, 0xff, 0xd2, 0x46, 0x2b, 0x70, 0xd1, 0x84, 0x06, 0x4b, + 0x22, 0x64, 0xee, 0x73, 0x9f, 0x65, 0xe4, 0x28, 0x4e, 0xc5, 0x21, 0x93, 0xca, 0xc4, 0xfd, 0xeb, + 0xdc, 0xc0, 0xde, 0xb7, 0xba, 0x8e, 0xef, 0x46, 0x42, 0xba, 0x0f, 0x2d, 0x0f, 0x1d, 0x81, 0x3e, + 0xd5, 0x8f, 0xe3, 0xbf, 0xb5, 0xaf, 0xfd, 0xa2, 0x00, 0xf5, 0x59, 0x93, 0x88, 0x1c, 0xa8, 0xf6, + 0xe9, 0x11, 0x8e, 0xa3, 0xd0, 0x5d, 0x40, 0x4d, 0xa8, 0xcf, 0x5a, 0x41, 0xb7, 0xa0, 0x1f, 0xdb, + 0xb3, 0x7e, 0xce, 0x5d, 0x44, 0x6d, 0x70, 0x72, 0xed, 0x9a, 0x79, 0xa0, 0xdf, 0xc9, 0x77, 0x5c, + 0x6e, 0x09, 0x2d, 0x83, 0x9b, 0x41, 0x59, 0x5f, 0xe5, 0x96, 0x91, 0x0b, 0x8d, 0x3b, 0xb9, 0xee, + 0xc8, 0xad, 0x28, 0x64, 0x33, 0x0c, 0xf7, 0xb3, 0x1f, 0xd8, 0x5c, 0x95, 0x00, 0x8d, 0x59, 0x43, + 0xa4, 0xbe, 0x57, 0x53, 0xdf, 0x9f, 0xb7, 0x39, 0x6e, 0xfd, 0xda, 0x2d, 0xa8, 0xcf, 0x6e, 0x65, + 0x54, 0x83, 0xd2, 0xe6, 0x54, 0x32, 0xc3, 0xfa, 0x36, 0x33, 0xbf, 0x10, 0x08, 0xb7, 0x80, 0x1a, + 0x50, 0xdb, 0x8a, 0xc6, 0x86, 0xe2, 0x22, 0xba, 0x00, 0xed, 0x1e, 0xa3, 0x32, 0xa2, 0x53, 0x36, + 0x15, 0xfa, 0x47, 0x25, 0xb7, 0xb8, 0xf5, 0xe6, 0xc7, 0x9f, 0x5f, 0x2e, 0x7c, 0xf2, 0xf9, 0xe5, + 0xc2, 0xc3, 0xcf, 0x2f, 0x2f, 0x7c, 0xf8, 0xf7, 0xcb, 0x85, 0xf7, 0xbe, 0x91, 0xfb, 0xb9, 0x3a, + 0xc1, 0x92, 0x47, 0xc7, 0x8c, 0x47, 0xe3, 0x88, 0x66, 0x13, 0x4a, 0xae, 0xa7, 0x93, 0xf1, 0xf5, + 0x74, 0x74, 0x1d, 0xa7, 0xd1, 0xa8, 0xa2, 0x7f, 0x97, 0x7e, 0xf9, 0xdf, 0x01, 0x00, 0x00, 0xff, + 0xff, 0xce, 0x98, 0x3c, 0xea, 0xf5, 0x16, 0x00, 0x00, } func (m *TNPingRequest) Marshal() (dAtA []byte, err error) { @@ -3098,13 +3097,6 @@ func (m *PrecommitWriteCmd) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } - if len(m.SyncProtectionJobId) > 0 { - i -= len(m.SyncProtectionJobId) - copy(dAtA[i:], m.SyncProtectionJobId) - i = encodeVarintApi(dAtA, i, uint64(len(m.SyncProtectionJobId))) - i-- - dAtA[i] = 0x12 - } if len(m.EntryList) > 0 { for iNdEx := len(m.EntryList) - 1; iNdEx >= 0; iNdEx-- { { @@ -3146,6 +3138,21 @@ func (m *Entry) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if m.HasLazyCatalogAccountId { + i-- + if m.HasLazyCatalogAccountId { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x50 + } + if m.LazyCatalogAccountId != 0 { + i = encodeVarintApi(dAtA, i, uint64(m.LazyCatalogAccountId)) + i-- + dAtA[i] = 0x48 + } if m.PkCheckByTn != 0 { i = encodeVarintApi(dAtA, i, uint64(m.PkCheckByTn)) i-- @@ -3974,16 +3981,6 @@ func (m *SchemaExtra) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } - if m.FromPublication { - i-- - if m.FromPublication { - dAtA[i] = 1 - } else { - dAtA[i] = 0 - } - i-- - dAtA[i] = 0x78 - } if m.ParentTableID != 0 { i = encodeVarintApi(dAtA, i, uint64(m.ParentTableID)) i-- @@ -4716,10 +4713,6 @@ func (m *PrecommitWriteCmd) ProtoSize() (n int) { n += 1 + l + sovApi(uint64(l)) } } - l = len(m.SyncProtectionJobId) - if l > 0 { - n += 1 + l + sovApi(uint64(l)) - } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -4760,6 +4753,12 @@ func (m *Entry) ProtoSize() (n int) { if m.PkCheckByTn != 0 { n += 1 + sovApi(uint64(m.PkCheckByTn)) } + if m.LazyCatalogAccountId != 0 { + n += 1 + sovApi(uint64(m.LazyCatalogAccountId)) + } + if m.HasLazyCatalogAccountId { + n += 2 + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -5177,9 +5176,6 @@ func (m *SchemaExtra) ProtoSize() (n int) { if m.ParentTableID != 0 { n += 1 + sovApi(uint64(m.ParentTableID)) } - if m.FromPublication { - n += 2 - } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -6637,38 +6633,6 @@ func (m *PrecommitWriteCmd) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SyncProtectionJobId", wireType) - } - var stringLen uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowApi - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - stringLen |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - intStringLen := int(stringLen) - if intStringLen < 0 { - return ErrInvalidLengthApi - } - postIndex := iNdEx + intStringLen - if postIndex < 0 { - return ErrInvalidLengthApi - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - m.SyncProtectionJobId = string(dAtA[iNdEx:postIndex]) - iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -6928,6 +6892,45 @@ func (m *Entry) Unmarshal(dAtA []byte) error { break } } + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LazyCatalogAccountId", wireType) + } + m.LazyCatalogAccountId = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.LazyCatalogAccountId |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 10: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field HasLazyCatalogAccountId", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.HasLazyCatalogAccountId = bool(v != 0) default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -9139,26 +9142,6 @@ func (m *SchemaExtra) Unmarshal(dAtA []byte) error { break } } - case 15: - if wireType != 0 { - return fmt.Errorf("proto: wrong wireType = %d for field FromPublication", wireType) - } - var v int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowApi - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - v |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - m.FromPublication = bool(v != 0) default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) diff --git a/pkg/pb/logtail/logtail.pb.go b/pkg/pb/logtail/logtail.pb.go index 7588ce783f3b7..776355c42d5ca 100644 --- a/pkg/pb/logtail/logtail.pb.go +++ b/pkg/pb/logtail/logtail.pb.go @@ -28,10 +28,12 @@ const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package // SubscribeRequest is the request for subscription. type SubscribeRequest struct { - Table *api.TableID `protobuf:"bytes,1,opt,name=table,proto3" json:"table,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + Table *api.TableID `protobuf:"bytes,1,opt,name=table,proto3" json:"table,omitempty"` + LazyCatalog bool `protobuf:"varint,2,opt,name=lazy_catalog,json=lazyCatalog,proto3" json:"lazy_catalog,omitempty"` + InitialActiveAccounts []uint32 `protobuf:"varint,3,rep,packed,name=initial_active_accounts,json=initialActiveAccounts,proto3" json:"initial_active_accounts,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *SubscribeRequest) Reset() { *m = SubscribeRequest{} } @@ -74,6 +76,20 @@ func (m *SubscribeRequest) GetTable() *api.TableID { return nil } +func (m *SubscribeRequest) GetLazyCatalog() bool { + if m != nil { + return m.LazyCatalog + } + return false +} + +func (m *SubscribeRequest) GetInitialActiveAccounts() []uint32 { + if m != nil { + return m.InitialActiveAccounts + } + return nil +} + // UnsubscribeRequest is the request for unsubscription. type UnsubscribeRequest struct { Table *api.TableID `protobuf:"bytes,1,opt,name=table,proto3" json:"table,omitempty"` @@ -122,6 +138,132 @@ func (m *UnsubscribeRequest) GetTable() *api.TableID { return nil } +type ActivateAccountForCatalogRequest struct { + AccountId uint32 `protobuf:"varint,1,opt,name=account_id,json=accountId,proto3" json:"account_id,omitempty"` + Seq uint64 `protobuf:"varint,2,opt,name=seq,proto3" json:"seq,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ActivateAccountForCatalogRequest) Reset() { *m = ActivateAccountForCatalogRequest{} } +func (m *ActivateAccountForCatalogRequest) String() string { return proto.CompactTextString(m) } +func (*ActivateAccountForCatalogRequest) ProtoMessage() {} +func (*ActivateAccountForCatalogRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_3604137dacc8e6bf, []int{2} +} +func (m *ActivateAccountForCatalogRequest) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ActivateAccountForCatalogRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ActivateAccountForCatalogRequest.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *ActivateAccountForCatalogRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ActivateAccountForCatalogRequest.Merge(m, src) +} +func (m *ActivateAccountForCatalogRequest) XXX_Size() int { + return m.ProtoSize() +} +func (m *ActivateAccountForCatalogRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ActivateAccountForCatalogRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_ActivateAccountForCatalogRequest proto.InternalMessageInfo + +func (m *ActivateAccountForCatalogRequest) GetAccountId() uint32 { + if m != nil { + return m.AccountId + } + return 0 +} + +func (m *ActivateAccountForCatalogRequest) GetSeq() uint64 { + if m != nil { + return m.Seq + } + return 0 +} + +type ActivateAccountForCatalogResponse struct { + AccountId uint32 `protobuf:"varint,1,opt,name=account_id,json=accountId,proto3" json:"account_id,omitempty"` + Seq uint64 `protobuf:"varint,2,opt,name=seq,proto3" json:"seq,omitempty"` + TargetTs *timestamp.Timestamp `protobuf:"bytes,3,opt,name=target_ts,json=targetTs,proto3" json:"target_ts,omitempty"` + Tails []TableLogtail `protobuf:"bytes,4,rep,name=tails,proto3" json:"tails"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ActivateAccountForCatalogResponse) Reset() { *m = ActivateAccountForCatalogResponse{} } +func (m *ActivateAccountForCatalogResponse) String() string { return proto.CompactTextString(m) } +func (*ActivateAccountForCatalogResponse) ProtoMessage() {} +func (*ActivateAccountForCatalogResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_3604137dacc8e6bf, []int{3} +} +func (m *ActivateAccountForCatalogResponse) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ActivateAccountForCatalogResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_ActivateAccountForCatalogResponse.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *ActivateAccountForCatalogResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_ActivateAccountForCatalogResponse.Merge(m, src) +} +func (m *ActivateAccountForCatalogResponse) XXX_Size() int { + return m.ProtoSize() +} +func (m *ActivateAccountForCatalogResponse) XXX_DiscardUnknown() { + xxx_messageInfo_ActivateAccountForCatalogResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_ActivateAccountForCatalogResponse proto.InternalMessageInfo + +func (m *ActivateAccountForCatalogResponse) GetAccountId() uint32 { + if m != nil { + return m.AccountId + } + return 0 +} + +func (m *ActivateAccountForCatalogResponse) GetSeq() uint64 { + if m != nil { + return m.Seq + } + return 0 +} + +func (m *ActivateAccountForCatalogResponse) GetTargetTs() *timestamp.Timestamp { + if m != nil { + return m.TargetTs + } + return nil +} + +func (m *ActivateAccountForCatalogResponse) GetTails() []TableLogtail { + if m != nil { + return m.Tails + } + return nil +} + // TableLogtail describes total or additional logtail for a table. type TableLogtail struct { CkpLocation string `protobuf:"bytes,1,opt,name=ckp_location,json=ckpLocation,proto3" json:"ckp_location,omitempty"` @@ -137,7 +279,7 @@ func (m *TableLogtail) Reset() { *m = TableLogtail{} } func (m *TableLogtail) String() string { return proto.CompactTextString(m) } func (*TableLogtail) ProtoMessage() {} func (*TableLogtail) Descriptor() ([]byte, []int) { - return fileDescriptor_3604137dacc8e6bf, []int{2} + return fileDescriptor_3604137dacc8e6bf, []int{4} } func (m *TableLogtail) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -209,7 +351,7 @@ func (m *Status) Reset() { *m = Status{} } func (m *Status) String() string { return proto.CompactTextString(m) } func (*Status) ProtoMessage() {} func (*Status) Descriptor() ([]byte, []int) { - return fileDescriptor_3604137dacc8e6bf, []int{3} + return fileDescriptor_3604137dacc8e6bf, []int{5} } func (m *Status) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -265,7 +407,7 @@ func (m *ErrorResponse) Reset() { *m = ErrorResponse{} } func (m *ErrorResponse) String() string { return proto.CompactTextString(m) } func (*ErrorResponse) ProtoMessage() {} func (*ErrorResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_3604137dacc8e6bf, []int{4} + return fileDescriptor_3604137dacc8e6bf, []int{6} } func (m *ErrorResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -322,7 +464,7 @@ func (m *SubscribeResponse) Reset() { *m = SubscribeResponse{} } func (m *SubscribeResponse) String() string { return proto.CompactTextString(m) } func (*SubscribeResponse) ProtoMessage() {} func (*SubscribeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_3604137dacc8e6bf, []int{5} + return fileDescriptor_3604137dacc8e6bf, []int{7} } func (m *SubscribeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -375,7 +517,7 @@ func (m *UpdateResponse) Reset() { *m = UpdateResponse{} } func (m *UpdateResponse) String() string { return proto.CompactTextString(m) } func (*UpdateResponse) ProtoMessage() {} func (*UpdateResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_3604137dacc8e6bf, []int{6} + return fileDescriptor_3604137dacc8e6bf, []int{8} } func (m *UpdateResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -437,7 +579,7 @@ func (m *UnSubscribeResponse) Reset() { *m = UnSubscribeResponse{} } func (m *UnSubscribeResponse) String() string { return proto.CompactTextString(m) } func (*UnSubscribeResponse) ProtoMessage() {} func (*UnSubscribeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_3604137dacc8e6bf, []int{7} + return fileDescriptor_3604137dacc8e6bf, []int{9} } func (m *UnSubscribeResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -479,6 +621,7 @@ type LogtailRequest struct { // Types that are valid to be assigned to Request: // *LogtailRequest_SubscribeTable // *LogtailRequest_UnsubscribeTable + // *LogtailRequest_ActivateAccountForCatalog Request isLogtailRequest_Request `protobuf_oneof:"request"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` @@ -489,7 +632,7 @@ func (m *LogtailRequest) Reset() { *m = LogtailRequest{} } func (m *LogtailRequest) String() string { return proto.CompactTextString(m) } func (*LogtailRequest) ProtoMessage() {} func (*LogtailRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_3604137dacc8e6bf, []int{8} + return fileDescriptor_3604137dacc8e6bf, []int{10} } func (m *LogtailRequest) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -530,9 +673,13 @@ type LogtailRequest_SubscribeTable struct { type LogtailRequest_UnsubscribeTable struct { UnsubscribeTable *UnsubscribeRequest `protobuf:"bytes,3,opt,name=unsubscribe_table,json=unsubscribeTable,proto3,oneof" json:"unsubscribe_table,omitempty"` } +type LogtailRequest_ActivateAccountForCatalog struct { + ActivateAccountForCatalog *ActivateAccountForCatalogRequest `protobuf:"bytes,4,opt,name=activate_account_for_catalog,json=activateAccountForCatalog,proto3,oneof" json:"activate_account_for_catalog,omitempty"` +} -func (*LogtailRequest_SubscribeTable) isLogtailRequest_Request() {} -func (*LogtailRequest_UnsubscribeTable) isLogtailRequest_Request() {} +func (*LogtailRequest_SubscribeTable) isLogtailRequest_Request() {} +func (*LogtailRequest_UnsubscribeTable) isLogtailRequest_Request() {} +func (*LogtailRequest_ActivateAccountForCatalog) isLogtailRequest_Request() {} func (m *LogtailRequest) GetRequest() isLogtailRequest_Request { if m != nil { @@ -562,11 +709,19 @@ func (m *LogtailRequest) GetUnsubscribeTable() *UnsubscribeRequest { return nil } +func (m *LogtailRequest) GetActivateAccountForCatalog() *ActivateAccountForCatalogRequest { + if x, ok := m.GetRequest().(*LogtailRequest_ActivateAccountForCatalog); ok { + return x.ActivateAccountForCatalog + } + return nil +} + // XXX_OneofWrappers is for the internal use of the proto package. func (*LogtailRequest) XXX_OneofWrappers() []interface{} { return []interface{}{ (*LogtailRequest_SubscribeTable)(nil), (*LogtailRequest_UnsubscribeTable)(nil), + (*LogtailRequest_ActivateAccountForCatalog)(nil), } } @@ -578,6 +733,7 @@ type LogtailResponse struct { // *LogtailResponse_UnsubscribeResponse // *LogtailResponse_UpdateResponse // *LogtailResponse_Error + // *LogtailResponse_ActivateAccountForCatalogResponse Response isLogtailResponse_Response `protobuf_oneof:"response"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` @@ -588,7 +744,7 @@ func (m *LogtailResponse) Reset() { *m = LogtailResponse{} } func (m *LogtailResponse) String() string { return proto.CompactTextString(m) } func (*LogtailResponse) ProtoMessage() {} func (*LogtailResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_3604137dacc8e6bf, []int{9} + return fileDescriptor_3604137dacc8e6bf, []int{11} } func (m *LogtailResponse) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -635,11 +791,15 @@ type LogtailResponse_UpdateResponse struct { type LogtailResponse_Error struct { Error *ErrorResponse `protobuf:"bytes,5,opt,name=error,proto3,oneof" json:"error,omitempty"` } +type LogtailResponse_ActivateAccountForCatalogResponse struct { + ActivateAccountForCatalogResponse *ActivateAccountForCatalogResponse `protobuf:"bytes,6,opt,name=activate_account_for_catalog_response,json=activateAccountForCatalogResponse,proto3,oneof" json:"activate_account_for_catalog_response,omitempty"` +} -func (*LogtailResponse_SubscribeResponse) isLogtailResponse_Response() {} -func (*LogtailResponse_UnsubscribeResponse) isLogtailResponse_Response() {} -func (*LogtailResponse_UpdateResponse) isLogtailResponse_Response() {} -func (*LogtailResponse_Error) isLogtailResponse_Response() {} +func (*LogtailResponse_SubscribeResponse) isLogtailResponse_Response() {} +func (*LogtailResponse_UnsubscribeResponse) isLogtailResponse_Response() {} +func (*LogtailResponse_UpdateResponse) isLogtailResponse_Response() {} +func (*LogtailResponse_Error) isLogtailResponse_Response() {} +func (*LogtailResponse_ActivateAccountForCatalogResponse) isLogtailResponse_Response() {} func (m *LogtailResponse) GetResponse() isLogtailResponse_Response { if m != nil { @@ -683,6 +843,13 @@ func (m *LogtailResponse) GetError() *ErrorResponse { return nil } +func (m *LogtailResponse) GetActivateAccountForCatalogResponse() *ActivateAccountForCatalogResponse { + if x, ok := m.GetResponse().(*LogtailResponse_ActivateAccountForCatalogResponse); ok { + return x.ActivateAccountForCatalogResponse + } + return nil +} + // XXX_OneofWrappers is for the internal use of the proto package. func (*LogtailResponse) XXX_OneofWrappers() []interface{} { return []interface{}{ @@ -690,6 +857,7 @@ func (*LogtailResponse) XXX_OneofWrappers() []interface{} { (*LogtailResponse_UnsubscribeResponse)(nil), (*LogtailResponse_UpdateResponse)(nil), (*LogtailResponse_Error)(nil), + (*LogtailResponse_ActivateAccountForCatalogResponse)(nil), } } @@ -709,7 +877,7 @@ func (m *MessageSegment) Reset() { *m = MessageSegment{} } func (m *MessageSegment) String() string { return proto.CompactTextString(m) } func (*MessageSegment) ProtoMessage() {} func (*MessageSegment) Descriptor() ([]byte, []int) { - return fileDescriptor_3604137dacc8e6bf, []int{10} + return fileDescriptor_3604137dacc8e6bf, []int{12} } func (m *MessageSegment) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -776,6 +944,8 @@ func (m *MessageSegment) GetPayload() []byte { func init() { proto.RegisterType((*SubscribeRequest)(nil), "logtail.SubscribeRequest") proto.RegisterType((*UnsubscribeRequest)(nil), "logtail.UnsubscribeRequest") + proto.RegisterType((*ActivateAccountForCatalogRequest)(nil), "logtail.ActivateAccountForCatalogRequest") + proto.RegisterType((*ActivateAccountForCatalogResponse)(nil), "logtail.ActivateAccountForCatalogResponse") proto.RegisterType((*TableLogtail)(nil), "logtail.TableLogtail") proto.RegisterType((*Status)(nil), "logtail.Status") proto.RegisterType((*ErrorResponse)(nil), "logtail.ErrorResponse") @@ -790,53 +960,65 @@ func init() { func init() { proto.RegisterFile("logtail.proto", fileDescriptor_3604137dacc8e6bf) } var fileDescriptor_3604137dacc8e6bf = []byte{ - // 734 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x55, 0x4f, 0x4f, 0xdb, 0x4a, - 0x10, 0x8f, 0x43, 0x42, 0xc8, 0x24, 0x24, 0xb0, 0xf0, 0xde, 0xcb, 0xcb, 0x7b, 0x0d, 0xd4, 0xea, - 0x21, 0x87, 0x92, 0x20, 0xaa, 0xa2, 0xf6, 0xd2, 0x43, 0x04, 0x55, 0x42, 0x41, 0x6a, 0x37, 0xe1, - 0xd2, 0x4b, 0xb4, 0x76, 0xb6, 0xae, 0x45, 0xec, 0x75, 0xbd, 0x6b, 0x09, 0xfa, 0x69, 0x38, 0x55, - 0xea, 0x37, 0xe1, 0xd0, 0x43, 0xd5, 0x0f, 0x50, 0x55, 0xf4, 0x8b, 0x54, 0xd9, 0x5d, 0xff, 0x09, - 0x01, 0x54, 0xf5, 0x36, 0x33, 0x3b, 0xf3, 0x9b, 0xfd, 0xfd, 0x66, 0xd6, 0x86, 0xd5, 0x29, 0x73, - 0x04, 0x71, 0xa7, 0x9d, 0x20, 0x64, 0x82, 0xa1, 0x92, 0x76, 0x9b, 0x3b, 0x8e, 0x2b, 0xde, 0x47, - 0x56, 0xc7, 0x66, 0x5e, 0xd7, 0x61, 0x0e, 0xeb, 0xca, 0x73, 0x2b, 0x7a, 0x27, 0x3d, 0xe9, 0x48, - 0x4b, 0xd5, 0x35, 0xeb, 0xc2, 0xf5, 0x28, 0x17, 0xc4, 0x0b, 0x74, 0xa0, 0x4c, 0x02, 0x57, 0x99, - 0xe6, 0x3e, 0xac, 0x0d, 0x23, 0x8b, 0xdb, 0xa1, 0x6b, 0x51, 0x4c, 0x3f, 0x44, 0x94, 0x0b, 0x64, - 0x42, 0x51, 0x10, 0x6b, 0x4a, 0x1b, 0xc6, 0xb6, 0xd1, 0xae, 0xec, 0x55, 0x3b, 0xb3, 0xf4, 0xd1, - 0x2c, 0x32, 0x38, 0xc0, 0xea, 0xc8, 0x7c, 0x06, 0xe8, 0xd4, 0xe7, 0x7f, 0x52, 0xf9, 0xd9, 0x80, - 0xaa, 0x0c, 0x1d, 0x2b, 0x36, 0xe8, 0x21, 0x54, 0xed, 0xb3, 0x60, 0x3c, 0x65, 0x36, 0x11, 0x2e, - 0xf3, 0x65, 0x6d, 0x19, 0x57, 0xec, 0xb3, 0xe0, 0x58, 0x87, 0xd0, 0x23, 0xc8, 0x0b, 0xde, 0xc8, - 0x4b, 0xd0, 0xcd, 0x4e, 0x4a, 0x67, 0x14, 0x5b, 0x38, 0x2f, 0x78, 0xda, 0x7d, 0xe9, 0xce, 0xee, - 0xe8, 0x31, 0xac, 0xd8, 0xcc, 0xf3, 0x88, 0x3f, 0xe1, 0x8d, 0xc2, 0xf6, 0x52, 0xbb, 0xb2, 0x07, - 0x32, 0xed, 0xd0, 0x17, 0xe1, 0x45, 0xaf, 0x70, 0xf5, 0x7d, 0x2b, 0x87, 0x93, 0x0c, 0x73, 0x1f, - 0x96, 0x87, 0x82, 0x88, 0x88, 0x23, 0x04, 0x05, 0x9b, 0x4d, 0x14, 0xb1, 0x55, 0x2c, 0x6d, 0xd4, - 0x80, 0x92, 0x47, 0x39, 0x27, 0x0e, 0x95, 0x57, 0x2b, 0xe3, 0xd8, 0x35, 0x2d, 0x58, 0x3d, 0x0c, - 0x43, 0x16, 0x62, 0xca, 0x03, 0xe6, 0x73, 0x8a, 0x76, 0x60, 0x99, 0x4b, 0x20, 0xad, 0x4c, 0xbd, - 0x13, 0x8f, 0x56, 0xe1, 0xeb, 0xce, 0x3a, 0x29, 0x65, 0x92, 0xbf, 0x5b, 0xc7, 0x23, 0x58, 0xcf, - 0x4c, 0x4e, 0xf7, 0x79, 0x0a, 0xf1, 0x92, 0xe8, 0x46, 0x7f, 0x25, 0x8d, 0xb2, 0x9a, 0xeb, 0x76, - 0x71, 0xae, 0x79, 0x69, 0x40, 0xed, 0x34, 0x98, 0x10, 0x91, 0x22, 0xb5, 0xa1, 0xf0, 0x32, 0x64, - 0x9e, 0x86, 0xb9, 0x5d, 0x74, 0x99, 0x31, 0x1b, 0xce, 0x88, 0xdd, 0x3f, 0x9c, 0x11, 0x43, 0x2f, - 0xa0, 0xaa, 0xbb, 0x8d, 0xa7, 0x2e, 0x17, 0x8d, 0x25, 0x29, 0xfe, 0xbd, 0xd7, 0xab, 0xe8, 0xb3, - 0x63, 0x97, 0x0b, 0xf3, 0x39, 0x6c, 0x9c, 0xfa, 0x8b, 0x84, 0x7f, 0x67, 0xe3, 0xbe, 0x18, 0x50, - 0xd3, 0xc8, 0xf1, 0xa2, 0x3e, 0x00, 0x08, 0x95, 0x39, 0x76, 0x27, 0xb2, 0xb6, 0x80, 0xcb, 0x3a, - 0x32, 0x98, 0xa0, 0x03, 0xa8, 0x27, 0xbb, 0x3d, 0xce, 0x4e, 0xe2, 0xdf, 0x74, 0x6e, 0x37, 0x76, - 0xbf, 0x9f, 0xc3, 0xb5, 0xa4, 0x46, 0x5e, 0x00, 0x1d, 0xc1, 0x7a, 0xe4, 0xdf, 0xc4, 0x51, 0xbb, - 0xf9, 0x5f, 0x82, 0xb3, 0xf8, 0x8a, 0xfa, 0x39, 0xbc, 0x96, 0xa9, 0x93, 0x58, 0xbd, 0x32, 0x94, - 0xf4, 0xf5, 0xcc, 0x6f, 0x79, 0xa8, 0x27, 0x74, 0xb4, 0x0c, 0x5b, 0x50, 0x09, 0xb5, 0x9d, 0x12, - 0x82, 0x38, 0x34, 0x98, 0xa0, 0x57, 0x80, 0xd2, 0x9b, 0xc4, 0x71, 0x4d, 0xaa, 0x79, 0x1b, 0x29, - 0x95, 0xd1, 0xcf, 0xe1, 0x75, 0xbe, 0x20, 0xfa, 0x1b, 0xd8, 0xcc, 0x12, 0x4b, 0xe0, 0x14, 0xb7, - 0xff, 0x33, 0xdc, 0x6e, 0x03, 0xdc, 0x88, 0xfc, 0x45, 0xc8, 0x1e, 0xd4, 0x23, 0xb9, 0x80, 0x29, - 0x5a, 0x41, 0xa2, 0xfd, 0x93, 0xa2, 0xcd, 0x2d, 0xe8, 0x4c, 0xef, 0x68, 0x7e, 0x65, 0x3b, 0x50, - 0xa4, 0xb3, 0x57, 0xd7, 0x28, 0xca, 0xca, 0xbf, 0x93, 0xca, 0xb9, 0xb7, 0xd8, 0xcf, 0x61, 0x95, - 0xd6, 0x03, 0x58, 0x89, 0x9b, 0x99, 0x9f, 0x0c, 0xa8, 0x9d, 0xa8, 0xd7, 0x3b, 0xa4, 0x8e, 0x47, - 0x7d, 0x81, 0x9a, 0xb0, 0x32, 0x14, 0x21, 0x25, 0xde, 0xe0, 0x40, 0x0b, 0x9a, 0xf8, 0x68, 0x1b, - 0x2a, 0x71, 0xb6, 0xfb, 0x51, 0xe9, 0x58, 0xc4, 0xd9, 0x90, 0xac, 0x9e, 0x0d, 0xcc, 0xb7, 0x95, - 0x2e, 0x45, 0x9c, 0xf8, 0xb2, 0x9a, 0x9c, 0x27, 0xc7, 0x05, 0x5d, 0x9d, 0x86, 0x66, 0x9f, 0x96, - 0xd7, 0xe4, 0x62, 0xca, 0xc8, 0x44, 0x92, 0xa9, 0xe2, 0xd8, 0xdd, 0x3b, 0x81, 0x52, 0xfc, 0xe1, - 0xec, 0xa5, 0x66, 0xaa, 0xd2, 0xfc, 0xa2, 0x37, 0x1b, 0x8b, 0x07, 0x9a, 0x71, 0xae, 0x6d, 0xec, - 0x1a, 0xbd, 0xde, 0xd5, 0x75, 0xcb, 0xf8, 0x7a, 0xdd, 0x32, 0x7e, 0x5c, 0xb7, 0x72, 0x97, 0x3f, - 0x5b, 0xc6, 0xdb, 0xdd, 0xcc, 0xcf, 0xc5, 0x23, 0x22, 0x74, 0xcf, 0x59, 0xe8, 0x3a, 0xae, 0x1f, - 0x3b, 0x3e, 0xed, 0x06, 0x67, 0x4e, 0x37, 0xb0, 0xba, 0x1a, 0xd9, 0x5a, 0x96, 0xbf, 0x92, 0x27, - 0xbf, 0x02, 0x00, 0x00, 0xff, 0xff, 0xbd, 0xae, 0x7f, 0xca, 0xaf, 0x06, 0x00, 0x00, + // 921 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x56, 0xdb, 0x8e, 0xdb, 0x44, + 0x18, 0x8e, 0x73, 0xd8, 0x4d, 0xfe, 0x64, 0x37, 0xdd, 0x69, 0x4b, 0xd3, 0x50, 0xb6, 0x59, 0x0b, + 0xa4, 0x80, 0x68, 0xd2, 0x2e, 0xa2, 0x82, 0x1b, 0xa4, 0x86, 0x6d, 0x95, 0x94, 0xad, 0x04, 0x93, + 0xec, 0x0d, 0x37, 0xd6, 0xc4, 0x71, 0xcd, 0x68, 0x6d, 0x8f, 0xeb, 0x19, 0xa3, 0x6e, 0x2f, 0x78, + 0x07, 0xde, 0xa0, 0x57, 0x48, 0xbc, 0x42, 0x9f, 0xa0, 0x97, 0x88, 0x07, 0x40, 0x68, 0x79, 0x11, + 0xe4, 0x39, 0xd8, 0xde, 0x66, 0x0f, 0x85, 0xbb, 0xf9, 0x4f, 0xdf, 0xfc, 0xf3, 0x7d, 0xf3, 0x8f, + 0x0d, 0x5b, 0x01, 0xf3, 0x05, 0xa1, 0xc1, 0x28, 0x4e, 0x98, 0x60, 0x68, 0x53, 0x9b, 0xfd, 0x7b, + 0x3e, 0x15, 0x3f, 0xa5, 0xcb, 0x91, 0xcb, 0xc2, 0xb1, 0xcf, 0x7c, 0x36, 0x96, 0xf1, 0x65, 0xfa, + 0x5c, 0x5a, 0xd2, 0x90, 0x2b, 0x55, 0xd7, 0xef, 0x0a, 0x1a, 0x7a, 0x5c, 0x90, 0x30, 0xd6, 0x8e, + 0x16, 0x89, 0xa9, 0x5a, 0xda, 0xbf, 0x5a, 0x70, 0x6d, 0x9e, 0x2e, 0xb9, 0x9b, 0xd0, 0xa5, 0x87, + 0xbd, 0x17, 0xa9, 0xc7, 0x05, 0xb2, 0xa1, 0x21, 0xc8, 0x32, 0xf0, 0x7a, 0xd6, 0xc0, 0x1a, 0xb6, + 0xf7, 0x3b, 0xa3, 0x2c, 0x7f, 0x91, 0x79, 0x66, 0x07, 0x58, 0x85, 0xd0, 0x1e, 0x74, 0x02, 0xf2, + 0xea, 0xc4, 0x71, 0x89, 0x20, 0x01, 0xf3, 0x7b, 0xd5, 0x81, 0x35, 0x6c, 0xe2, 0x76, 0xe6, 0xfb, + 0x56, 0xb9, 0xd0, 0x43, 0xb8, 0x45, 0x23, 0x2a, 0x28, 0x09, 0x1c, 0xe2, 0x0a, 0xfa, 0xb3, 0xe7, + 0x10, 0xd7, 0x65, 0x69, 0x24, 0x78, 0xaf, 0x36, 0xa8, 0x0d, 0xb7, 0xf0, 0x4d, 0x1d, 0x7e, 0x24, + 0xa3, 0x8f, 0x74, 0xd0, 0xfe, 0x0a, 0xd0, 0x51, 0xc4, 0xff, 0x47, 0x53, 0xf6, 0x1c, 0x06, 0x12, + 0x8b, 0x08, 0x83, 0xf6, 0x84, 0x25, 0xba, 0x1d, 0x83, 0xf3, 0x11, 0x80, 0x6e, 0xc3, 0xa1, 0x2b, + 0x09, 0xb6, 0x85, 0x5b, 0xda, 0x33, 0x5b, 0xa1, 0x6b, 0x50, 0xe3, 0xde, 0x0b, 0x79, 0x9c, 0x3a, + 0xce, 0x96, 0xf6, 0x1b, 0x0b, 0xf6, 0x2e, 0x41, 0xe5, 0x31, 0x8b, 0xb8, 0xf7, 0x9f, 0x61, 0xd1, + 0x03, 0x68, 0x09, 0x92, 0xf8, 0x9e, 0x70, 0x24, 0x1f, 0xd9, 0x99, 0x6e, 0x8c, 0x0a, 0xa5, 0x16, + 0x66, 0x85, 0x9b, 0x2a, 0x6d, 0xc1, 0xd1, 0x83, 0x8c, 0x02, 0x1a, 0xf0, 0x5e, 0x7d, 0x50, 0x1b, + 0xb6, 0xf7, 0x6f, 0x8e, 0xcc, 0xfd, 0x90, 0x34, 0x1c, 0x2a, 0x63, 0x52, 0x7f, 0xfb, 0xd7, 0xdd, + 0x0a, 0x56, 0x99, 0xf6, 0xef, 0x16, 0x74, 0xca, 0xd1, 0x4c, 0x37, 0xf7, 0x38, 0x76, 0x02, 0xe6, + 0x12, 0x41, 0x59, 0x24, 0x3b, 0x6d, 0xe1, 0xb6, 0x7b, 0x1c, 0x1f, 0x6a, 0x17, 0xfa, 0x18, 0xaa, + 0x82, 0xcb, 0x56, 0x2f, 0x6a, 0xa9, 0x2a, 0x78, 0xa1, 0x47, 0xed, 0xe2, 0x4b, 0xf2, 0x39, 0x34, + 0x5d, 0x16, 0x86, 0x24, 0x5a, 0x99, 0x9e, 0x41, 0xa6, 0x3d, 0x8e, 0x44, 0x72, 0xa2, 0x1b, 0xcd, + 0x33, 0xec, 0x87, 0xb0, 0x31, 0x17, 0x44, 0xa4, 0x1c, 0x21, 0xa8, 0xbb, 0x6c, 0xe5, 0x69, 0x1a, + 0xe5, 0x1a, 0xf5, 0x60, 0x33, 0xf4, 0x38, 0x27, 0xbe, 0x27, 0x5b, 0x6b, 0x61, 0x63, 0xda, 0x4b, + 0xd8, 0x7a, 0x9c, 0x24, 0x2c, 0xc9, 0xb5, 0xb8, 0x07, 0x1b, 0x5c, 0x02, 0xe9, 0xbb, 0xd2, 0xcd, + 0x89, 0x52, 0xf8, 0x7a, 0x67, 0x9d, 0x54, 0x9c, 0xa4, 0x7a, 0xf1, 0xcd, 0x7a, 0x0a, 0x3b, 0xa5, + 0x31, 0xd1, 0xfb, 0x7c, 0x09, 0x66, 0x24, 0xf5, 0x46, 0x97, 0x2a, 0x62, 0x72, 0xed, 0xd7, 0x16, + 0x6c, 0x1f, 0xc5, 0x2b, 0x22, 0x0a, 0xa4, 0x21, 0xd4, 0x9f, 0x24, 0x2c, 0xd4, 0x30, 0xe7, 0x93, + 0x2e, 0x33, 0x32, 0x71, 0x16, 0xec, 0x72, 0x71, 0x16, 0x0c, 0x7d, 0x03, 0x1d, 0xbd, 0x9b, 0x13, + 0x50, 0x2e, 0xe4, 0xbc, 0x5d, 0xd1, 0x5e, 0x5b, 0xc7, 0x0e, 0x29, 0x17, 0xf6, 0xd7, 0x70, 0xfd, + 0x28, 0x5a, 0x3f, 0xf0, 0xfb, 0xcc, 0xe0, 0x9b, 0x2a, 0x6c, 0x6b, 0xe4, 0xd2, 0xc8, 0x25, 0x6a, + 0x69, 0x66, 0xa3, 0x8e, 0x5b, 0xda, 0x33, 0x5b, 0xa1, 0x03, 0xe8, 0xe6, 0xd3, 0xee, 0x94, 0x95, + 0xb8, 0x5d, 0xe8, 0xf6, 0xce, 0x6b, 0x30, 0xad, 0xe0, 0xed, 0xbc, 0x46, 0x36, 0x80, 0x9e, 0xc2, + 0x4e, 0x1a, 0xbd, 0x8b, 0xa3, 0xee, 0xe6, 0x87, 0x39, 0xce, 0xfa, 0xbb, 0x32, 0xad, 0xe0, 0x6b, + 0xa5, 0x3a, 0x85, 0x15, 0xc0, 0x1d, 0xa2, 0x27, 0xde, 0xbc, 0x59, 0xce, 0x73, 0x96, 0xe4, 0x8f, + 0x5d, 0x5d, 0xc2, 0x7e, 0x9a, 0xc3, 0x5e, 0xf5, 0xe8, 0x4c, 0x2b, 0xf8, 0x36, 0xb9, 0x28, 0x67, + 0xd2, 0x82, 0x4d, 0x4d, 0x86, 0xfd, 0x67, 0x0d, 0xba, 0x39, 0x79, 0x9a, 0xf4, 0xbb, 0xd0, 0x4e, + 0xf4, 0xba, 0xa0, 0x0f, 0x8c, 0x6b, 0xb6, 0x42, 0xdf, 0x01, 0x2a, 0xce, 0x6d, 0xfc, 0x9a, 0xc2, + 0xfe, 0x79, 0x14, 0xaa, 0x8c, 0x69, 0x05, 0xef, 0xf0, 0x35, 0x89, 0x7f, 0x80, 0x1b, 0x65, 0x1a, + 0x73, 0x38, 0xc5, 0xe4, 0x9d, 0x12, 0x93, 0xe7, 0x01, 0x5e, 0x4f, 0xa3, 0x75, 0xc8, 0x09, 0x74, + 0x53, 0x79, 0xdd, 0x0b, 0x34, 0x45, 0xe0, 0xad, 0x02, 0xed, 0xcc, 0x38, 0x64, 0xea, 0xa6, 0x67, + 0x07, 0x64, 0x04, 0x0d, 0x2f, 0x9b, 0xf1, 0x5e, 0x43, 0x56, 0x7e, 0x90, 0x57, 0x9e, 0x99, 0xfc, + 0x69, 0x05, 0xab, 0x34, 0xf4, 0x0b, 0x7c, 0x72, 0x99, 0x82, 0x45, 0x27, 0x1b, 0x12, 0xef, 0xb3, + 0xf7, 0x91, 0x32, 0xdf, 0x63, 0x8f, 0x5c, 0x95, 0x34, 0x01, 0x68, 0x9a, 0x2d, 0xec, 0xdf, 0x2c, + 0xd8, 0x7e, 0xa6, 0xde, 0xaa, 0xb9, 0xe7, 0x87, 0x5e, 0x24, 0x50, 0x1f, 0x9a, 0x73, 0x91, 0x78, + 0x24, 0x9c, 0x1d, 0x68, 0x41, 0x73, 0x1b, 0x0d, 0xa0, 0x6d, 0xb2, 0xe9, 0x2b, 0xa5, 0x63, 0x03, + 0x97, 0x5d, 0xb2, 0x3a, 0xbb, 0x30, 0x91, 0xab, 0x74, 0x69, 0xe0, 0xdc, 0x96, 0xd5, 0xe4, 0x65, + 0x1e, 0xae, 0xeb, 0xea, 0xc2, 0x95, 0x3d, 0xa4, 0xdf, 0x93, 0x93, 0x80, 0x91, 0x95, 0x24, 0xb3, + 0x83, 0x8d, 0xb9, 0xff, 0x0c, 0x36, 0xcd, 0x67, 0x62, 0x52, 0x2c, 0x0b, 0x95, 0xce, 0x8e, 0x75, + 0xbf, 0xb7, 0x1e, 0xd0, 0x27, 0xae, 0x0c, 0xad, 0xfb, 0xd6, 0x64, 0xf2, 0xf6, 0x74, 0xd7, 0xfa, + 0xe3, 0x74, 0xd7, 0xfa, 0xfb, 0x74, 0xb7, 0xf2, 0xfa, 0x9f, 0x5d, 0xeb, 0xc7, 0xfb, 0xa5, 0x1f, + 0x97, 0x90, 0x88, 0x84, 0xbe, 0x64, 0x09, 0xf5, 0x69, 0x64, 0x8c, 0xc8, 0x1b, 0xc7, 0xc7, 0xfe, + 0x38, 0x5e, 0x8e, 0x35, 0xf2, 0x72, 0x43, 0xfe, 0xa6, 0x7c, 0xf1, 0x6f, 0x00, 0x00, 0x00, 0xff, + 0xff, 0x7a, 0x16, 0x6b, 0x5b, 0x0b, 0x09, 0x00, 0x00, } func (m *SubscribeRequest) Marshal() (dAtA []byte, err error) { @@ -863,6 +1045,34 @@ func (m *SubscribeRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if len(m.InitialActiveAccounts) > 0 { + dAtA2 := make([]byte, len(m.InitialActiveAccounts)*10) + var j1 int + for _, num := range m.InitialActiveAccounts { + for num >= 1<<7 { + dAtA2[j1] = uint8(uint64(num)&0x7f | 0x80) + num >>= 7 + j1++ + } + dAtA2[j1] = uint8(num) + j1++ + } + i -= j1 + copy(dAtA[i:], dAtA2[:j1]) + i = encodeVarintLogtail(dAtA, i, uint64(j1)) + i-- + dAtA[i] = 0x1a + } + if m.LazyCatalog { + i-- + if m.LazyCatalog { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x10 + } if m.Table != nil { { size, err := m.Table.MarshalToSizedBuffer(dAtA[:i]) @@ -917,6 +1127,106 @@ func (m *UnsubscribeRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *ActivateAccountForCatalogRequest) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ActivateAccountForCatalogRequest) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ActivateAccountForCatalogRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if m.Seq != 0 { + i = encodeVarintLogtail(dAtA, i, uint64(m.Seq)) + i-- + dAtA[i] = 0x10 + } + if m.AccountId != 0 { + i = encodeVarintLogtail(dAtA, i, uint64(m.AccountId)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *ActivateAccountForCatalogResponse) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ActivateAccountForCatalogResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *ActivateAccountForCatalogResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.XXX_unrecognized != nil { + i -= len(m.XXX_unrecognized) + copy(dAtA[i:], m.XXX_unrecognized) + } + if len(m.Tails) > 0 { + for iNdEx := len(m.Tails) - 1; iNdEx >= 0; iNdEx-- { + { + size, err := m.Tails[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintLogtail(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x22 + } + } + if m.TargetTs != nil { + { + size, err := m.TargetTs.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintLogtail(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x1a + } + if m.Seq != 0 { + i = encodeVarintLogtail(dAtA, i, uint64(m.Seq)) + i-- + dAtA[i] = 0x10 + } + if m.AccountId != 0 { + i = encodeVarintLogtail(dAtA, i, uint64(m.AccountId)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + func (m *TableLogtail) Marshal() (dAtA []byte, err error) { size := m.ProtoSize() dAtA = make([]byte, size) @@ -1301,6 +1611,27 @@ func (m *LogtailRequest_UnsubscribeTable) MarshalToSizedBuffer(dAtA []byte) (int } return len(dAtA) - i, nil } +func (m *LogtailRequest_ActivateAccountForCatalog) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LogtailRequest_ActivateAccountForCatalog) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.ActivateAccountForCatalog != nil { + { + size, err := m.ActivateAccountForCatalog.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintLogtail(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x22 + } + return len(dAtA) - i, nil +} func (m *LogtailResponse) Marshal() (dAtA []byte, err error) { size := m.ProtoSize() dAtA = make([]byte, size) @@ -1426,6 +1757,27 @@ func (m *LogtailResponse_Error) MarshalToSizedBuffer(dAtA []byte) (int, error) { } return len(dAtA) - i, nil } +func (m *LogtailResponse_ActivateAccountForCatalogResponse) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LogtailResponse_ActivateAccountForCatalogResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.ActivateAccountForCatalogResponse != nil { + { + size, err := m.ActivateAccountForCatalogResponse.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintLogtail(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x32 + } + return len(dAtA) - i, nil +} func (m *MessageSegment) Marshal() (dAtA []byte, err error) { size := m.ProtoSize() dAtA = make([]byte, size) @@ -1501,6 +1853,16 @@ func (m *SubscribeRequest) ProtoSize() (n int) { l = m.Table.ProtoSize() n += 1 + l + sovLogtail(uint64(l)) } + if m.LazyCatalog { + n += 2 + } + if len(m.InitialActiveAccounts) > 0 { + l = 0 + for _, e := range m.InitialActiveAccounts { + l += sovLogtail(uint64(e)) + } + n += 1 + sovLogtail(uint64(l)) + l + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -1523,23 +1885,69 @@ func (m *UnsubscribeRequest) ProtoSize() (n int) { return n } -func (m *TableLogtail) ProtoSize() (n int) { +func (m *ActivateAccountForCatalogRequest) ProtoSize() (n int) { if m == nil { return 0 } var l int _ = l - l = len(m.CkpLocation) - if l > 0 { - n += 1 + l + sovLogtail(uint64(l)) - } - if m.Ts != nil { - l = m.Ts.ProtoSize() - n += 1 + l + sovLogtail(uint64(l)) + if m.AccountId != 0 { + n += 1 + sovLogtail(uint64(m.AccountId)) } - if m.Table != nil { - l = m.Table.ProtoSize() - n += 1 + l + sovLogtail(uint64(l)) + if m.Seq != 0 { + n += 1 + sovLogtail(uint64(m.Seq)) + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *ActivateAccountForCatalogResponse) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.AccountId != 0 { + n += 1 + sovLogtail(uint64(m.AccountId)) + } + if m.Seq != 0 { + n += 1 + sovLogtail(uint64(m.Seq)) + } + if m.TargetTs != nil { + l = m.TargetTs.ProtoSize() + n += 1 + l + sovLogtail(uint64(l)) + } + if len(m.Tails) > 0 { + for _, e := range m.Tails { + l = e.ProtoSize() + n += 1 + l + sovLogtail(uint64(l)) + } + } + if m.XXX_unrecognized != nil { + n += len(m.XXX_unrecognized) + } + return n +} + +func (m *TableLogtail) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.CkpLocation) + if l > 0 { + n += 1 + l + sovLogtail(uint64(l)) + } + if m.Ts != nil { + l = m.Ts.ProtoSize() + n += 1 + l + sovLogtail(uint64(l)) + } + if m.Table != nil { + l = m.Table.ProtoSize() + n += 1 + l + sovLogtail(uint64(l)) } if len(m.Commands) > 0 { for _, e := range m.Commands { @@ -1688,6 +2096,18 @@ func (m *LogtailRequest_UnsubscribeTable) ProtoSize() (n int) { } return n } +func (m *LogtailRequest_ActivateAccountForCatalog) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ActivateAccountForCatalog != nil { + l = m.ActivateAccountForCatalog.ProtoSize() + n += 1 + l + sovLogtail(uint64(l)) + } + return n +} func (m *LogtailResponse) ProtoSize() (n int) { if m == nil { return 0 @@ -1754,6 +2174,18 @@ func (m *LogtailResponse_Error) ProtoSize() (n int) { } return n } +func (m *LogtailResponse_ActivateAccountForCatalogResponse) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.ActivateAccountForCatalogResponse != nil { + l = m.ActivateAccountForCatalogResponse.ProtoSize() + n += 1 + l + sovLogtail(uint64(l)) + } + return n +} func (m *MessageSegment) ProtoSize() (n int) { if m == nil { return 0 @@ -1853,6 +2285,102 @@ func (m *SubscribeRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field LazyCatalog", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogtail + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.LazyCatalog = bool(v != 0) + case 3: + if wireType == 0 { + var v uint32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogtail + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.InitialActiveAccounts = append(m.InitialActiveAccounts, v) + } else if wireType == 2 { + var packedLen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogtail + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + packedLen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if packedLen < 0 { + return ErrInvalidLengthLogtail + } + postIndex := iNdEx + packedLen + if postIndex < 0 { + return ErrInvalidLengthLogtail + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + var elementCount int + var count int + for _, integer := range dAtA[iNdEx:postIndex] { + if integer < 128 { + count++ + } + } + elementCount = count + if elementCount != 0 && len(m.InitialActiveAccounts) == 0 { + m.InitialActiveAccounts = make([]uint32, 0, elementCount) + } + for iNdEx < postIndex { + var v uint32 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogtail + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.InitialActiveAccounts = append(m.InitialActiveAccounts, v) + } + } else { + return fmt.Errorf("proto: wrong wireType = %d for field InitialActiveAccounts", wireType) + } default: iNdEx = preIndex skippy, err := skipLogtail(dAtA[iNdEx:]) @@ -1962,6 +2490,254 @@ func (m *UnsubscribeRequest) Unmarshal(dAtA []byte) error { } return nil } +func (m *ActivateAccountForCatalogRequest) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogtail + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ActivateAccountForCatalogRequest: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ActivateAccountForCatalogRequest: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field AccountId", wireType) + } + m.AccountId = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogtail + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.AccountId |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Seq", wireType) + } + m.Seq = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogtail + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Seq |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipLogtail(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthLogtail + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ActivateAccountForCatalogResponse) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogtail + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ActivateAccountForCatalogResponse: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ActivateAccountForCatalogResponse: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field AccountId", wireType) + } + m.AccountId = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogtail + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.AccountId |= uint32(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Seq", wireType) + } + m.Seq = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogtail + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Seq |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field TargetTs", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogtail + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthLogtail + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthLogtail + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.TargetTs == nil { + m.TargetTs = ×tamp.Timestamp{} + } + if err := m.TargetTs.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Tails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogtail + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthLogtail + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthLogtail + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Tails = append(m.Tails, TableLogtail{}) + if err := m.Tails[len(m.Tails)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipLogtail(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthLogtail + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...) + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *TableLogtail) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -2819,6 +3595,41 @@ func (m *LogtailRequest) Unmarshal(dAtA []byte) error { } m.Request = &LogtailRequest_UnsubscribeTable{v} iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ActivateAccountForCatalog", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogtail + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthLogtail + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthLogtail + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &ActivateAccountForCatalogRequest{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Request = &LogtailRequest_ActivateAccountForCatalog{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipLogtail(dAtA[iNdEx:]) @@ -3029,6 +3840,41 @@ func (m *LogtailResponse) Unmarshal(dAtA []byte) error { } m.Response = &LogtailResponse_Error{v} iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field ActivateAccountForCatalogResponse", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowLogtail + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthLogtail + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthLogtail + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &ActivateAccountForCatalogResponse{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Response = &LogtailResponse_ActivateAccountForCatalogResponse{v} + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipLogtail(dAtA[iNdEx:]) diff --git a/pkg/tests/issues/lazy_catalog_reconnect_test.go b/pkg/tests/issues/lazy_catalog_reconnect_test.go new file mode 100644 index 0000000000000..8e963378d757f --- /dev/null +++ b/pkg/tests/issues/lazy_catalog_reconnect_test.go @@ -0,0 +1,279 @@ +// Copyright 2021 - 2024 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package issues + +import ( + "context" + "fmt" + "testing" + "time" + + "github.com/matrixorigin/matrixone/pkg/cnservice" + "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/container/vector" + "github.com/matrixorigin/matrixone/pkg/embed" + "github.com/matrixorigin/matrixone/pkg/tests/testutils" + "github.com/matrixorigin/matrixone/pkg/util/executor" + "github.com/matrixorigin/matrixone/pkg/vm/engine/disttae" + "github.com/stretchr/testify/require" +) + +func TestLazyCatalogReconnectRestoresReadyAccounts(t *testing.T) { + c, err := embed.NewCluster(embed.WithCNCount(1)) + require.NoError(t, err) + require.NoError(t, c.Start()) + defer func() { + require.NoError(t, c.Close()) + }() + + ctx, cancel := context.WithTimeout(context.Background(), time.Second*180) + defer cancel() + + op, err := c.GetCNService(0) + require.NoError(t, err) + + cn := op.RawService().(cnservice.Service) + eng := cn.GetEngine().(*disttae.Engine) + logtailClient := eng.PushClient() + + reconnectC := make(chan struct{}, 4) + logtailClient.SetReconnectHandler(func() { + select { + case reconnectC <- struct{}{}: + default: + } + }) + + accountIDA := testutils.CreateAccount(t, c, "lazyreconnecta", "111") + accountIDB := testutils.CreateAccount(t, c, "lazyreconnectb", "111") + accountIDC := testutils.CreateAccount(t, c, "lazyreconnectc", "111") + + dbA := "reconnect_dba" + dbB := "reconnect_dbb" + dbC := "reconnect_dbc" + + require.NoError(t, eng.ActivateTenantCatalog(ctx, uint32(accountIDA))) + require.NoError(t, eng.ActivateTenantCatalog(ctx, uint32(accountIDB))) + + testutils.CreateTestDatabaseWithAccount(t, accountIDA, dbA, op) + testutils.ExecSQLWithAccount(t, accountIDA, dbA, op, + "create table ta(a int)", + "insert into ta values (1), (2)", + ) + testutils.CreateTestDatabaseWithAccount(t, accountIDB, dbB, op) + testutils.ExecSQLWithAccount(t, accountIDB, dbB, op, + "create table tb(b int)", + "insert into tb values (21), (22), (23)", + ) + + maxTS := types.MaxTs().ToTimestamp() + require.True(t, eng.PushClient().CanServeAccount(uint32(accountIDA), maxTS)) + require.True(t, eng.PushClient().CanServeAccount(uint32(accountIDB), maxTS)) + require.False(t, eng.PushClient().CanServeAccount(uint32(accountIDC), maxTS)) + + require.NoError(t, logtailClient.Disconnect()) + waitReconnectReady(t, cn, reconnectC) + + require.True(t, eng.PushClient().CanServeAccount(uint32(accountIDA), maxTS)) + require.True(t, eng.PushClient().CanServeAccount(uint32(accountIDB), maxTS)) + require.False(t, eng.PushClient().CanServeAccount(uint32(accountIDC), maxTS)) + + require.True(t, databaseExistsWithAccount(t, accountIDA, dbA, op)) + require.False(t, databaseExistsWithAccount(t, accountIDA, dbB, op)) + require.False(t, databaseExistsWithAccount(t, accountIDA, dbC, op)) + require.True(t, testutils.TableExistsWithAccount(t, accountIDA, dbA, "ta", op)) + require.Equal(t, int64(2), queryCountWithAccount(t, accountIDA, dbA, op, "select count(*) from ta")) + + require.True(t, databaseExistsWithAccount(t, accountIDB, dbB, op)) + require.False(t, databaseExistsWithAccount(t, accountIDB, dbA, op)) + require.False(t, databaseExistsWithAccount(t, accountIDB, dbC, op)) + require.True(t, testutils.TableExistsWithAccount(t, accountIDB, dbB, "tb", op)) + require.Equal(t, int64(3), queryCountWithAccount(t, accountIDB, dbB, op, "select count(*) from tb")) + + require.NoError(t, eng.ActivateTenantCatalog(ctx, uint32(accountIDC))) + testutils.CreateTestDatabaseWithAccount(t, accountIDC, dbC, op) + testutils.ExecSQLWithAccount(t, accountIDC, dbC, op, + "create table tc(c int)", + "insert into tc values (31)", + ) + require.True(t, eng.PushClient().CanServeAccount(uint32(accountIDC), maxTS)) + + require.NoError(t, logtailClient.Disconnect()) + waitReconnectReady(t, cn, reconnectC) + + require.True(t, eng.PushClient().CanServeAccount(uint32(accountIDA), maxTS)) + require.True(t, eng.PushClient().CanServeAccount(uint32(accountIDB), maxTS)) + require.True(t, eng.PushClient().CanServeAccount(uint32(accountIDC), maxTS)) + + require.True(t, databaseExistsWithAccount(t, accountIDA, dbA, op)) + require.False(t, databaseExistsWithAccount(t, accountIDA, dbB, op)) + require.False(t, databaseExistsWithAccount(t, accountIDA, dbC, op)) + require.True(t, testutils.TableExistsWithAccount(t, accountIDA, dbA, "ta", op)) + require.Equal(t, int64(2), queryCountWithAccount(t, accountIDA, dbA, op, "select count(*) from ta")) + + require.True(t, databaseExistsWithAccount(t, accountIDB, dbB, op)) + require.False(t, databaseExistsWithAccount(t, accountIDB, dbA, op)) + require.False(t, databaseExistsWithAccount(t, accountIDB, dbC, op)) + require.True(t, testutils.TableExistsWithAccount(t, accountIDB, dbB, "tb", op)) + require.Equal(t, int64(3), queryCountWithAccount(t, accountIDB, dbB, op, "select count(*) from tb")) + + require.True(t, databaseExistsWithAccount(t, accountIDC, dbC, op)) + require.False(t, databaseExistsWithAccount(t, accountIDC, dbA, op)) + require.False(t, databaseExistsWithAccount(t, accountIDC, dbB, op)) + require.True(t, testutils.TableExistsWithAccount(t, accountIDC, dbC, "tc", op)) + require.Equal(t, int64(1), queryCountWithAccount(t, accountIDC, dbC, op, "select count(*) from tc")) +} + +func TestLazyCatalogReconnectPreservesCheckpointAndInMemoryCatalogMix(t *testing.T) { + c, err := embed.NewCluster(embed.WithCNCount(1)) + require.NoError(t, err) + require.NoError(t, c.Start()) + defer func() { + require.NoError(t, c.Close()) + }() + + ctx, cancel := context.WithTimeout(context.Background(), time.Second*180) + defer cancel() + + op, err := c.GetCNService(0) + require.NoError(t, err) + + cn := op.RawService().(cnservice.Service) + eng := cn.GetEngine().(*disttae.Engine) + logtailClient := eng.PushClient() + + reconnectC := make(chan struct{}, 4) + logtailClient.SetReconnectHandler(func() { + select { + case reconnectC <- struct{}{}: + default: + } + }) + + accountIDA := testutils.CreateAccount(t, c, "lazyckpmixa", "111") + accountIDB := testutils.CreateAccount(t, c, "lazyckpmixb", "111") + + dbA := "mix_ckp_a" + dbB := "mix_ckp_b" + + require.NoError(t, eng.ActivateTenantCatalog(ctx, uint32(accountIDA))) + require.NoError(t, eng.ActivateTenantCatalog(ctx, uint32(accountIDB))) + + testutils.CreateTestDatabaseWithAccount(t, accountIDA, dbA, op) + testutils.ExecSQLWithAccount(t, accountIDA, dbA, op, + "create table t_flush(a int)", + "insert into t_flush values (1), (2)", + ) + testutils.CreateTestDatabaseWithAccount(t, accountIDB, dbB, op) + testutils.ExecSQLWithAccount(t, accountIDB, dbB, op, + "create table t_flush(b int)", + "insert into t_flush values (21), (22), (23)", + ) + + testutils.ExecSQLWithAccount(t, accountIDA, dbA, op, + fmt.Sprintf("select mo_ctl('dn','flush','%s.t_flush')", dbA), + ) + testutils.ExecSQLWithAccount(t, accountIDB, dbB, op, + fmt.Sprintf("select mo_ctl('dn','flush','%s.t_flush')", dbB), + ) + testutils.ExecSQL(t, "mo_catalog", op, "select mo_ctl('dn','checkpoint','')") + + testutils.ExecSQLWithAccount(t, accountIDA, dbA, op, + "create table t_mem(a int)", + "insert into t_mem values (3), (4)", + ) + testutils.ExecSQLWithAccount(t, accountIDB, dbB, op, + "create table t_mem(b int)", + "insert into t_mem values (24), (25)", + ) + + require.NoError(t, logtailClient.Disconnect()) + waitReconnectReady(t, cn, reconnectC) + + maxTS := types.MaxTs().ToTimestamp() + require.True(t, eng.PushClient().CanServeAccount(uint32(accountIDA), maxTS)) + require.True(t, eng.PushClient().CanServeAccount(uint32(accountIDB), maxTS)) + + require.True(t, databaseExistsWithAccount(t, accountIDA, dbA, op)) + require.False(t, databaseExistsWithAccount(t, accountIDA, dbB, op)) + require.True(t, testutils.TableExistsWithAccount(t, accountIDA, dbA, "t_flush", op)) + require.True(t, testutils.TableExistsWithAccount(t, accountIDA, dbA, "t_mem", op)) + require.Equal(t, int64(2), queryCountWithAccount(t, accountIDA, dbA, op, "select count(*) from t_flush")) + require.Equal(t, int64(2), queryCountWithAccount(t, accountIDA, dbA, op, "select count(*) from t_mem")) + + require.True(t, databaseExistsWithAccount(t, accountIDB, dbB, op)) + require.False(t, databaseExistsWithAccount(t, accountIDB, dbA, op)) + require.True(t, testutils.TableExistsWithAccount(t, accountIDB, dbB, "t_flush", op)) + require.True(t, testutils.TableExistsWithAccount(t, accountIDB, dbB, "t_mem", op)) + require.Equal(t, int64(3), queryCountWithAccount(t, accountIDB, dbB, op, "select count(*) from t_flush")) + require.Equal(t, int64(2), queryCountWithAccount(t, accountIDB, dbB, op, "select count(*) from t_mem")) +} + +func waitReconnectReady(t *testing.T, cn cnservice.Service, reconnectC <-chan struct{}) { + t.Helper() + select { + case <-reconnectC: + case <-time.After(time.Second * 30): + t.Fatal("timed out waiting for reconnect signal") + } + waitLogtailResume(cn) +} + +func databaseExistsWithAccount( + t *testing.T, + account int32, + name string, + cn embed.ServiceOperator, +) bool { + t.Helper() + exists := false + testutils.ExecSQLWithReadResultAndAccount( + t, + account, + "mo_catalog", + cn, + func(i int, s string, r executor.Result) { + exists = testutils.HasName(name, r) + }, + fmt.Sprintf("show databases like '%s'", name), + ) + return exists +} + +func queryCountWithAccount( + t *testing.T, + account int32, + db string, + cn embed.ServiceOperator, + statement string, +) int64 { + t.Helper() + var count int64 = -1 + testutils.ExecSQLWithReadResultAndAccount( + t, + account, + db, + cn, + func(i int, s string, r executor.Result) { + r.ReadRows(func(rows int, cols []*vector.Vector) bool { + count = executor.GetFixedRows[int64](cols[0])[0] + return false + }) + }, + statement, + ) + require.NotEqual(t, int64(-1), count) + return count +} diff --git a/pkg/util/status/server.go b/pkg/util/status/server.go index fa663ca886ca8..9e7bfa4ed07cc 100644 --- a/pkg/util/status/server.go +++ b/pkg/util/status/server.go @@ -16,11 +16,16 @@ package status import ( "encoding/json" + "fmt" "net/http" + "sort" + "strconv" + "strings" "sync" "github.com/matrixorigin/matrixone/pkg/lockservice" "github.com/matrixorigin/matrixone/pkg/logservice" + "github.com/matrixorigin/matrixone/pkg/pb/timestamp" "github.com/matrixorigin/matrixone/pkg/txn/client" "github.com/matrixorigin/matrixone/pkg/vm/engine/disttae" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logtail/service" @@ -32,6 +37,7 @@ type CNInstance struct { TxnClient client.TxnClient LockService lockservice.LockService logtailClient *disttae.PushClient + Engine *disttae.Engine } type Server struct { @@ -93,13 +99,38 @@ func (s *Server) SetLogTailClient(uuid string, c *disttae.PushClient) { s.mu.CNInstances[uuid].logtailClient = c } -func (s *Server) ServeHTTP(w http.ResponseWriter, _ *http.Request) { - data, err := s.Dump() - if err != nil { - w.Write([]byte(err.Error())) - return +func (s *Server) SetEngine(uuid string, e *disttae.Engine) { + s.mu.Lock() + defer s.mu.Unlock() + _, ok := s.mu.CNInstances[uuid] + if !ok { + s.mu.CNInstances[uuid] = &CNInstance{} + } + s.mu.CNInstances[uuid].Engine = e +} + +func (s *Server) ServeHTTP(w http.ResponseWriter, r *http.Request) { + path := strings.TrimSuffix(r.URL.Path, "/") + switch path { + case "/debug/status": + data, err := s.Dump() + if err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + w.Header().Set("Content-Type", "application/json") + w.Write(data) + case "/debug/status/catalog": + s.serveCatalog(w, r) + case "/debug/status/catalog-cache": + s.serveCatalogCache(w, r) + case "/debug/status/catalog-activation": + s.serveCatalogActivation(w, r) + case "/debug/status/partitions": + s.servePartitions(w, r) + default: + http.NotFound(w, r) } - w.Write(data) } func (s *Server) Dump() ([]byte, error) { @@ -115,3 +146,264 @@ func (s *Server) Dump() ([]byte, error) { status.fillCNStatus(s.mu.CNInstances) return json.MarshalIndent(status, "", JsonIdent) } + +func (s *Server) serveCatalog(w http.ResponseWriter, r *http.Request) { + accountFilter, err := optionalUint32Param(r, "account") + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + snapshotFilter, err := optionalTimestampParam(r, "snapshot") + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + + cnUUID, instance, err := s.selectCNInstance(r.URL.Query().Get("cn")) + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + if instance.Engine == nil { + http.Error(w, "selected CN instance does not have a disttae engine registered", http.StatusBadRequest) + return + } + + resp := struct { + CNUUID string `json:"cn_uuid"` + Catalog disttae.DebugCatalogState `json:"catalog"` + }{ + CNUUID: cnUUID, + Catalog: instance.Engine.DebugCatalogState(accountFilter, snapshotFilter), + } + writeJSON(w, resp) +} + +func (s *Server) serveCatalogCache(w http.ResponseWriter, r *http.Request) { + accountID, err := requiredUint32Param(r, "account") + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + snapshotFilter, err := optionalTimestampParam(r, "snapshot") + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + limit, err := intParam(r, "limit", 100) + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + + cnUUID, instance, err := s.selectCNInstance(r.URL.Query().Get("cn")) + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + if instance.Engine == nil { + http.Error(w, "selected CN instance does not have a disttae engine registered", http.StatusBadRequest) + return + } + + resp := struct { + CNUUID string `json:"cn_uuid"` + CatalogCache disttae.DebugCatalogCacheState `json:"catalog_cache"` + }{ + CNUUID: cnUUID, + CatalogCache: instance.Engine.DebugCatalogCache(accountID, snapshotFilter, r.URL.Query().Get("db"), limit), + } + writeJSON(w, resp) +} + +func (s *Server) serveCatalogActivation(w http.ResponseWriter, r *http.Request) { + accountFilter, err := optionalUint32Param(r, "account") + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + limit, err := intParam(r, "limit", 50) + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + + cnUUID, instance, err := s.selectCNInstance(r.URL.Query().Get("cn")) + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + if instance.Engine == nil { + http.Error(w, "selected CN instance does not have a disttae engine registered", http.StatusBadRequest) + return + } + + resp := struct { + CNUUID string `json:"cn_uuid"` + CatalogActivation disttae.DebugCatalogActivationState `json:"catalog_activation"` + }{ + CNUUID: cnUUID, + CatalogActivation: instance.Engine.DebugCatalogActivationHistory(accountFilter, limit), + } + writeJSON(w, resp) +} + +func (s *Server) servePartitions(w http.ResponseWriter, r *http.Request) { + dbFilter, err := optionalUint64Param(r, "db") + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + tableFilter, err := optionalUint64Param(r, "table") + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + limit, err := intParam(r, "limit", 100) + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + + cnUUID, instance, err := s.selectCNInstance(r.URL.Query().Get("cn")) + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + if instance.Engine == nil { + http.Error(w, "selected CN instance does not have a disttae engine registered", http.StatusBadRequest) + return + } + + resp := struct { + CNUUID string `json:"cn_uuid"` + Partitions disttae.DebugPartitionsState `json:"partitions"` + }{ + CNUUID: cnUUID, + Partitions: instance.Engine.DebugPartitions(dbFilter, tableFilter, limit), + } + writeJSON(w, resp) +} + +func (s *Server) selectCNInstance(cnUUID string) (string, *CNInstance, error) { + s.mu.Lock() + defer s.mu.Unlock() + + if cnUUID != "" { + instance, ok := s.mu.CNInstances[cnUUID] + if !ok { + return "", nil, fmt.Errorf("cn %q not found", cnUUID) + } + copy := *instance + return cnUUID, ©, nil + } + + switch len(s.mu.CNInstances) { + case 0: + return "", nil, fmt.Errorf("no CN instances registered") + case 1: + for uuid, instance := range s.mu.CNInstances { + copy := *instance + return uuid, ©, nil + } + } + + uuids := make([]string, 0, len(s.mu.CNInstances)) + for uuid := range s.mu.CNInstances { + uuids = append(uuids, uuid) + } + sort.Strings(uuids) + return "", nil, fmt.Errorf("multiple CN instances registered, specify ?cn= (%s)", strings.Join(uuids, ", ")) +} + +func optionalUint32Param(r *http.Request, key string) (*uint32, error) { + value := r.URL.Query().Get(key) + if value == "" { + return nil, nil + } + parsed, err := strconv.ParseUint(value, 10, 32) + if err != nil { + return nil, fmt.Errorf("invalid %s: %w", key, err) + } + result := uint32(parsed) + return &result, nil +} + +func optionalUint64Param(r *http.Request, key string) (*uint64, error) { + value := r.URL.Query().Get(key) + if value == "" { + return nil, nil + } + parsed, err := strconv.ParseUint(value, 10, 64) + if err != nil { + return nil, fmt.Errorf("invalid %s: %w", key, err) + } + return &parsed, nil +} + +func requiredUint32Param(r *http.Request, key string) (uint32, error) { + value := r.URL.Query().Get(key) + if value == "" { + return 0, fmt.Errorf("%s is required", key) + } + parsed, err := strconv.ParseUint(value, 10, 32) + if err != nil { + return 0, fmt.Errorf("invalid %s: %w", key, err) + } + return uint32(parsed), nil +} + +func optionalTimestampParam(r *http.Request, key string) (*timestamp.Timestamp, error) { + value := r.URL.Query().Get(key) + if value == "" { + return nil, nil + } + + parts := strings.Split(value, ":") + if len(parts) > 2 { + return nil, fmt.Errorf("invalid %s: expected physical[:logical]", key) + } + + physical, err := strconv.ParseInt(parts[0], 10, 64) + if err != nil { + return nil, fmt.Errorf("invalid %s physical time: %w", key, err) + } + if physical < 0 { + return nil, fmt.Errorf("%s physical time must be non-negative", key) + } + + ts := ×tamp.Timestamp{PhysicalTime: physical} + if len(parts) == 2 { + logical, err := strconv.ParseUint(parts[1], 10, 32) + if err != nil { + return nil, fmt.Errorf("invalid %s logical time: %w", key, err) + } + ts.LogicalTime = uint32(logical) + } + return ts, nil +} + +func intParam(r *http.Request, key string, defaultValue int) (int, error) { + value := r.URL.Query().Get(key) + if value == "" { + return defaultValue, nil + } + parsed, err := strconv.Atoi(value) + if err != nil { + return 0, fmt.Errorf("invalid %s: %w", key, err) + } + if parsed <= 0 { + return 0, fmt.Errorf("%s must be greater than zero", key) + } + return parsed, nil +} + +func writeJSON(w http.ResponseWriter, value any) { + data, err := json.MarshalIndent(value, "", JsonIdent) + if err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + w.Header().Set("Content-Type", "application/json") + w.Write(data) +} diff --git a/pkg/util/status/server_test.go b/pkg/util/status/server_test.go new file mode 100644 index 0000000000000..091e034a322d1 --- /dev/null +++ b/pkg/util/status/server_test.go @@ -0,0 +1,133 @@ +// Copyright 2021 -2023 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package status + +import ( + "encoding/json" + "net/http" + "net/http/httptest" + "testing" + + "github.com/matrixorigin/matrixone/pkg/vm/engine/disttae" + "github.com/stretchr/testify/require" +) + +func TestServeHTTPCatalog(t *testing.T) { + server := NewServer() + server.SetEngine("cn1", &disttae.Engine{}) + + req := httptest.NewRequest(http.MethodGet, "/debug/status/catalog", nil) + resp := httptest.NewRecorder() + server.ServeHTTP(resp, req) + + require.Equal(t, http.StatusOK, resp.Code) + + var payload struct { + CNUUID string `json:"cn_uuid"` + Catalog map[string]any `json:"catalog"` + } + require.NoError(t, json.Unmarshal(resp.Body.Bytes(), &payload)) + require.Equal(t, "cn1", payload.CNUUID) +} + +func TestServeHTTPCatalogRejectsInvalidSnapshot(t *testing.T) { + server := NewServer() + server.SetEngine("cn1", &disttae.Engine{}) + + req := httptest.NewRequest(http.MethodGet, "/debug/status/catalog?snapshot=bad:value", nil) + resp := httptest.NewRecorder() + server.ServeHTTP(resp, req) + require.Equal(t, http.StatusBadRequest, resp.Code) +} + +func TestServeHTTPCatalogRequiresCNWhenMultipleInstancesRegistered(t *testing.T) { + server := NewServer() + server.SetEngine("cn1", &disttae.Engine{}) + server.SetEngine("cn2", &disttae.Engine{}) + + req := httptest.NewRequest(http.MethodGet, "/debug/status/catalog", nil) + resp := httptest.NewRecorder() + server.ServeHTTP(resp, req) + require.Equal(t, http.StatusBadRequest, resp.Code) + + req = httptest.NewRequest(http.MethodGet, "/debug/status/catalog?cn=cn2", nil) + resp = httptest.NewRecorder() + server.ServeHTTP(resp, req) + require.Equal(t, http.StatusOK, resp.Code) +} + +func TestServeHTTPCatalogCache(t *testing.T) { + server := NewServer() + server.SetEngine("cn1", &disttae.Engine{}) + + req := httptest.NewRequest(http.MethodGet, "/debug/status/catalog-cache?account=1&snapshot=10:2&limit=5", nil) + resp := httptest.NewRecorder() + server.ServeHTTP(resp, req) + + require.Equal(t, http.StatusOK, resp.Code) + + var payload struct { + CNUUID string `json:"cn_uuid"` + CatalogCache map[string]any `json:"catalog_cache"` + } + require.NoError(t, json.Unmarshal(resp.Body.Bytes(), &payload)) + require.Equal(t, "cn1", payload.CNUUID) +} + +func TestServeHTTPCatalogCacheRequiresAccount(t *testing.T) { + server := NewServer() + server.SetEngine("cn1", &disttae.Engine{}) + + req := httptest.NewRequest(http.MethodGet, "/debug/status/catalog-cache", nil) + resp := httptest.NewRecorder() + server.ServeHTTP(resp, req) + require.Equal(t, http.StatusBadRequest, resp.Code) +} + +func TestServeHTTPCatalogActivation(t *testing.T) { + server := NewServer() + server.SetEngine("cn1", &disttae.Engine{}) + + req := httptest.NewRequest(http.MethodGet, "/debug/status/catalog-activation?limit=3", nil) + resp := httptest.NewRecorder() + server.ServeHTTP(resp, req) + + require.Equal(t, http.StatusOK, resp.Code) + + var payload struct { + CNUUID string `json:"cn_uuid"` + CatalogActivation map[string]any `json:"catalog_activation"` + } + require.NoError(t, json.Unmarshal(resp.Body.Bytes(), &payload)) + require.Equal(t, "cn1", payload.CNUUID) +} + +func TestServeHTTPPartitions(t *testing.T) { + server := NewServer() + server.SetEngine("cn1", &disttae.Engine{}) + + req := httptest.NewRequest(http.MethodGet, "/debug/status/partitions?limit=10", nil) + resp := httptest.NewRecorder() + server.ServeHTTP(resp, req) + + require.Equal(t, http.StatusOK, resp.Code) + + var payload struct { + CNUUID string `json:"cn_uuid"` + Partitions map[string]any `json:"partitions"` + } + require.NoError(t, json.Unmarshal(resp.Body.Bytes(), &payload)) + require.Equal(t, "cn1", payload.CNUUID) +} diff --git a/pkg/vm/engine/disttae/cache/debug.go b/pkg/vm/engine/disttae/cache/debug.go new file mode 100644 index 0000000000000..d67cc12ac1805 --- /dev/null +++ b/pkg/vm/engine/disttae/cache/debug.go @@ -0,0 +1,171 @@ +// Copyright 2021 - 2023 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package cache + +import ( + "strconv" + + "github.com/matrixorigin/matrixone/pkg/pb/timestamp" +) + +type DebugCatalogSummary struct { + Start timestamp.Timestamp `json:"start"` + End timestamp.Timestamp `json:"end"` + DatabaseItems int `json:"database_items"` + TableItems int `json:"table_items"` + CanServeLatest bool `json:"can_serve_latest"` +} + +type DebugCatalogDatabaseItem struct { + ID uint64 `json:"id"` + Name string `json:"name"` + Type string `json:"type"` + TableCount int `json:"table_count"` +} + +type DebugCatalogTableItem struct { + ID uint64 `json:"id"` + Name string `json:"name"` + DatabaseID uint64 `json:"database_id"` + DatabaseName string `json:"database_name"` + Kind string `json:"kind"` + DefinitionCount int `json:"definition_count"` + ColumnCount int `json:"column_count"` + CreateSQL string `json:"create_sql,omitempty"` + LogicalID uint64 `json:"logical_id"` +} + +type DebugCatalogAccountContents struct { + SnapshotTS timestamp.Timestamp `json:"snapshot_ts"` + VisibleDatabaseCount int `json:"visible_database_count"` + ReturnedDatabaseCount int `json:"returned_database_count"` + VisibleTableCount int `json:"visible_table_count"` + ReturnedTableCount int `json:"returned_table_count"` + Databases []DebugCatalogDatabaseItem `json:"databases"` + Tables []DebugCatalogTableItem `json:"tables"` +} + +func (cc *CatalogCache) DebugSummary() DebugCatalogSummary { + if cc == nil { + return DebugCatalogSummary{} + } + + cc.mu.Lock() + start, end := cc.mu.start, cc.mu.end + cc.mu.Unlock() + + return DebugCatalogSummary{ + Start: start.ToTimestamp(), + End: end.ToTimestamp(), + DatabaseItems: cc.databases.data.Len(), + TableItems: cc.tables.data.Len(), + CanServeLatest: cc.CanServe(end), + } +} + +func (cc *CatalogCache) DebugAccountContents( + accountID uint32, + ts timestamp.Timestamp, + dbFilter string, + limit int, +) DebugCatalogAccountContents { + contents := DebugCatalogAccountContents{ + SnapshotTS: ts, + Databases: make([]DebugCatalogDatabaseItem, 0), + Tables: make([]DebugCatalogTableItem, 0), + } + if cc == nil { + return contents + } + if limit <= 0 { + limit = 100 + } + + key := &DatabaseItem{ + AccountId: accountID, + } + seen := make(map[string]struct{}) + cc.databases.data.Ascend(key, func(item *DatabaseItem) bool { + if item.AccountId != accountID { + return false + } + if item.Ts.Greater(ts) { + return true + } + if _, ok := seen[item.Name]; ok { + return true + } + seen[item.Name] = struct{}{} + if item.deleted || !matchesDebugDatabaseFilter(dbFilter, item) { + return true + } + + tableNames, tableIDs := cc.Tables(accountID, item.Id, ts) + contents.VisibleDatabaseCount++ + contents.VisibleTableCount += len(tableNames) + + if len(contents.Databases) < limit { + contents.Databases = append(contents.Databases, DebugCatalogDatabaseItem{ + ID: item.Id, + Name: item.Name, + Type: item.Typ, + TableCount: len(tableNames), + }) + } + + for i, tableName := range tableNames { + if len(contents.Tables) >= limit { + continue + } + + var tableID uint64 + if i < len(tableIDs) { + tableID = tableIDs[i] + } + tableItem := cc.GetTableByIdAndTime(accountID, item.Id, tableID, ts) + debugItem := DebugCatalogTableItem{ + ID: tableID, + Name: tableName, + DatabaseID: item.Id, + DatabaseName: item.Name, + } + if tableItem != nil { + debugItem.ID = tableItem.Id + debugItem.Name = tableItem.Name + debugItem.Kind = tableItem.Kind + debugItem.DefinitionCount = len(tableItem.Defs) + debugItem.CreateSQL = tableItem.CreateSql + debugItem.LogicalID = tableItem.LogicalId + if tableItem.TableDef != nil { + debugItem.ColumnCount = len(tableItem.TableDef.Cols) + } + } + contents.Tables = append(contents.Tables, debugItem) + } + + return true + }) + + contents.ReturnedDatabaseCount = len(contents.Databases) + contents.ReturnedTableCount = len(contents.Tables) + return contents +} + +func matchesDebugDatabaseFilter(filter string, item *DatabaseItem) bool { + if filter == "" || item == nil { + return true + } + return filter == item.Name || filter == strconv.FormatUint(item.Id, 10) +} diff --git a/pkg/vm/engine/disttae/cache/debug_test.go b/pkg/vm/engine/disttae/cache/debug_test.go new file mode 100644 index 0000000000000..f38023f3d52cd --- /dev/null +++ b/pkg/vm/engine/disttae/cache/debug_test.go @@ -0,0 +1,123 @@ +// Copyright 2024 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package cache + +import ( + "testing" + + "github.com/matrixorigin/matrixone/pkg/pb/timestamp" + "github.com/matrixorigin/matrixone/pkg/vm/engine" + "github.com/stretchr/testify/require" +) + +func TestDebugAccountContents(t *testing.T) { + cc := NewCatalog() + ts := timestamp.Timestamp{PhysicalTime: 100} + + cc.databases.data.Set(&DatabaseItem{ + AccountId: 1, + Name: "db1", + Ts: ts, + Id: 10, + Typ: "USER", + }) + cc.databases.data.Set(&DatabaseItem{ + AccountId: 1, + Name: "db2", + Ts: ts, + Id: 20, + Typ: "USER", + }) + cc.databases.data.Set(&DatabaseItem{ + AccountId: 2, + Name: "dbx", + Ts: ts, + Id: 30, + Typ: "USER", + }) + + cc.tables.data.Set(&TableItem{ + AccountId: 1, + DatabaseId: 10, + Name: "t1", + Ts: ts, + Id: 11, + DatabaseName: "db1", + Kind: "TABLE", + Defs: make([]engine.TableDef, 2), + LogicalId: 1011, + }) + cc.tables.data.Set(&TableItem{ + AccountId: 1, + DatabaseId: 10, + Name: "t2", + Ts: ts, + Id: 12, + DatabaseName: "db1", + Kind: "TABLE", + Defs: make([]engine.TableDef, 3), + LogicalId: 1012, + }) + cc.tables.data.Set(&TableItem{ + AccountId: 1, + DatabaseId: 20, + Name: "t3", + Ts: ts, + Id: 21, + DatabaseName: "db2", + Kind: "VIEW", + Defs: make([]engine.TableDef, 1), + LogicalId: 2021, + }) + cc.tables.data.Set(&TableItem{ + AccountId: 2, + DatabaseId: 30, + Name: "other", + Ts: ts, + Id: 31, + DatabaseName: "dbx", + Kind: "TABLE", + }) + + got := cc.DebugAccountContents(1, ts, "db1", 10) + require.Equal(t, ts, got.SnapshotTS) + require.Equal(t, 1, got.VisibleDatabaseCount) + require.Equal(t, 2, got.VisibleTableCount) + require.Equal(t, 1, got.ReturnedDatabaseCount) + require.Equal(t, 2, got.ReturnedTableCount) + require.Len(t, got.Databases, 1) + require.Equal(t, uint64(10), got.Databases[0].ID) + require.Equal(t, "db1", got.Databases[0].Name) + require.Equal(t, 2, got.Databases[0].TableCount) + require.Len(t, got.Tables, 2) + require.Equal(t, "db1", got.Tables[0].DatabaseName) + require.Equal(t, 2, got.Tables[0].DefinitionCount) + require.Equal(t, uint64(1011), got.Tables[0].LogicalID) + + got = cc.DebugAccountContents(1, ts, "20", 1) + require.Equal(t, 1, got.VisibleDatabaseCount) + require.Equal(t, 1, got.VisibleTableCount) + require.Equal(t, 1, got.ReturnedDatabaseCount) + require.Equal(t, 1, got.ReturnedTableCount) + require.Equal(t, "db2", got.Databases[0].Name) + require.Equal(t, "t3", got.Tables[0].Name) + require.Equal(t, "VIEW", got.Tables[0].Kind) + + empty := cc.DebugAccountContents(99, ts, "", 10) + require.NotNil(t, empty.Databases) + require.NotNil(t, empty.Tables) + require.Empty(t, empty.Databases) + require.Empty(t, empty.Tables) +} diff --git a/pkg/vm/engine/disttae/debug_state.go b/pkg/vm/engine/disttae/debug_state.go new file mode 100644 index 0000000000000..2f73638c8a9a5 --- /dev/null +++ b/pkg/vm/engine/disttae/debug_state.go @@ -0,0 +1,526 @@ +// Copyright 2024 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package disttae + +import ( + "sort" + "time" + + "github.com/matrixorigin/matrixone/pkg/catalog" + "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/pb/timestamp" + "github.com/matrixorigin/matrixone/pkg/vm/engine/disttae/cache" + "github.com/matrixorigin/matrixone/pkg/vm/engine/disttae/logtailreplay" +) + +type DebugCatalogState struct { + LatestLogtailAppliedTS timestamp.Timestamp `json:"latest_logtail_applied_ts"` + CatalogCache cache.DebugCatalogSummary `json:"catalog_cache"` + LazyCatalog DebugLazyCatalogState `json:"lazy_catalog"` + Account *DebugCatalogAccountState `json:"account,omitempty"` +} + +type DebugLazyCatalogState struct { + Enabled bool `json:"enabled"` + CatchingUpCount int `json:"catching_up_count"` + PendingActivationResponseCount int `json:"pending_activation_response_count"` + InflightActivationCount int `json:"inflight_activation_count"` + WantedAccounts []uint32 `json:"wanted_accounts"` + Accounts []DebugLazyAccountState `json:"accounts"` +} + +type DebugLazyAccountState struct { + AccountID uint32 `json:"account_id"` + State string `json:"state"` + ReadyTS *timestamp.Timestamp `json:"ready_ts,omitempty"` + PendingSeq uint64 `json:"pending_seq"` + DelayedApplyCount int `json:"delayed_apply_count"` +} + +type DebugCatalogAccountState struct { + AccountID uint32 `json:"account_id"` + Present bool `json:"present"` + State string `json:"state"` + InWantedAccounts bool `json:"in_wanted_accounts"` + PendingSeq uint64 `json:"pending_seq"` + DelayedApplyCount int `json:"delayed_apply_count"` + ReadyTS *timestamp.Timestamp `json:"ready_ts,omitempty"` + LatestLogtailAppliedTS timestamp.Timestamp `json:"latest_logtail_applied_ts"` + RequestedSnapshotTS *timestamp.Timestamp `json:"requested_snapshot_ts,omitempty"` + CanServeLatest bool `json:"can_serve_latest"` + CanServeSnapshot *bool `json:"can_serve_snapshot,omitempty"` +} + +type DebugCatalogCacheState struct { + AccountID uint32 `json:"account_id"` + LatestLogtailAppliedTS timestamp.Timestamp `json:"latest_logtail_applied_ts"` + SnapshotTS timestamp.Timestamp `json:"snapshot_ts"` + DatabaseFilter string `json:"database_filter,omitempty"` + CanServeLatest bool `json:"can_serve_latest"` + CanServeSnapshot bool `json:"can_serve_snapshot"` + Contents cache.DebugCatalogAccountContents `json:"contents"` +} + +type DebugCatalogActivationState struct { + LatestLogtailAppliedTS timestamp.Timestamp `json:"latest_logtail_applied_ts"` + ReturnedEventCount int `json:"returned_event_count"` + Events []DebugCatalogActivationEvent `json:"events"` +} + +type DebugCatalogActivationEvent struct { + AccountID uint32 `json:"account_id"` + Seq uint64 `json:"seq"` + Source string `json:"source"` + Phase string `json:"phase"` + Result string `json:"result"` + Error string `json:"error,omitempty"` + TargetTS *timestamp.Timestamp `json:"target_ts,omitempty"` + ReplayTS *timestamp.Timestamp `json:"replay_ts,omitempty"` + DelayedApplyCount int `json:"delayed_apply_count,omitempty"` + StartedAt *time.Time `json:"started_at,omitempty"` + FinishedAt *time.Time `json:"finished_at,omitempty"` +} + +type DebugPartitionsState struct { + LatestLogtailAppliedTS timestamp.Timestamp `json:"latest_logtail_applied_ts"` + LivePartitionCount int `json:"live_partition_count"` + ReturnedPartitionCount int `json:"returned_partition_count"` + SnapshotConfig DebugSnapshotGCConfig `json:"snapshot_config"` + SnapshotMetrics DebugSnapshotMetrics `json:"snapshot_metrics"` + Partitions []DebugPartitionInfo `json:"partitions"` +} + +type DebugSnapshotGCConfig struct { + Enabled bool `json:"enabled"` + GCInterval string `json:"gc_interval"` + MaxAge string `json:"max_age"` + MaxSnapshotsPerTable int `json:"max_snapshots_per_table"` + MaxTotalSnapshots int `json:"max_total_snapshots"` +} + +type DebugSnapshotMetrics struct { + TotalSnapshots int64 `json:"total_snapshots"` + TotalTables int64 `json:"total_tables"` + GCRuns int64 `json:"gc_runs"` + SnapshotsGCed int64 `json:"snapshots_gced"` + LastGCDuration int64 `json:"last_gc_duration_ns"` + SnapshotHits int64 `json:"snapshot_hits"` + SnapshotMisses int64 `json:"snapshot_misses"` + SnapshotCreates int64 `json:"snapshot_creates"` + LRUEvictions int64 `json:"lru_evictions"` + AgeEvictions int64 `json:"age_evictions"` +} + +type DebugPartitionInfo struct { + DatabaseID uint64 `json:"database_id"` + DatabaseName string `json:"database_name,omitempty"` + TableID uint64 `json:"table_id"` + TableName string `json:"table_name,omitempty"` + TableInfoKnown bool `json:"table_info_known"` + StateMaterialized bool `json:"state_materialized"` + CheckpointConsumed bool `json:"checkpoint_consumed"` + State logtailreplay.DebugPartitionStateSummary `json:"state"` +} + +type debugPartitionRef struct { + dbID uint64 + tblID uint64 + part *logtailreplay.Partition +} + +const debugActivationHistoryLimit = 128 + +func (e *Engine) DebugCatalogState(accountFilter *uint32, snapshotFilter *timestamp.Timestamp) DebugCatalogState { + var state DebugCatalogState + if e == nil { + return state + } + + state.LatestLogtailAppliedTS = e.PushClient().LatestLogtailAppliedTime() + if cc := e.GetLatestCatalogCache(); cc != nil { + state.CatalogCache = cc.DebugSummary() + } + state.LazyCatalog = e.PushClient().DebugLazyCatalogState(accountFilter) + if accountFilter != nil { + accountState := e.PushClient().DebugCatalogAccountState(*accountFilter) + accountState.LatestLogtailAppliedTS = state.LatestLogtailAppliedTS + accountState.CanServeLatest = debugCanServeAccountAt(e, *accountFilter, state.LatestLogtailAppliedTS) + if snapshotFilter != nil { + ts := *snapshotFilter + accountState.RequestedSnapshotTS = &ts + canServe := debugCanServeAccountAt(e, *accountFilter, ts) + accountState.CanServeSnapshot = &canServe + } + state.Account = &accountState + } + return state +} + +func (c *PushClient) DebugLazyCatalogState(accountFilter *uint32) DebugLazyCatalogState { + if c == nil || c.lazyCatalog == nil { + return DebugLazyCatalogState{} + } + return c.lazyCatalog.debugState(accountFilter) +} + +func (c *PushClient) DebugCatalogAccountState(accountID uint32) DebugCatalogAccountState { + state := DebugCatalogAccountState{ + AccountID: accountID, + State: "ungated", + } + if c == nil || c.lazyCatalog == nil { + return state + } + return c.lazyCatalog.debugAccountState(accountID) +} + +func (c *PushClient) DebugCatalogActivationHistory(accountFilter *uint32, limit int) DebugCatalogActivationState { + state := DebugCatalogActivationState{ + Events: make([]DebugCatalogActivationEvent, 0), + } + if c == nil { + return state + } + state.LatestLogtailAppliedTS = c.LatestLogtailAppliedTime() + if c.lazyCatalog == nil { + return state + } + state.Events = c.lazyCatalog.debugActivationHistory(accountFilter, limit) + state.ReturnedEventCount = len(state.Events) + return state +} + +func (s *lazyCatalogCNState) debugState(accountFilter *uint32) DebugLazyCatalogState { + if s == nil { + return DebugLazyCatalogState{} + } + + s.mu.Lock() + defer s.mu.Unlock() + + state := DebugLazyCatalogState{ + Enabled: s.enabled.Load(), + CatchingUpCount: int(s.catchingUpCount.Load()), + PendingActivationResponseCount: len(s.pendingActivationResponses), + WantedAccounts: make([]uint32, 0), + Accounts: make([]DebugLazyAccountState, 0), + } + + s.inflightActivations.Range(func(_, _ any) bool { + state.InflightActivationCount++ + return true + }) + + for accountID := range s.wantedAccounts { + if accountFilter != nil && *accountFilter != accountID { + continue + } + state.WantedAccounts = append(state.WantedAccounts, accountID) + } + sort.Slice(state.WantedAccounts, func(i, j int) bool { + return state.WantedAccounts[i] < state.WantedAccounts[j] + }) + + for accountID, entry := range s.accounts { + if accountFilter != nil && *accountFilter != accountID { + continue + } + accountState := DebugLazyAccountState{ + AccountID: accountID, + State: accountReadyStateString(entry.state), + PendingSeq: s.pendingSeq[accountID], + DelayedApplyCount: len(s.accountDCA[accountID]), + } + if entry.state == accountReady && (entry.readyTS.PhysicalTime != 0 || entry.readyTS.LogicalTime != 0) { + ts := entry.readyTS + accountState.ReadyTS = &ts + } + state.Accounts = append(state.Accounts, accountState) + } + sort.Slice(state.Accounts, func(i, j int) bool { + return state.Accounts[i].AccountID < state.Accounts[j].AccountID + }) + + return state +} + +func (s *lazyCatalogCNState) debugAccountState(accountID uint32) DebugCatalogAccountState { + state := DebugCatalogAccountState{ + AccountID: accountID, + State: "ungated", + } + if s == nil { + return state + } + + s.mu.Lock() + defer s.mu.Unlock() + + if !s.enabled.Load() { + return state + } + state.State = "absent" + _, state.InWantedAccounts = s.wantedAccounts[accountID] + + entry, ok := s.accounts[accountID] + if !ok { + return state + } + state.Present = true + state.State = accountReadyStateString(entry.state) + state.PendingSeq = s.pendingSeq[accountID] + state.DelayedApplyCount = len(s.accountDCA[accountID]) + if entry.state == accountReady && (entry.readyTS.PhysicalTime != 0 || entry.readyTS.LogicalTime != 0) { + ts := entry.readyTS + state.ReadyTS = &ts + } + return state +} + +func (s *lazyCatalogCNState) recordActivationEvent(event DebugCatalogActivationEvent) { + if s == nil { + return + } + cloned := cloneDebugCatalogActivationEvent(event) + + s.mu.Lock() + defer s.mu.Unlock() + if len(s.activationHistory) >= debugActivationHistoryLimit { + copy(s.activationHistory, s.activationHistory[1:]) + s.activationHistory[len(s.activationHistory)-1] = cloned + return + } + s.activationHistory = append(s.activationHistory, cloned) +} + +func (s *lazyCatalogCNState) debugActivationHistory(accountFilter *uint32, limit int) []DebugCatalogActivationEvent { + if s == nil { + return []DebugCatalogActivationEvent{} + } + if limit <= 0 { + limit = 50 + } + + s.mu.Lock() + defer s.mu.Unlock() + + events := make([]DebugCatalogActivationEvent, 0, min(limit, len(s.activationHistory))) + for i := len(s.activationHistory) - 1; i >= 0 && len(events) < limit; i-- { + event := s.activationHistory[i] + if accountFilter != nil && event.AccountID != *accountFilter { + continue + } + events = append(events, cloneDebugCatalogActivationEvent(event)) + } + return events +} + +func accountReadyStateString(state catalogReadyState) string { + switch state { + case accountInactive: + return "inactive" + case accountCatchingUp: + return "catching_up" + case accountReadyDraining: + return "ready_draining" + case accountReady: + return "ready" + default: + return "unknown" + } +} + +func (e *Engine) DebugCatalogCache( + accountID uint32, + snapshotFilter *timestamp.Timestamp, + dbFilter string, + limit int, +) DebugCatalogCacheState { + state := DebugCatalogCacheState{ + AccountID: accountID, + } + if e == nil { + return state + } + + state.LatestLogtailAppliedTS = e.PushClient().LatestLogtailAppliedTime() + state.SnapshotTS = state.LatestLogtailAppliedTS + if snapshotFilter != nil { + state.SnapshotTS = *snapshotFilter + } + state.DatabaseFilter = dbFilter + state.CanServeLatest = debugCanServeAccountAt(e, accountID, state.LatestLogtailAppliedTS) + state.CanServeSnapshot = debugCanServeAccountAt(e, accountID, state.SnapshotTS) + + if cc := e.GetLatestCatalogCache(); cc != nil { + state.Contents = cc.DebugAccountContents(accountID, state.SnapshotTS, dbFilter, limit) + } + return state +} + +func (e *Engine) DebugCatalogActivationHistory( + accountFilter *uint32, + limit int, +) DebugCatalogActivationState { + if e == nil { + return DebugCatalogActivationState{} + } + return e.PushClient().DebugCatalogActivationHistory(accountFilter, limit) +} + +func (e *Engine) DebugPartitions(dbFilter, tableFilter *uint64, limit int) DebugPartitionsState { + state := DebugPartitionsState{} + if e == nil { + return state + } + + state.LatestLogtailAppliedTS = e.PushClient().LatestLogtailAppliedTime() + if e.snapshotMgr != nil { + cfg := e.snapshotMgr.GetConfig() + metrics := e.snapshotMgr.GetMetrics() + state.SnapshotConfig = DebugSnapshotGCConfig{ + Enabled: cfg.Enabled, + GCInterval: cfg.GCInterval.String(), + MaxAge: cfg.MaxAge.String(), + MaxSnapshotsPerTable: cfg.MaxSnapshotsPerTable, + MaxTotalSnapshots: cfg.MaxTotalSnapshots, + } + state.SnapshotMetrics = DebugSnapshotMetrics{ + TotalSnapshots: metrics.TotalSnapshots.Load(), + TotalTables: metrics.TotalTables.Load(), + GCRuns: metrics.GCRuns.Load(), + SnapshotsGCed: metrics.SnapshotsGCed.Load(), + LastGCDuration: metrics.LastGCDuration.Load(), + SnapshotHits: metrics.SnapshotHits.Load(), + SnapshotMisses: metrics.SnapshotMisses.Load(), + SnapshotCreates: metrics.SnapshotCreates.Load(), + LRUEvictions: metrics.LRUEvictions.Load(), + AgeEvictions: metrics.AgeEvictions.Load(), + } + } + + if limit <= 0 { + limit = 100 + } + + refs := make([]debugPartitionRef, 0) + e.RLock() + for key, part := range e.partitions { + if dbFilter != nil && key[0] != *dbFilter { + continue + } + if tableFilter != nil && key[1] != *tableFilter { + continue + } + refs = append(refs, debugPartitionRef{ + dbID: key[0], + tblID: key[1], + part: part, + }) + } + e.RUnlock() + + sort.Slice(refs, func(i, j int) bool { + if refs[i].dbID == refs[j].dbID { + return refs[i].tblID < refs[j].tblID + } + return refs[i].dbID < refs[j].dbID + }) + + state.LivePartitionCount = len(refs) + if len(refs) > limit { + refs = refs[:limit] + } + state.ReturnedPartitionCount = len(refs) + + state.Partitions = make([]DebugPartitionInfo, 0, len(refs)) + for _, ref := range refs { + info := DebugPartitionInfo{ + DatabaseID: ref.dbID, + DatabaseName: debugPartitionDatabaseName(ref.dbID), + TableID: ref.tblID, + CheckpointConsumed: ref.part.CheckpointConsumed(), + } + if ref.part.TableInfoOK && ref.part.TableInfo.Name != "" { + info.TableName = ref.part.TableInfo.Name + info.TableInfoKnown = true + } else if name := debugPartitionFallbackTableName(ref.tblID); name != "" { + info.TableName = name + info.TableInfoKnown = true + } + if ps := ref.part.Snapshot(); ps != nil { + info.StateMaterialized = true + info.State = ps.DebugSummary() + } + state.Partitions = append(state.Partitions, info) + } + + return state +} + +func debugCanServeAccountAt(e *Engine, accountID uint32, ts timestamp.Timestamp) bool { + if e == nil || ts.IsEmpty() { + return false + } + cc := e.GetLatestCatalogCache() + if cc == nil || !cc.CanServe(types.TimestampToTS(ts)) { + return false + } + return e.PushClient().CanServeAccount(accountID, ts) +} + +func debugPartitionDatabaseName(databaseID uint64) string { + if databaseID == catalog.MO_CATALOG_ID { + return catalog.MO_CATALOG + } + return "" +} + +func debugPartitionFallbackTableName(tableID uint64) string { + switch tableID { + case catalog.MO_DATABASE_ID: + return catalog.MO_DATABASE + case catalog.MO_TABLES_ID: + return catalog.MO_TABLES + case catalog.MO_COLUMNS_ID: + return catalog.MO_COLUMNS + default: + return "" + } +} + +func cloneDebugCatalogActivationEvent(event DebugCatalogActivationEvent) DebugCatalogActivationEvent { + return DebugCatalogActivationEvent{ + AccountID: event.AccountID, + Seq: event.Seq, + Source: event.Source, + Phase: event.Phase, + Result: event.Result, + Error: event.Error, + TargetTS: clonePtr(event.TargetTS), + ReplayTS: clonePtr(event.ReplayTS), + DelayedApplyCount: event.DelayedApplyCount, + StartedAt: clonePtr(event.StartedAt), + FinishedAt: clonePtr(event.FinishedAt), + } +} + +func clonePtr[T any](p *T) *T { + if p == nil { + return nil + } + v := *p + return &v +} diff --git a/pkg/vm/engine/disttae/debug_state_test.go b/pkg/vm/engine/disttae/debug_state_test.go new file mode 100644 index 0000000000000..f30c880928071 --- /dev/null +++ b/pkg/vm/engine/disttae/debug_state_test.go @@ -0,0 +1,135 @@ +// Copyright 2024 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package disttae + +import ( + "sync/atomic" + "testing" + "time" + + "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/pb/logtail" + "github.com/matrixorigin/matrixone/pkg/pb/timestamp" + "github.com/matrixorigin/matrixone/pkg/vm/engine/disttae/cache" + "github.com/stretchr/testify/require" +) + +func TestLazyCatalogDebugState(t *testing.T) { + state := newLazyCatalogCNState() + state.enabled.Store(true) + state.accounts[20] = &accountCatalogEntry{ + state: accountReady, + readyTS: timestamp.Timestamp{PhysicalTime: 200, LogicalTime: 2}, + } + state.accounts[10] = &accountCatalogEntry{ + state: accountCatchingUp, + } + state.pendingSeq[10] = 7 + state.accountDCA[10] = []func(){func() {}, func() {}} + state.catchingUpCount.Store(1) + state.wantedAccounts[0] = struct{}{} + state.wantedAccounts[20] = struct{}{} + state.pendingActivationResponses[activationResponseKey{accountID: 10, seq: 7}] = make(chan *logtail.ActivateAccountForCatalogResponse) + state.inflightActivations.Store(uint32(10), &inflightActivation{done: make(chan struct{})}) + + got := state.debugState(nil) + require.True(t, got.Enabled) + require.Equal(t, 1, got.PendingActivationResponseCount) + require.Equal(t, 1, got.InflightActivationCount) + require.Equal(t, []uint32{0, 20}, got.WantedAccounts) + require.Len(t, got.Accounts, 2) + + require.Equal(t, uint32(10), got.Accounts[0].AccountID) + require.Equal(t, "catching_up", got.Accounts[0].State) + require.Nil(t, got.Accounts[0].ReadyTS) + require.Equal(t, uint64(7), got.Accounts[0].PendingSeq) + require.Equal(t, 2, got.Accounts[0].DelayedApplyCount) + + require.Equal(t, uint32(20), got.Accounts[1].AccountID) + require.Equal(t, "ready", got.Accounts[1].State) + require.NotNil(t, got.Accounts[1].ReadyTS) + require.Equal(t, int64(200), got.Accounts[1].ReadyTS.PhysicalTime) + require.Equal(t, uint32(2), got.Accounts[1].ReadyTS.LogicalTime) + + account := state.debugAccountState(20) + require.True(t, account.Present) + require.True(t, account.InWantedAccounts) + require.Equal(t, "ready", account.State) + require.NotNil(t, account.ReadyTS) +} + +func TestLazyCatalogDebugActivationHistory(t *testing.T) { + state := newLazyCatalogCNState() + now := time.Now() + later := now.Add(time.Second) + targetTS := timestamp.Timestamp{PhysicalTime: 300} + replayTS := timestamp.Timestamp{PhysicalTime: 320} + + state.recordActivationEvent(DebugCatalogActivationEvent{ + AccountID: 10, + Seq: 7, + Source: "activation", + Phase: "complete", + Result: "ok", + TargetTS: &targetTS, + ReplayTS: &replayTS, + StartedAt: &now, + FinishedAt: &later, + }) + state.recordActivationEvent(DebugCatalogActivationEvent{ + AccountID: 20, + Source: "reconnect_restore", + Phase: "complete", + Result: "ok", + }) + + accountID := uint32(10) + got := state.debugActivationHistory(&accountID, 10) + require.Len(t, got, 1) + require.Equal(t, uint32(10), got[0].AccountID) + require.Equal(t, uint64(7), got[0].Seq) + require.Equal(t, "activation", got[0].Source) + require.Equal(t, "complete", got[0].Phase) + require.Equal(t, "ok", got[0].Result) + require.NotNil(t, got[0].TargetTS) + require.NotNil(t, got[0].ReplayTS) + require.NotNil(t, got[0].StartedAt) + require.NotNil(t, got[0].FinishedAt) +} + +func TestDebugCatalogStateWithAccountInspection(t *testing.T) { + eng := &Engine{} + cc := cache.NewCatalog() + cc.UpdateDuration(types.TS{}, types.MaxTs()) + eng.catalog.Store(cc) + eng.pClient.lazyCatalog = newLazyCatalogCNState() + eng.pClient.lazyCatalog.enable() + eng.pClient.lazyCatalog.setAccountReady(42, timestamp.Timestamp{PhysicalTime: 80}) + eng.pClient.receivedLogTailTime.tList = make([]atomic.Value, 1) + latest := timestamp.Timestamp{PhysicalTime: 100} + eng.pClient.receivedLogTailTime.tList[0].Store(latest) + + snapshot := timestamp.Timestamp{PhysicalTime: 90} + accountID := uint32(42) + got := eng.DebugCatalogState(&accountID, &snapshot) + require.Equal(t, latest, got.LatestLogtailAppliedTS) + require.NotNil(t, got.Account) + require.Equal(t, uint32(42), got.Account.AccountID) + require.Equal(t, "ready", got.Account.State) + require.True(t, got.Account.InWantedAccounts) + require.True(t, got.Account.CanServeLatest) + require.NotNil(t, got.Account.CanServeSnapshot) + require.True(t, *got.Account.CanServeSnapshot) +} diff --git a/pkg/vm/engine/disttae/engine.go b/pkg/vm/engine/disttae/engine.go index 913529eeafd77..30a08c7e84254 100644 --- a/pkg/vm/engine/disttae/engine.go +++ b/pkg/vm/engine/disttae/engine.go @@ -400,7 +400,8 @@ func (e *Engine) Database( catalog := e.GetLatestCatalogCache() if ok := catalog.GetDatabase(item); !ok { - if !catalog.CanServe(types.TimestampToTS(op.SnapshotTS())) { + if !catalog.CanServe(types.TimestampToTS(op.SnapshotTS())) || + !e.pClient.CanServeAccount(accountId, op.SnapshotTS()) { logutil.Info( "engine.database.load.from.storage", zap.String("name", name), @@ -557,7 +558,8 @@ func (e *Engine) GetRelationById(ctx context.Context, op client.TxnOperator, tab zap.String("snapshot-ts", types.TimestampToTS(op.SnapshotTS()).ToString()), zap.String("txn", op.Txn().DebugString()), ) - } else if !cache.CanServe(types.TimestampToTS(op.SnapshotTS())) { + } else if !cache.CanServe(types.TimestampToTS(op.SnapshotTS())) || + !e.pClient.CanServeAccount(accountId, op.SnapshotTS()) { // not found in cache, try storage logutil.Info( "engine.relation.load.from.storage", @@ -902,6 +904,11 @@ func (e *Engine) PushClient() *PushClient { return &e.pClient } +// ActivateTenantCatalog implements engine.TenantCatalogActivator. +func (e *Engine) ActivateTenantCatalog(ctx context.Context, accountID uint32) error { + return e.pClient.ActivateTenantCatalog(ctx, e, accountID) +} + // TryToSubscribeTable implements the LogtailEngine interface. func (e *Engine) TryToSubscribeTable(ctx context.Context, accId, dbID, tbID uint64, dbName, tblName string) error { return e.PushClient().TryToSubscribeTable(ctx, accId, dbID, tbID, dbName, tblName) diff --git a/pkg/vm/engine/disttae/lazy_catalog_cn.go b/pkg/vm/engine/disttae/lazy_catalog_cn.go new file mode 100644 index 0000000000000..b37569ec881d8 --- /dev/null +++ b/pkg/vm/engine/disttae/lazy_catalog_cn.go @@ -0,0 +1,361 @@ +// Copyright 2024 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package disttae + +import ( + "sync" + "sync/atomic" + + "github.com/matrixorigin/matrixone/pkg/catalog" + "github.com/matrixorigin/matrixone/pkg/pb/api" + "github.com/matrixorigin/matrixone/pkg/pb/logtail" + "github.com/matrixorigin/matrixone/pkg/pb/timestamp" +) + +type catalogReadyState int + +const ( + accountInactive catalogReadyState = iota + accountCatchingUp + accountReadyDraining + accountReady +) + +type accountCatalogEntry struct { + state catalogReadyState + readyTS timestamp.Timestamp +} + +type activationResponseKey struct { + accountID uint32 + seq uint64 +} + +// lazyCatalogCNState holds per-account catalog readiness state for a CN. +// All fields are guarded by mu unless noted otherwise. +type lazyCatalogCNState struct { + mu sync.RWMutex + + enabled atomic.Bool + + accounts map[uint32]*accountCatalogEntry + + // pendingSeq maps accountID → seq for in-flight activation requests. + pendingSeq map[uint32]uint64 + + // accountDCA holds delayed cache-apply closures for accounts in + // catching_up state. Flushed after full replay. + accountDCA map[uint32][]func() + + // wantedAccounts records which accounts were ready before a reconnect, + // so the reconnect subscribe can restore them in one batch. + wantedAccounts map[uint32]struct{} + + // pendingActivationResponses delivers activation responses from the + // receive loop to the waiting ActivateTenantCatalog caller. + pendingActivationResponses map[activationResponseKey]chan *logtail.ActivateAccountForCatalogResponse + + // seqCounter allocates monotonically increasing seq values. + seqCounter atomic.Uint64 + + // catchingUpCount is a fast-path counter. Positive means at least one + // account is in catching_up state, so consumeEntry should check + // per-account DCA. It is updated while holding mu, then published through + // the atomic for cheap read-side checks. + catchingUpCount atomic.Int32 + + // inflightActivations prevents duplicate concurrent activations for the + // same account. Maps accountID → *inflightActivation. + inflightActivations sync.Map + + // activationHistory keeps a bounded recent history of structured + // activation/startup/reconnect events for debug introspection. + activationHistory []DebugCatalogActivationEvent +} + +type inflightActivation struct { + done chan struct{} + err error +} + +func newLazyCatalogCNState() *lazyCatalogCNState { + return &lazyCatalogCNState{ + accounts: make(map[uint32]*accountCatalogEntry), + pendingSeq: make(map[uint32]uint64), + accountDCA: make(map[uint32][]func()), + wantedAccounts: make(map[uint32]struct{}), + pendingActivationResponses: make(map[activationResponseKey]chan *logtail.ActivateAccountForCatalogResponse), + } +} + +func (s *lazyCatalogCNState) enable() { + s.enabled.Store(true) +} + +func (s *lazyCatalogCNState) isEnabled() bool { + return s.enabled.Load() +} + +func (s *lazyCatalogCNState) isAccountReady(accountID uint32) bool { + if !s.enabled.Load() { + return true + } + s.mu.RLock() + defer s.mu.RUnlock() + e, ok := s.accounts[accountID] + return ok && e.state == accountReady +} + +func (s *lazyCatalogCNState) getAccountReadyTS(accountID uint32) (timestamp.Timestamp, bool) { + if !s.enabled.Load() { + return timestamp.Timestamp{}, true + } + s.mu.RLock() + defer s.mu.RUnlock() + e, ok := s.accounts[accountID] + if !ok || e.state != accountReady { + return timestamp.Timestamp{}, false + } + return e.readyTS, true +} + +// beginCatchingUp transitions the account to catching_up and returns a new seq. +// The caller should only call this under singleflight protection. +func (s *lazyCatalogCNState) beginCatchingUp(accountID uint32) uint64 { + s.mu.Lock() + defer s.mu.Unlock() + + seq := s.seqCounter.Add(1) + old := s.accounts[accountID] + if old == nil || old.state != accountCatchingUp { + s.catchingUpCount.Add(1) + } + s.accounts[accountID] = &accountCatalogEntry{state: accountCatchingUp} + s.pendingSeq[accountID] = seq + s.accountDCA[accountID] = nil + return seq +} + +// getOrCreateEntry returns the existing entry or creates and stores a new one. +// Must be called with mu held. +func (s *lazyCatalogCNState) getOrCreateEntry(accountID uint32) *accountCatalogEntry { + e := s.accounts[accountID] + if e == nil { + e = &accountCatalogEntry{} + s.accounts[accountID] = e + } + return e +} + +// setAccountReady transitions the account to ready, sets its readyTS, and +// records it in wantedAccounts for reconnect survival. Startup/reconnect use +// this directly because they do not rely on per-account DCA. +func (s *lazyCatalogCNState) setAccountReady(accountID uint32, readyTS timestamp.Timestamp) { + s.mu.Lock() + defer s.mu.Unlock() + + e := s.getOrCreateEntry(accountID) + if e.state == accountCatchingUp { + s.catchingUpCount.Add(-1) + } + e.state = accountReady + e.readyTS = readyTS + delete(s.pendingSeq, accountID) + s.wantedAccounts[accountID] = struct{}{} +} + +// delayAccountCacheApply queues f for later flush if the account is currently +// catching_up. Returns true if delayed, false if the caller should apply now. +func (s *lazyCatalogCNState) delayAccountCacheApply(accountID uint32, f func()) bool { + s.mu.Lock() + defer s.mu.Unlock() + + e, ok := s.accounts[accountID] + if !ok || e.state != accountCatchingUp { + return false + } + s.accountDCA[accountID] = append(s.accountDCA[accountID], f) + return true +} + +// beginAccountReadyTransition detaches any delayed cache-apply closures and +// moves the account out of catching_up, but does not publish readyTS yet. The +// caller must finish the transition under catalogCacheMu after the detached +// closures have run. +func (s *lazyCatalogCNState) beginAccountReadyTransition(accountID uint32) []func() { + s.mu.Lock() + defer s.mu.Unlock() + + e := s.getOrCreateEntry(accountID) + if e.state == accountCatchingUp { + s.catchingUpCount.Add(-1) + } + e.state = accountReadyDraining + delete(s.pendingSeq, accountID) + + fns := s.accountDCA[accountID] + delete(s.accountDCA, accountID) + return fns +} + +func (s *lazyCatalogCNState) finishAccountReady(accountID uint32, readyTS timestamp.Timestamp) { + s.mu.Lock() + defer s.mu.Unlock() + + e := s.getOrCreateEntry(accountID) + e.state = accountReady + e.readyTS = readyTS + s.wantedAccounts[accountID] = struct{}{} +} + +// cleanupFailedActivation reverts a catching_up account to inactive if the seq +// still matches. Returns true if cleanup was performed. +func (s *lazyCatalogCNState) cleanupFailedActivation(accountID uint32, seq uint64) bool { + s.mu.Lock() + defer s.mu.Unlock() + + current, ok := s.pendingSeq[accountID] + if !ok || current != seq { + return false + } + + delete(s.pendingSeq, accountID) + delete(s.pendingActivationResponses, activationResponseKey{accountID: accountID, seq: seq}) + s.accountDCA[accountID] = nil + + e := s.accounts[accountID] + if e != nil && e.state == accountCatchingUp { + e.state = accountInactive + s.catchingUpCount.Add(-1) + } + return true +} + +// snapshotWantedAccounts returns a copy of the wantedAccounts set, suitable +// for passing to subSysTables during reconnect. +func (s *lazyCatalogCNState) snapshotWantedAccounts() []uint32 { + s.mu.RLock() + defer s.mu.RUnlock() + + result := make([]uint32, 0, len(s.wantedAccounts)) + for id := range s.wantedAccounts { + result = append(result, id) + } + return result +} + +// collectWantedAccounts snapshots all currently ready accounts into +// wantedAccounts. Called before reconnect. +func (s *lazyCatalogCNState) collectWantedAccounts() { + s.mu.Lock() + defer s.mu.Unlock() + + s.wantedAccounts = make(map[uint32]struct{}) + for id, e := range s.accounts { + if e.state == accountReady { + s.wantedAccounts[id] = struct{}{} + } + } +} + +// resetAllStates clears all per-account state. Called on reconnect after +// wantedAccounts has been collected. +func (s *lazyCatalogCNState) resetAllStates() { + s.mu.Lock() + pendingResponses := make([]chan *logtail.ActivateAccountForCatalogResponse, 0, len(s.pendingActivationResponses)) + for _, ch := range s.pendingActivationResponses { + pendingResponses = append(pendingResponses, ch) + } + + s.accounts = make(map[uint32]*accountCatalogEntry) + s.pendingSeq = make(map[uint32]uint64) + s.accountDCA = make(map[uint32][]func()) + s.pendingActivationResponses = make(map[activationResponseKey]chan *logtail.ActivateAccountForCatalogResponse) + s.catchingUpCount.Store(0) + // Do NOT reassign inflightActivations here. sync.Map must not be + // overwritten after first use (data race with concurrent LoadOrStore). + // Old activation goroutines' defers will clean up their own entries. + s.mu.Unlock() + + for _, ch := range pendingResponses { + select { + case ch <- nil: + default: + } + } +} + +// registerPendingResponse creates a buffered channel for delivering the +// activation response from the receive loop to the ActivateTenantCatalog +// caller. +func (s *lazyCatalogCNState) registerPendingResponse( + accountID uint32, + seq uint64, +) chan *logtail.ActivateAccountForCatalogResponse { + ch := make(chan *logtail.ActivateAccountForCatalogResponse, 1) + s.mu.Lock() + defer s.mu.Unlock() + s.pendingActivationResponses[activationResponseKey{accountID: accountID, seq: seq}] = ch + return ch +} + +// deliverActivationResponse sends the activation response to the waiting +// caller. Returns false if no caller is waiting. +func (s *lazyCatalogCNState) deliverActivationResponse(resp *logtail.ActivateAccountForCatalogResponse) bool { + key := activationResponseKey{accountID: resp.GetAccountId(), seq: resp.GetSeq()} + s.mu.Lock() + ch, ok := s.pendingActivationResponses[key] + if ok { + delete(s.pendingActivationResponses, key) + } + s.mu.Unlock() + if !ok { + return false + } + ch <- resp + return true +} + +func (s *lazyCatalogCNState) hasCatchingUp() bool { + return s.catchingUpCount.Load() > 0 +} + +// shouldDelayCatalogCacheApplyEntry routes a pushed in-memory catalog entry by +// account. This intentionally relies on the current lazy-catalog assumption +// that pushed entry batches for the three catalog tables belong to a single +// account; subscribe snapshots are filtered separately on the TN side. +func (s *lazyCatalogCNState) shouldDelayCatalogCacheApplyEntry( + entry api.Entry, +) (uint32, bool, error) { + if !s.isEnabled() || !catalog.IsLazyCatalogTableID(entry.TableId) { + return 0, false, nil + } + + accountID, ok, err := catalog.LazyCatalogEntryAccountID(entry) + if err != nil || !ok { + return 0, false, err + } + return accountID, s.isAccountCatchingUp(accountID), nil +} + +func (s *lazyCatalogCNState) isAccountCatchingUp(accountID uint32) bool { + if s.catchingUpCount.Load() == 0 { + return false + } + s.mu.RLock() + defer s.mu.RUnlock() + entry := s.accounts[accountID] + return entry != nil && entry.state == accountCatchingUp +} diff --git a/pkg/vm/engine/disttae/lazy_catalog_cn_test.go b/pkg/vm/engine/disttae/lazy_catalog_cn_test.go new file mode 100644 index 0000000000000..baf239666a20c --- /dev/null +++ b/pkg/vm/engine/disttae/lazy_catalog_cn_test.go @@ -0,0 +1,420 @@ +// Copyright 2024 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package disttae + +import ( + "testing" + + "github.com/matrixorigin/matrixone/pkg/catalog" + "github.com/matrixorigin/matrixone/pkg/common/mpool" + "github.com/matrixorigin/matrixone/pkg/container/batch" + "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/pb/api" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/matrixorigin/matrixone/pkg/pb/logtail" + "github.com/matrixorigin/matrixone/pkg/pb/timestamp" +) + +func TestLazyCatalogCNState_BasicStateTransitions(t *testing.T) { + s := newLazyCatalogCNState() + s.enable() + + assert.False(t, s.isAccountReady(10)) + _, ok := s.getAccountReadyTS(10) + assert.False(t, ok) + + seq := s.beginCatchingUp(10) + assert.True(t, seq > 0) + assert.False(t, s.isAccountReady(10)) + assert.True(t, s.hasCatchingUp()) + + readyTS := timestamp.Timestamp{PhysicalTime: 100} + s.setAccountReady(10, readyTS) + + assert.True(t, s.isAccountReady(10)) + assert.False(t, s.hasCatchingUp()) + + gotTS, ok := s.getAccountReadyTS(10) + assert.True(t, ok) + assert.Equal(t, readyTS, gotTS) +} + +func TestLazyCatalogCNState_CatchingUpCount(t *testing.T) { + s := newLazyCatalogCNState() + + assert.False(t, s.hasCatchingUp()) + + s.beginCatchingUp(1) + assert.True(t, s.hasCatchingUp()) + + s.beginCatchingUp(2) + assert.True(t, s.hasCatchingUp()) + + s.setAccountReady(1, timestamp.Timestamp{PhysicalTime: 100}) + assert.True(t, s.hasCatchingUp()) + + s.setAccountReady(2, timestamp.Timestamp{PhysicalTime: 200}) + assert.False(t, s.hasCatchingUp()) +} + +func TestLazyCatalogCNState_CleanupFailedActivation(t *testing.T) { + s := newLazyCatalogCNState() + s.enable() + + seq := s.beginCatchingUp(10) + assert.True(t, s.hasCatchingUp()) + assert.True(t, s.isAccountCatchingUp(10)) + + ok := s.cleanupFailedActivation(10, seq) + assert.True(t, ok) + assert.False(t, s.hasCatchingUp()) + assert.False(t, s.isAccountReady(10)) + + ok = s.cleanupFailedActivation(10, seq) + assert.False(t, ok) +} + +func TestLazyCatalogCNState_StaleSeqCleanup(t *testing.T) { + s := newLazyCatalogCNState() + + seq1 := s.beginCatchingUp(10) + seq2 := s.beginCatchingUp(10) + assert.NotEqual(t, seq1, seq2) + + ok := s.cleanupFailedActivation(10, seq1) + assert.False(t, ok) + + ok = s.cleanupFailedActivation(10, seq2) + assert.True(t, ok) +} + +func TestLazyCatalogCNState_DelayAccountCacheApply(t *testing.T) { + s := newLazyCatalogCNState() + + called := false + f := func() { called = true } + + delayed := s.delayAccountCacheApply(10, f) + assert.False(t, delayed) + + s.beginCatchingUp(10) + + delayed = s.delayAccountCacheApply(10, f) + assert.True(t, delayed) + assert.False(t, called) + + for _, fn := range s.beginAccountReadyTransition(10) { + fn() + } + s.finishAccountReady(10, timestamp.Timestamp{PhysicalTime: 100}) + assert.True(t, called) +} + +func TestLazyCatalogCNState_ReadyTransitionDrainsAccountDCA(t *testing.T) { + s := newLazyCatalogCNState() + s.enable() + s.beginCatchingUp(10) + + count := 0 + for i := 0; i < 5; i++ { + s.delayAccountCacheApply(10, func() { count++ }) + } + + fns := s.beginAccountReadyTransition(10) + assert.False(t, s.delayAccountCacheApply(10, func() { count = 999 })) + _, ok := s.getAccountReadyTS(10) + assert.False(t, ok) + for _, fn := range fns { + fn() + } + s.finishAccountReady(10, timestamp.Timestamp{PhysicalTime: 200}) + assert.Equal(t, 5, count) + + readyTS, ok := s.getAccountReadyTS(10) + assert.True(t, ok) + assert.Equal(t, int64(200), readyTS.PhysicalTime) +} + +func TestLazyCatalogCNState_WantedAccounts(t *testing.T) { + s := newLazyCatalogCNState() + + s.setAccountReady(0, timestamp.Timestamp{PhysicalTime: 10}) + s.setAccountReady(5, timestamp.Timestamp{PhysicalTime: 20}) + s.setAccountReady(10, timestamp.Timestamp{PhysicalTime: 30}) + + s.collectWantedAccounts() + wanted := s.snapshotWantedAccounts() + + assert.Len(t, wanted, 3) + wantedSet := make(map[uint32]struct{}) + for _, id := range wanted { + wantedSet[id] = struct{}{} + } + assert.Contains(t, wantedSet, uint32(0)) + assert.Contains(t, wantedSet, uint32(5)) + assert.Contains(t, wantedSet, uint32(10)) +} + +func TestLazyCatalogCNState_ResetAllStates(t *testing.T) { + s := newLazyCatalogCNState() + s.enable() + + s.beginCatchingUp(10) + s.setAccountReady(20, timestamp.Timestamp{PhysicalTime: 100}) + s.inflightActivations.Store(uint32(30), &inflightActivation{done: make(chan struct{})}) + + s.collectWantedAccounts() + s.resetAllStates() + + assert.False(t, s.isAccountReady(20)) + assert.False(t, s.hasCatchingUp()) + + wanted := s.snapshotWantedAccounts() + wantedSet := make(map[uint32]struct{}) + for _, id := range wanted { + wantedSet[id] = struct{}{} + } + assert.Contains(t, wantedSet, uint32(20)) + + // inflightActivations is NOT cleared by resetAllStates to avoid a data + // race with concurrent LoadOrStore. Old entries are cleaned by their + // activation goroutines' defer blocks. + _, ok := s.inflightActivations.Load(uint32(30)) + assert.True(t, ok) +} + +func TestLazyCatalogCNState_ResetAllStatesNotifiesPendingResponses(t *testing.T) { + s := newLazyCatalogCNState() + ch := s.registerPendingResponse(10, 42) + + s.resetAllStates() + + resp := <-ch + assert.Nil(t, resp) +} + +func TestLazyCatalogCNState_PendingResponse(t *testing.T) { + s := newLazyCatalogCNState() + + ch := s.registerPendingResponse(10, 42) + require.NotNil(t, ch) + + resp := &logtail.ActivateAccountForCatalogResponse{ + AccountId: 10, + Seq: 42, + } + ok := s.deliverActivationResponse(resp) + assert.True(t, ok) + + received := <-ch + assert.Equal(t, uint32(10), received.AccountId) + assert.Equal(t, uint64(42), received.Seq) + + ok = s.deliverActivationResponse(resp) + assert.False(t, ok) +} + +func TestLazyCatalogCNState_PendingResponseIgnoresStaleSeq(t *testing.T) { + s := newLazyCatalogCNState() + + seq1 := s.beginCatchingUp(10) + ch1 := s.registerPendingResponse(10, seq1) + require.NotNil(t, ch1) + require.True(t, s.cleanupFailedActivation(10, seq1)) + + seq2 := s.beginCatchingUp(10) + ch2 := s.registerPendingResponse(10, seq2) + require.NotNil(t, ch2) + + ok := s.deliverActivationResponse(&logtail.ActivateAccountForCatalogResponse{ + AccountId: 10, + Seq: seq1, + }) + assert.False(t, ok) + + select { + case <-ch2: + t.Fatal("stale response should not be delivered to new waiter") + default: + } + + ok = s.deliverActivationResponse(&logtail.ActivateAccountForCatalogResponse{ + AccountId: 10, + Seq: seq2, + }) + require.True(t, ok) + got := <-ch2 + require.Equal(t, seq2, got.Seq) +} + +func TestCanServeAccount(t *testing.T) { + c := &PushClient{} + assert.True(t, c.CanServeAccount(10, timestamp.Timestamp{PhysicalTime: 100})) + + c.lazyCatalog = newLazyCatalogCNState() + assert.True(t, c.CanServeAccount(10, timestamp.Timestamp{PhysicalTime: 100})) + + c.lazyCatalog.enable() + assert.False(t, c.CanServeAccount(10, timestamp.Timestamp{PhysicalTime: 100})) + + c.lazyCatalog.setAccountReady(10, timestamp.Timestamp{PhysicalTime: 50}) + assert.True(t, c.CanServeAccount(10, timestamp.Timestamp{PhysicalTime: 50})) + assert.True(t, c.CanServeAccount(10, timestamp.Timestamp{PhysicalTime: 100})) + assert.False(t, c.CanServeAccount(10, timestamp.Timestamp{PhysicalTime: 30})) +} + +func TestIsLazyCatalogTableID(t *testing.T) { + require.True(t, catalog.IsLazyCatalogTableID(catalog.MO_DATABASE_ID)) + require.True(t, catalog.IsLazyCatalogTableID(catalog.MO_TABLES_ID)) + require.True(t, catalog.IsLazyCatalogTableID(catalog.MO_COLUMNS_ID)) + require.False(t, catalog.IsLazyCatalogTableID(catalog.MO_TABLES_LOGICAL_ID_INDEX_ID)) +} + +func TestReconnectInitialActiveAccounts(t *testing.T) { + c := &PushClient{lazyCatalog: newLazyCatalogCNState()} + c.lazyCatalog.setAccountReady(5, timestamp.Timestamp{PhysicalTime: 10}) + c.lazyCatalog.collectWantedAccounts() + + require.ElementsMatch(t, []uint32{0, 5}, c.reconnectInitialActiveAccounts()) +} + +func TestLazyCatalogCNStateShouldDelayCatalogCacheApplyEntryInsert(t *testing.T) { + s := newLazyCatalogCNState() + s.enable() + s.beginCatchingUp(10) + + accountID, shouldDelay, err := s.shouldDelayCatalogCacheApplyEntry(mustCatalogColumnInsertEntry(t, []uint32{10, 10})) + require.NoError(t, err) + require.Equal(t, uint32(10), accountID) + require.True(t, shouldDelay) + + accountID, shouldDelay, err = s.shouldDelayCatalogCacheApplyEntry(mustCatalogColumnInsertEntry(t, []uint32{20, 20})) + require.NoError(t, err) + require.Equal(t, uint32(20), accountID) + require.False(t, shouldDelay) +} + +func TestLazyCatalogCNStateShouldDelayCatalogCacheApplyEntryDelete(t *testing.T) { + s := newLazyCatalogCNState() + s.enable() + s.beginCatchingUp(10) + + accountID, shouldDelay, err := s.shouldDelayCatalogCacheApplyEntry(mustCatalogTableDeleteEntry(t, []uint32{10, 10})) + require.NoError(t, err) + require.Equal(t, uint32(10), accountID) + require.True(t, shouldDelay) + + accountID, shouldDelay, err = s.shouldDelayCatalogCacheApplyEntry(mustCatalogTableDeleteEntry(t, []uint32{20, 20})) + require.NoError(t, err) + require.Equal(t, uint32(20), accountID) + require.False(t, shouldDelay) +} + +func TestLazyCatalogCNStateShouldDelayCatalogCacheApplyEntryUsesSummary(t *testing.T) { + s := newLazyCatalogCNState() + s.enable() + s.beginCatchingUp(10) + + entry := api.Entry{ + EntryType: api.Entry_Insert, + TableId: catalog.MO_COLUMNS_ID, + DatabaseId: catalog.MO_CATALOG_ID, + } + catalog.SetLazyCatalogEntryAccountSummary(&entry, 10) + + accountID, shouldDelay, err := s.shouldDelayCatalogCacheApplyEntry(entry) + require.NoError(t, err) + require.Equal(t, uint32(10), accountID) + require.True(t, shouldDelay) +} + +func mustCatalogColumnInsertEntry(t *testing.T, accountIDs []uint32) api.Entry { + t.Helper() + mp := mpool.MustNew("lazy-catalog-cn-columns") + packer := types.NewPacker() + defer packer.Close() + typ := types.T_int64.ToType() + typEncoded, err := types.Encode(&typ) + require.NoError(t, err) + + cols := make([]catalog.Column, 0, len(accountIDs)) + for i, accountID := range accountIDs { + cols = append(cols, catalog.Column{ + AccountId: accountID, + DatabaseId: uint64(100 + i), + DatabaseName: "db", + TableId: uint64(200 + i), + TableName: "tbl", + Name: string(rune('a' + i)), + Num: int32(i), + Typ: typEncoded, + TypLen: int32(len(typEncoded)), + }) + } + + bat, err := catalog.GenCreateColumnTuples(cols, mp, packer) + require.NoError(t, err) + t.Cleanup(func() { bat.Clean(mp) }) + + pbBat, err := batch.BatchToProtoBatch(bat) + require.NoError(t, err) + return api.Entry{ + EntryType: api.Entry_Insert, + TableId: catalog.MO_COLUMNS_ID, + DatabaseId: catalog.MO_CATALOG_ID, + Bat: pbBat, + } +} + +func mustCatalogTableDeleteEntry(t *testing.T, accountIDs []uint32) api.Entry { + t.Helper() + mp := mpool.MustNew("lazy-catalog-cn-delete") + packer := types.NewPacker() + defer packer.Close() + + var batAcc *batch.Batch + for i, accountID := range accountIDs { + bat, err := catalog.GenDropTableTuple( + types.RandomRowid(), + accountID, + uint64(200+i), + uint64(100+i), + "tbl", + "db", + mp, + packer, + ) + require.NoError(t, err) + if batAcc == nil { + batAcc = bat + continue + } + require.NoError(t, batAcc.UnionOne(bat, 0, mp)) + bat.Clean(mp) + } + t.Cleanup(func() { batAcc.Clean(mp) }) + + pbBat, err := batch.BatchToProtoBatch(batAcc) + require.NoError(t, err) + return api.Entry{ + EntryType: api.Entry_Delete, + TableId: catalog.MO_TABLES_ID, + DatabaseId: catalog.MO_CATALOG_ID, + Bat: pbBat, + } +} diff --git a/pkg/vm/engine/disttae/logtail.go b/pkg/vm/engine/disttae/logtail.go index c83be58ad3af4..54386e9b6a8a0 100644 --- a/pkg/vm/engine/disttae/logtail.go +++ b/pkg/vm/engine/disttae/logtail.go @@ -56,8 +56,8 @@ func consumeEntry( v2.LogtailUpdatePartitonConsumeLogtailOneEntryLogtailReplayDurationHistogram.Observe(time.Since(t0).Seconds()) } - // Try to handle the memory records of the three tables - if !catalog.IsSystemTable(e.TableId) || logtailreplay.IsMetaEntry(e.TableName) || e.EntryType == api.Entry_DataObject || e.EntryType == api.Entry_TombstoneObject { + // Lazy catalog CN logic is scoped only to the three catalog tables. + if !catalog.IsLazyCatalogTableID(e.TableId) || logtailreplay.IsMetaEntry(e.TableName) || e.EntryType == api.Entry_DataObject || e.EntryType == api.Entry_TombstoneObject { return nil } @@ -65,7 +65,20 @@ func consumeEntry( return nil } - applyToCatalogCache(cache, e) + lc := engine.PushClient().lazyCatalog + if lc != nil { + accountID, shouldDelay, err := lc.shouldDelayCatalogCacheApplyEntry(*e) + if err != nil { + return err + } + if shouldDelay && lc.delayAccountCacheApply(accountID, func() { applyToCatalogCache(cache, e) }) { + return nil + } + } + + engine.PushClient().applyCatalogCacheChange(func() { + applyToCatalogCache(cache, e) + }) return nil } diff --git a/pkg/vm/engine/disttae/logtail_consumer.go b/pkg/vm/engine/disttae/logtail_consumer.go index ca2e60420d7bd..906ed87c92f8d 100644 --- a/pkg/vm/engine/disttae/logtail_consumer.go +++ b/pkg/vm/engine/disttae/logtail_consumer.go @@ -19,6 +19,7 @@ import ( "errors" "fmt" "math/rand" + "slices" "strings" "sync" "sync/atomic" @@ -39,8 +40,8 @@ import ( "github.com/matrixorigin/matrixone/pkg/pb/timestamp" "github.com/matrixorigin/matrixone/pkg/txn/client" "github.com/matrixorigin/matrixone/pkg/util/address" - "github.com/matrixorigin/matrixone/pkg/util/executor" v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2" + "github.com/matrixorigin/matrixone/pkg/vm/engine/disttae/cache" "github.com/matrixorigin/matrixone/pkg/vm/engine/disttae/logtailreplay" taeLogtail "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logtail" "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logtail/service" @@ -137,6 +138,12 @@ type PushClient struct { // Record the subscription status of a tables. subscribed subscribedTable + // lazyCatalog tracks per-account catalog readiness for lazy loading. + lazyCatalog *lazyCatalogCNState + // catalogCacheMu serializes shared CatalogCache mutation across replay, + // delayed flush, and steady-state push updates. + catalogCacheMu sync.Mutex + // timestampWaiter is used to notify the latest commit timestamp timestampWaiter client.TimestampWaiter @@ -176,21 +183,30 @@ type delayedCacheApply struct { // and on the other goroutine, replayCatalogCache will attempt to acquire the lock only once. // Therefore, the lock contention is not serious. sync.Mutex - replayed bool - flist []func() + phase atomic.Uint32 + flist []func() } +const ( + dcaPhaseBuffering uint32 = iota + dcaPhaseDraining + dcaPhaseReplayed +) + func (c *PushClient) dcaReset() { c.dca.Lock() - defer c.dca.Unlock() - c.dca.replayed = false - c.dca.flist = c.dca.flist[:0] + c.dca.phase.Store(dcaPhaseBuffering) + c.dca.flist = nil + c.dca.Unlock() } func (c *PushClient) dcaTryDelay(isSub bool, f func()) (delayed bool) { + if c.dca.phase.Load() == dcaPhaseReplayed { + return false + } c.dca.Lock() defer c.dca.Unlock() - if c.dca.replayed { + if c.dca.phase.Load() == dcaPhaseReplayed { // replay finished, no need to delay return false } @@ -202,13 +218,34 @@ func (c *PushClient) dcaTryDelay(isSub bool, f func()) (delayed bool) { } func (c *PushClient) dcaConfirmAndApply() { - c.dca.Lock() - defer c.dca.Unlock() - c.dca.replayed = true - for _, f := range c.dca.flist { - f() + for { + c.dca.Lock() + if c.dca.phase.Load() == dcaPhaseReplayed { + c.dca.Unlock() + return + } + c.dca.phase.Store(dcaPhaseDraining) + if len(c.dca.flist) == 0 { + c.dca.phase.Store(dcaPhaseReplayed) + c.dca.Unlock() + return + } + fns := c.dca.flist + c.dca.flist = nil + c.dca.Unlock() + + c.applyCatalogCacheChange(func() { + for _, f := range fns { + f() + } + }) } - c.dca.flist = c.dca.flist[:0] +} + +func (c *PushClient) applyCatalogCacheChange(fn func()) { + c.catalogCacheMu.Lock() + defer c.catalogCacheMu.Unlock() + fn() } type State struct { @@ -327,6 +364,7 @@ func (c *PushClient) init( c.consumeErrC = make(chan error, consumerNumber) c.pauseC = make(chan bool, 1) c.resumeC = make(chan struct{}) + c.lazyCatalog = newLazyCatalogCNState() } c.initialized = true @@ -510,6 +548,33 @@ func (c *PushClient) forcedSubscribeTable( return moerr.NewInternalError(ctx, "forced subscribe table timeout") } +func (c *PushClient) forcedSubscribeCatalogTable( + ctx context.Context, + dbId, tblId uint64, + initialActiveAccounts []uint32, +) error { + s := c.subscriber + + tbl := api.TableID{DbId: dbId, TbId: tblId} + if err := s.subscribeCatalogTable(ctx, tbl, initialActiveAccounts); err != nil { + return err + } + ticker := time.NewTicker(periodToCheckTableSubscribeSucceed) + defer ticker.Stop() + + for i := 0; i < maxCheckRangeTableSubscribeSucceed; i++ { + select { + case <-ctx.Done(): + return ctx.Err() + case <-ticker.C: + if ok := c.subscribed.isSubscribed(dbId, tblId); ok { + return nil + } + } + } + return moerr.NewInternalError(ctx, "forced subscribe catalog table timeout") +} + func (c *PushClient) subscribeTable( ctx context.Context, tblId api.TableID) error { select { @@ -529,17 +594,19 @@ func (c *PushClient) subscribeTable( } } -func (c *PushClient) subSysTables(ctx context.Context) error { +func (c *PushClient) subSysTables(ctx context.Context, initialActiveAccounts []uint32) error { if enabled, p := objectio.CNSubSysErrInjected(); enabled && rand.Intn(100000) < p { return moerr.NewInternalError(ctx, "FIND_TABLE sub sys error injected") } - // push subscription to Table `mo_database`, `mo_table`, `mo_column` of mo_catalog. + if c.lazyCatalog != nil { + c.lazyCatalog.enable() + } databaseId := uint64(catalog.MO_CATALOG_ID) tableIds := []uint64{catalog.MO_DATABASE_ID, catalog.MO_TABLES_ID, catalog.MO_COLUMNS_ID} var err error for _, ti := range tableIds { - err = c.forcedSubscribeTable(ctx, databaseId, ti) + err = c.forcedSubscribeCatalogTable(ctx, databaseId, ti, initialActiveAccounts) if err != nil { break } @@ -644,6 +711,8 @@ func (c *PushClient) receiveOneLogtail(ctx context.Context, e *Engine) error { ) return err } + } else if actResp := resp.response.GetActivateAccountForCatalogResponse(); actResp != nil { + c.handleActivationResponse(ctx, e, actResp, receiveAt) } else if errRsp := resp.response.GetError(); errRsp != nil { status := errRsp.GetStatus() if uint16(status.GetCode()) == moerr.OkExpectedEOB { @@ -655,6 +724,34 @@ func (c *PushClient) receiveOneLogtail(ctx context.Context, e *Engine) error { return nil } +// handleActivationResponse applies activation tails to shared PartitionState +// and delivers the response to the waiting ActivateTenantCatalog caller. +func (c *PushClient) handleActivationResponse( + ctx context.Context, e *Engine, + resp *logtail.ActivateAccountForCatalogResponse, + receiveAt time.Time, +) { + for i := range resp.Tails { + tail := resp.Tails[i] + if err := updatePartitionOfPush(ctx, e, &tail, true, receiveAt, false); err != nil { + logutil.Error("logtail.consumer.activation.apply.tail.failed", + zap.Uint32("account-id", resp.AccountId), + zap.Uint64("seq", resp.Seq), + zap.Error(err), + ) + } + } + + if c.lazyCatalog != nil { + if !c.lazyCatalog.deliverActivationResponse(resp) { + logutil.Warn("logtail.consumer.activation.response.no.waiter", + zap.Uint32("account-id", resp.AccountId), + zap.Uint64("seq", resp.Seq), + ) + } + } +} + func (c *PushClient) receiveLogtails(ctx context.Context, e *Engine) { for { select { @@ -768,10 +865,92 @@ func (c *PushClient) replayCatalogCache(ctx context.Context, e *Engine) (err err if enabled, p := objectio.CNReplayCacheErrInjected(); enabled && rand.Intn(100000) < p { return moerr.NewInternalError(ctx, "FIND_TABLE replay catalog cache error injected") } - // replay mo_catalog cache - var op client.TxnOperator - var result executor.Result ts := c.receivedLogTailTime.getTimestamp() + if err = c.replayCatalogCacheAt(ctx, e, ts, []uint32{0}); err != nil { + return err + } + c.dcaConfirmAndApply() + if c.lazyCatalog != nil { + c.lazyCatalog.setAccountReady(0, ts) + now := time.Now() + c.lazyCatalog.recordActivationEvent(DebugCatalogActivationEvent{ + AccountID: 0, + Source: "startup", + Phase: "complete", + Result: "ok", + ReplayTS: &ts, + StartedAt: &now, + FinishedAt: &now, + }) + } + return nil +} + +// reconnectInitialActiveAccounts builds wantedAccounts ∪ {0} for use as +// initial_active_accounts during reconnect. +func (c *PushClient) reconnectInitialActiveAccounts() []uint32 { + if c.lazyCatalog == nil { + return []uint32{0} + } + wanted := c.lazyCatalog.snapshotWantedAccounts() + if !slices.Contains(wanted, uint32(0)) { + wanted = append(wanted, 0) + } + return wanted +} + +// replayCatalogCacheForReconnect replays the catalog for sys + wanted +// accounts after reconnect and sets them all ready. +func (c *PushClient) replayCatalogCacheForReconnect( + ctx context.Context, e *Engine, accountIDs []uint32, +) error { + if enabled, p := objectio.CNReplayCacheErrInjected(); enabled && rand.Intn(100000) < p { + return moerr.NewInternalError(ctx, "FIND_TABLE replay catalog cache error injected") + } + ts := c.receivedLogTailTime.getTimestamp() + if err := c.replayCatalogCacheAt(ctx, e, ts, accountIDs); err != nil { + return err + } + c.dcaConfirmAndApply() + + if c.lazyCatalog != nil { + now := time.Now() + for _, id := range accountIDs { + c.lazyCatalog.setAccountReady(id, ts) + c.lazyCatalog.recordActivationEvent(DebugCatalogActivationEvent{ + AccountID: id, + Source: "reconnect_restore", + Phase: "complete", + Result: "ok", + ReplayTS: &ts, + StartedAt: &now, + FinishedAt: &now, + }) + } + } + return nil +} + +// CanServeAccount returns true if the account's catalog cache is ready to +// serve the given snapshot timestamp. +func (c *PushClient) CanServeAccount(accountID uint32, ts timestamp.Timestamp) bool { + if c.lazyCatalog == nil { + return true + } + readyTS, ok := c.lazyCatalog.getAccountReadyTS(accountID) + if ok && readyTS.IsEmpty() { + return true + } + if !ok { + return false + } + return ts.GreaterEq(readyTS) +} + +func (c *PushClient) replayCatalogCacheAt( + ctx context.Context, e *Engine, ts timestamp.Timestamp, accountIDs []uint32, +) (err error) { + var op client.TxnOperator ccache := e.catalog.Load() typeTs := types.TimestampToTS(ts) createByOpt := client.WithTxnCreateBy( @@ -798,74 +977,260 @@ func (c *PushClient) replayCatalogCache(ctx context.Context, e *Engine) (err err return err } - // read databases - result, err = execReadSql(ctx, op, catalog.MoDatabaseBatchQuery, true) + if err = replayCatalogDatabaseCache(ctx, op, ccache, typeTs, accountIDs, c.applyCatalogCacheChange); err != nil { + return err + } + if err = replayCatalogTableCache(ctx, e, op, ccache, typeTs, accountIDs, c.applyCatalogCacheChange); err != nil { + return err + } + if err = replayCatalogColumnCache(ctx, op, ccache, typeTs, accountIDs, c.applyCatalogCacheChange); err != nil { + return err + } + + c.applyCatalogCacheChange(func() { + ccache.UpdateDuration(typeTs, types.MaxTs()) + }) + return nil + +} + +// ActivateTenantCatalog ensures the catalog cache for the given account is +// ready. If the account is already ready, returns immediately. Otherwise it +// sends an activation request to TN, waits for the response, replays the +// catalog from storage, and marks the account ready. +func (c *PushClient) ActivateTenantCatalog(ctx context.Context, e *Engine, accountID uint32) error { + if c.lazyCatalog == nil || !c.lazyCatalog.isEnabled() { + return nil + } + if c.lazyCatalog.isAccountReady(accountID) { + return nil + } + + act := &inflightActivation{done: make(chan struct{})} + if existing, loaded := c.lazyCatalog.inflightActivations.LoadOrStore(accountID, act); loaded { + inf := existing.(*inflightActivation) + select { + case <-ctx.Done(): + return ctx.Err() + case <-inf.done: + return inf.err + } + } + + defer func() { + close(act.done) + c.lazyCatalog.inflightActivations.Delete(accountID) + }() + + if c.lazyCatalog.isAccountReady(accountID) { + return nil + } + + act.err = c.doActivateTenantCatalog(ctx, e, accountID) + return act.err +} + +func (c *PushClient) doActivateTenantCatalog(ctx context.Context, e *Engine, accountID uint32) error { + startedAt := time.Now() + seq := c.lazyCatalog.beginCatchingUp(accountID) + respCh := c.lazyCatalog.registerPendingResponse(accountID, seq) + + // Concise event recording: builds the struct with common fields, caller + // supplies only the varying parts. + record := func(phase, result string, err error, targetTS, replayTS *timestamp.Timestamp, delayedCount int) { + now := time.Now() + evt := DebugCatalogActivationEvent{ + AccountID: accountID, Seq: seq, Source: "activation", + Phase: phase, Result: result, + TargetTS: targetTS, ReplayTS: replayTS, + DelayedApplyCount: delayedCount, + StartedAt: &startedAt, FinishedAt: &now, + } + if err != nil { + evt.Error = err.Error() + } + c.lazyCatalog.recordActivationEvent(evt) + } + + fail := func(phase string, err error, targetTS *timestamp.Timestamp) error { + c.lazyCatalog.cleanupFailedActivation(accountID, seq) + record(phase, "error", err, targetTS, nil, 0) + return err + } + + if err := c.subscriber.logTailClient.ActivateAccountForCatalog(ctx, accountID, seq); err != nil { + return fail("request_send", err, nil) + } + + logutil.Info("logtail.consumer.activation.request.sent", + zap.Uint32("account-id", accountID), + zap.Uint64("seq", seq), + ) + record("request_sent", "in_progress", nil, nil, nil, 0) + + var resp *logtail.ActivateAccountForCatalogResponse + select { + case <-ctx.Done(): + return fail("wait_response", ctx.Err(), nil) + case resp = <-respCh: + } + if resp == nil { + err := moerr.NewInternalErrorNoCtx("tenant catalog activation interrupted by reconnect") + record("wait_response", "interrupted", err, nil, nil, 0) + return err + } + + targetTS := resp.GetTargetTs() + var targetTSCopy *timestamp.Timestamp + if targetTS != nil { + tsCopy := *targetTS + targetTSCopy = &tsCopy + } + record("response_received", "in_progress", nil, targetTSCopy, nil, 0) + + if resp.GetSeq() != seq { + return fail("response_received", + moerr.NewInternalErrorf(ctx, "activation seq mismatch: expected %d, got %d", seq, resp.GetSeq()), + targetTSCopy) + } + + replayTS, err := e.cli.WaitLogTailAppliedAt(ctx, *targetTS) if err != nil { + return fail("wait_logtail", err, targetTSCopy) + } + + if err := c.replayCatalogCacheAt(ctx, e, replayTS, []uint32{accountID}); err != nil { + c.lazyCatalog.cleanupFailedActivation(accountID, seq) + replayTSCopy := replayTS + record("replay_catalog", "error", err, targetTSCopy, &replayTSCopy, 0) return err } - rowCntF := func(bat []*batch.Batch) string { - return stringifySlice(bat, func(b any) string { - return fmt.Sprintf("%d", b.(*batch.Batch).RowCount()) - }) + + delayedApplyCount := 0 + c.applyCatalogCacheChange(func() { + fns := c.lazyCatalog.beginAccountReadyTransition(accountID) + delayedApplyCount = len(fns) + for _, f := range fns { + f() + } + c.lazyCatalog.finishAccountReady(accountID, replayTS) + }) + + replayTSCopy := replayTS + record("complete", "ok", nil, targetTSCopy, &replayTSCopy, delayedApplyCount) + + logutil.Info("logtail.consumer.activation.complete", + zap.Uint32("account-id", accountID), + zap.Uint64("seq", seq), + zap.String("target-ts", targetTS.String()), + zap.String("replay-ts", replayTS.String()), + ) + return nil +} + +func replayCatalogDatabaseCache( + ctx context.Context, + op client.TxnOperator, + ccache *cache.CatalogCache, + typeTs types.TS, + accountIDs []uint32, + applyCatalogCacheChange func(func()), +) (err error) { + result, err := execReadSql(ctx, op, catalog.BuildMoDatabaseBatchQuery(accountIDs), true) + if err != nil { + return err } - logutil.Infof("FIND_TABLE read mo_catalog.mo_databases %v rows", rowCntF(result.Batches)) defer result.Close() + + logutil.Infof("FIND_TABLE read mo_catalog.mo_databases %v rows", rowCountString(result.Batches)) for _, b := range result.Batches { if err = fillTsVecForSysTableQueryBatch(b, typeTs, result.Mp); err != nil { return err } - ccache.InsertDatabase(b) + applyCatalogCacheChange(func() { + ccache.InsertDatabase(b) + }) } + return nil +} - // read tables - result, err = execReadSql(ctx, op, catalog.MoTablesBatchQuery, true) +func replayCatalogTableCache( + ctx context.Context, + e *Engine, + op client.TxnOperator, + ccache *cache.CatalogCache, + typeTs types.TS, + accountIDs []uint32, + applyCatalogCacheChange func(func()), +) (err error) { + result, err := execReadSql(ctx, op, catalog.BuildMoTablesBatchQuery(accountIDs), true) if err != nil { return err } - logutil.Infof("FIND_TABLE read mo_catalog.mo_tables %v rows", rowCntF(result.Batches)) defer result.Close() + + logutil.Infof("FIND_TABLE read mo_catalog.mo_tables %v rows", rowCountString(result.Batches)) for _, b := range result.Batches { if err = fillTsVecForSysTableQueryBatch(b, typeTs, result.Mp); err != nil { return err } e.tryAdjustSysTablesCreatedTimeWithBatch(b) - ccache.InsertTable(b) + applyCatalogCacheChange(func() { + ccache.InsertTable(b) + }) } + return nil +} - // read columns - result, err = execReadSql(ctx, op, catalog.MoColumnsBatchQuery, true) +func replayCatalogColumnCache( + ctx context.Context, + op client.TxnOperator, + ccache *cache.CatalogCache, + typeTs types.TS, + accountIDs []uint32, + applyCatalogCacheChange func(func()), +) (err error) { + result, err := execReadSql(ctx, op, catalog.BuildMoColumnsBatchQuery(accountIDs), true) if err != nil { return err } defer result.Close() - logutil.Infof("FIND_TABLE read mo_catalog.mo_columns %v rows", rowCntF(result.Batches)) + + logutil.Infof("FIND_TABLE read mo_catalog.mo_columns %v rows", rowCountString(result.Batches)) if isColumnsBatchPerfectlySplitted(result.Batches) { for _, b := range result.Batches { if err = fillTsVecForSysTableQueryBatch(b, typeTs, result.Mp); err != nil { return err } - ccache.InsertColumns(b) + applyCatalogCacheChange(func() { + ccache.InsertColumns(b) + }) } - } else { - logutil.Info("FIND_TABLE merge mo_columns results") - bat := result.Batches[0] - for _, b := range result.Batches[1:] { - bat, err = bat.Append(ctx, result.Mp, b) - if err != nil { - return err - } - } - if err = fillTsVecForSysTableQueryBatch(bat, typeTs, result.Mp); err != nil { + return nil + } + + logutil.Info("FIND_TABLE merge mo_columns results") + bat := result.Batches[0] + for _, b := range result.Batches[1:] { + bat, err = bat.Append(ctx, result.Mp, b) + if err != nil { return err } - ccache.InsertColumns(bat) } - - ccache.UpdateDuration(typeTs, types.MaxTs()) - c.dcaConfirmAndApply() + if err = fillTsVecForSysTableQueryBatch(bat, typeTs, result.Mp); err != nil { + return err + } + applyCatalogCacheChange(func() { + ccache.InsertColumns(bat) + }) return nil +} +func rowCountString(batches []*batch.Batch) string { + return stringifySlice(batches, func(b any) string { + return fmt.Sprintf("%d", b.(*batch.Batch).RowCount()) + }) } func (c *PushClient) connect(ctx context.Context, e *Engine) { @@ -874,7 +1239,7 @@ func (c *PushClient) connect(ctx context.Context, e *Engine) { for { c.dcaReset() - err := c.subSysTables(ctx) + err := c.subSysTables(ctx, []uint32{0}) if err != nil { if errors.Is(err, context.Canceled) { logutil.Errorf("%s connect failed as context canceled: %v", logTag, ctx.Err()) @@ -908,6 +1273,11 @@ func (c *PushClient) connect(ctx context.Context, e *Engine) { e.setPushClientStatus(false) + if c.lazyCatalog != nil { + c.lazyCatalog.collectWantedAccounts() + c.lazyCatalog.resetAllStates() + } + // the consumer goroutine is supposed to be stopped. c.stopConsumers() @@ -946,14 +1316,10 @@ func (c *PushClient) connect(ctx context.Context, e *Engine) { c.resume() c.dcaReset() - err = c.subSysTables(ctx) - if err != nil { - // send on closed channel error: - // receive logtail error -> pause -> reconnect -------------------------> stop - // |-> forced subscribe table timeout -> continue ----> resume - // Any errors related to the logtail consumer should not be retried within the inner connect loop; they should be handled by the outer caller. - // So we break the loop here. + reconnectAccounts := c.reconnectInitialActiveAccounts() + err = c.subSysTables(ctx, reconnectAccounts) + if err != nil { c.pause(true) logutil.Errorf("%s subscribe system tables failed, err %v", logTag, err) break @@ -961,7 +1327,7 @@ func (c *PushClient) connect(ctx context.Context, e *Engine) { c.waitTimestamp() - if err := c.replayCatalogCache(ctx, e); err != nil { + if err := c.replayCatalogCacheForReconnect(ctx, e, reconnectAccounts); err != nil { c.pause(true) logutil.Errorf("%s replay catalog cache failed, err %v", logTag, err) break @@ -1721,6 +2087,13 @@ func (s *logTailSubscriber) subscribeTable( return moerr.AttachCause(ctx, err) } +func (s *logTailSubscriber) subscribeCatalogTable( + ctx context.Context, tblId api.TableID, initialActiveAccounts []uint32, +) error { + err := s.logTailClient.SubscribeCatalogTable(ctx, tblId, initialActiveAccounts) + return moerr.AttachCause(ctx, err) +} + // can't call this method directly. func (s *logTailSubscriber) unSubscribeTable( ctx context.Context, tblId api.TableID) error { diff --git a/pkg/vm/engine/disttae/logtail_consumer_test.go b/pkg/vm/engine/disttae/logtail_consumer_test.go index dbdf01af88f68..184eabed68805 100644 --- a/pkg/vm/engine/disttae/logtail_consumer_test.go +++ b/pkg/vm/engine/disttae/logtail_consumer_test.go @@ -319,6 +319,7 @@ func TestBlockInfoSlice(t *testing.T) { func TestDca(t *testing.T) { pClient := &PushClient{} + pClient.dcaReset() signalCnt := 0 assert.True(t, pClient.dcaTryDelay(true, func() { signalCnt++ })) // skip for sub response @@ -329,6 +330,23 @@ func TestDca(t *testing.T) { } +func TestDcaConfirmAndApplyDrainsNewClosuresBeforeReplay(t *testing.T) { + pClient := &PushClient{} + pClient.dcaReset() + + order := make([]int, 0, 2) + assert.True(t, pClient.dcaTryDelay(false, func() { + order = append(order, 1) + assert.True(t, pClient.dcaTryDelay(false, func() { + order = append(order, 2) + })) + })) + + pClient.dcaConfirmAndApply() + assert.Equal(t, []int{1, 2}, order) + assert.False(t, pClient.dcaTryDelay(false, func() {})) +} + type testHAKeeperClient struct { sync.RWMutex value log.ClusterDetails diff --git a/pkg/vm/engine/disttae/logtailreplay/debug.go b/pkg/vm/engine/disttae/logtailreplay/debug.go new file mode 100644 index 0000000000000..b3ac6c1d1a748 --- /dev/null +++ b/pkg/vm/engine/disttae/logtailreplay/debug.go @@ -0,0 +1,53 @@ +// Copyright 2022 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package logtailreplay + +import "github.com/matrixorigin/matrixone/pkg/pb/timestamp" + +type DebugPartitionStateSummary struct { + Start timestamp.Timestamp `json:"start"` + End timestamp.Timestamp `json:"end"` + Rows int `json:"rows"` + Checkpoints int `json:"checkpoints"` + DataObjects int `json:"data_objects"` + TombstoneObjects int `json:"tombstone_objects"` + Prefetch bool `json:"prefetch"` + NoData bool `json:"no_data"` + LastFlushTimestamp timestamp.Timestamp `json:"last_flush_timestamp"` +} + +func (p *PartitionState) DebugSummary() DebugPartitionStateSummary { + if p == nil { + return DebugPartitionStateSummary{} + } + return DebugPartitionStateSummary{ + Start: p.start.ToTimestamp(), + End: p.end.ToTimestamp(), + Rows: p.rows.Len(), + Checkpoints: len(p.checkpoints), + DataObjects: p.dataObjectsNameIndex.Len(), + TombstoneObjects: p.tombstoneObjectsNameIndex.Len(), + Prefetch: p.prefetch, + NoData: p.noData, + LastFlushTimestamp: p.lastFlushTimestamp.ToTimestamp(), + } +} + +func (p *Partition) CheckpointConsumed() bool { + if p == nil { + return false + } + return p.checkpointConsumed.Load() +} diff --git a/pkg/vm/engine/disttae/mo_table_stats.go b/pkg/vm/engine/disttae/mo_table_stats.go index bcf6dbe308c3c..985d3b7d21b15 100644 --- a/pkg/vm/engine/disttae/mo_table_stats.go +++ b/pkg/vm/engine/disttae/mo_table_stats.go @@ -1502,8 +1502,16 @@ func buildTablePairFromCache( snapshot types.TS, onlyUpdateTS bool, ) (tbl tablePair, ok bool) { + distEng := eng.(*Engine) + cache := distEng.GetLatestCatalogCache() + snapshotTS := snapshot.ToTimestamp() + if distEng.pClient.lazyCatalog != nil && distEng.pClient.lazyCatalog.isEnabled() { + if !cache.CanServe(snapshot) || !distEng.PushClient().CanServeAccount(uint32(accId), snapshotTS) { + return + } + } - item := eng.(*Engine).GetLatestCatalogCache().GetTableByIdAndTime(uint32(accId), dbId, tblId, snapshot.ToTimestamp()) + item := cache.GetTableByIdAndTime(uint32(accId), dbId, tblId, snapshotTS) if item == nil || item.IsDeleted() { // account, db, tbl may delete already // the `update_time` not change anymore diff --git a/pkg/vm/engine/disttae/stats.go b/pkg/vm/engine/disttae/stats.go index ff99f1216c8dc..d7f448a830750 100644 --- a/pkg/vm/engine/disttae/stats.go +++ b/pkg/vm/engine/disttae/stats.go @@ -944,6 +944,11 @@ func (gs *GlobalStats) RefreshWithMode(ctx context.Context, key pb.StatsInfoKey, return moerr.NewInternalErrorNoCtxf("failed to subscribe table: %v", err) } + if !gs.canServeCatalogForAccount(key.AccId) { + logutil.Infof("skip stats refresh before catalog ready, key=%v", key) + return nil + } + // Get table definition table := gs.engine.GetLatestCatalogCache().GetTableById(key.AccId, key.DatabaseID, key.TableID) if table == nil || table.TableDef == nil { @@ -989,7 +994,26 @@ func (gs *GlobalStats) RefreshWithMode(ctx context.Context, key pb.StatsInfoKey, return nil } +func (gs *GlobalStats) canServeCatalogForAccount(accountID uint32) bool { + if gs.engine.pClient.lazyCatalog == nil || !gs.engine.pClient.lazyCatalog.isEnabled() { + return true + } + + latestTS := gs.engine.pClient.LatestLogtailAppliedTime() + if latestTS.IsEmpty() { + return false + } + cache := gs.engine.GetLatestCatalogCache() + return cache.CanServe(types.TimestampToTS(latestTS)) && + gs.engine.pClient.CanServeAccount(accountID, latestTS) +} + func (gs *GlobalStats) executeStatsUpdate(ctx context.Context, ps *logtailreplay.PartitionState, key pb.StatsInfoKey, stats *pb.StatsInfo) (bool, float64) { + if !gs.canServeCatalogForAccount(key.AccId) { + logutil.Infof("skip stats update before catalog ready, key=%v", key) + return false, 0 + } + table := gs.engine.GetLatestCatalogCache().GetTableById(key.AccId, key.DatabaseID, key.TableID) // table or its definition is nil, means that the table is created but not committed yet. if table == nil || table.TableDef == nil { diff --git a/pkg/vm/engine/disttae/txn_database.go b/pkg/vm/engine/disttae/txn_database.go index bd6e2dfaf79f9..ff2429d3bac74 100644 --- a/pkg/vm/engine/disttae/txn_database.go +++ b/pkg/vm/engine/disttae/txn_database.go @@ -682,7 +682,8 @@ func (db *txnDatabase) getTableItem( c := engine.GetLatestCatalogCache() if ok := c.GetTable(&item); !ok { var tableitem *cache.TableItem - if !c.CanServe(types.TimestampToTS(db.op.SnapshotTS())) { + if !c.CanServe(types.TimestampToTS(db.op.SnapshotTS())) || + !engine.pClient.CanServeAccount(accountID, db.op.SnapshotTS()) { logutil.Info("FIND_TABLE loadTableFromStorage", zap.String("table", name), zap.Uint32("accountID", accountID), zap.String("txn", db.op.Txn().DebugString()), zap.String("cacheTS", c.GetStartTS().ToString())) if tableitem, err = db.loadTableFromStorage(ctx, accountID, name); err != nil { return nil, err diff --git a/pkg/vm/engine/entire_engine.go b/pkg/vm/engine/entire_engine.go index b763db71fc4ad..7c7a51904e54d 100644 --- a/pkg/vm/engine/entire_engine.go +++ b/pkg/vm/engine/entire_engine.go @@ -104,3 +104,10 @@ func (e *EntireEngine) GetMessageCenter() any { func (e *EntireEngine) GetService() string { return e.Engine.GetService() } + +func (e *EntireEngine) ActivateTenantCatalog(ctx context.Context, accountID uint32) error { + if activator, ok := e.Engine.(TenantCatalogActivator); ok { + return activator.ActivateTenantCatalog(ctx, accountID) + } + return nil +} diff --git a/pkg/vm/engine/entire_engine_test.go b/pkg/vm/engine/entire_engine_test.go index 95e60d81ae00c..a598982255cce 100644 --- a/pkg/vm/engine/entire_engine_test.go +++ b/pkg/vm/engine/entire_engine_test.go @@ -44,8 +44,9 @@ const ( type testEntireEngine struct { EntireEngine - step int - state int + step int + state int + activatedAccount uint32 } type testEngine struct { @@ -137,6 +138,12 @@ func TestEntireEngineHints(t *testing.T) { } +func TestEntireEngineActivateTenantCatalog(t *testing.T) { + ee := buildTestEntireEngine() + assert.NoError(t, ee.ActivateTenantCatalog(context.Background(), 42)) + assert.Equal(t, uint32(42), ee.activatedAccount) +} + func buildTestEntireEngine() *testEntireEngine { ee := new(testEntireEngine) ee.state = 1 @@ -172,6 +179,11 @@ func (e *testEngine) New(_ context.Context, _ client.TxnOperator) error { return nil } +func (e *testEngine) ActivateTenantCatalog(_ context.Context, accountID uint32) error { + e.parent.activatedAccount = accountID + return nil +} + func (e *testEngine) Commit(_ context.Context, _ client.TxnOperator) error { e.parent.step = e.parent.step + 1 if e.name == origin { diff --git a/pkg/vm/engine/tae/logtail/service/catalog_filter.go b/pkg/vm/engine/tae/logtail/service/catalog_filter.go new file mode 100644 index 0000000000000..3f6dca6b1ae08 --- /dev/null +++ b/pkg/vm/engine/tae/logtail/service/catalog_filter.go @@ -0,0 +1,265 @@ +// Copyright 2024 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package service + +import ( + "github.com/matrixorigin/matrixone/pkg/catalog" + "github.com/matrixorigin/matrixone/pkg/common/moerr" + "github.com/matrixorigin/matrixone/pkg/common/mpool" + "github.com/matrixorigin/matrixone/pkg/container/batch" + "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/container/vector" + "github.com/matrixorigin/matrixone/pkg/pb/api" + "github.com/matrixorigin/matrixone/pkg/pb/logtail" +) + +var lazyCatalogSubscribeFilterMP = mpool.MustNew("lazy-catalog-subscribe-filter") + +func filterLazyCatalogPulledTail( + tail logtail.TableLogtail, + allowedAccounts *lazyCatalogAllowedAccounts, +) (logtail.TableLogtail, func(), error) { + if allowedAccounts == nil || !isLazyCatalogTableID(tail.Table) { + return tail, nil, nil + } + return filterLazyCatalogSubscribeRowsInTail(tail, allowedAccounts) +} + +func filterLazyCatalogSubscribeRowsInTail( + tail logtail.TableLogtail, + allowedAccounts *lazyCatalogAllowedAccounts, +) (logtail.TableLogtail, func(), error) { + // Subscribe snapshots can still carry mixed-account api.Entry batches for the + // three catalog tables, so the TN side must copy only the target rows. + if len(tail.Commands) == 0 { + return tail, nil, nil + } + + filtered := tail + var closeCBs []func() + // Raw checkpoint locations are not account-filtered. Lazy catalog subscribe + // and activation responses must therefore forward only filtered row data. + filtered.CkpLocation = "" + filtered.Commands = make([]api.Entry, 0, len(tail.Commands)) + for i := range tail.Commands { + entry, keep, closeCB, err := filterLazyCatalogSubscribeEntry(tail.Commands[i], allowedAccounts) + if err != nil { + closeCallbacks(closeCB) + closeCallbacks(closeCBs...) + return logtail.TableLogtail{}, nil, err + } + if keep { + filtered.Commands = append(filtered.Commands, entry) + } + if closeCB != nil { + closeCBs = append(closeCBs, closeCB) + } + } + return filtered, composeCloseCallback(closeCBs...), nil +} + +func filterLazyCatalogPublishRowsInTail( + tail logtail.TableLogtail, + allowedAccounts *lazyCatalogAllowedAccounts, +) (logtail.TableLogtail, bool, error) { + // This helper is only for the three lazy-load catalog tables. Callers are + // expected to keep normal-table publish/subscribe paths out of here. + if len(tail.Commands) == 0 { + return tail, false, nil + } + + var filteredCommands []api.Entry + for i := range tail.Commands { + keep, err := filterLazyCatalogPublishEntry(tail.Commands[i], allowedAccounts) + if err != nil { + return logtail.TableLogtail{}, false, err + } + if filteredCommands == nil { + if keep { + continue + } + filteredCommands = make([]api.Entry, 0, len(tail.Commands)-1) + filteredCommands = append(filteredCommands, tail.Commands[:i]...) + continue + } + if keep { + filteredCommands = append(filteredCommands, tail.Commands[i]) + } + } + if filteredCommands == nil { + return tail, false, nil + } + + filtered := tail + filtered.Commands = filteredCommands + return filtered, true, nil +} + +func filterLazyCatalogSubscribeEntry( + entry api.Entry, + allowedAccounts *lazyCatalogAllowedAccounts, +) (api.Entry, bool, func(), error) { + switch entry.GetEntryType() { + case api.Entry_Insert, api.Entry_Update: + return filterLazyCatalogSubscribeInsertOrUpdateEntry(entry, allowedAccounts) + case api.Entry_Delete: + return filterLazyCatalogSubscribeDeleteEntry(entry, allowedAccounts) + default: + // Object/meta entries are not row-level tenant data; keep them untouched. + return entry, true, nil, nil + } +} + +func filterLazyCatalogPublishEntry( + entry api.Entry, + allowedAccounts *lazyCatalogAllowedAccounts, +) (bool, error) { + switch entry.GetEntryType() { + case api.Entry_Insert, api.Entry_Update, api.Entry_Delete: + accountID, ok, err := catalog.LazyCatalogEntryAccountID(entry) + if err != nil { + return false, err + } + if !ok { + return true, nil + } + return allowedAccounts.contains(accountID), nil + default: + // Object/meta entries are not row-level tenant data; keep them untouched. + return true, nil + } +} + +func filterLazyCatalogSubscribeInsertOrUpdateEntry( + entry api.Entry, + allowedAccounts *lazyCatalogAllowedAccounts, +) (api.Entry, bool, func(), error) { + bat, err := mustProtoBatch(entry) + if err != nil { + return api.Entry{}, false, nil, err + } + if bat.RowCount() == 0 { + return entry, true, nil, nil + } + + accountIdx := catalog.FindBatchAttrIndex(bat.Attrs, catalog.SystemDBAttr_AccID) + if accountIdx < 0 { + return api.Entry{}, false, nil, moerr.NewInternalErrorNoCtxf( + "catalog logtail entry %s missing account_id column, attrs=%v", + entry.GetEntryType().String(), + bat.Attrs, + ) + } + + accounts := vector.MustFixedColWithTypeCheck[uint32](bat.GetVector(int32(accountIdx))) + selectedRows := make([]int64, 0, len(accounts)) + for row, accountID := range accounts { + if allowedAccounts.contains(accountID) { + selectedRows = append(selectedRows, int64(row)) + } + } + return buildFilteredCatalogEntry(entry, bat, selectedRows) +} + +func filterLazyCatalogSubscribeDeleteEntry( + entry api.Entry, + allowedAccounts *lazyCatalogAllowedAccounts, +) (api.Entry, bool, func(), error) { + bat, err := mustProtoBatch(entry) + if err != nil { + return api.Entry{}, false, nil, err + } + if bat.RowCount() == 0 { + return entry, true, nil, nil + } + + // Insert/update entries use __mo_cpkey_col; tombstone/delete entries + // use __mo_%1_pk_val. Both contain the same compound-key bytes. + cpkeyIdx := catalog.FindCatalogDeletePKIndex(bat.Attrs) + if cpkeyIdx < 0 { + return api.Entry{}, false, nil, moerr.NewInternalErrorNoCtxf( + "catalog delete logtail entry missing cpkey/pk column, attrs=%v", + bat.Attrs, + ) + } + + selectedRows := make([]int64, 0, bat.RowCount()) + cpkeyVec := bat.GetVector(int32(cpkeyIdx)) + for row := 0; row < bat.RowCount(); row++ { + accountID, err := catalog.DecodeLazyCatalogAccountFromCPKey(cpkeyVec.GetBytesAt(row)) + if err != nil { + return api.Entry{}, false, nil, err + } + if allowedAccounts.contains(accountID) { + selectedRows = append(selectedRows, int64(row)) + } + } + return buildFilteredCatalogEntry(entry, bat, selectedRows) +} + +func buildFilteredCatalogEntry( + entry api.Entry, + bat *batch.Batch, + selectedRows []int64, +) (api.Entry, bool, func(), error) { + switch { + case len(selectedRows) == 0: + return api.Entry{}, false, nil, nil + case len(selectedRows) == bat.RowCount(): + return entry, true, nil, nil + } + + copiedBat, closeCB, err := copyProtoBatchRows(bat, selectedRows) + if err != nil { + return api.Entry{}, false, nil, err + } + filtered := entry + filtered.Bat = copiedBat + return filtered, true, closeCB, nil +} + +func copyProtoBatchRows(src *batch.Batch, selectedRows []int64) (*api.Batch, func(), error) { + attrs := append([]string(nil), src.Attrs...) + typesByAttr := make([]types.Type, len(src.Vecs)) + for i, vec := range src.Vecs { + typesByAttr[i] = *vec.GetType() + } + + copied := batch.NewWithSchema(false, attrs, typesByAttr) + + if err := copied.Union(src, selectedRows, lazyCatalogSubscribeFilterMP); err != nil { + copied.Clean(lazyCatalogSubscribeFilterMP) + return nil, nil, err + } + + pbBat, err := batch.BatchToProtoBatch(copied) + if err != nil { + copied.Clean(lazyCatalogSubscribeFilterMP) + return nil, nil, err + } + return pbBat, func() { + copied.Clean(lazyCatalogSubscribeFilterMP) + }, nil +} + +func mustProtoBatch(entry api.Entry) (*batch.Batch, error) { + if entry.Bat == nil { + return nil, moerr.NewInternalErrorNoCtxf( + "catalog logtail entry %s missing batch", + entry.GetEntryType().String(), + ) + } + return batch.ProtoBatchToBatch(entry.Bat) +} diff --git a/pkg/vm/engine/tae/logtail/service/catalog_filter_test.go b/pkg/vm/engine/tae/logtail/service/catalog_filter_test.go new file mode 100644 index 0000000000000..961fc1fb4a958 --- /dev/null +++ b/pkg/vm/engine/tae/logtail/service/catalog_filter_test.go @@ -0,0 +1,300 @@ +// Copyright 2024 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package service + +import ( + "context" + "slices" + "testing" + "time" + + "github.com/stretchr/testify/require" + + "github.com/matrixorigin/matrixone/pkg/catalog" + "github.com/matrixorigin/matrixone/pkg/common/mpool" + "github.com/matrixorigin/matrixone/pkg/container/batch" + "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/container/vector" + "github.com/matrixorigin/matrixone/pkg/logutil" + "github.com/matrixorigin/matrixone/pkg/pb/api" + "github.com/matrixorigin/matrixone/pkg/pb/logtail" + "github.com/matrixorigin/matrixone/pkg/pb/timestamp" +) + +func TestSessionConfigureLazyCatalogSubscription(t *testing.T) { + ss := newCatalogTestSession(t) + + req := &logtail.SubscribeRequest{ + Table: &api.TableID{DbId: catalog.MO_CATALOG_ID, TbId: catalog.MO_TABLES_ID}, + LazyCatalog: true, + InitialActiveAccounts: []uint32{0, 10}, + } + require.NoError(t, ss.configureLazyCatalogSubscription(req)) + require.True(t, ss.lazyCatalog.enabled) + _, ok := ss.lazyCatalog.activeAccounts[0] + require.True(t, ok) + _, ok = ss.lazyCatalog.activeAccounts[10] + require.True(t, ok) + require.Nil(t, ss.lazyCatalog.activatingSeqByAccount) + + err := ss.configureLazyCatalogSubscription(&logtail.SubscribeRequest{ + Table: &api.TableID{DbId: 10, TbId: 20}, + LazyCatalog: true, + }) + require.Error(t, err) +} + +func TestSessionPrepareLazyCatalogPublishWrapsFiltersCatalogRowsByAccount(t *testing.T) { + ss := newCatalogTestSession(t) + table := api.TableID{DbId: catalog.MO_CATALOG_ID, TbId: catalog.MO_COLUMNS_ID} + id := MarshalTableID(&table) + + require.False(t, ss.Register(id, table)) + require.NoError(t, ss.configureLazyCatalogSubscription(&logtail.SubscribeRequest{ + Table: &table, + LazyCatalog: true, + InitialActiveAccounts: []uint32{0}, + })) + ss.AdvanceState(id) + + wraps := []wrapLogtail{{ + id: id, + tail: logtail.TableLogtail{ + Table: &table, + Ts: ×tamp.Timestamp{PhysicalTime: 1}, + Commands: []api.Entry{ + mustCatalogColumnInsertEntry(t, []uint32{0}), + mustCatalogColumnInsertEntry(t, []uint32{10}), + }, + }, + }} + filtered, err := ss.prepareLazyCatalogPublishWrapsFromIndex(wraps, slices.IndexFunc(wraps, func(w wrapLogtail) bool { + return catalog.IsLazyCatalogTableID(w.tail.Table.TbId) + })) + require.NoError(t, err) + require.Len(t, filtered, 1) + require.Len(t, filtered[0].tail.Commands, 1) + require.Equal(t, []uint32{0}, mustAccountIDsFromEntry(t, filtered[0].tail.Commands[0])) +} + +func TestFilterLazyCatalogSubscribeRowsInTailCopiesMixedInsertEntry(t *testing.T) { + filtered, closeCB, err := filterLazyCatalogSubscribeRowsInTail(logtail.TableLogtail{ + Table: &api.TableID{DbId: catalog.MO_CATALOG_ID, TbId: catalog.MO_COLUMNS_ID}, + CkpLocation: "ckp:should-be-stripped", + Commands: []api.Entry{ + mustCatalogColumnInsertEntry(t, []uint32{0, 10, 20}), + }, + }, &lazyCatalogAllowedAccounts{accounts: map[uint32]struct{}{10: {}}}) + require.NoError(t, err) + require.NotNil(t, closeCB) + if closeCB != nil { + t.Cleanup(closeCB) + } + require.Empty(t, filtered.CkpLocation) + require.Len(t, filtered.Commands, 1) + require.Equal(t, []uint32{10}, mustAccountIDsFromEntry(t, filtered.Commands[0])) +} + +func TestSessionPrepareLazyCatalogPublishWrapsLeavesNormalTableUntouched(t *testing.T) { + ss := newCatalogTestSession(t) + require.NoError(t, ss.configureLazyCatalogSubscription(&logtail.SubscribeRequest{ + Table: &api.TableID{DbId: catalog.MO_CATALOG_ID, TbId: catalog.MO_TABLES_ID}, + LazyCatalog: true, + InitialActiveAccounts: []uint32{0}, + })) + + normal := wrapLogtail{ + id: MarshalTableID(&api.TableID{DbId: 10, TbId: 20}), + tail: logtail.TableLogtail{ + Table: &api.TableID{DbId: 10, TbId: 20}, + Ts: ×tamp.Timestamp{PhysicalTime: 1}, + Commands: []api.Entry{{ + EntryType: api.Entry_Insert, + }}, + }, + } + + wraps := []wrapLogtail{normal} + filtered, err := ss.prepareLazyCatalogPublishWrapsFromIndex(wraps, slices.IndexFunc(wraps, func(w wrapLogtail) bool { + return catalog.IsLazyCatalogTableID(w.tail.Table.TbId) + })) + require.NoError(t, err) + require.Len(t, filtered, 1) + require.Equal(t, normal.tail.Table.String(), filtered[0].tail.Table.String()) + require.Len(t, filtered[0].tail.Commands, 1) +} + +func TestFilterLazyCatalogSubscribeRowsInTailUsesCPKeyForDelete(t *testing.T) { + filtered, closeCB, err := filterLazyCatalogSubscribeRowsInTail(logtail.TableLogtail{ + Table: &api.TableID{DbId: catalog.MO_CATALOG_ID, TbId: catalog.MO_TABLES_ID}, + Commands: []api.Entry{ + mustCatalogTableDeleteEntry(t, []uint32{0, 10}), + }, + }, &lazyCatalogAllowedAccounts{accounts: map[uint32]struct{}{10: {}}}) + require.NoError(t, err) + require.NotNil(t, closeCB) + if closeCB != nil { + t.Cleanup(closeCB) + } + require.Len(t, filtered.Commands, 1) + require.Equal(t, []uint32{10}, mustCPKeyAccountsFromEntry(t, filtered.Commands[0])) +} + +func TestFilterLazyCatalogPublishRowsInTailUsesEntryAccountSummary(t *testing.T) { + table := api.TableID{DbId: catalog.MO_CATALOG_ID, TbId: catalog.MO_COLUMNS_ID} + entry := api.Entry{ + EntryType: api.Entry_Insert, + TableId: catalog.MO_COLUMNS_ID, + DatabaseId: catalog.MO_CATALOG_ID, + } + catalog.SetLazyCatalogEntryAccountSummary(&entry, 10) + + tail := logtail.TableLogtail{ + Table: &table, + Commands: []api.Entry{entry}, + } + + filtered, changed, err := filterLazyCatalogPublishRowsInTail( + tail, + &lazyCatalogAllowedAccounts{accounts: map[uint32]struct{}{10: {}}}, + ) + require.NoError(t, err) + require.False(t, changed) + require.True(t, &filtered.Commands[0] == &tail.Commands[0]) + + filtered, changed, err = filterLazyCatalogPublishRowsInTail( + tail, + &lazyCatalogAllowedAccounts{accounts: map[uint32]struct{}{20: {}}}, + ) + require.NoError(t, err) + require.True(t, changed) + require.Empty(t, filtered.Commands) +} + +func newCatalogTestSession(t *testing.T) *Session { + t.Helper() + return NewSession( + context.Background(), + mockMOLogger(), + NewLogtailResponsePool(), + mockSessionErrorNotifier(logutil.GetGlobalLogger()), + mockMorpcStream(&normalStream{}, 1, 1024), + time.Second, + time.Second, + time.Second, + ) +} + +func mustCatalogColumnInsertEntry(t *testing.T, accountIDs []uint32) api.Entry { + t.Helper() + mp := mpool.MustNew("catalog-filter-columns") + packer := types.NewPacker() + defer packer.Close() + typ := types.T_int64.ToType() + typEncoded, err := types.Encode(&typ) + require.NoError(t, err) + + cols := make([]catalog.Column, 0, len(accountIDs)) + for i, accountID := range accountIDs { + cols = append(cols, catalog.Column{ + AccountId: accountID, + DatabaseId: uint64(100 + i), + DatabaseName: "db", + TableId: uint64(200 + i), + TableName: "tbl", + Name: string(rune('a' + i)), + Num: int32(i), + Typ: typEncoded, + TypLen: int32(len(typEncoded)), + }) + } + + bat, err := catalog.GenCreateColumnTuples(cols, mp, packer) + require.NoError(t, err) + t.Cleanup(func() { bat.Clean(mp) }) + + pbBat, err := batch.BatchToProtoBatch(bat) + require.NoError(t, err) + return api.Entry{ + EntryType: api.Entry_Insert, + TableId: catalog.MO_COLUMNS_ID, + DatabaseId: catalog.MO_CATALOG_ID, + Bat: pbBat, + } +} + +func mustCatalogTableDeleteEntry(t *testing.T, accountIDs []uint32) api.Entry { + t.Helper() + mp := mpool.MustNew("catalog-filter-delete") + packer := types.NewPacker() + defer packer.Close() + + var batAcc *batch.Batch + for i, accountID := range accountIDs { + bat, err := catalog.GenDropTableTuple( + types.RandomRowid(), + accountID, + uint64(200+i), + uint64(100+i), + "tbl", + "db", + mp, + packer, + ) + require.NoError(t, err) + if batAcc == nil { + batAcc = bat + continue + } + require.NoError(t, batAcc.UnionOne(bat, 0, mp)) + bat.Clean(mp) + } + t.Cleanup(func() { batAcc.Clean(mp) }) + + pbBat, err := batch.BatchToProtoBatch(batAcc) + require.NoError(t, err) + return api.Entry{ + EntryType: api.Entry_Delete, + TableId: catalog.MO_TABLES_ID, + DatabaseId: catalog.MO_CATALOG_ID, + Bat: pbBat, + } +} + +func mustAccountIDsFromEntry(t *testing.T, entry api.Entry) []uint32 { + t.Helper() + bat, err := batch.ProtoBatchToBatch(entry.Bat) + require.NoError(t, err) + accountIdx := catalog.FindBatchAttrIndex(bat.Attrs, catalog.SystemDBAttr_AccID) + require.GreaterOrEqual(t, accountIdx, 0) + return append([]uint32(nil), vector.MustFixedColWithTypeCheck[uint32](bat.GetVector(int32(accountIdx)))...) +} + +func mustCPKeyAccountsFromEntry(t *testing.T, entry api.Entry) []uint32 { + t.Helper() + bat, err := batch.ProtoBatchToBatch(entry.Bat) + require.NoError(t, err) + cpkeyIdx := catalog.FindBatchAttrIndex(bat.Attrs, catalog.CPrimaryKeyColName) + require.GreaterOrEqual(t, cpkeyIdx, 0) + + accounts := make([]uint32, 0, bat.RowCount()) + cpkeyVec := bat.GetVector(int32(cpkeyIdx)) + for row := 0; row < bat.RowCount(); row++ { + accountID, err := catalog.DecodeLazyCatalogAccountFromCPKey(cpkeyVec.GetBytesAt(row)) + require.NoError(t, err) + accounts = append(accounts, accountID) + } + return accounts +} diff --git a/pkg/vm/engine/tae/logtail/service/client.go b/pkg/vm/engine/tae/logtail/service/client.go index ef52e6954b0cb..1c2fc691838c2 100644 --- a/pkg/vm/engine/tae/logtail/service/client.go +++ b/pkg/vm/engine/tae/logtail/service/client.go @@ -49,7 +49,7 @@ type LogtailClient struct { ctx context.Context cancel context.CancelFunc - // requestC is a chan, which receives all sub/unsub request. + // requestC is a chan, which receives all logtail stream requests. // There is another worker send the items in the chan to stream. requestC chan *LogtailRequest @@ -115,6 +115,31 @@ func (c *LogtailClient) Close() error { // Subscribe subscribes table. func (c *LogtailClient) Subscribe( ctx context.Context, table api.TableID, +) error { + return c.sendSubscribeRequest(ctx, &logtail.SubscribeRequest{ + Table: &table, + }) +} + +// SubscribeCatalogTable subscribes one of the three catalog system tables with lazy-catalog metadata. +func (c *LogtailClient) SubscribeCatalogTable( + ctx context.Context, table api.TableID, initialActiveAccounts []uint32, +) error { + if !isLazyCatalogTableID(&table) { + return moerr.NewNotSupportedf(ctx, + "lazy catalog subscribe only supports mo_database/mo_tables/mo_columns, got %s", + table.String(), + ) + } + return c.sendSubscribeRequest(ctx, &logtail.SubscribeRequest{ + Table: &table, + LazyCatalog: true, + InitialActiveAccounts: append([]uint32(nil), initialActiveAccounts...), + }) +} + +func (c *LogtailClient) sendSubscribeRequest( + ctx context.Context, req *logtail.SubscribeRequest, ) error { if c.streamBroken() { logutil.Error("logtail client: subscribe via broken morpc stream") @@ -125,9 +150,34 @@ func (c *LogtailClient) Subscribe( request := &LogtailRequest{} request.Request = &logtail.LogtailRequest_SubscribeTable{ - SubscribeTable: &logtail.SubscribeRequest{ - Table: &table, - }, + SubscribeTable: cloneSubscribeRequest(req), + } + return c.sendRequest(request) +} + +// ActivateAccountForCatalog sends the account-level lazy catalog activation request. +func (c *LogtailClient) ActivateAccountForCatalog( + ctx context.Context, accountID uint32, seq uint64, +) error { + return c.sendActivateAccountForCatalogRequest(ctx, &logtail.ActivateAccountForCatalogRequest{ + AccountId: accountID, + Seq: seq, + }) +} + +func (c *LogtailClient) sendActivateAccountForCatalogRequest( + ctx context.Context, req *logtail.ActivateAccountForCatalogRequest, +) error { + if c.streamBroken() { + logutil.Error("logtail client: activate account via broken morpc stream") + return moerr.NewStreamClosedNoCtx() + } + + c.limiter.Take() + + request := &LogtailRequest{} + request.Request = &logtail.LogtailRequest_ActivateAccountForCatalog{ + ActivateAccountForCatalog: cloneActivateAccountForCatalogRequest(req), } return c.sendRequest(request) } @@ -161,7 +211,8 @@ func (c *LogtailClient) BreakoutReceive() { // 1. response for error: *LogtailResponse.GetError() != nil // 2. response for subscription: *LogtailResponse.GetSubscribeResponse() != nil // 3. response for unsubscription: *LogtailResponse.GetUnsubscribeResponse() != nil -// 3. response for incremental logtail: *LogtailResponse.GetUpdateResponse() != nil +// 4. response for incremental logtail: *LogtailResponse.GetUpdateResponse() != nil +// 5. response for account activation: *LogtailResponse.GetActivateAccountForCatalogResponse() != nil func (c *LogtailClient) Receive(ctx context.Context) (*LogtailResponse, error) { recvFunc := func() (*LogtailResponseSegment, error) { select { @@ -249,8 +300,35 @@ func (c *LogtailClient) sendWorker() error { case request := <-c.requestC: if err := sendFn(request); err != nil { - logutil.Error("logtail client: fail to send sub/unsub request via morpc stream", zap.Error(err)) + logutil.Error("logtail client: fail to send logtail request via morpc stream", zap.Error(err)) } } } } + +func cloneSubscribeRequest(req *logtail.SubscribeRequest) *logtail.SubscribeRequest { + if req == nil { + return &logtail.SubscribeRequest{} + } + clone := &logtail.SubscribeRequest{ + LazyCatalog: req.GetLazyCatalog(), + InitialActiveAccounts: append([]uint32(nil), req.GetInitialActiveAccounts()...), + } + if table := req.GetTable(); table != nil { + tableCopy := *table + clone.Table = &tableCopy + } + return clone +} + +func cloneActivateAccountForCatalogRequest( + req *logtail.ActivateAccountForCatalogRequest, +) *logtail.ActivateAccountForCatalogRequest { + if req == nil { + return &logtail.ActivateAccountForCatalogRequest{} + } + return &logtail.ActivateAccountForCatalogRequest{ + AccountId: req.GetAccountId(), + Seq: req.GetSeq(), + } +} diff --git a/pkg/vm/engine/tae/logtail/service/client_test.go b/pkg/vm/engine/tae/logtail/service/client_test.go new file mode 100644 index 0000000000000..f086d5629a542 --- /dev/null +++ b/pkg/vm/engine/tae/logtail/service/client_test.go @@ -0,0 +1,160 @@ +// Copyright 2021 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package service + +import ( + "context" + "sync" + "testing" + "time" + + "github.com/stretchr/testify/require" + + "github.com/matrixorigin/matrixone/pkg/catalog" + "github.com/matrixorigin/matrixone/pkg/common/morpc" + "github.com/matrixorigin/matrixone/pkg/pb/api" +) + +type testMorpcStream struct { + id uint64 + recv chan morpc.Message + + mu sync.Mutex + sent []*LogtailRequest +} + +func newTestMorpcStream(id uint64) *testMorpcStream { + return &testMorpcStream{ + id: id, + recv: make(chan morpc.Message), + } +} + +func (s *testMorpcStream) ID() uint64 { + return s.id +} + +func (s *testMorpcStream) Send(_ context.Context, request morpc.Message) error { + s.mu.Lock() + defer s.mu.Unlock() + s.sent = append(s.sent, request.(*LogtailRequest)) + return nil +} + +func (s *testMorpcStream) Receive() (chan morpc.Message, error) { + return s.recv, nil +} + +func (s *testMorpcStream) Close(bool) error { + close(s.recv) + return nil +} + +func (s *testMorpcStream) latestRequest() *LogtailRequest { + s.mu.Lock() + defer s.mu.Unlock() + if len(s.sent) == 0 { + return nil + } + return s.sent[len(s.sent)-1] +} + +func TestLogtailClientSubscribe(t *testing.T) { + stream := newTestMorpcStream(42) + client, err := NewLogtailClient(context.Background(), stream, WithClientRequestPerSecond(100)) + require.NoError(t, err) + defer func() { + require.NoError(t, client.Close()) + }() + + table := api.TableID{DbId: 1, TbId: 2, PartitionId: 3} + err = client.Subscribe(context.Background(), table) + require.NoError(t, err) + + require.Eventually(t, func() bool { + return stream.latestRequest() != nil + }, time.Second, time.Millisecond*10) + + req := stream.latestRequest() + require.Equal(t, stream.id, req.GetRequestId()) + sub := req.GetSubscribeTable() + require.NotNil(t, sub) + require.False(t, sub.GetLazyCatalog()) + require.Empty(t, sub.GetInitialActiveAccounts()) + require.Equal(t, table.String(), sub.GetTable().String()) +} + +func TestLogtailClientSubscribeCatalogTable(t *testing.T) { + stream := newTestMorpcStream(43) + client, err := NewLogtailClient(context.Background(), stream, WithClientRequestPerSecond(100)) + require.NoError(t, err) + defer func() { + require.NoError(t, client.Close()) + }() + + table := api.TableID{DbId: catalog.MO_CATALOG_ID, TbId: catalog.MO_TABLES_ID} + initialActiveAccounts := []uint32{0, 10} + err = client.SubscribeCatalogTable(context.Background(), table, initialActiveAccounts) + require.NoError(t, err) + initialActiveAccounts[0] = 99 + + require.Eventually(t, func() bool { + return stream.latestRequest() != nil + }, time.Second, time.Millisecond*10) + + req := stream.latestRequest() + require.Equal(t, stream.id, req.GetRequestId()) + sub := req.GetSubscribeTable() + require.NotNil(t, sub) + require.True(t, sub.GetLazyCatalog()) + require.Equal(t, []uint32{0, 10}, sub.GetInitialActiveAccounts()) + require.Equal(t, table.String(), sub.GetTable().String()) +} + +func TestLogtailClientSubscribeCatalogTableRejectsNonCatalogTable(t *testing.T) { + stream := newTestMorpcStream(44) + client, err := NewLogtailClient(context.Background(), stream, WithClientRequestPerSecond(100)) + require.NoError(t, err) + defer func() { + require.NoError(t, client.Close()) + }() + + err = client.SubscribeCatalogTable(context.Background(), api.TableID{DbId: 10, TbId: 100}, []uint32{0}) + require.Error(t, err) + require.Nil(t, stream.latestRequest()) +} + +func TestLogtailClientActivateAccountForCatalog(t *testing.T) { + stream := newTestMorpcStream(99) + client, err := NewLogtailClient(context.Background(), stream, WithClientRequestPerSecond(100)) + require.NoError(t, err) + defer func() { + require.NoError(t, client.Close()) + }() + + err = client.ActivateAccountForCatalog(context.Background(), 7, 88) + require.NoError(t, err) + + require.Eventually(t, func() bool { + return stream.latestRequest() != nil + }, time.Second, time.Millisecond*10) + + req := stream.latestRequest() + require.Equal(t, stream.id, req.GetRequestId()) + activate := req.GetActivateAccountForCatalog() + require.NotNil(t, activate) + require.Equal(t, uint32(7), activate.GetAccountId()) + require.Equal(t, uint64(88), activate.GetSeq()) +} diff --git a/pkg/vm/engine/tae/logtail/service/lazy_catalog_session.go b/pkg/vm/engine/tae/logtail/service/lazy_catalog_session.go new file mode 100644 index 0000000000000..9fa779f2de348 --- /dev/null +++ b/pkg/vm/engine/tae/logtail/service/lazy_catalog_session.go @@ -0,0 +1,255 @@ +// Copyright 2024 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package service + +import ( + "sync/atomic" + + "github.com/matrixorigin/matrixone/pkg/catalog" + "github.com/matrixorigin/matrixone/pkg/common/moerr" + "github.com/matrixorigin/matrixone/pkg/pb/api" + "github.com/matrixorigin/matrixone/pkg/pb/logtail" +) + +type lazyCatalogAllowedAccounts struct { + accounts map[uint32]struct{} +} + +func newLazyCatalogAllowedAccounts(accountIDs ...uint32) *lazyCatalogAllowedAccounts { + allowed := &lazyCatalogAllowedAccounts{ + accounts: make(map[uint32]struct{}, len(accountIDs)), + } + for _, accountID := range accountIDs { + allowed.accounts[accountID] = struct{}{} + } + return allowed +} + +func (s *lazyCatalogAllowedAccounts) contains(accountID uint32) bool { + if s == nil { + return false + } + _, ok := s.accounts[accountID] + return ok +} + +// lazyCatalogFilterState keeps the per-session tenant filter state for the three +// catalog system tables. The whole struct is guarded by Session.mu. +type lazyCatalogFilterState struct { + enabled bool + + // activeAccounts are already allowed in steady-state push. + activeAccounts map[uint32]struct{} + // activeAccountsSnapshot is rebuilt only when the active set changes so the + // publish fast path can read a stable view without cloning on every push. + activeAccountsSnapshot atomic.Pointer[lazyCatalogAllowedAccounts] + // activatingSeqByAccount belongs to the later tn-activation-sender step. + // The tn-filter refactor keeps the state slot here, but does not manipulate it yet. + activatingSeqByAccount map[uint32]uint64 +} + +func (s *lazyCatalogFilterState) ensureActiveAccounts() { + if s.activeAccounts == nil { + s.activeAccounts = make(map[uint32]struct{}) + } +} + +func (s *lazyCatalogFilterState) configure(initialActiveAccounts []uint32) { + s.enabled = true + s.ensureActiveAccounts() + for _, accountID := range initialActiveAccounts { + s.activeAccounts[accountID] = struct{}{} + } + s.storeActiveAccountsSnapshot() +} + +func (s *lazyCatalogFilterState) storeActiveAccountsSnapshot() { + snapshot := &lazyCatalogAllowedAccounts{ + accounts: make(map[uint32]struct{}, len(s.activeAccounts)), + } + for accountID := range s.activeAccounts { + snapshot.accounts[accountID] = struct{}{} + } + s.activeAccountsSnapshot.Store(snapshot) +} + +// beginActivation records that an activation is in progress for the given +// account with the given seq. Returns false if the session is not in lazy +// catalog mode. +func (s *lazyCatalogFilterState) beginActivation(accountID uint32, seq uint64) bool { + if !s.enabled { + return false + } + if s.activatingSeqByAccount == nil { + s.activatingSeqByAccount = make(map[uint32]uint64) + } + s.activatingSeqByAccount[accountID] = seq + return true +} + +// completeActivation finalises an activation: verifies the seq still matches, +// adds the account to activeAccounts, and removes the pending entry. Returns +// false if a newer seq has superseded this one (stale activation). +func (s *lazyCatalogFilterState) completeActivation(accountID uint32, seq uint64) bool { + if !s.enabled { + return false + } + current, ok := s.activatingSeqByAccount[accountID] + if !ok || current != seq { + return false + } + delete(s.activatingSeqByAccount, accountID) + s.ensureActiveAccounts() + s.activeAccounts[accountID] = struct{}{} + s.storeActiveAccountsSnapshot() + return true +} + +func (s *lazyCatalogFilterState) abortActivation(accountID uint32, seq uint64) bool { + if !s.enabled { + return false + } + current, ok := s.activatingSeqByAccount[accountID] + if !ok || current != seq { + return false + } + delete(s.activatingSeqByAccount, accountID) + return true +} + +func (ss *Session) configureLazyCatalogSubscription(req *logtail.SubscribeRequest) error { + if !isLazyCatalogSubscribe(req) { + return nil + } + if !isLazyCatalogTableID(req.GetTable()) { + return moerr.NewNotSupportedNoCtxf( + "lazy catalog subscribe only supports mo_database/mo_tables/mo_columns, got %v", + req.GetTable(), + ) + } + + ss.mu.Lock() + defer ss.mu.Unlock() + ss.lazyCatalog.configure(req.GetInitialActiveAccounts()) + return nil +} + +func (ss *Session) snapshotLazyCatalogActiveAccountsForFilter() (*lazyCatalogAllowedAccounts, bool) { + ss.mu.RLock() + defer ss.mu.RUnlock() + + if !ss.lazyCatalog.enabled { + return nil, false + } + return ss.lazyCatalog.activeAccountsSnapshot.Load(), true +} + +func (ss *Session) lazyCatalogSubscribeAccountsForFilter( + req *logtail.SubscribeRequest, +) (*lazyCatalogAllowedAccounts, bool) { + if !isLazyCatalogSubscribe(req) { + return nil, false + } + return ss.snapshotLazyCatalogActiveAccountsForFilter() +} + +func (ss *Session) beginLazyCatalogActivation(accountID uint32, seq uint64) bool { + ss.mu.Lock() + defer ss.mu.Unlock() + return ss.lazyCatalog.beginActivation(accountID, seq) +} + +func (ss *Session) completeLazyCatalogActivation(accountID uint32, seq uint64) bool { + ss.mu.Lock() + defer ss.mu.Unlock() + return ss.lazyCatalog.completeActivation(accountID, seq) +} + +func (ss *Session) abortLazyCatalogActivation(accountID uint32, seq uint64) bool { + ss.mu.Lock() + defer ss.mu.Unlock() + return ss.lazyCatalog.abortActivation(accountID, seq) +} + +func (ss *Session) prepareLazyCatalogPublishWrapsFromIndex( + wraps []wrapLogtail, + firstLazyIndex int, +) ([]wrapLogtail, error) { + // Hot path fast path: non-catalog events should return before taking locks or + // cloning account state. + if firstLazyIndex < 0 { + return wraps, nil + } + + allowedAccounts, ok := ss.snapshotLazyCatalogActiveAccountsForFilter() + if !ok { + return wraps, nil + } + return rewriteLazyCatalogPublishWraps(wraps, firstLazyIndex, allowedAccounts) +} + +// rewriteLazyCatalogPublishWraps assumes firstLazyIndex already points at the +// first lazy-catalog table in wraps, so the caller can keep the hot-path scan +// outside the rewrite loop. +func rewriteLazyCatalogPublishWraps( + wraps []wrapLogtail, + firstLazyIndex int, + allowedAccounts *lazyCatalogAllowedAccounts, +) ([]wrapLogtail, error) { + var filtered []wrapLogtail + for idx := firstLazyIndex; idx < len(wraps); idx++ { + wrap := wraps[idx] + if !isLazyCatalogTableID(wrap.tail.Table) { + if filtered != nil { + filtered = append(filtered, wrap) + } + continue + } + + filteredTail, changed, err := filterLazyCatalogPublishRowsInTail(wrap.tail, allowedAccounts) + if err != nil { + return nil, err + } + if filtered == nil { + if !changed { + continue + } + filtered = make([]wrapLogtail, 0, len(wraps)) + filtered = append(filtered, wraps[:idx]...) + } + if isEmptyLogtail(filteredTail) { + continue + } + filtered = append(filtered, wrapLogtail{id: wrap.id, tail: filteredTail}) + } + if filtered == nil { + return wraps, nil + } + return filtered, nil +} + +// --- lazy catalog scope helpers --- + +func isLazyCatalogTableID(table *api.TableID) bool { + return table != nil && catalog.IsLazyCatalogTableID(table.TbId) +} + +func isLazyCatalogSubscribe(req *logtail.SubscribeRequest) bool { + return req != nil && req.GetLazyCatalog() +} + +func isEmptyLogtail(tail logtail.TableLogtail) bool { + return tail.CkpLocation == "" && len(tail.Commands) == 0 +} diff --git a/pkg/vm/engine/tae/logtail/service/lazy_catalog_session_test.go b/pkg/vm/engine/tae/logtail/service/lazy_catalog_session_test.go new file mode 100644 index 0000000000000..234ea56b45d0d --- /dev/null +++ b/pkg/vm/engine/tae/logtail/service/lazy_catalog_session_test.go @@ -0,0 +1,110 @@ +// Copyright 2024 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package service + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestLazyCatalogFilterState_BeginActivation(t *testing.T) { + s := &lazyCatalogFilterState{} + + ok := s.beginActivation(10, 1) + assert.False(t, ok) + + s.configure([]uint32{0}) + assert.True(t, s.enabled) + + ok = s.beginActivation(10, 1) + assert.True(t, ok) + assert.Equal(t, uint64(1), s.activatingSeqByAccount[10]) +} + +func TestLazyCatalogFilterState_CompleteActivation(t *testing.T) { + s := &lazyCatalogFilterState{} + s.configure([]uint32{0}) + + s.beginActivation(10, 1) + + ok := s.completeActivation(10, 1) + assert.True(t, ok) + + _, inActivating := s.activatingSeqByAccount[10] + assert.False(t, inActivating) + + _, inActive := s.activeAccounts[10] + assert.True(t, inActive) +} + +func TestLazyCatalogFilterState_CompleteActivation_StaleSeq(t *testing.T) { + s := &lazyCatalogFilterState{} + s.configure([]uint32{0}) + + s.beginActivation(10, 1) + s.beginActivation(10, 2) + + ok := s.completeActivation(10, 1) + assert.False(t, ok) + + ok = s.completeActivation(10, 2) + assert.True(t, ok) +} + +func TestLazyCatalogFilterState_ConfigureIdempotent(t *testing.T) { + s := &lazyCatalogFilterState{} + s.configure([]uint32{0}) + s.configure([]uint32{5}) + + _, ok := s.activeAccounts[0] + assert.True(t, ok) + _, ok = s.activeAccounts[5] + assert.True(t, ok) +} + +func TestLazyCatalogFilterState_RefreshActiveAccountsSnapshot(t *testing.T) { + s := &lazyCatalogFilterState{} + s.configure([]uint32{0, 5, 10}) + + snapshot := s.activeAccountsSnapshot.Load() + if assert.NotNil(t, snapshot) { + assert.True(t, snapshot.contains(0)) + assert.True(t, snapshot.contains(5)) + assert.True(t, snapshot.contains(10)) + } + + s.beginActivation(20, 1) + assert.True(t, s.completeActivation(20, 1)) + + snapshot = s.activeAccountsSnapshot.Load() + if assert.NotNil(t, snapshot) { + assert.True(t, snapshot.contains(20)) + } +} + +func TestLazyCatalogFilterState_AbortActivation(t *testing.T) { + s := &lazyCatalogFilterState{} + s.configure([]uint32{0}) + s.beginActivation(10, 1) + + assert.True(t, s.abortActivation(10, 1)) + _, ok := s.activatingSeqByAccount[10] + assert.False(t, ok) + + s.beginActivation(10, 2) + assert.False(t, s.abortActivation(10, 1)) + assert.True(t, s.abortActivation(10, 2)) +} diff --git a/pkg/vm/engine/tae/logtail/service/server.go b/pkg/vm/engine/tae/logtail/service/server.go index c1aee95704d26..8456100bdd20d 100644 --- a/pkg/vm/engine/tae/logtail/service/server.go +++ b/pkg/vm/engine/tae/logtail/service/server.go @@ -17,6 +17,8 @@ package service import ( "context" "fmt" + "slices" + "sync" "sync/atomic" "time" @@ -24,12 +26,14 @@ import ( "github.com/google/uuid" "go.uber.org/zap" + "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/common/log" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/common/moprobe" "github.com/matrixorigin/matrixone/pkg/common/morpc" "github.com/matrixorigin/matrixone/pkg/common/runtime" "github.com/matrixorigin/matrixone/pkg/common/stopper" + "github.com/matrixorigin/matrixone/pkg/pb/api" "github.com/matrixorigin/matrixone/pkg/pb/logtail" "github.com/matrixorigin/matrixone/pkg/pb/timestamp" v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2" @@ -120,6 +124,13 @@ type LogtailServer struct { // the second phase of collecting logtails. subTailChan chan *LogtailPhase + // activationTailChan carries phase-1 results for account activation + // requests, consumed by logtailSender for serialized phase-2 + response. + activationTailChan chan *catalogActivationPhase1 + // activationReqChan is a bounded queue feeding a fixed activation worker + // pool, so bursts do not turn directly into unbounded parked goroutines. + activationReqChan chan catalogActivation + // pullWorkerPool is used to control the parallel of the pull workers. pullWorkerPool chan struct{} @@ -134,6 +145,76 @@ type LogtailServer struct { stopper *stopper.Stopper } +// --- activation types and callback utilities --- + +// catalogActivation represents an in-flight activation request. +type catalogActivation struct { + timeout time.Duration + accountID uint32 + seq uint64 + session *Session +} + +const lazyCatalogTableCount = 3 + +// catalogActivationPhase1 carries phase-1 pull results for all three catalog +// tables. It is sent from the pull goroutine to the logtailSender for +// serialized phase-2 completion. +type catalogActivationPhase1 struct { + activation catalogActivation + tails [lazyCatalogTableCount]logtail.TableLogtail + closeCBs [lazyCatalogTableCount]func() +} + +// lazyCatalogTableIDs lists the three catalog tables in a fixed order that +// aligns with the tails/closeCBs arrays in catalogActivationPhase1. +var lazyCatalogTableIDs = [lazyCatalogTableCount]api.TableID{ + {DbId: catalog.MO_CATALOG_ID, TbId: catalog.MO_DATABASE_ID}, + {DbId: catalog.MO_CATALOG_ID, TbId: catalog.MO_TABLES_ID}, + {DbId: catalog.MO_CATALOG_ID, TbId: catalog.MO_COLUMNS_ID}, +} + +func (p *catalogActivationPhase1) closeAll() { + for i := range p.closeCBs { + if p.closeCBs[i] != nil { + p.closeCBs[i]() + p.closeCBs[i] = nil + } + } +} + +func (p *catalogActivationPhase1) takeCloseCB(idx int) func() { + cb := p.closeCBs[idx] + p.closeCBs[idx] = nil + return cb +} + +func closeCallbacks(callbacks ...func()) { + for _, cb := range callbacks { + if cb != nil { + cb() + } + } +} + +func composeCloseCallback(callbacks ...func()) func() { + var nonNil []func() + for _, cb := range callbacks { + if cb != nil { + nonNil = append(nonNil, cb) + } + } + if len(nonNil) == 0 { + return nil + } + if len(nonNil) == 1 { + return nonNil[0] + } + return func() { + closeCallbacks(nonNil...) + } +} + func defaultRPCServerFactory( name string, address string, @@ -175,16 +256,18 @@ func NewLogtailServer( rpcServerFactory func(string, string, *LogtailServer, ...morpc.ServerOption) (morpc.RPCServer, error), opts ...ServerOption, ) (*LogtailServer, error) { s := &LogtailServer{ - rt: rt, - logger: rt.Logger(), - cfg: cfg, - ssmgr: NewSessionManager(), - waterline: NewWaterliner(), - errChan: make(chan sessionError, 1), - subReqChan: make(chan subscription, 100), - subTailChan: make(chan *LogtailPhase, 300), - pullWorkerPool: make(chan struct{}, cfg.PullWorkerPoolSize), - logtailer: logtailer, + rt: rt, + logger: rt.Logger(), + cfg: cfg, + ssmgr: NewSessionManager(), + waterline: NewWaterliner(), + errChan: make(chan sessionError, 1), + subReqChan: make(chan subscription, 100), + subTailChan: make(chan *LogtailPhase, 300), + activationTailChan: make(chan *catalogActivationPhase1, 64), + activationReqChan: make(chan catalogActivation, activationWorkerCount(cfg)), + pullWorkerPool: make(chan struct{}, cfg.PullWorkerPoolSize), + logtailer: logtailer, } for _, opt := range opts { @@ -275,6 +358,10 @@ func (s *LogtailServer) onMessage( return s.onUnsubscription(ctx, stream, req) } + if req := msg.GetActivateAccountForCatalog(); req != nil { + return s.onActivateAccountForCatalog(ctx, stream, req) + } + return moerr.NewInvalidArg(ctx, "request", msg) } @@ -298,6 +385,10 @@ func (s *LogtailServer) onSubscription( logger.Info("repeated sub request", zap.String("table ID", string(tableID))) return nil } + if err := session.configureLazyCatalogSubscription(req); err != nil { + session.Unregister(tableID) + return err + } sub := subscription{ timeout: ContextTimeout(sendCtx, s.cfg.ResponseSendTimeout), @@ -346,6 +437,43 @@ func (s *LogtailServer) onUnsubscription( return session.SendUnsubscriptionResponse(sendCtx, *req.Table) } +func (s *LogtailServer) onActivateAccountForCatalog( + ctx context.Context, stream morpcStream, req *logtail.ActivateAccountForCatalogRequest, +) error { + logger := s.logger + session := s.ssmgr.GetSession( + s.rootCtx, logger, s.pool.responses, s, stream, + s.cfg.ResponseSendTimeout, + s.cfg.RPCStreamPoisonTime, + s.cfg.LogtailCollectInterval, + ) + + accountID := req.GetAccountId() + seq := req.GetSeq() + + if !session.beginLazyCatalogActivation(accountID, seq) { + return moerr.NewNotSupported(ctx, "activate account for catalog on non-lazy session") + } + + act := catalogActivation{ + timeout: ContextTimeout(ctx, s.cfg.ResponseSendTimeout), + accountID: accountID, + seq: seq, + session: session, + } + + select { + case <-s.rootCtx.Done(): + session.abortLazyCatalogActivation(accountID, seq) + return moerr.AttachCause(s.rootCtx, s.rootCtx.Err()) + case <-ctx.Done(): + session.abortLazyCatalogActivation(accountID, seq) + return moerr.AttachCause(ctx, ctx.Err()) + case s.activationReqChan <- act: + return nil + } +} + // NotifySessionError notifies session manager with session error. func (s *LogtailServer) NotifySessionError( session *Session, err error, @@ -448,6 +576,127 @@ func (s *LogtailServer) pullLogtailsPhase1(ctx context.Context, sub subscription } } +func activationWorkerCount(cfg *options.LogtailServerCfg) int { + if cfg == nil || cfg.PullWorkerPoolSize <= 0 { + return 1 + } + return int(cfg.PullWorkerPoolSize) +} + +func (s *LogtailServer) activationPullWorker(ctx context.Context) { + for { + select { + case <-ctx.Done(): + s.logger.Error("stop activation pull worker", zap.Error(ctx.Err())) + return + case act, ok := <-s.activationReqChan: + if !ok { + s.logger.Info("activation request channel closed") + return + } + s.pullActivationPhase1(ctx, act) + } + } +} + +// pullActivationPhase1 concurrently pulls historical row-level delta for all +// three catalog tables and sends the combined result to activationTailChan. +func (s *LogtailServer) pullActivationPhase1(ctx context.Context, act catalogActivation) { + s.pullWorkerPool <- struct{}{} + defer func() { <-s.pullWorkerPool }() + + s.logger.Info("activation phase1 start", + zap.Uint32("account-id", act.accountID), + zap.Uint64("seq", act.seq), + ) + + waterline := s.waterline.Waterline() + allowedAccounts := newLazyCatalogAllowedAccounts(act.accountID) + + var result catalogActivationPhase1 + result.activation = act + + enqueued := false + defer func() { + if !enqueued { + result.closeAll() + act.session.abortLazyCatalogActivation(act.accountID, act.seq) + } + }() + + var wg sync.WaitGroup + var mu sync.Mutex + var firstErr error + + for i, table := range lazyCatalogTableIDs { + wg.Add(1) + go func(idx int, tbl api.TableID) { + defer wg.Done() + tail, closeCB, err := s.pullTableLogtail( + ctx, + tbl, + timestamp.Timestamp{}, + waterline, + allowedAccounts, + ) + mu.Lock() + defer mu.Unlock() + if err != nil { + closeCallbacks(closeCB) + if firstErr == nil { + firstErr = err + } + return + } + result.tails[idx] = tail + result.closeCBs[idx] = closeCB + }(i, table) + } + wg.Wait() + + if firstErr != nil { + s.logger.Error("activation phase1 failed", + zap.Uint32("account-id", act.accountID), + zap.Uint64("seq", act.seq), + zap.Error(firstErr), + ) + return + } + + for { + select { + case <-ctx.Done(): + s.logger.Error("context done during activation phase1 enqueue", zap.Error(ctx.Err())) + return + case s.activationTailChan <- &result: + enqueued = true + return + default: + s.logger.Warn("activation tail chan full, retrying") + time.Sleep(time.Second) + } + } +} + +func (s *LogtailServer) pullTableLogtail( + ctx context.Context, + table api.TableID, + from, to timestamp.Timestamp, + allowedAccounts *lazyCatalogAllowedAccounts, +) (logtail.TableLogtail, func(), error) { + tail, closeCB, err := s.logtailer.TableLogtail(ctx, table, from, to) + if err != nil { + return logtail.TableLogtail{}, closeCB, err + } + + filtered, filterCloseCB, err := filterLazyCatalogPulledTail(tail, allowedAccounts) + if err != nil { + closeCallbacks(closeCB, filterCloseCB) + return logtail.TableLogtail{}, nil, err + } + return filtered, composeCloseCallback(closeCB, filterCloseCB), nil +} + // logtailSender sends total or incremental logtail. func (s *LogtailServer) logtailSender(ctx context.Context) { select { @@ -496,6 +745,13 @@ func (s *LogtailServer) logtailSender(ctx context.Context) { s.sendSubscription(ctx, tailPhase1, tailPhase2) } + case actPhase1, ok := <-s.activationTailChan: + if !ok { + s.logger.Info("activation channel closed") + return + } + s.sendActivation(ctx, actPhase1) + case e, ok := <-s.event.C: if !ok { s.logger.Info("publishment channel closed") @@ -506,6 +762,93 @@ func (s *LogtailServer) logtailSender(ctx context.Context) { } } +// sendActivation completes phase-2 for each catalog table, filters rows for +// the target account, builds and sends the ActivateAccountForCatalogResponse, +// and only then adds the account to activeAccounts. +func (s *LogtailServer) sendActivation(ctx context.Context, p1 *catalogActivationPhase1) { + act := p1.activation + sendCtx, cancel := context.WithTimeoutCause(ctx, act.timeout, moerr.CauseSendSubscription) + defer cancel() + + targetTS := s.waterline.Waterline() + allowedAccounts := newLazyCatalogAllowedAccounts(act.accountID) + + var responseTails []logtail.TableLogtail + var allCloseCBs []func() + sent := false + + defer func() { + if !sent { + p1.closeAll() + closeCallbacks(allCloseCBs...) + act.session.abortLazyCatalogActivation(act.accountID, act.seq) + } + }() + + for i, table := range lazyCatalogTableIDs { + phase1Ts := timestamp.Timestamp{} + if p1.tails[i].Ts != nil { + phase1Ts = *p1.tails[i].Ts + } + + phase2Tail, closeCB, err := s.pullTableLogtail(sendCtx, table, phase1Ts, targetTS, allowedAccounts) + if err != nil { + closeCallbacks(closeCB) + s.logger.Error("activation phase2 failed", + zap.Uint32("account-id", act.accountID), + zap.Uint64("seq", act.seq), + zap.Error(err), + ) + return + } + merged, mergedCloseCB := newLogtailMerger( + &LogtailPhase{tail: p1.tails[i], closeCB: p1.takeCloseCB(i)}, + &LogtailPhase{tail: phase2Tail, closeCB: closeCB}, + ).Merge() + allCloseCBs = append(allCloseCBs, mergedCloseCB) + + if !isEmptyLogtail(merged) { + responseTails = append(responseTails, merged) + } + } + + // Transfer cleanup ownership to the response path. + responseCB := composeCloseCallback(allCloseCBs...) + allCloseCBs = nil + + resp := logtail.ActivateAccountForCatalogResponse{ + AccountId: act.accountID, + Seq: act.seq, + TargetTs: &targetTS, + Tails: responseTails, + } + if err := act.session.SendActivateAccountForCatalogResponse(sendCtx, resp, responseCB); err != nil { + // SendResponse.Release already called responseCB for cleanup. + s.logger.Error("fail to send activation response", + zap.Uint32("account-id", act.accountID), + zap.Uint64("seq", act.seq), + zap.Error(err), + ) + return + } + sent = true + + // Only after the response has successfully entered the session's FIFO + // sendChan do we promote the account to active for steady-state push. + if !act.session.completeLazyCatalogActivation(act.accountID, act.seq) { + s.logger.Warn("activation seq superseded, account not promoted", + zap.Uint32("account-id", act.accountID), + zap.Uint64("seq", act.seq), + ) + } + + s.logger.Info("activation complete", + zap.Uint32("account-id", act.accountID), + zap.Uint64("seq", act.seq), + zap.String("target-ts", targetTS.String()), + ) +} + func (s *LogtailServer) getSubLogtailPhase( ctx context.Context, sub subscription, from, to timestamp.Timestamp, ) (*LogtailPhase, error) { @@ -520,18 +863,17 @@ func (s *LogtailServer) getSubLogtailPhase( }() table := *sub.req.Table + allowedAccounts, _ := sub.session.lazyCatalogSubscribeAccountsForFilter(sub.req) var tail logtail.TableLogtail var closeCB func() moprobe.WithRegion(ctx, moprobe.SubscriptionPullLogTail, func() { - tail, closeCB, subErr = s.logtailer.TableLogtail(sendCtx, table, from, to) + tail, closeCB, subErr = s.pullTableLogtail(sendCtx, table, from, to, allowedAccounts) subErr = moerr.AttachCause(sendCtx, subErr) }) if subErr != nil { // if error occurs, just send the error immediately. - if closeCB != nil { - closeCB() - } + closeCallbacks(closeCB) s.logger.Error("fail to fetch table total logtail", zap.Error(subErr), zap.Any("table", table)) subErrCode, ok := moerr.GetMoErrCode(subErr) @@ -581,7 +923,7 @@ func (s *LogtailServer) publishEvent(ctx context.Context, e event) { wraps := make([]wrapLogtail, 0, len(e.logtails)) for _, tail := range e.logtails { // skip empty logtail - if tail.CkpLocation == "" && len(tail.Commands) == 0 { + if isEmptyLogtail(tail) { continue } wraps = append(wraps, wrapLogtail{ @@ -606,9 +948,28 @@ func (s *LogtailServer) publishEvent(ctx context.Context, e event) { } } } + firstLazyCatalogIndex := slices.IndexFunc(wraps, func(w wrapLogtail) bool { + return catalog.IsLazyCatalogTableID(w.tail.Table.TbId) + }) refcount.Add(int32(len(sessions))) for _, session := range sessions { - if err := session.Publish(ctx, from, to, closeCB, wraps...); err != nil { + publishWraps := wraps + if firstLazyCatalogIndex >= 0 { + // Event-level fast path: if this batch does not contain the three lazy + // catalog tables, no session should even enter the lazy publish helper. + var err error + publishWraps, err = session.prepareLazyCatalogPublishWrapsFromIndex(wraps, firstLazyCatalogIndex) + if err != nil { + err = moerr.AttachCause(ctx, err) + closeCB() + s.NotifySessionError(session, err) + s.logger.Error("fail to filter catalog incremental logtail", zap.Error(err), + zap.Uint64("stream-id", session.stream.streamID), zap.String("remote", session.stream.remote), + ) + continue + } + } + if err := session.Publish(ctx, from, to, closeCB, publishWraps...); err != nil { s.logger.Error("fail to publish incremental logtail", zap.Error(err), zap.Uint64("stream-id", session.stream.streamID), zap.String("remote", session.stream.remote), ) @@ -670,6 +1031,14 @@ func (s *LogtailServer) Start() error { return err } + for i := 0; i < activationWorkerCount(s.cfg); i++ { + name := fmt.Sprintf("activation pull worker %d", i) + if err := s.stopper.RunNamedTask(name, s.activationPullWorker); err != nil { + s.logger.Error("fail to start activation pull worker", zap.Int("worker", i), zap.Error(err)) + return err + } + } + if err := s.stopper.RunNamedTask("logtail sender", s.logtailSender); err != nil { s.logger.Error("fail to start logtail sender", zap.Error(err)) return err diff --git a/pkg/vm/engine/tae/logtail/service/server_test.go b/pkg/vm/engine/tae/logtail/service/server_test.go index 411ca905baecb..f731852ea3989 100644 --- a/pkg/vm/engine/tae/logtail/service/server_test.go +++ b/pkg/vm/engine/tae/logtail/service/server_test.go @@ -20,6 +20,7 @@ import ( "testing" "time" + "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/common/morpc" "github.com/matrixorigin/matrixone/pkg/common/mpool" "github.com/matrixorigin/matrixone/pkg/common/runtime" @@ -126,6 +127,10 @@ type logtailer struct { tables []api.TableID } +type tableLogtailer struct { + tails map[uint64]logtail.TableLogtail +} + func mockLocktailer(tables ...api.TableID) taelogtail.Logtailer { return &logtailer{ tables: tables, @@ -160,6 +165,36 @@ func (m *logtailer) Now() (timestamp.Timestamp, timestamp.Timestamp) { panic("not implemented") } +func (m *tableLogtailer) RangeLogtail( + ctx context.Context, from, to timestamp.Timestamp, +) ([]logtail.TableLogtail, []func(), error) { + return nil, nil, nil +} + +func (m *tableLogtailer) RegisterCallback(cb func(from, to timestamp.Timestamp, closeCB func(), tails ...logtail.TableLogtail) error) { +} + +func (m *tableLogtailer) TableLogtail( + ctx context.Context, table api.TableID, from, to timestamp.Timestamp, +) (logtail.TableLogtail, func(), error) { + if tail, ok := m.tails[table.TbId]; ok { + tailCopy := tail + if tailCopy.Table == nil { + tailCopy.Table = &table + } + if tailCopy.Ts == nil { + ts := to + tailCopy.Ts = &ts + } + return tailCopy, func() {}, nil + } + return logtail.TableLogtail{Table: &table, Ts: &to}, func() {}, nil +} + +func (m *tableLogtailer) Now() (timestamp.Timestamp, timestamp.Timestamp) { + panic("not implemented") +} + func mockRuntime() runtime.Runtime { return runtime.NewRuntime( metadata.ServiceType_TN, @@ -235,3 +270,159 @@ func startLogtailServer( } return stop } + +func TestOnActivateAccountForCatalogQueuesWork(t *testing.T) { + server, stream, session := newActivationQueueTestServer(t, 1) + + err := server.onActivateAccountForCatalog(context.Background(), stream, &logtail.ActivateAccountForCatalogRequest{ + AccountId: 10, + Seq: 7, + }) + require.NoError(t, err) + + select { + case act := <-server.activationReqChan: + require.Equal(t, uint32(10), act.accountID) + require.Equal(t, uint64(7), act.seq) + require.Same(t, session, act.session) + case <-time.After(time.Second): + t.Fatal("activation request was not queued") + } +} + +func TestOnActivateAccountForCatalogQueueFailureCleansSessionState(t *testing.T) { + server, stream, session := newActivationQueueTestServer(t, 0) + + ctx, cancel := context.WithCancel(context.Background()) + cancel() + + err := server.onActivateAccountForCatalog(ctx, stream, &logtail.ActivateAccountForCatalogRequest{ + AccountId: 10, + Seq: 7, + }) + require.Error(t, err) + _, ok := session.lazyCatalog.activatingSeqByAccount[10] + require.False(t, ok) +} + +func TestGetSubLogtailPhaseFiltersLazyCatalogRowsEarly(t *testing.T) { + table := api.TableID{DbId: catalog.MO_CATALOG_ID, TbId: catalog.MO_COLUMNS_ID} + server := &LogtailServer{ + logger: mockMOLogger(), + logtailer: &tableLogtailer{ + tails: map[uint64]logtail.TableLogtail{ + catalog.MO_COLUMNS_ID: { + Table: &table, + Ts: ×tamp.Timestamp{PhysicalTime: 10}, + CkpLocation: "ckp:phase1", + Commands: []api.Entry{ + mustCatalogColumnInsertEntry(t, []uint32{0, 10, 20}), + }, + }, + }, + }, + } + + session := newCatalogTestSession(t) + require.NoError(t, session.configureLazyCatalogSubscription(&logtail.SubscribeRequest{ + Table: &table, + LazyCatalog: true, + InitialActiveAccounts: []uint32{10}, + })) + + phase, err := server.getSubLogtailPhase(context.Background(), subscription{ + timeout: time.Second, + tableID: MarshalTableID(&table), + req: &logtail.SubscribeRequest{ + Table: &table, + LazyCatalog: true, + }, + session: session, + }, timestamp.Timestamp{}, timestamp.Timestamp{PhysicalTime: 10}) + require.NoError(t, err) + t.Cleanup(func() { + if phase.closeCB != nil { + phase.closeCB() + } + }) + require.Empty(t, phase.tail.CkpLocation) + require.Len(t, phase.tail.Commands, 1) + require.Equal(t, []uint32{10}, mustAccountIDsFromEntry(t, phase.tail.Commands[0])) +} + +func TestPullActivationPhase1FiltersRowsBeforeEnqueue(t *testing.T) { + table := api.TableID{DbId: catalog.MO_CATALOG_ID, TbId: catalog.MO_COLUMNS_ID} + server := &LogtailServer{ + logger: mockMOLogger(), + logtailer: &tableLogtailer{tails: map[uint64]logtail.TableLogtail{ + catalog.MO_COLUMNS_ID: { + Table: &table, + Ts: ×tamp.Timestamp{PhysicalTime: 10}, + CkpLocation: "ckp:phase1", + Commands: []api.Entry{ + mustCatalogColumnInsertEntry(t, []uint32{0, 10, 20}), + }, + }, + }}, + waterline: NewWaterliner(), + activationTailChan: make(chan *catalogActivationPhase1, 1), + pullWorkerPool: make(chan struct{}, 1), + } + server.waterline.Advance(timestamp.Timestamp{PhysicalTime: 10}) + + session := newCatalogTestSession(t) + require.NoError(t, session.configureLazyCatalogSubscription(&logtail.SubscribeRequest{ + Table: &api.TableID{DbId: catalog.MO_CATALOG_ID, TbId: catalog.MO_TABLES_ID}, + LazyCatalog: true, + InitialActiveAccounts: []uint32{0}, + })) + + server.pullActivationPhase1(context.Background(), catalogActivation{ + timeout: time.Second, + accountID: 10, + seq: 7, + session: session, + }) + + select { + case phase := <-server.activationTailChan: + t.Cleanup(phase.closeAll) + require.Empty(t, phase.tails[2].CkpLocation) + require.Len(t, phase.tails[2].Commands, 1) + require.Equal(t, []uint32{10}, mustAccountIDsFromEntry(t, phase.tails[2].Commands[0])) + case <-time.After(time.Second): + t.Fatal("activation phase1 was not enqueued") + } +} + +func newActivationQueueTestServer(t *testing.T, queueCap int) (*LogtailServer, morpcStream, *Session) { + t.Helper() + + cfg := options.NewDefaultLogtailServerCfg() + server := &LogtailServer{ + logger: mockMOLogger(), + cfg: cfg, + ssmgr: NewSessionManager(), + rootCtx: context.Background(), + activationReqChan: make(chan catalogActivation, queueCap), + } + server.pool.responses = NewLogtailResponsePool() + + stream := mockMorpcStream(&normalStream{}, 1, 1024) + session := server.ssmgr.GetSession( + server.rootCtx, + server.logger, + server.pool.responses, + server, + stream, + server.cfg.ResponseSendTimeout, + server.cfg.RPCStreamPoisonTime, + server.cfg.LogtailCollectInterval, + ) + require.NoError(t, session.configureLazyCatalogSubscription(&logtail.SubscribeRequest{ + Table: &api.TableID{DbId: catalog.MO_CATALOG_ID, TbId: catalog.MO_TABLES_ID}, + LazyCatalog: true, + InitialActiveAccounts: []uint32{0}, + })) + return server, stream, session +} diff --git a/pkg/vm/engine/tae/logtail/service/session.go b/pkg/vm/engine/tae/logtail/service/session.go index 9b04fa5840afa..c86cbb8543631 100644 --- a/pkg/vm/engine/tae/logtail/service/session.go +++ b/pkg/vm/engine/tae/logtail/service/session.go @@ -232,6 +232,8 @@ type Session struct { mu sync.RWMutex tables map[TableID]TableState + lazyCatalog lazyCatalogFilterState + heartbeatInterval time.Duration heartbeatTimer *time.Timer exactFrom timestamp.Timestamp @@ -428,6 +430,8 @@ func (ss *Session) ListSubscribedTable() []TableID { // FilterLogtail selects logtail for expected tables. func (ss *Session) FilterLogtail(tails ...wrapLogtail) []logtail.TableLogtail { + // This is the original generic subscription filter used by normal publish flow. + // It intentionally does not know anything about lazy catalog account filtering. ss.mu.RLock() defer ss.mu.RUnlock() @@ -551,6 +555,23 @@ func (ss *Session) SendUnsubscriptionResponse( return err } +func (ss *Session) SendActivateAccountForCatalogResponse( + sendCtx context.Context, + activate logtail.ActivateAccountForCatalogResponse, + closeCB func(), +) error { + ss.logger.Info( + "send activate account for catalog response", + zap.Uint32("account-id", activate.AccountId), + zap.Uint64("seq", activate.Seq), + ) + + resp := ss.responses.Acquire() + resp.closeCB = closeCB + resp.Response = newActivateAccountForCatalogResponse(activate) + return ss.SendResponse(sendCtx, resp) +} + // SendUpdateResponse sends publishment response. func (ss *Session) SendUpdateResponse( sendCtx context.Context, from, to timestamp.Timestamp, closeCB func(), tails ...logtail.TableLogtail, @@ -686,6 +707,14 @@ func newUpdateResponse( } } +func newActivateAccountForCatalogResponse( + activate logtail.ActivateAccountForCatalogResponse, +) *logtail.LogtailResponse_ActivateAccountForCatalogResponse { + return &logtail.LogtailResponse_ActivateAccountForCatalogResponse{ + ActivateAccountForCatalogResponse: &activate, + } +} + // newSubscritpionResponse constructs response for subscription. func newSubscritpionResponse( tail logtail.TableLogtail, diff --git a/pkg/vm/engine/tae/logtail/service/session_test.go b/pkg/vm/engine/tae/logtail/service/session_test.go index b939c45ae29b0..2019143d13f01 100644 --- a/pkg/vm/engine/tae/logtail/service/session_test.go +++ b/pkg/vm/engine/tae/logtail/service/session_test.go @@ -213,6 +213,7 @@ func TestSession(t *testing.T) { // promote state for table A ss.AdvanceState(idA) require.Equal(t, 1, len(ss.ListSubscribedTable())) + var err error // promote state for non-exist table ss.AdvanceState(TableID("non-exist")) require.Equal(t, 1, len(ss.ListSubscribedTable())) @@ -235,7 +236,7 @@ func TestSession(t *testing.T) { require.Equal(t, 2, len(qualified)) /* ---- 5. send error response ---- */ - err := ss.SendErrorResponse( + err = ss.SendErrorResponse( context.Background(), tableA, moerr.ErrInternal, @@ -275,6 +276,25 @@ func TestSession(t *testing.T) { require.NoError(t, err) } + /* ---- 8.5 send activate-account response ---- */ + { + target := mockTimestamp(4, 0) + err = ss.SendActivateAccountForCatalogResponse( + context.Background(), + logtail.ActivateAccountForCatalogResponse{ + AccountId: 1, + Seq: 10, + TargetTs: &target, + Tails: []logtail.TableLogtail{ + mockLogtail(tableA, target), + }, + }, + nil, + ) + require.NoError(t, err) + require.Equal(t, 0, ss.Active()) + } + /* ---- 9. publish update response ---- */ err = ss.Publish( context.Background(), diff --git a/pkg/vm/engine/tae/logtail/txn_handle.go b/pkg/vm/engine/tae/logtail/txn_handle.go index b793d16d17c78..e122015a784d8 100644 --- a/pkg/vm/engine/tae/logtail/txn_handle.go +++ b/pkg/vm/engine/tae/logtail/txn_handle.go @@ -22,6 +22,7 @@ import ( "github.com/RoaringBitmap/roaring/v2" pkgcatalog "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/container/vector" "github.com/matrixorigin/matrixone/pkg/logutil" "github.com/matrixorigin/matrixone/pkg/objectio" "github.com/matrixorigin/matrixone/pkg/pb/api" @@ -241,6 +242,7 @@ func (b *TxnLogtailRespBuilder) buildLogtailEntry(tid, dbid uint64, tableName, d DatabaseName: dbName, Bat: apiBat, } + b.annotateLazyCatalogEntryAccountSummary(entry, bat) ts := b.txn.GetPrepareTS().ToTimestamp() tableID := &api.TableID{ AccId: b.currentAccID, @@ -269,6 +271,103 @@ func (b *TxnLogtailRespBuilder) buildLogtailEntry(tid, dbid uint64, tableName, d b.currentLogtail.Commands = append(b.currentLogtail.Commands, *entry) } +func (b *TxnLogtailRespBuilder) annotateLazyCatalogEntryAccountSummary( + entry *api.Entry, + bat *containers.Batch, +) { + if entry == nil || bat == nil || !pkgcatalog.IsLazyCatalogTableID(entry.GetTableId()) { + return + } + + switch entry.GetEntryType() { + case api.Entry_Insert, api.Entry_Update: + accountID, ok := batchSingleAccountID(bat, pkgcatalog.SystemDBAttr_AccID) + if !ok { + return + } + pkgcatalog.SetLazyCatalogEntryAccountSummary(entry, accountID) + case api.Entry_Delete: + accountID, ok, err := batchSingleCPKeyAccountID(bat) + if err != nil { + panic(err) + } + if !ok { + return + } + pkgcatalog.SetLazyCatalogEntryAccountSummary(entry, accountID) + } +} + +func batchSingleAccountID(bat *containers.Batch, attr string) (uint32, bool) { + attrIdx, ok := bat.Nameidx[attr] + if !ok || bat.Length() == 0 { + return 0, false + } + + accounts := vector.MustFixedColWithTypeCheck[uint32](bat.Vecs[attrIdx].GetDownstreamVector()) + if len(accounts) == 0 { + return 0, false + } + + first := accounts[0] + for row := 1; row < len(accounts); row++ { + if accounts[row] != first { + // Mixed-account batch (e.g., from restore). Don't set + // entry-level summary; TN filter will fall back to row-level. + return 0, false + } + } + return first, true +} + +func batchSingleCPKeyAccountID(bat *containers.Batch) (uint32, bool, error) { + attrIdx, ok := bat.Nameidx[pkgcatalog.CPrimaryKeyColName] + if !ok || bat.Length() == 0 { + return 0, false, nil + } + + var ( + first uint32 + initialized bool + mixed bool + ) + err := containers.ForeachWindowBytes( + bat.Vecs[attrIdx].GetDownstreamVector(), + 0, + bat.Length(), + func(cpkey []byte, isNull bool, row int) error { + accountID, err := pkgcatalog.DecodeLazyCatalogAccountFromCPKey(cpkey) + if err != nil { + return err + } + if !initialized { + first = accountID + initialized = true + return nil + } + if accountID != first { + // Mixed-account delete batch (e.g., from restore). + mixed = true + return fmt.Errorf("break") + } + return nil + }, + nil, + ) + if mixed { + // Don't set entry-level summary; TN filter will fall back to + // row-level filtering for this entry. + return 0, false, nil + } + if err != nil { + return 0, false, err + } + if !initialized { + return 0, false, nil + } + return first, true, nil +} + func (b *TxnLogtailRespBuilder) rotateTable(aid uint32, dbName, tableName string, dbid, tid uint64, pkSeqnum uint16) { b.buildLogtailEntry(b.currTableID, b.currDBID, b.currTableName, b.currDBName, dataObjectInfoBatch, api.Entry_DataObject) diff --git a/pkg/vm/engine/tae/logtail/txn_handle_test.go b/pkg/vm/engine/tae/logtail/txn_handle_test.go new file mode 100644 index 0000000000000..9609b3e80a158 --- /dev/null +++ b/pkg/vm/engine/tae/logtail/txn_handle_test.go @@ -0,0 +1,51 @@ +// Copyright 2024 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package logtail + +import ( + "testing" + + "github.com/stretchr/testify/require" + + catalogpkg "github.com/matrixorigin/matrixone/pkg/catalog" + "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/pb/api" + "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/containers" +) + +func TestAnnotateLazyCatalogEntryAccountSummaryUsesEntryAccountNotCurrentAccount(t *testing.T) { + bat := containers.BuildBatch( + []string{catalogpkg.SystemDBAttr_AccID}, + []types.Type{types.T_uint32.ToType()}, + containers.Options{}, + ) + t.Cleanup(func() { bat.Close() }) + bat.Vecs[0].Append(uint32(10001), false) + + entry := api.Entry{ + EntryType: api.Entry_Insert, + TableId: catalogpkg.MO_DATABASE_ID, + DatabaseId: catalogpkg.MO_CATALOG_ID, + } + + builder := &TxnLogtailRespBuilder{currentAccID: 0} + require.NotPanics(t, func() { + builder.annotateLazyCatalogEntryAccountSummary(&entry, bat) + }) + + accountID, ok := catalogpkg.LazyCatalogEntryAccountSummary(entry) + require.True(t, ok) + require.Equal(t, uint32(10001), accountID) +} diff --git a/pkg/vm/engine/types.go b/pkg/vm/engine/types.go index a16981e1c4a4d..fb83456fc1fc1 100644 --- a/pkg/vm/engine/types.go +++ b/pkg/vm/engine/types.go @@ -1202,6 +1202,14 @@ type Engine interface { LatestLogtailAppliedTime() timestamp.Timestamp } +// TenantCatalogActivator is an optional interface for engines that support +// lazy per-account catalog loading. If the engine implements this interface, +// the frontend should call ActivateTenantCatalog during authentication before +// the first tenant-context SQL. +type TenantCatalogActivator interface { + ActivateTenantCatalog(ctx context.Context, accountID uint32) error +} + type VectorPool interface { PutBatch(bat *batch.Batch) GetVector(typ types.Type) *vector.Vector diff --git a/proto/api.proto b/proto/api.proto index e929c195239d9..db520bf97a9e5 100644 --- a/proto/api.proto +++ b/proto/api.proto @@ -244,6 +244,10 @@ message Entry { Batch bat = 7; // whether TN do the PK uniqueness check against txn's workspace or not. int32 pk_check_by_tn = 8; + // Presence-safe account summary for lazy-catalog push entries. Account 0 is + // real, so the boolean is required to distinguish "absent" from "sys". + uint32 lazy_catalog_account_id = 9; + bool has_lazy_catalog_account_id = 10; }; // There are two kinds of checkpoint: delta checkpoint and base checkpoint, diff --git a/proto/logtail.proto b/proto/logtail.proto index eab3c465e45e4..cc3043737b5cc 100644 --- a/proto/logtail.proto +++ b/proto/logtail.proto @@ -28,6 +28,8 @@ option (gogoproto.protosizer_all) = true; // SubscribeRequest is the request for subscription. message SubscribeRequest { api.TableID table = 1; + bool lazy_catalog = 2; + repeated uint32 initial_active_accounts = 3; } // UnsubscribeRequest is the request for unsubscription. @@ -35,6 +37,18 @@ message UnsubscribeRequest { api.TableID table = 1; } +message ActivateAccountForCatalogRequest { + uint32 account_id = 1; + uint64 seq = 2; +} + +message ActivateAccountForCatalogResponse { + uint32 account_id = 1; + uint64 seq = 2; + timestamp.Timestamp target_ts = 3; + repeated TableLogtail tails = 4 [(gogoproto.nullable) = false]; +} + // TableLogtail describes total or additional logtail for a table. message TableLogtail { string ckp_location = 1; @@ -84,8 +98,9 @@ message UnSubscribeResponse { message LogtailRequest { uint64 request_id = 1; oneof request { - SubscribeRequest subscribe_table = 2; - UnsubscribeRequest unsubscribe_table = 3; + SubscribeRequest subscribe_table = 2; + UnsubscribeRequest unsubscribe_table = 3; + ActivateAccountForCatalogRequest activate_account_for_catalog = 4; } }; @@ -93,10 +108,11 @@ message LogtailRequest { message LogtailResponse { uint64 response_id = 1; oneof response { - SubscribeResponse subscribe_response = 2; - UnSubscribeResponse unsubscribe_response = 3; - UpdateResponse update_response = 4; - ErrorResponse error = 5; + SubscribeResponse subscribe_response = 2; + UnSubscribeResponse unsubscribe_response = 3; + UpdateResponse update_response = 4; + ErrorResponse error = 5; + ActivateAccountForCatalogResponse activate_account_for_catalog_response = 6; } }; From b4348409acc48a2a7dcff28f18097b9687525437 Mon Sep 17 00:00:00 2001 From: aptend Date: Sun, 29 Mar 2026 16:01:51 +0800 Subject: [PATCH 02/11] fix: show accounts visibility and restore activation for lazy catalog - show_account.go: Change INNER JOIN to LEFT JOIN from mo_account to db_tbl_counts so freshly-created but not-yet-activated accounts still appear in 'show accounts'. TN-side lazy catalog publish filtering strips mo_tables/mo_database entries for non-activated accounts, so the INNER JOIN was hiding them. COALESCE handles NULL counts. - snapshot.go: Add activateAccountCatalogIfNeeded() before the first internal SQL in restore flows (doRestoreSnapshot, restoreAccountUsingClusterSnapshotToNew) so background SQL sessions that bypass AuthenticateUser/ActivateTenantCatalog can see catalog entries for the target account. Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com> --- pkg/frontend/show_account.go | 8 ++++---- pkg/frontend/snapshot.go | 25 +++++++++++++++++++++++++ 2 files changed, 29 insertions(+), 4 deletions(-) diff --git a/pkg/frontend/show_account.go b/pkg/frontend/show_account.go index 2102b09918b72..c2807c8e6d528 100644 --- a/pkg/frontend/show_account.go +++ b/pkg/frontend/show_account.go @@ -84,19 +84,19 @@ const ( " ma.created_time," + " ma.status," + " ma.suspended_time," + - " db_counts.db_count," + - " tbl_counts.tbl_count," + + " COALESCE(db_counts.db_count, 0) AS db_count," + + " COALESCE(tbl_counts.tbl_count, 0) AS tbl_count," + " CAST(0 AS DOUBLE) AS size," + " CAST(0 AS DOUBLE) AS snapshot_size," + " ma.comments" + " %s" + // possible placeholder for object count " FROM" + " mo_catalog.mo_account AS ma " + - " JOIN" + + " LEFT JOIN" + " db_counts " + " ON " + " ma.account_id = db_counts.account_id " + - " JOIN" + + " LEFT JOIN" + " tbl_counts " + " ON " + " ma.account_id = tbl_counts.account_id " + diff --git a/pkg/frontend/snapshot.go b/pkg/frontend/snapshot.go index 63915cb965967..02b2cb9e1de22 100644 --- a/pkg/frontend/snapshot.go +++ b/pkg/frontend/snapshot.go @@ -38,6 +38,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/sql/parsers/tree" "github.com/matrixorigin/matrixone/pkg/sql/plan" "github.com/matrixorigin/matrixone/pkg/util/trace/impl/motrace/statistic" + "github.com/matrixorigin/matrixone/pkg/vm/engine" ) type tableType string @@ -554,6 +555,20 @@ func doDropSnapshot(ctx context.Context, ses *Session, stmt *tree.DropSnapShot) return err } +// activateAccountCatalogIfNeeded ensures the lazy catalog for the given +// account is activated before restore runs internal SQL as that account. +// Without this, logtail entries for the target account's catalog tables +// would be delayed and invisible to background SQL sessions. +func activateAccountCatalogIfNeeded(ctx context.Context, ses *Session, accountID uint32) error { + if accountID == 0 { + return nil // sys account is always activated + } + if activator, ok := ses.GetTxnHandler().GetStorage().(engine.TenantCatalogActivator); ok { + return activator.ActivateTenantCatalog(ctx, accountID) + } + return nil +} + func doRestoreSnapshot(ctx context.Context, ses *Session, stmt *tree.RestoreSnapShot) (stats statistic.StatsArray, err error) { bh := ses.GetBackgroundExec(ctx) bh.SetRestore(true) @@ -633,6 +648,11 @@ func doRestoreSnapshot(ctx context.Context, ses *Session, stmt *tree.RestoreSnap return } + // activate the target account's catalog so internal SQL can see its tables + if err = activateAccountCatalogIfNeeded(ctx, ses, toAccountId); err != nil { + return + } + // drop foreign key related tables first if err = deleteCurFkTables(ctx, ses.GetService(), bh, dbName, tblName, toAccountId); err != nil { return @@ -2406,6 +2426,11 @@ func restoreAccountUsingClusterSnapshotToNew(ctx context.Context, getLogger(ses.GetService()).Debug(fmt.Sprintf("[%s] start to restore dropped account: %v, account id: %d to new account id: %d, restore timestamp: %d", snapshotName, account.accountName, account.accountId, toAccountId, snapshotTs)) fromAccount := account.accountId + // activate the target account's catalog so internal SQL can see its tables + if err = activateAccountCatalogIfNeeded(ctx, ses, uint32(toAccountId)); err != nil { + return + } + // drop foreign key related tables first if isNeedToCleanToDatabase { if err = deleteCurFkTables(ctx, ses.GetService(), bh, "", "", uint32(toAccountId)); err != nil { From 8c57c6487a3bd97e41e63c340be4d22be77693d3 Mon Sep 17 00:00:00 2001 From: aptend Date: Sun, 29 Mar 2026 16:32:40 +0800 Subject: [PATCH 03/11] fix: activate source account catalog before restore FK resolution When restoring a dropped account (via cluster snapshot or account-level snapshot), the source account may have never been activated on this CN (e.g., no tenant session was created before the drop). Without activation, the source account's catalog entries are absent from both the catalog cache and the mo_tables partition state (filtered by TN publish), causing table resolution for mo_foreign_keys to fail with 'table does not exist'. Fix: activate the source (from) account's catalog before FK table resolution when from != to, in both the cluster restore path (restoreAccountUsingClusterSnapshotToNew) and the account-level restore path (doRestoreSnapshot). BVT: snapshot/cluster 3772/3772 (100%), snapshot/ 17597/17598 (99.99%, the 1 remaining failure is a pre-existing stale expected-result in snapshotRead.sql unrelated to this change) Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com> --- docs/design/lazy_catalog_load_for_cn.flow.md | 243 +++-------- docs/design/lazy_catalog_load_for_cn.plan.md | 389 +++++------------- .../docker-compose.yml | 5 + etc/docker-multi-cn-local-disk/start.sh | 3 + pkg/container/types/types.go | 2 +- pkg/frontend/authenticate.go | 8 + pkg/frontend/back_exec.go | 6 + pkg/frontend/clone.go | 13 + pkg/frontend/compiler_context.go | 38 +- pkg/frontend/output.go | 16 +- pkg/frontend/predefined.go | 2 +- pkg/frontend/show_account.go | 12 +- pkg/frontend/snapshot.go | 45 ++ pkg/frontend/snapshot_restore_with_ts.go | 11 + pkg/sql/colexec/aggexec/sumavg2.go | 2 +- .../ctl/cmd_activate_tenant_catalog.go | 76 ++++ pkg/sql/plan/function/ctl/types.go | 6 +- pkg/sql/plan/make.go | 5 +- pkg/vm/engine/disttae/cache/catalog.go | 68 ++- pkg/vm/engine/disttae/logtail.go | 16 + pkg/vm/engine/disttae/logtail_consumer.go | 109 ++++- .../engine/disttae/logtailreplay/partition.go | 4 + .../disttae/logtailreplay/partition_state.go | 89 ++-- .../logtailreplay/partition_state_test.go | 111 +++++ pkg/vm/engine/disttae/txn_database.go | 65 ++- pkg/vm/engine/disttae/txn_table.go | 5 +- .../tae/logtail/service/catalog_filter.go | 33 +- .../logtail/service/catalog_filter_test.go | 62 ++- pkg/vm/engine/tae/logtail/service/server.go | 9 +- .../engine/tae/logtail/service/server_test.go | 4 +- .../metadata/branch_metadata_tenant.result | 9 +- .../metadata/branch_metadata_tenant.sql | 11 +- 32 files changed, 871 insertions(+), 606 deletions(-) create mode 100644 pkg/sql/plan/function/ctl/cmd_activate_tenant_catalog.go diff --git a/docs/design/lazy_catalog_load_for_cn.flow.md b/docs/design/lazy_catalog_load_for_cn.flow.md index 3a29e6eeccd98..251961380f2ce 100644 --- a/docs/design/lazy_catalog_load_for_cn.flow.md +++ b/docs/design/lazy_catalog_load_for_cn.flow.md @@ -1,224 +1,107 @@ -# Lazy Catalog Flow(coding-ready) +# Lazy Catalog Flow ## 关键规则 -1. **lazy subscribe / activation 不再透传 raw checkpoint** - - 三张系统表的 `CkpLocation` 不是 account-filtered; - - 所以 startup / reconnect subscribe 和 activation response 只转发过滤后的 row-level data,不直接把 raw checkpoint 送到 CN。 - -2. **共享 `PartitionState` 只保存全局基线** - - object / metadata 持续全局推进; - - 只有 row-level in-memory delta 按 account 过滤。 - -3. **account 的 ready 点是 `replayTS`,不是 `targetTS`** - - `targetTS` 是 TN 返回的 barrier; - - `replayTS` 是 CN 在 barrier 真正落地后拿到的确定 snapshot ts。 - -4. **activation 期间普通 push 仍然推进 `PartitionState`** - - 只是 cache 侧不能直接对外可见; - - cache apply 要先进入 `accountDCA[account]`,full replay 后再 flush。 - -5. **row delete 也必须按 account 过滤** - - delete 不看 delete batch 里的 `account_id`; - - 统一按 `cpkey` 解出的 account 判断。 - -6. **push 与 pull 的 account 粒度不一样** - - steady-state push 的三表 in-memory entry 视为只属于一个 account; - - 所以 TN 的 publish 过滤和 CN 的 `accountDCA` 路由都按整条 entry keep/drop,不做 row-splitting; - - 但 subscribe / activation pull 回来的 entry 仍可能混合多个 account,TN 需要在发送前复制出目标行。 +1. **subscribe/activation 不透传 raw checkpoint**:只转发过滤后的 row-level data。 +2. **`PartitionState` 只保存全局基线**:object/metadata 全局推进;row-level delta 按 account 过滤。 +3. **account ready 点是 `replayTS`,不是 `targetTS`**。 +4. **activation 期间 push 仍推进 `PartitionState`**:cache apply 进 `accountDCA`,replay 后 drain。 +5. **delete 也按 account 过滤**:统一按 `cpkey` 解出 account。 +6. **push 与 pull 粒度不同**:push entry 视为单 account(整条 keep/drop);pull batch 可能混合 account(行级复制)。 --- ## Flow 1:首次 startup -### CN 发起 - -1. 建立 logtail stream。 -2. `subSysTables()` 对三张系统表发送 subscribe request: - -```text -lazy_catalog = true -initial_active_accounts = [0] -``` - -### TN 返回 - -3. startup subscribe response 返回: - - object / metadata; - - 过滤后的 sys account row-level in-memory delta; - - 不转发 raw `CkpLocation`。 - -### CN 落地 - -4. 把 subscribe response 应用到共享 `PartitionState`。 -5. 继续走当前 startup 骨架: - - `waitTimestamp()` - - `replayCatalogCache()` -6. startup replay 只加载 sys account。 -7. replay 期间普通 cache apply 继续走现有 global DCA 语义。 - - global DCA 只负责 startup / reconnect 的共享基线; - - `accountDCA` 只负责运行中单 account activation,不会阻塞其他 ready account 的正常 cache apply。 -8. replay 完成后: - - flush startup DCA; - - `sys.readyTS = startupReplayTS`; - - `sys.state = ready`。 +1. CN `subSysTables()` 发送 `lazy_catalog=true, initial_active_accounts=[0]`。 +2. TN 返回:object/metadata + sys account row-level delta(无 raw `CkpLocation`)。 +3. CN 应用到 `PartitionState`,走 `waitTimestamp()` / `replayCatalogCache()`(仅 sys account)。 +4. Flush global DCA → `sys.readyTS = startupReplayTS` → `sys.state = ready`。 -最终结果: - -- `PartitionState`:包含全局 object / metadata,以及 sys 已放行的 row-level delta; -- `CatalogCache`:只有 sys account。 - -说明:startup 仍沿用现有 `waitTimestamp()` / `replayCatalogCache()` 路径,不新增单独的 startup `targetTS`。 +结果:`PartitionState` 包含全局基线 + sys delta;`CatalogCache` 只有 sys。 --- ## Flow 2:运行中新增 account X -### A. 触发 activation - -1. `Session.AuthenticateUser()` 完成租户识别。 -2. 在 `tenant.SetTenantID(X)` 之后、第一条 tenant SQL 之前调用: - -```go -ActivateTenantCatalog(ctx, X) -``` - -### B. TN 执行 catch-up - -3. 若 X 已 ready,CN 直接返回。 -4. 否则 CN: - - `singleflight(key = X)`; - - `state[X] = catching_up`; - - 初始化 `accountDCA[X]`; - - 分配本地 `seq`; - - 发送 activation request。 - -5. TN 收到 request 后: - - 记录 `activatingAccounts[X] = seq`; - - phase1 worker 从 `0` 开始按表拉历史 row-level delta,实际下界由 checkpoint 逻辑处理; - - 三张表 catch-up 可以并发执行,直到各自补到 `phase1To`; - - `logtailSender()` 串行完成 phase2,并取 `targetTS`; - - response 成功进入当前 session FIFO `sendChan` 后,才把 X 加入 `activeCatalogAccounts`。 - -这里锁死的边界是: - -- `targetTS` 之前的数据属于 activation response; -- `targetTS` 之后的数据属于 steady-state push。 - -### C. CN 先补 `PartitionState` - -6. CN 收到 `ActivateAccountForCatalogResponse{account_id, seq, target_ts, tails}`。 -7. 用 `(X, seq)` 命中当前 pending activation。 -8. 先把 `tails` 应用到共享 `PartitionState`。 +**A. 触发**:`AuthenticateUser` → `tenant.SetTenantID(X)` 后 → `ActivateTenantCatalog(ctx, X)`。 -这里不要求 `tails` 固定顺序;CN 必须按表身份消费,而不是按位置消费。 +**B. CN 侧**: +1. X 已 ready → 返回。 +2. Inflight dedup(sync.Map leader/waiter) → `catching_up` + `accountDCA` + 分配 `seq` + 发送 request。 -此时语义: +**C. TN 侧**: +1. Phase1 worker 从 0 拉三表历史 delta(**立即过滤**),入 `activationTailChan`。 +2. Sender 串行完成 phase2(`phase1To → targetTS`),合并,发送 response。 +3. Response 入 FIFO 后才将 X 加入 `activeCatalogAccounts`。 -- X 在 `targetTS` 之前缺失的 row-level delta 已补齐; -- X 仍然不能 serve cache。 +**D. CN 落地**: +1. Apply tails → `PartitionState`。 +2. `WaitLogTailAppliedAt(targetTS)` → `replayTS`。 +3. `replayCatalogCacheForAccount(X, replayTS)` → drain `accountDCA[X]` → `readyTS` → `ready`。 -### D. CN 取确定的 `replayTS` +**E. 期间 push**:`PartitionState` 正常更新;X 的 cache apply 进 `accountDCA`。 -9. 执行: - -```go -replayTS = WaitLogTailAppliedAt(targetTS) -``` +--- -按当前 waiter 语义: +## Flow 3:steady state -- 会先等到 `latestApplied >= targetTS`; -- 返回值通常是 `latestApplied.Next()`; -- 所以 `replayTS` 一般严格大于 `targetTS`。 +- TN publish:object/metadata 全局推进;row-level delta 只对 `sys ∪ activeCatalogAccounts` 放行。 +- CN:ready account 正常更新 PS + cache。 +- `CanServeAccount(X, ts)` = `globalCanServe(ts) && readyTS[X] exists && ts >= readyTS[X]`。 -### E. activation 期间普通 push 的处理 +--- -10. 如果 X 的实时 update push 在 replay 完成前到达: - - `PartitionState` 正常更新; - - cache 不直接 apply; - - 写入 `accountDCA[X]`。 +## Flow 4:reconnect -这里依赖的是 push-entry 单 account 假设:CN 只对运行中收到的 pushed entry 按 entry 的 account 决定是立刻 apply 还是进入 `accountDCA[X]`。 +1. CN 保存 `wantedAccounts`(断线前已 ready)。 +2. Reconnect `subSysTables()`:`initial_active_accounts = wantedAccounts ∪ {0}`。 +3. TN 返回 sys + wanted 的 row-level delta。 +4. CN 走 startup 骨架 → `reconnectReplayTS` → replay sys + wanted → 批量 `ready`。 -### F. full replay + ready +--- -11. 在 `catalogCacheMu` 下执行: +## Flow 5:存储回退(account 未激活) -```go -replayCatalogCacheForAccount(ctx, e, X, replayTS) -``` +当 `CanServeAccount(X) == false` 时,engine 退回 `loadTableFromStorage()`: -12. full replay 完成后,在同一个 `catalogCacheMu` 临界区里: - - drain `accountDCA[X]`; - - `X.readyTS = replayTS`; - - `state[X] = ready`。 +1. `Engine.Database("mo_catalog")` 短路(`engine.go:369`),不查 cache。 +2. `relation("mo_tables")` 强制 `accountId=0`(`txn_database.go:107`),sys 始终 ready。 +3. `execReadSql(SELECT ... FROM mo_tables WHERE account_id=X)` 扫描 `PartitionState`。 -13. `ActivateTenantCatalog(ctx, X)` 返回。 -14. auth 继续执行 tenant SQL。 +**数据新鲜度**:返回 subscribe-time 快照。subscribe 后 TN push 按 account 过滤,inactive account 增量不到达 CN。 +**安全性**:login 在 tenant SQL 前激活;后台升级有版本守卫;同事务写入通过 workspace 可见。 --- -## Flow 3:X 激活后的 steady state - -1. TN `publishEvent()`: - - object / metadata 继续全局推进; - - row-level delta 只对 `sys ∪ activeCatalogAccounts` 放行。 - -2. CN 对 ready 的 X: - - `PartitionState` 正常更新; - - cache 正常更新。 +## Flow 6:activation 被 reconnect 打断 -3. `CanServeAccount(X, ts)` 必须同时满足: - - 全局 `CatalogCache.CanServe(ts)` 为 true; - - `ts >= X.readyTS`。 +1. `doActivateTenantCatalog` 等待 `respCh` → reconnect → `resetAllStates()` 发送 nil。 +2. Sentinel `errActivationInterruptedByReconnect` → 回退 `catching_up → inactive`。 +3. `ActivateTenantCatalog` 指数退避重试(500ms/1s/2s/4s,最多 4 次)。 +4. 所有 goroutine 独立重试,sync.Map 自动选举新 leader。 --- -## Flow 4:reconnect - -1. CN 保存断线前已 ready 的账户集合: +## Flow 7:consumeEntry DCA 路由 -```text -wantedAccounts ``` - -2. reconnect 后第一次 `subSysTables()` 直接发送: - -```text -initial_active_accounts = wantedAccounts ∪ {0} +entry 到达 + ├─ PartitionState 无条件更新 + ├─ 非 lazy catalog table → return + ├─ Global DCA buffering? → 缓冲(startup/reconnect) + ├─ Account catching_up? → accountDCA[X] + └─ 直接 apply ``` -3. TN 返回: - - object / metadata; - - sys + wanted accounts 的 row-level in-memory delta; - - 不转发 raw `CkpLocation`。 - -4. CN 继续走 startup 骨架: - - `waitTimestamp()` - - 拿到 `reconnectReplayTS` - - replay sys + wanted accounts - -5. replay 完成后,对 `wantedAccounts ∪ {0}` 统一设置: - - `readyTS = reconnectReplayTS` - - `state = ready` - -这里的建模是: - -> reconnect 是“带着 wanted accounts 的重新 startup”,不是“运行中新增一个 account”。 - --- ## 一句话版 -### startup - -`PS 先到位 -> startupReplayTS -> sys full replay -> sys ready` - -### 新增 account - -`PS 先补到 targetTS -> WaitLogTailAppliedAt 得到 replayTS -> replayTS 上 full replay -> flush accountDCA -> account ready` - -### reconnect - -`wanted accounts 一次性带给 TN -> reconnectReplayTS -> sys + wanted 全量 replay -> 批量 ready` +| 场景 | 流程 | +|------|------| +| startup | PS → startupReplayTS → sys replay → sys ready | +| 新增 account | PS 补到 targetTS → replayTS → replay → drain DCA → ready | +| reconnect | wanted ∪ {0} → reconnectReplayTS → 批量 replay → 批量 ready | +| 存储回退 | CanServe=false → mo_catalog 短路 + accountId=0 → 读 PS 快照 | +| reconnect 打断 | nil response → sentinel → 退避重试 → 新 leader | diff --git a/docs/design/lazy_catalog_load_for_cn.plan.md b/docs/design/lazy_catalog_load_for_cn.plan.md index e71d549ca3a30..b7f3d7f1b1c83 100644 --- a/docs/design/lazy_catalog_load_for_cn.plan.md +++ b/docs/design/lazy_catalog_load_for_cn.plan.md @@ -1,372 +1,171 @@ --- name: Lazy Catalog Load for CN -overview: "Coding-facing design reference for lazy catalog load: keep shared catalog PartitionState semantics unchanged, add account-scoped cache readiness, and make runtime activation follow the same PS-first then exact-replay model as startup." -todos: - - id: proto-contract - content: "proto/logtail.proto + pkg/pb/logtail: add `SubscribeRequest.lazy_catalog`, `initial_active_accounts`, and dedicated `ActivateAccountForCatalogRequest/Response{account_id, seq, target_ts, tails}`; keep request_id/response_id as stream ids and use `seq` for activation correlation" - status: done - - id: stream-plumbing - content: "pkg/vm/engine/tae/logtail/service/{client.go,server.go,request.go,response.go}: plumb activation messages over the existing segmented logtail stream and dispatch them separately from subscribe/update" - status: done - - id: tn-session-filter - content: "pkg/vm/engine/tae/logtail/service/session.go + helpers: track lazyCatalogMode, activeCatalogAccounts, activatingAccounts, and reuse one catalog entry-account filter helper for subscribe/publish/activation (delete still decodes account from cpkey)" - status: done - - id: cn-replay-refactor - content: "pkg/catalog/types.go + pkg/vm/engine/disttae/logtail_consumer.go: split `replayCatalogCache()` into reusable startup/reconnect/account replay helpers, and add account-filtered SQL templates so replay no longer assumes a global all-account load" - status: done - - id: startup-sys-baseline - content: "pkg/vm/engine/disttae/logtail_consumer.go: make `subSysTables()` send `lazy_catalog=true` with `initial_active_accounts=[0]`, and change startup replay/cache ready so only sys account becomes ready after startup" - status: done - - id: tn-activation-sender - content: "pkg/vm/engine/tae/logtail/service/server.go: implement activation phase1 worker plus sender-serialized phase2/targetTS/response enqueue, and only add an account to `activeCatalogAccounts` after the activation response enters `sendChan`" - status: done - - id: cn-account-state - content: "pkg/vm/engine/disttae/{logtail_consumer.go,engine.go,types.go}: add pending(account, seq), inactive/catching_up/ready, per-account readyTS, per-account accountDCA, and wantedAccounts that survives reconnect" - status: done - - id: cn-activation-flow - content: "pkg/vm/engine/disttae/logtail_consumer.go + pkg/vm/engine/tae/logtail/service/client.go: send activation, apply activation tails to shared PartitionState, call `WaitLogTailAppliedAt(targetTS)`, replay one account at replayTS, flush accountDCA, and remember the account in wantedAccounts" - status: done - - id: serve-gate-audit - content: "pkg/vm/engine/disttae/{engine.go,txn_database.go,txn.go,mo_table_stats.go} + cache/catalog.go: replace global-only `CatalogCache.CanServe(ts)` decisions with account-aware readyTS checks and storage fallback" - status: done - - id: reconnect-restore - content: "pkg/vm/engine/disttae/logtail_consumer.go: preserve wantedAccounts across reconnect, send them in the first system subscribe, replay sys + wanted accounts at one reconnectReplayTS, and batch-mark the whole set ready" - status: done - - id: frontend-auth-hook - content: "pkg/frontend/session.go: trigger activation immediately after `tenant.SetTenantID(...)` and before the first tenant-context SQL in `Session.AuthenticateUser`; return activation errors directly" - status: done - - id: tests-audit - content: "Add coverage for startup sys-only baseline, activation seq/ordering, global/per-account delayed cache apply, subscribe-time mixed-entry row copying, readyTS serve gate, reconnect batch restore, auth-path failure handling, and lazy-off compatibility" - status: done +overview: "Keep shared catalog PartitionState semantics unchanged, add account-scoped cache readiness, and make runtime activation follow the same PS-first then exact-replay model as startup." isProject: false --- # Lazy Catalog Load for CN(主设计文档) -> 这份文档只保留做代码前必须锁死的设计约束、协议语义、状态机和落点。更细的执行时序见 `lazy_flow.md`。 +> 设计约束、协议语义、实现要点。执行时序见 `lazy_catalog_load_for_cn.flow.md`。 ## 1. 目标与边界 -### 目标 +**目标**:startup 后 `CatalogCache` 只加载 sys account;运行中新增 account 走 activation;reconnect 按批量恢复处理。 -1. **startup** 保持现有系统表启动骨架,但 `CatalogCache` 在 startup 阶段只加载 sys account。 -2. **运行中新增 account** 复用 startup 语义:先把数据补进共享 `PartitionState`,再在一个确定的 snapshot ts 上做 full replay,最后再开放 cache serve。 -3. **reconnect** 按“批量恢复已 ready account”处理,而不是逐个 post-reconnect activation。 - -### 非目标 - -- 不优化 system-table checkpoint 内存。 -- 不把 checkpoint 改成按账户懒化。 -- 不把 catalog `PartitionState` 改成 per-account 隔离。 -- 不改普通用户表的订阅 / 推送 / 反订阅流程;lazy catalog 只针对 `mo_database` / `mo_tables` / `mo_columns`。 - -一句话原则: +**非目标**:不改 checkpoint 按账户懒化;不改 `PartitionState` 为 per-account 隔离;不改普通表的订阅/推送/反订阅。 > **共享 `PartitionState` 保留全局基线;account 是否可见,由 per-account replay 和 per-account `readyTS` 决定。** --- -## 2. 必须保留的基线语义 - -当前 startup 的真实语义是: +## 2. 核心不变量 -```text -PS first -> exact replay ts -> full replay -> flush delayed cache apply -> ready -``` - -本轮实现只是在这个基线上扩展: - -- startup 继续走现有 `waitTimestamp()` / `replayCatalogCache()` 骨架; -- runtime activation 改成显式拿 `targetTS`,再通过 `WaitLogTailAppliedAt(targetTS)` 取得确定的 `replayTS`; -- reconnect 沿 startup 骨架一次性恢复 sys + wanted accounts。 +1. **不透传 raw checkpoint**:三表的 `CkpLocation` 不是 account-filtered,subscribe/activation 只转发过滤后的 row-level data。 +2. **account 在 full replay 后才能 serve**:activation response 只补 `PartitionState`,`readyTS` 在 replay 后发布。 +3. **`targetTS` 在 TN sender 路径上确定**:保证和 steady-state push 无 gap。 +4. **response 入 FIFO 后才标记 active**:不能让 post-activation update 先于 response 到达 CN。 +5. **object/metadata 全局推进,只有 row-level delta 按 account 过滤**:insert/update 按 `account_id`,delete 按 `cpkey`。 +6. **reconnect 视为批量恢复**:`wantedAccounts ∪ {0}` 一次性 subscribe + 统一 replay。 +7. **只落在三张 catalog 系统表**:`mo_database`/`mo_tables`/`mo_columns`,不污染普通表。 --- -## 3. 核心不变量 - -1. **lazy subscribe / activation 不直接透传 raw checkpoint** - - 三张系统表的 `CkpLocation` 不是按 account 过滤的; - - startup / reconnect subscribe 和 activation response 都只转发过滤后的 row-level data,不把 raw checkpoint 直接送到 CN。 -2. **account 只有在 full replay 完成后才能 serve** - - activation response 只负责把缺失 row-level delta 补进 `PartitionState`; - - account 真正 ready 的时点是 full replay 完成后的 `readyTS`。 -3. `**targetTS` 必须在 TN sender 路径上确定** - - 只有这样,`targetTS` 才是和 steady-state push 无 gap 的真实边界。 -4. **account 变 active 必须晚于 activation response 成功入 FIFO send queue** - - 先 response enqueue,再打开 `activeCatalogAccounts`; - - 不能让 post-activation update 先于 activation response 到达 CN。 -5. **object / metadata 全局推进,只有 row-level in-memory delta 按 account 过滤** - - insert / update 按行内容里的 account 过滤; - - row delete 按 `cpkey` 解出的 account 过滤; - - startup / reconnect subscribe、activation response、steady-state push 共用同一套过滤契约。 -6. **reconnect 视为批量恢复,不视为单账户新增** - - reconnect 时第一轮 subscribe 直接带 `wantedAccounts ∪ {0}`; - - 用一个统一 replay ts 恢复 sys + wanted accounts。 -7. **lazy catalog 逻辑必须只落在三张 catalog 系统表** - - `SubscribeRequest.lazy_catalog`、activation、row-level catalog filter、account readyTS 都只服务 `mo_database` / `mo_tables` / `mo_columns`; - - 普通表的订阅 / steady-state push / 反订阅语义不能被这轮改动污染。 +## 3. 关键术语 ---- +| 术语 | 含义 | +|------|------| +| `targetTS` | TN activation response barrier:row-level delta 已补到此点 | +| `replayTS` | CN 调 `WaitLogTailAppliedAt(targetTS)` 后拿到的确定 snapshot ts(通常 > targetTS) | +| `readyTS` | account 开始可 serve 的时间点 = replayTS | +| `seq` | activation 相关键(不复用 stream id) | +| `accountDCA` | per-account 延迟 cache apply 暂存区 | -## 4. 关键术语与状态 - -### 时间戳语义 - -- `**targetTS`** - - TN 在 activation response 中返回的 barrier; - - 只表示“目标 account 缺失的 row-level delta 已补到这个点”。 -- `**replayTS`** - - CN 在确认 `targetTS` 已应用后拿到的确定 snapshot ts; - - 用于 full replay,也作为该 account 的 `readyTS`; - - 按当前 waiter 语义,通常满足 `replayTS > targetTS`。 -- `**readyTS`** - - 某个 account 在某个 CN 上开始可以安全使用 cache serve 的时间点; - - sys account、新增 account、reconnect 恢复的一批 account 都各自依赖这个语义。 - -### 相关键 - -- `**seq**` - - activation 的相关键; - - 不能依赖 `RequestId/ResponseId`,因为它们在当前协议里承担 stream ID 角色。 - -### CN account 状态 - -```go -type accountCatalogState struct { - state catalogReadyState - readyTS timestamp.Timestamp -} - -const ( - accountInactive catalogReadyState = iota - accountCatchingUp - accountReady -) -``` - -语义固定为: - -- `inactive`:不能 serve,也不能直接把该 account 的增量写入 cache。 -- `catching_up`:activation 中,`PartitionState` 正常推进,但 cache 更新进入 `accountDCA`。 -- `ready`:full replay 完成,`readyTS` 已确定,cache 可正常 serve / apply。 +CN account 状态:`inactive → catching_up → ready`。 --- -## 5. 协议语义 - -### `SubscribeRequest` +## 4. 协议 ```protobuf message SubscribeRequest { api.TableID table = 1; bool lazy_catalog = 2; - repeated uint32 initial_active_accounts = 3; -} -``` - -用途: - -- startup:`initial_active_accounts = [0]` -- reconnect:`initial_active_accounts = wantedAccounts ∪ {0}` - -### `ActivateAccountForCatalogRequest/Response` - -```protobuf -message ActivateAccountForCatalogRequest { - uint32 account_id = 1; - uint64 seq = 2; + repeated uint32 initial_active_accounts = 3; // startup=[0], reconnect=wanted∪{0} } - +message ActivateAccountForCatalogRequest { uint32 account_id = 1; uint64 seq = 2; } message ActivateAccountForCatalogResponse { - uint32 account_id = 1; - uint64 seq = 2; + uint32 account_id = 1; uint64 seq = 2; timestamp.Timestamp target_ts = 3; - repeated TableLogtail tails = 4; + repeated TableLogtail tails = 4; // 三表 row-level delta,不要求固定顺序 } ``` -锁死两条语义: - -1. activation 是 **account 级** 协议,不是 table 级协议; -2. `tails` 只承载三张 catalog 表的 **row-level in-memory delta**,但**不要求固定顺序**;TN 内部可以并发拉取三表,CN 必须按 table identity 消费,而不是按位置消费。 - --- -## 6. TN 侧职责 - -### session 侧状态 - -```go -type Session struct { - // existing fields... - lazyCatalogMode bool - activeCatalogAccounts map[uint32]struct{} - activatingAccounts map[uint32]uint64 -} -``` - -### startup / reconnect - -- checkpoint 继续正常返回并进入共享 `PartitionState`; -- object / metadata 继续全局推进; -- row-level in-memory delta 只对 `initial_active_accounts` 放行。 - -### runtime activation - -activation 保持“两阶段、sender 收口”的模型: - -1. `onMessage()` 只做 request 分发和 session 定位; -2. worker 执行 phase1,按表拉历史 row-level delta; -3. `logtailSender()` 串行完成: - - 校验 `activatingAccounts[account] == seq`; - - 取 `targetTS`; - - 拉 `phase1To -> targetTS` 的 phase2; - - 合并 phase1 + phase2; - - 发送 response; - - **只有 response 成功进入 session FIFO `sendChan` 后** 才把 account 加入 `activeCatalogAccounts`。 - -phase1 的下界固定为从 `0` 开始;实际有效下限由现有 checkpoint 处理逻辑决定,不在 activation 协议里额外维护 per-table catch-up 起点。 - -三张 catalog 表的 catch-up 可以并发执行;response 里的 `tails` 只要求完整覆盖三表,不要求固定顺序。 - -### steady state +## 5. TN 侧 -- `publishEvent()` 继续全局推 object / metadata; -- row-level catalog delta 只对 `sys ∪ activeCatalogAccounts` 放行。 +### Session 状态 -### 过滤契约 +`lazyCatalogMode` / `activeAccounts` / `activatingSeqByAccount`。`activeAccountsSnapshot`(copy-on-write `atomic.Pointer`)供 publish 热路径无锁读。 -TN 侧必须复用一套 catalog filter helper,用在: +### 过滤 -- startup / reconnect subscribe response(过滤 row,且去掉 raw `CkpLocation`) -- activation response(过滤 row,且去掉 raw `CkpLocation`) -- steady-state `publishEvent()` +- **subscribe/activation(pulled batch)**:可能混合 account → 用 `batch.Union(sels)` 按 `account_id`/`cpkey` 行级复制。Phase1 在 pull 后**立即**过滤,不进串行 send 路径。Filtered batch 用 cleanup callback 保活,不做 proto-batch deep clone。`stripObjectMeta` 参数控制 object metadata 是否保留(subscribe 保留,activation 去除)。 +- **steady-state push**:entry 级单 account 假设 → `prepareLazyCatalogPublishWrapsFromIndex` 读 entry summary 整条 keep/drop。无 summary 时退回行级扫描。 +- **entry-level summary**(`txn_handle.go`):`api.Entry` 携带 `lazy_catalog_account_id` + validity bit,仅当 batch 内所有行属于同一 account 时设置。混合 batch(如 restore)不设置。 -过滤规则固定为: +### Activation worker -- steady-state push 的三张 catalog 表 in-memory entry batch 仍假定各自只属于一个 account,因此 TN publish 过滤和 CN `accountDCA` 路由都可以按 entry 整体 keep/drop; -- startup / reconnect subscribe response 和 activation response 里的 pulled entry batch 仍可能混合多个 account,因此 TN 在发送前必须复制出目标行; -- steady-state push 的 insert / update:按 entry 首行里的 `account_id` 判断整个 entry; -- steady-state push 的 row delete:按 entry 首行 `cpkey` 解出的 account 判断整个 entry。 +有界 `activationReqChan` + 固定 worker pool(默认 1)。Phase1(worker 并发拉三表)→ Phase2(sender goroutine 串行:补齐 gap、合并、发送)。`targetTS = waterline.Waterline()`,response 进入 FIFO 后才 `completeActivation`。 --- -## 7. CN 侧职责 +## 6. CN 侧 -### 需要维护的状态 +### 状态 -- `pending(account, seq)`:request / response 关联; -- `state[account]`:`inactive / catching_up / ready`; -- `readyTS[account]`; -- `accountDCA[account]`:activation 期间普通 push 的暂存区; -- `wantedAccounts`:reconnect 时要恢复的账户集合。 +`lazyCatalogCNState`:per-account `{state, readyTS}`、`pendingSeq`、`accountDCA`、`wantedAccounts`、`inflightActivations`(sync.Map)、`activationHistory`(circular buffer)、`catchingUpCount`(atomic fast path)。 -### startup +### Activation 流程 -- 继续走当前 startup 骨架; -- `replayCatalogCache()` 仍只加载 sys account; -- startup replay 完成后设置 `sys.readyTS = startupReplayTS`。 +1. 已 ready → 直接返回。 +2. Inflight dedup(sync.Map leader/waiter)。 +3. `catching_up` + 初始化 `accountDCA` + 分配 `seq` + 发送 request。 +4. Response → apply tails 到 `PartitionState` → `WaitLogTailAppliedAt(targetTS)` → `replayCatalogCacheForAccount` → drain `accountDCA` → `readyTS` → `ready`。 -### runtime activation +失败回退:`catching_up → inactive`,丢弃 `accountDCA`,TN `abortActivation`。 -同步路径固定为: +### Reconnect 重试 -1. 如果 account 已 ready,直接返回; -2. `singleflight(key=accountID)`; -3. 置 `state[account] = catching_up` 并初始化 `accountDCA`; -4. 发送 `ActivateAccountForCatalogRequest{account, seq}`; -5. 收到 response 后,先把 `tails` 应用到 `PartitionState`; -6. 调 `WaitLogTailAppliedAt(targetTS)` 得到确定的 `replayTS`; -7. 执行 `replayCatalogCacheForAccount(account, replayTS)`; -8. 在 `catalogCacheMu` 下先 drain `accountDCA[account]`,再发布 `readyTS[account] = replayTS`; -9. 最后把状态切到 `ready`。 +`resetAllStates()` 向 pending channel 发送 nil → sentinel `errActivationInterruptedByReconnect` → 指数退避重试(500ms/1s/2s/4s,最多 4 次)。所有 goroutine 独立重试,新 leader 自动选出。 -### cache apply 规则 +### DCA 两级架构 -- `inactive`:不直接写 cache; -- `catching_up`:普通 push 继续更新 `PartitionState`,cache apply 写入 `accountDCA`; -- `ready`:正常更新 cache。 +- **Global DCA**:startup/reconnect 期间阻止 cache apply 抢跑。Replay 后 drain。 +- **Per-account DCA**:runtime activation 期间隔离一个 tenant 的 catch-up。 -这里依赖的是 steady-state push-entry 单 account 假设:CN 只对运行中收到的 pushed entry 按 entry 归属 account 决定是立即 apply 还是进入 `accountDCA`,不再按 row 拆分 batch。 +两级互不阻塞。`consumeEntry` 路由:`PartitionState` 无条件更新 → global DCA 判断 → per-account DCA 判断 → 直接 apply。 -### serve gate - -最终读路径必须按 account 判断: +### Serve gate 与存储回退 ```go -func CanServeAccount(account uint32, ts timestamp.Timestamp) bool { - return globalCatalogCacheCanServe(ts) && - !accountReadyTS[account].IsEmpty() && - ts.GreaterEq(accountReadyTS[account]) -} +CanServeAccount(X, ts) = globalCanServe(ts) && accountReadyTS[X] exists && ts >= readyTS[X] ``` -不能只依赖全局 `CatalogCache.CanServe(ts)`。 - -### 失败清理 - -如果 activation 失败或超时,且没有新的 seq 接管,必须: - -- 清理 `pending(account, seq)`; -- 丢弃 `accountDCA[account]`; -- 把 `state[account]` 从 `catching_up` 回退到 `inactive`。 +回退时 `loadTableFromStorage` → `execReadSql` → 扫描 `PartitionState`(subscribe-time 快照)。安全性: +- `mo_catalog` 数据库短路(`engine.go:369`),不查 cache。 +- 系统表强制 `accountId=0`(`txn_database.go:107-113`),sys 始终 ready。 +- login 路径在任何 tenant SQL 前激活;后台升级有版本守卫。 +- **注意**:subscribe 后 TN push 按 account 过滤,inactive account 的 `PartitionState` 不含后续增量。 --- -## 8. Frontend 与 reconnect 钩子 - -### Frontend - -`pkg/frontend/session.go::(*Session).AuthenticateUser` 中: +## 7. Frontend -- 在 `tenant.SetTenantID(...)` 之后触发 activation; -- 在第一个 tenant-context SQL 之前完成 activation; -- 出错时直接返回,不能吞错后继续 tenant SQL。 +`activateAccountCatalogIfNeeded`(`snapshot.go:533`):对 `accountID==0` 直接返回。`EntireEngine` 必须转发 `TenantCatalogActivator` 接口。 -### reconnect - -- CN 记住断线前哪些 account 已 ready,记为 `wantedAccounts`; -- reconnect 后第一次 `subSysTables()` 直接携带 `wantedAccounts ∪ {0}`; -- 用一个 `reconnectReplayTS` 重建 sys + wanted accounts cache; -- 为这一批 account 统一设置 `readyTS` 和 `state=ready`。 +| 调用位置 | 场景 | +|---------|------| +| `session.go:1395` | 登录 | +| `authenticate.go:3289/4048` | ALTER/DROP ACCOUNT | +| `show_account.go:717` | SHOW ACCOUNTS | +| `snapshot.go:623,631,2385,2394` | RESTORE | +| `clone.go:257,378` | CLONE TABLE/DATABASE | --- -## 9. 主要实现落点 - -### 协议 - -- `proto/logtail.proto` -- `pkg/pb/logtail/logtail.pb.go` - -### TN +## 8. Debug 诊断 -- `pkg/vm/engine/tae/logtail/service/session.go` -- `pkg/vm/engine/tae/logtail/service/server.go` -- `pkg/vm/engine/tae/logtail/service/client.go` -- `pkg/vm/engine/tae/logtail/service/` 下新增 catalog row-level filter helper +| 端点 | 用途 | +|------|------| +| `/debug/status/catalog?account=X` | 全局状态 + per-account readiness | +| `/debug/status/catalog-cache?account=X` | 直接查看 catalog cache 内容 | +| `/debug/status/catalog-activation?account=X` | 最近 activation 事件(进程级,重启后重置) | +| `/debug/status/partitions` | 分区状态摘要 | -### CN - -- `pkg/vm/engine/disttae/logtail_consumer.go` -- `pkg/vm/engine/disttae/logtail.go` -- `pkg/vm/engine/disttae/engine.go` -- `pkg/vm/engine/disttae/cache/catalog.go` -- `pkg/catalog/types.go` +--- -### Frontend +## 9. 实现落点 -- `pkg/frontend/session.go` +**协议**:`proto/logtail.proto`、`pkg/pb/logtail/` +**TN**:`tae/logtail/service/{server,session,lazy_catalog_session,catalog_filter}.go`、`tae/logtail/txn_handle.go` +**CN**:`disttae/{lazy_catalog_cn,logtail_consumer,logtail,engine,txn_database,debug_state}.go` +**Frontend**:`frontend/{session,snapshot,authenticate,show_account,clone}.go` +**其他**:`engine/{engine,entire_engine}.go`、`catalog/types.go`、`util/status/server.go` --- -## 10. 代码前最后核对项 - -1. startup 后 `CatalogCache` 只有 sys account; -2. activation response 只补 `PartitionState`,不直接宣告 cache ready; -3. `replayTS` 是确定值,并且来自 `WaitLogTailAppliedAt(targetTS)`; -4. activation 期间实时 cache 更新会暂存,replay 后再 flush; -5. `CanServeAccount(account, ts)` 依赖 per-account `readyTS`; -6. reconnect 一次性恢复 wanted accounts,而不是逐个 activation; -7. auth 路径必须在 tenant SQL 前完成 activation; -8. row delete 和 insert / update 一样,保持严格 account-scoped 过滤。 +## 10. 核对项 + +1. startup 后 cache 只有 sys account +2. activation response 只补 PS,不直接宣告 ready +3. `replayTS` 来自 `WaitLogTailAppliedAt(targetTS)` +4. activation 期间 cache 暂存,replay 后 drain +5. `CanServeAccount` 依赖 per-account `readyTS` +6. reconnect 批量恢复,不逐个 activation +7. auth 路径在 tenant SQL 前完成 activation +8. delete 和 insert/update 同样按 account 过滤 +9. 存储回退读 subscribe-time 快照,login 路径保证不触发 +10. TN push 对三表做 per-session account 过滤 +11. phase1 在 pull 后立即过滤,不进串行路径 +12. `EntireEngine` 必须转发 `TenantCatalogActivator` diff --git a/etc/docker-multi-cn-local-disk/docker-compose.yml b/etc/docker-multi-cn-local-disk/docker-compose.yml index 0b21746c628d3..99b42e85346ea 100644 --- a/etc/docker-multi-cn-local-disk/docker-compose.yml +++ b/etc/docker-multi-cn-local-disk/docker-compose.yml @@ -70,6 +70,7 @@ services: - ../../mo-data:/mo-data - ../../logs:/logs - .:/etc/docker-multi-cn-local-disk:ro + - ../../test:${MO_REPO_ROOT}/test # BVT test data (LOAD DATA + INTO OUTFILE) # Note: Local mo-service mount removed due to GLIBC version mismatch # Use image-built binary or build inside container for compatibility environment: @@ -106,6 +107,7 @@ services: - ../../mo-data:/mo-data - ../../logs:/logs - .:/etc/docker-multi-cn-local-disk:ro + - ../../test:${MO_REPO_ROOT}/test # BVT test data (LOAD DATA + INTO OUTFILE) # Note: Local mo-service mount removed due to GLIBC version mismatch # Use image-built binary or build inside container for compatibility environment: @@ -140,6 +142,7 @@ services: - ../../mo-data:/mo-data - ../../logs:/logs - .:/etc/docker-multi-cn-local-disk:ro + - ../../test:${MO_REPO_ROOT}/test # BVT test data (LOAD DATA + INTO OUTFILE) # Note: Local mo-service mount removed due to GLIBC version mismatch # Use image-built binary or build inside container for compatibility environment: @@ -174,6 +177,7 @@ services: - ../../mo-data:/mo-data - ../../logs:/logs - .:/etc/docker-multi-cn-local-disk:ro + - ../../test:${MO_REPO_ROOT}/test # BVT test data (LOAD DATA + INTO OUTFILE) # Note: Local mo-service mount removed due to GLIBC version mismatch # Use image-built binary or build inside container for compatibility environment: @@ -200,6 +204,7 @@ services: - ../../mo-data:/mo-data - ../../logs:/logs - .:/etc/docker-multi-cn-local-disk:ro + - ../../test:${MO_REPO_ROOT}/test # BVT test data (LOAD DATA + INTO OUTFILE) # Note: Local mo-service mount removed due to GLIBC version mismatch # Use image-built binary or build inside container for compatibility environment: diff --git a/etc/docker-multi-cn-local-disk/start.sh b/etc/docker-multi-cn-local-disk/start.sh index 0dd2dc2c7b669..fce9a95c4ad27 100755 --- a/etc/docker-multi-cn-local-disk/start.sh +++ b/etc/docker-multi-cn-local-disk/start.sh @@ -190,6 +190,9 @@ fi export DOCKER_UID=$(id -u) export DOCKER_GID=$(id -g) +# Auto-detect repo root (two levels up from this script's directory) +export MO_REPO_ROOT="$(cd "$SCRIPT_DIR/../.." && pwd)" + # Set image name based on version if [ "$IMAGE_VERSION" = "local" ]; then export IMAGE_NAME="matrixorigin/matrixone:local" diff --git a/pkg/container/types/types.go b/pkg/container/types/types.go index a90455e1cfc57..b301feabad08a 100644 --- a/pkg/container/types/types.go +++ b/pkg/container/types/types.go @@ -316,7 +316,7 @@ type FixedBytes interface { } type Ints interface { - int8 | int16 | int32 | int64 + int8 | int16 | int32 | int64 | MoYear } type UInts interface { diff --git a/pkg/frontend/authenticate.go b/pkg/frontend/authenticate.go index 2ded360b8f4b5..3d4531edf8c60 100644 --- a/pkg/frontend/authenticate.go +++ b/pkg/frontend/authenticate.go @@ -3288,6 +3288,10 @@ func doAlterAccount(ctx context.Context, ses *Session, aa *alterAccount) (err er if accountExist { // Option 1: alter the password of admin for the account if aa.AuthExist { + // Activate target account's catalog so internal SQL can resolve its tables. + if rtnErr = activateAccountCatalogIfNeeded(ctx, ses, uint32(targetAccountId)); rtnErr != nil { + return rtnErr + } // !!!NOTE!!!:switch into the target account's context, then update the table mo_user. accountCtx := defines.AttachAccountId(ctx, uint32(targetAccountId)) @@ -4043,6 +4047,10 @@ func doDropAccount(ctx context.Context, bh BackgroundExec, ses *Session, da *dro // drop tables of the tenant // NOTE!!!: single DDL drop statement per single transaction // SWITCH TO THE CONTEXT of the deleted context + // Activate target account's catalog so internal SQL can resolve its tables. + if rtnErr = activateAccountCatalogIfNeeded(ctx, ses, uint32(accountId)); rtnErr != nil { + return rtnErr + } deleteCtx = defines.AttachAccountId(ctx, uint32(accountId)) // step 2 : drop table mo_user diff --git a/pkg/frontend/back_exec.go b/pkg/frontend/back_exec.go index a61dd28c119ca..7e24a59f58a18 100644 --- a/pkg/frontend/back_exec.go +++ b/pkg/frontend/back_exec.go @@ -867,6 +867,12 @@ func newBackSession(ses FeSession, txnOp TxnOperator, db string, callBack output if ses.GetTxnHandler() != nil { connCtx = ses.GetTxnHandler().GetConnCtx() } + // TxnHandler.Close() clears connCtx while the handler pointer may still be non-nil + // (e.g. async temp-table cleanup in Session.Close). context.WithCancel(nil) panics; + // Background is a safe parent for the back session's txn context tree. + if connCtx == nil { + connCtx = context.Background() + } txnHandler := InitTxnHandler(ses.GetService(), getPu(service).StorageEngine, connCtx, txnOp) backSes := &backSession{} backSes.initFeSes(ses, txnHandler, db, callBack) diff --git a/pkg/frontend/clone.go b/pkg/frontend/clone.go index 1c9d97fa53b53..807f866fc8d62 100644 --- a/pkg/frontend/clone.go +++ b/pkg/frontend/clone.go @@ -252,6 +252,12 @@ func handleCloneTable( return } + // Activate the target account's catalog so internal SQL that runs under + // the target account context can find per-account system tables. + if err = activateAccountCatalogIfNeeded(reqCtx, ses, toAccountId); err != nil { + return + } + ctx = defines.AttachAccountId(reqCtx, toAccountId) sql := execCtx.input.sql @@ -366,6 +372,13 @@ func handleCloneDatabase( return } + // Activate the target account's catalog so internal SQL that runs under + // the target account context (e.g. FK resolution) can find per-account + // system tables like mo_foreign_keys. + if err = activateAccountCatalogIfNeeded(reqCtx, ses, toAccountId); err != nil { + return + } + ctx1 = defines.AttachAccountId(reqCtx, toAccountId) if err = bh.Exec(ctx1, fmt.Sprintf("create database `%s`", stmt.DstDatabase), diff --git a/pkg/frontend/compiler_context.go b/pkg/frontend/compiler_context.go index 6cb76dddbf7f2..ab85e7568f788 100644 --- a/pkg/frontend/compiler_context.go +++ b/pkg/frontend/compiler_context.go @@ -983,7 +983,24 @@ func (tcc *TxnCompilerContext) GetSubscriptionMeta(dbName string, snapshot *plan bh := tcc.getOrCreateBackExec(tempCtx) bh.ClearExecResultSet() - return getSubscriptionMeta(tempCtx, dbName, tcc.GetSession(), txn, bh) + sub, err := getSubscriptionMeta(tempCtx, dbName, tcc.GetSession(), txn, bh) + if err != nil { + return nil, err + } + + // Lazy catalog: if this is a subscription database, ensure the + // publisher's account catalog is activated on this CN so that any + // subsequent catalog queries (show tables, desc, select) can see + // the publisher's table metadata. + if sub != nil && sub.AccountId != 0 { + if activator, ok := tcc.GetTxnHandler().GetStorage().(engine.TenantCatalogActivator); ok { + if err = activator.ActivateTenantCatalog(tempCtx, uint32(sub.AccountId)); err != nil { + return nil, err + } + } + } + + return sub, nil } func (tcc *TxnCompilerContext) CheckSubscriptionValid(subName, accName, pubName string) error { @@ -1041,7 +1058,24 @@ func (tcc *TxnCompilerContext) ResolveSnapshotWithSnapshotName(snapshotName stri if snapshot := tcc.GetSnapshot(); snapshot != nil && snapshot.GetTenant() != nil { tenantCtx = defines.AttachAccount(tenantCtx, snapshot.Tenant.TenantID, GetAdminUserId(), GetAccountAdminRoleId()) } - return doResolveSnapshotWithSnapshotName(tenantCtx, tcc.GetSession(), snapshotName) + snap, err := doResolveSnapshotWithSnapshotName(tenantCtx, tcc.GetSession(), snapshotName) + if err != nil { + return nil, err + } + + // Lazy catalog: if the snapshot targets a non-sys tenant account, + // activate its catalog on this CN so that cross-account snapshot + // reads (show databases/tables {snapshot = 'xxx'}) can see the + // target account's metadata. + if snap != nil && snap.GetTenant() != nil && snap.Tenant.TenantID != 0 { + if activator, ok := tcc.GetTxnHandler().GetStorage().(engine.TenantCatalogActivator); ok { + if err = activator.ActivateTenantCatalog(tenantCtx, snap.Tenant.TenantID); err != nil { + return nil, err + } + } + } + + return snap, nil } func (tcc *TxnCompilerContext) CheckTimeStampValid(ts int64) (bool, error) { diff --git a/pkg/frontend/output.go b/pkg/frontend/output.go index 2d01ddf976412..975c208aa7d86 100644 --- a/pkg/frontend/output.go +++ b/pkg/frontend/output.go @@ -205,7 +205,7 @@ func extractRowFromVector2(ctx context.Context, ses FeSession, vec *vector.Vecto case types.T_int16: row[i] = colSlices.arrInt16[sliceIdx][rowIndex] case types.T_year: - row[i] = types.MoYear(colSlices.arrInt16[sliceIdx][rowIndex]) + row[i] = colSlices.arrMoYear[sliceIdx][rowIndex] case types.T_uint16: row[i] = colSlices.arrUint16[sliceIdx][rowIndex] case types.T_int32: @@ -316,6 +316,7 @@ type ColumnSlices struct { arrBlockid [][]types.Blockid arrTS [][]types.TS arrEnum [][]types.Enum + arrMoYear [][]types.MoYear safeRefSlice bool } @@ -350,6 +351,7 @@ func (slices *ColumnSlices) Close() { slices.arrBlockid = nil slices.arrTS = nil slices.arrEnum = nil + slices.arrMoYear = nil slices.safeRefSlice = false } @@ -415,8 +417,10 @@ func (slices *ColumnSlices) GetUint64(r uint64, i uint64) (uint64, error) { return uint64(slices.arrInt8[sliceIdx][r]), nil case types.T_uint8: return uint64(slices.arrUint8[sliceIdx][r]), nil - case types.T_int16, types.T_year: + case types.T_int16: return uint64(slices.arrInt16[sliceIdx][r]), nil + case types.T_year: + return uint64(slices.arrMoYear[sliceIdx][r]), nil case types.T_uint16: return uint64(slices.arrUint16[sliceIdx][r]), nil case types.T_int32: @@ -447,8 +451,10 @@ func (slices *ColumnSlices) GetInt64(r uint64, i uint64) (int64, error) { return int64(slices.arrInt8[sliceIdx][r]), nil case types.T_uint8: return int64(slices.arrUint8[sliceIdx][r]), nil - case types.T_int16, types.T_year: + case types.T_int16: return int64(slices.arrInt16[sliceIdx][r]), nil + case types.T_year: + return int64(slices.arrMoYear[sliceIdx][r]), nil case types.T_uint16: return int64(slices.arrUint16[sliceIdx][r]), nil case types.T_int32: @@ -742,8 +748,8 @@ func convertVectorToSlice(ctx context.Context, ses FeSession, vec *vector.Vector colSlices.colIdx2SliceIdx[i] = len(colSlices.arrInt16) colSlices.arrInt16 = append(colSlices.arrInt16, vector.ToSliceNoTypeCheck2[int16](vec)) case types.T_year: - colSlices.colIdx2SliceIdx[i] = len(colSlices.arrInt16) - colSlices.arrInt16 = append(colSlices.arrInt16, vector.ToSliceNoTypeCheck2[int16](vec)) + colSlices.colIdx2SliceIdx[i] = len(colSlices.arrMoYear) + colSlices.arrMoYear = append(colSlices.arrMoYear, vector.ToSliceNoTypeCheck2[types.MoYear](vec)) case types.T_uint16: colSlices.colIdx2SliceIdx[i] = len(colSlices.arrUint16) colSlices.arrUint16 = append(colSlices.arrUint16, vector.ToSliceNoTypeCheck2[uint16](vec)) diff --git a/pkg/frontend/predefined.go b/pkg/frontend/predefined.go index 96026a01ce7c2..5753930e61284 100644 --- a/pkg/frontend/predefined.go +++ b/pkg/frontend/predefined.go @@ -386,7 +386,7 @@ var ( clone_ts bigint signed not null comment 'branch creation timestamp in nanoseconds', p_table_id bigint unsigned not null comment 'id of the parent table this branch is based on', creator bigint unsigned not null comment 'account id of the creator', - level varchar not null, + level varchar(10) not null, table_deleted bool not null default false, index(p_table_id), index(creator), diff --git a/pkg/frontend/show_account.go b/pkg/frontend/show_account.go index c2807c8e6d528..a1a652348562d 100644 --- a/pkg/frontend/show_account.go +++ b/pkg/frontend/show_account.go @@ -576,7 +576,7 @@ func doShowAccounts(ctx context.Context, ses *Session, sa *tree.ShowAccounts) (e } } - specialTableCnt, specialDBCnt, err = getSpecialTableCnt(ctx, bh, accIds) + specialTableCnt, specialDBCnt, err = getSpecialTableCnt(ctx, ses, bh, accIds) t1 := time.Now() v2.TaskShowAccountsGetTableStatsDurationHistogram.Observe(t1.Sub(t0).Seconds()) if err != nil { @@ -726,20 +726,24 @@ func getAccountInfo(ctx context.Context, return rsOfMoAccount, accountIds, err } -func getSpecialTableCnt(ctx context.Context, bh BackgroundExec, accIds [][]int64) (dbCnt, tblCnt int64, err error) { +func getSpecialTableCnt(ctx context.Context, ses *Session, bh BackgroundExec, accIds [][]int64) (dbCnt, tblCnt int64, err error) { for x := range accIds { for y := range accIds[x] { if accIds[x][y] == sysAccountID { continue } - dbCnt, tblCnt, err = getSpecialTableInfo(ctx, bh, accIds[x][y]) + dbCnt, tblCnt, err = getSpecialTableInfo(ctx, ses, bh, accIds[x][y]) return } } return } -func getSpecialTableInfo(ctx context.Context, bh BackgroundExec, accId int64) (dbCnt, tblCnt int64, err error) { +func getSpecialTableInfo(ctx context.Context, ses *Session, bh BackgroundExec, accId int64) (dbCnt, tblCnt int64, err error) { + // Activate target account's catalog so table resolution uses cache. + if err = activateAccountCatalogIfNeeded(ctx, ses, uint32(accId)); err != nil { + return 0, 0, err + } sql := fmt.Sprintf(getSpecialTablesInfoFormat, sysAccountID, sysAccountID) newCtx := defines.AttachAccountId(ctx, uint32(accId)) diff --git a/pkg/frontend/snapshot.go b/pkg/frontend/snapshot.go index 02b2cb9e1de22..bae5e05834b43 100644 --- a/pkg/frontend/snapshot.go +++ b/pkg/frontend/snapshot.go @@ -633,6 +633,7 @@ func doRestoreSnapshot(ctx context.Context, ses *Session, stmt *tree.RestoreSnap return } } + getLogger(ses.GetService()).Debug(fmt.Sprintf("[%s]restore cluster success", snapshotName)) return } @@ -645,6 +646,7 @@ func doRestoreSnapshot(ctx context.Context, ses *Session, stmt *tree.RestoreSnap if err != nil { return stats, err } + return } @@ -653,6 +655,14 @@ func doRestoreSnapshot(ctx context.Context, ses *Session, stmt *tree.RestoreSnap return } + // Activate the source (restore) account's catalog — same rationale as + // restoreAccountUsingClusterSnapshotToNew. + if restoreAccount != toAccountId { + if err = activateAccountCatalogIfNeeded(ctx, ses, restoreAccount); err != nil { + return + } + } + // drop foreign key related tables first if err = deleteCurFkTables(ctx, ses.GetService(), bh, dbName, tblName, toAccountId); err != nil { return @@ -2055,6 +2065,28 @@ func getFkDeps( bh.ClearExecResultSet() if err = bh.Exec(newCtx, sql); err != nil { + // getFkDeps is called exclusively from the restore flow + // (deleteCurFkTables and fkTablesTopoSort). The source or + // target account's catalog data may not be fully available: + // + // - ErrNoSuchTable / ErrBadDB: dropped-account data compacted. + // - ErrInvalidInput ("column X does not exist"): a transient + // catalog-cache race. Push logtail delivers mo_tables and + // mo_columns as separate entries; during DCA flush or between + // the two push entries, InsertTable creates a cache item with + // nil/partial column definitions. A concurrent reader can + // then observe a table whose columns are incomplete. + // + // Returning empty FK deps is safe: restore drops all tables + // (DROP TABLE IF EXISTS) so ordering is best-effort, and tables + // are recreated from the snapshot with correct FK constraints. + if moerr.IsMoErrCode(err, moerr.ErrNoSuchTable) || + moerr.IsMoErrCode(err, moerr.ErrBadDB) || + moerr.IsMoErrCode(err, moerr.ErrInvalidInput) || + moerr.IsMoErrCode(err, moerr.ErrParseError) { + getLogger("").Warn(fmt.Sprintf("FK dep query failed (source catalog unavailable), proceeding without FK ordering: %v", err)) + return make(map[string][]string), nil + } return } @@ -2431,6 +2463,19 @@ func restoreAccountUsingClusterSnapshotToNew(ctx context.Context, return } + // Activate the source (from) account's catalog so restore queries + // (SHOW DATABASES, SHOW TABLES, CLONE TABLE, etc.) can discover the + // from-account's databases and tables in the partition state. + // If TN checkpoint has compacted the dropped account's data, the + // activation will return empty data — the restore will find no + // databases/tables and effectively do nothing for that account. + // The downstream FK and table-resolution code handles that gracefully. + if uint64(fromAccount) != toAccountId { + if err = activateAccountCatalogIfNeeded(ctx, ses, uint32(fromAccount)); err != nil { + return + } + } + // drop foreign key related tables first if isNeedToCleanToDatabase { if err = deleteCurFkTables(ctx, ses.GetService(), bh, "", "", uint32(toAccountId)); err != nil { diff --git a/pkg/frontend/snapshot_restore_with_ts.go b/pkg/frontend/snapshot_restore_with_ts.go index e591972501fb8..86363fbe222db 100644 --- a/pkg/frontend/snapshot_restore_with_ts.go +++ b/pkg/frontend/snapshot_restore_with_ts.go @@ -69,6 +69,17 @@ func getFkDepsWithTS(ctx context.Context, bh BackgroundExec, db string, tbl stri bh.ClearExecResultSet() if err = bh.ExecRestore(ctx, sql, from, to); err != nil { + // With lazy catalog the source account's catalog data may not be + // available (e.g. a dropped account whose data was compacted by a + // TN checkpoint). Treat table-resolution errors as "no FK deps" + // instead of aborting the entire restore — the CLONE TABLE path + // already bypasses FK constraint checks via the IsRestore flag. + if moerr.IsMoErrCode(err, moerr.ErrNoSuchTable) || + moerr.IsMoErrCode(err, moerr.ErrBadDB) || + moerr.IsMoErrCode(err, moerr.ErrParseError) { + getLogger("").Warn(fmt.Sprintf("[%d:%d] FK dep query failed (source catalog unavailable), proceeding without FK ordering: %v", from, ts, err)) + return make(map[string][]string), nil + } return } resultSet, err := getResultSet(ctx, bh) diff --git a/pkg/sql/colexec/aggexec/sumavg2.go b/pkg/sql/colexec/aggexec/sumavg2.go index fcb8adbfd313a..7d0230f17b91e 100644 --- a/pkg/sql/colexec/aggexec/sumavg2.go +++ b/pkg/sql/colexec/aggexec/sumavg2.go @@ -666,7 +666,7 @@ func makeSumAvgExec( case types.T_int16: return newSumAvgExec[int64, int16](mp, int64OfCheck, isSum, aggID, isDistinct, param) case types.T_year: - return newSumAvgExec[int64, int16](mp, int64OfCheck, isSum, aggID, isDistinct, param) + return newSumAvgExec[int64, types.MoYear](mp, int64OfCheck, isSum, aggID, isDistinct, param) case types.T_int32: return newSumAvgExec[int64, int32](mp, int64OfCheck, isSum, aggID, isDistinct, param) case types.T_int64: diff --git a/pkg/sql/plan/function/ctl/cmd_activate_tenant_catalog.go b/pkg/sql/plan/function/ctl/cmd_activate_tenant_catalog.go new file mode 100644 index 0000000000000..23f5b2ddd320f --- /dev/null +++ b/pkg/sql/plan/function/ctl/cmd_activate_tenant_catalog.go @@ -0,0 +1,76 @@ +// Copyright 2024 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package ctl + +import ( + "fmt" + "strconv" + "strings" + + "github.com/matrixorigin/matrixone/pkg/common/moerr" + "github.com/matrixorigin/matrixone/pkg/defines" + "github.com/matrixorigin/matrixone/pkg/vm/engine" + "github.com/matrixorigin/matrixone/pkg/vm/process" +) + +// handleActivateTenantCatalog activates the catalog cache for a specific +// tenant account on the current CN. This is only callable by the sys account. +// +// Usage: select mo_ctl('cn', 'ActivateTenantCatalog', ''); +func handleActivateTenantCatalog( + proc *process.Process, + service serviceType, + parameter string, + _ requestSender, +) (Result, error) { + if service != cn { + return Result{}, moerr.NewNotSupportedf( + proc.Ctx, "ActivateTenantCatalog only supports CN service") + } + + callerID, err := defines.GetAccountId(proc.Ctx) + if err != nil { + return Result{}, err + } + if callerID != 0 { + return Result{}, moerr.NewInternalError( + proc.Ctx, "ActivateTenantCatalog can only be called by sys account") + } + + targetID, err := strconv.ParseUint(strings.TrimSpace(parameter), 10, 32) + if err != nil { + return Result{}, moerr.NewInvalidInput( + proc.Ctx, "invalid account id: "+parameter) + } + + eng := proc.GetSessionInfo().StorageEngine + activator, ok := eng.(engine.TenantCatalogActivator) + if !ok { + // Engine does not support lazy catalog (e.g. unit-test engine). + return Result{ + Method: ActivateTenantCatalogMethod, + Data: "not supported by engine, skipped", + }, nil + } + + if err = activator.ActivateTenantCatalog(proc.Ctx, uint32(targetID)); err != nil { + return Result{}, err + } + + return Result{ + Method: ActivateTenantCatalogMethod, + Data: fmt.Sprintf("account %d activated on this CN", targetID), + }, nil +} diff --git a/pkg/sql/plan/function/ctl/types.go b/pkg/sql/plan/function/ctl/types.go index 82b55394ed4ce..77365b1d44e3f 100644 --- a/pkg/sql/plan/function/ctl/types.go +++ b/pkg/sql/plan/function/ctl/types.go @@ -59,7 +59,8 @@ var ( MoTableStats = strings.ToUpper("MoTableStats") WorkspaceThreshold = strings.ToUpper("WorkspaceThreshold") TableExtra = strings.ToUpper("table-extra") - PrefetchOnSubscribed = strings.ToUpper("prefetch-on-subscribed") + PrefetchOnSubscribed = strings.ToUpper("prefetch-on-subscribed") + ActivateTenantCatalogMethod = strings.ToUpper("ActivateTenantCatalog") ) var ( @@ -103,7 +104,8 @@ var ( MoTableStats: handleMoTableStats, WorkspaceThreshold: handleWorkspaceThreshold, TableExtra: handleTableExtra, - PrefetchOnSubscribed: handlePrefetchOnSubscribed, + PrefetchOnSubscribed: handlePrefetchOnSubscribed, + ActivateTenantCatalogMethod: handleActivateTenantCatalog, } ) diff --git a/pkg/sql/plan/make.go b/pkg/sql/plan/make.go index aea10de38b0bc..6813b9c8a5849 100644 --- a/pkg/sql/plan/make.go +++ b/pkg/sql/plan/make.go @@ -826,12 +826,13 @@ func isSameColumnType(t1 Type, t2 Type) bool { } // GetColDefFromTable Find the target column definition from the predefined -// table columns and return its deep copy +// table columns and return its deep copy. Returns nil if the column is not +// found (e.g. incomplete catalog data for a dropped account). func GetColDefFromTable(Cols []*ColDef, hidenColName string) *ColDef { for _, coldef := range Cols { if coldef.Name == hidenColName { return DeepCopyColDef(coldef) } } - panic("Unable to find target column from predefined table columns") + return nil } diff --git a/pkg/vm/engine/disttae/cache/catalog.go b/pkg/vm/engine/disttae/cache/catalog.go index 28566cbc39841..284aa1d247e12 100644 --- a/pkg/vm/engine/disttae/cache/catalog.go +++ b/pkg/vm/engine/disttae/cache/catalog.go @@ -15,6 +15,8 @@ package cache import ( + "bytes" + "fmt" "sort" "strings" "sync" @@ -395,12 +397,17 @@ func (cc *CatalogCache) HasNewerVersion(qry *TableChangeQuery) bool { } if item.Ts.Greater(qry.Ts) { - if item.deleted || item.Id != qry.TableId || item.Version < qry.Version { + if item.deleted { + find = true + } else if item.Id != qry.TableId { + find = true + } else if item.Version < qry.Version { find = true } } return false }) + return find } @@ -429,6 +436,9 @@ func (cc *CatalogCache) DeleteTable(bat *batch.Batch) { for i, ts := range timestamps { pk := cpks.GetBytesAt(i) cc.tables.cpkeyIndex.Ascend(&TableItem{CPKey: pk, Ts: ts.ToTimestamp()}, func(item *TableItem) bool { + if !bytes.Equal(item.CPKey, pk) { + return false + } newItem := &TableItem{ deleted: true, Id: item.Id, @@ -450,6 +460,9 @@ func (cc *CatalogCache) DeleteDatabase(bat *batch.Batch) { for i, ts := range timestamps { pk := cpks.GetBytesAt(i) cc.databases.cpkeyIndex.Ascend(&DatabaseItem{CPKey: pk, Ts: ts.ToTimestamp()}, func(item *DatabaseItem) bool { + if !bytes.Equal(item.CPKey, pk) { + return false + } newItem := &DatabaseItem{ deleted: true, Id: item.Id, @@ -580,6 +593,36 @@ func ParseColumnsBatchAnd(bat *batch.Batch, f func(map[TableItemKey]Columns)) { func InitTableItemWithColumns(item *TableItem, cols Columns) { sort.Sort(cols) + // Detection-only: log duplicate column names WITHOUT removing them so the + // downstream "ambiguous column" error is still exposed for root-cause tracing. + if len(cols) > 1 { + seen := make(map[string]int, len(cols)) + var duplicates []string + for _, col := range cols { + seen[col.Name]++ + if seen[col.Name] == 2 { + duplicates = append(duplicates, col.Name) + } + } + if len(duplicates) > 0 { + colDetails := make([]string, 0, len(cols)) + for _, col := range cols { + colDetails = append(colDetails, fmt.Sprintf( + "%s(seqnum=%d,num=%d)", col.Name, col.Seqnum, col.Num)) + } + logutil.Error("catalog-cache: DUPLICATE COLUMNS DETECTED in table definition", + zap.Uint32("account-id", item.AccountId), + zap.Uint64("database-id", item.DatabaseId), + zap.Uint64("table-id", item.Id), + zap.String("table-name", item.Name), + zap.String("ts", item.Ts.String()), + zap.Int("total-columns", len(cols)), + zap.Strings("duplicate-names", duplicates), + zap.Strings("all-columns", colDetails), + zap.Stack("stack"), + ) + } + } coldefs := make([]engine.TableDef, 0, len(cols)) for i, col := range cols { if col.ConstraintType == catalog.SystemColPKConstraint { @@ -598,6 +641,16 @@ func (cc *CatalogCache) InsertColumns(bat *batch.Batch) { ParseColumnsBatchAnd(bat, func(mp map[TableItemKey]Columns) { queryKey := new(TableItem) for k, cols := range mp { + // Diagnostic: log column insertion details per table for duplication tracing. + ts := k.Ts.toTs() + logutil.Info("catalog-cache.InsertColumns", + zap.Uint32("account-id", k.AccountId), + zap.Uint64("database-id", k.DatabaseId), + zap.Uint64("table-id", k.Id), + zap.String("table-name", k.Name), + zap.String("ts", ts.String()), + zap.Int("column-count", len(cols)), + ) queryKey.Name = k.Name queryKey.AccountId = k.AccountId queryKey.DatabaseId = k.DatabaseId @@ -611,7 +664,18 @@ func (cc *CatalogCache) InsertColumns(bat *batch.Batch) { ) continue } - InitTableItemWithColumns(item, cols) + // Copy-on-write: create a new item and populate its columns + // instead of mutating the existing BTree item in-place. This + // eliminates the data race where a concurrent GetTable reader + // (holding only the BTree read lock, not catalogCacheMu) could + // observe a partially-written Defs slice during in-place + // mutation. After Set, readers atomically see either the old + // item (nil Defs) or the new item (full Defs). + newItem := new(TableItem) + *newItem = *item + InitTableItemWithColumns(newItem, cols) + cc.tables.data.Set(newItem) + cc.tables.cpkeyIndex.Set(newItem) } }) } diff --git a/pkg/vm/engine/disttae/logtail.go b/pkg/vm/engine/disttae/logtail.go index 54386e9b6a8a0..2f10361aaa742 100644 --- a/pkg/vm/engine/disttae/logtail.go +++ b/pkg/vm/engine/disttae/logtail.go @@ -18,6 +18,9 @@ import ( "context" "time" + "go.uber.org/zap" + + "github.com/matrixorigin/matrixone/pkg/logutil" v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2" "github.com/matrixorigin/matrixone/pkg/vm/engine/disttae/cache" @@ -36,6 +39,7 @@ func consumeEntry( state *logtailreplay.PartitionState, e *api.Entry, isSub bool, + skipCatalogCache bool, ) error { // for test only. if engine.skipConsume { @@ -61,6 +65,13 @@ func consumeEntry( return nil } + // Activation response data should only populate PartitionState; the + // catalog cache is built later by replayCatalogCacheAt. Skip all + // catalog-cache operations (both global DCA and per-account DCA). + if skipCatalogCache { + return nil + } + if engine.PushClient().dcaTryDelay(isSub, func() { applyToCatalogCache(cache, e) }) { return nil } @@ -72,6 +83,11 @@ func consumeEntry( return err } if shouldDelay && lc.delayAccountCacheApply(accountID, func() { applyToCatalogCache(cache, e) }) { + logutil.Warn("DIAG-consumeEntry per-account DCA delayed", + zap.Uint32("delayedAccount", accountID), + zap.Uint64("tableId", e.TableId), + zap.String("entryType", e.EntryType.String()), + ) return nil } } diff --git a/pkg/vm/engine/disttae/logtail_consumer.go b/pkg/vm/engine/disttae/logtail_consumer.go index 906ed87c92f8d..19be52c98855a 100644 --- a/pkg/vm/engine/disttae/logtail_consumer.go +++ b/pkg/vm/engine/disttae/logtail_consumer.go @@ -31,6 +31,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/common/morpc" + "github.com/matrixorigin/matrixone/pkg/defines" "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/logutil" @@ -85,8 +86,18 @@ const ( defaultRPCReadTimeout = time.Second * 30 logTag = "[logtail-consumer]" + + // maxActivationReconnectRetries is the number of times ActivateTenantCatalog + // retries when the in-flight request is interrupted by a reconnect cycle. + maxActivationReconnectRetries = 4 ) +// errActivationInterruptedByReconnect is a sentinel error returned by +// doActivateTenantCatalog when the pending response channel receives nil, +// which means a reconnect cycle cleared it. The outer ActivateTenantCatalog +// loop retries on this specific error. +var errActivationInterruptedByReconnect = errors.New("tenant catalog activation interrupted by reconnect") + type SubscribeState int32 const ( @@ -733,13 +744,30 @@ func (c *PushClient) handleActivationResponse( ) { for i := range resp.Tails { tail := resp.Tails[i] - if err := updatePartitionOfPush(ctx, e, &tail, true, receiveAt, false); err != nil { + // Diagnostic: log per-tail entry/row details to detect if TN sends duplicate data + totalRows := 0 + for j := range tail.Commands { + if tail.Commands[j].Bat != nil && len(tail.Commands[j].Bat.Vecs) > 0 { + totalRows += int(tail.Commands[j].Bat.Vecs[0].Len) + } + } + logutil.Info("logtail.consumer.activation.response.tail", + zap.Uint32("account-id", resp.AccountId), + zap.Uint64("seq", resp.Seq), + zap.Uint64("table-id", tail.Table.TbId), + zap.String("table-name", tail.Table.TbName), + zap.Int("entries", len(tail.Commands)), + zap.Int("total-rows", totalRows), + zap.String("ckp-location", tail.CkpLocation), + ) + if err := updatePartitionOfPush(ctx, e, &tail, true, receiveAt, false, true); err != nil { logutil.Error("logtail.consumer.activation.apply.tail.failed", zap.Uint32("account-id", resp.AccountId), zap.Uint64("seq", resp.Seq), zap.Error(err), ) } + } if c.lazyCatalog != nil { @@ -869,6 +897,10 @@ func (c *PushClient) replayCatalogCache(ctx context.Context, e *Engine) (err err if err = c.replayCatalogCacheAt(ctx, e, ts, []uint32{0}); err != nil { return err } + ccache := e.catalog.Load() + c.applyCatalogCacheChange(func() { + ccache.UpdateDuration(types.TimestampToTS(ts), types.MaxTs()) + }) c.dcaConfirmAndApply() if c.lazyCatalog != nil { c.lazyCatalog.setAccountReady(0, ts) @@ -911,6 +943,10 @@ func (c *PushClient) replayCatalogCacheForReconnect( if err := c.replayCatalogCacheAt(ctx, e, ts, accountIDs); err != nil { return err } + ccache := e.catalog.Load() + c.applyCatalogCacheChange(func() { + ccache.UpdateDuration(types.TimestampToTS(ts), types.MaxTs()) + }) c.dcaConfirmAndApply() if c.lazyCatalog != nil { @@ -950,6 +986,13 @@ func (c *PushClient) CanServeAccount(accountID uint32, ts timestamp.Timestamp) b func (c *PushClient) replayCatalogCacheAt( ctx context.Context, e *Engine, ts timestamp.Timestamp, accountIDs []uint32, ) (err error) { + // The replay SQL reads shared system tables (mo_database, mo_tables, + // mo_columns) with an explicit account_id filter. It must always run + // under the sys account (TenantIDKey=0) so that table resolution goes + // through the always-ready sys catalog, not the not-yet-ready tenant + // catalog that we are in the process of activating. + ctx = context.WithValue(ctx, defines.TenantIDKey{}, uint32(0)) + var op client.TxnOperator ccache := e.catalog.Load() typeTs := types.TimestampToTS(ts) @@ -987,21 +1030,52 @@ func (c *PushClient) replayCatalogCacheAt( return err } - c.applyCatalogCacheChange(func() { - ccache.UpdateDuration(typeTs, types.MaxTs()) - }) return nil - } // ActivateTenantCatalog ensures the catalog cache for the given account is // ready. If the account is already ready, returns immediately. Otherwise it // sends an activation request to TN, waits for the response, replays the // catalog from storage, and marks the account ready. +// +// If an in-flight activation is interrupted by a reconnect cycle, the +// method retries up to maxActivationReconnectRetries times with exponential +// backoff (500ms, 1s, 2s, 4s) so that a transient reconnect does not +// surface as a frontend error. func (c *PushClient) ActivateTenantCatalog(ctx context.Context, e *Engine, accountID uint32) error { if c.lazyCatalog == nil || !c.lazyCatalog.isEnabled() { return nil } + + // Backoff schedule: 500ms, 1s, 2s, 4s (total ~7.5s worst case). + backoff := 500 * time.Millisecond + + for attempt := 0; ; attempt++ { + err := c.tryActivateTenantCatalog(ctx, e, accountID) + if err == nil { + return nil + } + if !errors.Is(err, errActivationInterruptedByReconnect) || + attempt >= maxActivationReconnectRetries { + return err + } + logutil.Info("logtail.consumer.activation.reconnect.retry", + zap.Uint32("account-id", accountID), + zap.Int("attempt", attempt+1), + zap.Duration("backoff", backoff), + ) + select { + case <-ctx.Done(): + return ctx.Err() + case <-time.After(backoff): + } + backoff *= 2 + } +} + +// tryActivateTenantCatalog performs a single activation attempt with inflight +// deduplication. +func (c *PushClient) tryActivateTenantCatalog(ctx context.Context, e *Engine, accountID uint32) error { if c.lazyCatalog.isAccountReady(accountID) { return nil } @@ -1075,9 +1149,7 @@ func (c *PushClient) doActivateTenantCatalog(ctx context.Context, e *Engine, acc case resp = <-respCh: } if resp == nil { - err := moerr.NewInternalErrorNoCtx("tenant catalog activation interrupted by reconnect") - record("wait_response", "interrupted", err, nil, nil, 0) - return err + return fail("wait_response", errActivationInterruptedByReconnect, nil) } targetTS := resp.GetTargetTs() @@ -1110,6 +1182,11 @@ func (c *PushClient) doActivateTenantCatalog(ctx context.Context, e *Engine, acc c.applyCatalogCacheChange(func() { fns := c.lazyCatalog.beginAccountReadyTransition(accountID) delayedApplyCount = len(fns) + logutil.Info("logtail.consumer.activation.dca.flush", + zap.Uint32("account-id", accountID), + zap.Uint64("seq", seq), + zap.Int("delayed-apply-count", delayedApplyCount), + ) for _, f := range fns { f() } @@ -2526,7 +2603,7 @@ func (e *Engine) consumeSubscribeResponse( lazyLoad bool, receiveAt time.Time) error { lt := rp.GetLogtail() - return updatePartitionOfPush(ctx, e, <, lazyLoad, receiveAt, true) + return updatePartitionOfPush(ctx, e, <, lazyLoad, receiveAt, true, false) } func (e *Engine) consumeUpdateLogTail( @@ -2534,7 +2611,7 @@ func (e *Engine) consumeUpdateLogTail( rp logtail.TableLogtail, lazyLoad bool, receiveAt time.Time) error { - return updatePartitionOfPush(ctx, e, &rp, lazyLoad, receiveAt, false) + return updatePartitionOfPush(ctx, e, &rp, lazyLoad, receiveAt, false, false) } // updatePartitionOfPush is the partition update method of log tail push model. @@ -2544,7 +2621,8 @@ func updatePartitionOfPush( tl *logtail.TableLogtail, lazyLoad bool, receiveAt time.Time, - isSub bool) (err error) { + isSub bool, + skipCatalogCache bool) (err error) { start := time.Now() v2.LogTailApplyLatencyDurationHistogram.Observe(start.Sub(receiveAt).Seconds()) defer func() { @@ -2603,6 +2681,7 @@ func updatePartitionOfPush( state, tl, isSub, + skipCatalogCache, ) v2.LogtailUpdatePartitonConsumeLogtailDurationHistogram.Observe(time.Since(t0).Seconds()) @@ -2614,7 +2693,7 @@ func updatePartitionOfPush( v2.LogtailUpdatePartitonHandleCheckpointDurationHistogram.Observe(time.Since(t0).Seconds()) } t0 = time.Now() - err = consumeCkpsAndLogTail(ctx, partition.TableInfo.PrimarySeqnum, e, state, tl, dbId, tblId, isSub) + err = consumeCkpsAndLogTail(ctx, partition.TableInfo.PrimarySeqnum, e, state, tl, dbId, tblId, isSub, skipCatalogCache) v2.LogtailUpdatePartitonConsumeLogtailDurationHistogram.Observe(time.Since(t0).Seconds()) } @@ -2666,6 +2745,7 @@ func consumeLogTail( state *logtailreplay.PartitionState, lt *logtail.TableLogtail, isSub bool, + skipCatalogCache bool, ) error { // return hackConsumeLogtail(ctx, primarySeqnum, engine, state, lt) if lt.Table.DbName == "" { @@ -2674,7 +2754,7 @@ func consumeLogTail( t0 := time.Now() for i := 0; i < len(lt.Commands); i++ { if err := consumeEntry(ctx, primarySeqnum, - engine, engine.GetLatestCatalogCache(), state, <.Commands[i], isSub); err != nil { + engine, engine.GetLatestCatalogCache(), state, <.Commands[i], isSub, skipCatalogCache); err != nil { return err } } @@ -2714,6 +2794,7 @@ func consumeCkpsAndLogTail( databaseId uint64, tableId uint64, isSub bool, + skipCatalogCache bool, ) (err error) { var closeCBs []func() if err = taeLogtail.ConsumeCheckpointEntries( @@ -2733,5 +2814,5 @@ func consumeCkpsAndLogTail( } } }() - return consumeLogTail(ctx, primarySeqnum, engine, state, lt, isSub) + return consumeLogTail(ctx, primarySeqnum, engine, state, lt, isSub, skipCatalogCache) } diff --git a/pkg/vm/engine/disttae/logtailreplay/partition.go b/pkg/vm/engine/disttae/logtailreplay/partition.go index 4294e82c41d07..500f77b04372d 100644 --- a/pkg/vm/engine/disttae/logtailreplay/partition.go +++ b/pkg/vm/engine/disttae/logtailreplay/partition.go @@ -126,6 +126,10 @@ func (p *Partition) Snapshot() *PartitionState { return p.state.Load() } +func (p *Partition) IsCheckpointConsumed() bool { + return p.checkpointConsumed.Load() +} + func (*Partition) CheckPoint(ctx context.Context, ts timestamp.Timestamp) error { panic("unimplemented") } diff --git a/pkg/vm/engine/disttae/logtailreplay/partition_state.go b/pkg/vm/engine/disttae/logtailreplay/partition_state.go index 42aa031c7626d..b05e03b3af319 100644 --- a/pkg/vm/engine/disttae/logtailreplay/partition_state.go +++ b/pkg/vm/engine/disttae/logtailreplay/partition_state.go @@ -167,16 +167,15 @@ func (p *PartitionState) handleDataObjectEntry( old, exist := p.dataObjectsNameIndex.Get(objEntry) if exist { - // why check the deleteTime here? consider this situation: - // 1. insert on an object, then these insert operations recorded into a CKP. - // 2. and delete this object, this operation recorded into WAL. - // 3. restart - // 4. replay CKP(lazily) into partition state --> replay WAL into partition state - // the delete record in WAL could be overwritten by insert record in CKP, - // causing logic err of the objects' visibility(dead object back to life!!). + // Dead-object protection — see the detailed comment in + // HandleDataObjectList for the full explanation. // - // if this happened, just skip this object will be fine, - if !old.DeleteTime.IsEmpty() { + // Only skip when the incoming entry is a CREATE (DeleteTime empty) + // and the existing entry is already dead (DeleteTime non-empty). + // When both are DELETEs, let it proceed so GC can clean up btree + // rows that may have been inserted between two arrivals of the + // same sealed-appendable DELETE (push-then-activation race). + if !old.DeleteTime.IsEmpty() && objEntry.DeleteTime.IsEmpty() { return } } else { @@ -244,27 +243,8 @@ func (p *PartitionState) handleDataObjectEntry( } numDeleted++ } - - //it's tricky here. - //Due to consuming lazily the checkpoint, - //we have to take the following scenario into account: - //1. CN receives deletes for a non-appendable block from the log tail, - // then apply the deletes into PartitionState.rows. - //2. CN receives block meta of the above non-appendable block to be inserted - // from the checkpoint, then apply the block meta into PartitionState.blocks. - // So , if the above scenario happens, we need to set the non-appendable block into - // PartitionState.dirtyBlocks. - //if !objEntry.EntryState && !objEntry.HasDeltaLoc { - // p.dirtyBlocks.Set(entry.BlockID) - // break - //} } iter.Release() - - // if there are no rows for the block, delete the block from the dirty - //if objEntry.EntryState && scanCnt == blockDeleted && p.dirtyBlocks.Len() > 0 { - // p.dirtyBlocks.Delete(*blkID) - //} } p.prefetchObject(fs, objEntry) @@ -451,16 +431,30 @@ func (p *PartitionState) HandleDataObjectList( old, exist := p.dataObjectsNameIndex.Get(objEntry) if exist { - // why check the deleteTime here? consider this situation: - // 1. insert on an object, then these insert operations recorded into a CKP. - // 2. and delete this object, this operation recorded into WAL. - // 3. restart - // 4. replay CKP(lazily) into partition state --> replay WAL into partition state - // the delete record in WAL could be overwritten by insert record in CKP, - // causing logic err of the objects' visibility(dead object back to life!!). + // Dead-object protection: when this object already exists in nameIndex + // with a non-empty DeleteTime (i.e. it's already dead), we must prevent + // a CREATE record from overwriting it. // - // if this happened, just skip this object will be fine, - if !old.DeleteTime.IsEmpty() { + // This guards the CKP/WAL replay scenario: + // 1. An INSERT on this object is recorded into a CKP. + // 2. A DELETE on this object is recorded into WAL. + // 3. On restart, CKP is replayed first, then WAL. + // 4. Without this guard, the CKP's CREATE would overwrite the + // WAL's DELETE, bringing a dead object back to life. + // + // We only skip when the incoming entry is a CREATE (DeleteTime empty). + // When it is also a DELETE (DeleteTime non-empty), we let it proceed + // so that the GC loop below can run. This matters because the same + // sealed-appendable DELETE can arrive twice on CN: + // - First via a regular push (GC is a no-op because the btree is + // still empty at that point). + // - Then inside the activation response, by which time the + // activation's INSERT entries have already populated the btree. + // The second GC pass must run to clean up those rows; skipping it + // would leave duplicate rows and cause "ambiguous column" errors. + // Index updates (nameIndex, TSIndex) are idempotent for the same + // data, so letting them execute again is harmless. + if !old.DeleteTime.IsEmpty() && objEntry.DeleteTime.IsEmpty() { continue } } else { @@ -486,7 +480,7 @@ func (p *PartitionState) HandleDataObjectList( p.dataObjectTSIndex.Set(e) } - // for appendable object, gc rows when delete object + // For appendable objects, gc in-memory rows when the object is sealed/deleted. iter := p.rows.Copy().Iter() defer iter.Release() objID := objEntry.ObjectStats.ObjectName().ObjectId() @@ -530,26 +524,7 @@ func (p *PartitionState) HandleDataObjectList( blockDeleted++ } } - - //it's tricky here. - //Due to consuming lazily the checkpoint, - //we have to take the following scenario into account: - //1. CN receives deletes for a non-appendable block from the log tail, - // then apply the deletes into PartitionState.rows. - //2. CN receives block meta of the above non-appendable block to be inserted - // from the checkpoint, then apply the block meta into PartitionState.blocks. - // So , if the above scenario happens, we need to set the non-appendable block into - // PartitionState.dirtyBlocks. - //if !objEntry.EntryState && !objEntry.HasDeltaLoc { - // p.dirtyBlocks.Set(entry.BlockID) - // break - //} } - - // if there are no rows for the block, delete the block from the dirty - //if objEntry.EntryState && scanCnt == blockDeleted && p.dirtyBlocks.Len() > 0 { - // p.dirtyBlocks.Delete(*blkID) - //} } p.prefetchObject(fs, objEntry) diff --git a/pkg/vm/engine/disttae/logtailreplay/partition_state_test.go b/pkg/vm/engine/disttae/logtailreplay/partition_state_test.go index 3fa0fb9d894c4..fadc6348e33b1 100644 --- a/pkg/vm/engine/disttae/logtailreplay/partition_state_test.go +++ b/pkg/vm/engine/disttae/logtailreplay/partition_state_test.go @@ -2901,3 +2901,114 @@ func TestCountRows_VisibleAppendableDataObjects(t *testing.T) { require.Error(t, err, "CountRows must return error when CollectDataStats fails") }) } + +// TestHandleDataObjectEntry_DuplicateDeleteRunsGC verifies the fix for the +// push-vs-activation race condition where a sealed appendable's DELETE +// DataObject arrives twice on CN — first via a regular push (when the btree +// is still empty, so GC is a no-op) and then inside the activation response +// (when btree rows have been inserted in between by the activation's INSERT +// entries). Before the fix, the dead-object protection `continue` at the +// second DELETE skipped GC entirely, leaving duplicate btree rows. +func TestHandleDataObjectEntry_DuplicateDeleteRunsGC(t *testing.T) { + ctx := context.Background() + state := NewPartitionState("", true, 42, false) + + // Build an appendable ObjectEntry with BlkCnt=1, Size>0 so it is + // not filtered out by the early-skip checks. + objID := objectio.NewObjectid() + stats := objectio.NewObjectStatsWithObjectID(&objID, true /*appendable*/, false, false) + objectio.SetObjectStatsBlkCnt(stats, 1) + objectio.SetObjectStatsSize(stats, 100) + + createTS := types.BuildTS(10, 0) + deleteTS := types.BuildTS(20, 0) + + sealedEntry := objectio.ObjectEntry{ + ObjectStats: *stats, + CreateTime: createTS, + DeleteTime: deleteTS, + } + + // ── Step 1: push delivers the sealed-appendable DELETE first ── + // At this point the btree is empty, so GC is a no-op. + err := state.handleDataObjectEntry(ctx, nil, sealedEntry) + require.NoError(t, err) + + // Verify the entry is in nameIndex with DeleteTime set. + got, ok := state.dataObjectsNameIndex.Get(sealedEntry) + require.True(t, ok) + require.False(t, got.DeleteTime.IsEmpty()) + + // Verify btree is empty. + require.Equal(t, 0, state.rows.Len(), "btree should be empty before insert") + + // ── Step 2: activation INSERT entries add rows to btree ── + // Simulate rows that belong to this appendable object's block. + blkID := types.NewBlockidWithObjectID(&objID, 0) + numRows := 10 + for i := 0; i < numRows; i++ { + rowID := types.NewRowid(&blkID, uint32(i)) + state.rows.Set(&RowEntry{ + BlockID: blkID, + RowID: rowID, + Time: types.BuildTS(5, uint32(i)), // Time <= createTS, eligible for GC + ID: int64(i), + Deleted: false, + }) + } + require.Equal(t, numRows, state.rows.Len(), "btree should have %d rows after insert", numRows) + + // ── Step 3: activation DataObject for the same sealed appendable arrives ── + // Before the fix: the dead-object protection would `continue`, skipping GC. + // After the fix: GC still runs and cleans up all btree rows. + err = state.handleDataObjectEntry(ctx, nil, sealedEntry) + require.NoError(t, err) + + // All rows with Time <= deleteTS should have been GC'd. + require.Equal(t, 0, state.rows.Len(), + "btree should be empty after second DELETE processing runs GC") +} + +// TestHandleDataObjectEntry_CreateOverDeleteStillBlocked verifies that the +// dead-object protection is preserved: a CREATE (DeleteTime empty) must NOT +// overwrite a DELETE (DeleteTime set) in nameIndex. +func TestHandleDataObjectEntry_CreateOverDeleteStillBlocked(t *testing.T) { + ctx := context.Background() + state := NewPartitionState("", true, 42, false) + + objID := objectio.NewObjectid() + stats := objectio.NewObjectStatsWithObjectID(&objID, false /*non-appendable*/, false, false) + objectio.SetObjectStatsBlkCnt(stats, 1) + objectio.SetObjectStatsSize(stats, 100) + + createTS := types.BuildTS(10, 0) + deleteTS := types.BuildTS(20, 0) + + // First: process the DELETE entry. + deleteEntry := objectio.ObjectEntry{ + ObjectStats: *stats, + CreateTime: createTS, + DeleteTime: deleteTS, + } + err := state.handleDataObjectEntry(ctx, nil, deleteEntry) + require.NoError(t, err) + + got, ok := state.dataObjectsNameIndex.Get(deleteEntry) + require.True(t, ok) + require.False(t, got.DeleteTime.IsEmpty()) + + // Second: process a CREATE entry for the same object (CKP replay scenario). + createEntry := objectio.ObjectEntry{ + ObjectStats: *stats, + CreateTime: createTS, + // DeleteTime is zero — this is a CREATE. + } + err = state.handleDataObjectEntry(ctx, nil, createEntry) + require.NoError(t, err) + + // The DELETE should NOT be overwritten — object stays dead. + got, ok = state.dataObjectsNameIndex.Get(deleteEntry) + require.True(t, ok) + require.False(t, got.DeleteTime.IsEmpty(), + "CREATE must not overwrite DELETE in nameIndex (dead-object protection)") +} diff --git a/pkg/vm/engine/disttae/txn_database.go b/pkg/vm/engine/disttae/txn_database.go index ff2429d3bac74..d4a6e9fb18b68 100644 --- a/pkg/vm/engine/disttae/txn_database.go +++ b/pkg/vm/engine/disttae/txn_database.go @@ -617,7 +617,14 @@ func (db *txnDatabase) loadTableFromStorage( return } if row := res.Batches[0].RowCount(); row != 1 { - panic(fmt.Sprintf("FIND_TABLE loadTableFromStorage failed: table result row cnt: %v, sql : %s", row, tblSql)) + // Zero or multiple rows may happen for dropped accounts whose + // catalog was compacted by a checkpoint or hasn't been fully + // activated yet. Return nil instead of panicking so the caller + // can handle the missing table gracefully. + logutil.Warn("FIND_TABLE loadTableFromStorage unexpected row count", + zap.Int("rows", row), + zap.String("sql", tblSql)) + return } bat := res.Batches[0] @@ -651,12 +658,24 @@ func (db *txnDatabase) loadTableFromStorage( return } } + logutil.Info("FIND_TABLE loadTableFromStorage columns", + zap.String("table", name), + zap.Uint32("accountID", accountID), + zap.Int("batches", len(res.Batches)), + zap.Int("totalRows", bat.RowCount()), + zap.Uint64("tableID", tblid), + ) if err := fillTsVecForSysTableQueryBatch(bat, ts, res.Mp); err != nil { return nil, err } cache.ParseColumnsBatchAnd(bat, func(m map[cache.TableItemKey]cache.Columns) { if len(m) != 1 { - panic(fmt.Sprintf("FIND_TABLE loadTableFromStorage failed: columns touch %d tables", len(m))) + logutil.Warn("FIND_TABLE loadTableFromStorage columns touch unexpected tables", + zap.Int("count", len(m)), + zap.String("table", name)) + // Clear tableitem so the caller sees nil. + tableitem = nil + return } for _, v := range m { cache.InitTableItemWithColumns(tableitem, v) @@ -680,21 +699,37 @@ func (db *txnDatabase) getTableItem( } var err error c := engine.GetLatestCatalogCache() - if ok := c.GetTable(&item); !ok { - var tableitem *cache.TableItem - if !c.CanServe(types.TimestampToTS(db.op.SnapshotTS())) || - !engine.pClient.CanServeAccount(accountID, db.op.SnapshotTS()) { - logutil.Info("FIND_TABLE loadTableFromStorage", zap.String("table", name), zap.Uint32("accountID", accountID), zap.String("txn", db.op.Txn().DebugString()), zap.String("cacheTS", c.GetStartTS().ToString())) - if tableitem, err = db.loadTableFromStorage(ctx, accountID, name); err != nil { - return nil, err - } - } - if tableitem == nil { - return nil, nil + if ok := c.GetTable(&item); ok { + // Guard against a transient catalog-cache window: push logtail + // delivers mo_tables and mo_columns entries separately, each + // applied under a distinct catalogCacheMu acquisition. + // InsertTable creates a BTree item with nil Defs; InsertColumns + // later replaces it with a fully-populated copy (COW). Between + // the two, a concurrent GetTable reader can observe the + // intermediate item whose Defs is nil. Fall through to + // loadTableFromStorage so callers never see a column-less + // table definition. + if item.Defs != nil { + return &item, nil + } + logutil.Warn("FIND_TABLE catalog-cache item has no column defs, falling through to storage", + zap.String("table", name), + zap.Uint32("accountID", accountID), + zap.Uint64("tableID", item.Id), + ) + } + var tableitem *cache.TableItem + if !c.CanServe(types.TimestampToTS(db.op.SnapshotTS())) || + !engine.pClient.CanServeAccount(accountID, db.op.SnapshotTS()) { + logutil.Info("FIND_TABLE loadTableFromStorage", zap.String("table", name), zap.Uint32("accountID", accountID), zap.String("txn", db.op.Txn().DebugString()), zap.String("cacheTS", c.GetStartTS().ToString())) + if tableitem, err = db.loadTableFromStorage(ctx, accountID, name); err != nil { + return nil, err } - return tableitem, nil } - return &item, nil + if tableitem == nil { + return nil, nil + } + return tableitem, nil } // syncLogicalIdIndexInsert synchronizes the logical_id index table for INSERT/UPDATE operations diff --git a/pkg/vm/engine/disttae/txn_table.go b/pkg/vm/engine/disttae/txn_table.go index 51686db1ecefb..3dd7ead0b7409 100644 --- a/pkg/vm/engine/disttae/txn_table.go +++ b/pkg/vm/engine/disttae/txn_table.go @@ -940,8 +940,9 @@ func (tbl *txnTable) doRanges(ctx context.Context, rangesParam engine.RangesPara var part *logtailreplay.PartitionState var uncommittedObjects []objectio.ObjectStats blocks := objectio.PreAllocBlockInfoSlice(rangesParam.PreAllocBlocks) - if rangesParam.Policy&engine.Policy_CollectCommittedInmemData != 0 || - rangesParam.Policy&engine.Policy_CollectUncommittedInmemData != 0 { + hasInmem := rangesParam.Policy&engine.Policy_CollectCommittedInmemData != 0 || + rangesParam.Policy&engine.Policy_CollectUncommittedInmemData != 0 + if hasInmem { blocks.AppendBlockInfo(&objectio.EmptyBlockInfo) } diff --git a/pkg/vm/engine/tae/logtail/service/catalog_filter.go b/pkg/vm/engine/tae/logtail/service/catalog_filter.go index 3f6dca6b1ae08..b848fae9f642c 100644 --- a/pkg/vm/engine/tae/logtail/service/catalog_filter.go +++ b/pkg/vm/engine/tae/logtail/service/catalog_filter.go @@ -27,19 +27,29 @@ import ( var lazyCatalogSubscribeFilterMP = mpool.MustNew("lazy-catalog-subscribe-filter") +// filterLazyCatalogPulledTail filters a pulled logtail for the three lazy +// catalog tables, keeping only rows belonging to allowedAccounts. +// +// When stripObjectMeta is true, object-metadata entries (Entry_DataObject, +// Entry_TombstoneObject) are dropped entirely. This is correct for activation +// responses because partition state already has all object metadata from the +// earlier subscribe + steady-state push; activation only needs the row-level +// catalog entries for the activated account. func filterLazyCatalogPulledTail( tail logtail.TableLogtail, allowedAccounts *lazyCatalogAllowedAccounts, + stripObjectMeta bool, ) (logtail.TableLogtail, func(), error) { if allowedAccounts == nil || !isLazyCatalogTableID(tail.Table) { return tail, nil, nil } - return filterLazyCatalogSubscribeRowsInTail(tail, allowedAccounts) + return filterLazyCatalogSubscribeRowsInTail(tail, allowedAccounts, stripObjectMeta) } func filterLazyCatalogSubscribeRowsInTail( tail logtail.TableLogtail, allowedAccounts *lazyCatalogAllowedAccounts, + stripObjectMeta bool, ) (logtail.TableLogtail, func(), error) { // Subscribe snapshots can still carry mixed-account api.Entry batches for the // three catalog tables, so the TN side must copy only the target rows. @@ -49,12 +59,14 @@ func filterLazyCatalogSubscribeRowsInTail( filtered := tail var closeCBs []func() - // Raw checkpoint locations are not account-filtered. Lazy catalog subscribe - // and activation responses must therefore forward only filtered row data. - filtered.CkpLocation = "" + // Keep CkpLocation as-is. Checkpoints are not account-filtered, but the + // CN needs them to populate partition state (object entries). Account-level + // filtering happens at the catalog-cache replay SQL layer via + // "account_id IN (...)" predicates. Stripping CkpLocation would leave + // partition state empty after a checkpoint, breaking replay queries. filtered.Commands = make([]api.Entry, 0, len(tail.Commands)) for i := range tail.Commands { - entry, keep, closeCB, err := filterLazyCatalogSubscribeEntry(tail.Commands[i], allowedAccounts) + entry, keep, closeCB, err := filterLazyCatalogSubscribeEntry(tail.Commands[i], allowedAccounts, stripObjectMeta) if err != nil { closeCallbacks(closeCB) closeCallbacks(closeCBs...) @@ -110,14 +122,23 @@ func filterLazyCatalogPublishRowsInTail( func filterLazyCatalogSubscribeEntry( entry api.Entry, allowedAccounts *lazyCatalogAllowedAccounts, + stripObjectMeta bool, ) (api.Entry, bool, func(), error) { switch entry.GetEntryType() { case api.Entry_Insert, api.Entry_Update: return filterLazyCatalogSubscribeInsertOrUpdateEntry(entry, allowedAccounts) case api.Entry_Delete: return filterLazyCatalogSubscribeDeleteEntry(entry, allowedAccounts) + case api.Entry_DataObject, api.Entry_TombstoneObject: + // Object metadata entries are table-level (not account-specific). + // For subscribe: keep them so CN can populate partition state. + // For activation: drop them because partition state already has + // all objects from the earlier subscribe + steady-state push. + if stripObjectMeta { + return api.Entry{}, false, nil, nil + } + return entry, true, nil, nil default: - // Object/meta entries are not row-level tenant data; keep them untouched. return entry, true, nil, nil } } diff --git a/pkg/vm/engine/tae/logtail/service/catalog_filter_test.go b/pkg/vm/engine/tae/logtail/service/catalog_filter_test.go index 961fc1fb4a958..a8b1dc00b525b 100644 --- a/pkg/vm/engine/tae/logtail/service/catalog_filter_test.go +++ b/pkg/vm/engine/tae/logtail/service/catalog_filter_test.go @@ -92,17 +92,19 @@ func TestSessionPrepareLazyCatalogPublishWrapsFiltersCatalogRowsByAccount(t *tes func TestFilterLazyCatalogSubscribeRowsInTailCopiesMixedInsertEntry(t *testing.T) { filtered, closeCB, err := filterLazyCatalogSubscribeRowsInTail(logtail.TableLogtail{ Table: &api.TableID{DbId: catalog.MO_CATALOG_ID, TbId: catalog.MO_COLUMNS_ID}, - CkpLocation: "ckp:should-be-stripped", + CkpLocation: "ckp:should-be-preserved", Commands: []api.Entry{ mustCatalogColumnInsertEntry(t, []uint32{0, 10, 20}), }, - }, &lazyCatalogAllowedAccounts{accounts: map[uint32]struct{}{10: {}}}) + }, &lazyCatalogAllowedAccounts{accounts: map[uint32]struct{}{10: {}}}, false) require.NoError(t, err) require.NotNil(t, closeCB) if closeCB != nil { t.Cleanup(closeCB) } - require.Empty(t, filtered.CkpLocation) + // CkpLocation is preserved so the CN can load checkpoint data into + // partition state; account filtering happens at the SQL replay layer. + require.Equal(t, "ckp:should-be-preserved", filtered.CkpLocation) require.Len(t, filtered.Commands, 1) require.Equal(t, []uint32{10}, mustAccountIDsFromEntry(t, filtered.Commands[0])) } @@ -142,7 +144,7 @@ func TestFilterLazyCatalogSubscribeRowsInTailUsesCPKeyForDelete(t *testing.T) { Commands: []api.Entry{ mustCatalogTableDeleteEntry(t, []uint32{0, 10}), }, - }, &lazyCatalogAllowedAccounts{accounts: map[uint32]struct{}{10: {}}}) + }, &lazyCatalogAllowedAccounts{accounts: map[uint32]struct{}{10: {}}}, false) require.NoError(t, err) require.NotNil(t, closeCB) if closeCB != nil { @@ -152,6 +154,58 @@ func TestFilterLazyCatalogSubscribeRowsInTailUsesCPKeyForDelete(t *testing.T) { require.Equal(t, []uint32{10}, mustCPKeyAccountsFromEntry(t, filtered.Commands[0])) } +func TestFilterLazyCatalogActivationStripsObjectMetaEntries(t *testing.T) { + rowEntry := mustCatalogColumnInsertEntry(t, []uint32{10}) + // Override table ID to mo_tables for this test. + rowEntry.TableId = catalog.MO_TABLES_ID + objEntry := api.Entry{ + EntryType: api.Entry_DataObject, + TableId: catalog.MO_TABLES_ID, + DatabaseId: catalog.MO_CATALOG_ID, + TableName: "mo_tables", + DatabaseName: "mo_catalog", + Bat: &api.Batch{}, + } + tombEntry := api.Entry{ + EntryType: api.Entry_TombstoneObject, + TableId: catalog.MO_TABLES_ID, + DatabaseId: catalog.MO_CATALOG_ID, + TableName: "mo_tables", + DatabaseName: "mo_catalog", + Bat: &api.Batch{}, + } + // With stripObjectMeta=true (activation), object entries should be dropped. + filtered, closeCB, err := filterLazyCatalogSubscribeRowsInTail(logtail.TableLogtail{ + Table: &api.TableID{DbId: catalog.MO_CATALOG_ID, TbId: catalog.MO_TABLES_ID}, + Commands: []api.Entry{ + objEntry, + rowEntry, + tombEntry, + }, + }, &lazyCatalogAllowedAccounts{accounts: map[uint32]struct{}{10: {}}}, true) + require.NoError(t, err) + if closeCB != nil { + t.Cleanup(closeCB) + } + require.Len(t, filtered.Commands, 1, "only row-level entry should survive") + require.Equal(t, api.Entry_Insert, filtered.Commands[0].GetEntryType()) + + // With stripObjectMeta=false (subscribe), object entries should be kept. + filtered2, closeCB2, err := filterLazyCatalogSubscribeRowsInTail(logtail.TableLogtail{ + Table: &api.TableID{DbId: catalog.MO_CATALOG_ID, TbId: catalog.MO_TABLES_ID}, + Commands: []api.Entry{ + objEntry, + rowEntry, + tombEntry, + }, + }, &lazyCatalogAllowedAccounts{accounts: map[uint32]struct{}{10: {}}}, false) + require.NoError(t, err) + if closeCB2 != nil { + t.Cleanup(closeCB2) + } + require.Len(t, filtered2.Commands, 3, "subscribe keeps all entry types") +} + func TestFilterLazyCatalogPublishRowsInTailUsesEntryAccountSummary(t *testing.T) { table := api.TableID{DbId: catalog.MO_CATALOG_ID, TbId: catalog.MO_COLUMNS_ID} entry := api.Entry{ diff --git a/pkg/vm/engine/tae/logtail/service/server.go b/pkg/vm/engine/tae/logtail/service/server.go index 8456100bdd20d..79bc61586c3bd 100644 --- a/pkg/vm/engine/tae/logtail/service/server.go +++ b/pkg/vm/engine/tae/logtail/service/server.go @@ -638,6 +638,8 @@ func (s *LogtailServer) pullActivationPhase1(ctx context.Context, act catalogAct timestamp.Timestamp{}, waterline, allowedAccounts, + false, // activation: keep object meta so HandleDataObjectList GC + // can clean up rows that overlap with concurrent flushes ) mu.Lock() defer mu.Unlock() @@ -683,13 +685,14 @@ func (s *LogtailServer) pullTableLogtail( table api.TableID, from, to timestamp.Timestamp, allowedAccounts *lazyCatalogAllowedAccounts, + stripObjectMeta bool, ) (logtail.TableLogtail, func(), error) { tail, closeCB, err := s.logtailer.TableLogtail(ctx, table, from, to) if err != nil { return logtail.TableLogtail{}, closeCB, err } - filtered, filterCloseCB, err := filterLazyCatalogPulledTail(tail, allowedAccounts) + filtered, filterCloseCB, err := filterLazyCatalogPulledTail(tail, allowedAccounts, stripObjectMeta) if err != nil { closeCallbacks(closeCB, filterCloseCB) return logtail.TableLogtail{}, nil, err @@ -791,7 +794,7 @@ func (s *LogtailServer) sendActivation(ctx context.Context, p1 *catalogActivatio phase1Ts = *p1.tails[i].Ts } - phase2Tail, closeCB, err := s.pullTableLogtail(sendCtx, table, phase1Ts, targetTS, allowedAccounts) + phase2Tail, closeCB, err := s.pullTableLogtail(sendCtx, table, phase1Ts, targetTS, allowedAccounts, false) if err != nil { closeCallbacks(closeCB) s.logger.Error("activation phase2 failed", @@ -868,7 +871,7 @@ func (s *LogtailServer) getSubLogtailPhase( var tail logtail.TableLogtail var closeCB func() moprobe.WithRegion(ctx, moprobe.SubscriptionPullLogTail, func() { - tail, closeCB, subErr = s.pullTableLogtail(sendCtx, table, from, to, allowedAccounts) + tail, closeCB, subErr = s.pullTableLogtail(sendCtx, table, from, to, allowedAccounts, false) subErr = moerr.AttachCause(sendCtx, subErr) }) if subErr != nil { diff --git a/pkg/vm/engine/tae/logtail/service/server_test.go b/pkg/vm/engine/tae/logtail/service/server_test.go index f731852ea3989..bfa2de0123a52 100644 --- a/pkg/vm/engine/tae/logtail/service/server_test.go +++ b/pkg/vm/engine/tae/logtail/service/server_test.go @@ -345,7 +345,7 @@ func TestGetSubLogtailPhaseFiltersLazyCatalogRowsEarly(t *testing.T) { phase.closeCB() } }) - require.Empty(t, phase.tail.CkpLocation) + require.Equal(t, "ckp:phase1", phase.tail.CkpLocation) require.Len(t, phase.tail.Commands, 1) require.Equal(t, []uint32{10}, mustAccountIDsFromEntry(t, phase.tail.Commands[0])) } @@ -387,7 +387,7 @@ func TestPullActivationPhase1FiltersRowsBeforeEnqueue(t *testing.T) { select { case phase := <-server.activationTailChan: t.Cleanup(phase.closeAll) - require.Empty(t, phase.tails[2].CkpLocation) + require.Equal(t, "ckp:phase1", phase.tails[2].CkpLocation) require.Len(t, phase.tails[2].Commands, 1) require.Equal(t, []uint32{10}, mustAccountIDsFromEntry(t, phase.tails[2].Commands[0])) case <-time.After(time.Second): diff --git a/test/distributed/cases/git4data/branch/metadata/branch_metadata_tenant.result b/test/distributed/cases/git4data/branch/metadata/branch_metadata_tenant.result index d66bd2abde8a7..390974fef8d08 100644 --- a/test/distributed/cases/git4data/branch/metadata/branch_metadata_tenant.result +++ b/test/distributed/cases/git4data/branch/metadata/branch_metadata_tenant.result @@ -1,5 +1,10 @@ drop account if exists acc_branch_meta; create account acc_branch_meta admin_name "root1" identified by "111"; +set @acc_id = ( +select account_id from mo_catalog.mo_account +where account_name = 'acc_branch_meta' +); +set @_activate = (select mo_ctl('cn', 'ActivateTenantCatalog', cast(@acc_id as char))); drop database if exists br_meta_db; create database br_meta_db; use br_meta_db; @@ -10,10 +15,6 @@ drop snapshot if exists sp_base_tbl; create snapshot sp_base_tbl for table br_meta_db base_tbl; drop table if exists branch_tbl; data branch create table branch_tbl from base_tbl{snapshot="sp_base_tbl"}; -set @acc_id = ( -select account_id from mo_catalog.mo_account -where account_name = 'acc_branch_meta' -); set @branch_tbl_id = ( select rel_id from mo_catalog.mo_tables where account_id = @acc_id and reldatabase = 'br_meta_db' and relname = 'branch_tbl' diff --git a/test/distributed/cases/git4data/branch/metadata/branch_metadata_tenant.sql b/test/distributed/cases/git4data/branch/metadata/branch_metadata_tenant.sql index 2c244792937b0..da27b5fd086c5 100644 --- a/test/distributed/cases/git4data/branch/metadata/branch_metadata_tenant.sql +++ b/test/distributed/cases/git4data/branch/metadata/branch_metadata_tenant.sql @@ -3,6 +3,13 @@ drop account if exists acc_branch_meta; create account acc_branch_meta admin_name "root1" identified by "111"; +-- Activate the new account on this CN for subsequent cross-account catalog queries. +set @acc_id = ( + select account_id from mo_catalog.mo_account + where account_name = 'acc_branch_meta' +); +set @_activate = (select mo_ctl('cn', 'ActivateTenantCatalog', cast(@acc_id as char))); + -- @session:id=1&user=acc_branch_meta:root1&password=111 -- Case 1: data branch create/delete table in normal tenant. drop database if exists br_meta_db; @@ -20,10 +27,6 @@ drop table if exists branch_tbl; data branch create table branch_tbl from base_tbl{snapshot="sp_base_tbl"}; -- @session -set @acc_id = ( - select account_id from mo_catalog.mo_account - where account_name = 'acc_branch_meta' -); set @branch_tbl_id = ( select rel_id from mo_catalog.mo_tables where account_id = @acc_id and reldatabase = 'br_meta_db' and relname = 'branch_tbl' From 10e94f39556d97f19c71540977f4224c963f9ca5 Mon Sep 17 00:00:00 2001 From: aptend Date: Tue, 21 Apr 2026 12:03:18 +0800 Subject: [PATCH 04/11] chore: regenerate pb files after rebase onto main Regenerated proto-derived files after rebasing onto origin/main. Also includes gofmt formatting fixes from make pb. Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com> --- pkg/pb/api/api.pb.go | 460 ++++++++++++-------- pkg/sql/plan/function/ctl/types.go | 132 +++--- pkg/vm/engine/disttae/logtail_consumer.go | 4 +- pkg/vm/engine/tae/logtail/service/server.go | 2 +- 4 files changed, 348 insertions(+), 250 deletions(-) diff --git a/pkg/pb/api/api.pb.go b/pkg/pb/api/api.pb.go index 98e10670ead0d..10af658e249aa 100644 --- a/pkg/pb/api/api.pb.go +++ b/pkg/pb/api/api.pb.go @@ -773,7 +773,10 @@ func (m *SyncLogTailResp) GetCommands() []*Entry { // How to parse and handle PrecommiWriteCmd , pls ref to // tae/rpc/handle.go/HandlePreCommit function type PrecommitWriteCmd struct { - EntryList []*Entry `protobuf:"bytes,1,rep,name=entry_list,json=entryList,proto3" json:"entry_list,omitempty"` + EntryList []*Entry `protobuf:"bytes,1,rep,name=entry_list,json=entryList,proto3" json:"entry_list,omitempty"` + // Job ID for sync protection validation during CCPR commits. + // When non-empty, TN validates the sync protection is still valid at prepareTS. + SyncProtectionJobId string `protobuf:"bytes,2,opt,name=sync_protection_job_id,json=syncProtectionJobId,proto3" json:"sync_protection_job_id,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -819,6 +822,13 @@ func (m *PrecommitWriteCmd) GetEntryList() []*Entry { return nil } +func (m *PrecommitWriteCmd) GetSyncProtectionJobId() string { + if m != nil { + return m.SyncProtectionJobId + } + return "" +} + type Entry struct { EntryType Entry_EntryType `protobuf:"varint,1,opt,name=entry_type,json=entryType,proto3,enum=api.Entry_EntryType" json:"entry_type,omitempty"` TableId uint64 `protobuf:"varint,2,opt,name=table_id,json=tableId,proto3" json:"table_id,omitempty"` @@ -1791,20 +1801,22 @@ type SchemaExtra struct { DroppedAttrs []string `protobuf:"bytes,2,rep,name=dropped_attrs,json=droppedAttrs,proto3" json:"dropped_attrs,omitempty"` ColumnChanged bool `protobuf:"varint,3,opt,name=column_changed,json=columnChanged,proto3" json:"column_changed,omitempty"` // sending mo_tables deletes by this. - OldName string `protobuf:"bytes,4,opt,name=old_name,json=oldName,proto3" json:"old_name,omitempty"` - MinOsizeQuailifed uint32 `protobuf:"varint,5,opt,name=min_osize_quailifed,json=minOsizeQuailifed,proto3" json:"min_osize_quailifed,omitempty"` - MaxObjOnerun uint32 `protobuf:"varint,6,opt,name=max_obj_onerun,json=maxObjOnerun,proto3" json:"max_obj_onerun,omitempty"` - MaxOsizeMergedObj uint32 `protobuf:"varint,7,opt,name=max_osize_merged_obj,json=maxOsizeMergedObj,proto3" json:"max_osize_merged_obj,omitempty"` - Hints []MergeHint `protobuf:"varint,8,rep,packed,name=hints,proto3,enum=api.MergeHint" json:"hints,omitempty"` - MinCnMergeSize uint64 `protobuf:"varint,9,opt,name=min_cn_merge_size,json=minCnMergeSize,proto3" json:"min_cn_merge_size,omitempty"` - BlockMaxRows uint32 `protobuf:"varint,10,opt,name=block_max_rows,json=blockMaxRows,proto3" json:"block_max_rows,omitempty"` - ObjectMaxBlocks uint32 `protobuf:"varint,11,opt,name=object_max_blocks,json=objectMaxBlocks,proto3" json:"object_max_blocks,omitempty"` - FeatureFlag uint64 `protobuf:"varint,12,opt,name=FeatureFlag,proto3" json:"FeatureFlag,omitempty"` - IndexTables []uint64 `protobuf:"varint,13,rep,packed,name=IndexTables,proto3" json:"IndexTables,omitempty"` - ParentTableID uint64 `protobuf:"varint,14,opt,name=ParentTableID,proto3" json:"ParentTableID,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + OldName string `protobuf:"bytes,4,opt,name=old_name,json=oldName,proto3" json:"old_name,omitempty"` + MinOsizeQuailifed uint32 `protobuf:"varint,5,opt,name=min_osize_quailifed,json=minOsizeQuailifed,proto3" json:"min_osize_quailifed,omitempty"` + MaxObjOnerun uint32 `protobuf:"varint,6,opt,name=max_obj_onerun,json=maxObjOnerun,proto3" json:"max_obj_onerun,omitempty"` + MaxOsizeMergedObj uint32 `protobuf:"varint,7,opt,name=max_osize_merged_obj,json=maxOsizeMergedObj,proto3" json:"max_osize_merged_obj,omitempty"` + Hints []MergeHint `protobuf:"varint,8,rep,packed,name=hints,proto3,enum=api.MergeHint" json:"hints,omitempty"` + MinCnMergeSize uint64 `protobuf:"varint,9,opt,name=min_cn_merge_size,json=minCnMergeSize,proto3" json:"min_cn_merge_size,omitempty"` + BlockMaxRows uint32 `protobuf:"varint,10,opt,name=block_max_rows,json=blockMaxRows,proto3" json:"block_max_rows,omitempty"` + ObjectMaxBlocks uint32 `protobuf:"varint,11,opt,name=object_max_blocks,json=objectMaxBlocks,proto3" json:"object_max_blocks,omitempty"` + FeatureFlag uint64 `protobuf:"varint,12,opt,name=FeatureFlag,proto3" json:"FeatureFlag,omitempty"` + IndexTables []uint64 `protobuf:"varint,13,rep,packed,name=IndexTables,proto3" json:"IndexTables,omitempty"` + ParentTableID uint64 `protobuf:"varint,14,opt,name=ParentTableID,proto3" json:"ParentTableID,omitempty"` + // mark if table is created by publication (CCPR), should skip merge + FromPublication bool `protobuf:"varint,15,opt,name=FromPublication,proto3" json:"FromPublication,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *SchemaExtra) Reset() { *m = SchemaExtra{} } @@ -1938,6 +1950,13 @@ func (m *SchemaExtra) GetParentTableID() uint64 { return 0 } +func (m *SchemaExtra) GetFromPublication() bool { + if m != nil { + return m.FromPublication + } + return false +} + // Int64Map mainly used in unit test type Int64Map struct { M map[int64]int64 `protobuf:"bytes,1,rep,name=m,proto3" json:"m,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` @@ -2471,172 +2490,175 @@ func init() { func init() { proto.RegisterFile("api.proto", fileDescriptor_00212fb1f9d3bf1c) } var fileDescriptor_00212fb1f9d3bf1c = []byte{ - // 2633 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x58, 0x4b, 0x6f, 0x1c, 0xc7, - 0x11, 0xe6, 0x72, 0xdf, 0x35, 0xfb, 0x18, 0xb6, 0x28, 0x6b, 0x4d, 0x3b, 0x12, 0xb3, 0x96, 0x6d, - 0x5a, 0x8e, 0x29, 0x84, 0xb6, 0x13, 0xdb, 0x10, 0x6c, 0x90, 0x4b, 0x4b, 0xdc, 0x84, 0xd4, 0x32, - 0xc3, 0x95, 0x0d, 0x18, 0x01, 0x06, 0xbd, 0x33, 0xad, 0xe5, 0x68, 0x67, 0xba, 0x47, 0xdd, 0xbd, - 0x14, 0xe9, 0x6b, 0xe2, 0x63, 0x2e, 0xb9, 0xe5, 0x66, 0xff, 0x83, 0x04, 0x08, 0x90, 0x7f, 0x10, - 0xf8, 0xe8, 0x20, 0xce, 0x3b, 0x71, 0x14, 0xe7, 0x92, 0xe4, 0x57, 0x04, 0xfd, 0x98, 0xdd, 0x21, - 0x45, 0x39, 0x71, 0x10, 0xc0, 0x07, 0x12, 0xdd, 0x5f, 0x55, 0xf5, 0x7c, 0x55, 0x5d, 0x5d, 0x5d, - 0xbd, 0x50, 0xc7, 0x69, 0xb4, 0x9e, 0x72, 0x26, 0x19, 0x2a, 0xe2, 0x34, 0x5a, 0x79, 0x69, 0x1c, - 0xc9, 0xc3, 0xe9, 0x68, 0x3d, 0x60, 0xc9, 0xf5, 0x31, 0x1b, 0xb3, 0xeb, 0x5a, 0x36, 0x9a, 0xde, - 0xd5, 0x33, 0x3d, 0xd1, 0x23, 0x63, 0xb3, 0xd2, 0x96, 0x51, 0x42, 0x84, 0xc4, 0x49, 0x6a, 0x01, - 0x48, 0x63, 0x4c, 0xcd, 0xb8, 0xfb, 0x6d, 0x68, 0x0e, 0x6f, 0xef, 0x47, 0x74, 0xec, 0x91, 0xfb, - 0x53, 0x22, 0x24, 0x7a, 0x1a, 0xea, 0x29, 0xe6, 0x38, 0x21, 0x92, 0xf0, 0x4e, 0x61, 0xb5, 0xb0, - 0x56, 0xf7, 0xe6, 0xc0, 0x1b, 0xb5, 0x0f, 0x3f, 0xba, 0x52, 0x78, 0xf8, 0xd1, 0x95, 0x85, 0xee, - 0xcf, 0x0b, 0xd0, 0xca, 0x2c, 0x45, 0xca, 0xa8, 0x20, 0xa8, 0x03, 0x55, 0x21, 0x19, 0x27, 0xfd, - 0x6d, 0x6b, 0x98, 0x4d, 0xd1, 0x73, 0xd0, 0x12, 0x84, 0x1f, 0x45, 0x01, 0xd9, 0x0c, 0x43, 0x4e, - 0x84, 0xe8, 0x2c, 0x6a, 0x85, 0x33, 0xa8, 0x5e, 0xe1, 0x10, 0xf3, 0xb0, 0xbf, 0xdd, 0x29, 0xae, - 0x16, 0xd6, 0x4a, 0x5e, 0x36, 0x55, 0xb4, 0x38, 0x49, 0xe3, 0x28, 0xc0, 0xfd, 0xed, 0x4e, 0x49, - 0xcb, 0xe6, 0x00, 0xba, 0x0c, 0x10, 0xb3, 0xf1, 0x81, 0x35, 0x2d, 0x6b, 0x71, 0x0e, 0xc9, 0xd1, - 0x7e, 0x03, 0xdc, 0xe1, 0xed, 0x03, 0xc9, 0xf3, 0xbc, 0xf5, 0xda, 0x72, 0xca, 0xe9, 0x81, 0x9c, - 0xb9, 0x3c, 0x03, 0x72, 0xb6, 0x3f, 0x2b, 0x40, 0xe5, 0x1d, 0x12, 0x48, 0xc6, 0x11, 0x82, 0x52, - 0x88, 0x25, 0xd6, 0xda, 0x0d, 0x4f, 0x8f, 0xd1, 0x55, 0x28, 0xc9, 0x93, 0x94, 0x68, 0xd7, 0x9c, - 0x0d, 0x58, 0xd7, 0x51, 0x1e, 0x9e, 0xa4, 0x64, 0xab, 0xf4, 0xf1, 0x67, 0x57, 0x16, 0x3c, 0x2d, - 0x45, 0x2b, 0x50, 0xa3, 0xd3, 0x38, 0xc6, 0xa3, 0x98, 0x68, 0x1f, 0x6b, 0xde, 0x6c, 0x8e, 0x5c, - 0x28, 0x52, 0x91, 0x6a, 0xf7, 0x1a, 0x9e, 0x1a, 0xa2, 0x27, 0xa1, 0x16, 0x09, 0x3f, 0x60, 0x54, - 0x48, 0xed, 0x56, 0xcd, 0xab, 0x46, 0xa2, 0xa7, 0xa6, 0x4a, 0x39, 0x26, 0xb4, 0x53, 0x59, 0x2d, - 0xac, 0x35, 0x3d, 0x35, 0x54, 0xa4, 0x30, 0x27, 0xb8, 0x53, 0x35, 0xa4, 0xd4, 0xb8, 0xfb, 0x1d, - 0x28, 0x6f, 0x61, 0x19, 0x1c, 0xa2, 0x15, 0x28, 0x63, 0x29, 0xb9, 0xe8, 0x14, 0x56, 0x8b, 0x6b, - 0x75, 0x4b, 0xc9, 0x40, 0xe8, 0x59, 0x28, 0x1d, 0x91, 0x40, 0x6d, 0x4a, 0x71, 0xcd, 0xd9, 0x70, - 0xd6, 0x55, 0xbe, 0x19, 0x47, 0x33, 0xea, 0x4a, 0xdc, 0xfd, 0x65, 0x01, 0xaa, 0x43, 0x45, 0xb4, - 0xbf, 0x8d, 0x2e, 0x40, 0x39, 0x1c, 0xf9, 0x51, 0xa8, 0x23, 0x50, 0xf2, 0x4a, 0xe1, 0xa8, 0x1f, - 0x2a, 0x50, 0x6a, 0x70, 0xd1, 0x80, 0x52, 0x81, 0x5f, 0x87, 0x46, 0x8a, 0xb9, 0x8c, 0x64, 0xc4, - 0xa8, 0x92, 0x99, 0x8d, 0x75, 0x66, 0x58, 0x3f, 0x44, 0x17, 0xa1, 0x82, 0x83, 0x40, 0x09, 0x4b, - 0xda, 0x9b, 0x32, 0x0e, 0x82, 0x7e, 0x88, 0x2e, 0x41, 0x35, 0x1c, 0xf9, 0x14, 0x27, 0x44, 0xfb, - 0x5e, 0xf7, 0x2a, 0xe1, 0xe8, 0x36, 0x4e, 0x88, 0x12, 0x48, 0x2b, 0xa8, 0x18, 0x81, 0x34, 0x82, - 0x67, 0xa1, 0x95, 0xf2, 0x28, 0xc1, 0xfc, 0xc4, 0x17, 0xe4, 0x3e, 0x9d, 0x26, 0x3a, 0x16, 0x4d, - 0xaf, 0x69, 0xd1, 0x03, 0x0d, 0x76, 0x7f, 0x5c, 0x80, 0xd6, 0xc1, 0x09, 0x0d, 0x76, 0xd9, 0x78, - 0x88, 0xa3, 0xd8, 0x23, 0xf7, 0xd1, 0x4b, 0x50, 0x0d, 0xa8, 0x7f, 0x88, 0x8f, 0x88, 0xf6, 0xc8, - 0xd9, 0x58, 0x5e, 0x9f, 0x1f, 0x9b, 0x61, 0x36, 0xf2, 0x2a, 0x01, 0xdd, 0xc1, 0x47, 0xc4, 0xaa, - 0x3f, 0xc0, 0x54, 0xda, 0xed, 0x7e, 0xac, 0xfa, 0xbb, 0x98, 0x4a, 0xd4, 0x85, 0xb2, 0x9c, 0xed, - 0xb8, 0xb3, 0xd1, 0xd0, 0x11, 0xb6, 0xa1, 0xf4, 0x8c, 0xa8, 0xfb, 0x7d, 0x68, 0x9f, 0xe2, 0x24, - 0x52, 0x15, 0xba, 0x60, 0x92, 0xfa, 0x31, 0x0b, 0xb0, 0x8a, 0x94, 0xcd, 0x4d, 0x27, 0x98, 0xa4, - 0xbb, 0x16, 0x42, 0xcf, 0x41, 0x2d, 0x60, 0x49, 0x82, 0x69, 0x98, 0x6d, 0x1f, 0xe8, 0xc5, 0xdf, - 0xa6, 0x92, 0x9f, 0x78, 0x33, 0x59, 0xf7, 0x4d, 0x58, 0xda, 0xe7, 0x44, 0x4d, 0x23, 0xf9, 0x2e, - 0x8f, 0x24, 0xe9, 0x25, 0x21, 0x7a, 0x01, 0x80, 0x28, 0x3d, 0x3f, 0x8e, 0x84, 0xd4, 0x89, 0x71, - 0xda, 0xbc, 0xae, 0xa5, 0xbb, 0x91, 0x90, 0xdd, 0x0f, 0x4a, 0x50, 0xd6, 0x20, 0x7a, 0x39, 0x33, - 0xd2, 0xc9, 0xae, 0x28, 0xb5, 0x36, 0x96, 0xe7, 0x46, 0xe6, 0xbf, 0x4a, 0x7b, 0x6b, 0xae, 0x86, - 0x2a, 0x8f, 0xb5, 0x97, 0xf3, 0xe4, 0xa8, 0xea, 0x79, 0x3f, 0x44, 0x57, 0xc0, 0x51, 0xc7, 0x67, - 0x84, 0x05, 0x99, 0xa7, 0x07, 0x64, 0x50, 0x3f, 0x44, 0x5f, 0x03, 0x30, 0xb6, 0x7a, 0xc3, 0x4b, - 0xe6, 0x7c, 0x6a, 0x44, 0xef, 0xf9, 0x33, 0xd0, 0x9c, 0xd9, 0xe7, 0x72, 0xa5, 0x91, 0x81, 0x5a, - 0xe9, 0x29, 0xa8, 0xdf, 0x8d, 0xb2, 0x25, 0x4c, 0xce, 0xd4, 0x14, 0xa0, 0x85, 0x4f, 0x43, 0x71, - 0x84, 0xa5, 0x4e, 0x95, 0xcc, 0x7f, 0x7d, 0x66, 0x3c, 0x05, 0xa3, 0x67, 0xa0, 0x95, 0x4e, 0xfc, - 0xe0, 0x90, 0x04, 0x13, 0x7f, 0x74, 0xe2, 0x4b, 0xda, 0xa9, 0xad, 0x16, 0xd6, 0xca, 0x9e, 0x93, - 0x4e, 0x7a, 0x0a, 0xdc, 0x3a, 0x19, 0x52, 0xf4, 0x2a, 0x5c, 0x8a, 0xf1, 0xfb, 0x27, 0x7e, 0x80, - 0x25, 0x8e, 0xd9, 0xd8, 0xc7, 0x41, 0xc0, 0xa6, 0x54, 0x2a, 0x87, 0xea, 0x3a, 0x03, 0x97, 0x95, - 0xb8, 0x67, 0xa4, 0x9b, 0x46, 0xd8, 0x0f, 0xd1, 0x0d, 0x78, 0xea, 0x10, 0x0b, 0xff, 0x71, 0xa6, - 0xa0, 0x4f, 0xfc, 0xa5, 0x43, 0x2c, 0x76, 0xcf, 0xb1, 0xee, 0x72, 0xa8, 0xcf, 0x82, 0x8d, 0x00, - 0x2a, 0x7d, 0x2a, 0x08, 0x97, 0xee, 0x82, 0x1a, 0x6f, 0x93, 0x98, 0x48, 0xe2, 0x16, 0xd4, 0xf8, - 0x4e, 0x1a, 0x62, 0x49, 0xdc, 0x45, 0x54, 0x87, 0xf2, 0x66, 0x2c, 0x09, 0x77, 0x8b, 0x68, 0x09, - 0x9a, 0x07, 0x29, 0x09, 0x22, 0x1c, 0x5b, 0xcd, 0x12, 0x6a, 0x01, 0x6c, 0x63, 0x89, 0x07, 0xa3, - 0x7b, 0x24, 0x90, 0x6e, 0x19, 0x5d, 0x80, 0xf6, 0x90, 0x25, 0x23, 0x21, 0x19, 0x25, 0x16, 0xac, - 0x74, 0x7f, 0x58, 0x00, 0xd0, 0x6e, 0xa7, 0x2c, 0xa2, 0x12, 0xbd, 0x08, 0x95, 0x24, 0xa2, 0xbe, - 0x14, 0x5f, 0x78, 0x6a, 0xca, 0x49, 0x44, 0x87, 0x42, 0x2b, 0xe3, 0x63, 0xa5, 0xbc, 0xf8, 0x85, - 0xca, 0xf8, 0x78, 0x28, 0xb2, 0x4d, 0x29, 0x9e, 0xbb, 0x29, 0x86, 0x86, 0x89, 0x47, 0x6f, 0x92, - 0x7e, 0x65, 0x34, 0x3e, 0x28, 0x80, 0xb3, 0x47, 0x24, 0x56, 0xb9, 0xf6, 0x55, 0xf2, 0xf8, 0x57, - 0x01, 0x5c, 0xbd, 0xb3, 0xba, 0xa6, 0xec, 0xb3, 0x38, 0x0a, 0x4e, 0xd0, 0x3a, 0x5c, 0x50, 0x64, - 0x98, 0x88, 0xde, 0x27, 0xfe, 0xfd, 0x29, 0x8e, 0xe2, 0xe8, 0x2e, 0x31, 0x05, 0xbb, 0xe9, 0x2d, - 0x25, 0x11, 0x1d, 0x28, 0xc9, 0xf7, 0x32, 0x01, 0xba, 0x0a, 0x2d, 0xc5, 0x87, 0x8d, 0xee, 0xf9, - 0x8c, 0x12, 0x3e, 0xa5, 0x9a, 0x57, 0xd3, 0x6b, 0x24, 0xf8, 0x78, 0x30, 0xba, 0x37, 0xd0, 0x18, - 0xba, 0x0e, 0xcb, 0x5a, 0x4b, 0xaf, 0x9a, 0x10, 0x3e, 0x26, 0xa1, 0x32, 0xd1, 0xcc, 0xd4, 0xb2, - 0xf8, 0x58, 0x2f, 0xbb, 0xa7, 0x25, 0x83, 0xd1, 0x3d, 0x74, 0x15, 0xca, 0x87, 0x11, 0x95, 0xa2, - 0x53, 0x5a, 0x2d, 0xae, 0xb5, 0x36, 0x5a, 0x9a, 0xbb, 0x16, 0xef, 0x44, 0x54, 0x7a, 0x46, 0x88, - 0x5e, 0x00, 0xc5, 0xc8, 0x0f, 0xa8, 0x59, 0xd3, 0x57, 0x6b, 0xd8, 0x8b, 0xbc, 0x95, 0x44, 0xb4, - 0x47, 0xb5, 0xc5, 0x41, 0xf4, 0x3e, 0xe9, 0xbe, 0x06, 0xcb, 0x73, 0x5f, 0xf5, 0x5d, 0xc8, 0xb1, - 0xca, 0xc5, 0x55, 0x70, 0x82, 0xd9, 0x4c, 0xd8, 0xab, 0x39, 0x0f, 0x75, 0x5f, 0x82, 0xa5, 0xbc, - 0x65, 0x92, 0x10, 0x2a, 0x55, 0xcf, 0x11, 0x98, 0x61, 0xd6, 0xb5, 0xd8, 0x69, 0x77, 0x0f, 0x2e, - 0xce, 0xd5, 0x3d, 0xa2, 0x6a, 0x87, 0x1e, 0xaa, 0x6a, 0xc6, 0xe2, 0xd0, 0x14, 0x13, 0x6b, 0xc3, - 0xe2, 0x50, 0xd7, 0x92, 0x27, 0xa1, 0x46, 0xc9, 0x03, 0x23, 0x32, 0x3d, 0x4e, 0x95, 0x92, 0x07, - 0x4a, 0xd4, 0xa5, 0x70, 0xe1, 0xec, 0x72, 0x3d, 0x16, 0xff, 0x6f, 0x8b, 0xa9, 0xab, 0x41, 0xa8, - 0x8e, 0x8d, 0x06, 0xc4, 0x57, 0xf7, 0x9c, 0x09, 0xbf, 0x93, 0x61, 0xb7, 0xa7, 0x49, 0x37, 0xcc, - 0x7f, 0x6f, 0x33, 0x0c, 0x7b, 0x2c, 0x9e, 0x26, 0x14, 0x5d, 0x85, 0x4a, 0xa0, 0x47, 0x36, 0x47, - 0x1b, 0xa6, 0x51, 0xe9, 0xb1, 0x78, 0x9b, 0xdc, 0xf5, 0xac, 0x0c, 0x3d, 0x0f, 0xed, 0x48, 0x97, - 0x13, 0x3f, 0x65, 0x42, 0xdf, 0xd3, 0x9a, 0x41, 0xd9, 0x6b, 0x19, 0x78, 0xdf, 0xa2, 0xa7, 0x77, - 0xc3, 0x23, 0x69, 0x8c, 0x03, 0xb2, 0x4d, 0xee, 0xa2, 0x55, 0x28, 0x86, 0xe4, 0xae, 0xfd, 0x46, - 0xcb, 0x36, 0x43, 0x4a, 0x47, 0x7d, 0x45, 0x89, 0xba, 0xef, 0xe5, 0x2d, 0xb7, 0x39, 0x4b, 0x2d, - 0xc1, 0x2b, 0xe0, 0xc4, 0x6c, 0x1c, 0x05, 0x38, 0xf6, 0xa3, 0xf0, 0xd8, 0xe6, 0x2b, 0x58, 0xa8, - 0x1f, 0x1e, 0x3f, 0xe2, 0xfb, 0xe2, 0xa3, 0xbe, 0x3f, 0x2c, 0x41, 0x33, 0x4f, 0xeb, 0xfe, 0xa9, - 0x1b, 0xa8, 0x70, 0xfa, 0x06, 0x9a, 0xf5, 0x32, 0x8b, 0xb9, 0x5e, 0xa6, 0x0b, 0xa5, 0x49, 0x44, - 0xcd, 0x7d, 0x94, 0x65, 0xad, 0x5e, 0xf1, 0xbb, 0x11, 0x0d, 0x3d, 0x2d, 0x43, 0xaf, 0x03, 0xe0, - 0x30, 0xf4, 0x6d, 0x38, 0x4b, 0xda, 0xd5, 0xce, 0x5c, 0xf3, 0x74, 0xe0, 0x77, 0x16, 0xbc, 0x3a, - 0x9e, 0xed, 0xc2, 0x0d, 0x70, 0x42, 0xce, 0xd2, 0xcc, 0xb6, 0xac, 0x6d, 0x9f, 0x3c, 0x63, 0x3b, - 0x0f, 0xca, 0xce, 0x82, 0x07, 0xe1, 0x3c, 0x44, 0x6f, 0x41, 0x83, 0xeb, 0x04, 0xf2, 0x4d, 0x5b, - 0x51, 0xd1, 0xe6, 0x2b, 0x67, 0xcc, 0x73, 0x29, 0xbb, 0xb3, 0xe0, 0x39, 0x3c, 0x97, 0xc1, 0x6f, - 0x41, 0x6b, 0xaa, 0x6f, 0x05, 0x3f, 0xcb, 0x7d, 0x73, 0xfb, 0x3d, 0x71, 0x66, 0x09, 0x7b, 0x48, - 0x76, 0x16, 0xbc, 0xa6, 0xd1, 0xcf, 0x4e, 0xcd, 0x0d, 0x70, 0xb2, 0x05, 0x84, 0xe4, 0xfa, 0x4a, - 0x7c, 0x94, 0xff, 0xfc, 0x70, 0x2a, 0xfe, 0x76, 0x01, 0x21, 0x39, 0xba, 0x01, 0x76, 0x39, 0x3f, - 0xd5, 0xb5, 0x4a, 0x5f, 0x92, 0xce, 0xc6, 0xc5, 0x33, 0xf6, 0xa6, 0x90, 0xed, 0x2c, 0x78, 0x0d, - 0xa3, 0x6d, 0x0b, 0xdb, 0xeb, 0x00, 0xd6, 0xfb, 0x80, 0xc5, 0x1d, 0xe7, 0xdc, 0xb0, 0xcf, 0xce, - 0x97, 0x0a, 0x3b, 0x9f, 0x1d, 0xb6, 0x1b, 0xe0, 0x70, 0x93, 0xa3, 0xbe, 0xca, 0xce, 0xc6, 0xb9, - 0xb4, 0xe7, 0x59, 0xac, 0x68, 0xf3, 0xd9, 0x6c, 0xcb, 0x81, 0x3a, 0x4b, 0x09, 0xd7, 0x9d, 0x57, - 0xf7, 0xa7, 0x25, 0x70, 0x0e, 0x82, 0x43, 0x92, 0xe0, 0xb7, 0x8f, 0x25, 0xc7, 0xe8, 0x39, 0x68, - 0x53, 0x72, 0x2c, 0x15, 0xa7, 0xac, 0xf9, 0x34, 0xa9, 0xdb, 0x54, 0x70, 0x8f, 0xc5, 0xa6, 0xf9, - 0xd4, 0xfd, 0x0a, 0x67, 0x69, 0x4a, 0x42, 0xdf, 0x34, 0xe4, 0xaa, 0x6d, 0x53, 0xfd, 0x8a, 0x01, - 0x37, 0x6d, 0x47, 0xde, 0x32, 0x99, 0xe1, 0x07, 0x87, 0x98, 0x8e, 0x49, 0x68, 0xdf, 0x0a, 0x4d, - 0x83, 0xf6, 0x0c, 0x78, 0xaa, 0x76, 0x94, 0x4e, 0xd7, 0x8e, 0xc7, 0x54, 0xff, 0xf2, 0x7f, 0x5f, - 0xfd, 0x2b, 0x5f, 0xa2, 0xfa, 0x57, 0xff, 0x63, 0xf5, 0xaf, 0x7d, 0xe9, 0xea, 0x5f, 0x3f, 0xaf, - 0xfa, 0x2b, 0x9e, 0xa3, 0x98, 0x05, 0x13, 0x5f, 0xf1, 0xe0, 0xec, 0x81, 0xd0, 0x5d, 0x52, 0xd3, - 0x6b, 0x68, 0x74, 0x0f, 0x1f, 0x7b, 0xec, 0x81, 0x40, 0xd7, 0x60, 0x89, 0xe9, 0x96, 0x45, 0xab, - 0x69, 0x91, 0xd0, 0x99, 0xd2, 0xf4, 0xda, 0x46, 0xb0, 0x87, 0x8f, 0xb7, 0x34, 0xac, 0xee, 0x8d, - 0x9b, 0x04, 0xcb, 0x29, 0x27, 0x37, 0x63, 0x3c, 0xd6, 0x39, 0x51, 0xf2, 0xf2, 0x90, 0xd2, 0xe8, - 0xd3, 0x90, 0x1c, 0xeb, 0xec, 0x10, 0x9d, 0xe6, 0x6a, 0x51, 0x69, 0xe4, 0x20, 0x74, 0x15, 0x9a, - 0xfb, 0x98, 0x13, 0x2a, 0x6d, 0x53, 0xdf, 0x69, 0xe9, 0x55, 0x4e, 0x83, 0xdd, 0x10, 0x6a, 0x7d, - 0x2a, 0xbf, 0xf5, 0xca, 0x1e, 0x4e, 0x51, 0x17, 0x0a, 0x89, 0x6d, 0xb9, 0x4d, 0xf7, 0x9c, 0x49, - 0xd6, 0xf7, 0x4c, 0xf3, 0x5d, 0x48, 0x56, 0x5e, 0x81, 0x8a, 0x99, 0xa8, 0xc7, 0xde, 0x84, 0x9c, - 0xe8, 0x84, 0x2a, 0x7a, 0x6a, 0x88, 0x96, 0xa1, 0x7c, 0x84, 0xe3, 0xa9, 0xb9, 0x18, 0x8a, 0x9e, - 0x99, 0xbc, 0xb1, 0xf8, 0x5a, 0xa1, 0xfb, 0x0e, 0x34, 0x86, 0x1c, 0x53, 0xb1, 0x4d, 0x84, 0x2a, - 0xd3, 0xe8, 0x09, 0xa8, 0xb0, 0xd1, 0xbd, 0xbe, 0x2d, 0xa5, 0x65, 0xcf, 0xce, 0x14, 0x3e, 0x8a, - 0x27, 0x0a, 0x37, 0x95, 0xdd, 0xce, 0x14, 0xce, 0xd9, 0x03, 0x85, 0x17, 0x0d, 0x6e, 0x66, 0xdd, - 0x1f, 0x14, 0xc0, 0xd9, 0x8a, 0x27, 0x7a, 0x6d, 0xe5, 0xc1, 0x8b, 0x73, 0x0f, 0x2e, 0x99, 0x86, - 0x64, 0x2e, 0xb4, 0x4e, 0xd8, 0xe7, 0x63, 0x21, 0x59, 0xb9, 0x75, 0x9e, 0x2b, 0x65, 0xe3, 0xca, - 0xf3, 0x79, 0x57, 0x9c, 0x8d, 0x25, 0xf3, 0x3a, 0xca, 0xb9, 0x90, 0xf7, 0x6e, 0x07, 0x50, 0xf6, - 0x9d, 0xbb, 0x84, 0x6f, 0x31, 0x36, 0x89, 0xe8, 0x18, 0x6d, 0x40, 0x2d, 0xc1, 0x69, 0x1a, 0xd1, - 0xb1, 0xb0, 0x94, 0xdc, 0xb3, 0x94, 0x2c, 0x97, 0x99, 0x5e, 0xf7, 0xd3, 0x45, 0x70, 0x75, 0x5e, - 0xf5, 0xf4, 0xab, 0xc8, 0xb0, 0x3b, 0xf7, 0x5d, 0x7b, 0x11, 0x2a, 0x72, 0x14, 0xcf, 0x6f, 0x88, - 0xb2, 0x1c, 0xc5, 0x8f, 0x3c, 0x4c, 0x8a, 0x67, 0x1f, 0x26, 0xaf, 0x42, 0x4d, 0x48, 0xcc, 0xa5, - 0xaf, 0x7b, 0x9f, 0xc7, 0x76, 0x78, 0x96, 0x57, 0x55, 0xeb, 0x0e, 0x85, 0xba, 0xfe, 0xe6, 0x07, - 0x4b, 0x74, 0xca, 0xab, 0xc5, 0xb5, 0x86, 0x07, 0x49, 0x76, 0xa2, 0x84, 0x7e, 0x15, 0x72, 0x82, - 0x65, 0xa6, 0x51, 0xd1, 0x1a, 0x8e, 0xc5, 0xb4, 0xca, 0x37, 0xa1, 0x3a, 0x32, 0x91, 0xb1, 0x75, - 0xfd, 0xf4, 0x06, 0xcd, 0x03, 0xe7, 0x65, 0x7a, 0xea, 0xb3, 0x76, 0xa8, 0xde, 0x9b, 0xfa, 0xb8, - 0xd6, 0x3d, 0xb0, 0xd0, 0x2e, 0x0b, 0xd4, 0xbe, 0x11, 0xce, 0xf5, 0xa9, 0xac, 0x7b, 0x6a, 0xa8, - 0x52, 0x30, 0x26, 0x47, 0x24, 0xd6, 0x27, 0xb0, 0xec, 0x99, 0x49, 0xf7, 0x27, 0x8b, 0xd0, 0xd2, - 0x61, 0x1d, 0x62, 0x31, 0xf9, 0xbf, 0x07, 0x35, 0xf7, 0x9b, 0x40, 0xe9, 0xd4, 0x6f, 0x02, 0x5d, - 0x68, 0x4a, 0x66, 0xcb, 0x47, 0x2e, 0x70, 0x8e, 0x64, 0x9a, 0x8c, 0x0e, 0xcb, 0x3a, 0x5c, 0x20, - 0x42, 0x46, 0x89, 0x8e, 0x5d, 0x42, 0x12, 0x7f, 0x2a, 0xf0, 0xd8, 0xdc, 0x9e, 0x25, 0x6f, 0x69, - 0x26, 0xda, 0x23, 0xc9, 0x1d, 0x25, 0x50, 0x5c, 0x72, 0xaf, 0x31, 0x53, 0xe3, 0xea, 0x78, 0xf6, - 0x7a, 0xbb, 0x04, 0xd5, 0xa9, 0x20, 0x5c, 0xc9, 0x6a, 0x5a, 0x56, 0x51, 0x53, 0x23, 0xe0, 0xcc, - 0xb4, 0x1a, 0xe6, 0xf5, 0x57, 0x51, 0xd3, 0x7e, 0xd8, 0xbd, 0x0d, 0xad, 0xf9, 0xe3, 0x49, 0x3f, - 0xf1, 0x57, 0xa0, 0xb6, 0x7b, 0xfa, 0x79, 0x3f, 0x9b, 0xab, 0xb2, 0x23, 0xf9, 0x94, 0x06, 0x58, - 0x92, 0x5d, 0x41, 0x6d, 0x98, 0xf2, 0xd0, 0xb5, 0x1f, 0x15, 0xa1, 0x32, 0x48, 0x7b, 0x2c, 0x24, - 0xa8, 0x0a, 0xc5, 0xdb, 0x2c, 0x75, 0x17, 0xd0, 0x12, 0x34, 0x06, 0xe9, 0x2d, 0x22, 0xed, 0x0f, - 0x09, 0xee, 0x3f, 0xaa, 0xc8, 0x05, 0x67, 0x90, 0xee, 0x73, 0x9b, 0xe8, 0xee, 0x3f, 0xab, 0xc8, - 0x51, 0x76, 0xfb, 0x11, 0x1d, 0xbb, 0x9f, 0xb4, 0x51, 0x03, 0xaa, 0x83, 0xf4, 0x66, 0x3c, 0x15, - 0x87, 0xee, 0xaf, 0xda, 0xc6, 0x7e, 0xce, 0xd2, 0xfd, 0x75, 0x1b, 0xb5, 0xa0, 0x3e, 0x48, 0xfb, - 0x54, 0xa4, 0xea, 0x0d, 0xf8, 0x69, 0x1b, 0x2d, 0x43, 0x7b, 0x90, 0x6e, 0x86, 0xe1, 0x4d, 0x3c, - 0x8d, 0xe5, 0xbe, 0xd6, 0xfa, 0x4d, 0x1b, 0x35, 0xa1, 0x36, 0x48, 0xb7, 0x70, 0x30, 0x99, 0xa6, - 0xee, 0x6f, 0xdb, 0xe6, 0xa3, 0x43, 0x8e, 0x03, 0x72, 0x90, 0x62, 0xea, 0xfe, 0xae, 0x8d, 0x2e, - 0x40, 0x6b, 0x90, 0x1e, 0x48, 0xc6, 0xf1, 0x98, 0xe8, 0x00, 0xbb, 0xbf, 0x6f, 0xa3, 0x4b, 0x80, - 0x06, 0xe9, 0xad, 0x98, 0x8d, 0x70, 0x9c, 0xfb, 0xe8, 0x1f, 0xda, 0xe8, 0x09, 0x58, 0x52, 0x1f, - 0x95, 0x84, 0x07, 0x24, 0x95, 0x96, 0xfa, 0x1f, 0xdb, 0x08, 0x41, 0x53, 0xb9, 0xac, 0xa6, 0x7a, - 0x67, 0xdd, 0x3f, 0x59, 0xdd, 0xed, 0x48, 0x4c, 0xd4, 0x5f, 0x2f, 0x26, 0x98, 0x12, 0xee, 0xfe, - 0xd9, 0x52, 0xf2, 0x08, 0x0e, 0x09, 0x77, 0xff, 0xd2, 0x46, 0x2b, 0x70, 0xd1, 0x84, 0x06, 0x4b, - 0x22, 0x64, 0xee, 0x73, 0x9f, 0x65, 0xe4, 0x28, 0x4e, 0xc5, 0x21, 0x93, 0xca, 0xc4, 0xfd, 0xeb, - 0xdc, 0xc0, 0xde, 0xb7, 0xba, 0x8e, 0xef, 0x46, 0x42, 0xba, 0x0f, 0x2d, 0x0f, 0x1d, 0x81, 0x3e, - 0xd5, 0x8f, 0xe3, 0xbf, 0xb5, 0xaf, 0xfd, 0xa2, 0x00, 0xf5, 0x59, 0x93, 0x88, 0x1c, 0xa8, 0xf6, - 0xe9, 0x11, 0x8e, 0xa3, 0xd0, 0x5d, 0x40, 0x4d, 0xa8, 0xcf, 0x5a, 0x41, 0xb7, 0xa0, 0x1f, 0xdb, - 0xb3, 0x7e, 0xce, 0x5d, 0x44, 0x6d, 0x70, 0x72, 0xed, 0x9a, 0x79, 0xa0, 0xdf, 0xc9, 0x77, 0x5c, - 0x6e, 0x09, 0x2d, 0x83, 0x9b, 0x41, 0x59, 0x5f, 0xe5, 0x96, 0x91, 0x0b, 0x8d, 0x3b, 0xb9, 0xee, - 0xc8, 0xad, 0x28, 0x64, 0x33, 0x0c, 0xf7, 0xb3, 0x1f, 0xd8, 0x5c, 0x95, 0x00, 0x8d, 0x59, 0x43, - 0xa4, 0xbe, 0x57, 0x53, 0xdf, 0x9f, 0xb7, 0x39, 0x6e, 0xfd, 0xda, 0x2d, 0xa8, 0xcf, 0x6e, 0x65, - 0x54, 0x83, 0xd2, 0xe6, 0x54, 0x32, 0xc3, 0xfa, 0x36, 0x33, 0xbf, 0x10, 0x08, 0xb7, 0x80, 0x1a, - 0x50, 0xdb, 0x8a, 0xc6, 0x86, 0xe2, 0x22, 0xba, 0x00, 0xed, 0x1e, 0xa3, 0x32, 0xa2, 0x53, 0x36, - 0x15, 0xfa, 0x47, 0x25, 0xb7, 0xb8, 0xf5, 0xe6, 0xc7, 0x9f, 0x5f, 0x2e, 0x7c, 0xf2, 0xf9, 0xe5, - 0xc2, 0xc3, 0xcf, 0x2f, 0x2f, 0x7c, 0xf8, 0xf7, 0xcb, 0x85, 0xf7, 0xbe, 0x91, 0xfb, 0xb9, 0x3a, - 0xc1, 0x92, 0x47, 0xc7, 0x8c, 0x47, 0xe3, 0x88, 0x66, 0x13, 0x4a, 0xae, 0xa7, 0x93, 0xf1, 0xf5, - 0x74, 0x74, 0x1d, 0xa7, 0xd1, 0xa8, 0xa2, 0x7f, 0x97, 0x7e, 0xf9, 0xdf, 0x01, 0x00, 0x00, 0xff, - 0xff, 0xce, 0x98, 0x3c, 0xea, 0xf5, 0x16, 0x00, 0x00, + // 2684 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x58, 0x4f, 0x6f, 0x1c, 0xc7, + 0xb1, 0xe7, 0x72, 0xff, 0xd7, 0xec, 0x9f, 0x61, 0x93, 0x92, 0xd6, 0xb4, 0x9f, 0xc4, 0xb7, 0x96, + 0x6d, 0x5a, 0x7e, 0xa6, 0xf0, 0x68, 0x3b, 0xb1, 0x0d, 0x21, 0x06, 0xb9, 0xb4, 0xc4, 0x75, 0x48, + 0x91, 0x19, 0x52, 0x36, 0x60, 0x04, 0x18, 0xf4, 0xce, 0xb4, 0x96, 0xa3, 0x9d, 0xe9, 0x1e, 0x75, + 0xf7, 0x4a, 0xa4, 0xaf, 0x89, 0x8f, 0xb9, 0xe4, 0x96, 0x9b, 0xfd, 0x11, 0x02, 0x04, 0xc8, 0x37, + 0x08, 0x7c, 0x74, 0x10, 0xe7, 0x9f, 0x93, 0x38, 0x8a, 0x73, 0x49, 0xf2, 0x29, 0x82, 0xae, 0x9e, + 0xd9, 0x5d, 0x52, 0x94, 0x13, 0x07, 0x01, 0x7c, 0xd8, 0x45, 0xf7, 0xaf, 0xaa, 0xba, 0xab, 0xaa, + 0xab, 0xab, 0xaa, 0x07, 0xea, 0x34, 0x8d, 0xd6, 0x52, 0x29, 0xb4, 0x20, 0x45, 0x9a, 0x46, 0xcb, + 0x2f, 0x0f, 0x23, 0x7d, 0x34, 0x1e, 0xac, 0x05, 0x22, 0xb9, 0x3e, 0x14, 0x43, 0x71, 0x1d, 0x69, + 0x83, 0xf1, 0x5d, 0x9c, 0xe1, 0x04, 0x47, 0x56, 0x66, 0xb9, 0xad, 0xa3, 0x84, 0x29, 0x4d, 0x93, + 0x34, 0x03, 0x20, 0x8d, 0x29, 0xb7, 0xe3, 0xee, 0xb7, 0xa1, 0x79, 0x78, 0x7b, 0x3f, 0xe2, 0x43, + 0x8f, 0xdd, 0x1f, 0x33, 0xa5, 0xc9, 0x33, 0x50, 0x4f, 0xa9, 0xa4, 0x09, 0xd3, 0x4c, 0x76, 0x0a, + 0x2b, 0x85, 0xd5, 0xba, 0x37, 0x05, 0xde, 0xac, 0x7d, 0xf4, 0xf1, 0x95, 0xc2, 0xa3, 0x8f, 0xaf, + 0xcc, 0x75, 0x7f, 0x56, 0x80, 0x56, 0x2e, 0xa9, 0x52, 0xc1, 0x15, 0x23, 0x1d, 0xa8, 0x2a, 0x2d, + 0x24, 0xeb, 0x6f, 0x65, 0x82, 0xf9, 0x94, 0x3c, 0x0f, 0x2d, 0xc5, 0xe4, 0x83, 0x28, 0x60, 0x1b, + 0x61, 0x28, 0x99, 0x52, 0x9d, 0x79, 0x64, 0x38, 0x83, 0xe2, 0x0a, 0x47, 0x54, 0x86, 0xfd, 0xad, + 0x4e, 0x71, 0xa5, 0xb0, 0x5a, 0xf2, 0xf2, 0xa9, 0x51, 0x4b, 0xb2, 0x34, 0x8e, 0x02, 0xda, 0xdf, + 0xea, 0x94, 0x90, 0x36, 0x05, 0xc8, 0x65, 0x80, 0x58, 0x0c, 0x0f, 0x32, 0xd1, 0x32, 0x92, 0x67, + 0x90, 0x19, 0xb5, 0xdf, 0x04, 0xf7, 0xf0, 0xf6, 0x81, 0x96, 0xb3, 0x7a, 0xe3, 0xda, 0x7a, 0x2c, + 0xf9, 0x81, 0x9e, 0x98, 0x3c, 0x01, 0x66, 0x64, 0x7f, 0x5a, 0x80, 0xca, 0xbb, 0x2c, 0xd0, 0x42, + 0x12, 0x02, 0xa5, 0x90, 0x6a, 0x8a, 0xdc, 0x0d, 0x0f, 0xc7, 0xe4, 0x2a, 0x94, 0xf4, 0x49, 0xca, + 0xd0, 0x34, 0x67, 0x1d, 0xd6, 0xd0, 0xcb, 0x87, 0x27, 0x29, 0xdb, 0x2c, 0x7d, 0xf2, 0xc5, 0x95, + 0x39, 0x0f, 0xa9, 0x64, 0x19, 0x6a, 0x7c, 0x1c, 0xc7, 0x74, 0x10, 0x33, 0xb4, 0xb1, 0xe6, 0x4d, + 0xe6, 0xc4, 0x85, 0x22, 0x57, 0x29, 0x9a, 0xd7, 0xf0, 0xcc, 0x90, 0x3c, 0x05, 0xb5, 0x48, 0xf9, + 0x81, 0xe0, 0x4a, 0xa3, 0x59, 0x35, 0xaf, 0x1a, 0xa9, 0x9e, 0x99, 0x1a, 0xe6, 0x98, 0xf1, 0x4e, + 0x65, 0xa5, 0xb0, 0xda, 0xf4, 0xcc, 0xd0, 0x28, 0x45, 0x25, 0xa3, 0x9d, 0xaa, 0x55, 0xca, 0x8c, + 0xbb, 0xef, 0x40, 0x79, 0x93, 0xea, 0xe0, 0x88, 0x2c, 0x43, 0x99, 0x6a, 0x2d, 0x55, 0xa7, 0xb0, + 0x52, 0x5c, 0xad, 0x67, 0x2a, 0x59, 0x88, 0x3c, 0x07, 0xa5, 0x07, 0x2c, 0x30, 0x87, 0x52, 0x5c, + 0x75, 0xd6, 0x9d, 0x35, 0x13, 0x6f, 0xd6, 0xd0, 0x5c, 0x75, 0x43, 0xee, 0xfe, 0xa2, 0x00, 0xd5, + 0x43, 0xa3, 0x68, 0x7f, 0x8b, 0x2c, 0x42, 0x39, 0x1c, 0xf8, 0x51, 0x88, 0x1e, 0x28, 0x79, 0xa5, + 0x70, 0xd0, 0x0f, 0x0d, 0xa8, 0x11, 0x9c, 0xb7, 0xa0, 0x36, 0xe0, 0xff, 0x42, 0x23, 0xa5, 0x52, + 0x47, 0x3a, 0x12, 0xdc, 0xd0, 0xec, 0xc1, 0x3a, 0x13, 0xac, 0x1f, 0x92, 0x0b, 0x50, 0xa1, 0x41, + 0x60, 0x88, 0x25, 0xb4, 0xa6, 0x4c, 0x83, 0xa0, 0x1f, 0x92, 0x4b, 0x50, 0x0d, 0x07, 0x3e, 0xa7, + 0x09, 0x43, 0xdb, 0xeb, 0x5e, 0x25, 0x1c, 0xdc, 0xa6, 0x09, 0x33, 0x04, 0x9d, 0x11, 0x2a, 0x96, + 0xa0, 0x2d, 0xe1, 0x39, 0x68, 0xa5, 0x32, 0x4a, 0xa8, 0x3c, 0xf1, 0x15, 0xbb, 0xcf, 0xc7, 0x09, + 0xfa, 0xa2, 0xe9, 0x35, 0x33, 0xf4, 0x00, 0xc1, 0xee, 0x8f, 0x0b, 0xd0, 0x3a, 0x38, 0xe1, 0xc1, + 0x8e, 0x18, 0x1e, 0xd2, 0x28, 0xf6, 0xd8, 0x7d, 0xf2, 0x32, 0x54, 0x03, 0xee, 0x1f, 0xd1, 0x07, + 0x0c, 0x2d, 0x72, 0xd6, 0x97, 0xd6, 0xa6, 0xd7, 0xe6, 0x30, 0x1f, 0x79, 0x95, 0x80, 0x6f, 0xd3, + 0x07, 0x2c, 0x63, 0x7f, 0x48, 0xb9, 0xce, 0x8e, 0xfb, 0x89, 0xec, 0xef, 0x51, 0xae, 0x49, 0x17, + 0xca, 0x7a, 0x72, 0xe2, 0xce, 0x7a, 0x03, 0x3d, 0x9c, 0xb9, 0xd2, 0xb3, 0xa4, 0xee, 0xf7, 0xa1, + 0x7d, 0x4a, 0x27, 0x95, 0x1a, 0xd7, 0x05, 0xa3, 0xd4, 0x8f, 0x45, 0x40, 0x8d, 0xa7, 0xb2, 0xd8, + 0x74, 0x82, 0x51, 0xba, 0x93, 0x41, 0xe4, 0x79, 0xa8, 0x05, 0x22, 0x49, 0x28, 0x0f, 0xf3, 0xe3, + 0x03, 0x5c, 0xfc, 0x6d, 0xae, 0xe5, 0x89, 0x37, 0xa1, 0x75, 0x15, 0x2c, 0xec, 0x4b, 0x66, 0xa6, + 0x91, 0x7e, 0x4f, 0x46, 0x9a, 0xf5, 0x92, 0x90, 0xbc, 0x08, 0xc0, 0x0c, 0x9f, 0x1f, 0x47, 0x4a, + 0x63, 0x60, 0x9c, 0x16, 0xaf, 0x23, 0x75, 0x27, 0x52, 0x9a, 0xbc, 0x02, 0x17, 0xd5, 0x09, 0x0f, + 0x7c, 0x93, 0x35, 0x58, 0x80, 0x67, 0x79, 0x4f, 0x4c, 0xce, 0xba, 0xee, 0x2d, 0x1a, 0xea, 0xfe, + 0x84, 0xf8, 0x8e, 0x18, 0xf4, 0xc3, 0xee, 0x87, 0x25, 0x28, 0xe3, 0x4a, 0xe4, 0x95, 0x7c, 0x27, + 0xbc, 0x21, 0xc6, 0x8e, 0xd6, 0xfa, 0xd2, 0x74, 0x27, 0xfb, 0x6f, 0xee, 0x4a, 0xb6, 0xa7, 0x19, + 0x9a, 0xe0, 0x47, 0xd7, 0x4c, 0x23, 0xaa, 0x8a, 0xf3, 0x7e, 0x48, 0xae, 0x80, 0x63, 0xee, 0xdc, + 0x80, 0x2a, 0x36, 0x8d, 0x29, 0xc8, 0xa1, 0x7e, 0x48, 0xfe, 0x07, 0xc0, 0xca, 0x62, 0x94, 0x94, + 0xec, 0xa5, 0x46, 0x04, 0x03, 0xe5, 0x59, 0x68, 0x4e, 0xe4, 0x67, 0x02, 0xac, 0x91, 0x83, 0xc8, + 0xf4, 0x34, 0xd4, 0xef, 0x46, 0xf9, 0x12, 0x36, 0xd0, 0x6a, 0x06, 0x40, 0xe2, 0x33, 0x50, 0x1c, + 0x50, 0x8d, 0xf1, 0x95, 0x3b, 0x0d, 0x2f, 0x9a, 0x67, 0x60, 0xf2, 0x2c, 0xb4, 0xd2, 0x91, 0x1f, + 0x1c, 0xb1, 0x60, 0xe4, 0x0f, 0x4e, 0x7c, 0xcd, 0x3b, 0xb5, 0x95, 0xc2, 0x6a, 0xd9, 0x73, 0xd2, + 0x51, 0xcf, 0x80, 0x9b, 0x27, 0x87, 0x9c, 0xbc, 0x06, 0x97, 0x62, 0xfa, 0xc1, 0x89, 0x1f, 0x50, + 0x4d, 0x63, 0x31, 0xf4, 0x69, 0x10, 0x88, 0x31, 0xd7, 0xc6, 0xa0, 0x3a, 0x86, 0xed, 0x92, 0x21, + 0xf7, 0x2c, 0x75, 0xc3, 0x12, 0xfb, 0x21, 0xb9, 0x01, 0x4f, 0x1f, 0x51, 0xe5, 0x3f, 0x49, 0x14, + 0x30, 0x4d, 0x5c, 0x3a, 0xa2, 0x6a, 0xe7, 0x1c, 0xe9, 0xae, 0x84, 0xfa, 0xc4, 0xd9, 0x04, 0xa0, + 0xd2, 0xe7, 0x8a, 0x49, 0xed, 0xce, 0x99, 0xf1, 0x16, 0x8b, 0x99, 0x66, 0x6e, 0xc1, 0x8c, 0xef, + 0xa4, 0x21, 0xd5, 0xcc, 0x9d, 0x27, 0x75, 0x28, 0x6f, 0xc4, 0x9a, 0x49, 0xb7, 0x48, 0x16, 0xa0, + 0x79, 0x90, 0xb2, 0x20, 0xa2, 0x71, 0xc6, 0x59, 0x22, 0x2d, 0x80, 0x2d, 0xaa, 0xe9, 0xde, 0xe0, + 0x1e, 0x0b, 0xb4, 0x5b, 0x26, 0x8b, 0xd0, 0x3e, 0x14, 0xc9, 0x40, 0x69, 0xc1, 0x59, 0x06, 0x56, + 0xba, 0x3f, 0x2c, 0x00, 0xa0, 0xd9, 0xa9, 0x88, 0xb8, 0x26, 0x2f, 0x41, 0x25, 0x89, 0xb8, 0xaf, + 0xd5, 0x57, 0x5e, 0xb5, 0x72, 0x12, 0xf1, 0x43, 0x85, 0xcc, 0xf4, 0xd8, 0x30, 0xcf, 0x7f, 0x25, + 0x33, 0x3d, 0x3e, 0x54, 0xf9, 0xa1, 0x14, 0xcf, 0x3d, 0x14, 0xab, 0x86, 0xf5, 0x47, 0x6f, 0x94, + 0x7e, 0x63, 0x6a, 0x7c, 0x58, 0x00, 0x67, 0x97, 0x69, 0x6a, 0x62, 0xed, 0x9b, 0xd4, 0xe3, 0x1f, + 0x05, 0x70, 0xf1, 0x64, 0x31, 0x11, 0xed, 0x8b, 0x38, 0x0a, 0x4e, 0xc8, 0x1a, 0x2c, 0x1a, 0x65, + 0x84, 0x8a, 0x3e, 0x60, 0xfe, 0xfd, 0x31, 0x8d, 0xe2, 0xe8, 0x2e, 0xb3, 0x59, 0xbe, 0xe9, 0x2d, + 0x24, 0x11, 0xdf, 0x33, 0x94, 0xef, 0xe5, 0x04, 0x72, 0x15, 0x5a, 0x46, 0x1f, 0x31, 0xb8, 0xe7, + 0x0b, 0xce, 0xe4, 0x98, 0xa3, 0x5e, 0x4d, 0xaf, 0x91, 0xd0, 0xe3, 0xbd, 0xc1, 0xbd, 0x3d, 0xc4, + 0xc8, 0x75, 0x58, 0x42, 0x2e, 0x5c, 0x35, 0x61, 0x72, 0xc8, 0x42, 0x23, 0x82, 0x9a, 0x99, 0x65, + 0xe9, 0x31, 0x2e, 0xbb, 0x8b, 0x94, 0xbd, 0xc1, 0x3d, 0x72, 0x15, 0xca, 0x47, 0x11, 0xd7, 0xaa, + 0x53, 0x5a, 0x29, 0xae, 0xb6, 0xd6, 0x5b, 0xa8, 0x3b, 0x92, 0xb7, 0x23, 0xae, 0x3d, 0x4b, 0x24, + 0x2f, 0x82, 0xd1, 0xc8, 0x0f, 0xb8, 0x5d, 0xd3, 0x37, 0x6b, 0x64, 0xd5, 0xbf, 0x95, 0x44, 0xbc, + 0xc7, 0x51, 0xe2, 0x20, 0xfa, 0x80, 0x75, 0x5f, 0x87, 0xa5, 0xa9, 0xad, 0x58, 0x40, 0x25, 0x35, + 0xb1, 0xb8, 0x02, 0x4e, 0x30, 0x99, 0xa9, 0xac, 0x9e, 0xcf, 0x42, 0xdd, 0x97, 0x61, 0x61, 0x56, + 0x32, 0x49, 0x18, 0xd7, 0xa6, 0x51, 0x09, 0xec, 0x30, 0x6f, 0x75, 0xb2, 0x69, 0x77, 0x17, 0x2e, + 0x4c, 0xd9, 0x3d, 0x66, 0x72, 0x07, 0x0e, 0x4d, 0x36, 0x13, 0x71, 0x68, 0x93, 0x49, 0x26, 0x23, + 0xe2, 0x10, 0x73, 0xc9, 0x53, 0x50, 0xe3, 0xec, 0xa1, 0x25, 0xd9, 0x74, 0x5a, 0xe5, 0xec, 0xa1, + 0x21, 0x75, 0x39, 0x2c, 0x9e, 0x5d, 0xae, 0x27, 0xe2, 0xff, 0x6c, 0x31, 0x53, 0x4f, 0x94, 0x69, + 0xf3, 0x78, 0xc0, 0x7c, 0x53, 0x1c, 0xad, 0xfb, 0x9d, 0x1c, 0xbb, 0x3d, 0x4e, 0xba, 0xe1, 0xec, + 0x7e, 0x1b, 0x61, 0xd8, 0x13, 0xf1, 0x38, 0xe1, 0xe4, 0x2a, 0x54, 0x02, 0x1c, 0x65, 0x31, 0xda, + 0xb0, 0xdd, 0x4d, 0x4f, 0xc4, 0x5b, 0xec, 0xae, 0x97, 0xd1, 0xc8, 0x0b, 0xd0, 0x8e, 0x30, 0x9d, + 0xf8, 0xa9, 0x50, 0x58, 0xdc, 0x51, 0x83, 0xb2, 0xd7, 0xb2, 0xf0, 0x7e, 0x86, 0x9e, 0x3e, 0x0d, + 0x8f, 0xa5, 0x31, 0x0d, 0xd8, 0x16, 0xbb, 0x4b, 0x56, 0xa0, 0x18, 0xb2, 0xbb, 0xd9, 0x1e, 0xad, + 0xac, 0x83, 0x32, 0x3c, 0x66, 0x17, 0x43, 0xea, 0xbe, 0x3f, 0x2b, 0xb9, 0x25, 0x45, 0x9a, 0x29, + 0x78, 0x05, 0x9c, 0x58, 0x0c, 0xa3, 0x80, 0xc6, 0x7e, 0x14, 0x1e, 0x67, 0xf1, 0x0a, 0x19, 0xd4, + 0x0f, 0x8f, 0x1f, 0xb3, 0x7d, 0xfe, 0x71, 0xdb, 0x1f, 0x95, 0xa0, 0x39, 0xab, 0xd6, 0xfd, 0x53, + 0x15, 0xa8, 0x70, 0xba, 0x02, 0x4d, 0x1a, 0xa0, 0xf9, 0x99, 0x06, 0xa8, 0x0b, 0xa5, 0x51, 0xc4, + 0x6d, 0x3d, 0xca, 0xa3, 0x16, 0x57, 0xfc, 0x6e, 0xc4, 0x43, 0x0f, 0x69, 0xe4, 0x0d, 0x00, 0x1a, + 0x86, 0x7e, 0xe6, 0xce, 0x12, 0x9a, 0xda, 0x99, 0x72, 0x9e, 0x76, 0xfc, 0xf6, 0x9c, 0x57, 0xa7, + 0x93, 0x53, 0xb8, 0x01, 0x4e, 0x28, 0x45, 0x9a, 0xcb, 0x96, 0x51, 0xf6, 0xa9, 0x33, 0xb2, 0x53, + 0xa7, 0x6c, 0xcf, 0x79, 0x10, 0x4e, 0x5d, 0xf4, 0x16, 0x34, 0x24, 0x06, 0x90, 0x6f, 0x7b, 0x91, + 0x0a, 0x8a, 0x2f, 0x9f, 0x11, 0x9f, 0x09, 0xd9, 0xed, 0x39, 0xcf, 0x91, 0x33, 0x11, 0xfc, 0x16, + 0xb4, 0xc6, 0x58, 0x15, 0xfc, 0x3c, 0xf6, 0x6d, 0xf5, 0xbb, 0x78, 0x66, 0x89, 0xec, 0x92, 0x6c, + 0xcf, 0x79, 0x4d, 0xcb, 0x9f, 0xdf, 0x9a, 0x1b, 0xe0, 0xe4, 0x0b, 0x28, 0x2d, 0xb1, 0x24, 0x3e, + 0xae, 0xff, 0xf4, 0x72, 0x1a, 0xfd, 0xb3, 0x05, 0x94, 0x96, 0xe4, 0x06, 0x64, 0xcb, 0xf9, 0x29, + 0xe6, 0x2a, 0x2c, 0x92, 0xce, 0xfa, 0x85, 0x33, 0xf2, 0x36, 0x91, 0x6d, 0xcf, 0x79, 0x0d, 0xcb, + 0x9d, 0x25, 0xb6, 0x37, 0x00, 0x32, 0xeb, 0x03, 0x11, 0x77, 0x9c, 0x73, 0xdd, 0x3e, 0xb9, 0x5f, + 0xc6, 0xed, 0x72, 0x72, 0xd9, 0x6e, 0x80, 0x23, 0x6d, 0x8c, 0xfa, 0x26, 0x3a, 0x1b, 0xe7, 0xaa, + 0x3d, 0x8d, 0x62, 0xa3, 0xb6, 0x9c, 0xcc, 0x36, 0x1d, 0xa8, 0x8b, 0x94, 0x49, 0x6c, 0xd7, 0xba, + 0x9f, 0x97, 0xc0, 0x39, 0x08, 0x8e, 0x58, 0x42, 0xdf, 0x3e, 0xd6, 0x92, 0x92, 0xe7, 0xa1, 0xcd, + 0xd9, 0xb1, 0x36, 0x3a, 0xe5, 0x1d, 0xab, 0x0d, 0xdd, 0xa6, 0x81, 0x7b, 0x22, 0xb6, 0x1d, 0x2b, + 0xf6, 0x2b, 0x52, 0xa4, 0x29, 0x0b, 0x7d, 0xdb, 0xc5, 0x9b, 0x5e, 0xcf, 0xf4, 0x2b, 0x16, 0xdc, + 0xc8, 0xda, 0xf8, 0x96, 0x8d, 0x0c, 0x3f, 0x38, 0xa2, 0x7c, 0xc8, 0xc2, 0xec, 0x81, 0xd1, 0xb4, + 0x68, 0xcf, 0x82, 0xa7, 0x72, 0x47, 0xe9, 0x74, 0xee, 0x78, 0x42, 0xf6, 0x2f, 0xff, 0xfb, 0xd9, + 0xbf, 0xf2, 0x35, 0xb2, 0x7f, 0xf5, 0x5f, 0x66, 0xff, 0xda, 0xd7, 0xce, 0xfe, 0xf5, 0xf3, 0xb2, + 0xbf, 0xd1, 0x73, 0x10, 0x8b, 0x60, 0xe4, 0x1b, 0x3d, 0xa4, 0x78, 0xa8, 0xb0, 0x4b, 0x6a, 0x7a, + 0x0d, 0x44, 0x77, 0xe9, 0xb1, 0x27, 0x1e, 0x2a, 0x72, 0x0d, 0x16, 0x04, 0xb6, 0x2c, 0xc8, 0x86, + 0x24, 0x85, 0x91, 0xd2, 0xf4, 0xda, 0x96, 0xb0, 0x4b, 0x8f, 0x37, 0x11, 0x36, 0x75, 0xe3, 0x26, + 0xa3, 0x7a, 0x2c, 0xd9, 0xcd, 0x98, 0x0e, 0x31, 0x26, 0x4a, 0xde, 0x2c, 0x64, 0x38, 0xfa, 0x3c, + 0x64, 0xc7, 0x18, 0x1d, 0xaa, 0xd3, 0x5c, 0x29, 0x1a, 0x8e, 0x19, 0x88, 0x5c, 0x85, 0xe6, 0x3e, + 0x95, 0x8c, 0xeb, 0xec, 0x25, 0xd0, 0x69, 0xe1, 0x2a, 0xa7, 0x41, 0xb2, 0x0a, 0xed, 0x9b, 0x52, + 0x24, 0xfb, 0xe3, 0x81, 0x79, 0xec, 0x62, 0x52, 0x6d, 0xe3, 0xb1, 0x9e, 0x85, 0xbb, 0x21, 0xd4, + 0xfa, 0x5c, 0x7f, 0xeb, 0xd5, 0x5d, 0x9a, 0x92, 0x2e, 0x14, 0x92, 0xac, 0xa3, 0xb7, 0x7d, 0x76, + 0x4e, 0x59, 0xdb, 0xb5, 0xbd, 0x7d, 0x21, 0x59, 0x7e, 0x15, 0x2a, 0x76, 0x62, 0xde, 0x92, 0x23, + 0x76, 0x82, 0xa1, 0x57, 0xf4, 0xcc, 0x90, 0x2c, 0x41, 0xf9, 0x01, 0x8d, 0xc7, 0xb6, 0x84, 0x14, + 0x3d, 0x3b, 0x79, 0x73, 0xfe, 0xf5, 0x42, 0xf7, 0x5d, 0x68, 0x1c, 0x4a, 0xca, 0xd5, 0x16, 0x53, + 0x26, 0xa1, 0x93, 0x8b, 0x50, 0x11, 0x83, 0x7b, 0xfd, 0x2c, 0xe9, 0x96, 0xbd, 0x6c, 0x66, 0xf0, + 0x41, 0x3c, 0x32, 0xb8, 0xad, 0x01, 0xd9, 0xcc, 0xe0, 0x52, 0x3c, 0x34, 0x78, 0xd1, 0xe2, 0x76, + 0xd6, 0xfd, 0x41, 0x01, 0x9c, 0xcd, 0x78, 0x84, 0x6b, 0x1b, 0x0b, 0x5e, 0x9a, 0x5a, 0x70, 0xc9, + 0xb6, 0x2e, 0x53, 0x62, 0x66, 0x44, 0xf6, 0x3a, 0x2d, 0x24, 0xcb, 0xb7, 0xce, 0x33, 0xa5, 0x6c, + 0x4d, 0x79, 0x61, 0xd6, 0x14, 0x67, 0x7d, 0xc1, 0x3e, 0xbe, 0x66, 0x4c, 0x98, 0xb5, 0x6e, 0x1b, + 0x48, 0xbe, 0xcf, 0x5d, 0x26, 0x37, 0x85, 0x18, 0x45, 0x7c, 0x48, 0xd6, 0xa1, 0x96, 0xd0, 0x34, + 0x8d, 0xf8, 0x50, 0x65, 0x2a, 0xb9, 0x67, 0x55, 0xca, 0x74, 0x99, 0xf0, 0x75, 0x3f, 0x9b, 0x07, + 0x17, 0x23, 0xb0, 0x87, 0x8f, 0x2e, 0xab, 0xdd, 0xb9, 0xcf, 0xe6, 0x0b, 0x50, 0xd1, 0x83, 0x78, + 0x5a, 0x4b, 0xca, 0x7a, 0x10, 0x3f, 0xf6, 0x84, 0x29, 0x9e, 0x7d, 0xc2, 0xbc, 0x06, 0x35, 0xa5, + 0xa9, 0xd4, 0x3e, 0x76, 0x49, 0x4f, 0xec, 0x05, 0x33, 0xbd, 0xaa, 0xc8, 0x7b, 0xa8, 0x4c, 0xa1, + 0x9c, 0x5e, 0x41, 0xd5, 0x29, 0xaf, 0x14, 0x57, 0x1b, 0x1e, 0x24, 0xf9, 0xdd, 0x53, 0xf8, 0xe8, + 0x94, 0x8c, 0xea, 0x9c, 0xa3, 0x82, 0x1c, 0x4e, 0x86, 0x21, 0xcb, 0xff, 0x43, 0x75, 0x60, 0x3d, + 0x93, 0x55, 0x80, 0xd3, 0x07, 0x34, 0x75, 0x9c, 0x97, 0xf3, 0x99, 0x6d, 0xb3, 0xa1, 0x79, 0xce, + 0xe2, 0xc5, 0xae, 0x7b, 0x90, 0x41, 0x3b, 0x22, 0x30, 0xe7, 0xc6, 0xa4, 0xc4, 0xfb, 0x5b, 0xf7, + 0xcc, 0xd0, 0x84, 0x60, 0xcc, 0x1e, 0xb0, 0x18, 0xef, 0x6a, 0xd9, 0xb3, 0x93, 0xee, 0x4f, 0xe6, + 0xa1, 0x85, 0x6e, 0x3d, 0xa4, 0x6a, 0xf4, 0x5f, 0x77, 0xea, 0xcc, 0x27, 0x87, 0xd2, 0xa9, 0x4f, + 0x0e, 0x5d, 0x68, 0x6a, 0x91, 0x25, 0x9a, 0x19, 0xc7, 0x39, 0x5a, 0xa0, 0x32, 0xe8, 0x96, 0x35, + 0x58, 0x64, 0x4a, 0x47, 0x09, 0xfa, 0x2e, 0x61, 0x89, 0x3f, 0x56, 0x74, 0x68, 0xeb, 0x6c, 0xc9, + 0x5b, 0x98, 0x90, 0x76, 0x59, 0x72, 0xc7, 0x10, 0x8c, 0x2e, 0x33, 0xef, 0x36, 0x9b, 0x0d, 0xeb, + 0x74, 0xf2, 0xce, 0xbb, 0x04, 0xd5, 0xb1, 0x62, 0xd2, 0xd0, 0x6a, 0x48, 0xab, 0x98, 0xa9, 0x25, + 0x48, 0x61, 0x9b, 0x12, 0xfb, 0x4e, 0xac, 0x98, 0x69, 0x3f, 0xec, 0xde, 0x86, 0xd6, 0xf4, 0x99, + 0x85, 0x5f, 0x10, 0x96, 0xa1, 0xb6, 0x73, 0xfa, 0xeb, 0xc1, 0x64, 0x6e, 0x12, 0x94, 0x96, 0x63, + 0x1e, 0x50, 0xcd, 0x76, 0x14, 0xcf, 0xdc, 0x34, 0x0b, 0x5d, 0xfb, 0x51, 0x11, 0x2a, 0x7b, 0x69, + 0x4f, 0x84, 0x8c, 0x54, 0xa1, 0x78, 0x5b, 0xa4, 0xee, 0x1c, 0x59, 0x80, 0xc6, 0x5e, 0x7a, 0x8b, + 0xe9, 0xec, 0x3b, 0x85, 0xfb, 0xb7, 0x2a, 0x71, 0xc1, 0xd9, 0x4b, 0xf7, 0x65, 0x16, 0xe8, 0xee, + 0xdf, 0xab, 0xc4, 0x31, 0x72, 0xfb, 0x11, 0x1f, 0xba, 0x9f, 0xb6, 0x49, 0x03, 0xaa, 0x7b, 0xe9, + 0xcd, 0x78, 0xac, 0x8e, 0xdc, 0x5f, 0xb6, 0xad, 0xfc, 0x54, 0x4b, 0xf7, 0x57, 0x6d, 0xd2, 0x82, + 0xfa, 0x5e, 0xda, 0xe7, 0x2a, 0x35, 0xaf, 0xc5, 0xcf, 0xda, 0x64, 0x09, 0xda, 0x7b, 0xe9, 0x46, + 0x18, 0xde, 0xa4, 0xe3, 0x58, 0xef, 0x23, 0xd7, 0xaf, 0xdb, 0xa4, 0x09, 0xb5, 0xbd, 0x74, 0x93, + 0x06, 0xa3, 0x71, 0xea, 0xfe, 0xa6, 0x6d, 0x37, 0x3d, 0x94, 0x34, 0x60, 0x07, 0x29, 0xe5, 0xee, + 0x6f, 0xdb, 0x64, 0x11, 0x5a, 0x7b, 0xe9, 0x81, 0x16, 0x92, 0x0e, 0x19, 0x3a, 0xd8, 0xfd, 0x5d, + 0x9b, 0x5c, 0x02, 0xb2, 0x97, 0xde, 0x8a, 0xc5, 0x80, 0xc6, 0x33, 0x9b, 0xfe, 0xbe, 0x4d, 0x2e, + 0xc2, 0x82, 0xd9, 0x54, 0x33, 0x19, 0xb0, 0x54, 0x67, 0xaa, 0x7f, 0xde, 0x26, 0x04, 0x9a, 0xc6, + 0x64, 0x33, 0xc5, 0x93, 0x75, 0xff, 0x90, 0xf1, 0x6e, 0x45, 0x6a, 0x64, 0x7e, 0xbd, 0x98, 0x51, + 0xce, 0xa4, 0xfb, 0xc7, 0x4c, 0x25, 0x8f, 0xd1, 0x90, 0x49, 0xf7, 0x4f, 0x6d, 0xb2, 0x0c, 0x17, + 0xac, 0x6b, 0xa8, 0x66, 0x4a, 0xcf, 0x6c, 0xf7, 0x45, 0xae, 0x1c, 0xa7, 0xa9, 0x3a, 0x12, 0xda, + 0x88, 0xb8, 0x7f, 0x9e, 0x0a, 0x64, 0x95, 0x19, 0x33, 0xfe, 0x4e, 0xa4, 0xb4, 0xfb, 0x28, 0xd3, + 0x03, 0x3d, 0xd0, 0xe7, 0xf8, 0x8c, 0xfe, 0x4b, 0xfb, 0xda, 0xcf, 0x0b, 0x50, 0x9f, 0xb4, 0x93, + 0xc4, 0x81, 0x6a, 0x9f, 0x3f, 0xa0, 0x71, 0x14, 0xba, 0x73, 0xa4, 0x09, 0xf5, 0x49, 0xd3, 0xe8, + 0x16, 0xf0, 0x59, 0x3e, 0xe9, 0xfc, 0xdc, 0x79, 0xd2, 0x06, 0x67, 0xa6, 0xb1, 0xb3, 0x4f, 0xf9, + 0x3b, 0xb3, 0xbd, 0x99, 0x5b, 0x22, 0x4b, 0xe0, 0xe6, 0x50, 0xde, 0x81, 0xb9, 0x65, 0xe2, 0x42, + 0xe3, 0xce, 0x4c, 0x1f, 0xe5, 0x56, 0x0c, 0xb2, 0x11, 0x86, 0xfb, 0xf9, 0xf7, 0x3b, 0xd7, 0x04, + 0x40, 0x63, 0xd2, 0x3a, 0x99, 0xfd, 0x6a, 0x66, 0xff, 0x69, 0x43, 0xe4, 0xd6, 0xaf, 0xdd, 0x82, + 0xfa, 0xa4, 0x7e, 0x93, 0x1a, 0x94, 0x36, 0xc6, 0x5a, 0x58, 0xad, 0x6f, 0x0b, 0xfb, 0x2d, 0x41, + 0xb9, 0x05, 0xd2, 0x80, 0xda, 0x66, 0x34, 0xb4, 0x2a, 0xce, 0x93, 0x45, 0x68, 0xf7, 0x04, 0xd7, + 0x11, 0x1f, 0x8b, 0xb1, 0xc2, 0x6f, 0x56, 0x6e, 0x71, 0xf3, 0x3b, 0x9f, 0x7c, 0x79, 0xb9, 0xf0, + 0xe9, 0x97, 0x97, 0x0b, 0x8f, 0xbe, 0xbc, 0x3c, 0xf7, 0xd1, 0x5f, 0x2f, 0x17, 0xde, 0xff, 0xbf, + 0x99, 0xaf, 0xe1, 0x09, 0xd5, 0x32, 0x3a, 0x16, 0x32, 0x1a, 0x46, 0x3c, 0x9f, 0x70, 0x76, 0x3d, + 0x1d, 0x0d, 0xaf, 0xa7, 0x83, 0xeb, 0x34, 0x8d, 0x06, 0x15, 0xfc, 0xec, 0xfd, 0xca, 0x3f, 0x03, + 0x00, 0x00, 0xff, 0xff, 0x09, 0x35, 0x33, 0xc6, 0x54, 0x17, 0x00, 0x00, } func (m *TNPingRequest) Marshal() (dAtA []byte, err error) { @@ -3097,6 +3119,13 @@ func (m *PrecommitWriteCmd) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if len(m.SyncProtectionJobId) > 0 { + i -= len(m.SyncProtectionJobId) + copy(dAtA[i:], m.SyncProtectionJobId) + i = encodeVarintApi(dAtA, i, uint64(len(m.SyncProtectionJobId))) + i-- + dAtA[i] = 0x12 + } if len(m.EntryList) > 0 { for iNdEx := len(m.EntryList) - 1; iNdEx >= 0; iNdEx-- { { @@ -3981,6 +4010,16 @@ func (m *SchemaExtra) MarshalToSizedBuffer(dAtA []byte) (int, error) { i -= len(m.XXX_unrecognized) copy(dAtA[i:], m.XXX_unrecognized) } + if m.FromPublication { + i-- + if m.FromPublication { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x78 + } if m.ParentTableID != 0 { i = encodeVarintApi(dAtA, i, uint64(m.ParentTableID)) i-- @@ -4713,6 +4752,10 @@ func (m *PrecommitWriteCmd) ProtoSize() (n int) { n += 1 + l + sovApi(uint64(l)) } } + l = len(m.SyncProtectionJobId) + if l > 0 { + n += 1 + l + sovApi(uint64(l)) + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -5176,6 +5219,9 @@ func (m *SchemaExtra) ProtoSize() (n int) { if m.ParentTableID != 0 { n += 1 + sovApi(uint64(m.ParentTableID)) } + if m.FromPublication { + n += 2 + } if m.XXX_unrecognized != nil { n += len(m.XXX_unrecognized) } @@ -6633,6 +6679,38 @@ func (m *PrecommitWriteCmd) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SyncProtectionJobId", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthApi + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthApi + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.SyncProtectionJobId = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) @@ -9142,6 +9220,26 @@ func (m *SchemaExtra) Unmarshal(dAtA []byte) error { break } } + case 15: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field FromPublication", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowApi + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.FromPublication = bool(v != 0) default: iNdEx = preIndex skippy, err := skipApi(dAtA[iNdEx:]) diff --git a/pkg/sql/plan/function/ctl/types.go b/pkg/sql/plan/function/ctl/types.go index 77365b1d44e3f..bd7a6b5df0174 100644 --- a/pkg/sql/plan/function/ctl/types.go +++ b/pkg/sql/plan/function/ctl/types.go @@ -28,39 +28,39 @@ var ( tn serviceType = "DN" cn serviceType = "CN" - PingMethod = "PING" - FlushMethod = "FLUSH" - TaskMethod = "TASK" - UseSnapshotMethod = "USESNAPSHOT" - GetSnapshotMethod = "GETSNAPSHOT" - CheckpointMethod = "CHECKPOINT" - GlobalCheckpointMethod = "GLOBALCHECKPOINT" - ForceGCMethod = "FORCEGC" - InspectMethod = "INSPECT" - LabelMethod = "LABEL" - WorkStateMethod = "WORKSTATE" - SyncCommitMethod = "SYNCCOMMIT" - AddFaultPointMethod = "ADDFAULTPOINT" - BackupMethod = "BACKUP" - TraceSpanMethod = "TRACESPAN" - CoreDumpMethod = "COREDUMP" - InterceptCommitMethod = "INTERCEPTCOMMIT" - MergeObjectsMethod = "MERGEOBJECTS" - DiskCleanerMethod = "DISKCLEANER" - GetProtocolVersionMethod = "GETPROTOCOLVERSION" - SetProtocolVersionMethod = "SETPROTOCOLVERSION" - RemoveRemoteLockTable = strings.ToUpper("RemoveRemoteLockTable") - GetLatestBind = strings.ToUpper("GetLatestBind") - UnsubscribeTable = "UNSUBSCRIBE_TABLE" - HandleTxnTrace = strings.ToUpper("txn-trace") - ReloadAutoIncrementCache = strings.ToUpper("reload-auto-increment-cache") - CtlReaderMethod = strings.ToUpper("reader") - GetTableShards = strings.ToUpper("get-table-shards") - MoTableStats = strings.ToUpper("MoTableStats") - WorkspaceThreshold = strings.ToUpper("WorkspaceThreshold") - TableExtra = strings.ToUpper("table-extra") - PrefetchOnSubscribed = strings.ToUpper("prefetch-on-subscribed") - ActivateTenantCatalogMethod = strings.ToUpper("ActivateTenantCatalog") + PingMethod = "PING" + FlushMethod = "FLUSH" + TaskMethod = "TASK" + UseSnapshotMethod = "USESNAPSHOT" + GetSnapshotMethod = "GETSNAPSHOT" + CheckpointMethod = "CHECKPOINT" + GlobalCheckpointMethod = "GLOBALCHECKPOINT" + ForceGCMethod = "FORCEGC" + InspectMethod = "INSPECT" + LabelMethod = "LABEL" + WorkStateMethod = "WORKSTATE" + SyncCommitMethod = "SYNCCOMMIT" + AddFaultPointMethod = "ADDFAULTPOINT" + BackupMethod = "BACKUP" + TraceSpanMethod = "TRACESPAN" + CoreDumpMethod = "COREDUMP" + InterceptCommitMethod = "INTERCEPTCOMMIT" + MergeObjectsMethod = "MERGEOBJECTS" + DiskCleanerMethod = "DISKCLEANER" + GetProtocolVersionMethod = "GETPROTOCOLVERSION" + SetProtocolVersionMethod = "SETPROTOCOLVERSION" + RemoveRemoteLockTable = strings.ToUpper("RemoveRemoteLockTable") + GetLatestBind = strings.ToUpper("GetLatestBind") + UnsubscribeTable = "UNSUBSCRIBE_TABLE" + HandleTxnTrace = strings.ToUpper("txn-trace") + ReloadAutoIncrementCache = strings.ToUpper("reload-auto-increment-cache") + CtlReaderMethod = strings.ToUpper("reader") + GetTableShards = strings.ToUpper("get-table-shards") + MoTableStats = strings.ToUpper("MoTableStats") + WorkspaceThreshold = strings.ToUpper("WorkspaceThreshold") + TableExtra = strings.ToUpper("table-extra") + PrefetchOnSubscribed = strings.ToUpper("prefetch-on-subscribed") + ActivateTenantCatalogMethod = strings.ToUpper("ActivateTenantCatalog") ) var ( @@ -73,39 +73,39 @@ var ( var ( // register all supported debug command here supportedCmds = map[string]handleFunc{ - PingMethod: handlePing(), - FlushMethod: handleFlush(), - TaskMethod: handleTask, - UseSnapshotMethod: handleUseSnapshotTS, - GetSnapshotMethod: handleGetSnapshotTS, - CheckpointMethod: handleCheckpoint(), - GlobalCheckpointMethod: handleGlobalCheckpoint(), - ForceGCMethod: handleCNGC, - InspectMethod: handleInspectTN(), - LabelMethod: handleSetLabel, - WorkStateMethod: handleSetWorkState, - SyncCommitMethod: handleSyncCommit, - AddFaultPointMethod: handleAddFaultPoint(), - BackupMethod: handleBackup(), - TraceSpanMethod: handleTraceSpan, - CoreDumpMethod: handleCoreDump, - InterceptCommitMethod: handleInterceptCommit(), - MergeObjectsMethod: handleCNMerge, - DiskCleanerMethod: handleDiskCleaner(), - GetProtocolVersionMethod: handleGetProtocolVersion, - SetProtocolVersionMethod: handleSetProtocolVersion, - RemoveRemoteLockTable: handleRemoveRemoteLockTable, - GetLatestBind: handleGetLatestBind, - UnsubscribeTable: handleUnsubscribeTable, - HandleTxnTrace: handleTxnTrace, - ReloadAutoIncrementCache: handleReloadAutoIncrementCache, - CtlReaderMethod: handleCtlReader, - GetTableShards: handleGetTableShards, - MoTableStats: handleMoTableStats, - WorkspaceThreshold: handleWorkspaceThreshold, - TableExtra: handleTableExtra, - PrefetchOnSubscribed: handlePrefetchOnSubscribed, - ActivateTenantCatalogMethod: handleActivateTenantCatalog, + PingMethod: handlePing(), + FlushMethod: handleFlush(), + TaskMethod: handleTask, + UseSnapshotMethod: handleUseSnapshotTS, + GetSnapshotMethod: handleGetSnapshotTS, + CheckpointMethod: handleCheckpoint(), + GlobalCheckpointMethod: handleGlobalCheckpoint(), + ForceGCMethod: handleCNGC, + InspectMethod: handleInspectTN(), + LabelMethod: handleSetLabel, + WorkStateMethod: handleSetWorkState, + SyncCommitMethod: handleSyncCommit, + AddFaultPointMethod: handleAddFaultPoint(), + BackupMethod: handleBackup(), + TraceSpanMethod: handleTraceSpan, + CoreDumpMethod: handleCoreDump, + InterceptCommitMethod: handleInterceptCommit(), + MergeObjectsMethod: handleCNMerge, + DiskCleanerMethod: handleDiskCleaner(), + GetProtocolVersionMethod: handleGetProtocolVersion, + SetProtocolVersionMethod: handleSetProtocolVersion, + RemoveRemoteLockTable: handleRemoveRemoteLockTable, + GetLatestBind: handleGetLatestBind, + UnsubscribeTable: handleUnsubscribeTable, + HandleTxnTrace: handleTxnTrace, + ReloadAutoIncrementCache: handleReloadAutoIncrementCache, + CtlReaderMethod: handleCtlReader, + GetTableShards: handleGetTableShards, + MoTableStats: handleMoTableStats, + WorkspaceThreshold: handleWorkspaceThreshold, + TableExtra: handleTableExtra, + PrefetchOnSubscribed: handlePrefetchOnSubscribed, + ActivateTenantCatalogMethod: handleActivateTenantCatalog, } ) diff --git a/pkg/vm/engine/disttae/logtail_consumer.go b/pkg/vm/engine/disttae/logtail_consumer.go index 19be52c98855a..1a492526893e3 100644 --- a/pkg/vm/engine/disttae/logtail_consumer.go +++ b/pkg/vm/engine/disttae/logtail_consumer.go @@ -31,9 +31,9 @@ import ( "github.com/matrixorigin/matrixone/pkg/catalog" "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/common/morpc" - "github.com/matrixorigin/matrixone/pkg/defines" "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" + "github.com/matrixorigin/matrixone/pkg/defines" "github.com/matrixorigin/matrixone/pkg/logutil" "github.com/matrixorigin/matrixone/pkg/objectio" "github.com/matrixorigin/matrixone/pkg/pb/api" @@ -1118,7 +1118,7 @@ func (c *PushClient) doActivateTenantCatalog(ctx context.Context, e *Engine, acc Phase: phase, Result: result, TargetTS: targetTS, ReplayTS: replayTS, DelayedApplyCount: delayedCount, - StartedAt: &startedAt, FinishedAt: &now, + StartedAt: &startedAt, FinishedAt: &now, } if err != nil { evt.Error = err.Error() diff --git a/pkg/vm/engine/tae/logtail/service/server.go b/pkg/vm/engine/tae/logtail/service/server.go index 79bc61586c3bd..eb0f78c0c7728 100644 --- a/pkg/vm/engine/tae/logtail/service/server.go +++ b/pkg/vm/engine/tae/logtail/service/server.go @@ -639,7 +639,7 @@ func (s *LogtailServer) pullActivationPhase1(ctx context.Context, act catalogAct waterline, allowedAccounts, false, // activation: keep object meta so HandleDataObjectList GC - // can clean up rows that overlap with concurrent flushes + // can clean up rows that overlap with concurrent flushes ) mu.Lock() defer mu.Unlock() From f92449cbd74faca8f7332cc3d86b2bf7a722e231 Mon Sep 17 00:00:00 2001 From: aptend Date: Fri, 24 Apr 2026 11:34:36 +0800 Subject: [PATCH 05/11] fix: use ts-relative drop check in visitObjMeta to avoid aobj race Previously visitObjMeta used the global HasDropCommitted() flag to decide whether to skip an appendable object's in-memory data scan. This races with concurrent compaction: if the drop commits AFTER the scan-end of the current logtail window, the aobj was still alive at scan-end and its rows must be included in the response, but the global flag causes us to skip the scan. The newly created merged nobj has CreateNode > scan-end, so both objects fall outside the window and the rows are silently lost. This was the root cause of flaky failures in the high-concurrent CREATE/ DROP ACCOUNT tests: mo_columns aobj drops committed a few ms after the activation-pull scan-end, causing CN to receive an empty-column TableDef for information_schema.character_sets, which later surfaced as 'no such table information_schema.character_sets' during drop-account FK cleanup. CI evidence: mo-auto-test run 24869137344 job 72813171794 step 7 shows logtail.pull.skip.just.dropped.aobj firing on mo_columns ~1.5s before the client-visible 'no such table' error. Fix: skip the data scan only if DeleteNode is committed AND its End <= scan-end, so the decision is ts-relative to the scan window, matching the semantics already used by ForeachMVCCNodeInRange for metadata. Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com> --- pkg/vm/engine/tae/logtail/handle.go | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/pkg/vm/engine/tae/logtail/handle.go b/pkg/vm/engine/tae/logtail/handle.go index c0158b6957909..fbcac0684033c 100644 --- a/pkg/vm/engine/tae/logtail/handle.go +++ b/pkg/vm/engine/tae/logtail/handle.go @@ -260,8 +260,19 @@ func (b *TableLogtailRespBuilder) visitObjMeta(e *catalog.ObjectEntry) (bool, er return nil }) - if e.IsAppendable() && !e.HasDropCommitted() { - return false, nil + if e.IsAppendable() { + // Scan the aobj's in-memory rows unless its drop is visible within + // our snapshot window [b.start, b.end]. Using the global + // HasDropCommitted() races with concurrent compaction: if the drop + // commits AFTER b.end, the aobj is still alive at b.end and its rows + // must be included in this pull. Otherwise both this aobj (skipped) + // and the new merged nobj (CreateNode > b.end, so also out of range) + // fall outside the window, and the rows are silently dropped from the + // response. This was the root cause of the lazy-catalog partial-rows + // bug seen in mo_columns during high-concurrent CREATE/DROP ACCOUNT. + if e.DeleteNode.IsEmpty() || !e.DeleteNode.IsCommitted() || e.DeleteNode.End.GT(&b.end) { + return false, nil + } } return true, nil } From b6995c8b895d8daca615790cd45e69b9d7e625f2 Mon Sep 17 00:00:00 2001 From: aptend Date: Fri, 24 Apr 2026 19:18:29 +0800 Subject: [PATCH 06/11] fix(tae/tables): collect aobj rows from pnode after flush-on-logtail When TN's HandleSyncLogTailReq hits a tail larger than Size90M it calls FlushTable and retries the same (from, end) range. The flush upgrades the appendable object's mnode to a pnode and writes a DeleteNode at flushTS > end. On retry visitObjMeta still emits the aobj meta (its CreateNode falls in range, its DeleteNode does not), but baseObject.ScanInMemory returned nil for any persisted node, dropping all of the aobj's row data from the retry tail. For lazy catalog (mo_database / mo_tables / mo_columns) this silently produced 0-row activation replays on CN, which later manifested as 'delete table ... failed' / 'no such table' panics during concurrent account creation. Fix: add persistedNode.ScanDataInRange, modeled after CollectObjectTombstoneInRange, that reads per-row data + commit_ts from the flushed aobj's block and filters by [start, end]. Wire it into ScanInMemory for appendable, non-tombstone persisted objects. Non-appendable persisted objects stay meta-only (unchanged). Lock the behavior down with TestCollectInsertAfterFlush: append three transactions into an aobj, flush it, then verify RangeScanInMemoryByObject still returns the same 6 / 9 / 3 / 6 row windows for [0,p1] / [0,p2] / (p1,p2] / (p1,p3] that TestCollectInsert checks before the flush. Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com> --- pkg/vm/engine/tae/db/test/db_test.go | 89 +++++++++++++++++- pkg/vm/engine/tae/tables/base.go | 8 ++ pkg/vm/engine/tae/tables/pnode.go | 130 +++++++++++++++++++++++++++ 3 files changed, 226 insertions(+), 1 deletion(-) diff --git a/pkg/vm/engine/tae/db/test/db_test.go b/pkg/vm/engine/tae/db/test/db_test.go index 225751c0c20c5..4d92e8f6665e8 100644 --- a/pkg/vm/engine/tae/db/test/db_test.go +++ b/pkg/vm/engine/tae/db/test/db_test.go @@ -4214,10 +4214,97 @@ func TestCollectInsert(t *testing.T) { batches[schema.Version].Close() } -func TestAppendnode(t *testing.T) { +// TestCollectInsertAfterFlush appends three transactions into an appendable +// object, flushes that aobj so its in-memory node is upgraded to a persisted +// node, and then verifies that RangeScanInMemoryByObject (used by logtail +// emission) still returns the same row windows it would have returned before +// the flush. This locks in persistedNode.ScanDataInRange's behavior: if the +// pnode branch returns nil again, rows disappear from logtail tails during +// concurrent aobj flushes (the lazy-catalog bug this path was added to fix). +func TestCollectInsertAfterFlush(t *testing.T) { defer testutils.AfterTest(t)() ctx := context.Background() + opts := config.WithLongScanAndCKPOpts(nil) + tae := testutil.NewTestEngine(ctx, ModuleName, t, opts) + defer tae.Close() + + worker := ops.NewOpWorker(ctx, "flush-for-scan-range") + worker.Start() + defer worker.Stop() + + schema := catalog.MockSchemaAll(1, -1) + schema.Extra.BlockMaxRows = 20 + tae.BindSchema(schema) + bat := catalog.MockBatch(schema, 12) + bats := bat.Split(4) + + tae.CreateRelAndAppend(bats[0], true) + + txn1, rel := tae.GetRelation() + assert.NoError(t, rel.Append(ctx, bats[1])) + assert.NoError(t, txn1.Commit(ctx)) + p1 := txn1.GetPrepareTS() + + txn2, rel := tae.GetRelation() + assert.NoError(t, rel.Append(ctx, bats[2])) + assert.NoError(t, txn2.Commit(ctx)) + p2 := txn2.GetPrepareTS() + + txn3, rel := tae.GetRelation() + assert.NoError(t, rel.Append(ctx, bats[3])) + assert.NoError(t, txn3.Commit(ctx)) + p3 := txn3.GetPrepareTS() + + _, rel = tae.GetRelation() + objEntry := testutil.GetOneObject(rel).GetMeta().(*catalog.ObjectEntry) + require.True(t, objEntry.IsAppendable(), "expected appendable object before flush") + + flushTxn, _ := tae.GetRelation() + task, err := jobs.NewFlushTableTailTask( + tasks.WaitableCtx, flushTxn, + []*catalog.ObjectEntry{objEntry}, + nil, + tae.DB.Runtime, + ) + assert.NoError(t, err) + worker.SendOp(task) + assert.NoError(t, task.WaitDone(ctx)) + assert.NoError(t, flushTxn.Commit(ctx)) + + flushedEntry, err := rel.GetMeta().(*catalog.TableEntry).GetObjectByID(objEntry.ID(), false) + assert.NoError(t, err) + require.True(t, flushedEntry.IsAppendable(), + "flushed aobj should still be marked appendable") + deleteAt := flushedEntry.GetDeleteAt() + require.False(t, (&deleteAt).IsEmpty(), + "flushed aobj should carry a non-empty DeleteAt") + + check := func(name string, start, end types.TS, expectedRows int) { + t.Helper() + batches := make(map[uint32]*containers.BatchWithVersion) + err := tables.RangeScanInMemoryByObject( + ctx, flushedEntry, batches, start, end, common.DefaultAllocator, + ) + assert.NoErrorf(t, err, "%s: scan error", name) + bwv := batches[schema.Version] + require.NotNilf(t, bwv, "%s: expected batch at schema version %d", name, schema.Version) + for _, vec := range bwv.Vecs { + assert.Equalf(t, expectedRows, vec.Length(), + "%s: unexpected row count for attrs=%v", name, bwv.Attrs) + } + bwv.Close() + } + + check("[0,p1]", types.TS{}, p1, 6) + check("[0,p2]", types.TS{}, p2, 9) + check("(p1,p2]", p1.Next(), p2, 3) + check("(p1,p3]", p1.Next(), p3, 6) +} + +func TestAppendnode(t *testing.T) { + defer testutils.AfterTest(t)() + ctx := context.Background() opts := config.WithLongScanAndCKPOpts(nil) tae := testutil.NewTestEngine(ctx, ModuleName, t, opts) defer tae.Close() diff --git a/pkg/vm/engine/tae/tables/base.go b/pkg/vm/engine/tae/tables/base.go index ba553c1512a2a..c669f74ac49d4 100644 --- a/pkg/vm/engine/tae/tables/base.go +++ b/pkg/vm/engine/tae/tables/base.go @@ -527,6 +527,14 @@ func (obj *baseObject) ScanInMemory( node := obj.PinNode() defer node.Unref() if node.IsPersisted() { + // A flushed appendable object still needs to contribute per-row inserts + // to logtail tails that span its creation time, because the aobj's + // mnode data is gone but its row data lives on the persisted block. + // Non-appendable persisted objects are carried by object-meta only and + // do not need row-level collection here. + if obj.meta.Load().IsAppendable() && !obj.meta.Load().IsTombstone { + return node.MustPNode().ScanDataInRange(ctx, batches, start, end, mp) + } return nil } mnode := node.MustMNode() diff --git a/pkg/vm/engine/tae/tables/pnode.go b/pkg/vm/engine/tae/tables/pnode.go index b8c872a196810..e71cd93be445d 100644 --- a/pkg/vm/engine/tae/tables/pnode.go +++ b/pkg/vm/engine/tae/tables/pnode.go @@ -222,6 +222,136 @@ func (node *persistedNode) Scan( return } +// ScanDataInRange is the pnode counterpart of memoryNode.getDataWindowOnWriteSchema +// for appendable data objects that have been flushed to disk but whose rows still +// fall inside a logtail request range [start, end]. +// +// This is used by the lazy-catalog-tail path: when visitObjMeta decides to emit +// an aobj whose DeleteNode is out of the request range but whose CreateNode is +// visible, the caller needs per-row inserts for that aobj. For mnode we read +// from the in-memory node.data + appendMVCC; after TryUpgrade the mnode is gone +// and the data only lives on disk, so we must fetch it from the persisted block. +func (node *persistedNode) ScanDataInRange( + ctx context.Context, + batches map[uint32]*containers.BatchWithVersion, + start, end types.TS, + mp *mpool.MPool, +) (err error) { + meta := node.object.meta.Load() + if !meta.IsAppendable() || meta.IsTombstone { + return nil + } + createAt := meta.GetCreatedAt() + deleteAt := meta.GetDeleteAt() + if createAt.GT(&end) { + return nil + } + if !deleteAt.IsEmpty() && deleteAt.LT(&start) { + return nil + } + + readSchema := meta.GetTable().GetLastestSchema(false) + colIdxes := make([]int, 0, len(readSchema.ColDefs)) + for i := range readSchema.ColDefs { + colIdxes = append(colIdxes, i) + } + + id := meta.AsCommonID() + id.SetBlockOffset(0) + location, err := node.object.buildMetalocation(0) + if err != nil { + return err + } + if location.IsEmpty() { + return nil + } + + vecs, _, _, err := LoadPersistedColumnData( + ctx, readSchema, node.object.rt, id, colIdxes, location, mp, nil, true, + ) + if err != nil { + return err + } + closeVecs := func() { + for i := range vecs { + if vecs[i] != nil { + vecs[i].Close() + } + } + } + + commitTSVec, err := node.object.LoadPersistedCommitTS(0) + if err != nil { + closeVecs() + return err + } + if commitTSVec == nil { + closeVecs() + return nil + } + defer commitTSVec.Close() + + commitTSs := vector.MustFixedColWithTypeCheck[types.TS](commitTSVec.GetDownstreamVector()) + total := len(commitTSs) + if total == 0 { + closeVecs() + return nil + } + + var toDelete *nulls.Nulls + selected := 0 + for i := 0; i < total; i++ { + ts := commitTSs[i] + if ts.GE(&start) && ts.LE(&end) { + selected++ + continue + } + if toDelete == nil { + toDelete = nulls.NewWithSize(total) + } + toDelete.Add(uint64(i)) + } + if selected == 0 { + closeVecs() + return nil + } + + srcTSVec := node.object.rt.VectorPool.Transient.GetVector(&objectio.TSType) + if err = vector.AppendFixedList( + srcTSVec.GetDownstreamVector(), commitTSs, nil, mp, + ); err != nil { + srcTSVec.Close() + closeVecs() + return err + } + + src := containers.NewBatch() + for i, colIdx := range colIdxes { + src.AddVector(readSchema.ColDefs[colIdx].Name, vecs[i]) + } + src.AddVector(objectio.TombstoneAttr_CommitTs_Attr, srcTSVec) + if toDelete != nil && !toDelete.IsEmpty() { + src.Deletes = toDelete + src.Compact() + } + + dest, ok := batches[readSchema.Version] + if ok { + dest.Extend(src) + src.Close() + return nil + } + seqnums := readSchema.AllSeqnums() + seqnums = append(seqnums, objectio.SEQNUM_COMMITTS) + batches[readSchema.Version] = &containers.BatchWithVersion{ + Version: readSchema.Version, + NextSeqnum: uint16(readSchema.Extra.NextColSeqnum), + Seqnums: seqnums, + Batch: src, + } + return nil +} + func (node *persistedNode) CollectObjectTombstoneInRange( ctx context.Context, start, end types.TS, From 13dc87283052b99a606508d8f029200ad48035d7 Mon Sep 17 00:00:00 2001 From: aptend Date: Thu, 30 Apr 2026 16:00:09 +0800 Subject: [PATCH 07/11] chore(disttae): rename FIND_TABLE log prefix to catalog-load and remove DIAG- noise - Rename the diagnostic FIND_TABLE prefix used across loadTableFromStorage, replayCatalog* and a few error-injection sites to the more descriptive catalog-load tag, so the logs read as production observability rather than ad-hoc debug markers. - Drop the noisy DIAG-consumeEntry per-account DCA delayed Warn: the path fires on every per-account catch-up entry by design and added no signal beyond what the activation lifecycle logs already cover. - Replace the panic in loadDatabaseFromStorage's row-count guard with a proper InternalError return; the caller already handles errors and the panic was added for diagnosis, not as a real invariant. Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com> --- pkg/vm/engine/disttae/engine.go | 4 +++- pkg/vm/engine/disttae/logtail.go | 8 -------- pkg/vm/engine/disttae/logtail_consumer.go | 16 ++++++++-------- pkg/vm/engine/disttae/txn_database.go | 18 +++++++++--------- 4 files changed, 20 insertions(+), 26 deletions(-) diff --git a/pkg/vm/engine/disttae/engine.go b/pkg/vm/engine/disttae/engine.go index 30a08c7e84254..c4eb6f84f214f 100644 --- a/pkg/vm/engine/disttae/engine.go +++ b/pkg/vm/engine/disttae/engine.go @@ -336,7 +336,9 @@ func (e *Engine) loadDatabaseFromStorage( } if row := res.Batches[0].RowCount(); row != 1 { logerror() - panic("FIND_TABLE loadDatabaseFromStorage failed: table result row cnt != 1") + return nil, moerr.NewInternalErrorf(ctx, + "catalog-load loadDatabaseFromStorage: unexpected row count %d for %s.%v", + row, name, accountID) } bat := res.Batches[0] diff --git a/pkg/vm/engine/disttae/logtail.go b/pkg/vm/engine/disttae/logtail.go index 2f10361aaa742..5959b7dbabef9 100644 --- a/pkg/vm/engine/disttae/logtail.go +++ b/pkg/vm/engine/disttae/logtail.go @@ -18,9 +18,6 @@ import ( "context" "time" - "go.uber.org/zap" - - "github.com/matrixorigin/matrixone/pkg/logutil" v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2" "github.com/matrixorigin/matrixone/pkg/vm/engine/disttae/cache" @@ -83,11 +80,6 @@ func consumeEntry( return err } if shouldDelay && lc.delayAccountCacheApply(accountID, func() { applyToCatalogCache(cache, e) }) { - logutil.Warn("DIAG-consumeEntry per-account DCA delayed", - zap.Uint32("delayedAccount", accountID), - zap.Uint64("tableId", e.TableId), - zap.String("entryType", e.EntryType.String()), - ) return nil } } diff --git a/pkg/vm/engine/disttae/logtail_consumer.go b/pkg/vm/engine/disttae/logtail_consumer.go index 1a492526893e3..24e5e37948bfc 100644 --- a/pkg/vm/engine/disttae/logtail_consumer.go +++ b/pkg/vm/engine/disttae/logtail_consumer.go @@ -607,7 +607,7 @@ func (c *PushClient) subscribeTable( func (c *PushClient) subSysTables(ctx context.Context, initialActiveAccounts []uint32) error { if enabled, p := objectio.CNSubSysErrInjected(); enabled && rand.Intn(100000) < p { - return moerr.NewInternalError(ctx, "FIND_TABLE sub sys error injected") + return moerr.NewInternalError(ctx, "catalog-load sub sys error injected") } if c.lazyCatalog != nil { c.lazyCatalog.enable() @@ -670,7 +670,7 @@ func (c *PushClient) receiveOneLogtail(ctx context.Context, e *Engine) error { defer v2.LogTailClientReceiveCounter.Add(1) if enabled, p := objectio.CNRecvErrInjected(); enabled && rand.Intn(100000) < p { - return moerr.NewInternalError(ctx, "FIND_TABLE random error") + return moerr.NewInternalError(ctx, "catalog-load random error") } resp := c.subscriber.receiveResponse(ctx) @@ -891,7 +891,7 @@ func (c *PushClient) waitTimestamp() { func (c *PushClient) replayCatalogCache(ctx context.Context, e *Engine) (err error) { if enabled, p := objectio.CNReplayCacheErrInjected(); enabled && rand.Intn(100000) < p { - return moerr.NewInternalError(ctx, "FIND_TABLE replay catalog cache error injected") + return moerr.NewInternalError(ctx, "catalog-load replay catalog cache error injected") } ts := c.receivedLogTailTime.getTimestamp() if err = c.replayCatalogCacheAt(ctx, e, ts, []uint32{0}); err != nil { @@ -937,7 +937,7 @@ func (c *PushClient) replayCatalogCacheForReconnect( ctx context.Context, e *Engine, accountIDs []uint32, ) error { if enabled, p := objectio.CNReplayCacheErrInjected(); enabled && rand.Intn(100000) < p { - return moerr.NewInternalError(ctx, "FIND_TABLE replay catalog cache error injected") + return moerr.NewInternalError(ctx, "catalog-load replay catalog cache error injected") } ts := c.receivedLogTailTime.getTimestamp() if err := c.replayCatalogCacheAt(ctx, e, ts, accountIDs); err != nil { @@ -1219,7 +1219,7 @@ func replayCatalogDatabaseCache( } defer result.Close() - logutil.Infof("FIND_TABLE read mo_catalog.mo_databases %v rows", rowCountString(result.Batches)) + logutil.Infof("catalog-load read mo_catalog.mo_databases %v rows", rowCountString(result.Batches)) for _, b := range result.Batches { if err = fillTsVecForSysTableQueryBatch(b, typeTs, result.Mp); err != nil { return err @@ -1246,7 +1246,7 @@ func replayCatalogTableCache( } defer result.Close() - logutil.Infof("FIND_TABLE read mo_catalog.mo_tables %v rows", rowCountString(result.Batches)) + logutil.Infof("catalog-load read mo_catalog.mo_tables %v rows", rowCountString(result.Batches)) for _, b := range result.Batches { if err = fillTsVecForSysTableQueryBatch(b, typeTs, result.Mp); err != nil { return err @@ -1273,7 +1273,7 @@ func replayCatalogColumnCache( } defer result.Close() - logutil.Infof("FIND_TABLE read mo_catalog.mo_columns %v rows", rowCountString(result.Batches)) + logutil.Infof("catalog-load read mo_catalog.mo_columns %v rows", rowCountString(result.Batches)) if isColumnsBatchPerfectlySplitted(result.Batches) { for _, b := range result.Batches { @@ -1287,7 +1287,7 @@ func replayCatalogColumnCache( return nil } - logutil.Info("FIND_TABLE merge mo_columns results") + logutil.Info("catalog-load merge mo_columns results") bat := result.Batches[0] for _, b := range result.Batches[1:] { bat, err = bat.Append(ctx, result.Mp, b) diff --git a/pkg/vm/engine/disttae/txn_database.go b/pkg/vm/engine/disttae/txn_database.go index d4a6e9fb18b68..8d4c3fbcaff24 100644 --- a/pkg/vm/engine/disttae/txn_database.go +++ b/pkg/vm/engine/disttae/txn_database.go @@ -253,7 +253,7 @@ func (db *txnDatabase) deleteTable(ctx context.Context, name string, forAlter bo } if len(res.Batches) != 1 || res.Batches[0].Vecs[0].Length() != 1 { logutil.Error( - "FIND_TABLE deleteTableError", + "catalog-load deleteTableError", zap.String("bat", stringifySlice(res.Batches, func(a any) string { bat := a.(*batch.Batch) return common.MoBatchToString(bat, 10) @@ -280,7 +280,7 @@ func (db *txnDatabase) deleteTable(ctx context.Context, name string, forAlter bo if len(rowids) != len(colPKs) { logutil.Error( - "FIND_TABLE deleteTableError", + "catalog-load deleteTableError", zap.String("bat", stringifySlice(rowids, func(a any) string { r := a.(types.Rowid) return r.ShortStringEx() @@ -591,7 +591,7 @@ func (db *txnDatabase) loadTableFromStorage( now := time.Now() defer func() { if time.Since(now) > time.Second { - logutil.Info("FIND_TABLE slow loadTableFromStorage", + logutil.Info("catalog-load slow loadTableFromStorage", zap.Duration("cost", time.Since(now)), zap.String("table", name), zap.Uint32("accountID", accountID), @@ -621,7 +621,7 @@ func (db *txnDatabase) loadTableFromStorage( // catalog was compacted by a checkpoint or hasn't been fully // activated yet. Return nil instead of panicking so the caller // can handle the missing table gracefully. - logutil.Warn("FIND_TABLE loadTableFromStorage unexpected row count", + logutil.Warn("catalog-load loadTableFromStorage unexpected row count", zap.Int("rows", row), zap.String("sql", tblSql)) return @@ -648,7 +648,7 @@ func (db *txnDatabase) loadTableFromStorage( } defer res.Close() if len(res.Batches) == 0 { - err = moerr.NewParseErrorf(ctx, "FIND_TABLE columns of table %q does not exist, cnt: %v, sql:%v", name, len(res.Batches), colSql) + err = moerr.NewParseErrorf(ctx, "catalog-load columns of table %q does not exist, cnt: %v, sql:%v", name, len(res.Batches), colSql) return } bat := res.Batches[0] @@ -658,7 +658,7 @@ func (db *txnDatabase) loadTableFromStorage( return } } - logutil.Info("FIND_TABLE loadTableFromStorage columns", + logutil.Info("catalog-load loadTableFromStorage columns", zap.String("table", name), zap.Uint32("accountID", accountID), zap.Int("batches", len(res.Batches)), @@ -670,7 +670,7 @@ func (db *txnDatabase) loadTableFromStorage( } cache.ParseColumnsBatchAnd(bat, func(m map[cache.TableItemKey]cache.Columns) { if len(m) != 1 { - logutil.Warn("FIND_TABLE loadTableFromStorage columns touch unexpected tables", + logutil.Warn("catalog-load loadTableFromStorage columns touch unexpected tables", zap.Int("count", len(m)), zap.String("table", name)) // Clear tableitem so the caller sees nil. @@ -712,7 +712,7 @@ func (db *txnDatabase) getTableItem( if item.Defs != nil { return &item, nil } - logutil.Warn("FIND_TABLE catalog-cache item has no column defs, falling through to storage", + logutil.Warn("catalog-load catalog-cache item has no column defs, falling through to storage", zap.String("table", name), zap.Uint32("accountID", accountID), zap.Uint64("tableID", item.Id), @@ -721,7 +721,7 @@ func (db *txnDatabase) getTableItem( var tableitem *cache.TableItem if !c.CanServe(types.TimestampToTS(db.op.SnapshotTS())) || !engine.pClient.CanServeAccount(accountID, db.op.SnapshotTS()) { - logutil.Info("FIND_TABLE loadTableFromStorage", zap.String("table", name), zap.Uint32("accountID", accountID), zap.String("txn", db.op.Txn().DebugString()), zap.String("cacheTS", c.GetStartTS().ToString())) + logutil.Info("catalog-load loadTableFromStorage", zap.String("table", name), zap.Uint32("accountID", accountID), zap.String("txn", db.op.Txn().DebugString()), zap.String("cacheTS", c.GetStartTS().ToString())) if tableitem, err = db.loadTableFromStorage(ctx, accountID, name); err != nil { return nil, err } From 6729a6d97b785f185d592f9670b23231a5c080e4 Mon Sep 17 00:00:00 2001 From: aptend Date: Thu, 30 Apr 2026 17:44:27 +0800 Subject: [PATCH 08/11] test(ut): activate tenant catalog in helpers that bypass frontend The lazy catalog activation hook normally fires inside the frontend authentication flow. UTs that bypass the frontend by calling cnservice.GetSQLExecutor() directly with a non-sys account context must activate per-account catalog explicitly, otherwise per-account mo_columns/mo_foreign_keys lookups miss for the new tenant. - pkg/vm/engine/test/cdc_testutil.go: add activateTenantFromCtx and call it from exec_sql, mock_mo_indexes, mock_mo_foreign_keys, mock_mo_intra_system_change_propagation_log, execSql, and CreateDBAndTableForCNConsumerAndGetAppendData. - pkg/tests/testutils/test_utils.go: add activateTenantCatalogIfNeeded and call it from CreateTestDatabaseWithAccount, ExecSQLWithReadResultAndAccount, ExecSQLWithMinCommittedTSAndAccount, DBExistsWithAccount, TableExistsWithAccount. - pkg/frontend/show_account_test.go: align Test_getSqlForAccountInfo assertion with the LEFT JOIN show-account SQL. Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com> --- pkg/frontend/show_account_test.go | 1 - pkg/tests/testutils/test_utils.go | 28 ++++++++++++++++++++++++++++ pkg/vm/engine/test/cdc_testutil.go | 26 ++++++++++++++++++++++++++ 3 files changed, 54 insertions(+), 1 deletion(-) diff --git a/pkg/frontend/show_account_test.go b/pkg/frontend/show_account_test.go index 56c6d6d7c1456..cd479a63547bb 100644 --- a/pkg/frontend/show_account_test.go +++ b/pkg/frontend/show_account_test.go @@ -64,7 +64,6 @@ func Test_getSqlForAccountInfo(t *testing.T) { wantNotContains: []string{ "mo_catalog.mo_tables AS mt JOIN mo_catalog.mo_database AS md", "db_tbl_counts", - "LEFT JOIN", }, }, { diff --git a/pkg/tests/testutils/test_utils.go b/pkg/tests/testutils/test_utils.go index 3c88044210dc5..47fa0a2150815 100644 --- a/pkg/tests/testutils/test_utils.go +++ b/pkg/tests/testutils/test_utils.go @@ -33,9 +33,32 @@ import ( "github.com/matrixorigin/matrixone/pkg/pb/timestamp" "github.com/matrixorigin/matrixone/pkg/txn/client" "github.com/matrixorigin/matrixone/pkg/util/executor" + moengine "github.com/matrixorigin/matrixone/pkg/vm/engine" "github.com/stretchr/testify/require" ) +// activateTenantCatalogIfNeeded activates the lazy catalog for the given +// account on the CN. The frontend normally activates during login; tests +// that bypass the frontend by calling cnservice.GetSQLExecutor directly +// must activate explicitly before issuing tenant-scoped SQL. +func activateTenantCatalogIfNeeded( + ctx context.Context, + cn embed.ServiceOperator, + account int32, +) error { + if account <= 0 { + return nil + } + eng := cn.RawService().(cnservice.Service).GetEngine() + if activator, ok := eng.(moengine.TenantCatalogActivator); ok { + return activator.ActivateTenantCatalog(ctx, uint32(account)) + } + if ee, ok := eng.(*moengine.EntireEngine); ok { + return ee.ActivateTenantCatalog(ctx, uint32(account)) + } + return nil +} + func CreateTableAndWaitCNApplied( t *testing.T, db string, @@ -86,6 +109,7 @@ func CreateTestDatabaseWithAccount( defer cancel() ctx = defines.AttachAccountId(ctx, uint32(account)) + require.NoError(t, activateTenantCatalogIfNeeded(ctx, cn, account)) res, err := sql.Exec( ctx, fmt.Sprintf("create database %s", name), @@ -234,6 +258,7 @@ func ExecSQLWithReadResultAndAccount( moerr.CauseExecSQL, ) defer cancel() + require.NoError(t, activateTenantCatalogIfNeeded(ctx, cn, account)) var txnOp client.TxnOperator err := exec.ExecTxn( @@ -302,6 +327,7 @@ func ExecSQLWithMinCommittedTSAndAccount( moerr.CauseExecSQLWithMinCommittedTS, ) defer cancel() + require.NoError(t, activateTenantCatalogIfNeeded(ctx, cn, account)) var txnOp client.TxnOperator err := exec.ExecTxn( @@ -385,6 +411,7 @@ func DBExistsWithAccount( ctx = defines.AttachAccountId(ctx, uint32(account)) exec := cn.RawService().(cnservice.Service).GetSQLExecutor() + require.NoError(t, activateTenantCatalogIfNeeded(ctx, cn, account)) res, err := exec.Exec( ctx, "show databases", @@ -423,6 +450,7 @@ func TableExistsWithAccount( ctx = defines.AttachAccountId(ctx, uint32(account)) exec := cn.RawService().(cnservice.Service).GetSQLExecutor() + require.NoError(t, activateTenantCatalogIfNeeded(ctx, cn, account)) res, err := exec.Exec( ctx, "show tables", diff --git a/pkg/vm/engine/test/cdc_testutil.go b/pkg/vm/engine/test/cdc_testutil.go index 9abd672c41fce..dfeaa1a260c70 100644 --- a/pkg/vm/engine/test/cdc_testutil.go +++ b/pkg/vm/engine/test/cdc_testutil.go @@ -24,6 +24,7 @@ import ( "github.com/matrixorigin/matrixone/pkg/container/batch" "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/container/vector" + "github.com/matrixorigin/matrixone/pkg/defines" "github.com/matrixorigin/matrixone/pkg/frontend" "github.com/matrixorigin/matrixone/pkg/iscp" "github.com/matrixorigin/matrixone/pkg/util/executor" @@ -35,6 +36,18 @@ import ( "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/containers" ) +// activateTenantFromCtx activates lazy catalog for the tenant in ctx if it +// is a non-system account. Unit tests bypass the frontend, which is the +// normal activation point, so test helpers must do it explicitly before +// issuing tenant-scoped DDL/DML through the internal executor. +func activateTenantFromCtx(de *testutil.TestDisttaeEngine, ctx context.Context) error { + accountID, err := defines.GetAccountId(ctx) + if err != nil || accountID == 0 { + return nil + } + return de.Engine.ActivateTenantCatalog(ctx, accountID) +} + type internalExecResult struct { batch *batch.Batch err error @@ -134,6 +147,9 @@ func mock_mo_intra_system_change_propagation_log( ) (err error) { sql := frontend.MoCatalogMoISCPLogDDL + if err = activateTenantFromCtx(de, ctx); err != nil { + return err + } v, ok := moruntime.ServiceRuntime("").GetGlobalVariables(moruntime.InternalSQLExecutor) if !ok { panic("missing lock service") @@ -166,6 +182,9 @@ func exec_sql( sql string, ) (err error) { + if err = activateTenantFromCtx(de, ctx); err != nil { + return err + } v, ok := moruntime.ServiceRuntime("").GetGlobalVariables(moruntime.InternalSQLExecutor) if !ok { panic("missing lock service") @@ -201,6 +220,9 @@ func mock_mo_indexes( "PRIMARY KEY (`table_id`,`column_name`)" + // use table_id as primary key instead of id to avoid duplicate ")" + if err = activateTenantFromCtx(de, ctx); err != nil { + return err + } v, ok := moruntime.ServiceRuntime("").GetGlobalVariables(moruntime.InternalSQLExecutor) if !ok { panic("missing lock service") @@ -260,6 +282,9 @@ func execSql( ctx context.Context, sql string, ) (result executor.Result, err error) { + if err = activateTenantFromCtx(de, ctx); err != nil { + return + } v, ok := moruntime.ServiceRuntime("").GetGlobalVariables(moruntime.InternalSQLExecutor) if !ok { panic("missing lock service") @@ -357,6 +382,7 @@ func CreateDBAndTableForCNConsumerAndGetAppendData( } exec := v.(executor.SQLExecutor) + assert.NoError(t, activateTenantFromCtx(de, ctx)) _, err := exec.Exec(ctx, createDBSql, executor.Options{}) assert.NoError(t, err) _, err = exec.Exec(ctx, createTableSql, executor.Options{}) From 02ffe11c35331239d00579cadccc889c17595b49 Mon Sep 17 00:00:00 2001 From: aptend Date: Thu, 30 Apr 2026 17:49:55 +0800 Subject: [PATCH 09/11] fix(static-check): replace fmt.Errorf and errors.New with moerr The lazy-catalog feature commit introduced fmt.Errorf in pkg/util/status/server.go and an errors.New sentinel in pkg/vm/engine/disttae/logtail_consumer.go, plus a fmt.Errorf sentinel in pkg/vm/engine/tae/logtail/txn_handle.go. err-check rejects all three. Convert to moerr.NewInternalErrorNoCtx[f] and replace %w with %v where wrap-formatting was used. Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com> --- pkg/util/status/server.go | 28 +++++++++++------------ pkg/vm/engine/disttae/logtail_consumer.go | 2 +- pkg/vm/engine/tae/logtail/txn_handle.go | 3 ++- 3 files changed, 17 insertions(+), 16 deletions(-) diff --git a/pkg/util/status/server.go b/pkg/util/status/server.go index 9e7bfa4ed07cc..83c3e92fd1ff5 100644 --- a/pkg/util/status/server.go +++ b/pkg/util/status/server.go @@ -16,13 +16,13 @@ package status import ( "encoding/json" - "fmt" "net/http" "sort" "strconv" "strings" "sync" + "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/lockservice" "github.com/matrixorigin/matrixone/pkg/logservice" "github.com/matrixorigin/matrixone/pkg/pb/timestamp" @@ -292,7 +292,7 @@ func (s *Server) selectCNInstance(cnUUID string) (string, *CNInstance, error) { if cnUUID != "" { instance, ok := s.mu.CNInstances[cnUUID] if !ok { - return "", nil, fmt.Errorf("cn %q not found", cnUUID) + return "", nil, moerr.NewInternalErrorNoCtxf("cn %q not found", cnUUID) } copy := *instance return cnUUID, ©, nil @@ -300,7 +300,7 @@ func (s *Server) selectCNInstance(cnUUID string) (string, *CNInstance, error) { switch len(s.mu.CNInstances) { case 0: - return "", nil, fmt.Errorf("no CN instances registered") + return "", nil, moerr.NewInternalErrorNoCtxf("no CN instances registered") case 1: for uuid, instance := range s.mu.CNInstances { copy := *instance @@ -313,7 +313,7 @@ func (s *Server) selectCNInstance(cnUUID string) (string, *CNInstance, error) { uuids = append(uuids, uuid) } sort.Strings(uuids) - return "", nil, fmt.Errorf("multiple CN instances registered, specify ?cn= (%s)", strings.Join(uuids, ", ")) + return "", nil, moerr.NewInternalErrorNoCtxf("multiple CN instances registered, specify ?cn= (%s)", strings.Join(uuids, ", ")) } func optionalUint32Param(r *http.Request, key string) (*uint32, error) { @@ -323,7 +323,7 @@ func optionalUint32Param(r *http.Request, key string) (*uint32, error) { } parsed, err := strconv.ParseUint(value, 10, 32) if err != nil { - return nil, fmt.Errorf("invalid %s: %w", key, err) + return nil, moerr.NewInternalErrorNoCtxf("invalid %s: %v", key, err) } result := uint32(parsed) return &result, nil @@ -336,7 +336,7 @@ func optionalUint64Param(r *http.Request, key string) (*uint64, error) { } parsed, err := strconv.ParseUint(value, 10, 64) if err != nil { - return nil, fmt.Errorf("invalid %s: %w", key, err) + return nil, moerr.NewInternalErrorNoCtxf("invalid %s: %v", key, err) } return &parsed, nil } @@ -344,11 +344,11 @@ func optionalUint64Param(r *http.Request, key string) (*uint64, error) { func requiredUint32Param(r *http.Request, key string) (uint32, error) { value := r.URL.Query().Get(key) if value == "" { - return 0, fmt.Errorf("%s is required", key) + return 0, moerr.NewInternalErrorNoCtxf("%s is required", key) } parsed, err := strconv.ParseUint(value, 10, 32) if err != nil { - return 0, fmt.Errorf("invalid %s: %w", key, err) + return 0, moerr.NewInternalErrorNoCtxf("invalid %s: %v", key, err) } return uint32(parsed), nil } @@ -361,22 +361,22 @@ func optionalTimestampParam(r *http.Request, key string) (*timestamp.Timestamp, parts := strings.Split(value, ":") if len(parts) > 2 { - return nil, fmt.Errorf("invalid %s: expected physical[:logical]", key) + return nil, moerr.NewInternalErrorNoCtxf("invalid %s: expected physical[:logical]", key) } physical, err := strconv.ParseInt(parts[0], 10, 64) if err != nil { - return nil, fmt.Errorf("invalid %s physical time: %w", key, err) + return nil, moerr.NewInternalErrorNoCtxf("invalid %s physical time: %v", key, err) } if physical < 0 { - return nil, fmt.Errorf("%s physical time must be non-negative", key) + return nil, moerr.NewInternalErrorNoCtxf("%s physical time must be non-negative", key) } ts := ×tamp.Timestamp{PhysicalTime: physical} if len(parts) == 2 { logical, err := strconv.ParseUint(parts[1], 10, 32) if err != nil { - return nil, fmt.Errorf("invalid %s logical time: %w", key, err) + return nil, moerr.NewInternalErrorNoCtxf("invalid %s logical time: %v", key, err) } ts.LogicalTime = uint32(logical) } @@ -390,10 +390,10 @@ func intParam(r *http.Request, key string, defaultValue int) (int, error) { } parsed, err := strconv.Atoi(value) if err != nil { - return 0, fmt.Errorf("invalid %s: %w", key, err) + return 0, moerr.NewInternalErrorNoCtxf("invalid %s: %v", key, err) } if parsed <= 0 { - return 0, fmt.Errorf("%s must be greater than zero", key) + return 0, moerr.NewInternalErrorNoCtxf("%s must be greater than zero", key) } return parsed, nil } diff --git a/pkg/vm/engine/disttae/logtail_consumer.go b/pkg/vm/engine/disttae/logtail_consumer.go index 24e5e37948bfc..fc9c20258e044 100644 --- a/pkg/vm/engine/disttae/logtail_consumer.go +++ b/pkg/vm/engine/disttae/logtail_consumer.go @@ -96,7 +96,7 @@ const ( // doActivateTenantCatalog when the pending response channel receives nil, // which means a reconnect cycle cleared it. The outer ActivateTenantCatalog // loop retries on this specific error. -var errActivationInterruptedByReconnect = errors.New("tenant catalog activation interrupted by reconnect") +var errActivationInterruptedByReconnect = moerr.NewInternalErrorNoCtx("tenant catalog activation interrupted by reconnect") type SubscribeState int32 diff --git a/pkg/vm/engine/tae/logtail/txn_handle.go b/pkg/vm/engine/tae/logtail/txn_handle.go index e122015a784d8..d9f1a9ec92bbb 100644 --- a/pkg/vm/engine/tae/logtail/txn_handle.go +++ b/pkg/vm/engine/tae/logtail/txn_handle.go @@ -21,6 +21,7 @@ import ( "github.com/RoaringBitmap/roaring/v2" pkgcatalog "github.com/matrixorigin/matrixone/pkg/catalog" + "github.com/matrixorigin/matrixone/pkg/common/moerr" "github.com/matrixorigin/matrixone/pkg/container/types" "github.com/matrixorigin/matrixone/pkg/container/vector" "github.com/matrixorigin/matrixone/pkg/logutil" @@ -348,7 +349,7 @@ func batchSingleCPKeyAccountID(bat *containers.Batch) (uint32, bool, error) { if accountID != first { // Mixed-account delete batch (e.g., from restore). mixed = true - return fmt.Errorf("break") + return moerr.NewInternalErrorNoCtx("break") } return nil }, From f83e0fe8b0784ccd2a3ef67d7550f5484f78569e Mon Sep 17 00:00:00 2001 From: aptend Date: Thu, 30 Apr 2026 18:32:06 +0800 Subject: [PATCH 10/11] fix(static-check): drop unused forcedSubscribeTable, prealloc allCloseCBs golangci-lint flagged two issues introduced by the lazy-catalog branch: - pkg/vm/engine/disttae/logtail_consumer.go: forcedSubscribeTable is unused (its single caller was replaced by forcedSubscribeCatalogTable during the lazy-catalog refactor); remove it. - pkg/vm/engine/tae/logtail/service/server.go: pre-allocate the allCloseCBs slice with the known table count to satisfy prealloc. Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com> --- pkg/vm/engine/disttae/logtail_consumer.go | 27 --------------------- pkg/vm/engine/tae/logtail/service/server.go | 2 +- 2 files changed, 1 insertion(+), 28 deletions(-) diff --git a/pkg/vm/engine/disttae/logtail_consumer.go b/pkg/vm/engine/disttae/logtail_consumer.go index fc9c20258e044..1b838463a477a 100644 --- a/pkg/vm/engine/disttae/logtail_consumer.go +++ b/pkg/vm/engine/disttae/logtail_consumer.go @@ -532,33 +532,6 @@ func (c *PushClient) TryToSubscribeTable( return } -// this method will ignore lock check, subscribe a table and block until subscribe succeed. -// developer should use this method carefully. -// in most time, developer should use TryToSubscribeTable instead. -func (c *PushClient) forcedSubscribeTable( - ctx context.Context, - dbId, tblId uint64) error { - s := c.subscriber - - if err := s.sendSubscribe(ctx, api.TableID{DbId: dbId, TbId: tblId}); err != nil { - return err - } - ticker := time.NewTicker(periodToCheckTableSubscribeSucceed) - defer ticker.Stop() - - for i := 0; i < maxCheckRangeTableSubscribeSucceed; i++ { - select { - case <-ctx.Done(): - return ctx.Err() - case <-ticker.C: - if ok := c.subscribed.isSubscribed(dbId, tblId); ok { - return nil - } - } - } - return moerr.NewInternalError(ctx, "forced subscribe table timeout") -} - func (c *PushClient) forcedSubscribeCatalogTable( ctx context.Context, dbId, tblId uint64, diff --git a/pkg/vm/engine/tae/logtail/service/server.go b/pkg/vm/engine/tae/logtail/service/server.go index eb0f78c0c7728..c24c0e3a037b4 100644 --- a/pkg/vm/engine/tae/logtail/service/server.go +++ b/pkg/vm/engine/tae/logtail/service/server.go @@ -777,7 +777,7 @@ func (s *LogtailServer) sendActivation(ctx context.Context, p1 *catalogActivatio allowedAccounts := newLazyCatalogAllowedAccounts(act.accountID) var responseTails []logtail.TableLogtail - var allCloseCBs []func() + allCloseCBs := make([]func(), 0, len(lazyCatalogTableIDs)) sent := false defer func() { From 6cbbf9f9b67eeb81669a84312a0e1ebf3f62c9a0 Mon Sep 17 00:00:00 2001 From: aptend Date: Wed, 6 May 2026 10:48:10 +0800 Subject: [PATCH 11/11] test(coverage): cover status param helpers and ActivateTenantCatalog ctl Add unit tests for the server param-parsing helpers in pkg/util/status (optional/required uint, optional timestamp, intParam, selectCNInstance) plus the full handleActivateTenantCatalog branch matrix. These exercise the previously-uncovered error paths that drag down PR coverage. Co-authored-by: Copilot <223556219+Copilot@users.noreply.github.com> --- .../ctl/cmd_activate_tenant_catalog_test.go | 99 +++++++ pkg/util/status/server_helpers_test.go | 248 ++++++++++++++++++ 2 files changed, 347 insertions(+) create mode 100644 pkg/sql/plan/function/ctl/cmd_activate_tenant_catalog_test.go create mode 100644 pkg/util/status/server_helpers_test.go diff --git a/pkg/sql/plan/function/ctl/cmd_activate_tenant_catalog_test.go b/pkg/sql/plan/function/ctl/cmd_activate_tenant_catalog_test.go new file mode 100644 index 0000000000000..a0b6bbe900d86 --- /dev/null +++ b/pkg/sql/plan/function/ctl/cmd_activate_tenant_catalog_test.go @@ -0,0 +1,99 @@ +// Copyright 2024 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package ctl + +import ( + "context" + "testing" + + "github.com/matrixorigin/matrixone/pkg/defines" + "github.com/matrixorigin/matrixone/pkg/vm/engine" + "github.com/matrixorigin/matrixone/pkg/vm/process" + "github.com/stretchr/testify/require" +) + +type fakeActivatorEngine struct { + engine.Engine + called bool + accountID uint32 + err error +} + +func (f *fakeActivatorEngine) ActivateTenantCatalog(_ context.Context, accountID uint32) error { + f.called = true + f.accountID = accountID + return f.err +} + +func makeActivateProc(ctx context.Context, eng engine.Engine) *process.Process { + proc := process.NewTopProcess(ctx, nil, nil, nil, nil, nil, nil, nil, nil, nil, nil) + proc.Base.SessionInfo.StorageEngine = eng + return proc +} + +func TestHandleActivateTenantCatalog_NotCN(t *testing.T) { + proc := makeActivateProc(context.Background(), nil) + _, err := handleActivateTenantCatalog(proc, tn, "1", nil) + require.Error(t, err) +} + +func TestHandleActivateTenantCatalog_NoAccountInCtx(t *testing.T) { + proc := makeActivateProc(context.Background(), nil) + _, err := handleActivateTenantCatalog(proc, cn, "1", nil) + require.Error(t, err) +} + +func TestHandleActivateTenantCatalog_NotSysAccount(t *testing.T) { + ctx := defines.AttachAccountId(context.Background(), 42) + proc := makeActivateProc(ctx, nil) + _, err := handleActivateTenantCatalog(proc, cn, "1", nil) + require.Error(t, err) +} + +func TestHandleActivateTenantCatalog_InvalidParameter(t *testing.T) { + ctx := defines.AttachAccountId(context.Background(), 0) + proc := makeActivateProc(ctx, nil) + _, err := handleActivateTenantCatalog(proc, cn, "not-a-number", nil) + require.Error(t, err) +} + +type plainEngine struct{ engine.Engine } + +func TestHandleActivateTenantCatalog_EngineDoesNotSupport(t *testing.T) { + ctx := defines.AttachAccountId(context.Background(), 0) + proc := makeActivateProc(ctx, &plainEngine{}) + res, err := handleActivateTenantCatalog(proc, cn, "5", nil) + require.NoError(t, err) + require.Equal(t, ActivateTenantCatalogMethod, res.Method) +} + +func TestHandleActivateTenantCatalog_Success(t *testing.T) { + ctx := defines.AttachAccountId(context.Background(), 0) + eng := &fakeActivatorEngine{} + proc := makeActivateProc(ctx, eng) + res, err := handleActivateTenantCatalog(proc, cn, " 7 ", nil) + require.NoError(t, err) + require.Equal(t, ActivateTenantCatalogMethod, res.Method) + require.True(t, eng.called) + require.Equal(t, uint32(7), eng.accountID) +} + +func TestHandleActivateTenantCatalog_ActivatorError(t *testing.T) { + ctx := defines.AttachAccountId(context.Background(), 0) + eng := &fakeActivatorEngine{err: context.Canceled} + proc := makeActivateProc(ctx, eng) + _, err := handleActivateTenantCatalog(proc, cn, "9", nil) + require.Error(t, err) +} diff --git a/pkg/util/status/server_helpers_test.go b/pkg/util/status/server_helpers_test.go new file mode 100644 index 0000000000000..3403835e5a74f --- /dev/null +++ b/pkg/util/status/server_helpers_test.go @@ -0,0 +1,248 @@ +// Copyright 2021 -2023 Matrix Origin +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package status + +import ( + "net/http" + "net/http/httptest" + "testing" + + "github.com/matrixorigin/matrixone/pkg/vm/engine/disttae" + "github.com/stretchr/testify/require" +) + +func newRequestWithQuery(t *testing.T, q string) *http.Request { + t.Helper() + url := "/x" + if q != "" { + url += "?" + q + } + return httptest.NewRequest(http.MethodGet, url, nil) +} + +func TestOptionalUint32Param(t *testing.T) { + v, err := optionalUint32Param(newRequestWithQuery(t, ""), "k") + require.NoError(t, err) + require.Nil(t, v) + + v, err = optionalUint32Param(newRequestWithQuery(t, "k=42"), "k") + require.NoError(t, err) + require.NotNil(t, v) + require.Equal(t, uint32(42), *v) + + _, err = optionalUint32Param(newRequestWithQuery(t, "k=abc"), "k") + require.Error(t, err) + + _, err = optionalUint32Param(newRequestWithQuery(t, "k=99999999999"), "k") + require.Error(t, err) +} + +func TestOptionalUint64Param(t *testing.T) { + v, err := optionalUint64Param(newRequestWithQuery(t, ""), "k") + require.NoError(t, err) + require.Nil(t, v) + + v, err = optionalUint64Param(newRequestWithQuery(t, "k=12345"), "k") + require.NoError(t, err) + require.NotNil(t, v) + require.Equal(t, uint64(12345), *v) + + _, err = optionalUint64Param(newRequestWithQuery(t, "k=oops"), "k") + require.Error(t, err) +} + +func TestRequiredUint32Param(t *testing.T) { + _, err := requiredUint32Param(newRequestWithQuery(t, ""), "k") + require.Error(t, err) + + v, err := requiredUint32Param(newRequestWithQuery(t, "k=7"), "k") + require.NoError(t, err) + require.Equal(t, uint32(7), v) + + _, err = requiredUint32Param(newRequestWithQuery(t, "k=bad"), "k") + require.Error(t, err) +} + +func TestOptionalTimestampParam(t *testing.T) { + ts, err := optionalTimestampParam(newRequestWithQuery(t, ""), "k") + require.NoError(t, err) + require.Nil(t, ts) + + ts, err = optionalTimestampParam(newRequestWithQuery(t, "k=10"), "k") + require.NoError(t, err) + require.NotNil(t, ts) + require.Equal(t, int64(10), ts.PhysicalTime) + require.Equal(t, uint32(0), ts.LogicalTime) + + ts, err = optionalTimestampParam(newRequestWithQuery(t, "k=10:5"), "k") + require.NoError(t, err) + require.NotNil(t, ts) + require.Equal(t, int64(10), ts.PhysicalTime) + require.Equal(t, uint32(5), ts.LogicalTime) + + _, err = optionalTimestampParam(newRequestWithQuery(t, "k=1:2:3"), "k") + require.Error(t, err) + + _, err = optionalTimestampParam(newRequestWithQuery(t, "k=bad"), "k") + require.Error(t, err) + + _, err = optionalTimestampParam(newRequestWithQuery(t, "k=-1"), "k") + require.Error(t, err) + + _, err = optionalTimestampParam(newRequestWithQuery(t, "k=1:bad"), "k") + require.Error(t, err) +} + +func TestIntParam(t *testing.T) { + v, err := intParam(newRequestWithQuery(t, ""), "k", 7) + require.NoError(t, err) + require.Equal(t, 7, v) + + v, err = intParam(newRequestWithQuery(t, "k=42"), "k", 0) + require.NoError(t, err) + require.Equal(t, 42, v) + + _, err = intParam(newRequestWithQuery(t, "k=bad"), "k", 0) + require.Error(t, err) + + _, err = intParam(newRequestWithQuery(t, "k=0"), "k", 0) + require.Error(t, err) + + _, err = intParam(newRequestWithQuery(t, "k=-1"), "k", 0) + require.Error(t, err) +} + +func TestSelectCNInstance(t *testing.T) { + server := NewServer() + + // no instances registered. + _, _, err := server.selectCNInstance("") + require.Error(t, err) + + // explicit cn, not found. + _, _, err = server.selectCNInstance("missing") + require.Error(t, err) + + server.SetEngine("cn1", &disttae.Engine{}) + uuid, inst, err := server.selectCNInstance("") + require.NoError(t, err) + require.Equal(t, "cn1", uuid) + require.NotNil(t, inst) + + uuid, inst, err = server.selectCNInstance("cn1") + require.NoError(t, err) + require.Equal(t, "cn1", uuid) + require.NotNil(t, inst) + + // multiple instances - empty cn must fail. + server.SetEngine("cn2", &disttae.Engine{}) + _, _, err = server.selectCNInstance("") + require.Error(t, err) + + uuid, _, err = server.selectCNInstance("cn2") + require.NoError(t, err) + require.Equal(t, "cn2", uuid) +} + +func TestServeHTTPNotFound(t *testing.T) { + server := NewServer() + req := httptest.NewRequest(http.MethodGet, "/debug/status/unknown", nil) + resp := httptest.NewRecorder() + server.ServeHTTP(resp, req) + require.Equal(t, http.StatusNotFound, resp.Code) +} + +func TestServeHTTPDump(t *testing.T) { + server := NewServer() + req := httptest.NewRequest(http.MethodGet, "/debug/status", nil) + resp := httptest.NewRecorder() + server.ServeHTTP(resp, req) + require.Equal(t, http.StatusOK, resp.Code) +} + +func TestServeHTTPCatalogRejectsInvalidAccount(t *testing.T) { + server := NewServer() + server.SetEngine("cn1", &disttae.Engine{}) + + req := httptest.NewRequest(http.MethodGet, "/debug/status/catalog?account=bad", nil) + resp := httptest.NewRecorder() + server.ServeHTTP(resp, req) + require.Equal(t, http.StatusBadRequest, resp.Code) +} + +func TestServeHTTPCatalogCacheRejectsInvalidParams(t *testing.T) { + server := NewServer() + server.SetEngine("cn1", &disttae.Engine{}) + + for _, q := range []string{ + "account=bad", + "account=1&snapshot=bad", + "account=1&limit=bad", + } { + req := httptest.NewRequest(http.MethodGet, "/debug/status/catalog-cache?"+q, nil) + resp := httptest.NewRecorder() + server.ServeHTTP(resp, req) + require.Equalf(t, http.StatusBadRequest, resp.Code, "query=%s", q) + } +} + +func TestServeHTTPCatalogActivationRejectsInvalidParams(t *testing.T) { + server := NewServer() + server.SetEngine("cn1", &disttae.Engine{}) + + for _, q := range []string{ + "account=bad", + "limit=bad", + } { + req := httptest.NewRequest(http.MethodGet, "/debug/status/catalog-activation?"+q, nil) + resp := httptest.NewRecorder() + server.ServeHTTP(resp, req) + require.Equalf(t, http.StatusBadRequest, resp.Code, "query=%s", q) + } +} + +func TestServeHTTPPartitionsRejectsInvalidParams(t *testing.T) { + server := NewServer() + server.SetEngine("cn1", &disttae.Engine{}) + + for _, q := range []string{ + "db=bad", + "table=bad", + "limit=bad", + } { + req := httptest.NewRequest(http.MethodGet, "/debug/status/partitions?"+q, nil) + resp := httptest.NewRecorder() + server.ServeHTTP(resp, req) + require.Equalf(t, http.StatusBadRequest, resp.Code, "query=%s", q) + } +} + +func TestServeHTTPHandlersRejectMissingEngine(t *testing.T) { + // Register an instance with no Engine to hit the "engine nil" branches. + server := NewServer() + server.mu.CNInstances["cn1"] = &CNInstance{} + + for _, p := range []string{ + "/debug/status/catalog", + "/debug/status/catalog-cache?account=1", + "/debug/status/catalog-activation", + "/debug/status/partitions", + } { + req := httptest.NewRequest(http.MethodGet, p, nil) + resp := httptest.NewRecorder() + server.ServeHTTP(resp, req) + require.Equalf(t, http.StatusBadRequest, resp.Code, "path=%s", p) + } +}