Compaction过程中 产生大量读I/O 的背景

项目中因大value 需求,引入了PingCap 参考Wisckey 思想实现的key-value分离存储 titan, 使用过程中因为有用到Rocksdb本身的 CompactionFilter功能,所以就直接用TitanDB的option 传入了compaction filter。

使用过程中,单纯的通过db->Put接口写入 就会发现磁盘上大量的读I/O。

Ps : 相关的现象产生时的基本配置就不贴上来了,这个现象用过titan的compaction filter的同学应该都会比较清楚。

如果没有用过,但也发现了一些异常,也可以直接向后看。

我们数据写入量是 key:10B, value: 8K , 磁盘上的读本身是由于compaction引起的,compaction过程中需要将选择的sst文件中的key-value通过迭代器一个一个读取上来做堆排序。这个过程会产生读I/O,也就是只有Compaction 本身会有读I/O。

问题现象是单次compaction的量也就几十M,但磁盘上却产生了数百M的读I/O

更加直观的体现就是通过命令sudo iotop,可以看到此时大量的compaction 线程产生了读IO

问题分析

这里显然不合理,rocksdb的日志打印出来的LOG 中总共的compaction的带宽也就几十M,因为在titandb的key-value分离存储之后LSM-tree中仅仅存储了key和key-index,所以单次compaction的过程中理论上并不会携带着value参与,这样的大量I/O不太合理。

继续向下看,从iotop的输出中取出来一个compaction的线程ID,sudo strace -ttt -T -p 209278 抓取它的系统调用,可以看到大量的pread64系统调用

1617853714.972743 pread64(14224, "\203\250\206p\20/\0\0\0\fuid:11288154\201^\365.\0\0\n\362]\22"..., 8190, 13057621) = 12057 <0.000445>
1617853714.973241 pread64(13772, "\357\212\255y\20/\0\0\0\fuid:11288198\201^\365.\0\0\n\362]\22"..., 8190, 3267429) = 12057 <0.000013>
1617853714.973284 pread64(15591, "\343\3602\373\20/\0\0\0\fuid:11288239\201^\365.\0\0\n\362]\22"..., 8190, 3279482) = 12057 <0.000230>
...

可以看到pread64读到的数据大小是8190B,显然是我们写入的value大小,这货肯定读了存放value的blobfile

随机抽样几个fd ,也就是pread64(14224, "\20...)的第一参数,从进程的fd列表中看看它链接得是哪个文件ls -l /proc/xxx/fd | grep 209278

lr-x------ 1 kiwi2 kiwi2 64 Apr  8 11:49 /proc/209235/fd/10029 -> /mnt/db/14/titandb/000681.blob

果然是从blobfile中读取的数据,到这里我们就知道为什么compaction线程会有这么多的读,因为compaction过程中竟然读了blob file中的value。。。陷入沉思,梳理一下titan的写入逻辑。

  1. Key-value 都和以前rocksdb一样,先写入memtable
  2. 在Flush过程中形成sst文件的时候,通过titan自己的table-builder add的过程中来做区分,大于一个阈值时 分离value写入到blobfile中,key+key-index 存放到LSM-tree 的sst文件中
  3. 后续LSM-tree继续自己的compaction, blobfiles 则在达到触发gc条件的时候由一个线程池的一个线程调度blobfile的过期清理

也就是titan compaction过程中理论上仅仅是sst文件中的key + key-index参与,并不会涉及blobfiles 中的value,要不然key-value分离的意义何在?带宽还是没有降下来。

接下来的分析就更加明了了,看看这个时候大量读的compaction线程调用栈,直接上命令sudo pstack 209278(pstack底层也是调用gdb 执行的,不过是quiet指令执行,并不会阻塞线程),最后能看到如下调用栈

#0  0x00007faa05d93f73 in pread64 () from /lib64/libpthread.so.0
#1  0x000000000095f85e in pread (__offset=16132142, __nbytes=12057, __buf=0x3c074a000, __fd=<optimized out>)
#2  rocksdb::PosixRandomAccessFile::Read(unsigned long, unsigned long, rocksdb::Slice*, char*) const ()
#3  0x0000000000a0c0b1 in rocksdb::RandomAccessFileReader::Read(unsigned long, unsigned long, rocksdb::Slice*, char*, bool) const ()
#4  0x000000000081b197 in rocksdb::titandb::BlobFileReader::ReadRecord(rocksdb::titandb::BlobHandle const&, rocksdb::titandb::BlobRecord*, rocksdb::titandb::OwnedSlice*) ()
#5  0x000000000081ba21 in rocksdb::titandb::BlobFileReader::Get(rocksdb::ReadOptions const&, rocksdb::titandb::BlobHandle const&, rocksdb::titandb::BlobRecord*, rocksdb::PinnableSlice*) ()
#6  0x00000000008428e3 in rocksdb::titandb::BlobFileCache::Get(rocksdb::ReadOptions const&, unsigned long, unsigned long, rocksdb::titandb::BlobHandle const&, rocksdb::titandb::BlobRecord*, rocksdb::PinnableSlice*) ()
#7  0x00000000008396b8 in rocksdb::titandb::BlobStorage::Get(rocksdb::ReadOptions const&, rocksdb::titandb::BlobIndex const&, rocksdb::titandb::BlobRecord*, rocksdb::PinnableSlice*) ()
#8  0x00000000007f4a3b in rocksdb::titandb::TitanCompactionFilter::FilterV2 (this=0x3ceb05b00, level=0, key=..., value_type=<optimized out>, value=..., new_value=0x1be783cf8, skip_until=0x1be783d18)
#9  0x0000000000a2fa1a in InvokeFilterIfNeeded (skip_until=0x7fa9f786e730, need_skip=0x7fa9f786e72f, this=0x1be783b00)
#10 rocksdb::CompactionIterator::InvokeFilterIfNeeded (this=0x1be783b00, need_skip=0x7fa9f786e72f, skip_until=0x7fa9f786e730)
#11 0x0000000000a3039a in rocksdb::CompactionIterator::NextFromInput() ()
#12 0x0000000000a31c5a in rocksdb::CompactionIterator::Next (this=0x1be783b00)
#13 0x0000000000a39658 in rocksdb::CompactionJob::ProcessKeyValueCompaction(rocksdb::CompactionJob::SubcompactionState*) ()
#14 0x0000000000a3aa1c in rocksdb::CompactionJob::Run() ()
#15 0x0000000000887a5b in rocksdb::DBImpl::BackgroundCompaction(bool*, rocksdb::JobContext*, rocksdb::LogBuffer*, rocksdb::DBImpl::PrepickedCompaction*, rocksdb::Env::Priority) ()
#16 0x000000000088ab44 in rocksdb::DBImpl::BackgroundCallCompaction(rocksdb::DBImpl::PrepickedCompaction*, rocksdb::Env::Priority) ()
#17 0x000000000088b028 in rocksdb::DBImpl::BGWorkCompaction (arg=<optimized out>)
#18 0x0000000000a1437c in operator() (this=0x7fa9f7870370)
#19 rocksdb::ThreadPoolImpl::Impl::BGThread(unsigned long) ()
#20 0x0000000000a144d3 in rocksdb::ThreadPoolImpl::Impl::BGThreadWrapper (arg=<optimized out>)

这个调用栈中可以看到

#11 0x0000000000a3039a in rocksdb::CompactionIterator::NextFromInput() ()
#12 0x0000000000a31c5a in rocksdb::CompactionIterator::Next (this=0x1be783b00)
#13 0x0000000000a39658 in rocksdb::CompactionJob::ProcessKeyValueCompaction(rocksdb::CompactionJob::SubcompactionState*) ()
#14 0x0000000000a3aa1c in rocksdb::CompactionJob::Run() ()

这一些都是正常的compaction逻辑,但是再往上走就进入了compaction filter之中,使用了Titandb的filter函数,并且调用了rocksdb::titandb::BlobStorage::Get,确实,我们用户态用了compaction filter,但不应该调用到blob的Get,好吧。。。
直接看Titan的源代码。

Titan的Compaction Filter实现

在打开TitanDB的时候会将用户传入的compaction_filter作为一个子filter传进来,并且交给titan自己的TitanCompactionFilterFactory来处理

Status TitanDBImpl::OpenImpl(const std::vector<TitanCFDescriptor>& descs,std::vector<ColumnFamilyHandle*>* handles) {......std::vector<ColumnFamilyDescriptor> base_descs;std::vector<std::shared_ptr<TitanTableFactory>> titan_table_factories;for (auto& desc : descs) {......if (cf_opts.compaction_filter != nullptr ||cf_opts.compaction_filter_factory != nullptr) {std::shared_ptr<TitanCompactionFilterFactory> titan_cf_factory =std::make_shared<TitanCompactionFilterFactory>(cf_opts.compaction_filter, cf_opts.compaction_filter_factory,this, desc.options.skip_value_in_compaction_filter, desc.name);cf_opts.compaction_filter = nullptr;cf_opts.compaction_filter_factory = titan_cf_factory;}}// Open base DB.s = DB::Open(db_options_, dbname_, base_descs, handles, &db_);\......
}

进入TitanCompactionFilterFactoryCreateCompactionFilter函数
之前介绍Rocksdb的ComapctionFilter实现的时候知道,引擎对外暴漏了这一些接口,能够由用户来指定自己想要过滤什么样的key。

Rocskdb CompactionFilter实现

std::unique_ptr<CompactionFilter> CreateCompactionFilter(const CompactionFilter::Context &context) override {assert(original_filter_ != nullptr || original_filter_factory_ != nullptr);std::shared_ptr<BlobStorage> blob_storage;{MutexLock l(&titan_db_impl_->mutex_);blob_storage = titan_db_impl_->blob_file_set_->GetBlobStorage(context.column_family_id).lock();}if (blob_storage == nullptr) {assert(false);// Shouldn't be here, but ignore compaction filter when we hit error.return nullptr;}const CompactionFilter *original_filter = original_filter_;std::unique_ptr<CompactionFilter> original_filter_from_factory;if (original_filter == nullptr) {original_filter_from_factory =original_filter_factory_->CreateCompactionFilter(context);original_filter = original_filter_from_factory.get();}return std::unique_ptr<CompactionFilter>(new TitanCompactionFilter(titan_db_impl_, cf_name_, original_filter,std::move(original_filter_from_factory), blob_storage, skip_value_));
}

Factory会将TitanCompactionFilter返回,且这个filter也携带着用户自定义的Filteroriginal_filter。也就是comapction 过程中会先执行TitanCompactionFilterFilterV2函数,接着看一下titandb 的FilterV2函数:

Decision FilterV2(int level, const Slice &key, ValueType value_type,const Slice &value, std::string *new_value,std::string *skip_until) const override {......BlobRecord record;PinnableSlice buffer;ReadOptions read_options;// 问题源头s = blob_storage_->Get(read_options, blob_index, &record, &buffer);if (s.IsCorruption()) {// Could be cause by blob file beinged GC-ed, or real corruption.// TODO(yiwu): Tell the two cases apart.return Decision::kKeep;} else if (s.ok()) {// 用户自定义的Filter逻辑auto decision = original_filter_->FilterV2(level, key, kValue, record.value, new_value, skip_until);...}
}

可以看到这里会有一个blob_storage_->Get,到此我们就知道为什么会有一个blobfile 的Get了。

因为用户在回掉使用original_filter_->FilterV2逻辑的时候需要知道具体的value,所以Titan这里需要将blobfile中的value传回去。

OK。。。这样啊,那确实没有办法,毕竟想要拥有灵活性,代价是必不可少的。

解决

如果业务中针对compaction filter的需求是不需要value的数据,这里可以避免掉blobfile的Get,设置titan options skip_value_in_compaction_filter = true 即可。

TitanDB 中使用Compaction Filter ,产生了预期之外几十倍的读I/O相关推荐

  1. 一招连环追销,让顾客在第一次成交中买多个产品,客单价提高十倍

    怎样组合产品,才能让客户在第一次成交中就购买多个产品? 在推销产品过程中应该如何连环追销,才能几何倍增实体店的销售额? 先来介绍一下,什么是连环追销. 简单点来说,连环追销就是客户买了A,你推B,然后 ...

  2. ufserver文件上传服务器-用来管理你项目中的图片-性能可比Java、Python高十倍不止

    大家在做项目的时候如果是web项目,项目又不大的情况下上传图片一般都是上传到项目的webapp目录下面,webapp下面会有一个upload的文件夹,今天给大家分享的是在项目中如何把图片上传到其他服务 ...

  3. influxdb tsm文件_Influxdb中的Compaction操作

    Influxdb中的Compaction操作 Compaction概述 Influxdb的存储引擎使用了TSM文件结构,这其实也是在LSM-Tree基础针对时序特点作了改进,因此其与LSM-Tree类 ...

  4. ArcEngine中IFeatureClass.Search(filter, Recycling)方法中Recycling参数的理解

    转自 ArcEngine中IFeatureClass.Search(filter, Recycling)方法中Recycling参数的理解 ArcGIS Engine中总调用IFeatureClass ...

  5. 简述angular中constant和$filter的用法

    这里是修真院前端小课堂,每篇分享文从 [背景介绍][知识剖析][常见问题][解决方案][编码实战][扩展思考][更多讨论][参考文献] 八个方面深度解析前端知识/技能,本篇分享的是: [简述angul ...

  6. Vue中的过滤器(filter)

    一.Vue中的过滤器是什么 过滤器(filter)是输送介质管道上不可缺少的一种装置,大白话,就是把一些不必要的东西过滤掉,过滤器实质不改变原始数据,只是对数据进行加工处理后返回过滤后的数据再进行调用 ...

  7. HBASE中的compaction策略,日期分层

    DTCP(日期分层Compaction) 问题描述 ExploringCompactionPolicy是hbase minor compaction的默认策略.图一表现了这个算法默认设置下的表现情况 ...

  8. Java中的过滤器Filter

    本文用于学习所用,有不足及错误之处欢迎指出和补充. 目录 一:Filter简介 二:关于Filter (一)如何使用Filter进行过滤 (二)过滤器链 (FilterChain) (三)Filter ...

  9. spring web.xml中 过滤器(Filter)的工作原理和代码演示

    一.Filter简介 Filter也称之为过滤器,它是Servlet技术中最激动人心的技术之一,WEB开发人员通过Filter技术,对web服务器管理的所有web资源:例如Jsp, Servlet, ...

最新文章

  1. python获利模式_Python 分段利润提成
  2. android 设置允许http请求_网络请求框架----OkHttp原理
  3. dpi重启后会恢复_Linux 系统的备份恢复
  4. (转)UITableViewCell复用问题
  5. SQL Server常用函数 -- 更新中
  6. 路由器上的lookback是什么?有什么作用?
  7. JAVA中自增自减运算符(i++与++i的区别)
  8. webgl之helloworld
  9. oracle怎样查询能利用索引,oracle怎样查询索引的使用情况
  10. 用WebBrowser实现HTML界面的应用
  11. TextView用法及实例
  12. python写彩票预测软件_python写彩票预测软件
  13. 如何选择配置管理工具
  14. 23王道——建立中序线索树,找前驱后继,并用其遍历
  15. mysql 主从 只读,mysql主从复制(从库只读)
  16. 跳台阶算法的三种实现方案(PHP)
  17. import和export在浏览器中的使用方式
  18. Day18--拉氏变换与控制系统模型
  19. centos 7.6 ——远程访问及控制——(ssh密钥登录、ssh客户端、TCP Wrappers)
  20. 【东大自控笔记9】一文掌握根轨迹法

热门文章

  1. 【云安全与同态加密_调研分析(3)】国内云安全组织及标准——By Me
  2. DataCleaner 3.1.1 发布,数据质量分析管理
  3. php类退出魔术方法,php类中常用的魔术方法
  4. 百度云api android,帮助文档首页/百度移动统计API/百度移动统计 Android版SDK - 百度开放云平台...
  5. linux的逻辑运算参数,Linux基础之bash脚本编程初级-逻辑运算与测试
  6. java 生成pdf itext_使用Java组件itext 生成pdf介绍
  7. java vagrant_vagrant 做一个java web开发环境
  8. linux下从git获取有权限的代码,linux下从源代码安装git
  9. python生成试卷制卷系统_Python 读写文件 小应用:生成随机的测验试卷文件
  10. 【python语言基础】疑难点整理2