Skip to content

Transaction

liuhuicong edited this page Feb 19, 2019 · 4 revisions

概述

分布式事务主要用于在分布式存储环境下,保证数据更新的ACID特性。同一事务内部在不同region上的数据更新,要么全部成功,要么全部失败。

目前BaikalDB实现了Read Committed 级别的分布式事务,支持单语句事务(autocommit=1模式下的DML形成单语句事务)和使用显式控制命令的多语句事务。 BaikalDB的分布式事务实现利用了RocksDB单机引擎的两阶段提交 和Savepoint机制。其中Savepoint仅用来支持跨region的更新语句的一致性,并不支持用户通过SQL设置保存点。

事务相关功能列表

BaikalDB支持的事务相关的SQL语句包括:

  • 事务开始 BEGIN / START TRANSACTION
  • 事务提交COMMIT
  • 事务回滚ROLLBACK
  • 设置自动提交状态SET AUTOCOMMIT=0/1

BaikalDB支持的事务相关gflags如下:

模块 gflags 类型 含义
baikaldb fetch_instance_id bool baikaldb模块启动时是否通过baikalMeta获取全局唯一的实例id, 如果为false,则该baikaldb实例无法使用事务功能。
baikaldb recovery_db_path string baikaldb模块事务LOG存储路径,用户baikaldb实例挂掉后的事务恢复。
baikaldb default_2pc bool 是否对单条DML开启分布式事务,如果为false,无法保证修改操作在多个region上的一致性,但性能会比true时有提升
baikalStore rocks_transaction_lock_timeout_ms integer 事务中DML对记录加锁时的等待超时(单位ms),具体含义请参考RocksDB TransactionDBOptions::transaction_lock_timeout
baikalStore rocks_default_lock_timeout_ms integer 事务中DML对记录加锁时的等待超时(单位ms),具体含义请参考RocksDB TransactionDBOptions::default_lock_timeout

对于单DML语句,可以通过在SQL前添加/*{"enable_2pc":true}*/ 或 /*{"enable_2pc":false}*/ 注释来强制打开/关闭两阶段提交。如果不加注释,则使用default_2pc gflag的默认设置。

查询计划

查询计划的生成逻辑

事务控制命令(Begin,Commit,Rollback)生成逻辑查询计划的代码在文件src/new_logical_plan/transaction_planner.cpp中,自动提交命令(set autocommit=0/1)生成逻辑查询计划的代码在文件src/new_logical_plan/setkv_planner.cpp的set_autocommit函数中。

对于单条DML语句的隐式事务,如果开启两阶段提交功能,则会通过LogicalPlanner::set_dml_txn_state(src/new_logical_plan/logical_planner.cpp)函数获取事务id,并设置事务状态。

逻辑查询计划生成后,会生成相应的物理查询计划,在该阶段对事务相关的逻辑查询计划树进行变幻,分别构造出在BaikalStore和BaikalDB端执行的计划节点,并在两部分中间插入FetchNode节点来处理两个服务之间的网络交互。物理查询计划的代码参考src/physical_plan/separate.cpp的separate_begin,separate_commit, separate_rollback和separate_autocommit_dml_2pc函数。

单语句事务的执行计划

根据用户对一致性和性能的权衡,单DML语句可以开启或者关闭两阶段提交(2PC)功能。在2PC开启时,生成的查询计划如下图所示:

单语句事务的执行计划

其中PacketNode, CommitNode和所有的FetchNode均在baikaldb端执行,FetchNode的child在BaikalStore端执行。具体流程如下:

  1. CommitNode首先调用FetchNode1, 为了减少baikaldb和baikalStore之间的网络交互次数,FetchNode1并不会将StoreBegin(对应seq_id=1)立即发送到对应的baikalStore端并执行,而是缓存在baikaldb的当前mysql会话(NetworkSocket)中。
  2. CommitNode调用FetchNode2, FetchNode2将StoreDML(对应seq_id=2)缓存到当前会话中。
  3. CommitNode调用FetchNode3, FetchNode3将StoreBegin,StoreDML和StorePrepare(seq_id=3)打包,根据DML操作对应的keys(或key的范围)路由信息,将打包请求通过rpc发送到BaikalStore端对应的Region上,按照seq_id的顺序依次执行Begin,DML和Prepare命令。在Region上,这三个命令也被打包在一条RAFT指令中执行。
  4. 如果FetchNode3返回成功,即所有的Region上均执行成功,CommitNode调用FetchNode4,进而调用StoreCommit(seq_id=4)节点,在DML操作影响的所有region上执行本地Commit操作。如果FetchNode3返回失败,CommitNode则调用FetchNode5,进而调用StoreRollback(seq_id=5)节点,在所有DML的Region上执行回滚操作。

多语句事务的执行计划

多语句事务是由BEGIN (或SET AUTOCOMMIT=0) ... COMMIT/ROLLBACK(SET AUTOCOMMIT=1)及其中间包围的若干条连续查询或修改语句组成的。 多语句事务的执行计划

上图展示了一个多语句事务的执行计划,每条用户语句(事务控制语句/DML语句/SELECT查询语句)对应一个执行计划树。每条语句的seq_id递增(Commit语句占用两个seq_id,因为内部会拆分成Prepare和Commit两条命令)。 每个BaikalDB与Client的连接实例(NetworkSocket对象)上,保存了以下与事务相关的数据:

  • 事务ID(txn_id)。事务ID由两部分组成,BaikalDB全局唯一的实例ID(下文中的server_instance_id,占用3 bytes)和BaikalDB实例内部ID(5 bytes,实例内部全局唯一且递增)。
  • 事务内部序列ID(seq_id)。当前命令在事务内部的序列号,在事务内由1开始连续递增。
  • 自动提交状态(autocommit)。记录用户发送SET AUTOCOMMIT=0/1命令对自动提交状态的修改。
  • BaikalDB实例ID(server_instance_id)。BaikalDB全局唯一的实例ID(占用3 bytes,由BaikalMeta统一分配)。
  • 待回滚的命令ID(need_rollback_seq)。在“带保存点的局部回滚实现”一节将详细介绍。
  • 事务所有命令集合(cache_plans)。缓存了事务中的所有命令的执行计划,用于在事务执行过程中发生Region分裂或RAFT切主时进行重试。
  • 事务修改的所有Region信息(region_infos)。缓存了事务中所有DML命令修改的Region的元信息,用作事务控制命令(BEGIN, PREPARE, COMMIT/ROLLBACK)的路由信息。

查询执行

BaikalDB端

事务命令的缓存和重试机制

BaikalDB的连接实例上缓存了当前事务的所有已执行命令的执行计划(包括跳过执行直接做缓存的命令,如BEGIN命令和隐式单语句事务中的DML命令)。这些命令按照seq_id的顺序,保存在NetworkSocket.cache_plans结构中。缓存执行计划的作用包括两方面:一是减少BaikalDB和BaikalStore模块在两阶段提交过程中的rpc交互次数,进而降低事务的响应延迟;二是在Region分裂或RAFT切主(Leadership Transfer)时,新Region或者新Leader上需要从头开始执行事务命令,有了执行计划的缓存可以在这两种情况下快速重试。

在单语句事务的执行计划流程的1-2步中,我们提到BEGIN和DML语句都都跳过在Store上的执行,直接缓存在BaikalDB的连接上并返回,然后和第3步的PREPARE命令一起打包发送给BaikalStore去执行,即是利用了上述提到缓存作用一。

而在多语句事务中,用户输入BEGIN命令(seq_id=1)也是跳过在BaikalStore上的执行而缓存后立即返回成功。对于seq_id=2的DML或SELECT命令,FetchNode将缓存的BEGIN命令和当前的DML/SELECT命令一起打包发送到BaikalStore执行,因为client需要在事务提交前看到DML/SELECT操作的执行的结果。后续命令(seq_id>=3)执行时在正常情况无需再发送缓存命令,除非BaikalStore因为某些原因导致前面的事务命令缺失,返回TXN_FOLLOW_UP错误码和last_seq_id,表明目标Region上已经执行到了last_seq_id命令,仍缺少last_seq_id+1...seq_id-1区间的命令,BaikalDB需要补发该区间的缓存命令进行重试。

下图展示了BaikalDB向BaikalStore发送事务命令时因为BaikalStore端前序命令的缺失导致重试的流程:

事务请求重试

异常处理和恢复

BaikalDB与Client的连接超时 BaikalDB和Client之间的连接在超过connect_idle_timeout_s秒空闲后,会超时断开,此时如果连接上事务尚未提交,BaikalDB会向BaikalStore发送Rollback命令来回滚事务在Region上的修改操作。

BaikalDB实例Crash 根据事务处理的阶段分为几种情况讨论:

  • 实例在事务Prepare之前Crash。此时事务已经在Region上开启,但Prepare尚未执行,BaikalStore的事务管理器的GC线程会将空闲超过一定时间的未Prepare的事务回滚。
  • 实例在事务Prepare之后,Commit执行过程中Crash。BaikalDB在Prepare成功之后,先在本地写入一条CommitLog记录(记录内包含了Commit命令的执行计划,Log路径由recovery_db_path选项控制,代码参考TransactionNode::add_commit_log_entry函数),然后再向Store发送StoreCommit命令,StoreCommit在所有Region上执行成功后删除CommitLog记录。如果在Commit执行过程中BaikalDB Crash,则CommitLog记录不会被删除,而BaikalDB实例重启后,首先读取CommitLog并执行每条记录的Commit操作。如果在事务对应的CommitLog记录写入前BaikalDB Crash,则不会出现部分Region提交成功的情况,数据一致性能够得到保证。

单Region场景下两阶段提交的优化

如果一个事务中的DML操作仅仅影响了单个Region,则可以将两阶段提交优化为一阶段提交,从而减少一次BaikalDB和BaikalStore之间的网络交互。 此时BaikalDB只需要发送Prepare命令,无需发送StoreCommit命令,BaikalStore判断请求中的optimize_1pc状态如果为true, 则直接在Prepare阶段执行Commit操作。如果此时发生Region分裂需要重试,则BaikalDB会将optimize_1pc置为false,并给新老Region重发请求。

BaikalStore端

BaikalStore的每个Region包含一个TransactionPool实例,用于存放和管理当前Region上的事务实例。每个事务实例按序保存了该事务所有命令的执行计划(Transaction::_cache_plan_map)。 由于每条操作都需要提交并应用到RAFT状态机中,我们可以按照每个事务操作对应一条RAFT日志的方式,但该方式由于多个事务的DML操作交叉执行会导致死锁。如下图所示

RAFT LOG中事务交叉执行导致死锁

BaikalStore依赖RAFT的串行化机制和RocksDB TransactionDB内部的锁处理机制来进行锁冲突的检测和解决。针对上述死锁问题,提出将每条事务的DML操作打包在一个RAFT LOG中执行。每个事务仅提交两条RAFT LOG:第一条包含BEGIN...PREPARE的命令集合;第二条包含COMMIT命令。Prepare之前的命令首先在Leader上执行并缓存,确保无锁冲突,待收到BaikalDB发来的Prepare命令后,将缓存的所有命令打包提交到RAFT状态机。当RAFT LOG中的指令在Leader上执行时,同样利用了seq_id的特性保证按序执行,不重不丢。最后COMMIT命令作为一条单独的RAFT LOG来执行。整个流程如下图所示。

执行方案

RAFT切主

当RAFT发生Leadership Transfer时,如果此时事务已经执行了Prepare命令,则无需特殊处理。否则需要将Old Leader上的事务回滚(代码参考Region::on_leader_stop函数),并在收到BaikalDB发送的下一条事务命令时,在New Leader上利用BaikalDB的重试机制补发缺失的命令,从头开始执行事务。

Region分裂

为了保证Region分裂时数据的完整性,需要在数据复制阶段,将已经Prepare但尚未提交的事务发送到新Region重新执行。这部分的代码在Region::replay_txn_for_recovery函数中,该函数有两个重载实现,一个BaikalStore本地执行事务命令重放(用于非tail split场景),另一个是通过网络RPC发送事务命令给远程Region(用于tail split场景)。

由于在事务重放过程中,事务在Old Region上修改的记录有一部分转移到了New Region,需要对Old Region上该事务影响的记录行数做更新。在分裂操作的OP_VALIDATE_AND_ADD_VERSION阶段如果事务尚未提交,则根据每个事务重放在New Region上返回的affected_rows,在Old Region的事务实例上减去该值,该逻辑代码参考Region::validate_and_add_versionTransactionPool::update_txn_num_rows_after_split两个函数。

多语句事务中的局部操作回滚

非自动提交的多语句事务中,某个DML语句可能执行失败,这并不影响事务的最终提交。但由于失败的语句可能修改了多个Region并在部分Region上执行成功了,如果直接发送StoreCommit命令给这些Region,会造成多个Region上的数据不一致。针对该问题的解决方案如下

  1. BaikalDB端在DML语句执行完成后,根据执行成功/失败来记录DML语句是否需要回滚,保存事务中所有需要回滚的语句的seq_id,后续对Region的请求中会附带所有需要回滚的语句的seq_id。
  2. BaikalStore上执行每个DML/Select操作之前,先检查是否有需要回滚的请求,如果有,则根据seq_id回滚到对应的checkpoint位置。如果当前语句是DML则设置新的checkpoint,用于后续可能的局部回滚。然后执行DML操作并返回结果给BaikalDB。

下图显示了一个需要局部回滚的case(Region1 [-inf..k4) 表示region1的数据key的范围是负无穷到k4)。seq_id=2的Insert (k1, k8)在Region上执行失败,需要在后续命令执行前先回滚到seq_id=2之前的状态。

执行方案

Clone this wiki locally