diff --git a/CMakeLists.txt b/CMakeLists.txt index 717fb0568..56b914909 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -336,10 +336,12 @@ set(SOURCES table/format.cc table/full_filter_block.cc table/get_context.cc + table/index_builder.cc table/iterator.cc table/merging_iterator.cc table/sst_file_writer.cc table/meta_blocks.cc + table/partitioned_filter_block.cc table/plain_table_builder.cc table/plain_table_factory.cc table/plain_table_index.cc diff --git a/src.mk b/src.mk index f6e6d0168..b1f1aabbf 100644 --- a/src.mk +++ b/src.mk @@ -72,10 +72,12 @@ LIB_SOURCES = \ table/format.cc \ table/full_filter_block.cc \ table/get_context.cc \ + table/index_builder.cc \ table/iterator.cc \ table/merging_iterator.cc \ table/meta_blocks.cc \ table/sst_file_writer.cc \ + table/partitioned_filter_block.cc \ table/plain_table_builder.cc \ table/plain_table_factory.cc \ table/plain_table_index.cc \ diff --git a/table/block_based_table_builder.cc b/table/block_based_table_builder.cc index 23da9fcef..daf9c6319 100644 --- a/table/block_based_table_builder.cc +++ b/table/block_based_table_builder.cc @@ -31,14 +31,16 @@ #include "rocksdb/table.h" #include "table/block.h" +#include "table/block_based_filter_block.h" +#include "table/block_based_table_factory.h" #include "table/block_based_table_reader.h" #include "table/block_builder.h" #include "table/filter_block.h" -#include "table/block_based_filter_block.h" -#include "table/block_based_table_factory.h" -#include "table/full_filter_block.h" #include "table/format.h" +#include "table/full_filter_block.h" +#include "table/index_builder.h" #include "table/meta_blocks.h" +#include "table/partitioned_filter_block.h" #include "table/table_builder.h" #include "util/string_util.h" @@ -54,384 +56,10 @@ extern const std::string kHashIndexPrefixesBlock; extern const std::string kHashIndexPrefixesMetadataBlock; typedef BlockBasedTableOptions::IndexType IndexType; -class IndexBuilder; - -namespace { -rocksdb::IndexBuilder* CreateIndexBuilder( - IndexType index_type, const InternalKeyComparator* comparator, - const SliceTransform* prefix_extractor, int index_block_restart_interval, - uint64_t index_per_partition); -} - -// The interface for building index. -// Instruction for adding a new concrete IndexBuilder: -// 1. Create a subclass instantiated from IndexBuilder. -// 2. Add a new entry associated with that subclass in TableOptions::IndexType. -// 3. Add a create function for the new subclass in CreateIndexBuilder. -// Note: we can devise more advanced design to simplify the process for adding -// new subclass, which will, on the other hand, increase the code complexity and -// catch unwanted attention from readers. Given that we won't add/change -// indexes frequently, it makes sense to just embrace a more straightforward -// design that just works. -class IndexBuilder { - public: - // Index builder will construct a set of blocks which contain: - // 1. One primary index block. - // 2. (Optional) a set of metablocks that contains the metadata of the - // primary index. - struct IndexBlocks { - Slice index_block_contents; - std::unordered_map meta_blocks; - }; - explicit IndexBuilder(const InternalKeyComparator* comparator) - : comparator_(comparator) {} - - virtual ~IndexBuilder() {} - - // Add a new index entry to index block. - // To allow further optimization, we provide `last_key_in_current_block` and - // `first_key_in_next_block`, based on which the specific implementation can - // determine the best index key to be used for the index block. - // @last_key_in_current_block: this parameter maybe overridden with the value - // "substitute key". - // @first_key_in_next_block: it will be nullptr if the entry being added is - // the last one in the table - // - // REQUIRES: Finish() has not yet been called. - virtual void AddIndexEntry(std::string* last_key_in_current_block, - const Slice* first_key_in_next_block, - const BlockHandle& block_handle) = 0; - - // This method will be called whenever a key is added. The subclasses may - // override OnKeyAdded() if they need to collect additional information. - virtual void OnKeyAdded(const Slice& key) {} - - // Inform the index builder that all entries has been written. Block builder - // may therefore perform any operation required for block finalization. - // - // REQUIRES: Finish() has not yet been called. - inline Status Finish(IndexBlocks* index_blocks) { - // Throw away the changes to last_partition_block_handle. It has no effect - // on the first call to Finish anyway. - BlockHandle last_partition_block_handle; - return Finish(index_blocks, last_partition_block_handle); - } - - // This override of Finish can be utilized to build the 2nd level index in - // PartitionIndexBuilder. - // - // index_blocks will be filled with the resulting index data. If the return - // value is Status::InComplete() then it means that the index is partitioned - // and the callee should keep calling Finish until Status::OK() is returned. - // In that case, last_partition_block_handle is pointer to the block written - // with the result of the last call to Finish. This can be utilized to build - // the second level index pointing to each block of partitioned indexes. The - // last call to Finish() that returns Status::OK() populates index_blocks with - // the 2nd level index content. - virtual Status Finish(IndexBlocks* index_blocks, - const BlockHandle& last_partition_block_handle) = 0; - - // Get the estimated size for index block. - virtual size_t EstimatedSize() const = 0; - - protected: - const InternalKeyComparator* comparator_; -}; - -// This index builder builds space-efficient index block. -// -// Optimizations: -// 1. Made block's `block_restart_interval` to be 1, which will avoid linear -// search when doing index lookup (can be disabled by setting -// index_block_restart_interval). -// 2. Shorten the key length for index block. Other than honestly using the -// last key in the data block as the index key, we instead find a shortest -// substitute key that serves the same function. -class ShortenedIndexBuilder : public IndexBuilder { - public: - explicit ShortenedIndexBuilder(const InternalKeyComparator* comparator, - int index_block_restart_interval) - : IndexBuilder(comparator), - index_block_builder_(index_block_restart_interval) {} - - virtual void AddIndexEntry(std::string* last_key_in_current_block, - const Slice* first_key_in_next_block, - const BlockHandle& block_handle) override { - if (first_key_in_next_block != nullptr) { - comparator_->FindShortestSeparator(last_key_in_current_block, - *first_key_in_next_block); - } else { - comparator_->FindShortSuccessor(last_key_in_current_block); - } - - std::string handle_encoding; - block_handle.EncodeTo(&handle_encoding); - index_block_builder_.Add(*last_key_in_current_block, handle_encoding); - } - - virtual Status Finish( - IndexBlocks* index_blocks, - const BlockHandle& last_partition_block_handle) override { - index_blocks->index_block_contents = index_block_builder_.Finish(); - return Status::OK(); - } - - virtual size_t EstimatedSize() const override { - return index_block_builder_.CurrentSizeEstimate(); - } - - private: - BlockBuilder index_block_builder_; -}; - -/** - * IndexBuilder for two-level indexing. Internally it creates a new index for - * each partition and Finish then in order when Finish is called on it - * continiously until Status::OK() is returned. - * - * The format on the disk would be I I I I I I IP where I is block containing a - * partition of indexes built using ShortenedIndexBuilder and IP is a block - * containing a secondary index on the partitions, built using - * ShortenedIndexBuilder. - */ -class PartitionIndexBuilder : public IndexBuilder { - public: - explicit PartitionIndexBuilder(const InternalKeyComparator* comparator, - const SliceTransform* prefix_extractor, - const uint64_t index_per_partition, - int index_block_restart_interval) - : IndexBuilder(comparator), - prefix_extractor_(prefix_extractor), - index_block_builder_(index_block_restart_interval), - index_per_partition_(index_per_partition), - index_block_restart_interval_(index_block_restart_interval) { - sub_index_builder_ = - CreateIndexBuilder(sub_type_, comparator_, prefix_extractor_, - index_block_restart_interval_, index_per_partition_); - } - - virtual ~PartitionIndexBuilder() { delete sub_index_builder_; } - - virtual void AddIndexEntry(std::string* last_key_in_current_block, - const Slice* first_key_in_next_block, - const BlockHandle& block_handle) override { - sub_index_builder_->AddIndexEntry(last_key_in_current_block, - first_key_in_next_block, block_handle); - num_indexes++; - if (UNLIKELY(first_key_in_next_block == nullptr)) { // no more keys - entries_.push_back({std::string(*last_key_in_current_block), - std::unique_ptr(sub_index_builder_)}); - sub_index_builder_ = nullptr; - } else if (num_indexes % index_per_partition_ == 0) { - entries_.push_back({std::string(*last_key_in_current_block), - std::unique_ptr(sub_index_builder_)}); - sub_index_builder_ = CreateIndexBuilder( - sub_type_, comparator_, prefix_extractor_, - index_block_restart_interval_, index_per_partition_); - } - } - - virtual Status Finish( - IndexBlocks* index_blocks, - const BlockHandle& last_partition_block_handle) override { - assert(!entries_.empty()); - // It must be set to null after last key is added - assert(sub_index_builder_ == nullptr); - if (finishing == true) { - Entry& last_entry = entries_.front(); - std::string handle_encoding; - last_partition_block_handle.EncodeTo(&handle_encoding); - index_block_builder_.Add(last_entry.key, handle_encoding); - entries_.pop_front(); - } - // If there is no sub_index left, then return the 2nd level index. - if (UNLIKELY(entries_.empty())) { - index_blocks->index_block_contents = index_block_builder_.Finish(); - return Status::OK(); - } else { - // Finish the next partition index in line and Incomplete() to indicate we - // expect more calls to Finish - Entry& entry = entries_.front(); - auto s = entry.value->Finish(index_blocks); - finishing = true; - return s.ok() ? Status::Incomplete() : s; - } - } - - virtual size_t EstimatedSize() const override { - size_t total = 0; - for (auto it = entries_.begin(); it != entries_.end(); ++it) { - total += it->value->EstimatedSize(); - } - total += index_block_builder_.CurrentSizeEstimate(); - total += - sub_index_builder_ == nullptr ? 0 : sub_index_builder_->EstimatedSize(); - return total; - } - - private: - static const IndexType sub_type_ = BlockBasedTableOptions::kBinarySearch; - struct Entry { - std::string key; - std::unique_ptr value; - }; - std::list entries_; // list of partitioned indexes and their keys - const SliceTransform* prefix_extractor_; - BlockBuilder index_block_builder_; // top-level index builder - IndexBuilder* sub_index_builder_; // the active partition index builder - uint64_t index_per_partition_; - int index_block_restart_interval_; - uint64_t num_indexes = 0; - bool finishing = - false; // true if Finish is called once but not complete yet. -}; - -// HashIndexBuilder contains a binary-searchable primary index and the -// metadata for secondary hash index construction. -// The metadata for hash index consists two parts: -// - a metablock that compactly contains a sequence of prefixes. All prefixes -// are stored consectively without any metadata (like, prefix sizes) being -// stored, which is kept in the other metablock. -// - a metablock contains the metadata of the prefixes, including prefix size, -// restart index and number of block it spans. The format looks like: -// -// +-----------------+---------------------------+---------------------+ <=prefix 1 -// | length: 4 bytes | restart interval: 4 bytes | num-blocks: 4 bytes | -// +-----------------+---------------------------+---------------------+ <=prefix 2 -// | length: 4 bytes | restart interval: 4 bytes | num-blocks: 4 bytes | -// +-----------------+---------------------------+---------------------+ -// | | -// | .... | -// | | -// +-----------------+---------------------------+---------------------+ <=prefix n -// | length: 4 bytes | restart interval: 4 bytes | num-blocks: 4 bytes | -// +-----------------+---------------------------+---------------------+ -// -// The reason of separating these two metablocks is to enable the efficiently -// reuse the first metablock during hash index construction without unnecessary -// data copy or small heap allocations for prefixes. -class HashIndexBuilder : public IndexBuilder { - public: - explicit HashIndexBuilder(const InternalKeyComparator* comparator, - const SliceTransform* hash_key_extractor, - int index_block_restart_interval) - : IndexBuilder(comparator), - primary_index_builder_(comparator, index_block_restart_interval), - hash_key_extractor_(hash_key_extractor) {} - - virtual void AddIndexEntry(std::string* last_key_in_current_block, - const Slice* first_key_in_next_block, - const BlockHandle& block_handle) override { - ++current_restart_index_; - primary_index_builder_.AddIndexEntry(last_key_in_current_block, - first_key_in_next_block, block_handle); - } - - virtual void OnKeyAdded(const Slice& key) override { - auto key_prefix = hash_key_extractor_->Transform(key); - bool is_first_entry = pending_block_num_ == 0; - - // Keys may share the prefix - if (is_first_entry || pending_entry_prefix_ != key_prefix) { - if (!is_first_entry) { - FlushPendingPrefix(); - } - - // need a hard copy otherwise the underlying data changes all the time. - // TODO(kailiu) ToString() is expensive. We may speed up can avoid data - // copy. - pending_entry_prefix_ = key_prefix.ToString(); - pending_block_num_ = 1; - pending_entry_index_ = static_cast(current_restart_index_); - } else { - // entry number increments when keys share the prefix reside in - // different data blocks. - auto last_restart_index = pending_entry_index_ + pending_block_num_ - 1; - assert(last_restart_index <= current_restart_index_); - if (last_restart_index != current_restart_index_) { - ++pending_block_num_; - } - } - } - - virtual Status Finish( - IndexBlocks* index_blocks, - const BlockHandle& last_partition_block_handle) override { - FlushPendingPrefix(); - primary_index_builder_.Finish(index_blocks, last_partition_block_handle); - index_blocks->meta_blocks.insert( - {kHashIndexPrefixesBlock.c_str(), prefix_block_}); - index_blocks->meta_blocks.insert( - {kHashIndexPrefixesMetadataBlock.c_str(), prefix_meta_block_}); - return Status::OK(); - } - - virtual size_t EstimatedSize() const override { - return primary_index_builder_.EstimatedSize() + prefix_block_.size() + - prefix_meta_block_.size(); - } - - private: - void FlushPendingPrefix() { - prefix_block_.append(pending_entry_prefix_.data(), - pending_entry_prefix_.size()); - PutVarint32Varint32Varint32( - &prefix_meta_block_, - static_cast(pending_entry_prefix_.size()), - pending_entry_index_, pending_block_num_); - } - - ShortenedIndexBuilder primary_index_builder_; - const SliceTransform* hash_key_extractor_; - - // stores a sequence of prefixes - std::string prefix_block_; - // stores the metadata of prefixes - std::string prefix_meta_block_; - - // The following 3 variables keeps unflushed prefix and its metadata. - // The details of block_num and entry_index can be found in - // "block_hash_index.{h,cc}" - uint32_t pending_block_num_ = 0; - uint32_t pending_entry_index_ = 0; - std::string pending_entry_prefix_; - - uint64_t current_restart_index_ = 0; -}; // Without anonymous namespace here, we fail the warning -Wmissing-prototypes namespace { -// Create a index builder based on its type. -IndexBuilder* CreateIndexBuilder(IndexType index_type, - const InternalKeyComparator* comparator, - const SliceTransform* prefix_extractor, - int index_block_restart_interval, - uint64_t index_per_partition) { - switch (index_type) { - case BlockBasedTableOptions::kBinarySearch: { - return new ShortenedIndexBuilder(comparator, - index_block_restart_interval); - } - case BlockBasedTableOptions::kHashSearch: { - return new HashIndexBuilder(comparator, prefix_extractor, - index_block_restart_interval); - } - case BlockBasedTableOptions::kTwoLevelIndexSearch: { - return new PartitionIndexBuilder(comparator, prefix_extractor, - index_per_partition, - index_block_restart_interval); - } - default: { - assert(!"Do not recognize the index type "); - return nullptr; - } - } - // impossible. - assert(false); - return nullptr; -} - // Create a index builder based on its type. FilterBlockBuilder* CreateFilterBlockBuilder(const ImmutableCFOptions& opt, const BlockBasedTableOptions& table_opt) { @@ -649,11 +277,11 @@ struct BlockBasedTableBuilder::Rep { table_options.use_delta_encoding), range_del_block(1), // TODO(andrewkr): restart_interval unnecessary internal_prefix_transform(_ioptions.prefix_extractor), - index_builder( - CreateIndexBuilder(table_options.index_type, &internal_comparator, - &this->internal_prefix_transform, - table_options.index_block_restart_interval, - table_options.index_per_partition)), + index_builder(IndexBuilder::CreateIndexBuilder( + table_options.index_type, &internal_comparator, + &this->internal_prefix_transform, + table_options.index_block_restart_interval, + table_options.index_per_partition)), compression_type(_compression_type), compression_opts(_compression_opts), compression_dict(_compression_dict), diff --git a/table/block_based_table_reader.cc b/table/block_based_table_reader.cc index 2707f5300..6efba78d0 100644 --- a/table/block_based_table_reader.cc +++ b/table/block_based_table_reader.cc @@ -57,6 +57,11 @@ using std::unique_ptr; typedef BlockBasedTable::IndexReader IndexReader; +BlockBasedTable::~BlockBasedTable() { + Close(); + delete rep_; +} + namespace { // Read the block identified by "handle" from "file". // The only relevant option is options.verify_checksums for now. @@ -143,42 +148,6 @@ Cache::Handle* GetEntryFromCache(Cache* block_cache, const Slice& key, } // namespace -// -- IndexReader and its subclasses -// IndexReader is the interface that provide the functionality for index access. -class BlockBasedTable::IndexReader { - public: - explicit IndexReader(const Comparator* comparator, Statistics* stats) - : comparator_(comparator), statistics_(stats) {} - - virtual ~IndexReader() {} - - // Create an iterator for index access. - // If iter is null then a new object is created on heap and the callee will - // have the ownership. If a non-null iter is passed in it will be used, and - // the returned value is either the same as iter or a new on-heap object that - // wrapps the passed iter. In the latter case the return value would point to - // a different object then iter and the callee has the ownership of the - // returned object. - virtual InternalIterator* NewIterator(BlockIter* iter = nullptr, - bool total_order_seek = true) = 0; - - // The size of the index. - virtual size_t size() const = 0; - // Memory usage of the index block - virtual size_t usable_size() const = 0; - // return the statistics pointer - virtual Statistics* statistics() const { return statistics_; } - // Report an approximation of how much memory has been used other than memory - // that was allocated in block cache. - virtual size_t ApproximateMemoryUsage() const = 0; - - protected: - const Comparator* comparator_; - - private: - Statistics* statistics_; -}; - // Index that allows binary search lookup in a two-level index structure. class PartitionIndexReader : public IndexReader { public: @@ -397,118 +366,6 @@ class HashIndexReader : public IndexReader { BlockContents prefixes_contents_; }; -// CachableEntry represents the entries that *may* be fetched from block cache. -// field `value` is the item we want to get. -// field `cache_handle` is the cache handle to the block cache. If the value -// was not read from cache, `cache_handle` will be nullptr. -template -struct BlockBasedTable::CachableEntry { - CachableEntry(TValue* _value, Cache::Handle* _cache_handle) - : value(_value), cache_handle(_cache_handle) {} - CachableEntry() : CachableEntry(nullptr, nullptr) {} - void Release(Cache* cache) { - if (cache_handle) { - cache->Release(cache_handle); - value = nullptr; - cache_handle = nullptr; - } - } - bool IsSet() const { return cache_handle != nullptr; } - - TValue* value = nullptr; - // if the entry is from the cache, cache_handle will be populated. - Cache::Handle* cache_handle = nullptr; -}; - -struct BlockBasedTable::Rep { - Rep(const ImmutableCFOptions& _ioptions, const EnvOptions& _env_options, - const BlockBasedTableOptions& _table_opt, - const InternalKeyComparator& _internal_comparator, bool skip_filters) - : ioptions(_ioptions), - env_options(_env_options), - table_options(_table_opt), - filter_policy(skip_filters ? nullptr : _table_opt.filter_policy.get()), - internal_comparator(_internal_comparator), - filter_type(FilterType::kNoFilter), - whole_key_filtering(_table_opt.whole_key_filtering), - prefix_filtering(true), - range_del_handle(BlockHandle::NullBlockHandle()), - global_seqno(kDisableGlobalSequenceNumber) {} - - const ImmutableCFOptions& ioptions; - const EnvOptions& env_options; - const BlockBasedTableOptions& table_options; - const FilterPolicy* const filter_policy; - const InternalKeyComparator& internal_comparator; - Status status; - unique_ptr file; - char cache_key_prefix[kMaxCacheKeyPrefixSize]; - size_t cache_key_prefix_size = 0; - char persistent_cache_key_prefix[kMaxCacheKeyPrefixSize]; - size_t persistent_cache_key_prefix_size = 0; - char compressed_cache_key_prefix[kMaxCacheKeyPrefixSize]; - size_t compressed_cache_key_prefix_size = 0; - uint64_t dummy_index_reader_offset = - 0; // ID that is unique for the block cache. - PersistentCacheOptions persistent_cache_options; - - // Footer contains the fixed table information - Footer footer; - // index_reader and filter will be populated and used only when - // options.block_cache is nullptr; otherwise we will get the index block via - // the block cache. - unique_ptr index_reader; - unique_ptr filter; - - enum class FilterType { - kNoFilter, - kFullFilter, - kBlockFilter, - }; - FilterType filter_type; - BlockHandle filter_handle; - - std::shared_ptr table_properties; - // Block containing the data for the compression dictionary. We take ownership - // for the entire block struct, even though we only use its Slice member. This - // is easier because the Slice member depends on the continued existence of - // another member ("allocation"). - std::unique_ptr compression_dict_block; - BlockBasedTableOptions::IndexType index_type; - bool hash_index_allow_collision; - bool whole_key_filtering; - bool prefix_filtering; - // TODO(kailiu) It is very ugly to use internal key in table, since table - // module should not be relying on db module. However to make things easier - // and compatible with existing code, we introduce a wrapper that allows - // block to extract prefix without knowing if a key is internal or not. - unique_ptr internal_prefix_transform; - - // only used in level 0 files: - // when pin_l0_filter_and_index_blocks_in_cache is true, we do use the - // LRU cache, but we always keep the filter & idndex block's handle checked - // out here (=we don't call Release()), plus the parsed out objects - // the LRU cache will never push flush them out, hence they're pinned - CachableEntry filter_entry; - CachableEntry index_entry; - // range deletion meta-block is pinned through reader's lifetime when LRU - // cache is enabled. - CachableEntry range_del_entry; - BlockHandle range_del_handle; - - // If global_seqno is used, all Keys in this file will have the same - // seqno with value `global_seqno`. - // - // A value of kDisableGlobalSequenceNumber means that this feature is disabled - // and every key have it's own seqno. - SequenceNumber global_seqno; -}; - -BlockBasedTable::~BlockBasedTable() { - Close(); - delete rep_; -} - // Helper function to setup the cache key's prefix for the Table. void BlockBasedTable::SetupCacheKeyPrefix(Rep* rep, uint64_t file_size) { assert(kMaxCacheKeyPrefixSize >= 10); @@ -850,7 +707,7 @@ Status BlockBasedTable::Open(const ImmutableCFOptions& ioptions, // Set filter block if (rep->filter_policy) { - rep->filter.reset(ReadFilter(rep)); + rep->filter.reset(new_table->ReadFilter(rep)); } } else { delete index_reader; @@ -1087,7 +944,7 @@ Status BlockBasedTable::PutDataBlockToCache( return s; } -FilterBlockReader* BlockBasedTable::ReadFilter(Rep* rep) { +FilterBlockReader* BlockBasedTable::ReadFilter(Rep* rep) const { // TODO: We might want to unify with ReadBlockFromFile() if we start // requiring checksum verification in Table::Open. if (rep->filter_type == Rep::FilterType::kNoFilter) { diff --git a/table/block_based_table_reader.h b/table/block_based_table_reader.h index bcff8b3f7..ad3dbe8c4 100644 --- a/table/block_based_table_reader.h +++ b/table/block_based_table_reader.h @@ -20,6 +20,9 @@ #include "rocksdb/statistics.h" #include "rocksdb/status.h" #include "rocksdb/table.h" +#include "table/filter_block.h" +#include "table/format.h" +#include "table/persistent_cache_helper.h" #include "table/table_properties_internal.h" #include "table/table_reader.h" #include "table/two_level_iterator.h" @@ -139,8 +142,45 @@ class BlockBasedTable : public TableReader { bool TEST_filter_block_preloaded() const; bool TEST_index_reader_preloaded() const; - // Implementation of IndexReader will be exposed to internal cc file only. - class IndexReader; + + // IndexReader is the interface that provide the functionality for index + // access. + class IndexReader { + public: + explicit IndexReader(const Comparator* comparator, Statistics* stats) + : comparator_(comparator), statistics_(stats) {} + + virtual ~IndexReader() {} + + // Create an iterator for index access. + // If iter is null then a new object is created on heap and the callee will + // have the ownership. If a non-null iter is passed in it will be used, and + // the returned value is either the same as iter or a new on-heap object + // that + // wrapps the passed iter. In the latter case the return value would point + // to + // a different object then iter and the callee has the ownership of the + // returned object. + virtual InternalIterator* NewIterator(BlockIter* iter = nullptr, + bool total_order_seek = true) = 0; + + // The size of the index. + virtual size_t size() const = 0; + // Memory usage of the index block + virtual size_t usable_size() const = 0; + // return the statistics pointer + virtual Statistics* statistics() const { return statistics_; } + // Report an approximation of how much memory has been used other than + // memory + // that was allocated in block cache. + virtual size_t ApproximateMemoryUsage() const = 0; + + protected: + const Comparator* comparator_; + + private: + Statistics* statistics_; + }; static Slice GetCacheKey(const char* cache_key_prefix, size_t cache_key_prefix_size, @@ -155,7 +195,6 @@ class BlockBasedTable : public TableReader { private: template struct CachableEntry; - struct Rep; Rep* rep_; bool compaction_optimized_; @@ -251,7 +290,7 @@ class BlockBasedTable : public TableReader { std::unique_ptr* iter); // Create the filter from the filter block. - static FilterBlockReader* ReadFilter(Rep* rep); + FilterBlockReader* ReadFilter(Rep* rep) const; static void SetupCacheKeyPrefix(Rep* rep, uint64_t file_size); @@ -290,4 +329,112 @@ class BlockBasedTable::BlockEntryIteratorState : public TwoLevelIteratorState { bool skip_filters_; }; +// CachableEntry represents the entries that *may* be fetched from block cache. +// field `value` is the item we want to get. +// field `cache_handle` is the cache handle to the block cache. If the value +// was not read from cache, `cache_handle` will be nullptr. +template +struct BlockBasedTable::CachableEntry { + CachableEntry(TValue* _value, Cache::Handle* _cache_handle) + : value(_value), cache_handle(_cache_handle) {} + CachableEntry() : CachableEntry(nullptr, nullptr) {} + void Release(Cache* cache) { + if (cache_handle) { + cache->Release(cache_handle); + value = nullptr; + cache_handle = nullptr; + } + } + bool IsSet() const { return cache_handle != nullptr; } + + TValue* value = nullptr; + // if the entry is from the cache, cache_handle will be populated. + Cache::Handle* cache_handle = nullptr; +}; + +struct BlockBasedTable::Rep { + Rep(const ImmutableCFOptions& _ioptions, const EnvOptions& _env_options, + const BlockBasedTableOptions& _table_opt, + const InternalKeyComparator& _internal_comparator, bool skip_filters) + : ioptions(_ioptions), + env_options(_env_options), + table_options(_table_opt), + filter_policy(skip_filters ? nullptr : _table_opt.filter_policy.get()), + internal_comparator(_internal_comparator), + filter_type(FilterType::kNoFilter), + whole_key_filtering(_table_opt.whole_key_filtering), + prefix_filtering(true), + range_del_handle(BlockHandle::NullBlockHandle()), + global_seqno(kDisableGlobalSequenceNumber) {} + + const ImmutableCFOptions& ioptions; + const EnvOptions& env_options; + const BlockBasedTableOptions& table_options; + const FilterPolicy* const filter_policy; + const InternalKeyComparator& internal_comparator; + Status status; + unique_ptr file; + char cache_key_prefix[kMaxCacheKeyPrefixSize]; + size_t cache_key_prefix_size = 0; + char persistent_cache_key_prefix[kMaxCacheKeyPrefixSize]; + size_t persistent_cache_key_prefix_size = 0; + char compressed_cache_key_prefix[kMaxCacheKeyPrefixSize]; + size_t compressed_cache_key_prefix_size = 0; + uint64_t dummy_index_reader_offset = + 0; // ID that is unique for the block cache. + PersistentCacheOptions persistent_cache_options; + + // Footer contains the fixed table information + Footer footer; + // index_reader and filter will be populated and used only when + // options.block_cache is nullptr; otherwise we will get the index block via + // the block cache. + unique_ptr index_reader; + unique_ptr filter; + + enum class FilterType { + kNoFilter, + kFullFilter, + kBlockFilter, + kPartitionedFilter, + }; + FilterType filter_type; + BlockHandle filter_handle; + + std::shared_ptr table_properties; + // Block containing the data for the compression dictionary. We take ownership + // for the entire block struct, even though we only use its Slice member. This + // is easier because the Slice member depends on the continued existence of + // another member ("allocation"). + std::unique_ptr compression_dict_block; + BlockBasedTableOptions::IndexType index_type; + bool hash_index_allow_collision; + bool whole_key_filtering; + bool prefix_filtering; + // TODO(kailiu) It is very ugly to use internal key in table, since table + // module should not be relying on db module. However to make things easier + // and compatible with existing code, we introduce a wrapper that allows + // block to extract prefix without knowing if a key is internal or not. + unique_ptr internal_prefix_transform; + + // only used in level 0 files: + // when pin_l0_filter_and_index_blocks_in_cache is true, we do use the + // LRU cache, but we always keep the filter & idndex block's handle checked + // out here (=we don't call Release()), plus the parsed out objects + // the LRU cache will never push flush them out, hence they're pinned + CachableEntry filter_entry; + CachableEntry index_entry; + // range deletion meta-block is pinned through reader's lifetime when LRU + // cache is enabled. + CachableEntry range_del_entry; + BlockHandle range_del_handle; + + // If global_seqno is used, all Keys in this file will have the same + // seqno with value `global_seqno`. + // + // A value of kDisableGlobalSequenceNumber means that this feature is disabled + // and every key have it's own seqno. + SequenceNumber global_seqno; +}; + } // namespace rocksdb diff --git a/table/format.h b/table/format.h index 0d7cec5e7..f8c8c027e 100644 --- a/table/format.h +++ b/table/format.h @@ -15,8 +15,8 @@ #include "rocksdb/options.h" #include "rocksdb/table.h" -#include "port/port.h" // noexcept -#include "table/persistent_cache_helper.h" +#include "port/port.h" // noexcept +#include "table/persistent_cache_options.h" #include "util/cf_options.h" namespace rocksdb { diff --git a/table/index_builder.cc b/table/index_builder.cc new file mode 100644 index 000000000..4e4ea76f3 --- /dev/null +++ b/table/index_builder.cc @@ -0,0 +1,52 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under the BSD-style license found in the +// LICENSE file in the root directory of this source tree. An additional grant +// of patent rights can be found in the PATENTS file in the same directory. +// +// Copyright (c) 2011 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors. + +#include "table/index_builder.h" +#include +#include +#include +#include + +#include "rocksdb/comparator.h" +#include "table/format.h" +#include "table/partitioned_filter_block.h" + +// Without anonymous namespace here, we fail the warning -Wmissing-prototypes +namespace rocksdb { +// using namespace rocksdb; +// Create a index builder based on its type. +IndexBuilder* IndexBuilder::CreateIndexBuilder( + BlockBasedTableOptions::IndexType index_type, + const InternalKeyComparator* comparator, + const SliceTransform* prefix_extractor, int index_block_restart_interval, + uint64_t index_per_partition) { + switch (index_type) { + case BlockBasedTableOptions::kBinarySearch: { + return new ShortenedIndexBuilder(comparator, + index_block_restart_interval); + } + case BlockBasedTableOptions::kHashSearch: { + return new HashIndexBuilder(comparator, prefix_extractor, + index_block_restart_interval); + } + case BlockBasedTableOptions::kTwoLevelIndexSearch: { + return new PartitionIndexBuilder(comparator, prefix_extractor, + index_per_partition, + index_block_restart_interval); + } + default: { + assert(!"Do not recognize the index type "); + return nullptr; + } + } + // impossible. + assert(false); + return nullptr; +} +} // namespace rocksdb diff --git a/table/index_builder.h b/table/index_builder.h new file mode 100644 index 000000000..9cc629e37 --- /dev/null +++ b/table/index_builder.h @@ -0,0 +1,265 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under the BSD-style license found in the +// LICENSE file in the root directory of this source tree. An additional grant +// of patent rights can be found in the PATENTS file in the same directory. +// +// Copyright (c) 2011 The LevelDB Authors. All rights reserved. +// Use of this source code is governed by a BSD-style license that can be +// found in the LICENSE file. See the AUTHORS file for names of contributors. + +#pragma once + +#include +#include +#include +#include + +#include "rocksdb/comparator.h" +#include "table/block_based_table_factory.h" +#include "table/block_builder.h" +#include "table/format.h" + +namespace rocksdb { +// The interface for building index. +// Instruction for adding a new concrete IndexBuilder: +// 1. Create a subclass instantiated from IndexBuilder. +// 2. Add a new entry associated with that subclass in TableOptions::IndexType. +// 3. Add a create function for the new subclass in CreateIndexBuilder. +// Note: we can devise more advanced design to simplify the process for adding +// new subclass, which will, on the other hand, increase the code complexity and +// catch unwanted attention from readers. Given that we won't add/change +// indexes frequently, it makes sense to just embrace a more straightforward +// design that just works. +class IndexBuilder { + public: + static IndexBuilder* CreateIndexBuilder( + BlockBasedTableOptions::IndexType index_type, + const InternalKeyComparator* comparator, + const SliceTransform* prefix_extractor, int index_block_restart_interval, + uint64_t index_per_partition); + + // Index builder will construct a set of blocks which contain: + // 1. One primary index block. + // 2. (Optional) a set of metablocks that contains the metadata of the + // primary index. + struct IndexBlocks { + Slice index_block_contents; + std::unordered_map meta_blocks; + }; + explicit IndexBuilder(const InternalKeyComparator* comparator) + : comparator_(comparator) {} + + virtual ~IndexBuilder() {} + + // Add a new index entry to index block. + // To allow further optimization, we provide `last_key_in_current_block` and + // `first_key_in_next_block`, based on which the specific implementation can + // determine the best index key to be used for the index block. + // @last_key_in_current_block: this parameter maybe overridden with the value + // "substitute key". + // @first_key_in_next_block: it will be nullptr if the entry being added is + // the last one in the table + // + // REQUIRES: Finish() has not yet been called. + virtual void AddIndexEntry(std::string* last_key_in_current_block, + const Slice* first_key_in_next_block, + const BlockHandle& block_handle) = 0; + + // This method will be called whenever a key is added. The subclasses may + // override OnKeyAdded() if they need to collect additional information. + virtual void OnKeyAdded(const Slice& key) {} + + // Inform the index builder that all entries has been written. Block builder + // may therefore perform any operation required for block finalization. + // + // REQUIRES: Finish() has not yet been called. + inline Status Finish(IndexBlocks* index_blocks) { + // Throw away the changes to last_partition_block_handle. It has no effect + // on the first call to Finish anyway. + BlockHandle last_partition_block_handle; + return Finish(index_blocks, last_partition_block_handle); + } + + // This override of Finish can be utilized to build the 2nd level index in + // PartitionIndexBuilder. + // + // index_blocks will be filled with the resulting index data. If the return + // value is Status::InComplete() then it means that the index is partitioned + // and the callee should keep calling Finish until Status::OK() is returned. + // In that case, last_partition_block_handle is pointer to the block written + // with the result of the last call to Finish. This can be utilized to build + // the second level index pointing to each block of partitioned indexes. The + // last call to Finish() that returns Status::OK() populates index_blocks with + // the 2nd level index content. + virtual Status Finish(IndexBlocks* index_blocks, + const BlockHandle& last_partition_block_handle) = 0; + + // Get the estimated size for index block. + virtual size_t EstimatedSize() const = 0; + + protected: + const InternalKeyComparator* comparator_; +}; + +// This index builder builds space-efficient index block. +// +// Optimizations: +// 1. Made block's `block_restart_interval` to be 1, which will avoid linear +// search when doing index lookup (can be disabled by setting +// index_block_restart_interval). +// 2. Shorten the key length for index block. Other than honestly using the +// last key in the data block as the index key, we instead find a shortest +// substitute key that serves the same function. +class ShortenedIndexBuilder : public IndexBuilder { + public: + explicit ShortenedIndexBuilder(const InternalKeyComparator* comparator, + int index_block_restart_interval) + : IndexBuilder(comparator), + index_block_builder_(index_block_restart_interval) {} + + virtual void AddIndexEntry(std::string* last_key_in_current_block, + const Slice* first_key_in_next_block, + const BlockHandle& block_handle) override { + if (first_key_in_next_block != nullptr) { + comparator_->FindShortestSeparator(last_key_in_current_block, + *first_key_in_next_block); + } else { + comparator_->FindShortSuccessor(last_key_in_current_block); + } + + std::string handle_encoding; + block_handle.EncodeTo(&handle_encoding); + index_block_builder_.Add(*last_key_in_current_block, handle_encoding); + } + + virtual Status Finish( + IndexBlocks* index_blocks, + const BlockHandle& last_partition_block_handle) override { + index_blocks->index_block_contents = index_block_builder_.Finish(); + return Status::OK(); + } + + virtual size_t EstimatedSize() const override { + return index_block_builder_.CurrentSizeEstimate(); + } + + private: + BlockBuilder index_block_builder_; +}; + +// HashIndexBuilder contains a binary-searchable primary index and the +// metadata for secondary hash index construction. +// The metadata for hash index consists two parts: +// - a metablock that compactly contains a sequence of prefixes. All prefixes +// are stored consectively without any metadata (like, prefix sizes) being +// stored, which is kept in the other metablock. +// - a metablock contains the metadata of the prefixes, including prefix size, +// restart index and number of block it spans. The format looks like: +// +// +-----------------+---------------------------+---------------------+ +// <=prefix 1 +// | length: 4 bytes | restart interval: 4 bytes | num-blocks: 4 bytes | +// +-----------------+---------------------------+---------------------+ +// <=prefix 2 +// | length: 4 bytes | restart interval: 4 bytes | num-blocks: 4 bytes | +// +-----------------+---------------------------+---------------------+ +// | | +// | .... | +// | | +// +-----------------+---------------------------+---------------------+ +// <=prefix n +// | length: 4 bytes | restart interval: 4 bytes | num-blocks: 4 bytes | +// +-----------------+---------------------------+---------------------+ +// +// The reason of separating these two metablocks is to enable the efficiently +// reuse the first metablock during hash index construction without unnecessary +// data copy or small heap allocations for prefixes. +class HashIndexBuilder : public IndexBuilder { + public: + explicit HashIndexBuilder(const InternalKeyComparator* comparator, + const SliceTransform* hash_key_extractor, + int index_block_restart_interval) + : IndexBuilder(comparator), + primary_index_builder_(comparator, index_block_restart_interval), + hash_key_extractor_(hash_key_extractor) {} + + virtual void AddIndexEntry(std::string* last_key_in_current_block, + const Slice* first_key_in_next_block, + const BlockHandle& block_handle) override { + ++current_restart_index_; + primary_index_builder_.AddIndexEntry(last_key_in_current_block, + first_key_in_next_block, block_handle); + } + + virtual void OnKeyAdded(const Slice& key) override { + auto key_prefix = hash_key_extractor_->Transform(key); + bool is_first_entry = pending_block_num_ == 0; + + // Keys may share the prefix + if (is_first_entry || pending_entry_prefix_ != key_prefix) { + if (!is_first_entry) { + FlushPendingPrefix(); + } + + // need a hard copy otherwise the underlying data changes all the time. + // TODO(kailiu) ToString() is expensive. We may speed up can avoid data + // copy. + pending_entry_prefix_ = key_prefix.ToString(); + pending_block_num_ = 1; + pending_entry_index_ = static_cast(current_restart_index_); + } else { + // entry number increments when keys share the prefix reside in + // different data blocks. + auto last_restart_index = pending_entry_index_ + pending_block_num_ - 1; + assert(last_restart_index <= current_restart_index_); + if (last_restart_index != current_restart_index_) { + ++pending_block_num_; + } + } + } + + virtual Status Finish( + IndexBlocks* index_blocks, + const BlockHandle& last_partition_block_handle) override { + FlushPendingPrefix(); + primary_index_builder_.Finish(index_blocks, last_partition_block_handle); + index_blocks->meta_blocks.insert( + {kHashIndexPrefixesBlock.c_str(), prefix_block_}); + index_blocks->meta_blocks.insert( + {kHashIndexPrefixesMetadataBlock.c_str(), prefix_meta_block_}); + return Status::OK(); + } + + virtual size_t EstimatedSize() const override { + return primary_index_builder_.EstimatedSize() + prefix_block_.size() + + prefix_meta_block_.size(); + } + + private: + void FlushPendingPrefix() { + prefix_block_.append(pending_entry_prefix_.data(), + pending_entry_prefix_.size()); + PutVarint32Varint32Varint32( + &prefix_meta_block_, + static_cast(pending_entry_prefix_.size()), + pending_entry_index_, pending_block_num_); + } + + ShortenedIndexBuilder primary_index_builder_; + const SliceTransform* hash_key_extractor_; + + // stores a sequence of prefixes + std::string prefix_block_; + // stores the metadata of prefixes + std::string prefix_meta_block_; + + // The following 3 variables keeps unflushed prefix and its metadata. + // The details of block_num and entry_index can be found in + // "block_hash_index.{h,cc}" + uint32_t pending_block_num_ = 0; + uint32_t pending_entry_index_ = 0; + std::string pending_entry_prefix_; + + uint64_t current_restart_index_ = 0; +}; +} // namespace rocksdb diff --git a/table/partitioned_filter_block.cc b/table/partitioned_filter_block.cc new file mode 100644 index 000000000..28e9cc9aa --- /dev/null +++ b/table/partitioned_filter_block.cc @@ -0,0 +1,84 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under the BSD-style license found in the +// LICENSE file in the root directory of this source tree. An additional grant +// of patent rights can be found in the PATENTS file in the same directory. + +#include "table/partitioned_filter_block.h" + +#include "port/port.h" +#include "util/coding.h" + +namespace rocksdb { +PartitionIndexBuilder::PartitionIndexBuilder( + const InternalKeyComparator* comparator, + const SliceTransform* prefix_extractor, const uint64_t index_per_partition, + int index_block_restart_interval) + : IndexBuilder(comparator), + prefix_extractor_(prefix_extractor), + index_block_builder_(index_block_restart_interval), + index_per_partition_(index_per_partition), + index_block_restart_interval_(index_block_restart_interval) { + sub_index_builder_ = + CreateIndexBuilder(sub_type_, comparator_, prefix_extractor_, + index_block_restart_interval_, index_per_partition_); +} + +PartitionIndexBuilder::~PartitionIndexBuilder() { delete sub_index_builder_; } + +void PartitionIndexBuilder::AddIndexEntry( + std::string* last_key_in_current_block, + const Slice* first_key_in_next_block, const BlockHandle& block_handle) { + sub_index_builder_->AddIndexEntry(last_key_in_current_block, + first_key_in_next_block, block_handle); + num_indexes++; + if (UNLIKELY(first_key_in_next_block == nullptr)) { // no more keys + entries_.push_back({std::string(*last_key_in_current_block), + std::unique_ptr(sub_index_builder_)}); + sub_index_builder_ = nullptr; + } else if (num_indexes % index_per_partition_ == 0) { + entries_.push_back({std::string(*last_key_in_current_block), + std::unique_ptr(sub_index_builder_)}); + sub_index_builder_ = + CreateIndexBuilder(sub_type_, comparator_, prefix_extractor_, + index_block_restart_interval_, index_per_partition_); + } +} + +Status PartitionIndexBuilder::Finish( + IndexBlocks* index_blocks, const BlockHandle& last_partition_block_handle) { + assert(!entries_.empty()); + // It must be set to null after last key is added + assert(sub_index_builder_ == nullptr); + if (finishing == true) { + Entry& last_entry = entries_.front(); + std::string handle_encoding; + last_partition_block_handle.EncodeTo(&handle_encoding); + index_block_builder_.Add(last_entry.key, handle_encoding); + entries_.pop_front(); + } + // If there is no sub_index left, then return the 2nd level index. + if (UNLIKELY(entries_.empty())) { + index_blocks->index_block_contents = index_block_builder_.Finish(); + return Status::OK(); + } else { + // Finish the next partition index in line and Incomplete() to indicate we + // expect more calls to Finish + Entry& entry = entries_.front(); + auto s = entry.value->Finish(index_blocks); + finishing = true; + return s.ok() ? Status::Incomplete() : s; + } +} + +size_t PartitionIndexBuilder::EstimatedSize() const { + size_t total = 0; + for (auto it = entries_.begin(); it != entries_.end(); ++it) { + total += it->value->EstimatedSize(); + } + total += index_block_builder_.CurrentSizeEstimate(); + total += + sub_index_builder_ == nullptr ? 0 : sub_index_builder_->EstimatedSize(); + return total; +} + +} // namespace rocksdb diff --git a/table/partitioned_filter_block.h b/table/partitioned_filter_block.h new file mode 100644 index 000000000..57f5d8b90 --- /dev/null +++ b/table/partitioned_filter_block.h @@ -0,0 +1,65 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under the BSD-style license found in the +// LICENSE file in the root directory of this source tree. An additional grant +// of patent rights can be found in the PATENTS file in the same directory. + +#pragma once + +#include +#include +#include "rocksdb/options.h" +#include "rocksdb/slice.h" +#include "rocksdb/slice_transform.h" +#include "util/hash.h" + +#include "table/index_builder.h" + +namespace rocksdb { + +/** + * IndexBuilder for two-level indexing. Internally it creates a new index for + * each partition and Finish then in order when Finish is called on it + * continiously until Status::OK() is returned. + * + * The format on the disk would be I I I I I I IP where I is block containing a + * partition of indexes built using ShortenedIndexBuilder and IP is a block + * containing a secondary index on the partitions, built using + * ShortenedIndexBuilder. + */ +class PartitionIndexBuilder : public IndexBuilder { + public: + explicit PartitionIndexBuilder(const InternalKeyComparator* comparator, + const SliceTransform* prefix_extractor, + const uint64_t index_per_partition, + int index_block_restart_interval); + + virtual ~PartitionIndexBuilder(); + + virtual void AddIndexEntry(std::string* last_key_in_current_block, + const Slice* first_key_in_next_block, + const BlockHandle& block_handle); + + virtual Status Finish(IndexBlocks* index_blocks, + const BlockHandle& last_partition_block_handle); + + virtual size_t EstimatedSize() const; + + private: + static const BlockBasedTableOptions::IndexType sub_type_ = + BlockBasedTableOptions::kBinarySearch; + struct Entry { + std::string key; + std::unique_ptr value; + }; + std::list entries_; // list of partitioned indexes and their keys + const SliceTransform* prefix_extractor_; + BlockBuilder index_block_builder_; // top-level index builder + IndexBuilder* sub_index_builder_; // the active partition index builder + uint64_t index_per_partition_; + int index_block_restart_interval_; + uint64_t num_indexes = 0; + bool finishing = + false; // true if Finish is called once but not complete yet. +}; + +} // namespace rocksdb diff --git a/table/persistent_cache_helper.cc b/table/persistent_cache_helper.cc index d68c80735..c3a5b132d 100644 --- a/table/persistent_cache_helper.cc +++ b/table/persistent_cache_helper.cc @@ -2,7 +2,9 @@ // This source code is licensed under the BSD-style license found in the // LICENSE file in the root directory of this source tree. An additional grant // of patent rights can be found in the PATENTS file in the same directory. + #include "table/persistent_cache_helper.h" +#include "table/block_based_table_reader.h" #include "table/format.h" namespace rocksdb { diff --git a/table/persistent_cache_helper.h b/table/persistent_cache_helper.h index 45a1f87d2..7733b3fac 100644 --- a/table/persistent_cache_helper.h +++ b/table/persistent_cache_helper.h @@ -6,33 +6,14 @@ #include -#include "table/block_based_table_reader.h" +#include "table/format.h" +#include "table/persistent_cache_options.h" #include "util/statistics.h" namespace rocksdb { struct BlockContents; -// PersistentCacheOptions -// -// This describe the caching behavior for page cache -// This is used to pass the context for caching and the cache handle -struct PersistentCacheOptions { - PersistentCacheOptions() {} - explicit PersistentCacheOptions( - const std::shared_ptr& _persistent_cache, - const std::string _key_prefix, Statistics* const _statistics) - : persistent_cache(_persistent_cache), - key_prefix(_key_prefix), - statistics(_statistics) {} - - virtual ~PersistentCacheOptions() {} - - std::shared_ptr persistent_cache; - std::string key_prefix; - Statistics* statistics = nullptr; -}; - // PersistentCacheHelper // // Encapsulates some of the helper logic for read and writing from the cache diff --git a/table/persistent_cache_options.h b/table/persistent_cache_options.h new file mode 100644 index 000000000..c0873921c --- /dev/null +++ b/table/persistent_cache_options.h @@ -0,0 +1,34 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under the BSD-style license found in the +// LICENSE file in the root directory of this source tree. An additional grant +// of patent rights can be found in the PATENTS file in the same directory. +#pragma once + +#include + +#include "include/rocksdb/persistent_cache.h" +#include "util/statistics.h" + +namespace rocksdb { + +// PersistentCacheOptions +// +// This describe the caching behavior for page cache +// This is used to pass the context for caching and the cache handle +struct PersistentCacheOptions { + PersistentCacheOptions() {} + explicit PersistentCacheOptions( + const std::shared_ptr& _persistent_cache, + const std::string _key_prefix, Statistics* const _statistics) + : persistent_cache(_persistent_cache), + key_prefix(_key_prefix), + statistics(_statistics) {} + + virtual ~PersistentCacheOptions() {} + + std::shared_ptr persistent_cache; + std::string key_prefix; + Statistics* statistics = nullptr; +}; + +} // namespace rocksdb