LSM 优化系列(四) -- Rocksdb和Lethe 对Delete问题的优化
文章目錄
- 前言
- 1. 問題背景
- 2. 問題復現
- 3. Rocksdb 的 Delete-Aware 優化
- 3.1 可配置的 Delete-Aware調度
- 3.2 Compaction 邏輯對 delete key的優化
- 4. Lethe: A Tunable Delete-Aware LSM Engine . SIGMOD'20
前言
本文介紹過程中涉及到的源代碼是基于rocksdb 6.4.6 版本的。
同時需要對rocksdb的Compaction邏輯以及源代碼有較為深入的了解,相關的原理介紹之前有一些粗淺的分析,對理解不夠深入的同學應該有幫助。
1. SST文件詳細格式源碼解析
2. Compaction 完整實現過程 概覽
1. 問題背景
基于LSM的存儲引擎 在提供了優秀的寫性能的基礎上 卻因為compaction帶來了很多其他問題。
本文所要描述的一個場景是LSM引擎的標記刪除(將對一個key的刪除作為一個key-value 追加寫入到引擎之中),這種刪除邏輯會帶來如下幾個問題:
- 讀放大:range scan 的時候需要掃描更多的記錄,點查詢Bloom Filter 的false positive概率增加。
- 空間放大:因為delete的type 也是一個key-value,而這種類型的key只有在compaction到最后一層才會被刪除,中間層的很多空間就會被浪費在delete的存儲之上。
- 寫放大:本身LSM compaction帶來的原有問題,而針對delete key的頻繁調度compaction 帶來的寫放大問題更不友好。
- 用戶隱私:用戶已經標記刪除的數據并沒有及時刪除,未到最后一層,無法真正刪除。
如上圖:標記D的都是含有delete 的sst文件,加入我們想要查找的key在L2,這個key其實之前已經被刪除了。但是數據并沒有被清理,對應sst文件的bloom filter也沒有更新,所以無法準確判斷改key是否還在,則需要一直讀到 purpose key所在的sst文件的data block,發現這個key時delete type,最后才向客戶端返回not found。
這個查找過程存在數次I/O操作,讀sst文件的 filter block,index block, 整個data block,真刪除的不存在key的查找 多了 兩個block的IO流程(index block 和 data block),代價可想而知。
2. 問題復現
因為業務中存在這樣的問題,所以直接上代碼,更加直觀。
復現如下簡單且經典場景(sql刪除了一個表中80%的數據,后來想scan剩下的一部分數據):
寫入1千萬的相同前綴 且字典序遞增的key,刪除其中的的9百萬,從頭遍歷 直到取得未刪除的前500條key。
站在用戶的角度:
刪除了一批key, 讀取剩下的key中的100條
引擎的角度:
用戶調用了刪除接口,寫入了大量的delete type的key。但是 delete type的key還在LSM 上層,一點一點做compaction到最后一層。所以用戶調用一批scan 查找的過程相當于將已經標記delete 但是還未清理數據的key都得掃一遍。
通過實現一段rocksdb的復現代碼如下:
#include <unistd.h>
#include <atomic>
#include <iostream>
#include <random>
#include <string>
#include <thread>#include <sys/time.h>#include "rocksdb/db.h"
#include "rocksdb/table.h"
#include "rocksdb/cache.h"
#include "rocksdb/filter_policy.h"
#include "rocksdb/write_batch.h"
#include "rocksdb/rate_limiter.h"
#include "rocksdb/perf_context.h"
#include "rocksdb/iostats_context.h"
#include "rocksdb/table_properties.h"
#include "utilities/table_properties_collectors.h"
#include "gflags/gflags.h"#define VALUE_LEN 500 using namespace google;DEFINE_int64(time,1200,"read threads' ratio");
DEFINE_int64(thread_num,64,"total threads ");
DEFINE_string(db_dir, "srd_delete", "db's dir is delete");
DEFINE_bool(use_fullcompaction, false, "use full compaction to db");
DEFINE_int64(num_keys, 1000000,"write keys' num");
DEFINE_int64(wait_compaction,10,"wait");
DEFINE_bool(traverse, true, "use traverse");
DEFINE_bool(use_delete_collector, false, "use traverse");std::atomic<long> g_op_W;rocksdb::DB* src_db;
rocksdb::Options options;
std::mt19937_64 generator_; // 生成偽隨機數// 返回微秒
inline int64_t now() {struct timeval t;gettimeofday(&t, NULL);return static_cast<int64_t>(t.tv_sec)*1000000 + t.tv_usec;
}// 打開rocksdb
void OpenDB() {options.create_if_missing = true;options.compression = rocksdb::kNoCompression;options.disable_auto_compactions = FLAGS_use_fullcompaction;if(FLAGS_use_delete_collector) {options.table_properties_collector_factories.emplace_back(rocksdb::NewCompactOnDeletionCollectorFactory(1000000, 1000));}std::shared_ptr<rocksdb::Cache> cache = rocksdb::NewLRUCache(10737418240);rocksdb::BlockBasedTableOptions bbto;bbto.whole_key_filtering = true;bbto.cache_index_and_filter_blocks = true;bbto.filter_policy.reset(rocksdb::NewBloomFilterPolicy(16,false));bbto.block_cache = cache;options.table_factory.reset(rocksdb::NewBlockBasedTableFactory(bbto));options.max_background_compactions = 32;auto s = rocksdb::DB::Open(options, FLAGS_db_dir, &src_db);std::cout << "open src_db" << s.ToString() << std::endl;}// 配置手動執行full compaction
void FullCompaction() {rocksdb::Slice begin("");rocksdb::Iterator *it = src_db->NewIterator(rocksdb::ReadOptions());it->SeekToLast();std::string end_str = it->key().ToString();delete it;rocksdb::Slice end(end_str);auto s = src_db->CompactRange(rocksdb::CompactRangeOptions(), &begin, &end);if (!s.ok()) {std::cout << "CompactRange failed " << s.ToString() << std::endl;}
}// 寫入 + 刪除
void DOWrite(int num) {int count = FLAGS_num_keys;std::string value(VALUE_LEN, 'x'); while(count > 0) {src_db -> Put(rocksdb::WriteOptions(), std::string("test_compaction_129_")+std::to_string(count),value);++ g_op_W;count --;}// 刪除90% 的keystd::cout << "Begin delete "<< std::endl;for (int i = 0;i < FLAGS_num_keys - FLAGS_num_keys / 10; ++i) {src_db->Delete(rocksdb::WriteOptions(), std::string("test_compaction_129_")+std::to_string(i));}src_db->SetOptions(src_db->DefaultColumnFamily(),{{"diable_auto_compactions","true"}});std::cout << "End delete \n";// 開啟full compactionif(FLAGS_use_fullcompaction) {FullCompaction();sleep(FLAGS_wait_compaction);}
}void Traverse() {// open perf rocksdb::SetPerfLevel(rocksdb::PerfLevel::kEnableTimeExceptForMutex);rocksdb::get_perf_context()->Reset();rocksdb::get_iostats_context()->Reset();rocksdb::Iterator *it = src_db->NewIterator(rocksdb::ReadOptions());it->Seek("");std::cout << "Traverse begin :" << std::endl;int64_t ts = now();int seek_count = 0;for(; it->Valid()&& seek_count<500; it->Next() ) {std::cout << it->key().ToString() << std::endl; seek_count++;}delete it;// close perf,查看internal_delete_skipped_count指標rocksdb::SetPerfLevel(rocksdb::PerfLevel::kDisable);std::cout << "Traverse use time :" << now() - ts << std::endl;std::cout << "internal_delete_skipped_count " << rocksdb::get_perf_context()->internal_delete_skipped_count<< std::endl;
}int main(int argc, char** argv) {ParseCommandLineFlags(&argc,&argv, true);OpenDB();for(int i = 0;i < FLAGS_thread_num; i++) {new std::thread(DOWrite,i);}long last_opn_W = 0;int count = FLAGS_time;while(count > 0) {sleep(1);long nopn_W = g_op_W;std::cout << "write_speed : " << nopn_W - last_opn_W << std::endl;last_opn_W = nopn_W;count --;}if(FLAGS_traverse){Traverse();}return 0;
}
編譯時需要加入gflags 的鏈接,最終可以通過如下命令進行測試:
測試原生compaction邏輯下 traverse過程的耗時
./test_delete -use_fullcompaction=false -time=300 -db_dir=./src_delete -thread_num=1 -num_keys=10000000 -use_delete_collector=false
可以發現最終的輸出:
Traverse use time :2018292
也就是在客戶端看來,只獲取100個存在的key就話費2s的時間。。。這簡直不能忍,這樣的存儲引擎如何讓用戶安心使用。
這里嘗試使用 Full Compaction 進行優化,即rocksdb自己提供的手動compaction接口CompacRange,在刪除操作執行完成之后對整個db進行了一次全量的compaction操作,再次進行遍歷獲取100個存在key的耗時統計。
以上的代碼也有增加,使用如下命令運行
./test_delete -use_fullcompaction=true -time=300 -db_dir=./src_delete -thread_num=1 -num_keys=10000000 -use_delete_collector=false
可以看到這次的運行耗時下降了3個量級,僅僅只有364us:
Traverse use time :364
那是不是我們這個問題就解決了呢?顯然不可能。
全量compaction 帶來的代價是極大的,所有的sst文件不論之前有沒有參與過compaction都會被調度起來,巨量的I/O 和 CPU 洪峰,且跟隨本節點數據量的大小持續一段時間;這個過程可能直接讓系統的可用性極低甚至掉零,更為嚴重的是某位不講武德的仁兄看到優化效果如此給力,不經過嚴格的測試直接線上跑 ,那這位仁兄就可以準備好掛個p0 走人了。
況且存儲引擎作為持久化存儲的核心,這樣的危險操作顯然不會交給用戶來做,所以本身引擎內部要有足量機制降低 未compaction清理的delete key對scan性能的影響,這也是引擎的職責所在。
總結這個問題放在引擎中處理 的幾個關節點 如下:
- delete的過程是通過compaction進行的,而compaction對于用戶來說是黑盒,用戶很難精確控制compaction過程中的delete key清理時機。
- 實際LSM的應用場景中 用戶驅動刪除的workload 相對較少,除了用戶主動刪除之外LSM 引擎也會應用在關系數據庫之中 出現drop 表等被動刪除場景,所以在用戶不知道的情況下引擎也會產生大范圍的數據清理和遷移。
- 對非連續的key的刪除調度。比如key的相同前綴是基于table id,刪除時則基于某一時間戳進行刪除。這個過程需要調度seek進行大量的查找,代價極大。
接下來將介紹一下 rocksdb 以及 業界對該場景的優化手段。rocksdb這里比較熟悉,會深入源代碼詳細介紹一番;業界對該優化的過程將介紹幾篇已有的優化論文。
3. Rocksdb 的 Delete-Aware 優化
基于以上問題,接下來會介紹Rocksdb 在該場景所做的兩個優化feature。
Rocksdb 在保證不增加LSM 的讀代價的情況下 針對delete type的key增加了兩方面的優化邏輯:
- 用戶態可配置的delete 密集型 sst文件的compaction優先級調度。
- compaction本身針對 delete key 邏輯的 處理優化。
3.1 可配置的 Delete-Aware調度
OpenDB函數中增加如下rocksdb配置:
options.table_properties_collector_factories.emplace_back(rocksdb::NewCompactOnDeletionCollectorFactory(10000, 1000));
同時需要包含頭文件:
#include "rocksdb/table_properties.h"
#include "utilities/table_properties_collectors.h"
查看以上函數聲明:
std::shared_ptr<CompactOnDeletionCollectorFactory>NewCompactOnDeletionCollectorFactory(size_t sliding_window_size,size_t deletion_trigger) {return std::shared_ptr<CompactOnDeletionCollectorFactory>(new CompactOnDeletionCollectorFactory(sliding_window_size, deletion_trigger));
}// A factory of a table property collector that marks a SST
// file as need-compaction when it observe at least "D" deletion
// entries in any "N" consecutive entires.
//
// @param sliding_window_size "N"
// @param deletion_trigger "D"
CompactOnDeletionCollectorFactory(size_t sliding_window_size,size_t deletion_trigger): sliding_window_size_(sliding_window_size),
deletion_trigger_(deletion_trigger) {}
總結一下:
NewCompactOnDeletionCollectorFactory函數聲明一個用戶配置的表格屬性收集器,這里的表格指的是sstable。
該函數需要傳入的兩個參數sliding_window_size和 deletion_trigger 表示刪除收集器 的滑動窗口 和 觸發刪除的key的個數。
在每個sst文件內維護一個滑動窗口,滑動窗口維護了一個窗口大小,在當前窗口大小內出現的delete-key的個數超過了窗口的大小,那么這個sst文件會被標記為need_compaction_,從而在下一次的compaction過程中被優先調度。
達到將delete-key較多的sst文件快速下沉到底層,delete的數據被真正刪除的目的,可以理解為這是一種更加激進的compaction調度策略。
使用者僅僅需要調整兩個參數的比例,即可決定該sst文件是否應該被優先調度compaction。
接下來詳細說一下Rocksdb的這個配置如何實現deletes 超過一定比例之后被compaction優先調度:
用戶配置了table_properties_collector_factories
在Compaction的核心處理key-value的函數CompactionJob::ProcessKeyValueCompaction 中調度 BlockBasedTableBuilder::Add 函數 構造sst文件中不同存儲區域的builder,這里面會使用用戶構造的collector 進行 指定key的收集,通過如下函數NotifyCollectTableCollectorsOnAdd 進入 InternalAdd 。
bool NotifyCollectTableCollectorsOnAdd(const Slice& key, const Slice& value, uint64_t file_size,const std::vector<std::unique_ptr<IntTblPropCollector>>& collectors,Logger* info_log) {bool all_succeeded = true;for (auto& collector : collectors) {Status s = collector->InternalAdd(key, value, file_size); all_succeeded = all_succeeded && s.ok();if (!s.ok()) {LogPropertiesCollectionError(info_log, "Add" /* method */,collector->Name());}}return all_succeeded;
}
因我我們reset的是NewCompactOnDeletionCollectorFactory collector,所以會通過UserKeyTablePropertiesCollector::InternalAdd 進入CompactOnDeletionCollector::AddUserKey collector的AddUserKey函數:
- 增加滑動窗口內 真正key的數量,并根據是否達到了窗口閾值 來及時更新內部key的數量 以及 降低舊的delete key的數量
- 根據傳入的key的類型來 增加滑動窗口內 delete key的數量,達到刪除閾值時會打入
need_compaction_標記(這個標記會在后續的compaction邏輯中將當前sst文件標記為優先級較高的sst文件)
Status CompactOnDeletionCollector::AddUserKey(const Slice& /*key*/,const Slice& /*value*/,EntryType type,SequenceNumber /*seq*/,uint64_t /*file_size*/) {...// 這里的bucket_size_可以理解為我們設置的滑動窗口大小// 第一個if語句中做的事情主要是更新滑動窗口中 key的數量 以及 舊的delete key的數量if (num_keys_in_current_bucket_ == bucket_size_) {// When the current bucket is full, advance the cursor of the// ring buffer to the next bucket.current_bucket_ = (current_bucket_ + 1) % kNumBuckets;// Update the current count of observed deletion keys by excluding// the number of deletion keys in the oldest bucket in the// observation window.assert(num_deletions_in_observation_window_ >=num_deletions_in_buckets_[current_bucket_]);num_deletions_in_observation_window_ -=num_deletions_in_buckets_[current_bucket_];num_deletions_in_buckets_[current_bucket_] = 0;num_keys_in_current_bucket_ = 0;}num_keys_in_current_bucket_++;// 根據key的類型來調整是否達到觸發刪除的閾值,從而打入need_compaction_標記if (type == kEntryDelete) { num_deletions_in_observation_window_++;num_deletions_in_buckets_[current_bucket_]++;if (num_deletions_in_observation_window_ >= deletion_trigger_) {need_compaction_ = true;}}...
}
在collector中達到觸發刪除的標記之后會在BlockBasedTableBuilder::NeedCompact()中體現,而這個函數則會被CompactionJob::FinishCompactionOutputFile調用
bool BlockBasedTableBuilder::NeedCompact() const {for (const auto& collector : rep_->table_properties_collectors) {if (collector->NeedCompact()) {return true;}}return false;
}
CompactionJob::FinishCompactionOutputFile同樣是在compaction的核心邏輯ProcessKeyValueCompaciton 中調用,以sst文件為單位 將構造好的一個個builder寫入到具體的sst文件中的數據區域,寫入過程中對要寫入的sst文件打入標記(如果collector中確認delete keys超過了設置的閾值)
meta->marked_for_compaction = sub_compact->builder->NeedCompact();
marked_for_compaction標記會在函數VersionStorageInfo::ComputeFilesMarkedForCompaction 被使用,其中將有marked_for_compaction標記文件添加到一個動態數組之中files_marked_for_compaction_,這個數組會在當前compaction完成之后再次被調度,將動態數組中的sst文件優先進行compaction的調度。
void VersionStorageInfo::ComputeFilesMarkedForCompaction() {files_marked_for_compaction_.clear();int last_qualify_level = 0;// Do not include files from the last level with data// If table properties collector suggests a file on the last level,// we should not move it to a new level.for (int level = num_levels() - 1; level >= 1; level--) {if (!files_[level].empty()) {last_qualify_level = level - 1;break;}}for (int level = 0; level <= last_qualify_level; level++) {for (auto* f : files_[level]) {if (!f->being_compacted && f->marked_for_compaction) {// 將被打入標記的文件添加到動態數組之中files_marked_for_compaction_.emplace_back(level, f); }}}
}
后續的調度過程會在DBImpl::BackgroundCompaction中,通過如下邏輯調用 LevelCompactionPicker的NeedsCompaction來進行判斷,判斷的過程主要是確認files_marked_for_compaction_不為空即返回true,從而再次將文件添加到當前column family的待compaction隊列,最后通過MaybeScheduleFlushOrCompaction再次調度。
if (cfd->NeedsCompaction()) {// Yes, we need more compactions!AddToCompactionQueue(cfd);++unscheduled_compactions_;MaybeScheduleFlushOrCompaction();}
整個鏈路相對來說比較長且復雜的,需要對Compaction的調度代碼較為熟悉的話會更容易理解。
最后能夠在rocksdb的LOG日志來確認是否觸發了Marked的邏輯:
"compaction_reason": "ManualCompaction", 這樣的字段,且后續的Compaction job的輸出中能夠看到records in的數據 和records dropped的數據的比例滿足我們設置的collector中的 sliding-window-size 和 delete-trigger的比例即可。
最后的性能可以通過上一章節中的 問題復現 的代碼看到。
運行如下命令:
./test_delete -use_fullcompaction=false -time=300 -db_dir=./src_delete -thread_num=1 -num_keys=10000000 -use_delete_collector=true
可以看到相同的delete場景,最后的traverse性能相比于原來能夠提升4倍,這個數據在不同的delete 比重以及collector設置的參數下有差異的。
Traverse use time :674629
3.2 Compaction 邏輯對 delete key的優化
Compaction調度過程不再深入描述了,直接到上小節說過的ProccessKeyValueCompaction函數中會對key-value一個一個處理,會通過Compaction迭代器的移動(基本的SeekToFirst,Next等)來達到一個一個處理的目的。
處理的過程通過調用NextFromInput函數來進行,這個函數會處理不同的key-type,其中關于delete類型的主要處理邏輯如下代碼(這一部分全部粘貼到下面了,可以簡單看看):
我們知道原本LSM 中的delete key 只能在最后一層將之前所有的寫入包括自己都清理掉,因為到了最后一層就知道之前已經沒有人再使用這個key了。
這里可能有人會問,用戶已經刪除了,為什么還一定要到最后一層才將key的數據清理掉呢?
因為rocksdb提供了snapshot功能,即在當前delete之前可能存在用戶打入的snapshot,即rocksdb需要保證那個時刻的key能夠被讀到,所以刪除的過程中都需要確認當前key之前版本是否有snapshot的sequence number,如果沒有才允許后續的刪除判斷。
} else if (compaction_ != nullptr && ikey_.type == kTypeDeletion &&IN_EARLIEST_SNAPSHOT(ikey_.sequence) &&ikeyNotNeededForIncrementalSnapshot() &&compaction_->KeyNotExistsBeyondOutputLevel(ikey_.user_key,&level_ptrs_)) {// TODO(noetzli): This is the only place where we use compaction_// (besides the constructor). We should probably get rid of this// dependency and find a way to do similar filtering during flushes.//// For this user key:// (1) there is no data in higher levels// (2) data in lower levels will have larger sequence numbers// (3) data in layers that are being compacted here and have// smaller sequence numbers will be dropped in the next// few iterations of this loop (by rule (A) above).// Therefore this deletion marker is obsolete and can be dropped.//// Note: Dropping this Delete will not affect TransactionDB// write-conflict checking since it is earlier than any snapshot.//// It seems that we can also drop deletion later than earliest snapshot// given that:// (1) The deletion is earlier than earliest_write_conflict_snapshot, and// (2) No value exist earlier than the deletion.++iter_stats_.num_record_drop_obsolete;if (!bottommost_level_) {++iter_stats_.num_optimized_del_drop_obsolete;}input_->Next();} else if ((ikey_.type == kTypeDeletion) && bottommost_level_ &&ikeyNotNeededForIncrementalSnapshot()) {// Handle the case where we have a delete key at the bottom most level// We can skip outputting the key iff there are no subsequent puts for this// keyParsedInternalKey next_ikey;input_->Next();// Skip over all versions of this key that happen to occur in the same snapshot// range as the deletewhile (input_->Valid() && ParseInternalKey(input_->key(), &next_ikey) &&cmp_->Equal(ikey_.user_key, next_ikey.user_key) &&(prev_snapshot == 0 ||DEFINITELY_NOT_IN_SNAPSHOT(next_ikey.sequence, prev_snapshot))) {input_->Next();}// If you find you still need to output a row with this key, we need to output the// delete tooif (input_->Valid() && ParseInternalKey(input_->key(), &next_ikey) &&cmp_->Equal(ikey_.user_key, next_ikey.user_key)) {valid_ = true;at_next_ = true;}}
以上邏輯的優化主要體現在第一個if語句之中,即當前delete 的key可能不在最后一層,通過這個函數KeyNotExistsBeyondOutputLevel判斷后續的層中沒有當前key了, 那么就可以直接刪除;這個邏輯會增加一定的CPU比較代價,但是它的收益是可觀的,能夠在較高的層中將key直接刪除。
4. Lethe: A Tunable Delete-Aware LSM Engine . SIGMOD’20
這是一篇 今年頂會 SIGMOD中發表的論文,核心就是LSM 引擎帶來的delete問題。
在該篇論文中詳細描述了delete 對LSM帶來的問題,且現有的引擎沒有很好的辦法很好得解決大范圍刪除的問題。
Lethe通過:
- 增加統計信息和compaction策略,優化sort key大范圍刪除帶來的問題
- 設計新的storage layout,來提升secondary key的大范圍刪除。
提出了了兩種策略:
-
FADE(fast deletion): 更加激進的compaction調度策略。相比于rocksdb的可配置策略,這里增加了文件的TTL,優先針對達到TTL的文件進行挑選(可以根據TTL主動觸發調度),調度compaction刪除。
-
KiWi(Key Weaving storage layout): 提高支持非sort key的range delete,犧牲了一定的查詢代價,提升了范圍刪除的效率。這個設計 感覺和rocksdb的DeleteRange有點類似,就是不知道性能差異大嗎?
一個X-engine的大佬已經有一篇該論文的詳細分享了。
Lethe 如何優化 LSM-Tree delete 難題
論文原地址: https://cs-people.bu.edu/dstara/pdfs/Lethe.pdf
后續嘗試將Lethe這種更加激進的策略集成到rocksdb中,看看效果如何,理論上在Delete 比例重的場景,應該優于Rocksdb本身的優化。
總結
以上是生活随笔為你收集整理的LSM 优化系列(四) -- Rocksdb和Lethe 对Delete问题的优化的全部內容,希望文章能夠幫你解決所遇到的問題。
- 上一篇: 改名多少钱啊?
- 下一篇: 女性不孕不育一般多久能治好