RocksDB 中的 LSM-tree Compaction 算法概述(一)- Leveled Compaction

RocksDB 的 LSM-tree compaction 选取 sorted run 进行合并合并生成新的 sorted run,从而影响每次读取涉及的 sorted run。不同的 compaction 算法,可以在空间放大、读放大和写放大之间进行取舍,以适应特定的业务场景。

RUM Conjecture

RUM Conjecture

When designing access methods we set an upper bound for two of the RUM overheads, this implies a hard lower bound for the third overhead which cannot be further reduced.
RUM 猜想指在优化数据系统 read times (R),update cost (U) 以及 memory (or storage) overhead (M) 的开销过程中,其中两项的上限升高会导致第三项上限的降低。

Compaction 算法分类

Leveled

leveled compaction 的特点是以读放大和写放大为代价最小化空间放大。
LSM-tree 可以看作是包含若干 level 的序列,每个 level 是仅包括1个 sorted run。相邻 level 的大小之比通常被我们称为 fanout(扇出),当不同 level 之间的 fanout 相同时,LSM-tree 的写放大最小。compaction 选择 L(n) 的数据,与原有 L(n+1) 的数据进行合并,得到新的 L(n+1) 数据。每次 compaction 的最大写放大系数等同于 fanout。

Tiered

tiered compaction 特点是以读放大和空间放大为代价最小化写放大。
LSM-tree 依然可以看作是包含若干 level 的序列,每个 level 包括 N 个 sorted run。L(n) 的 sorted run 大小是 L(n-1) 的 N 倍。compaction 通常选择 L(n) 的数据合并得到新的 sorted run 输出到 L(n+1),但并不与 L(n+1) 的已有数据进行合并。每次 compaction 的最大写放大系数是 1。

Tiered + Leveled

对于较小的 level ,包括较多的活跃数据,涉及更新的可能性较大,采用 tiered compaction 减小写放大。对于较大的 level,需要存储更多的数据,则采用 leveled compaction 减小空间放大。RocksDB 的 leveled compaction 可以看作是 tiered&leveled 混合的 compaction 方法。

Compaction 触发条件

RocksDB 的 compaction 触发入口是 MaybeScheduleFlushOrCompaction。主要包括3种触发方式:

  1. switch wal:当 WAL 的文件大小超过阈值时
  2. writer buffer full:当 memtable 写满时
  3. schedule compaction:其余由更上层逻辑触发的 compaction,如 manual compaction

Leveled Compaction

image.png

Leveled Compaction 以提高读写放大为代价,最小化空间放大。Leveled Compaction LSM-tree 的 L0 通常包括多个 sorted run,L1+ 的每一个 level 是一个 sorted run。Leveled Compaction 是 RocksDB 默认的 compaction 算法,是在 LevelDB compaction 算法的基础上改进得到的。

Compaction 流程

  1. 当 L0 的文件数量超过 level0_file_num_compaction_trigger (默认值为4),选取交叠的 L0 的文件合并到 L1。
  2. L1 + 文件大小超过设定值,选取一个 L(n) 的文件合并到 L(n+1) 。

compaction 在某种程度上可以并行,并行度取决于 max_background_compactions (默认值为2),限定了 compaction 和 flush 操作后台任务数量上限。

Compaction Level 选取

在选取 level 进行 compaction 操作前,对每个 level 进行打分,比较所有 level 的分值,分值较高的 level 将会优先执行 compaction。

  1. Level 0
    分值取决于两个指标,文件总数量(也即 L0 的 sorted run 数量)除以 level0_file_num_compaction_trigger 得到的数值,以及当前数据量总大小除以 max_bytes_for_level_base 得到的数值。最终的分值取他们两者中的较大者。但是,在文件数量没有超过 level0_file_num_compaction_trigger 时,L0 的 compaction 不会执行。
    选择 Level 0 文件数量作为 compaction 的触发原因在 RocksDB 的代码中有做说明,主要有两点考虑:当 writer buffer 足够大时,较少的 level 0 compaction 依然可以得到较好的读取性能;由于每次读取都需要将 level 0 的若干个文件进行合并,因此较少的文件数量可以显著提升性能。
    如果选择了 L0 作为 compaction base level,当 L0 的文件数量超过 level0_file_num_compaction_trigger + 2,RocksDB 引入了 IntraL0Compaction 来减少 L0 的文件数量从而避免 Write Stall。
    此外,包括 Universal Compaction 和 FIFO Compaction,level 0 的分值计算都是照此方法进行。
void VersionStorageInfo::ComputeCompactionScore(
    const ImmutableCFOptions& immutable_cf_options,
    const MutableCFOptions& mutable_cf_options) {
...
        score = static_cast<double>(num_sorted_runs) /
                mutable_cf_options.level0_file_num_compaction_trigger;
        if (compaction_style_ == kCompactionStyleLevel && num_levels() > 1) {
          // Level-based involves L0->L0 compactions that can lead to oversized
          // L0 files. Take into account size as well to avoid later giant
          // compactions to the base level.
          uint64_t l0_target_size = mutable_cf_options.max_bytes_for_level_base;
          if (immutable_cf_options.level_compaction_dynamic_level_bytes &&
              level_multiplier_ != 0.0) {
            // Prevent L0 to Lbase fanout from growing larger than
            // `level_multiplier_`. This prevents us from getting stuck picking
            // L0 forever even when it is hurting write-amp. That could happen
            // in dynamic level compaction's write-burst mode where the base
            // level's target size can grow to be enormous.
            l0_target_size =
                std::max(l0_target_size,
                         static_cast<uint64_t>(level_max_bytes_[base_level_] /
                                               level_multiplier_));
          }
          score =
              std::max(score, static_cast<double>(total_size) / l0_target_size);
        }
...
}
  1. Level 1+
    分值取当前 level 数据量总大小与设定目标大小的比值。注意,之前已被选中进行 compaction 的文件不会被计算在内。
void VersionStorageInfo::ComputeCompactionScore(
    const ImmutableCFOptions& immutable_cf_options,
    const MutableCFOptions& mutable_cf_options) {
  ...
     // Compute the ratio of current size to size limit.
      uint64_t level_bytes_no_compacting = 0;
      for (auto f : files_[level]) {
        // 正在 compact 的文件除外
        if (!f->being_compacted) {
          level_bytes_no_compacting += f->compensated_file_size;
        }
      }
      // 计算分值
      score = static_cast<double>(level_bytes_no_compacting) /
              MaxBytesForLevel(level);
    }
    compaction_level_[level] = level;
    compaction_score_[level] = score;
  ...
}

Level's Target Size

  1. level_compaction_dynamic_level_bytes = false (默认值)
    L0: 取决于 max_bytes_for_level_base,默认值 256 MB
    L1+: 取决于 level_max_bytes_[L(n-1)] * max_bytes_for_level_multiplier,其中 max_bytes_for_level_multiplier 默认值为 10
  2. level_compaction_dynamic_level_bytes = true
    动态调整 level target size,实际上是为了 LSM-tree 的结构稳定性。由于 L0 compaction 的触发与 L0 的 SST 文件数量相关,可能会造成 L0 文件大小甚至超过 L1 的情况,如此将会产生雪崩式的 compaction,从而造成无谓的 IO 和计算。此时,通过动态调整 level size,将 L1 目标大小调整至 L0 的实际大小,并以相同的系数调整 L2+ 的目标文件大小,从而避免该问题。

Compaction 文件选取

compaction 文件选取会选择 L(n) 中最小交叠 SST 文件集合 clean cut 作为 start_level_inputs,选择 L(n+1) 中与 start_level_inputs 的 key 范围的最小交叠 SST 文件集合 clean cut 作为 output_level_inputs,将两者进行合并后得到新的 L(n) 的 SST 文件。

bool LevelCompactionBuilder::PickFileToCompact() {
...
    start_level_inputs_.files.push_back(f);
    start_level_inputs_.level = start_level_;
    if (!compaction_picker_->ExpandInputsToCleanCut(cf_name_, vstorage_,
                                                    &start_level_inputs_) ||
        compaction_picker_->FilesRangeOverlapWithCompaction(
            {start_level_inputs_}, output_level_)) {
    }

    //   计算 start_level_inputs 的 key 范围
    InternalKey smallest, largest;
    compaction_picker_->GetRange(start_level_inputs_, &smallest, &largest);
    CompactionInputFiles output_level_inputs;
    output_level_inputs.level = output_level_;
    // 寻找 output_level 的 clean cut 
    vstorage_->GetOverlappingInputs(output_level_, &smallest, &largest,
                                    &output_level_inputs.files);
    if (!output_level_inputs.empty() &&
        !compaction_picker_->ExpandInputsToCleanCut(cf_name_, vstorage_,
                                                    &output_level_inputs)) {
...
}

一个例子

image.png

L(n) 选取了 7-8 和 8-9 两个文件,L(n+1) 选取了 5-9、9-10 和 10-11 三个文件。这里选取 clean cut 不仅仅考虑存在交叠的部分,由于 SST 文件的 key 是 internal key,包括 user key、sequence number 和 key type 三部分,相邻的 SST 文件可能包含相同的 user key,也需要纳入 clean cut 考虑范围。

参考文献

RocksDB Compaction Wiki
RocksDB Leveled Compaction Wiki

最后编辑于
©著作权归作者所有,转载或内容合作请联系作者
  • 序言:七十年代末,一起剥皮案震惊了整个滨河市,随后出现的几起案子,更是在滨河造成了极大的恐慌,老刑警刘岩,带你破解...
    沈念sama阅读 216,125评论 6 498
  • 序言:滨河连续发生了三起死亡事件,死亡现场离奇诡异,居然都是意外死亡,警方通过查阅死者的电脑和手机,发现死者居然都...
    沈念sama阅读 92,293评论 3 392
  • 文/潘晓璐 我一进店门,熙熙楼的掌柜王于贵愁眉苦脸地迎上来,“玉大人,你说我怎么就摊上这事。” “怎么了?”我有些...
    开封第一讲书人阅读 162,054评论 0 351
  • 文/不坏的土叔 我叫张陵,是天一观的道长。 经常有香客问我,道长,这世上最难降的妖魔是什么? 我笑而不...
    开封第一讲书人阅读 58,077评论 1 291
  • 正文 为了忘掉前任,我火速办了婚礼,结果婚礼上,老公的妹妹穿的比我还像新娘。我一直安慰自己,他们只是感情好,可当我...
    茶点故事阅读 67,096评论 6 388
  • 文/花漫 我一把揭开白布。 她就那样静静地躺着,像睡着了一般。 火红的嫁衣衬着肌肤如雪。 梳的纹丝不乱的头发上,一...
    开封第一讲书人阅读 51,062评论 1 295
  • 那天,我揣着相机与录音,去河边找鬼。 笑死,一个胖子当着我的面吹牛,可吹牛的内容都是我干的。 我是一名探鬼主播,决...
    沈念sama阅读 39,988评论 3 417
  • 文/苍兰香墨 我猛地睁开眼,长吁一口气:“原来是场噩梦啊……” “哼!你这毒妇竟也来了?” 一声冷哼从身侧响起,我...
    开封第一讲书人阅读 38,817评论 0 273
  • 序言:老挝万荣一对情侣失踪,失踪者是张志新(化名)和其女友刘颖,没想到半个月后,有当地人在树林里发现了一具尸体,经...
    沈念sama阅读 45,266评论 1 310
  • 正文 独居荒郊野岭守林人离奇死亡,尸身上长有42处带血的脓包…… 初始之章·张勋 以下内容为张勋视角 年9月15日...
    茶点故事阅读 37,486评论 2 331
  • 正文 我和宋清朗相恋三年,在试婚纱的时候发现自己被绿了。 大学时的朋友给我发了我未婚夫和他白月光在一起吃饭的照片。...
    茶点故事阅读 39,646评论 1 347
  • 序言:一个原本活蹦乱跳的男人离奇死亡,死状恐怖,灵堂内的尸体忽然破棺而出,到底是诈尸还是另有隐情,我是刑警宁泽,带...
    沈念sama阅读 35,375评论 5 342
  • 正文 年R本政府宣布,位于F岛的核电站,受9级特大地震影响,放射性物质发生泄漏。R本人自食恶果不足惜,却给世界环境...
    茶点故事阅读 40,974评论 3 325
  • 文/蒙蒙 一、第九天 我趴在偏房一处隐蔽的房顶上张望。 院中可真热闹,春花似锦、人声如沸。这庄子的主人今日做“春日...
    开封第一讲书人阅读 31,621评论 0 21
  • 文/苍兰香墨 我抬头看了看天上的太阳。三九已至,却和暖如春,着一层夹袄步出监牢的瞬间,已是汗流浃背。 一阵脚步声响...
    开封第一讲书人阅读 32,796评论 1 268
  • 我被黑心中介骗来泰国打工, 没想到刚下飞机就差点儿被人妖公主榨干…… 1. 我叫王不留,地道东北人。 一个月前我还...
    沈念sama阅读 47,642评论 2 368
  • 正文 我出身青楼,却偏偏与公主长得像,于是被迫代替她去往敌国和亲。 传闻我的和亲对象是个残疾皇子,可洞房花烛夜当晚...
    茶点故事阅读 44,538评论 2 352

推荐阅读更多精彩内容