1.compaction任务的开启

1.1.机制

rocksdb最常用的compaction方式是Leveled Compaction,首先介绍一下Leveled Compaction。参考
https://github.com/facebook/rocksdb/wiki/Leveled-Compaction

数据库的数据存放在sst文件中,sst文件有多个,会分到不同的level(L0,L1,L2…)中。最新的数据先保存在内存中,满足一定条件后写入到L0的sst文件,sst文件写入后不会再进行修改,只能读取、删除。L0有几个文件数量相关的配置:

Options.level0_file_num_compaction_trigger
Options.level0_slowdown_writes_trigger
Options.level0_stop_writes_trigger

当L0层的sst文件个数大于等于level0_file_num_compaction_trigger时,就可以触发compaction了。L0层的compaction会将L0层的所有sst文件进行compaction,生成新的文件存放到L1层。除了会使用L0层的sst文件外,L1层中,与L0层中的文件的key范围有重叠的所有sst文件,也需要参与进行compaction。compaction会将参与的所有文件进行合并且重新排序,再按key的顺序拆分成多个新的sst文件,保存到L1层,因为经过了排序,所以L1层的所有sst文件的key范围是不会重叠的。

L0层的compaction结束后,L1层的大小发生了变化,如果L1层的大小超过了限制,则需要对L1层进行compaction。限制每层大小的配置:

Target_Size(L1) = max_bytes_for_level_base
Target_Size(Ln+1) = Target_Size(Ln) * max_bytes_for_level_multiplier * max_bytes_for_level_multiplier_additional[n]

L1层会挑选超过限制大小的sst文件进行compaction,不需要选择所有的sst文件,compaction后生成的文件存放到L2层中,因此可能触发L2层的compaction。更高层也是类似的。

1.2.代码逻辑

Compaction的触发:

  • L0层文件数量超过level0_file_num_compaction_trigger,代码中是在SwitchMemtable、Flush时检查
  • L1~Ln大小超过Target_Size,代码中是在一个Compaction任务完成后检查

以上的两个场景会调用InstallSuperVersionAndScheduleWork,后续调用流程:

 InstallSuperVersionAndScheduleWork->SchedulePendingCompaction->NeedsCompaction  判断cfd是否需要进行Compaction->compaction_picker_->NeedsCompaction 对应类型的NeedsCompaction函数,对于leveled是LevelCompactionPicker::NeedsCompaction->AddToCompactionQueue  将cfd加入到compaction_queue_->MaybeScheduleFlushOrCompaction->env_->Schedule 将compaction任务加入到线程池等待调度

LevelCompactionPicker::NeedsCompaction的代码如下,有多个判断条件

bool LevelCompactionPicker::NeedsCompaction(const VersionStorageInfo* vstorage) const {if (!vstorage->ExpiredTtlFiles().empty()) {return true;}if (!vstorage->FilesMarkedForPeriodicCompaction().empty()) {return true;}if (!vstorage->BottommostFilesMarkedForCompaction().empty()) {return true;}if (!vstorage->FilesMarkedForCompaction().empty()) {return true;}for (int i = 0; i <= vstorage->MaxInputLevel(); i++) {if (vstorage->CompactionScore(i) >= 1) {return true;}}return false;
}

ExpiredTtlFiles,FilesMarkedForPeriodicCompaction,BottommostFilesMarkedForCompaction,FilesMarkedForCompaction属于特殊情况,正常情况下是vstorage->CompactionScore(i) >= 1触发。

score是事先计算好的,这里只是比较一下,计算在ComputeCompactionScore中进行,如下场景进行调用:

flush
#0  rocksdb::VersionStorageInfo::ComputeCompactionScore () at db/version_set.cc:2555
#1  rocksdb::VersionSet::AppendVersion () at db/version_set.cc:3853
#2  rocksdb::VersionSet::ProcessManifestWrites () at db/version_set.cc:4289
#3  rocksdb::VersionSet::LogAndApply() at db/version_set.cc:4440
#4  rocksdb::VersionSet::LogAndApply() at ./db/version_set.h:996
#5  rocksdb::MemTableList::TryInstallMemtableFlushResults () at db/memtable_list.cc:507
#6  rocksdb::FlushJob::Run () at db/flush_job.cc:247
#7  rocksdb::DBImpl::FlushMemTableToOutputFile () at db/db_impl/db_impl_compaction_flush.cc:210
#8  rocksdb::DBImpl::FlushMemTablesToOutputFiles () at db/db_impl/db_impl_compaction_flush.cc:339
#9  rocksdb::DBImpl::BackgroundFlush () at db/db_impl/db_impl_compaction_flush.cc:2543
compaction的pick完成后,job开始前,这里是用来判断是否有其他compaction任务可以并行执行
#0  rocksdb::VersionStorageInfo::ComputeCompactionScore () at db/version_set.cc:2555
#1  rocksdb::(anonymous namespace)::LevelCompactionBuilder::GetCompaction () at db/compaction/compaction_picker_level.cc:350
#2  rocksdb::(anonymous namespace)::LevelCompactionBuilder::PickCompaction () at db/compaction/compaction_picker_level.cc:320
#3  rocksdb::LevelCompactionPicker::PickCompaction () at db/compaction/compaction_picker_level.cc:508
#4  rocksdb::ColumnFamilyData::PickCompaction () at db/column_family.cc:1082
#5  rocksdb::DBImpl::BackgroundCompaction () at db/db_impl/db_impl_compaction_flush.cc:2906
#6  rocksdb::DBImpl::BackgroundCallCompaction () at db/db_impl/db_impl_compaction_flush.cc:2666
#7  rocksdb::DBImpl::BGWorkCompaction () at db/db_impl/db_impl_compaction_flush.cc:2441
compaction结束后
#0  rocksdb::VersionStorageInfo::ComputeCompactionScore () at db/version_set.cc:2555
#1  rocksdb::VersionSet::AppendVersion () at db/version_set.cc:3853
#2  rocksdb::VersionSet::ProcessManifestWrites () at db/version_set.cc:4289
#3  rocksdb::VersionSet::LogAndApply() at db/version_set.cc:4440
#4  rocksdb::VersionSet::LogAndApply () at ./db/version_set.h:977
#5  rocksdb::CompactionJob::InstallCompactionResults () at db/compaction/compaction_job.cc:1643
#6  rocksdb::CompactionJob::Install () at db/compaction/compaction_job.cc:783
#7  rocksdb::DBImpl::BackgroundCompaction () at db/db_impl/db_impl_compaction_flush.cc:3126
#8  rocksdb::DBImpl::BackgroundCallCompaction () at db/db_impl/db_impl_compaction_flush.cc:2666
#9  in rocksdb::DBImpl::BGWorkCompaction () at db/db_impl/db_impl_compaction_flush.cc:2441

score的计算方法:待补充。

2.BackgroundCompaction

这个函数是compaction的主体函数。简单说明一下整体的流程,一些细节不会介绍。

首先调用PickCompactionFromQueue,从队列中取出一个cfd,对这个cfd执行compaction,这一步比较简单。

然后调用PickCompaction,选取需要进行compaction的文件,得到一个Compaction对象。

然后对不同的compaction类型会进行不同的处理,我们这里只对普通的compaction进行介绍。

首先生成一个CompactionJob对象,然后调用其Prepare方法,主要任务是将compaction任务划分成多个Subcompaction。

再调用CompactionJob的Run方法,这是主体流程。会在当前线程执行第一个Subcompaction任务,而其他Subcompaction任务将其加入到thread_pool中等待调度。一个Subcompaction的主要内容是读取输入的文件列表,进行归并排序,结果到达一定大小就写入一个新的sst文件并新开一个文件用于后续内容写入,直到所有输入被处理完毕。

精简一下就是:
a) PickCompactionFromQueue;
b) PickCompaction;
c) CompactionJob.prepare()
d) CompactionJob.run()
后面展开介绍。

3.PickCompaction

不同类型的compaction会调用不同的PickCompaction函数,对于leveled compaction调用的是LevelCompactionPicker::PickCompaction。

首先传入必要的信息创建一个LevelCompactionBuilder对象,compaction的选择过程就在这个对象的PickCompaction函数中进行。

文件的选取分为三步:

  1. SetupInitialFiles();
  2. SetupOtherL0FilesIfNeeded()
  3. SetupOtherInputsIfNeeded()

选取完成后,再调用GetCompaction()构建一个Compaction对象返回。

3.1.SetupInitialFiles

3.1.1. 选取level

这个函数是选取一个level作为start level进行compaction,并选择start level以及output 中需要进行compaction的文件。如果选择L0触发compaction,那么start level就是L0,output level就是L1。

选取level是通过score进行的,score的计算已经在ComputeCompactionScore中进行了,保存在VersionStorageInfo的compaction_score_数组中,并且按score从大到小进行了排序,所以这里就简单从score大的向score小的进行遍历,尝试选择。

首先需要满足的条件是score>1,这是基本条件。另外,可能有些清空下L0层的compaction需要pending(应该是L0正在进行compaction,则新的L0层sst文件需要等待正在进行的compaction完成后再进行compaction),则优先进行其他层的compaction。第一步是找到满足条件的第一个level,一般来说就是score最大的那个了。

3.1.2. 选取文件

找到level后,调用PickFileToCompact选择文件,按文件大小从大到小遍历start level的文件去选取,已经在进行compaction的文件会跳过。

a. 获取文件

vstorage_->LevelFiles

文件信息已经在缓存中,直接从vstorage中获取。

记录文件信息用的类是FileMetaData。

b. ExpandInputsToCleanCut

选取第一个文件后,调用ExpandInputsToCleanCut,clean cut的意思是清晰的边界,即与其他文件没有重叠,这个函数的作用就是根据当前的文件的key范围,将其他重叠的文件加入集合直到得到一个clean cut。

ExpandInputsToCleanCut中,会先清除inputs,再调用GetOverlappingInputs重新进行选择。

对于>0的level,调用GetOverlappingInputsRangeBinarySearch,先二分查找到start_index,再二分找到end_index,将中间的所有文件加入到inputs中。

对于L0,会比较复杂,首先将所有文件加入到一个集合,再遍历这个集合,找到一个与key范围有重叠的文件,将该文件从集合中删除。如果没有找到,则直接退出。找到后,根据新加入文件将key范围进行扩展,然后再遍历一般去寻找,如此循环直到找不到退出为止。

ExpandInputsToCleanCut中选取文件时不会判断是否有在进行compaction的文件,但在选取完成后,如果选取结果中有正在进行compaction的文件(ExpandInputsToCleanCut会返回false),或者和进行中的compaction的key range有重叠(调用FilesRangeOverlapWithCompaction进行判断),则表示目前选取的文件无法并行执行compaction,清空已选取的文件,再进行下一次选取。

c. output level的input文件选择

首先根据已选择的input文件,得到一个key的范围,记在smallest、largest变量中。

然后调用GetOverlappingInputs,这个函数在a里面已经使用过,现在的作用就是从output level中选取出与input文件的key范围有重叠的所有文件。

然后再调用ExpandInputsToCleanCut,但输入的是output level的文件,这里应该不是为了扩展文件,因为output level肯定是>0的,那么文件就是不重叠的,不需要扩展就是clean cut。这里调用ExpandInputsToCleanCut的作用是判断output level的文件中有没有已经在执行compaction的文件。

d. 循环

a和b两步结束后,如果正常的话,这个函数就结束了,完成了start_level_inputs_和output_level_inputs_。

但代码是一个循环,循环的作用是出现异常时用下一个文件再进行重试,异常情况指的主要是选取的文件无法和已经在执行的compaction并行执行的情况。也不算异常,就是需要重新选择的一种情况。

3.2.SetupOtherL0FilesIfNeeded

待补充。

3.3.SetupOtherInputsIfNeeded

待补充。

3.4.GetCompaction

将输入文件、level等进行进行组装,得到一个Compaction对象。

将compaction注册到compaction_picker_中,用于后续的其他compaction判断是否冲突,即能否并行,例如两个L0的compaction不能并行。

调用ComputeCompactionScore,因一些文件用于compaction了,需要重新计算score。

到这里PickCompaction就结束了,得到了一个Compaction对象。

4.CompactionJob::Prepare

主要是划分sub compaction。

sub compaction:配置options.max_subcompactions要大于1;对于leveled compaction的情况,需要start_level是1,且output_level>0且output level有input文件。

4.1. GenSubcompactionBoundaries

默认情况不开启这个配置,不会划分。
具体待补充。

5.CompactionJob::run

如果不止一个sub compaction,先给除了第一个以外的没有sub compaction在thread pool中添加一个任务等待调度。

然后调用ProcessKeyValueCompaction,这个函数中执行了compaction的主要内容。

完成后,等待所有sub compaction任务结束,再统计一些信息,再对文件进行校验(使用lambda表达式verify_table)。

5.1.ProcessKeyValueCompaction

5.1.1.MakeInputIterator

5.1.1.1. list

new一个InternalIterator指针的数组:

InternalIterator** list = new InternalIterator* [space];

L0中每个文件创建一个Iterator,L>0的每个level创建一个Iterator,依此计算space的大小。

list数据存放后续创建的Iterator,最后再合并成一个MergingIterator。

5.1.1.1. 循环

遍历所有level,对每个level进行处理。

a. L0

L0对每个文件创建一个Iterator。

调用cfd->table_cache()->NewIterator ==> TableCache::NewIterator

FindTable:

尝试从column family中的table_cache查找TableReader,TableReader中保存了meta和foot信息,

如果table_cache中没有找到,那个就需要从硬盘中读取,调用TableCache::GetTableReader,再调用到BlockBasedTable::Open,进行以下内容的读取:

// Read in the following order:
// 1. Footer
// 2. [metaindex block]
// 3. [meta block: properties]
// 4. [meta block: range deletion tombstone]
// 5. [meta block: compression dictionary]
// 6. [meta block: index]
// 7. [meta block: filter]

b. L>0

LevelIterator的创建比较简单,只是将信息组合在一起,还没有从硬盘读取数据。

5.1.1.1. NewMergingIterator

将list中的Iterator合并成一个Merging Iterator。

5.1.2.seek first

Slice* start = sub_compact->start;
Slice* end = sub_compact->end;
if (start != nullptr) {
IterKey start_iter;
start_iter.SetInternalKey(*start, kMaxSequenceNumber, kValueTypeForSeek);
input->Seek(start_iter.GetInternalKey());
} else {
input->SeekToFirst();
}

如果没有指定start,则调用SeekToFirst,如果指定了start则Seek到start。例如:

sub0|sub1|sub2|sub3

sub0没有start有end,sub1和sub2有start和end,sub3有start没有end。

seektofirst流程如下图,seek类似,只是指定了target

5.1.3.c_iter

c_iter是CompactionIterator的对象,MergingIterator只能对文件进行遍历,而CompactionIterator从MergingIterator中获取数据,并进行处理,例如新旧数据的合并,同一个key只保留新的,又例如数据的删除。

这部分逻辑也比较多,时间原因还没有整理完成,先不介绍了。

5.1.4.循环

接下来就是循环去获取Iterator中的数据,并合并成新的sst文件写入硬盘。
主要内容如下图:

到这里,compaction的主要内容就结束了,数据已经写入硬盘。后续还有一些收尾工作,比较细节还没有研究,先不介绍了。

Ps.

compaction内存占用分析:

  1. 所有参与compaction的文件中的meta,会放在table cache中。
  2. 正在进行compaction处理的文件的readahead(在iterator中),可以配置,但是开启direct io时必须要有。个数是L0的文件个数+其他level数量,因为其他level的文件是不重叠的,只需要一个iterator。
  3. 当前正在处理的block,会放在block cache中。其大小是解压后的大小,所以会大于block_size。
  4. 输出文件的缓存。数据只缓存一个block,block写入后清楚,再缓存下一个block。除了数据还有meta的缓存。

rocksdb介绍之compaction流程相关推荐

  1. rocksdb原理_Rocksdb Compaction原理

    概述 compaction主要包括两类:将内存中imutable 转储到磁盘上sst的过程称之为flush或者minor compaction:磁盘上的sst文件从低层向高层转储的过程称之为compa ...

  2. 【flink】RocksDB介绍以及Flink对RocksDB的支持

    1.概述 转载:「Flink」RocksDB介绍以及Flink对RocksDB的支持 2.RocksDB简介 RocksDB是基于C++语言编写的嵌入式KV存储引擎,它不是一个分布式的DB,而是一个高 ...

  3. html5/css3响应式布局介绍及设计流程

    html5/css3响应式布局介绍 html5/css3响应式布局介绍及设计流程,利用css3的media query媒体查询功能.移动终端一般都是对css3支持比较好的高级浏览器不需要考虑响应式布局 ...

  4. Shiro介绍及主要流程

    Shiro介绍及主要流程 什么是Shiro Apache Shiro是一个强大且灵活的开源安全框架,易于使用且好理解,撇开了搭建安全框架时的复杂性. Shiro可以帮助我们做以下几件事: 认证使用者的 ...

  5. 06【群面】(无领导小组)群面基本介绍考察内容流程,群面常见问题解题思路,群面角色分析考官追问

    群面? (无领导小组)群面基本介绍&考察&内容&流程,群面常见问题&解题思路,群面角色分析&考官追问 第一节 群面基本介绍&考察&内容& ...

  6. 【6】爬虫介绍/准备工作/构建流程/获取数据/BeautifulSoup/Re(正则表达式)/正则提取/标签解析/保存数据到excel

    爬虫介绍/准备工作/构建流程/获取数据/BeautifulSoup/Re(正则表达式)/正则提取/标签解析/保存数据到excel 更新时间:2021.9.16 vedio:15,16,17,18,19 ...

  7. Doris Compaction 流程

    背景知识:Doris中compaction分为两种:BASE_COMPACTION 和 CUMULATIVE_COMPACTION 1.首先从 olap_server.cpp 中 start_bg_t ...

  8. linux内核开发入门二(内核KO模块介绍、开发流程以及注意事项)

    linux内核开发入门二(内核KO模块介绍.开发流程以及注意事项) 一.什么是内核模块 内核模块:ko模块(Kernel Object Module)是Linux内核中的可加载模块,它可以动态地向内核 ...

  9. RocksDB 介绍

    因为使用到了MyRocks存储引擎,需要了解下下RocksDB,发现国内介绍的不详细就自己来写一遍吧. 概述: RocksDB 是针对KV数据存储的高性能嵌入式数据库,由Facebook的Dhruba ...

最新文章

  1. C语言中不同类型的循环(Different types of loops in C)
  2. activemqcpp编译及可能的错误处理
  3. 苹果公布WWDC20大会安排、Snapchat上线小程序功能、QUIC和HTTP/3 将定稿|Decode the Week...
  4. An example of parsing xml file using Scala
  5. [react] 函数式组件有没有生命周期?为什么?
  6. python 中的面向对象
  7. Centos7下安装python3
  8. 人工智能的Hello World!
  9. 制作app怎么连接服务器,App制作步骤、流程有哪些?
  10. HTML静态网页作业-HTML5+CSS大作业——年会抽奖网页设计(1页)
  11. 实现原理 扫描枪_扫描枪原理
  12. ionic 项目文件下载总结
  13. 该内存不能为 read/written解决办法
  14. Maven学习(基础部分)
  15. ajax心得体会论文,AJAX重点知识的心得体会
  16. MPX + Vant Weapp 在微信小程序中实现Picker选择器
  17. 我整理了一些Python测试的库,看一下有没有你需要的库
  18. 《月亮与六便士》书摘
  19. 转载一篇关于泰迪很好的文章
  20. matlab 物流 算法,遗传算法求物流配送路径VRP问题MATLAB源码

热门文章

  1. Windows安装达梦数据库及初始化
  2. 手机算通用计算机还是,电脑耳机和手机耳机通用吗
  3. CSS(一)字体样式属性
  4. windows定时备份mysql数据库并自动压缩
  5. 【Tensorflow教程笔记】常用模块 tf.train.Checkpoint :变量的保存与恢复
  6. Python图片处理库Wand的简单使用
  7. 【Linux系统】第12节 Linux系统日志管理及日志的异地备份
  8. presto 的web ui
  9. 缓冲区溢出攻击与防范
  10. DVWA通关攻略之命令注入