深入剖析MySQL group commit实现(下)

达芬奇密码2018-08-15 11:17

上篇见:深入剖析MySQL group commit实现 (上)

MySQL 5.6版本的group commit实现

MySQL 官方在5.6.6版本也实现了binary log group commit,但是在具体实现上,与Kristian还略有不同。整体上来说,Oracle MySQL将上层事务的提交过程划分为3个阶段:FLUSH_STAGE、SYNC_STAGE、COMMIT_STAGE,每个阶段分别设置一个队列,阶段之间可以并发执行。

从某种程度上来说,Kristian将最为耗时的binary log的fsync操作与redo log的fsync操作进行了解绑,同时保证了上层和下层数据的一致性。相在比较而言,MySQL官方则更加细化,上层同一个group中,又划分了3个阶段,这3个阶段同时是可以并发执行的。多个阶段的设置导致效率提升的程度主要取决于这3个阶段的耗时,由于这个三个阶段大多是内存操作,不涉及IO,所以我认为这更多的是一个锦上添花的优化。下面我们从代码的角度来具体分析一下Oracle MySQL 5.6 binary log group commit的实现。


Oracle MySQL 5.6的binary log group commit的逻辑主要在binlog.cc的ordered_commit方法实现,主干逻辑如下:

[Oracle MySQL 5.6]

int MYSQL_BIN_LOG::ordered_commit(THD *thd, bool all, bool skip_commit)
{
 ......
/*这里暂不做介绍,*/
  thd->durability_property= HA_IGNORE_DURABILITY;
        ......
        // 进入第一阶段FLUSH_STAGE,主要完成的是Flush 各个线程的binlog cache到binary log文件中。
  /*
    Stage #1: flushing transactions to binary log

    While flushing, we allow new threads to enter and will process
    them in due time. Once the queue was empty, we cannot reap
    anything more since it is possible that a thread entered and
    appointed itself leader for the flush phase.
  */
 change_stage(thd, Stage_manager::FLUSH_STAGE, thd, NULL, &LOCK_log)
    ......
 process_flush_stage_queue(&total_bytes, &do_rotate, &wait_queue);
    
  /*
    Stage #2: Syncing binary log file to disk
 */
change_stage(thd, Stage_manager::SYNC_STAGE, wait_queue,
                   need_LOCK_log ? NULL : &LOCK_log, &LOCK_sync)
     ......

  /*
    Stage #3: Commit all transactions in order.

    This stage is skipped if we do not need to order the commits and
    each thread have to execute the handlerton commit instead.

    Howver, since we are keeping the lock from the previous stage, we
    need to unlock it if we skip the stage.
   */

    change_stage(thd, Stage_manager::COMMIT_STAGE,
                     final_queue, &LOCK_sync, &LOCK_commit)

    ......
}

我们来看一下change_stage函数定义:

/**
  Enter a stage of the ordered commit procedure.

  Entering is stage is done by:

  - Atomically enqueueing a queue of processes (which is just one for
    the first phase).

  - If the queue was empty, the thread is the leader for that stage
    and it should process the entire queue for that stage.

  - If the queue was not empty, the thread is a follower and can go
    waiting for the commit to finish.

  The function will lock the stage mutex if it was designated the
  leader for the phase.

  @param thd    Session structure
  @param stage  The stage to enter
  @param queue  Queue of threads to enqueue for the stage
  @param stage_mutex Mutex for the stage

  @retval true  The thread should "bail out" and go waiting for the
                commit to finish
  @retval false The thread is the leader for the stage and should do
                the processing.
*/

bool MYSQL_BIN_LOG::change_stage(THD *thd,Stage_manager::StageID stage, THD *queue, mysql_mutex_t *leave_mutex, mysql_mutex_t *enter_mutex)

可以看出,每次change都首先进入排队,如果队列是空的,则进入队列的线程将成为队列的Leader,如果队列不空,则将成为follower,Leader将完成本队列的所有线程在本阶段的工作。follower线程则在mysql_cond_wait方法进行条件等待,等待group leader完成所有的本阶段操作后唤醒该线程。下面我们就分别详细分析一下三个阶段。

首先我们看一下FLUSH_STAGE阶段:FLUSH STAGE的核心逻辑是在process_flush_stage_queue中实现的,首先进入FLUSH_STAGE队列的线程为leader,leader线程会获取Log_Lock锁。

int
MYSQL_BIN_LOG::process_flush_stage_queue(my_off_t *total_bytes_var,
                                         bool *rotate_var,
                                         THD **out_queue_var){

    ....
    1. 逐个遍历整个队列的每一个元素,如果元素为空或者超过binlog_max_flush_queue_time设置的超时时间时,则停止,扫描过的元素为一个group
    2.对group中的每一个线程调用flush_thread_caches()方法,将binlog cache写入到binary log文件中。
    ......
}

Group leader 在完成FLUSH_STAGE阶段后,进入SYNC_STAGE,与FLUSH_STAGE类似,首先会带领整个group在本阶段排队,但是有一点值得注意,如果队列为空,在FLUSH_STAGE阶段的leader还是leader,但是如果队列不空,则该leader会变成follower,新的SYNC_STAGE的队头线程将会成为整个group的leader。sync_binlog_file()函数完成了binary log 的Sync操作。另外,在进入Sync阶段前,如果sync_binlog=1,则不能释放Log_Lock锁,因为在binary log sync前,如果释放Log_Lock,则会导致dump线程有可能将还未sync的binary log发送到从机,一旦主机宕机,会导致从节点比主节点数据多,所以如果sync_binlog =1时, SYNC_STAGE实际上不能与FLUSH_STAGE并发执行。在执行sync_binlog_file()操作前,Leader线程首先也需要先获取Sync_Lock锁。


Sync结束后,leader线程首先要做的就是如果sync_binlog=1,则要释放Lock_log锁,同时也 释放Sync_Lock锁,同时获取Commit_Lock锁,进入第三个 COMMIT_STAGE阶段。Commit阶段的核心处理逻辑在process_commit_stage_queue函数实现的。


void
MYSQL_BIN_LOG::process_commit_stage_queue(THD *thd, THD *first)
{
for (THD *head= first ; head ; head = head->next_to_commit){
      ......
      ha_commit_low(head, all, false);
      ......
      }
}

int ha_commit_low(THD *thd, bool all, bool run_after_commit)
{
     for (; ha_info; ha_info= ha_info_next)
    {
        ht->commit(ht, thd, all)
    }
}

我们来看Oracle MySQL 5.6 Innodb的commit函数实现:

innobase_commit(  handlerton*	hton, THD*	 thd, bool	 commit_trx)
{
    ......
    /* Don't do write + flush right now. For group commit
  to work we want to do the flush later. */
  trx->flush_log_later = TRUE;
  innobase_commit_low(trx);
  trx->flush_log_later = FALSE;
    .......
/* Now do a write + flush of logs. */
  trx_commit_complete_for_mysql(trx);
}

void
trx_commit_in_memory(  trx_t*	trx,  lsn_t	lsn){
    ......
    if (trx->flush_log_later) {
   trx->must_flush_log_later = TRUE;
  }
    ......
}


UNIV_INTERN
void
trx_commit_complete_for_mysql(
/*==========================*/
 trx_t*	trx)	/*!< in/out: transaction */
{
 ut_a(trx);

 if (!trx->must_flush_log_later
     || thd_requested_durability(trx->mysql_thd)
        == HA_IGNORE_DURABILITY) {
  return;
 }

 trx_flush_log_if_needed(trx->commit_lsn, trx);

 trx->must_flush_log_later = FALSE;
}

在trx_commit_complete_for_mysql()方法中,Innodb存储引擎会调用上层的方法thd_get_durability_property(thd)获取本线程的thd->durability_property属性,从我们抽取的order_commit函数的主干代码中可以看出,线程一进入order_commit方法,durability_property属性就被设置为HA_IGNORE_DURABILITY,在下层事务提交时,实际redo log fsync根本不会执行。当读到这段代码时,会让人产生一个困惑,如果redo commit日志不写,会不会造成已经提交的事务在recover的时候被回滚。


在不开启binary log的情况下,MySQL在recover时,check point之前的所有更新确实是根据 redo log上是否记录该事务commit记录决定事务是redo还是roll back。但是如果开启binary log,前面我们也提到,binary log作为多个事务引擎的协调者,如果binary log上记录的事务,即使 redo log上没有commit记录,MySQL也会在recover时自动提交,所以recover时起决定作用的不再是redo log,而是binary log,所以即使redo log不再强制写入刷新到外存中,通过binary log协调者的角色,也能保证已经提交的事务在recover阶段不会丢失。这个最早其实是由MariaDB的Kristian提出的GroupCommit的优化[ MDEV-232],后来在OracleMySQL 5.6中也吸收了这个优秀的设计改进。


后记

  1. 通过在网易私有云关系数据库服务RDS上的实践,相比物理环境,云环境下group commit将有更大的效果提升。随着计算资源与存储资源的分离,单次IO响应时间受到网络的影响急剧增长,IO成本的上升使得一次group能够积累更多的事务,提交效率也会大幅上升。
  2. group commit的优化也在不断的持续,在MySQL 5.7.6版本,Oracle MySQL官方吸收了淘宝提供的innodb prepare fsync的优化patch,进一步降低事务在prepare阶段的fsync次数。其实现思路是在prepare阶段,不再调用类似fil_flush方法强制sync硬盘,而是跳过硬盘prepare阶段的sync过程,将该过程后移到在commit阶段,在order_commit方法的Flush_Stage阶段,由leader增加一个额外的操作就是负责fsync一次redo log prepare,这个思路同样可以应用到MySQL 5.5中,虽然5.5的实现是使用了一个队列,但是同样也是放弃prepare阶段的fsync,在trx_group_commit_leader函数中增加一个redo prepare fsync的操作步骤。虽然innodb的prepare本身是有group commit的,但是上述优化可以进一步增加单次group的事务数量,提高group的效率。但是有一点值得注意,一味的增加单次Group的事务数量并非是完美的,相应的单次请求的响应时间也会随着增加,这里需要寻找到一个用户容忍的时间和提高group效率的平衡点。
  3. 本篇论文重点介绍的是binary log group commit的实现,但是在innodb的redo log的刷新同样也使用了group commit的技术,本篇博文并没有具体分析innodb层的group commit实现,具体代码见【Log0log.cc::log_write_up_to】方法。
  4. 虽然仅仅只是看了Oracle MySQL 5.6关于binary log group commit相关的代码,但是深深感到MySQL 5.6相比5.5在代码规范性上的进步,其他的不提,就单论注释,MySQL 5.6的代码每个方法都有完整详细的注释,这使得虽然5.6和5.5在代码架构上有较大的变化,但是还是能熟悉MySQL 5.5代码的快速的入手。


网易云新用户大礼包:https://www.163yun.com/gift

本文来自网易实践者社区,经作者郭忆授权发布。