Databend Meta-Service 架构概述

Databend 的 meta-service 是一个基于 Raft 共识算法的分布式服务。其核心架构可以概括为一个 Raft 应用,如图中虚线框所示。

Raft 协议的主要组成部分包括:

  1. 分布式日志(Log)
  2. 状态机(State Machine)

分布式日志

日志主要用于记录分布式系统的操作。当一条日志被写入本地并通过网络同步到其他 Raft 节点后,系统会等待日志在多数节点上完成持久化。一旦达到多数派写入,该日志条目就被视为已提交(committed)。

状态机

提交的日志随后会被应用到状态机中。我们的状态机设计相对简洁,由两个主要部分组成:

  1. 内存表(MemTable):一个基于内存的键值排序数组
  2. 磁盘快照(Snapshot):持久化在磁盘上的数据

工作流程

  1. 新的日志条目被提交后,会不断地应用到内存中的状态机。
  2. 状态机的数据不会立即写入磁盘,而是通过 Raft 的定时机制,定期将整个状态机的数据保存为快照。
  3. 数据被保存到快照后,相应的日志条目可以从 Raft 日志中删除,以节省空间。

这种设计既保证了数据的一致性和持久性,又提高了系统的效率和性能。

在系统架构中,持久化组件包括日志(log)和快照(Snapshot)。其他组件主要存在于内存中。值得注意的是,快照系统还包含一个块缓存(block cache)机制。该机制将快照分割成多个块,并加载到内存中,从而提高访问速度。对于数据访问模式较为集中的场景,这种设计可以减少直接访问磁盘的频率,提升整体性能。

Raft 模块是在 datafuselabs/openraft 项目中维护的,而我们的 meta-service 可视为 Raft 的一个具体应用实现。在这个架构中,meta Node 是一个核心数据结构,负责管理 Raft 节点和整个应用程序。它与 Raft 节点建立连接,并向 Raft 发送指令。meta Node 本质上是一个封装层,主要功能是处理接口对接,而非实现复杂的业务逻辑。

系统对外提供两种 API 接口:

  1. client API(基于 gRPC ):主要用于与 databend-query 进行交互,支持读写操作。

  2. Raft API:支持 Raft 协议内部通信,包括以下操作:

    • RequestVote
    • AppendEntries
    • Snapshot
    • Forward

这些 API 构成了系统的网络层,在架构图中以蓝色部分表示。

在本系统的读写操作设计中,所有的读写请求都会被路由到 Leader 节点处理。如图所示,虚线框内的部分代表Leader 节点。Leader 节点的 client API 直接接收来自 databend-query 的请求并进行处理。

对于其他非 Leader 节点(Follower),如果它们接收到客户端请求,会立即通过内部的 forward 接口将请求转发至 Leader 节点。在收到 Leader 的响应后,Follower 节点再将结果返回给客户端。这种设计使得客户端无需手动切换节点即可完成请求。

值得注意的是,当 Follower 节点处理转发请求时,会在返回给 databend-query 客户端的响应中包含一个特殊的 header。这个 header 包含以下信息:

  1. 指示该请求是被转发处理的
  2. 当前 Leader 节点的地址

通过提供这些信息,客户端在后续请求中可以直接访问 Leader 节点,而无需再经过 Follower 节点的转发。这种机制确保了所有的读写请求最终都在 Leader 节点上执行,从而保证了数据的一致性。

接下来,我们将详细探讨系统支持的写操作接口。

Write 操作

Write 操作的核心功能是 upsert(update or insert 的缩写),支持数据的更新和插入。以下是相关的数据结构定义:

复制代码
pub struct LogEntry {
    pub time_ms: Option<u64>,
    pub cmd: Cmd,
}

pub enum Cmd {
    AddNode { node_id: NodeId, node: Node, },
    RemoveNode { node_id: NodeId },
    UpsertKV(UpsertKV),
    Transaction(TxnRequest),
}

pub struct UpsertKV {
    pub key: String,
    pub seq: MatchSeq,
    //  enum MatchSeq {
    //    Exact(u64),
    //    GE(u64),
    //  }
    pub value: Operation<Vec<u8>>,
    //    enum Operation<T> {
    //        Update(T),
    //        Delete,
    //        AsIs,
    //    }
    pub value_meta: Option<MetaSpec>,
    //              struct MetaSpec {
    //                  expire_at: Option<u64>,
    //                  ttl: Option<Interval>,
    //              }
}

LogEntry 结构体包含一个可选的时间戳 time_ms 和一个 Cmd 枚举类型。

Cmd 枚举定义了可能的命令类型:

  • AddNode: 添加节点
  • RemoveNode: 移除节点
  • UpsertKV: 执行 upsert 操作
  • Transaction: 执行事务

UpsertKV 结构体定义了 upsert 操作的详细参数:

  • key: 操作的键
  • seq: 匹配序列,可能是 MatchSeq::Exact(u64)MatchSeq::GE(u64)
  • value: 值操作,可能是 Operation::Update(T)Operation::DeleteOperation::AsIs
  • value_meta: 可选的元数据,包含 expire_atttl 信息

Raft LogEntry: 提供一致的时钟

复制代码
pub struct LogEntry {
    pub time_ms: Option<u64>,
    pub cmd: Cmd,
}

LogEntry 结构体包含两个关键字段:time_mscmdtime_ms 字段代表日志条目的生成时间,而 cmd 字段包含了实际需要执行的操作命令。

time_ms 字段在日志条目中的存在具有重要意义。在分布式系统中,超时判断是一个关键操作,而这个判断通常发生在将日志应用到状态机的过程中。如果使用每个节点的本地时间来进行超时判断,可能会导致不一致性问题。这是因为 Leader 节点和 Follower 节点的本地时间可能存在差异,从而在判断某个条目是否超时时可能得出不同的结果。

例如,Leader 节点可能认为某个条目存在,而 Follower 节点认为该条目不存在。这种不一致性可能导致后续事务执行时出现偏差,最终使状态机陷入不一致状态。因此,时间在我们的系统中必须是可以在内部维护和复制的一致条件,而不能被视为随时可能变化的外部条件。

通过在每个日志条目中记录其生成时间,我们可以确保整个分布式状态机的一致性。当状态机应用一条日志时,我们将系统时间视为该日志条目中记录的时间,并基于这个时间进行超时判断。这样,只要应用的日志序列相同,我们就可以保证每次应用后的结果都是一致的。

这种设计确保了时间在系统内部是可控和一致的,从而维护了分布式状态机的整体一致性。

Cmd: 操作内容

复制代码
pub enum Cmd {
    AddNode { node_id: NodeId, node: Node, },
    RemoveNode { node_id: NodeId },
    UpsertKV(UpsertKV),
    Transaction(TxnRequest),
}

节点变更命令

Cmd 枚举类型定义了系统支持的具体操作。其中,AddNodeRemoveNode 这两个变体主要用于内部节点管理,不对外部业务逻辑开放。具体来说,databend-query 不会直接使用这两个命令,它们仅在 meta-service 内部使用。

虽然 Raft 协议提供了成员配置变更(membership config change)算法来实现集群节点的动态变化,但该算法主要关注于确保配置变更前后均能维持连续的仲裁(quorum)。这种设计带来了一些限制:

  1. 每次变更都需要提交两个成员配置日志。
  2. 必须等待前一个配置日志提交后才能提议下一个,这导致了变更过程的延迟。

然而,在某些场景下,例如仅更改节点的对外服务地址和端口,并不涉及 Raft 成员的实质性变化。这种情况下,使用 Raft 的联合共识(joint consensus)算法进行成员配置变更显得过于复杂和低效。

考虑到这一点,我们选择将节点信息单独存储在状态机中,而不完全依赖 Raft 的成员配置变更流程。这种设计在保证一致性的同时,提供了更高的灵活性和效率,特别是对于那些不影响 Raft 成员关系的节点信息更新操作。

数据更新命令

数据更新操作主要分为两类:单条数据更新(upsert)和多条数据更新(transaction)。transaction 可以视为多个 upsert 操作的集合,同时包含一些必须预先满足的条件。

Upsert
复制代码
pub struct UpsertKV {
    pub key: String,
    pub seq: MatchSeq,
    //  enum MatchSeq {
    //    Exact(u64),
    //    GE(u64),
    //  }
    pub value: Operation<Vec<u8>>,
    //    enum Operation<T> {
    //        Update(T),
    //        Delete,
    //        AsIs,
    //    }
    pub value_meta: Option<MetaSpec>,
    //              struct MetaSpec {
    //                  expire_at: Option<u64>,
    //                  ttl: Option<Interval>,
    //              }
}

UpsertKV 结构体定义了单条数据更新操作的详细参数:

  1. key: 标识需要更新的具体数据项。

  2. seq: 全局序列号,类型为 MatchSeq,主要有两种匹配方式:

    • Exact(u64): 要求当前 key 的版本或序列号必须精确匹配指定值,用于支持比较并交换(Compare-And-Swap,CAS)操作。
    • GE(u64): "Greater than or Equal"的缩写,通常用于更新操作,要求记录的序列号大于或等于指定值,确保记录存在。
  3. value: 类型为 Operation<Vec<u8>>,描述对值的操作:

    • Update(T): 直接用新值替换原值。
    • Delete: 删除该键值对。
    • AsIs: 保持值不变,通常用于修改 value_meta
  4. value_meta: 可选的元数据,类型为 Option<MetaSpec>,可能包含过期时间(expire_at)和生存时间(TTL)信息。

在 meta-service 中,对外提供的接口将 key 定义为 String 类型,value 定义为 Vec<u8> 类型。

AsIs 操作通常用于节点续期场景。每个节点可能有一个超时时间,通过延长租约(extend lease)来确保节点在超时之前继续存在。这种操作只修改 value_meta 而不改变 value 本身。

Expiration

超时机制主要用于判断计算节点是否存活。在这种场景下,通常采用 AsIs 操作,并更新 value_meta 中的超时信息。value_meta 的主要功能是设置超时时间,包含两个选项:

  1. expire_at: 绝对超时时间,指定一个具体的时间点,到达该时间点时立即超时。

  2. ttl (Time To Live): 相对超时时间,指定一个时间间隔。

超时时间的计算与 LogEntry 中的 time_ms 字段密切相关。对于 ttl,实际的绝对超时时间是由 time_ms 加上 ttl得出。这个绝对超时时间由 meta-service 的 Leader 节点生成,确保不会出现时间回退问题。

使用 expire_at 可能会遇到一些潜在问题。由于客户端直接指定 expire_at,如果客户端的时间落后于 meta-service 的时间,且指定的超时时间较短,可能会导致数据在写入时就立即超时。这种情况下,写入操作实际上没有产生预期效果,写入完成后客户端可能无法查看到该记录,从而引发一些问题。

因此,除非有特殊需求,通常建议使用 ttl 而非 expire_atttl 提供了更可靠和一致的超时机制,能够有效避免因时间差异导致的意外超时问题。

Transaction
复制代码
message TxnRequest {
  repeated TxnCondition condition = 1;
  repeated TxnOp if_then = 2;
  repeated TxnOp else_then = 3;
}

message TxnCondition {
  string key = 1;
  oneof target { bytes value = 2; uint64 seq = 3; }
  ConditionResult expected = 4;

  enum ConditionResult { EQ = 0; GT = 1; GE = 2; LT = 3; LE = 4; NE = 5; }
}

message TxnOp {
  oneof request {
    TxnGetRequest get = 1;
    TxnPutRequest put = 2;
    TxnDeleteRequest delete = 3;
    TxnDeleteByPrefixRequest delete_by_prefix = 4;
  }
}

Transaction(事务)由一系列条件(conditions)和两组操作序列构成:

  1. condition: 定义事务执行的前提条件。
  2. if_then: 当所有条件满足时执行的操作序列。
  3. else_then: 当任一条件不满足时执行的操作序列。

TxnCondition 结构定义了单个条件的组成:

  • key: 指定要比较的键。
  • target: 比较目标,可以是键对应的值(value)或序列号(seq)。
  • expected: 期望的比较结果,包括等于(EQ)、大于(GT)、大于等于(GE)、小于(LT)、小于等于(LE)和不等于(NE)。

在实际应用中,比较序列号(seq)是最常见的用例。这种方式主要用于实现比较并交换(Compare-And-Swap,CAS)操作,确保在执行后续操作时,键的状态与之前读取时保持一致。

TxnOp 定义了事务中可执行的操作类型:

  1. get: 获取键值。
  2. put: 写入键值。
  3. delete: 删除指定键。
  4. delete_by_prefix: 删除指定前缀的所有键。

这些操作类型涵盖了常见的数据操作需求,为事务提供了灵活的操作空间。

通过这种结构,事务机制能够在保证数据一致性的前提下,实现复杂的条件判断和多步骤操作。

数据写入流程

数据写入流程主要包含以下步骤:

  1. 将日志条目落盘到本地磁盘
  2. 将日志复制到 quorum 数量的节点
  3. 确认日志提交状态 - 即后续任何 Leader 都能看到该日志条目
  4. 将已提交的日志应用(apply)到状态机

具体的 apply 操作代码如下:

apply()

复制代码
pub async fn apply(&mut self, entry: &Entry) -> Result<AppliedState, io::Error> {
    let log_id = &entry.log_id;
    let log_time_ms = Self::get_log_time(entry);

    self.clean_expired_kvs(log_time_ms).await?;

    *self.sm.sys_data_mut().last_applied_mut() = Some(*log_id);

    let applied_state = match entry.payload {
        EntryPayload::Normal(ref data) => {
            info!("apply: normal: {} {}", log_id, data);
            assert!(data.txid.is_none(), "txid is disabled");

            self.apply_cmd(&data.cmd).await?
        }
        // ...
    };

    if let Some(subscriber) = &self.sm.subscriber {
        for event in self.changes.drain(..) {
            subscriber.kv_changed(event);
        }
    }

    Ok(applied_state)
}

apply 操作的主要步骤:

  1. 获取日志 ID,用于记录 apply 进度,避免重复 apply
  2. 从日志条目中获取时间戳,作为清理过期键值对的阈值
  3. 清理所有过期的键值对
  4. 记录当前 apply 到的日志位置
  5. 执行实际的 apply 命令
  6. 将此次 apply 产生的所有变更发送给 subscriber

最后一步是为了支持 watcher 接口,当监控的 key 范围发生变化时,可以接收到通知。

读操作

读操作与写操作不同,它不需要经过分布式提交过程,而是直接在每个节点的本地状态机中执行。虽然读操作可以在Leader 或 Follower 节点上进行,但为了减少数据不一致的风险,我们只允许在 Leader 节点上执行读操作。这样做可以最大程度地避免返回不一致的结果,因为使用过期或不一致的数据进行后续操作可能会导致更多的冲突,引发问题或增加重试次数。

如前所述,所有的写操作都在 Leader 节点上执行。以下是读操作的简化抽象实现:

复制代码
pub struct SMV003KVApi<'a> { sm: &'a SMV003, }

impl<'a> kvapi::KVApi for SMV003KVApi<'a> {
    type Error = io::Error;

    type KVStream<E> = BoxStream<'static, Result<StreamItem, E>>;

    async fn get_kv_stream(&self, keys: &[String]) -> Result<KVStream<Self::Error>, Self::Error> {
        let local_now_ms = SeqV::<()>::now_ms();

        let mut items = Vec::with_capacity(keys.len());

        for k in keys {
            let got = self.sm.get_maybe_expired_kv(k.as_str()).await?;
            let v = Self::non_expired(got, local_now_ms);
            items.push(Ok(StreamItem::from((k.clone(), v))));
        }

        Ok(futures::stream::iter(items).boxed())
    }

    async fn list_kv(&self, prefix: &str) -> Result<KVStream<Self::Error>, Self::Error> {
        let local_now_ms = SeqV::<()>::now_ms();

        let strm = self
            .sm
            .list_kv(prefix)
            .await?
            .try_filter(move |(_k, v)| future::ready(!v.is_expired(local_now_ms)))
            .map_ok(StreamItem::from);

        Ok(strm.boxed())
    }
}

这段代码实现了状态机的 KVApi,包含两个主要接口:get_kv_streamlist_kv。前者用于按键获取值,后者用于按前缀列出键值对。

在执行读操作时,必须处理数据过期的问题。即使一段时间内日志没有更新,某些数据可能已经过期。因此,在读取时需要清理过期的键。然而,需要注意的是,这种过期处理可能并不完全准确。

读操作的不一致性问题

  1. 写操作保证输出一致性结果:

    • 在应用新的日志条目之前,过期的键会通过 LogEntry.time_ms 被清理。
  2. Leader 切换期间,读操作可能产生不一致结果:

    • 一条带有过期时间的记录可能在Leader-A节点上不可见。
    • 但在切换到Leader-B节点后,同一条记录可能变得可见。

这种不一致性主要发生在 Leader 切换过程中。例如,当从 Leader-A 切换到 Leader-B 时,如果 Leader-B 的本地时间较小,那么在 Leader-A 上不可见的某条记录可能在 Leader-B 上变得可见。这种读操作的不一致性可能导致一些错误的重试操作。

然而,这个问题通常影响较小,因为节点间的数据不一致和 Leader 切换相对较少发生。写操作始终保证完全一致性,而读操作可能在某些情况下产生不一致的结果。

序列号(Seq number)

复制代码
pub struct SeqV<T = Vec<u8>> {
    pub seq: u64,
    pub meta: Option<KVMeta>,
    pub data: T,
}

序列号(Seq number)可以被理解为一个版本号。在我们的系统中,我们采用了全局序列号而非每个键(per-key)的版本号。每个键作用域内的版本号更新(例如,当前键的版本是 1,下次更新时版本必定是 2)这种方式并未被采用。相反,我们使用全局序列号,因为它提供了更可靠的状态追踪能力。

以下是全局单调递增(Globally monotonic)与每键单调递增(Per-key monotonic)的对比示例:

复制代码
                  per-key:        globally
insert key=foo -> version=1       seq=1
                                              Read   key=foo: version=1
update key=foo -> version=2       seq=2
delete key=foo -> version=ø       seq=2
insert key=foo -> version=1       seq=3
                                              Update key=foo(version=1)

注意,在第三个操作(DELETE)中,如果使用每键策略,版本号会变为空(ø)。而使用全局序列号策略,序列号仍保持为 2。在下一次插入该键时,每键策略的版本又变回 1,而全局序列号策略中,序列号递增到 3。

这种差异在并发操作中尤为重要。假设另一个线程想要执行比较并交换(CAS)操作,它首先读取键的值,获得版本 1。然后,当它尝试更新时,如果看到键的版本仍为 1,在每键策略中,它无法确定该键是否已经发生变化,这会导致问题。

因此,每键策略难以实现准确的 CAS 操作。而使用全局序列号,系统中任何键的变化都会导致序列号递增。如果序列号发生变化,就明确表示该键已经被修改。这为实现可靠的 CAS 操作提供了基础。

结语

Databend 的 meta-service 采用基于 Raft 的分布式架构,实现了高可用性和数据一致性。其核心组件包括分布式日志和状态机,通过精心设计的写入和读取流程,确保了数据操作的可靠性。

该系统的一些关键特性包括:

  1. 使用 Raft 日志中的时间戳来提供一致的时钟,解决了分布式系统中的时间同步问题。
  2. 支持灵活的节点管理和数据更新操作,包括单条数据更新(upsert)和多条数据更新(transaction)。
  3. 采用全局序列号机制,为实现可靠的比较并交换(CAS)操作提供了基础。
  4. 通过只在 Leader 节点执行读操作,最大程度地减少了数据不一致的风险。
  5. 实现了高效的过期数据清理机制,保证了数据的时效性。

尽管在某些特定情况下(如 Leader 切换期间)可能存在读操作的不一致性,但系统整体上保证了写操作的完全一致性,为分布式数据管理提供了强有力的支持。

随着 Databend 项目的不断发展,meta-service 将继续优化和改进,以应对更复杂的分布式系统挑战,为用户提供更可靠、高效的元数据管理服务。

关于 Databend

Databend 是一款开源、弹性、低成本,基于对象存储也可以做实时分析的新式数仓。期待您的关注,一起探索云原生数仓解决方案,打造新一代开源 Data Cloud。 👨‍💻‍ Databend Cloud:databend.cn

📖 Databend 文档:docs.databend.cn/

💻 Wechat:Databend

✨ GitHub:github.com/datafuselab...

相关推荐
子兮曰9 分钟前
后端字段又改了?我撸了一个 BFF 数据适配器,从此再也不怕接口“屎山”!
前端·javascript·架构
卓卓不是桌桌2 小时前
如何优雅地处理 iframe 跨域通信?这是我的开源方案
javascript·架构
Qlly3 小时前
DDD 架构为什么适合 MCP Server 开发?
人工智能·后端·架构
用户881586910911 天前
AI Agent 协作系统架构设计与实践
架构
鹏北海1 天前
Qiankun 微前端实战踩坑历程
前端·架构
货拉拉技术1 天前
货拉拉海豚平台-大模型推理加速工程化实践
人工智能·后端·架构
RoyLin1 天前
libkrun 深度解析:架构设计、模块实现与 Windows WHPX 后端
架构
CoovallyAIHub2 天前
实时视觉AI智能体框架来了!Vision Agents 狂揽7K Star,延迟低至30ms,YOLO+Gemini实时联动!
算法·架构·github
RoyLin2 天前
领域驱动设计:回归本质的工程实践
架构
CoovallyAIHub2 天前
OpenClaw:从“19万星标”到“行业封杀”,这只“赛博龙虾”究竟触动了谁的神经?
算法·架构·github