diff options
Diffstat (limited to 'src/rocksdb/table/block_based')
60 files changed, 21409 insertions, 0 deletions
diff --git a/src/rocksdb/table/block_based/binary_search_index_reader.cc b/src/rocksdb/table/block_based/binary_search_index_reader.cc new file mode 100644 index 000000000..21787cc1a --- /dev/null +++ b/src/rocksdb/table/block_based/binary_search_index_reader.cc @@ -0,0 +1,74 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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/block_based/binary_search_index_reader.h" + +namespace ROCKSDB_NAMESPACE { +Status BinarySearchIndexReader::Create( + const BlockBasedTable* table, const ReadOptions& ro, + FilePrefetchBuffer* prefetch_buffer, bool use_cache, bool prefetch, + bool pin, BlockCacheLookupContext* lookup_context, + std::unique_ptr<IndexReader>* index_reader) { + assert(table != nullptr); + assert(table->get_rep()); + assert(!pin || prefetch); + assert(index_reader != nullptr); + + CachableEntry<Block> index_block; + if (prefetch || !use_cache) { + const Status s = + ReadIndexBlock(table, prefetch_buffer, ro, use_cache, + /*get_context=*/nullptr, lookup_context, &index_block); + if (!s.ok()) { + return s; + } + + if (use_cache && !pin) { + index_block.Reset(); + } + } + + index_reader->reset( + new BinarySearchIndexReader(table, std::move(index_block))); + + return Status::OK(); +} + +InternalIteratorBase<IndexValue>* BinarySearchIndexReader::NewIterator( + const ReadOptions& read_options, bool /* disable_prefix_seek */, + IndexBlockIter* iter, GetContext* get_context, + BlockCacheLookupContext* lookup_context) { + const BlockBasedTable::Rep* rep = table()->get_rep(); + const bool no_io = (read_options.read_tier == kBlockCacheTier); + CachableEntry<Block> index_block; + const Status s = + GetOrReadIndexBlock(no_io, read_options.rate_limiter_priority, + get_context, lookup_context, &index_block); + if (!s.ok()) { + if (iter != nullptr) { + iter->Invalidate(s); + return iter; + } + + return NewErrorInternalIterator<IndexValue>(s); + } + + Statistics* kNullStats = nullptr; + // We don't return pinned data from index blocks, so no need + // to set `block_contents_pinned`. + auto it = index_block.GetValue()->NewIndexIterator( + internal_comparator()->user_comparator(), + rep->get_global_seqno(BlockType::kIndex), iter, kNullStats, true, + index_has_first_key(), index_key_includes_seq(), index_value_is_full()); + + assert(it != nullptr); + index_block.TransferTo(it); + + return it; +} +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/binary_search_index_reader.h b/src/rocksdb/table/block_based/binary_search_index_reader.h new file mode 100644 index 000000000..d4a611ecc --- /dev/null +++ b/src/rocksdb/table/block_based/binary_search_index_reader.h @@ -0,0 +1,48 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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 "table/block_based/index_reader_common.h" + +namespace ROCKSDB_NAMESPACE { +// Index that allows binary search lookup for the first key of each block. +// This class can be viewed as a thin wrapper for `Block` class which already +// supports binary search. +class BinarySearchIndexReader : public BlockBasedTable::IndexReaderCommon { + public: + // Read index from the file and create an intance for + // `BinarySearchIndexReader`. + // On success, index_reader will be populated; otherwise it will remain + // unmodified. + static Status Create(const BlockBasedTable* table, const ReadOptions& ro, + FilePrefetchBuffer* prefetch_buffer, bool use_cache, + bool prefetch, bool pin, + BlockCacheLookupContext* lookup_context, + std::unique_ptr<IndexReader>* index_reader); + + InternalIteratorBase<IndexValue>* NewIterator( + const ReadOptions& read_options, bool /* disable_prefix_seek */, + IndexBlockIter* iter, GetContext* get_context, + BlockCacheLookupContext* lookup_context) override; + + size_t ApproximateMemoryUsage() const override { + size_t usage = ApproximateIndexBlockMemoryUsage(); +#ifdef ROCKSDB_MALLOC_USABLE_SIZE + usage += malloc_usable_size(const_cast<BinarySearchIndexReader*>(this)); +#else + usage += sizeof(*this); +#endif // ROCKSDB_MALLOC_USABLE_SIZE + return usage; + } + + private: + BinarySearchIndexReader(const BlockBasedTable* t, + CachableEntry<Block>&& index_block) + : IndexReaderCommon(t, std::move(index_block)) {} +}; +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/block.cc b/src/rocksdb/table/block_based/block.cc new file mode 100644 index 000000000..7eb0b010f --- /dev/null +++ b/src/rocksdb/table/block_based/block.cc @@ -0,0 +1,1131 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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. +// +// Decodes the blocks generated by block_builder.cc. + +#include "table/block_based/block.h" + +#include <algorithm> +#include <string> +#include <unordered_map> +#include <vector> + +#include "monitoring/perf_context_imp.h" +#include "port/port.h" +#include "port/stack_trace.h" +#include "rocksdb/comparator.h" +#include "table/block_based/block_prefix_index.h" +#include "table/block_based/data_block_footer.h" +#include "table/format.h" +#include "util/coding.h" + +namespace ROCKSDB_NAMESPACE { + +// Helper routine: decode the next block entry starting at "p", +// storing the number of shared key bytes, non_shared key bytes, +// and the length of the value in "*shared", "*non_shared", and +// "*value_length", respectively. Will not derefence past "limit". +// +// If any errors are detected, returns nullptr. Otherwise, returns a +// pointer to the key delta (just past the three decoded values). +struct DecodeEntry { + inline const char* operator()(const char* p, const char* limit, + uint32_t* shared, uint32_t* non_shared, + uint32_t* value_length) { + // We need 2 bytes for shared and non_shared size. We also need one more + // byte either for value size or the actual value in case of value delta + // encoding. + assert(limit - p >= 3); + *shared = reinterpret_cast<const unsigned char*>(p)[0]; + *non_shared = reinterpret_cast<const unsigned char*>(p)[1]; + *value_length = reinterpret_cast<const unsigned char*>(p)[2]; + if ((*shared | *non_shared | *value_length) < 128) { + // Fast path: all three values are encoded in one byte each + p += 3; + } else { + if ((p = GetVarint32Ptr(p, limit, shared)) == nullptr) return nullptr; + if ((p = GetVarint32Ptr(p, limit, non_shared)) == nullptr) return nullptr; + if ((p = GetVarint32Ptr(p, limit, value_length)) == nullptr) { + return nullptr; + } + } + + // Using an assert in place of "return null" since we should not pay the + // cost of checking for corruption on every single key decoding + assert(!(static_cast<uint32_t>(limit - p) < (*non_shared + *value_length))); + return p; + } +}; + +// Helper routine: similar to DecodeEntry but does not have assertions. +// Instead, returns nullptr so that caller can detect and report failure. +struct CheckAndDecodeEntry { + inline const char* operator()(const char* p, const char* limit, + uint32_t* shared, uint32_t* non_shared, + uint32_t* value_length) { + // We need 2 bytes for shared and non_shared size. We also need one more + // byte either for value size or the actual value in case of value delta + // encoding. + if (limit - p < 3) { + return nullptr; + } + *shared = reinterpret_cast<const unsigned char*>(p)[0]; + *non_shared = reinterpret_cast<const unsigned char*>(p)[1]; + *value_length = reinterpret_cast<const unsigned char*>(p)[2]; + if ((*shared | *non_shared | *value_length) < 128) { + // Fast path: all three values are encoded in one byte each + p += 3; + } else { + if ((p = GetVarint32Ptr(p, limit, shared)) == nullptr) return nullptr; + if ((p = GetVarint32Ptr(p, limit, non_shared)) == nullptr) return nullptr; + if ((p = GetVarint32Ptr(p, limit, value_length)) == nullptr) { + return nullptr; + } + } + + if (static_cast<uint32_t>(limit - p) < (*non_shared + *value_length)) { + return nullptr; + } + return p; + } +}; + +struct DecodeKey { + inline const char* operator()(const char* p, const char* limit, + uint32_t* shared, uint32_t* non_shared) { + uint32_t value_length; + return DecodeEntry()(p, limit, shared, non_shared, &value_length); + } +}; + +// In format_version 4, which is used by index blocks, the value size is not +// encoded before the entry, as the value is known to be the handle with the +// known size. +struct DecodeKeyV4 { + inline const char* operator()(const char* p, const char* limit, + uint32_t* shared, uint32_t* non_shared) { + // We need 2 bytes for shared and non_shared size. We also need one more + // byte either for value size or the actual value in case of value delta + // encoding. + if (limit - p < 3) return nullptr; + *shared = reinterpret_cast<const unsigned char*>(p)[0]; + *non_shared = reinterpret_cast<const unsigned char*>(p)[1]; + if ((*shared | *non_shared) < 128) { + // Fast path: all three values are encoded in one byte each + p += 2; + } else { + if ((p = GetVarint32Ptr(p, limit, shared)) == nullptr) return nullptr; + if ((p = GetVarint32Ptr(p, limit, non_shared)) == nullptr) return nullptr; + } + return p; + } +}; + +struct DecodeEntryV4 { + inline const char* operator()(const char* p, const char* limit, + uint32_t* shared, uint32_t* non_shared, + uint32_t* value_length) { + assert(value_length); + + *value_length = 0; + return DecodeKeyV4()(p, limit, shared, non_shared); + } +}; +void DataBlockIter::NextImpl() { + bool is_shared = false; + ParseNextDataKey(&is_shared); +} + +void MetaBlockIter::NextImpl() { + bool is_shared = false; + ParseNextKey<CheckAndDecodeEntry>(&is_shared); +} + +void IndexBlockIter::NextImpl() { ParseNextIndexKey(); } + +void IndexBlockIter::PrevImpl() { + assert(Valid()); + // Scan backwards to a restart point before current_ + const uint32_t original = current_; + while (GetRestartPoint(restart_index_) >= original) { + if (restart_index_ == 0) { + // No more entries + current_ = restarts_; + restart_index_ = num_restarts_; + return; + } + restart_index_--; + } + SeekToRestartPoint(restart_index_); + // Loop until end of current entry hits the start of original entry + while (ParseNextIndexKey() && NextEntryOffset() < original) { + } +} + +void MetaBlockIter::PrevImpl() { + assert(Valid()); + // Scan backwards to a restart point before current_ + const uint32_t original = current_; + while (GetRestartPoint(restart_index_) >= original) { + if (restart_index_ == 0) { + // No more entries + current_ = restarts_; + restart_index_ = num_restarts_; + return; + } + restart_index_--; + } + SeekToRestartPoint(restart_index_); + bool is_shared = false; + // Loop until end of current entry hits the start of original entry + while (ParseNextKey<CheckAndDecodeEntry>(&is_shared) && + NextEntryOffset() < original) { + } +} + +// Similar to IndexBlockIter::PrevImpl but also caches the prev entries +void DataBlockIter::PrevImpl() { + assert(Valid()); + + assert(prev_entries_idx_ == -1 || + static_cast<size_t>(prev_entries_idx_) < prev_entries_.size()); + // Check if we can use cached prev_entries_ + if (prev_entries_idx_ > 0 && + prev_entries_[prev_entries_idx_].offset == current_) { + // Read cached CachedPrevEntry + prev_entries_idx_--; + const CachedPrevEntry& current_prev_entry = + prev_entries_[prev_entries_idx_]; + + const char* key_ptr = nullptr; + bool raw_key_cached; + if (current_prev_entry.key_ptr != nullptr) { + // The key is not delta encoded and stored in the data block + key_ptr = current_prev_entry.key_ptr; + raw_key_cached = false; + } else { + // The key is delta encoded and stored in prev_entries_keys_buff_ + key_ptr = prev_entries_keys_buff_.data() + current_prev_entry.key_offset; + raw_key_cached = true; + } + const Slice current_key(key_ptr, current_prev_entry.key_size); + + current_ = current_prev_entry.offset; + // TODO(ajkr): the copy when `raw_key_cached` is done here for convenience, + // not necessity. It is convenient since this class treats keys as pinned + // when `raw_key_` points to an outside buffer. So we cannot allow + // `raw_key_` point into Prev cache as it is a transient outside buffer + // (i.e., keys in it are not actually pinned). + raw_key_.SetKey(current_key, raw_key_cached /* copy */); + value_ = current_prev_entry.value; + + return; + } + + // Clear prev entries cache + prev_entries_idx_ = -1; + prev_entries_.clear(); + prev_entries_keys_buff_.clear(); + + // Scan backwards to a restart point before current_ + const uint32_t original = current_; + while (GetRestartPoint(restart_index_) >= original) { + if (restart_index_ == 0) { + // No more entries + current_ = restarts_; + restart_index_ = num_restarts_; + return; + } + restart_index_--; + } + + SeekToRestartPoint(restart_index_); + + do { + bool is_shared = false; + if (!ParseNextDataKey(&is_shared)) { + break; + } + Slice current_key = raw_key_.GetKey(); + + if (raw_key_.IsKeyPinned()) { + // The key is not delta encoded + prev_entries_.emplace_back(current_, current_key.data(), 0, + current_key.size(), value()); + } else { + // The key is delta encoded, cache decoded key in buffer + size_t new_key_offset = prev_entries_keys_buff_.size(); + prev_entries_keys_buff_.append(current_key.data(), current_key.size()); + + prev_entries_.emplace_back(current_, nullptr, new_key_offset, + current_key.size(), value()); + } + // Loop until end of current entry hits the start of original entry + } while (NextEntryOffset() < original); + prev_entries_idx_ = static_cast<int32_t>(prev_entries_.size()) - 1; +} + +void DataBlockIter::SeekImpl(const Slice& target) { + Slice seek_key = target; + PERF_TIMER_GUARD(block_seek_nanos); + if (data_ == nullptr) { // Not init yet + return; + } + uint32_t index = 0; + bool skip_linear_scan = false; + bool ok = BinarySeek<DecodeKey>(seek_key, &index, &skip_linear_scan); + + if (!ok) { + return; + } + FindKeyAfterBinarySeek(seek_key, index, skip_linear_scan); +} + +void MetaBlockIter::SeekImpl(const Slice& target) { + Slice seek_key = target; + PERF_TIMER_GUARD(block_seek_nanos); + if (data_ == nullptr) { // Not init yet + return; + } + uint32_t index = 0; + bool skip_linear_scan = false; + bool ok = BinarySeek<DecodeKey>(seek_key, &index, &skip_linear_scan); + + if (!ok) { + return; + } + FindKeyAfterBinarySeek(seek_key, index, skip_linear_scan); +} + +// Optimized Seek for point lookup for an internal key `target` +// target = "seek_user_key @ type | seqno". +// +// For any type other than kTypeValue, kTypeDeletion, kTypeSingleDeletion, +// kTypeBlobIndex, or kTypeWideColumnEntity, this function behaves identically +// to Seek(). +// +// For any type in kTypeValue, kTypeDeletion, kTypeSingleDeletion, +// kTypeBlobIndex, or kTypeWideColumnEntity: +// +// If the return value is FALSE, iter location is undefined, and it means: +// 1) there is no key in this block falling into the range: +// ["seek_user_key @ type | seqno", "seek_user_key @ kTypeDeletion | 0"], +// inclusive; AND +// 2) the last key of this block has a greater user_key from seek_user_key +// +// If the return value is TRUE, iter location has two possibilies: +// 1) If iter is valid, it is set to a location as if set by BinarySeek. In +// this case, it points to the first key with a larger user_key or a matching +// user_key with a seqno no greater than the seeking seqno. +// 2) If the iter is invalid, it means that either all the user_key is less +// than the seek_user_key, or the block ends with a matching user_key but +// with a smaller [ type | seqno ] (i.e. a larger seqno, or the same seqno +// but larger type). +bool DataBlockIter::SeekForGetImpl(const Slice& target) { + Slice target_user_key = ExtractUserKey(target); + uint32_t map_offset = restarts_ + num_restarts_ * sizeof(uint32_t); + uint8_t entry = + data_block_hash_index_->Lookup(data_, map_offset, target_user_key); + + if (entry == kCollision) { + // HashSeek not effective, falling back + SeekImpl(target); + return true; + } + + if (entry == kNoEntry) { + // Even if we cannot find the user_key in this block, the result may + // exist in the next block. Consider this example: + // + // Block N: [aab@100, ... , app@120] + // boundary key: axy@50 (we make minimal assumption about a boundary key) + // Block N+1: [axy@10, ... ] + // + // If seek_key = axy@60, the search will starts from Block N. + // Even if the user_key is not found in the hash map, the caller still + // have to continue searching the next block. + // + // In this case, we pretend the key is the the last restart interval. + // The while-loop below will search the last restart interval for the + // key. It will stop at the first key that is larger than the seek_key, + // or to the end of the block if no one is larger. + entry = static_cast<uint8_t>(num_restarts_ - 1); + } + + uint32_t restart_index = entry; + + // check if the key is in the restart_interval + assert(restart_index < num_restarts_); + SeekToRestartPoint(restart_index); + current_ = GetRestartPoint(restart_index); + + uint32_t limit = restarts_; + if (restart_index + 1 < num_restarts_) { + limit = GetRestartPoint(restart_index + 1); + } + while (current_ < limit) { + bool shared; + // Here we only linear seek the target key inside the restart interval. + // If a key does not exist inside a restart interval, we avoid + // further searching the block content across restart interval boundary. + // + // TODO(fwu): check the left and right boundary of the restart interval + // to avoid linear seek a target key that is out of range. + if (!ParseNextDataKey(&shared) || CompareCurrentKey(target) >= 0) { + // we stop at the first potential matching user key. + break; + } + } + + if (current_ == restarts_) { + // Search reaches to the end of the block. There are three possibilites: + // 1) there is only one user_key match in the block (otherwise collsion). + // the matching user_key resides in the last restart interval, and it + // is the last key of the restart interval and of the block as well. + // ParseNextKey() skiped it as its [ type | seqno ] is smaller. + // + // 2) The seek_key is not found in the HashIndex Lookup(), i.e. kNoEntry, + // AND all existing user_keys in the restart interval are smaller than + // seek_user_key. + // + // 3) The seek_key is a false positive and happens to be hashed to the + // last restart interval, AND all existing user_keys in the restart + // interval are smaller than seek_user_key. + // + // The result may exist in the next block each case, so we return true. + return true; + } + + if (icmp_->user_comparator()->Compare(raw_key_.GetUserKey(), + target_user_key) != 0) { + // the key is not in this block and cannot be at the next block either. + return false; + } + + // Here we are conservative and only support a limited set of cases + ValueType value_type = ExtractValueType(raw_key_.GetInternalKey()); + if (value_type != ValueType::kTypeValue && + value_type != ValueType::kTypeDeletion && + value_type != ValueType::kTypeSingleDeletion && + value_type != ValueType::kTypeBlobIndex && + value_type != ValueType::kTypeWideColumnEntity) { + SeekImpl(target); + return true; + } + + // Result found, and the iter is correctly set. + return true; +} + +void IndexBlockIter::SeekImpl(const Slice& target) { + TEST_SYNC_POINT("IndexBlockIter::Seek:0"); + PERF_TIMER_GUARD(block_seek_nanos); + if (data_ == nullptr) { // Not init yet + return; + } + Slice seek_key = target; + if (raw_key_.IsUserKey()) { + seek_key = ExtractUserKey(target); + } + status_ = Status::OK(); + uint32_t index = 0; + bool skip_linear_scan = false; + bool ok = false; + if (prefix_index_) { + bool prefix_may_exist = true; + ok = PrefixSeek(target, &index, &prefix_may_exist); + if (!prefix_may_exist) { + // This is to let the caller to distinguish between non-existing prefix, + // and when key is larger than the last key, which both set Valid() to + // false. + current_ = restarts_; + status_ = Status::NotFound(); + } + // restart interval must be one when hash search is enabled so the binary + // search simply lands at the right place. + skip_linear_scan = true; + } else if (value_delta_encoded_) { + ok = BinarySeek<DecodeKeyV4>(seek_key, &index, &skip_linear_scan); + } else { + ok = BinarySeek<DecodeKey>(seek_key, &index, &skip_linear_scan); + } + + if (!ok) { + return; + } + FindKeyAfterBinarySeek(seek_key, index, skip_linear_scan); +} + +void DataBlockIter::SeekForPrevImpl(const Slice& target) { + PERF_TIMER_GUARD(block_seek_nanos); + Slice seek_key = target; + if (data_ == nullptr) { // Not init yet + return; + } + uint32_t index = 0; + bool skip_linear_scan = false; + bool ok = BinarySeek<DecodeKey>(seek_key, &index, &skip_linear_scan); + + if (!ok) { + return; + } + FindKeyAfterBinarySeek(seek_key, index, skip_linear_scan); + + if (!Valid()) { + SeekToLastImpl(); + } else { + while (Valid() && CompareCurrentKey(seek_key) > 0) { + PrevImpl(); + } + } +} + +void MetaBlockIter::SeekForPrevImpl(const Slice& target) { + PERF_TIMER_GUARD(block_seek_nanos); + Slice seek_key = target; + if (data_ == nullptr) { // Not init yet + return; + } + uint32_t index = 0; + bool skip_linear_scan = false; + bool ok = BinarySeek<DecodeKey>(seek_key, &index, &skip_linear_scan); + + if (!ok) { + return; + } + FindKeyAfterBinarySeek(seek_key, index, skip_linear_scan); + + if (!Valid()) { + SeekToLastImpl(); + } else { + while (Valid() && CompareCurrentKey(seek_key) > 0) { + PrevImpl(); + } + } +} + +void DataBlockIter::SeekToFirstImpl() { + if (data_ == nullptr) { // Not init yet + return; + } + SeekToRestartPoint(0); + bool is_shared = false; + ParseNextDataKey(&is_shared); +} + +void MetaBlockIter::SeekToFirstImpl() { + if (data_ == nullptr) { // Not init yet + return; + } + SeekToRestartPoint(0); + bool is_shared = false; + ParseNextKey<CheckAndDecodeEntry>(&is_shared); +} + +void IndexBlockIter::SeekToFirstImpl() { + if (data_ == nullptr) { // Not init yet + return; + } + status_ = Status::OK(); + SeekToRestartPoint(0); + ParseNextIndexKey(); +} + +void DataBlockIter::SeekToLastImpl() { + if (data_ == nullptr) { // Not init yet + return; + } + SeekToRestartPoint(num_restarts_ - 1); + bool is_shared = false; + while (ParseNextDataKey(&is_shared) && NextEntryOffset() < restarts_) { + // Keep skipping + } +} + +void MetaBlockIter::SeekToLastImpl() { + if (data_ == nullptr) { // Not init yet + return; + } + SeekToRestartPoint(num_restarts_ - 1); + bool is_shared = false; + while (ParseNextKey<CheckAndDecodeEntry>(&is_shared) && + NextEntryOffset() < restarts_) { + // Keep skipping + } +} + +void IndexBlockIter::SeekToLastImpl() { + if (data_ == nullptr) { // Not init yet + return; + } + status_ = Status::OK(); + SeekToRestartPoint(num_restarts_ - 1); + while (ParseNextIndexKey() && NextEntryOffset() < restarts_) { + // Keep skipping + } +} + +template <class TValue> +void BlockIter<TValue>::CorruptionError() { + current_ = restarts_; + restart_index_ = num_restarts_; + status_ = Status::Corruption("bad entry in block"); + raw_key_.Clear(); + value_.clear(); +} + +template <class TValue> +template <typename DecodeEntryFunc> +bool BlockIter<TValue>::ParseNextKey(bool* is_shared) { + current_ = NextEntryOffset(); + const char* p = data_ + current_; + const char* limit = data_ + restarts_; // Restarts come right after data + + if (p >= limit) { + // No more entries to return. Mark as invalid. + current_ = restarts_; + restart_index_ = num_restarts_; + return false; + } + // Decode next entry + uint32_t shared, non_shared, value_length; + p = DecodeEntryFunc()(p, limit, &shared, &non_shared, &value_length); + if (p == nullptr || raw_key_.Size() < shared) { + CorruptionError(); + return false; + } else { + if (shared == 0) { + *is_shared = false; + // If this key doesn't share any bytes with prev key then we don't need + // to decode it and can use its address in the block directly. + raw_key_.SetKey(Slice(p, non_shared), false /* copy */); + } else { + // This key share `shared` bytes with prev key, we need to decode it + *is_shared = true; + raw_key_.TrimAppend(shared, p, non_shared); + } + value_ = Slice(p + non_shared, value_length); + if (shared == 0) { + while (restart_index_ + 1 < num_restarts_ && + GetRestartPoint(restart_index_ + 1) < current_) { + ++restart_index_; + } + } + // else we are in the middle of a restart interval and the restart_index_ + // thus has not changed + return true; + } +} + +bool DataBlockIter::ParseNextDataKey(bool* is_shared) { + if (ParseNextKey<DecodeEntry>(is_shared)) { +#ifndef NDEBUG + if (global_seqno_ != kDisableGlobalSequenceNumber) { + // If we are reading a file with a global sequence number we should + // expect that all encoded sequence numbers are zeros and any value + // type is kTypeValue, kTypeMerge, kTypeDeletion, + // kTypeDeletionWithTimestamp, or kTypeRangeDeletion. + uint64_t packed = ExtractInternalKeyFooter(raw_key_.GetKey()); + SequenceNumber seqno; + ValueType value_type; + UnPackSequenceAndType(packed, &seqno, &value_type); + assert(value_type == ValueType::kTypeValue || + value_type == ValueType::kTypeMerge || + value_type == ValueType::kTypeDeletion || + value_type == ValueType::kTypeDeletionWithTimestamp || + value_type == ValueType::kTypeRangeDeletion); + assert(seqno == 0); + } +#endif // NDEBUG + return true; + } else { + return false; + } +} + +bool IndexBlockIter::ParseNextIndexKey() { + bool is_shared = false; + bool ok = (value_delta_encoded_) ? ParseNextKey<DecodeEntryV4>(&is_shared) + : ParseNextKey<DecodeEntry>(&is_shared); + if (ok) { + if (value_delta_encoded_ || global_seqno_state_ != nullptr) { + DecodeCurrentValue(is_shared); + } + } + return ok; +} + +// The format: +// restart_point 0: k, v (off, sz), k, v (delta-sz), ..., k, v (delta-sz) +// restart_point 1: k, v (off, sz), k, v (delta-sz), ..., k, v (delta-sz) +// ... +// restart_point n-1: k, v (off, sz), k, v (delta-sz), ..., k, v (delta-sz) +// where, k is key, v is value, and its encoding is in parenthesis. +// The format of each key is (shared_size, non_shared_size, shared, non_shared) +// The format of each value, i.e., block handle, is (offset, size) whenever the +// is_shared is false, which included the first entry in each restart point. +// Otherwise the format is delta-size = block handle size - size of last block +// handle. +void IndexBlockIter::DecodeCurrentValue(bool is_shared) { + Slice v(value_.data(), data_ + restarts_ - value_.data()); + // Delta encoding is used if `shared` != 0. + Status decode_s __attribute__((__unused__)) = decoded_value_.DecodeFrom( + &v, have_first_key_, + (value_delta_encoded_ && is_shared) ? &decoded_value_.handle : nullptr); + assert(decode_s.ok()); + value_ = Slice(value_.data(), v.data() - value_.data()); + + if (global_seqno_state_ != nullptr) { + // Overwrite sequence number the same way as in DataBlockIter. + + IterKey& first_internal_key = global_seqno_state_->first_internal_key; + first_internal_key.SetInternalKey(decoded_value_.first_internal_key, + /* copy */ true); + + assert(GetInternalKeySeqno(first_internal_key.GetInternalKey()) == 0); + + ValueType value_type = ExtractValueType(first_internal_key.GetKey()); + assert(value_type == ValueType::kTypeValue || + value_type == ValueType::kTypeMerge || + value_type == ValueType::kTypeDeletion || + value_type == ValueType::kTypeRangeDeletion); + + first_internal_key.UpdateInternalKey(global_seqno_state_->global_seqno, + value_type); + decoded_value_.first_internal_key = first_internal_key.GetKey(); + } +} + +template <class TValue> +void BlockIter<TValue>::FindKeyAfterBinarySeek(const Slice& target, + uint32_t index, + bool skip_linear_scan) { + // SeekToRestartPoint() only does the lookup in the restart block. We need + // to follow it up with NextImpl() to position the iterator at the restart + // key. + SeekToRestartPoint(index); + NextImpl(); + + if (!skip_linear_scan) { + // Linear search (within restart block) for first key >= target + uint32_t max_offset; + if (index + 1 < num_restarts_) { + // We are in a non-last restart interval. Since `BinarySeek()` guarantees + // the next restart key is strictly greater than `target`, we can + // terminate upon reaching it without any additional key comparison. + max_offset = GetRestartPoint(index + 1); + } else { + // We are in the last restart interval. The while-loop will terminate by + // `Valid()` returning false upon advancing past the block's last key. + max_offset = std::numeric_limits<uint32_t>::max(); + } + while (true) { + NextImpl(); + if (!Valid()) { + break; + } + if (current_ == max_offset) { + assert(CompareCurrentKey(target) > 0); + break; + } else if (CompareCurrentKey(target) >= 0) { + break; + } + } + } +} + +// Binary searches in restart array to find the starting restart point for the +// linear scan, and stores it in `*index`. Assumes restart array does not +// contain duplicate keys. It is guaranteed that the restart key at `*index + 1` +// is strictly greater than `target` or does not exist (this can be used to +// elide a comparison when linear scan reaches all the way to the next restart +// key). Furthermore, `*skip_linear_scan` is set to indicate whether the +// `*index`th restart key is the final result so that key does not need to be +// compared again later. +template <class TValue> +template <typename DecodeKeyFunc> +bool BlockIter<TValue>::BinarySeek(const Slice& target, uint32_t* index, + bool* skip_linear_scan) { + if (restarts_ == 0) { + // SST files dedicated to range tombstones are written with index blocks + // that have no keys while also having `num_restarts_ == 1`. This would + // cause a problem for `BinarySeek()` as it'd try to access the first key + // which does not exist. We identify such blocks by the offset at which + // their restarts are stored, and return false to prevent any attempted + // key accesses. + return false; + } + + *skip_linear_scan = false; + // Loop invariants: + // - Restart key at index `left` is less than or equal to the target key. The + // sentinel index `-1` is considered to have a key that is less than all + // keys. + // - Any restart keys after index `right` are strictly greater than the target + // key. + int64_t left = -1, right = num_restarts_ - 1; + while (left != right) { + // The `mid` is computed by rounding up so it lands in (`left`, `right`]. + int64_t mid = left + (right - left + 1) / 2; + uint32_t region_offset = GetRestartPoint(static_cast<uint32_t>(mid)); + uint32_t shared, non_shared; + const char* key_ptr = DecodeKeyFunc()( + data_ + region_offset, data_ + restarts_, &shared, &non_shared); + if (key_ptr == nullptr || (shared != 0)) { + CorruptionError(); + return false; + } + Slice mid_key(key_ptr, non_shared); + raw_key_.SetKey(mid_key, false /* copy */); + int cmp = CompareCurrentKey(target); + if (cmp < 0) { + // Key at "mid" is smaller than "target". Therefore all + // blocks before "mid" are uninteresting. + left = mid; + } else if (cmp > 0) { + // Key at "mid" is >= "target". Therefore all blocks at or + // after "mid" are uninteresting. + right = mid - 1; + } else { + *skip_linear_scan = true; + left = right = mid; + } + } + + if (left == -1) { + // All keys in the block were strictly greater than `target`. So the very + // first key in the block is the final seek result. + *skip_linear_scan = true; + *index = 0; + } else { + *index = static_cast<uint32_t>(left); + } + return true; +} + +// Compare target key and the block key of the block of `block_index`. +// Return -1 if error. +int IndexBlockIter::CompareBlockKey(uint32_t block_index, const Slice& target) { + uint32_t region_offset = GetRestartPoint(block_index); + uint32_t shared, non_shared; + const char* key_ptr = + value_delta_encoded_ + ? DecodeKeyV4()(data_ + region_offset, data_ + restarts_, &shared, + &non_shared) + : DecodeKey()(data_ + region_offset, data_ + restarts_, &shared, + &non_shared); + if (key_ptr == nullptr || (shared != 0)) { + CorruptionError(); + return 1; // Return target is smaller + } + Slice block_key(key_ptr, non_shared); + raw_key_.SetKey(block_key, false /* copy */); + return CompareCurrentKey(target); +} + +// Binary search in block_ids to find the first block +// with a key >= target +bool IndexBlockIter::BinaryBlockIndexSeek(const Slice& target, + uint32_t* block_ids, uint32_t left, + uint32_t right, uint32_t* index, + bool* prefix_may_exist) { + assert(left <= right); + assert(index); + assert(prefix_may_exist); + *prefix_may_exist = true; + uint32_t left_bound = left; + + while (left <= right) { + uint32_t mid = (right + left) / 2; + + int cmp = CompareBlockKey(block_ids[mid], target); + if (!status_.ok()) { + return false; + } + if (cmp < 0) { + // Key at "target" is larger than "mid". Therefore all + // blocks before or at "mid" are uninteresting. + left = mid + 1; + } else { + // Key at "target" is <= "mid". Therefore all blocks + // after "mid" are uninteresting. + // If there is only one block left, we found it. + if (left == right) break; + right = mid; + } + } + + if (left == right) { + // In one of the two following cases: + // (1) left is the first one of block_ids + // (2) there is a gap of blocks between block of `left` and `left-1`. + // we can further distinguish the case of key in the block or key not + // existing, by comparing the target key and the key of the previous + // block to the left of the block found. + if (block_ids[left] > 0 && + (left == left_bound || block_ids[left - 1] != block_ids[left] - 1) && + CompareBlockKey(block_ids[left] - 1, target) > 0) { + current_ = restarts_; + *prefix_may_exist = false; + return false; + } + + *index = block_ids[left]; + return true; + } else { + assert(left > right); + + // If the next block key is larger than seek key, it is possible that + // no key shares the prefix with `target`, or all keys with the same + // prefix as `target` are smaller than prefix. In the latter case, + // we are mandated to set the position the same as the total order. + // In the latter case, either: + // (1) `target` falls into the range of the next block. In this case, + // we can place the iterator to the next block, or + // (2) `target` is larger than all block keys. In this case we can + // keep the iterator invalidate without setting `prefix_may_exist` + // to false. + // We might sometimes end up with setting the total order position + // while there is no key sharing the prefix as `target`, but it + // still follows the contract. + uint32_t right_index = block_ids[right]; + assert(right_index + 1 <= num_restarts_); + if (right_index + 1 < num_restarts_) { + if (CompareBlockKey(right_index + 1, target) >= 0) { + *index = right_index + 1; + return true; + } else { + // We have to set the flag here because we are not positioning + // the iterator to the total order position. + *prefix_may_exist = false; + } + } + + // Mark iterator invalid + current_ = restarts_; + return false; + } +} + +bool IndexBlockIter::PrefixSeek(const Slice& target, uint32_t* index, + bool* prefix_may_exist) { + assert(index); + assert(prefix_may_exist); + assert(prefix_index_); + *prefix_may_exist = true; + Slice seek_key = target; + if (raw_key_.IsUserKey()) { + seek_key = ExtractUserKey(target); + } + uint32_t* block_ids = nullptr; + uint32_t num_blocks = prefix_index_->GetBlocks(target, &block_ids); + + if (num_blocks == 0) { + current_ = restarts_; + *prefix_may_exist = false; + return false; + } else { + assert(block_ids); + return BinaryBlockIndexSeek(seek_key, block_ids, 0, num_blocks - 1, index, + prefix_may_exist); + } +} + +uint32_t Block::NumRestarts() const { + assert(size_ >= 2 * sizeof(uint32_t)); + uint32_t block_footer = DecodeFixed32(data_ + size_ - sizeof(uint32_t)); + uint32_t num_restarts = block_footer; + if (size_ > kMaxBlockSizeSupportedByHashIndex) { + // In BlockBuilder, we have ensured a block with HashIndex is less than + // kMaxBlockSizeSupportedByHashIndex (64KiB). + // + // Therefore, if we encounter a block with a size > 64KiB, the block + // cannot have HashIndex. So the footer will directly interpreted as + // num_restarts. + // + // Such check is for backward compatibility. We can ensure legacy block + // with a vary large num_restarts i.e. >= 0x80000000 can be interpreted + // correctly as no HashIndex even if the MSB of num_restarts is set. + return num_restarts; + } + BlockBasedTableOptions::DataBlockIndexType index_type; + UnPackIndexTypeAndNumRestarts(block_footer, &index_type, &num_restarts); + return num_restarts; +} + +BlockBasedTableOptions::DataBlockIndexType Block::IndexType() const { + assert(size_ >= 2 * sizeof(uint32_t)); + if (size_ > kMaxBlockSizeSupportedByHashIndex) { + // The check is for the same reason as that in NumRestarts() + return BlockBasedTableOptions::kDataBlockBinarySearch; + } + uint32_t block_footer = DecodeFixed32(data_ + size_ - sizeof(uint32_t)); + uint32_t num_restarts = block_footer; + BlockBasedTableOptions::DataBlockIndexType index_type; + UnPackIndexTypeAndNumRestarts(block_footer, &index_type, &num_restarts); + return index_type; +} + +Block::~Block() { + // This sync point can be re-enabled if RocksDB can control the + // initialization order of any/all static options created by the user. + // TEST_SYNC_POINT("Block::~Block"); +} + +Block::Block(BlockContents&& contents, size_t read_amp_bytes_per_bit, + Statistics* statistics) + : contents_(std::move(contents)), + data_(contents_.data.data()), + size_(contents_.data.size()), + restart_offset_(0), + num_restarts_(0) { + TEST_SYNC_POINT("Block::Block:0"); + if (size_ < sizeof(uint32_t)) { + size_ = 0; // Error marker + } else { + // Should only decode restart points for uncompressed blocks + num_restarts_ = NumRestarts(); + switch (IndexType()) { + case BlockBasedTableOptions::kDataBlockBinarySearch: + restart_offset_ = static_cast<uint32_t>(size_) - + (1 + num_restarts_) * sizeof(uint32_t); + if (restart_offset_ > size_ - sizeof(uint32_t)) { + // The size is too small for NumRestarts() and therefore + // restart_offset_ wrapped around. + size_ = 0; + } + break; + case BlockBasedTableOptions::kDataBlockBinaryAndHash: + if (size_ < sizeof(uint32_t) /* block footer */ + + sizeof(uint16_t) /* NUM_BUCK */) { + size_ = 0; + break; + } + + uint16_t map_offset; + data_block_hash_index_.Initialize( + contents.data.data(), + static_cast<uint16_t>(contents.data.size() - + sizeof(uint32_t)), /*chop off + NUM_RESTARTS*/ + &map_offset); + + restart_offset_ = map_offset - num_restarts_ * sizeof(uint32_t); + + if (restart_offset_ > map_offset) { + // map_offset is too small for NumRestarts() and + // therefore restart_offset_ wrapped around. + size_ = 0; + break; + } + break; + default: + size_ = 0; // Error marker + } + } + if (read_amp_bytes_per_bit != 0 && statistics && size_ != 0) { + read_amp_bitmap_.reset(new BlockReadAmpBitmap( + restart_offset_, read_amp_bytes_per_bit, statistics)); + } +} + +MetaBlockIter* Block::NewMetaIterator(bool block_contents_pinned) { + MetaBlockIter* iter = new MetaBlockIter(); + if (size_ < 2 * sizeof(uint32_t)) { + iter->Invalidate(Status::Corruption("bad block contents")); + return iter; + } else if (num_restarts_ == 0) { + // Empty block. + iter->Invalidate(Status::OK()); + } else { + iter->Initialize(data_, restart_offset_, num_restarts_, + block_contents_pinned); + } + return iter; +} + +DataBlockIter* Block::NewDataIterator(const Comparator* raw_ucmp, + SequenceNumber global_seqno, + DataBlockIter* iter, Statistics* stats, + bool block_contents_pinned) { + DataBlockIter* ret_iter; + if (iter != nullptr) { + ret_iter = iter; + } else { + ret_iter = new DataBlockIter; + } + if (size_ < 2 * sizeof(uint32_t)) { + ret_iter->Invalidate(Status::Corruption("bad block contents")); + return ret_iter; + } + if (num_restarts_ == 0) { + // Empty block. + ret_iter->Invalidate(Status::OK()); + return ret_iter; + } else { + ret_iter->Initialize( + raw_ucmp, data_, restart_offset_, num_restarts_, global_seqno, + read_amp_bitmap_.get(), block_contents_pinned, + data_block_hash_index_.Valid() ? &data_block_hash_index_ : nullptr); + if (read_amp_bitmap_) { + if (read_amp_bitmap_->GetStatistics() != stats) { + // DB changed the Statistics pointer, we need to notify read_amp_bitmap_ + read_amp_bitmap_->SetStatistics(stats); + } + } + } + + return ret_iter; +} + +IndexBlockIter* Block::NewIndexIterator( + const Comparator* raw_ucmp, SequenceNumber global_seqno, + IndexBlockIter* iter, Statistics* /*stats*/, bool total_order_seek, + bool have_first_key, bool key_includes_seq, bool value_is_full, + bool block_contents_pinned, BlockPrefixIndex* prefix_index) { + IndexBlockIter* ret_iter; + if (iter != nullptr) { + ret_iter = iter; + } else { + ret_iter = new IndexBlockIter; + } + if (size_ < 2 * sizeof(uint32_t)) { + ret_iter->Invalidate(Status::Corruption("bad block contents")); + return ret_iter; + } + if (num_restarts_ == 0) { + // Empty block. + ret_iter->Invalidate(Status::OK()); + return ret_iter; + } else { + BlockPrefixIndex* prefix_index_ptr = + total_order_seek ? nullptr : prefix_index; + ret_iter->Initialize(raw_ucmp, data_, restart_offset_, num_restarts_, + global_seqno, prefix_index_ptr, have_first_key, + key_includes_seq, value_is_full, + block_contents_pinned); + } + + return ret_iter; +} + +size_t Block::ApproximateMemoryUsage() const { + size_t usage = usable_size(); +#ifdef ROCKSDB_MALLOC_USABLE_SIZE + usage += malloc_usable_size((void*)this); +#else + usage += sizeof(*this); +#endif // ROCKSDB_MALLOC_USABLE_SIZE + if (read_amp_bitmap_) { + usage += read_amp_bitmap_->ApproximateMemoryUsage(); + } + return usage; +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/block.h b/src/rocksdb/table/block_based/block.h new file mode 100644 index 000000000..5d73f72f6 --- /dev/null +++ b/src/rocksdb/table/block_based/block.h @@ -0,0 +1,744 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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 <stddef.h> +#include <stdint.h> + +#include <string> +#include <vector> + +#include "db/pinned_iterators_manager.h" +#include "port/malloc.h" +#include "rocksdb/iterator.h" +#include "rocksdb/options.h" +#include "rocksdb/statistics.h" +#include "rocksdb/table.h" +#include "table/block_based/block_prefix_index.h" +#include "table/block_based/data_block_hash_index.h" +#include "table/format.h" +#include "table/internal_iterator.h" +#include "test_util/sync_point.h" +#include "util/random.h" + +namespace ROCKSDB_NAMESPACE { + +struct BlockContents; +class Comparator; +template <class TValue> +class BlockIter; +class DataBlockIter; +class IndexBlockIter; +class MetaBlockIter; +class BlockPrefixIndex; + +// BlockReadAmpBitmap is a bitmap that map the ROCKSDB_NAMESPACE::Block data +// bytes to a bitmap with ratio bytes_per_bit. Whenever we access a range of +// bytes in the Block we update the bitmap and increment +// READ_AMP_ESTIMATE_USEFUL_BYTES. +class BlockReadAmpBitmap { + public: + explicit BlockReadAmpBitmap(size_t block_size, size_t bytes_per_bit, + Statistics* statistics) + : bitmap_(nullptr), + bytes_per_bit_pow_(0), + statistics_(statistics), + rnd_(Random::GetTLSInstance()->Uniform( + static_cast<int>(bytes_per_bit))) { + TEST_SYNC_POINT_CALLBACK("BlockReadAmpBitmap:rnd", &rnd_); + assert(block_size > 0 && bytes_per_bit > 0); + + // convert bytes_per_bit to be a power of 2 + while (bytes_per_bit >>= 1) { + bytes_per_bit_pow_++; + } + + // num_bits_needed = ceil(block_size / bytes_per_bit) + size_t num_bits_needed = ((block_size - 1) >> bytes_per_bit_pow_) + 1; + assert(num_bits_needed > 0); + + // bitmap_size = ceil(num_bits_needed / kBitsPerEntry) + size_t bitmap_size = (num_bits_needed - 1) / kBitsPerEntry + 1; + + // Create bitmap and set all the bits to 0 + bitmap_ = new std::atomic<uint32_t>[bitmap_size](); + + RecordTick(GetStatistics(), READ_AMP_TOTAL_READ_BYTES, block_size); + } + + ~BlockReadAmpBitmap() { delete[] bitmap_; } + + void Mark(uint32_t start_offset, uint32_t end_offset) { + assert(end_offset >= start_offset); + // Index of first bit in mask + uint32_t start_bit = + (start_offset + (1 << bytes_per_bit_pow_) - rnd_ - 1) >> + bytes_per_bit_pow_; + // Index of last bit in mask + 1 + uint32_t exclusive_end_bit = + (end_offset + (1 << bytes_per_bit_pow_) - rnd_) >> bytes_per_bit_pow_; + if (start_bit >= exclusive_end_bit) { + return; + } + assert(exclusive_end_bit > 0); + + if (GetAndSet(start_bit) == 0) { + uint32_t new_useful_bytes = (exclusive_end_bit - start_bit) + << bytes_per_bit_pow_; + RecordTick(GetStatistics(), READ_AMP_ESTIMATE_USEFUL_BYTES, + new_useful_bytes); + } + } + + Statistics* GetStatistics() { + return statistics_.load(std::memory_order_relaxed); + } + + void SetStatistics(Statistics* stats) { statistics_.store(stats); } + + uint32_t GetBytesPerBit() { return 1 << bytes_per_bit_pow_; } + + size_t ApproximateMemoryUsage() const { +#ifdef ROCKSDB_MALLOC_USABLE_SIZE + return malloc_usable_size((void*)this); +#endif // ROCKSDB_MALLOC_USABLE_SIZE + return sizeof(*this); + } + + private: + // Get the current value of bit at `bit_idx` and set it to 1 + inline bool GetAndSet(uint32_t bit_idx) { + const uint32_t byte_idx = bit_idx / kBitsPerEntry; + const uint32_t bit_mask = 1 << (bit_idx % kBitsPerEntry); + + return bitmap_[byte_idx].fetch_or(bit_mask, std::memory_order_relaxed) & + bit_mask; + } + + const uint32_t kBytesPersEntry = sizeof(uint32_t); // 4 bytes + const uint32_t kBitsPerEntry = kBytesPersEntry * 8; // 32 bits + + // Bitmap used to record the bytes that we read, use atomic to protect + // against multiple threads updating the same bit + std::atomic<uint32_t>* bitmap_; + // (1 << bytes_per_bit_pow_) is bytes_per_bit. Use power of 2 to optimize + // muliplication and division + uint8_t bytes_per_bit_pow_; + // Pointer to DB Statistics object, Since this bitmap may outlive the DB + // this pointer maybe invalid, but the DB will update it to a valid pointer + // by using SetStatistics() before calling Mark() + std::atomic<Statistics*> statistics_; + uint32_t rnd_; +}; + +// class Block is the uncompressed and "parsed" form for blocks containing +// key-value pairs. (See BlockContents comments for more on terminology.) +// This includes the in-memory representation of data blocks, index blocks +// (including partitions), range deletion blocks, properties blocks, metaindex +// blocks, as well as the top level of the partitioned filter structure (which +// is actually an index of the filter partitions). It is NOT suitable for +// compressed blocks in general, filter blocks/partitions, or compression +// dictionaries. +// +// See https://github.com/facebook/rocksdb/wiki/Rocksdb-BlockBasedTable-Format +// for details of the format and the various block types. +// +// TODO: Rename to ParsedKvBlock? +class Block { + public: + // Initialize the block with the specified contents. + explicit Block(BlockContents&& contents, size_t read_amp_bytes_per_bit = 0, + Statistics* statistics = nullptr); + // No copying allowed + Block(const Block&) = delete; + void operator=(const Block&) = delete; + + ~Block(); + + size_t size() const { return size_; } + const char* data() const { return data_; } + // The additional memory space taken by the block data. + size_t usable_size() const { return contents_.usable_size(); } + uint32_t NumRestarts() const; + bool own_bytes() const { return contents_.own_bytes(); } + + BlockBasedTableOptions::DataBlockIndexType IndexType() const; + + // raw_ucmp is a raw (i.e., not wrapped by `UserComparatorWrapper`) user key + // comparator. + // + // If iter is null, return new Iterator + // If iter is not null, update this one and return it as Iterator* + // + // Updates read_amp_bitmap_ if it is not nullptr. + // + // If `block_contents_pinned` is true, the caller will guarantee that when + // the cleanup functions are transferred from the iterator to other + // classes, e.g. PinnableSlice, the pointer to the bytes will still be + // valid. Either the iterator holds cache handle or ownership of some resource + // and release them in a release function, or caller is sure that the data + // will not go away (for example, it's from mmapped file which will not be + // closed). + // + // NOTE: for the hash based lookup, if a key prefix doesn't match any key, + // the iterator will simply be set as "invalid", rather than returning + // the key that is just pass the target key. + DataBlockIter* NewDataIterator(const Comparator* raw_ucmp, + SequenceNumber global_seqno, + DataBlockIter* iter = nullptr, + Statistics* stats = nullptr, + bool block_contents_pinned = false); + + // Returns an MetaBlockIter for iterating over blocks containing metadata + // (like Properties blocks). Unlike data blocks, the keys for these blocks + // do not contain sequence numbers, do not use a user-define comparator, and + // do not track read amplification/statistics. Additionally, MetaBlocks will + // not assert if the block is formatted improperly. + // + // If `block_contents_pinned` is true, the caller will guarantee that when + // the cleanup functions are transferred from the iterator to other + // classes, e.g. PinnableSlice, the pointer to the bytes will still be + // valid. Either the iterator holds cache handle or ownership of some resource + // and release them in a release function, or caller is sure that the data + // will not go away (for example, it's from mmapped file which will not be + // closed). + MetaBlockIter* NewMetaIterator(bool block_contents_pinned = false); + + // raw_ucmp is a raw (i.e., not wrapped by `UserComparatorWrapper`) user key + // comparator. + // + // key_includes_seq, default true, means that the keys are in internal key + // format. + // value_is_full, default true, means that no delta encoding is + // applied to values. + // + // If `prefix_index` is not nullptr this block will do hash lookup for the key + // prefix. If total_order_seek is true, prefix_index_ is ignored. + // + // `have_first_key` controls whether IndexValue will contain + // first_internal_key. It affects data serialization format, so the same value + // have_first_key must be used when writing and reading index. + // It is determined by IndexType property of the table. + IndexBlockIter* NewIndexIterator(const Comparator* raw_ucmp, + SequenceNumber global_seqno, + IndexBlockIter* iter, Statistics* stats, + bool total_order_seek, bool have_first_key, + bool key_includes_seq, bool value_is_full, + bool block_contents_pinned = false, + BlockPrefixIndex* prefix_index = nullptr); + + // Report an approximation of how much memory has been used. + size_t ApproximateMemoryUsage() const; + + private: + BlockContents contents_; + const char* data_; // contents_.data.data() + size_t size_; // contents_.data.size() + uint32_t restart_offset_; // Offset in data_ of restart array + uint32_t num_restarts_; + std::unique_ptr<BlockReadAmpBitmap> read_amp_bitmap_; + DataBlockHashIndex data_block_hash_index_; +}; + +// A `BlockIter` iterates over the entries in a `Block`'s data buffer. The +// format of this data buffer is an uncompressed, sorted sequence of key-value +// pairs (see `Block` API for more details). +// +// Notably, the keys may either be in internal key format or user key format. +// Subclasses are responsible for configuring the key format. +// +// `BlockIter` intends to provide final overrides for all of +// `InternalIteratorBase` functions that can move the iterator. It does +// this to guarantee `UpdateKey()` is called exactly once after each key +// movement potentially visible to users. In this step, the key is prepared +// (e.g., serialized if global seqno is in effect) so it can be returned +// immediately when the user asks for it via calling `key() const`. +// +// For its subclasses, it provides protected variants of the above-mentioned +// final-overridden methods. They are named with the "Impl" suffix, e.g., +// `Seek()` logic would be implemented by subclasses in `SeekImpl()`. These +// "Impl" functions are responsible for positioning `raw_key_` but not +// invoking `UpdateKey()`. +template <class TValue> +class BlockIter : public InternalIteratorBase<TValue> { + public: + // Makes Valid() return false, status() return `s`, and Seek()/Prev()/etc do + // nothing. Calls cleanup functions. + virtual void Invalidate(const Status& s) { + // Assert that the BlockIter is never deleted while Pinning is Enabled. + assert(!pinned_iters_mgr_ || !pinned_iters_mgr_->PinningEnabled()); + + data_ = nullptr; + current_ = restarts_; + status_ = s; + + // Call cleanup callbacks. + Cleanable::Reset(); + } + + bool Valid() const override { return current_ < restarts_; } + + virtual void SeekToFirst() override final { + SeekToFirstImpl(); + UpdateKey(); + } + + virtual void SeekToLast() override final { + SeekToLastImpl(); + UpdateKey(); + } + + virtual void Seek(const Slice& target) override final { + SeekImpl(target); + UpdateKey(); + } + + virtual void SeekForPrev(const Slice& target) override final { + SeekForPrevImpl(target); + UpdateKey(); + } + + virtual void Next() override final { + NextImpl(); + UpdateKey(); + } + + virtual bool NextAndGetResult(IterateResult* result) override final { + // This does not need to call `UpdateKey()` as the parent class only has + // access to the `UpdateKey()`-invoking functions. + return InternalIteratorBase<TValue>::NextAndGetResult(result); + } + + virtual void Prev() override final { + PrevImpl(); + UpdateKey(); + } + + Status status() const override { return status_; } + Slice key() const override { + assert(Valid()); + return key_; + } + +#ifndef NDEBUG + ~BlockIter() override { + // Assert that the BlockIter is never deleted while Pinning is Enabled. + assert(!pinned_iters_mgr_ || + (pinned_iters_mgr_ && !pinned_iters_mgr_->PinningEnabled())); + status_.PermitUncheckedError(); + } + void SetPinnedItersMgr(PinnedIteratorsManager* pinned_iters_mgr) override { + pinned_iters_mgr_ = pinned_iters_mgr; + } + PinnedIteratorsManager* pinned_iters_mgr_ = nullptr; +#endif + + bool IsKeyPinned() const override { + return block_contents_pinned_ && key_pinned_; + } + + bool IsValuePinned() const override { return block_contents_pinned_; } + + size_t TEST_CurrentEntrySize() { return NextEntryOffset() - current_; } + + uint32_t ValueOffset() const { + return static_cast<uint32_t>(value_.data() - data_); + } + + void SetCacheHandle(Cache::Handle* handle) { cache_handle_ = handle; } + + Cache::Handle* cache_handle() { return cache_handle_; } + + protected: + std::unique_ptr<InternalKeyComparator> icmp_; + const char* data_; // underlying block contents + uint32_t num_restarts_; // Number of uint32_t entries in restart array + + // Index of restart block in which current_ or current_-1 falls + uint32_t restart_index_; + uint32_t restarts_; // Offset of restart array (list of fixed32) + // current_ is offset in data_ of current entry. >= restarts_ if !Valid + uint32_t current_; + // Raw key from block. + IterKey raw_key_; + // Buffer for key data when global seqno assignment is enabled. + IterKey key_buf_; + Slice value_; + Status status_; + // Key to be exposed to users. + Slice key_; + bool key_pinned_; + // Whether the block data is guaranteed to outlive this iterator, and + // as long as the cleanup functions are transferred to another class, + // e.g. PinnableSlice, the pointer to the bytes will still be valid. + bool block_contents_pinned_; + SequenceNumber global_seqno_; + + virtual void SeekToFirstImpl() = 0; + virtual void SeekToLastImpl() = 0; + virtual void SeekImpl(const Slice& target) = 0; + virtual void SeekForPrevImpl(const Slice& target) = 0; + virtual void NextImpl() = 0; + + virtual void PrevImpl() = 0; + + template <typename DecodeEntryFunc> + inline bool ParseNextKey(bool* is_shared); + + void InitializeBase(const Comparator* raw_ucmp, const char* data, + uint32_t restarts, uint32_t num_restarts, + SequenceNumber global_seqno, bool block_contents_pinned) { + assert(data_ == nullptr); // Ensure it is called only once + assert(num_restarts > 0); // Ensure the param is valid + + icmp_ = std::make_unique<InternalKeyComparator>(raw_ucmp); + data_ = data; + restarts_ = restarts; + num_restarts_ = num_restarts; + current_ = restarts_; + restart_index_ = num_restarts_; + global_seqno_ = global_seqno; + block_contents_pinned_ = block_contents_pinned; + cache_handle_ = nullptr; + } + + // Must be called every time a key is found that needs to be returned to user, + // and may be called when no key is found (as a no-op). Updates `key_`, + // `key_buf_`, and `key_pinned_` with info about the found key. + void UpdateKey() { + key_buf_.Clear(); + if (!Valid()) { + return; + } + if (raw_key_.IsUserKey()) { + assert(global_seqno_ == kDisableGlobalSequenceNumber); + key_ = raw_key_.GetUserKey(); + key_pinned_ = raw_key_.IsKeyPinned(); + } else if (global_seqno_ == kDisableGlobalSequenceNumber) { + key_ = raw_key_.GetInternalKey(); + key_pinned_ = raw_key_.IsKeyPinned(); + } else { + key_buf_.SetInternalKey(raw_key_.GetUserKey(), global_seqno_, + ExtractValueType(raw_key_.GetInternalKey())); + key_ = key_buf_.GetInternalKey(); + key_pinned_ = false; + } + } + + // Returns the result of `Comparator::Compare()`, where the appropriate + // comparator is used for the block contents, the LHS argument is the current + // key with global seqno applied, and the RHS argument is `other`. + int CompareCurrentKey(const Slice& other) { + if (raw_key_.IsUserKey()) { + assert(global_seqno_ == kDisableGlobalSequenceNumber); + return icmp_->user_comparator()->Compare(raw_key_.GetUserKey(), other); + } else if (global_seqno_ == kDisableGlobalSequenceNumber) { + return icmp_->Compare(raw_key_.GetInternalKey(), other); + } + return icmp_->Compare(raw_key_.GetInternalKey(), global_seqno_, other, + kDisableGlobalSequenceNumber); + } + + private: + // Store the cache handle, if the block is cached. We need this since the + // only other place the handle is stored is as an argument to the Cleanable + // function callback, which is hard to retrieve. When multiple value + // PinnableSlices reference the block, they need the cache handle in order + // to bump up the ref count + Cache::Handle* cache_handle_; + + public: + // Return the offset in data_ just past the end of the current entry. + inline uint32_t NextEntryOffset() const { + // NOTE: We don't support blocks bigger than 2GB + return static_cast<uint32_t>((value_.data() + value_.size()) - data_); + } + + uint32_t GetRestartPoint(uint32_t index) { + assert(index < num_restarts_); + return DecodeFixed32(data_ + restarts_ + index * sizeof(uint32_t)); + } + + void SeekToRestartPoint(uint32_t index) { + raw_key_.Clear(); + restart_index_ = index; + // current_ will be fixed by ParseNextKey(); + + // ParseNextKey() starts at the end of value_, so set value_ accordingly + uint32_t offset = GetRestartPoint(index); + value_ = Slice(data_ + offset, 0); + } + + void CorruptionError(); + + protected: + template <typename DecodeKeyFunc> + inline bool BinarySeek(const Slice& target, uint32_t* index, + bool* is_index_key_result); + + void FindKeyAfterBinarySeek(const Slice& target, uint32_t index, + bool is_index_key_result); +}; + +class DataBlockIter final : public BlockIter<Slice> { + public: + DataBlockIter() + : BlockIter(), read_amp_bitmap_(nullptr), last_bitmap_offset_(0) {} + DataBlockIter(const Comparator* raw_ucmp, const char* data, uint32_t restarts, + uint32_t num_restarts, SequenceNumber global_seqno, + BlockReadAmpBitmap* read_amp_bitmap, bool block_contents_pinned, + DataBlockHashIndex* data_block_hash_index) + : DataBlockIter() { + Initialize(raw_ucmp, data, restarts, num_restarts, global_seqno, + read_amp_bitmap, block_contents_pinned, data_block_hash_index); + } + void Initialize(const Comparator* raw_ucmp, const char* data, + uint32_t restarts, uint32_t num_restarts, + SequenceNumber global_seqno, + BlockReadAmpBitmap* read_amp_bitmap, + bool block_contents_pinned, + DataBlockHashIndex* data_block_hash_index) { + InitializeBase(raw_ucmp, data, restarts, num_restarts, global_seqno, + block_contents_pinned); + raw_key_.SetIsUserKey(false); + read_amp_bitmap_ = read_amp_bitmap; + last_bitmap_offset_ = current_ + 1; + data_block_hash_index_ = data_block_hash_index; + } + + Slice value() const override { + assert(Valid()); + if (read_amp_bitmap_ && current_ < restarts_ && + current_ != last_bitmap_offset_) { + read_amp_bitmap_->Mark(current_ /* current entry offset */, + NextEntryOffset() - 1); + last_bitmap_offset_ = current_; + } + return value_; + } + + inline bool SeekForGet(const Slice& target) { + if (!data_block_hash_index_) { + SeekImpl(target); + UpdateKey(); + return true; + } + bool res = SeekForGetImpl(target); + UpdateKey(); + return res; + } + + void Invalidate(const Status& s) override { + BlockIter::Invalidate(s); + // Clear prev entries cache. + prev_entries_keys_buff_.clear(); + prev_entries_.clear(); + prev_entries_idx_ = -1; + } + + protected: + friend Block; + inline bool ParseNextDataKey(bool* is_shared); + void SeekToFirstImpl() override; + void SeekToLastImpl() override; + void SeekImpl(const Slice& target) override; + void SeekForPrevImpl(const Slice& target) override; + void NextImpl() override; + void PrevImpl() override; + + private: + // read-amp bitmap + BlockReadAmpBitmap* read_amp_bitmap_; + // last `current_` value we report to read-amp bitmp + mutable uint32_t last_bitmap_offset_; + struct CachedPrevEntry { + explicit CachedPrevEntry(uint32_t _offset, const char* _key_ptr, + size_t _key_offset, size_t _key_size, Slice _value) + : offset(_offset), + key_ptr(_key_ptr), + key_offset(_key_offset), + key_size(_key_size), + value(_value) {} + + // offset of entry in block + uint32_t offset; + // Pointer to key data in block (nullptr if key is delta-encoded) + const char* key_ptr; + // offset of key in prev_entries_keys_buff_ (0 if key_ptr is not nullptr) + size_t key_offset; + // size of key + size_t key_size; + // value slice pointing to data in block + Slice value; + }; + std::string prev_entries_keys_buff_; + std::vector<CachedPrevEntry> prev_entries_; + int32_t prev_entries_idx_ = -1; + + DataBlockHashIndex* data_block_hash_index_; + + bool SeekForGetImpl(const Slice& target); +}; + +// Iterator over MetaBlocks. MetaBlocks are similar to Data Blocks and +// are used to store Properties associated with table. +// Meta blocks always store user keys (no sequence number) and always +// use the BytewiseComparator. Additionally, MetaBlock accesses are +// not recorded in the Statistics or for Read-Amplification. +class MetaBlockIter final : public BlockIter<Slice> { + public: + MetaBlockIter() : BlockIter() { raw_key_.SetIsUserKey(true); } + void Initialize(const char* data, uint32_t restarts, uint32_t num_restarts, + bool block_contents_pinned) { + // Initializes the iterator with a BytewiseComparator and + // the raw key being a user key. + InitializeBase(BytewiseComparator(), data, restarts, num_restarts, + kDisableGlobalSequenceNumber, block_contents_pinned); + raw_key_.SetIsUserKey(true); + } + + Slice value() const override { + assert(Valid()); + return value_; + } + + protected: + void SeekToFirstImpl() override; + void SeekToLastImpl() override; + void SeekImpl(const Slice& target) override; + void SeekForPrevImpl(const Slice& target) override; + void NextImpl() override; + void PrevImpl() override; +}; + +class IndexBlockIter final : public BlockIter<IndexValue> { + public: + IndexBlockIter() : BlockIter(), prefix_index_(nullptr) {} + + // key_includes_seq, default true, means that the keys are in internal key + // format. + // value_is_full, default true, means that no delta encoding is + // applied to values. + void Initialize(const Comparator* raw_ucmp, const char* data, + uint32_t restarts, uint32_t num_restarts, + SequenceNumber global_seqno, BlockPrefixIndex* prefix_index, + bool have_first_key, bool key_includes_seq, + bool value_is_full, bool block_contents_pinned) { + InitializeBase(raw_ucmp, data, restarts, num_restarts, + kDisableGlobalSequenceNumber, block_contents_pinned); + raw_key_.SetIsUserKey(!key_includes_seq); + prefix_index_ = prefix_index; + value_delta_encoded_ = !value_is_full; + have_first_key_ = have_first_key; + if (have_first_key_ && global_seqno != kDisableGlobalSequenceNumber) { + global_seqno_state_.reset(new GlobalSeqnoState(global_seqno)); + } else { + global_seqno_state_.reset(); + } + } + + Slice user_key() const override { + assert(Valid()); + return raw_key_.GetUserKey(); + } + + IndexValue value() const override { + assert(Valid()); + if (value_delta_encoded_ || global_seqno_state_ != nullptr) { + return decoded_value_; + } else { + IndexValue entry; + Slice v = value_; + Status decode_s __attribute__((__unused__)) = + entry.DecodeFrom(&v, have_first_key_, nullptr); + assert(decode_s.ok()); + return entry; + } + } + + bool IsValuePinned() const override { + return global_seqno_state_ != nullptr ? false : BlockIter::IsValuePinned(); + } + + protected: + // IndexBlockIter follows a different contract for prefix iterator + // from data iterators. + // If prefix of the seek key `target` exists in the file, it must + // return the same result as total order seek. + // If the prefix of `target` doesn't exist in the file, it can either + // return the result of total order seek, or set both of Valid() = false + // and status() = NotFound(). + void SeekImpl(const Slice& target) override; + + void SeekForPrevImpl(const Slice&) override { + assert(false); + current_ = restarts_; + restart_index_ = num_restarts_; + status_ = Status::InvalidArgument( + "RocksDB internal error: should never call SeekForPrev() on index " + "blocks"); + raw_key_.Clear(); + value_.clear(); + } + + void PrevImpl() override; + + void NextImpl() override; + + void SeekToFirstImpl() override; + + void SeekToLastImpl() override; + + private: + bool value_delta_encoded_; + bool have_first_key_; // value includes first_internal_key + BlockPrefixIndex* prefix_index_; + // Whether the value is delta encoded. In that case the value is assumed to be + // BlockHandle. The first value in each restart interval is the full encoded + // BlockHandle; the restart of encoded size part of the BlockHandle. The + // offset of delta encoded BlockHandles is computed by adding the size of + // previous delta encoded values in the same restart interval to the offset of + // the first value in that restart interval. + IndexValue decoded_value_; + + // When sequence number overwriting is enabled, this struct contains the seqno + // to overwrite with, and current first_internal_key with overwritten seqno. + // This is rarely used, so we put it behind a pointer and only allocate when + // needed. + struct GlobalSeqnoState { + // First internal key according to current index entry, but with sequence + // number overwritten to global_seqno. + IterKey first_internal_key; + SequenceNumber global_seqno; + + explicit GlobalSeqnoState(SequenceNumber seqno) : global_seqno(seqno) {} + }; + + std::unique_ptr<GlobalSeqnoState> global_seqno_state_; + + // Set *prefix_may_exist to false if no key possibly share the same prefix + // as `target`. If not set, the result position should be the same as total + // order Seek. + bool PrefixSeek(const Slice& target, uint32_t* index, bool* prefix_may_exist); + // Set *prefix_may_exist to false if no key can possibly share the same + // prefix as `target`. If not set, the result position should be the same + // as total order seek. + bool BinaryBlockIndexSeek(const Slice& target, uint32_t* block_ids, + uint32_t left, uint32_t right, uint32_t* index, + bool* prefix_may_exist); + inline int CompareBlockKey(uint32_t block_index, const Slice& target); + + inline bool ParseNextIndexKey(); + + // When value_delta_encoded_ is enabled it decodes the value which is assumed + // to be BlockHandle and put it to decoded_value_ + inline void DecodeCurrentValue(bool is_shared); +}; + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/block_based_table_builder.cc b/src/rocksdb/table/block_based/block_based_table_builder.cc new file mode 100644 index 000000000..fed69af07 --- /dev/null +++ b/src/rocksdb/table/block_based/block_based_table_builder.cc @@ -0,0 +1,2096 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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/block_based/block_based_table_builder.h" + +#include <assert.h> +#include <stdio.h> + +#include <atomic> +#include <list> +#include <map> +#include <memory> +#include <numeric> +#include <string> +#include <unordered_map> +#include <utility> + +#include "cache/cache_entry_roles.h" +#include "cache/cache_helpers.h" +#include "cache/cache_key.h" +#include "cache/cache_reservation_manager.h" +#include "db/dbformat.h" +#include "index_builder.h" +#include "logging/logging.h" +#include "memory/memory_allocator.h" +#include "rocksdb/cache.h" +#include "rocksdb/comparator.h" +#include "rocksdb/env.h" +#include "rocksdb/filter_policy.h" +#include "rocksdb/flush_block_policy.h" +#include "rocksdb/merge_operator.h" +#include "rocksdb/table.h" +#include "rocksdb/types.h" +#include "table/block_based/block.h" +#include "table/block_based/block_based_table_factory.h" +#include "table/block_based/block_based_table_reader.h" +#include "table/block_based/block_builder.h" +#include "table/block_based/block_like_traits.h" +#include "table/block_based/filter_block.h" +#include "table/block_based/filter_policy_internal.h" +#include "table/block_based/full_filter_block.h" +#include "table/block_based/partitioned_filter_block.h" +#include "table/format.h" +#include "table/meta_blocks.h" +#include "table/table_builder.h" +#include "util/coding.h" +#include "util/compression.h" +#include "util/stop_watch.h" +#include "util/string_util.h" +#include "util/work_queue.h" + +namespace ROCKSDB_NAMESPACE { + +extern const std::string kHashIndexPrefixesBlock; +extern const std::string kHashIndexPrefixesMetadataBlock; + +// Without anonymous namespace here, we fail the warning -Wmissing-prototypes +namespace { + +constexpr size_t kBlockTrailerSize = BlockBasedTable::kBlockTrailerSize; + +// Create a filter block builder based on its type. +FilterBlockBuilder* CreateFilterBlockBuilder( + const ImmutableCFOptions& /*opt*/, const MutableCFOptions& mopt, + const FilterBuildingContext& context, + const bool use_delta_encoding_for_index_values, + PartitionedIndexBuilder* const p_index_builder) { + const BlockBasedTableOptions& table_opt = context.table_options; + assert(table_opt.filter_policy); // precondition + + FilterBitsBuilder* filter_bits_builder = + BloomFilterPolicy::GetBuilderFromContext(context); + if (filter_bits_builder == nullptr) { + return nullptr; + } else { + if (table_opt.partition_filters) { + assert(p_index_builder != nullptr); + // Since after partition cut request from filter builder it takes time + // until index builder actully cuts the partition, until the end of a + // data block potentially with many keys, we take the lower bound as + // partition size. + assert(table_opt.block_size_deviation <= 100); + auto partition_size = + static_cast<uint32_t>(((table_opt.metadata_block_size * + (100 - table_opt.block_size_deviation)) + + 99) / + 100); + partition_size = std::max(partition_size, static_cast<uint32_t>(1)); + return new PartitionedFilterBlockBuilder( + mopt.prefix_extractor.get(), table_opt.whole_key_filtering, + filter_bits_builder, table_opt.index_block_restart_interval, + use_delta_encoding_for_index_values, p_index_builder, partition_size); + } else { + return new FullFilterBlockBuilder(mopt.prefix_extractor.get(), + table_opt.whole_key_filtering, + filter_bits_builder); + } + } +} + +bool GoodCompressionRatio(size_t compressed_size, size_t uncomp_size) { + // Check to see if compressed less than 12.5% + return compressed_size < uncomp_size - (uncomp_size / 8u); +} + +} // namespace + +// format_version is the block format as defined in include/rocksdb/table.h +Slice CompressBlock(const Slice& uncompressed_data, const CompressionInfo& info, + CompressionType* type, uint32_t format_version, + bool do_sample, std::string* compressed_output, + std::string* sampled_output_fast, + std::string* sampled_output_slow) { + assert(type); + assert(compressed_output); + assert(compressed_output->empty()); + + // If requested, we sample one in every N block with a + // fast and slow compression algorithm and report the stats. + // The users can use these stats to decide if it is worthwhile + // enabling compression and they also get a hint about which + // compression algorithm wil be beneficial. + if (do_sample && info.SampleForCompression() && + Random::GetTLSInstance()->OneIn( + static_cast<int>(info.SampleForCompression()))) { + // Sampling with a fast compression algorithm + if (sampled_output_fast && (LZ4_Supported() || Snappy_Supported())) { + CompressionType c = + LZ4_Supported() ? kLZ4Compression : kSnappyCompression; + CompressionContext context(c); + CompressionOptions options; + CompressionInfo info_tmp(options, context, + CompressionDict::GetEmptyDict(), c, + info.SampleForCompression()); + + CompressData(uncompressed_data, info_tmp, + GetCompressFormatForVersion(format_version), + sampled_output_fast); + } + + // Sampling with a slow but high-compression algorithm + if (sampled_output_slow && (ZSTD_Supported() || Zlib_Supported())) { + CompressionType c = ZSTD_Supported() ? kZSTD : kZlibCompression; + CompressionContext context(c); + CompressionOptions options; + CompressionInfo info_tmp(options, context, + CompressionDict::GetEmptyDict(), c, + info.SampleForCompression()); + + CompressData(uncompressed_data, info_tmp, + GetCompressFormatForVersion(format_version), + sampled_output_slow); + } + } + + if (info.type() == kNoCompression) { + *type = kNoCompression; + return uncompressed_data; + } + + // Actually compress the data; if the compression method is not supported, + // or the compression fails etc., just fall back to uncompressed + if (!CompressData(uncompressed_data, info, + GetCompressFormatForVersion(format_version), + compressed_output)) { + *type = kNoCompression; + return uncompressed_data; + } + + // Check the compression ratio; if it's not good enough, just fall back to + // uncompressed + if (!GoodCompressionRatio(compressed_output->size(), + uncompressed_data.size())) { + *type = kNoCompression; + return uncompressed_data; + } + + *type = info.type(); + return *compressed_output; +} + +// kBlockBasedTableMagicNumber was picked by running +// echo rocksdb.table.block_based | sha1sum +// and taking the leading 64 bits. +// Please note that kBlockBasedTableMagicNumber may also be accessed by other +// .cc files +// for that reason we declare it extern in the header but to get the space +// allocated +// it must be not extern in one place. +const uint64_t kBlockBasedTableMagicNumber = 0x88e241b785f4cff7ull; +// We also support reading and writing legacy block based table format (for +// backwards compatibility) +const uint64_t kLegacyBlockBasedTableMagicNumber = 0xdb4775248b80fb57ull; + +// A collector that collects properties of interest to block-based table. +// For now this class looks heavy-weight since we only write one additional +// property. +// But in the foreseeable future, we will add more and more properties that are +// specific to block-based table. +class BlockBasedTableBuilder::BlockBasedTablePropertiesCollector + : public IntTblPropCollector { + public: + explicit BlockBasedTablePropertiesCollector( + BlockBasedTableOptions::IndexType index_type, bool whole_key_filtering, + bool prefix_filtering) + : index_type_(index_type), + whole_key_filtering_(whole_key_filtering), + prefix_filtering_(prefix_filtering) {} + + Status InternalAdd(const Slice& /*key*/, const Slice& /*value*/, + uint64_t /*file_size*/) override { + // Intentionally left blank. Have no interest in collecting stats for + // individual key/value pairs. + return Status::OK(); + } + + virtual void BlockAdd(uint64_t /* block_uncomp_bytes */, + uint64_t /* block_compressed_bytes_fast */, + uint64_t /* block_compressed_bytes_slow */) override { + // Intentionally left blank. No interest in collecting stats for + // blocks. + return; + } + + Status Finish(UserCollectedProperties* properties) override { + std::string val; + PutFixed32(&val, static_cast<uint32_t>(index_type_)); + properties->insert({BlockBasedTablePropertyNames::kIndexType, val}); + properties->insert({BlockBasedTablePropertyNames::kWholeKeyFiltering, + whole_key_filtering_ ? kPropTrue : kPropFalse}); + properties->insert({BlockBasedTablePropertyNames::kPrefixFiltering, + prefix_filtering_ ? kPropTrue : kPropFalse}); + return Status::OK(); + } + + // The name of the properties collector can be used for debugging purpose. + const char* Name() const override { + return "BlockBasedTablePropertiesCollector"; + } + + UserCollectedProperties GetReadableProperties() const override { + // Intentionally left blank. + return UserCollectedProperties(); + } + + private: + BlockBasedTableOptions::IndexType index_type_; + bool whole_key_filtering_; + bool prefix_filtering_; +}; + +struct BlockBasedTableBuilder::Rep { + const ImmutableOptions ioptions; + const MutableCFOptions moptions; + const BlockBasedTableOptions table_options; + const InternalKeyComparator& internal_comparator; + WritableFileWriter* file; + std::atomic<uint64_t> offset; + size_t alignment; + BlockBuilder data_block; + // Buffers uncompressed data blocks to replay later. Needed when + // compression dictionary is enabled so we can finalize the dictionary before + // compressing any data blocks. + std::vector<std::string> data_block_buffers; + BlockBuilder range_del_block; + + InternalKeySliceTransform internal_prefix_transform; + std::unique_ptr<IndexBuilder> index_builder; + PartitionedIndexBuilder* p_index_builder_ = nullptr; + + std::string last_key; + const Slice* first_key_in_next_block = nullptr; + CompressionType compression_type; + uint64_t sample_for_compression; + std::atomic<uint64_t> compressible_input_data_bytes; + std::atomic<uint64_t> uncompressible_input_data_bytes; + std::atomic<uint64_t> sampled_input_data_bytes; + std::atomic<uint64_t> sampled_output_slow_data_bytes; + std::atomic<uint64_t> sampled_output_fast_data_bytes; + CompressionOptions compression_opts; + std::unique_ptr<CompressionDict> compression_dict; + std::vector<std::unique_ptr<CompressionContext>> compression_ctxs; + std::vector<std::unique_ptr<UncompressionContext>> verify_ctxs; + std::unique_ptr<UncompressionDict> verify_dict; + + size_t data_begin_offset = 0; + + TableProperties props; + + // States of the builder. + // + // - `kBuffered`: This is the initial state where zero or more data blocks are + // accumulated uncompressed in-memory. From this state, call + // `EnterUnbuffered()` to finalize the compression dictionary if enabled, + // compress/write out any buffered blocks, and proceed to the `kUnbuffered` + // state. + // + // - `kUnbuffered`: This is the state when compression dictionary is finalized + // either because it wasn't enabled in the first place or it's been created + // from sampling previously buffered data. In this state, blocks are simply + // compressed/written out as they fill up. From this state, call `Finish()` + // to complete the file (write meta-blocks, etc.), or `Abandon()` to delete + // the partially created file. + // + // - `kClosed`: This indicates either `Finish()` or `Abandon()` has been + // called, so the table builder is no longer usable. We must be in this + // state by the time the destructor runs. + enum class State { + kBuffered, + kUnbuffered, + kClosed, + }; + State state; + // `kBuffered` state is allowed only as long as the buffering of uncompressed + // data blocks (see `data_block_buffers`) does not exceed `buffer_limit`. + uint64_t buffer_limit; + std::shared_ptr<CacheReservationManager> + compression_dict_buffer_cache_res_mgr; + const bool use_delta_encoding_for_index_values; + std::unique_ptr<FilterBlockBuilder> filter_builder; + OffsetableCacheKey base_cache_key; + const TableFileCreationReason reason; + + BlockHandle pending_handle; // Handle to add to index block + + std::string compressed_output; + std::unique_ptr<FlushBlockPolicy> flush_block_policy; + + std::vector<std::unique_ptr<IntTblPropCollector>> table_properties_collectors; + + std::unique_ptr<ParallelCompressionRep> pc_rep; + + uint64_t get_offset() { return offset.load(std::memory_order_relaxed); } + void set_offset(uint64_t o) { offset.store(o, std::memory_order_relaxed); } + + bool IsParallelCompressionEnabled() const { + return compression_opts.parallel_threads > 1; + } + + Status GetStatus() { + // We need to make modifications of status visible when status_ok is set + // to false, and this is ensured by status_mutex, so no special memory + // order for status_ok is required. + if (status_ok.load(std::memory_order_relaxed)) { + return Status::OK(); + } else { + return CopyStatus(); + } + } + + Status CopyStatus() { + std::lock_guard<std::mutex> lock(status_mutex); + return status; + } + + IOStatus GetIOStatus() { + // We need to make modifications of io_status visible when status_ok is set + // to false, and this is ensured by io_status_mutex, so no special memory + // order for io_status_ok is required. + if (io_status_ok.load(std::memory_order_relaxed)) { + return IOStatus::OK(); + } else { + return CopyIOStatus(); + } + } + + IOStatus CopyIOStatus() { + std::lock_guard<std::mutex> lock(io_status_mutex); + return io_status; + } + + // Never erase an existing status that is not OK. + void SetStatus(Status s) { + if (!s.ok() && status_ok.load(std::memory_order_relaxed)) { + // Locking is an overkill for non compression_opts.parallel_threads + // case but since it's unlikely that s is not OK, we take this cost + // to be simplicity. + std::lock_guard<std::mutex> lock(status_mutex); + status = s; + status_ok.store(false, std::memory_order_relaxed); + } + } + + // Never erase an existing I/O status that is not OK. + // Calling this will also SetStatus(ios) + void SetIOStatus(IOStatus ios) { + if (!ios.ok() && io_status_ok.load(std::memory_order_relaxed)) { + // Locking is an overkill for non compression_opts.parallel_threads + // case but since it's unlikely that s is not OK, we take this cost + // to be simplicity. + std::lock_guard<std::mutex> lock(io_status_mutex); + io_status = ios; + io_status_ok.store(false, std::memory_order_relaxed); + } + SetStatus(ios); + } + + Rep(const BlockBasedTableOptions& table_opt, const TableBuilderOptions& tbo, + WritableFileWriter* f) + : ioptions(tbo.ioptions), + moptions(tbo.moptions), + table_options(table_opt), + internal_comparator(tbo.internal_comparator), + file(f), + offset(0), + alignment(table_options.block_align + ? std::min(static_cast<size_t>(table_options.block_size), + kDefaultPageSize) + : 0), + data_block(table_options.block_restart_interval, + table_options.use_delta_encoding, + false /* use_value_delta_encoding */, + tbo.internal_comparator.user_comparator() + ->CanKeysWithDifferentByteContentsBeEqual() + ? BlockBasedTableOptions::kDataBlockBinarySearch + : table_options.data_block_index_type, + table_options.data_block_hash_table_util_ratio), + range_del_block(1 /* block_restart_interval */), + internal_prefix_transform(tbo.moptions.prefix_extractor.get()), + compression_type(tbo.compression_type), + sample_for_compression(tbo.moptions.sample_for_compression), + compressible_input_data_bytes(0), + uncompressible_input_data_bytes(0), + sampled_input_data_bytes(0), + sampled_output_slow_data_bytes(0), + sampled_output_fast_data_bytes(0), + compression_opts(tbo.compression_opts), + compression_dict(), + compression_ctxs(tbo.compression_opts.parallel_threads), + verify_ctxs(tbo.compression_opts.parallel_threads), + verify_dict(), + state((tbo.compression_opts.max_dict_bytes > 0) ? State::kBuffered + : State::kUnbuffered), + use_delta_encoding_for_index_values(table_opt.format_version >= 4 && + !table_opt.block_align), + reason(tbo.reason), + flush_block_policy( + table_options.flush_block_policy_factory->NewFlushBlockPolicy( + table_options, data_block)), + status_ok(true), + io_status_ok(true) { + if (tbo.target_file_size == 0) { + buffer_limit = compression_opts.max_dict_buffer_bytes; + } else if (compression_opts.max_dict_buffer_bytes == 0) { + buffer_limit = tbo.target_file_size; + } else { + buffer_limit = std::min(tbo.target_file_size, + compression_opts.max_dict_buffer_bytes); + } + + const auto compress_dict_build_buffer_charged = + table_options.cache_usage_options.options_overrides + .at(CacheEntryRole::kCompressionDictionaryBuildingBuffer) + .charged; + if (table_options.block_cache && + (compress_dict_build_buffer_charged == + CacheEntryRoleOptions::Decision::kEnabled || + compress_dict_build_buffer_charged == + CacheEntryRoleOptions::Decision::kFallback)) { + compression_dict_buffer_cache_res_mgr = + std::make_shared<CacheReservationManagerImpl< + CacheEntryRole::kCompressionDictionaryBuildingBuffer>>( + table_options.block_cache); + } else { + compression_dict_buffer_cache_res_mgr = nullptr; + } + + for (uint32_t i = 0; i < compression_opts.parallel_threads; i++) { + compression_ctxs[i].reset(new CompressionContext(compression_type)); + } + if (table_options.index_type == + BlockBasedTableOptions::kTwoLevelIndexSearch) { + p_index_builder_ = PartitionedIndexBuilder::CreateIndexBuilder( + &internal_comparator, use_delta_encoding_for_index_values, + table_options); + index_builder.reset(p_index_builder_); + } else { + index_builder.reset(IndexBuilder::CreateIndexBuilder( + table_options.index_type, &internal_comparator, + &this->internal_prefix_transform, use_delta_encoding_for_index_values, + table_options)); + } + if (ioptions.optimize_filters_for_hits && tbo.is_bottommost) { + // Apply optimize_filters_for_hits setting here when applicable by + // skipping filter generation + filter_builder.reset(); + } else if (tbo.skip_filters) { + // For SstFileWriter skip_filters + filter_builder.reset(); + } else if (!table_options.filter_policy) { + // Null filter_policy -> no filter + filter_builder.reset(); + } else { + FilterBuildingContext filter_context(table_options); + + filter_context.info_log = ioptions.logger; + filter_context.column_family_name = tbo.column_family_name; + filter_context.reason = reason; + + // Only populate other fields if known to be in LSM rather than + // generating external SST file + if (reason != TableFileCreationReason::kMisc) { + filter_context.compaction_style = ioptions.compaction_style; + filter_context.num_levels = ioptions.num_levels; + filter_context.level_at_creation = tbo.level_at_creation; + filter_context.is_bottommost = tbo.is_bottommost; + assert(filter_context.level_at_creation < filter_context.num_levels); + } + + filter_builder.reset(CreateFilterBlockBuilder( + ioptions, moptions, filter_context, + use_delta_encoding_for_index_values, p_index_builder_)); + } + + assert(tbo.int_tbl_prop_collector_factories); + for (auto& factory : *tbo.int_tbl_prop_collector_factories) { + assert(factory); + + table_properties_collectors.emplace_back( + factory->CreateIntTblPropCollector(tbo.column_family_id, + tbo.level_at_creation)); + } + table_properties_collectors.emplace_back( + new BlockBasedTablePropertiesCollector( + table_options.index_type, table_options.whole_key_filtering, + moptions.prefix_extractor != nullptr)); + const Comparator* ucmp = tbo.internal_comparator.user_comparator(); + assert(ucmp); + if (ucmp->timestamp_size() > 0) { + table_properties_collectors.emplace_back( + new TimestampTablePropertiesCollector(ucmp)); + } + if (table_options.verify_compression) { + for (uint32_t i = 0; i < compression_opts.parallel_threads; i++) { + verify_ctxs[i].reset(new UncompressionContext(compression_type)); + } + } + + // These are only needed for populating table properties + props.column_family_id = tbo.column_family_id; + props.column_family_name = tbo.column_family_name; + props.oldest_key_time = tbo.oldest_key_time; + props.file_creation_time = tbo.file_creation_time; + props.orig_file_number = tbo.cur_file_num; + props.db_id = tbo.db_id; + props.db_session_id = tbo.db_session_id; + props.db_host_id = ioptions.db_host_id; + if (!ReifyDbHostIdProperty(ioptions.env, &props.db_host_id).ok()) { + ROCKS_LOG_INFO(ioptions.logger, "db_host_id property will not be set"); + } + } + + Rep(const Rep&) = delete; + Rep& operator=(const Rep&) = delete; + + private: + // Synchronize status & io_status accesses across threads from main thread, + // compression thread and write thread in parallel compression. + std::mutex status_mutex; + std::atomic<bool> status_ok; + Status status; + std::mutex io_status_mutex; + std::atomic<bool> io_status_ok; + IOStatus io_status; +}; + +struct BlockBasedTableBuilder::ParallelCompressionRep { + // Keys is a wrapper of vector of strings avoiding + // releasing string memories during vector clear() + // in order to save memory allocation overhead + class Keys { + public: + Keys() : keys_(kKeysInitSize), size_(0) {} + void PushBack(const Slice& key) { + if (size_ == keys_.size()) { + keys_.emplace_back(key.data(), key.size()); + } else { + keys_[size_].assign(key.data(), key.size()); + } + size_++; + } + void SwapAssign(std::vector<std::string>& keys) { + size_ = keys.size(); + std::swap(keys_, keys); + } + void Clear() { size_ = 0; } + size_t Size() { return size_; } + std::string& Back() { return keys_[size_ - 1]; } + std::string& operator[](size_t idx) { + assert(idx < size_); + return keys_[idx]; + } + + private: + const size_t kKeysInitSize = 32; + std::vector<std::string> keys_; + size_t size_; + }; + std::unique_ptr<Keys> curr_block_keys; + + class BlockRepSlot; + + // BlockRep instances are fetched from and recycled to + // block_rep_pool during parallel compression. + struct BlockRep { + Slice contents; + Slice compressed_contents; + std::unique_ptr<std::string> data; + std::unique_ptr<std::string> compressed_data; + CompressionType compression_type; + std::unique_ptr<std::string> first_key_in_next_block; + std::unique_ptr<Keys> keys; + std::unique_ptr<BlockRepSlot> slot; + Status status; + }; + // Use a vector of BlockRep as a buffer for a determined number + // of BlockRep structures. All data referenced by pointers in + // BlockRep will be freed when this vector is destructed. + using BlockRepBuffer = std::vector<BlockRep>; + BlockRepBuffer block_rep_buf; + // Use a thread-safe queue for concurrent access from block + // building thread and writer thread. + using BlockRepPool = WorkQueue<BlockRep*>; + BlockRepPool block_rep_pool; + + // Use BlockRepSlot to keep block order in write thread. + // slot_ will pass references to BlockRep + class BlockRepSlot { + public: + BlockRepSlot() : slot_(1) {} + template <typename T> + void Fill(T&& rep) { + slot_.push(std::forward<T>(rep)); + }; + void Take(BlockRep*& rep) { slot_.pop(rep); } + + private: + // slot_ will pass references to BlockRep in block_rep_buf, + // and those references are always valid before the destruction of + // block_rep_buf. + WorkQueue<BlockRep*> slot_; + }; + + // Compression queue will pass references to BlockRep in block_rep_buf, + // and those references are always valid before the destruction of + // block_rep_buf. + using CompressQueue = WorkQueue<BlockRep*>; + CompressQueue compress_queue; + std::vector<port::Thread> compress_thread_pool; + + // Write queue will pass references to BlockRep::slot in block_rep_buf, + // and those references are always valid before the corresponding + // BlockRep::slot is destructed, which is before the destruction of + // block_rep_buf. + using WriteQueue = WorkQueue<BlockRepSlot*>; + WriteQueue write_queue; + std::unique_ptr<port::Thread> write_thread; + + // Estimate output file size when parallel compression is enabled. This is + // necessary because compression & flush are no longer synchronized, + // and BlockBasedTableBuilder::FileSize() is no longer accurate. + // memory_order_relaxed suffices because accurate statistics is not required. + class FileSizeEstimator { + public: + explicit FileSizeEstimator() + : uncomp_bytes_compressed(0), + uncomp_bytes_curr_block(0), + uncomp_bytes_curr_block_set(false), + uncomp_bytes_inflight(0), + blocks_inflight(0), + curr_compression_ratio(0), + estimated_file_size(0) {} + + // Estimate file size when a block is about to be emitted to + // compression thread + void EmitBlock(uint64_t uncomp_block_size, uint64_t curr_file_size) { + uint64_t new_uncomp_bytes_inflight = + uncomp_bytes_inflight.fetch_add(uncomp_block_size, + std::memory_order_relaxed) + + uncomp_block_size; + + uint64_t new_blocks_inflight = + blocks_inflight.fetch_add(1, std::memory_order_relaxed) + 1; + + estimated_file_size.store( + curr_file_size + + static_cast<uint64_t>( + static_cast<double>(new_uncomp_bytes_inflight) * + curr_compression_ratio.load(std::memory_order_relaxed)) + + new_blocks_inflight * kBlockTrailerSize, + std::memory_order_relaxed); + } + + // Estimate file size when a block is already reaped from + // compression thread + void ReapBlock(uint64_t compressed_block_size, uint64_t curr_file_size) { + assert(uncomp_bytes_curr_block_set); + + uint64_t new_uncomp_bytes_compressed = + uncomp_bytes_compressed + uncomp_bytes_curr_block; + assert(new_uncomp_bytes_compressed > 0); + + curr_compression_ratio.store( + (curr_compression_ratio.load(std::memory_order_relaxed) * + uncomp_bytes_compressed + + compressed_block_size) / + static_cast<double>(new_uncomp_bytes_compressed), + std::memory_order_relaxed); + uncomp_bytes_compressed = new_uncomp_bytes_compressed; + + uint64_t new_uncomp_bytes_inflight = + uncomp_bytes_inflight.fetch_sub(uncomp_bytes_curr_block, + std::memory_order_relaxed) - + uncomp_bytes_curr_block; + + uint64_t new_blocks_inflight = + blocks_inflight.fetch_sub(1, std::memory_order_relaxed) - 1; + + estimated_file_size.store( + curr_file_size + + static_cast<uint64_t>( + static_cast<double>(new_uncomp_bytes_inflight) * + curr_compression_ratio.load(std::memory_order_relaxed)) + + new_blocks_inflight * kBlockTrailerSize, + std::memory_order_relaxed); + + uncomp_bytes_curr_block_set = false; + } + + void SetEstimatedFileSize(uint64_t size) { + estimated_file_size.store(size, std::memory_order_relaxed); + } + + uint64_t GetEstimatedFileSize() { + return estimated_file_size.load(std::memory_order_relaxed); + } + + void SetCurrBlockUncompSize(uint64_t size) { + uncomp_bytes_curr_block = size; + uncomp_bytes_curr_block_set = true; + } + + private: + // Input bytes compressed so far. + uint64_t uncomp_bytes_compressed; + // Size of current block being appended. + uint64_t uncomp_bytes_curr_block; + // Whether uncomp_bytes_curr_block has been set for next + // ReapBlock call. + bool uncomp_bytes_curr_block_set; + // Input bytes under compression and not appended yet. + std::atomic<uint64_t> uncomp_bytes_inflight; + // Number of blocks under compression and not appended yet. + std::atomic<uint64_t> blocks_inflight; + // Current compression ratio, maintained by BGWorkWriteMaybeCompressedBlock. + std::atomic<double> curr_compression_ratio; + // Estimated SST file size. + std::atomic<uint64_t> estimated_file_size; + }; + FileSizeEstimator file_size_estimator; + + // Facilities used for waiting first block completion. Need to Wait for + // the completion of first block compression and flush to get a non-zero + // compression ratio. + std::atomic<bool> first_block_processed; + std::condition_variable first_block_cond; + std::mutex first_block_mutex; + + explicit ParallelCompressionRep(uint32_t parallel_threads) + : curr_block_keys(new Keys()), + block_rep_buf(parallel_threads), + block_rep_pool(parallel_threads), + compress_queue(parallel_threads), + write_queue(parallel_threads), + first_block_processed(false) { + for (uint32_t i = 0; i < parallel_threads; i++) { + block_rep_buf[i].contents = Slice(); + block_rep_buf[i].compressed_contents = Slice(); + block_rep_buf[i].data.reset(new std::string()); + block_rep_buf[i].compressed_data.reset(new std::string()); + block_rep_buf[i].compression_type = CompressionType(); + block_rep_buf[i].first_key_in_next_block.reset(new std::string()); + block_rep_buf[i].keys.reset(new Keys()); + block_rep_buf[i].slot.reset(new BlockRepSlot()); + block_rep_buf[i].status = Status::OK(); + block_rep_pool.push(&block_rep_buf[i]); + } + } + + ~ParallelCompressionRep() { block_rep_pool.finish(); } + + // Make a block prepared to be emitted to compression thread + // Used in non-buffered mode + BlockRep* PrepareBlock(CompressionType compression_type, + const Slice* first_key_in_next_block, + BlockBuilder* data_block) { + BlockRep* block_rep = + PrepareBlockInternal(compression_type, first_key_in_next_block); + assert(block_rep != nullptr); + data_block->SwapAndReset(*(block_rep->data)); + block_rep->contents = *(block_rep->data); + std::swap(block_rep->keys, curr_block_keys); + curr_block_keys->Clear(); + return block_rep; + } + + // Used in EnterUnbuffered + BlockRep* PrepareBlock(CompressionType compression_type, + const Slice* first_key_in_next_block, + std::string* data_block, + std::vector<std::string>* keys) { + BlockRep* block_rep = + PrepareBlockInternal(compression_type, first_key_in_next_block); + assert(block_rep != nullptr); + std::swap(*(block_rep->data), *data_block); + block_rep->contents = *(block_rep->data); + block_rep->keys->SwapAssign(*keys); + return block_rep; + } + + // Emit a block to compression thread + void EmitBlock(BlockRep* block_rep) { + assert(block_rep != nullptr); + assert(block_rep->status.ok()); + if (!write_queue.push(block_rep->slot.get())) { + return; + } + if (!compress_queue.push(block_rep)) { + return; + } + + if (!first_block_processed.load(std::memory_order_relaxed)) { + std::unique_lock<std::mutex> lock(first_block_mutex); + first_block_cond.wait(lock, [this] { + return first_block_processed.load(std::memory_order_relaxed); + }); + } + } + + // Reap a block from compression thread + void ReapBlock(BlockRep* block_rep) { + assert(block_rep != nullptr); + block_rep->compressed_data->clear(); + block_rep_pool.push(block_rep); + + if (!first_block_processed.load(std::memory_order_relaxed)) { + std::lock_guard<std::mutex> lock(first_block_mutex); + first_block_processed.store(true, std::memory_order_relaxed); + first_block_cond.notify_one(); + } + } + + private: + BlockRep* PrepareBlockInternal(CompressionType compression_type, + const Slice* first_key_in_next_block) { + BlockRep* block_rep = nullptr; + block_rep_pool.pop(block_rep); + assert(block_rep != nullptr); + + assert(block_rep->data); + + block_rep->compression_type = compression_type; + + if (first_key_in_next_block == nullptr) { + block_rep->first_key_in_next_block.reset(nullptr); + } else { + block_rep->first_key_in_next_block->assign( + first_key_in_next_block->data(), first_key_in_next_block->size()); + } + + return block_rep; + } +}; + +BlockBasedTableBuilder::BlockBasedTableBuilder( + const BlockBasedTableOptions& table_options, const TableBuilderOptions& tbo, + WritableFileWriter* file) { + BlockBasedTableOptions sanitized_table_options(table_options); + if (sanitized_table_options.format_version == 0 && + sanitized_table_options.checksum != kCRC32c) { + ROCKS_LOG_WARN( + tbo.ioptions.logger, + "Silently converting format_version to 1 because checksum is " + "non-default"); + // silently convert format_version to 1 to keep consistent with current + // behavior + sanitized_table_options.format_version = 1; + } + + rep_ = new Rep(sanitized_table_options, tbo, file); + + TEST_SYNC_POINT_CALLBACK( + "BlockBasedTableBuilder::BlockBasedTableBuilder:PreSetupBaseCacheKey", + const_cast<TableProperties*>(&rep_->props)); + + BlockBasedTable::SetupBaseCacheKey(&rep_->props, tbo.db_session_id, + tbo.cur_file_num, &rep_->base_cache_key); + + if (rep_->IsParallelCompressionEnabled()) { + StartParallelCompression(); + } +} + +BlockBasedTableBuilder::~BlockBasedTableBuilder() { + // Catch errors where caller forgot to call Finish() + assert(rep_->state == Rep::State::kClosed); + delete rep_; +} + +void BlockBasedTableBuilder::Add(const Slice& key, const Slice& value) { + Rep* r = rep_; + assert(rep_->state != Rep::State::kClosed); + if (!ok()) return; + ValueType value_type = ExtractValueType(key); + if (IsValueType(value_type)) { +#ifndef NDEBUG + if (r->props.num_entries > r->props.num_range_deletions) { + assert(r->internal_comparator.Compare(key, Slice(r->last_key)) > 0); + } +#endif // !NDEBUG + + auto should_flush = r->flush_block_policy->Update(key, value); + if (should_flush) { + assert(!r->data_block.empty()); + r->first_key_in_next_block = &key; + Flush(); + if (r->state == Rep::State::kBuffered) { + bool exceeds_buffer_limit = + (r->buffer_limit != 0 && r->data_begin_offset > r->buffer_limit); + bool exceeds_global_block_cache_limit = false; + + // Increase cache charging for the last buffered data block + // only if the block is not going to be unbuffered immediately + // and there exists a cache reservation manager + if (!exceeds_buffer_limit && + r->compression_dict_buffer_cache_res_mgr != nullptr) { + Status s = + r->compression_dict_buffer_cache_res_mgr->UpdateCacheReservation( + r->data_begin_offset); + exceeds_global_block_cache_limit = s.IsMemoryLimit(); + } + + if (exceeds_buffer_limit || exceeds_global_block_cache_limit) { + EnterUnbuffered(); + } + } + + // Add item to index block. + // We do not emit the index entry for a block until we have seen the + // first key for the next data block. This allows us to use shorter + // keys in the index block. For example, consider a block boundary + // between the keys "the quick brown fox" and "the who". We can use + // "the r" as the key for the index block entry since it is >= all + // entries in the first block and < all entries in subsequent + // blocks. + if (ok() && r->state == Rep::State::kUnbuffered) { + if (r->IsParallelCompressionEnabled()) { + r->pc_rep->curr_block_keys->Clear(); + } else { + r->index_builder->AddIndexEntry(&r->last_key, &key, + r->pending_handle); + } + } + } + + // Note: PartitionedFilterBlockBuilder requires key being added to filter + // builder after being added to index builder. + if (r->state == Rep::State::kUnbuffered) { + if (r->IsParallelCompressionEnabled()) { + r->pc_rep->curr_block_keys->PushBack(key); + } else { + if (r->filter_builder != nullptr) { + size_t ts_sz = + r->internal_comparator.user_comparator()->timestamp_size(); + r->filter_builder->Add(ExtractUserKeyAndStripTimestamp(key, ts_sz)); + } + } + } + + r->data_block.AddWithLastKey(key, value, r->last_key); + r->last_key.assign(key.data(), key.size()); + if (r->state == Rep::State::kBuffered) { + // Buffered keys will be replayed from data_block_buffers during + // `Finish()` once compression dictionary has been finalized. + } else { + if (!r->IsParallelCompressionEnabled()) { + r->index_builder->OnKeyAdded(key); + } + } + // TODO offset passed in is not accurate for parallel compression case + NotifyCollectTableCollectorsOnAdd(key, value, r->get_offset(), + r->table_properties_collectors, + r->ioptions.logger); + + } else if (value_type == kTypeRangeDeletion) { + r->range_del_block.Add(key, value); + // TODO offset passed in is not accurate for parallel compression case + NotifyCollectTableCollectorsOnAdd(key, value, r->get_offset(), + r->table_properties_collectors, + r->ioptions.logger); + } else { + assert(false); + } + + r->props.num_entries++; + r->props.raw_key_size += key.size(); + r->props.raw_value_size += value.size(); + if (value_type == kTypeDeletion || value_type == kTypeSingleDeletion || + value_type == kTypeDeletionWithTimestamp) { + r->props.num_deletions++; + } else if (value_type == kTypeRangeDeletion) { + r->props.num_deletions++; + r->props.num_range_deletions++; + } else if (value_type == kTypeMerge) { + r->props.num_merge_operands++; + } +} + +void BlockBasedTableBuilder::Flush() { + Rep* r = rep_; + assert(rep_->state != Rep::State::kClosed); + if (!ok()) return; + if (r->data_block.empty()) return; + if (r->IsParallelCompressionEnabled() && + r->state == Rep::State::kUnbuffered) { + r->data_block.Finish(); + ParallelCompressionRep::BlockRep* block_rep = r->pc_rep->PrepareBlock( + r->compression_type, r->first_key_in_next_block, &(r->data_block)); + assert(block_rep != nullptr); + r->pc_rep->file_size_estimator.EmitBlock(block_rep->data->size(), + r->get_offset()); + r->pc_rep->EmitBlock(block_rep); + } else { + WriteBlock(&r->data_block, &r->pending_handle, BlockType::kData); + } +} + +void BlockBasedTableBuilder::WriteBlock(BlockBuilder* block, + BlockHandle* handle, + BlockType block_type) { + block->Finish(); + std::string uncompressed_block_data; + uncompressed_block_data.reserve(rep_->table_options.block_size); + block->SwapAndReset(uncompressed_block_data); + if (rep_->state == Rep::State::kBuffered) { + assert(block_type == BlockType::kData); + rep_->data_block_buffers.emplace_back(std::move(uncompressed_block_data)); + rep_->data_begin_offset += rep_->data_block_buffers.back().size(); + return; + } + WriteBlock(uncompressed_block_data, handle, block_type); +} + +void BlockBasedTableBuilder::WriteBlock(const Slice& uncompressed_block_data, + BlockHandle* handle, + BlockType block_type) { + Rep* r = rep_; + assert(r->state == Rep::State::kUnbuffered); + Slice block_contents; + CompressionType type; + Status compress_status; + bool is_data_block = block_type == BlockType::kData; + CompressAndVerifyBlock(uncompressed_block_data, is_data_block, + *(r->compression_ctxs[0]), r->verify_ctxs[0].get(), + &(r->compressed_output), &(block_contents), &type, + &compress_status); + r->SetStatus(compress_status); + if (!ok()) { + return; + } + + WriteMaybeCompressedBlock(block_contents, type, handle, block_type, + &uncompressed_block_data); + r->compressed_output.clear(); + if (is_data_block) { + r->props.data_size = r->get_offset(); + ++r->props.num_data_blocks; + } +} + +void BlockBasedTableBuilder::BGWorkCompression( + const CompressionContext& compression_ctx, + UncompressionContext* verify_ctx) { + ParallelCompressionRep::BlockRep* block_rep = nullptr; + while (rep_->pc_rep->compress_queue.pop(block_rep)) { + assert(block_rep != nullptr); + CompressAndVerifyBlock(block_rep->contents, true, /* is_data_block*/ + compression_ctx, verify_ctx, + block_rep->compressed_data.get(), + &block_rep->compressed_contents, + &(block_rep->compression_type), &block_rep->status); + block_rep->slot->Fill(block_rep); + } +} + +void BlockBasedTableBuilder::CompressAndVerifyBlock( + const Slice& uncompressed_block_data, bool is_data_block, + const CompressionContext& compression_ctx, UncompressionContext* verify_ctx, + std::string* compressed_output, Slice* block_contents, + CompressionType* type, Status* out_status) { + // File format contains a sequence of blocks where each block has: + // block_data: uint8[n] + // type: uint8 + // crc: uint32 + Rep* r = rep_; + bool is_status_ok = ok(); + if (!r->IsParallelCompressionEnabled()) { + assert(is_status_ok); + } + + *type = r->compression_type; + uint64_t sample_for_compression = r->sample_for_compression; + bool abort_compression = false; + + StopWatchNano timer( + r->ioptions.clock, + ShouldReportDetailedTime(r->ioptions.env, r->ioptions.stats)); + + if (is_status_ok && uncompressed_block_data.size() < kCompressionSizeLimit) { + if (is_data_block) { + r->compressible_input_data_bytes.fetch_add(uncompressed_block_data.size(), + std::memory_order_relaxed); + } + const CompressionDict* compression_dict; + if (!is_data_block || r->compression_dict == nullptr) { + compression_dict = &CompressionDict::GetEmptyDict(); + } else { + compression_dict = r->compression_dict.get(); + } + assert(compression_dict != nullptr); + CompressionInfo compression_info(r->compression_opts, compression_ctx, + *compression_dict, *type, + sample_for_compression); + + std::string sampled_output_fast; + std::string sampled_output_slow; + *block_contents = CompressBlock( + uncompressed_block_data, compression_info, type, + r->table_options.format_version, is_data_block /* do_sample */, + compressed_output, &sampled_output_fast, &sampled_output_slow); + + if (sampled_output_slow.size() > 0 || sampled_output_fast.size() > 0) { + // Currently compression sampling is only enabled for data block. + assert(is_data_block); + r->sampled_input_data_bytes.fetch_add(uncompressed_block_data.size(), + std::memory_order_relaxed); + r->sampled_output_slow_data_bytes.fetch_add(sampled_output_slow.size(), + std::memory_order_relaxed); + r->sampled_output_fast_data_bytes.fetch_add(sampled_output_fast.size(), + std::memory_order_relaxed); + } + // notify collectors on block add + NotifyCollectTableCollectorsOnBlockAdd( + r->table_properties_collectors, uncompressed_block_data.size(), + sampled_output_fast.size(), sampled_output_slow.size()); + + // Some of the compression algorithms are known to be unreliable. If + // the verify_compression flag is set then try to de-compress the + // compressed data and compare to the input. + if (*type != kNoCompression && r->table_options.verify_compression) { + // Retrieve the uncompressed contents into a new buffer + const UncompressionDict* verify_dict; + if (!is_data_block || r->verify_dict == nullptr) { + verify_dict = &UncompressionDict::GetEmptyDict(); + } else { + verify_dict = r->verify_dict.get(); + } + assert(verify_dict != nullptr); + BlockContents contents; + UncompressionInfo uncompression_info(*verify_ctx, *verify_dict, + r->compression_type); + Status stat = UncompressBlockData( + uncompression_info, block_contents->data(), block_contents->size(), + &contents, r->table_options.format_version, r->ioptions); + + if (stat.ok()) { + bool compressed_ok = + contents.data.compare(uncompressed_block_data) == 0; + if (!compressed_ok) { + // The result of the compression was invalid. abort. + abort_compression = true; + const char* const msg = + "Decompressed block did not match pre-compression block"; + ROCKS_LOG_ERROR(r->ioptions.logger, "%s", msg); + *out_status = Status::Corruption(msg); + } + } else { + // Decompression reported an error. abort. + *out_status = Status::Corruption(std::string("Could not decompress: ") + + stat.getState()); + abort_compression = true; + } + } + } else { + // Block is too big to be compressed. + if (is_data_block) { + r->uncompressible_input_data_bytes.fetch_add( + uncompressed_block_data.size(), std::memory_order_relaxed); + } + abort_compression = true; + } + if (is_data_block) { + r->uncompressible_input_data_bytes.fetch_add(kBlockTrailerSize, + std::memory_order_relaxed); + } + + // Abort compression if the block is too big, or did not pass + // verification. + if (abort_compression) { + RecordTick(r->ioptions.stats, NUMBER_BLOCK_NOT_COMPRESSED); + *type = kNoCompression; + *block_contents = uncompressed_block_data; + } else if (*type != kNoCompression) { + if (ShouldReportDetailedTime(r->ioptions.env, r->ioptions.stats)) { + RecordTimeToHistogram(r->ioptions.stats, COMPRESSION_TIMES_NANOS, + timer.ElapsedNanos()); + } + RecordInHistogram(r->ioptions.stats, BYTES_COMPRESSED, + uncompressed_block_data.size()); + RecordTick(r->ioptions.stats, NUMBER_BLOCK_COMPRESSED); + } else if (*type != r->compression_type) { + RecordTick(r->ioptions.stats, NUMBER_BLOCK_NOT_COMPRESSED); + } +} + +void BlockBasedTableBuilder::WriteMaybeCompressedBlock( + const Slice& block_contents, CompressionType type, BlockHandle* handle, + BlockType block_type, const Slice* uncompressed_block_data) { + Rep* r = rep_; + bool is_data_block = block_type == BlockType::kData; + // Old, misleading name of this function: WriteRawBlock + StopWatch sw(r->ioptions.clock, r->ioptions.stats, WRITE_RAW_BLOCK_MICROS); + handle->set_offset(r->get_offset()); + handle->set_size(block_contents.size()); + assert(status().ok()); + assert(io_status().ok()); + + { + IOStatus io_s = r->file->Append(block_contents); + if (!io_s.ok()) { + r->SetIOStatus(io_s); + return; + } + } + + std::array<char, kBlockTrailerSize> trailer; + trailer[0] = type; + uint32_t checksum = ComputeBuiltinChecksumWithLastByte( + r->table_options.checksum, block_contents.data(), block_contents.size(), + /*last_byte*/ type); + + if (block_type == BlockType::kFilter) { + Status s = r->filter_builder->MaybePostVerifyFilter(block_contents); + if (!s.ok()) { + r->SetStatus(s); + return; + } + } + + EncodeFixed32(trailer.data() + 1, checksum); + TEST_SYNC_POINT_CALLBACK( + "BlockBasedTableBuilder::WriteMaybeCompressedBlock:TamperWithChecksum", + trailer.data()); + { + IOStatus io_s = r->file->Append(Slice(trailer.data(), trailer.size())); + if (!io_s.ok()) { + r->SetIOStatus(io_s); + return; + } + } + + { + Status s = Status::OK(); + bool warm_cache; + switch (r->table_options.prepopulate_block_cache) { + case BlockBasedTableOptions::PrepopulateBlockCache::kFlushOnly: + warm_cache = (r->reason == TableFileCreationReason::kFlush); + break; + case BlockBasedTableOptions::PrepopulateBlockCache::kDisable: + warm_cache = false; + break; + default: + // missing case + assert(false); + warm_cache = false; + } + if (warm_cache) { + if (type == kNoCompression) { + s = InsertBlockInCacheHelper(block_contents, handle, block_type); + } else if (uncompressed_block_data != nullptr) { + s = InsertBlockInCacheHelper(*uncompressed_block_data, handle, + block_type); + } + if (!s.ok()) { + r->SetStatus(s); + return; + } + } + s = InsertBlockInCompressedCache(block_contents, type, handle); + if (!s.ok()) { + r->SetStatus(s); + return; + } + } + + r->set_offset(r->get_offset() + block_contents.size() + kBlockTrailerSize); + if (r->table_options.block_align && is_data_block) { + size_t pad_bytes = + (r->alignment - + ((block_contents.size() + kBlockTrailerSize) & (r->alignment - 1))) & + (r->alignment - 1); + IOStatus io_s = r->file->Pad(pad_bytes); + if (io_s.ok()) { + r->set_offset(r->get_offset() + pad_bytes); + } else { + r->SetIOStatus(io_s); + return; + } + } + + if (r->IsParallelCompressionEnabled()) { + if (is_data_block) { + r->pc_rep->file_size_estimator.ReapBlock(block_contents.size(), + r->get_offset()); + } else { + r->pc_rep->file_size_estimator.SetEstimatedFileSize(r->get_offset()); + } + } +} + +void BlockBasedTableBuilder::BGWorkWriteMaybeCompressedBlock() { + Rep* r = rep_; + ParallelCompressionRep::BlockRepSlot* slot = nullptr; + ParallelCompressionRep::BlockRep* block_rep = nullptr; + while (r->pc_rep->write_queue.pop(slot)) { + assert(slot != nullptr); + slot->Take(block_rep); + assert(block_rep != nullptr); + if (!block_rep->status.ok()) { + r->SetStatus(block_rep->status); + // Reap block so that blocked Flush() can finish + // if there is one, and Flush() will notice !ok() next time. + block_rep->status = Status::OK(); + r->pc_rep->ReapBlock(block_rep); + continue; + } + + for (size_t i = 0; i < block_rep->keys->Size(); i++) { + auto& key = (*block_rep->keys)[i]; + if (r->filter_builder != nullptr) { + size_t ts_sz = + r->internal_comparator.user_comparator()->timestamp_size(); + r->filter_builder->Add(ExtractUserKeyAndStripTimestamp(key, ts_sz)); + } + r->index_builder->OnKeyAdded(key); + } + + r->pc_rep->file_size_estimator.SetCurrBlockUncompSize( + block_rep->data->size()); + WriteMaybeCompressedBlock(block_rep->compressed_contents, + block_rep->compression_type, &r->pending_handle, + BlockType::kData, &block_rep->contents); + if (!ok()) { + break; + } + + r->props.data_size = r->get_offset(); + ++r->props.num_data_blocks; + + if (block_rep->first_key_in_next_block == nullptr) { + r->index_builder->AddIndexEntry(&(block_rep->keys->Back()), nullptr, + r->pending_handle); + } else { + Slice first_key_in_next_block = + Slice(*block_rep->first_key_in_next_block); + r->index_builder->AddIndexEntry(&(block_rep->keys->Back()), + &first_key_in_next_block, + r->pending_handle); + } + + r->pc_rep->ReapBlock(block_rep); + } +} + +void BlockBasedTableBuilder::StartParallelCompression() { + rep_->pc_rep.reset( + new ParallelCompressionRep(rep_->compression_opts.parallel_threads)); + rep_->pc_rep->compress_thread_pool.reserve( + rep_->compression_opts.parallel_threads); + for (uint32_t i = 0; i < rep_->compression_opts.parallel_threads; i++) { + rep_->pc_rep->compress_thread_pool.emplace_back([this, i] { + BGWorkCompression(*(rep_->compression_ctxs[i]), + rep_->verify_ctxs[i].get()); + }); + } + rep_->pc_rep->write_thread.reset( + new port::Thread([this] { BGWorkWriteMaybeCompressedBlock(); })); +} + +void BlockBasedTableBuilder::StopParallelCompression() { + rep_->pc_rep->compress_queue.finish(); + for (auto& thread : rep_->pc_rep->compress_thread_pool) { + thread.join(); + } + rep_->pc_rep->write_queue.finish(); + rep_->pc_rep->write_thread->join(); +} + +Status BlockBasedTableBuilder::status() const { return rep_->GetStatus(); } + +IOStatus BlockBasedTableBuilder::io_status() const { + return rep_->GetIOStatus(); +} + +// +// Make a copy of the block contents and insert into compressed block cache +// +Status BlockBasedTableBuilder::InsertBlockInCompressedCache( + const Slice& block_contents, const CompressionType type, + const BlockHandle* handle) { + Rep* r = rep_; + Cache* block_cache_compressed = r->table_options.block_cache_compressed.get(); + Status s; + if (type != kNoCompression && block_cache_compressed != nullptr) { + size_t size = block_contents.size(); + + auto ubuf = + AllocateBlock(size + 1, block_cache_compressed->memory_allocator()); + memcpy(ubuf.get(), block_contents.data(), size); + ubuf[size] = type; + + BlockContents* block_contents_to_cache = + new BlockContents(std::move(ubuf), size); +#ifndef NDEBUG + block_contents_to_cache->has_trailer = true; +#endif // NDEBUG + + CacheKey key = BlockBasedTable::GetCacheKey(rep_->base_cache_key, *handle); + + s = block_cache_compressed->Insert( + key.AsSlice(), block_contents_to_cache, + block_contents_to_cache->ApproximateMemoryUsage(), + &DeleteCacheEntry<BlockContents>); + if (s.ok()) { + RecordTick(rep_->ioptions.stats, BLOCK_CACHE_COMPRESSED_ADD); + } else { + RecordTick(rep_->ioptions.stats, BLOCK_CACHE_COMPRESSED_ADD_FAILURES); + } + // Invalidate OS cache. + r->file->InvalidateCache(static_cast<size_t>(r->get_offset()), size) + .PermitUncheckedError(); + } + return s; +} + +Status BlockBasedTableBuilder::InsertBlockInCacheHelper( + const Slice& block_contents, const BlockHandle* handle, + BlockType block_type) { + Status s; + switch (block_type) { + case BlockType::kData: + case BlockType::kIndex: + case BlockType::kFilterPartitionIndex: + s = InsertBlockInCache<Block>(block_contents, handle, block_type); + break; + case BlockType::kFilter: + s = InsertBlockInCache<ParsedFullFilterBlock>(block_contents, handle, + block_type); + break; + case BlockType::kCompressionDictionary: + s = InsertBlockInCache<UncompressionDict>(block_contents, handle, + block_type); + break; + default: + // no-op / not cached + break; + } + return s; +} + +template <typename TBlocklike> +Status BlockBasedTableBuilder::InsertBlockInCache(const Slice& block_contents, + const BlockHandle* handle, + BlockType block_type) { + // Uncompressed regular block cache + Cache* block_cache = rep_->table_options.block_cache.get(); + Status s; + if (block_cache != nullptr) { + size_t size = block_contents.size(); + auto buf = AllocateBlock(size, block_cache->memory_allocator()); + memcpy(buf.get(), block_contents.data(), size); + BlockContents results(std::move(buf), size); + + CacheKey key = BlockBasedTable::GetCacheKey(rep_->base_cache_key, *handle); + + const size_t read_amp_bytes_per_bit = + rep_->table_options.read_amp_bytes_per_bit; + + // TODO akanksha:: Dedup below code by calling + // BlockBasedTable::PutDataBlockToCache. + std::unique_ptr<TBlocklike> block_holder( + BlocklikeTraits<TBlocklike>::Create( + std::move(results), read_amp_bytes_per_bit, + rep_->ioptions.statistics.get(), + false /*rep_->blocks_definitely_zstd_compressed*/, + rep_->table_options.filter_policy.get())); + + assert(block_holder->own_bytes()); + size_t charge = block_holder->ApproximateMemoryUsage(); + s = block_cache->Insert( + key.AsSlice(), block_holder.get(), + BlocklikeTraits<TBlocklike>::GetCacheItemHelper(block_type), charge, + nullptr, Cache::Priority::LOW); + + if (s.ok()) { + // Release ownership of block_holder. + block_holder.release(); + BlockBasedTable::UpdateCacheInsertionMetrics( + block_type, nullptr /*get_context*/, charge, s.IsOkOverwritten(), + rep_->ioptions.stats); + } else { + RecordTick(rep_->ioptions.stats, BLOCK_CACHE_ADD_FAILURES); + } + } + return s; +} + +void BlockBasedTableBuilder::WriteFilterBlock( + MetaIndexBuilder* meta_index_builder) { + if (rep_->filter_builder == nullptr || rep_->filter_builder->IsEmpty()) { + // No filter block needed + return; + } + BlockHandle filter_block_handle; + bool is_partitioned_filter = rep_->table_options.partition_filters; + if (ok()) { + rep_->props.num_filter_entries += + rep_->filter_builder->EstimateEntriesAdded(); + Status s = Status::Incomplete(); + while (ok() && s.IsIncomplete()) { + // filter_data is used to store the transferred filter data payload from + // FilterBlockBuilder and deallocate the payload by going out of scope. + // Otherwise, the payload will unnecessarily remain until + // BlockBasedTableBuilder is deallocated. + // + // See FilterBlockBuilder::Finish() for more on the difference in + // transferred filter data payload among different FilterBlockBuilder + // subtypes. + std::unique_ptr<const char[]> filter_data; + Slice filter_content = + rep_->filter_builder->Finish(filter_block_handle, &s, &filter_data); + + assert(s.ok() || s.IsIncomplete() || s.IsCorruption()); + if (s.IsCorruption()) { + rep_->SetStatus(s); + break; + } + + rep_->props.filter_size += filter_content.size(); + + BlockType btype = is_partitioned_filter && /* last */ s.ok() + ? BlockType::kFilterPartitionIndex + : BlockType::kFilter; + WriteMaybeCompressedBlock(filter_content, kNoCompression, + &filter_block_handle, btype); + } + rep_->filter_builder->ResetFilterBitsBuilder(); + } + if (ok()) { + // Add mapping from "<filter_block_prefix>.Name" to location + // of filter data. + std::string key; + key = is_partitioned_filter ? BlockBasedTable::kPartitionedFilterBlockPrefix + : BlockBasedTable::kFullFilterBlockPrefix; + key.append(rep_->table_options.filter_policy->CompatibilityName()); + meta_index_builder->Add(key, filter_block_handle); + } +} + +void BlockBasedTableBuilder::WriteIndexBlock( + MetaIndexBuilder* meta_index_builder, BlockHandle* index_block_handle) { + if (!ok()) { + return; + } + IndexBuilder::IndexBlocks index_blocks; + auto index_builder_status = rep_->index_builder->Finish(&index_blocks); + if (index_builder_status.IsIncomplete()) { + // We we have more than one index partition then meta_blocks are not + // supported for the index. Currently meta_blocks are used only by + // HashIndexBuilder which is not multi-partition. + assert(index_blocks.meta_blocks.empty()); + } else if (ok() && !index_builder_status.ok()) { + rep_->SetStatus(index_builder_status); + } + if (ok()) { + for (const auto& item : index_blocks.meta_blocks) { + BlockHandle block_handle; + WriteBlock(item.second, &block_handle, BlockType::kIndex); + if (!ok()) { + break; + } + meta_index_builder->Add(item.first, block_handle); + } + } + if (ok()) { + if (rep_->table_options.enable_index_compression) { + WriteBlock(index_blocks.index_block_contents, index_block_handle, + BlockType::kIndex); + } else { + WriteMaybeCompressedBlock(index_blocks.index_block_contents, + kNoCompression, index_block_handle, + BlockType::kIndex); + } + } + // If there are more index partitions, finish them and write them out + if (index_builder_status.IsIncomplete()) { + bool index_building_finished = false; + while (ok() && !index_building_finished) { + Status s = + rep_->index_builder->Finish(&index_blocks, *index_block_handle); + if (s.ok()) { + index_building_finished = true; + } else if (s.IsIncomplete()) { + // More partitioned index after this one + assert(!index_building_finished); + } else { + // Error + rep_->SetStatus(s); + return; + } + + if (rep_->table_options.enable_index_compression) { + WriteBlock(index_blocks.index_block_contents, index_block_handle, + BlockType::kIndex); + } else { + WriteMaybeCompressedBlock(index_blocks.index_block_contents, + kNoCompression, index_block_handle, + BlockType::kIndex); + } + // The last index_block_handle will be for the partition index block + } + } +} + +void BlockBasedTableBuilder::WritePropertiesBlock( + MetaIndexBuilder* meta_index_builder) { + BlockHandle properties_block_handle; + if (ok()) { + PropertyBlockBuilder property_block_builder; + rep_->props.filter_policy_name = + rep_->table_options.filter_policy != nullptr + ? rep_->table_options.filter_policy->Name() + : ""; + rep_->props.index_size = + rep_->index_builder->IndexSize() + kBlockTrailerSize; + rep_->props.comparator_name = rep_->ioptions.user_comparator != nullptr + ? rep_->ioptions.user_comparator->Name() + : "nullptr"; + rep_->props.merge_operator_name = + rep_->ioptions.merge_operator != nullptr + ? rep_->ioptions.merge_operator->Name() + : "nullptr"; + rep_->props.compression_name = + CompressionTypeToString(rep_->compression_type); + rep_->props.compression_options = + CompressionOptionsToString(rep_->compression_opts); + rep_->props.prefix_extractor_name = + rep_->moptions.prefix_extractor != nullptr + ? rep_->moptions.prefix_extractor->AsString() + : "nullptr"; + std::string property_collectors_names = "["; + for (size_t i = 0; + i < rep_->ioptions.table_properties_collector_factories.size(); ++i) { + if (i != 0) { + property_collectors_names += ","; + } + property_collectors_names += + rep_->ioptions.table_properties_collector_factories[i]->Name(); + } + property_collectors_names += "]"; + rep_->props.property_collectors_names = property_collectors_names; + if (rep_->table_options.index_type == + BlockBasedTableOptions::kTwoLevelIndexSearch) { + assert(rep_->p_index_builder_ != nullptr); + rep_->props.index_partitions = rep_->p_index_builder_->NumPartitions(); + rep_->props.top_level_index_size = + rep_->p_index_builder_->TopLevelIndexSize(rep_->offset); + } + rep_->props.index_key_is_user_key = + !rep_->index_builder->seperator_is_key_plus_seq(); + rep_->props.index_value_is_delta_encoded = + rep_->use_delta_encoding_for_index_values; + if (rep_->sampled_input_data_bytes > 0) { + rep_->props.slow_compression_estimated_data_size = static_cast<uint64_t>( + static_cast<double>(rep_->sampled_output_slow_data_bytes) / + rep_->sampled_input_data_bytes * + rep_->compressible_input_data_bytes + + rep_->uncompressible_input_data_bytes + 0.5); + rep_->props.fast_compression_estimated_data_size = static_cast<uint64_t>( + static_cast<double>(rep_->sampled_output_fast_data_bytes) / + rep_->sampled_input_data_bytes * + rep_->compressible_input_data_bytes + + rep_->uncompressible_input_data_bytes + 0.5); + } else if (rep_->sample_for_compression > 0) { + // We tried to sample but none were found. Assume worst-case (compression + // ratio 1.0) so data is complete and aggregatable. + rep_->props.slow_compression_estimated_data_size = + rep_->compressible_input_data_bytes + + rep_->uncompressible_input_data_bytes; + rep_->props.fast_compression_estimated_data_size = + rep_->compressible_input_data_bytes + + rep_->uncompressible_input_data_bytes; + } + + // Add basic properties + property_block_builder.AddTableProperty(rep_->props); + + // Add use collected properties + NotifyCollectTableCollectorsOnFinish(rep_->table_properties_collectors, + rep_->ioptions.logger, + &property_block_builder); + + Slice block_data = property_block_builder.Finish(); + TEST_SYNC_POINT_CALLBACK( + "BlockBasedTableBuilder::WritePropertiesBlock:BlockData", &block_data); + WriteMaybeCompressedBlock(block_data, kNoCompression, + &properties_block_handle, BlockType::kProperties); + } + if (ok()) { +#ifndef NDEBUG + { + uint64_t props_block_offset = properties_block_handle.offset(); + uint64_t props_block_size = properties_block_handle.size(); + TEST_SYNC_POINT_CALLBACK( + "BlockBasedTableBuilder::WritePropertiesBlock:GetPropsBlockOffset", + &props_block_offset); + TEST_SYNC_POINT_CALLBACK( + "BlockBasedTableBuilder::WritePropertiesBlock:GetPropsBlockSize", + &props_block_size); + } +#endif // !NDEBUG + + const std::string* properties_block_meta = &kPropertiesBlockName; + TEST_SYNC_POINT_CALLBACK( + "BlockBasedTableBuilder::WritePropertiesBlock:Meta", + &properties_block_meta); + meta_index_builder->Add(*properties_block_meta, properties_block_handle); + } +} + +void BlockBasedTableBuilder::WriteCompressionDictBlock( + MetaIndexBuilder* meta_index_builder) { + if (rep_->compression_dict != nullptr && + rep_->compression_dict->GetRawDict().size()) { + BlockHandle compression_dict_block_handle; + if (ok()) { + WriteMaybeCompressedBlock(rep_->compression_dict->GetRawDict(), + kNoCompression, &compression_dict_block_handle, + BlockType::kCompressionDictionary); +#ifndef NDEBUG + Slice compression_dict = rep_->compression_dict->GetRawDict(); + TEST_SYNC_POINT_CALLBACK( + "BlockBasedTableBuilder::WriteCompressionDictBlock:RawDict", + &compression_dict); +#endif // NDEBUG + } + if (ok()) { + meta_index_builder->Add(kCompressionDictBlockName, + compression_dict_block_handle); + } + } +} + +void BlockBasedTableBuilder::WriteRangeDelBlock( + MetaIndexBuilder* meta_index_builder) { + if (ok() && !rep_->range_del_block.empty()) { + BlockHandle range_del_block_handle; + WriteMaybeCompressedBlock(rep_->range_del_block.Finish(), kNoCompression, + &range_del_block_handle, + BlockType::kRangeDeletion); + meta_index_builder->Add(kRangeDelBlockName, range_del_block_handle); + } +} + +void BlockBasedTableBuilder::WriteFooter(BlockHandle& metaindex_block_handle, + BlockHandle& index_block_handle) { + Rep* r = rep_; + // this is guaranteed by BlockBasedTableBuilder's constructor + assert(r->table_options.checksum == kCRC32c || + r->table_options.format_version != 0); + assert(ok()); + + FooterBuilder footer; + footer.Build(kBlockBasedTableMagicNumber, r->table_options.format_version, + r->get_offset(), r->table_options.checksum, + metaindex_block_handle, index_block_handle); + IOStatus ios = r->file->Append(footer.GetSlice()); + if (ios.ok()) { + r->set_offset(r->get_offset() + footer.GetSlice().size()); + } else { + r->SetIOStatus(ios); + } +} + +void BlockBasedTableBuilder::EnterUnbuffered() { + Rep* r = rep_; + assert(r->state == Rep::State::kBuffered); + r->state = Rep::State::kUnbuffered; + const size_t kSampleBytes = r->compression_opts.zstd_max_train_bytes > 0 + ? r->compression_opts.zstd_max_train_bytes + : r->compression_opts.max_dict_bytes; + const size_t kNumBlocksBuffered = r->data_block_buffers.size(); + if (kNumBlocksBuffered == 0) { + // The below code is neither safe nor necessary for handling zero data + // blocks. + return; + } + + // Abstract algebra teaches us that a finite cyclic group (such as the + // additive group of integers modulo N) can be generated by a number that is + // coprime with N. Since N is variable (number of buffered data blocks), we + // must then pick a prime number in order to guarantee coprimeness with any N. + // + // One downside of this approach is the spread will be poor when + // `kPrimeGeneratorRemainder` is close to zero or close to + // `kNumBlocksBuffered`. + // + // Picked a random number between one and one trillion and then chose the + // next prime number greater than or equal to it. + const uint64_t kPrimeGenerator = 545055921143ull; + // Can avoid repeated division by just adding the remainder repeatedly. + const size_t kPrimeGeneratorRemainder = static_cast<size_t>( + kPrimeGenerator % static_cast<uint64_t>(kNumBlocksBuffered)); + const size_t kInitSampleIdx = kNumBlocksBuffered / 2; + + std::string compression_dict_samples; + std::vector<size_t> compression_dict_sample_lens; + size_t buffer_idx = kInitSampleIdx; + for (size_t i = 0; + i < kNumBlocksBuffered && compression_dict_samples.size() < kSampleBytes; + ++i) { + size_t copy_len = std::min(kSampleBytes - compression_dict_samples.size(), + r->data_block_buffers[buffer_idx].size()); + compression_dict_samples.append(r->data_block_buffers[buffer_idx], 0, + copy_len); + compression_dict_sample_lens.emplace_back(copy_len); + + buffer_idx += kPrimeGeneratorRemainder; + if (buffer_idx >= kNumBlocksBuffered) { + buffer_idx -= kNumBlocksBuffered; + } + } + + // final data block flushed, now we can generate dictionary from the samples. + // OK if compression_dict_samples is empty, we'll just get empty dictionary. + std::string dict; + if (r->compression_opts.zstd_max_train_bytes > 0) { + if (r->compression_opts.use_zstd_dict_trainer) { + dict = ZSTD_TrainDictionary(compression_dict_samples, + compression_dict_sample_lens, + r->compression_opts.max_dict_bytes); + } else { + dict = ZSTD_FinalizeDictionary( + compression_dict_samples, compression_dict_sample_lens, + r->compression_opts.max_dict_bytes, r->compression_opts.level); + } + } else { + dict = std::move(compression_dict_samples); + } + r->compression_dict.reset(new CompressionDict(dict, r->compression_type, + r->compression_opts.level)); + r->verify_dict.reset(new UncompressionDict( + dict, r->compression_type == kZSTD || + r->compression_type == kZSTDNotFinalCompression)); + + auto get_iterator_for_block = [&r](size_t i) { + auto& data_block = r->data_block_buffers[i]; + assert(!data_block.empty()); + + Block reader{BlockContents{data_block}}; + DataBlockIter* iter = reader.NewDataIterator( + r->internal_comparator.user_comparator(), kDisableGlobalSequenceNumber); + + iter->SeekToFirst(); + assert(iter->Valid()); + return std::unique_ptr<DataBlockIter>(iter); + }; + + std::unique_ptr<DataBlockIter> iter = nullptr, next_block_iter = nullptr; + + for (size_t i = 0; ok() && i < r->data_block_buffers.size(); ++i) { + if (iter == nullptr) { + iter = get_iterator_for_block(i); + assert(iter != nullptr); + }; + + if (i + 1 < r->data_block_buffers.size()) { + next_block_iter = get_iterator_for_block(i + 1); + } + + auto& data_block = r->data_block_buffers[i]; + if (r->IsParallelCompressionEnabled()) { + Slice first_key_in_next_block; + const Slice* first_key_in_next_block_ptr = &first_key_in_next_block; + if (i + 1 < r->data_block_buffers.size()) { + assert(next_block_iter != nullptr); + first_key_in_next_block = next_block_iter->key(); + } else { + first_key_in_next_block_ptr = r->first_key_in_next_block; + } + + std::vector<std::string> keys; + for (; iter->Valid(); iter->Next()) { + keys.emplace_back(iter->key().ToString()); + } + + ParallelCompressionRep::BlockRep* block_rep = r->pc_rep->PrepareBlock( + r->compression_type, first_key_in_next_block_ptr, &data_block, &keys); + + assert(block_rep != nullptr); + r->pc_rep->file_size_estimator.EmitBlock(block_rep->data->size(), + r->get_offset()); + r->pc_rep->EmitBlock(block_rep); + } else { + for (; iter->Valid(); iter->Next()) { + Slice key = iter->key(); + if (r->filter_builder != nullptr) { + size_t ts_sz = + r->internal_comparator.user_comparator()->timestamp_size(); + r->filter_builder->Add(ExtractUserKeyAndStripTimestamp(key, ts_sz)); + } + r->index_builder->OnKeyAdded(key); + } + WriteBlock(Slice(data_block), &r->pending_handle, BlockType::kData); + if (ok() && i + 1 < r->data_block_buffers.size()) { + assert(next_block_iter != nullptr); + Slice first_key_in_next_block = next_block_iter->key(); + + Slice* first_key_in_next_block_ptr = &first_key_in_next_block; + + iter->SeekToLast(); + std::string last_key = iter->key().ToString(); + r->index_builder->AddIndexEntry(&last_key, first_key_in_next_block_ptr, + r->pending_handle); + } + } + std::swap(iter, next_block_iter); + } + r->data_block_buffers.clear(); + r->data_begin_offset = 0; + // Release all reserved cache for data block buffers + if (r->compression_dict_buffer_cache_res_mgr != nullptr) { + Status s = r->compression_dict_buffer_cache_res_mgr->UpdateCacheReservation( + r->data_begin_offset); + s.PermitUncheckedError(); + } +} + +Status BlockBasedTableBuilder::Finish() { + Rep* r = rep_; + assert(r->state != Rep::State::kClosed); + bool empty_data_block = r->data_block.empty(); + r->first_key_in_next_block = nullptr; + Flush(); + if (r->state == Rep::State::kBuffered) { + EnterUnbuffered(); + } + if (r->IsParallelCompressionEnabled()) { + StopParallelCompression(); +#ifndef NDEBUG + for (const auto& br : r->pc_rep->block_rep_buf) { + assert(br.status.ok()); + } +#endif // !NDEBUG + } else { + // To make sure properties block is able to keep the accurate size of index + // block, we will finish writing all index entries first. + if (ok() && !empty_data_block) { + r->index_builder->AddIndexEntry( + &r->last_key, nullptr /* no next data block */, r->pending_handle); + } + } + + // Write meta blocks, metaindex block and footer in the following order. + // 1. [meta block: filter] + // 2. [meta block: index] + // 3. [meta block: compression dictionary] + // 4. [meta block: range deletion tombstone] + // 5. [meta block: properties] + // 6. [metaindex block] + // 7. Footer + BlockHandle metaindex_block_handle, index_block_handle; + MetaIndexBuilder meta_index_builder; + WriteFilterBlock(&meta_index_builder); + WriteIndexBlock(&meta_index_builder, &index_block_handle); + WriteCompressionDictBlock(&meta_index_builder); + WriteRangeDelBlock(&meta_index_builder); + WritePropertiesBlock(&meta_index_builder); + if (ok()) { + // flush the meta index block + WriteMaybeCompressedBlock(meta_index_builder.Finish(), kNoCompression, + &metaindex_block_handle, BlockType::kMetaIndex); + } + if (ok()) { + WriteFooter(metaindex_block_handle, index_block_handle); + } + r->state = Rep::State::kClosed; + r->SetStatus(r->CopyIOStatus()); + Status ret_status = r->CopyStatus(); + assert(!ret_status.ok() || io_status().ok()); + return ret_status; +} + +void BlockBasedTableBuilder::Abandon() { + assert(rep_->state != Rep::State::kClosed); + if (rep_->IsParallelCompressionEnabled()) { + StopParallelCompression(); + } + rep_->state = Rep::State::kClosed; + rep_->CopyStatus().PermitUncheckedError(); + rep_->CopyIOStatus().PermitUncheckedError(); +} + +uint64_t BlockBasedTableBuilder::NumEntries() const { + return rep_->props.num_entries; +} + +bool BlockBasedTableBuilder::IsEmpty() const { + return rep_->props.num_entries == 0 && rep_->props.num_range_deletions == 0; +} + +uint64_t BlockBasedTableBuilder::FileSize() const { return rep_->offset; } + +uint64_t BlockBasedTableBuilder::EstimatedFileSize() const { + if (rep_->IsParallelCompressionEnabled()) { + // Use compression ratio so far and inflight uncompressed bytes to estimate + // final SST size. + return rep_->pc_rep->file_size_estimator.GetEstimatedFileSize(); + } else { + return FileSize(); + } +} + +bool BlockBasedTableBuilder::NeedCompact() const { + for (const auto& collector : rep_->table_properties_collectors) { + if (collector->NeedCompact()) { + return true; + } + } + return false; +} + +TableProperties BlockBasedTableBuilder::GetTableProperties() const { + TableProperties ret = rep_->props; + for (const auto& collector : rep_->table_properties_collectors) { + for (const auto& prop : collector->GetReadableProperties()) { + ret.readable_properties.insert(prop); + } + collector->Finish(&ret.user_collected_properties).PermitUncheckedError(); + } + return ret; +} + +std::string BlockBasedTableBuilder::GetFileChecksum() const { + if (rep_->file != nullptr) { + return rep_->file->GetFileChecksum(); + } else { + return kUnknownFileChecksum; + } +} + +const char* BlockBasedTableBuilder::GetFileChecksumFuncName() const { + if (rep_->file != nullptr) { + return rep_->file->GetFileChecksumFuncName(); + } else { + return kUnknownFileChecksumFuncName; + } +} +void BlockBasedTableBuilder::SetSeqnoTimeTableProperties( + const std::string& encoded_seqno_to_time_mapping, + uint64_t oldest_ancestor_time) { + rep_->props.seqno_to_time_mapping = encoded_seqno_to_time_mapping; + rep_->props.creation_time = oldest_ancestor_time; +} + +const std::string BlockBasedTable::kObsoleteFilterBlockPrefix = "filter."; +const std::string BlockBasedTable::kFullFilterBlockPrefix = "fullfilter."; +const std::string BlockBasedTable::kPartitionedFilterBlockPrefix = + "partitionedfilter."; +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/block_based_table_builder.h b/src/rocksdb/table/block_based/block_based_table_builder.h new file mode 100644 index 000000000..ecc13d0f7 --- /dev/null +++ b/src/rocksdb/table/block_based/block_based_table_builder.h @@ -0,0 +1,203 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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 <stdint.h> + +#include <array> +#include <limits> +#include <string> +#include <utility> +#include <vector> + +#include "db/version_edit.h" +#include "rocksdb/flush_block_policy.h" +#include "rocksdb/listener.h" +#include "rocksdb/options.h" +#include "rocksdb/status.h" +#include "rocksdb/table.h" +#include "table/meta_blocks.h" +#include "table/table_builder.h" +#include "util/compression.h" + +namespace ROCKSDB_NAMESPACE { + +class BlockBuilder; +class BlockHandle; +class WritableFile; +struct BlockBasedTableOptions; + +extern const uint64_t kBlockBasedTableMagicNumber; +extern const uint64_t kLegacyBlockBasedTableMagicNumber; + +class BlockBasedTableBuilder : public TableBuilder { + public: + // Create a builder that will store the contents of the table it is + // building in *file. Does not close the file. It is up to the + // caller to close the file after calling Finish(). + BlockBasedTableBuilder(const BlockBasedTableOptions& table_options, + const TableBuilderOptions& table_builder_options, + WritableFileWriter* file); + + // No copying allowed + BlockBasedTableBuilder(const BlockBasedTableBuilder&) = delete; + BlockBasedTableBuilder& operator=(const BlockBasedTableBuilder&) = delete; + + // REQUIRES: Either Finish() or Abandon() has been called. + ~BlockBasedTableBuilder(); + + // Add key,value to the table being constructed. + // REQUIRES: key is after any previously added key according to comparator. + // REQUIRES: Finish(), Abandon() have not been called + void Add(const Slice& key, const Slice& value) override; + + // Return non-ok iff some error has been detected. + Status status() const override; + + // Return non-ok iff some error happens during IO. + IOStatus io_status() const override; + + // Finish building the table. Stops using the file passed to the + // constructor after this function returns. + // REQUIRES: Finish(), Abandon() have not been called + Status Finish() override; + + // Indicate that the contents of this builder should be abandoned. Stops + // using the file passed to the constructor after this function returns. + // If the caller is not going to call Finish(), it must call Abandon() + // before destroying this builder. + // REQUIRES: Finish(), Abandon() have not been called + void Abandon() override; + + // Number of calls to Add() so far. + uint64_t NumEntries() const override; + + bool IsEmpty() const override; + + // Size of the file generated so far. If invoked after a successful + // Finish() call, returns the size of the final generated file. + uint64_t FileSize() const override; + + // Estimated size of the file generated so far. This is used when + // FileSize() cannot estimate final SST size, e.g. parallel compression + // is enabled. + uint64_t EstimatedFileSize() const override; + + bool NeedCompact() const override; + + // Get table properties + TableProperties GetTableProperties() const override; + + // Get file checksum + std::string GetFileChecksum() const override; + + // Get file checksum function name + const char* GetFileChecksumFuncName() const override; + + void SetSeqnoTimeTableProperties( + const std::string& encoded_seqno_to_time_mapping, + uint64_t oldest_ancestor_time) override; + + private: + bool ok() const { return status().ok(); } + + // Transition state from buffered to unbuffered. See `Rep::State` API comment + // for details of the states. + // REQUIRES: `rep_->state == kBuffered` + void EnterUnbuffered(); + + // Call block's Finish() method and then + // - in buffered mode, buffer the uncompressed block contents. + // - in unbuffered mode, write the compressed block contents to file. + void WriteBlock(BlockBuilder* block, BlockHandle* handle, + BlockType blocktype); + + // Compress and write block content to the file. + void WriteBlock(const Slice& block_contents, BlockHandle* handle, + BlockType block_type); + // Directly write data to the file. + void WriteMaybeCompressedBlock(const Slice& data, CompressionType, + BlockHandle* handle, BlockType block_type, + const Slice* raw_data = nullptr); + + void SetupCacheKeyPrefix(const TableBuilderOptions& tbo); + + template <typename TBlocklike> + Status InsertBlockInCache(const Slice& block_contents, + const BlockHandle* handle, BlockType block_type); + + Status InsertBlockInCacheHelper(const Slice& block_contents, + const BlockHandle* handle, + BlockType block_type); + + Status InsertBlockInCompressedCache(const Slice& block_contents, + const CompressionType type, + const BlockHandle* handle); + + void WriteFilterBlock(MetaIndexBuilder* meta_index_builder); + void WriteIndexBlock(MetaIndexBuilder* meta_index_builder, + BlockHandle* index_block_handle); + void WritePropertiesBlock(MetaIndexBuilder* meta_index_builder); + void WriteCompressionDictBlock(MetaIndexBuilder* meta_index_builder); + void WriteRangeDelBlock(MetaIndexBuilder* meta_index_builder); + void WriteFooter(BlockHandle& metaindex_block_handle, + BlockHandle& index_block_handle); + + struct Rep; + class BlockBasedTablePropertiesCollectorFactory; + class BlockBasedTablePropertiesCollector; + Rep* rep_; + + struct ParallelCompressionRep; + + // Advanced operation: flush any buffered key/value pairs to file. + // Can be used to ensure that two adjacent entries never live in + // the same data block. Most clients should not need to use this method. + // REQUIRES: Finish(), Abandon() have not been called + void Flush(); + + // Some compression libraries fail when the uncompressed size is bigger than + // int. If uncompressed size is bigger than kCompressionSizeLimit, don't + // compress it + const uint64_t kCompressionSizeLimit = std::numeric_limits<int>::max(); + + // Get blocks from mem-table walking thread, compress them and + // pass them to the write thread. Used in parallel compression mode only + void BGWorkCompression(const CompressionContext& compression_ctx, + UncompressionContext* verify_ctx); + + // Given uncompressed block content, try to compress it and return result and + // compression type + void CompressAndVerifyBlock(const Slice& uncompressed_block_data, + bool is_data_block, + const CompressionContext& compression_ctx, + UncompressionContext* verify_ctx, + std::string* compressed_output, + Slice* result_block_contents, + CompressionType* result_compression_type, + Status* out_status); + + // Get compressed blocks from BGWorkCompression and write them into SST + void BGWorkWriteMaybeCompressedBlock(); + + // Initialize parallel compression context and + // start BGWorkCompression and BGWorkWriteMaybeCompressedBlock threads + void StartParallelCompression(); + + // Stop BGWorkCompression and BGWorkWriteMaybeCompressedBlock threads + void StopParallelCompression(); +}; + +Slice CompressBlock(const Slice& uncompressed_data, const CompressionInfo& info, + CompressionType* type, uint32_t format_version, + bool do_sample, std::string* compressed_output, + std::string* sampled_output_fast, + std::string* sampled_output_slow); + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/block_based_table_factory.cc b/src/rocksdb/table/block_based/block_based_table_factory.cc new file mode 100644 index 000000000..09c1d2f62 --- /dev/null +++ b/src/rocksdb/table/block_based/block_based_table_factory.cc @@ -0,0 +1,1058 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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/block_based/block_based_table_factory.h" + +#include <stdint.h> + +#include <cinttypes> +#include <memory> +#include <string> + +#include "cache/cache_entry_roles.h" +#include "cache/cache_reservation_manager.h" +#include "logging/logging.h" +#include "options/options_helper.h" +#include "port/port.h" +#include "rocksdb/cache.h" +#include "rocksdb/convenience.h" +#include "rocksdb/filter_policy.h" +#include "rocksdb/flush_block_policy.h" +#include "rocksdb/rocksdb_namespace.h" +#include "rocksdb/table.h" +#include "rocksdb/utilities/options_type.h" +#include "table/block_based/block_based_table_builder.h" +#include "table/block_based/block_based_table_reader.h" +#include "table/format.h" +#include "util/mutexlock.h" +#include "util/string_util.h" + +namespace ROCKSDB_NAMESPACE { + +void TailPrefetchStats::RecordEffectiveSize(size_t len) { + MutexLock l(&mutex_); + if (num_records_ < kNumTracked) { + num_records_++; + } + records_[next_++] = len; + if (next_ == kNumTracked) { + next_ = 0; + } +} + +size_t TailPrefetchStats::GetSuggestedPrefetchSize() { + std::vector<size_t> sorted; + { + MutexLock l(&mutex_); + + if (num_records_ == 0) { + return 0; + } + sorted.assign(records_, records_ + num_records_); + } + + // Of the historic size, we find the maximum one that satisifis the condtiion + // that if prefetching all, less than 1/8 will be wasted. + std::sort(sorted.begin(), sorted.end()); + + // Assuming we have 5 data points, and after sorting it looks like this: + // + // +---+ + // +---+ | | + // | | | | + // | | | | + // | | | | + // | | | | + // +---+ | | | | + // | | | | | | + // +---+ | | | | | | + // | | | | | | | | + // +---+ | | | | | | | | + // | | | | | | | | | | + // | | | | | | | | | | + // | | | | | | | | | | + // | | | | | | | | | | + // | | | | | | | | | | + // +---+ +---+ +---+ +---+ +---+ + // + // and we use every of the value as a candidate, and estimate how much we + // wasted, compared to read. For example, when we use the 3rd record + // as candiate. This area is what we read: + // +---+ + // +---+ | | + // | | | | + // | | | | + // | | | | + // | | | | + // *** *** *** ***+ *** *** *** *** ** + // * | | | | | | + // +---+ | | | | | * + // * | | | | | | | | + // +---+ | | | | | | | * + // * | | | | X | | | | | + // | | | | | | | | | * + // * | | | | | | | | | + // | | | | | | | | | * + // * | | | | | | | | | + // *** *** ***-*** ***--*** ***--*** +**** + // which is (size of the record) X (number of records). + // + // While wasted is this area: + // +---+ + // +---+ | | + // | | | | + // | | | | + // | | | | + // | | | | + // *** *** *** ****---+ | | | | + // * * | | | | | + // * *-*** *** | | | | | + // * * | | | | | | | + // *--** *** | | | | | | | + // | | | | | X | | | | | + // | | | | | | | | | | + // | | | | | | | | | | + // | | | | | | | | | | + // | | | | | | | | | | + // +---+ +---+ +---+ +---+ +---+ + // + // Which can be calculated iteratively. + // The difference between wasted using 4st and 3rd record, will + // be following area: + // +---+ + // +--+ +-+ ++ +-+ +-+ +---+ | | + // + xxxxxxxxxxxxxxxxxxxxxxxx | | | | + // xxxxxxxxxxxxxxxxxxxxxxxx | | | | + // + xxxxxxxxxxxxxxxxxxxxxxxx | | | | + // | xxxxxxxxxxxxxxxxxxxxxxxx | | | | + // +-+ +-+ +-+ ++ +---+ +--+ | | | + // | | | | | | | + // +---+ ++ | | | | | | + // | | | | | | X | | | + // +---+ ++ | | | | | | | | + // | | | | | | | | | | + // | | | | | | | | | | + // | | | | | | | | | | + // | | | | | | | | | | + // | | | | | | | | | | + // +---+ +---+ +---+ +---+ +---+ + // + // which will be the size difference between 4st and 3rd record, + // times 3, which is number of records before the 4st. + // Here we assume that all data within the prefetch range will be useful. In + // reality, it may not be the case when a partial block is inside the range, + // or there are data in the middle that is not read. We ignore those cases + // for simplicity. + assert(!sorted.empty()); + size_t prev_size = sorted[0]; + size_t max_qualified_size = sorted[0]; + size_t wasted = 0; + for (size_t i = 1; i < sorted.size(); i++) { + size_t read = sorted[i] * sorted.size(); + wasted += (sorted[i] - prev_size) * i; + if (wasted <= read / 8) { + max_qualified_size = sorted[i]; + } + prev_size = sorted[i]; + } + const size_t kMaxPrefetchSize = 512 * 1024; // Never exceed 512KB + return std::min(kMaxPrefetchSize, max_qualified_size); +} + +#ifndef ROCKSDB_LITE + +const std::string kOptNameMetadataCacheOpts = "metadata_cache_options"; + +static std::unordered_map<std::string, PinningTier> + pinning_tier_type_string_map = { + {"kFallback", PinningTier::kFallback}, + {"kNone", PinningTier::kNone}, + {"kFlushedAndSimilar", PinningTier::kFlushedAndSimilar}, + {"kAll", PinningTier::kAll}}; + +static std::unordered_map<std::string, BlockBasedTableOptions::IndexType> + block_base_table_index_type_string_map = { + {"kBinarySearch", BlockBasedTableOptions::IndexType::kBinarySearch}, + {"kHashSearch", BlockBasedTableOptions::IndexType::kHashSearch}, + {"kTwoLevelIndexSearch", + BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch}, + {"kBinarySearchWithFirstKey", + BlockBasedTableOptions::IndexType::kBinarySearchWithFirstKey}}; + +static std::unordered_map<std::string, + BlockBasedTableOptions::DataBlockIndexType> + block_base_table_data_block_index_type_string_map = { + {"kDataBlockBinarySearch", + BlockBasedTableOptions::DataBlockIndexType::kDataBlockBinarySearch}, + {"kDataBlockBinaryAndHash", + BlockBasedTableOptions::DataBlockIndexType::kDataBlockBinaryAndHash}}; + +static std::unordered_map<std::string, + BlockBasedTableOptions::IndexShorteningMode> + block_base_table_index_shortening_mode_string_map = { + {"kNoShortening", + BlockBasedTableOptions::IndexShorteningMode::kNoShortening}, + {"kShortenSeparators", + BlockBasedTableOptions::IndexShorteningMode::kShortenSeparators}, + {"kShortenSeparatorsAndSuccessor", + BlockBasedTableOptions::IndexShorteningMode:: + kShortenSeparatorsAndSuccessor}}; + +static std::unordered_map<std::string, OptionTypeInfo> + metadata_cache_options_type_info = { + {"top_level_index_pinning", + OptionTypeInfo::Enum<PinningTier>( + offsetof(struct MetadataCacheOptions, top_level_index_pinning), + &pinning_tier_type_string_map)}, + {"partition_pinning", + OptionTypeInfo::Enum<PinningTier>( + offsetof(struct MetadataCacheOptions, partition_pinning), + &pinning_tier_type_string_map)}, + {"unpartitioned_pinning", + OptionTypeInfo::Enum<PinningTier>( + offsetof(struct MetadataCacheOptions, unpartitioned_pinning), + &pinning_tier_type_string_map)}}; + +static std::unordered_map<std::string, + BlockBasedTableOptions::PrepopulateBlockCache> + block_base_table_prepopulate_block_cache_string_map = { + {"kDisable", BlockBasedTableOptions::PrepopulateBlockCache::kDisable}, + {"kFlushOnly", + BlockBasedTableOptions::PrepopulateBlockCache::kFlushOnly}}; + +#endif // ROCKSDB_LITE + +static std::unordered_map<std::string, OptionTypeInfo> + block_based_table_type_info = { +#ifndef ROCKSDB_LITE + /* currently not supported + std::shared_ptr<Cache> block_cache = nullptr; + std::shared_ptr<Cache> block_cache_compressed = nullptr; + CacheUsageOptions cache_usage_options; + */ + {"flush_block_policy_factory", + OptionTypeInfo::AsCustomSharedPtr<FlushBlockPolicyFactory>( + offsetof(struct BlockBasedTableOptions, + flush_block_policy_factory), + OptionVerificationType::kByName, OptionTypeFlags::kCompareNever)}, + {"cache_index_and_filter_blocks", + {offsetof(struct BlockBasedTableOptions, + cache_index_and_filter_blocks), + OptionType::kBoolean, OptionVerificationType::kNormal, + OptionTypeFlags::kNone}}, + {"cache_index_and_filter_blocks_with_high_priority", + {offsetof(struct BlockBasedTableOptions, + cache_index_and_filter_blocks_with_high_priority), + OptionType::kBoolean, OptionVerificationType::kNormal, + OptionTypeFlags::kNone}}, + {"pin_l0_filter_and_index_blocks_in_cache", + {offsetof(struct BlockBasedTableOptions, + pin_l0_filter_and_index_blocks_in_cache), + OptionType::kBoolean, OptionVerificationType::kNormal, + OptionTypeFlags::kNone}}, + {"index_type", OptionTypeInfo::Enum<BlockBasedTableOptions::IndexType>( + offsetof(struct BlockBasedTableOptions, index_type), + &block_base_table_index_type_string_map)}, + {"hash_index_allow_collision", + {0, OptionType::kBoolean, OptionVerificationType::kDeprecated, + OptionTypeFlags::kNone}}, + {"data_block_index_type", + OptionTypeInfo::Enum<BlockBasedTableOptions::DataBlockIndexType>( + offsetof(struct BlockBasedTableOptions, data_block_index_type), + &block_base_table_data_block_index_type_string_map)}, + {"index_shortening", + OptionTypeInfo::Enum<BlockBasedTableOptions::IndexShorteningMode>( + offsetof(struct BlockBasedTableOptions, index_shortening), + &block_base_table_index_shortening_mode_string_map)}, + {"data_block_hash_table_util_ratio", + {offsetof(struct BlockBasedTableOptions, + data_block_hash_table_util_ratio), + OptionType::kDouble, OptionVerificationType::kNormal, + OptionTypeFlags::kNone}}, + {"checksum", + {offsetof(struct BlockBasedTableOptions, checksum), + OptionType::kChecksumType, OptionVerificationType::kNormal, + OptionTypeFlags::kNone}}, + {"no_block_cache", + {offsetof(struct BlockBasedTableOptions, no_block_cache), + OptionType::kBoolean, OptionVerificationType::kNormal, + OptionTypeFlags::kNone}}, + {"block_size", + {offsetof(struct BlockBasedTableOptions, block_size), + OptionType::kSizeT, OptionVerificationType::kNormal, + OptionTypeFlags::kMutable}}, + {"block_size_deviation", + {offsetof(struct BlockBasedTableOptions, block_size_deviation), + OptionType::kInt, OptionVerificationType::kNormal, + OptionTypeFlags::kNone}}, + {"block_restart_interval", + {offsetof(struct BlockBasedTableOptions, block_restart_interval), + OptionType::kInt, OptionVerificationType::kNormal, + OptionTypeFlags::kMutable}}, + {"index_block_restart_interval", + {offsetof(struct BlockBasedTableOptions, index_block_restart_interval), + OptionType::kInt, OptionVerificationType::kNormal, + OptionTypeFlags::kNone}}, + {"index_per_partition", + {0, OptionType::kUInt64T, OptionVerificationType::kDeprecated, + OptionTypeFlags::kNone}}, + {"metadata_block_size", + {offsetof(struct BlockBasedTableOptions, metadata_block_size), + OptionType::kUInt64T, OptionVerificationType::kNormal, + OptionTypeFlags::kNone}}, + {"partition_filters", + {offsetof(struct BlockBasedTableOptions, partition_filters), + OptionType::kBoolean, OptionVerificationType::kNormal, + OptionTypeFlags::kNone}}, + {"optimize_filters_for_memory", + {offsetof(struct BlockBasedTableOptions, optimize_filters_for_memory), + OptionType::kBoolean, OptionVerificationType::kNormal, + OptionTypeFlags::kNone}}, + {"filter_policy", + OptionTypeInfo::AsCustomSharedPtr<const FilterPolicy>( + offsetof(struct BlockBasedTableOptions, filter_policy), + OptionVerificationType::kByNameAllowFromNull, + OptionTypeFlags::kNone)}, + {"whole_key_filtering", + {offsetof(struct BlockBasedTableOptions, whole_key_filtering), + OptionType::kBoolean, OptionVerificationType::kNormal, + OptionTypeFlags::kNone}}, + {"detect_filter_construct_corruption", + {offsetof(struct BlockBasedTableOptions, + detect_filter_construct_corruption), + OptionType::kBoolean, OptionVerificationType::kNormal, + OptionTypeFlags::kMutable}}, + {"reserve_table_builder_memory", + {0, OptionType::kBoolean, OptionVerificationType::kDeprecated, + OptionTypeFlags::kNone}}, + {"reserve_table_reader_memory", + {0, OptionType::kBoolean, OptionVerificationType::kDeprecated, + OptionTypeFlags::kNone}}, + {"skip_table_builder_flush", + {0, OptionType::kBoolean, OptionVerificationType::kDeprecated, + OptionTypeFlags::kNone}}, + {"format_version", + {offsetof(struct BlockBasedTableOptions, format_version), + OptionType::kUInt32T, OptionVerificationType::kNormal, + OptionTypeFlags::kNone}}, + {"verify_compression", + {offsetof(struct BlockBasedTableOptions, verify_compression), + OptionType::kBoolean, OptionVerificationType::kNormal, + OptionTypeFlags::kNone}}, + {"read_amp_bytes_per_bit", + {offsetof(struct BlockBasedTableOptions, read_amp_bytes_per_bit), + OptionType::kUInt32T, OptionVerificationType::kNormal, + OptionTypeFlags::kNone, + [](const ConfigOptions& /*opts*/, const std::string& /*name*/, + const std::string& value, void* addr) { + // A workaround to fix a bug in 6.10, 6.11, 6.12, 6.13 + // and 6.14. The bug will write out 8 bytes to OPTIONS file from the + // starting address of BlockBasedTableOptions.read_amp_bytes_per_bit + // which is actually a uint32. Consequently, the value of + // read_amp_bytes_per_bit written in the OPTIONS file is wrong. + // From 6.15, RocksDB will try to parse the read_amp_bytes_per_bit + // from OPTIONS file as a uint32. To be able to load OPTIONS file + // generated by affected releases before the fix, we need to + // manually parse read_amp_bytes_per_bit with this special hack. + uint64_t read_amp_bytes_per_bit = ParseUint64(value); + *(static_cast<uint32_t*>(addr)) = + static_cast<uint32_t>(read_amp_bytes_per_bit); + return Status::OK(); + }}}, + {"enable_index_compression", + {offsetof(struct BlockBasedTableOptions, enable_index_compression), + OptionType::kBoolean, OptionVerificationType::kNormal, + OptionTypeFlags::kNone}}, + {"block_align", + {offsetof(struct BlockBasedTableOptions, block_align), + OptionType::kBoolean, OptionVerificationType::kNormal, + OptionTypeFlags::kNone}}, + {"pin_top_level_index_and_filter", + {offsetof(struct BlockBasedTableOptions, + pin_top_level_index_and_filter), + OptionType::kBoolean, OptionVerificationType::kNormal, + OptionTypeFlags::kNone}}, + {kOptNameMetadataCacheOpts, + OptionTypeInfo::Struct( + kOptNameMetadataCacheOpts, &metadata_cache_options_type_info, + offsetof(struct BlockBasedTableOptions, metadata_cache_options), + OptionVerificationType::kNormal, OptionTypeFlags::kNone)}, + {"block_cache", + {offsetof(struct BlockBasedTableOptions, block_cache), + OptionType::kUnknown, OptionVerificationType::kNormal, + (OptionTypeFlags::kCompareNever | OptionTypeFlags::kDontSerialize), + // Parses the input value as a Cache + [](const ConfigOptions& opts, const std::string&, + const std::string& value, void* addr) { + auto* cache = static_cast<std::shared_ptr<Cache>*>(addr); + return Cache::CreateFromString(opts, value, cache); + }}}, + {"block_cache_compressed", + {offsetof(struct BlockBasedTableOptions, block_cache_compressed), + OptionType::kUnknown, OptionVerificationType::kNormal, + (OptionTypeFlags::kCompareNever | OptionTypeFlags::kDontSerialize), + // Parses the input value as a Cache + [](const ConfigOptions& opts, const std::string&, + const std::string& value, void* addr) { + auto* cache = static_cast<std::shared_ptr<Cache>*>(addr); + return Cache::CreateFromString(opts, value, cache); + }}}, + {"max_auto_readahead_size", + {offsetof(struct BlockBasedTableOptions, max_auto_readahead_size), + OptionType::kSizeT, OptionVerificationType::kNormal, + OptionTypeFlags::kMutable}}, + {"prepopulate_block_cache", + OptionTypeInfo::Enum<BlockBasedTableOptions::PrepopulateBlockCache>( + offsetof(struct BlockBasedTableOptions, prepopulate_block_cache), + &block_base_table_prepopulate_block_cache_string_map, + OptionTypeFlags::kMutable)}, + {"initial_auto_readahead_size", + {offsetof(struct BlockBasedTableOptions, initial_auto_readahead_size), + OptionType::kSizeT, OptionVerificationType::kNormal, + OptionTypeFlags::kMutable}}, + {"num_file_reads_for_auto_readahead", + {offsetof(struct BlockBasedTableOptions, + num_file_reads_for_auto_readahead), + OptionType::kUInt64T, OptionVerificationType::kNormal, + OptionTypeFlags::kMutable}}, + +#endif // ROCKSDB_LITE +}; + +// TODO(myabandeh): We should return an error instead of silently changing the +// options +BlockBasedTableFactory::BlockBasedTableFactory( + const BlockBasedTableOptions& _table_options) + : table_options_(_table_options) { + InitializeOptions(); + RegisterOptions(&table_options_, &block_based_table_type_info); + + const auto table_reader_charged = + table_options_.cache_usage_options.options_overrides + .at(CacheEntryRole::kBlockBasedTableReader) + .charged; + if (table_options_.block_cache && + table_reader_charged == CacheEntryRoleOptions::Decision::kEnabled) { + table_reader_cache_res_mgr_.reset(new ConcurrentCacheReservationManager( + std::make_shared<CacheReservationManagerImpl< + CacheEntryRole::kBlockBasedTableReader>>( + table_options_.block_cache))); + } +} + +void BlockBasedTableFactory::InitializeOptions() { + if (table_options_.flush_block_policy_factory == nullptr) { + table_options_.flush_block_policy_factory.reset( + new FlushBlockBySizePolicyFactory()); + } + if (table_options_.no_block_cache) { + table_options_.block_cache.reset(); + } else if (table_options_.block_cache == nullptr) { + LRUCacheOptions co; + co.capacity = 8 << 20; + // It makes little sense to pay overhead for mid-point insertion while the + // block size is only 8MB. + co.high_pri_pool_ratio = 0.0; + co.low_pri_pool_ratio = 0.0; + table_options_.block_cache = NewLRUCache(co); + } + if (table_options_.block_size_deviation < 0 || + table_options_.block_size_deviation > 100) { + table_options_.block_size_deviation = 0; + } + if (table_options_.block_restart_interval < 1) { + table_options_.block_restart_interval = 1; + } + if (table_options_.index_block_restart_interval < 1) { + table_options_.index_block_restart_interval = 1; + } + if (table_options_.index_type == BlockBasedTableOptions::kHashSearch && + table_options_.index_block_restart_interval != 1) { + // Currently kHashSearch is incompatible with + // index_block_restart_interval > 1 + table_options_.index_block_restart_interval = 1; + } + if (table_options_.partition_filters && + table_options_.index_type != + BlockBasedTableOptions::kTwoLevelIndexSearch) { + // We do not support partitioned filters without partitioning indexes + table_options_.partition_filters = false; + } + auto& options_overrides = + table_options_.cache_usage_options.options_overrides; + const auto options = table_options_.cache_usage_options.options; + for (std::uint32_t i = 0; i < kNumCacheEntryRoles; ++i) { + CacheEntryRole role = static_cast<CacheEntryRole>(i); + auto options_overrides_iter = options_overrides.find(role); + if (options_overrides_iter == options_overrides.end()) { + options_overrides.insert({role, options}); + } else if (options_overrides_iter->second.charged == + CacheEntryRoleOptions::Decision::kFallback) { + options_overrides_iter->second.charged = options.charged; + } + } +} + +Status BlockBasedTableFactory::PrepareOptions(const ConfigOptions& opts) { + InitializeOptions(); + return TableFactory::PrepareOptions(opts); +} + +namespace { +// Different cache kinds use the same keys for physically different values, so +// they must not share an underlying key space with each other. +Status CheckCacheOptionCompatibility(const BlockBasedTableOptions& bbto) { + int cache_count = (bbto.block_cache != nullptr) + + (bbto.block_cache_compressed != nullptr) + + (bbto.persistent_cache != nullptr); + if (cache_count <= 1) { + // Nothing to share / overlap + return Status::OK(); + } + + // Simple pointer equality + if (bbto.block_cache == bbto.block_cache_compressed) { + return Status::InvalidArgument( + "block_cache same as block_cache_compressed not currently supported, " + "and would be bad for performance anyway"); + } + + // More complex test of shared key space, in case the instances are wrappers + // for some shared underlying cache. + CacheKey sentinel_key = CacheKey::CreateUniqueForProcessLifetime(); + static char kRegularBlockCacheMarker = 'b'; + static char kCompressedBlockCacheMarker = 'c'; + static char kPersistentCacheMarker = 'p'; + if (bbto.block_cache) { + bbto.block_cache + ->Insert(sentinel_key.AsSlice(), &kRegularBlockCacheMarker, 1, + GetNoopDeleterForRole<CacheEntryRole::kMisc>()) + .PermitUncheckedError(); + } + if (bbto.block_cache_compressed) { + bbto.block_cache_compressed + ->Insert(sentinel_key.AsSlice(), &kCompressedBlockCacheMarker, 1, + GetNoopDeleterForRole<CacheEntryRole::kMisc>()) + .PermitUncheckedError(); + } + if (bbto.persistent_cache) { + // Note: persistent cache copies the data, not keeping the pointer + bbto.persistent_cache + ->Insert(sentinel_key.AsSlice(), &kPersistentCacheMarker, 1) + .PermitUncheckedError(); + } + // If we get something different from what we inserted, that indicates + // dangerously overlapping key spaces. + if (bbto.block_cache) { + auto handle = bbto.block_cache->Lookup(sentinel_key.AsSlice()); + if (handle) { + auto v = static_cast<char*>(bbto.block_cache->Value(handle)); + char c = *v; + bbto.block_cache->Release(handle); + if (v == &kCompressedBlockCacheMarker) { + return Status::InvalidArgument( + "block_cache and block_cache_compressed share the same key space, " + "which is not supported"); + } else if (c == kPersistentCacheMarker) { + return Status::InvalidArgument( + "block_cache and persistent_cache share the same key space, " + "which is not supported"); + } else if (v != &kRegularBlockCacheMarker) { + return Status::Corruption("Unexpected mutation to block_cache"); + } + } + } + if (bbto.block_cache_compressed) { + auto handle = bbto.block_cache_compressed->Lookup(sentinel_key.AsSlice()); + if (handle) { + auto v = static_cast<char*>(bbto.block_cache_compressed->Value(handle)); + char c = *v; + bbto.block_cache_compressed->Release(handle); + if (v == &kRegularBlockCacheMarker) { + return Status::InvalidArgument( + "block_cache_compressed and block_cache share the same key space, " + "which is not supported"); + } else if (c == kPersistentCacheMarker) { + return Status::InvalidArgument( + "block_cache_compressed and persistent_cache share the same key " + "space, " + "which is not supported"); + } else if (v != &kCompressedBlockCacheMarker) { + return Status::Corruption( + "Unexpected mutation to block_cache_compressed"); + } + } + } + if (bbto.persistent_cache) { + std::unique_ptr<char[]> data; + size_t size = 0; + bbto.persistent_cache->Lookup(sentinel_key.AsSlice(), &data, &size) + .PermitUncheckedError(); + if (data && size > 0) { + if (data[0] == kRegularBlockCacheMarker) { + return Status::InvalidArgument( + "persistent_cache and block_cache share the same key space, " + "which is not supported"); + } else if (data[0] == kCompressedBlockCacheMarker) { + return Status::InvalidArgument( + "persistent_cache and block_cache_compressed share the same key " + "space, " + "which is not supported"); + } else if (data[0] != kPersistentCacheMarker) { + return Status::Corruption("Unexpected mutation to persistent_cache"); + } + } + } + return Status::OK(); +} + +} // namespace + +Status BlockBasedTableFactory::NewTableReader( + const ReadOptions& ro, const TableReaderOptions& table_reader_options, + std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size, + std::unique_ptr<TableReader>* table_reader, + bool prefetch_index_and_filter_in_cache) const { + return BlockBasedTable::Open( + ro, table_reader_options.ioptions, table_reader_options.env_options, + table_options_, table_reader_options.internal_comparator, std::move(file), + file_size, table_reader, table_reader_cache_res_mgr_, + table_reader_options.prefix_extractor, prefetch_index_and_filter_in_cache, + table_reader_options.skip_filters, table_reader_options.level, + table_reader_options.immortal, table_reader_options.largest_seqno, + table_reader_options.force_direct_prefetch, &tail_prefetch_stats_, + table_reader_options.block_cache_tracer, + table_reader_options.max_file_size_for_l0_meta_pin, + table_reader_options.cur_db_session_id, table_reader_options.cur_file_num, + table_reader_options.unique_id); +} + +TableBuilder* BlockBasedTableFactory::NewTableBuilder( + const TableBuilderOptions& table_builder_options, + WritableFileWriter* file) const { + return new BlockBasedTableBuilder(table_options_, table_builder_options, + file); +} + +Status BlockBasedTableFactory::ValidateOptions( + const DBOptions& db_opts, const ColumnFamilyOptions& cf_opts) const { + if (table_options_.index_type == BlockBasedTableOptions::kHashSearch && + cf_opts.prefix_extractor == nullptr) { + return Status::InvalidArgument( + "Hash index is specified for block-based " + "table, but prefix_extractor is not given"); + } + if (table_options_.cache_index_and_filter_blocks && + table_options_.no_block_cache) { + return Status::InvalidArgument( + "Enable cache_index_and_filter_blocks, " + ", but block cache is disabled"); + } + if (table_options_.pin_l0_filter_and_index_blocks_in_cache && + table_options_.no_block_cache) { + return Status::InvalidArgument( + "Enable pin_l0_filter_and_index_blocks_in_cache, " + ", but block cache is disabled"); + } + if (!IsSupportedFormatVersion(table_options_.format_version)) { + return Status::InvalidArgument( + "Unsupported BlockBasedTable format_version. Please check " + "include/rocksdb/table.h for more info"); + } + if (table_options_.block_align && (cf_opts.compression != kNoCompression)) { + return Status::InvalidArgument( + "Enable block_align, but compression " + "enabled"); + } + if (table_options_.block_align && + (table_options_.block_size & (table_options_.block_size - 1))) { + return Status::InvalidArgument( + "Block alignment requested but block size is not a power of 2"); + } + if (table_options_.block_size > std::numeric_limits<uint32_t>::max()) { + return Status::InvalidArgument( + "block size exceeds maximum number (4GiB) allowed"); + } + if (table_options_.data_block_index_type == + BlockBasedTableOptions::kDataBlockBinaryAndHash && + table_options_.data_block_hash_table_util_ratio <= 0) { + return Status::InvalidArgument( + "data_block_hash_table_util_ratio should be greater than 0 when " + "data_block_index_type is set to kDataBlockBinaryAndHash"); + } + if (db_opts.unordered_write && cf_opts.max_successive_merges > 0) { + // TODO(myabandeh): support it + return Status::InvalidArgument( + "max_successive_merges larger than 0 is currently inconsistent with " + "unordered_write"); + } + const auto& options_overrides = + table_options_.cache_usage_options.options_overrides; + for (auto options_overrides_iter = options_overrides.cbegin(); + options_overrides_iter != options_overrides.cend(); + ++options_overrides_iter) { + const CacheEntryRole role = options_overrides_iter->first; + const CacheEntryRoleOptions options = options_overrides_iter->second; + static const std::set<CacheEntryRole> kMemoryChargingSupported = { + CacheEntryRole::kCompressionDictionaryBuildingBuffer, + CacheEntryRole::kFilterConstruction, + CacheEntryRole::kBlockBasedTableReader, CacheEntryRole::kFileMetadata, + CacheEntryRole::kBlobCache}; + if (options.charged != CacheEntryRoleOptions::Decision::kFallback && + kMemoryChargingSupported.count(role) == 0) { + return Status::NotSupported( + "Enable/Disable CacheEntryRoleOptions::charged" + " for CacheEntryRole " + + kCacheEntryRoleToCamelString[static_cast<uint32_t>(role)] + + " is not supported"); + } + if (table_options_.no_block_cache && + options.charged == CacheEntryRoleOptions::Decision::kEnabled) { + return Status::InvalidArgument( + "Enable CacheEntryRoleOptions::charged" + " for CacheEntryRole " + + kCacheEntryRoleToCamelString[static_cast<uint32_t>(role)] + + " but block cache is disabled"); + } + if (role == CacheEntryRole::kBlobCache && + options.charged == CacheEntryRoleOptions::Decision::kEnabled) { + if (cf_opts.blob_cache == nullptr) { + return Status::InvalidArgument( + "Enable CacheEntryRoleOptions::charged" + " for CacheEntryRole " + + kCacheEntryRoleToCamelString[static_cast<uint32_t>(role)] + + " but blob cache is not configured"); + } + if (table_options_.no_block_cache) { + return Status::InvalidArgument( + "Enable CacheEntryRoleOptions::charged" + " for CacheEntryRole " + + kCacheEntryRoleToCamelString[static_cast<uint32_t>(role)] + + " but block cache is disabled"); + } + if (table_options_.block_cache == cf_opts.blob_cache) { + return Status::InvalidArgument( + "Enable CacheEntryRoleOptions::charged" + " for CacheEntryRole " + + kCacheEntryRoleToCamelString[static_cast<uint32_t>(role)] + + " but blob cache is the same as block cache"); + } + if (cf_opts.blob_cache->GetCapacity() > + table_options_.block_cache->GetCapacity()) { + return Status::InvalidArgument( + "Enable CacheEntryRoleOptions::charged" + " for CacheEntryRole " + + kCacheEntryRoleToCamelString[static_cast<uint32_t>(role)] + + " but blob cache capacity is larger than block cache capacity"); + } + } + } + { + Status s = CheckCacheOptionCompatibility(table_options_); + if (!s.ok()) { + return s; + } + } + std::string garbage; + if (!SerializeEnum<ChecksumType>(checksum_type_string_map, + table_options_.checksum, &garbage)) { + return Status::InvalidArgument( + "Unrecognized ChecksumType for checksum: " + + std::to_string(static_cast<uint32_t>(table_options_.checksum))); + } + return TableFactory::ValidateOptions(db_opts, cf_opts); +} + +std::string BlockBasedTableFactory::GetPrintableOptions() const { + std::string ret; + ret.reserve(20000); + const int kBufferSize = 200; + char buffer[kBufferSize]; + + snprintf(buffer, kBufferSize, " flush_block_policy_factory: %s (%p)\n", + table_options_.flush_block_policy_factory->Name(), + static_cast<void*>(table_options_.flush_block_policy_factory.get())); + ret.append(buffer); + snprintf(buffer, kBufferSize, " cache_index_and_filter_blocks: %d\n", + table_options_.cache_index_and_filter_blocks); + ret.append(buffer); + snprintf(buffer, kBufferSize, + " cache_index_and_filter_blocks_with_high_priority: %d\n", + table_options_.cache_index_and_filter_blocks_with_high_priority); + ret.append(buffer); + snprintf(buffer, kBufferSize, + " pin_l0_filter_and_index_blocks_in_cache: %d\n", + table_options_.pin_l0_filter_and_index_blocks_in_cache); + ret.append(buffer); + snprintf(buffer, kBufferSize, " pin_top_level_index_and_filter: %d\n", + table_options_.pin_top_level_index_and_filter); + ret.append(buffer); + snprintf(buffer, kBufferSize, " index_type: %d\n", + table_options_.index_type); + ret.append(buffer); + snprintf(buffer, kBufferSize, " data_block_index_type: %d\n", + table_options_.data_block_index_type); + ret.append(buffer); + snprintf(buffer, kBufferSize, " index_shortening: %d\n", + static_cast<int>(table_options_.index_shortening)); + ret.append(buffer); + snprintf(buffer, kBufferSize, " data_block_hash_table_util_ratio: %lf\n", + table_options_.data_block_hash_table_util_ratio); + ret.append(buffer); + snprintf(buffer, kBufferSize, " checksum: %d\n", table_options_.checksum); + ret.append(buffer); + snprintf(buffer, kBufferSize, " no_block_cache: %d\n", + table_options_.no_block_cache); + ret.append(buffer); + snprintf(buffer, kBufferSize, " block_cache: %p\n", + static_cast<void*>(table_options_.block_cache.get())); + ret.append(buffer); + if (table_options_.block_cache) { + const char* block_cache_name = table_options_.block_cache->Name(); + if (block_cache_name != nullptr) { + snprintf(buffer, kBufferSize, " block_cache_name: %s\n", + block_cache_name); + ret.append(buffer); + } + ret.append(" block_cache_options:\n"); + ret.append(table_options_.block_cache->GetPrintableOptions()); + } + snprintf(buffer, kBufferSize, " block_cache_compressed: %p\n", + static_cast<void*>(table_options_.block_cache_compressed.get())); + ret.append(buffer); + if (table_options_.block_cache_compressed) { + const char* block_cache_compressed_name = + table_options_.block_cache_compressed->Name(); + if (block_cache_compressed_name != nullptr) { + snprintf(buffer, kBufferSize, " block_cache_name: %s\n", + block_cache_compressed_name); + ret.append(buffer); + } + ret.append(" block_cache_compressed_options:\n"); + ret.append(table_options_.block_cache_compressed->GetPrintableOptions()); + } + snprintf(buffer, kBufferSize, " persistent_cache: %p\n", + static_cast<void*>(table_options_.persistent_cache.get())); + ret.append(buffer); + if (table_options_.persistent_cache) { + snprintf(buffer, kBufferSize, " persistent_cache_options:\n"); + ret.append(buffer); + ret.append(table_options_.persistent_cache->GetPrintableOptions()); + } + snprintf(buffer, kBufferSize, " block_size: %" PRIu64 "\n", + table_options_.block_size); + ret.append(buffer); + snprintf(buffer, kBufferSize, " block_size_deviation: %d\n", + table_options_.block_size_deviation); + ret.append(buffer); + snprintf(buffer, kBufferSize, " block_restart_interval: %d\n", + table_options_.block_restart_interval); + ret.append(buffer); + snprintf(buffer, kBufferSize, " index_block_restart_interval: %d\n", + table_options_.index_block_restart_interval); + ret.append(buffer); + snprintf(buffer, kBufferSize, " metadata_block_size: %" PRIu64 "\n", + table_options_.metadata_block_size); + ret.append(buffer); + snprintf(buffer, kBufferSize, " partition_filters: %d\n", + table_options_.partition_filters); + ret.append(buffer); + snprintf(buffer, kBufferSize, " use_delta_encoding: %d\n", + table_options_.use_delta_encoding); + ret.append(buffer); + snprintf(buffer, kBufferSize, " filter_policy: %s\n", + table_options_.filter_policy == nullptr + ? "nullptr" + : table_options_.filter_policy->Name()); + ret.append(buffer); + snprintf(buffer, kBufferSize, " whole_key_filtering: %d\n", + table_options_.whole_key_filtering); + ret.append(buffer); + snprintf(buffer, kBufferSize, " verify_compression: %d\n", + table_options_.verify_compression); + ret.append(buffer); + snprintf(buffer, kBufferSize, " read_amp_bytes_per_bit: %d\n", + table_options_.read_amp_bytes_per_bit); + ret.append(buffer); + snprintf(buffer, kBufferSize, " format_version: %d\n", + table_options_.format_version); + ret.append(buffer); + snprintf(buffer, kBufferSize, " enable_index_compression: %d\n", + table_options_.enable_index_compression); + ret.append(buffer); + snprintf(buffer, kBufferSize, " block_align: %d\n", + table_options_.block_align); + ret.append(buffer); + snprintf(buffer, kBufferSize, + " max_auto_readahead_size: %" ROCKSDB_PRIszt "\n", + table_options_.max_auto_readahead_size); + ret.append(buffer); + snprintf(buffer, kBufferSize, " prepopulate_block_cache: %d\n", + static_cast<int>(table_options_.prepopulate_block_cache)); + ret.append(buffer); + snprintf(buffer, kBufferSize, + " initial_auto_readahead_size: %" ROCKSDB_PRIszt "\n", + table_options_.initial_auto_readahead_size); + ret.append(buffer); + snprintf(buffer, kBufferSize, + " num_file_reads_for_auto_readahead: %" PRIu64 "\n", + table_options_.num_file_reads_for_auto_readahead); + ret.append(buffer); + return ret; +} + +const void* BlockBasedTableFactory::GetOptionsPtr( + const std::string& name) const { + if (name == kBlockCacheOpts()) { + if (table_options_.no_block_cache) { + return nullptr; + } else { + return table_options_.block_cache.get(); + } + } else { + return TableFactory::GetOptionsPtr(name); + } +} + +#ifndef ROCKSDB_LITE +// Take a default BlockBasedTableOptions "table_options" in addition to a +// map "opts_map" of option name to option value to construct the new +// BlockBasedTableOptions "new_table_options". +// +// Below are the instructions of how to config some non-primitive-typed +// options in BlockBasedTableOptions: +// +// * filter_policy: +// We currently only support the following FilterPolicy in the convenience +// functions: +// - BloomFilter: use "bloomfilter:[bits_per_key]:[use_block_based_builder]" +// to specify BloomFilter. The above string is equivalent to calling +// NewBloomFilterPolicy(bits_per_key, use_block_based_builder). +// [Example]: +// - Pass {"filter_policy", "bloomfilter:4:true"} in +// GetBlockBasedTableOptionsFromMap to use a BloomFilter with 4-bits +// per key and use_block_based_builder enabled. +// +// * block_cache / block_cache_compressed: +// We currently only support LRU cache in the GetOptions API. The LRU +// cache can be set by directly specifying its size. +// [Example]: +// - Passing {"block_cache", "1M"} in GetBlockBasedTableOptionsFromMap is +// equivalent to setting block_cache using NewLRUCache(1024 * 1024). +// +// @param table_options the default options of the output "new_table_options". +// @param opts_map an option name to value map for specifying how +// "new_table_options" should be set. +// @param new_table_options the resulting options based on "table_options" +// with the change specified in "opts_map". +// @param input_strings_escaped when set to true, each escaped characters +// prefixed by '\' in the values of the opts_map will be further converted +// back to the raw string before assigning to the associated options. +// @param ignore_unknown_options when set to true, unknown options are ignored +// instead of resulting in an unknown-option error. +// @return Status::OK() on success. Otherwise, a non-ok status indicating +// error will be returned, and "new_table_options" will be set to +// "table_options". +Status BlockBasedTableFactory::ParseOption(const ConfigOptions& config_options, + const OptionTypeInfo& opt_info, + const std::string& opt_name, + const std::string& opt_value, + void* opt_ptr) { + Status status = TableFactory::ParseOption(config_options, opt_info, opt_name, + opt_value, opt_ptr); + if (config_options.input_strings_escaped && !status.ok()) { // Got an error + // !input_strings_escaped indicates the old API, where everything is + // parsable. + if (opt_info.IsByName()) { + status = Status::OK(); + } + } + return status; +} + +Status GetBlockBasedTableOptionsFromString( + const BlockBasedTableOptions& table_options, const std::string& opts_str, + BlockBasedTableOptions* new_table_options) { + ConfigOptions config_options; + config_options.input_strings_escaped = false; + config_options.ignore_unknown_options = false; + config_options.invoke_prepare_options = false; + config_options.ignore_unsupported_options = false; + + return GetBlockBasedTableOptionsFromString(config_options, table_options, + opts_str, new_table_options); +} +Status GetBlockBasedTableOptionsFromString( + const ConfigOptions& config_options, + const BlockBasedTableOptions& table_options, const std::string& opts_str, + BlockBasedTableOptions* new_table_options) { + std::unordered_map<std::string, std::string> opts_map; + Status s = StringToMap(opts_str, &opts_map); + if (!s.ok()) { + return s; + } + s = GetBlockBasedTableOptionsFromMap(config_options, table_options, opts_map, + new_table_options); + // Translate any errors (NotFound, NotSupported, to InvalidArgument + if (s.ok() || s.IsInvalidArgument()) { + return s; + } else { + return Status::InvalidArgument(s.getState()); + } +} + +Status GetBlockBasedTableOptionsFromMap( + const BlockBasedTableOptions& table_options, + const std::unordered_map<std::string, std::string>& opts_map, + BlockBasedTableOptions* new_table_options, bool input_strings_escaped, + bool ignore_unknown_options) { + ConfigOptions config_options; + config_options.input_strings_escaped = input_strings_escaped; + config_options.ignore_unknown_options = ignore_unknown_options; + config_options.invoke_prepare_options = false; + + return GetBlockBasedTableOptionsFromMap(config_options, table_options, + opts_map, new_table_options); +} + +Status GetBlockBasedTableOptionsFromMap( + const ConfigOptions& config_options, + const BlockBasedTableOptions& table_options, + const std::unordered_map<std::string, std::string>& opts_map, + BlockBasedTableOptions* new_table_options) { + assert(new_table_options); + BlockBasedTableFactory bbtf(table_options); + Status s = bbtf.ConfigureFromMap(config_options, opts_map); + if (s.ok()) { + *new_table_options = *(bbtf.GetOptions<BlockBasedTableOptions>()); + } else { + *new_table_options = table_options; + } + return s; +} +#endif // !ROCKSDB_LITE + +TableFactory* NewBlockBasedTableFactory( + const BlockBasedTableOptions& _table_options) { + return new BlockBasedTableFactory(_table_options); +} + +const std::string BlockBasedTablePropertyNames::kIndexType = + "rocksdb.block.based.table.index.type"; +const std::string BlockBasedTablePropertyNames::kWholeKeyFiltering = + "rocksdb.block.based.table.whole.key.filtering"; +const std::string BlockBasedTablePropertyNames::kPrefixFiltering = + "rocksdb.block.based.table.prefix.filtering"; +const std::string kHashIndexPrefixesBlock = "rocksdb.hashindex.prefixes"; +const std::string kHashIndexPrefixesMetadataBlock = + "rocksdb.hashindex.metadata"; +const std::string kPropTrue = "1"; +const std::string kPropFalse = "0"; + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/block_based_table_factory.h b/src/rocksdb/table/block_based/block_based_table_factory.h new file mode 100644 index 000000000..3166cd3cc --- /dev/null +++ b/src/rocksdb/table/block_based/block_based_table_factory.h @@ -0,0 +1,101 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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 <stdint.h> + +#include <memory> +#include <string> + +#include "cache/cache_reservation_manager.h" +#include "port/port.h" +#include "rocksdb/flush_block_policy.h" +#include "rocksdb/table.h" + +namespace ROCKSDB_NAMESPACE { +struct ColumnFamilyOptions; +struct ConfigOptions; +struct DBOptions; +struct EnvOptions; + +class BlockBasedTableBuilder; +class RandomAccessFileReader; +class WritableFileWriter; + +// A class used to track actual bytes written from the tail in the recent SST +// file opens, and provide a suggestion for following open. +class TailPrefetchStats { + public: + void RecordEffectiveSize(size_t len); + // 0 indicates no information to determine. + size_t GetSuggestedPrefetchSize(); + + private: + const static size_t kNumTracked = 32; + size_t records_[kNumTracked]; + port::Mutex mutex_; + size_t next_ = 0; + size_t num_records_ = 0; +}; + +class BlockBasedTableFactory : public TableFactory { + public: + explicit BlockBasedTableFactory( + const BlockBasedTableOptions& table_options = BlockBasedTableOptions()); + + ~BlockBasedTableFactory() {} + + // Method to allow CheckedCast to work for this class + static const char* kClassName() { return kBlockBasedTableName(); } + + const char* Name() const override { return kBlockBasedTableName(); } + + using TableFactory::NewTableReader; + Status NewTableReader( + const ReadOptions& ro, const TableReaderOptions& table_reader_options, + std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size, + std::unique_ptr<TableReader>* table_reader, + bool prefetch_index_and_filter_in_cache = true) const override; + + TableBuilder* NewTableBuilder( + const TableBuilderOptions& table_builder_options, + WritableFileWriter* file) const override; + + // Valdates the specified DB Options. + Status ValidateOptions(const DBOptions& db_opts, + const ColumnFamilyOptions& cf_opts) const override; + Status PrepareOptions(const ConfigOptions& opts) override; + + std::string GetPrintableOptions() const override; + + bool IsDeleteRangeSupported() const override { return true; } + + TailPrefetchStats* tail_prefetch_stats() { return &tail_prefetch_stats_; } + + protected: + const void* GetOptionsPtr(const std::string& name) const override; +#ifndef ROCKSDB_LITE + Status ParseOption(const ConfigOptions& config_options, + const OptionTypeInfo& opt_info, + const std::string& opt_name, const std::string& opt_value, + void* opt_ptr) override; +#endif + void InitializeOptions(); + + private: + BlockBasedTableOptions table_options_; + std::shared_ptr<CacheReservationManager> table_reader_cache_res_mgr_; + mutable TailPrefetchStats tail_prefetch_stats_; +}; + +extern const std::string kHashIndexPrefixesBlock; +extern const std::string kHashIndexPrefixesMetadataBlock; +extern const std::string kPropTrue; +extern const std::string kPropFalse; +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/block_based_table_iterator.cc b/src/rocksdb/table/block_based/block_based_table_iterator.cc new file mode 100644 index 000000000..d2605670f --- /dev/null +++ b/src/rocksdb/table/block_based/block_based_table_iterator.cc @@ -0,0 +1,459 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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/block_based/block_based_table_iterator.h" + +namespace ROCKSDB_NAMESPACE { + +void BlockBasedTableIterator::SeekToFirst() { SeekImpl(nullptr, false); } + +void BlockBasedTableIterator::Seek(const Slice& target) { + SeekImpl(&target, true); +} + +void BlockBasedTableIterator::SeekImpl(const Slice* target, + bool async_prefetch) { + bool is_first_pass = true; + if (async_read_in_progress_) { + AsyncInitDataBlock(false); + is_first_pass = false; + } + + is_out_of_bound_ = false; + is_at_first_key_from_index_ = false; + if (target && !CheckPrefixMayMatch(*target, IterDirection::kForward)) { + ResetDataIter(); + return; + } + + bool need_seek_index = true; + if (block_iter_points_to_real_block_ && block_iter_.Valid()) { + // Reseek. + prev_block_offset_ = index_iter_->value().handle.offset(); + + if (target) { + // We can avoid an index seek if: + // 1. The new seek key is larger than the current key + // 2. The new seek key is within the upper bound of the block + // Since we don't necessarily know the internal key for either + // the current key or the upper bound, we check user keys and + // exclude the equality case. Considering internal keys can + // improve for the boundary cases, but it would complicate the + // code. + if (user_comparator_.Compare(ExtractUserKey(*target), + block_iter_.user_key()) > 0 && + user_comparator_.Compare(ExtractUserKey(*target), + index_iter_->user_key()) < 0) { + need_seek_index = false; + } + } + } + + if (need_seek_index) { + if (target) { + index_iter_->Seek(*target); + } else { + index_iter_->SeekToFirst(); + } + + if (!index_iter_->Valid()) { + ResetDataIter(); + return; + } + } + + IndexValue v = index_iter_->value(); + const bool same_block = block_iter_points_to_real_block_ && + v.handle.offset() == prev_block_offset_; + + if (!v.first_internal_key.empty() && !same_block && + (!target || icomp_.Compare(*target, v.first_internal_key) <= 0) && + allow_unprepared_value_) { + // Index contains the first key of the block, and it's >= target. + // We can defer reading the block. + is_at_first_key_from_index_ = true; + // ResetDataIter() will invalidate block_iter_. Thus, there is no need to + // call CheckDataBlockWithinUpperBound() to check for iterate_upper_bound + // as that will be done later when the data block is actually read. + ResetDataIter(); + } else { + // Need to use the data block. + if (!same_block) { + if (read_options_.async_io && async_prefetch) { + if (is_first_pass) { + AsyncInitDataBlock(is_first_pass); + } + if (async_read_in_progress_) { + // Status::TryAgain indicates asynchronous request for retrieval of + // data blocks has been submitted. So it should return at this point + // and Seek should be called again to retrieve the requested block and + // execute the remaining code. + return; + } + } else { + InitDataBlock(); + } + } else { + // When the user does a reseek, the iterate_upper_bound might have + // changed. CheckDataBlockWithinUpperBound() needs to be called + // explicitly if the reseek ends up in the same data block. + // If the reseek ends up in a different block, InitDataBlock() will do + // the iterator upper bound check. + CheckDataBlockWithinUpperBound(); + } + + if (target) { + block_iter_.Seek(*target); + } else { + block_iter_.SeekToFirst(); + } + FindKeyForward(); + } + + CheckOutOfBound(); + + if (target) { + assert(!Valid() || icomp_.Compare(*target, key()) <= 0); + } +} + +void BlockBasedTableIterator::SeekForPrev(const Slice& target) { + is_out_of_bound_ = false; + is_at_first_key_from_index_ = false; + // For now totally disable prefix seek in auto prefix mode because we don't + // have logic + if (!CheckPrefixMayMatch(target, IterDirection::kBackward)) { + ResetDataIter(); + return; + } + + SavePrevIndexValue(); + + // Call Seek() rather than SeekForPrev() in the index block, because the + // target data block will likely to contain the position for `target`, the + // same as Seek(), rather than than before. + // For example, if we have three data blocks, each containing two keys: + // [2, 4] [6, 8] [10, 12] + // (the keys in the index block would be [4, 8, 12]) + // and the user calls SeekForPrev(7), we need to go to the second block, + // just like if they call Seek(7). + // The only case where the block is difference is when they seek to a position + // in the boundary. For example, if they SeekForPrev(5), we should go to the + // first block, rather than the second. However, we don't have the information + // to distinguish the two unless we read the second block. In this case, we'll + // end up with reading two blocks. + index_iter_->Seek(target); + + if (!index_iter_->Valid()) { + auto seek_status = index_iter_->status(); + // Check for IO error + if (!seek_status.IsNotFound() && !seek_status.ok()) { + ResetDataIter(); + return; + } + + // With prefix index, Seek() returns NotFound if the prefix doesn't exist + if (seek_status.IsNotFound()) { + // Any key less than the target is fine for prefix seek + ResetDataIter(); + return; + } else { + index_iter_->SeekToLast(); + } + // Check for IO error + if (!index_iter_->Valid()) { + ResetDataIter(); + return; + } + } + + InitDataBlock(); + + block_iter_.SeekForPrev(target); + + FindKeyBackward(); + CheckDataBlockWithinUpperBound(); + assert(!block_iter_.Valid() || + icomp_.Compare(target, block_iter_.key()) >= 0); +} + +void BlockBasedTableIterator::SeekToLast() { + is_out_of_bound_ = false; + is_at_first_key_from_index_ = false; + SavePrevIndexValue(); + index_iter_->SeekToLast(); + if (!index_iter_->Valid()) { + ResetDataIter(); + return; + } + InitDataBlock(); + block_iter_.SeekToLast(); + FindKeyBackward(); + CheckDataBlockWithinUpperBound(); +} + +void BlockBasedTableIterator::Next() { + if (is_at_first_key_from_index_ && !MaterializeCurrentBlock()) { + return; + } + assert(block_iter_points_to_real_block_); + block_iter_.Next(); + FindKeyForward(); + CheckOutOfBound(); +} + +bool BlockBasedTableIterator::NextAndGetResult(IterateResult* result) { + Next(); + bool is_valid = Valid(); + if (is_valid) { + result->key = key(); + result->bound_check_result = UpperBoundCheckResult(); + result->value_prepared = !is_at_first_key_from_index_; + } + return is_valid; +} + +void BlockBasedTableIterator::Prev() { + if (is_at_first_key_from_index_) { + is_at_first_key_from_index_ = false; + + index_iter_->Prev(); + if (!index_iter_->Valid()) { + return; + } + + InitDataBlock(); + block_iter_.SeekToLast(); + } else { + assert(block_iter_points_to_real_block_); + block_iter_.Prev(); + } + + FindKeyBackward(); +} + +void BlockBasedTableIterator::InitDataBlock() { + BlockHandle data_block_handle = index_iter_->value().handle; + if (!block_iter_points_to_real_block_ || + data_block_handle.offset() != prev_block_offset_ || + // if previous attempt of reading the block missed cache, try again + block_iter_.status().IsIncomplete()) { + if (block_iter_points_to_real_block_) { + ResetDataIter(); + } + auto* rep = table_->get_rep(); + + bool is_for_compaction = + lookup_context_.caller == TableReaderCaller::kCompaction; + // Prefetch additional data for range scans (iterators). + // Implicit auto readahead: + // Enabled after 2 sequential IOs when ReadOptions.readahead_size == 0. + // Explicit user requested readahead: + // Enabled from the very first IO when ReadOptions.readahead_size is set. + block_prefetcher_.PrefetchIfNeeded( + rep, data_block_handle, read_options_.readahead_size, is_for_compaction, + /*no_sequential_checking=*/false, read_options_.rate_limiter_priority); + Status s; + table_->NewDataBlockIterator<DataBlockIter>( + read_options_, data_block_handle, &block_iter_, BlockType::kData, + /*get_context=*/nullptr, &lookup_context_, + block_prefetcher_.prefetch_buffer(), + /*for_compaction=*/is_for_compaction, /*async_read=*/false, s); + block_iter_points_to_real_block_ = true; + CheckDataBlockWithinUpperBound(); + } +} + +void BlockBasedTableIterator::AsyncInitDataBlock(bool is_first_pass) { + BlockHandle data_block_handle = index_iter_->value().handle; + bool is_for_compaction = + lookup_context_.caller == TableReaderCaller::kCompaction; + if (is_first_pass) { + if (!block_iter_points_to_real_block_ || + data_block_handle.offset() != prev_block_offset_ || + // if previous attempt of reading the block missed cache, try again + block_iter_.status().IsIncomplete()) { + if (block_iter_points_to_real_block_) { + ResetDataIter(); + } + auto* rep = table_->get_rep(); + // Prefetch additional data for range scans (iterators). + // Implicit auto readahead: + // Enabled after 2 sequential IOs when ReadOptions.readahead_size == 0. + // Explicit user requested readahead: + // Enabled from the very first IO when ReadOptions.readahead_size is + // set. + // In case of async_io with Implicit readahead, block_prefetcher_ will + // always the create the prefetch buffer by setting no_sequential_checking + // = true. + block_prefetcher_.PrefetchIfNeeded( + rep, data_block_handle, read_options_.readahead_size, + is_for_compaction, /*no_sequential_checking=*/read_options_.async_io, + read_options_.rate_limiter_priority); + + Status s; + table_->NewDataBlockIterator<DataBlockIter>( + read_options_, data_block_handle, &block_iter_, BlockType::kData, + /*get_context=*/nullptr, &lookup_context_, + block_prefetcher_.prefetch_buffer(), + /*for_compaction=*/is_for_compaction, /*async_read=*/true, s); + + if (s.IsTryAgain()) { + async_read_in_progress_ = true; + return; + } + } + } else { + // Second pass will call the Poll to get the data block which has been + // requested asynchronously. + Status s; + table_->NewDataBlockIterator<DataBlockIter>( + read_options_, data_block_handle, &block_iter_, BlockType::kData, + /*get_context=*/nullptr, &lookup_context_, + block_prefetcher_.prefetch_buffer(), + /*for_compaction=*/is_for_compaction, /*async_read=*/false, s); + } + block_iter_points_to_real_block_ = true; + CheckDataBlockWithinUpperBound(); + async_read_in_progress_ = false; +} + +bool BlockBasedTableIterator::MaterializeCurrentBlock() { + assert(is_at_first_key_from_index_); + assert(!block_iter_points_to_real_block_); + assert(index_iter_->Valid()); + + is_at_first_key_from_index_ = false; + InitDataBlock(); + assert(block_iter_points_to_real_block_); + + if (!block_iter_.status().ok()) { + return false; + } + + block_iter_.SeekToFirst(); + + if (!block_iter_.Valid() || + icomp_.Compare(block_iter_.key(), + index_iter_->value().first_internal_key) != 0) { + block_iter_.Invalidate(Status::Corruption( + "first key in index doesn't match first key in block")); + return false; + } + + return true; +} + +void BlockBasedTableIterator::FindKeyForward() { + // This method's code is kept short to make it likely to be inlined. + + assert(!is_out_of_bound_); + assert(block_iter_points_to_real_block_); + + if (!block_iter_.Valid()) { + // This is the only call site of FindBlockForward(), but it's extracted into + // a separate method to keep FindKeyForward() short and likely to be + // inlined. When transitioning to a different block, we call + // FindBlockForward(), which is much longer and is probably not inlined. + FindBlockForward(); + } else { + // This is the fast path that avoids a function call. + } +} + +void BlockBasedTableIterator::FindBlockForward() { + // TODO the while loop inherits from two-level-iterator. We don't know + // whether a block can be empty so it can be replaced by an "if". + do { + if (!block_iter_.status().ok()) { + return; + } + // Whether next data block is out of upper bound, if there is one. + const bool next_block_is_out_of_bound = + read_options_.iterate_upper_bound != nullptr && + block_iter_points_to_real_block_ && + block_upper_bound_check_ == BlockUpperBound::kUpperBoundInCurBlock; + assert(!next_block_is_out_of_bound || + user_comparator_.CompareWithoutTimestamp( + *read_options_.iterate_upper_bound, /*a_has_ts=*/false, + index_iter_->user_key(), /*b_has_ts=*/true) <= 0); + ResetDataIter(); + index_iter_->Next(); + if (next_block_is_out_of_bound) { + // The next block is out of bound. No need to read it. + TEST_SYNC_POINT_CALLBACK("BlockBasedTableIterator:out_of_bound", nullptr); + // We need to make sure this is not the last data block before setting + // is_out_of_bound_, since the index key for the last data block can be + // larger than smallest key of the next file on the same level. + if (index_iter_->Valid()) { + is_out_of_bound_ = true; + } + return; + } + + if (!index_iter_->Valid()) { + return; + } + + IndexValue v = index_iter_->value(); + + if (!v.first_internal_key.empty() && allow_unprepared_value_) { + // Index contains the first key of the block. Defer reading the block. + is_at_first_key_from_index_ = true; + return; + } + + InitDataBlock(); + block_iter_.SeekToFirst(); + } while (!block_iter_.Valid()); +} + +void BlockBasedTableIterator::FindKeyBackward() { + while (!block_iter_.Valid()) { + if (!block_iter_.status().ok()) { + return; + } + + ResetDataIter(); + index_iter_->Prev(); + + if (index_iter_->Valid()) { + InitDataBlock(); + block_iter_.SeekToLast(); + } else { + return; + } + } + + // We could have check lower bound here too, but we opt not to do it for + // code simplicity. +} + +void BlockBasedTableIterator::CheckOutOfBound() { + if (read_options_.iterate_upper_bound != nullptr && + block_upper_bound_check_ != BlockUpperBound::kUpperBoundBeyondCurBlock && + Valid()) { + is_out_of_bound_ = + user_comparator_.CompareWithoutTimestamp( + *read_options_.iterate_upper_bound, /*a_has_ts=*/false, user_key(), + /*b_has_ts=*/true) <= 0; + } +} + +void BlockBasedTableIterator::CheckDataBlockWithinUpperBound() { + if (read_options_.iterate_upper_bound != nullptr && + block_iter_points_to_real_block_) { + block_upper_bound_check_ = (user_comparator_.CompareWithoutTimestamp( + *read_options_.iterate_upper_bound, + /*a_has_ts=*/false, index_iter_->user_key(), + /*b_has_ts=*/true) > 0) + ? BlockUpperBound::kUpperBoundBeyondCurBlock + : BlockUpperBound::kUpperBoundInCurBlock; + } +} +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/block_based_table_iterator.h b/src/rocksdb/table/block_based/block_based_table_iterator.h new file mode 100644 index 000000000..a2918b248 --- /dev/null +++ b/src/rocksdb/table/block_based/block_based_table_iterator.h @@ -0,0 +1,280 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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 "table/block_based/block_based_table_reader.h" +#include "table/block_based/block_based_table_reader_impl.h" +#include "table/block_based/block_prefetcher.h" +#include "table/block_based/reader_common.h" + +namespace ROCKSDB_NAMESPACE { +// Iterates over the contents of BlockBasedTable. +class BlockBasedTableIterator : public InternalIteratorBase<Slice> { + // compaction_readahead_size: its value will only be used if for_compaction = + // true + // @param read_options Must outlive this iterator. + public: + BlockBasedTableIterator( + const BlockBasedTable* table, const ReadOptions& read_options, + const InternalKeyComparator& icomp, + std::unique_ptr<InternalIteratorBase<IndexValue>>&& index_iter, + bool check_filter, bool need_upper_bound_check, + const SliceTransform* prefix_extractor, TableReaderCaller caller, + size_t compaction_readahead_size = 0, bool allow_unprepared_value = false) + : index_iter_(std::move(index_iter)), + table_(table), + read_options_(read_options), + icomp_(icomp), + user_comparator_(icomp.user_comparator()), + pinned_iters_mgr_(nullptr), + prefix_extractor_(prefix_extractor), + lookup_context_(caller), + block_prefetcher_( + compaction_readahead_size, + table_->get_rep()->table_options.initial_auto_readahead_size), + allow_unprepared_value_(allow_unprepared_value), + block_iter_points_to_real_block_(false), + check_filter_(check_filter), + need_upper_bound_check_(need_upper_bound_check), + async_read_in_progress_(false) {} + + ~BlockBasedTableIterator() {} + + void Seek(const Slice& target) override; + void SeekForPrev(const Slice& target) override; + void SeekToFirst() override; + void SeekToLast() override; + void Next() final override; + bool NextAndGetResult(IterateResult* result) override; + void Prev() override; + bool Valid() const override { + return !is_out_of_bound_ && + (is_at_first_key_from_index_ || + (block_iter_points_to_real_block_ && block_iter_.Valid())); + } + Slice key() const override { + assert(Valid()); + if (is_at_first_key_from_index_) { + return index_iter_->value().first_internal_key; + } else { + return block_iter_.key(); + } + } + Slice user_key() const override { + assert(Valid()); + if (is_at_first_key_from_index_) { + return ExtractUserKey(index_iter_->value().first_internal_key); + } else { + return block_iter_.user_key(); + } + } + bool PrepareValue() override { + assert(Valid()); + + if (!is_at_first_key_from_index_) { + return true; + } + + return const_cast<BlockBasedTableIterator*>(this) + ->MaterializeCurrentBlock(); + } + Slice value() const override { + // PrepareValue() must have been called. + assert(!is_at_first_key_from_index_); + assert(Valid()); + + return block_iter_.value(); + } + Status status() const override { + // Prefix index set status to NotFound when the prefix does not exist + if (!index_iter_->status().ok() && !index_iter_->status().IsNotFound()) { + return index_iter_->status(); + } else if (block_iter_points_to_real_block_) { + return block_iter_.status(); + } else if (async_read_in_progress_) { + return Status::TryAgain(); + } else { + return Status::OK(); + } + } + + inline IterBoundCheck UpperBoundCheckResult() override { + if (is_out_of_bound_) { + return IterBoundCheck::kOutOfBound; + } else if (block_upper_bound_check_ == + BlockUpperBound::kUpperBoundBeyondCurBlock) { + assert(!is_out_of_bound_); + return IterBoundCheck::kInbound; + } else { + return IterBoundCheck::kUnknown; + } + } + + void SetPinnedItersMgr(PinnedIteratorsManager* pinned_iters_mgr) override { + pinned_iters_mgr_ = pinned_iters_mgr; + } + bool IsKeyPinned() const override { + // Our key comes either from block_iter_'s current key + // or index_iter_'s current *value*. + return pinned_iters_mgr_ && pinned_iters_mgr_->PinningEnabled() && + ((is_at_first_key_from_index_ && index_iter_->IsValuePinned()) || + (block_iter_points_to_real_block_ && block_iter_.IsKeyPinned())); + } + bool IsValuePinned() const override { + assert(!is_at_first_key_from_index_); + assert(Valid()); + + // BlockIter::IsValuePinned() is always true. No need to check + return pinned_iters_mgr_ && pinned_iters_mgr_->PinningEnabled() && + block_iter_points_to_real_block_; + } + + void ResetDataIter() { + if (block_iter_points_to_real_block_) { + if (pinned_iters_mgr_ != nullptr && pinned_iters_mgr_->PinningEnabled()) { + block_iter_.DelegateCleanupsTo(pinned_iters_mgr_); + } + block_iter_.Invalidate(Status::OK()); + block_iter_points_to_real_block_ = false; + } + block_upper_bound_check_ = BlockUpperBound::kUnknown; + } + + void SavePrevIndexValue() { + if (block_iter_points_to_real_block_) { + // Reseek. If they end up with the same data block, we shouldn't re-fetch + // the same data block. + prev_block_offset_ = index_iter_->value().handle.offset(); + } + } + + void GetReadaheadState(ReadaheadFileInfo* readahead_file_info) override { + if (block_prefetcher_.prefetch_buffer() != nullptr && + read_options_.adaptive_readahead) { + block_prefetcher_.prefetch_buffer()->GetReadaheadState( + &(readahead_file_info->data_block_readahead_info)); + if (index_iter_) { + index_iter_->GetReadaheadState(readahead_file_info); + } + } + } + + void SetReadaheadState(ReadaheadFileInfo* readahead_file_info) override { + if (read_options_.adaptive_readahead) { + block_prefetcher_.SetReadaheadState( + &(readahead_file_info->data_block_readahead_info)); + if (index_iter_) { + index_iter_->SetReadaheadState(readahead_file_info); + } + } + } + + std::unique_ptr<InternalIteratorBase<IndexValue>> index_iter_; + + private: + enum class IterDirection { + kForward, + kBackward, + }; + // This enum indicates whether the upper bound falls into current block + // or beyond. + // +-------------+ + // | cur block | <-- (1) + // +-------------+ + // <-- (2) + // --- <boundary key> --- + // <-- (3) + // +-------------+ + // | next block | <-- (4) + // ...... + // + // When the block is smaller than <boundary key>, kUpperBoundInCurBlock + // is the value to use. The examples are (1) or (2) in the graph. It means + // all keys in the next block or beyond will be out of bound. Keys within + // the current block may or may not be out of bound. + // When the block is larger or equal to <boundary key>, + // kUpperBoundBeyondCurBlock is to be used. The examples are (3) and (4) + // in the graph. It means that all keys in the current block is within the + // upper bound and keys in the next block may or may not be within the uppder + // bound. + // If the boundary key hasn't been checked against the upper bound, + // kUnknown can be used. + enum class BlockUpperBound { + kUpperBoundInCurBlock, + kUpperBoundBeyondCurBlock, + kUnknown, + }; + + const BlockBasedTable* table_; + const ReadOptions& read_options_; + const InternalKeyComparator& icomp_; + UserComparatorWrapper user_comparator_; + PinnedIteratorsManager* pinned_iters_mgr_; + DataBlockIter block_iter_; + const SliceTransform* prefix_extractor_; + uint64_t prev_block_offset_ = std::numeric_limits<uint64_t>::max(); + BlockCacheLookupContext lookup_context_; + + BlockPrefetcher block_prefetcher_; + + const bool allow_unprepared_value_; + // True if block_iter_ is initialized and points to the same block + // as index iterator. + bool block_iter_points_to_real_block_; + // See InternalIteratorBase::IsOutOfBound(). + bool is_out_of_bound_ = false; + // How current data block's boundary key with the next block is compared with + // iterate upper bound. + BlockUpperBound block_upper_bound_check_ = BlockUpperBound::kUnknown; + // True if we're standing at the first key of a block, and we haven't loaded + // that block yet. A call to PrepareValue() will trigger loading the block. + bool is_at_first_key_from_index_ = false; + bool check_filter_; + // TODO(Zhongyi): pick a better name + bool need_upper_bound_check_; + + bool async_read_in_progress_; + + // If `target` is null, seek to first. + void SeekImpl(const Slice* target, bool async_prefetch); + + void InitDataBlock(); + void AsyncInitDataBlock(bool is_first_pass); + bool MaterializeCurrentBlock(); + void FindKeyForward(); + void FindBlockForward(); + void FindKeyBackward(); + void CheckOutOfBound(); + + // Check if data block is fully within iterate_upper_bound. + // + // Note MyRocks may update iterate bounds between seek. To workaround it, + // we need to check and update data_block_within_upper_bound_ accordingly. + void CheckDataBlockWithinUpperBound(); + + bool CheckPrefixMayMatch(const Slice& ikey, IterDirection direction) { + if (need_upper_bound_check_ && direction == IterDirection::kBackward) { + // Upper bound check isn't sufficient for backward direction to + // guarantee the same result as total order, so disable prefix + // check. + return true; + } + if (check_filter_ && !table_->PrefixRangeMayMatch( + ikey, read_options_, prefix_extractor_, + need_upper_bound_check_, &lookup_context_)) { + // TODO remember the iterator is invalidated because of prefix + // match. This can avoid the upper level file iterator to falsely + // believe the position is the end of the SST file and move to + // the first key of the next file. + ResetDataIter(); + return false; + } + return true; + } +}; +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/block_based_table_reader.cc b/src/rocksdb/table/block_based/block_based_table_reader.cc new file mode 100644 index 000000000..43962ba1d --- /dev/null +++ b/src/rocksdb/table/block_based/block_based_table_reader.cc @@ -0,0 +1,3092 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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/block_based/block_based_table_reader.h" + +#include <algorithm> +#include <array> +#include <atomic> +#include <cstdint> +#include <limits> +#include <memory> +#include <string> +#include <unordered_set> +#include <utility> +#include <vector> + +#include "cache/cache_entry_roles.h" +#include "cache/cache_key.h" +#include "db/compaction/compaction_picker.h" +#include "db/dbformat.h" +#include "db/pinned_iterators_manager.h" +#include "file/file_prefetch_buffer.h" +#include "file/file_util.h" +#include "file/random_access_file_reader.h" +#include "logging/logging.h" +#include "monitoring/perf_context_imp.h" +#include "port/lang.h" +#include "rocksdb/cache.h" +#include "rocksdb/comparator.h" +#include "rocksdb/convenience.h" +#include "rocksdb/env.h" +#include "rocksdb/file_system.h" +#include "rocksdb/filter_policy.h" +#include "rocksdb/iterator.h" +#include "rocksdb/options.h" +#include "rocksdb/snapshot.h" +#include "rocksdb/statistics.h" +#include "rocksdb/system_clock.h" +#include "rocksdb/table.h" +#include "rocksdb/table_properties.h" +#include "rocksdb/trace_record.h" +#include "table/block_based/binary_search_index_reader.h" +#include "table/block_based/block.h" +#include "table/block_based/block_based_table_factory.h" +#include "table/block_based/block_based_table_iterator.h" +#include "table/block_based/block_like_traits.h" +#include "table/block_based/block_prefix_index.h" +#include "table/block_based/block_type.h" +#include "table/block_based/filter_block.h" +#include "table/block_based/filter_policy_internal.h" +#include "table/block_based/full_filter_block.h" +#include "table/block_based/hash_index_reader.h" +#include "table/block_based/partitioned_filter_block.h" +#include "table/block_based/partitioned_index_reader.h" +#include "table/block_fetcher.h" +#include "table/format.h" +#include "table/get_context.h" +#include "table/internal_iterator.h" +#include "table/meta_blocks.h" +#include "table/multiget_context.h" +#include "table/persistent_cache_helper.h" +#include "table/persistent_cache_options.h" +#include "table/sst_file_writer_collectors.h" +#include "table/two_level_iterator.h" +#include "test_util/sync_point.h" +#include "util/coding.h" +#include "util/crc32c.h" +#include "util/stop_watch.h" +#include "util/string_util.h" + +namespace ROCKSDB_NAMESPACE { +namespace { + +CacheAllocationPtr CopyBufferToHeap(MemoryAllocator* allocator, Slice& buf) { + CacheAllocationPtr heap_buf; + heap_buf = AllocateBlock(buf.size(), allocator); + memcpy(heap_buf.get(), buf.data(), buf.size()); + return heap_buf; +} +} // namespace +} // namespace ROCKSDB_NAMESPACE + +// Generate the regular and coroutine versions of some methods by +// including block_based_table_reader_sync_and_async.h twice +// Macros in the header will expand differently based on whether +// WITH_COROUTINES or WITHOUT_COROUTINES is defined +// clang-format off +#define WITHOUT_COROUTINES +#include "table/block_based/block_based_table_reader_sync_and_async.h" +#undef WITHOUT_COROUTINES +#define WITH_COROUTINES +#include "table/block_based/block_based_table_reader_sync_and_async.h" +#undef WITH_COROUTINES +// clang-format on + +namespace ROCKSDB_NAMESPACE { + +extern const uint64_t kBlockBasedTableMagicNumber; +extern const std::string kHashIndexPrefixesBlock; +extern const std::string kHashIndexPrefixesMetadataBlock; + +BlockBasedTable::~BlockBasedTable() { delete rep_; } + +namespace { +// Read the block identified by "handle" from "file". +// The only relevant option is options.verify_checksums for now. +// On failure return non-OK. +// On success fill *result and return OK - caller owns *result +// @param uncompression_dict Data for presetting the compression library's +// dictionary. +template <typename TBlocklike> +Status ReadBlockFromFile( + RandomAccessFileReader* file, FilePrefetchBuffer* prefetch_buffer, + const Footer& footer, const ReadOptions& options, const BlockHandle& handle, + std::unique_ptr<TBlocklike>* result, const ImmutableOptions& ioptions, + bool do_uncompress, bool maybe_compressed, BlockType block_type, + const UncompressionDict& uncompression_dict, + const PersistentCacheOptions& cache_options, size_t read_amp_bytes_per_bit, + MemoryAllocator* memory_allocator, bool for_compaction, bool using_zstd, + const FilterPolicy* filter_policy, bool async_read) { + assert(result); + + BlockContents contents; + BlockFetcher block_fetcher( + file, prefetch_buffer, footer, options, handle, &contents, ioptions, + do_uncompress, maybe_compressed, block_type, uncompression_dict, + cache_options, memory_allocator, nullptr, for_compaction); + Status s; + // If prefetch_buffer is not allocated, it will fallback to synchronous + // reading of block contents. + if (async_read && prefetch_buffer != nullptr) { + s = block_fetcher.ReadAsyncBlockContents(); + if (!s.ok()) { + return s; + } + } else { + s = block_fetcher.ReadBlockContents(); + } + if (s.ok()) { + result->reset(BlocklikeTraits<TBlocklike>::Create( + std::move(contents), read_amp_bytes_per_bit, ioptions.stats, using_zstd, + filter_policy)); + } + + return s; +} + +// For hash based index, return false if table_properties->prefix_extractor_name +// and prefix_extractor both exist and match, otherwise true. +inline bool PrefixExtractorChangedHelper( + const TableProperties* table_properties, + const SliceTransform* prefix_extractor) { + // BlockBasedTableOptions::kHashSearch requires prefix_extractor to be set. + // Turn off hash index in prefix_extractor is not set; if prefix_extractor + // is set but prefix_extractor_block is not set, also disable hash index + if (prefix_extractor == nullptr || table_properties == nullptr || + table_properties->prefix_extractor_name.empty()) { + return true; + } + + // prefix_extractor and prefix_extractor_block are both non-empty + if (table_properties->prefix_extractor_name != prefix_extractor->AsString()) { + return true; + } else { + return false; + } +} + +} // namespace + +void BlockBasedTable::UpdateCacheHitMetrics(BlockType block_type, + GetContext* get_context, + size_t usage) const { + Statistics* const statistics = rep_->ioptions.stats; + + PERF_COUNTER_ADD(block_cache_hit_count, 1); + PERF_COUNTER_BY_LEVEL_ADD(block_cache_hit_count, 1, + static_cast<uint32_t>(rep_->level)); + + if (get_context) { + ++get_context->get_context_stats_.num_cache_hit; + get_context->get_context_stats_.num_cache_bytes_read += usage; + } else { + RecordTick(statistics, BLOCK_CACHE_HIT); + RecordTick(statistics, BLOCK_CACHE_BYTES_READ, usage); + } + + switch (block_type) { + case BlockType::kFilter: + case BlockType::kFilterPartitionIndex: + PERF_COUNTER_ADD(block_cache_filter_hit_count, 1); + + if (get_context) { + ++get_context->get_context_stats_.num_cache_filter_hit; + } else { + RecordTick(statistics, BLOCK_CACHE_FILTER_HIT); + } + break; + + case BlockType::kCompressionDictionary: + // TODO: introduce perf counter for compression dictionary hit count + if (get_context) { + ++get_context->get_context_stats_.num_cache_compression_dict_hit; + } else { + RecordTick(statistics, BLOCK_CACHE_COMPRESSION_DICT_HIT); + } + break; + + case BlockType::kIndex: + PERF_COUNTER_ADD(block_cache_index_hit_count, 1); + + if (get_context) { + ++get_context->get_context_stats_.num_cache_index_hit; + } else { + RecordTick(statistics, BLOCK_CACHE_INDEX_HIT); + } + break; + + default: + // TODO: introduce dedicated tickers/statistics/counters + // for range tombstones + if (get_context) { + ++get_context->get_context_stats_.num_cache_data_hit; + } else { + RecordTick(statistics, BLOCK_CACHE_DATA_HIT); + } + break; + } +} + +void BlockBasedTable::UpdateCacheMissMetrics(BlockType block_type, + GetContext* get_context) const { + Statistics* const statistics = rep_->ioptions.stats; + + // TODO: introduce aggregate (not per-level) block cache miss count + PERF_COUNTER_BY_LEVEL_ADD(block_cache_miss_count, 1, + static_cast<uint32_t>(rep_->level)); + + if (get_context) { + ++get_context->get_context_stats_.num_cache_miss; + } else { + RecordTick(statistics, BLOCK_CACHE_MISS); + } + + // TODO: introduce perf counters for misses per block type + switch (block_type) { + case BlockType::kFilter: + case BlockType::kFilterPartitionIndex: + if (get_context) { + ++get_context->get_context_stats_.num_cache_filter_miss; + } else { + RecordTick(statistics, BLOCK_CACHE_FILTER_MISS); + } + break; + + case BlockType::kCompressionDictionary: + if (get_context) { + ++get_context->get_context_stats_.num_cache_compression_dict_miss; + } else { + RecordTick(statistics, BLOCK_CACHE_COMPRESSION_DICT_MISS); + } + break; + + case BlockType::kIndex: + if (get_context) { + ++get_context->get_context_stats_.num_cache_index_miss; + } else { + RecordTick(statistics, BLOCK_CACHE_INDEX_MISS); + } + break; + + default: + // TODO: introduce dedicated tickers/statistics/counters + // for range tombstones + if (get_context) { + ++get_context->get_context_stats_.num_cache_data_miss; + } else { + RecordTick(statistics, BLOCK_CACHE_DATA_MISS); + } + break; + } +} + +void BlockBasedTable::UpdateCacheInsertionMetrics( + BlockType block_type, GetContext* get_context, size_t usage, bool redundant, + Statistics* const statistics) { + // TODO: introduce perf counters for block cache insertions + if (get_context) { + ++get_context->get_context_stats_.num_cache_add; + if (redundant) { + ++get_context->get_context_stats_.num_cache_add_redundant; + } + get_context->get_context_stats_.num_cache_bytes_write += usage; + } else { + RecordTick(statistics, BLOCK_CACHE_ADD); + if (redundant) { + RecordTick(statistics, BLOCK_CACHE_ADD_REDUNDANT); + } + RecordTick(statistics, BLOCK_CACHE_BYTES_WRITE, usage); + } + + switch (block_type) { + case BlockType::kFilter: + case BlockType::kFilterPartitionIndex: + if (get_context) { + ++get_context->get_context_stats_.num_cache_filter_add; + if (redundant) { + ++get_context->get_context_stats_.num_cache_filter_add_redundant; + } + get_context->get_context_stats_.num_cache_filter_bytes_insert += usage; + } else { + RecordTick(statistics, BLOCK_CACHE_FILTER_ADD); + if (redundant) { + RecordTick(statistics, BLOCK_CACHE_FILTER_ADD_REDUNDANT); + } + RecordTick(statistics, BLOCK_CACHE_FILTER_BYTES_INSERT, usage); + } + break; + + case BlockType::kCompressionDictionary: + if (get_context) { + ++get_context->get_context_stats_.num_cache_compression_dict_add; + if (redundant) { + ++get_context->get_context_stats_ + .num_cache_compression_dict_add_redundant; + } + get_context->get_context_stats_ + .num_cache_compression_dict_bytes_insert += usage; + } else { + RecordTick(statistics, BLOCK_CACHE_COMPRESSION_DICT_ADD); + if (redundant) { + RecordTick(statistics, BLOCK_CACHE_COMPRESSION_DICT_ADD_REDUNDANT); + } + RecordTick(statistics, BLOCK_CACHE_COMPRESSION_DICT_BYTES_INSERT, + usage); + } + break; + + case BlockType::kIndex: + if (get_context) { + ++get_context->get_context_stats_.num_cache_index_add; + if (redundant) { + ++get_context->get_context_stats_.num_cache_index_add_redundant; + } + get_context->get_context_stats_.num_cache_index_bytes_insert += usage; + } else { + RecordTick(statistics, BLOCK_CACHE_INDEX_ADD); + if (redundant) { + RecordTick(statistics, BLOCK_CACHE_INDEX_ADD_REDUNDANT); + } + RecordTick(statistics, BLOCK_CACHE_INDEX_BYTES_INSERT, usage); + } + break; + + default: + // TODO: introduce dedicated tickers/statistics/counters + // for range tombstones + if (get_context) { + ++get_context->get_context_stats_.num_cache_data_add; + if (redundant) { + ++get_context->get_context_stats_.num_cache_data_add_redundant; + } + get_context->get_context_stats_.num_cache_data_bytes_insert += usage; + } else { + RecordTick(statistics, BLOCK_CACHE_DATA_ADD); + if (redundant) { + RecordTick(statistics, BLOCK_CACHE_DATA_ADD_REDUNDANT); + } + RecordTick(statistics, BLOCK_CACHE_DATA_BYTES_INSERT, usage); + } + break; + } +} + +Cache::Handle* BlockBasedTable::GetEntryFromCache( + const CacheTier& cache_tier, Cache* block_cache, const Slice& key, + BlockType block_type, const bool wait, GetContext* get_context, + const Cache::CacheItemHelper* cache_helper, + const Cache::CreateCallback& create_cb, Cache::Priority priority) const { + Cache::Handle* cache_handle = nullptr; + if (cache_tier == CacheTier::kNonVolatileBlockTier) { + cache_handle = block_cache->Lookup(key, cache_helper, create_cb, priority, + wait, rep_->ioptions.statistics.get()); + } else { + cache_handle = block_cache->Lookup(key, rep_->ioptions.statistics.get()); + } + + // Avoid updating metrics here if the handle is not complete yet. This + // happens with MultiGet and secondary cache. So update the metrics only + // if its a miss, or a hit and value is ready + if (!cache_handle || block_cache->Value(cache_handle)) { + if (cache_handle != nullptr) { + UpdateCacheHitMetrics(block_type, get_context, + block_cache->GetUsage(cache_handle)); + } else { + UpdateCacheMissMetrics(block_type, get_context); + } + } + + return cache_handle; +} + +template <typename TBlocklike> +Status BlockBasedTable::InsertEntryToCache( + const CacheTier& cache_tier, Cache* block_cache, const Slice& key, + const Cache::CacheItemHelper* cache_helper, + std::unique_ptr<TBlocklike>&& block_holder, size_t charge, + Cache::Handle** cache_handle, Cache::Priority priority) const { + Status s = Status::OK(); + if (cache_tier == CacheTier::kNonVolatileBlockTier) { + s = block_cache->Insert(key, block_holder.get(), cache_helper, charge, + cache_handle, priority); + } else { + s = block_cache->Insert(key, block_holder.get(), charge, + cache_helper->del_cb, cache_handle, priority); + } + if (s.ok()) { + // Cache took ownership + block_holder.release(); + } + s.MustCheck(); + return s; +} + +namespace { +// Return True if table_properties has `user_prop_name` has a `true` value +// or it doesn't contain this property (for backward compatible). +bool IsFeatureSupported(const TableProperties& table_properties, + const std::string& user_prop_name, Logger* info_log) { + auto& props = table_properties.user_collected_properties; + auto pos = props.find(user_prop_name); + // Older version doesn't have this value set. Skip this check. + if (pos != props.end()) { + if (pos->second == kPropFalse) { + return false; + } else if (pos->second != kPropTrue) { + ROCKS_LOG_WARN(info_log, "Property %s has invalidate value %s", + user_prop_name.c_str(), pos->second.c_str()); + } + } + return true; +} + +// Caller has to ensure seqno is not nullptr. +Status GetGlobalSequenceNumber(const TableProperties& table_properties, + SequenceNumber largest_seqno, + SequenceNumber* seqno) { + const auto& props = table_properties.user_collected_properties; + const auto version_pos = props.find(ExternalSstFilePropertyNames::kVersion); + const auto seqno_pos = props.find(ExternalSstFilePropertyNames::kGlobalSeqno); + + *seqno = kDisableGlobalSequenceNumber; + if (version_pos == props.end()) { + if (seqno_pos != props.end()) { + std::array<char, 200> msg_buf; + // This is not an external sst file, global_seqno is not supported. + snprintf( + msg_buf.data(), msg_buf.max_size(), + "A non-external sst file have global seqno property with value %s", + seqno_pos->second.c_str()); + return Status::Corruption(msg_buf.data()); + } + return Status::OK(); + } + + uint32_t version = DecodeFixed32(version_pos->second.c_str()); + if (version < 2) { + if (seqno_pos != props.end() || version != 1) { + std::array<char, 200> msg_buf; + // This is a v1 external sst file, global_seqno is not supported. + snprintf(msg_buf.data(), msg_buf.max_size(), + "An external sst file with version %u have global seqno " + "property with value %s", + version, seqno_pos->second.c_str()); + return Status::Corruption(msg_buf.data()); + } + return Status::OK(); + } + + // Since we have a plan to deprecate global_seqno, we do not return failure + // if seqno_pos == props.end(). We rely on version_pos to detect whether the + // SST is external. + SequenceNumber global_seqno(0); + if (seqno_pos != props.end()) { + global_seqno = DecodeFixed64(seqno_pos->second.c_str()); + } + // SstTableReader open table reader with kMaxSequenceNumber as largest_seqno + // to denote it is unknown. + if (largest_seqno < kMaxSequenceNumber) { + if (global_seqno == 0) { + global_seqno = largest_seqno; + } + if (global_seqno != largest_seqno) { + std::array<char, 200> msg_buf; + snprintf( + msg_buf.data(), msg_buf.max_size(), + "An external sst file with version %u have global seqno property " + "with value %s, while largest seqno in the file is %llu", + version, seqno_pos->second.c_str(), + static_cast<unsigned long long>(largest_seqno)); + return Status::Corruption(msg_buf.data()); + } + } + *seqno = global_seqno; + + if (global_seqno > kMaxSequenceNumber) { + std::array<char, 200> msg_buf; + snprintf(msg_buf.data(), msg_buf.max_size(), + "An external sst file with version %u have global seqno property " + "with value %llu, which is greater than kMaxSequenceNumber", + version, static_cast<unsigned long long>(global_seqno)); + return Status::Corruption(msg_buf.data()); + } + + return Status::OK(); +} +} // namespace + +void BlockBasedTable::SetupBaseCacheKey(const TableProperties* properties, + const std::string& cur_db_session_id, + uint64_t cur_file_number, + OffsetableCacheKey* out_base_cache_key, + bool* out_is_stable) { + // Use a stable cache key if sufficient data is in table properties + std::string db_session_id; + uint64_t file_num; + std::string db_id; + if (properties && !properties->db_session_id.empty() && + properties->orig_file_number > 0) { + // (Newer SST file case) + // We must have both properties to get a stable unique id because + // CreateColumnFamilyWithImport or IngestExternalFiles can change the + // file numbers on a file. + db_session_id = properties->db_session_id; + file_num = properties->orig_file_number; + // Less critical, populated in earlier release than above + db_id = properties->db_id; + if (out_is_stable) { + *out_is_stable = true; + } + } else { + // (Old SST file case) + // We use (unique) cache keys based on current identifiers. These are at + // least stable across table file close and re-open, but not across + // different DBs nor DB close and re-open. + db_session_id = cur_db_session_id; + file_num = cur_file_number; + // Plumbing through the DB ID to here would be annoying, and of limited + // value because of the case of VersionSet::Recover opening some table + // files and later setting the DB ID. So we just rely on uniqueness + // level provided by session ID. + db_id = "unknown"; + if (out_is_stable) { + *out_is_stable = false; + } + } + + // Too many tests to update to get these working + // assert(file_num > 0); + // assert(!db_session_id.empty()); + // assert(!db_id.empty()); + + // Minimum block size is 5 bytes; therefore we can trim off two lower bits + // from offsets. See GetCacheKey. + *out_base_cache_key = OffsetableCacheKey(db_id, db_session_id, file_num); +} + +CacheKey BlockBasedTable::GetCacheKey(const OffsetableCacheKey& base_cache_key, + const BlockHandle& handle) { + // Minimum block size is 5 bytes; therefore we can trim off two lower bits + // from offet. + return base_cache_key.WithOffset(handle.offset() >> 2); +} + +Status BlockBasedTable::Open( + const ReadOptions& read_options, const ImmutableOptions& ioptions, + const EnvOptions& env_options, const BlockBasedTableOptions& table_options, + const InternalKeyComparator& internal_comparator, + std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size, + std::unique_ptr<TableReader>* table_reader, + std::shared_ptr<CacheReservationManager> table_reader_cache_res_mgr, + const std::shared_ptr<const SliceTransform>& prefix_extractor, + const bool prefetch_index_and_filter_in_cache, const bool skip_filters, + const int level, const bool immortal_table, + const SequenceNumber largest_seqno, const bool force_direct_prefetch, + TailPrefetchStats* tail_prefetch_stats, + BlockCacheTracer* const block_cache_tracer, + size_t max_file_size_for_l0_meta_pin, const std::string& cur_db_session_id, + uint64_t cur_file_num, UniqueId64x2 expected_unique_id) { + table_reader->reset(); + + Status s; + Footer footer; + std::unique_ptr<FilePrefetchBuffer> prefetch_buffer; + + // From read_options, retain deadline, io_timeout, and rate_limiter_priority. + // In future, we may retain more + // options. Specifically, we ignore verify_checksums and default to + // checksum verification anyway when creating the index and filter + // readers. + ReadOptions ro; + ro.deadline = read_options.deadline; + ro.io_timeout = read_options.io_timeout; + ro.rate_limiter_priority = read_options.rate_limiter_priority; + + // prefetch both index and filters, down to all partitions + const bool prefetch_all = prefetch_index_and_filter_in_cache || level == 0; + const bool preload_all = !table_options.cache_index_and_filter_blocks; + + if (!ioptions.allow_mmap_reads) { + s = PrefetchTail(ro, file.get(), file_size, force_direct_prefetch, + tail_prefetch_stats, prefetch_all, preload_all, + &prefetch_buffer); + // Return error in prefetch path to users. + if (!s.ok()) { + return s; + } + } else { + // Should not prefetch for mmap mode. + prefetch_buffer.reset(new FilePrefetchBuffer( + 0 /* readahead_size */, 0 /* max_readahead_size */, false /* enable */, + true /* track_min_offset */)); + } + + // 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] + IOOptions opts; + s = file->PrepareIOOptions(ro, opts); + if (s.ok()) { + s = ReadFooterFromFile(opts, file.get(), prefetch_buffer.get(), file_size, + &footer, kBlockBasedTableMagicNumber); + } + if (!s.ok()) { + return s; + } + if (!IsSupportedFormatVersion(footer.format_version())) { + return Status::Corruption( + "Unknown Footer version. Maybe this file was created with newer " + "version of RocksDB?"); + } + + BlockCacheLookupContext lookup_context{TableReaderCaller::kPrefetch}; + Rep* rep = new BlockBasedTable::Rep(ioptions, env_options, table_options, + internal_comparator, skip_filters, + file_size, level, immortal_table); + rep->file = std::move(file); + rep->footer = footer; + + // For fully portable/stable cache keys, we need to read the properties + // block before setting up cache keys. TODO: consider setting up a bootstrap + // cache key for PersistentCache to use for metaindex and properties blocks. + rep->persistent_cache_options = PersistentCacheOptions(); + + // Meta-blocks are not dictionary compressed. Explicitly set the dictionary + // handle to null, otherwise it may be seen as uninitialized during the below + // meta-block reads. + rep->compression_dict_handle = BlockHandle::NullBlockHandle(); + + // Read metaindex + std::unique_ptr<BlockBasedTable> new_table( + new BlockBasedTable(rep, block_cache_tracer)); + std::unique_ptr<Block> metaindex; + std::unique_ptr<InternalIterator> metaindex_iter; + s = new_table->ReadMetaIndexBlock(ro, prefetch_buffer.get(), &metaindex, + &metaindex_iter); + if (!s.ok()) { + return s; + } + + // Populates table_properties and some fields that depend on it, + // such as index_type. + s = new_table->ReadPropertiesBlock(ro, prefetch_buffer.get(), + metaindex_iter.get(), largest_seqno); + if (!s.ok()) { + return s; + } + + // Check expected unique id if provided + if (expected_unique_id != kNullUniqueId64x2) { + auto props = rep->table_properties; + if (!props) { + return Status::Corruption("Missing table properties on file " + + std::to_string(cur_file_num) + + " with known unique ID"); + } + UniqueId64x2 actual_unique_id{}; + s = GetSstInternalUniqueId(props->db_id, props->db_session_id, + props->orig_file_number, &actual_unique_id, + /*force*/ true); + assert(s.ok()); // because force=true + if (expected_unique_id != actual_unique_id) { + return Status::Corruption( + "Mismatch in unique ID on table file " + + std::to_string(cur_file_num) + + ". Expected: " + InternalUniqueIdToHumanString(&expected_unique_id) + + " Actual: " + InternalUniqueIdToHumanString(&actual_unique_id)); + } + TEST_SYNC_POINT_CALLBACK("BlockBasedTable::Open::PassedVerifyUniqueId", + &actual_unique_id); + } else { + TEST_SYNC_POINT_CALLBACK("BlockBasedTable::Open::SkippedVerifyUniqueId", + nullptr); + if (ioptions.verify_sst_unique_id_in_manifest && ioptions.logger) { + // A crude but isolated way of reporting unverified files. This should not + // be an ongoing concern so doesn't deserve a place in Statistics IMHO. + static std::atomic<uint64_t> unverified_count{0}; + auto prev_count = + unverified_count.fetch_add(1, std::memory_order_relaxed); + if (prev_count == 0) { + ROCKS_LOG_WARN( + ioptions.logger, + "At least one SST file opened without unique ID to verify: %" PRIu64 + ".sst", + cur_file_num); + } else if (prev_count % 1000 == 0) { + ROCKS_LOG_WARN( + ioptions.logger, + "Another ~1000 SST files opened without unique ID to verify"); + } + } + } + + // Set up prefix extracto as needed + bool force_null_table_prefix_extractor = false; + TEST_SYNC_POINT_CALLBACK( + "BlockBasedTable::Open::ForceNullTablePrefixExtractor", + &force_null_table_prefix_extractor); + if (force_null_table_prefix_extractor) { + assert(!rep->table_prefix_extractor); + } else if (!PrefixExtractorChangedHelper(rep->table_properties.get(), + prefix_extractor.get())) { + // Establish fast path for unchanged prefix_extractor + rep->table_prefix_extractor = prefix_extractor; + } else { + // Current prefix_extractor doesn't match table +#ifndef ROCKSDB_LITE + if (rep->table_properties) { + //**TODO: If/When the DBOptions has a registry in it, the ConfigOptions + // will need to use it + ConfigOptions config_options; + Status st = SliceTransform::CreateFromString( + config_options, rep->table_properties->prefix_extractor_name, + &(rep->table_prefix_extractor)); + if (!st.ok()) { + //**TODO: Should this be error be returned or swallowed? + ROCKS_LOG_ERROR(rep->ioptions.logger, + "Failed to create prefix extractor[%s]: %s", + rep->table_properties->prefix_extractor_name.c_str(), + st.ToString().c_str()); + } + } +#endif // ROCKSDB_LITE + } + + // With properties loaded, we can set up portable/stable cache keys + SetupBaseCacheKey(rep->table_properties.get(), cur_db_session_id, + cur_file_num, &rep->base_cache_key); + + rep->persistent_cache_options = + PersistentCacheOptions(rep->table_options.persistent_cache, + rep->base_cache_key, rep->ioptions.stats); + + s = new_table->ReadRangeDelBlock(ro, prefetch_buffer.get(), + metaindex_iter.get(), internal_comparator, + &lookup_context); + if (!s.ok()) { + return s; + } + s = new_table->PrefetchIndexAndFilterBlocks( + ro, prefetch_buffer.get(), metaindex_iter.get(), new_table.get(), + prefetch_all, table_options, level, file_size, + max_file_size_for_l0_meta_pin, &lookup_context); + + if (s.ok()) { + // Update tail prefetch stats + assert(prefetch_buffer.get() != nullptr); + if (tail_prefetch_stats != nullptr) { + assert(prefetch_buffer->min_offset_read() < file_size); + tail_prefetch_stats->RecordEffectiveSize( + static_cast<size_t>(file_size) - prefetch_buffer->min_offset_read()); + } + } + + if (s.ok() && table_reader_cache_res_mgr) { + std::size_t mem_usage = new_table->ApproximateMemoryUsage(); + s = table_reader_cache_res_mgr->MakeCacheReservation( + mem_usage, &(rep->table_reader_cache_res_handle)); + if (s.IsMemoryLimit()) { + s = Status::MemoryLimit( + "Can't allocate " + + kCacheEntryRoleToCamelString[static_cast<std::uint32_t>( + CacheEntryRole::kBlockBasedTableReader)] + + " due to memory limit based on " + "cache capacity for memory allocation"); + } + } + + if (s.ok()) { + *table_reader = std::move(new_table); + } + return s; +} + +Status BlockBasedTable::PrefetchTail( + const ReadOptions& ro, RandomAccessFileReader* file, uint64_t file_size, + bool force_direct_prefetch, TailPrefetchStats* tail_prefetch_stats, + const bool prefetch_all, const bool preload_all, + std::unique_ptr<FilePrefetchBuffer>* prefetch_buffer) { + size_t tail_prefetch_size = 0; + if (tail_prefetch_stats != nullptr) { + // Multiple threads may get a 0 (no history) when running in parallel, + // but it will get cleared after the first of them finishes. + tail_prefetch_size = tail_prefetch_stats->GetSuggestedPrefetchSize(); + } + if (tail_prefetch_size == 0) { + // Before read footer, readahead backwards to prefetch data. Do more + // readahead if we're going to read index/filter. + // TODO: This may incorrectly select small readahead in case partitioned + // index/filter is enabled and top-level partition pinning is enabled. + // That's because we need to issue readahead before we read the properties, + // at which point we don't yet know the index type. + tail_prefetch_size = prefetch_all || preload_all ? 512 * 1024 : 4 * 1024; + } + size_t prefetch_off; + size_t prefetch_len; + if (file_size < tail_prefetch_size) { + prefetch_off = 0; + prefetch_len = static_cast<size_t>(file_size); + } else { + prefetch_off = static_cast<size_t>(file_size - tail_prefetch_size); + prefetch_len = tail_prefetch_size; + } + TEST_SYNC_POINT_CALLBACK("BlockBasedTable::Open::TailPrefetchLen", + &tail_prefetch_size); + + // Try file system prefetch + if (!file->use_direct_io() && !force_direct_prefetch) { + if (!file->Prefetch(prefetch_off, prefetch_len, ro.rate_limiter_priority) + .IsNotSupported()) { + prefetch_buffer->reset(new FilePrefetchBuffer( + 0 /* readahead_size */, 0 /* max_readahead_size */, + false /* enable */, true /* track_min_offset */)); + return Status::OK(); + } + } + + // Use `FilePrefetchBuffer` + prefetch_buffer->reset( + new FilePrefetchBuffer(0 /* readahead_size */, 0 /* max_readahead_size */, + true /* enable */, true /* track_min_offset */)); + + IOOptions opts; + Status s = file->PrepareIOOptions(ro, opts); + if (s.ok()) { + s = (*prefetch_buffer) + ->Prefetch(opts, file, prefetch_off, prefetch_len, + ro.rate_limiter_priority); + } + return s; +} + +Status BlockBasedTable::ReadPropertiesBlock( + const ReadOptions& ro, FilePrefetchBuffer* prefetch_buffer, + InternalIterator* meta_iter, const SequenceNumber largest_seqno) { + Status s; + BlockHandle handle; + s = FindOptionalMetaBlock(meta_iter, kPropertiesBlockName, &handle); + + if (!s.ok()) { + ROCKS_LOG_WARN(rep_->ioptions.logger, + "Error when seeking to properties block from file: %s", + s.ToString().c_str()); + } else if (!handle.IsNull()) { + s = meta_iter->status(); + std::unique_ptr<TableProperties> table_properties; + if (s.ok()) { + s = ReadTablePropertiesHelper( + ro, handle, rep_->file.get(), prefetch_buffer, rep_->footer, + rep_->ioptions, &table_properties, nullptr /* memory_allocator */); + } + IGNORE_STATUS_IF_ERROR(s); + + if (!s.ok()) { + ROCKS_LOG_WARN(rep_->ioptions.logger, + "Encountered error while reading data from properties " + "block %s", + s.ToString().c_str()); + } else { + assert(table_properties != nullptr); + rep_->table_properties = std::move(table_properties); + rep_->blocks_maybe_compressed = + rep_->table_properties->compression_name != + CompressionTypeToString(kNoCompression); + rep_->blocks_definitely_zstd_compressed = + (rep_->table_properties->compression_name == + CompressionTypeToString(kZSTD) || + rep_->table_properties->compression_name == + CompressionTypeToString(kZSTDNotFinalCompression)); + } + } else { + ROCKS_LOG_ERROR(rep_->ioptions.logger, + "Cannot find Properties block from file."); + } + + // Read the table properties, if provided. + if (rep_->table_properties) { + rep_->whole_key_filtering &= + IsFeatureSupported(*(rep_->table_properties), + BlockBasedTablePropertyNames::kWholeKeyFiltering, + rep_->ioptions.logger); + rep_->prefix_filtering &= IsFeatureSupported( + *(rep_->table_properties), + BlockBasedTablePropertyNames::kPrefixFiltering, rep_->ioptions.logger); + + rep_->index_key_includes_seq = + rep_->table_properties->index_key_is_user_key == 0; + rep_->index_value_is_full = + rep_->table_properties->index_value_is_delta_encoded == 0; + + // Update index_type with the true type. + // If table properties don't contain index type, we assume that the table + // is in very old format and has kBinarySearch index type. + auto& props = rep_->table_properties->user_collected_properties; + auto pos = props.find(BlockBasedTablePropertyNames::kIndexType); + if (pos != props.end()) { + rep_->index_type = static_cast<BlockBasedTableOptions::IndexType>( + DecodeFixed32(pos->second.c_str())); + } + + rep_->index_has_first_key = + rep_->index_type == BlockBasedTableOptions::kBinarySearchWithFirstKey; + + s = GetGlobalSequenceNumber(*(rep_->table_properties), largest_seqno, + &(rep_->global_seqno)); + if (!s.ok()) { + ROCKS_LOG_ERROR(rep_->ioptions.logger, "%s", s.ToString().c_str()); + } + } + return s; +} + +Status BlockBasedTable::ReadRangeDelBlock( + const ReadOptions& read_options, FilePrefetchBuffer* prefetch_buffer, + InternalIterator* meta_iter, + const InternalKeyComparator& internal_comparator, + BlockCacheLookupContext* lookup_context) { + Status s; + BlockHandle range_del_handle; + s = FindOptionalMetaBlock(meta_iter, kRangeDelBlockName, &range_del_handle); + if (!s.ok()) { + ROCKS_LOG_WARN( + rep_->ioptions.logger, + "Error when seeking to range delete tombstones block from file: %s", + s.ToString().c_str()); + } else if (!range_del_handle.IsNull()) { + Status tmp_status; + std::unique_ptr<InternalIterator> iter(NewDataBlockIterator<DataBlockIter>( + read_options, range_del_handle, + /*input_iter=*/nullptr, BlockType::kRangeDeletion, + /*get_context=*/nullptr, lookup_context, prefetch_buffer, + /*for_compaction= */ false, /*async_read= */ false, tmp_status)); + assert(iter != nullptr); + s = iter->status(); + if (!s.ok()) { + ROCKS_LOG_WARN( + rep_->ioptions.logger, + "Encountered error while reading data from range del block %s", + s.ToString().c_str()); + IGNORE_STATUS_IF_ERROR(s); + } else { + rep_->fragmented_range_dels = + std::make_shared<FragmentedRangeTombstoneList>(std::move(iter), + internal_comparator); + } + } + return s; +} + +Status BlockBasedTable::PrefetchIndexAndFilterBlocks( + const ReadOptions& ro, FilePrefetchBuffer* prefetch_buffer, + InternalIterator* meta_iter, BlockBasedTable* new_table, bool prefetch_all, + const BlockBasedTableOptions& table_options, const int level, + size_t file_size, size_t max_file_size_for_l0_meta_pin, + BlockCacheLookupContext* lookup_context) { + // Find filter handle and filter type + if (rep_->filter_policy) { + auto name = rep_->filter_policy->CompatibilityName(); + bool builtin_compatible = + strcmp(name, BuiltinFilterPolicy::kCompatibilityName()) == 0; + + for (const auto& [filter_type, prefix] : + {std::make_pair(Rep::FilterType::kFullFilter, kFullFilterBlockPrefix), + std::make_pair(Rep::FilterType::kPartitionedFilter, + kPartitionedFilterBlockPrefix), + std::make_pair(Rep::FilterType::kNoFilter, + kObsoleteFilterBlockPrefix)}) { + if (builtin_compatible) { + // This code is only here to deal with a hiccup in early 7.0.x where + // there was an unintentional name change in the SST files metadata. + // It should be OK to remove this in the future (late 2022) and just + // have the 'else' code. + // NOTE: the test:: names below are likely not needed but included + // out of caution + static const std::unordered_set<std::string> kBuiltinNameAndAliases = { + BuiltinFilterPolicy::kCompatibilityName(), + test::LegacyBloomFilterPolicy::kClassName(), + test::FastLocalBloomFilterPolicy::kClassName(), + test::Standard128RibbonFilterPolicy::kClassName(), + "rocksdb.internal.DeprecatedBlockBasedBloomFilter", + BloomFilterPolicy::kClassName(), + RibbonFilterPolicy::kClassName(), + }; + + // For efficiency, do a prefix seek and see if the first match is + // good. + meta_iter->Seek(prefix); + if (meta_iter->status().ok() && meta_iter->Valid()) { + Slice key = meta_iter->key(); + if (key.starts_with(prefix)) { + key.remove_prefix(prefix.size()); + if (kBuiltinNameAndAliases.find(key.ToString()) != + kBuiltinNameAndAliases.end()) { + Slice v = meta_iter->value(); + Status s = rep_->filter_handle.DecodeFrom(&v); + if (s.ok()) { + rep_->filter_type = filter_type; + if (filter_type == Rep::FilterType::kNoFilter) { + ROCKS_LOG_WARN(rep_->ioptions.logger, + "Detected obsolete filter type in %s. Read " + "performance might suffer until DB is fully " + "re-compacted.", + rep_->file->file_name().c_str()); + } + break; + } + } + } + } + } else { + std::string filter_block_key = prefix + name; + if (FindMetaBlock(meta_iter, filter_block_key, &rep_->filter_handle) + .ok()) { + rep_->filter_type = filter_type; + if (filter_type == Rep::FilterType::kNoFilter) { + ROCKS_LOG_WARN( + rep_->ioptions.logger, + "Detected obsolete filter type in %s. Read performance might " + "suffer until DB is fully re-compacted.", + rep_->file->file_name().c_str()); + } + break; + } + } + } + } + // Partition filters cannot be enabled without partition indexes + assert(rep_->filter_type != Rep::FilterType::kPartitionedFilter || + rep_->index_type == BlockBasedTableOptions::kTwoLevelIndexSearch); + + // Find compression dictionary handle + Status s = FindOptionalMetaBlock(meta_iter, kCompressionDictBlockName, + &rep_->compression_dict_handle); + if (!s.ok()) { + return s; + } + + BlockBasedTableOptions::IndexType index_type = rep_->index_type; + + const bool use_cache = table_options.cache_index_and_filter_blocks; + + const bool maybe_flushed = + level == 0 && file_size <= max_file_size_for_l0_meta_pin; + std::function<bool(PinningTier, PinningTier)> is_pinned = + [maybe_flushed, &is_pinned](PinningTier pinning_tier, + PinningTier fallback_pinning_tier) { + // Fallback to fallback would lead to infinite recursion. Disallow it. + assert(fallback_pinning_tier != PinningTier::kFallback); + + switch (pinning_tier) { + case PinningTier::kFallback: + return is_pinned(fallback_pinning_tier, + PinningTier::kNone /* fallback_pinning_tier */); + case PinningTier::kNone: + return false; + case PinningTier::kFlushedAndSimilar: + return maybe_flushed; + case PinningTier::kAll: + return true; + }; + + // In GCC, this is needed to suppress `control reaches end of non-void + // function [-Werror=return-type]`. + assert(false); + return false; + }; + const bool pin_top_level_index = is_pinned( + table_options.metadata_cache_options.top_level_index_pinning, + table_options.pin_top_level_index_and_filter ? PinningTier::kAll + : PinningTier::kNone); + const bool pin_partition = + is_pinned(table_options.metadata_cache_options.partition_pinning, + table_options.pin_l0_filter_and_index_blocks_in_cache + ? PinningTier::kFlushedAndSimilar + : PinningTier::kNone); + const bool pin_unpartitioned = + is_pinned(table_options.metadata_cache_options.unpartitioned_pinning, + table_options.pin_l0_filter_and_index_blocks_in_cache + ? PinningTier::kFlushedAndSimilar + : PinningTier::kNone); + + // pin the first level of index + const bool pin_index = + index_type == BlockBasedTableOptions::kTwoLevelIndexSearch + ? pin_top_level_index + : pin_unpartitioned; + // prefetch the first level of index + // WART: this might be redundant (unnecessary cache hit) if !pin_index, + // depending on prepopulate_block_cache option + const bool prefetch_index = prefetch_all || pin_index; + + std::unique_ptr<IndexReader> index_reader; + s = new_table->CreateIndexReader(ro, prefetch_buffer, meta_iter, use_cache, + prefetch_index, pin_index, lookup_context, + &index_reader); + if (!s.ok()) { + return s; + } + + rep_->index_reader = std::move(index_reader); + + // The partitions of partitioned index are always stored in cache. They + // are hence follow the configuration for pin and prefetch regardless of + // the value of cache_index_and_filter_blocks + if (prefetch_all || pin_partition) { + s = rep_->index_reader->CacheDependencies(ro, pin_partition); + } + if (!s.ok()) { + return s; + } + + // pin the first level of filter + const bool pin_filter = + rep_->filter_type == Rep::FilterType::kPartitionedFilter + ? pin_top_level_index + : pin_unpartitioned; + // prefetch the first level of filter + // WART: this might be redundant (unnecessary cache hit) if !pin_filter, + // depending on prepopulate_block_cache option + const bool prefetch_filter = prefetch_all || pin_filter; + + if (rep_->filter_policy) { + auto filter = new_table->CreateFilterBlockReader( + ro, prefetch_buffer, use_cache, prefetch_filter, pin_filter, + lookup_context); + + if (filter) { + // Refer to the comment above about paritioned indexes always being cached + if (prefetch_all || pin_partition) { + s = filter->CacheDependencies(ro, pin_partition); + if (!s.ok()) { + return s; + } + } + rep_->filter = std::move(filter); + } + } + + if (!rep_->compression_dict_handle.IsNull()) { + std::unique_ptr<UncompressionDictReader> uncompression_dict_reader; + s = UncompressionDictReader::Create( + this, ro, prefetch_buffer, use_cache, prefetch_all || pin_unpartitioned, + pin_unpartitioned, lookup_context, &uncompression_dict_reader); + if (!s.ok()) { + return s; + } + + rep_->uncompression_dict_reader = std::move(uncompression_dict_reader); + } + + assert(s.ok()); + return s; +} + +void BlockBasedTable::SetupForCompaction() { + switch (rep_->ioptions.access_hint_on_compaction_start) { + case Options::NONE: + break; + case Options::NORMAL: + rep_->file->file()->Hint(FSRandomAccessFile::kNormal); + break; + case Options::SEQUENTIAL: + rep_->file->file()->Hint(FSRandomAccessFile::kSequential); + break; + case Options::WILLNEED: + rep_->file->file()->Hint(FSRandomAccessFile::kWillNeed); + break; + default: + assert(false); + } +} + +std::shared_ptr<const TableProperties> BlockBasedTable::GetTableProperties() + const { + return rep_->table_properties; +} + +size_t BlockBasedTable::ApproximateMemoryUsage() const { + size_t usage = 0; + if (rep_) { + usage += rep_->ApproximateMemoryUsage(); + } else { + return usage; + } + if (rep_->filter) { + usage += rep_->filter->ApproximateMemoryUsage(); + } + if (rep_->index_reader) { + usage += rep_->index_reader->ApproximateMemoryUsage(); + } + if (rep_->uncompression_dict_reader) { + usage += rep_->uncompression_dict_reader->ApproximateMemoryUsage(); + } + if (rep_->table_properties) { + usage += rep_->table_properties->ApproximateMemoryUsage(); + } + return usage; +} + +// Load the meta-index-block from the file. On success, return the loaded +// metaindex +// block and its iterator. +Status BlockBasedTable::ReadMetaIndexBlock( + const ReadOptions& ro, FilePrefetchBuffer* prefetch_buffer, + std::unique_ptr<Block>* metaindex_block, + std::unique_ptr<InternalIterator>* iter) { + // TODO(sanjay): Skip this if footer.metaindex_handle() size indicates + // it is an empty block. + std::unique_ptr<Block> metaindex; + Status s = ReadBlockFromFile( + rep_->file.get(), prefetch_buffer, rep_->footer, ro, + rep_->footer.metaindex_handle(), &metaindex, rep_->ioptions, + true /* decompress */, true /*maybe_compressed*/, BlockType::kMetaIndex, + UncompressionDict::GetEmptyDict(), rep_->persistent_cache_options, + 0 /* read_amp_bytes_per_bit */, GetMemoryAllocator(rep_->table_options), + false /* for_compaction */, rep_->blocks_definitely_zstd_compressed, + nullptr /* filter_policy */, false /* async_read */); + + if (!s.ok()) { + ROCKS_LOG_ERROR(rep_->ioptions.logger, + "Encountered error while reading data from properties" + " block %s", + s.ToString().c_str()); + return s; + } + + *metaindex_block = std::move(metaindex); + // meta block uses bytewise comparator. + iter->reset(metaindex_block->get()->NewMetaIterator()); + return Status::OK(); +} + +template <typename TBlocklike> +Status BlockBasedTable::GetDataBlockFromCache( + const Slice& cache_key, Cache* block_cache, Cache* block_cache_compressed, + const ReadOptions& read_options, + CachableEntry<TBlocklike>* out_parsed_block, + const UncompressionDict& uncompression_dict, BlockType block_type, + const bool wait, GetContext* get_context) const { + const size_t read_amp_bytes_per_bit = + block_type == BlockType::kData + ? rep_->table_options.read_amp_bytes_per_bit + : 0; + assert(out_parsed_block); + assert(out_parsed_block->IsEmpty()); + // Here we treat the legacy name "...index_and_filter_blocks..." to mean all + // metadata blocks that might go into block cache, EXCEPT only those needed + // for the read path (Get, etc.). TableProperties should not be needed on the + // read path (prefix extractor setting is an O(1) size special case that we + // are working not to require from TableProperties), so it is not given + // high-priority treatment if it should go into BlockCache. + const Cache::Priority priority = + rep_->table_options.cache_index_and_filter_blocks_with_high_priority && + block_type != BlockType::kData && + block_type != BlockType::kProperties + ? Cache::Priority::HIGH + : Cache::Priority::LOW; + + Status s; + BlockContents* compressed_block = nullptr; + Cache::Handle* block_cache_compressed_handle = nullptr; + Statistics* statistics = rep_->ioptions.statistics.get(); + bool using_zstd = rep_->blocks_definitely_zstd_compressed; + const FilterPolicy* filter_policy = rep_->filter_policy; + Cache::CreateCallback create_cb = GetCreateCallback<TBlocklike>( + read_amp_bytes_per_bit, statistics, using_zstd, filter_policy); + + // Lookup uncompressed cache first + if (block_cache != nullptr) { + assert(!cache_key.empty()); + Cache::Handle* cache_handle = nullptr; + cache_handle = GetEntryFromCache( + rep_->ioptions.lowest_used_cache_tier, block_cache, cache_key, + block_type, wait, get_context, + BlocklikeTraits<TBlocklike>::GetCacheItemHelper(block_type), create_cb, + priority); + if (cache_handle != nullptr) { + out_parsed_block->SetCachedValue( + reinterpret_cast<TBlocklike*>(block_cache->Value(cache_handle)), + block_cache, cache_handle); + return s; + } + } + + // If not found, search from the compressed block cache. + assert(out_parsed_block->IsEmpty()); + + if (block_cache_compressed == nullptr) { + return s; + } + + assert(!cache_key.empty()); + BlockContents contents; + block_cache_compressed_handle = + block_cache_compressed->Lookup(cache_key, statistics); + + // if we found in the compressed cache, then uncompress and insert into + // uncompressed cache + if (block_cache_compressed_handle == nullptr) { + RecordTick(statistics, BLOCK_CACHE_COMPRESSED_MISS); + return s; + } + + // found compressed block + RecordTick(statistics, BLOCK_CACHE_COMPRESSED_HIT); + compressed_block = reinterpret_cast<BlockContents*>( + block_cache_compressed->Value(block_cache_compressed_handle)); + CompressionType compression_type = GetBlockCompressionType(*compressed_block); + assert(compression_type != kNoCompression); + + // Retrieve the uncompressed contents into a new buffer + UncompressionContext context(compression_type); + UncompressionInfo info(context, uncompression_dict, compression_type); + s = UncompressSerializedBlock( + info, compressed_block->data.data(), compressed_block->data.size(), + &contents, rep_->table_options.format_version, rep_->ioptions, + GetMemoryAllocator(rep_->table_options)); + + // Insert parsed block into block cache, the priority is based on the + // data block type. + if (s.ok()) { + std::unique_ptr<TBlocklike> block_holder( + BlocklikeTraits<TBlocklike>::Create( + std::move(contents), read_amp_bytes_per_bit, statistics, + rep_->blocks_definitely_zstd_compressed, + rep_->table_options.filter_policy.get())); + + if (block_cache != nullptr && block_holder->own_bytes() && + read_options.fill_cache) { + size_t charge = block_holder->ApproximateMemoryUsage(); + Cache::Handle* cache_handle = nullptr; + auto block_holder_raw_ptr = block_holder.get(); + s = InsertEntryToCache( + rep_->ioptions.lowest_used_cache_tier, block_cache, cache_key, + BlocklikeTraits<TBlocklike>::GetCacheItemHelper(block_type), + std::move(block_holder), charge, &cache_handle, priority); + if (s.ok()) { + assert(cache_handle != nullptr); + out_parsed_block->SetCachedValue(block_holder_raw_ptr, block_cache, + cache_handle); + + UpdateCacheInsertionMetrics(block_type, get_context, charge, + s.IsOkOverwritten(), rep_->ioptions.stats); + } else { + RecordTick(statistics, BLOCK_CACHE_ADD_FAILURES); + } + } else { + out_parsed_block->SetOwnedValue(std::move(block_holder)); + } + } + + // Release hold on compressed cache entry + block_cache_compressed->Release(block_cache_compressed_handle); + return s; +} + +template <typename TBlocklike> +Status BlockBasedTable::PutDataBlockToCache( + const Slice& cache_key, Cache* block_cache, Cache* block_cache_compressed, + CachableEntry<TBlocklike>* out_parsed_block, BlockContents&& block_contents, + CompressionType block_comp_type, + const UncompressionDict& uncompression_dict, + MemoryAllocator* memory_allocator, BlockType block_type, + GetContext* get_context) const { + const ImmutableOptions& ioptions = rep_->ioptions; + const uint32_t format_version = rep_->table_options.format_version; + const size_t read_amp_bytes_per_bit = + block_type == BlockType::kData + ? rep_->table_options.read_amp_bytes_per_bit + : 0; + const Cache::Priority priority = + rep_->table_options.cache_index_and_filter_blocks_with_high_priority && + block_type != BlockType::kData + ? Cache::Priority::HIGH + : Cache::Priority::LOW; + assert(out_parsed_block); + assert(out_parsed_block->IsEmpty()); + + Status s; + Statistics* statistics = ioptions.stats; + + std::unique_ptr<TBlocklike> block_holder; + if (block_comp_type != kNoCompression) { + // Retrieve the uncompressed contents into a new buffer + BlockContents uncompressed_block_contents; + UncompressionContext context(block_comp_type); + UncompressionInfo info(context, uncompression_dict, block_comp_type); + s = UncompressBlockData(info, block_contents.data.data(), + block_contents.data.size(), + &uncompressed_block_contents, format_version, + ioptions, memory_allocator); + if (!s.ok()) { + return s; + } + + block_holder.reset(BlocklikeTraits<TBlocklike>::Create( + std::move(uncompressed_block_contents), read_amp_bytes_per_bit, + statistics, rep_->blocks_definitely_zstd_compressed, + rep_->table_options.filter_policy.get())); + } else { + block_holder.reset(BlocklikeTraits<TBlocklike>::Create( + std::move(block_contents), read_amp_bytes_per_bit, statistics, + rep_->blocks_definitely_zstd_compressed, + rep_->table_options.filter_policy.get())); + } + + // Insert compressed block into compressed block cache. + // Release the hold on the compressed cache entry immediately. + if (block_cache_compressed != nullptr && block_comp_type != kNoCompression && + block_contents.own_bytes()) { + assert(block_contents.has_trailer); + assert(!cache_key.empty()); + + // We cannot directly put block_contents because this could point to + // an object in the stack. + auto block_cont_for_comp_cache = + std::make_unique<BlockContents>(std::move(block_contents)); + size_t charge = block_cont_for_comp_cache->ApproximateMemoryUsage(); + + s = block_cache_compressed->Insert( + cache_key, block_cont_for_comp_cache.get(), charge, + &DeleteCacheEntry<BlockContents>, nullptr /*handle*/, + Cache::Priority::LOW); + + if (s.ok()) { + // Cache took ownership + block_cont_for_comp_cache.release(); + RecordTick(statistics, BLOCK_CACHE_COMPRESSED_ADD); + } else { + RecordTick(statistics, BLOCK_CACHE_COMPRESSED_ADD_FAILURES); + } + } + + // insert into uncompressed block cache + if (block_cache != nullptr && block_holder->own_bytes()) { + size_t charge = block_holder->ApproximateMemoryUsage(); + auto block_holder_raw_ptr = block_holder.get(); + Cache::Handle* cache_handle = nullptr; + s = InsertEntryToCache( + rep_->ioptions.lowest_used_cache_tier, block_cache, cache_key, + BlocklikeTraits<TBlocklike>::GetCacheItemHelper(block_type), + std::move(block_holder), charge, &cache_handle, priority); + if (s.ok()) { + assert(cache_handle != nullptr); + out_parsed_block->SetCachedValue(block_holder_raw_ptr, block_cache, + cache_handle); + + UpdateCacheInsertionMetrics(block_type, get_context, charge, + s.IsOkOverwritten(), rep_->ioptions.stats); + } else { + RecordTick(statistics, BLOCK_CACHE_ADD_FAILURES); + } + } else { + out_parsed_block->SetOwnedValue(std::move(block_holder)); + } + + return s; +} + +std::unique_ptr<FilterBlockReader> BlockBasedTable::CreateFilterBlockReader( + const ReadOptions& ro, FilePrefetchBuffer* prefetch_buffer, bool use_cache, + bool prefetch, bool pin, BlockCacheLookupContext* lookup_context) { + auto& rep = rep_; + auto filter_type = rep->filter_type; + if (filter_type == Rep::FilterType::kNoFilter) { + return std::unique_ptr<FilterBlockReader>(); + } + + assert(rep->filter_policy); + + switch (filter_type) { + case Rep::FilterType::kPartitionedFilter: + return PartitionedFilterBlockReader::Create( + this, ro, prefetch_buffer, use_cache, prefetch, pin, lookup_context); + + case Rep::FilterType::kFullFilter: + return FullFilterBlockReader::Create(this, ro, prefetch_buffer, use_cache, + prefetch, pin, lookup_context); + + default: + // filter_type is either kNoFilter (exited the function at the first if), + // or it must be covered in this switch block + assert(false); + return std::unique_ptr<FilterBlockReader>(); + } +} + +// disable_prefix_seek should be set to true when prefix_extractor found in SST +// differs from the one in mutable_cf_options and index type is HashBasedIndex +InternalIteratorBase<IndexValue>* BlockBasedTable::NewIndexIterator( + const ReadOptions& read_options, bool disable_prefix_seek, + IndexBlockIter* input_iter, GetContext* get_context, + BlockCacheLookupContext* lookup_context) const { + assert(rep_ != nullptr); + assert(rep_->index_reader != nullptr); + + // We don't return pinned data from index blocks, so no need + // to set `block_contents_pinned`. + return rep_->index_reader->NewIterator(read_options, disable_prefix_seek, + input_iter, get_context, + lookup_context); +} + +template <> +DataBlockIter* BlockBasedTable::InitBlockIterator<DataBlockIter>( + const Rep* rep, Block* block, BlockType block_type, + DataBlockIter* input_iter, bool block_contents_pinned) { + return block->NewDataIterator(rep->internal_comparator.user_comparator(), + rep->get_global_seqno(block_type), input_iter, + rep->ioptions.stats, block_contents_pinned); +} + +template <> +IndexBlockIter* BlockBasedTable::InitBlockIterator<IndexBlockIter>( + const Rep* rep, Block* block, BlockType block_type, + IndexBlockIter* input_iter, bool block_contents_pinned) { + return block->NewIndexIterator( + rep->internal_comparator.user_comparator(), + rep->get_global_seqno(block_type), input_iter, rep->ioptions.stats, + /* total_order_seek */ true, rep->index_has_first_key, + rep->index_key_includes_seq, rep->index_value_is_full, + block_contents_pinned); +} + +// If contents is nullptr, this function looks up the block caches for the +// data block referenced by handle, and read the block from disk if necessary. +// If contents is non-null, it skips the cache lookup and disk read, since +// the caller has already read it. In both cases, if ro.fill_cache is true, +// it inserts the block into the block cache. +template <typename TBlocklike> +Status BlockBasedTable::MaybeReadBlockAndLoadToCache( + FilePrefetchBuffer* prefetch_buffer, const ReadOptions& ro, + const BlockHandle& handle, const UncompressionDict& uncompression_dict, + const bool wait, const bool for_compaction, + CachableEntry<TBlocklike>* out_parsed_block, BlockType block_type, + GetContext* get_context, BlockCacheLookupContext* lookup_context, + BlockContents* contents, bool async_read) const { + assert(out_parsed_block != nullptr); + const bool no_io = (ro.read_tier == kBlockCacheTier); + Cache* block_cache = rep_->table_options.block_cache.get(); + Cache* block_cache_compressed = + rep_->table_options.block_cache_compressed.get(); + + // First, try to get the block from the cache + // + // If either block cache is enabled, we'll try to read from it. + Status s; + CacheKey key_data; + Slice key; + bool is_cache_hit = false; + if (block_cache != nullptr || block_cache_compressed != nullptr) { + // create key for block cache + key_data = GetCacheKey(rep_->base_cache_key, handle); + key = key_data.AsSlice(); + + if (!contents) { + s = GetDataBlockFromCache(key, block_cache, block_cache_compressed, ro, + out_parsed_block, uncompression_dict, + block_type, wait, get_context); + // Value could still be null at this point, so check the cache handle + // and update the read pattern for prefetching + if (out_parsed_block->GetValue() || out_parsed_block->GetCacheHandle()) { + // TODO(haoyu): Differentiate cache hit on uncompressed block cache and + // compressed block cache. + is_cache_hit = true; + if (prefetch_buffer) { + // Update the block details so that PrefetchBuffer can use the read + // pattern to determine if reads are sequential or not for + // prefetching. It should also take in account blocks read from cache. + prefetch_buffer->UpdateReadPattern( + handle.offset(), BlockSizeWithTrailer(handle), + ro.adaptive_readahead /*decrease_readahead_size*/); + } + } + } + + // Can't find the block from the cache. If I/O is allowed, read from the + // file. + if (out_parsed_block->GetValue() == nullptr && + out_parsed_block->GetCacheHandle() == nullptr && !no_io && + ro.fill_cache) { + Statistics* statistics = rep_->ioptions.stats; + const bool maybe_compressed = + block_type != BlockType::kFilter && + block_type != BlockType::kCompressionDictionary && + rep_->blocks_maybe_compressed; + const bool do_uncompress = maybe_compressed && !block_cache_compressed; + CompressionType contents_comp_type; + // Maybe serialized or uncompressed + BlockContents tmp_contents; + if (!contents) { + Histograms histogram = for_compaction ? READ_BLOCK_COMPACTION_MICROS + : READ_BLOCK_GET_MICROS; + StopWatch sw(rep_->ioptions.clock, statistics, histogram); + BlockFetcher block_fetcher( + rep_->file.get(), prefetch_buffer, rep_->footer, ro, handle, + &tmp_contents, rep_->ioptions, do_uncompress, maybe_compressed, + block_type, uncompression_dict, rep_->persistent_cache_options, + GetMemoryAllocator(rep_->table_options), + GetMemoryAllocatorForCompressedBlock(rep_->table_options)); + + // If prefetch_buffer is not allocated, it will fallback to synchronous + // reading of block contents. + if (async_read && prefetch_buffer != nullptr) { + s = block_fetcher.ReadAsyncBlockContents(); + if (!s.ok()) { + return s; + } + } else { + s = block_fetcher.ReadBlockContents(); + } + + contents_comp_type = block_fetcher.get_compression_type(); + contents = &tmp_contents; + if (get_context) { + switch (block_type) { + case BlockType::kIndex: + ++get_context->get_context_stats_.num_index_read; + break; + case BlockType::kFilter: + case BlockType::kFilterPartitionIndex: + ++get_context->get_context_stats_.num_filter_read; + break; + default: + break; + } + } + } else { + contents_comp_type = GetBlockCompressionType(*contents); + } + + if (s.ok()) { + // If filling cache is allowed and a cache is configured, try to put the + // block to the cache. + s = PutDataBlockToCache( + key, block_cache, block_cache_compressed, out_parsed_block, + std::move(*contents), contents_comp_type, uncompression_dict, + GetMemoryAllocator(rep_->table_options), block_type, get_context); + } + } + } + + // Fill lookup_context. + if (block_cache_tracer_ && block_cache_tracer_->is_tracing_enabled() && + lookup_context) { + size_t usage = 0; + uint64_t nkeys = 0; + if (out_parsed_block->GetValue()) { + // Approximate the number of keys in the block using restarts. + nkeys = rep_->table_options.block_restart_interval * + BlocklikeTraits<TBlocklike>::GetNumRestarts( + *out_parsed_block->GetValue()); + usage = out_parsed_block->GetValue()->ApproximateMemoryUsage(); + } + TraceType trace_block_type = TraceType::kTraceMax; + switch (block_type) { + case BlockType::kData: + trace_block_type = TraceType::kBlockTraceDataBlock; + break; + case BlockType::kFilter: + case BlockType::kFilterPartitionIndex: + trace_block_type = TraceType::kBlockTraceFilterBlock; + break; + case BlockType::kCompressionDictionary: + trace_block_type = TraceType::kBlockTraceUncompressionDictBlock; + break; + case BlockType::kRangeDeletion: + trace_block_type = TraceType::kBlockTraceRangeDeletionBlock; + break; + case BlockType::kIndex: + trace_block_type = TraceType::kBlockTraceIndexBlock; + break; + default: + // This cannot happen. + assert(false); + break; + } + bool no_insert = no_io || !ro.fill_cache; + if (BlockCacheTraceHelper::IsGetOrMultiGetOnDataBlock( + trace_block_type, lookup_context->caller)) { + // Defer logging the access to Get() and MultiGet() to trace additional + // information, e.g., referenced_key_exist_in_block. + + // Make a copy of the block key here since it will be logged later. + lookup_context->FillLookupContext( + is_cache_hit, no_insert, trace_block_type, + /*block_size=*/usage, /*block_key=*/key.ToString(), nkeys); + } else { + // Avoid making copy of block_key and cf_name when constructing the access + // record. + BlockCacheTraceRecord access_record( + rep_->ioptions.clock->NowMicros(), + /*block_key=*/"", trace_block_type, + /*block_size=*/usage, rep_->cf_id_for_tracing(), + /*cf_name=*/"", rep_->level_for_tracing(), + rep_->sst_number_for_tracing(), lookup_context->caller, is_cache_hit, + no_insert, lookup_context->get_id, + lookup_context->get_from_user_specified_snapshot, + /*referenced_key=*/""); + // TODO: Should handle this error? + block_cache_tracer_ + ->WriteBlockAccess(access_record, key, rep_->cf_name_for_tracing(), + lookup_context->referenced_key) + .PermitUncheckedError(); + } + } + + assert(s.ok() || out_parsed_block->GetValue() == nullptr); + return s; +} + +template <typename TBlocklike> +Status BlockBasedTable::RetrieveBlock( + FilePrefetchBuffer* prefetch_buffer, const ReadOptions& ro, + const BlockHandle& handle, const UncompressionDict& uncompression_dict, + CachableEntry<TBlocklike>* out_parsed_block, BlockType block_type, + GetContext* get_context, BlockCacheLookupContext* lookup_context, + bool for_compaction, bool use_cache, bool wait_for_cache, + bool async_read) const { + assert(out_parsed_block); + assert(out_parsed_block->IsEmpty()); + + Status s; + if (use_cache) { + s = MaybeReadBlockAndLoadToCache(prefetch_buffer, ro, handle, + uncompression_dict, wait_for_cache, + for_compaction, out_parsed_block, + block_type, get_context, lookup_context, + /*contents=*/nullptr, async_read); + + if (!s.ok()) { + return s; + } + + if (out_parsed_block->GetValue() != nullptr || + out_parsed_block->GetCacheHandle() != nullptr) { + assert(s.ok()); + return s; + } + } + + assert(out_parsed_block->IsEmpty()); + + const bool no_io = ro.read_tier == kBlockCacheTier; + if (no_io) { + return Status::Incomplete("no blocking io"); + } + + const bool maybe_compressed = + block_type != BlockType::kFilter && + block_type != BlockType::kCompressionDictionary && + rep_->blocks_maybe_compressed; + const bool do_uncompress = maybe_compressed; + std::unique_ptr<TBlocklike> block; + + { + Histograms histogram = + for_compaction ? READ_BLOCK_COMPACTION_MICROS : READ_BLOCK_GET_MICROS; + StopWatch sw(rep_->ioptions.clock, rep_->ioptions.stats, histogram); + s = ReadBlockFromFile( + rep_->file.get(), prefetch_buffer, rep_->footer, ro, handle, &block, + rep_->ioptions, do_uncompress, maybe_compressed, block_type, + uncompression_dict, rep_->persistent_cache_options, + block_type == BlockType::kData + ? rep_->table_options.read_amp_bytes_per_bit + : 0, + GetMemoryAllocator(rep_->table_options), for_compaction, + rep_->blocks_definitely_zstd_compressed, + rep_->table_options.filter_policy.get(), async_read); + + if (get_context) { + switch (block_type) { + case BlockType::kIndex: + ++(get_context->get_context_stats_.num_index_read); + break; + case BlockType::kFilter: + case BlockType::kFilterPartitionIndex: + ++(get_context->get_context_stats_.num_filter_read); + break; + default: + break; + } + } + } + + if (!s.ok()) { + return s; + } + + out_parsed_block->SetOwnedValue(std::move(block)); + + assert(s.ok()); + return s; +} + +// Explicitly instantiate templates for each "blocklike" type we use. +// This makes it possible to keep the template definitions in the .cc file. +template Status BlockBasedTable::RetrieveBlock<ParsedFullFilterBlock>( + FilePrefetchBuffer* prefetch_buffer, const ReadOptions& ro, + const BlockHandle& handle, const UncompressionDict& uncompression_dict, + CachableEntry<ParsedFullFilterBlock>* out_parsed_block, + BlockType block_type, GetContext* get_context, + BlockCacheLookupContext* lookup_context, bool for_compaction, + bool use_cache, bool wait_for_cache, bool async_read) const; + +template Status BlockBasedTable::RetrieveBlock<Block>( + FilePrefetchBuffer* prefetch_buffer, const ReadOptions& ro, + const BlockHandle& handle, const UncompressionDict& uncompression_dict, + CachableEntry<Block>* out_parsed_block, BlockType block_type, + GetContext* get_context, BlockCacheLookupContext* lookup_context, + bool for_compaction, bool use_cache, bool wait_for_cache, + bool async_read) const; + +template Status BlockBasedTable::RetrieveBlock<UncompressionDict>( + FilePrefetchBuffer* prefetch_buffer, const ReadOptions& ro, + const BlockHandle& handle, const UncompressionDict& uncompression_dict, + CachableEntry<UncompressionDict>* out_parsed_block, BlockType block_type, + GetContext* get_context, BlockCacheLookupContext* lookup_context, + bool for_compaction, bool use_cache, bool wait_for_cache, + bool async_read) const; + +BlockBasedTable::PartitionedIndexIteratorState::PartitionedIndexIteratorState( + const BlockBasedTable* table, + UnorderedMap<uint64_t, CachableEntry<Block>>* block_map) + : table_(table), block_map_(block_map) {} + +InternalIteratorBase<IndexValue>* +BlockBasedTable::PartitionedIndexIteratorState::NewSecondaryIterator( + const BlockHandle& handle) { + // Return a block iterator on the index partition + auto block = block_map_->find(handle.offset()); + // block_map_ must be exhaustive + if (block == block_map_->end()) { + assert(false); + // Signal problem to caller + return nullptr; + } + const Rep* rep = table_->get_rep(); + assert(rep); + + Statistics* kNullStats = nullptr; + // We don't return pinned data from index blocks, so no need + // to set `block_contents_pinned`. + return block->second.GetValue()->NewIndexIterator( + rep->internal_comparator.user_comparator(), + rep->get_global_seqno(BlockType::kIndex), nullptr, kNullStats, true, + rep->index_has_first_key, rep->index_key_includes_seq, + rep->index_value_is_full); +} + +// This will be broken if the user specifies an unusual implementation +// of Options.comparator, or if the user specifies an unusual +// definition of prefixes in BlockBasedTableOptions.filter_policy. +// In particular, we require the following three properties: +// +// 1) key.starts_with(prefix(key)) +// 2) Compare(prefix(key), key) <= 0. +// 3) If Compare(key1, key2) <= 0, then Compare(prefix(key1), prefix(key2)) <= 0 +// +// If read_options.read_tier == kBlockCacheTier, this method will do no I/O and +// will return true if the filter block is not in memory and not found in block +// cache. +// +// REQUIRES: this method shouldn't be called while the DB lock is held. +bool BlockBasedTable::PrefixRangeMayMatch( + const Slice& internal_key, const ReadOptions& read_options, + const SliceTransform* options_prefix_extractor, + const bool need_upper_bound_check, + BlockCacheLookupContext* lookup_context) const { + if (!rep_->filter_policy) { + return true; + } + + const SliceTransform* prefix_extractor; + + if (rep_->table_prefix_extractor == nullptr) { + if (need_upper_bound_check) { + return true; + } + prefix_extractor = options_prefix_extractor; + } else { + prefix_extractor = rep_->table_prefix_extractor.get(); + } + auto ts_sz = rep_->internal_comparator.user_comparator()->timestamp_size(); + auto user_key_without_ts = + ExtractUserKeyAndStripTimestamp(internal_key, ts_sz); + if (!prefix_extractor->InDomain(user_key_without_ts)) { + return true; + } + + bool may_match = true; + + FilterBlockReader* const filter = rep_->filter.get(); + bool filter_checked = false; + if (filter != nullptr) { + const bool no_io = read_options.read_tier == kBlockCacheTier; + + const Slice* const const_ikey_ptr = &internal_key; + may_match = filter->RangeMayExist( + read_options.iterate_upper_bound, user_key_without_ts, prefix_extractor, + rep_->internal_comparator.user_comparator(), const_ikey_ptr, + &filter_checked, need_upper_bound_check, no_io, lookup_context, + read_options.rate_limiter_priority); + } + + if (filter_checked) { + Statistics* statistics = rep_->ioptions.stats; + RecordTick(statistics, BLOOM_FILTER_PREFIX_CHECKED); + if (!may_match) { + RecordTick(statistics, BLOOM_FILTER_PREFIX_USEFUL); + } + } + + return may_match; +} + +bool BlockBasedTable::PrefixExtractorChanged( + const SliceTransform* prefix_extractor) const { + if (prefix_extractor == nullptr) { + return true; + } else if (prefix_extractor == rep_->table_prefix_extractor.get()) { + return false; + } else { + return PrefixExtractorChangedHelper(rep_->table_properties.get(), + prefix_extractor); + } +} + +InternalIterator* BlockBasedTable::NewIterator( + const ReadOptions& read_options, const SliceTransform* prefix_extractor, + Arena* arena, bool skip_filters, TableReaderCaller caller, + size_t compaction_readahead_size, bool allow_unprepared_value) { + BlockCacheLookupContext lookup_context{caller}; + bool need_upper_bound_check = + read_options.auto_prefix_mode || PrefixExtractorChanged(prefix_extractor); + std::unique_ptr<InternalIteratorBase<IndexValue>> index_iter(NewIndexIterator( + read_options, + /*disable_prefix_seek=*/need_upper_bound_check && + rep_->index_type == BlockBasedTableOptions::kHashSearch, + /*input_iter=*/nullptr, /*get_context=*/nullptr, &lookup_context)); + if (arena == nullptr) { + return new BlockBasedTableIterator( + this, read_options, rep_->internal_comparator, std::move(index_iter), + !skip_filters && !read_options.total_order_seek && + prefix_extractor != nullptr, + need_upper_bound_check, prefix_extractor, caller, + compaction_readahead_size, allow_unprepared_value); + } else { + auto* mem = arena->AllocateAligned(sizeof(BlockBasedTableIterator)); + return new (mem) BlockBasedTableIterator( + this, read_options, rep_->internal_comparator, std::move(index_iter), + !skip_filters && !read_options.total_order_seek && + prefix_extractor != nullptr, + need_upper_bound_check, prefix_extractor, caller, + compaction_readahead_size, allow_unprepared_value); + } +} + +FragmentedRangeTombstoneIterator* BlockBasedTable::NewRangeTombstoneIterator( + const ReadOptions& read_options) { + if (rep_->fragmented_range_dels == nullptr) { + return nullptr; + } + SequenceNumber snapshot = kMaxSequenceNumber; + if (read_options.snapshot != nullptr) { + snapshot = read_options.snapshot->GetSequenceNumber(); + } + return new FragmentedRangeTombstoneIterator(rep_->fragmented_range_dels, + rep_->internal_comparator, + snapshot, read_options.timestamp); +} + +bool BlockBasedTable::FullFilterKeyMayMatch( + FilterBlockReader* filter, const Slice& internal_key, const bool no_io, + const SliceTransform* prefix_extractor, GetContext* get_context, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) const { + if (filter == nullptr) { + return true; + } + Slice user_key = ExtractUserKey(internal_key); + const Slice* const const_ikey_ptr = &internal_key; + bool may_match = true; + size_t ts_sz = rep_->internal_comparator.user_comparator()->timestamp_size(); + Slice user_key_without_ts = StripTimestampFromUserKey(user_key, ts_sz); + if (rep_->whole_key_filtering) { + may_match = + filter->KeyMayMatch(user_key_without_ts, no_io, const_ikey_ptr, + get_context, lookup_context, rate_limiter_priority); + } else if (!PrefixExtractorChanged(prefix_extractor) && + prefix_extractor->InDomain(user_key_without_ts) && + !filter->PrefixMayMatch( + prefix_extractor->Transform(user_key_without_ts), no_io, + const_ikey_ptr, get_context, lookup_context, + rate_limiter_priority)) { + // FIXME ^^^: there should be no reason for Get() to depend on current + // prefix_extractor at all. It should always use table_prefix_extractor. + may_match = false; + } + if (may_match) { + RecordTick(rep_->ioptions.stats, BLOOM_FILTER_FULL_POSITIVE); + PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_full_positive, 1, rep_->level); + } + return may_match; +} + +void BlockBasedTable::FullFilterKeysMayMatch( + FilterBlockReader* filter, MultiGetRange* range, const bool no_io, + const SliceTransform* prefix_extractor, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) const { + if (filter == nullptr) { + return; + } + uint64_t before_keys = range->KeysLeft(); + assert(before_keys > 0); // Caller should ensure + if (rep_->whole_key_filtering) { + filter->KeysMayMatch(range, no_io, lookup_context, rate_limiter_priority); + uint64_t after_keys = range->KeysLeft(); + if (after_keys) { + RecordTick(rep_->ioptions.stats, BLOOM_FILTER_FULL_POSITIVE, after_keys); + PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_full_positive, after_keys, + rep_->level); + } + uint64_t filtered_keys = before_keys - after_keys; + if (filtered_keys) { + RecordTick(rep_->ioptions.stats, BLOOM_FILTER_USEFUL, filtered_keys); + PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_useful, filtered_keys, + rep_->level); + } + } else if (!PrefixExtractorChanged(prefix_extractor)) { + // FIXME ^^^: there should be no reason for MultiGet() to depend on current + // prefix_extractor at all. It should always use table_prefix_extractor. + filter->PrefixesMayMatch(range, prefix_extractor, false, lookup_context, + rate_limiter_priority); + RecordTick(rep_->ioptions.stats, BLOOM_FILTER_PREFIX_CHECKED, before_keys); + uint64_t after_keys = range->KeysLeft(); + uint64_t filtered_keys = before_keys - after_keys; + if (filtered_keys) { + RecordTick(rep_->ioptions.stats, BLOOM_FILTER_PREFIX_USEFUL, + filtered_keys); + } + } +} + +Status BlockBasedTable::ApproximateKeyAnchors(const ReadOptions& read_options, + std::vector<Anchor>& anchors) { + // We iterator the whole index block here. More efficient implementation + // is possible if we push this operation into IndexReader. For example, we + // can directly sample from restart block entries in the index block and + // only read keys needed. Here we take a simple solution. Performance is + // likely not to be a problem. We are compacting the whole file, so all + // keys will be read out anyway. An extra read to index block might be + // a small share of the overhead. We can try to optimize if needed. + IndexBlockIter iiter_on_stack; + auto iiter = NewIndexIterator( + read_options, /*disable_prefix_seek=*/false, &iiter_on_stack, + /*get_context=*/nullptr, /*lookup_context=*/nullptr); + std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr; + if (iiter != &iiter_on_stack) { + iiter_unique_ptr.reset(iiter); + } + + // If needed the threshold could be more adaptive. For example, it can be + // based on size, so that a larger will be sampled to more partitions than a + // smaller file. The size might also need to be passed in by the caller based + // on total compaction size. + const uint64_t kMaxNumAnchors = uint64_t{128}; + uint64_t num_blocks = this->GetTableProperties()->num_data_blocks; + uint64_t num_blocks_per_anchor = num_blocks / kMaxNumAnchors; + if (num_blocks_per_anchor == 0) { + num_blocks_per_anchor = 1; + } + + uint64_t count = 0; + std::string last_key; + uint64_t range_size = 0; + uint64_t prev_offset = 0; + for (iiter->SeekToFirst(); iiter->Valid(); iiter->Next()) { + const BlockHandle& bh = iiter->value().handle; + range_size += bh.offset() + bh.size() - prev_offset; + prev_offset = bh.offset() + bh.size(); + if (++count % num_blocks_per_anchor == 0) { + count = 0; + anchors.emplace_back(iiter->user_key(), range_size); + range_size = 0; + } else { + last_key = iiter->user_key().ToString(); + } + } + if (count != 0) { + anchors.emplace_back(last_key, range_size); + } + return Status::OK(); +} + +Status BlockBasedTable::Get(const ReadOptions& read_options, const Slice& key, + GetContext* get_context, + const SliceTransform* prefix_extractor, + bool skip_filters) { + assert(key.size() >= 8); // key must be internal key + assert(get_context != nullptr); + Status s; + const bool no_io = read_options.read_tier == kBlockCacheTier; + + FilterBlockReader* const filter = + !skip_filters ? rep_->filter.get() : nullptr; + + // First check the full filter + // If full filter not useful, Then go into each block + uint64_t tracing_get_id = get_context->get_tracing_get_id(); + BlockCacheLookupContext lookup_context{ + TableReaderCaller::kUserGet, tracing_get_id, + /*get_from_user_specified_snapshot=*/read_options.snapshot != nullptr}; + if (block_cache_tracer_ && block_cache_tracer_->is_tracing_enabled()) { + // Trace the key since it contains both user key and sequence number. + lookup_context.referenced_key = key.ToString(); + lookup_context.get_from_user_specified_snapshot = + read_options.snapshot != nullptr; + } + TEST_SYNC_POINT("BlockBasedTable::Get:BeforeFilterMatch"); + const bool may_match = FullFilterKeyMayMatch( + filter, key, no_io, prefix_extractor, get_context, &lookup_context, + read_options.rate_limiter_priority); + TEST_SYNC_POINT("BlockBasedTable::Get:AfterFilterMatch"); + if (!may_match) { + RecordTick(rep_->ioptions.stats, BLOOM_FILTER_USEFUL); + PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_useful, 1, rep_->level); + } else { + IndexBlockIter iiter_on_stack; + // if prefix_extractor found in block differs from options, disable + // BlockPrefixIndex. Only do this check when index_type is kHashSearch. + bool need_upper_bound_check = false; + if (rep_->index_type == BlockBasedTableOptions::kHashSearch) { + need_upper_bound_check = PrefixExtractorChanged(prefix_extractor); + } + auto iiter = + NewIndexIterator(read_options, need_upper_bound_check, &iiter_on_stack, + get_context, &lookup_context); + std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr; + if (iiter != &iiter_on_stack) { + iiter_unique_ptr.reset(iiter); + } + + size_t ts_sz = + rep_->internal_comparator.user_comparator()->timestamp_size(); + bool matched = false; // if such user key matched a key in SST + bool done = false; + for (iiter->Seek(key); iiter->Valid() && !done; iiter->Next()) { + IndexValue v = iiter->value(); + + if (!v.first_internal_key.empty() && !skip_filters && + UserComparatorWrapper(rep_->internal_comparator.user_comparator()) + .CompareWithoutTimestamp( + ExtractUserKey(key), + ExtractUserKey(v.first_internal_key)) < 0) { + // The requested key falls between highest key in previous block and + // lowest key in current block. + break; + } + + BlockCacheLookupContext lookup_data_block_context{ + TableReaderCaller::kUserGet, tracing_get_id, + /*get_from_user_specified_snapshot=*/read_options.snapshot != + nullptr}; + bool does_referenced_key_exist = false; + DataBlockIter biter; + uint64_t referenced_data_size = 0; + Status tmp_status; + NewDataBlockIterator<DataBlockIter>( + read_options, v.handle, &biter, BlockType::kData, get_context, + &lookup_data_block_context, /*prefetch_buffer=*/nullptr, + /*for_compaction=*/false, /*async_read=*/false, tmp_status); + + if (no_io && biter.status().IsIncomplete()) { + // couldn't get block from block_cache + // Update Saver.state to Found because we are only looking for + // whether we can guarantee the key is not there when "no_io" is set + get_context->MarkKeyMayExist(); + s = biter.status(); + break; + } + if (!biter.status().ok()) { + s = biter.status(); + break; + } + + bool may_exist = biter.SeekForGet(key); + // If user-specified timestamp is supported, we cannot end the search + // just because hash index lookup indicates the key+ts does not exist. + if (!may_exist && ts_sz == 0) { + // HashSeek cannot find the key this block and the the iter is not + // the end of the block, i.e. cannot be in the following blocks + // either. In this case, the seek_key cannot be found, so we break + // from the top level for-loop. + done = true; + } else { + // Call the *saver function on each entry/block until it returns false + for (; biter.Valid(); biter.Next()) { + ParsedInternalKey parsed_key; + Status pik_status = ParseInternalKey( + biter.key(), &parsed_key, false /* log_err_key */); // TODO + if (!pik_status.ok()) { + s = pik_status; + } + + if (!get_context->SaveValue( + parsed_key, biter.value(), &matched, + biter.IsValuePinned() ? &biter : nullptr)) { + if (get_context->State() == GetContext::GetState::kFound) { + does_referenced_key_exist = true; + referenced_data_size = biter.key().size() + biter.value().size(); + } + done = true; + break; + } + } + s = biter.status(); + } + // Write the block cache access record. + if (block_cache_tracer_ && block_cache_tracer_->is_tracing_enabled()) { + // Avoid making copy of block_key, cf_name, and referenced_key when + // constructing the access record. + Slice referenced_key; + if (does_referenced_key_exist) { + referenced_key = biter.key(); + } else { + referenced_key = key; + } + BlockCacheTraceRecord access_record( + rep_->ioptions.clock->NowMicros(), + /*block_key=*/"", lookup_data_block_context.block_type, + lookup_data_block_context.block_size, rep_->cf_id_for_tracing(), + /*cf_name=*/"", rep_->level_for_tracing(), + rep_->sst_number_for_tracing(), lookup_data_block_context.caller, + lookup_data_block_context.is_cache_hit, + lookup_data_block_context.no_insert, + lookup_data_block_context.get_id, + lookup_data_block_context.get_from_user_specified_snapshot, + /*referenced_key=*/"", referenced_data_size, + lookup_data_block_context.num_keys_in_block, + does_referenced_key_exist); + // TODO: Should handle status here? + block_cache_tracer_ + ->WriteBlockAccess(access_record, + lookup_data_block_context.block_key, + rep_->cf_name_for_tracing(), referenced_key) + .PermitUncheckedError(); + } + + if (done) { + // Avoid the extra Next which is expensive in two-level indexes + break; + } + } + if (matched && filter != nullptr) { + RecordTick(rep_->ioptions.stats, BLOOM_FILTER_FULL_TRUE_POSITIVE); + PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_full_true_positive, 1, + rep_->level); + } + if (s.ok() && !iiter->status().IsNotFound()) { + s = iiter->status(); + } + } + + return s; +} + +Status BlockBasedTable::MultiGetFilter(const ReadOptions& read_options, + const SliceTransform* prefix_extractor, + MultiGetRange* mget_range) { + if (mget_range->empty()) { + // Caller should ensure non-empty (performance bug) + assert(false); + return Status::OK(); // Nothing to do + } + + FilterBlockReader* const filter = rep_->filter.get(); + if (!filter) { + return Status::OK(); + } + + // First check the full filter + // If full filter not useful, Then go into each block + const bool no_io = read_options.read_tier == kBlockCacheTier; + uint64_t tracing_mget_id = BlockCacheTraceHelper::kReservedGetId; + if (mget_range->begin()->get_context) { + tracing_mget_id = mget_range->begin()->get_context->get_tracing_get_id(); + } + BlockCacheLookupContext lookup_context{ + TableReaderCaller::kUserMultiGet, tracing_mget_id, + /*_get_from_user_specified_snapshot=*/read_options.snapshot != nullptr}; + FullFilterKeysMayMatch(filter, mget_range, no_io, prefix_extractor, + &lookup_context, read_options.rate_limiter_priority); + + return Status::OK(); +} + +Status BlockBasedTable::Prefetch(const Slice* const begin, + const Slice* const end) { + auto& comparator = rep_->internal_comparator; + UserComparatorWrapper user_comparator(comparator.user_comparator()); + // pre-condition + if (begin && end && comparator.Compare(*begin, *end) > 0) { + return Status::InvalidArgument(*begin, *end); + } + BlockCacheLookupContext lookup_context{TableReaderCaller::kPrefetch}; + IndexBlockIter iiter_on_stack; + auto iiter = NewIndexIterator(ReadOptions(), /*need_upper_bound_check=*/false, + &iiter_on_stack, /*get_context=*/nullptr, + &lookup_context); + std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr; + if (iiter != &iiter_on_stack) { + iiter_unique_ptr = std::unique_ptr<InternalIteratorBase<IndexValue>>(iiter); + } + + if (!iiter->status().ok()) { + // error opening index iterator + return iiter->status(); + } + + // indicates if we are on the last page that need to be pre-fetched + bool prefetching_boundary_page = false; + + for (begin ? iiter->Seek(*begin) : iiter->SeekToFirst(); iiter->Valid(); + iiter->Next()) { + BlockHandle block_handle = iiter->value().handle; + const bool is_user_key = !rep_->index_key_includes_seq; + if (end && + ((!is_user_key && comparator.Compare(iiter->key(), *end) >= 0) || + (is_user_key && + user_comparator.Compare(iiter->key(), ExtractUserKey(*end)) >= 0))) { + if (prefetching_boundary_page) { + break; + } + + // The index entry represents the last key in the data block. + // We should load this page into memory as well, but no more + prefetching_boundary_page = true; + } + + // Load the block specified by the block_handle into the block cache + DataBlockIter biter; + Status tmp_status; + NewDataBlockIterator<DataBlockIter>( + ReadOptions(), block_handle, &biter, /*type=*/BlockType::kData, + /*get_context=*/nullptr, &lookup_context, + /*prefetch_buffer=*/nullptr, /*for_compaction=*/false, + /*async_read=*/false, tmp_status); + + if (!biter.status().ok()) { + // there was an unexpected error while pre-fetching + return biter.status(); + } + } + + return Status::OK(); +} + +Status BlockBasedTable::VerifyChecksum(const ReadOptions& read_options, + TableReaderCaller caller) { + Status s; + // Check Meta blocks + std::unique_ptr<Block> metaindex; + std::unique_ptr<InternalIterator> metaindex_iter; + ReadOptions ro; + s = ReadMetaIndexBlock(ro, nullptr /* prefetch buffer */, &metaindex, + &metaindex_iter); + if (s.ok()) { + s = VerifyChecksumInMetaBlocks(metaindex_iter.get()); + if (!s.ok()) { + return s; + } + } else { + return s; + } + // Check Data blocks + IndexBlockIter iiter_on_stack; + BlockCacheLookupContext context{caller}; + InternalIteratorBase<IndexValue>* iiter = NewIndexIterator( + read_options, /*disable_prefix_seek=*/false, &iiter_on_stack, + /*get_context=*/nullptr, &context); + std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr; + if (iiter != &iiter_on_stack) { + iiter_unique_ptr = std::unique_ptr<InternalIteratorBase<IndexValue>>(iiter); + } + if (!iiter->status().ok()) { + // error opening index iterator + return iiter->status(); + } + s = VerifyChecksumInBlocks(read_options, iiter); + return s; +} + +Status BlockBasedTable::VerifyChecksumInBlocks( + const ReadOptions& read_options, + InternalIteratorBase<IndexValue>* index_iter) { + Status s; + // We are scanning the whole file, so no need to do exponential + // increasing of the buffer size. + size_t readahead_size = (read_options.readahead_size != 0) + ? read_options.readahead_size + : rep_->table_options.max_auto_readahead_size; + // FilePrefetchBuffer doesn't work in mmap mode and readahead is not + // needed there. + FilePrefetchBuffer prefetch_buffer( + readahead_size /* readahead_size */, + readahead_size /* max_readahead_size */, + !rep_->ioptions.allow_mmap_reads /* enable */); + + for (index_iter->SeekToFirst(); index_iter->Valid(); index_iter->Next()) { + s = index_iter->status(); + if (!s.ok()) { + break; + } + BlockHandle handle = index_iter->value().handle; + BlockContents contents; + BlockFetcher block_fetcher( + rep_->file.get(), &prefetch_buffer, rep_->footer, read_options, handle, + &contents, rep_->ioptions, false /* decompress */, + false /*maybe_compressed*/, BlockType::kData, + UncompressionDict::GetEmptyDict(), rep_->persistent_cache_options); + s = block_fetcher.ReadBlockContents(); + if (!s.ok()) { + break; + } + } + if (s.ok()) { + // In the case of two level indexes, we would have exited the above loop + // by checking index_iter->Valid(), but Valid() might have returned false + // due to an IO error. So check the index_iter status + s = index_iter->status(); + } + return s; +} + +BlockType BlockBasedTable::GetBlockTypeForMetaBlockByName( + const Slice& meta_block_name) { + if (meta_block_name.starts_with(kFullFilterBlockPrefix)) { + return BlockType::kFilter; + } + + if (meta_block_name.starts_with(kPartitionedFilterBlockPrefix)) { + return BlockType::kFilterPartitionIndex; + } + + if (meta_block_name == kPropertiesBlockName) { + return BlockType::kProperties; + } + + if (meta_block_name == kCompressionDictBlockName) { + return BlockType::kCompressionDictionary; + } + + if (meta_block_name == kRangeDelBlockName) { + return BlockType::kRangeDeletion; + } + + if (meta_block_name == kHashIndexPrefixesBlock) { + return BlockType::kHashIndexPrefixes; + } + + if (meta_block_name == kHashIndexPrefixesMetadataBlock) { + return BlockType::kHashIndexMetadata; + } + + if (meta_block_name.starts_with(kObsoleteFilterBlockPrefix)) { + // Obsolete but possible in old files + return BlockType::kInvalid; + } + + assert(false); + return BlockType::kInvalid; +} + +Status BlockBasedTable::VerifyChecksumInMetaBlocks( + InternalIteratorBase<Slice>* index_iter) { + Status s; + for (index_iter->SeekToFirst(); index_iter->Valid(); index_iter->Next()) { + s = index_iter->status(); + if (!s.ok()) { + break; + } + BlockHandle handle; + Slice input = index_iter->value(); + s = handle.DecodeFrom(&input); + BlockContents contents; + const Slice meta_block_name = index_iter->key(); + if (meta_block_name == kPropertiesBlockName) { + // Unfortunate special handling for properties block checksum w/ + // global seqno + std::unique_ptr<TableProperties> table_properties; + s = ReadTablePropertiesHelper(ReadOptions(), handle, rep_->file.get(), + nullptr /* prefetch_buffer */, rep_->footer, + rep_->ioptions, &table_properties, + nullptr /* memory_allocator */); + } else { + s = BlockFetcher( + rep_->file.get(), nullptr /* prefetch buffer */, rep_->footer, + ReadOptions(), handle, &contents, rep_->ioptions, + false /* decompress */, false /*maybe_compressed*/, + GetBlockTypeForMetaBlockByName(meta_block_name), + UncompressionDict::GetEmptyDict(), rep_->persistent_cache_options) + .ReadBlockContents(); + } + if (!s.ok()) { + break; + } + } + return s; +} + +bool BlockBasedTable::TEST_BlockInCache(const BlockHandle& handle) const { + assert(rep_ != nullptr); + + Cache* const cache = rep_->table_options.block_cache.get(); + if (cache == nullptr) { + return false; + } + + CacheKey key = GetCacheKey(rep_->base_cache_key, handle); + + Cache::Handle* const cache_handle = cache->Lookup(key.AsSlice()); + if (cache_handle == nullptr) { + return false; + } + + cache->Release(cache_handle); + + return true; +} + +bool BlockBasedTable::TEST_KeyInCache(const ReadOptions& options, + const Slice& key) { + std::unique_ptr<InternalIteratorBase<IndexValue>> iiter(NewIndexIterator( + options, /*need_upper_bound_check=*/false, /*input_iter=*/nullptr, + /*get_context=*/nullptr, /*lookup_context=*/nullptr)); + iiter->Seek(key); + assert(iiter->Valid()); + + return TEST_BlockInCache(iiter->value().handle); +} + +// REQUIRES: The following fields of rep_ should have already been populated: +// 1. file +// 2. index_handle, +// 3. options +// 4. internal_comparator +// 5. index_type +Status BlockBasedTable::CreateIndexReader( + const ReadOptions& ro, FilePrefetchBuffer* prefetch_buffer, + InternalIterator* meta_iter, bool use_cache, bool prefetch, bool pin, + BlockCacheLookupContext* lookup_context, + std::unique_ptr<IndexReader>* index_reader) { + switch (rep_->index_type) { + case BlockBasedTableOptions::kTwoLevelIndexSearch: { + return PartitionIndexReader::Create(this, ro, prefetch_buffer, use_cache, + prefetch, pin, lookup_context, + index_reader); + } + case BlockBasedTableOptions::kBinarySearch: + FALLTHROUGH_INTENDED; + case BlockBasedTableOptions::kBinarySearchWithFirstKey: { + return BinarySearchIndexReader::Create(this, ro, prefetch_buffer, + use_cache, prefetch, pin, + lookup_context, index_reader); + } + case BlockBasedTableOptions::kHashSearch: { + if (!rep_->table_prefix_extractor) { + ROCKS_LOG_WARN(rep_->ioptions.logger, + "Missing prefix extractor for hash index. Fall back to" + " binary search index."); + return BinarySearchIndexReader::Create(this, ro, prefetch_buffer, + use_cache, prefetch, pin, + lookup_context, index_reader); + } else { + return HashIndexReader::Create(this, ro, prefetch_buffer, meta_iter, + use_cache, prefetch, pin, lookup_context, + index_reader); + } + } + default: { + std::string error_message = + "Unrecognized index type: " + std::to_string(rep_->index_type); + return Status::InvalidArgument(error_message.c_str()); + } + } +} + +uint64_t BlockBasedTable::ApproximateDataOffsetOf( + const InternalIteratorBase<IndexValue>& index_iter, + uint64_t data_size) const { + assert(index_iter.status().ok()); + if (index_iter.Valid()) { + BlockHandle handle = index_iter.value().handle; + return handle.offset(); + } else { + // The iterator is past the last key in the file. + return data_size; + } +} + +uint64_t BlockBasedTable::GetApproximateDataSize() { + // Should be in table properties unless super old version + if (rep_->table_properties) { + return rep_->table_properties->data_size; + } + // Fall back to rough estimate from footer + return rep_->footer.metaindex_handle().offset(); +} + +uint64_t BlockBasedTable::ApproximateOffsetOf(const Slice& key, + TableReaderCaller caller) { + uint64_t data_size = GetApproximateDataSize(); + if (UNLIKELY(data_size == 0)) { + // Hmm. Let's just split in half to avoid skewing one way or another, + // since we don't know whether we're operating on lower bound or + // upper bound. + return rep_->file_size / 2; + } + + BlockCacheLookupContext context(caller); + IndexBlockIter iiter_on_stack; + ReadOptions ro; + ro.total_order_seek = true; + auto index_iter = + NewIndexIterator(ro, /*disable_prefix_seek=*/true, + /*input_iter=*/&iiter_on_stack, /*get_context=*/nullptr, + /*lookup_context=*/&context); + std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr; + if (index_iter != &iiter_on_stack) { + iiter_unique_ptr.reset(index_iter); + } + + index_iter->Seek(key); + uint64_t offset; + if (index_iter->status().ok()) { + offset = ApproximateDataOffsetOf(*index_iter, data_size); + } else { + // Split in half to avoid skewing one way or another, + // since we don't know whether we're operating on lower bound or + // upper bound. + return rep_->file_size / 2; + } + + // Pro-rate file metadata (incl filters) size-proportionally across data + // blocks. + double size_ratio = + static_cast<double>(offset) / static_cast<double>(data_size); + return static_cast<uint64_t>(size_ratio * + static_cast<double>(rep_->file_size)); +} + +uint64_t BlockBasedTable::ApproximateSize(const Slice& start, const Slice& end, + TableReaderCaller caller) { + assert(rep_->internal_comparator.Compare(start, end) <= 0); + + uint64_t data_size = GetApproximateDataSize(); + if (UNLIKELY(data_size == 0)) { + // Hmm. Assume whole file is involved, since we have lower and upper + // bound. This likely skews the estimate if we consider that this function + // is typically called with `[start, end]` fully contained in the file's + // key-range. + return rep_->file_size; + } + + BlockCacheLookupContext context(caller); + IndexBlockIter iiter_on_stack; + ReadOptions ro; + ro.total_order_seek = true; + auto index_iter = + NewIndexIterator(ro, /*disable_prefix_seek=*/true, + /*input_iter=*/&iiter_on_stack, /*get_context=*/nullptr, + /*lookup_context=*/&context); + std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr; + if (index_iter != &iiter_on_stack) { + iiter_unique_ptr.reset(index_iter); + } + + index_iter->Seek(start); + uint64_t start_offset; + if (index_iter->status().ok()) { + start_offset = ApproximateDataOffsetOf(*index_iter, data_size); + } else { + // Assume file is involved from the start. This likely skews the estimate + // but is consistent with the above error handling. + start_offset = 0; + } + + index_iter->Seek(end); + uint64_t end_offset; + if (index_iter->status().ok()) { + end_offset = ApproximateDataOffsetOf(*index_iter, data_size); + } else { + // Assume file is involved until the end. This likely skews the estimate + // but is consistent with the above error handling. + end_offset = data_size; + } + + assert(end_offset >= start_offset); + // Pro-rate file metadata (incl filters) size-proportionally across data + // blocks. + double size_ratio = static_cast<double>(end_offset - start_offset) / + static_cast<double>(data_size); + return static_cast<uint64_t>(size_ratio * + static_cast<double>(rep_->file_size)); +} + +bool BlockBasedTable::TEST_FilterBlockInCache() const { + assert(rep_ != nullptr); + return rep_->filter_type != Rep::FilterType::kNoFilter && + TEST_BlockInCache(rep_->filter_handle); +} + +bool BlockBasedTable::TEST_IndexBlockInCache() const { + assert(rep_ != nullptr); + + return TEST_BlockInCache(rep_->footer.index_handle()); +} + +Status BlockBasedTable::GetKVPairsFromDataBlocks( + std::vector<KVPairBlock>* kv_pair_blocks) { + std::unique_ptr<InternalIteratorBase<IndexValue>> blockhandles_iter( + NewIndexIterator(ReadOptions(), /*need_upper_bound_check=*/false, + /*input_iter=*/nullptr, /*get_context=*/nullptr, + /*lookup_contex=*/nullptr)); + + Status s = blockhandles_iter->status(); + if (!s.ok()) { + // Cannot read Index Block + return s; + } + + for (blockhandles_iter->SeekToFirst(); blockhandles_iter->Valid(); + blockhandles_iter->Next()) { + s = blockhandles_iter->status(); + + if (!s.ok()) { + break; + } + + std::unique_ptr<InternalIterator> datablock_iter; + Status tmp_status; + datablock_iter.reset(NewDataBlockIterator<DataBlockIter>( + ReadOptions(), blockhandles_iter->value().handle, + /*input_iter=*/nullptr, /*type=*/BlockType::kData, + /*get_context=*/nullptr, /*lookup_context=*/nullptr, + /*prefetch_buffer=*/nullptr, /*for_compaction=*/false, + /*async_read=*/false, tmp_status)); + s = datablock_iter->status(); + + if (!s.ok()) { + // Error reading the block - Skipped + continue; + } + + KVPairBlock kv_pair_block; + for (datablock_iter->SeekToFirst(); datablock_iter->Valid(); + datablock_iter->Next()) { + s = datablock_iter->status(); + if (!s.ok()) { + // Error reading the block - Skipped + break; + } + const Slice& key = datablock_iter->key(); + const Slice& value = datablock_iter->value(); + std::string key_copy = std::string(key.data(), key.size()); + std::string value_copy = std::string(value.data(), value.size()); + + kv_pair_block.push_back( + std::make_pair(std::move(key_copy), std::move(value_copy))); + } + kv_pair_blocks->push_back(std::move(kv_pair_block)); + } + return Status::OK(); +} + +Status BlockBasedTable::DumpTable(WritableFile* out_file) { + WritableFileStringStreamAdapter out_file_wrapper(out_file); + std::ostream out_stream(&out_file_wrapper); + // Output Footer + out_stream << "Footer Details:\n" + "--------------------------------------\n"; + out_stream << " " << rep_->footer.ToString() << "\n"; + + // Output MetaIndex + out_stream << "Metaindex Details:\n" + "--------------------------------------\n"; + std::unique_ptr<Block> metaindex; + std::unique_ptr<InternalIterator> metaindex_iter; + ReadOptions ro; + Status s = ReadMetaIndexBlock(ro, nullptr /* prefetch_buffer */, &metaindex, + &metaindex_iter); + if (s.ok()) { + for (metaindex_iter->SeekToFirst(); metaindex_iter->Valid(); + metaindex_iter->Next()) { + s = metaindex_iter->status(); + if (!s.ok()) { + return s; + } + if (metaindex_iter->key() == kPropertiesBlockName) { + out_stream << " Properties block handle: " + << metaindex_iter->value().ToString(true) << "\n"; + } else if (metaindex_iter->key() == kCompressionDictBlockName) { + out_stream << " Compression dictionary block handle: " + << metaindex_iter->value().ToString(true) << "\n"; + } else if (strstr(metaindex_iter->key().ToString().c_str(), + "filter.rocksdb.") != nullptr) { + out_stream << " Filter block handle: " + << metaindex_iter->value().ToString(true) << "\n"; + } else if (metaindex_iter->key() == kRangeDelBlockName) { + out_stream << " Range deletion block handle: " + << metaindex_iter->value().ToString(true) << "\n"; + } + } + out_stream << "\n"; + } else { + return s; + } + + // Output TableProperties + const ROCKSDB_NAMESPACE::TableProperties* table_properties; + table_properties = rep_->table_properties.get(); + + if (table_properties != nullptr) { + out_stream << "Table Properties:\n" + "--------------------------------------\n"; + out_stream << " " << table_properties->ToString("\n ", ": ") << "\n"; + } + + if (rep_->filter) { + out_stream << "Filter Details:\n" + "--------------------------------------\n"; + out_stream << " " << rep_->filter->ToString() << "\n"; + } + + // Output Index block + s = DumpIndexBlock(out_stream); + if (!s.ok()) { + return s; + } + + // Output compression dictionary + if (rep_->uncompression_dict_reader) { + CachableEntry<UncompressionDict> uncompression_dict; + s = rep_->uncompression_dict_reader->GetOrReadUncompressionDictionary( + nullptr /* prefetch_buffer */, false /* no_io */, + false, /* verify_checksums */ + nullptr /* get_context */, nullptr /* lookup_context */, + &uncompression_dict); + if (!s.ok()) { + return s; + } + + assert(uncompression_dict.GetValue()); + + const Slice& raw_dict = uncompression_dict.GetValue()->GetRawDict(); + out_stream << "Compression Dictionary:\n" + "--------------------------------------\n"; + out_stream << " size (bytes): " << raw_dict.size() << "\n\n"; + out_stream << " HEX " << raw_dict.ToString(true) << "\n\n"; + } + + // Output range deletions block + auto* range_del_iter = NewRangeTombstoneIterator(ReadOptions()); + if (range_del_iter != nullptr) { + range_del_iter->SeekToFirst(); + if (range_del_iter->Valid()) { + out_stream << "Range deletions:\n" + "--------------------------------------\n"; + for (; range_del_iter->Valid(); range_del_iter->Next()) { + DumpKeyValue(range_del_iter->key(), range_del_iter->value(), + out_stream); + } + out_stream << "\n"; + } + delete range_del_iter; + } + // Output Data blocks + s = DumpDataBlocks(out_stream); + + if (!s.ok()) { + return s; + } + + if (!out_stream.good()) { + return Status::IOError("Failed to write to output file"); + } + return Status::OK(); +} + +Status BlockBasedTable::DumpIndexBlock(std::ostream& out_stream) { + out_stream << "Index Details:\n" + "--------------------------------------\n"; + std::unique_ptr<InternalIteratorBase<IndexValue>> blockhandles_iter( + NewIndexIterator(ReadOptions(), /*need_upper_bound_check=*/false, + /*input_iter=*/nullptr, /*get_context=*/nullptr, + /*lookup_contex=*/nullptr)); + Status s = blockhandles_iter->status(); + if (!s.ok()) { + out_stream << "Can not read Index Block \n\n"; + return s; + } + + out_stream << " Block key hex dump: Data block handle\n"; + out_stream << " Block key ascii\n\n"; + for (blockhandles_iter->SeekToFirst(); blockhandles_iter->Valid(); + blockhandles_iter->Next()) { + s = blockhandles_iter->status(); + if (!s.ok()) { + break; + } + Slice key = blockhandles_iter->key(); + Slice user_key; + InternalKey ikey; + if (!rep_->index_key_includes_seq) { + user_key = key; + } else { + ikey.DecodeFrom(key); + user_key = ikey.user_key(); + } + + out_stream << " HEX " << user_key.ToString(true) << ": " + << blockhandles_iter->value().ToString(true, + rep_->index_has_first_key) + << " offset " << blockhandles_iter->value().handle.offset() + << " size " << blockhandles_iter->value().handle.size() << "\n"; + + std::string str_key = user_key.ToString(); + std::string res_key(""); + char cspace = ' '; + for (size_t i = 0; i < str_key.size(); i++) { + res_key.append(&str_key[i], 1); + res_key.append(1, cspace); + } + out_stream << " ASCII " << res_key << "\n"; + out_stream << " ------\n"; + } + out_stream << "\n"; + return Status::OK(); +} + +Status BlockBasedTable::DumpDataBlocks(std::ostream& out_stream) { + std::unique_ptr<InternalIteratorBase<IndexValue>> blockhandles_iter( + NewIndexIterator(ReadOptions(), /*need_upper_bound_check=*/false, + /*input_iter=*/nullptr, /*get_context=*/nullptr, + /*lookup_contex=*/nullptr)); + Status s = blockhandles_iter->status(); + if (!s.ok()) { + out_stream << "Can not read Index Block \n\n"; + return s; + } + + uint64_t datablock_size_min = std::numeric_limits<uint64_t>::max(); + uint64_t datablock_size_max = 0; + uint64_t datablock_size_sum = 0; + + size_t block_id = 1; + for (blockhandles_iter->SeekToFirst(); blockhandles_iter->Valid(); + block_id++, blockhandles_iter->Next()) { + s = blockhandles_iter->status(); + if (!s.ok()) { + break; + } + + BlockHandle bh = blockhandles_iter->value().handle; + uint64_t datablock_size = bh.size(); + datablock_size_min = std::min(datablock_size_min, datablock_size); + datablock_size_max = std::max(datablock_size_max, datablock_size); + datablock_size_sum += datablock_size; + + out_stream << "Data Block # " << block_id << " @ " + << blockhandles_iter->value().handle.ToString(true) << "\n"; + out_stream << "--------------------------------------\n"; + + std::unique_ptr<InternalIterator> datablock_iter; + Status tmp_status; + datablock_iter.reset(NewDataBlockIterator<DataBlockIter>( + ReadOptions(), blockhandles_iter->value().handle, + /*input_iter=*/nullptr, /*type=*/BlockType::kData, + /*get_context=*/nullptr, /*lookup_context=*/nullptr, + /*prefetch_buffer=*/nullptr, /*for_compaction=*/false, + /*async_read=*/false, tmp_status)); + s = datablock_iter->status(); + + if (!s.ok()) { + out_stream << "Error reading the block - Skipped \n\n"; + continue; + } + + for (datablock_iter->SeekToFirst(); datablock_iter->Valid(); + datablock_iter->Next()) { + s = datablock_iter->status(); + if (!s.ok()) { + out_stream << "Error reading the block - Skipped \n"; + break; + } + DumpKeyValue(datablock_iter->key(), datablock_iter->value(), out_stream); + } + out_stream << "\n"; + } + + uint64_t num_datablocks = block_id - 1; + if (num_datablocks) { + double datablock_size_avg = + static_cast<double>(datablock_size_sum) / num_datablocks; + out_stream << "Data Block Summary:\n"; + out_stream << "--------------------------------------\n"; + out_stream << " # data blocks: " << num_datablocks << "\n"; + out_stream << " min data block size: " << datablock_size_min << "\n"; + out_stream << " max data block size: " << datablock_size_max << "\n"; + out_stream << " avg data block size: " + << std::to_string(datablock_size_avg) << "\n"; + } + + return Status::OK(); +} + +void BlockBasedTable::DumpKeyValue(const Slice& key, const Slice& value, + std::ostream& out_stream) { + InternalKey ikey; + ikey.DecodeFrom(key); + + out_stream << " HEX " << ikey.user_key().ToString(true) << ": " + << value.ToString(true) << "\n"; + + std::string str_key = ikey.user_key().ToString(); + std::string str_value = value.ToString(); + std::string res_key(""), res_value(""); + char cspace = ' '; + for (size_t i = 0; i < str_key.size(); i++) { + if (str_key[i] == '\0') { + res_key.append("\\0", 2); + } else { + res_key.append(&str_key[i], 1); + } + res_key.append(1, cspace); + } + for (size_t i = 0; i < str_value.size(); i++) { + if (str_value[i] == '\0') { + res_value.append("\\0", 2); + } else { + res_value.append(&str_value[i], 1); + } + res_value.append(1, cspace); + } + + out_stream << " ASCII " << res_key << ": " << res_value << "\n"; + out_stream << " ------\n"; +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/block_based_table_reader.h b/src/rocksdb/table/block_based/block_based_table_reader.h new file mode 100644 index 000000000..89de891c9 --- /dev/null +++ b/src/rocksdb/table/block_based/block_based_table_reader.h @@ -0,0 +1,739 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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 <cstdint> +#include <memory> + +#include "cache/cache_entry_roles.h" +#include "cache/cache_key.h" +#include "cache/cache_reservation_manager.h" +#include "db/range_tombstone_fragmenter.h" +#include "file/filename.h" +#include "rocksdb/slice_transform.h" +#include "rocksdb/table_properties.h" +#include "table/block_based/block.h" +#include "table/block_based/block_based_table_factory.h" +#include "table/block_based/block_type.h" +#include "table/block_based/cachable_entry.h" +#include "table/block_based/filter_block.h" +#include "table/block_based/uncompression_dict_reader.h" +#include "table/format.h" +#include "table/persistent_cache_options.h" +#include "table/table_properties_internal.h" +#include "table/table_reader.h" +#include "table/two_level_iterator.h" +#include "trace_replay/block_cache_tracer.h" +#include "util/coro_utils.h" +#include "util/hash_containers.h" + +namespace ROCKSDB_NAMESPACE { + +class Cache; +class FilterBlockReader; +class FullFilterBlockReader; +class Footer; +class InternalKeyComparator; +class Iterator; +class FSRandomAccessFile; +class TableCache; +class TableReader; +class WritableFile; +struct BlockBasedTableOptions; +struct EnvOptions; +struct ReadOptions; +class GetContext; + +using KVPairBlock = std::vector<std::pair<std::string, std::string>>; + +// Reader class for BlockBasedTable format. +// For the format of BlockBasedTable refer to +// https://github.com/facebook/rocksdb/wiki/Rocksdb-BlockBasedTable-Format. +// This is the default table type. Data is chucked into fixed size blocks and +// each block in-turn stores entries. When storing data, we can compress and/or +// encode data efficiently within a block, which often results in a much smaller +// data size compared with the raw data size. As for the record retrieval, we'll +// first locate the block where target record may reside, then read the block to +// memory, and finally search that record within the block. Of course, to avoid +// frequent reads of the same block, we introduced the block cache to keep the +// loaded blocks in the memory. +class BlockBasedTable : public TableReader { + public: + static const std::string kObsoleteFilterBlockPrefix; + static const std::string kFullFilterBlockPrefix; + static const std::string kPartitionedFilterBlockPrefix; + + // 1-byte compression type + 32-bit checksum + static constexpr size_t kBlockTrailerSize = 5; + + // Attempt to open the table that is stored in bytes [0..file_size) + // of "file", and read the metadata entries necessary to allow + // retrieving data from the table. + // + // If successful, returns ok and sets "*table_reader" to the newly opened + // table. The client should delete "*table_reader" when no longer needed. + // If there was an error while initializing the table, sets "*table_reader" + // to nullptr and returns a non-ok status. + // + // @param file must remain live while this Table is in use. + // @param prefetch_index_and_filter_in_cache can be used to disable + // prefetching of + // index and filter blocks into block cache at startup + // @param skip_filters Disables loading/accessing the filter block. Overrides + // prefetch_index_and_filter_in_cache, so filter will be skipped if both + // are set. + // @param force_direct_prefetch if true, always prefetching to RocksDB + // buffer, rather than calling RandomAccessFile::Prefetch(). + static Status Open( + const ReadOptions& ro, const ImmutableOptions& ioptions, + const EnvOptions& env_options, + const BlockBasedTableOptions& table_options, + const InternalKeyComparator& internal_key_comparator, + std::unique_ptr<RandomAccessFileReader>&& file, uint64_t file_size, + std::unique_ptr<TableReader>* table_reader, + std::shared_ptr<CacheReservationManager> table_reader_cache_res_mgr = + nullptr, + const std::shared_ptr<const SliceTransform>& prefix_extractor = nullptr, + bool prefetch_index_and_filter_in_cache = true, bool skip_filters = false, + int level = -1, const bool immortal_table = false, + const SequenceNumber largest_seqno = 0, + bool force_direct_prefetch = false, + TailPrefetchStats* tail_prefetch_stats = nullptr, + BlockCacheTracer* const block_cache_tracer = nullptr, + size_t max_file_size_for_l0_meta_pin = 0, + const std::string& cur_db_session_id = "", uint64_t cur_file_num = 0, + UniqueId64x2 expected_unique_id = {}); + + bool PrefixRangeMayMatch(const Slice& internal_key, + const ReadOptions& read_options, + const SliceTransform* options_prefix_extractor, + const bool need_upper_bound_check, + BlockCacheLookupContext* lookup_context) const; + + // Returns a new iterator over the table contents. + // The result of NewIterator() is initially invalid (caller must + // call one of the Seek methods on the iterator before using it). + // @param read_options Must outlive the returned iterator. + // @param skip_filters Disables loading/accessing the filter block + // compaction_readahead_size: its value will only be used if caller = + // kCompaction. + InternalIterator* NewIterator(const ReadOptions&, + const SliceTransform* prefix_extractor, + Arena* arena, bool skip_filters, + TableReaderCaller caller, + size_t compaction_readahead_size = 0, + bool allow_unprepared_value = false) override; + + FragmentedRangeTombstoneIterator* NewRangeTombstoneIterator( + const ReadOptions& read_options) override; + + // @param skip_filters Disables loading/accessing the filter block + Status Get(const ReadOptions& readOptions, const Slice& key, + GetContext* get_context, const SliceTransform* prefix_extractor, + bool skip_filters = false) override; + + Status MultiGetFilter(const ReadOptions& read_options, + const SliceTransform* prefix_extractor, + MultiGetRange* mget_range) override; + + DECLARE_SYNC_AND_ASYNC_OVERRIDE(void, MultiGet, + const ReadOptions& readOptions, + const MultiGetContext::Range* mget_range, + const SliceTransform* prefix_extractor, + bool skip_filters = false); + + // Pre-fetch the disk blocks that correspond to the key range specified by + // (kbegin, kend). The call will return error status in the event of + // IO or iteration error. + Status Prefetch(const Slice* begin, const Slice* end) override; + + // Given a key, return an approximate byte offset in the file where + // the data for that key begins (or would begin if the key were + // present in the file). The returned value is in terms of file + // bytes, and so includes effects like compression of the underlying data. + // E.g., the approximate offset of the last key in the table will + // be close to the file length. + uint64_t ApproximateOffsetOf(const Slice& key, + TableReaderCaller caller) override; + + // Given start and end keys, return the approximate data size in the file + // between the keys. The returned value is in terms of file bytes, and so + // includes effects like compression of the underlying data. + // The start key must not be greater than the end key. + uint64_t ApproximateSize(const Slice& start, const Slice& end, + TableReaderCaller caller) override; + + Status ApproximateKeyAnchors(const ReadOptions& read_options, + std::vector<Anchor>& anchors) override; + + bool TEST_BlockInCache(const BlockHandle& handle) const; + + // Returns true if the block for the specified key is in cache. + // REQUIRES: key is in this table && block cache enabled + bool TEST_KeyInCache(const ReadOptions& options, const Slice& key); + + // Set up the table for Compaction. Might change some parameters with + // posix_fadvise + void SetupForCompaction() override; + + std::shared_ptr<const TableProperties> GetTableProperties() const override; + + size_t ApproximateMemoryUsage() const override; + + // convert SST file to a human readable form + Status DumpTable(WritableFile* out_file) override; + + Status VerifyChecksum(const ReadOptions& readOptions, + TableReaderCaller caller) override; + + ~BlockBasedTable(); + + bool TEST_FilterBlockInCache() const; + bool TEST_IndexBlockInCache() const; + + // IndexReader is the interface that provides the functionality for index + // access. + class IndexReader { + public: + virtual ~IndexReader() = default; + + // Create an iterator for index access. If iter is null, then a new object + // is created on the 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 + // wraps the passed iter. In the latter case the return value points + // to a different object then iter, and the callee has the ownership of the + // returned object. + virtual InternalIteratorBase<IndexValue>* NewIterator( + const ReadOptions& read_options, bool disable_prefix_seek, + IndexBlockIter* iter, GetContext* get_context, + BlockCacheLookupContext* lookup_context) = 0; + + // 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; + // Cache the dependencies of the index reader (e.g. the partitions + // of a partitioned index). + virtual Status CacheDependencies(const ReadOptions& /*ro*/, + bool /* pin */) { + return Status::OK(); + } + }; + + class IndexReaderCommon; + + static void SetupBaseCacheKey(const TableProperties* properties, + const std::string& cur_db_session_id, + uint64_t cur_file_number, + OffsetableCacheKey* out_base_cache_key, + bool* out_is_stable = nullptr); + + static CacheKey GetCacheKey(const OffsetableCacheKey& base_cache_key, + const BlockHandle& handle); + + static void UpdateCacheInsertionMetrics(BlockType block_type, + GetContext* get_context, size_t usage, + bool redundant, + Statistics* const statistics); + + // Get the size to read from storage for a BlockHandle. size_t because we + // are about to load into memory. + static inline size_t BlockSizeWithTrailer(const BlockHandle& handle) { + return static_cast<size_t>(handle.size() + kBlockTrailerSize); + } + + // It is the caller's responsibility to make sure that this is called with + // block-based table serialized block contents, which contains the compression + // byte in the trailer after `block_size`. + static inline CompressionType GetBlockCompressionType(const char* block_data, + size_t block_size) { + return static_cast<CompressionType>(block_data[block_size]); + } + static inline CompressionType GetBlockCompressionType( + const BlockContents& contents) { + assert(contents.has_trailer); + return GetBlockCompressionType(contents.data.data(), contents.data.size()); + } + + // Retrieve all key value pairs from data blocks in the table. + // The key retrieved are internal keys. + Status GetKVPairsFromDataBlocks(std::vector<KVPairBlock>* kv_pair_blocks); + + struct Rep; + + Rep* get_rep() { return rep_; } + const Rep* get_rep() const { return rep_; } + + // input_iter: if it is not null, update this one and return it as Iterator + template <typename TBlockIter> + TBlockIter* NewDataBlockIterator(const ReadOptions& ro, + const BlockHandle& block_handle, + TBlockIter* input_iter, BlockType block_type, + GetContext* get_context, + BlockCacheLookupContext* lookup_context, + FilePrefetchBuffer* prefetch_buffer, + bool for_compaction, bool async_read, + Status& s) const; + + // input_iter: if it is not null, update this one and return it as Iterator + template <typename TBlockIter> + TBlockIter* NewDataBlockIterator(const ReadOptions& ro, + CachableEntry<Block>& block, + TBlockIter* input_iter, Status s) const; + + class PartitionedIndexIteratorState; + + template <typename TBlocklike> + friend class FilterBlockReaderCommon; + + friend class PartitionIndexReader; + + friend class UncompressionDictReader; + + protected: + Rep* rep_; + explicit BlockBasedTable(Rep* rep, BlockCacheTracer* const block_cache_tracer) + : rep_(rep), block_cache_tracer_(block_cache_tracer) {} + // No copying allowed + explicit BlockBasedTable(const TableReader&) = delete; + void operator=(const TableReader&) = delete; + + private: + friend class MockedBlockBasedTable; + friend class BlockBasedTableReaderTestVerifyChecksum_ChecksumMismatch_Test; + BlockCacheTracer* const block_cache_tracer_; + + void UpdateCacheHitMetrics(BlockType block_type, GetContext* get_context, + size_t usage) const; + void UpdateCacheMissMetrics(BlockType block_type, + GetContext* get_context) const; + + Cache::Handle* GetEntryFromCache(const CacheTier& cache_tier, + Cache* block_cache, const Slice& key, + BlockType block_type, const bool wait, + GetContext* get_context, + const Cache::CacheItemHelper* cache_helper, + const Cache::CreateCallback& create_cb, + Cache::Priority priority) const; + + template <typename TBlocklike> + Status InsertEntryToCache(const CacheTier& cache_tier, Cache* block_cache, + const Slice& key, + const Cache::CacheItemHelper* cache_helper, + std::unique_ptr<TBlocklike>&& block_holder, + size_t charge, Cache::Handle** cache_handle, + Cache::Priority priority) const; + + // Either Block::NewDataIterator() or Block::NewIndexIterator(). + template <typename TBlockIter> + static TBlockIter* InitBlockIterator(const Rep* rep, Block* block, + BlockType block_type, + TBlockIter* input_iter, + bool block_contents_pinned); + + // If block cache enabled (compressed or uncompressed), looks for the block + // identified by handle in (1) uncompressed cache, (2) compressed cache, and + // then (3) file. If found, inserts into the cache(s) that were searched + // unsuccessfully (e.g., if found in file, will add to both uncompressed and + // compressed caches if they're enabled). + // + // @param block_entry value is set to the uncompressed block if found. If + // in uncompressed block cache, also sets cache_handle to reference that + // block. + template <typename TBlocklike> + Status MaybeReadBlockAndLoadToCache( + FilePrefetchBuffer* prefetch_buffer, const ReadOptions& ro, + const BlockHandle& handle, const UncompressionDict& uncompression_dict, + const bool wait, const bool for_compaction, + CachableEntry<TBlocklike>* block_entry, BlockType block_type, + GetContext* get_context, BlockCacheLookupContext* lookup_context, + BlockContents* contents, bool async_read) const; + + // Similar to the above, with one crucial difference: it will retrieve the + // block from the file even if there are no caches configured (assuming the + // read options allow I/O). + template <typename TBlocklike> + Status RetrieveBlock(FilePrefetchBuffer* prefetch_buffer, + const ReadOptions& ro, const BlockHandle& handle, + const UncompressionDict& uncompression_dict, + CachableEntry<TBlocklike>* block_entry, + BlockType block_type, GetContext* get_context, + BlockCacheLookupContext* lookup_context, + bool for_compaction, bool use_cache, bool wait_for_cache, + bool async_read) const; + + DECLARE_SYNC_AND_ASYNC_CONST( + void, RetrieveMultipleBlocks, const ReadOptions& options, + const MultiGetRange* batch, + const autovector<BlockHandle, MultiGetContext::MAX_BATCH_SIZE>* handles, + autovector<Status, MultiGetContext::MAX_BATCH_SIZE>* statuses, + autovector<CachableEntry<Block>, MultiGetContext::MAX_BATCH_SIZE>* + results, + char* scratch, const UncompressionDict& uncompression_dict); + + // Get the iterator from the index reader. + // + // If input_iter is not set, return a new Iterator. + // If input_iter is set, try to update it and return it as Iterator. + // However note that in some cases the returned iterator may be different + // from input_iter. In such case the returned iterator should be freed. + // + // Note: ErrorIterator with Status::Incomplete shall be returned if all the + // following conditions are met: + // 1. We enabled table_options.cache_index_and_filter_blocks. + // 2. index is not present in block cache. + // 3. We disallowed any io to be performed, that is, read_options == + // kBlockCacheTier + InternalIteratorBase<IndexValue>* NewIndexIterator( + const ReadOptions& read_options, bool need_upper_bound_check, + IndexBlockIter* input_iter, GetContext* get_context, + BlockCacheLookupContext* lookup_context) const; + + // Read block cache from block caches (if set): block_cache and + // block_cache_compressed. + // On success, Status::OK with be returned and @block will be populated with + // pointer to the block as well as its block handle. + // @param uncompression_dict Data for presetting the compression library's + // dictionary. + template <typename TBlocklike> + Status GetDataBlockFromCache(const Slice& cache_key, Cache* block_cache, + Cache* block_cache_compressed, + const ReadOptions& read_options, + CachableEntry<TBlocklike>* block, + const UncompressionDict& uncompression_dict, + BlockType block_type, const bool wait, + GetContext* get_context) const; + + // Put a maybe compressed block to the corresponding block caches. + // This method will perform decompression against block_contents if needed + // and then populate the block caches. + // On success, Status::OK will be returned; also @block will be populated with + // uncompressed block and its cache handle. + // + // Allocated memory managed by block_contents will be transferred to + // PutDataBlockToCache(). After the call, the object will be invalid. + // @param uncompression_dict Data for presetting the compression library's + // dictionary. + template <typename TBlocklike> + Status PutDataBlockToCache(const Slice& cache_key, Cache* block_cache, + Cache* block_cache_compressed, + CachableEntry<TBlocklike>* cached_block, + BlockContents&& block_contents, + CompressionType block_comp_type, + const UncompressionDict& uncompression_dict, + MemoryAllocator* memory_allocator, + BlockType block_type, + GetContext* get_context) const; + + // Calls (*handle_result)(arg, ...) repeatedly, starting with the entry found + // after a call to Seek(key), until handle_result returns false. + // May not make such a call if filter policy says that key is not present. + friend class TableCache; + friend class BlockBasedTableBuilder; + + // Create a index reader based on the index type stored in the table. + // Optionally, user can pass a preloaded meta_index_iter for the index that + // need to access extra meta blocks for index construction. This parameter + // helps avoid re-reading meta index block if caller already created one. + Status CreateIndexReader(const ReadOptions& ro, + FilePrefetchBuffer* prefetch_buffer, + InternalIterator* preloaded_meta_index_iter, + bool use_cache, bool prefetch, bool pin, + BlockCacheLookupContext* lookup_context, + std::unique_ptr<IndexReader>* index_reader); + + bool FullFilterKeyMayMatch(FilterBlockReader* filter, const Slice& user_key, + const bool no_io, + const SliceTransform* prefix_extractor, + GetContext* get_context, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) const; + + void FullFilterKeysMayMatch(FilterBlockReader* filter, MultiGetRange* range, + const bool no_io, + const SliceTransform* prefix_extractor, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) const; + + // If force_direct_prefetch is true, always prefetching to RocksDB + // buffer, rather than calling RandomAccessFile::Prefetch(). + static Status PrefetchTail( + const ReadOptions& ro, RandomAccessFileReader* file, uint64_t file_size, + bool force_direct_prefetch, TailPrefetchStats* tail_prefetch_stats, + const bool prefetch_all, const bool preload_all, + std::unique_ptr<FilePrefetchBuffer>* prefetch_buffer); + Status ReadMetaIndexBlock(const ReadOptions& ro, + FilePrefetchBuffer* prefetch_buffer, + std::unique_ptr<Block>* metaindex_block, + std::unique_ptr<InternalIterator>* iter); + Status ReadPropertiesBlock(const ReadOptions& ro, + FilePrefetchBuffer* prefetch_buffer, + InternalIterator* meta_iter, + const SequenceNumber largest_seqno); + Status ReadRangeDelBlock(const ReadOptions& ro, + FilePrefetchBuffer* prefetch_buffer, + InternalIterator* meta_iter, + const InternalKeyComparator& internal_comparator, + BlockCacheLookupContext* lookup_context); + Status PrefetchIndexAndFilterBlocks( + const ReadOptions& ro, FilePrefetchBuffer* prefetch_buffer, + InternalIterator* meta_iter, BlockBasedTable* new_table, + bool prefetch_all, const BlockBasedTableOptions& table_options, + const int level, size_t file_size, size_t max_file_size_for_l0_meta_pin, + BlockCacheLookupContext* lookup_context); + + static BlockType GetBlockTypeForMetaBlockByName(const Slice& meta_block_name); + + Status VerifyChecksumInMetaBlocks(InternalIteratorBase<Slice>* index_iter); + Status VerifyChecksumInBlocks(const ReadOptions& read_options, + InternalIteratorBase<IndexValue>* index_iter); + + // Create the filter from the filter block. + std::unique_ptr<FilterBlockReader> CreateFilterBlockReader( + const ReadOptions& ro, FilePrefetchBuffer* prefetch_buffer, + bool use_cache, bool prefetch, bool pin, + BlockCacheLookupContext* lookup_context); + + // Size of all data blocks, maybe approximate + uint64_t GetApproximateDataSize(); + + // Given an iterator return its offset in data block section of file. + uint64_t ApproximateDataOffsetOf( + const InternalIteratorBase<IndexValue>& index_iter, + uint64_t data_size) const; + + // Helper functions for DumpTable() + Status DumpIndexBlock(std::ostream& out_stream); + Status DumpDataBlocks(std::ostream& out_stream); + void DumpKeyValue(const Slice& key, const Slice& value, + std::ostream& out_stream); + + // Returns false if prefix_extractor exists and is compatible with that used + // in building the table file, otherwise true. + bool PrefixExtractorChanged(const SliceTransform* prefix_extractor) const; + + // A cumulative data block file read in MultiGet lower than this size will + // use a stack buffer + static constexpr size_t kMultiGetReadStackBufSize = 8192; + + friend class PartitionedFilterBlockReader; + friend class PartitionedFilterBlockTest; + friend class DBBasicTest_MultiGetIOBufferOverrun_Test; +}; + +// Maintaining state of a two-level iteration on a partitioned index structure. +class BlockBasedTable::PartitionedIndexIteratorState + : public TwoLevelIteratorState { + public: + PartitionedIndexIteratorState( + const BlockBasedTable* table, + UnorderedMap<uint64_t, CachableEntry<Block>>* block_map); + InternalIteratorBase<IndexValue>* NewSecondaryIterator( + const BlockHandle& index_value) override; + + private: + // Don't own table_ + const BlockBasedTable* table_; + UnorderedMap<uint64_t, CachableEntry<Block>>* block_map_; +}; + +// Stores all the properties associated with a BlockBasedTable. +// These are immutable. +struct BlockBasedTable::Rep { + Rep(const ImmutableOptions& _ioptions, const EnvOptions& _env_options, + const BlockBasedTableOptions& _table_opt, + const InternalKeyComparator& _internal_comparator, bool skip_filters, + uint64_t _file_size, int _level, const bool _immortal_table) + : 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), + index_type(BlockBasedTableOptions::IndexType::kBinarySearch), + whole_key_filtering(_table_opt.whole_key_filtering), + prefix_filtering(true), + global_seqno(kDisableGlobalSequenceNumber), + file_size(_file_size), + level(_level), + immortal_table(_immortal_table) {} + ~Rep() { status.PermitUncheckedError(); } + const ImmutableOptions& ioptions; + const EnvOptions& env_options; + const BlockBasedTableOptions table_options; + const FilterPolicy* const filter_policy; + const InternalKeyComparator& internal_comparator; + Status status; + std::unique_ptr<RandomAccessFileReader> file; + OffsetableCacheKey base_cache_key; + PersistentCacheOptions persistent_cache_options; + + // Footer contains the fixed table information + Footer footer; + + std::unique_ptr<IndexReader> index_reader; + std::unique_ptr<FilterBlockReader> filter; + std::unique_ptr<UncompressionDictReader> uncompression_dict_reader; + + enum class FilterType { + kNoFilter, + kFullFilter, + kPartitionedFilter, + }; + FilterType filter_type; + BlockHandle filter_handle; + BlockHandle compression_dict_handle; + + std::shared_ptr<const TableProperties> table_properties; + BlockBasedTableOptions::IndexType index_type; + bool whole_key_filtering; + bool prefix_filtering; + std::shared_ptr<const SliceTransform> table_prefix_extractor; + + std::shared_ptr<FragmentedRangeTombstoneList> fragmented_range_dels; + + // 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; + + // Size of the table file on disk + uint64_t file_size; + + // the level when the table is opened, could potentially change when trivial + // move is involved + int level; + + // If false, blocks in this file are definitely all uncompressed. Knowing this + // before reading individual blocks enables certain optimizations. + bool blocks_maybe_compressed = true; + + // If true, data blocks in this file are definitely ZSTD compressed. If false + // they might not be. When false we skip creating a ZSTD digested + // uncompression dictionary. Even if we get a false negative, things should + // still work, just not as quickly. + bool blocks_definitely_zstd_compressed = false; + + // These describe how index is encoded. + bool index_has_first_key = false; + bool index_key_includes_seq = true; + bool index_value_is_full = true; + + const bool immortal_table; + + std::unique_ptr<CacheReservationManager::CacheReservationHandle> + table_reader_cache_res_handle = nullptr; + + SequenceNumber get_global_seqno(BlockType block_type) const { + return (block_type == BlockType::kFilterPartitionIndex || + block_type == BlockType::kCompressionDictionary) + ? kDisableGlobalSequenceNumber + : global_seqno; + } + + uint64_t cf_id_for_tracing() const { + return table_properties + ? table_properties->column_family_id + : ROCKSDB_NAMESPACE::TablePropertiesCollectorFactory::Context:: + kUnknownColumnFamily; + } + + Slice cf_name_for_tracing() const { + return table_properties ? table_properties->column_family_name + : BlockCacheTraceHelper::kUnknownColumnFamilyName; + } + + uint32_t level_for_tracing() const { return level >= 0 ? level : UINT32_MAX; } + + uint64_t sst_number_for_tracing() const { + return file ? TableFileNameToNumber(file->file_name()) : UINT64_MAX; + } + void CreateFilePrefetchBuffer( + size_t readahead_size, size_t max_readahead_size, + std::unique_ptr<FilePrefetchBuffer>* fpb, bool implicit_auto_readahead, + uint64_t num_file_reads, + uint64_t num_file_reads_for_auto_readahead) const { + fpb->reset(new FilePrefetchBuffer( + readahead_size, max_readahead_size, + !ioptions.allow_mmap_reads /* enable */, false /* track_min_offset */, + implicit_auto_readahead, num_file_reads, + num_file_reads_for_auto_readahead, ioptions.fs.get(), ioptions.clock, + ioptions.stats)); + } + + void CreateFilePrefetchBufferIfNotExists( + size_t readahead_size, size_t max_readahead_size, + std::unique_ptr<FilePrefetchBuffer>* fpb, bool implicit_auto_readahead, + uint64_t num_file_reads, + uint64_t num_file_reads_for_auto_readahead) const { + if (!(*fpb)) { + CreateFilePrefetchBuffer(readahead_size, max_readahead_size, fpb, + implicit_auto_readahead, num_file_reads, + num_file_reads_for_auto_readahead); + } + } + + std::size_t ApproximateMemoryUsage() const { + std::size_t usage = 0; +#ifdef ROCKSDB_MALLOC_USABLE_SIZE + usage += malloc_usable_size(const_cast<BlockBasedTable::Rep*>(this)); +#else + usage += sizeof(*this); +#endif // ROCKSDB_MALLOC_USABLE_SIZE + return usage; + } +}; + +// This is an adapter class for `WritableFile` to be used for `std::ostream`. +// The adapter wraps a `WritableFile`, which can be passed to a `std::ostream` +// constructor for storing streaming data. +// Note: +// * This adapter doesn't provide any buffering, each write is forwarded to +// `WritableFile->Append()` directly. +// * For a failed write, the user needs to check the status by `ostream.good()` +class WritableFileStringStreamAdapter : public std::stringbuf { + public: + explicit WritableFileStringStreamAdapter(WritableFile* writable_file) + : file_(writable_file) {} + + // Override overflow() to handle `sputc()`. There are cases that will not go + // through `xsputn()` e.g. `std::endl` or an unsigned long long is written by + // `os.put()` directly and will call `sputc()` By internal implementation: + // int_type __CLR_OR_THIS_CALL sputc(_Elem _Ch) { // put a character + // return 0 < _Pnavail() ? _Traits::to_int_type(*_Pninc() = _Ch) : + // overflow(_Traits::to_int_type(_Ch)); + // } + // As we explicitly disabled buffering (_Pnavail() is always 0), every write, + // not captured by xsputn(), becomes an overflow here. + int overflow(int ch = EOF) override { + if (ch != EOF) { + Status s = file_->Append(Slice((char*)&ch, 1)); + if (s.ok()) { + return ch; + } + } + return EOF; + } + + std::streamsize xsputn(char const* p, std::streamsize n) override { + Status s = file_->Append(Slice(p, n)); + if (!s.ok()) { + return 0; + } + return n; + } + + private: + WritableFile* file_; +}; + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/block_based_table_reader_impl.h b/src/rocksdb/table/block_based/block_based_table_reader_impl.h new file mode 100644 index 000000000..1f6f5f223 --- /dev/null +++ b/src/rocksdb/table/block_based/block_based_table_reader_impl.h @@ -0,0 +1,171 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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 "table/block_based/block_based_table_reader.h" +#include "table/block_based/reader_common.h" + +// The file contains some member functions of BlockBasedTable that +// cannot be implemented in block_based_table_reader.cc because +// it's called by other files (e.g. block_based_iterator.h) and +// are templates. + +namespace ROCKSDB_NAMESPACE { +// Convert an index iterator value (i.e., an encoded BlockHandle) +// into an iterator over the contents of the corresponding block. +// If input_iter is null, new a iterator +// If input_iter is not null, update this iter and return it +template <typename TBlockIter> +TBlockIter* BlockBasedTable::NewDataBlockIterator( + const ReadOptions& ro, const BlockHandle& handle, TBlockIter* input_iter, + BlockType block_type, GetContext* get_context, + BlockCacheLookupContext* lookup_context, + FilePrefetchBuffer* prefetch_buffer, bool for_compaction, bool async_read, + Status& s) const { + PERF_TIMER_GUARD(new_table_block_iter_nanos); + + TBlockIter* iter = input_iter != nullptr ? input_iter : new TBlockIter; + if (!s.ok()) { + iter->Invalidate(s); + return iter; + } + + CachableEntry<Block> block; + if (rep_->uncompression_dict_reader && block_type == BlockType::kData) { + CachableEntry<UncompressionDict> uncompression_dict; + const bool no_io = (ro.read_tier == kBlockCacheTier); + s = rep_->uncompression_dict_reader->GetOrReadUncompressionDictionary( + prefetch_buffer, no_io, ro.verify_checksums, get_context, + lookup_context, &uncompression_dict); + if (!s.ok()) { + iter->Invalidate(s); + return iter; + } + const UncompressionDict& dict = uncompression_dict.GetValue() + ? *uncompression_dict.GetValue() + : UncompressionDict::GetEmptyDict(); + s = RetrieveBlock(prefetch_buffer, ro, handle, dict, &block, block_type, + get_context, lookup_context, for_compaction, + /* use_cache */ true, /* wait_for_cache */ true, + async_read); + } else { + s = RetrieveBlock( + prefetch_buffer, ro, handle, UncompressionDict::GetEmptyDict(), &block, + block_type, get_context, lookup_context, for_compaction, + /* use_cache */ true, /* wait_for_cache */ true, async_read); + } + + if (s.IsTryAgain() && async_read) { + return iter; + } + + if (!s.ok()) { + assert(block.IsEmpty()); + iter->Invalidate(s); + return iter; + } + + assert(block.GetValue() != nullptr); + + // Block contents are pinned and it is still pinned after the iterator + // is destroyed as long as cleanup functions are moved to another object, + // when: + // 1. block cache handle is set to be released in cleanup function, or + // 2. it's pointing to immortal source. If own_bytes is true then we are + // not reading data from the original source, whether immortal or not. + // Otherwise, the block is pinned iff the source is immortal. + const bool block_contents_pinned = + block.IsCached() || + (!block.GetValue()->own_bytes() && rep_->immortal_table); + iter = InitBlockIterator<TBlockIter>(rep_, block.GetValue(), block_type, iter, + block_contents_pinned); + + if (!block.IsCached()) { + if (!ro.fill_cache) { + Cache* const block_cache = rep_->table_options.block_cache.get(); + if (block_cache) { + // insert a dummy record to block cache to track the memory usage + Cache::Handle* cache_handle = nullptr; + CacheKey key = CacheKey::CreateUniqueForCacheLifetime(block_cache); + s = block_cache->Insert(key.AsSlice(), nullptr, + block.GetValue()->ApproximateMemoryUsage(), + nullptr, &cache_handle); + + if (s.ok()) { + assert(cache_handle != nullptr); + iter->RegisterCleanup(&ForceReleaseCachedEntry, block_cache, + cache_handle); + } + } + } + } else { + iter->SetCacheHandle(block.GetCacheHandle()); + } + + block.TransferTo(iter); + + return iter; +} + +// Convert an uncompressed data block (i.e CachableEntry<Block>) +// into an iterator over the contents of the corresponding block. +// If input_iter is null, new a iterator +// If input_iter is not null, update this iter and return it +template <typename TBlockIter> +TBlockIter* BlockBasedTable::NewDataBlockIterator(const ReadOptions& ro, + CachableEntry<Block>& block, + TBlockIter* input_iter, + Status s) const { + PERF_TIMER_GUARD(new_table_block_iter_nanos); + + TBlockIter* iter = input_iter != nullptr ? input_iter : new TBlockIter; + if (!s.ok()) { + iter->Invalidate(s); + return iter; + } + + assert(block.GetValue() != nullptr); + // Block contents are pinned and it is still pinned after the iterator + // is destroyed as long as cleanup functions are moved to another object, + // when: + // 1. block cache handle is set to be released in cleanup function, or + // 2. it's pointing to immortal source. If own_bytes is true then we are + // not reading data from the original source, whether immortal or not. + // Otherwise, the block is pinned iff the source is immortal. + const bool block_contents_pinned = + block.IsCached() || + (!block.GetValue()->own_bytes() && rep_->immortal_table); + iter = InitBlockIterator<TBlockIter>(rep_, block.GetValue(), BlockType::kData, + iter, block_contents_pinned); + + if (!block.IsCached()) { + if (!ro.fill_cache) { + Cache* const block_cache = rep_->table_options.block_cache.get(); + if (block_cache) { + // insert a dummy record to block cache to track the memory usage + Cache::Handle* cache_handle = nullptr; + CacheKey key = CacheKey::CreateUniqueForCacheLifetime(block_cache); + s = block_cache->Insert(key.AsSlice(), nullptr, + block.GetValue()->ApproximateMemoryUsage(), + nullptr, &cache_handle); + + if (s.ok()) { + assert(cache_handle != nullptr); + iter->RegisterCleanup(&ForceReleaseCachedEntry, block_cache, + cache_handle); + } + } + } + } else { + iter->SetCacheHandle(block.GetCacheHandle()); + } + + block.TransferTo(iter); + return iter; +} +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/block_based_table_reader_sync_and_async.h b/src/rocksdb/table/block_based/block_based_table_reader_sync_and_async.h new file mode 100644 index 000000000..8c7547a2a --- /dev/null +++ b/src/rocksdb/table/block_based/block_based_table_reader_sync_and_async.h @@ -0,0 +1,760 @@ +// Copyright (c) Meta Platforms, Inc. and affiliates. +// +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#include "util/async_file_reader.h" +#include "util/coro_utils.h" + +#if defined(WITHOUT_COROUTINES) || \ + (defined(USE_COROUTINES) && defined(WITH_COROUTINES)) + +namespace ROCKSDB_NAMESPACE { + +// This function reads multiple data blocks from disk using Env::MultiRead() +// and optionally inserts them into the block cache. It uses the scratch +// buffer provided by the caller, which is contiguous. If scratch is a nullptr +// it allocates a separate buffer for each block. Typically, if the blocks +// need to be uncompressed and there is no compressed block cache, callers +// can allocate a temporary scratch buffer in order to minimize memory +// allocations. +// If options.fill_cache is true, it inserts the blocks into cache. If its +// false and scratch is non-null and the blocks are uncompressed, it copies +// the buffers to heap. In any case, the CachableEntry<Block> returned will +// own the data bytes. +// If compression is enabled and also there is no compressed block cache, +// the adjacent blocks are read out in one IO (combined read) +// batch - A MultiGetRange with only those keys with unique data blocks not +// found in cache +// handles - A vector of block handles. Some of them me be NULL handles +// scratch - An optional contiguous buffer to read compressed blocks into +DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::RetrieveMultipleBlocks) +(const ReadOptions& options, const MultiGetRange* batch, + const autovector<BlockHandle, MultiGetContext::MAX_BATCH_SIZE>* handles, + autovector<Status, MultiGetContext::MAX_BATCH_SIZE>* statuses, + autovector<CachableEntry<Block>, MultiGetContext::MAX_BATCH_SIZE>* results, + char* scratch, const UncompressionDict& uncompression_dict) const { + RandomAccessFileReader* file = rep_->file.get(); + const Footer& footer = rep_->footer; + const ImmutableOptions& ioptions = rep_->ioptions; + size_t read_amp_bytes_per_bit = rep_->table_options.read_amp_bytes_per_bit; + MemoryAllocator* memory_allocator = GetMemoryAllocator(rep_->table_options); + + if (ioptions.allow_mmap_reads) { + size_t idx_in_batch = 0; + for (auto mget_iter = batch->begin(); mget_iter != batch->end(); + ++mget_iter, ++idx_in_batch) { + BlockCacheLookupContext lookup_data_block_context( + TableReaderCaller::kUserMultiGet); + const BlockHandle& handle = (*handles)[idx_in_batch]; + if (handle.IsNull()) { + continue; + } + + (*statuses)[idx_in_batch] = + RetrieveBlock(nullptr, options, handle, uncompression_dict, + &(*results)[idx_in_batch], BlockType::kData, + mget_iter->get_context, &lookup_data_block_context, + /* for_compaction */ false, /* use_cache */ true, + /* wait_for_cache */ true, /* async_read */ false); + } + CO_RETURN; + } + + // In direct IO mode, blocks share the direct io buffer. + // Otherwise, blocks share the scratch buffer. + const bool use_shared_buffer = file->use_direct_io() || scratch != nullptr; + + autovector<FSReadRequest, MultiGetContext::MAX_BATCH_SIZE> read_reqs; + size_t buf_offset = 0; + size_t idx_in_batch = 0; + + uint64_t prev_offset = 0; + size_t prev_len = 0; + autovector<size_t, MultiGetContext::MAX_BATCH_SIZE> req_idx_for_block; + autovector<size_t, MultiGetContext::MAX_BATCH_SIZE> req_offset_for_block; + for (auto mget_iter = batch->begin(); mget_iter != batch->end(); + ++mget_iter, ++idx_in_batch) { + const BlockHandle& handle = (*handles)[idx_in_batch]; + if (handle.IsNull()) { + continue; + } + + size_t prev_end = static_cast<size_t>(prev_offset) + prev_len; + + // If current block is adjacent to the previous one, at the same time, + // compression is enabled and there is no compressed cache, we combine + // the two block read as one. + // We don't combine block reads here in direct IO mode, because when doing + // direct IO read, the block requests will be realigned and merged when + // necessary. + if (use_shared_buffer && !file->use_direct_io() && + prev_end == handle.offset()) { + req_offset_for_block.emplace_back(prev_len); + prev_len += BlockSizeWithTrailer(handle); + } else { + // No compression or current block and previous one is not adjacent: + // Step 1, create a new request for previous blocks + if (prev_len != 0) { + FSReadRequest req; + req.offset = prev_offset; + req.len = prev_len; + if (file->use_direct_io()) { + req.scratch = nullptr; + } else if (use_shared_buffer) { + req.scratch = scratch + buf_offset; + buf_offset += req.len; + } else { + req.scratch = new char[req.len]; + } + read_reqs.emplace_back(req); + } + + // Step 2, remeber the previous block info + prev_offset = handle.offset(); + prev_len = BlockSizeWithTrailer(handle); + req_offset_for_block.emplace_back(0); + } + req_idx_for_block.emplace_back(read_reqs.size()); + + PERF_COUNTER_ADD(block_read_count, 1); + PERF_COUNTER_ADD(block_read_byte, BlockSizeWithTrailer(handle)); + } + // Handle the last block and process the pending last request + if (prev_len != 0) { + FSReadRequest req; + req.offset = prev_offset; + req.len = prev_len; + if (file->use_direct_io()) { + req.scratch = nullptr; + } else if (use_shared_buffer) { + req.scratch = scratch + buf_offset; + } else { + req.scratch = new char[req.len]; + } + read_reqs.emplace_back(req); + } + + AlignedBuf direct_io_buf; + { + IOOptions opts; + IOStatus s = file->PrepareIOOptions(options, opts); + if (s.ok()) { +#if defined(WITH_COROUTINES) + if (file->use_direct_io()) { +#endif // WITH_COROUTINES + s = file->MultiRead(opts, &read_reqs[0], read_reqs.size(), + &direct_io_buf, options.rate_limiter_priority); +#if defined(WITH_COROUTINES) + } else { + co_await batch->context()->reader().MultiReadAsync( + file, opts, &read_reqs[0], read_reqs.size(), &direct_io_buf); + } +#endif // WITH_COROUTINES + } + if (!s.ok()) { + // Discard all the results in this batch if there is any time out + // or overall MultiRead error + for (FSReadRequest& req : read_reqs) { + req.status = s; + } + } + } + + idx_in_batch = 0; + size_t valid_batch_idx = 0; + for (auto mget_iter = batch->begin(); mget_iter != batch->end(); + ++mget_iter, ++idx_in_batch) { + const BlockHandle& handle = (*handles)[idx_in_batch]; + + if (handle.IsNull()) { + continue; + } + + assert(valid_batch_idx < req_idx_for_block.size()); + assert(valid_batch_idx < req_offset_for_block.size()); + assert(req_idx_for_block[valid_batch_idx] < read_reqs.size()); + size_t& req_idx = req_idx_for_block[valid_batch_idx]; + size_t& req_offset = req_offset_for_block[valid_batch_idx]; + valid_batch_idx++; + FSReadRequest& req = read_reqs[req_idx]; + Status s = req.status; + if (s.ok()) { + if ((req.result.size() != req.len) || + (req_offset + BlockSizeWithTrailer(handle) > req.result.size())) { + s = Status::Corruption("truncated block read from " + + rep_->file->file_name() + " offset " + + std::to_string(handle.offset()) + ", expected " + + std::to_string(req.len) + " bytes, got " + + std::to_string(req.result.size())); + } + } + + BlockContents serialized_block; + if (s.ok()) { + if (!use_shared_buffer) { + // We allocated a buffer for this block. Give ownership of it to + // BlockContents so it can free the memory + assert(req.result.data() == req.scratch); + assert(req.result.size() == BlockSizeWithTrailer(handle)); + assert(req_offset == 0); + serialized_block = + BlockContents(std::unique_ptr<char[]>(req.scratch), handle.size()); + } else { + // We used the scratch buffer or direct io buffer + // which are shared by the blocks. + // serialized_block does not have the ownership. + serialized_block = + BlockContents(Slice(req.result.data() + req_offset, handle.size())); + } +#ifndef NDEBUG + serialized_block.has_trailer = true; +#endif + + if (options.verify_checksums) { + PERF_TIMER_GUARD(block_checksum_time); + const char* data = req.result.data(); + // Since the scratch might be shared, the offset of the data block in + // the buffer might not be 0. req.result.data() only point to the + // begin address of each read request, we need to add the offset + // in each read request. Checksum is stored in the block trailer, + // beyond the payload size. + s = VerifyBlockChecksum(footer.checksum_type(), data + req_offset, + handle.size(), rep_->file->file_name(), + handle.offset()); + TEST_SYNC_POINT_CALLBACK("RetrieveMultipleBlocks:VerifyChecksum", &s); + } + } else if (!use_shared_buffer) { + // Free the allocated scratch buffer. + delete[] req.scratch; + } + + if (s.ok()) { + // When the blocks share the same underlying buffer (scratch or direct io + // buffer), we may need to manually copy the block into heap if the + // serialized block has to be inserted into a cache. That falls into the + // following cases - + // 1. serialized block is not compressed, it needs to be inserted into + // the uncompressed block cache if there is one + // 2. If the serialized block is compressed, it needs to be inserted + // into the compressed block cache if there is one + // + // In all other cases, the serialized block is either uncompressed into a + // heap buffer or there is no cache at all. + CompressionType compression_type = + GetBlockCompressionType(serialized_block); + if (use_shared_buffer && (compression_type == kNoCompression || + (compression_type != kNoCompression && + rep_->table_options.block_cache_compressed))) { + Slice serialized = + Slice(req.result.data() + req_offset, BlockSizeWithTrailer(handle)); + serialized_block = BlockContents( + CopyBufferToHeap(GetMemoryAllocator(rep_->table_options), + serialized), + handle.size()); +#ifndef NDEBUG + serialized_block.has_trailer = true; +#endif + } + } + + if (s.ok()) { + if (options.fill_cache) { + BlockCacheLookupContext lookup_data_block_context( + TableReaderCaller::kUserMultiGet); + CachableEntry<Block>* block_entry = &(*results)[idx_in_batch]; + // MaybeReadBlockAndLoadToCache will insert into the block caches if + // necessary. Since we're passing the serialized block contents, it + // will avoid looking up the block cache + s = MaybeReadBlockAndLoadToCache( + nullptr, options, handle, uncompression_dict, /*wait=*/true, + /*for_compaction=*/false, block_entry, BlockType::kData, + mget_iter->get_context, &lookup_data_block_context, + &serialized_block, /*async_read=*/false); + + // block_entry value could be null if no block cache is present, i.e + // BlockBasedTableOptions::no_block_cache is true and no compressed + // block cache is configured. In that case, fall + // through and set up the block explicitly + if (block_entry->GetValue() != nullptr) { + s.PermitUncheckedError(); + continue; + } + } + + CompressionType compression_type = + GetBlockCompressionType(serialized_block); + BlockContents contents; + if (compression_type != kNoCompression) { + UncompressionContext context(compression_type); + UncompressionInfo info(context, uncompression_dict, compression_type); + s = UncompressSerializedBlock( + info, req.result.data() + req_offset, handle.size(), &contents, + footer.format_version(), rep_->ioptions, memory_allocator); + } else { + // There are two cases here: + // 1) caller uses the shared buffer (scratch or direct io buffer); + // 2) we use the requst buffer. + // If scratch buffer or direct io buffer is used, we ensure that + // all serialized blocks are copyed to the heap as single blocks. If + // scratch buffer is not used, we also have no combined read, so the + // serialized block can be used directly. + contents = std::move(serialized_block); + } + if (s.ok()) { + (*results)[idx_in_batch].SetOwnedValue(std::make_unique<Block>( + std::move(contents), read_amp_bytes_per_bit, ioptions.stats)); + } + } + (*statuses)[idx_in_batch] = s; + } +} + +using MultiGetRange = MultiGetContext::Range; +DEFINE_SYNC_AND_ASYNC(void, BlockBasedTable::MultiGet) +(const ReadOptions& read_options, const MultiGetRange* mget_range, + const SliceTransform* prefix_extractor, bool skip_filters) { + if (mget_range->empty()) { + // Caller should ensure non-empty (performance bug) + assert(false); + CO_RETURN; // Nothing to do + } + + FilterBlockReader* const filter = + !skip_filters ? rep_->filter.get() : nullptr; + MultiGetRange sst_file_range(*mget_range, mget_range->begin(), + mget_range->end()); + + // First check the full filter + // If full filter not useful, Then go into each block + const bool no_io = read_options.read_tier == kBlockCacheTier; + uint64_t tracing_mget_id = BlockCacheTraceHelper::kReservedGetId; + if (sst_file_range.begin()->get_context) { + tracing_mget_id = sst_file_range.begin()->get_context->get_tracing_get_id(); + } + BlockCacheLookupContext lookup_context{ + TableReaderCaller::kUserMultiGet, tracing_mget_id, + /*_get_from_user_specified_snapshot=*/read_options.snapshot != nullptr}; + FullFilterKeysMayMatch(filter, &sst_file_range, no_io, prefix_extractor, + &lookup_context, read_options.rate_limiter_priority); + + if (!sst_file_range.empty()) { + IndexBlockIter iiter_on_stack; + // if prefix_extractor found in block differs from options, disable + // BlockPrefixIndex. Only do this check when index_type is kHashSearch. + bool need_upper_bound_check = false; + if (rep_->index_type == BlockBasedTableOptions::kHashSearch) { + need_upper_bound_check = PrefixExtractorChanged(prefix_extractor); + } + auto iiter = + NewIndexIterator(read_options, need_upper_bound_check, &iiter_on_stack, + sst_file_range.begin()->get_context, &lookup_context); + std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr; + if (iiter != &iiter_on_stack) { + iiter_unique_ptr.reset(iiter); + } + + uint64_t prev_offset = std::numeric_limits<uint64_t>::max(); + autovector<BlockHandle, MultiGetContext::MAX_BATCH_SIZE> block_handles; + autovector<CachableEntry<Block>, MultiGetContext::MAX_BATCH_SIZE> results; + autovector<Status, MultiGetContext::MAX_BATCH_SIZE> statuses; + MultiGetContext::Mask reused_mask = 0; + char stack_buf[kMultiGetReadStackBufSize]; + std::unique_ptr<char[]> block_buf; + { + MultiGetRange data_block_range(sst_file_range, sst_file_range.begin(), + sst_file_range.end()); + std::vector<Cache::Handle*> cache_handles; + bool wait_for_cache_results = false; + + CachableEntry<UncompressionDict> uncompression_dict; + Status uncompression_dict_status; + uncompression_dict_status.PermitUncheckedError(); + bool uncompression_dict_inited = false; + size_t total_len = 0; + ReadOptions ro = read_options; + ro.read_tier = kBlockCacheTier; + + for (auto miter = data_block_range.begin(); + miter != data_block_range.end(); ++miter) { + const Slice& key = miter->ikey; + iiter->Seek(miter->ikey); + + IndexValue v; + if (iiter->Valid()) { + v = iiter->value(); + } + if (!iiter->Valid() || + (!v.first_internal_key.empty() && !skip_filters && + UserComparatorWrapper(rep_->internal_comparator.user_comparator()) + .CompareWithoutTimestamp( + ExtractUserKey(key), + ExtractUserKey(v.first_internal_key)) < 0)) { + // The requested key falls between highest key in previous block and + // lowest key in current block. + if (!iiter->status().IsNotFound()) { + *(miter->s) = iiter->status(); + } + data_block_range.SkipKey(miter); + sst_file_range.SkipKey(miter); + continue; + } + + if (!uncompression_dict_inited && rep_->uncompression_dict_reader) { + uncompression_dict_status = + rep_->uncompression_dict_reader->GetOrReadUncompressionDictionary( + nullptr /* prefetch_buffer */, no_io, + read_options.verify_checksums, + sst_file_range.begin()->get_context, &lookup_context, + &uncompression_dict); + uncompression_dict_inited = true; + } + + if (!uncompression_dict_status.ok()) { + assert(!uncompression_dict_status.IsNotFound()); + *(miter->s) = uncompression_dict_status; + data_block_range.SkipKey(miter); + sst_file_range.SkipKey(miter); + continue; + } + + statuses.emplace_back(); + results.emplace_back(); + if (v.handle.offset() == prev_offset) { + // This key can reuse the previous block (later on). + // Mark previous as "reused" + reused_mask |= MultiGetContext::Mask{1} << (block_handles.size() - 1); + // Use null handle to indicate this one reuses same block as + // previous. + block_handles.emplace_back(BlockHandle::NullBlockHandle()); + continue; + } + // Lookup the cache for the given data block referenced by an index + // iterator value (i.e BlockHandle). If it exists in the cache, + // initialize block to the contents of the data block. + prev_offset = v.handle.offset(); + BlockHandle handle = v.handle; + BlockCacheLookupContext lookup_data_block_context( + TableReaderCaller::kUserMultiGet); + const UncompressionDict& dict = uncompression_dict.GetValue() + ? *uncompression_dict.GetValue() + : UncompressionDict::GetEmptyDict(); + Status s = RetrieveBlock( + nullptr, ro, handle, dict, &(results.back()), BlockType::kData, + miter->get_context, &lookup_data_block_context, + /* for_compaction */ false, /* use_cache */ true, + /* wait_for_cache */ false, /* async_read */ false); + if (s.IsIncomplete()) { + s = Status::OK(); + } + if (s.ok() && !results.back().IsEmpty()) { + // Since we have a valid handle, check the value. If its nullptr, + // it means the cache is waiting for the final result and we're + // supposed to call WaitAll() to wait for the result. + if (results.back().GetValue() != nullptr) { + // Found it in the cache. Add NULL handle to indicate there is + // nothing to read from disk. + if (results.back().GetCacheHandle()) { + results.back().UpdateCachedValue(); + } + block_handles.emplace_back(BlockHandle::NullBlockHandle()); + } else { + // We have to wait for the cache lookup to finish in the + // background, and then we may have to read the block from disk + // anyway + assert(results.back().GetCacheHandle()); + wait_for_cache_results = true; + block_handles.emplace_back(handle); + cache_handles.emplace_back(results.back().GetCacheHandle()); + } + } else { + block_handles.emplace_back(handle); + total_len += BlockSizeWithTrailer(handle); + } + } + + if (wait_for_cache_results) { + Cache* block_cache = rep_->table_options.block_cache.get(); + block_cache->WaitAll(cache_handles); + for (size_t i = 0; i < block_handles.size(); ++i) { + // If this block was a success or failure or not needed because + // the corresponding key is in the same block as a prior key, skip + if (block_handles[i] == BlockHandle::NullBlockHandle() || + results[i].IsEmpty()) { + continue; + } + results[i].UpdateCachedValue(); + void* val = results[i].GetValue(); + Cache::Handle* handle = results[i].GetCacheHandle(); + // GetContext for any key will do, as the stats will be aggregated + // anyway + GetContext* get_context = sst_file_range.begin()->get_context; + if (!val) { + // The async cache lookup failed - could be due to an error + // or a false positive. We need to read the data block from + // the SST file + results[i].Reset(); + total_len += BlockSizeWithTrailer(block_handles[i]); + UpdateCacheMissMetrics(BlockType::kData, get_context); + } else { + block_handles[i] = BlockHandle::NullBlockHandle(); + UpdateCacheHitMetrics(BlockType::kData, get_context, + block_cache->GetUsage(handle)); + } + } + } + + if (total_len) { + char* scratch = nullptr; + const UncompressionDict& dict = uncompression_dict.GetValue() + ? *uncompression_dict.GetValue() + : UncompressionDict::GetEmptyDict(); + assert(uncompression_dict_inited || !rep_->uncompression_dict_reader); + assert(uncompression_dict_status.ok()); + // If using direct IO, then scratch is not used, so keep it nullptr. + // If the blocks need to be uncompressed and we don't need the + // compressed blocks, then we can use a contiguous block of + // memory to read in all the blocks as it will be temporary + // storage + // 1. If blocks are compressed and compressed block cache is there, + // alloc heap bufs + // 2. If blocks are uncompressed, alloc heap bufs + // 3. If blocks are compressed and no compressed block cache, use + // stack buf + if (!rep_->file->use_direct_io() && + rep_->table_options.block_cache_compressed == nullptr && + rep_->blocks_maybe_compressed) { + if (total_len <= kMultiGetReadStackBufSize) { + scratch = stack_buf; + } else { + scratch = new char[total_len]; + block_buf.reset(scratch); + } + } + CO_AWAIT(RetrieveMultipleBlocks) + (read_options, &data_block_range, &block_handles, &statuses, &results, + scratch, dict); + if (sst_file_range.begin()->get_context) { + ++(sst_file_range.begin() + ->get_context->get_context_stats_.num_sst_read); + } + } + } + + DataBlockIter first_biter; + DataBlockIter next_biter; + size_t idx_in_batch = 0; + SharedCleanablePtr shared_cleanable; + for (auto miter = sst_file_range.begin(); miter != sst_file_range.end(); + ++miter) { + Status s; + GetContext* get_context = miter->get_context; + const Slice& key = miter->ikey; + bool matched = false; // if such user key matched a key in SST + bool done = false; + bool first_block = true; + do { + DataBlockIter* biter = nullptr; + bool reusing_prev_block; + bool later_reused; + uint64_t referenced_data_size = 0; + bool does_referenced_key_exist = false; + BlockCacheLookupContext lookup_data_block_context( + TableReaderCaller::kUserMultiGet, tracing_mget_id, + /*_get_from_user_specified_snapshot=*/read_options.snapshot != + nullptr); + if (first_block) { + if (!block_handles[idx_in_batch].IsNull() || + !results[idx_in_batch].IsEmpty()) { + first_biter.Invalidate(Status::OK()); + NewDataBlockIterator<DataBlockIter>( + read_options, results[idx_in_batch], &first_biter, + statuses[idx_in_batch]); + reusing_prev_block = false; + } else { + // If handler is null and result is empty, then the status is never + // set, which should be the initial value: ok(). + assert(statuses[idx_in_batch].ok()); + reusing_prev_block = true; + } + biter = &first_biter; + later_reused = + (reused_mask & (MultiGetContext::Mask{1} << idx_in_batch)) != 0; + idx_in_batch++; + } else { + IndexValue v = iiter->value(); + if (!v.first_internal_key.empty() && !skip_filters && + UserComparatorWrapper(rep_->internal_comparator.user_comparator()) + .CompareWithoutTimestamp( + ExtractUserKey(key), + ExtractUserKey(v.first_internal_key)) < 0) { + // The requested key falls between highest key in previous block and + // lowest key in current block. + break; + } + + next_biter.Invalidate(Status::OK()); + Status tmp_s; + NewDataBlockIterator<DataBlockIter>( + read_options, iiter->value().handle, &next_biter, + BlockType::kData, get_context, &lookup_data_block_context, + /* prefetch_buffer= */ nullptr, /* for_compaction = */ false, + /*async_read = */ false, tmp_s); + biter = &next_biter; + reusing_prev_block = false; + later_reused = false; + } + + if (read_options.read_tier == kBlockCacheTier && + biter->status().IsIncomplete()) { + // couldn't get block from block_cache + // Update Saver.state to Found because we are only looking for + // whether we can guarantee the key is not there when "no_io" is set + get_context->MarkKeyMayExist(); + break; + } + if (!biter->status().ok()) { + s = biter->status(); + break; + } + + // Reusing blocks complicates pinning/Cleanable, because the cache + // entry referenced by biter can only be released once all returned + // pinned values are released. This code previously did an extra + // block_cache Ref for each reuse, but that unnecessarily increases + // block cache contention. Instead we can use a variant of shared_ptr + // to release in block cache only once. + // + // Although the biter loop below might SaveValue multiple times for + // merges, just one value_pinner suffices, as MultiGet will merge + // the operands before returning to the API user. + Cleanable* value_pinner; + if (biter->IsValuePinned()) { + if (reusing_prev_block) { + // Note that we don't yet know if the MultiGet results will need + // to pin this block, so we might wrap a block for sharing and + // still end up with 1 (or 0) pinning ref. Not ideal but OK. + // + // Here we avoid adding redundant cleanups if we didn't end up + // delegating the cleanup from last time around. + if (!biter->HasCleanups()) { + assert(shared_cleanable.get()); + if (later_reused) { + shared_cleanable.RegisterCopyWith(biter); + } else { + shared_cleanable.MoveAsCleanupTo(biter); + } + } + } else if (later_reused) { + assert(biter->HasCleanups()); + // Make the existing cleanups on `biter` sharable: + shared_cleanable.Allocate(); + // Move existing `biter` cleanup(s) to `shared_cleanable` + biter->DelegateCleanupsTo(&*shared_cleanable); + // Reference `shared_cleanable` as new cleanup for `biter` + shared_cleanable.RegisterCopyWith(biter); + } + assert(biter->HasCleanups()); + value_pinner = biter; + } else { + value_pinner = nullptr; + } + + bool may_exist = biter->SeekForGet(key); + if (!may_exist) { + // HashSeek cannot find the key this block and the the iter is not + // the end of the block, i.e. cannot be in the following blocks + // either. In this case, the seek_key cannot be found, so we break + // from the top level for-loop. + break; + } + + // Call the *saver function on each entry/block until it returns false + for (; biter->Valid(); biter->Next()) { + ParsedInternalKey parsed_key; + Status pik_status = ParseInternalKey( + biter->key(), &parsed_key, false /* log_err_key */); // TODO + if (!pik_status.ok()) { + s = pik_status; + } + if (!get_context->SaveValue(parsed_key, biter->value(), &matched, + value_pinner)) { + if (get_context->State() == GetContext::GetState::kFound) { + does_referenced_key_exist = true; + referenced_data_size = + biter->key().size() + biter->value().size(); + } + done = true; + break; + } + s = biter->status(); + } + // Write the block cache access. + // XXX: There appear to be 'break' statements above that bypass this + // writing of the block cache trace record + if (block_cache_tracer_ && block_cache_tracer_->is_tracing_enabled() && + !reusing_prev_block) { + // Avoid making copy of block_key, cf_name, and referenced_key when + // constructing the access record. + Slice referenced_key; + if (does_referenced_key_exist) { + referenced_key = biter->key(); + } else { + referenced_key = key; + } + BlockCacheTraceRecord access_record( + rep_->ioptions.clock->NowMicros(), + /*_block_key=*/"", lookup_data_block_context.block_type, + lookup_data_block_context.block_size, rep_->cf_id_for_tracing(), + /*_cf_name=*/"", rep_->level_for_tracing(), + rep_->sst_number_for_tracing(), lookup_data_block_context.caller, + lookup_data_block_context.is_cache_hit, + lookup_data_block_context.no_insert, + lookup_data_block_context.get_id, + lookup_data_block_context.get_from_user_specified_snapshot, + /*_referenced_key=*/"", referenced_data_size, + lookup_data_block_context.num_keys_in_block, + does_referenced_key_exist); + // TODO: Should handle status here? + block_cache_tracer_ + ->WriteBlockAccess(access_record, + lookup_data_block_context.block_key, + rep_->cf_name_for_tracing(), referenced_key) + .PermitUncheckedError(); + } + s = biter->status(); + if (done) { + // Avoid the extra Next which is expensive in two-level indexes + break; + } + if (first_block) { + iiter->Seek(key); + if (!iiter->Valid()) { + break; + } + } + first_block = false; + iiter->Next(); + } while (iiter->Valid()); + + if (matched && filter != nullptr) { + RecordTick(rep_->ioptions.stats, BLOOM_FILTER_FULL_TRUE_POSITIVE); + PERF_COUNTER_BY_LEVEL_ADD(bloom_filter_full_true_positive, 1, + rep_->level); + } + if (s.ok() && !iiter->status().IsNotFound()) { + s = iiter->status(); + } + *(miter->s) = s; + } +#ifdef ROCKSDB_ASSERT_STATUS_CHECKED + // Not sure why we need to do it. Should investigate more. + for (auto& st : statuses) { + st.PermitUncheckedError(); + } +#endif // ROCKSDB_ASSERT_STATUS_CHECKED + } +} +} // namespace ROCKSDB_NAMESPACE +#endif diff --git a/src/rocksdb/table/block_based/block_based_table_reader_test.cc b/src/rocksdb/table/block_based/block_based_table_reader_test.cc new file mode 100644 index 000000000..c5a615dfc --- /dev/null +++ b/src/rocksdb/table/block_based/block_based_table_reader_test.cc @@ -0,0 +1,572 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#include "table/block_based/block_based_table_reader.h" + +#include <cmath> +#include <memory> +#include <string> + +#include "cache/cache_reservation_manager.h" +#include "db/db_test_util.h" +#include "db/table_properties_collector.h" +#include "file/file_util.h" +#include "options/options_helper.h" +#include "port/port.h" +#include "port/stack_trace.h" +#include "rocksdb/compression_type.h" +#include "rocksdb/db.h" +#include "rocksdb/file_system.h" +#include "table/block_based/block_based_table_builder.h" +#include "table/block_based/block_based_table_factory.h" +#include "table/block_based/partitioned_index_iterator.h" +#include "table/format.h" +#include "test_util/testharness.h" +#include "test_util/testutil.h" +#include "util/random.h" + +namespace ROCKSDB_NAMESPACE { + +class BlockBasedTableReaderBaseTest : public testing::Test { + protected: + // Prepare key-value pairs to occupy multiple blocks. + // Each value is 256B, every 16 pairs constitute 1 block. + // If mixed_with_human_readable_string_value == true, + // then adjacent blocks contain values with different compression + // complexity: human readable strings are easier to compress than random + // strings. + static std::map<std::string, std::string> GenerateKVMap( + int num_block = 100, + bool mixed_with_human_readable_string_value = false) { + std::map<std::string, std::string> kv; + + Random rnd(101); + uint32_t key = 0; + for (int block = 0; block < num_block; block++) { + for (int i = 0; i < 16; i++) { + char k[9] = {0}; + // Internal key is constructed directly from this key, + // and internal key size is required to be >= 8 bytes, + // so use %08u as the format string. + sprintf(k, "%08u", key); + std::string v; + if (mixed_with_human_readable_string_value) { + v = (block % 2) ? rnd.HumanReadableString(256) + : rnd.RandomString(256); + } else { + v = rnd.RandomString(256); + } + kv[std::string(k)] = v; + key++; + } + } + return kv; + } + + void SetUp() override { + SetupSyncPointsToMockDirectIO(); + test_dir_ = test::PerThreadDBPath("block_based_table_reader_test"); + env_ = Env::Default(); + fs_ = FileSystem::Default(); + ASSERT_OK(fs_->CreateDir(test_dir_, IOOptions(), nullptr)); + ConfigureTableFactory(); + } + + virtual void ConfigureTableFactory() = 0; + + void TearDown() override { EXPECT_OK(DestroyDir(env_, test_dir_)); } + + // Creates a table with the specificied key value pairs (kv). + void CreateTable(const std::string& table_name, + const CompressionType& compression_type, + const std::map<std::string, std::string>& kv) { + std::unique_ptr<WritableFileWriter> writer; + NewFileWriter(table_name, &writer); + + // Create table builder. + ImmutableOptions ioptions(options_); + InternalKeyComparator comparator(options_.comparator); + ColumnFamilyOptions cf_options; + MutableCFOptions moptions(cf_options); + IntTblPropCollectorFactories factories; + std::unique_ptr<TableBuilder> table_builder( + options_.table_factory->NewTableBuilder( + TableBuilderOptions(ioptions, moptions, comparator, &factories, + compression_type, CompressionOptions(), + 0 /* column_family_id */, + kDefaultColumnFamilyName, -1 /* level */), + writer.get())); + + // Build table. + for (auto it = kv.begin(); it != kv.end(); it++) { + std::string k = ToInternalKey(it->first); + std::string v = it->second; + table_builder->Add(k, v); + } + ASSERT_OK(table_builder->Finish()); + } + + void NewBlockBasedTableReader(const FileOptions& foptions, + const ImmutableOptions& ioptions, + const InternalKeyComparator& comparator, + const std::string& table_name, + std::unique_ptr<BlockBasedTable>* table, + bool prefetch_index_and_filter_in_cache = true, + Status* status = nullptr) { + const MutableCFOptions moptions(options_); + TableReaderOptions table_reader_options = TableReaderOptions( + ioptions, moptions.prefix_extractor, EnvOptions(), comparator); + + std::unique_ptr<RandomAccessFileReader> file; + NewFileReader(table_name, foptions, &file); + + uint64_t file_size = 0; + ASSERT_OK(env_->GetFileSize(Path(table_name), &file_size)); + + std::unique_ptr<TableReader> general_table; + Status s = options_.table_factory->NewTableReader( + ReadOptions(), table_reader_options, std::move(file), file_size, + &general_table, prefetch_index_and_filter_in_cache); + + if (s.ok()) { + table->reset(reinterpret_cast<BlockBasedTable*>(general_table.release())); + } + + if (status) { + *status = s; + } + } + + std::string Path(const std::string& fname) { return test_dir_ + "/" + fname; } + + std::string test_dir_; + Env* env_; + std::shared_ptr<FileSystem> fs_; + Options options_; + + private: + void WriteToFile(const std::string& content, const std::string& filename) { + std::unique_ptr<FSWritableFile> f; + ASSERT_OK(fs_->NewWritableFile(Path(filename), FileOptions(), &f, nullptr)); + ASSERT_OK(f->Append(content, IOOptions(), nullptr)); + ASSERT_OK(f->Close(IOOptions(), nullptr)); + } + + void NewFileWriter(const std::string& filename, + std::unique_ptr<WritableFileWriter>* writer) { + std::string path = Path(filename); + EnvOptions env_options; + FileOptions foptions; + std::unique_ptr<FSWritableFile> file; + ASSERT_OK(fs_->NewWritableFile(path, foptions, &file, nullptr)); + writer->reset(new WritableFileWriter(std::move(file), path, env_options)); + } + + void NewFileReader(const std::string& filename, const FileOptions& opt, + std::unique_ptr<RandomAccessFileReader>* reader) { + std::string path = Path(filename); + std::unique_ptr<FSRandomAccessFile> f; + ASSERT_OK(fs_->NewRandomAccessFile(path, opt, &f, nullptr)); + reader->reset(new RandomAccessFileReader(std::move(f), path, + env_->GetSystemClock().get())); + } + + std::string ToInternalKey(const std::string& key) { + InternalKey internal_key(key, 0, ValueType::kTypeValue); + return internal_key.Encode().ToString(); + } +}; + +class BlockBasedTableReaderTest + : public BlockBasedTableReaderBaseTest, + public testing::WithParamInterface<std::tuple< + CompressionType, bool, BlockBasedTableOptions::IndexType, bool>> { + protected: + void SetUp() override { + compression_type_ = std::get<0>(GetParam()); + use_direct_reads_ = std::get<1>(GetParam()); + BlockBasedTableReaderBaseTest::SetUp(); + } + + void ConfigureTableFactory() override { + BlockBasedTableOptions opts; + opts.index_type = std::get<2>(GetParam()); + opts.no_block_cache = std::get<3>(GetParam()); + options_.table_factory.reset( + static_cast<BlockBasedTableFactory*>(NewBlockBasedTableFactory(opts))); + } + + CompressionType compression_type_; + bool use_direct_reads_; +}; + +// Tests MultiGet in both direct IO and non-direct IO mode. +// The keys should be in cache after MultiGet. +TEST_P(BlockBasedTableReaderTest, MultiGet) { + std::map<std::string, std::string> kv = + BlockBasedTableReaderBaseTest::GenerateKVMap( + 100 /* num_block */, + true /* mixed_with_human_readable_string_value */); + + // Prepare keys, values, and statuses for MultiGet. + autovector<Slice, MultiGetContext::MAX_BATCH_SIZE> keys; + autovector<PinnableSlice, MultiGetContext::MAX_BATCH_SIZE> values; + autovector<Status, MultiGetContext::MAX_BATCH_SIZE> statuses; + { + const int step = + static_cast<int>(kv.size()) / MultiGetContext::MAX_BATCH_SIZE; + auto it = kv.begin(); + for (int i = 0; i < MultiGetContext::MAX_BATCH_SIZE; i++) { + keys.emplace_back(it->first); + values.emplace_back(); + statuses.emplace_back(); + std::advance(it, step); + } + } + + std::string table_name = + "BlockBasedTableReaderTest" + CompressionTypeToString(compression_type_); + CreateTable(table_name, compression_type_, kv); + + std::unique_ptr<BlockBasedTable> table; + Options options; + ImmutableOptions ioptions(options); + FileOptions foptions; + foptions.use_direct_reads = use_direct_reads_; + InternalKeyComparator comparator(options.comparator); + NewBlockBasedTableReader(foptions, ioptions, comparator, table_name, &table); + + // Ensure that keys are not in cache before MultiGet. + for (auto& key : keys) { + ASSERT_FALSE(table->TEST_KeyInCache(ReadOptions(), key)); + } + + // Prepare MultiGetContext. + autovector<GetContext, MultiGetContext::MAX_BATCH_SIZE> get_context; + autovector<KeyContext, MultiGetContext::MAX_BATCH_SIZE> key_context; + autovector<KeyContext*, MultiGetContext::MAX_BATCH_SIZE> sorted_keys; + for (size_t i = 0; i < keys.size(); ++i) { + get_context.emplace_back(BytewiseComparator(), nullptr, nullptr, nullptr, + GetContext::kNotFound, keys[i], &values[i], + nullptr, nullptr, nullptr, nullptr, + true /* do_merge */, nullptr, nullptr, nullptr, + nullptr, nullptr, nullptr); + key_context.emplace_back(nullptr, keys[i], &values[i], nullptr, + &statuses.back()); + key_context.back().get_context = &get_context.back(); + } + for (auto& key_ctx : key_context) { + sorted_keys.emplace_back(&key_ctx); + } + MultiGetContext ctx(&sorted_keys, 0, sorted_keys.size(), 0, ReadOptions(), + fs_.get(), nullptr); + + // Execute MultiGet. + MultiGetContext::Range range = ctx.GetMultiGetRange(); + PerfContext* perf_ctx = get_perf_context(); + perf_ctx->Reset(); + table->MultiGet(ReadOptions(), &range, nullptr); + + ASSERT_GE(perf_ctx->block_read_count - perf_ctx->index_block_read_count - + perf_ctx->filter_block_read_count - + perf_ctx->compression_dict_block_read_count, + 1); + ASSERT_GE(perf_ctx->block_read_byte, 1); + + for (const Status& status : statuses) { + ASSERT_OK(status); + } + // Check that keys are in cache after MultiGet. + for (size_t i = 0; i < keys.size(); i++) { + ASSERT_TRUE(table->TEST_KeyInCache(ReadOptions(), keys[i])); + ASSERT_EQ(values[i].ToString(), kv[keys[i].ToString()]); + } +} + +class ChargeTableReaderTest + : public BlockBasedTableReaderBaseTest, + public testing::WithParamInterface< + CacheEntryRoleOptions::Decision /* charge_table_reader_mem */> { + protected: + static std::size_t CalculateMaxTableReaderNumBeforeCacheFull( + std::size_t cache_capacity, std::size_t approx_table_reader_mem) { + // To make calculation easier for testing + assert(cache_capacity % CacheReservationManagerImpl< + CacheEntryRole::kBlockBasedTableReader>:: + GetDummyEntrySize() == + 0 && + cache_capacity >= 2 * CacheReservationManagerImpl< + CacheEntryRole::kBlockBasedTableReader>:: + GetDummyEntrySize()); + + // We need to subtract 1 for max_num_dummy_entry to account for dummy + // entries' overhead, assumed the overhead is no greater than 1 dummy entry + // size + std::size_t max_num_dummy_entry = + (size_t)std::floor(( + 1.0 * cache_capacity / + CacheReservationManagerImpl< + CacheEntryRole::kBlockBasedTableReader>::GetDummyEntrySize())) - + 1; + std::size_t cache_capacity_rounded_to_dummy_entry_multiples = + max_num_dummy_entry * + CacheReservationManagerImpl< + CacheEntryRole::kBlockBasedTableReader>::GetDummyEntrySize(); + std::size_t max_table_reader_num_capped = static_cast<std::size_t>( + std::floor(1.0 * cache_capacity_rounded_to_dummy_entry_multiples / + approx_table_reader_mem)); + + return max_table_reader_num_capped; + } + + void SetUp() override { + // To cache and re-use the same kv map and compression type in the test + // suite for elimiating variance caused by these two factors + kv_ = BlockBasedTableReaderBaseTest::GenerateKVMap(); + compression_type_ = CompressionType::kNoCompression; + + table_reader_charge_tracking_cache_ = std::make_shared< + TargetCacheChargeTrackingCache< + CacheEntryRole::kBlockBasedTableReader>>((NewLRUCache( + 4 * CacheReservationManagerImpl< + CacheEntryRole::kBlockBasedTableReader>::GetDummyEntrySize(), + 0 /* num_shard_bits */, true /* strict_capacity_limit */))); + + // To ApproximateTableReaderMem() without being affected by + // the feature of charging its memory, we turn off the feature + charge_table_reader_ = CacheEntryRoleOptions::Decision::kDisabled; + BlockBasedTableReaderBaseTest::SetUp(); + approx_table_reader_mem_ = ApproximateTableReaderMem(); + + // Now we condtionally turn on the feature to test + charge_table_reader_ = GetParam(); + ConfigureTableFactory(); + } + + void ConfigureTableFactory() override { + BlockBasedTableOptions table_options; + table_options.cache_usage_options.options_overrides.insert( + {CacheEntryRole::kBlockBasedTableReader, + {/*.charged = */ charge_table_reader_}}); + table_options.block_cache = table_reader_charge_tracking_cache_; + + table_options.cache_index_and_filter_blocks = false; + table_options.filter_policy.reset(NewBloomFilterPolicy(10, false)); + table_options.partition_filters = true; + table_options.index_type = BlockBasedTableOptions::kTwoLevelIndexSearch; + + options_.table_factory.reset(NewBlockBasedTableFactory(table_options)); + } + + CacheEntryRoleOptions::Decision charge_table_reader_; + std::shared_ptr< + TargetCacheChargeTrackingCache<CacheEntryRole::kBlockBasedTableReader>> + table_reader_charge_tracking_cache_; + std::size_t approx_table_reader_mem_; + std::map<std::string, std::string> kv_; + CompressionType compression_type_; + + private: + std::size_t ApproximateTableReaderMem() { + std::size_t approx_table_reader_mem = 0; + + std::string table_name = "table_for_approx_table_reader_mem"; + CreateTable(table_name, compression_type_, kv_); + + std::unique_ptr<BlockBasedTable> table; + Status s; + NewBlockBasedTableReader( + FileOptions(), ImmutableOptions(options_), + InternalKeyComparator(options_.comparator), table_name, &table, + false /* prefetch_index_and_filter_in_cache */, &s); + assert(s.ok()); + + approx_table_reader_mem = table->ApproximateMemoryUsage(); + assert(approx_table_reader_mem > 0); + return approx_table_reader_mem; + } +}; + +INSTANTIATE_TEST_CASE_P( + ChargeTableReaderTest, ChargeTableReaderTest, + ::testing::Values(CacheEntryRoleOptions::Decision::kEnabled, + CacheEntryRoleOptions::Decision::kDisabled)); + +TEST_P(ChargeTableReaderTest, Basic) { + const std::size_t max_table_reader_num_capped = + ChargeTableReaderTest::CalculateMaxTableReaderNumBeforeCacheFull( + table_reader_charge_tracking_cache_->GetCapacity(), + approx_table_reader_mem_); + + // Acceptable estimtation errors coming from + // 1. overstimate max_table_reader_num_capped due to # dummy entries is high + // and results in metadata charge overhead greater than 1 dummy entry size + // (violating our assumption in calculating max_table_reader_num_capped) + // 2. overestimate/underestimate max_table_reader_num_capped due to the gap + // between ApproximateTableReaderMem() and actual table reader mem + std::size_t max_table_reader_num_capped_upper_bound = + (std::size_t)(max_table_reader_num_capped * 1.05); + std::size_t max_table_reader_num_capped_lower_bound = + (std::size_t)(max_table_reader_num_capped * 0.95); + std::size_t max_table_reader_num_uncapped = + (std::size_t)(max_table_reader_num_capped * 1.1); + ASSERT_GT(max_table_reader_num_uncapped, + max_table_reader_num_capped_upper_bound) + << "We need `max_table_reader_num_uncapped` > " + "`max_table_reader_num_capped_upper_bound` to differentiate cases " + "between " + "charge_table_reader_ == kDisabled and == kEnabled)"; + + Status s = Status::OK(); + std::size_t opened_table_reader_num = 0; + std::string table_name; + std::vector<std::unique_ptr<BlockBasedTable>> tables; + // Keep creating BlockBasedTableReader till hiting the memory limit based on + // cache capacity and creation fails (when charge_table_reader_ == + // kEnabled) or reaching a specfied big number of table readers (when + // charge_table_reader_ == kDisabled) + while (s.ok() && opened_table_reader_num < max_table_reader_num_uncapped) { + table_name = "table_" + std::to_string(opened_table_reader_num); + CreateTable(table_name, compression_type_, kv_); + tables.push_back(std::unique_ptr<BlockBasedTable>()); + NewBlockBasedTableReader( + FileOptions(), ImmutableOptions(options_), + InternalKeyComparator(options_.comparator), table_name, &tables.back(), + false /* prefetch_index_and_filter_in_cache */, &s); + if (s.ok()) { + ++opened_table_reader_num; + } + } + + if (charge_table_reader_ == CacheEntryRoleOptions::Decision::kEnabled) { + EXPECT_TRUE(s.IsMemoryLimit()) << "s: " << s.ToString(); + EXPECT_TRUE(s.ToString().find( + kCacheEntryRoleToCamelString[static_cast<std::uint32_t>( + CacheEntryRole::kBlockBasedTableReader)]) != + std::string::npos); + EXPECT_TRUE(s.ToString().find("memory limit based on cache capacity") != + std::string::npos); + + EXPECT_GE(opened_table_reader_num, max_table_reader_num_capped_lower_bound); + EXPECT_LE(opened_table_reader_num, max_table_reader_num_capped_upper_bound); + + std::size_t updated_max_table_reader_num_capped = + ChargeTableReaderTest::CalculateMaxTableReaderNumBeforeCacheFull( + table_reader_charge_tracking_cache_->GetCapacity() / 2, + approx_table_reader_mem_); + + // Keep deleting BlockBasedTableReader to lower down memory usage from the + // memory limit to make the next creation succeeds + while (opened_table_reader_num >= updated_max_table_reader_num_capped) { + tables.pop_back(); + --opened_table_reader_num; + } + table_name = "table_for_successful_table_reader_open"; + CreateTable(table_name, compression_type_, kv_); + tables.push_back(std::unique_ptr<BlockBasedTable>()); + NewBlockBasedTableReader( + FileOptions(), ImmutableOptions(options_), + InternalKeyComparator(options_.comparator), table_name, &tables.back(), + false /* prefetch_index_and_filter_in_cache */, &s); + EXPECT_TRUE(s.ok()) << s.ToString(); + + tables.clear(); + EXPECT_EQ(table_reader_charge_tracking_cache_->GetCacheCharge(), 0); + } else { + EXPECT_TRUE(s.ok() && + opened_table_reader_num == max_table_reader_num_uncapped) + << "s: " << s.ToString() << " opened_table_reader_num: " + << std::to_string(opened_table_reader_num); + EXPECT_EQ(table_reader_charge_tracking_cache_->GetCacheCharge(), 0); + } +} + +class BlockBasedTableReaderTestVerifyChecksum + : public BlockBasedTableReaderTest { + public: + BlockBasedTableReaderTestVerifyChecksum() : BlockBasedTableReaderTest() {} +}; + +TEST_P(BlockBasedTableReaderTestVerifyChecksum, ChecksumMismatch) { + std::map<std::string, std::string> kv = + BlockBasedTableReaderBaseTest::GenerateKVMap(800 /* num_block */); + + std::string table_name = + "BlockBasedTableReaderTest" + CompressionTypeToString(compression_type_); + CreateTable(table_name, compression_type_, kv); + + std::unique_ptr<BlockBasedTable> table; + Options options; + ImmutableOptions ioptions(options); + FileOptions foptions; + foptions.use_direct_reads = use_direct_reads_; + InternalKeyComparator comparator(options.comparator); + NewBlockBasedTableReader(foptions, ioptions, comparator, table_name, &table); + + // Use the top level iterator to find the offset/size of the first + // 2nd level index block and corrupt the block + IndexBlockIter iiter_on_stack; + BlockCacheLookupContext context{TableReaderCaller::kUserVerifyChecksum}; + InternalIteratorBase<IndexValue>* iiter = table->NewIndexIterator( + ReadOptions(), /*disable_prefix_seek=*/false, &iiter_on_stack, + /*get_context=*/nullptr, &context); + std::unique_ptr<InternalIteratorBase<IndexValue>> iiter_unique_ptr; + if (iiter != &iiter_on_stack) { + iiter_unique_ptr = std::unique_ptr<InternalIteratorBase<IndexValue>>(iiter); + } + ASSERT_OK(iiter->status()); + iiter->SeekToFirst(); + BlockHandle handle = static_cast<PartitionedIndexIterator*>(iiter) + ->index_iter_->value() + .handle; + table.reset(); + + // Corrupt the block pointed to by handle + ASSERT_OK(test::CorruptFile(options.env, Path(table_name), + static_cast<int>(handle.offset()), 128)); + + NewBlockBasedTableReader(foptions, ioptions, comparator, table_name, &table); + Status s = table->VerifyChecksum(ReadOptions(), + TableReaderCaller::kUserVerifyChecksum); + ASSERT_EQ(s.code(), Status::kCorruption); +} + +// Param 1: compression type +// Param 2: whether to use direct reads +// Param 3: Block Based Table Index type +// Param 4: BBTO no_block_cache option +#ifdef ROCKSDB_LITE +// Skip direct I/O tests in lite mode since direct I/O is unsupported. +INSTANTIATE_TEST_CASE_P( + MultiGet, BlockBasedTableReaderTest, + ::testing::Combine( + ::testing::ValuesIn(GetSupportedCompressions()), + ::testing::Values(false), + ::testing::Values(BlockBasedTableOptions::IndexType::kBinarySearch), + ::testing::Values(false))); +#else // ROCKSDB_LITE +INSTANTIATE_TEST_CASE_P( + MultiGet, BlockBasedTableReaderTest, + ::testing::Combine( + ::testing::ValuesIn(GetSupportedCompressions()), ::testing::Bool(), + ::testing::Values(BlockBasedTableOptions::IndexType::kBinarySearch), + ::testing::Values(false))); +#endif // ROCKSDB_LITE +INSTANTIATE_TEST_CASE_P( + VerifyChecksum, BlockBasedTableReaderTestVerifyChecksum, + ::testing::Combine( + ::testing::ValuesIn(GetSupportedCompressions()), + ::testing::Values(false), + ::testing::Values( + BlockBasedTableOptions::IndexType::kTwoLevelIndexSearch), + ::testing::Values(true))); + +} // namespace ROCKSDB_NAMESPACE + +int main(int argc, char** argv) { + ROCKSDB_NAMESPACE::port::InstallStackTraceHandler(); + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/src/rocksdb/table/block_based/block_builder.cc b/src/rocksdb/table/block_based/block_builder.cc new file mode 100644 index 000000000..92702b17d --- /dev/null +++ b/src/rocksdb/table/block_based/block_builder.cc @@ -0,0 +1,234 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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. +// +// BlockBuilder generates blocks where keys are prefix-compressed: +// +// When we store a key, we drop the prefix shared with the previous +// string. This helps reduce the space requirement significantly. +// Furthermore, once every K keys, we do not apply the prefix +// compression and store the entire key. We call this a "restart +// point". The tail end of the block stores the offsets of all of the +// restart points, and can be used to do a binary search when looking +// for a particular key. Values are stored as-is (without compression) +// immediately following the corresponding key. +// +// An entry for a particular key-value pair has the form: +// shared_bytes: varint32 +// unshared_bytes: varint32 +// value_length: varint32 +// key_delta: char[unshared_bytes] +// value: char[value_length] +// shared_bytes == 0 for restart points. +// +// The trailer of the block has the form: +// restarts: uint32[num_restarts] +// num_restarts: uint32 +// restarts[i] contains the offset within the block of the ith restart point. + +#include "table/block_based/block_builder.h" + +#include <assert.h> + +#include <algorithm> + +#include "db/dbformat.h" +#include "rocksdb/comparator.h" +#include "table/block_based/data_block_footer.h" +#include "util/coding.h" + +namespace ROCKSDB_NAMESPACE { + +BlockBuilder::BlockBuilder( + int block_restart_interval, bool use_delta_encoding, + bool use_value_delta_encoding, + BlockBasedTableOptions::DataBlockIndexType index_type, + double data_block_hash_table_util_ratio) + : block_restart_interval_(block_restart_interval), + use_delta_encoding_(use_delta_encoding), + use_value_delta_encoding_(use_value_delta_encoding), + restarts_(1, 0), // First restart point is at offset 0 + counter_(0), + finished_(false) { + switch (index_type) { + case BlockBasedTableOptions::kDataBlockBinarySearch: + break; + case BlockBasedTableOptions::kDataBlockBinaryAndHash: + data_block_hash_index_builder_.Initialize( + data_block_hash_table_util_ratio); + break; + default: + assert(0); + } + assert(block_restart_interval_ >= 1); + estimate_ = sizeof(uint32_t) + sizeof(uint32_t); +} + +void BlockBuilder::Reset() { + buffer_.clear(); + restarts_.resize(1); // First restart point is at offset 0 + assert(restarts_[0] == 0); + estimate_ = sizeof(uint32_t) + sizeof(uint32_t); + counter_ = 0; + finished_ = false; + last_key_.clear(); + if (data_block_hash_index_builder_.Valid()) { + data_block_hash_index_builder_.Reset(); + } +#ifndef NDEBUG + add_with_last_key_called_ = false; +#endif +} + +void BlockBuilder::SwapAndReset(std::string& buffer) { + std::swap(buffer_, buffer); + Reset(); +} + +size_t BlockBuilder::EstimateSizeAfterKV(const Slice& key, + const Slice& value) const { + size_t estimate = CurrentSizeEstimate(); + // Note: this is an imprecise estimate as it accounts for the whole key size + // instead of non-shared key size. + estimate += key.size(); + // In value delta encoding we estimate the value delta size as half the full + // value size since only the size field of block handle is encoded. + estimate += + !use_value_delta_encoding_ || (counter_ >= block_restart_interval_) + ? value.size() + : value.size() / 2; + + if (counter_ >= block_restart_interval_) { + estimate += sizeof(uint32_t); // a new restart entry. + } + + estimate += sizeof(int32_t); // varint for shared prefix length. + // Note: this is an imprecise estimate as we will have to encoded size, one + // for shared key and one for non-shared key. + estimate += VarintLength(key.size()); // varint for key length. + if (!use_value_delta_encoding_ || (counter_ >= block_restart_interval_)) { + estimate += VarintLength(value.size()); // varint for value length. + } + + return estimate; +} + +Slice BlockBuilder::Finish() { + // Append restart array + for (size_t i = 0; i < restarts_.size(); i++) { + PutFixed32(&buffer_, restarts_[i]); + } + + uint32_t num_restarts = static_cast<uint32_t>(restarts_.size()); + BlockBasedTableOptions::DataBlockIndexType index_type = + BlockBasedTableOptions::kDataBlockBinarySearch; + if (data_block_hash_index_builder_.Valid() && + CurrentSizeEstimate() <= kMaxBlockSizeSupportedByHashIndex) { + data_block_hash_index_builder_.Finish(buffer_); + index_type = BlockBasedTableOptions::kDataBlockBinaryAndHash; + } + + // footer is a packed format of data_block_index_type and num_restarts + uint32_t block_footer = PackIndexTypeAndNumRestarts(index_type, num_restarts); + + PutFixed32(&buffer_, block_footer); + finished_ = true; + return Slice(buffer_); +} + +void BlockBuilder::Add(const Slice& key, const Slice& value, + const Slice* const delta_value) { + // Ensure no unsafe mixing of Add and AddWithLastKey + assert(!add_with_last_key_called_); + + AddWithLastKeyImpl(key, value, last_key_, delta_value, buffer_.size()); + if (use_delta_encoding_) { + // Update state + // We used to just copy the changed data, but it appears to be + // faster to just copy the whole thing. + last_key_.assign(key.data(), key.size()); + } +} + +void BlockBuilder::AddWithLastKey(const Slice& key, const Slice& value, + const Slice& last_key_param, + const Slice* const delta_value) { + // Ensure no unsafe mixing of Add and AddWithLastKey + assert(last_key_.empty()); +#ifndef NDEBUG + add_with_last_key_called_ = false; +#endif + + // Here we make sure to use an empty `last_key` on first call after creation + // or Reset. This is more convenient for the caller and we can be more + // clever inside BlockBuilder. On this hot code path, we want to avoid + // conditional jumps like `buffer_.empty() ? ... : ...` so we can use a + // fast min operation instead, with an assertion to be sure our logic is + // sound. + size_t buffer_size = buffer_.size(); + size_t last_key_size = last_key_param.size(); + assert(buffer_size == 0 || buffer_size >= last_key_size); + + Slice last_key(last_key_param.data(), std::min(buffer_size, last_key_size)); + + AddWithLastKeyImpl(key, value, last_key, delta_value, buffer_size); +} + +inline void BlockBuilder::AddWithLastKeyImpl(const Slice& key, + const Slice& value, + const Slice& last_key, + const Slice* const delta_value, + size_t buffer_size) { + assert(!finished_); + assert(counter_ <= block_restart_interval_); + assert(!use_value_delta_encoding_ || delta_value); + size_t shared = 0; // number of bytes shared with prev key + if (counter_ >= block_restart_interval_) { + // Restart compression + restarts_.push_back(static_cast<uint32_t>(buffer_size)); + estimate_ += sizeof(uint32_t); + counter_ = 0; + } else if (use_delta_encoding_) { + // See how much sharing to do with previous string + shared = key.difference_offset(last_key); + } + + const size_t non_shared = key.size() - shared; + + if (use_value_delta_encoding_) { + // Add "<shared><non_shared>" to buffer_ + PutVarint32Varint32(&buffer_, static_cast<uint32_t>(shared), + static_cast<uint32_t>(non_shared)); + } else { + // Add "<shared><non_shared><value_size>" to buffer_ + PutVarint32Varint32Varint32(&buffer_, static_cast<uint32_t>(shared), + static_cast<uint32_t>(non_shared), + static_cast<uint32_t>(value.size())); + } + + // Add string delta to buffer_ followed by value + buffer_.append(key.data() + shared, non_shared); + // Use value delta encoding only when the key has shared bytes. This would + // simplify the decoding, where it can figure which decoding to use simply by + // looking at the shared bytes size. + if (shared != 0 && use_value_delta_encoding_) { + buffer_.append(delta_value->data(), delta_value->size()); + } else { + buffer_.append(value.data(), value.size()); + } + + if (data_block_hash_index_builder_.Valid()) { + data_block_hash_index_builder_.Add(ExtractUserKey(key), + restarts_.size() - 1); + } + + counter_++; + estimate_ += buffer_.size() - buffer_size; +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/block_builder.h b/src/rocksdb/table/block_based/block_builder.h new file mode 100644 index 000000000..5f68b449b --- /dev/null +++ b/src/rocksdb/table/block_based/block_builder.h @@ -0,0 +1,102 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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 <stdint.h> + +#include <vector> + +#include "rocksdb/slice.h" +#include "rocksdb/table.h" +#include "table/block_based/data_block_hash_index.h" + +namespace ROCKSDB_NAMESPACE { + +class BlockBuilder { + public: + BlockBuilder(const BlockBuilder&) = delete; + void operator=(const BlockBuilder&) = delete; + + explicit BlockBuilder(int block_restart_interval, + bool use_delta_encoding = true, + bool use_value_delta_encoding = false, + BlockBasedTableOptions::DataBlockIndexType index_type = + BlockBasedTableOptions::kDataBlockBinarySearch, + double data_block_hash_table_util_ratio = 0.75); + + // Reset the contents as if the BlockBuilder was just constructed. + void Reset(); + + // Swap the contents in BlockBuilder with buffer, then reset the BlockBuilder. + void SwapAndReset(std::string& buffer); + + // REQUIRES: Finish() has not been called since the last call to Reset(). + // REQUIRES: key is larger than any previously added key + // DO NOT mix with AddWithLastKey() between Resets. For efficiency, use + // AddWithLastKey() in contexts where previous added key is already known + // and delta encoding might be used. + void Add(const Slice& key, const Slice& value, + const Slice* const delta_value = nullptr); + + // A faster version of Add() if the previous key is already known for all + // Add()s. + // REQUIRES: Finish() has not been called since the last call to Reset(). + // REQUIRES: key is larger than any previously added key + // REQUIRES: if AddWithLastKey has been called since last Reset(), last_key + // is the key from most recent AddWithLastKey. (For convenience, last_key + // is ignored on first call after creation or Reset().) + // DO NOT mix with Add() between Resets. + void AddWithLastKey(const Slice& key, const Slice& value, + const Slice& last_key, + const Slice* const delta_value = nullptr); + + // Finish building the block and return a slice that refers to the + // block contents. The returned slice will remain valid for the + // lifetime of this builder or until Reset() is called. + Slice Finish(); + + // Returns an estimate of the current (uncompressed) size of the block + // we are building. + inline size_t CurrentSizeEstimate() const { + return estimate_ + (data_block_hash_index_builder_.Valid() + ? data_block_hash_index_builder_.EstimateSize() + : 0); + } + + // Returns an estimated block size after appending key and value. + size_t EstimateSizeAfterKV(const Slice& key, const Slice& value) const; + + // Return true iff no entries have been added since the last Reset() + bool empty() const { return buffer_.empty(); } + + private: + inline void AddWithLastKeyImpl(const Slice& key, const Slice& value, + const Slice& last_key, + const Slice* const delta_value, + size_t buffer_size); + + const int block_restart_interval_; + // TODO(myabandeh): put it into a separate IndexBlockBuilder + const bool use_delta_encoding_; + // Refer to BlockIter::DecodeCurrentValue for format of delta encoded values + const bool use_value_delta_encoding_; + + std::string buffer_; // Destination buffer + std::vector<uint32_t> restarts_; // Restart points + size_t estimate_; + int counter_; // Number of entries emitted since restart + bool finished_; // Has Finish() been called? + std::string last_key_; + DataBlockHashIndexBuilder data_block_hash_index_builder_; +#ifndef NDEBUG + bool add_with_last_key_called_ = false; +#endif +}; + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/block_like_traits.h b/src/rocksdb/table/block_based/block_like_traits.h new file mode 100644 index 000000000..d406dbb5d --- /dev/null +++ b/src/rocksdb/table/block_based/block_like_traits.h @@ -0,0 +1,182 @@ +// Copyright (c) Facebook, Inc. and its affiliates. All Rights Reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#pragma once + +#include "cache/cache_entry_roles.h" +#include "port/lang.h" +#include "table/block_based/block.h" +#include "table/block_based/block_type.h" +#include "table/block_based/parsed_full_filter_block.h" +#include "table/format.h" + +namespace ROCKSDB_NAMESPACE { + +template <typename TBlocklike> +class BlocklikeTraits; + +template <typename T, CacheEntryRole R> +Cache::CacheItemHelper* GetCacheItemHelperForRole(); + +template <typename TBlocklike> +Cache::CreateCallback GetCreateCallback(size_t read_amp_bytes_per_bit, + Statistics* statistics, bool using_zstd, + const FilterPolicy* filter_policy) { + return [read_amp_bytes_per_bit, statistics, using_zstd, filter_policy]( + const void* buf, size_t size, void** out_obj, + size_t* charge) -> Status { + assert(buf != nullptr); + std::unique_ptr<char[]> buf_data(new char[size]()); + memcpy(buf_data.get(), buf, size); + BlockContents bc = BlockContents(std::move(buf_data), size); + TBlocklike* ucd_ptr = BlocklikeTraits<TBlocklike>::Create( + std::move(bc), read_amp_bytes_per_bit, statistics, using_zstd, + filter_policy); + *out_obj = reinterpret_cast<void*>(ucd_ptr); + *charge = size; + return Status::OK(); + }; +} + +template <> +class BlocklikeTraits<ParsedFullFilterBlock> { + public: + static ParsedFullFilterBlock* Create(BlockContents&& contents, + size_t /* read_amp_bytes_per_bit */, + Statistics* /* statistics */, + bool /* using_zstd */, + const FilterPolicy* filter_policy) { + return new ParsedFullFilterBlock(filter_policy, std::move(contents)); + } + + static uint32_t GetNumRestarts(const ParsedFullFilterBlock& /* block */) { + return 0; + } + + static size_t SizeCallback(void* obj) { + assert(obj != nullptr); + ParsedFullFilterBlock* ptr = static_cast<ParsedFullFilterBlock*>(obj); + return ptr->GetBlockContentsData().size(); + } + + static Status SaveToCallback(void* from_obj, size_t from_offset, + size_t length, void* out) { + assert(from_obj != nullptr); + ParsedFullFilterBlock* ptr = static_cast<ParsedFullFilterBlock*>(from_obj); + const char* buf = ptr->GetBlockContentsData().data(); + assert(length == ptr->GetBlockContentsData().size()); + (void)from_offset; + memcpy(out, buf, length); + return Status::OK(); + } + + static Cache::CacheItemHelper* GetCacheItemHelper(BlockType block_type) { + (void)block_type; + assert(block_type == BlockType::kFilter); + return GetCacheItemHelperForRole<ParsedFullFilterBlock, + CacheEntryRole::kFilterBlock>(); + } +}; + +template <> +class BlocklikeTraits<Block> { + public: + static Block* Create(BlockContents&& contents, size_t read_amp_bytes_per_bit, + Statistics* statistics, bool /* using_zstd */, + const FilterPolicy* /* filter_policy */) { + return new Block(std::move(contents), read_amp_bytes_per_bit, statistics); + } + + static uint32_t GetNumRestarts(const Block& block) { + return block.NumRestarts(); + } + + static size_t SizeCallback(void* obj) { + assert(obj != nullptr); + Block* ptr = static_cast<Block*>(obj); + return ptr->size(); + } + + static Status SaveToCallback(void* from_obj, size_t from_offset, + size_t length, void* out) { + assert(from_obj != nullptr); + Block* ptr = static_cast<Block*>(from_obj); + const char* buf = ptr->data(); + assert(length == ptr->size()); + (void)from_offset; + memcpy(out, buf, length); + return Status::OK(); + } + + static Cache::CacheItemHelper* GetCacheItemHelper(BlockType block_type) { + switch (block_type) { + case BlockType::kData: + return GetCacheItemHelperForRole<Block, CacheEntryRole::kDataBlock>(); + case BlockType::kIndex: + return GetCacheItemHelperForRole<Block, CacheEntryRole::kIndexBlock>(); + case BlockType::kFilterPartitionIndex: + return GetCacheItemHelperForRole<Block, + CacheEntryRole::kFilterMetaBlock>(); + default: + // Not a recognized combination + assert(false); + FALLTHROUGH_INTENDED; + case BlockType::kRangeDeletion: + return GetCacheItemHelperForRole<Block, CacheEntryRole::kOtherBlock>(); + } + } +}; + +template <> +class BlocklikeTraits<UncompressionDict> { + public: + static UncompressionDict* Create(BlockContents&& contents, + size_t /* read_amp_bytes_per_bit */, + Statistics* /* statistics */, + bool using_zstd, + const FilterPolicy* /* filter_policy */) { + return new UncompressionDict(contents.data, std::move(contents.allocation), + using_zstd); + } + + static uint32_t GetNumRestarts(const UncompressionDict& /* dict */) { + return 0; + } + + static size_t SizeCallback(void* obj) { + assert(obj != nullptr); + UncompressionDict* ptr = static_cast<UncompressionDict*>(obj); + return ptr->slice_.size(); + } + + static Status SaveToCallback(void* from_obj, size_t from_offset, + size_t length, void* out) { + assert(from_obj != nullptr); + UncompressionDict* ptr = static_cast<UncompressionDict*>(from_obj); + const char* buf = ptr->slice_.data(); + assert(length == ptr->slice_.size()); + (void)from_offset; + memcpy(out, buf, length); + return Status::OK(); + } + + static Cache::CacheItemHelper* GetCacheItemHelper(BlockType block_type) { + (void)block_type; + assert(block_type == BlockType::kCompressionDictionary); + return GetCacheItemHelperForRole<UncompressionDict, + CacheEntryRole::kOtherBlock>(); + } +}; + +// Get an CacheItemHelper pointer for value type T and role R. +template <typename T, CacheEntryRole R> +Cache::CacheItemHelper* GetCacheItemHelperForRole() { + static Cache::CacheItemHelper cache_helper( + BlocklikeTraits<T>::SizeCallback, BlocklikeTraits<T>::SaveToCallback, + GetCacheEntryDeleterForRole<T, R>()); + return &cache_helper; +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/block_prefetcher.cc b/src/rocksdb/table/block_based/block_prefetcher.cc new file mode 100644 index 000000000..83ec2cb06 --- /dev/null +++ b/src/rocksdb/table/block_based/block_prefetcher.cc @@ -0,0 +1,120 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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/block_based/block_prefetcher.h" + +#include "rocksdb/file_system.h" +#include "table/block_based/block_based_table_reader.h" + +namespace ROCKSDB_NAMESPACE { +void BlockPrefetcher::PrefetchIfNeeded( + const BlockBasedTable::Rep* rep, const BlockHandle& handle, + const size_t readahead_size, bool is_for_compaction, + const bool no_sequential_checking, + const Env::IOPriority rate_limiter_priority) { + // num_file_reads is used by FilePrefetchBuffer only when + // implicit_auto_readahead is set. + if (is_for_compaction) { + rep->CreateFilePrefetchBufferIfNotExists( + compaction_readahead_size_, compaction_readahead_size_, + &prefetch_buffer_, /*implicit_auto_readahead=*/false, + /*num_file_reads=*/0, /*num_file_reads_for_auto_readahead=*/0); + return; + } + + // Explicit user requested readahead. + if (readahead_size > 0) { + rep->CreateFilePrefetchBufferIfNotExists( + readahead_size, readahead_size, &prefetch_buffer_, + /*implicit_auto_readahead=*/false, /*num_file_reads=*/0, + /*num_file_reads_for_auto_readahead=*/0); + return; + } + + // Implicit readahead. + + // If max_auto_readahead_size is set to be 0 by user, no data will be + // prefetched. + size_t max_auto_readahead_size = rep->table_options.max_auto_readahead_size; + if (max_auto_readahead_size == 0 || initial_auto_readahead_size_ == 0) { + return; + } + + if (initial_auto_readahead_size_ > max_auto_readahead_size) { + initial_auto_readahead_size_ = max_auto_readahead_size; + } + + // In case of no_sequential_checking, it will skip the num_file_reads_ and + // will always creates the FilePrefetchBuffer. + if (no_sequential_checking) { + rep->CreateFilePrefetchBufferIfNotExists( + initial_auto_readahead_size_, max_auto_readahead_size, + &prefetch_buffer_, /*implicit_auto_readahead=*/true, + /*num_file_reads=*/0, + rep->table_options.num_file_reads_for_auto_readahead); + return; + } + + size_t len = BlockBasedTable::BlockSizeWithTrailer(handle); + size_t offset = handle.offset(); + + // If FS supports prefetching (readahead_limit_ will be non zero in that case) + // and current block exists in prefetch buffer then return. + if (offset + len <= readahead_limit_) { + UpdateReadPattern(offset, len); + return; + } + + if (!IsBlockSequential(offset)) { + UpdateReadPattern(offset, len); + ResetValues(rep->table_options.initial_auto_readahead_size); + return; + } + UpdateReadPattern(offset, len); + + // Implicit auto readahead, which will be enabled if the number of reads + // reached `table_options.num_file_reads_for_auto_readahead` (default: 2) and + // scans are sequential. + num_file_reads_++; + if (num_file_reads_ <= rep->table_options.num_file_reads_for_auto_readahead) { + return; + } + + if (rep->file->use_direct_io()) { + rep->CreateFilePrefetchBufferIfNotExists( + initial_auto_readahead_size_, max_auto_readahead_size, + &prefetch_buffer_, /*implicit_auto_readahead=*/true, num_file_reads_, + rep->table_options.num_file_reads_for_auto_readahead); + return; + } + + if (readahead_size_ > max_auto_readahead_size) { + readahead_size_ = max_auto_readahead_size; + } + + // If prefetch is not supported, fall back to use internal prefetch buffer. + // Discarding other return status of Prefetch calls intentionally, as + // we can fallback to reading from disk if Prefetch fails. + Status s = rep->file->Prefetch( + handle.offset(), + BlockBasedTable::BlockSizeWithTrailer(handle) + readahead_size_, + rate_limiter_priority); + if (s.IsNotSupported()) { + rep->CreateFilePrefetchBufferIfNotExists( + initial_auto_readahead_size_, max_auto_readahead_size, + &prefetch_buffer_, /*implicit_auto_readahead=*/true, num_file_reads_, + rep->table_options.num_file_reads_for_auto_readahead); + return; + } + + readahead_limit_ = offset + len + readahead_size_; + // Keep exponentially increasing readahead size until + // max_auto_readahead_size. + readahead_size_ = std::min(max_auto_readahead_size, readahead_size_ * 2); +} +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/block_prefetcher.h b/src/rocksdb/table/block_based/block_prefetcher.h new file mode 100644 index 000000000..518868a30 --- /dev/null +++ b/src/rocksdb/table/block_based/block_prefetcher.h @@ -0,0 +1,72 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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 "table/block_based/block_based_table_reader.h" + +namespace ROCKSDB_NAMESPACE { +class BlockPrefetcher { + public: + explicit BlockPrefetcher(size_t compaction_readahead_size, + size_t initial_auto_readahead_size) + : compaction_readahead_size_(compaction_readahead_size), + readahead_size_(initial_auto_readahead_size), + initial_auto_readahead_size_(initial_auto_readahead_size) {} + + void PrefetchIfNeeded(const BlockBasedTable::Rep* rep, + const BlockHandle& handle, size_t readahead_size, + bool is_for_compaction, + const bool no_sequential_checking, + Env::IOPriority rate_limiter_priority); + FilePrefetchBuffer* prefetch_buffer() { return prefetch_buffer_.get(); } + + void UpdateReadPattern(const uint64_t& offset, const size_t& len) { + prev_offset_ = offset; + prev_len_ = len; + } + + bool IsBlockSequential(const uint64_t& offset) { + return (prev_len_ == 0 || (prev_offset_ + prev_len_ == offset)); + } + + void ResetValues(size_t initial_auto_readahead_size) { + num_file_reads_ = 1; + // Since initial_auto_readahead_size_ can be different from + // the value passed to BlockBasedTableOptions.initial_auto_readahead_size in + // case of adaptive_readahead, so fallback the readahead_size_ to that value + // in case of reset. + initial_auto_readahead_size_ = initial_auto_readahead_size; + readahead_size_ = initial_auto_readahead_size_; + readahead_limit_ = 0; + return; + } + + void SetReadaheadState(ReadaheadFileInfo::ReadaheadInfo* readahead_info) { + num_file_reads_ = readahead_info->num_file_reads; + initial_auto_readahead_size_ = readahead_info->readahead_size; + TEST_SYNC_POINT_CALLBACK("BlockPrefetcher::SetReadaheadState", + &initial_auto_readahead_size_); + } + + private: + // Readahead size used in compaction, its value is used only if + // lookup_context_.caller = kCompaction. + size_t compaction_readahead_size_; + + // readahead_size_ is used if underlying FS supports prefetching. + size_t readahead_size_; + size_t readahead_limit_ = 0; + // initial_auto_readahead_size_ is used if RocksDB uses internal prefetch + // buffer. + uint64_t initial_auto_readahead_size_; + uint64_t num_file_reads_ = 0; + uint64_t prev_offset_ = 0; + size_t prev_len_ = 0; + std::unique_ptr<FilePrefetchBuffer> prefetch_buffer_; +}; +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/block_prefix_index.cc b/src/rocksdb/table/block_based/block_prefix_index.cc new file mode 100644 index 000000000..c83701d69 --- /dev/null +++ b/src/rocksdb/table/block_based/block_prefix_index.cc @@ -0,0 +1,226 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#include "table/block_based/block_prefix_index.h" + +#include <vector> + +#include "memory/arena.h" +#include "rocksdb/comparator.h" +#include "rocksdb/slice.h" +#include "rocksdb/slice_transform.h" +#include "util/coding.h" +#include "util/hash.h" + +namespace ROCKSDB_NAMESPACE { + +inline uint32_t Hash(const Slice& s) { + return ROCKSDB_NAMESPACE::Hash(s.data(), s.size(), 0); +} + +inline uint32_t PrefixToBucket(const Slice& prefix, uint32_t num_buckets) { + return Hash(prefix) % num_buckets; +} + +// The prefix block index is simply a bucket array, with each entry pointing to +// the blocks that span the prefixes hashed to this bucket. +// +// To reduce memory footprint, if there is only one block per bucket, the entry +// stores the block id directly. If there are more than one blocks per bucket, +// because of hash collision or a single prefix spanning multiple blocks, +// the entry points to an array of block ids. The block array is an array of +// uint32_t's. The first uint32_t indicates the total number of blocks, followed +// by the block ids. +// +// To differentiate the two cases, the high order bit of the entry indicates +// whether it is a 'pointer' into a separate block array. +// 0x7FFFFFFF is reserved for empty bucket. + +const uint32_t kNoneBlock = 0x7FFFFFFF; +const uint32_t kBlockArrayMask = 0x80000000; + +inline bool IsNone(uint32_t block_id) { return block_id == kNoneBlock; } + +inline bool IsBlockId(uint32_t block_id) { + return (block_id & kBlockArrayMask) == 0; +} + +inline uint32_t DecodeIndex(uint32_t block_id) { + uint32_t index = block_id ^ kBlockArrayMask; + assert(index < kBlockArrayMask); + return index; +} + +inline uint32_t EncodeIndex(uint32_t index) { + assert(index < kBlockArrayMask); + return index | kBlockArrayMask; +} + +// temporary storage for prefix information during index building +struct PrefixRecord { + Slice prefix; + uint32_t start_block; + uint32_t end_block; + uint32_t num_blocks; + PrefixRecord* next; +}; + +class BlockPrefixIndex::Builder { + public: + void Add(const Slice& key_prefix, uint32_t start_block, uint32_t num_blocks) { + PrefixRecord* record = reinterpret_cast<PrefixRecord*>( + arena_.AllocateAligned(sizeof(PrefixRecord))); + record->prefix = key_prefix; + record->start_block = start_block; + record->end_block = start_block + num_blocks - 1; + record->num_blocks = num_blocks; + prefixes_.push_back(record); + } + + BlockPrefixIndex* Finish(const SliceTransform* prefix_extractor) { + // For now, use roughly 1:1 prefix to bucket ratio. + uint32_t num_buckets = static_cast<uint32_t>(prefixes_.size()) + 1; + + // Collect prefix records that hash to the same bucket, into a single + // linklist. + std::vector<PrefixRecord*> prefixes_per_bucket(num_buckets, nullptr); + std::vector<uint32_t> num_blocks_per_bucket(num_buckets, 0); + for (PrefixRecord* current : prefixes_) { + uint32_t bucket = PrefixToBucket(current->prefix, num_buckets); + // merge the prefix block span if the first block of this prefix is + // connected to the last block of the previous prefix. + PrefixRecord* prev = prefixes_per_bucket[bucket]; + if (prev) { + assert(current->start_block >= prev->end_block); + auto distance = current->start_block - prev->end_block; + if (distance <= 1) { + prev->end_block = current->end_block; + prev->num_blocks = prev->end_block - prev->start_block + 1; + num_blocks_per_bucket[bucket] += (current->num_blocks + distance - 1); + continue; + } + } + current->next = prev; + prefixes_per_bucket[bucket] = current; + num_blocks_per_bucket[bucket] += current->num_blocks; + } + + // Calculate the block array buffer size + uint32_t total_block_array_entries = 0; + for (uint32_t i = 0; i < num_buckets; i++) { + uint32_t num_blocks = num_blocks_per_bucket[i]; + if (num_blocks > 1) { + total_block_array_entries += (num_blocks + 1); + } + } + + // Populate the final prefix block index + uint32_t* block_array_buffer = new uint32_t[total_block_array_entries]; + uint32_t* buckets = new uint32_t[num_buckets]; + uint32_t offset = 0; + for (uint32_t i = 0; i < num_buckets; i++) { + uint32_t num_blocks = num_blocks_per_bucket[i]; + if (num_blocks == 0) { + assert(prefixes_per_bucket[i] == nullptr); + buckets[i] = kNoneBlock; + } else if (num_blocks == 1) { + assert(prefixes_per_bucket[i] != nullptr); + assert(prefixes_per_bucket[i]->next == nullptr); + buckets[i] = prefixes_per_bucket[i]->start_block; + } else { + assert(total_block_array_entries > 0); + assert(prefixes_per_bucket[i] != nullptr); + buckets[i] = EncodeIndex(offset); + block_array_buffer[offset] = num_blocks; + uint32_t* last_block = &block_array_buffer[offset + num_blocks]; + auto current = prefixes_per_bucket[i]; + // populate block ids from largest to smallest + while (current != nullptr) { + for (uint32_t iter = 0; iter < current->num_blocks; iter++) { + *last_block = current->end_block - iter; + last_block--; + } + current = current->next; + } + assert(last_block == &block_array_buffer[offset]); + offset += (num_blocks + 1); + } + } + + assert(offset == total_block_array_entries); + + return new BlockPrefixIndex(prefix_extractor, num_buckets, buckets, + total_block_array_entries, block_array_buffer); + } + + private: + std::vector<PrefixRecord*> prefixes_; + Arena arena_; +}; + +Status BlockPrefixIndex::Create(const SliceTransform* prefix_extractor, + const Slice& prefixes, const Slice& prefix_meta, + BlockPrefixIndex** prefix_index) { + uint64_t pos = 0; + auto meta_pos = prefix_meta; + Status s; + Builder builder; + + while (!meta_pos.empty()) { + uint32_t prefix_size = 0; + uint32_t entry_index = 0; + uint32_t num_blocks = 0; + if (!GetVarint32(&meta_pos, &prefix_size) || + !GetVarint32(&meta_pos, &entry_index) || + !GetVarint32(&meta_pos, &num_blocks)) { + s = Status::Corruption( + "Corrupted prefix meta block: unable to read from it."); + break; + } + if (pos + prefix_size > prefixes.size()) { + s = Status::Corruption( + "Corrupted prefix meta block: size inconsistency."); + break; + } + Slice prefix(prefixes.data() + pos, prefix_size); + builder.Add(prefix, entry_index, num_blocks); + + pos += prefix_size; + } + + if (s.ok() && pos != prefixes.size()) { + s = Status::Corruption("Corrupted prefix meta block"); + } + + if (s.ok()) { + *prefix_index = builder.Finish(prefix_extractor); + } + + return s; +} + +uint32_t BlockPrefixIndex::GetBlocks(const Slice& key, uint32_t** blocks) { + Slice prefix = internal_prefix_extractor_.Transform(key); + + uint32_t bucket = PrefixToBucket(prefix, num_buckets_); + uint32_t block_id = buckets_[bucket]; + + if (IsNone(block_id)) { + return 0; + } else if (IsBlockId(block_id)) { + *blocks = &buckets_[bucket]; + return 1; + } else { + uint32_t index = DecodeIndex(block_id); + assert(index < num_block_array_buffer_entries_); + *blocks = &block_array_buffer_[index + 1]; + uint32_t num_blocks = block_array_buffer_[index]; + assert(num_blocks > 1); + assert(index + num_blocks < num_block_array_buffer_entries_); + return num_blocks; + } +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/block_prefix_index.h b/src/rocksdb/table/block_based/block_prefix_index.h new file mode 100644 index 000000000..4db8e2c65 --- /dev/null +++ b/src/rocksdb/table/block_based/block_prefix_index.h @@ -0,0 +1,70 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +#pragma once + +#include <stdint.h> + +#include "db/dbformat.h" +#include "rocksdb/status.h" + +namespace ROCKSDB_NAMESPACE { + +class Comparator; +class Iterator; +class Slice; +class SliceTransform; + +// Build a hash-based index to speed up the lookup for "index block". +// BlockHashIndex accepts a key and, if found, returns its restart index within +// that index block. +class BlockPrefixIndex { + public: + // Maps a key to a list of data blocks that could potentially contain + // the key, based on the prefix. + // Returns the total number of relevant blocks, 0 means the key does + // not exist. + uint32_t GetBlocks(const Slice& key, uint32_t** blocks); + + size_t ApproximateMemoryUsage() const { + return sizeof(BlockPrefixIndex) + + (num_block_array_buffer_entries_ + num_buckets_) * sizeof(uint32_t); + } + + // Create hash index by reading from the metadata blocks. + // Note: table reader (caller) is responsible for keeping shared_ptr to + // underlying prefix extractor + // @params prefixes: a sequence of prefixes. + // @params prefix_meta: contains the "metadata" to of the prefixes. + static Status Create(const SliceTransform* hash_key_extractor, + const Slice& prefixes, const Slice& prefix_meta, + BlockPrefixIndex** prefix_index); + + ~BlockPrefixIndex() { + delete[] buckets_; + delete[] block_array_buffer_; + } + + private: + class Builder; + friend Builder; + + BlockPrefixIndex(const SliceTransform* prefix_extractor, uint32_t num_buckets, + uint32_t* buckets, uint32_t num_block_array_buffer_entries, + uint32_t* block_array_buffer) + : internal_prefix_extractor_(prefix_extractor), + num_buckets_(num_buckets), + num_block_array_buffer_entries_(num_block_array_buffer_entries), + buckets_(buckets), + block_array_buffer_(block_array_buffer) {} + + InternalKeySliceTransform internal_prefix_extractor_; + + uint32_t num_buckets_; + uint32_t num_block_array_buffer_entries_; + uint32_t* buckets_; + uint32_t* block_array_buffer_; +}; + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/block_test.cc b/src/rocksdb/table/block_based/block_test.cc new file mode 100644 index 000000000..83b87fe79 --- /dev/null +++ b/src/rocksdb/table/block_based/block_test.cc @@ -0,0 +1,627 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// + +#include "table/block_based/block.h" + +#include <stdio.h> + +#include <algorithm> +#include <set> +#include <string> +#include <unordered_set> +#include <utility> +#include <vector> + +#include "db/dbformat.h" +#include "db/memtable.h" +#include "db/write_batch_internal.h" +#include "rocksdb/db.h" +#include "rocksdb/env.h" +#include "rocksdb/iterator.h" +#include "rocksdb/slice_transform.h" +#include "rocksdb/table.h" +#include "table/block_based/block_based_table_reader.h" +#include "table/block_based/block_builder.h" +#include "table/format.h" +#include "test_util/testharness.h" +#include "test_util/testutil.h" +#include "util/random.h" + +namespace ROCKSDB_NAMESPACE { + +std::string GenerateInternalKey(int primary_key, int secondary_key, + int padding_size, Random *rnd) { + char buf[50]; + char *p = &buf[0]; + snprintf(buf, sizeof(buf), "%6d%4d", primary_key, secondary_key); + std::string k(p); + if (padding_size) { + k += rnd->RandomString(padding_size); + } + AppendInternalKeyFooter(&k, 0 /* seqno */, kTypeValue); + + return k; +} + +// Generate random key value pairs. +// The generated key will be sorted. You can tune the parameters to generated +// different kinds of test key/value pairs for different scenario. +void GenerateRandomKVs(std::vector<std::string> *keys, + std::vector<std::string> *values, const int from, + const int len, const int step = 1, + const int padding_size = 0, + const int keys_share_prefix = 1) { + Random rnd(302); + + // generate different prefix + for (int i = from; i < from + len; i += step) { + // generating keys that shares the prefix + for (int j = 0; j < keys_share_prefix; ++j) { + // `DataBlockIter` assumes it reads only internal keys. + keys->emplace_back(GenerateInternalKey(i, j, padding_size, &rnd)); + + // 100 bytes values + values->emplace_back(rnd.RandomString(100)); + } + } +} + +class BlockTest : public testing::Test {}; + +// block test +TEST_F(BlockTest, SimpleTest) { + Random rnd(301); + Options options = Options(); + + std::vector<std::string> keys; + std::vector<std::string> values; + BlockBuilder builder(16); + int num_records = 100000; + + GenerateRandomKVs(&keys, &values, 0, num_records); + // add a bunch of records to a block + for (int i = 0; i < num_records; i++) { + builder.Add(keys[i], values[i]); + } + + // read serialized contents of the block + Slice rawblock = builder.Finish(); + + // create block reader + BlockContents contents; + contents.data = rawblock; + Block reader(std::move(contents)); + + // read contents of block sequentially + int count = 0; + InternalIterator *iter = + reader.NewDataIterator(options.comparator, kDisableGlobalSequenceNumber); + for (iter->SeekToFirst(); iter->Valid(); count++, iter->Next()) { + // read kv from block + Slice k = iter->key(); + Slice v = iter->value(); + + // compare with lookaside array + ASSERT_EQ(k.ToString().compare(keys[count]), 0); + ASSERT_EQ(v.ToString().compare(values[count]), 0); + } + delete iter; + + // read block contents randomly + iter = + reader.NewDataIterator(options.comparator, kDisableGlobalSequenceNumber); + for (int i = 0; i < num_records; i++) { + // find a random key in the lookaside array + int index = rnd.Uniform(num_records); + Slice k(keys[index]); + + // search in block for this key + iter->Seek(k); + ASSERT_TRUE(iter->Valid()); + Slice v = iter->value(); + ASSERT_EQ(v.ToString().compare(values[index]), 0); + } + delete iter; +} + +// return the block contents +BlockContents GetBlockContents(std::unique_ptr<BlockBuilder> *builder, + const std::vector<std::string> &keys, + const std::vector<std::string> &values, + const int /*prefix_group_size*/ = 1) { + builder->reset(new BlockBuilder(1 /* restart interval */)); + + // Add only half of the keys + for (size_t i = 0; i < keys.size(); ++i) { + (*builder)->Add(keys[i], values[i]); + } + Slice rawblock = (*builder)->Finish(); + + BlockContents contents; + contents.data = rawblock; + + return contents; +} + +void CheckBlockContents(BlockContents contents, const int max_key, + const std::vector<std::string> &keys, + const std::vector<std::string> &values) { + const size_t prefix_size = 6; + // create block reader + BlockContents contents_ref(contents.data); + Block reader1(std::move(contents)); + Block reader2(std::move(contents_ref)); + + std::unique_ptr<const SliceTransform> prefix_extractor( + NewFixedPrefixTransform(prefix_size)); + + std::unique_ptr<InternalIterator> regular_iter(reader2.NewDataIterator( + BytewiseComparator(), kDisableGlobalSequenceNumber)); + + // Seek existent keys + for (size_t i = 0; i < keys.size(); i++) { + regular_iter->Seek(keys[i]); + ASSERT_OK(regular_iter->status()); + ASSERT_TRUE(regular_iter->Valid()); + + Slice v = regular_iter->value(); + ASSERT_EQ(v.ToString().compare(values[i]), 0); + } + + // Seek non-existent keys. + // For hash index, if no key with a given prefix is not found, iterator will + // simply be set as invalid; whereas the binary search based iterator will + // return the one that is closest. + for (int i = 1; i < max_key - 1; i += 2) { + // `DataBlockIter` assumes its APIs receive only internal keys. + auto key = GenerateInternalKey(i, 0, 0, nullptr); + regular_iter->Seek(key); + ASSERT_TRUE(regular_iter->Valid()); + } +} + +// In this test case, no two key share same prefix. +TEST_F(BlockTest, SimpleIndexHash) { + const int kMaxKey = 100000; + std::vector<std::string> keys; + std::vector<std::string> values; + GenerateRandomKVs(&keys, &values, 0 /* first key id */, + kMaxKey /* last key id */, 2 /* step */, + 8 /* padding size (8 bytes randomly generated suffix) */); + + std::unique_ptr<BlockBuilder> builder; + auto contents = GetBlockContents(&builder, keys, values); + + CheckBlockContents(std::move(contents), kMaxKey, keys, values); +} + +TEST_F(BlockTest, IndexHashWithSharedPrefix) { + const int kMaxKey = 100000; + // for each prefix, there will be 5 keys starts with it. + const int kPrefixGroup = 5; + std::vector<std::string> keys; + std::vector<std::string> values; + // Generate keys with same prefix. + GenerateRandomKVs(&keys, &values, 0, // first key id + kMaxKey, // last key id + 2, // step + 10, // padding size, + kPrefixGroup); + + std::unique_ptr<BlockBuilder> builder; + auto contents = GetBlockContents(&builder, keys, values, kPrefixGroup); + + CheckBlockContents(std::move(contents), kMaxKey, keys, values); +} + +// A slow and accurate version of BlockReadAmpBitmap that simply store +// all the marked ranges in a set. +class BlockReadAmpBitmapSlowAndAccurate { + public: + void Mark(size_t start_offset, size_t end_offset) { + assert(end_offset >= start_offset); + marked_ranges_.emplace(end_offset, start_offset); + } + + void ResetCheckSequence() { iter_valid_ = false; } + + // Return true if any byte in this range was Marked + // This does linear search from the previous position. When calling + // multiple times, `offset` needs to be incremental to get correct results. + // Call ResetCheckSequence() to reset it. + bool IsPinMarked(size_t offset) { + if (iter_valid_) { + // Has existing iterator, try linear search from + // the iterator. + for (int i = 0; i < 64; i++) { + if (offset < iter_->second) { + return false; + } + if (offset <= iter_->first) { + return true; + } + + iter_++; + if (iter_ == marked_ranges_.end()) { + iter_valid_ = false; + return false; + } + } + } + // Initial call or have linear searched too many times. + // Do binary search. + iter_ = marked_ranges_.lower_bound( + std::make_pair(offset, static_cast<size_t>(0))); + if (iter_ == marked_ranges_.end()) { + iter_valid_ = false; + return false; + } + iter_valid_ = true; + return offset <= iter_->first && offset >= iter_->second; + } + + private: + std::set<std::pair<size_t, size_t>> marked_ranges_; + std::set<std::pair<size_t, size_t>>::iterator iter_; + bool iter_valid_ = false; +}; + +TEST_F(BlockTest, BlockReadAmpBitmap) { + uint32_t pin_offset = 0; + SyncPoint::GetInstance()->SetCallBack( + "BlockReadAmpBitmap:rnd", [&pin_offset](void *arg) { + pin_offset = *(static_cast<uint32_t *>(arg)); + }); + SyncPoint::GetInstance()->EnableProcessing(); + std::vector<size_t> block_sizes = { + 1, // 1 byte + 32, // 32 bytes + 61, // 61 bytes + 64, // 64 bytes + 512, // 0.5 KB + 1024, // 1 KB + 1024 * 4, // 4 KB + 1024 * 10, // 10 KB + 1024 * 50, // 50 KB + 1024 * 1024 * 4, // 5 MB + 777, + 124653, + }; + const size_t kBytesPerBit = 64; + + Random rnd(301); + for (size_t block_size : block_sizes) { + std::shared_ptr<Statistics> stats = ROCKSDB_NAMESPACE::CreateDBStatistics(); + BlockReadAmpBitmap read_amp_bitmap(block_size, kBytesPerBit, stats.get()); + BlockReadAmpBitmapSlowAndAccurate read_amp_slow_and_accurate; + + size_t needed_bits = (block_size / kBytesPerBit); + if (block_size % kBytesPerBit != 0) { + needed_bits++; + } + + ASSERT_EQ(stats->getTickerCount(READ_AMP_TOTAL_READ_BYTES), block_size); + + // Generate some random entries + std::vector<size_t> random_entry_offsets; + for (int i = 0; i < 1000; i++) { + random_entry_offsets.push_back(rnd.Next() % block_size); + } + std::sort(random_entry_offsets.begin(), random_entry_offsets.end()); + auto it = + std::unique(random_entry_offsets.begin(), random_entry_offsets.end()); + random_entry_offsets.resize( + std::distance(random_entry_offsets.begin(), it)); + + std::vector<std::pair<size_t, size_t>> random_entries; + for (size_t i = 0; i < random_entry_offsets.size(); i++) { + size_t entry_start = random_entry_offsets[i]; + size_t entry_end; + if (i + 1 < random_entry_offsets.size()) { + entry_end = random_entry_offsets[i + 1] - 1; + } else { + entry_end = block_size - 1; + } + random_entries.emplace_back(entry_start, entry_end); + } + + for (size_t i = 0; i < random_entries.size(); i++) { + read_amp_slow_and_accurate.ResetCheckSequence(); + auto ¤t_entry = random_entries[rnd.Next() % random_entries.size()]; + + read_amp_bitmap.Mark(static_cast<uint32_t>(current_entry.first), + static_cast<uint32_t>(current_entry.second)); + read_amp_slow_and_accurate.Mark(current_entry.first, + current_entry.second); + + size_t total_bits = 0; + for (size_t bit_idx = 0; bit_idx < needed_bits; bit_idx++) { + total_bits += read_amp_slow_and_accurate.IsPinMarked( + bit_idx * kBytesPerBit + pin_offset); + } + size_t expected_estimate_useful = total_bits * kBytesPerBit; + size_t got_estimate_useful = + stats->getTickerCount(READ_AMP_ESTIMATE_USEFUL_BYTES); + ASSERT_EQ(expected_estimate_useful, got_estimate_useful); + } + } + SyncPoint::GetInstance()->DisableProcessing(); + SyncPoint::GetInstance()->ClearAllCallBacks(); +} + +TEST_F(BlockTest, BlockWithReadAmpBitmap) { + Random rnd(301); + Options options = Options(); + + std::vector<std::string> keys; + std::vector<std::string> values; + BlockBuilder builder(16); + int num_records = 10000; + + GenerateRandomKVs(&keys, &values, 0, num_records, 1); + // add a bunch of records to a block + for (int i = 0; i < num_records; i++) { + builder.Add(keys[i], values[i]); + } + + Slice rawblock = builder.Finish(); + const size_t kBytesPerBit = 8; + + // Read the block sequentially using Next() + { + std::shared_ptr<Statistics> stats = ROCKSDB_NAMESPACE::CreateDBStatistics(); + + // create block reader + BlockContents contents; + contents.data = rawblock; + Block reader(std::move(contents), kBytesPerBit, stats.get()); + + // read contents of block sequentially + size_t read_bytes = 0; + DataBlockIter *iter = reader.NewDataIterator( + options.comparator, kDisableGlobalSequenceNumber, nullptr, stats.get()); + for (iter->SeekToFirst(); iter->Valid(); iter->Next()) { + iter->value(); + read_bytes += iter->TEST_CurrentEntrySize(); + + double semi_acc_read_amp = + static_cast<double>(read_bytes) / rawblock.size(); + double read_amp = static_cast<double>(stats->getTickerCount( + READ_AMP_ESTIMATE_USEFUL_BYTES)) / + stats->getTickerCount(READ_AMP_TOTAL_READ_BYTES); + + // Error in read amplification will be less than 1% if we are reading + // sequentially + double error_pct = fabs(semi_acc_read_amp - read_amp) * 100; + EXPECT_LT(error_pct, 1); + } + + delete iter; + } + + // Read the block sequentially using Seek() + { + std::shared_ptr<Statistics> stats = ROCKSDB_NAMESPACE::CreateDBStatistics(); + + // create block reader + BlockContents contents; + contents.data = rawblock; + Block reader(std::move(contents), kBytesPerBit, stats.get()); + + size_t read_bytes = 0; + DataBlockIter *iter = reader.NewDataIterator( + options.comparator, kDisableGlobalSequenceNumber, nullptr, stats.get()); + for (int i = 0; i < num_records; i++) { + Slice k(keys[i]); + + // search in block for this key + iter->Seek(k); + iter->value(); + read_bytes += iter->TEST_CurrentEntrySize(); + + double semi_acc_read_amp = + static_cast<double>(read_bytes) / rawblock.size(); + double read_amp = static_cast<double>(stats->getTickerCount( + READ_AMP_ESTIMATE_USEFUL_BYTES)) / + stats->getTickerCount(READ_AMP_TOTAL_READ_BYTES); + + // Error in read amplification will be less than 1% if we are reading + // sequentially + double error_pct = fabs(semi_acc_read_amp - read_amp) * 100; + EXPECT_LT(error_pct, 1); + } + delete iter; + } + + // Read the block randomly + { + std::shared_ptr<Statistics> stats = ROCKSDB_NAMESPACE::CreateDBStatistics(); + + // create block reader + BlockContents contents; + contents.data = rawblock; + Block reader(std::move(contents), kBytesPerBit, stats.get()); + + size_t read_bytes = 0; + DataBlockIter *iter = reader.NewDataIterator( + options.comparator, kDisableGlobalSequenceNumber, nullptr, stats.get()); + std::unordered_set<int> read_keys; + for (int i = 0; i < num_records; i++) { + int index = rnd.Uniform(num_records); + Slice k(keys[index]); + + iter->Seek(k); + iter->value(); + if (read_keys.find(index) == read_keys.end()) { + read_keys.insert(index); + read_bytes += iter->TEST_CurrentEntrySize(); + } + + double semi_acc_read_amp = + static_cast<double>(read_bytes) / rawblock.size(); + double read_amp = static_cast<double>(stats->getTickerCount( + READ_AMP_ESTIMATE_USEFUL_BYTES)) / + stats->getTickerCount(READ_AMP_TOTAL_READ_BYTES); + + double error_pct = fabs(semi_acc_read_amp - read_amp) * 100; + // Error in read amplification will be less than 2% if we are reading + // randomly + EXPECT_LT(error_pct, 2); + } + delete iter; + } +} + +TEST_F(BlockTest, ReadAmpBitmapPow2) { + std::shared_ptr<Statistics> stats = ROCKSDB_NAMESPACE::CreateDBStatistics(); + ASSERT_EQ(BlockReadAmpBitmap(100, 1, stats.get()).GetBytesPerBit(), 1u); + ASSERT_EQ(BlockReadAmpBitmap(100, 2, stats.get()).GetBytesPerBit(), 2u); + ASSERT_EQ(BlockReadAmpBitmap(100, 4, stats.get()).GetBytesPerBit(), 4u); + ASSERT_EQ(BlockReadAmpBitmap(100, 8, stats.get()).GetBytesPerBit(), 8u); + ASSERT_EQ(BlockReadAmpBitmap(100, 16, stats.get()).GetBytesPerBit(), 16u); + ASSERT_EQ(BlockReadAmpBitmap(100, 32, stats.get()).GetBytesPerBit(), 32u); + + ASSERT_EQ(BlockReadAmpBitmap(100, 3, stats.get()).GetBytesPerBit(), 2u); + ASSERT_EQ(BlockReadAmpBitmap(100, 7, stats.get()).GetBytesPerBit(), 4u); + ASSERT_EQ(BlockReadAmpBitmap(100, 11, stats.get()).GetBytesPerBit(), 8u); + ASSERT_EQ(BlockReadAmpBitmap(100, 17, stats.get()).GetBytesPerBit(), 16u); + ASSERT_EQ(BlockReadAmpBitmap(100, 33, stats.get()).GetBytesPerBit(), 32u); + ASSERT_EQ(BlockReadAmpBitmap(100, 35, stats.get()).GetBytesPerBit(), 32u); +} + +class IndexBlockTest + : public testing::Test, + public testing::WithParamInterface<std::tuple<bool, bool>> { + public: + IndexBlockTest() = default; + + bool useValueDeltaEncoding() const { return std::get<0>(GetParam()); } + bool includeFirstKey() const { return std::get<1>(GetParam()); } +}; + +// Similar to GenerateRandomKVs but for index block contents. +void GenerateRandomIndexEntries(std::vector<std::string> *separators, + std::vector<BlockHandle> *block_handles, + std::vector<std::string> *first_keys, + const int len) { + Random rnd(42); + + // For each of `len` blocks, we need to generate a first and last key. + // Let's generate n*2 random keys, sort them, group into consecutive pairs. + std::set<std::string> keys; + while ((int)keys.size() < len * 2) { + // Keys need to be at least 8 bytes long to look like internal keys. + keys.insert(test::RandomKey(&rnd, 12)); + } + + uint64_t offset = 0; + for (auto it = keys.begin(); it != keys.end();) { + first_keys->emplace_back(*it++); + separators->emplace_back(*it++); + uint64_t size = rnd.Uniform(1024 * 16); + BlockHandle handle(offset, size); + offset += size + BlockBasedTable::kBlockTrailerSize; + block_handles->emplace_back(handle); + } +} + +TEST_P(IndexBlockTest, IndexValueEncodingTest) { + Random rnd(301); + Options options = Options(); + + std::vector<std::string> separators; + std::vector<BlockHandle> block_handles; + std::vector<std::string> first_keys; + const bool kUseDeltaEncoding = true; + BlockBuilder builder(16, kUseDeltaEncoding, useValueDeltaEncoding()); + int num_records = 100; + + GenerateRandomIndexEntries(&separators, &block_handles, &first_keys, + num_records); + BlockHandle last_encoded_handle; + for (int i = 0; i < num_records; i++) { + IndexValue entry(block_handles[i], first_keys[i]); + std::string encoded_entry; + std::string delta_encoded_entry; + entry.EncodeTo(&encoded_entry, includeFirstKey(), nullptr); + if (useValueDeltaEncoding() && i > 0) { + entry.EncodeTo(&delta_encoded_entry, includeFirstKey(), + &last_encoded_handle); + } + last_encoded_handle = entry.handle; + const Slice delta_encoded_entry_slice(delta_encoded_entry); + builder.Add(separators[i], encoded_entry, &delta_encoded_entry_slice); + } + + // read serialized contents of the block + Slice rawblock = builder.Finish(); + + // create block reader + BlockContents contents; + contents.data = rawblock; + Block reader(std::move(contents)); + + const bool kTotalOrderSeek = true; + const bool kIncludesSeq = true; + const bool kValueIsFull = !useValueDeltaEncoding(); + IndexBlockIter *kNullIter = nullptr; + Statistics *kNullStats = nullptr; + // read contents of block sequentially + InternalIteratorBase<IndexValue> *iter = reader.NewIndexIterator( + options.comparator, kDisableGlobalSequenceNumber, kNullIter, kNullStats, + kTotalOrderSeek, includeFirstKey(), kIncludesSeq, kValueIsFull); + iter->SeekToFirst(); + for (int index = 0; index < num_records; ++index) { + ASSERT_TRUE(iter->Valid()); + + Slice k = iter->key(); + IndexValue v = iter->value(); + + EXPECT_EQ(separators[index], k.ToString()); + EXPECT_EQ(block_handles[index].offset(), v.handle.offset()); + EXPECT_EQ(block_handles[index].size(), v.handle.size()); + EXPECT_EQ(includeFirstKey() ? first_keys[index] : "", + v.first_internal_key.ToString()); + + iter->Next(); + } + delete iter; + + // read block contents randomly + iter = reader.NewIndexIterator( + options.comparator, kDisableGlobalSequenceNumber, kNullIter, kNullStats, + kTotalOrderSeek, includeFirstKey(), kIncludesSeq, kValueIsFull); + for (int i = 0; i < num_records * 2; i++) { + // find a random key in the lookaside array + int index = rnd.Uniform(num_records); + Slice k(separators[index]); + + // search in block for this key + iter->Seek(k); + ASSERT_TRUE(iter->Valid()); + IndexValue v = iter->value(); + EXPECT_EQ(separators[index], iter->key().ToString()); + EXPECT_EQ(block_handles[index].offset(), v.handle.offset()); + EXPECT_EQ(block_handles[index].size(), v.handle.size()); + EXPECT_EQ(includeFirstKey() ? first_keys[index] : "", + v.first_internal_key.ToString()); + } + delete iter; +} + +INSTANTIATE_TEST_CASE_P(P, IndexBlockTest, + ::testing::Values(std::make_tuple(false, false), + std::make_tuple(false, true), + std::make_tuple(true, false), + std::make_tuple(true, true))); + +} // namespace ROCKSDB_NAMESPACE + +int main(int argc, char **argv) { + ROCKSDB_NAMESPACE::port::InstallStackTraceHandler(); + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/src/rocksdb/table/block_based/block_type.h b/src/rocksdb/table/block_based/block_type.h new file mode 100644 index 000000000..a9d6a1a77 --- /dev/null +++ b/src/rocksdb/table/block_based/block_type.h @@ -0,0 +1,34 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#pragma once + +#include <cstdint> + +#include "rocksdb/rocksdb_namespace.h" + +namespace ROCKSDB_NAMESPACE { + +// Represents the types of blocks used in the block based table format. +// See https://github.com/facebook/rocksdb/wiki/Rocksdb-BlockBasedTable-Format +// for details. +// For code sanity, BlockType should imply a specific TBlocklike for +// BlocklikeTraits. +enum class BlockType : uint8_t { + kData, + kFilter, // for second level partitioned filters and full filters + kFilterPartitionIndex, // for top-level index of filter partitions + kProperties, + kCompressionDictionary, + kRangeDeletion, + kHashIndexPrefixes, + kHashIndexMetadata, + kMetaIndex, + kIndex, + // Note: keep kInvalid the last value when adding new enum values. + kInvalid +}; + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/cachable_entry.h b/src/rocksdb/table/block_based/cachable_entry.h new file mode 100644 index 000000000..ad8acb18d --- /dev/null +++ b/src/rocksdb/table/block_based/cachable_entry.h @@ -0,0 +1,232 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// +// Copyright (c) 2012 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 <cassert> + +#include "port/likely.h" +#include "rocksdb/cache.h" +#include "rocksdb/cleanable.h" + +namespace ROCKSDB_NAMESPACE { + +// CachableEntry is a handle to an object that may or may not be in the block +// cache. It is used in a variety of ways: +// +// 1) It may refer to an object in the block cache. In this case, cache_ and +// cache_handle_ are not nullptr, and the cache handle has to be released when +// the CachableEntry is destroyed (the lifecycle of the cached object, on the +// other hand, is managed by the cache itself). +// 2) It may uniquely own the (non-cached) object it refers to (examples include +// a block read directly from file, or uncompressed blocks when there is a +// compressed block cache but no uncompressed block cache). In such cases, the +// object has to be destroyed when the CachableEntry is destroyed. +// 3) It may point to an object (cached or not) without owning it. In this case, +// no action is needed when the CachableEntry is destroyed. +// 4) Sometimes, management of a cached or owned object (see #1 and #2 above) +// is transferred to some other object. This is used for instance with iterators +// (where cleanup is performed using a chain of cleanup functions, +// see Cleanable). +// +// Because of #1 and #2 above, copying a CachableEntry is not safe (and thus not +// allowed); hence, this is a move-only type, where a move transfers the +// management responsibilities, and leaves the source object in an empty state. + +template <class T> +class CachableEntry { + public: + CachableEntry() = default; + + CachableEntry(T* value, Cache* cache, Cache::Handle* cache_handle, + bool own_value) + : value_(value), + cache_(cache), + cache_handle_(cache_handle), + own_value_(own_value) { + assert(value_ != nullptr || + (cache_ == nullptr && cache_handle_ == nullptr && !own_value_)); + assert(!!cache_ == !!cache_handle_); + assert(!cache_handle_ || !own_value_); + } + + CachableEntry(const CachableEntry&) = delete; + CachableEntry& operator=(const CachableEntry&) = delete; + + CachableEntry(CachableEntry&& rhs) noexcept + : value_(rhs.value_), + cache_(rhs.cache_), + cache_handle_(rhs.cache_handle_), + own_value_(rhs.own_value_) { + assert(value_ != nullptr || + (cache_ == nullptr && cache_handle_ == nullptr && !own_value_)); + assert(!!cache_ == !!cache_handle_); + assert(!cache_handle_ || !own_value_); + + rhs.ResetFields(); + } + + CachableEntry& operator=(CachableEntry&& rhs) noexcept { + if (UNLIKELY(this == &rhs)) { + return *this; + } + + ReleaseResource(); + + value_ = rhs.value_; + cache_ = rhs.cache_; + cache_handle_ = rhs.cache_handle_; + own_value_ = rhs.own_value_; + + assert(value_ != nullptr || + (cache_ == nullptr && cache_handle_ == nullptr && !own_value_)); + assert(!!cache_ == !!cache_handle_); + assert(!cache_handle_ || !own_value_); + + rhs.ResetFields(); + + return *this; + } + + ~CachableEntry() { ReleaseResource(); } + + bool IsEmpty() const { + return value_ == nullptr && cache_ == nullptr && cache_handle_ == nullptr && + !own_value_; + } + + bool IsCached() const { + assert(!!cache_ == !!cache_handle_); + + return cache_handle_ != nullptr; + } + + T* GetValue() const { return value_; } + Cache* GetCache() const { return cache_; } + Cache::Handle* GetCacheHandle() const { return cache_handle_; } + bool GetOwnValue() const { return own_value_; } + + void Reset() { + ReleaseResource(); + ResetFields(); + } + + void TransferTo(Cleanable* cleanable) { + if (cleanable) { + if (cache_handle_ != nullptr) { + assert(cache_ != nullptr); + cleanable->RegisterCleanup(&ReleaseCacheHandle, cache_, cache_handle_); + } else if (own_value_) { + cleanable->RegisterCleanup(&DeleteValue, value_, nullptr); + } + } + + ResetFields(); + } + + void SetOwnedValue(std::unique_ptr<T>&& value) { + assert(value.get() != nullptr); + + if (UNLIKELY(value_ == value.get() && own_value_)) { + assert(cache_ == nullptr && cache_handle_ == nullptr); + return; + } + + Reset(); + + value_ = value.release(); + own_value_ = true; + } + + void SetUnownedValue(T* value) { + assert(value != nullptr); + + if (UNLIKELY(value_ == value && cache_ == nullptr && + cache_handle_ == nullptr && !own_value_)) { + return; + } + + Reset(); + + value_ = value; + assert(!own_value_); + } + + void SetCachedValue(T* value, Cache* cache, Cache::Handle* cache_handle) { + assert(cache != nullptr); + assert(cache_handle != nullptr); + + if (UNLIKELY(value_ == value && cache_ == cache && + cache_handle_ == cache_handle && !own_value_)) { + return; + } + + Reset(); + + value_ = value; + cache_ = cache; + cache_handle_ = cache_handle; + assert(!own_value_); + } + + void UpdateCachedValue() { + assert(cache_ != nullptr); + assert(cache_handle_ != nullptr); + + value_ = static_cast<T*>(cache_->Value(cache_handle_)); + } + + bool IsReady() { + if (!own_value_) { + assert(cache_ != nullptr); + assert(cache_handle_ != nullptr); + return cache_->IsReady(cache_handle_); + } + return true; + } + + private: + void ReleaseResource() noexcept { + if (LIKELY(cache_handle_ != nullptr)) { + assert(cache_ != nullptr); + cache_->Release(cache_handle_); + } else if (own_value_) { + delete value_; + } + } + + void ResetFields() noexcept { + value_ = nullptr; + cache_ = nullptr; + cache_handle_ = nullptr; + own_value_ = false; + } + + static void ReleaseCacheHandle(void* arg1, void* arg2) { + Cache* const cache = static_cast<Cache*>(arg1); + assert(cache); + + Cache::Handle* const cache_handle = static_cast<Cache::Handle*>(arg2); + assert(cache_handle); + + cache->Release(cache_handle); + } + + static void DeleteValue(void* arg1, void* /* arg2 */) { + delete static_cast<T*>(arg1); + } + + private: + T* value_ = nullptr; + Cache* cache_ = nullptr; + Cache::Handle* cache_handle_ = nullptr; + bool own_value_ = false; +}; + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/data_block_footer.cc b/src/rocksdb/table/block_based/data_block_footer.cc new file mode 100644 index 000000000..5d5d8ed55 --- /dev/null +++ b/src/rocksdb/table/block_based/data_block_footer.cc @@ -0,0 +1,59 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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/block_based/data_block_footer.h" + +#include "rocksdb/table.h" + +namespace ROCKSDB_NAMESPACE { + +const int kDataBlockIndexTypeBitShift = 31; + +// 0x7FFFFFFF +const uint32_t kMaxNumRestarts = (1u << kDataBlockIndexTypeBitShift) - 1u; + +// 0x7FFFFFFF +const uint32_t kNumRestartsMask = (1u << kDataBlockIndexTypeBitShift) - 1u; + +uint32_t PackIndexTypeAndNumRestarts( + BlockBasedTableOptions::DataBlockIndexType index_type, + uint32_t num_restarts) { + if (num_restarts > kMaxNumRestarts) { + assert(0); // mute travis "unused" warning + } + + uint32_t block_footer = num_restarts; + if (index_type == BlockBasedTableOptions::kDataBlockBinaryAndHash) { + block_footer |= 1u << kDataBlockIndexTypeBitShift; + } else if (index_type != BlockBasedTableOptions::kDataBlockBinarySearch) { + assert(0); + } + + return block_footer; +} + +void UnPackIndexTypeAndNumRestarts( + uint32_t block_footer, + BlockBasedTableOptions::DataBlockIndexType* index_type, + uint32_t* num_restarts) { + if (index_type) { + if (block_footer & 1u << kDataBlockIndexTypeBitShift) { + *index_type = BlockBasedTableOptions::kDataBlockBinaryAndHash; + } else { + *index_type = BlockBasedTableOptions::kDataBlockBinarySearch; + } + } + + if (num_restarts) { + *num_restarts = block_footer & kNumRestartsMask; + assert(*num_restarts <= kMaxNumRestarts); + } +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/data_block_footer.h b/src/rocksdb/table/block_based/data_block_footer.h new file mode 100644 index 000000000..c1cfd4730 --- /dev/null +++ b/src/rocksdb/table/block_based/data_block_footer.h @@ -0,0 +1,25 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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 "rocksdb/table.h" + +namespace ROCKSDB_NAMESPACE { + +uint32_t PackIndexTypeAndNumRestarts( + BlockBasedTableOptions::DataBlockIndexType index_type, + uint32_t num_restarts); + +void UnPackIndexTypeAndNumRestarts( + uint32_t block_footer, + BlockBasedTableOptions::DataBlockIndexType* index_type, + uint32_t* num_restarts); + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/data_block_hash_index.cc b/src/rocksdb/table/block_based/data_block_hash_index.cc new file mode 100644 index 000000000..c579dcc43 --- /dev/null +++ b/src/rocksdb/table/block_based/data_block_hash_index.cc @@ -0,0 +1,94 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +#include "table/block_based/data_block_hash_index.h" + +#include <string> +#include <vector> + +#include "rocksdb/slice.h" +#include "util/coding.h" +#include "util/hash.h" + +namespace ROCKSDB_NAMESPACE { + +void DataBlockHashIndexBuilder::Add(const Slice& key, + const size_t restart_index) { + assert(Valid()); + if (restart_index > kMaxRestartSupportedByHashIndex) { + valid_ = false; + return; + } + + uint32_t hash_value = GetSliceHash(key); + hash_and_restart_pairs_.emplace_back(hash_value, + static_cast<uint8_t>(restart_index)); + estimated_num_buckets_ += bucket_per_key_; +} + +void DataBlockHashIndexBuilder::Finish(std::string& buffer) { + assert(Valid()); + uint16_t num_buckets = static_cast<uint16_t>(estimated_num_buckets_); + + if (num_buckets == 0) { + num_buckets = 1; // sanity check + } + + // The build-in hash cannot well distribute strings when into different + // buckets when num_buckets is power of two, resulting in high hash + // collision. + // We made the num_buckets to be odd to avoid this issue. + num_buckets |= 1; + + std::vector<uint8_t> buckets(num_buckets, kNoEntry); + // write the restart_index array + for (auto& entry : hash_and_restart_pairs_) { + uint32_t hash_value = entry.first; + uint8_t restart_index = entry.second; + uint16_t buck_idx = static_cast<uint16_t>(hash_value % num_buckets); + if (buckets[buck_idx] == kNoEntry) { + buckets[buck_idx] = restart_index; + } else if (buckets[buck_idx] != restart_index) { + // same bucket cannot store two different restart_index, mark collision + buckets[buck_idx] = kCollision; + } + } + + for (uint8_t restart_index : buckets) { + buffer.append( + const_cast<const char*>(reinterpret_cast<char*>(&restart_index)), + sizeof(restart_index)); + } + + // write NUM_BUCK + PutFixed16(&buffer, num_buckets); + + assert(buffer.size() <= kMaxBlockSizeSupportedByHashIndex); +} + +void DataBlockHashIndexBuilder::Reset() { + estimated_num_buckets_ = 0; + valid_ = true; + hash_and_restart_pairs_.clear(); +} + +void DataBlockHashIndex::Initialize(const char* data, uint16_t size, + uint16_t* map_offset) { + assert(size >= sizeof(uint16_t)); // NUM_BUCKETS + num_buckets_ = DecodeFixed16(data + size - sizeof(uint16_t)); + assert(num_buckets_ > 0); + assert(size > num_buckets_ * sizeof(uint8_t)); + *map_offset = static_cast<uint16_t>(size - sizeof(uint16_t) - + num_buckets_ * sizeof(uint8_t)); +} + +uint8_t DataBlockHashIndex::Lookup(const char* data, uint32_t map_offset, + const Slice& key) const { + uint32_t hash_value = GetSliceHash(key); + uint16_t idx = static_cast<uint16_t>(hash_value % num_buckets_); + const char* bucket_table = data + map_offset; + return static_cast<uint8_t>(*(bucket_table + idx * sizeof(uint8_t))); +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/data_block_hash_index.h b/src/rocksdb/table/block_based/data_block_hash_index.h new file mode 100644 index 000000000..321522175 --- /dev/null +++ b/src/rocksdb/table/block_based/data_block_hash_index.h @@ -0,0 +1,137 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#pragma once + +#include <cstdint> +#include <string> +#include <vector> + +#include "rocksdb/slice.h" + +namespace ROCKSDB_NAMESPACE { +// This is an experimental feature aiming to reduce the CPU utilization of +// point-lookup within a data-block. It is only used in data blocks, and not +// in meta-data blocks or per-table index blocks. +// +// It only used to support BlockBasedTable::Get(). +// +// A serialized hash index is appended to the data-block. The new block data +// format is as follows: +// +// DATA_BLOCK: [RI RI RI ... RI RI_IDX HASH_IDX FOOTER] +// +// RI: Restart Interval (the same as the default data-block format) +// RI_IDX: Restart Interval index (the same as the default data-block format) +// HASH_IDX: The new data-block hash index feature. +// FOOTER: A 32bit block footer, which is the NUM_RESTARTS with the MSB as +// the flag indicating if this hash index is in use. Note that +// given a data block < 32KB, the MSB is never used. So we can +// borrow the MSB as the hash index flag. Therefore, this format is +// compatible with the legacy data-blocks with num_restarts < 32768, +// as the MSB is 0. +// +// The format of the data-block hash index is as follows: +// +// HASH_IDX: [B B B ... B NUM_BUCK] +// +// B: bucket, an array of restart index. Each buckets is uint8_t. +// NUM_BUCK: Number of buckets, which is the length of the bucket array. +// +// We reserve two special flag: +// kNoEntry=255, +// kCollision=254. +// +// Therefore, the max number of restarts this hash index can supoport is 253. +// +// Buckets are initialized to be kNoEntry. +// +// When storing a key in the hash index, the key is first hashed to a bucket. +// If there the bucket is empty (kNoEntry), the restart index is stored in +// the bucket. If there is already a restart index there, we will update the +// existing restart index to a collision marker (kCollision). If the +// the bucket is already marked as collision, we do not store the restart +// index either. +// +// During query process, a key is first hashed to a bucket. Then we examine if +// the buckets store nothing (kNoEntry) or the bucket had a collision +// (kCollision). If either of those happens, we get the restart index of +// the key and will directly go to the restart interval to search the key. +// +// Note that we only support blocks with #restart_interval < 254. If a block +// has more restart interval than that, hash index will not be create for it. + +const uint8_t kNoEntry = 255; +const uint8_t kCollision = 254; +const uint8_t kMaxRestartSupportedByHashIndex = 253; + +// Because we use uint16_t address, we only support block no more than 64KB +const size_t kMaxBlockSizeSupportedByHashIndex = 1u << 16; +const double kDefaultUtilRatio = 0.75; + +class DataBlockHashIndexBuilder { + public: + DataBlockHashIndexBuilder() + : bucket_per_key_(-1 /*uninitialized marker*/), + estimated_num_buckets_(0), + valid_(false) {} + + void Initialize(double util_ratio) { + if (util_ratio <= 0) { + util_ratio = kDefaultUtilRatio; // sanity check + } + bucket_per_key_ = 1 / util_ratio; + valid_ = true; + } + + inline bool Valid() const { return valid_ && bucket_per_key_ > 0; } + void Add(const Slice& key, const size_t restart_index); + void Finish(std::string& buffer); + void Reset(); + inline size_t EstimateSize() const { + uint16_t estimated_num_buckets = + static_cast<uint16_t>(estimated_num_buckets_); + + // Maching the num_buckets number in DataBlockHashIndexBuilder::Finish. + estimated_num_buckets |= 1; + + return sizeof(uint16_t) + + static_cast<size_t>(estimated_num_buckets * sizeof(uint8_t)); + } + + private: + double bucket_per_key_; // is the multiplicative inverse of util_ratio_ + double estimated_num_buckets_; + + // Now the only usage for `valid_` is to mark false when the inserted + // restart_index is larger than supported. In this case HashIndex is not + // appended to the block content. + bool valid_; + + std::vector<std::pair<uint32_t, uint8_t>> hash_and_restart_pairs_; + friend class DataBlockHashIndex_DataBlockHashTestSmall_Test; +}; + +class DataBlockHashIndex { + public: + DataBlockHashIndex() : num_buckets_(0) {} + + void Initialize(const char* data, uint16_t size, uint16_t* map_offset); + + uint8_t Lookup(const char* data, uint32_t map_offset, const Slice& key) const; + + inline bool Valid() { return num_buckets_ != 0; } + + private: + // To make the serialized hash index compact and to save the space overhead, + // here all the data fields persisted in the block are in uint16 format. + // We find that a uint16 is large enough to index every offset of a 64KiB + // block. + // So in other words, DataBlockHashIndex does not support block size equal + // or greater then 64KiB. + uint16_t num_buckets_; +}; + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/data_block_hash_index_test.cc b/src/rocksdb/table/block_based/data_block_hash_index_test.cc new file mode 100644 index 000000000..cd2e30833 --- /dev/null +++ b/src/rocksdb/table/block_based/data_block_hash_index_test.cc @@ -0,0 +1,717 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#include "table/block_based/data_block_hash_index.h" + +#include <cstdlib> +#include <string> +#include <unordered_map> + +#include "db/table_properties_collector.h" +#include "rocksdb/slice.h" +#include "table/block_based/block.h" +#include "table/block_based/block_based_table_reader.h" +#include "table/block_based/block_builder.h" +#include "table/get_context.h" +#include "table/table_builder.h" +#include "test_util/testharness.h" +#include "test_util/testutil.h" +#include "util/random.h" + +namespace ROCKSDB_NAMESPACE { + +bool SearchForOffset(DataBlockHashIndex& index, const char* data, + uint16_t map_offset, const Slice& key, + uint8_t& restart_point) { + uint8_t entry = index.Lookup(data, map_offset, key); + if (entry == kCollision) { + return true; + } + + if (entry == kNoEntry) { + return false; + } + + return entry == restart_point; +} + +std::string GenerateKey(int primary_key, int secondary_key, int padding_size, + Random* rnd) { + char buf[50]; + char* p = &buf[0]; + snprintf(buf, sizeof(buf), "%6d%4d", primary_key, secondary_key); + std::string k(p); + if (padding_size) { + k += rnd->RandomString(padding_size); + } + + return k; +} + +// Generate random key value pairs. +// The generated key will be sorted. You can tune the parameters to generated +// different kinds of test key/value pairs for different scenario. +void GenerateRandomKVs(std::vector<std::string>* keys, + std::vector<std::string>* values, const int from, + const int len, const int step = 1, + const int padding_size = 0, + const int keys_share_prefix = 1) { + Random rnd(302); + + // generate different prefix + for (int i = from; i < from + len; i += step) { + // generating keys that shares the prefix + for (int j = 0; j < keys_share_prefix; ++j) { + keys->emplace_back(GenerateKey(i, j, padding_size, &rnd)); + + // 100 bytes values + values->emplace_back(rnd.RandomString(100)); + } + } +} + +TEST(DataBlockHashIndex, DataBlockHashTestSmall) { + DataBlockHashIndexBuilder builder; + builder.Initialize(0.75 /*util_ratio*/); + for (int j = 0; j < 5; j++) { + for (uint8_t i = 0; i < 2 + j; i++) { + std::string key("key" + std::to_string(i)); + uint8_t restart_point = i; + builder.Add(key, restart_point); + } + + size_t estimated_size = builder.EstimateSize(); + + std::string buffer("fake"), buffer2; + size_t original_size = buffer.size(); + estimated_size += original_size; + builder.Finish(buffer); + + ASSERT_EQ(buffer.size(), estimated_size); + + buffer2 = buffer; // test for the correctness of relative offset + + Slice s(buffer2); + DataBlockHashIndex index; + uint16_t map_offset; + index.Initialize(s.data(), static_cast<uint16_t>(s.size()), &map_offset); + + // the additional hash map should start at the end of the buffer + ASSERT_EQ(original_size, map_offset); + for (uint8_t i = 0; i < 2; i++) { + std::string key("key" + std::to_string(i)); + uint8_t restart_point = i; + ASSERT_TRUE( + SearchForOffset(index, s.data(), map_offset, key, restart_point)); + } + builder.Reset(); + } +} + +TEST(DataBlockHashIndex, DataBlockHashTest) { + // bucket_num = 200, #keys = 100. 50% utilization + DataBlockHashIndexBuilder builder; + builder.Initialize(0.75 /*util_ratio*/); + + for (uint8_t i = 0; i < 100; i++) { + std::string key("key" + std::to_string(i)); + uint8_t restart_point = i; + builder.Add(key, restart_point); + } + + size_t estimated_size = builder.EstimateSize(); + + std::string buffer("fake content"), buffer2; + size_t original_size = buffer.size(); + estimated_size += original_size; + builder.Finish(buffer); + + ASSERT_EQ(buffer.size(), estimated_size); + + buffer2 = buffer; // test for the correctness of relative offset + + Slice s(buffer2); + DataBlockHashIndex index; + uint16_t map_offset; + index.Initialize(s.data(), static_cast<uint16_t>(s.size()), &map_offset); + + // the additional hash map should start at the end of the buffer + ASSERT_EQ(original_size, map_offset); + for (uint8_t i = 0; i < 100; i++) { + std::string key("key" + std::to_string(i)); + uint8_t restart_point = i; + ASSERT_TRUE( + SearchForOffset(index, s.data(), map_offset, key, restart_point)); + } +} + +TEST(DataBlockHashIndex, DataBlockHashTestCollision) { + // bucket_num = 2. There will be intense hash collisions + DataBlockHashIndexBuilder builder; + builder.Initialize(0.75 /*util_ratio*/); + + for (uint8_t i = 0; i < 100; i++) { + std::string key("key" + std::to_string(i)); + uint8_t restart_point = i; + builder.Add(key, restart_point); + } + + size_t estimated_size = builder.EstimateSize(); + + std::string buffer("some other fake content to take up space"), buffer2; + size_t original_size = buffer.size(); + estimated_size += original_size; + builder.Finish(buffer); + + ASSERT_EQ(buffer.size(), estimated_size); + + buffer2 = buffer; // test for the correctness of relative offset + + Slice s(buffer2); + DataBlockHashIndex index; + uint16_t map_offset; + index.Initialize(s.data(), static_cast<uint16_t>(s.size()), &map_offset); + + // the additional hash map should start at the end of the buffer + ASSERT_EQ(original_size, map_offset); + for (uint8_t i = 0; i < 100; i++) { + std::string key("key" + std::to_string(i)); + uint8_t restart_point = i; + ASSERT_TRUE( + SearchForOffset(index, s.data(), map_offset, key, restart_point)); + } +} + +TEST(DataBlockHashIndex, DataBlockHashTestLarge) { + DataBlockHashIndexBuilder builder; + builder.Initialize(0.75 /*util_ratio*/); + std::unordered_map<std::string, uint8_t> m; + + for (uint8_t i = 0; i < 100; i++) { + if (i % 2) { + continue; // leave half of the keys out + } + std::string key = "key" + std::to_string(i); + uint8_t restart_point = i; + builder.Add(key, restart_point); + m[key] = restart_point; + } + + size_t estimated_size = builder.EstimateSize(); + + std::string buffer("filling stuff"), buffer2; + size_t original_size = buffer.size(); + estimated_size += original_size; + builder.Finish(buffer); + + ASSERT_EQ(buffer.size(), estimated_size); + + buffer2 = buffer; // test for the correctness of relative offset + + Slice s(buffer2); + DataBlockHashIndex index; + uint16_t map_offset; + index.Initialize(s.data(), static_cast<uint16_t>(s.size()), &map_offset); + + // the additional hash map should start at the end of the buffer + ASSERT_EQ(original_size, map_offset); + for (uint8_t i = 0; i < 100; i++) { + std::string key = "key" + std::to_string(i); + uint8_t restart_point = i; + if (m.count(key)) { + ASSERT_TRUE(m[key] == restart_point); + ASSERT_TRUE( + SearchForOffset(index, s.data(), map_offset, key, restart_point)); + } else { + // we allow false positve, so don't test the nonexisting keys. + // when false positive happens, the search will continue to the + // restart intervals to see if the key really exist. + } + } +} + +TEST(DataBlockHashIndex, RestartIndexExceedMax) { + DataBlockHashIndexBuilder builder; + builder.Initialize(0.75 /*util_ratio*/); + std::unordered_map<std::string, uint8_t> m; + + for (uint8_t i = 0; i <= 253; i++) { + std::string key = "key" + std::to_string(i); + uint8_t restart_point = i; + builder.Add(key, restart_point); + } + ASSERT_TRUE(builder.Valid()); + + builder.Reset(); + + for (uint8_t i = 0; i <= 254; i++) { + std::string key = "key" + std::to_string(i); + uint8_t restart_point = i; + builder.Add(key, restart_point); + } + + ASSERT_FALSE(builder.Valid()); + + builder.Reset(); + ASSERT_TRUE(builder.Valid()); +} + +TEST(DataBlockHashIndex, BlockRestartIndexExceedMax) { + Options options = Options(); + + BlockBuilder builder(1 /* block_restart_interval */, + true /* use_delta_encoding */, + false /* use_value_delta_encoding */, + BlockBasedTableOptions::kDataBlockBinaryAndHash); + + // #restarts <= 253. HashIndex is valid + for (int i = 0; i <= 253; i++) { + std::string ukey = "key" + std::to_string(i); + InternalKey ikey(ukey, 0, kTypeValue); + builder.Add(ikey.Encode().ToString(), "value"); + } + + { + // read serialized contents of the block + Slice rawblock = builder.Finish(); + + // create block reader + BlockContents contents; + contents.data = rawblock; + Block reader(std::move(contents)); + + ASSERT_EQ(reader.IndexType(), + BlockBasedTableOptions::kDataBlockBinaryAndHash); + } + + builder.Reset(); + + // #restarts > 253. HashIndex is not used + for (int i = 0; i <= 254; i++) { + std::string ukey = "key" + std::to_string(i); + InternalKey ikey(ukey, 0, kTypeValue); + builder.Add(ikey.Encode().ToString(), "value"); + } + + { + // read serialized contents of the block + Slice rawblock = builder.Finish(); + + // create block reader + BlockContents contents; + contents.data = rawblock; + Block reader(std::move(contents)); + + ASSERT_EQ(reader.IndexType(), + BlockBasedTableOptions::kDataBlockBinarySearch); + } +} + +TEST(DataBlockHashIndex, BlockSizeExceedMax) { + Options options = Options(); + std::string ukey(10, 'k'); + InternalKey ikey(ukey, 0, kTypeValue); + + BlockBuilder builder(1 /* block_restart_interval */, + false /* use_delta_encoding */, + false /* use_value_delta_encoding */, + BlockBasedTableOptions::kDataBlockBinaryAndHash); + + { + // insert a large value. The block size plus HashIndex is 65536. + std::string value(65502, 'v'); + + builder.Add(ikey.Encode().ToString(), value); + + // read serialized contents of the block + Slice rawblock = builder.Finish(); + ASSERT_LE(rawblock.size(), kMaxBlockSizeSupportedByHashIndex); + std::cerr << "block size: " << rawblock.size() << std::endl; + + // create block reader + BlockContents contents; + contents.data = rawblock; + Block reader(std::move(contents)); + + ASSERT_EQ(reader.IndexType(), + BlockBasedTableOptions::kDataBlockBinaryAndHash); + } + + builder.Reset(); + + { + // insert a large value. The block size plus HashIndex would be 65537. + // This excceed the max block size supported by HashIndex (65536). + // So when build finishes HashIndex will not be created for the block. + std::string value(65503, 'v'); + + builder.Add(ikey.Encode().ToString(), value); + + // read serialized contents of the block + Slice rawblock = builder.Finish(); + ASSERT_LE(rawblock.size(), kMaxBlockSizeSupportedByHashIndex); + std::cerr << "block size: " << rawblock.size() << std::endl; + + // create block reader + BlockContents contents; + contents.data = rawblock; + Block reader(std::move(contents)); + + // the index type have fallen back to binary when build finish. + ASSERT_EQ(reader.IndexType(), + BlockBasedTableOptions::kDataBlockBinarySearch); + } +} + +TEST(DataBlockHashIndex, BlockTestSingleKey) { + Options options = Options(); + + BlockBuilder builder(16 /* block_restart_interval */, + true /* use_delta_encoding */, + false /* use_value_delta_encoding */, + BlockBasedTableOptions::kDataBlockBinaryAndHash); + + std::string ukey("gopher"); + std::string value("gold"); + InternalKey ikey(ukey, 10, kTypeValue); + builder.Add(ikey.Encode().ToString(), value /*value*/); + + // read serialized contents of the block + Slice rawblock = builder.Finish(); + + // create block reader + BlockContents contents; + contents.data = rawblock; + Block reader(std::move(contents)); + + const InternalKeyComparator icmp(BytewiseComparator()); + auto iter = reader.NewDataIterator(icmp.user_comparator(), + kDisableGlobalSequenceNumber); + bool may_exist; + // search in block for the key just inserted + { + InternalKey seek_ikey(ukey, 10, kValueTypeForSeek); + may_exist = iter->SeekForGet(seek_ikey.Encode().ToString()); + ASSERT_TRUE(may_exist); + ASSERT_TRUE(iter->Valid()); + ASSERT_EQ( + options.comparator->Compare(iter->key(), ikey.Encode().ToString()), 0); + ASSERT_EQ(iter->value(), value); + } + + // search in block for the existing ukey, but with higher seqno + { + InternalKey seek_ikey(ukey, 20, kValueTypeForSeek); + + // HashIndex should be able to set the iter correctly + may_exist = iter->SeekForGet(seek_ikey.Encode().ToString()); + ASSERT_TRUE(may_exist); + ASSERT_TRUE(iter->Valid()); + + // user key should match + ASSERT_EQ(options.comparator->Compare(ExtractUserKey(iter->key()), ukey), + 0); + + // seek_key seqno number should be greater than that of iter result + ASSERT_GT(GetInternalKeySeqno(seek_ikey.Encode()), + GetInternalKeySeqno(iter->key())); + + ASSERT_EQ(iter->value(), value); + } + + // Search in block for the existing ukey, but with lower seqno + // in this case, hash can find the only occurrence of the user_key, but + // ParseNextDataKey() will skip it as it does not have a older seqno. + // In this case, GetForSeek() is effective to locate the user_key, and + // iter->Valid() == false indicates that we've reached to the end of + // the block and the caller should continue searching the next block. + { + InternalKey seek_ikey(ukey, 5, kValueTypeForSeek); + may_exist = iter->SeekForGet(seek_ikey.Encode().ToString()); + ASSERT_TRUE(may_exist); + ASSERT_FALSE(iter->Valid()); // should have reached to the end of block + } + + delete iter; +} + +TEST(DataBlockHashIndex, BlockTestLarge) { + Random rnd(1019); + Options options = Options(); + std::vector<std::string> keys; + std::vector<std::string> values; + + BlockBuilder builder(16 /* block_restart_interval */, + true /* use_delta_encoding */, + false /* use_value_delta_encoding */, + BlockBasedTableOptions::kDataBlockBinaryAndHash); + int num_records = 500; + + GenerateRandomKVs(&keys, &values, 0, num_records); + + // Generate keys. Adding a trailing "1" to indicate existent keys. + // Later will Seeking for keys with a trailing "0" to test seeking + // non-existent keys. + for (int i = 0; i < num_records; i++) { + std::string ukey(keys[i] + "1" /* existing key marker */); + InternalKey ikey(ukey, 0, kTypeValue); + builder.Add(ikey.Encode().ToString(), values[i]); + } + + // read serialized contents of the block + Slice rawblock = builder.Finish(); + + // create block reader + BlockContents contents; + contents.data = rawblock; + Block reader(std::move(contents)); + const InternalKeyComparator icmp(BytewiseComparator()); + + // random seek existent keys + for (int i = 0; i < num_records; i++) { + auto iter = reader.NewDataIterator(icmp.user_comparator(), + kDisableGlobalSequenceNumber); + // find a random key in the lookaside array + int index = rnd.Uniform(num_records); + std::string ukey(keys[index] + "1" /* existing key marker */); + InternalKey ikey(ukey, 0, kTypeValue); + + // search in block for this key + bool may_exist = iter->SeekForGet(ikey.Encode().ToString()); + ASSERT_TRUE(may_exist); + ASSERT_TRUE(iter->Valid()); + ASSERT_EQ(values[index], iter->value()); + + delete iter; + } + + // random seek non-existent user keys + // In this case A), the user_key cannot be found in HashIndex. The key may + // exist in the next block. So the iter is set invalidated to tell the + // caller to search the next block. This test case belongs to this case A). + // + // Note that for non-existent keys, there is possibility of false positive, + // i.e. the key is still hashed into some restart interval. + // Two additional possible outcome: + // B) linear seek the restart interval and not found, the iter stops at the + // starting of the next restart interval. The key does not exist + // anywhere. + // C) linear seek the restart interval and not found, the iter stops at the + // the end of the block, i.e. restarts_. The key may exist in the next + // block. + // So these combinations are possible when searching non-existent user_key: + // + // case# may_exist iter->Valid() + // A true false + // B false true + // C true false + + for (int i = 0; i < num_records; i++) { + auto iter = reader.NewDataIterator(icmp.user_comparator(), + kDisableGlobalSequenceNumber); + // find a random key in the lookaside array + int index = rnd.Uniform(num_records); + std::string ukey(keys[index] + "0" /* non-existing key marker */); + InternalKey ikey(ukey, 0, kTypeValue); + + // search in block for this key + bool may_exist = iter->SeekForGet(ikey.Encode().ToString()); + if (!may_exist) { + ASSERT_TRUE(iter->Valid()); + } + if (!iter->Valid()) { + ASSERT_TRUE(may_exist); + } + + delete iter; + } +} + +// helper routine for DataBlockHashIndex.BlockBoundary +void TestBoundary(InternalKey& ik1, std::string& v1, InternalKey& ik2, + std::string& v2, InternalKey& seek_ikey, + GetContext& get_context, Options& options) { + std::unique_ptr<WritableFileWriter> file_writer; + std::unique_ptr<RandomAccessFileReader> file_reader; + std::unique_ptr<TableReader> table_reader; + int level_ = -1; + + std::vector<std::string> keys; + const ImmutableOptions ioptions(options); + const MutableCFOptions moptions(options); + const InternalKeyComparator internal_comparator(options.comparator); + + EnvOptions soptions; + + soptions.use_mmap_reads = ioptions.allow_mmap_reads; + test::StringSink* sink = new test::StringSink(); + std::unique_ptr<FSWritableFile> f(sink); + file_writer.reset( + new WritableFileWriter(std::move(f), "" /* don't care */, FileOptions())); + std::unique_ptr<TableBuilder> builder; + IntTblPropCollectorFactories int_tbl_prop_collector_factories; + std::string column_family_name; + builder.reset(ioptions.table_factory->NewTableBuilder( + TableBuilderOptions( + ioptions, moptions, internal_comparator, + &int_tbl_prop_collector_factories, options.compression, + CompressionOptions(), + TablePropertiesCollectorFactory::Context::kUnknownColumnFamily, + column_family_name, level_), + file_writer.get())); + + builder->Add(ik1.Encode().ToString(), v1); + builder->Add(ik2.Encode().ToString(), v2); + EXPECT_TRUE(builder->status().ok()); + + Status s = builder->Finish(); + ASSERT_OK(file_writer->Flush()); + EXPECT_TRUE(s.ok()) << s.ToString(); + + EXPECT_EQ(sink->contents().size(), builder->FileSize()); + + // Open the table + test::StringSource* source = new test::StringSource( + sink->contents(), 0 /*uniq_id*/, ioptions.allow_mmap_reads); + std::unique_ptr<FSRandomAccessFile> file(source); + file_reader.reset(new RandomAccessFileReader(std::move(file), "test")); + const bool kSkipFilters = true; + const bool kImmortal = true; + ASSERT_OK(ioptions.table_factory->NewTableReader( + TableReaderOptions(ioptions, moptions.prefix_extractor, soptions, + internal_comparator, !kSkipFilters, !kImmortal, + level_), + std::move(file_reader), sink->contents().size(), &table_reader)); + // Search using Get() + ReadOptions ro; + + ASSERT_OK(table_reader->Get(ro, seek_ikey.Encode().ToString(), &get_context, + moptions.prefix_extractor.get())); +} + +TEST(DataBlockHashIndex, BlockBoundary) { + BlockBasedTableOptions table_options; + table_options.data_block_index_type = + BlockBasedTableOptions::kDataBlockBinaryAndHash; + table_options.block_restart_interval = 1; + table_options.block_size = 4096; + + Options options; + options.comparator = BytewiseComparator(); + + options.table_factory.reset(NewBlockBasedTableFactory(table_options)); + + // insert two large k/v pair. Given that the block_size is 4096, one k/v + // pair will take up one block. + // [ k1/v1 ][ k2/v2 ] + // [ Block N ][ Block N+1 ] + + { + // [ "aab"@100 ][ "axy"@10 ] + // | Block N ][ Block N+1 ] + // seek for "axy"@60 + std::string uk1("aab"); + InternalKey ik1(uk1, 100, kTypeValue); + std::string v1(4100, '1'); // large value + + std::string uk2("axy"); + InternalKey ik2(uk2, 10, kTypeValue); + std::string v2(4100, '2'); // large value + + PinnableSlice value; + std::string seek_ukey("axy"); + InternalKey seek_ikey(seek_ukey, 60, kTypeValue); + GetContext get_context(options.comparator, nullptr, nullptr, nullptr, + GetContext::kNotFound, seek_ukey, &value, nullptr, + nullptr, nullptr, true, nullptr, nullptr); + + TestBoundary(ik1, v1, ik2, v2, seek_ikey, get_context, options); + ASSERT_EQ(get_context.State(), GetContext::kFound); + ASSERT_EQ(value, v2); + value.Reset(); + } + + { + // [ "axy"@100 ][ "axy"@10 ] + // | Block N ][ Block N+1 ] + // seek for "axy"@60 + std::string uk1("axy"); + InternalKey ik1(uk1, 100, kTypeValue); + std::string v1(4100, '1'); // large value + + std::string uk2("axy"); + InternalKey ik2(uk2, 10, kTypeValue); + std::string v2(4100, '2'); // large value + + PinnableSlice value; + std::string seek_ukey("axy"); + InternalKey seek_ikey(seek_ukey, 60, kTypeValue); + GetContext get_context(options.comparator, nullptr, nullptr, nullptr, + GetContext::kNotFound, seek_ukey, &value, nullptr, + nullptr, nullptr, true, nullptr, nullptr); + + TestBoundary(ik1, v1, ik2, v2, seek_ikey, get_context, options); + ASSERT_EQ(get_context.State(), GetContext::kFound); + ASSERT_EQ(value, v2); + value.Reset(); + } + + { + // [ "axy"@100 ][ "axy"@10 ] + // | Block N ][ Block N+1 ] + // seek for "axy"@120 + std::string uk1("axy"); + InternalKey ik1(uk1, 100, kTypeValue); + std::string v1(4100, '1'); // large value + + std::string uk2("axy"); + InternalKey ik2(uk2, 10, kTypeValue); + std::string v2(4100, '2'); // large value + + PinnableSlice value; + std::string seek_ukey("axy"); + InternalKey seek_ikey(seek_ukey, 120, kTypeValue); + GetContext get_context(options.comparator, nullptr, nullptr, nullptr, + GetContext::kNotFound, seek_ukey, &value, nullptr, + nullptr, nullptr, true, nullptr, nullptr); + + TestBoundary(ik1, v1, ik2, v2, seek_ikey, get_context, options); + ASSERT_EQ(get_context.State(), GetContext::kFound); + ASSERT_EQ(value, v1); + value.Reset(); + } + + { + // [ "axy"@100 ][ "axy"@10 ] + // | Block N ][ Block N+1 ] + // seek for "axy"@5 + std::string uk1("axy"); + InternalKey ik1(uk1, 100, kTypeValue); + std::string v1(4100, '1'); // large value + + std::string uk2("axy"); + InternalKey ik2(uk2, 10, kTypeValue); + std::string v2(4100, '2'); // large value + + PinnableSlice value; + std::string seek_ukey("axy"); + InternalKey seek_ikey(seek_ukey, 5, kTypeValue); + GetContext get_context(options.comparator, nullptr, nullptr, nullptr, + GetContext::kNotFound, seek_ukey, &value, nullptr, + nullptr, nullptr, true, nullptr, nullptr); + + TestBoundary(ik1, v1, ik2, v2, seek_ikey, get_context, options); + ASSERT_EQ(get_context.State(), GetContext::kNotFound); + value.Reset(); + } +} + +} // namespace ROCKSDB_NAMESPACE + +int main(int argc, char** argv) { + ROCKSDB_NAMESPACE::port::InstallStackTraceHandler(); + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/src/rocksdb/table/block_based/filter_block.h b/src/rocksdb/table/block_based/filter_block.h new file mode 100644 index 000000000..e1e206990 --- /dev/null +++ b/src/rocksdb/table/block_based/filter_block.h @@ -0,0 +1,182 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// +// Copyright (c) 2012 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. +// +// A filter block is stored near the end of a Table file. It contains +// filters (e.g., bloom filters) for all data blocks in the table combined +// into a single filter block. + +#pragma once + +#include <stddef.h> +#include <stdint.h> + +#include <memory> +#include <string> +#include <vector> + +#include "rocksdb/options.h" +#include "rocksdb/slice.h" +#include "rocksdb/slice_transform.h" +#include "rocksdb/table.h" +#include "table/format.h" +#include "table/multiget_context.h" +#include "trace_replay/block_cache_tracer.h" +#include "util/hash.h" + +namespace ROCKSDB_NAMESPACE { + +const uint64_t kNotValid = ULLONG_MAX; +class FilterPolicy; + +class GetContext; +using MultiGetRange = MultiGetContext::Range; + +// A FilterBlockBuilder is used to construct all of the filters for a +// particular Table. It generates a single string which is stored as +// a special block in the Table, or partitioned into smaller filters. +// +// The sequence of calls to FilterBlockBuilder must match the regexp: +// Add* Finish +class FilterBlockBuilder { + public: + explicit FilterBlockBuilder() {} + // No copying allowed + FilterBlockBuilder(const FilterBlockBuilder&) = delete; + void operator=(const FilterBlockBuilder&) = delete; + + virtual ~FilterBlockBuilder() {} + + virtual void Add( + const Slice& key_without_ts) = 0; // Add a key to current filter + virtual bool IsEmpty() const = 0; // Empty == none added + // For reporting stats on how many entries the builder considered unique + virtual size_t EstimateEntriesAdded() = 0; + Slice Finish() { // Generate Filter + const BlockHandle empty_handle; + Status dont_care_status; + auto ret = Finish(empty_handle, &dont_care_status); + assert(dont_care_status.ok()); + return ret; + } + // If filter_data is not nullptr, Finish() may transfer ownership of + // underlying filter data to the caller, so that it can be freed as soon as + // possible. BlockBasedFilterBlock will ignore this parameter. + // + virtual Slice Finish( + const BlockHandle& tmp /* only used in PartitionedFilterBlock as + last_partition_block_handle */ + , + Status* status, std::unique_ptr<const char[]>* filter_data = nullptr) = 0; + + // This is called when finishes using the FilterBitsBuilder + // in order to release memory usage and cache charge + // associated with it timely + virtual void ResetFilterBitsBuilder() {} + + // To optionally post-verify the filter returned from + // FilterBlockBuilder::Finish. + // Return Status::OK() if skipped. + virtual Status MaybePostVerifyFilter(const Slice& /* filter_content */) { + return Status::OK(); + } +}; + +// A FilterBlockReader is used to parse filter from SST table. +// KeyMayMatch and PrefixMayMatch would trigger filter checking +// +// BlockBased/Full FilterBlock would be called in the same way. +class FilterBlockReader { + public: + FilterBlockReader() = default; + virtual ~FilterBlockReader() = default; + + FilterBlockReader(const FilterBlockReader&) = delete; + FilterBlockReader& operator=(const FilterBlockReader&) = delete; + + /** + * If no_io is set, then it returns true if it cannot answer the query without + * reading data from disk. This is used in PartitionedFilterBlockReader to + * avoid reading partitions that are not in block cache already + * + * Normally filters are built on only the user keys and the InternalKey is not + * needed for a query. The index in PartitionedFilterBlockReader however is + * built upon InternalKey and must be provided via const_ikey_ptr when running + * queries. + */ + virtual bool KeyMayMatch(const Slice& key, const bool no_io, + const Slice* const const_ikey_ptr, + GetContext* get_context, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) = 0; + + virtual void KeysMayMatch(MultiGetRange* range, const bool no_io, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) { + for (auto iter = range->begin(); iter != range->end(); ++iter) { + const Slice ukey_without_ts = iter->ukey_without_ts; + const Slice ikey = iter->ikey; + GetContext* const get_context = iter->get_context; + if (!KeyMayMatch(ukey_without_ts, no_io, &ikey, get_context, + lookup_context, rate_limiter_priority)) { + range->SkipKey(iter); + } + } + } + + /** + * no_io and const_ikey_ptr here means the same as in KeyMayMatch + */ + virtual bool PrefixMayMatch(const Slice& prefix, const bool no_io, + const Slice* const const_ikey_ptr, + GetContext* get_context, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) = 0; + + virtual void PrefixesMayMatch(MultiGetRange* range, + const SliceTransform* prefix_extractor, + const bool no_io, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) { + for (auto iter = range->begin(); iter != range->end(); ++iter) { + const Slice ukey_without_ts = iter->ukey_without_ts; + const Slice ikey = iter->ikey; + GetContext* const get_context = iter->get_context; + if (prefix_extractor->InDomain(ukey_without_ts) && + !PrefixMayMatch(prefix_extractor->Transform(ukey_without_ts), no_io, + &ikey, get_context, lookup_context, + rate_limiter_priority)) { + range->SkipKey(iter); + } + } + } + + virtual size_t ApproximateMemoryUsage() const = 0; + + // convert this object to a human readable form + virtual std::string ToString() const { + std::string error_msg("Unsupported filter \n"); + return error_msg; + } + + virtual Status CacheDependencies(const ReadOptions& /*ro*/, bool /*pin*/) { + return Status::OK(); + } + + virtual bool RangeMayExist(const Slice* /*iterate_upper_bound*/, + const Slice& user_key_without_ts, + const SliceTransform* prefix_extractor, + const Comparator* /*comparator*/, + const Slice* const const_ikey_ptr, + bool* filter_checked, bool need_upper_bound_check, + bool no_io, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) = 0; +}; + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/filter_block_reader_common.cc b/src/rocksdb/table/block_based/filter_block_reader_common.cc new file mode 100644 index 000000000..7dc49e83e --- /dev/null +++ b/src/rocksdb/table/block_based/filter_block_reader_common.cc @@ -0,0 +1,164 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// + +#include "table/block_based/filter_block_reader_common.h" + +#include "monitoring/perf_context_imp.h" +#include "table/block_based/block_based_table_reader.h" +#include "table/block_based/parsed_full_filter_block.h" + +namespace ROCKSDB_NAMESPACE { + +template <typename TBlocklike> +Status FilterBlockReaderCommon<TBlocklike>::ReadFilterBlock( + const BlockBasedTable* table, FilePrefetchBuffer* prefetch_buffer, + const ReadOptions& read_options, bool use_cache, GetContext* get_context, + BlockCacheLookupContext* lookup_context, + CachableEntry<TBlocklike>* filter_block, BlockType block_type) { + PERF_TIMER_GUARD(read_filter_block_nanos); + + assert(table); + assert(filter_block); + assert(filter_block->IsEmpty()); + + const BlockBasedTable::Rep* const rep = table->get_rep(); + assert(rep); + + const Status s = + table->RetrieveBlock(prefetch_buffer, read_options, rep->filter_handle, + UncompressionDict::GetEmptyDict(), filter_block, + block_type, get_context, lookup_context, + /* for_compaction */ false, use_cache, + /* wait_for_cache */ true, /* async_read */ false); + + return s; +} + +template <typename TBlocklike> +const SliceTransform* +FilterBlockReaderCommon<TBlocklike>::table_prefix_extractor() const { + assert(table_); + + const BlockBasedTable::Rep* const rep = table_->get_rep(); + assert(rep); + + return rep->prefix_filtering ? rep->table_prefix_extractor.get() : nullptr; +} + +template <typename TBlocklike> +bool FilterBlockReaderCommon<TBlocklike>::whole_key_filtering() const { + assert(table_); + assert(table_->get_rep()); + + return table_->get_rep()->whole_key_filtering; +} + +template <typename TBlocklike> +bool FilterBlockReaderCommon<TBlocklike>::cache_filter_blocks() const { + assert(table_); + assert(table_->get_rep()); + + return table_->get_rep()->table_options.cache_index_and_filter_blocks; +} + +template <typename TBlocklike> +Status FilterBlockReaderCommon<TBlocklike>::GetOrReadFilterBlock( + bool no_io, GetContext* get_context, + BlockCacheLookupContext* lookup_context, + CachableEntry<TBlocklike>* filter_block, BlockType block_type, + Env::IOPriority rate_limiter_priority) const { + assert(filter_block); + + if (!filter_block_.IsEmpty()) { + filter_block->SetUnownedValue(filter_block_.GetValue()); + return Status::OK(); + } + + ReadOptions read_options; + read_options.rate_limiter_priority = rate_limiter_priority; + if (no_io) { + read_options.read_tier = kBlockCacheTier; + } + + return ReadFilterBlock(table_, nullptr /* prefetch_buffer */, read_options, + cache_filter_blocks(), get_context, lookup_context, + filter_block, block_type); +} + +template <typename TBlocklike> +size_t FilterBlockReaderCommon<TBlocklike>::ApproximateFilterBlockMemoryUsage() + const { + assert(!filter_block_.GetOwnValue() || filter_block_.GetValue() != nullptr); + return filter_block_.GetOwnValue() + ? filter_block_.GetValue()->ApproximateMemoryUsage() + : 0; +} + +template <typename TBlocklike> +bool FilterBlockReaderCommon<TBlocklike>::RangeMayExist( + const Slice* iterate_upper_bound, const Slice& user_key_without_ts, + const SliceTransform* prefix_extractor, const Comparator* comparator, + const Slice* const const_ikey_ptr, bool* filter_checked, + bool need_upper_bound_check, bool no_io, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) { + if (!prefix_extractor || !prefix_extractor->InDomain(user_key_without_ts)) { + *filter_checked = false; + return true; + } + Slice prefix = prefix_extractor->Transform(user_key_without_ts); + if (need_upper_bound_check && + !IsFilterCompatible(iterate_upper_bound, prefix, comparator)) { + *filter_checked = false; + return true; + } else { + *filter_checked = true; + return PrefixMayMatch(prefix, no_io, const_ikey_ptr, + /* get_context */ nullptr, lookup_context, + rate_limiter_priority); + } +} + +template <typename TBlocklike> +bool FilterBlockReaderCommon<TBlocklike>::IsFilterCompatible( + const Slice* iterate_upper_bound, const Slice& prefix, + const Comparator* comparator) const { + // Try to reuse the bloom filter in the SST table if prefix_extractor in + // mutable_cf_options has changed. If range [user_key, upper_bound) all + // share the same prefix then we may still be able to use the bloom filter. + const SliceTransform* const prefix_extractor = table_prefix_extractor(); + if (iterate_upper_bound != nullptr && prefix_extractor) { + if (!prefix_extractor->InDomain(*iterate_upper_bound)) { + return false; + } + Slice upper_bound_xform = prefix_extractor->Transform(*iterate_upper_bound); + // first check if user_key and upper_bound all share the same prefix + if (comparator->CompareWithoutTimestamp(prefix, false, upper_bound_xform, + false) != 0) { + // second check if user_key's prefix is the immediate predecessor of + // upper_bound and have the same length. If so, we know for sure all + // keys in the range [user_key, upper_bound) share the same prefix. + // Also need to make sure upper_bound are full length to ensure + // correctness + if (!full_length_enabled_ || + iterate_upper_bound->size() != prefix_extractor_full_length_ || + !comparator->IsSameLengthImmediateSuccessor(prefix, + *iterate_upper_bound)) { + return false; + } + } + return true; + } else { + return false; + } +} + +// Explicitly instantiate templates for both "blocklike" types we use. +// This makes it possible to keep the template definitions in the .cc file. +template class FilterBlockReaderCommon<Block>; +template class FilterBlockReaderCommon<ParsedFullFilterBlock>; + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/filter_block_reader_common.h b/src/rocksdb/table/block_based/filter_block_reader_common.h new file mode 100644 index 000000000..ca07f5050 --- /dev/null +++ b/src/rocksdb/table/block_based/filter_block_reader_common.h @@ -0,0 +1,79 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// + +#pragma once + +#include <cassert> + +#include "block_type.h" +#include "table/block_based/cachable_entry.h" +#include "table/block_based/filter_block.h" + +namespace ROCKSDB_NAMESPACE { + +class BlockBasedTable; +class FilePrefetchBuffer; + +// Encapsulates common functionality for the various filter block reader +// implementations. Provides access to the filter block regardless of whether +// it is owned by the reader or stored in the cache, or whether it is pinned +// in the cache or not. +template <typename TBlocklike> +class FilterBlockReaderCommon : public FilterBlockReader { + public: + FilterBlockReaderCommon(const BlockBasedTable* t, + CachableEntry<TBlocklike>&& filter_block) + : table_(t), filter_block_(std::move(filter_block)) { + assert(table_); + const SliceTransform* const prefix_extractor = table_prefix_extractor(); + if (prefix_extractor) { + full_length_enabled_ = + prefix_extractor->FullLengthEnabled(&prefix_extractor_full_length_); + } + } + + bool RangeMayExist(const Slice* iterate_upper_bound, const Slice& user_key, + const SliceTransform* prefix_extractor, + const Comparator* comparator, + const Slice* const const_ikey_ptr, bool* filter_checked, + bool need_upper_bound_check, bool no_io, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) override; + + protected: + static Status ReadFilterBlock(const BlockBasedTable* table, + FilePrefetchBuffer* prefetch_buffer, + const ReadOptions& read_options, bool use_cache, + GetContext* get_context, + BlockCacheLookupContext* lookup_context, + CachableEntry<TBlocklike>* filter_block, + BlockType block_type); + + const BlockBasedTable* table() const { return table_; } + const SliceTransform* table_prefix_extractor() const; + bool whole_key_filtering() const; + bool cache_filter_blocks() const; + + Status GetOrReadFilterBlock(bool no_io, GetContext* get_context, + BlockCacheLookupContext* lookup_context, + CachableEntry<TBlocklike>* filter_block, + BlockType block_type, + Env::IOPriority rate_limiter_priority) const; + + size_t ApproximateFilterBlockMemoryUsage() const; + + private: + bool IsFilterCompatible(const Slice* iterate_upper_bound, const Slice& prefix, + const Comparator* comparator) const; + + private: + const BlockBasedTable* table_; + CachableEntry<TBlocklike> filter_block_; + size_t prefix_extractor_full_length_ = 0; + bool full_length_enabled_ = false; +}; + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/filter_policy.cc b/src/rocksdb/table/block_based/filter_policy.cc new file mode 100644 index 000000000..f84f804dd --- /dev/null +++ b/src/rocksdb/table/block_based/filter_policy.cc @@ -0,0 +1,1973 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// +// Copyright (c) 2012 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 "rocksdb/filter_policy.h" + +#include <array> +#include <climits> +#include <cstring> +#include <deque> +#include <limits> +#include <memory> + +#include "cache/cache_entry_roles.h" +#include "cache/cache_reservation_manager.h" +#include "logging/logging.h" +#include "port/lang.h" +#include "rocksdb/convenience.h" +#include "rocksdb/rocksdb_namespace.h" +#include "rocksdb/slice.h" +#include "rocksdb/utilities/object_registry.h" +#include "table/block_based/block_based_table_reader.h" +#include "table/block_based/filter_policy_internal.h" +#include "table/block_based/full_filter_block.h" +#include "util/bloom_impl.h" +#include "util/coding.h" +#include "util/hash.h" +#include "util/math.h" +#include "util/ribbon_config.h" +#include "util/ribbon_impl.h" +#include "util/string_util.h" + +namespace ROCKSDB_NAMESPACE { + +namespace { + +// Metadata trailer size for built-in filters. (This is separate from +// block-based table block trailer.) +// +// Originally this was 1 byte for num_probes and 4 bytes for number of +// cache lines in the Bloom filter, but now the first trailer byte is +// usually an implementation marker and remaining 4 bytes have various +// meanings. +static constexpr uint32_t kMetadataLen = 5; + +Slice FinishAlwaysFalse(std::unique_ptr<const char[]>* /*buf*/) { + // Missing metadata, treated as zero entries + return Slice(nullptr, 0); +} + +Slice FinishAlwaysTrue(std::unique_ptr<const char[]>* /*buf*/) { + return Slice("\0\0\0\0\0\0", 6); +} + +// Base class for filter builders using the XXH3 preview hash, +// also known as Hash64 or GetSliceHash64. +class XXPH3FilterBitsBuilder : public BuiltinFilterBitsBuilder { + public: + explicit XXPH3FilterBitsBuilder( + std::atomic<int64_t>* aggregate_rounding_balance, + std::shared_ptr<CacheReservationManager> cache_res_mgr, + bool detect_filter_construct_corruption) + : aggregate_rounding_balance_(aggregate_rounding_balance), + cache_res_mgr_(cache_res_mgr), + detect_filter_construct_corruption_( + detect_filter_construct_corruption) {} + + ~XXPH3FilterBitsBuilder() override {} + + virtual void AddKey(const Slice& key) override { + uint64_t hash = GetSliceHash64(key); + // Especially with prefixes, it is common to have repetition, + // though only adjacent repetition, which we want to immediately + // recognize and collapse for estimating true filter space + // requirements. + if (hash_entries_info_.entries.empty() || + hash != hash_entries_info_.entries.back()) { + if (detect_filter_construct_corruption_) { + hash_entries_info_.xor_checksum ^= hash; + } + hash_entries_info_.entries.push_back(hash); + if (cache_res_mgr_ && + // Traditional rounding to whole bucket size + ((hash_entries_info_.entries.size() % + kUint64tHashEntryCacheResBucketSize) == + kUint64tHashEntryCacheResBucketSize / 2)) { + hash_entries_info_.cache_res_bucket_handles.emplace_back(nullptr); + Status s = cache_res_mgr_->MakeCacheReservation( + kUint64tHashEntryCacheResBucketSize * sizeof(hash), + &hash_entries_info_.cache_res_bucket_handles.back()); + s.PermitUncheckedError(); + } + } + } + + virtual size_t EstimateEntriesAdded() override { + return hash_entries_info_.entries.size(); + } + + virtual Status MaybePostVerify(const Slice& filter_content) override; + + protected: + static constexpr uint32_t kMetadataLen = 5; + + // Number of hash entries to accumulate before charging their memory usage to + // the cache when cache charging is available + static const std::size_t kUint64tHashEntryCacheResBucketSize = + CacheReservationManagerImpl< + CacheEntryRole::kFilterConstruction>::GetDummyEntrySize() / + sizeof(uint64_t); + + // For delegating between XXPH3FilterBitsBuilders + void SwapEntriesWith(XXPH3FilterBitsBuilder* other) { + assert(other != nullptr); + hash_entries_info_.Swap(&(other->hash_entries_info_)); + } + + void ResetEntries() { hash_entries_info_.Reset(); } + + virtual size_t RoundDownUsableSpace(size_t available_size) = 0; + + // To choose size using malloc_usable_size, we have to actually allocate. + size_t AllocateMaybeRounding(size_t target_len_with_metadata, + size_t num_entries, + std::unique_ptr<char[]>* buf) { + // Return value set to a default; overwritten in some cases + size_t rv = target_len_with_metadata; +#ifdef ROCKSDB_MALLOC_USABLE_SIZE + if (aggregate_rounding_balance_ != nullptr) { + // Do optimize_filters_for_memory, using malloc_usable_size. + // Approach: try to keep FP rate balance better than or on + // target (negative aggregate_rounding_balance_). We can then select a + // lower bound filter size (within reasonable limits) that gets us as + // close to on target as possible. We request allocation for that filter + // size and use malloc_usable_size to "round up" to the actual + // allocation size. + + // Although it can be considered bad practice to use malloc_usable_size + // to access an object beyond its original size, this approach should be + // quite general: working for all allocators that properly support + // malloc_usable_size. + + // Race condition on balance is OK because it can only cause temporary + // skew in rounding up vs. rounding down, as long as updates are atomic + // and relative. + int64_t balance = aggregate_rounding_balance_->load(); + + double target_fp_rate = + EstimatedFpRate(num_entries, target_len_with_metadata); + double rv_fp_rate = target_fp_rate; + + if (balance < 0) { + // See formula for BloomFilterPolicy::aggregate_rounding_balance_ + double for_balance_fp_rate = + -balance / double{0x100000000} + target_fp_rate; + + // To simplify, we just try a few modified smaller sizes. This also + // caps how much we vary filter size vs. target, to avoid outlier + // behavior from excessive variance. + size_t target_len = target_len_with_metadata - kMetadataLen; + assert(target_len < target_len_with_metadata); // check underflow + for (uint64_t maybe_len_rough : + {uint64_t{3} * target_len / 4, uint64_t{13} * target_len / 16, + uint64_t{7} * target_len / 8, uint64_t{15} * target_len / 16}) { + size_t maybe_len_with_metadata = + RoundDownUsableSpace(maybe_len_rough + kMetadataLen); + double maybe_fp_rate = + EstimatedFpRate(num_entries, maybe_len_with_metadata); + if (maybe_fp_rate <= for_balance_fp_rate) { + rv = maybe_len_with_metadata; + rv_fp_rate = maybe_fp_rate; + break; + } + } + } + + // Filter blocks are loaded into block cache with their block trailer. + // We need to make sure that's accounted for in choosing a + // fragmentation-friendly size. + const size_t kExtraPadding = BlockBasedTable::kBlockTrailerSize; + size_t requested = rv + kExtraPadding; + + // Allocate and get usable size + buf->reset(new char[requested]); + size_t usable = malloc_usable_size(buf->get()); + + if (usable - usable / 4 > requested) { + // Ratio greater than 4/3 is too much for utilizing, if it's + // not a buggy or mislinked malloc_usable_size implementation. + // Non-linearity of FP rates with bits/key means rapidly + // diminishing returns in overall accuracy for additional + // storage on disk. + // Nothing to do, except assert that the result is accurate about + // the usable size. (Assignment never used.) + assert(((*buf)[usable - 1] = 'x')); + } else if (usable > requested) { + rv = RoundDownUsableSpace(usable - kExtraPadding); + assert(rv <= usable - kExtraPadding); + rv_fp_rate = EstimatedFpRate(num_entries, rv); + } else { + // Too small means bad malloc_usable_size + assert(usable == requested); + } + memset(buf->get(), 0, rv); + + // Update balance + int64_t diff = static_cast<int64_t>((rv_fp_rate - target_fp_rate) * + double{0x100000000}); + *aggregate_rounding_balance_ += diff; + } else { + buf->reset(new char[rv]()); + } +#else + (void)num_entries; + buf->reset(new char[rv]()); +#endif // ROCKSDB_MALLOC_USABLE_SIZE + return rv; + } + + // TODO: Ideally we want to verify the hash entry + // as it is added to the filter and eliminate this function + // for speeding up and leaving fewer spaces for undetected memory/CPU + // corruption. For Ribbon Filter, it's bit harder. + // Possible solution: + // pass a custom iterator that tracks the xor checksum as + // it iterates to ResetAndFindSeedToSolve + Status MaybeVerifyHashEntriesChecksum() { + if (!detect_filter_construct_corruption_) { + return Status::OK(); + } + + uint64_t actual_hash_entries_xor_checksum = 0; + for (uint64_t h : hash_entries_info_.entries) { + actual_hash_entries_xor_checksum ^= h; + } + + if (actual_hash_entries_xor_checksum == hash_entries_info_.xor_checksum) { + return Status::OK(); + } else { + // Since these hash entries are corrupted and they will not be used + // anymore, we can reset them and release memory. + ResetEntries(); + return Status::Corruption("Filter's hash entries checksum mismatched"); + } + } + + // See BloomFilterPolicy::aggregate_rounding_balance_. If nullptr, + // always "round up" like historic behavior. + std::atomic<int64_t>* aggregate_rounding_balance_; + + // For reserving memory used in (new) Bloom and Ribbon Filter construction + std::shared_ptr<CacheReservationManager> cache_res_mgr_; + + // For managing cache charge for final filter in (new) Bloom and Ribbon + // Filter construction + std::deque<std::unique_ptr<CacheReservationManager::CacheReservationHandle>> + final_filter_cache_res_handles_; + + bool detect_filter_construct_corruption_; + + struct HashEntriesInfo { + // A deque avoids unnecessary copying of already-saved values + // and has near-minimal peak memory use. + std::deque<uint64_t> entries; + + // If cache_res_mgr_ != nullptr, + // it manages cache charge for buckets of hash entries in (new) Bloom + // or Ribbon Filter construction. + // Otherwise, it is empty. + std::deque<std::unique_ptr<CacheReservationManager::CacheReservationHandle>> + cache_res_bucket_handles; + + // If detect_filter_construct_corruption_ == true, + // it records the xor checksum of hash entries. + // Otherwise, it is 0. + uint64_t xor_checksum = 0; + + void Swap(HashEntriesInfo* other) { + assert(other != nullptr); + std::swap(entries, other->entries); + std::swap(cache_res_bucket_handles, other->cache_res_bucket_handles); + std::swap(xor_checksum, other->xor_checksum); + } + + void Reset() { + entries.clear(); + cache_res_bucket_handles.clear(); + xor_checksum = 0; + } + }; + + HashEntriesInfo hash_entries_info_; +}; + +// #################### FastLocalBloom implementation ################## // +// ############## also known as format_version=5 Bloom filter ########## // + +// See description in FastLocalBloomImpl +class FastLocalBloomBitsBuilder : public XXPH3FilterBitsBuilder { + public: + // Non-null aggregate_rounding_balance implies optimize_filters_for_memory + explicit FastLocalBloomBitsBuilder( + const int millibits_per_key, + std::atomic<int64_t>* aggregate_rounding_balance, + std::shared_ptr<CacheReservationManager> cache_res_mgr, + bool detect_filter_construct_corruption) + : XXPH3FilterBitsBuilder(aggregate_rounding_balance, cache_res_mgr, + detect_filter_construct_corruption), + millibits_per_key_(millibits_per_key) { + assert(millibits_per_key >= 1000); + } + + // No Copy allowed + FastLocalBloomBitsBuilder(const FastLocalBloomBitsBuilder&) = delete; + void operator=(const FastLocalBloomBitsBuilder&) = delete; + + ~FastLocalBloomBitsBuilder() override {} + + using FilterBitsBuilder::Finish; + + virtual Slice Finish(std::unique_ptr<const char[]>* buf) override { + return Finish(buf, nullptr); + } + + virtual Slice Finish(std::unique_ptr<const char[]>* buf, + Status* status) override { + size_t num_entries = hash_entries_info_.entries.size(); + size_t len_with_metadata = CalculateSpace(num_entries); + + std::unique_ptr<char[]> mutable_buf; + std::unique_ptr<CacheReservationManager::CacheReservationHandle> + final_filter_cache_res_handle; + len_with_metadata = + AllocateMaybeRounding(len_with_metadata, num_entries, &mutable_buf); + // Cache charging for mutable_buf + if (cache_res_mgr_) { + Status s = cache_res_mgr_->MakeCacheReservation( + len_with_metadata * sizeof(char), &final_filter_cache_res_handle); + s.PermitUncheckedError(); + } + + assert(mutable_buf); + assert(len_with_metadata >= kMetadataLen); + + // Max size supported by implementation + assert(len_with_metadata <= 0xffffffffU); + + // Compute num_probes after any rounding / adjustments + int num_probes = GetNumProbes(num_entries, len_with_metadata); + + uint32_t len = static_cast<uint32_t>(len_with_metadata - kMetadataLen); + if (len > 0) { + TEST_SYNC_POINT_CALLBACK( + "XXPH3FilterBitsBuilder::Finish::" + "TamperHashEntries", + &hash_entries_info_.entries); + AddAllEntries(mutable_buf.get(), len, num_probes); + Status verify_hash_entries_checksum_status = + MaybeVerifyHashEntriesChecksum(); + if (!verify_hash_entries_checksum_status.ok()) { + if (status) { + *status = verify_hash_entries_checksum_status; + } + return FinishAlwaysTrue(buf); + } + } + + bool keep_entries_for_postverify = detect_filter_construct_corruption_; + if (!keep_entries_for_postverify) { + ResetEntries(); + } + + // See BloomFilterPolicy::GetBloomBitsReader re: metadata + // -1 = Marker for newer Bloom implementations + mutable_buf[len] = static_cast<char>(-1); + // 0 = Marker for this sub-implementation + mutable_buf[len + 1] = static_cast<char>(0); + // num_probes (and 0 in upper bits for 64-byte block size) + mutable_buf[len + 2] = static_cast<char>(num_probes); + // rest of metadata stays zero + + auto TEST_arg_pair __attribute__((__unused__)) = + std::make_pair(&mutable_buf, len_with_metadata); + TEST_SYNC_POINT_CALLBACK("XXPH3FilterBitsBuilder::Finish::TamperFilter", + &TEST_arg_pair); + + Slice rv(mutable_buf.get(), len_with_metadata); + *buf = std::move(mutable_buf); + final_filter_cache_res_handles_.push_back( + std::move(final_filter_cache_res_handle)); + if (status) { + *status = Status::OK(); + } + return rv; + } + + size_t ApproximateNumEntries(size_t bytes) override { + size_t bytes_no_meta = + bytes >= kMetadataLen ? RoundDownUsableSpace(bytes) - kMetadataLen : 0; + return static_cast<size_t>(uint64_t{8000} * bytes_no_meta / + millibits_per_key_); + } + + size_t CalculateSpace(size_t num_entries) override { + // If not for cache line blocks in the filter, what would the target + // length in bytes be? + size_t raw_target_len = static_cast<size_t>( + (uint64_t{num_entries} * millibits_per_key_ + 7999) / 8000); + + if (raw_target_len >= size_t{0xffffffc0}) { + // Max supported for this data structure implementation + raw_target_len = size_t{0xffffffc0}; + } + + // Round up to nearest multiple of 64 (block size). This adjustment is + // used for target FP rate only so that we don't receive complaints about + // lower FP rate vs. historic Bloom filter behavior. + return ((raw_target_len + 63) & ~size_t{63}) + kMetadataLen; + } + + double EstimatedFpRate(size_t keys, size_t len_with_metadata) override { + int num_probes = GetNumProbes(keys, len_with_metadata); + return FastLocalBloomImpl::EstimatedFpRate( + keys, len_with_metadata - kMetadataLen, num_probes, /*hash bits*/ 64); + } + + protected: + size_t RoundDownUsableSpace(size_t available_size) override { + size_t rv = available_size - kMetadataLen; + + if (rv >= size_t{0xffffffc0}) { + // Max supported for this data structure implementation + rv = size_t{0xffffffc0}; + } + + // round down to multiple of 64 (block size) + rv &= ~size_t{63}; + + return rv + kMetadataLen; + } + + private: + // Compute num_probes after any rounding / adjustments + int GetNumProbes(size_t keys, size_t len_with_metadata) { + uint64_t millibits = uint64_t{len_with_metadata - kMetadataLen} * 8000; + int actual_millibits_per_key = + static_cast<int>(millibits / std::max(keys, size_t{1})); + // BEGIN XXX/TODO(peterd): preserving old/default behavior for now to + // minimize unit test churn. Remove this some time. + if (!aggregate_rounding_balance_) { + actual_millibits_per_key = millibits_per_key_; + } + // END XXX/TODO + return FastLocalBloomImpl::ChooseNumProbes(actual_millibits_per_key); + } + + void AddAllEntries(char* data, uint32_t len, int num_probes) { + // Simple version without prefetching: + // + // for (auto h : hash_entries_info_.entries) { + // FastLocalBloomImpl::AddHash(Lower32of64(h), Upper32of64(h), len, + // num_probes, data); + // } + + const size_t num_entries = hash_entries_info_.entries.size(); + constexpr size_t kBufferMask = 7; + static_assert(((kBufferMask + 1) & kBufferMask) == 0, + "Must be power of 2 minus 1"); + + std::array<uint32_t, kBufferMask + 1> hashes; + std::array<uint32_t, kBufferMask + 1> byte_offsets; + + // Prime the buffer + size_t i = 0; + std::deque<uint64_t>::iterator hash_entries_it = + hash_entries_info_.entries.begin(); + for (; i <= kBufferMask && i < num_entries; ++i) { + uint64_t h = *hash_entries_it; + FastLocalBloomImpl::PrepareHash(Lower32of64(h), len, data, + /*out*/ &byte_offsets[i]); + hashes[i] = Upper32of64(h); + ++hash_entries_it; + } + + // Process and buffer + for (; i < num_entries; ++i) { + uint32_t& hash_ref = hashes[i & kBufferMask]; + uint32_t& byte_offset_ref = byte_offsets[i & kBufferMask]; + // Process (add) + FastLocalBloomImpl::AddHashPrepared(hash_ref, num_probes, + data + byte_offset_ref); + // And buffer + uint64_t h = *hash_entries_it; + FastLocalBloomImpl::PrepareHash(Lower32of64(h), len, data, + /*out*/ &byte_offset_ref); + hash_ref = Upper32of64(h); + ++hash_entries_it; + } + + // Finish processing + for (i = 0; i <= kBufferMask && i < num_entries; ++i) { + FastLocalBloomImpl::AddHashPrepared(hashes[i], num_probes, + data + byte_offsets[i]); + } + } + + // Target allocation per added key, in thousandths of a bit. + int millibits_per_key_; +}; + +// See description in FastLocalBloomImpl +class FastLocalBloomBitsReader : public BuiltinFilterBitsReader { + public: + FastLocalBloomBitsReader(const char* data, int num_probes, uint32_t len_bytes) + : data_(data), num_probes_(num_probes), len_bytes_(len_bytes) {} + + // No Copy allowed + FastLocalBloomBitsReader(const FastLocalBloomBitsReader&) = delete; + void operator=(const FastLocalBloomBitsReader&) = delete; + + ~FastLocalBloomBitsReader() override {} + + bool MayMatch(const Slice& key) override { + uint64_t h = GetSliceHash64(key); + uint32_t byte_offset; + FastLocalBloomImpl::PrepareHash(Lower32of64(h), len_bytes_, data_, + /*out*/ &byte_offset); + return FastLocalBloomImpl::HashMayMatchPrepared(Upper32of64(h), num_probes_, + data_ + byte_offset); + } + + virtual void MayMatch(int num_keys, Slice** keys, bool* may_match) override { + std::array<uint32_t, MultiGetContext::MAX_BATCH_SIZE> hashes; + std::array<uint32_t, MultiGetContext::MAX_BATCH_SIZE> byte_offsets; + for (int i = 0; i < num_keys; ++i) { + uint64_t h = GetSliceHash64(*keys[i]); + FastLocalBloomImpl::PrepareHash(Lower32of64(h), len_bytes_, data_, + /*out*/ &byte_offsets[i]); + hashes[i] = Upper32of64(h); + } + for (int i = 0; i < num_keys; ++i) { + may_match[i] = FastLocalBloomImpl::HashMayMatchPrepared( + hashes[i], num_probes_, data_ + byte_offsets[i]); + } + } + + bool HashMayMatch(const uint64_t h) override { + return FastLocalBloomImpl::HashMayMatch(Lower32of64(h), Upper32of64(h), + len_bytes_, num_probes_, data_); + } + + private: + const char* data_; + const int num_probes_; + const uint32_t len_bytes_; +}; + +// ##################### Ribbon filter implementation ################### // + +// Implements concept RehasherTypesAndSettings in ribbon_impl.h +struct Standard128RibbonRehasherTypesAndSettings { + // These are schema-critical. Any change almost certainly changes + // underlying data. + static constexpr bool kIsFilter = true; + static constexpr bool kHomogeneous = false; + static constexpr bool kFirstCoeffAlwaysOne = true; + static constexpr bool kUseSmash = false; + using CoeffRow = ROCKSDB_NAMESPACE::Unsigned128; + using Hash = uint64_t; + using Seed = uint32_t; + // Changing these doesn't necessarily change underlying data, + // but might affect supported scalability of those dimensions. + using Index = uint32_t; + using ResultRow = uint32_t; + // Save a conditional in Ribbon queries + static constexpr bool kAllowZeroStarts = false; +}; + +using Standard128RibbonTypesAndSettings = + ribbon::StandardRehasherAdapter<Standard128RibbonRehasherTypesAndSettings>; + +class Standard128RibbonBitsBuilder : public XXPH3FilterBitsBuilder { + public: + explicit Standard128RibbonBitsBuilder( + double desired_one_in_fp_rate, int bloom_millibits_per_key, + std::atomic<int64_t>* aggregate_rounding_balance, + std::shared_ptr<CacheReservationManager> cache_res_mgr, + bool detect_filter_construct_corruption, Logger* info_log) + : XXPH3FilterBitsBuilder(aggregate_rounding_balance, cache_res_mgr, + detect_filter_construct_corruption), + desired_one_in_fp_rate_(desired_one_in_fp_rate), + info_log_(info_log), + bloom_fallback_(bloom_millibits_per_key, aggregate_rounding_balance, + cache_res_mgr, detect_filter_construct_corruption) { + assert(desired_one_in_fp_rate >= 1.0); + } + + // No Copy allowed + Standard128RibbonBitsBuilder(const Standard128RibbonBitsBuilder&) = delete; + void operator=(const Standard128RibbonBitsBuilder&) = delete; + + ~Standard128RibbonBitsBuilder() override {} + + using FilterBitsBuilder::Finish; + + virtual Slice Finish(std::unique_ptr<const char[]>* buf) override { + return Finish(buf, nullptr); + } + + virtual Slice Finish(std::unique_ptr<const char[]>* buf, + Status* status) override { + if (hash_entries_info_.entries.size() > kMaxRibbonEntries) { + ROCKS_LOG_WARN( + info_log_, "Too many keys for Ribbon filter: %llu", + static_cast<unsigned long long>(hash_entries_info_.entries.size())); + SwapEntriesWith(&bloom_fallback_); + assert(hash_entries_info_.entries.empty()); + return bloom_fallback_.Finish(buf, status); + } + if (hash_entries_info_.entries.size() == 0) { + // Save a conditional in Ribbon queries by using alternate reader + // for zero entries added. + if (status) { + *status = Status::OK(); + } + return FinishAlwaysFalse(buf); + } + uint32_t num_entries = + static_cast<uint32_t>(hash_entries_info_.entries.size()); + uint32_t num_slots; + size_t len_with_metadata; + + CalculateSpaceAndSlots(num_entries, &len_with_metadata, &num_slots); + + // Bloom fall-back indicator + if (num_slots == 0) { + SwapEntriesWith(&bloom_fallback_); + assert(hash_entries_info_.entries.empty()); + return bloom_fallback_.Finish(buf, status); + } + + uint32_t entropy = 0; + if (!hash_entries_info_.entries.empty()) { + entropy = Lower32of64(hash_entries_info_.entries.front()); + } + + BandingType banding; + std::size_t bytes_banding = ribbon::StandardBanding< + Standard128RibbonTypesAndSettings>::EstimateMemoryUsage(num_slots); + Status status_banding_cache_res = Status::OK(); + + // Cache charging for banding + std::unique_ptr<CacheReservationManager::CacheReservationHandle> + banding_res_handle; + if (cache_res_mgr_) { + status_banding_cache_res = cache_res_mgr_->MakeCacheReservation( + bytes_banding, &banding_res_handle); + } + + if (status_banding_cache_res.IsMemoryLimit()) { + ROCKS_LOG_WARN(info_log_, + "Cache charging for Ribbon filter banding failed due " + "to cache full"); + SwapEntriesWith(&bloom_fallback_); + assert(hash_entries_info_.entries.empty()); + // Release cache for banding since the banding won't be allocated + banding_res_handle.reset(); + return bloom_fallback_.Finish(buf, status); + } + + TEST_SYNC_POINT_CALLBACK( + "XXPH3FilterBitsBuilder::Finish::" + "TamperHashEntries", + &hash_entries_info_.entries); + + bool success = banding.ResetAndFindSeedToSolve( + num_slots, hash_entries_info_.entries.begin(), + hash_entries_info_.entries.end(), + /*starting seed*/ entropy & 255, /*seed mask*/ 255); + if (!success) { + ROCKS_LOG_WARN( + info_log_, "Too many re-seeds (256) for Ribbon filter, %llu / %llu", + static_cast<unsigned long long>(hash_entries_info_.entries.size()), + static_cast<unsigned long long>(num_slots)); + SwapEntriesWith(&bloom_fallback_); + assert(hash_entries_info_.entries.empty()); + return bloom_fallback_.Finish(buf, status); + } + + Status verify_hash_entries_checksum_status = + MaybeVerifyHashEntriesChecksum(); + if (!verify_hash_entries_checksum_status.ok()) { + ROCKS_LOG_WARN(info_log_, "Verify hash entries checksum error: %s", + verify_hash_entries_checksum_status.getState()); + if (status) { + *status = verify_hash_entries_checksum_status; + } + return FinishAlwaysTrue(buf); + } + + bool keep_entries_for_postverify = detect_filter_construct_corruption_; + if (!keep_entries_for_postverify) { + ResetEntries(); + } + + uint32_t seed = banding.GetOrdinalSeed(); + assert(seed < 256); + + std::unique_ptr<char[]> mutable_buf; + std::unique_ptr<CacheReservationManager::CacheReservationHandle> + final_filter_cache_res_handle; + len_with_metadata = + AllocateMaybeRounding(len_with_metadata, num_entries, &mutable_buf); + // Cache charging for mutable_buf + if (cache_res_mgr_) { + Status s = cache_res_mgr_->MakeCacheReservation( + len_with_metadata * sizeof(char), &final_filter_cache_res_handle); + s.PermitUncheckedError(); + } + + SolnType soln(mutable_buf.get(), len_with_metadata); + soln.BackSubstFrom(banding); + uint32_t num_blocks = soln.GetNumBlocks(); + // This should be guaranteed: + // num_entries < 2^30 + // => (overhead_factor < 2.0) + // num_entries * overhead_factor == num_slots < 2^31 + // => (num_blocks = num_slots / 128) + // num_blocks < 2^24 + assert(num_blocks < 0x1000000U); + + // See BloomFilterPolicy::GetBloomBitsReader re: metadata + // -2 = Marker for Standard128 Ribbon + mutable_buf[len_with_metadata - 5] = static_cast<char>(-2); + // Hash seed + mutable_buf[len_with_metadata - 4] = static_cast<char>(seed); + // Number of blocks, in 24 bits + // (Along with bytes, we can derive other settings) + mutable_buf[len_with_metadata - 3] = static_cast<char>(num_blocks & 255); + mutable_buf[len_with_metadata - 2] = + static_cast<char>((num_blocks >> 8) & 255); + mutable_buf[len_with_metadata - 1] = + static_cast<char>((num_blocks >> 16) & 255); + + auto TEST_arg_pair __attribute__((__unused__)) = + std::make_pair(&mutable_buf, len_with_metadata); + TEST_SYNC_POINT_CALLBACK("XXPH3FilterBitsBuilder::Finish::TamperFilter", + &TEST_arg_pair); + + Slice rv(mutable_buf.get(), len_with_metadata); + *buf = std::move(mutable_buf); + final_filter_cache_res_handles_.push_back( + std::move(final_filter_cache_res_handle)); + if (status) { + *status = Status::OK(); + } + return rv; + } + + // Setting num_slots to 0 means "fall back on Bloom filter." + // And note this implementation does not support num_entries or num_slots + // beyond uint32_t; see kMaxRibbonEntries. + void CalculateSpaceAndSlots(size_t num_entries, + size_t* target_len_with_metadata, + uint32_t* num_slots) { + if (num_entries > kMaxRibbonEntries) { + // More entries than supported by this Ribbon + *num_slots = 0; // use Bloom + *target_len_with_metadata = bloom_fallback_.CalculateSpace(num_entries); + return; + } + uint32_t entropy = 0; + if (!hash_entries_info_.entries.empty()) { + entropy = Upper32of64(hash_entries_info_.entries.front()); + } + + *num_slots = NumEntriesToNumSlots(static_cast<uint32_t>(num_entries)); + *target_len_with_metadata = + SolnType::GetBytesForOneInFpRate(*num_slots, desired_one_in_fp_rate_, + /*rounding*/ entropy) + + kMetadataLen; + + // Consider possible Bloom fallback for small filters + if (*num_slots < 1024) { + size_t bloom = bloom_fallback_.CalculateSpace(num_entries); + if (bloom < *target_len_with_metadata) { + *num_slots = 0; // use Bloom + *target_len_with_metadata = bloom; + return; + } + } + } + + size_t CalculateSpace(size_t num_entries) override { + if (num_entries == 0) { + // See FinishAlwaysFalse + return 0; + } + size_t target_len_with_metadata; + uint32_t num_slots; + CalculateSpaceAndSlots(num_entries, &target_len_with_metadata, &num_slots); + (void)num_slots; + return target_len_with_metadata; + } + + // This is a somewhat ugly but reasonably fast and reasonably accurate + // reversal of CalculateSpace. + size_t ApproximateNumEntries(size_t bytes) override { + size_t len_no_metadata = + RoundDownUsableSpace(std::max(bytes, size_t{kMetadataLen})) - + kMetadataLen; + + if (!(desired_one_in_fp_rate_ > 1.0)) { + // Effectively asking for 100% FP rate, or NaN etc. + // Note that NaN is neither < 1.0 nor > 1.0 + return kMaxRibbonEntries; + } + + // Find a slight under-estimate for actual average bits per slot + double min_real_bits_per_slot; + if (desired_one_in_fp_rate_ >= 1.0 + std::numeric_limits<uint32_t>::max()) { + // Max of 32 solution columns (result bits) + min_real_bits_per_slot = 32.0; + } else { + // Account for mix of b and b+1 solution columns being slightly + // suboptimal vs. ideal log2(1/fp_rate) bits. + uint32_t rounded = static_cast<uint32_t>(desired_one_in_fp_rate_); + int upper_bits_per_key = 1 + FloorLog2(rounded); + double fp_rate_for_upper = std::pow(2.0, -upper_bits_per_key); + double portion_lower = + (1.0 / desired_one_in_fp_rate_ - fp_rate_for_upper) / + fp_rate_for_upper; + min_real_bits_per_slot = upper_bits_per_key - portion_lower; + assert(min_real_bits_per_slot > 0.0); + assert(min_real_bits_per_slot <= 32.0); + } + + // An overestimate, but this should only be O(1) slots away from truth. + double max_slots = len_no_metadata * 8.0 / min_real_bits_per_slot; + + // Let's not bother accounting for overflow to Bloom filter + // (Includes NaN case) + if (!(max_slots < ConfigHelper::GetNumSlots(kMaxRibbonEntries))) { + return kMaxRibbonEntries; + } + + // Set up for short iteration + uint32_t slots = static_cast<uint32_t>(max_slots); + slots = SolnType::RoundUpNumSlots(slots); + + // Assert that we have a valid upper bound on slots + assert(SolnType::GetBytesForOneInFpRate( + SolnType::RoundUpNumSlots(slots + 1), desired_one_in_fp_rate_, + /*rounding*/ 0) > len_no_metadata); + + // Iterate up to a few times to rather precisely account for small effects + for (int i = 0; slots > 0; ++i) { + size_t reqd_bytes = + SolnType::GetBytesForOneInFpRate(slots, desired_one_in_fp_rate_, + /*rounding*/ 0); + if (reqd_bytes <= len_no_metadata) { + break; // done + } + if (i >= 2) { + // should have been enough iterations + assert(false); + break; + } + slots = SolnType::RoundDownNumSlots(slots - 1); + } + + uint32_t num_entries = ConfigHelper::GetNumToAdd(slots); + + // Consider possible Bloom fallback for small filters + if (slots < 1024) { + size_t bloom = bloom_fallback_.ApproximateNumEntries(bytes); + if (bloom > num_entries) { + return bloom; + } else { + return num_entries; + } + } else { + return std::min(num_entries, kMaxRibbonEntries); + } + } + + double EstimatedFpRate(size_t num_entries, + size_t len_with_metadata) override { + if (num_entries > kMaxRibbonEntries) { + // More entries than supported by this Ribbon + return bloom_fallback_.EstimatedFpRate(num_entries, len_with_metadata); + } + uint32_t num_slots = + NumEntriesToNumSlots(static_cast<uint32_t>(num_entries)); + SolnType fake_soln(nullptr, len_with_metadata); + fake_soln.ConfigureForNumSlots(num_slots); + return fake_soln.ExpectedFpRate(); + } + + Status MaybePostVerify(const Slice& filter_content) override { + bool fall_back = (bloom_fallback_.EstimateEntriesAdded() > 0); + return fall_back ? bloom_fallback_.MaybePostVerify(filter_content) + : XXPH3FilterBitsBuilder::MaybePostVerify(filter_content); + } + + protected: + size_t RoundDownUsableSpace(size_t available_size) override { + size_t rv = available_size - kMetadataLen; + + // round down to multiple of 16 (segment size) + rv &= ~size_t{15}; + + return rv + kMetadataLen; + } + + private: + using TS = Standard128RibbonTypesAndSettings; + using SolnType = ribbon::SerializableInterleavedSolution<TS>; + using BandingType = ribbon::StandardBanding<TS>; + using ConfigHelper = ribbon::BandingConfigHelper1TS<ribbon::kOneIn20, TS>; + + static uint32_t NumEntriesToNumSlots(uint32_t num_entries) { + uint32_t num_slots1 = ConfigHelper::GetNumSlots(num_entries); + return SolnType::RoundUpNumSlots(num_slots1); + } + + // Approximate num_entries to ensure number of bytes fits in 32 bits, which + // is not an inherent limitation but does ensure somewhat graceful Bloom + // fallback for crazy high number of entries, since the Bloom implementation + // does not support number of bytes bigger than fits in 32 bits. This is + // within an order of magnitude of implementation limit on num_slots + // fitting in 32 bits, and even closer for num_blocks fitting in 24 bits + // (for filter metadata). + static constexpr uint32_t kMaxRibbonEntries = 950000000; // ~ 1 billion + + // A desired value for 1/fp_rate. For example, 100 -> 1% fp rate. + double desired_one_in_fp_rate_; + + // For warnings, or can be nullptr + Logger* info_log_; + + // For falling back on Bloom filter in some exceptional cases and + // very small filter cases + FastLocalBloomBitsBuilder bloom_fallback_; +}; + +// for the linker, at least with DEBUG_LEVEL=2 +constexpr uint32_t Standard128RibbonBitsBuilder::kMaxRibbonEntries; + +class Standard128RibbonBitsReader : public BuiltinFilterBitsReader { + public: + Standard128RibbonBitsReader(const char* data, size_t len_bytes, + uint32_t num_blocks, uint32_t seed) + : soln_(const_cast<char*>(data), len_bytes) { + soln_.ConfigureForNumBlocks(num_blocks); + hasher_.SetOrdinalSeed(seed); + } + + // No Copy allowed + Standard128RibbonBitsReader(const Standard128RibbonBitsReader&) = delete; + void operator=(const Standard128RibbonBitsReader&) = delete; + + ~Standard128RibbonBitsReader() override {} + + bool MayMatch(const Slice& key) override { + uint64_t h = GetSliceHash64(key); + return soln_.FilterQuery(h, hasher_); + } + + virtual void MayMatch(int num_keys, Slice** keys, bool* may_match) override { + struct SavedData { + uint64_t seeded_hash; + uint32_t segment_num; + uint32_t num_columns; + uint32_t start_bits; + }; + std::array<SavedData, MultiGetContext::MAX_BATCH_SIZE> saved; + for (int i = 0; i < num_keys; ++i) { + ribbon::InterleavedPrepareQuery( + GetSliceHash64(*keys[i]), hasher_, soln_, &saved[i].seeded_hash, + &saved[i].segment_num, &saved[i].num_columns, &saved[i].start_bits); + } + for (int i = 0; i < num_keys; ++i) { + may_match[i] = ribbon::InterleavedFilterQuery( + saved[i].seeded_hash, saved[i].segment_num, saved[i].num_columns, + saved[i].start_bits, hasher_, soln_); + } + } + + bool HashMayMatch(const uint64_t h) override { + return soln_.FilterQuery(h, hasher_); + } + + private: + using TS = Standard128RibbonTypesAndSettings; + ribbon::SerializableInterleavedSolution<TS> soln_; + ribbon::StandardHasher<TS> hasher_; +}; + +// ##################### Legacy Bloom implementation ################### // + +using LegacyBloomImpl = LegacyLocalityBloomImpl</*ExtraRotates*/ false>; + +class LegacyBloomBitsBuilder : public BuiltinFilterBitsBuilder { + public: + explicit LegacyBloomBitsBuilder(const int bits_per_key, Logger* info_log); + + // No Copy allowed + LegacyBloomBitsBuilder(const LegacyBloomBitsBuilder&) = delete; + void operator=(const LegacyBloomBitsBuilder&) = delete; + + ~LegacyBloomBitsBuilder() override; + + void AddKey(const Slice& key) override; + + virtual size_t EstimateEntriesAdded() override { + return hash_entries_.size(); + } + + using FilterBitsBuilder::Finish; + + Slice Finish(std::unique_ptr<const char[]>* buf) override; + + size_t CalculateSpace(size_t num_entries) override { + uint32_t dont_care1; + uint32_t dont_care2; + return CalculateSpace(num_entries, &dont_care1, &dont_care2); + } + + double EstimatedFpRate(size_t keys, size_t bytes) override { + return LegacyBloomImpl::EstimatedFpRate(keys, bytes - kMetadataLen, + num_probes_); + } + + size_t ApproximateNumEntries(size_t bytes) override; + + private: + int bits_per_key_; + int num_probes_; + std::vector<uint32_t> hash_entries_; + Logger* info_log_; + + // Get totalbits that optimized for cpu cache line + uint32_t GetTotalBitsForLocality(uint32_t total_bits); + + // Reserve space for new filter + char* ReserveSpace(size_t num_entries, uint32_t* total_bits, + uint32_t* num_lines); + + // Implementation-specific variant of public CalculateSpace + uint32_t CalculateSpace(size_t num_entries, uint32_t* total_bits, + uint32_t* num_lines); + + // Assuming single threaded access to this function. + void AddHash(uint32_t h, char* data, uint32_t num_lines, uint32_t total_bits); +}; + +LegacyBloomBitsBuilder::LegacyBloomBitsBuilder(const int bits_per_key, + Logger* info_log) + : bits_per_key_(bits_per_key), + num_probes_(LegacyNoLocalityBloomImpl::ChooseNumProbes(bits_per_key_)), + info_log_(info_log) { + assert(bits_per_key_); +} + +LegacyBloomBitsBuilder::~LegacyBloomBitsBuilder() {} + +void LegacyBloomBitsBuilder::AddKey(const Slice& key) { + uint32_t hash = BloomHash(key); + if (hash_entries_.size() == 0 || hash != hash_entries_.back()) { + hash_entries_.push_back(hash); + } +} + +Slice LegacyBloomBitsBuilder::Finish(std::unique_ptr<const char[]>* buf) { + uint32_t total_bits, num_lines; + size_t num_entries = hash_entries_.size(); + char* data = + ReserveSpace(static_cast<int>(num_entries), &total_bits, &num_lines); + assert(data); + + if (total_bits != 0 && num_lines != 0) { + for (auto h : hash_entries_) { + AddHash(h, data, num_lines, total_bits); + } + + // Check for excessive entries for 32-bit hash function + if (num_entries >= /* minimum of 3 million */ 3000000U) { + // More specifically, we can detect that the 32-bit hash function + // is causing significant increase in FP rate by comparing current + // estimated FP rate to what we would get with a normal number of + // keys at same memory ratio. + double est_fp_rate = LegacyBloomImpl::EstimatedFpRate( + num_entries, total_bits / 8, num_probes_); + double vs_fp_rate = LegacyBloomImpl::EstimatedFpRate( + 1U << 16, (1U << 16) * bits_per_key_ / 8, num_probes_); + + if (est_fp_rate >= 1.50 * vs_fp_rate) { + // For more details, see + // https://github.com/facebook/rocksdb/wiki/RocksDB-Bloom-Filter + ROCKS_LOG_WARN( + info_log_, + "Using legacy SST/BBT Bloom filter with excessive key count " + "(%.1fM @ %dbpk), causing estimated %.1fx higher filter FP rate. " + "Consider using new Bloom with format_version>=5, smaller SST " + "file size, or partitioned filters.", + num_entries / 1000000.0, bits_per_key_, est_fp_rate / vs_fp_rate); + } + } + } + // See BloomFilterPolicy::GetFilterBitsReader for metadata + data[total_bits / 8] = static_cast<char>(num_probes_); + EncodeFixed32(data + total_bits / 8 + 1, static_cast<uint32_t>(num_lines)); + + const char* const_data = data; + buf->reset(const_data); + hash_entries_.clear(); + + return Slice(data, total_bits / 8 + kMetadataLen); +} + +size_t LegacyBloomBitsBuilder::ApproximateNumEntries(size_t bytes) { + assert(bits_per_key_); + assert(bytes > 0); + + uint64_t total_bits_tmp = bytes * 8; + // total bits, including temporary computations, cannot exceed 2^32 + // for compatibility + total_bits_tmp = std::min(total_bits_tmp, uint64_t{0xffff0000}); + + uint32_t high = static_cast<uint32_t>(total_bits_tmp) / + static_cast<uint32_t>(bits_per_key_) + + 1; + uint32_t low = 1; + uint32_t n = high; + for (; n >= low; n--) { + if (CalculateSpace(n) <= bytes) { + break; + } + } + return n; +} + +uint32_t LegacyBloomBitsBuilder::GetTotalBitsForLocality(uint32_t total_bits) { + uint32_t num_lines = + (total_bits + CACHE_LINE_SIZE * 8 - 1) / (CACHE_LINE_SIZE * 8); + + // Make num_lines an odd number to make sure more bits are involved + // when determining which block. + if (num_lines % 2 == 0) { + num_lines++; + } + return num_lines * (CACHE_LINE_SIZE * 8); +} + +uint32_t LegacyBloomBitsBuilder::CalculateSpace(size_t num_entries, + uint32_t* total_bits, + uint32_t* num_lines) { + assert(bits_per_key_); + if (num_entries != 0) { + size_t total_bits_tmp = num_entries * bits_per_key_; + // total bits, including temporary computations, cannot exceed 2^32 + // for compatibility + total_bits_tmp = std::min(total_bits_tmp, size_t{0xffff0000}); + + *total_bits = + GetTotalBitsForLocality(static_cast<uint32_t>(total_bits_tmp)); + *num_lines = *total_bits / (CACHE_LINE_SIZE * 8); + assert(*total_bits > 0 && *total_bits % 8 == 0); + } else { + // filter is empty, just leave space for metadata + *total_bits = 0; + *num_lines = 0; + } + + // Reserve space for Filter + uint32_t sz = *total_bits / 8; + sz += kMetadataLen; // 4 bytes for num_lines, 1 byte for num_probes + return sz; +} + +char* LegacyBloomBitsBuilder::ReserveSpace(size_t num_entries, + uint32_t* total_bits, + uint32_t* num_lines) { + uint32_t sz = CalculateSpace(num_entries, total_bits, num_lines); + char* data = new char[sz]; + memset(data, 0, sz); + return data; +} + +inline void LegacyBloomBitsBuilder::AddHash(uint32_t h, char* data, + uint32_t num_lines, + uint32_t total_bits) { +#ifdef NDEBUG + static_cast<void>(total_bits); +#endif + assert(num_lines > 0 && total_bits > 0); + + LegacyBloomImpl::AddHash(h, num_lines, num_probes_, data, + ConstexprFloorLog2(CACHE_LINE_SIZE)); +} + +class LegacyBloomBitsReader : public BuiltinFilterBitsReader { + public: + LegacyBloomBitsReader(const char* data, int num_probes, uint32_t num_lines, + uint32_t log2_cache_line_size) + : data_(data), + num_probes_(num_probes), + num_lines_(num_lines), + log2_cache_line_size_(log2_cache_line_size) {} + + // No Copy allowed + LegacyBloomBitsReader(const LegacyBloomBitsReader&) = delete; + void operator=(const LegacyBloomBitsReader&) = delete; + + ~LegacyBloomBitsReader() override {} + + // "contents" contains the data built by a preceding call to + // FilterBitsBuilder::Finish. MayMatch must return true if the key was + // passed to FilterBitsBuilder::AddKey. This method may return true or false + // if the key was not on the list, but it should aim to return false with a + // high probability. + bool MayMatch(const Slice& key) override { + uint32_t hash = BloomHash(key); + uint32_t byte_offset; + LegacyBloomImpl::PrepareHashMayMatch( + hash, num_lines_, data_, /*out*/ &byte_offset, log2_cache_line_size_); + return LegacyBloomImpl::HashMayMatchPrepared( + hash, num_probes_, data_ + byte_offset, log2_cache_line_size_); + } + + virtual void MayMatch(int num_keys, Slice** keys, bool* may_match) override { + std::array<uint32_t, MultiGetContext::MAX_BATCH_SIZE> hashes; + std::array<uint32_t, MultiGetContext::MAX_BATCH_SIZE> byte_offsets; + for (int i = 0; i < num_keys; ++i) { + hashes[i] = BloomHash(*keys[i]); + LegacyBloomImpl::PrepareHashMayMatch(hashes[i], num_lines_, data_, + /*out*/ &byte_offsets[i], + log2_cache_line_size_); + } + for (int i = 0; i < num_keys; ++i) { + may_match[i] = LegacyBloomImpl::HashMayMatchPrepared( + hashes[i], num_probes_, data_ + byte_offsets[i], + log2_cache_line_size_); + } + } + + bool HashMayMatch(const uint64_t /* h */) override { return false; } + + private: + const char* data_; + const int num_probes_; + const uint32_t num_lines_; + const uint32_t log2_cache_line_size_; +}; + +class AlwaysTrueFilter : public BuiltinFilterBitsReader { + public: + bool MayMatch(const Slice&) override { return true; } + using FilterBitsReader::MayMatch; // inherit overload + bool HashMayMatch(const uint64_t) override { return true; } + using BuiltinFilterBitsReader::HashMayMatch; // inherit overload +}; + +class AlwaysFalseFilter : public BuiltinFilterBitsReader { + public: + bool MayMatch(const Slice&) override { return false; } + using FilterBitsReader::MayMatch; // inherit overload + bool HashMayMatch(const uint64_t) override { return false; } + using BuiltinFilterBitsReader::HashMayMatch; // inherit overload +}; + +Status XXPH3FilterBitsBuilder::MaybePostVerify(const Slice& filter_content) { + Status s = Status::OK(); + + if (!detect_filter_construct_corruption_) { + return s; + } + + std::unique_ptr<BuiltinFilterBitsReader> bits_reader( + BuiltinFilterPolicy::GetBuiltinFilterBitsReader(filter_content)); + + for (uint64_t h : hash_entries_info_.entries) { + // The current approach will not detect corruption from XXPH3Filter to + // AlwaysTrueFilter, which can lead to performance cost later due to + // AlwaysTrueFilter not filtering anything. But this cost is acceptable + // given the extra implementation complixity to detect such case. + bool may_match = bits_reader->HashMayMatch(h); + if (!may_match) { + s = Status::Corruption("Corrupted filter content"); + break; + } + } + + ResetEntries(); + return s; +} +} // namespace + +const char* BuiltinFilterPolicy::kClassName() { + return "rocksdb.internal.BuiltinFilter"; +} + +bool BuiltinFilterPolicy::IsInstanceOf(const std::string& name) const { + if (name == kClassName()) { + return true; + } else { + return FilterPolicy::IsInstanceOf(name); + } +} + +static const char* kBuiltinFilterMetadataName = "rocksdb.BuiltinBloomFilter"; + +const char* BuiltinFilterPolicy::kCompatibilityName() { + return kBuiltinFilterMetadataName; +} + +const char* BuiltinFilterPolicy::CompatibilityName() const { + return kBuiltinFilterMetadataName; +} + +BloomLikeFilterPolicy::BloomLikeFilterPolicy(double bits_per_key) + : warned_(false), aggregate_rounding_balance_(0) { + // Sanitize bits_per_key + if (bits_per_key < 0.5) { + // Round down to no filter + bits_per_key = 0; + } else if (bits_per_key < 1.0) { + // Minimum 1 bit per key (equiv) when creating filter + bits_per_key = 1.0; + } else if (!(bits_per_key < 100.0)) { // including NaN + bits_per_key = 100.0; + } + + // Includes a nudge toward rounding up, to ensure on all platforms + // that doubles specified with three decimal digits after the decimal + // point are interpreted accurately. + millibits_per_key_ = static_cast<int>(bits_per_key * 1000.0 + 0.500001); + + // For now configure Ribbon filter to match Bloom FP rate and save + // memory. (Ribbon bits per key will be ~30% less than Bloom bits per key + // for same FP rate.) + desired_one_in_fp_rate_ = + 1.0 / BloomMath::CacheLocalFpRate( + bits_per_key, + FastLocalBloomImpl::ChooseNumProbes(millibits_per_key_), + /*cache_line_bits*/ 512); + + // For better or worse, this is a rounding up of a nudged rounding up, + // e.g. 7.4999999999999 will round up to 8, but that provides more + // predictability against small arithmetic errors in floating point. + whole_bits_per_key_ = (millibits_per_key_ + 500) / 1000; +} + +BloomLikeFilterPolicy::~BloomLikeFilterPolicy() {} +const char* BloomLikeFilterPolicy::kClassName() { + return "rocksdb.internal.BloomLikeFilter"; +} + +bool BloomLikeFilterPolicy::IsInstanceOf(const std::string& name) const { + if (name == kClassName()) { + return true; + } else { + return BuiltinFilterPolicy::IsInstanceOf(name); + } +} + +const char* ReadOnlyBuiltinFilterPolicy::kClassName() { + return kBuiltinFilterMetadataName; +} + +std::string BloomLikeFilterPolicy::GetId() const { + return Name() + GetBitsPerKeySuffix(); +} + +BloomFilterPolicy::BloomFilterPolicy(double bits_per_key) + : BloomLikeFilterPolicy(bits_per_key) {} + +FilterBitsBuilder* BloomFilterPolicy::GetBuilderWithContext( + const FilterBuildingContext& context) const { + if (GetMillibitsPerKey() == 0) { + // "No filter" special case + return nullptr; + } else if (context.table_options.format_version < 5) { + return GetLegacyBloomBuilderWithContext(context); + } else { + return GetFastLocalBloomBuilderWithContext(context); + } +} + +const char* BloomFilterPolicy::kClassName() { return "bloomfilter"; } +const char* BloomFilterPolicy::kNickName() { return "rocksdb.BloomFilter"; } + +std::string BloomFilterPolicy::GetId() const { + // Including ":false" for better forward-compatibility with 6.29 and earlier + // which required a boolean `use_block_based_builder` parameter + return BloomLikeFilterPolicy::GetId() + ":false"; +} + +FilterBitsBuilder* BloomLikeFilterPolicy::GetFastLocalBloomBuilderWithContext( + const FilterBuildingContext& context) const { + bool offm = context.table_options.optimize_filters_for_memory; + const auto options_overrides_iter = + context.table_options.cache_usage_options.options_overrides.find( + CacheEntryRole::kFilterConstruction); + const auto filter_construction_charged = + options_overrides_iter != + context.table_options.cache_usage_options.options_overrides.end() + ? options_overrides_iter->second.charged + : context.table_options.cache_usage_options.options.charged; + + std::shared_ptr<CacheReservationManager> cache_res_mgr; + if (context.table_options.block_cache && + filter_construction_charged == + CacheEntryRoleOptions::Decision::kEnabled) { + cache_res_mgr = std::make_shared< + CacheReservationManagerImpl<CacheEntryRole::kFilterConstruction>>( + context.table_options.block_cache); + } + return new FastLocalBloomBitsBuilder( + millibits_per_key_, offm ? &aggregate_rounding_balance_ : nullptr, + cache_res_mgr, context.table_options.detect_filter_construct_corruption); +} + +FilterBitsBuilder* BloomLikeFilterPolicy::GetLegacyBloomBuilderWithContext( + const FilterBuildingContext& context) const { + if (whole_bits_per_key_ >= 14 && context.info_log && + !warned_.load(std::memory_order_relaxed)) { + warned_ = true; + const char* adjective; + if (whole_bits_per_key_ >= 20) { + adjective = "Dramatic"; + } else { + adjective = "Significant"; + } + // For more details, see + // https://github.com/facebook/rocksdb/wiki/RocksDB-Bloom-Filter + ROCKS_LOG_WARN(context.info_log, + "Using legacy Bloom filter with high (%d) bits/key. " + "%s filter space and/or accuracy improvement is available " + "with format_version>=5.", + whole_bits_per_key_, adjective); + } + return new LegacyBloomBitsBuilder(whole_bits_per_key_, context.info_log); +} + +FilterBitsBuilder* +BloomLikeFilterPolicy::GetStandard128RibbonBuilderWithContext( + const FilterBuildingContext& context) const { + // FIXME: code duplication with GetFastLocalBloomBuilderWithContext + bool offm = context.table_options.optimize_filters_for_memory; + const auto options_overrides_iter = + context.table_options.cache_usage_options.options_overrides.find( + CacheEntryRole::kFilterConstruction); + const auto filter_construction_charged = + options_overrides_iter != + context.table_options.cache_usage_options.options_overrides.end() + ? options_overrides_iter->second.charged + : context.table_options.cache_usage_options.options.charged; + + std::shared_ptr<CacheReservationManager> cache_res_mgr; + if (context.table_options.block_cache && + filter_construction_charged == + CacheEntryRoleOptions::Decision::kEnabled) { + cache_res_mgr = std::make_shared< + CacheReservationManagerImpl<CacheEntryRole::kFilterConstruction>>( + context.table_options.block_cache); + } + return new Standard128RibbonBitsBuilder( + desired_one_in_fp_rate_, millibits_per_key_, + offm ? &aggregate_rounding_balance_ : nullptr, cache_res_mgr, + context.table_options.detect_filter_construct_corruption, + context.info_log); +} + +std::string BloomLikeFilterPolicy::GetBitsPerKeySuffix() const { + std::string rv = ":" + std::to_string(millibits_per_key_ / 1000); + int frac = millibits_per_key_ % 1000; + if (frac > 0) { + rv.push_back('.'); + rv.push_back(static_cast<char>('0' + (frac / 100))); + frac %= 100; + if (frac > 0) { + rv.push_back(static_cast<char>('0' + (frac / 10))); + frac %= 10; + if (frac > 0) { + rv.push_back(static_cast<char>('0' + frac)); + } + } + } + return rv; +} + +FilterBitsBuilder* BuiltinFilterPolicy::GetBuilderFromContext( + const FilterBuildingContext& context) { + if (context.table_options.filter_policy) { + return context.table_options.filter_policy->GetBuilderWithContext(context); + } else { + return nullptr; + } +} + +// For testing only, but always constructable with internal names +namespace test { + +const char* LegacyBloomFilterPolicy::kClassName() { + return "rocksdb.internal.LegacyBloomFilter"; +} + +FilterBitsBuilder* LegacyBloomFilterPolicy::GetBuilderWithContext( + const FilterBuildingContext& context) const { + if (GetMillibitsPerKey() == 0) { + // "No filter" special case + return nullptr; + } + return GetLegacyBloomBuilderWithContext(context); +} + +const char* FastLocalBloomFilterPolicy::kClassName() { + return "rocksdb.internal.FastLocalBloomFilter"; +} + +FilterBitsBuilder* FastLocalBloomFilterPolicy::GetBuilderWithContext( + const FilterBuildingContext& context) const { + if (GetMillibitsPerKey() == 0) { + // "No filter" special case + return nullptr; + } + return GetFastLocalBloomBuilderWithContext(context); +} + +const char* Standard128RibbonFilterPolicy::kClassName() { + return "rocksdb.internal.Standard128RibbonFilter"; +} + +FilterBitsBuilder* Standard128RibbonFilterPolicy::GetBuilderWithContext( + const FilterBuildingContext& context) const { + if (GetMillibitsPerKey() == 0) { + // "No filter" special case + return nullptr; + } + return GetStandard128RibbonBuilderWithContext(context); +} + +} // namespace test + +BuiltinFilterBitsReader* BuiltinFilterPolicy::GetBuiltinFilterBitsReader( + const Slice& contents) { + uint32_t len_with_meta = static_cast<uint32_t>(contents.size()); + if (len_with_meta <= kMetadataLen) { + // filter is empty or broken. Treat like zero keys added. + return new AlwaysFalseFilter(); + } + + // Legacy Bloom filter data: + // 0 +-----------------------------------+ + // | Raw Bloom filter data | + // | ... | + // len +-----------------------------------+ + // | byte for num_probes or | + // | marker for new implementations | + // len+1 +-----------------------------------+ + // | four bytes for number of cache | + // | lines | + // len_with_meta +-----------------------------------+ + + int8_t raw_num_probes = + static_cast<int8_t>(contents.data()[len_with_meta - kMetadataLen]); + // NB: *num_probes > 30 and < 128 probably have not been used, because of + // BloomFilterPolicy::initialize, unless directly calling + // LegacyBloomBitsBuilder as an API, but we are leaving those cases in + // limbo with LegacyBloomBitsReader for now. + + if (raw_num_probes < 1) { + // Note: < 0 (or unsigned > 127) indicate special new implementations + // (or reserved for future use) + switch (raw_num_probes) { + case 0: + // Treat as zero probes (always FP) + return new AlwaysTrueFilter(); + case -1: + // Marker for newer Bloom implementations + return GetBloomBitsReader(contents); + case -2: + // Marker for Ribbon implementations + return GetRibbonBitsReader(contents); + default: + // Reserved (treat as zero probes, always FP, for now) + return new AlwaysTrueFilter(); + } + } + // else attempt decode for LegacyBloomBitsReader + + int num_probes = raw_num_probes; + assert(num_probes >= 1); + assert(num_probes <= 127); + + uint32_t len = len_with_meta - kMetadataLen; + assert(len > 0); + + uint32_t num_lines = DecodeFixed32(contents.data() + len_with_meta - 4); + uint32_t log2_cache_line_size; + + if (num_lines * CACHE_LINE_SIZE == len) { + // Common case + log2_cache_line_size = ConstexprFloorLog2(CACHE_LINE_SIZE); + } else if (num_lines == 0 || len % num_lines != 0) { + // Invalid (no solution to num_lines * x == len) + // Treat as zero probes (always FP) for now. + return new AlwaysTrueFilter(); + } else { + // Determine the non-native cache line size (from another system) + log2_cache_line_size = 0; + while ((num_lines << log2_cache_line_size) < len) { + ++log2_cache_line_size; + } + if ((num_lines << log2_cache_line_size) != len) { + // Invalid (block size not a power of two) + // Treat as zero probes (always FP) for now. + return new AlwaysTrueFilter(); + } + } + // if not early return + return new LegacyBloomBitsReader(contents.data(), num_probes, num_lines, + log2_cache_line_size); +} + +// Read metadata to determine what kind of FilterBitsReader is needed +// and return a new one. +FilterBitsReader* BuiltinFilterPolicy::GetFilterBitsReader( + const Slice& contents) const { + return BuiltinFilterPolicy::GetBuiltinFilterBitsReader(contents); +} + +BuiltinFilterBitsReader* BuiltinFilterPolicy::GetRibbonBitsReader( + const Slice& contents) { + uint32_t len_with_meta = static_cast<uint32_t>(contents.size()); + uint32_t len = len_with_meta - kMetadataLen; + + assert(len > 0); // precondition + + uint32_t seed = static_cast<uint8_t>(contents.data()[len + 1]); + uint32_t num_blocks = static_cast<uint8_t>(contents.data()[len + 2]); + num_blocks |= static_cast<uint8_t>(contents.data()[len + 3]) << 8; + num_blocks |= static_cast<uint8_t>(contents.data()[len + 4]) << 16; + if (num_blocks < 2) { + // Not supported + // num_blocks == 1 is not used because num_starts == 1 is problematic + // for the hashing scheme. num_blocks == 0 is unused because there's + // already a concise encoding of an "always false" filter. + // Return something safe: + return new AlwaysTrueFilter(); + } + return new Standard128RibbonBitsReader(contents.data(), len, num_blocks, + seed); +} + +// For newer Bloom filter implementations +BuiltinFilterBitsReader* BuiltinFilterPolicy::GetBloomBitsReader( + const Slice& contents) { + uint32_t len_with_meta = static_cast<uint32_t>(contents.size()); + uint32_t len = len_with_meta - kMetadataLen; + + assert(len > 0); // precondition + + // New Bloom filter data: + // 0 +-----------------------------------+ + // | Raw Bloom filter data | + // | ... | + // len +-----------------------------------+ + // | char{-1} byte -> new Bloom filter | + // len+1 +-----------------------------------+ + // | byte for subimplementation | + // | 0: FastLocalBloom | + // | other: reserved | + // len+2 +-----------------------------------+ + // | byte for block_and_probes | + // | 0 in top 3 bits -> 6 -> 64-byte | + // | reserved: | + // | 1 in top 3 bits -> 7 -> 128-byte| + // | 2 in top 3 bits -> 8 -> 256-byte| + // | ... | + // | num_probes in bottom 5 bits, | + // | except 0 and 31 reserved | + // len+3 +-----------------------------------+ + // | two bytes reserved | + // | possibly for hash seed | + // len_with_meta +-----------------------------------+ + + // Read more metadata (see above) + char sub_impl_val = contents.data()[len_with_meta - 4]; + char block_and_probes = contents.data()[len_with_meta - 3]; + int log2_block_bytes = ((block_and_probes >> 5) & 7) + 6; + + int num_probes = (block_and_probes & 31); + if (num_probes < 1 || num_probes > 30) { + // Reserved / future safe + return new AlwaysTrueFilter(); + } + + uint16_t rest = DecodeFixed16(contents.data() + len_with_meta - 2); + if (rest != 0) { + // Reserved, possibly for hash seed + // Future safe + return new AlwaysTrueFilter(); + } + + if (sub_impl_val == 0) { // FastLocalBloom + if (log2_block_bytes == 6) { // Only block size supported for now + return new FastLocalBloomBitsReader(contents.data(), num_probes, len); + } + } + // otherwise + // Reserved / future safe + return new AlwaysTrueFilter(); +} + +const FilterPolicy* NewBloomFilterPolicy(double bits_per_key, + bool /*use_block_based_builder*/) { + // NOTE: use_block_based_builder now ignored so block-based filter is no + // longer accessible in public API. + return new BloomFilterPolicy(bits_per_key); +} + +RibbonFilterPolicy::RibbonFilterPolicy(double bloom_equivalent_bits_per_key, + int bloom_before_level) + : BloomLikeFilterPolicy(bloom_equivalent_bits_per_key), + bloom_before_level_(bloom_before_level) {} + +FilterBitsBuilder* RibbonFilterPolicy::GetBuilderWithContext( + const FilterBuildingContext& context) const { + if (GetMillibitsPerKey() == 0) { + // "No filter" special case + return nullptr; + } + // Treat unknown same as bottommost + int levelish = INT_MAX; + + switch (context.compaction_style) { + case kCompactionStyleLevel: + case kCompactionStyleUniversal: { + if (context.reason == TableFileCreationReason::kFlush) { + // Treat flush as level -1 + assert(context.level_at_creation == 0); + levelish = -1; + } else if (context.level_at_creation == -1) { + // Unknown level + assert(levelish == INT_MAX); + } else { + levelish = context.level_at_creation; + } + break; + } + case kCompactionStyleFIFO: + case kCompactionStyleNone: + // Treat as bottommost + assert(levelish == INT_MAX); + break; + } + if (levelish < bloom_before_level_) { + return GetFastLocalBloomBuilderWithContext(context); + } else { + return GetStandard128RibbonBuilderWithContext(context); + } +} + +const char* RibbonFilterPolicy::kClassName() { return "ribbonfilter"; } +const char* RibbonFilterPolicy::kNickName() { return "rocksdb.RibbonFilter"; } + +std::string RibbonFilterPolicy::GetId() const { + return BloomLikeFilterPolicy::GetId() + ":" + + std::to_string(bloom_before_level_); +} + +const FilterPolicy* NewRibbonFilterPolicy(double bloom_equivalent_bits_per_key, + int bloom_before_level) { + return new RibbonFilterPolicy(bloom_equivalent_bits_per_key, + bloom_before_level); +} + +FilterBuildingContext::FilterBuildingContext( + const BlockBasedTableOptions& _table_options) + : table_options(_table_options) {} + +FilterPolicy::~FilterPolicy() {} + +std::shared_ptr<const FilterPolicy> BloomLikeFilterPolicy::Create( + const std::string& name, double bits_per_key) { + if (name == test::LegacyBloomFilterPolicy::kClassName()) { + return std::make_shared<test::LegacyBloomFilterPolicy>(bits_per_key); + } else if (name == test::FastLocalBloomFilterPolicy::kClassName()) { + return std::make_shared<test::FastLocalBloomFilterPolicy>(bits_per_key); + } else if (name == test::Standard128RibbonFilterPolicy::kClassName()) { + return std::make_shared<test::Standard128RibbonFilterPolicy>(bits_per_key); + } else if (name == BloomFilterPolicy::kClassName()) { + // For testing + return std::make_shared<BloomFilterPolicy>(bits_per_key); + } else if (name == RibbonFilterPolicy::kClassName()) { + // For testing + return std::make_shared<RibbonFilterPolicy>(bits_per_key, + /*bloom_before_level*/ 0); + } else { + return nullptr; + } +} + +#ifndef ROCKSDB_LITE +namespace { +static ObjectLibrary::PatternEntry FilterPatternEntryWithBits( + const char* name) { + return ObjectLibrary::PatternEntry(name, false).AddNumber(":", false); +} + +template <typename T> +T* NewBuiltinFilterPolicyWithBits(const std::string& uri) { + const std::vector<std::string> vals = StringSplit(uri, ':'); + double bits_per_key = ParseDouble(vals[1]); + return new T(bits_per_key); +} +static int RegisterBuiltinFilterPolicies(ObjectLibrary& library, + const std::string& /*arg*/) { + library.AddFactory<const FilterPolicy>( + ReadOnlyBuiltinFilterPolicy::kClassName(), + [](const std::string& /*uri*/, std::unique_ptr<const FilterPolicy>* guard, + std::string* /* errmsg */) { + guard->reset(new ReadOnlyBuiltinFilterPolicy()); + return guard->get(); + }); + + library.AddFactory<const FilterPolicy>( + FilterPatternEntryWithBits(BloomFilterPolicy::kClassName()) + .AnotherName(BloomFilterPolicy::kNickName()), + [](const std::string& uri, std::unique_ptr<const FilterPolicy>* guard, + std::string* /* errmsg */) { + guard->reset(NewBuiltinFilterPolicyWithBits<BloomFilterPolicy>(uri)); + return guard->get(); + }); + library.AddFactory<const FilterPolicy>( + FilterPatternEntryWithBits(BloomFilterPolicy::kClassName()) + .AnotherName(BloomFilterPolicy::kNickName()) + .AddSuffix(":false"), + [](const std::string& uri, std::unique_ptr<const FilterPolicy>* guard, + std::string* /* errmsg */) { + guard->reset(NewBuiltinFilterPolicyWithBits<BloomFilterPolicy>(uri)); + return guard->get(); + }); + library.AddFactory<const FilterPolicy>( + FilterPatternEntryWithBits(BloomFilterPolicy::kClassName()) + .AnotherName(BloomFilterPolicy::kNickName()) + .AddSuffix(":true"), + [](const std::string& uri, std::unique_ptr<const FilterPolicy>* guard, + std::string* /* errmsg */) { + const std::vector<std::string> vals = StringSplit(uri, ':'); + double bits_per_key = ParseDouble(vals[1]); + // NOTE: This case previously configured the deprecated block-based + // filter, but old ways of configuring that now map to full filter. We + // defer to the corresponding API to ensure consistency in case that + // change is reverted. + guard->reset(NewBloomFilterPolicy(bits_per_key, true)); + return guard->get(); + }); + library.AddFactory<const FilterPolicy>( + FilterPatternEntryWithBits(RibbonFilterPolicy::kClassName()) + .AnotherName(RibbonFilterPolicy::kNickName()), + [](const std::string& uri, std::unique_ptr<const FilterPolicy>* guard, + std::string* /* errmsg */) { + const std::vector<std::string> vals = StringSplit(uri, ':'); + double bits_per_key = ParseDouble(vals[1]); + guard->reset(NewRibbonFilterPolicy(bits_per_key)); + return guard->get(); + }); + library.AddFactory<const FilterPolicy>( + FilterPatternEntryWithBits(RibbonFilterPolicy::kClassName()) + .AnotherName(RibbonFilterPolicy::kNickName()) + .AddNumber(":", true), + [](const std::string& uri, std::unique_ptr<const FilterPolicy>* guard, + std::string* /* errmsg */) { + const std::vector<std::string> vals = StringSplit(uri, ':'); + double bits_per_key = ParseDouble(vals[1]); + int bloom_before_level = ParseInt(vals[2]); + guard->reset(NewRibbonFilterPolicy(bits_per_key, bloom_before_level)); + return guard->get(); + }); + library.AddFactory<const FilterPolicy>( + FilterPatternEntryWithBits(test::LegacyBloomFilterPolicy::kClassName()), + [](const std::string& uri, std::unique_ptr<const FilterPolicy>* guard, + std::string* /* errmsg */) { + guard->reset( + NewBuiltinFilterPolicyWithBits<test::LegacyBloomFilterPolicy>(uri)); + return guard->get(); + }); + library.AddFactory<const FilterPolicy>( + FilterPatternEntryWithBits( + test::FastLocalBloomFilterPolicy::kClassName()), + [](const std::string& uri, std::unique_ptr<const FilterPolicy>* guard, + std::string* /* errmsg */) { + guard->reset( + NewBuiltinFilterPolicyWithBits<test::FastLocalBloomFilterPolicy>( + uri)); + return guard->get(); + }); + library.AddFactory<const FilterPolicy>( + FilterPatternEntryWithBits( + test::Standard128RibbonFilterPolicy::kClassName()), + [](const std::string& uri, std::unique_ptr<const FilterPolicy>* guard, + std::string* /* errmsg */) { + guard->reset( + NewBuiltinFilterPolicyWithBits<test::Standard128RibbonFilterPolicy>( + uri)); + return guard->get(); + }); + size_t num_types; + return static_cast<int>(library.GetFactoryCount(&num_types)); +} +} // namespace +#endif // ROCKSDB_LITE + +Status FilterPolicy::CreateFromString( + const ConfigOptions& options, const std::string& value, + std::shared_ptr<const FilterPolicy>* policy) { + if (value == kNullptrString || value.empty()) { + policy->reset(); + return Status::OK(); + } else if (value == ReadOnlyBuiltinFilterPolicy::kClassName()) { + *policy = std::make_shared<ReadOnlyBuiltinFilterPolicy>(); + return Status::OK(); + } + + std::string id; + std::unordered_map<std::string, std::string> opt_map; + Status status = + Customizable::GetOptionsMap(options, policy->get(), value, &id, &opt_map); + if (!status.ok()) { // GetOptionsMap failed + return status; + } else if (id.empty()) { // We have no Id but have options. Not good + return Status::NotSupported("Cannot reset object ", id); + } else { +#ifndef ROCKSDB_LITE + static std::once_flag loaded; + std::call_once(loaded, [&]() { + RegisterBuiltinFilterPolicies(*(ObjectLibrary::Default().get()), ""); + }); + status = options.registry->NewSharedObject(id, policy); +#else + status = + Status::NotSupported("Cannot load filter policy in LITE mode ", value); +#endif // ROCKSDB_LITE + } + if (options.ignore_unsupported_options && status.IsNotSupported()) { + return Status::OK(); + } else if (status.ok()) { + status = Customizable::ConfigureNewObject( + options, const_cast<FilterPolicy*>(policy->get()), opt_map); + } + return status; +} + +const std::vector<std::string>& BloomLikeFilterPolicy::GetAllFixedImpls() { + STATIC_AVOID_DESTRUCTION(std::vector<std::string>, impls){ + // Match filter_bench -impl=x ordering + test::LegacyBloomFilterPolicy::kClassName(), + test::FastLocalBloomFilterPolicy::kClassName(), + test::Standard128RibbonFilterPolicy::kClassName(), + }; + return impls; +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/filter_policy_internal.h b/src/rocksdb/table/block_based/filter_policy_internal.h new file mode 100644 index 000000000..9bc3a2482 --- /dev/null +++ b/src/rocksdb/table/block_based/filter_policy_internal.h @@ -0,0 +1,340 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// Copyright (c) 2012 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 <atomic> +#include <memory> +#include <string> +#include <vector> + +#include "rocksdb/filter_policy.h" +#include "rocksdb/table.h" + +namespace ROCKSDB_NAMESPACE { + +// A class that takes a bunch of keys, then generates filter +class FilterBitsBuilder { + public: + virtual ~FilterBitsBuilder() {} + + // Add a key (or prefix) to the filter. Typically, a builder will keep + // a set of 64-bit key hashes and only build the filter in Finish + // when the final number of keys is known. Keys are added in sorted order + // and duplicated keys are possible, so typically, the builder will + // only add this key if its hash is different from the most recently + // added. + virtual void AddKey(const Slice& key) = 0; + + // Called by RocksDB before Finish to populate + // TableProperties::num_filter_entries, so should represent the + // number of unique keys (and/or prefixes) added, but does not have + // to be exact. `return 0;` may be used to conspicuously indicate "unknown". + virtual size_t EstimateEntriesAdded() = 0; + + // Generate the filter using the keys that are added + // The return value of this function would be the filter bits, + // The ownership of actual data is set to buf + virtual Slice Finish(std::unique_ptr<const char[]>* buf) = 0; + + // Similar to Finish(std::unique_ptr<const char[]>* buf), except that + // for a non-null status pointer argument, it will point to + // Status::Corruption() when there is any corruption during filter + // construction or Status::OK() otherwise. + // + // WARNING: do not use a filter resulted from a corrupted construction + // TODO: refactor this to have a better signature, consolidate + virtual Slice Finish(std::unique_ptr<const char[]>* buf, + Status* /* status */) { + return Finish(buf); + } + + // Verify the filter returned from calling FilterBitsBuilder::Finish. + // The function returns Status::Corruption() if there is any corruption in the + // constructed filter or Status::OK() otherwise. + // + // Implementations should normally consult + // FilterBuildingContext::table_options.detect_filter_construct_corruption + // to determine whether to perform verification or to skip by returning + // Status::OK(). The decision is left to the FilterBitsBuilder so that + // verification prerequisites before PostVerify can be skipped when not + // configured. + // + // RocksDB internal will always call MaybePostVerify() on the filter after + // it is returned from calling FilterBitsBuilder::Finish + // except for FilterBitsBuilder::Finish resulting a corruption + // status, which indicates the filter is already in a corrupted state and + // there is no need to post-verify + virtual Status MaybePostVerify(const Slice& /* filter_content */) { + return Status::OK(); + } + + // Approximate the number of keys that can be added and generate a filter + // <= the specified number of bytes. Callers (including RocksDB) should + // only use this result for optimizing performance and not as a guarantee. + virtual size_t ApproximateNumEntries(size_t bytes) = 0; +}; + +// A class that checks if a key can be in filter +// It should be initialized by Slice generated by BitsBuilder +class FilterBitsReader { + public: + virtual ~FilterBitsReader() {} + + // Check if the entry match the bits in filter + virtual bool MayMatch(const Slice& entry) = 0; + + // Check if an array of entries match the bits in filter + virtual void MayMatch(int num_keys, Slice** keys, bool* may_match) { + for (int i = 0; i < num_keys; ++i) { + may_match[i] = MayMatch(*keys[i]); + } + } +}; + +// Exposes any extra information needed for testing built-in +// FilterBitsBuilders +class BuiltinFilterBitsBuilder : public FilterBitsBuilder { + public: + // Calculate number of bytes needed for a new filter, including + // metadata. Passing the result to ApproximateNumEntries should + // (ideally, usually) return >= the num_entry passed in. + // When optimize_filters_for_memory is enabled, this function + // is not authoritative but represents a target size that should + // be close to the average size. + virtual size_t CalculateSpace(size_t num_entries) = 0; + + // Returns an estimate of the FP rate of the returned filter if + // `num_entries` keys are added and the filter returned by Finish + // is `bytes` bytes. + virtual double EstimatedFpRate(size_t num_entries, size_t bytes) = 0; +}; + +// Base class for RocksDB built-in filter reader with +// extra useful functionalities for inernal. +class BuiltinFilterBitsReader : public FilterBitsReader { + public: + // Check if the hash of the entry match the bits in filter + virtual bool HashMayMatch(const uint64_t /* h */) { return true; } +}; + +// Base class for RocksDB built-in filter policies. This provides the +// ability to read all kinds of built-in filters (so that old filters can +// be used even when you change between built-in policies). +class BuiltinFilterPolicy : public FilterPolicy { + public: // overrides + // Read metadata to determine what kind of FilterBitsReader is needed + // and return a new one. This must successfully process any filter data + // generated by a built-in FilterBitsBuilder, regardless of the impl + // chosen for this BloomFilterPolicy. + FilterBitsReader* GetFilterBitsReader(const Slice& contents) const override; + static const char* kClassName(); + bool IsInstanceOf(const std::string& id) const override; + // All variants of BuiltinFilterPolicy can read each others filters. + const char* CompatibilityName() const override; + static const char* kCompatibilityName(); + + public: // new + // An internal function for the implementation of + // BuiltinFilterBitsReader::GetFilterBitsReader without requiring an instance + // or working around potential virtual overrides. + static BuiltinFilterBitsReader* GetBuiltinFilterBitsReader( + const Slice& contents); + + // Returns a new FilterBitsBuilder from the filter_policy in + // table_options of a context, or nullptr if not applicable. + // (An internal convenience function to save boilerplate.) + static FilterBitsBuilder* GetBuilderFromContext(const FilterBuildingContext&); + + private: + // For Bloom filter implementation(s) + static BuiltinFilterBitsReader* GetBloomBitsReader(const Slice& contents); + + // For Ribbon filter implementation(s) + static BuiltinFilterBitsReader* GetRibbonBitsReader(const Slice& contents); +}; + +// A "read only" filter policy used for backward compatibility with old +// OPTIONS files, which did not specifying a Bloom configuration, just +// "rocksdb.BuiltinBloomFilter". Although this can read existing filters, +// this policy does not build new filters, so new SST files generated +// under the policy will get no filters (like nullptr FilterPolicy). +// This class is considered internal API and subject to change. +class ReadOnlyBuiltinFilterPolicy : public BuiltinFilterPolicy { + public: + const char* Name() const override { return kClassName(); } + static const char* kClassName(); + + // Does not write filters. + FilterBitsBuilder* GetBuilderWithContext( + const FilterBuildingContext&) const override { + return nullptr; + } +}; + +// RocksDB built-in filter policy for Bloom or Bloom-like filters including +// Ribbon filters. +// This class is considered internal API and subject to change. +// See NewBloomFilterPolicy and NewRibbonFilterPolicy. +class BloomLikeFilterPolicy : public BuiltinFilterPolicy { + public: + explicit BloomLikeFilterPolicy(double bits_per_key); + + ~BloomLikeFilterPolicy() override; + static const char* kClassName(); + bool IsInstanceOf(const std::string& id) const override; + + std::string GetId() const override; + + // Essentially for testing only: configured millibits/key + int GetMillibitsPerKey() const { return millibits_per_key_; } + // Essentially for testing only: legacy whole bits/key + int GetWholeBitsPerKey() const { return whole_bits_per_key_; } + + // All the different underlying implementations that a BloomLikeFilterPolicy + // might use, as a configuration string name for a testing mode for + // "always use this implementation." Only appropriate for unit tests. + static const std::vector<std::string>& GetAllFixedImpls(); + + // Convenience function for creating by name for fixed impls + static std::shared_ptr<const FilterPolicy> Create(const std::string& name, + double bits_per_key); + + protected: + // Some implementations used by aggregating policies + FilterBitsBuilder* GetLegacyBloomBuilderWithContext( + const FilterBuildingContext& context) const; + FilterBitsBuilder* GetFastLocalBloomBuilderWithContext( + const FilterBuildingContext& context) const; + FilterBitsBuilder* GetStandard128RibbonBuilderWithContext( + const FilterBuildingContext& context) const; + + std::string GetBitsPerKeySuffix() const; + + private: + // Bits per key settings are for configuring Bloom filters. + + // Newer filters support fractional bits per key. For predictable behavior + // of 0.001-precision values across floating point implementations, we + // round to thousandths of a bit (on average) per key. + int millibits_per_key_; + + // Older filters round to whole number bits per key. (There *should* be no + // compatibility issue with fractional bits per key, but preserving old + // behavior with format_version < 5 just in case.) + int whole_bits_per_key_; + + // For configuring Ribbon filter: a desired value for 1/fp_rate. For + // example, 100 -> 1% fp rate. + double desired_one_in_fp_rate_; + + // Whether relevant warnings have been logged already. (Remember so we + // only report once per BloomFilterPolicy instance, to keep the noise down.) + mutable std::atomic<bool> warned_; + + // State for implementing optimize_filters_for_memory. Essentially, this + // tracks a surplus or deficit in total FP rate of filters generated by + // builders under this policy vs. what would have been generated without + // optimize_filters_for_memory. + // + // To avoid floating point weirdness, the actual value is + // Sum over all generated filters f: + // (predicted_fp_rate(f) - predicted_fp_rate(f|o_f_f_m=false)) * 2^32 + mutable std::atomic<int64_t> aggregate_rounding_balance_; +}; + +// For NewBloomFilterPolicy +// +// This is a user-facing policy that automatically choose between +// LegacyBloom and FastLocalBloom based on context at build time, +// including compatibility with format_version. +class BloomFilterPolicy : public BloomLikeFilterPolicy { + public: + explicit BloomFilterPolicy(double bits_per_key); + + // To use this function, call BuiltinFilterPolicy::GetBuilderFromContext(). + // + // Neither the context nor any objects therein should be saved beyond + // the call to this function, unless it's shared_ptr. + FilterBitsBuilder* GetBuilderWithContext( + const FilterBuildingContext&) const override; + + static const char* kClassName(); + const char* Name() const override { return kClassName(); } + static const char* kNickName(); + const char* NickName() const override { return kNickName(); } + std::string GetId() const override; +}; + +// For NewRibbonFilterPolicy +// +// This is a user-facing policy that chooses between Standard128Ribbon +// and FastLocalBloom based on context at build time (LSM level and other +// factors in extreme cases). +class RibbonFilterPolicy : public BloomLikeFilterPolicy { + public: + explicit RibbonFilterPolicy(double bloom_equivalent_bits_per_key, + int bloom_before_level); + + FilterBitsBuilder* GetBuilderWithContext( + const FilterBuildingContext&) const override; + + int GetBloomBeforeLevel() const { return bloom_before_level_; } + + static const char* kClassName(); + const char* Name() const override { return kClassName(); } + static const char* kNickName(); + const char* NickName() const override { return kNickName(); } + std::string GetId() const override; + + private: + const int bloom_before_level_; +}; + +// For testing only, but always constructable with internal names +namespace test { + +class LegacyBloomFilterPolicy : public BloomLikeFilterPolicy { + public: + explicit LegacyBloomFilterPolicy(double bits_per_key) + : BloomLikeFilterPolicy(bits_per_key) {} + + FilterBitsBuilder* GetBuilderWithContext( + const FilterBuildingContext& context) const override; + + static const char* kClassName(); + const char* Name() const override { return kClassName(); } +}; + +class FastLocalBloomFilterPolicy : public BloomLikeFilterPolicy { + public: + explicit FastLocalBloomFilterPolicy(double bits_per_key) + : BloomLikeFilterPolicy(bits_per_key) {} + + FilterBitsBuilder* GetBuilderWithContext( + const FilterBuildingContext& context) const override; + + static const char* kClassName(); + const char* Name() const override { return kClassName(); } +}; + +class Standard128RibbonFilterPolicy : public BloomLikeFilterPolicy { + public: + explicit Standard128RibbonFilterPolicy(double bloom_equiv_bits_per_key) + : BloomLikeFilterPolicy(bloom_equiv_bits_per_key) {} + + FilterBitsBuilder* GetBuilderWithContext( + const FilterBuildingContext& context) const override; + + static const char* kClassName(); + const char* Name() const override { return kClassName(); } +}; + +} // namespace test + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/flush_block_policy.cc b/src/rocksdb/table/block_based/flush_block_policy.cc new file mode 100644 index 000000000..9bb1f334b --- /dev/null +++ b/src/rocksdb/table/block_based/flush_block_policy.cc @@ -0,0 +1,146 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#include "rocksdb/flush_block_policy.h" + +#include <cassert> +#include <mutex> + +#include "rocksdb/options.h" +#include "rocksdb/slice.h" +#include "rocksdb/utilities/customizable_util.h" +#include "table/block_based/block_based_table_reader.h" +#include "table/block_based/block_builder.h" +#include "table/block_based/flush_block_policy.h" +#include "table/format.h" + +namespace ROCKSDB_NAMESPACE { + +// Flush block by size +class FlushBlockBySizePolicy : public FlushBlockPolicy { + public: + // @params block_size: Approximate size of user data packed per + // block. + // @params block_size_deviation: This is used to close a block before it + // reaches the configured + FlushBlockBySizePolicy(const uint64_t block_size, + const uint64_t block_size_deviation, const bool align, + const BlockBuilder& data_block_builder) + : block_size_(block_size), + block_size_deviation_limit_( + ((block_size * (100 - block_size_deviation)) + 99) / 100), + align_(align), + data_block_builder_(data_block_builder) {} + + bool Update(const Slice& key, const Slice& value) override { + // it makes no sense to flush when the data block is empty + if (data_block_builder_.empty()) { + return false; + } + + auto curr_size = data_block_builder_.CurrentSizeEstimate(); + + // Do flush if one of the below two conditions is true: + // 1) if the current estimated size already exceeds the block size, + // 2) block_size_deviation is set and the estimated size after appending + // the kv will exceed the block size and the current size is under the + // the deviation. + return curr_size >= block_size_ || BlockAlmostFull(key, value); + } + + private: + bool BlockAlmostFull(const Slice& key, const Slice& value) const { + if (block_size_deviation_limit_ == 0) { + return false; + } + + const auto curr_size = data_block_builder_.CurrentSizeEstimate(); + auto estimated_size_after = + data_block_builder_.EstimateSizeAfterKV(key, value); + + if (align_) { + estimated_size_after += BlockBasedTable::kBlockTrailerSize; + return estimated_size_after > block_size_; + } + + return estimated_size_after > block_size_ && + curr_size > block_size_deviation_limit_; + } + + const uint64_t block_size_; + const uint64_t block_size_deviation_limit_; + const bool align_; + const BlockBuilder& data_block_builder_; +}; + +FlushBlockPolicy* FlushBlockBySizePolicyFactory::NewFlushBlockPolicy( + const BlockBasedTableOptions& table_options, + const BlockBuilder& data_block_builder) const { + return new FlushBlockBySizePolicy( + table_options.block_size, table_options.block_size_deviation, + table_options.block_align, data_block_builder); +} + +FlushBlockPolicy* FlushBlockBySizePolicyFactory::NewFlushBlockPolicy( + const uint64_t size, const int deviation, + const BlockBuilder& data_block_builder) { + return new FlushBlockBySizePolicy(size, deviation, false, data_block_builder); +} + +#ifndef ROCKSDB_LITE +static int RegisterFlushBlockPolicyFactories(ObjectLibrary& library, + const std::string& /*arg*/) { + library.AddFactory<FlushBlockPolicyFactory>( + FlushBlockBySizePolicyFactory::kClassName(), + [](const std::string& /*uri*/, + std::unique_ptr<FlushBlockPolicyFactory>* guard, + std::string* /* errmsg */) { + guard->reset(new FlushBlockBySizePolicyFactory()); + return guard->get(); + }); + library.AddFactory<FlushBlockPolicyFactory>( + FlushBlockEveryKeyPolicyFactory::kClassName(), + [](const std::string& /*uri*/, + std::unique_ptr<FlushBlockPolicyFactory>* guard, + std::string* /* errmsg */) { + guard->reset(new FlushBlockEveryKeyPolicyFactory()); + return guard->get(); + }); + return 2; +} +#endif // ROCKSDB_LITE + +static bool LoadFlushPolicyFactory( + const std::string& id, std::shared_ptr<FlushBlockPolicyFactory>* result) { + if (id.empty()) { + result->reset(new FlushBlockBySizePolicyFactory()); +#ifdef ROCKSDB_LITE + } else if (id == FlushBlockBySizePolicyFactory::kClassName()) { + result->reset(new FlushBlockBySizePolicyFactory()); + } else if (id == FlushBlockEveryKeyPolicyFactory::kClassName()) { + result->reset(new FlushBlockEveryKeyPolicyFactory()); +#endif // ROCKSDB_LITE + } else { + return false; + } + return true; +} + +FlushBlockBySizePolicyFactory::FlushBlockBySizePolicyFactory() + : FlushBlockPolicyFactory() {} + +Status FlushBlockPolicyFactory::CreateFromString( + const ConfigOptions& config_options, const std::string& value, + std::shared_ptr<FlushBlockPolicyFactory>* factory) { +#ifndef ROCKSDB_LITE + static std::once_flag once; + std::call_once(once, [&]() { + RegisterFlushBlockPolicyFactories(*(ObjectLibrary::Default().get()), ""); + }); +#endif // ROCKSDB_LITE + return LoadSharedObject<FlushBlockPolicyFactory>( + config_options, value, LoadFlushPolicyFactory, factory); +} +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/flush_block_policy.h b/src/rocksdb/table/block_based/flush_block_policy.h new file mode 100644 index 000000000..4f79682bc --- /dev/null +++ b/src/rocksdb/table/block_based/flush_block_policy.h @@ -0,0 +1,40 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#include "rocksdb/flush_block_policy.h" + +namespace ROCKSDB_NAMESPACE { + +// FlushBlockEveryKeyPolicy currently used only in tests. + +class FlushBlockEveryKeyPolicy : public FlushBlockPolicy { + public: + bool Update(const Slice& /*key*/, const Slice& /*value*/) override { + if (!start_) { + start_ = true; + return false; + } + return true; + } + + private: + bool start_ = false; +}; + +class FlushBlockEveryKeyPolicyFactory : public FlushBlockPolicyFactory { + public: + explicit FlushBlockEveryKeyPolicyFactory() {} + + static const char* kClassName() { return "FlushBlockEveryKeyPolicyFactory"; } + const char* Name() const override { return kClassName(); } + + FlushBlockPolicy* NewFlushBlockPolicy( + const BlockBasedTableOptions& /*table_options*/, + const BlockBuilder& /*data_block_builder*/) const override { + return new FlushBlockEveryKeyPolicy; + } +}; + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/full_filter_block.cc b/src/rocksdb/table/block_based/full_filter_block.cc new file mode 100644 index 000000000..62b7a9eca --- /dev/null +++ b/src/rocksdb/table/block_based/full_filter_block.cc @@ -0,0 +1,296 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#include "table/block_based/full_filter_block.h" + +#include <array> + +#include "block_type.h" +#include "monitoring/perf_context_imp.h" +#include "port/malloc.h" +#include "port/port.h" +#include "rocksdb/filter_policy.h" +#include "table/block_based/block_based_table_reader.h" +#include "util/coding.h" + +namespace ROCKSDB_NAMESPACE { + +FullFilterBlockBuilder::FullFilterBlockBuilder( + const SliceTransform* _prefix_extractor, bool whole_key_filtering, + FilterBitsBuilder* filter_bits_builder) + : prefix_extractor_(_prefix_extractor), + whole_key_filtering_(whole_key_filtering), + last_whole_key_recorded_(false), + last_prefix_recorded_(false), + last_key_in_domain_(false), + any_added_(false) { + assert(filter_bits_builder != nullptr); + filter_bits_builder_.reset(filter_bits_builder); +} + +size_t FullFilterBlockBuilder::EstimateEntriesAdded() { + return filter_bits_builder_->EstimateEntriesAdded(); +} + +void FullFilterBlockBuilder::Add(const Slice& key_without_ts) { + const bool add_prefix = + prefix_extractor_ && prefix_extractor_->InDomain(key_without_ts); + + if (!last_prefix_recorded_ && last_key_in_domain_) { + // We can reach here when a new filter partition starts in partitioned + // filter. The last prefix in the previous partition should be added if + // necessary regardless of key_without_ts, to support prefix SeekForPrev. + AddKey(last_prefix_str_); + last_prefix_recorded_ = true; + } + + if (whole_key_filtering_) { + if (!add_prefix) { + AddKey(key_without_ts); + } else { + // if both whole_key and prefix are added to bloom then we will have whole + // key_without_ts and prefix addition being interleaved and thus cannot + // rely on the bits builder to properly detect the duplicates by comparing + // with the last item. + Slice last_whole_key = Slice(last_whole_key_str_); + if (!last_whole_key_recorded_ || + last_whole_key.compare(key_without_ts) != 0) { + AddKey(key_without_ts); + last_whole_key_recorded_ = true; + last_whole_key_str_.assign(key_without_ts.data(), + key_without_ts.size()); + } + } + } + if (add_prefix) { + last_key_in_domain_ = true; + AddPrefix(key_without_ts); + } else { + last_key_in_domain_ = false; + } +} + +// Add key to filter if needed +inline void FullFilterBlockBuilder::AddKey(const Slice& key) { + filter_bits_builder_->AddKey(key); + any_added_ = true; +} + +// Add prefix to filter if needed +void FullFilterBlockBuilder::AddPrefix(const Slice& key) { + assert(prefix_extractor_ && prefix_extractor_->InDomain(key)); + Slice prefix = prefix_extractor_->Transform(key); + if (whole_key_filtering_) { + // if both whole_key and prefix are added to bloom then we will have whole + // key and prefix addition being interleaved and thus cannot rely on the + // bits builder to properly detect the duplicates by comparing with the last + // item. + Slice last_prefix = Slice(last_prefix_str_); + if (!last_prefix_recorded_ || last_prefix.compare(prefix) != 0) { + AddKey(prefix); + last_prefix_recorded_ = true; + last_prefix_str_.assign(prefix.data(), prefix.size()); + } + } else { + AddKey(prefix); + } +} + +void FullFilterBlockBuilder::Reset() { + last_whole_key_recorded_ = false; + last_prefix_recorded_ = false; +} + +Slice FullFilterBlockBuilder::Finish( + const BlockHandle& /*tmp*/, Status* status, + std::unique_ptr<const char[]>* filter_data) { + Reset(); + // In this impl we ignore BlockHandle + *status = Status::OK(); + if (any_added_) { + any_added_ = false; + Slice filter_content = filter_bits_builder_->Finish( + filter_data ? filter_data : &filter_data_, status); + return filter_content; + } + return Slice(); +} + +FullFilterBlockReader::FullFilterBlockReader( + const BlockBasedTable* t, + CachableEntry<ParsedFullFilterBlock>&& filter_block) + : FilterBlockReaderCommon(t, std::move(filter_block)) {} + +bool FullFilterBlockReader::KeyMayMatch(const Slice& key, const bool no_io, + const Slice* const /*const_ikey_ptr*/, + GetContext* get_context, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) { + if (!whole_key_filtering()) { + return true; + } + return MayMatch(key, no_io, get_context, lookup_context, + rate_limiter_priority); +} + +std::unique_ptr<FilterBlockReader> FullFilterBlockReader::Create( + const BlockBasedTable* table, const ReadOptions& ro, + FilePrefetchBuffer* prefetch_buffer, bool use_cache, bool prefetch, + bool pin, BlockCacheLookupContext* lookup_context) { + assert(table); + assert(table->get_rep()); + assert(!pin || prefetch); + + CachableEntry<ParsedFullFilterBlock> filter_block; + if (prefetch || !use_cache) { + const Status s = ReadFilterBlock(table, prefetch_buffer, ro, use_cache, + nullptr /* get_context */, lookup_context, + &filter_block, BlockType::kFilter); + if (!s.ok()) { + IGNORE_STATUS_IF_ERROR(s); + return std::unique_ptr<FilterBlockReader>(); + } + + if (use_cache && !pin) { + filter_block.Reset(); + } + } + + return std::unique_ptr<FilterBlockReader>( + new FullFilterBlockReader(table, std::move(filter_block))); +} + +bool FullFilterBlockReader::PrefixMayMatch( + const Slice& prefix, const bool no_io, + const Slice* const /*const_ikey_ptr*/, GetContext* get_context, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) { + return MayMatch(prefix, no_io, get_context, lookup_context, + rate_limiter_priority); +} + +bool FullFilterBlockReader::MayMatch( + const Slice& entry, bool no_io, GetContext* get_context, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) const { + CachableEntry<ParsedFullFilterBlock> filter_block; + + const Status s = + GetOrReadFilterBlock(no_io, get_context, lookup_context, &filter_block, + BlockType::kFilter, rate_limiter_priority); + if (!s.ok()) { + IGNORE_STATUS_IF_ERROR(s); + return true; + } + + assert(filter_block.GetValue()); + + FilterBitsReader* const filter_bits_reader = + filter_block.GetValue()->filter_bits_reader(); + + if (filter_bits_reader) { + if (filter_bits_reader->MayMatch(entry)) { + PERF_COUNTER_ADD(bloom_sst_hit_count, 1); + return true; + } else { + PERF_COUNTER_ADD(bloom_sst_miss_count, 1); + return false; + } + } + return true; +} + +void FullFilterBlockReader::KeysMayMatch( + MultiGetRange* range, const bool no_io, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) { + if (!whole_key_filtering()) { + // Simply return. Don't skip any key - consider all keys as likely to be + // present + return; + } + MayMatch(range, no_io, nullptr, lookup_context, rate_limiter_priority); +} + +void FullFilterBlockReader::PrefixesMayMatch( + MultiGetRange* range, const SliceTransform* prefix_extractor, + const bool no_io, BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) { + MayMatch(range, no_io, prefix_extractor, lookup_context, + rate_limiter_priority); +} + +void FullFilterBlockReader::MayMatch( + MultiGetRange* range, bool no_io, const SliceTransform* prefix_extractor, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) const { + CachableEntry<ParsedFullFilterBlock> filter_block; + + const Status s = GetOrReadFilterBlock( + no_io, range->begin()->get_context, lookup_context, &filter_block, + BlockType::kFilter, rate_limiter_priority); + if (!s.ok()) { + IGNORE_STATUS_IF_ERROR(s); + return; + } + + assert(filter_block.GetValue()); + + FilterBitsReader* const filter_bits_reader = + filter_block.GetValue()->filter_bits_reader(); + + if (!filter_bits_reader) { + return; + } + + // We need to use an array instead of autovector for may_match since + // &may_match[0] doesn't work for autovector<bool> (compiler error). So + // declare both keys and may_match as arrays, which is also slightly less + // expensive compared to autovector + std::array<Slice*, MultiGetContext::MAX_BATCH_SIZE> keys; + std::array<bool, MultiGetContext::MAX_BATCH_SIZE> may_match = {{true}}; + autovector<Slice, MultiGetContext::MAX_BATCH_SIZE> prefixes; + int num_keys = 0; + MultiGetRange filter_range(*range, range->begin(), range->end()); + for (auto iter = filter_range.begin(); iter != filter_range.end(); ++iter) { + if (!prefix_extractor) { + keys[num_keys++] = &iter->ukey_without_ts; + } else if (prefix_extractor->InDomain(iter->ukey_without_ts)) { + prefixes.emplace_back(prefix_extractor->Transform(iter->ukey_without_ts)); + keys[num_keys++] = &prefixes.back(); + } else { + filter_range.SkipKey(iter); + } + } + + filter_bits_reader->MayMatch(num_keys, &keys[0], &may_match[0]); + + int i = 0; + for (auto iter = filter_range.begin(); iter != filter_range.end(); ++iter) { + if (!may_match[i]) { + // Update original MultiGet range to skip this key. The filter_range + // was temporarily used just to skip keys not in prefix_extractor domain + range->SkipKey(iter); + PERF_COUNTER_ADD(bloom_sst_miss_count, 1); + } else { + // PERF_COUNTER_ADD(bloom_sst_hit_count, 1); + PerfContext* perf_ctx = get_perf_context(); + perf_ctx->bloom_sst_hit_count++; + } + ++i; + } +} + +size_t FullFilterBlockReader::ApproximateMemoryUsage() const { + size_t usage = ApproximateFilterBlockMemoryUsage(); +#ifdef ROCKSDB_MALLOC_USABLE_SIZE + usage += malloc_usable_size(const_cast<FullFilterBlockReader*>(this)); +#else + usage += sizeof(*this); +#endif // ROCKSDB_MALLOC_USABLE_SIZE + return usage; +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/full_filter_block.h b/src/rocksdb/table/block_based/full_filter_block.h new file mode 100644 index 000000000..cd1771a38 --- /dev/null +++ b/src/rocksdb/table/block_based/full_filter_block.h @@ -0,0 +1,147 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#pragma once + +#include <stddef.h> +#include <stdint.h> + +#include <memory> +#include <string> +#include <vector> + +#include "rocksdb/options.h" +#include "rocksdb/slice.h" +#include "rocksdb/slice_transform.h" +#include "table/block_based/filter_block_reader_common.h" +#include "table/block_based/filter_policy_internal.h" +#include "table/block_based/parsed_full_filter_block.h" +#include "util/hash.h" + +namespace ROCKSDB_NAMESPACE { + +class FilterPolicy; +class FilterBitsBuilder; +class FilterBitsReader; + +// A FullFilterBlockBuilder is used to construct a full filter for a +// particular Table. It generates a single string which is stored as +// a special block in the Table. +// The format of full filter block is: +// +----------------------------------------------------------------+ +// | full filter for all keys in sst file | +// +----------------------------------------------------------------+ +// The full filter can be very large. At the end of it, we put +// num_probes: how many hash functions are used in bloom filter +// +class FullFilterBlockBuilder : public FilterBlockBuilder { + public: + explicit FullFilterBlockBuilder(const SliceTransform* prefix_extractor, + bool whole_key_filtering, + FilterBitsBuilder* filter_bits_builder); + // No copying allowed + FullFilterBlockBuilder(const FullFilterBlockBuilder&) = delete; + void operator=(const FullFilterBlockBuilder&) = delete; + + // bits_builder is created in filter_policy, it should be passed in here + // directly. and be deleted here + ~FullFilterBlockBuilder() {} + + virtual void Add(const Slice& key_without_ts) override; + virtual bool IsEmpty() const override { return !any_added_; } + virtual size_t EstimateEntriesAdded() override; + virtual Slice Finish( + const BlockHandle& tmp, Status* status, + std::unique_ptr<const char[]>* filter_data = nullptr) override; + using FilterBlockBuilder::Finish; + + virtual void ResetFilterBitsBuilder() override { + filter_bits_builder_.reset(); + } + + virtual Status MaybePostVerifyFilter(const Slice& filter_content) override { + return filter_bits_builder_->MaybePostVerify(filter_content); + } + + protected: + virtual void AddKey(const Slice& key); + std::unique_ptr<FilterBitsBuilder> filter_bits_builder_; + virtual void Reset(); + void AddPrefix(const Slice& key); + const SliceTransform* prefix_extractor() { return prefix_extractor_; } + const std::string& last_prefix_str() const { return last_prefix_str_; } + + private: + // important: all of these might point to invalid addresses + // at the time of destruction of this filter block. destructor + // should NOT dereference them. + const SliceTransform* prefix_extractor_; + bool whole_key_filtering_; + bool last_whole_key_recorded_; + std::string last_whole_key_str_; + bool last_prefix_recorded_; + std::string last_prefix_str_; + // Whether prefix_extractor_->InDomain(last_whole_key_) is true. + // Used in partitioned filters so that the last prefix from the previous + // filter partition will be added to the current partition if + // last_key_in_domain_ is true, regardless of the current key. + bool last_key_in_domain_; + bool any_added_; + std::unique_ptr<const char[]> filter_data_; +}; + +// A FilterBlockReader is used to parse filter from SST table. +// KeyMayMatch and PrefixMayMatch would trigger filter checking +class FullFilterBlockReader + : public FilterBlockReaderCommon<ParsedFullFilterBlock> { + public: + FullFilterBlockReader(const BlockBasedTable* t, + CachableEntry<ParsedFullFilterBlock>&& filter_block); + + static std::unique_ptr<FilterBlockReader> Create( + const BlockBasedTable* table, const ReadOptions& ro, + FilePrefetchBuffer* prefetch_buffer, bool use_cache, bool prefetch, + bool pin, BlockCacheLookupContext* lookup_context); + + bool KeyMayMatch(const Slice& key, const bool no_io, + const Slice* const const_ikey_ptr, GetContext* get_context, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) override; + + bool PrefixMayMatch(const Slice& prefix, const bool no_io, + const Slice* const const_ikey_ptr, + GetContext* get_context, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) override; + + void KeysMayMatch(MultiGetRange* range, const bool no_io, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) override; + // Used in partitioned filter code + void KeysMayMatch2(MultiGetRange* range, + const SliceTransform* /*prefix_extractor*/, + const bool no_io, BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) { + KeysMayMatch(range, no_io, lookup_context, rate_limiter_priority); + } + + void PrefixesMayMatch(MultiGetRange* range, + const SliceTransform* prefix_extractor, + const bool no_io, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) override; + size_t ApproximateMemoryUsage() const override; + + private: + bool MayMatch(const Slice& entry, bool no_io, GetContext* get_context, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) const; + void MayMatch(MultiGetRange* range, bool no_io, + const SliceTransform* prefix_extractor, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) const; +}; + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/full_filter_block_test.cc b/src/rocksdb/table/block_based/full_filter_block_test.cc new file mode 100644 index 000000000..bd98638e5 --- /dev/null +++ b/src/rocksdb/table/block_based/full_filter_block_test.cc @@ -0,0 +1,339 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#include "table/block_based/full_filter_block.h" + +#include <set> + +#include "rocksdb/filter_policy.h" +#include "rocksdb/status.h" +#include "table/block_based/block_based_table_reader.h" +#include "table/block_based/filter_policy_internal.h" +#include "table/block_based/mock_block_based_table.h" +#include "table/format.h" +#include "test_util/testharness.h" +#include "test_util/testutil.h" +#include "util/coding.h" +#include "util/hash.h" +#include "util/string_util.h" + +namespace ROCKSDB_NAMESPACE { + +class TestFilterBitsBuilder : public FilterBitsBuilder { + public: + explicit TestFilterBitsBuilder() {} + + // Add Key to filter + void AddKey(const Slice& key) override { + hash_entries_.push_back(Hash(key.data(), key.size(), 1)); + } + + using FilterBitsBuilder::Finish; + + // Generate the filter using the keys that are added + Slice Finish(std::unique_ptr<const char[]>* buf) override { + uint32_t len = static_cast<uint32_t>(hash_entries_.size()) * 4; + char* data = new char[len]; + for (size_t i = 0; i < hash_entries_.size(); i++) { + EncodeFixed32(data + i * 4, hash_entries_[i]); + } + const char* const_data = data; + buf->reset(const_data); + return Slice(data, len); + } + + size_t EstimateEntriesAdded() override { return hash_entries_.size(); } + + size_t ApproximateNumEntries(size_t bytes) override { return bytes / 4; } + + private: + std::vector<uint32_t> hash_entries_; +}; + +class MockBlockBasedTable : public BlockBasedTable { + public: + explicit MockBlockBasedTable(Rep* rep) + : BlockBasedTable(rep, nullptr /* block_cache_tracer */) {} +}; + +class TestFilterBitsReader : public FilterBitsReader { + public: + explicit TestFilterBitsReader(const Slice& contents) + : data_(contents.data()), len_(static_cast<uint32_t>(contents.size())) {} + + // Silence compiler warning about overloaded virtual + using FilterBitsReader::MayMatch; + bool MayMatch(const Slice& entry) override { + uint32_t h = Hash(entry.data(), entry.size(), 1); + for (size_t i = 0; i + 4 <= len_; i += 4) { + if (h == DecodeFixed32(data_ + i)) { + return true; + } + } + return false; + } + + private: + const char* data_; + uint32_t len_; +}; + +class TestHashFilter : public FilterPolicy { + public: + const char* Name() const override { return "TestHashFilter"; } + const char* CompatibilityName() const override { return Name(); } + + FilterBitsBuilder* GetBuilderWithContext( + const FilterBuildingContext&) const override { + return new TestFilterBitsBuilder(); + } + + FilterBitsReader* GetFilterBitsReader(const Slice& contents) const override { + return new TestFilterBitsReader(contents); + } +}; + +class PluginFullFilterBlockTest : public mock::MockBlockBasedTableTester, + public testing::Test { + public: + PluginFullFilterBlockTest() + : mock::MockBlockBasedTableTester(new TestHashFilter) {} +}; + +TEST_F(PluginFullFilterBlockTest, PluginEmptyBuilder) { + FullFilterBlockBuilder builder(nullptr, true, GetBuilder()); + Slice slice = builder.Finish(); + ASSERT_EQ("", EscapeString(slice)); + + CachableEntry<ParsedFullFilterBlock> block( + new ParsedFullFilterBlock(table_options_.filter_policy.get(), + BlockContents(slice)), + nullptr /* cache */, nullptr /* cache_handle */, true /* own_value */); + + FullFilterBlockReader reader(table_.get(), std::move(block)); + // Remain same symantic with blockbased filter + ASSERT_TRUE(reader.KeyMayMatch("foo", + /*no_io=*/false, /*const_ikey_ptr=*/nullptr, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, Env::IO_TOTAL)); +} + +TEST_F(PluginFullFilterBlockTest, PluginSingleChunk) { + FullFilterBlockBuilder builder(nullptr, true, GetBuilder()); + builder.Add("foo"); + builder.Add("bar"); + builder.Add("box"); + builder.Add("box"); + builder.Add("hello"); + Slice slice = builder.Finish(); + + CachableEntry<ParsedFullFilterBlock> block( + new ParsedFullFilterBlock(table_options_.filter_policy.get(), + BlockContents(slice)), + nullptr /* cache */, nullptr /* cache_handle */, true /* own_value */); + + FullFilterBlockReader reader(table_.get(), std::move(block)); + Env::IOPriority rate_limiter_priority = Env::IO_TOTAL; + ASSERT_TRUE(reader.KeyMayMatch("foo", + /*no_io=*/false, /*const_ikey_ptr=*/nullptr, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, + rate_limiter_priority)); + ASSERT_TRUE(reader.KeyMayMatch("bar", + /*no_io=*/false, /*const_ikey_ptr=*/nullptr, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, + rate_limiter_priority)); + ASSERT_TRUE(reader.KeyMayMatch("box", + /*no_io=*/false, /*const_ikey_ptr=*/nullptr, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, + rate_limiter_priority)); + ASSERT_TRUE(reader.KeyMayMatch("hello", + /*no_io=*/false, /*const_ikey_ptr=*/nullptr, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, + rate_limiter_priority)); + ASSERT_TRUE(reader.KeyMayMatch("foo", + /*no_io=*/false, /*const_ikey_ptr=*/nullptr, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, + rate_limiter_priority)); + ASSERT_TRUE(!reader.KeyMayMatch("missing", + /*no_io=*/false, /*const_ikey_ptr=*/nullptr, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, + rate_limiter_priority)); + ASSERT_TRUE(!reader.KeyMayMatch("other", + /*no_io=*/false, /*const_ikey_ptr=*/nullptr, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, + rate_limiter_priority)); +} + +class FullFilterBlockTest : public mock::MockBlockBasedTableTester, + public testing::Test { + public: + FullFilterBlockTest() + : mock::MockBlockBasedTableTester(NewBloomFilterPolicy(10, false)) {} +}; + +TEST_F(FullFilterBlockTest, EmptyBuilder) { + FullFilterBlockBuilder builder(nullptr, true, GetBuilder()); + Slice slice = builder.Finish(); + ASSERT_EQ("", EscapeString(slice)); + + CachableEntry<ParsedFullFilterBlock> block( + new ParsedFullFilterBlock(table_options_.filter_policy.get(), + BlockContents(slice)), + nullptr /* cache */, nullptr /* cache_handle */, true /* own_value */); + + FullFilterBlockReader reader(table_.get(), std::move(block)); + // Remain same symantic with blockbased filter + ASSERT_TRUE(reader.KeyMayMatch("foo", + /*no_io=*/false, /*const_ikey_ptr=*/nullptr, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, Env::IO_TOTAL)); +} + +class CountUniqueFilterBitsBuilderWrapper : public FilterBitsBuilder { + std::unique_ptr<FilterBitsBuilder> b_; + std::set<std::string> uniq_; + + public: + explicit CountUniqueFilterBitsBuilderWrapper(FilterBitsBuilder* b) : b_(b) {} + + ~CountUniqueFilterBitsBuilderWrapper() override {} + + void AddKey(const Slice& key) override { + b_->AddKey(key); + uniq_.insert(key.ToString()); + } + + using FilterBitsBuilder::Finish; + + Slice Finish(std::unique_ptr<const char[]>* buf) override { + Slice rv = b_->Finish(buf); + Status s_dont_care = b_->MaybePostVerify(rv); + s_dont_care.PermitUncheckedError(); + uniq_.clear(); + return rv; + } + + size_t EstimateEntriesAdded() override { return b_->EstimateEntriesAdded(); } + + size_t ApproximateNumEntries(size_t bytes) override { + return b_->ApproximateNumEntries(bytes); + } + + size_t CountUnique() { return uniq_.size(); } +}; + +TEST_F(FullFilterBlockTest, DuplicateEntries) { + { // empty prefixes + std::unique_ptr<const SliceTransform> prefix_extractor( + NewFixedPrefixTransform(0)); + auto bits_builder = new CountUniqueFilterBitsBuilderWrapper(GetBuilder()); + const bool WHOLE_KEY = true; + FullFilterBlockBuilder builder(prefix_extractor.get(), WHOLE_KEY, + bits_builder); + ASSERT_EQ(0, bits_builder->CountUnique()); + // adds key and empty prefix; both abstractions count them + builder.Add("key1"); + ASSERT_EQ(2, bits_builder->CountUnique()); + // Add different key (unique) and also empty prefix (not unique). + // From here in this test, it's immaterial whether the block builder + // can count unique keys. + builder.Add("key2"); + ASSERT_EQ(3, bits_builder->CountUnique()); + // Empty key -> nothing unique + builder.Add(""); + ASSERT_EQ(3, bits_builder->CountUnique()); + } + + // mix of empty and non-empty + std::unique_ptr<const SliceTransform> prefix_extractor( + NewFixedPrefixTransform(7)); + auto bits_builder = new CountUniqueFilterBitsBuilderWrapper(GetBuilder()); + const bool WHOLE_KEY = true; + FullFilterBlockBuilder builder(prefix_extractor.get(), WHOLE_KEY, + bits_builder); + builder.Add(""); // test with empty key too + builder.Add("prefix1key1"); + builder.Add("prefix1key1"); + builder.Add("prefix1key2"); + builder.Add("prefix1key3"); + builder.Add("prefix2key4"); + // 1 empty, 2 non-empty prefixes, and 4 non-empty keys + ASSERT_EQ(1 + 2 + 4, bits_builder->CountUnique()); +} + +TEST_F(FullFilterBlockTest, SingleChunk) { + FullFilterBlockBuilder builder(nullptr, true, GetBuilder()); + ASSERT_TRUE(builder.IsEmpty()); + builder.Add("foo"); + ASSERT_FALSE(builder.IsEmpty()); + builder.Add("bar"); + builder.Add("box"); + builder.Add("box"); + builder.Add("hello"); + // "box" only counts once + ASSERT_EQ(4, builder.EstimateEntriesAdded()); + ASSERT_FALSE(builder.IsEmpty()); + Status s; + Slice slice = builder.Finish(BlockHandle(), &s); + ASSERT_OK(s); + + CachableEntry<ParsedFullFilterBlock> block( + new ParsedFullFilterBlock(table_options_.filter_policy.get(), + BlockContents(slice)), + nullptr /* cache */, nullptr /* cache_handle */, true /* own_value */); + + FullFilterBlockReader reader(table_.get(), std::move(block)); + Env::IOPriority rate_limiter_priority = Env::IO_TOTAL; + ASSERT_TRUE(reader.KeyMayMatch("foo", + /*no_io=*/false, /*const_ikey_ptr=*/nullptr, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, + rate_limiter_priority)); + ASSERT_TRUE(reader.KeyMayMatch("bar", + /*no_io=*/false, /*const_ikey_ptr=*/nullptr, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, + rate_limiter_priority)); + ASSERT_TRUE(reader.KeyMayMatch("box", + /*no_io=*/false, /*const_ikey_ptr=*/nullptr, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, + rate_limiter_priority)); + ASSERT_TRUE(reader.KeyMayMatch("hello", + /*no_io=*/false, /*const_ikey_ptr=*/nullptr, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, + rate_limiter_priority)); + ASSERT_TRUE(reader.KeyMayMatch("foo", + /*no_io=*/false, /*const_ikey_ptr=*/nullptr, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, + rate_limiter_priority)); + ASSERT_TRUE(!reader.KeyMayMatch("missing", + /*no_io=*/false, /*const_ikey_ptr=*/nullptr, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, + rate_limiter_priority)); + ASSERT_TRUE(!reader.KeyMayMatch("other", + /*no_io=*/false, /*const_ikey_ptr=*/nullptr, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, + rate_limiter_priority)); +} + +} // namespace ROCKSDB_NAMESPACE + +int main(int argc, char** argv) { + ROCKSDB_NAMESPACE::port::InstallStackTraceHandler(); + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/src/rocksdb/table/block_based/hash_index_reader.cc b/src/rocksdb/table/block_based/hash_index_reader.cc new file mode 100644 index 000000000..bcaba17a2 --- /dev/null +++ b/src/rocksdb/table/block_based/hash_index_reader.cc @@ -0,0 +1,148 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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/block_based/hash_index_reader.h" + +#include "table/block_fetcher.h" +#include "table/meta_blocks.h" + +namespace ROCKSDB_NAMESPACE { +Status HashIndexReader::Create(const BlockBasedTable* table, + const ReadOptions& ro, + FilePrefetchBuffer* prefetch_buffer, + InternalIterator* meta_index_iter, + bool use_cache, bool prefetch, bool pin, + BlockCacheLookupContext* lookup_context, + std::unique_ptr<IndexReader>* index_reader) { + assert(table != nullptr); + assert(index_reader != nullptr); + assert(!pin || prefetch); + + const BlockBasedTable::Rep* rep = table->get_rep(); + assert(rep != nullptr); + + CachableEntry<Block> index_block; + if (prefetch || !use_cache) { + const Status s = + ReadIndexBlock(table, prefetch_buffer, ro, use_cache, + /*get_context=*/nullptr, lookup_context, &index_block); + if (!s.ok()) { + return s; + } + + if (use_cache && !pin) { + index_block.Reset(); + } + } + + // Note, failure to create prefix hash index does not need to be a + // hard error. We can still fall back to the original binary search index. + // So, Create will succeed regardless, from this point on. + + index_reader->reset(new HashIndexReader(table, std::move(index_block))); + + // Get prefixes block + BlockHandle prefixes_handle; + Status s = + FindMetaBlock(meta_index_iter, kHashIndexPrefixesBlock, &prefixes_handle); + if (!s.ok()) { + // TODO: log error + return Status::OK(); + } + + // Get index metadata block + BlockHandle prefixes_meta_handle; + s = FindMetaBlock(meta_index_iter, kHashIndexPrefixesMetadataBlock, + &prefixes_meta_handle); + if (!s.ok()) { + // TODO: log error + return Status::OK(); + } + + RandomAccessFileReader* const file = rep->file.get(); + const Footer& footer = rep->footer; + const ImmutableOptions& ioptions = rep->ioptions; + const PersistentCacheOptions& cache_options = rep->persistent_cache_options; + MemoryAllocator* const memory_allocator = + GetMemoryAllocator(rep->table_options); + + // Read contents for the blocks + BlockContents prefixes_contents; + BlockFetcher prefixes_block_fetcher( + file, prefetch_buffer, footer, ReadOptions(), prefixes_handle, + &prefixes_contents, ioptions, true /*decompress*/, + true /*maybe_compressed*/, BlockType::kHashIndexPrefixes, + UncompressionDict::GetEmptyDict(), cache_options, memory_allocator); + s = prefixes_block_fetcher.ReadBlockContents(); + if (!s.ok()) { + return s; + } + BlockContents prefixes_meta_contents; + BlockFetcher prefixes_meta_block_fetcher( + file, prefetch_buffer, footer, ReadOptions(), prefixes_meta_handle, + &prefixes_meta_contents, ioptions, true /*decompress*/, + true /*maybe_compressed*/, BlockType::kHashIndexMetadata, + UncompressionDict::GetEmptyDict(), cache_options, memory_allocator); + s = prefixes_meta_block_fetcher.ReadBlockContents(); + if (!s.ok()) { + // TODO: log error + return Status::OK(); + } + + BlockPrefixIndex* prefix_index = nullptr; + assert(rep->table_prefix_extractor); + s = BlockPrefixIndex::Create(rep->table_prefix_extractor.get(), + prefixes_contents.data, + prefixes_meta_contents.data, &prefix_index); + // TODO: log error + if (s.ok()) { + HashIndexReader* const hash_index_reader = + static_cast<HashIndexReader*>(index_reader->get()); + hash_index_reader->prefix_index_.reset(prefix_index); + } + + return Status::OK(); +} + +InternalIteratorBase<IndexValue>* HashIndexReader::NewIterator( + const ReadOptions& read_options, bool disable_prefix_seek, + IndexBlockIter* iter, GetContext* get_context, + BlockCacheLookupContext* lookup_context) { + const BlockBasedTable::Rep* rep = table()->get_rep(); + const bool no_io = (read_options.read_tier == kBlockCacheTier); + CachableEntry<Block> index_block; + const Status s = + GetOrReadIndexBlock(no_io, read_options.rate_limiter_priority, + get_context, lookup_context, &index_block); + if (!s.ok()) { + if (iter != nullptr) { + iter->Invalidate(s); + return iter; + } + + return NewErrorInternalIterator<IndexValue>(s); + } + + Statistics* kNullStats = nullptr; + const bool total_order_seek = + read_options.total_order_seek || disable_prefix_seek; + // We don't return pinned data from index blocks, so no need + // to set `block_contents_pinned`. + auto it = index_block.GetValue()->NewIndexIterator( + internal_comparator()->user_comparator(), + rep->get_global_seqno(BlockType::kIndex), iter, kNullStats, + total_order_seek, index_has_first_key(), index_key_includes_seq(), + index_value_is_full(), false /* block_contents_pinned */, + prefix_index_.get()); + + assert(it != nullptr); + index_block.TransferTo(it); + + return it; +} +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/hash_index_reader.h b/src/rocksdb/table/block_based/hash_index_reader.h new file mode 100644 index 000000000..9037efc87 --- /dev/null +++ b/src/rocksdb/table/block_based/hash_index_reader.h @@ -0,0 +1,49 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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 "table/block_based/index_reader_common.h" + +namespace ROCKSDB_NAMESPACE { +// Index that leverages an internal hash table to quicken the lookup for a given +// key. +class HashIndexReader : public BlockBasedTable::IndexReaderCommon { + public: + static Status Create(const BlockBasedTable* table, const ReadOptions& ro, + FilePrefetchBuffer* prefetch_buffer, + InternalIterator* meta_index_iter, bool use_cache, + bool prefetch, bool pin, + BlockCacheLookupContext* lookup_context, + std::unique_ptr<IndexReader>* index_reader); + + InternalIteratorBase<IndexValue>* NewIterator( + const ReadOptions& read_options, bool disable_prefix_seek, + IndexBlockIter* iter, GetContext* get_context, + BlockCacheLookupContext* lookup_context) override; + + size_t ApproximateMemoryUsage() const override { + size_t usage = ApproximateIndexBlockMemoryUsage(); +#ifdef ROCKSDB_MALLOC_USABLE_SIZE + usage += malloc_usable_size(const_cast<HashIndexReader*>(this)); +#else + if (prefix_index_) { + usage += prefix_index_->ApproximateMemoryUsage(); + } + usage += sizeof(*this); +#endif // ROCKSDB_MALLOC_USABLE_SIZE + return usage; + } + + private: + HashIndexReader(const BlockBasedTable* t, CachableEntry<Block>&& index_block) + : IndexReaderCommon(t, std::move(index_block)) {} + + std::unique_ptr<BlockPrefixIndex> prefix_index_; +}; +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/index_builder.cc b/src/rocksdb/table/block_based/index_builder.cc new file mode 100644 index 000000000..024730178 --- /dev/null +++ b/src/rocksdb/table/block_based/index_builder.cc @@ -0,0 +1,282 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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/block_based/index_builder.h" + +#include <assert.h> + +#include <cinttypes> +#include <list> +#include <string> + +#include "db/dbformat.h" +#include "rocksdb/comparator.h" +#include "rocksdb/flush_block_policy.h" +#include "table/block_based/partitioned_filter_block.h" +#include "table/format.h" + +namespace ROCKSDB_NAMESPACE { + +// Create a index builder based on its type. +IndexBuilder* IndexBuilder::CreateIndexBuilder( + BlockBasedTableOptions::IndexType index_type, + const InternalKeyComparator* comparator, + const InternalKeySliceTransform* int_key_slice_transform, + const bool use_value_delta_encoding, + const BlockBasedTableOptions& table_opt) { + IndexBuilder* result = nullptr; + switch (index_type) { + case BlockBasedTableOptions::kBinarySearch: { + result = new ShortenedIndexBuilder( + comparator, table_opt.index_block_restart_interval, + table_opt.format_version, use_value_delta_encoding, + table_opt.index_shortening, /* include_first_key */ false); + break; + } + case BlockBasedTableOptions::kHashSearch: { + // Currently kHashSearch is incompatible with index_block_restart_interval + // > 1 + assert(table_opt.index_block_restart_interval == 1); + result = new HashIndexBuilder( + comparator, int_key_slice_transform, + table_opt.index_block_restart_interval, table_opt.format_version, + use_value_delta_encoding, table_opt.index_shortening); + break; + } + case BlockBasedTableOptions::kTwoLevelIndexSearch: { + result = PartitionedIndexBuilder::CreateIndexBuilder( + comparator, use_value_delta_encoding, table_opt); + break; + } + case BlockBasedTableOptions::kBinarySearchWithFirstKey: { + result = new ShortenedIndexBuilder( + comparator, table_opt.index_block_restart_interval, + table_opt.format_version, use_value_delta_encoding, + table_opt.index_shortening, /* include_first_key */ true); + break; + } + default: { + assert(!"Do not recognize the index type "); + break; + } + } + return result; +} + +void ShortenedIndexBuilder::FindShortestInternalKeySeparator( + const Comparator& comparator, std::string* start, const Slice& limit) { + // Attempt to shorten the user portion of the key + Slice user_start = ExtractUserKey(*start); + Slice user_limit = ExtractUserKey(limit); + std::string tmp(user_start.data(), user_start.size()); + comparator.FindShortestSeparator(&tmp, user_limit); + if (tmp.size() <= user_start.size() && + comparator.Compare(user_start, tmp) < 0) { + // User key has become shorter physically, but larger logically. + // Tack on the earliest possible number to the shortened user key. + PutFixed64(&tmp, + PackSequenceAndType(kMaxSequenceNumber, kValueTypeForSeek)); + assert(InternalKeyComparator(&comparator).Compare(*start, tmp) < 0); + assert(InternalKeyComparator(&comparator).Compare(tmp, limit) < 0); + start->swap(tmp); + } +} + +void ShortenedIndexBuilder::FindShortInternalKeySuccessor( + const Comparator& comparator, std::string* key) { + Slice user_key = ExtractUserKey(*key); + std::string tmp(user_key.data(), user_key.size()); + comparator.FindShortSuccessor(&tmp); + if (tmp.size() <= user_key.size() && comparator.Compare(user_key, tmp) < 0) { + // User key has become shorter physically, but larger logically. + // Tack on the earliest possible number to the shortened user key. + PutFixed64(&tmp, + PackSequenceAndType(kMaxSequenceNumber, kValueTypeForSeek)); + assert(InternalKeyComparator(&comparator).Compare(*key, tmp) < 0); + key->swap(tmp); + } +} + +PartitionedIndexBuilder* PartitionedIndexBuilder::CreateIndexBuilder( + const InternalKeyComparator* comparator, + const bool use_value_delta_encoding, + const BlockBasedTableOptions& table_opt) { + return new PartitionedIndexBuilder(comparator, table_opt, + use_value_delta_encoding); +} + +PartitionedIndexBuilder::PartitionedIndexBuilder( + const InternalKeyComparator* comparator, + const BlockBasedTableOptions& table_opt, + const bool use_value_delta_encoding) + : IndexBuilder(comparator), + index_block_builder_(table_opt.index_block_restart_interval, + true /*use_delta_encoding*/, + use_value_delta_encoding), + index_block_builder_without_seq_(table_opt.index_block_restart_interval, + true /*use_delta_encoding*/, + use_value_delta_encoding), + sub_index_builder_(nullptr), + table_opt_(table_opt), + // We start by false. After each partition we revise the value based on + // what the sub_index_builder has decided. If the feature is disabled + // entirely, this will be set to true after switching the first + // sub_index_builder. Otherwise, it could be set to true even one of the + // sub_index_builders could not safely exclude seq from the keys, then it + // wil be enforced on all sub_index_builders on ::Finish. + seperator_is_key_plus_seq_(false), + use_value_delta_encoding_(use_value_delta_encoding) {} + +PartitionedIndexBuilder::~PartitionedIndexBuilder() { + delete sub_index_builder_; +} + +void PartitionedIndexBuilder::MakeNewSubIndexBuilder() { + assert(sub_index_builder_ == nullptr); + sub_index_builder_ = new ShortenedIndexBuilder( + comparator_, table_opt_.index_block_restart_interval, + table_opt_.format_version, use_value_delta_encoding_, + table_opt_.index_shortening, /* include_first_key */ false); + + // Set sub_index_builder_->seperator_is_key_plus_seq_ to true if + // seperator_is_key_plus_seq_ is true (internal-key mode) (set to false by + // default on Creation) so that flush policy can point to + // sub_index_builder_->index_block_builder_ + if (seperator_is_key_plus_seq_) { + sub_index_builder_->seperator_is_key_plus_seq_ = true; + } + + flush_policy_.reset(FlushBlockBySizePolicyFactory::NewFlushBlockPolicy( + table_opt_.metadata_block_size, table_opt_.block_size_deviation, + // Note: this is sub-optimal since sub_index_builder_ could later reset + // seperator_is_key_plus_seq_ but the probability of that is low. + sub_index_builder_->seperator_is_key_plus_seq_ + ? sub_index_builder_->index_block_builder_ + : sub_index_builder_->index_block_builder_without_seq_)); + partition_cut_requested_ = false; +} + +void PartitionedIndexBuilder::RequestPartitionCut() { + partition_cut_requested_ = true; +} + +void PartitionedIndexBuilder::AddIndexEntry( + std::string* last_key_in_current_block, + const Slice* first_key_in_next_block, const BlockHandle& block_handle) { + // Note: to avoid two consecuitive flush in the same method call, we do not + // check flush policy when adding the last key + if (UNLIKELY(first_key_in_next_block == nullptr)) { // no more keys + if (sub_index_builder_ == nullptr) { + MakeNewSubIndexBuilder(); + } + sub_index_builder_->AddIndexEntry(last_key_in_current_block, + first_key_in_next_block, block_handle); + if (!seperator_is_key_plus_seq_ && + sub_index_builder_->seperator_is_key_plus_seq_) { + // then we need to apply it to all sub-index builders and reset + // flush_policy to point to Block Builder of sub_index_builder_ that store + // internal keys. + seperator_is_key_plus_seq_ = true; + flush_policy_.reset(FlushBlockBySizePolicyFactory::NewFlushBlockPolicy( + table_opt_.metadata_block_size, table_opt_.block_size_deviation, + sub_index_builder_->index_block_builder_)); + } + sub_index_last_key_ = std::string(*last_key_in_current_block); + entries_.push_back( + {sub_index_last_key_, + std::unique_ptr<ShortenedIndexBuilder>(sub_index_builder_)}); + sub_index_builder_ = nullptr; + cut_filter_block = true; + } else { + // apply flush policy only to non-empty sub_index_builder_ + if (sub_index_builder_ != nullptr) { + std::string handle_encoding; + block_handle.EncodeTo(&handle_encoding); + bool do_flush = + partition_cut_requested_ || + flush_policy_->Update(*last_key_in_current_block, handle_encoding); + if (do_flush) { + entries_.push_back( + {sub_index_last_key_, + std::unique_ptr<ShortenedIndexBuilder>(sub_index_builder_)}); + cut_filter_block = true; + sub_index_builder_ = nullptr; + } + } + if (sub_index_builder_ == nullptr) { + MakeNewSubIndexBuilder(); + } + sub_index_builder_->AddIndexEntry(last_key_in_current_block, + first_key_in_next_block, block_handle); + sub_index_last_key_ = std::string(*last_key_in_current_block); + if (!seperator_is_key_plus_seq_ && + sub_index_builder_->seperator_is_key_plus_seq_) { + // then we need to apply it to all sub-index builders and reset + // flush_policy to point to Block Builder of sub_index_builder_ that store + // internal keys. + seperator_is_key_plus_seq_ = true; + flush_policy_.reset(FlushBlockBySizePolicyFactory::NewFlushBlockPolicy( + table_opt_.metadata_block_size, table_opt_.block_size_deviation, + sub_index_builder_->index_block_builder_)); + } + } +} + +Status PartitionedIndexBuilder::Finish( + IndexBlocks* index_blocks, const BlockHandle& last_partition_block_handle) { + if (partition_cnt_ == 0) { + partition_cnt_ = entries_.size(); + } + // It must be set to null after last key is added + assert(sub_index_builder_ == nullptr); + if (finishing_indexes == true) { + Entry& last_entry = entries_.front(); + std::string handle_encoding; + last_partition_block_handle.EncodeTo(&handle_encoding); + std::string handle_delta_encoding; + PutVarsignedint64( + &handle_delta_encoding, + last_partition_block_handle.size() - last_encoded_handle_.size()); + last_encoded_handle_ = last_partition_block_handle; + const Slice handle_delta_encoding_slice(handle_delta_encoding); + index_block_builder_.Add(last_entry.key, handle_encoding, + &handle_delta_encoding_slice); + if (!seperator_is_key_plus_seq_) { + index_block_builder_without_seq_.Add(ExtractUserKey(last_entry.key), + handle_encoding, + &handle_delta_encoding_slice); + } + entries_.pop_front(); + } + // If there is no sub_index left, then return the 2nd level index. + if (UNLIKELY(entries_.empty())) { + if (seperator_is_key_plus_seq_) { + index_blocks->index_block_contents = index_block_builder_.Finish(); + } else { + index_blocks->index_block_contents = + index_block_builder_without_seq_.Finish(); + } + top_level_index_size_ = index_blocks->index_block_contents.size(); + index_size_ += top_level_index_size_; + 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(); + // Apply the policy to all sub-indexes + entry.value->seperator_is_key_plus_seq_ = seperator_is_key_plus_seq_; + auto s = entry.value->Finish(index_blocks); + index_size_ += index_blocks->index_block_contents.size(); + finishing_indexes = true; + return s.ok() ? Status::Incomplete() : s; + } +} + +size_t PartitionedIndexBuilder::NumPartitions() const { return partition_cnt_; } +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/index_builder.h b/src/rocksdb/table/block_based/index_builder.h new file mode 100644 index 000000000..dd3be0331 --- /dev/null +++ b/src/rocksdb/table/block_based/index_builder.h @@ -0,0 +1,455 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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 <assert.h> + +#include <cinttypes> +#include <list> +#include <string> +#include <unordered_map> + +#include "rocksdb/comparator.h" +#include "table/block_based/block_based_table_factory.h" +#include "table/block_based/block_builder.h" +#include "table/format.h" + +namespace ROCKSDB_NAMESPACE { +// 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 ROCKSDB_NAMESPACE::InternalKeyComparator* comparator, + const InternalKeySliceTransform* int_key_slice_transform, + const bool use_value_delta_encoding, + const BlockBasedTableOptions& table_opt); + + // 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<std::string, Slice> 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. + // Called before the OnKeyAdded() call for first_key_in_next_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 size for index block. Must be called after ::Finish. + virtual size_t IndexSize() const = 0; + + virtual bool seperator_is_key_plus_seq() { return true; } + + protected: + const InternalKeyComparator* comparator_; + // Set after ::Finish is called + size_t index_size_ = 0; +}; + +// 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, + const int index_block_restart_interval, const uint32_t format_version, + const bool use_value_delta_encoding, + BlockBasedTableOptions::IndexShorteningMode shortening_mode, + bool include_first_key) + : IndexBuilder(comparator), + index_block_builder_(index_block_restart_interval, + true /*use_delta_encoding*/, + use_value_delta_encoding), + index_block_builder_without_seq_(index_block_restart_interval, + true /*use_delta_encoding*/, + use_value_delta_encoding), + use_value_delta_encoding_(use_value_delta_encoding), + include_first_key_(include_first_key), + shortening_mode_(shortening_mode) { + // Making the default true will disable the feature for old versions + seperator_is_key_plus_seq_ = (format_version <= 2); + } + + virtual void OnKeyAdded(const Slice& key) override { + if (include_first_key_ && current_block_first_internal_key_.empty()) { + current_block_first_internal_key_.assign(key.data(), key.size()); + } + } + + 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) { + if (shortening_mode_ != + BlockBasedTableOptions::IndexShorteningMode::kNoShortening) { + FindShortestInternalKeySeparator(*comparator_->user_comparator(), + last_key_in_current_block, + *first_key_in_next_block); + } + if (!seperator_is_key_plus_seq_ && + comparator_->user_comparator()->Compare( + ExtractUserKey(*last_key_in_current_block), + ExtractUserKey(*first_key_in_next_block)) == 0) { + seperator_is_key_plus_seq_ = true; + } + } else { + if (shortening_mode_ == BlockBasedTableOptions::IndexShorteningMode:: + kShortenSeparatorsAndSuccessor) { + FindShortInternalKeySuccessor(*comparator_->user_comparator(), + last_key_in_current_block); + } + } + auto sep = Slice(*last_key_in_current_block); + + assert(!include_first_key_ || !current_block_first_internal_key_.empty()); + IndexValue entry(block_handle, current_block_first_internal_key_); + std::string encoded_entry; + std::string delta_encoded_entry; + entry.EncodeTo(&encoded_entry, include_first_key_, nullptr); + if (use_value_delta_encoding_ && !last_encoded_handle_.IsNull()) { + entry.EncodeTo(&delta_encoded_entry, include_first_key_, + &last_encoded_handle_); + } else { + // If it's the first block, or delta encoding is disabled, + // BlockBuilder::Add() below won't use delta-encoded slice. + } + last_encoded_handle_ = block_handle; + const Slice delta_encoded_entry_slice(delta_encoded_entry); + index_block_builder_.Add(sep, encoded_entry, &delta_encoded_entry_slice); + if (!seperator_is_key_plus_seq_) { + index_block_builder_without_seq_.Add(ExtractUserKey(sep), encoded_entry, + &delta_encoded_entry_slice); + } + + current_block_first_internal_key_.clear(); + } + + using IndexBuilder::Finish; + virtual Status Finish( + IndexBlocks* index_blocks, + const BlockHandle& /*last_partition_block_handle*/) override { + if (seperator_is_key_plus_seq_) { + index_blocks->index_block_contents = index_block_builder_.Finish(); + } else { + index_blocks->index_block_contents = + index_block_builder_without_seq_.Finish(); + } + index_size_ = index_blocks->index_block_contents.size(); + return Status::OK(); + } + + virtual size_t IndexSize() const override { return index_size_; } + + virtual bool seperator_is_key_plus_seq() override { + return seperator_is_key_plus_seq_; + } + + // Changes *key to a short string >= *key. + // + static void FindShortestInternalKeySeparator(const Comparator& comparator, + std::string* start, + const Slice& limit); + + static void FindShortInternalKeySuccessor(const Comparator& comparator, + std::string* key); + + friend class PartitionedIndexBuilder; + + private: + BlockBuilder index_block_builder_; + BlockBuilder index_block_builder_without_seq_; + const bool use_value_delta_encoding_; + bool seperator_is_key_plus_seq_; + const bool include_first_key_; + BlockBasedTableOptions::IndexShorteningMode shortening_mode_; + BlockHandle last_encoded_handle_ = BlockHandle::NullBlockHandle(); + std::string current_block_first_internal_key_; +}; + +// 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, int format_version, + bool use_value_delta_encoding, + BlockBasedTableOptions::IndexShorteningMode shortening_mode) + : IndexBuilder(comparator), + primary_index_builder_(comparator, index_block_restart_interval, + format_version, use_value_delta_encoding, + shortening_mode, /* include_first_key */ false), + 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) std::to_string() 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<uint32_t>(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 { + if (pending_block_num_ != 0) { + FlushPendingPrefix(); + } + Status s = 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 s; + } + + virtual size_t IndexSize() const override { + return primary_index_builder_.IndexSize() + prefix_block_.size() + + prefix_meta_block_.size(); + } + + virtual bool seperator_is_key_plus_seq() override { + return primary_index_builder_.seperator_is_key_plus_seq(); + } + + private: + void FlushPendingPrefix() { + prefix_block_.append(pending_entry_prefix_.data(), + pending_entry_prefix_.size()); + PutVarint32Varint32Varint32( + &prefix_meta_block_, + static_cast<uint32_t>(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; +}; + +/** + * 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 PartitionedIndexBuilder : public IndexBuilder { + public: + static PartitionedIndexBuilder* CreateIndexBuilder( + const ROCKSDB_NAMESPACE::InternalKeyComparator* comparator, + const bool use_value_delta_encoding, + const BlockBasedTableOptions& table_opt); + + explicit PartitionedIndexBuilder(const InternalKeyComparator* comparator, + const BlockBasedTableOptions& table_opt, + const bool use_value_delta_encoding); + + virtual ~PartitionedIndexBuilder(); + + virtual void AddIndexEntry(std::string* last_key_in_current_block, + const Slice* first_key_in_next_block, + const BlockHandle& block_handle) override; + + virtual Status Finish( + IndexBlocks* index_blocks, + const BlockHandle& last_partition_block_handle) override; + + virtual size_t IndexSize() const override { return index_size_; } + size_t TopLevelIndexSize(uint64_t) const { return top_level_index_size_; } + size_t NumPartitions() const; + + inline bool ShouldCutFilterBlock() { + // Current policy is to align the partitions of index and filters + if (cut_filter_block) { + cut_filter_block = false; + return true; + } + return false; + } + + std::string& GetPartitionKey() { return sub_index_last_key_; } + + // Called when an external entity (such as filter partition builder) request + // cutting the next partition + void RequestPartitionCut(); + + virtual bool seperator_is_key_plus_seq() override { + return seperator_is_key_plus_seq_; + } + + bool get_use_value_delta_encoding() { return use_value_delta_encoding_; } + + private: + // Set after ::Finish is called + size_t top_level_index_size_ = 0; + // Set after ::Finish is called + size_t partition_cnt_ = 0; + + void MakeNewSubIndexBuilder(); + + struct Entry { + std::string key; + std::unique_ptr<ShortenedIndexBuilder> value; + }; + std::list<Entry> entries_; // list of partitioned indexes and their keys + BlockBuilder index_block_builder_; // top-level index builder + BlockBuilder index_block_builder_without_seq_; // same for user keys + // the active partition index builder + ShortenedIndexBuilder* sub_index_builder_; + // the last key in the active partition index builder + std::string sub_index_last_key_; + std::unique_ptr<FlushBlockPolicy> flush_policy_; + // true if Finish is called once but not complete yet. + bool finishing_indexes = false; + const BlockBasedTableOptions& table_opt_; + bool seperator_is_key_plus_seq_; + bool use_value_delta_encoding_; + // true if an external entity (such as filter partition builder) request + // cutting the next partition + bool partition_cut_requested_ = true; + // true if it should cut the next filter partition block + bool cut_filter_block = false; + BlockHandle last_encoded_handle_; +}; +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/index_reader_common.cc b/src/rocksdb/table/block_based/index_reader_common.cc new file mode 100644 index 000000000..6584586c9 --- /dev/null +++ b/src/rocksdb/table/block_based/index_reader_common.cc @@ -0,0 +1,56 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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/block_based/index_reader_common.h" + +namespace ROCKSDB_NAMESPACE { +Status BlockBasedTable::IndexReaderCommon::ReadIndexBlock( + const BlockBasedTable* table, FilePrefetchBuffer* prefetch_buffer, + const ReadOptions& read_options, bool use_cache, GetContext* get_context, + BlockCacheLookupContext* lookup_context, + CachableEntry<Block>* index_block) { + PERF_TIMER_GUARD(read_index_block_nanos); + + assert(table != nullptr); + assert(index_block != nullptr); + assert(index_block->IsEmpty()); + + const Rep* const rep = table->get_rep(); + assert(rep != nullptr); + + const Status s = table->RetrieveBlock( + prefetch_buffer, read_options, rep->footer.index_handle(), + UncompressionDict::GetEmptyDict(), index_block, BlockType::kIndex, + get_context, lookup_context, /* for_compaction */ false, use_cache, + /* wait_for_cache */ true, /* async_read */ false); + + return s; +} + +Status BlockBasedTable::IndexReaderCommon::GetOrReadIndexBlock( + bool no_io, Env::IOPriority rate_limiter_priority, GetContext* get_context, + BlockCacheLookupContext* lookup_context, + CachableEntry<Block>* index_block) const { + assert(index_block != nullptr); + + if (!index_block_.IsEmpty()) { + index_block->SetUnownedValue(index_block_.GetValue()); + return Status::OK(); + } + + ReadOptions read_options; + read_options.rate_limiter_priority = rate_limiter_priority; + if (no_io) { + read_options.read_tier = kBlockCacheTier; + } + + return ReadIndexBlock(table_, /*prefetch_buffer=*/nullptr, read_options, + cache_index_blocks(), get_context, lookup_context, + index_block); +} +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/index_reader_common.h b/src/rocksdb/table/block_based/index_reader_common.h new file mode 100644 index 000000000..5627b0eeb --- /dev/null +++ b/src/rocksdb/table/block_based/index_reader_common.h @@ -0,0 +1,85 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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 "table/block_based/block_based_table_reader.h" +#include "table/block_based/reader_common.h" + +namespace ROCKSDB_NAMESPACE { +// Encapsulates common functionality for the various index reader +// implementations. Provides access to the index block regardless of whether +// it is owned by the reader or stored in the cache, or whether it is pinned +// in the cache or not. +class BlockBasedTable::IndexReaderCommon : public BlockBasedTable::IndexReader { + public: + IndexReaderCommon(const BlockBasedTable* t, + CachableEntry<Block>&& index_block) + : table_(t), index_block_(std::move(index_block)) { + assert(table_ != nullptr); + } + + protected: + static Status ReadIndexBlock(const BlockBasedTable* table, + FilePrefetchBuffer* prefetch_buffer, + const ReadOptions& read_options, bool use_cache, + GetContext* get_context, + BlockCacheLookupContext* lookup_context, + CachableEntry<Block>* index_block); + + const BlockBasedTable* table() const { return table_; } + + const InternalKeyComparator* internal_comparator() const { + assert(table_ != nullptr); + assert(table_->get_rep() != nullptr); + + return &table_->get_rep()->internal_comparator; + } + + bool index_has_first_key() const { + assert(table_ != nullptr); + assert(table_->get_rep() != nullptr); + return table_->get_rep()->index_has_first_key; + } + + bool index_key_includes_seq() const { + assert(table_ != nullptr); + assert(table_->get_rep() != nullptr); + return table_->get_rep()->index_key_includes_seq; + } + + bool index_value_is_full() const { + assert(table_ != nullptr); + assert(table_->get_rep() != nullptr); + return table_->get_rep()->index_value_is_full; + } + + bool cache_index_blocks() const { + assert(table_ != nullptr); + assert(table_->get_rep() != nullptr); + return table_->get_rep()->table_options.cache_index_and_filter_blocks; + } + + Status GetOrReadIndexBlock(bool no_io, Env::IOPriority rate_limiter_priority, + GetContext* get_context, + BlockCacheLookupContext* lookup_context, + CachableEntry<Block>* index_block) const; + + size_t ApproximateIndexBlockMemoryUsage() const { + assert(!index_block_.GetOwnValue() || index_block_.GetValue() != nullptr); + return index_block_.GetOwnValue() + ? index_block_.GetValue()->ApproximateMemoryUsage() + : 0; + } + + private: + const BlockBasedTable* table_; + CachableEntry<Block> index_block_; +}; + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/mock_block_based_table.h b/src/rocksdb/table/block_based/mock_block_based_table.h new file mode 100644 index 000000000..13f3dfaee --- /dev/null +++ b/src/rocksdb/table/block_based/mock_block_based_table.h @@ -0,0 +1,62 @@ +// Copyright (c) 2019-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +#pragma once + +#include <memory> + +#include "rocksdb/filter_policy.h" +#include "table/block_based/block_based_table_reader.h" +#include "table/block_based/filter_policy_internal.h" + +namespace ROCKSDB_NAMESPACE { +namespace mock { + +class MockBlockBasedTable : public BlockBasedTable { + public: + explicit MockBlockBasedTable(Rep* rep) + : BlockBasedTable(rep, nullptr /* block_cache_tracer */) {} +}; + +class MockBlockBasedTableTester { + static constexpr int kMockLevel = 0; + + public: + Options options_; + ImmutableOptions ioptions_; + EnvOptions env_options_; + BlockBasedTableOptions table_options_; + InternalKeyComparator icomp_; + std::unique_ptr<BlockBasedTable> table_; + + explicit MockBlockBasedTableTester(const FilterPolicy* filter_policy) + : MockBlockBasedTableTester( + std::shared_ptr<const FilterPolicy>(filter_policy)){}; + + explicit MockBlockBasedTableTester( + std::shared_ptr<const FilterPolicy> filter_policy) + : ioptions_(options_), + env_options_(options_), + icomp_(options_.comparator) { + table_options_.filter_policy = std::move(filter_policy); + + constexpr bool skip_filters = false; + constexpr bool immortal_table = false; + table_.reset(new MockBlockBasedTable(new BlockBasedTable::Rep( + ioptions_, env_options_, table_options_, icomp_, skip_filters, + 12345 /*file_size*/, kMockLevel, immortal_table))); + } + + FilterBitsBuilder* GetBuilder() const { + FilterBuildingContext context(table_options_); + context.column_family_name = "mock_cf"; + context.compaction_style = ioptions_.compaction_style; + context.level_at_creation = kMockLevel; + context.info_log = ioptions_.logger; + return BloomFilterPolicy::GetBuilderFromContext(context); + } +}; + +} // namespace mock +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/parsed_full_filter_block.cc b/src/rocksdb/table/block_based/parsed_full_filter_block.cc new file mode 100644 index 000000000..9184a48d2 --- /dev/null +++ b/src/rocksdb/table/block_based/parsed_full_filter_block.cc @@ -0,0 +1,23 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// + +#include "table/block_based/parsed_full_filter_block.h" + +#include "table/block_based/filter_policy_internal.h" + +namespace ROCKSDB_NAMESPACE { + +ParsedFullFilterBlock::ParsedFullFilterBlock(const FilterPolicy* filter_policy, + BlockContents&& contents) + : block_contents_(std::move(contents)), + filter_bits_reader_( + !block_contents_.data.empty() + ? filter_policy->GetFilterBitsReader(block_contents_.data) + : nullptr) {} + +ParsedFullFilterBlock::~ParsedFullFilterBlock() = default; + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/parsed_full_filter_block.h b/src/rocksdb/table/block_based/parsed_full_filter_block.h new file mode 100644 index 000000000..95d7b5208 --- /dev/null +++ b/src/rocksdb/table/block_based/parsed_full_filter_block.h @@ -0,0 +1,42 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#pragma once + +#include <memory> + +#include "table/format.h" + +namespace ROCKSDB_NAMESPACE { + +class FilterBitsReader; +class FilterPolicy; + +// The sharable/cachable part of the full filter. +class ParsedFullFilterBlock { + public: + ParsedFullFilterBlock(const FilterPolicy* filter_policy, + BlockContents&& contents); + ~ParsedFullFilterBlock(); + + FilterBitsReader* filter_bits_reader() const { + return filter_bits_reader_.get(); + } + + // TODO: consider memory usage of the FilterBitsReader + size_t ApproximateMemoryUsage() const { + return block_contents_.ApproximateMemoryUsage(); + } + + bool own_bytes() const { return block_contents_.own_bytes(); } + + const Slice GetBlockContentsData() const { return block_contents_.data; } + + private: + BlockContents block_contents_; + std::unique_ptr<FilterBitsReader> filter_bits_reader_; +}; + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/partitioned_filter_block.cc b/src/rocksdb/table/block_based/partitioned_filter_block.cc new file mode 100644 index 000000000..af30925b7 --- /dev/null +++ b/src/rocksdb/table/block_based/partitioned_filter_block.cc @@ -0,0 +1,561 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#include "table/block_based/partitioned_filter_block.h" + +#include <utility> + +#include "block_type.h" +#include "file/random_access_file_reader.h" +#include "logging/logging.h" +#include "monitoring/perf_context_imp.h" +#include "port/malloc.h" +#include "port/port.h" +#include "rocksdb/filter_policy.h" +#include "table/block_based/block.h" +#include "table/block_based/block_based_table_reader.h" +#include "util/coding.h" + +namespace ROCKSDB_NAMESPACE { + +PartitionedFilterBlockBuilder::PartitionedFilterBlockBuilder( + const SliceTransform* _prefix_extractor, bool whole_key_filtering, + FilterBitsBuilder* filter_bits_builder, int index_block_restart_interval, + const bool use_value_delta_encoding, + PartitionedIndexBuilder* const p_index_builder, + const uint32_t partition_size) + : FullFilterBlockBuilder(_prefix_extractor, whole_key_filtering, + filter_bits_builder), + index_on_filter_block_builder_(index_block_restart_interval, + true /*use_delta_encoding*/, + use_value_delta_encoding), + index_on_filter_block_builder_without_seq_(index_block_restart_interval, + true /*use_delta_encoding*/, + use_value_delta_encoding), + p_index_builder_(p_index_builder), + keys_added_to_partition_(0), + total_added_in_built_(0) { + keys_per_partition_ = static_cast<uint32_t>( + filter_bits_builder_->ApproximateNumEntries(partition_size)); + if (keys_per_partition_ < 1) { + // partition_size (minus buffer, ~10%) might be smaller than minimum + // filter size, sometimes based on cache line size. Try to find that + // minimum size without CalculateSpace (not necessarily available). + uint32_t larger = std::max(partition_size + 4, uint32_t{16}); + for (;;) { + keys_per_partition_ = static_cast<uint32_t>( + filter_bits_builder_->ApproximateNumEntries(larger)); + if (keys_per_partition_ >= 1) { + break; + } + larger += larger / 4; + if (larger > 100000) { + // might be a broken implementation. substitute something reasonable: + // 1 key / byte. + keys_per_partition_ = partition_size; + break; + } + } + } +} + +PartitionedFilterBlockBuilder::~PartitionedFilterBlockBuilder() { + partitioned_filters_construction_status_.PermitUncheckedError(); +} + +void PartitionedFilterBlockBuilder::MaybeCutAFilterBlock( + const Slice* next_key) { + // Use == to send the request only once + if (keys_added_to_partition_ == keys_per_partition_) { + // Currently only index builder is in charge of cutting a partition. We keep + // requesting until it is granted. + p_index_builder_->RequestPartitionCut(); + } + if (!p_index_builder_->ShouldCutFilterBlock()) { + return; + } + + // Add the prefix of the next key before finishing the partition without + // updating last_prefix_str_. This hack, fixes a bug with format_verison=3 + // where seeking for the prefix would lead us to the previous partition. + const bool maybe_add_prefix = + next_key && prefix_extractor() && prefix_extractor()->InDomain(*next_key); + if (maybe_add_prefix) { + const Slice next_key_prefix = prefix_extractor()->Transform(*next_key); + if (next_key_prefix.compare(last_prefix_str()) != 0) { + AddKey(next_key_prefix); + } + } + + total_added_in_built_ += filter_bits_builder_->EstimateEntriesAdded(); + std::unique_ptr<const char[]> filter_data; + Status filter_construction_status = Status::OK(); + Slice filter = + filter_bits_builder_->Finish(&filter_data, &filter_construction_status); + if (filter_construction_status.ok()) { + filter_construction_status = filter_bits_builder_->MaybePostVerify(filter); + } + std::string& index_key = p_index_builder_->GetPartitionKey(); + filters.push_back({index_key, std::move(filter_data), filter}); + if (!filter_construction_status.ok() && + partitioned_filters_construction_status_.ok()) { + partitioned_filters_construction_status_ = filter_construction_status; + } + keys_added_to_partition_ = 0; + Reset(); +} + +void PartitionedFilterBlockBuilder::Add(const Slice& key) { + MaybeCutAFilterBlock(&key); + FullFilterBlockBuilder::Add(key); +} + +void PartitionedFilterBlockBuilder::AddKey(const Slice& key) { + FullFilterBlockBuilder::AddKey(key); + keys_added_to_partition_++; +} + +size_t PartitionedFilterBlockBuilder::EstimateEntriesAdded() { + return total_added_in_built_ + filter_bits_builder_->EstimateEntriesAdded(); +} + +Slice PartitionedFilterBlockBuilder::Finish( + const BlockHandle& last_partition_block_handle, Status* status, + std::unique_ptr<const char[]>* filter_data) { + if (finishing_filters == true) { + // Record the handle of the last written filter block in the index + std::string handle_encoding; + last_partition_block_handle.EncodeTo(&handle_encoding); + std::string handle_delta_encoding; + PutVarsignedint64( + &handle_delta_encoding, + last_partition_block_handle.size() - last_encoded_handle_.size()); + last_encoded_handle_ = last_partition_block_handle; + const Slice handle_delta_encoding_slice(handle_delta_encoding); + index_on_filter_block_builder_.Add(last_filter_entry_key, handle_encoding, + &handle_delta_encoding_slice); + if (!p_index_builder_->seperator_is_key_plus_seq()) { + index_on_filter_block_builder_without_seq_.Add( + ExtractUserKey(last_filter_entry_key), handle_encoding, + &handle_delta_encoding_slice); + } + } else { + MaybeCutAFilterBlock(nullptr); + } + + if (!partitioned_filters_construction_status_.ok()) { + *status = partitioned_filters_construction_status_; + return Slice(); + } + + // If there is no filter partition left, then return the index on filter + // partitions + if (UNLIKELY(filters.empty())) { + *status = Status::OK(); + last_filter_data.reset(); + if (finishing_filters) { + // Simplest to just add them all at the end + total_added_in_built_ = 0; + if (p_index_builder_->seperator_is_key_plus_seq()) { + return index_on_filter_block_builder_.Finish(); + } else { + return index_on_filter_block_builder_without_seq_.Finish(); + } + } else { + // This is the rare case where no key was added to the filter + return Slice(); + } + } else { + // Return the next filter partition in line and set Incomplete() status to + // indicate we expect more calls to Finish + *status = Status::Incomplete(); + finishing_filters = true; + + last_filter_entry_key = filters.front().key; + Slice filter = filters.front().filter; + last_filter_data = std::move(filters.front().filter_data); + if (filter_data != nullptr) { + *filter_data = std::move(last_filter_data); + } + filters.pop_front(); + return filter; + } +} + +PartitionedFilterBlockReader::PartitionedFilterBlockReader( + const BlockBasedTable* t, CachableEntry<Block>&& filter_block) + : FilterBlockReaderCommon(t, std::move(filter_block)) {} + +std::unique_ptr<FilterBlockReader> PartitionedFilterBlockReader::Create( + const BlockBasedTable* table, const ReadOptions& ro, + FilePrefetchBuffer* prefetch_buffer, bool use_cache, bool prefetch, + bool pin, BlockCacheLookupContext* lookup_context) { + assert(table); + assert(table->get_rep()); + assert(!pin || prefetch); + + CachableEntry<Block> filter_block; + if (prefetch || !use_cache) { + const Status s = ReadFilterBlock( + table, prefetch_buffer, ro, use_cache, nullptr /* get_context */, + lookup_context, &filter_block, BlockType::kFilterPartitionIndex); + if (!s.ok()) { + IGNORE_STATUS_IF_ERROR(s); + return std::unique_ptr<FilterBlockReader>(); + } + + if (use_cache && !pin) { + filter_block.Reset(); + } + } + + return std::unique_ptr<FilterBlockReader>( + new PartitionedFilterBlockReader(table, std::move(filter_block))); +} + +bool PartitionedFilterBlockReader::KeyMayMatch( + const Slice& key, const bool no_io, const Slice* const const_ikey_ptr, + GetContext* get_context, BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) { + assert(const_ikey_ptr != nullptr); + if (!whole_key_filtering()) { + return true; + } + + return MayMatch(key, no_io, const_ikey_ptr, get_context, lookup_context, + rate_limiter_priority, &FullFilterBlockReader::KeyMayMatch); +} + +void PartitionedFilterBlockReader::KeysMayMatch( + MultiGetRange* range, const bool no_io, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) { + if (!whole_key_filtering()) { + return; // Any/all may match + } + + MayMatch(range, nullptr, no_io, lookup_context, rate_limiter_priority, + &FullFilterBlockReader::KeysMayMatch2); +} + +bool PartitionedFilterBlockReader::PrefixMayMatch( + const Slice& prefix, const bool no_io, const Slice* const const_ikey_ptr, + GetContext* get_context, BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) { + assert(const_ikey_ptr != nullptr); + return MayMatch(prefix, no_io, const_ikey_ptr, get_context, lookup_context, + rate_limiter_priority, + &FullFilterBlockReader::PrefixMayMatch); +} + +void PartitionedFilterBlockReader::PrefixesMayMatch( + MultiGetRange* range, const SliceTransform* prefix_extractor, + const bool no_io, BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) { + assert(prefix_extractor); + MayMatch(range, prefix_extractor, no_io, lookup_context, + rate_limiter_priority, &FullFilterBlockReader::PrefixesMayMatch); +} + +BlockHandle PartitionedFilterBlockReader::GetFilterPartitionHandle( + const CachableEntry<Block>& filter_block, const Slice& entry) const { + IndexBlockIter iter; + const InternalKeyComparator* const comparator = internal_comparator(); + Statistics* kNullStats = nullptr; + filter_block.GetValue()->NewIndexIterator( + comparator->user_comparator(), + table()->get_rep()->get_global_seqno(BlockType::kFilterPartitionIndex), + &iter, kNullStats, true /* total_order_seek */, + false /* have_first_key */, index_key_includes_seq(), + index_value_is_full()); + iter.Seek(entry); + if (UNLIKELY(!iter.Valid())) { + // entry is larger than all the keys. However its prefix might still be + // present in the last partition. If this is called by PrefixMayMatch this + // is necessary for correct behavior. Otherwise it is unnecessary but safe. + // Assuming this is an unlikely case for full key search, the performance + // overhead should be negligible. + iter.SeekToLast(); + } + assert(iter.Valid()); + BlockHandle fltr_blk_handle = iter.value().handle; + return fltr_blk_handle; +} + +Status PartitionedFilterBlockReader::GetFilterPartitionBlock( + FilePrefetchBuffer* prefetch_buffer, const BlockHandle& fltr_blk_handle, + bool no_io, GetContext* get_context, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority, + CachableEntry<ParsedFullFilterBlock>* filter_block) const { + assert(table()); + assert(filter_block); + assert(filter_block->IsEmpty()); + + if (!filter_map_.empty()) { + auto iter = filter_map_.find(fltr_blk_handle.offset()); + // This is a possible scenario since block cache might not have had space + // for the partition + if (iter != filter_map_.end()) { + filter_block->SetUnownedValue(iter->second.GetValue()); + return Status::OK(); + } + } + + ReadOptions read_options; + read_options.rate_limiter_priority = rate_limiter_priority; + if (no_io) { + read_options.read_tier = kBlockCacheTier; + } + + const Status s = + table()->RetrieveBlock(prefetch_buffer, read_options, fltr_blk_handle, + UncompressionDict::GetEmptyDict(), filter_block, + BlockType::kFilter, get_context, lookup_context, + /* for_compaction */ false, /* use_cache */ true, + /* wait_for_cache */ true, /* async_read */ false); + + return s; +} + +bool PartitionedFilterBlockReader::MayMatch( + const Slice& slice, bool no_io, const Slice* const_ikey_ptr, + GetContext* get_context, BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority, + FilterFunction filter_function) const { + CachableEntry<Block> filter_block; + Status s = GetOrReadFilterBlock( + no_io, get_context, lookup_context, &filter_block, + BlockType::kFilterPartitionIndex, rate_limiter_priority); + if (UNLIKELY(!s.ok())) { + IGNORE_STATUS_IF_ERROR(s); + return true; + } + + if (UNLIKELY(filter_block.GetValue()->size() == 0)) { + return true; + } + + auto filter_handle = GetFilterPartitionHandle(filter_block, *const_ikey_ptr); + if (UNLIKELY(filter_handle.size() == 0)) { // key is out of range + return false; + } + + CachableEntry<ParsedFullFilterBlock> filter_partition_block; + s = GetFilterPartitionBlock(nullptr /* prefetch_buffer */, filter_handle, + no_io, get_context, lookup_context, + rate_limiter_priority, &filter_partition_block); + if (UNLIKELY(!s.ok())) { + IGNORE_STATUS_IF_ERROR(s); + return true; + } + + FullFilterBlockReader filter_partition(table(), + std::move(filter_partition_block)); + return (filter_partition.*filter_function)(slice, no_io, const_ikey_ptr, + get_context, lookup_context, + rate_limiter_priority); +} + +void PartitionedFilterBlockReader::MayMatch( + MultiGetRange* range, const SliceTransform* prefix_extractor, bool no_io, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority, + FilterManyFunction filter_function) const { + CachableEntry<Block> filter_block; + Status s = GetOrReadFilterBlock( + no_io, range->begin()->get_context, lookup_context, &filter_block, + BlockType::kFilterPartitionIndex, rate_limiter_priority); + if (UNLIKELY(!s.ok())) { + IGNORE_STATUS_IF_ERROR(s); + return; // Any/all may match + } + + if (UNLIKELY(filter_block.GetValue()->size() == 0)) { + return; // Any/all may match + } + + auto start_iter_same_handle = range->begin(); + BlockHandle prev_filter_handle = BlockHandle::NullBlockHandle(); + + // For all keys mapping to same partition (must be adjacent in sorted order) + // share block cache lookup and use full filter multiget on the partition + // filter. + for (auto iter = start_iter_same_handle; iter != range->end(); ++iter) { + // TODO: re-use one top-level index iterator + BlockHandle this_filter_handle = + GetFilterPartitionHandle(filter_block, iter->ikey); + if (!prev_filter_handle.IsNull() && + this_filter_handle != prev_filter_handle) { + MultiGetRange subrange(*range, start_iter_same_handle, iter); + MayMatchPartition(&subrange, prefix_extractor, prev_filter_handle, no_io, + lookup_context, rate_limiter_priority, filter_function); + range->AddSkipsFrom(subrange); + start_iter_same_handle = iter; + } + if (UNLIKELY(this_filter_handle.size() == 0)) { // key is out of range + // Not reachable with current behavior of GetFilterPartitionHandle + assert(false); + range->SkipKey(iter); + prev_filter_handle = BlockHandle::NullBlockHandle(); + } else { + prev_filter_handle = this_filter_handle; + } + } + if (!prev_filter_handle.IsNull()) { + MultiGetRange subrange(*range, start_iter_same_handle, range->end()); + MayMatchPartition(&subrange, prefix_extractor, prev_filter_handle, no_io, + lookup_context, rate_limiter_priority, filter_function); + range->AddSkipsFrom(subrange); + } +} + +void PartitionedFilterBlockReader::MayMatchPartition( + MultiGetRange* range, const SliceTransform* prefix_extractor, + BlockHandle filter_handle, bool no_io, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority, + FilterManyFunction filter_function) const { + CachableEntry<ParsedFullFilterBlock> filter_partition_block; + Status s = GetFilterPartitionBlock( + nullptr /* prefetch_buffer */, filter_handle, no_io, + range->begin()->get_context, lookup_context, rate_limiter_priority, + &filter_partition_block); + if (UNLIKELY(!s.ok())) { + IGNORE_STATUS_IF_ERROR(s); + return; // Any/all may match + } + + FullFilterBlockReader filter_partition(table(), + std::move(filter_partition_block)); + (filter_partition.*filter_function)(range, prefix_extractor, no_io, + lookup_context, rate_limiter_priority); +} + +size_t PartitionedFilterBlockReader::ApproximateMemoryUsage() const { + size_t usage = ApproximateFilterBlockMemoryUsage(); +#ifdef ROCKSDB_MALLOC_USABLE_SIZE + usage += malloc_usable_size(const_cast<PartitionedFilterBlockReader*>(this)); +#else + usage += sizeof(*this); +#endif // ROCKSDB_MALLOC_USABLE_SIZE + return usage; + // TODO(myabandeh): better estimation for filter_map_ size +} + +// TODO(myabandeh): merge this with the same function in IndexReader +Status PartitionedFilterBlockReader::CacheDependencies(const ReadOptions& ro, + bool pin) { + assert(table()); + + const BlockBasedTable::Rep* const rep = table()->get_rep(); + assert(rep); + + BlockCacheLookupContext lookup_context{TableReaderCaller::kPrefetch}; + + CachableEntry<Block> filter_block; + + Status s = GetOrReadFilterBlock(false /* no_io */, nullptr /* get_context */, + &lookup_context, &filter_block, + BlockType::kFilterPartitionIndex, + ro.rate_limiter_priority); + if (!s.ok()) { + ROCKS_LOG_ERROR(rep->ioptions.logger, + "Error retrieving top-level filter block while trying to " + "cache filter partitions: %s", + s.ToString().c_str()); + return s; + } + + // Before read partitions, prefetch them to avoid lots of IOs + assert(filter_block.GetValue()); + + IndexBlockIter biter; + const InternalKeyComparator* const comparator = internal_comparator(); + Statistics* kNullStats = nullptr; + filter_block.GetValue()->NewIndexIterator( + comparator->user_comparator(), + rep->get_global_seqno(BlockType::kFilterPartitionIndex), &biter, + kNullStats, true /* total_order_seek */, false /* have_first_key */, + index_key_includes_seq(), index_value_is_full()); + // Index partitions are assumed to be consecuitive. Prefetch them all. + // Read the first block offset + biter.SeekToFirst(); + BlockHandle handle = biter.value().handle; + uint64_t prefetch_off = handle.offset(); + + // Read the last block's offset + biter.SeekToLast(); + handle = biter.value().handle; + uint64_t last_off = + handle.offset() + handle.size() + BlockBasedTable::kBlockTrailerSize; + uint64_t prefetch_len = last_off - prefetch_off; + std::unique_ptr<FilePrefetchBuffer> prefetch_buffer; + rep->CreateFilePrefetchBuffer( + 0, 0, &prefetch_buffer, false /* Implicit autoreadahead */, + 0 /*num_reads_*/, 0 /*num_file_reads_for_auto_readahead*/); + + IOOptions opts; + s = rep->file->PrepareIOOptions(ro, opts); + if (s.ok()) { + s = prefetch_buffer->Prefetch(opts, rep->file.get(), prefetch_off, + static_cast<size_t>(prefetch_len), + ro.rate_limiter_priority); + } + if (!s.ok()) { + return s; + } + + // After prefetch, read the partitions one by one + for (biter.SeekToFirst(); biter.Valid(); biter.Next()) { + handle = biter.value().handle; + + CachableEntry<ParsedFullFilterBlock> block; + // TODO: Support counter batch update for partitioned index and + // filter blocks + s = table()->MaybeReadBlockAndLoadToCache( + prefetch_buffer.get(), ro, handle, UncompressionDict::GetEmptyDict(), + /* wait */ true, /* for_compaction */ false, &block, BlockType::kFilter, + nullptr /* get_context */, &lookup_context, nullptr /* contents */, + false); + if (!s.ok()) { + return s; + } + assert(s.ok() || block.GetValue() == nullptr); + + if (block.GetValue() != nullptr) { + if (block.IsCached()) { + if (pin) { + filter_map_[handle.offset()] = std::move(block); + } + } + } + } + return biter.status(); +} + +const InternalKeyComparator* PartitionedFilterBlockReader::internal_comparator() + const { + assert(table()); + assert(table()->get_rep()); + + return &table()->get_rep()->internal_comparator; +} + +bool PartitionedFilterBlockReader::index_key_includes_seq() const { + assert(table()); + assert(table()->get_rep()); + + return table()->get_rep()->index_key_includes_seq; +} + +bool PartitionedFilterBlockReader::index_value_is_full() const { + assert(table()); + assert(table()->get_rep()); + + return table()->get_rep()->index_value_is_full; +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/partitioned_filter_block.h b/src/rocksdb/table/block_based/partitioned_filter_block.h new file mode 100644 index 000000000..955b50739 --- /dev/null +++ b/src/rocksdb/table/block_based/partitioned_filter_block.h @@ -0,0 +1,178 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#pragma once + +#include <deque> +#include <list> +#include <string> +#include <unordered_map> + +#include "rocksdb/options.h" +#include "rocksdb/slice.h" +#include "rocksdb/slice_transform.h" +#include "table/block_based/block.h" +#include "table/block_based/filter_block_reader_common.h" +#include "table/block_based/full_filter_block.h" +#include "table/block_based/index_builder.h" +#include "util/autovector.h" +#include "util/hash_containers.h" + +namespace ROCKSDB_NAMESPACE { +class InternalKeyComparator; + +class PartitionedFilterBlockBuilder : public FullFilterBlockBuilder { + public: + explicit PartitionedFilterBlockBuilder( + const SliceTransform* prefix_extractor, bool whole_key_filtering, + FilterBitsBuilder* filter_bits_builder, int index_block_restart_interval, + const bool use_value_delta_encoding, + PartitionedIndexBuilder* const p_index_builder, + const uint32_t partition_size); + + virtual ~PartitionedFilterBlockBuilder(); + + void AddKey(const Slice& key) override; + void Add(const Slice& key) override; + size_t EstimateEntriesAdded() override; + + virtual Slice Finish( + const BlockHandle& last_partition_block_handle, Status* status, + std::unique_ptr<const char[]>* filter_data = nullptr) override; + + virtual void ResetFilterBitsBuilder() override { + // Previously constructed partitioned filters by + // this to-be-reset FiterBitsBuilder can also be + // cleared + filters.clear(); + FullFilterBlockBuilder::ResetFilterBitsBuilder(); + } + + // For PartitionFilter, optional post-verifing the filter is done + // as part of PartitionFilterBlockBuilder::Finish + // to avoid implementation complexity of doing it elsewhere. + // Therefore we are skipping it in here. + virtual Status MaybePostVerifyFilter( + const Slice& /* filter_content */) override { + return Status::OK(); + } + + private: + // Filter data + BlockBuilder index_on_filter_block_builder_; // top-level index builder + BlockBuilder + index_on_filter_block_builder_without_seq_; // same for user keys + struct FilterEntry { + std::string key; + std::unique_ptr<const char[]> filter_data; + Slice filter; + }; + std::deque<FilterEntry> filters; // list of partitioned filters and keys used + // in building the index + + // Set to the first non-okay status if any of the filter + // partitions experiences construction error. + // If partitioned_filters_construction_status_ is non-okay, + // then the whole partitioned filters should not be used. + Status partitioned_filters_construction_status_; + std::string last_filter_entry_key; + std::unique_ptr<const char[]> last_filter_data; + std::unique_ptr<IndexBuilder> value; + bool finishing_filters = + false; // true if Finish is called once but not complete yet. + // The policy of when cut a filter block and Finish it + void MaybeCutAFilterBlock(const Slice* next_key); + // Currently we keep the same number of partitions for filters and indexes. + // This would allow for some potentioal optimizations in future. If such + // optimizations did not realize we can use different number of partitions and + // eliminate p_index_builder_ + PartitionedIndexBuilder* const p_index_builder_; + // The desired number of keys per partition + uint32_t keys_per_partition_; + // The number of keys added to the last partition so far + uint32_t keys_added_to_partition_; + // According to the bits builders, how many keys/prefixes added + // in all the filters we have fully built + uint64_t total_added_in_built_; + BlockHandle last_encoded_handle_; +}; + +class PartitionedFilterBlockReader : public FilterBlockReaderCommon<Block> { + public: + PartitionedFilterBlockReader(const BlockBasedTable* t, + CachableEntry<Block>&& filter_block); + + static std::unique_ptr<FilterBlockReader> Create( + const BlockBasedTable* table, const ReadOptions& ro, + FilePrefetchBuffer* prefetch_buffer, bool use_cache, bool prefetch, + bool pin, BlockCacheLookupContext* lookup_context); + + bool KeyMayMatch(const Slice& key, const bool no_io, + const Slice* const const_ikey_ptr, GetContext* get_context, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) override; + void KeysMayMatch(MultiGetRange* range, const bool no_io, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) override; + + bool PrefixMayMatch(const Slice& prefix, const bool no_io, + const Slice* const const_ikey_ptr, + GetContext* get_context, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) override; + void PrefixesMayMatch(MultiGetRange* range, + const SliceTransform* prefix_extractor, + const bool no_io, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority) override; + + size_t ApproximateMemoryUsage() const override; + + private: + BlockHandle GetFilterPartitionHandle(const CachableEntry<Block>& filter_block, + const Slice& entry) const; + Status GetFilterPartitionBlock( + FilePrefetchBuffer* prefetch_buffer, const BlockHandle& handle, + bool no_io, GetContext* get_context, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority, + CachableEntry<ParsedFullFilterBlock>* filter_block) const; + + using FilterFunction = bool (FullFilterBlockReader::*)( + const Slice& slice, const bool no_io, const Slice* const const_ikey_ptr, + GetContext* get_context, BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority); + bool MayMatch(const Slice& slice, bool no_io, const Slice* const_ikey_ptr, + GetContext* get_context, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority, + FilterFunction filter_function) const; + using FilterManyFunction = void (FullFilterBlockReader::*)( + MultiGetRange* range, const SliceTransform* prefix_extractor, + const bool no_io, BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority); + void MayMatch(MultiGetRange* range, const SliceTransform* prefix_extractor, + bool no_io, BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority, + FilterManyFunction filter_function) const; + void MayMatchPartition(MultiGetRange* range, + const SliceTransform* prefix_extractor, + BlockHandle filter_handle, bool no_io, + BlockCacheLookupContext* lookup_context, + Env::IOPriority rate_limiter_priority, + FilterManyFunction filter_function) const; + Status CacheDependencies(const ReadOptions& ro, bool pin) override; + + const InternalKeyComparator* internal_comparator() const; + bool index_key_includes_seq() const; + bool index_value_is_full() const; + + protected: + // For partition blocks pinned in cache. Can be a subset of blocks + // in case some fail insertion on attempt to pin. + UnorderedMap<uint64_t, CachableEntry<ParsedFullFilterBlock>> filter_map_; +}; + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/partitioned_filter_block_test.cc b/src/rocksdb/table/block_based/partitioned_filter_block_test.cc new file mode 100644 index 000000000..0ce50d2bc --- /dev/null +++ b/src/rocksdb/table/block_based/partitioned_filter_block_test.cc @@ -0,0 +1,436 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). + +#include "table/block_based/partitioned_filter_block.h" + +#include <map> + +#include "index_builder.h" +#include "rocksdb/filter_policy.h" +#include "table/block_based/block_based_table_reader.h" +#include "table/block_based/filter_policy_internal.h" +#include "table/format.h" +#include "test_util/testharness.h" +#include "test_util/testutil.h" +#include "util/coding.h" +#include "util/hash.h" + +namespace ROCKSDB_NAMESPACE { + +std::map<uint64_t, std::string> blooms; + +class MockedBlockBasedTable : public BlockBasedTable { + public: + MockedBlockBasedTable(Rep* rep, PartitionedIndexBuilder* pib) + : BlockBasedTable(rep, /*block_cache_tracer=*/nullptr) { + // Initialize what Open normally does as much as necessary for the test + rep->index_key_includes_seq = pib->seperator_is_key_plus_seq(); + rep->index_value_is_full = !pib->get_use_value_delta_encoding(); + } +}; + +class MyPartitionedFilterBlockReader : public PartitionedFilterBlockReader { + public: + MyPartitionedFilterBlockReader(BlockBasedTable* t, + CachableEntry<Block>&& filter_block) + : PartitionedFilterBlockReader(t, std::move(filter_block)) { + for (const auto& pair : blooms) { + const uint64_t offset = pair.first; + const std::string& bloom = pair.second; + + assert(t); + assert(t->get_rep()); + CachableEntry<ParsedFullFilterBlock> block( + new ParsedFullFilterBlock( + t->get_rep()->table_options.filter_policy.get(), + BlockContents(Slice(bloom))), + nullptr /* cache */, nullptr /* cache_handle */, + true /* own_value */); + filter_map_[offset] = std::move(block); + } + } +}; + +class PartitionedFilterBlockTest + : public testing::Test, + virtual public ::testing::WithParamInterface<uint32_t> { + public: + Options options_; + ImmutableOptions ioptions_; + EnvOptions env_options_; + BlockBasedTableOptions table_options_; + InternalKeyComparator icomp_; + std::unique_ptr<BlockBasedTable> table_; + std::shared_ptr<Cache> cache_; + int bits_per_key_; + + PartitionedFilterBlockTest() + : ioptions_(options_), + env_options_(options_), + icomp_(options_.comparator), + bits_per_key_(10) { + table_options_.filter_policy.reset( + NewBloomFilterPolicy(bits_per_key_, false)); + table_options_.format_version = GetParam(); + table_options_.index_block_restart_interval = 3; + } + + ~PartitionedFilterBlockTest() override {} + + const std::string keys[4] = {"afoo", "bar", "box", "hello"}; + const std::string missing_keys[2] = {"missing", "other"}; + + uint64_t MaxIndexSize() { + int num_keys = sizeof(keys) / sizeof(*keys); + uint64_t max_key_size = 0; + for (int i = 1; i < num_keys; i++) { + max_key_size = + std::max(max_key_size, static_cast<uint64_t>(keys[i].size())); + } + uint64_t max_index_size = num_keys * (max_key_size + 8 /*handle*/); + return max_index_size; + } + + uint64_t MaxFilterSize() { + int num_keys = sizeof(keys) / sizeof(*keys); + // General, rough over-approximation + return num_keys * bits_per_key_ + (CACHE_LINE_SIZE * 8 + /*metadata*/ 5); + } + + uint64_t last_offset = 10; + BlockHandle Write(const Slice& slice) { + BlockHandle bh(last_offset + 1, slice.size()); + blooms[bh.offset()] = slice.ToString(); + last_offset += bh.size(); + return bh; + } + + PartitionedIndexBuilder* NewIndexBuilder() { + const bool kValueDeltaEncoded = true; + return PartitionedIndexBuilder::CreateIndexBuilder( + &icomp_, !kValueDeltaEncoded, table_options_); + } + + PartitionedFilterBlockBuilder* NewBuilder( + PartitionedIndexBuilder* const p_index_builder, + const SliceTransform* prefix_extractor = nullptr) { + assert(table_options_.block_size_deviation <= 100); + auto partition_size = + static_cast<uint32_t>(((table_options_.metadata_block_size * + (100 - table_options_.block_size_deviation)) + + 99) / + 100); + partition_size = std::max(partition_size, static_cast<uint32_t>(1)); + const bool kValueDeltaEncoded = true; + return new PartitionedFilterBlockBuilder( + prefix_extractor, table_options_.whole_key_filtering, + BloomFilterPolicy::GetBuilderFromContext( + FilterBuildingContext(table_options_)), + table_options_.index_block_restart_interval, !kValueDeltaEncoded, + p_index_builder, partition_size); + } + + PartitionedFilterBlockReader* NewReader( + PartitionedFilterBlockBuilder* builder, PartitionedIndexBuilder* pib) { + BlockHandle bh; + Status status; + Slice slice; + std::unique_ptr<const char[]> filter_data; + do { + slice = builder->Finish(bh, &status, &filter_data); + bh = Write(slice); + } while (status.IsIncomplete()); + + constexpr bool skip_filters = false; + constexpr uint64_t file_size = 12345; + constexpr int level = 0; + constexpr bool immortal_table = false; + table_.reset(new MockedBlockBasedTable( + new BlockBasedTable::Rep(ioptions_, env_options_, table_options_, + icomp_, skip_filters, file_size, level, + immortal_table), + pib)); + BlockContents contents(slice); + CachableEntry<Block> block( + new Block(std::move(contents), 0 /* read_amp_bytes_per_bit */, nullptr), + nullptr /* cache */, nullptr /* cache_handle */, true /* own_value */); + auto reader = + new MyPartitionedFilterBlockReader(table_.get(), std::move(block)); + return reader; + } + + void VerifyReader(PartitionedFilterBlockBuilder* builder, + PartitionedIndexBuilder* pib, bool empty = false) { + std::unique_ptr<PartitionedFilterBlockReader> reader( + NewReader(builder, pib)); + Env::IOPriority rate_limiter_priority = Env::IO_TOTAL; + // Querying added keys + const bool no_io = true; + for (auto key : keys) { + auto ikey = InternalKey(key, 0, ValueType::kTypeValue); + const Slice ikey_slice = Slice(*ikey.rep()); + ASSERT_TRUE(reader->KeyMayMatch(key, !no_io, &ikey_slice, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, + rate_limiter_priority)); + } + { + // querying a key twice + auto ikey = InternalKey(keys[0], 0, ValueType::kTypeValue); + const Slice ikey_slice = Slice(*ikey.rep()); + ASSERT_TRUE(reader->KeyMayMatch(keys[0], !no_io, &ikey_slice, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, + rate_limiter_priority)); + } + // querying missing keys + for (auto key : missing_keys) { + auto ikey = InternalKey(key, 0, ValueType::kTypeValue); + const Slice ikey_slice = Slice(*ikey.rep()); + if (empty) { + ASSERT_TRUE(reader->KeyMayMatch(key, !no_io, &ikey_slice, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, + rate_limiter_priority)); + } else { + // assuming a good hash function + ASSERT_FALSE(reader->KeyMayMatch(key, !no_io, &ikey_slice, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, + rate_limiter_priority)); + } + } + } + + int TestBlockPerKey() { + std::unique_ptr<PartitionedIndexBuilder> pib(NewIndexBuilder()); + std::unique_ptr<PartitionedFilterBlockBuilder> builder( + NewBuilder(pib.get())); + int i = 0; + builder->Add(keys[i]); + CutABlock(pib.get(), keys[i], keys[i + 1]); + i++; + builder->Add(keys[i]); + CutABlock(pib.get(), keys[i], keys[i + 1]); + i++; + builder->Add(keys[i]); + builder->Add(keys[i]); + CutABlock(pib.get(), keys[i], keys[i + 1]); + i++; + builder->Add(keys[i]); + CutABlock(pib.get(), keys[i]); + + VerifyReader(builder.get(), pib.get()); + return CountNumOfIndexPartitions(pib.get()); + } + + void TestBlockPerTwoKeys(const SliceTransform* prefix_extractor = nullptr) { + std::unique_ptr<PartitionedIndexBuilder> pib(NewIndexBuilder()); + std::unique_ptr<PartitionedFilterBlockBuilder> builder( + NewBuilder(pib.get(), prefix_extractor)); + int i = 0; + builder->Add(keys[i]); + i++; + builder->Add(keys[i]); + CutABlock(pib.get(), keys[i], keys[i + 1]); + i++; + builder->Add(keys[i]); + builder->Add(keys[i]); + i++; + builder->Add(keys[i]); + CutABlock(pib.get(), keys[i]); + + VerifyReader(builder.get(), pib.get(), prefix_extractor); + } + + void TestBlockPerAllKeys() { + std::unique_ptr<PartitionedIndexBuilder> pib(NewIndexBuilder()); + std::unique_ptr<PartitionedFilterBlockBuilder> builder( + NewBuilder(pib.get())); + int i = 0; + builder->Add(keys[i]); + i++; + builder->Add(keys[i]); + i++; + builder->Add(keys[i]); + builder->Add(keys[i]); + i++; + builder->Add(keys[i]); + CutABlock(pib.get(), keys[i]); + + VerifyReader(builder.get(), pib.get()); + } + + void CutABlock(PartitionedIndexBuilder* builder, + const std::string& user_key) { + // Assuming a block is cut, add an entry to the index + std::string key = + std::string(*InternalKey(user_key, 0, ValueType::kTypeValue).rep()); + BlockHandle dont_care_block_handle(1, 1); + builder->AddIndexEntry(&key, nullptr, dont_care_block_handle); + } + + void CutABlock(PartitionedIndexBuilder* builder, const std::string& user_key, + const std::string& next_user_key) { + // Assuming a block is cut, add an entry to the index + std::string key = + std::string(*InternalKey(user_key, 0, ValueType::kTypeValue).rep()); + std::string next_key = std::string( + *InternalKey(next_user_key, 0, ValueType::kTypeValue).rep()); + BlockHandle dont_care_block_handle(1, 1); + Slice slice = Slice(next_key.data(), next_key.size()); + builder->AddIndexEntry(&key, &slice, dont_care_block_handle); + } + + int CountNumOfIndexPartitions(PartitionedIndexBuilder* builder) { + IndexBuilder::IndexBlocks dont_care_ib; + BlockHandle dont_care_bh(10, 10); + Status s; + int cnt = 0; + do { + s = builder->Finish(&dont_care_ib, dont_care_bh); + cnt++; + } while (s.IsIncomplete()); + return cnt - 1; // 1 is 2nd level index + } +}; + +// Format versions potentially intersting to partitioning +INSTANTIATE_TEST_CASE_P(FormatVersions, PartitionedFilterBlockTest, + testing::ValuesIn(std::set<uint32_t>{ + 2, 3, 4, test::kDefaultFormatVersion, + kLatestFormatVersion})); + +TEST_P(PartitionedFilterBlockTest, EmptyBuilder) { + std::unique_ptr<PartitionedIndexBuilder> pib(NewIndexBuilder()); + std::unique_ptr<PartitionedFilterBlockBuilder> builder(NewBuilder(pib.get())); + const bool empty = true; + VerifyReader(builder.get(), pib.get(), empty); +} + +TEST_P(PartitionedFilterBlockTest, OneBlock) { + uint64_t max_index_size = MaxIndexSize(); + for (uint64_t i = 1; i < max_index_size + 1; i++) { + table_options_.metadata_block_size = i; + TestBlockPerAllKeys(); + } +} + +TEST_P(PartitionedFilterBlockTest, TwoBlocksPerKey) { + uint64_t max_index_size = MaxIndexSize(); + for (uint64_t i = 1; i < max_index_size + 1; i++) { + table_options_.metadata_block_size = i; + TestBlockPerTwoKeys(); + } +} + +// This reproduces the bug that a prefix is the same among multiple consecutive +// blocks but the bug would add it only to the first block. +TEST_P(PartitionedFilterBlockTest, SamePrefixInMultipleBlocks) { + // some small number to cause partition cuts + table_options_.metadata_block_size = 1; + std::unique_ptr<const SliceTransform> prefix_extractor( + ROCKSDB_NAMESPACE::NewFixedPrefixTransform(1)); + std::unique_ptr<PartitionedIndexBuilder> pib(NewIndexBuilder()); + std::unique_ptr<PartitionedFilterBlockBuilder> builder( + NewBuilder(pib.get(), prefix_extractor.get())); + const std::string pkeys[3] = {"p-key10", "p-key20", "p-key30"}; + builder->Add(pkeys[0]); + CutABlock(pib.get(), pkeys[0], pkeys[1]); + builder->Add(pkeys[1]); + CutABlock(pib.get(), pkeys[1], pkeys[2]); + builder->Add(pkeys[2]); + CutABlock(pib.get(), pkeys[2]); + std::unique_ptr<PartitionedFilterBlockReader> reader( + NewReader(builder.get(), pib.get())); + for (auto key : pkeys) { + auto ikey = InternalKey(key, 0, ValueType::kTypeValue); + const Slice ikey_slice = Slice(*ikey.rep()); + ASSERT_TRUE(reader->PrefixMayMatch(prefix_extractor->Transform(key), + /*no_io=*/false, &ikey_slice, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, + Env::IO_TOTAL)); + } + // Non-existent keys but with the same prefix + const std::string pnonkeys[4] = {"p-key9", "p-key11", "p-key21", "p-key31"}; + for (auto key : pnonkeys) { + auto ikey = InternalKey(key, 0, ValueType::kTypeValue); + const Slice ikey_slice = Slice(*ikey.rep()); + ASSERT_TRUE(reader->PrefixMayMatch(prefix_extractor->Transform(key), + /*no_io=*/false, &ikey_slice, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, + Env::IO_TOTAL)); + } +} + +// This reproduces the bug in format_version=3 that the seeking the prefix will +// lead us to the partition before the one that has filter for the prefix. +TEST_P(PartitionedFilterBlockTest, PrefixInWrongPartitionBug) { + // some small number to cause partition cuts + table_options_.metadata_block_size = 1; + std::unique_ptr<const SliceTransform> prefix_extractor( + ROCKSDB_NAMESPACE::NewFixedPrefixTransform(2)); + std::unique_ptr<PartitionedIndexBuilder> pib(NewIndexBuilder()); + std::unique_ptr<PartitionedFilterBlockBuilder> builder( + NewBuilder(pib.get(), prefix_extractor.get())); + // In the bug, searching for prefix "p3" on an index with format version 3, + // will give the key "p3" and the partition of the keys that are <= p3, i.e., + // p2-keys, where the filter for prefix "p3" does not exist. + const std::string pkeys[] = {"p1-key1", "p2-key2", "p3-key3", "p4-key3", + "p5-key3"}; + builder->Add(pkeys[0]); + CutABlock(pib.get(), pkeys[0], pkeys[1]); + builder->Add(pkeys[1]); + CutABlock(pib.get(), pkeys[1], pkeys[2]); + builder->Add(pkeys[2]); + CutABlock(pib.get(), pkeys[2], pkeys[3]); + builder->Add(pkeys[3]); + CutABlock(pib.get(), pkeys[3], pkeys[4]); + builder->Add(pkeys[4]); + CutABlock(pib.get(), pkeys[4]); + std::unique_ptr<PartitionedFilterBlockReader> reader( + NewReader(builder.get(), pib.get())); + Env::IOPriority rate_limiter_priority = Env::IO_TOTAL; + for (auto key : pkeys) { + auto prefix = prefix_extractor->Transform(key); + auto ikey = InternalKey(prefix, 0, ValueType::kTypeValue); + const Slice ikey_slice = Slice(*ikey.rep()); + ASSERT_TRUE(reader->PrefixMayMatch(prefix, + /*no_io=*/false, &ikey_slice, + /*get_context=*/nullptr, + /*lookup_context=*/nullptr, + rate_limiter_priority)); + } +} + +TEST_P(PartitionedFilterBlockTest, OneBlockPerKey) { + uint64_t max_index_size = MaxIndexSize(); + for (uint64_t i = 1; i < max_index_size + 1; i++) { + table_options_.metadata_block_size = i; + TestBlockPerKey(); + } +} + +TEST_P(PartitionedFilterBlockTest, PartitionCount) { + int num_keys = sizeof(keys) / sizeof(*keys); + table_options_.metadata_block_size = + std::max(MaxIndexSize(), MaxFilterSize()); + int partitions = TestBlockPerKey(); + ASSERT_EQ(partitions, 1); + // A low number ensures cutting a block after each key + table_options_.metadata_block_size = 1; + partitions = TestBlockPerKey(); + ASSERT_EQ(partitions, num_keys - 1 /* last two keys make one flush */); +} + +} // namespace ROCKSDB_NAMESPACE + +int main(int argc, char** argv) { + ROCKSDB_NAMESPACE::port::InstallStackTraceHandler(); + ::testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} diff --git a/src/rocksdb/table/block_based/partitioned_index_iterator.cc b/src/rocksdb/table/block_based/partitioned_index_iterator.cc new file mode 100644 index 000000000..b9bc2155a --- /dev/null +++ b/src/rocksdb/table/block_based/partitioned_index_iterator.cc @@ -0,0 +1,163 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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/block_based/partitioned_index_iterator.h" + +namespace ROCKSDB_NAMESPACE { +void PartitionedIndexIterator::Seek(const Slice& target) { SeekImpl(&target); } + +void PartitionedIndexIterator::SeekToFirst() { SeekImpl(nullptr); } + +void PartitionedIndexIterator::SeekImpl(const Slice* target) { + SavePrevIndexValue(); + + if (target) { + index_iter_->Seek(*target); + } else { + index_iter_->SeekToFirst(); + } + + if (!index_iter_->Valid()) { + ResetPartitionedIndexIter(); + return; + } + + InitPartitionedIndexBlock(); + + if (target) { + block_iter_.Seek(*target); + } else { + block_iter_.SeekToFirst(); + } + FindKeyForward(); + + // We could check upper bound here, but that would be too complicated + // and checking index upper bound is less useful than for data blocks. + + if (target) { + assert(!Valid() || (table_->get_rep()->index_key_includes_seq + ? (icomp_.Compare(*target, key()) <= 0) + : (user_comparator_.Compare(ExtractUserKey(*target), + key()) <= 0))); + } +} + +void PartitionedIndexIterator::SeekToLast() { + SavePrevIndexValue(); + index_iter_->SeekToLast(); + if (!index_iter_->Valid()) { + ResetPartitionedIndexIter(); + return; + } + InitPartitionedIndexBlock(); + block_iter_.SeekToLast(); + FindKeyBackward(); +} + +void PartitionedIndexIterator::Next() { + assert(block_iter_points_to_real_block_); + block_iter_.Next(); + FindKeyForward(); +} + +void PartitionedIndexIterator::Prev() { + assert(block_iter_points_to_real_block_); + block_iter_.Prev(); + + FindKeyBackward(); +} + +void PartitionedIndexIterator::InitPartitionedIndexBlock() { + BlockHandle partitioned_index_handle = index_iter_->value().handle; + if (!block_iter_points_to_real_block_ || + partitioned_index_handle.offset() != prev_block_offset_ || + // if previous attempt of reading the block missed cache, try again + block_iter_.status().IsIncomplete()) { + if (block_iter_points_to_real_block_) { + ResetPartitionedIndexIter(); + } + auto* rep = table_->get_rep(); + bool is_for_compaction = + lookup_context_.caller == TableReaderCaller::kCompaction; + // Prefetch additional data for range scans (iterators). + // Implicit auto readahead: + // Enabled after 2 sequential IOs when ReadOptions.readahead_size == 0. + // Explicit user requested readahead: + // Enabled from the very first IO when ReadOptions.readahead_size is set. + block_prefetcher_.PrefetchIfNeeded( + rep, partitioned_index_handle, read_options_.readahead_size, + is_for_compaction, /*no_sequential_checking=*/false, + read_options_.rate_limiter_priority); + Status s; + table_->NewDataBlockIterator<IndexBlockIter>( + read_options_, partitioned_index_handle, &block_iter_, + BlockType::kIndex, + /*get_context=*/nullptr, &lookup_context_, + block_prefetcher_.prefetch_buffer(), + /*for_compaction=*/is_for_compaction, /*async_read=*/false, s); + block_iter_points_to_real_block_ = true; + // We could check upper bound here but it is complicated to reason about + // upper bound in index iterator. On the other than, in large scans, index + // iterators are moved much less frequently compared to data blocks. So + // the upper bound check is skipped for simplicity. + } +} + +void PartitionedIndexIterator::FindKeyForward() { + // This method's code is kept short to make it likely to be inlined. + + assert(block_iter_points_to_real_block_); + + if (!block_iter_.Valid()) { + // This is the only call site of FindBlockForward(), but it's extracted into + // a separate method to keep FindKeyForward() short and likely to be + // inlined. When transitioning to a different block, we call + // FindBlockForward(), which is much longer and is probably not inlined. + FindBlockForward(); + } else { + // This is the fast path that avoids a function call. + } +} + +void PartitionedIndexIterator::FindBlockForward() { + // TODO the while loop inherits from two-level-iterator. We don't know + // whether a block can be empty so it can be replaced by an "if". + do { + if (!block_iter_.status().ok()) { + return; + } + ResetPartitionedIndexIter(); + index_iter_->Next(); + + if (!index_iter_->Valid()) { + return; + } + + InitPartitionedIndexBlock(); + block_iter_.SeekToFirst(); + } while (!block_iter_.Valid()); +} + +void PartitionedIndexIterator::FindKeyBackward() { + while (!block_iter_.Valid()) { + if (!block_iter_.status().ok()) { + return; + } + + ResetPartitionedIndexIter(); + index_iter_->Prev(); + + if (index_iter_->Valid()) { + InitPartitionedIndexBlock(); + block_iter_.SeekToLast(); + } else { + return; + } + } +} +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/partitioned_index_iterator.h b/src/rocksdb/table/block_based/partitioned_index_iterator.h new file mode 100644 index 000000000..6412fe239 --- /dev/null +++ b/src/rocksdb/table/block_based/partitioned_index_iterator.h @@ -0,0 +1,160 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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 "table/block_based/block_based_table_reader.h" +#include "table/block_based/block_based_table_reader_impl.h" +#include "table/block_based/block_prefetcher.h" +#include "table/block_based/reader_common.h" + +namespace ROCKSDB_NAMESPACE { +// Iterator that iterates over partitioned index. +// Some upper and lower bound tricks played in block based table iterators +// could be played here, but it's too complicated to reason about index +// keys with upper or lower bound, so we skip it for simplicity. +class PartitionedIndexIterator : public InternalIteratorBase<IndexValue> { + // compaction_readahead_size: its value will only be used if for_compaction = + // true + public: + PartitionedIndexIterator( + const BlockBasedTable* table, const ReadOptions& read_options, + const InternalKeyComparator& icomp, + std::unique_ptr<InternalIteratorBase<IndexValue>>&& index_iter, + TableReaderCaller caller, size_t compaction_readahead_size = 0) + : index_iter_(std::move(index_iter)), + table_(table), + read_options_(read_options), +#ifndef NDEBUG + icomp_(icomp), +#endif + user_comparator_(icomp.user_comparator()), + block_iter_points_to_real_block_(false), + lookup_context_(caller), + block_prefetcher_( + compaction_readahead_size, + table_->get_rep()->table_options.initial_auto_readahead_size) { + } + + ~PartitionedIndexIterator() override {} + + void Seek(const Slice& target) override; + void SeekForPrev(const Slice&) override { + // Shouldn't be called. + assert(false); + } + void SeekToFirst() override; + void SeekToLast() override; + void Next() final override; + bool NextAndGetResult(IterateResult*) override { + assert(false); + return false; + } + void Prev() override; + bool Valid() const override { + return block_iter_points_to_real_block_ && block_iter_.Valid(); + } + Slice key() const override { + assert(Valid()); + return block_iter_.key(); + } + Slice user_key() const override { + assert(Valid()); + return block_iter_.user_key(); + } + IndexValue value() const override { + assert(Valid()); + return block_iter_.value(); + } + Status status() const override { + // Prefix index set status to NotFound when the prefix does not exist + if (!index_iter_->status().ok() && !index_iter_->status().IsNotFound()) { + return index_iter_->status(); + } else if (block_iter_points_to_real_block_) { + return block_iter_.status(); + } else { + return Status::OK(); + } + } + inline IterBoundCheck UpperBoundCheckResult() override { + // Shouldn't be called. + assert(false); + return IterBoundCheck::kUnknown; + } + void SetPinnedItersMgr(PinnedIteratorsManager*) override { + // Shouldn't be called. + assert(false); + } + bool IsKeyPinned() const override { + // Shouldn't be called. + assert(false); + return false; + } + bool IsValuePinned() const override { + // Shouldn't be called. + assert(false); + return false; + } + + void ResetPartitionedIndexIter() { + if (block_iter_points_to_real_block_) { + block_iter_.Invalidate(Status::OK()); + block_iter_points_to_real_block_ = false; + } + } + + void SavePrevIndexValue() { + if (block_iter_points_to_real_block_) { + // Reseek. If they end up with the same data block, we shouldn't re-fetch + // the same data block. + prev_block_offset_ = index_iter_->value().handle.offset(); + } + } + + void GetReadaheadState(ReadaheadFileInfo* readahead_file_info) override { + if (block_prefetcher_.prefetch_buffer() != nullptr && + read_options_.adaptive_readahead) { + block_prefetcher_.prefetch_buffer()->GetReadaheadState( + &(readahead_file_info->index_block_readahead_info)); + } + } + + void SetReadaheadState(ReadaheadFileInfo* readahead_file_info) override { + if (read_options_.adaptive_readahead) { + block_prefetcher_.SetReadaheadState( + &(readahead_file_info->index_block_readahead_info)); + } + } + + std::unique_ptr<InternalIteratorBase<IndexValue>> index_iter_; + + private: + friend class BlockBasedTableReaderTestVerifyChecksum_ChecksumMismatch_Test; + const BlockBasedTable* table_; + const ReadOptions read_options_; +#ifndef NDEBUG + const InternalKeyComparator& icomp_; +#endif + UserComparatorWrapper user_comparator_; + IndexBlockIter block_iter_; + + // True if block_iter_ is initialized and points to the same block + // as index iterator. + bool block_iter_points_to_real_block_; + uint64_t prev_block_offset_ = std::numeric_limits<uint64_t>::max(); + BlockCacheLookupContext lookup_context_; + BlockPrefetcher block_prefetcher_; + + // If `target` is null, seek to first. + void SeekImpl(const Slice* target); + + void InitPartitionedIndexBlock(); + void FindKeyForward(); + void FindBlockForward(); + void FindKeyBackward(); +}; +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/partitioned_index_reader.cc b/src/rocksdb/table/block_based/partitioned_index_reader.cc new file mode 100644 index 000000000..017ea4a3a --- /dev/null +++ b/src/rocksdb/table/block_based/partitioned_index_reader.cc @@ -0,0 +1,215 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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/block_based/partitioned_index_reader.h" + +#include "file/random_access_file_reader.h" +#include "table/block_based/block_based_table_reader.h" +#include "table/block_based/partitioned_index_iterator.h" + +namespace ROCKSDB_NAMESPACE { +Status PartitionIndexReader::Create( + const BlockBasedTable* table, const ReadOptions& ro, + FilePrefetchBuffer* prefetch_buffer, bool use_cache, bool prefetch, + bool pin, BlockCacheLookupContext* lookup_context, + std::unique_ptr<IndexReader>* index_reader) { + assert(table != nullptr); + assert(table->get_rep()); + assert(!pin || prefetch); + assert(index_reader != nullptr); + + CachableEntry<Block> index_block; + if (prefetch || !use_cache) { + const Status s = + ReadIndexBlock(table, prefetch_buffer, ro, use_cache, + /*get_context=*/nullptr, lookup_context, &index_block); + if (!s.ok()) { + return s; + } + + if (use_cache && !pin) { + index_block.Reset(); + } + } + + index_reader->reset(new PartitionIndexReader(table, std::move(index_block))); + + return Status::OK(); +} + +InternalIteratorBase<IndexValue>* PartitionIndexReader::NewIterator( + const ReadOptions& read_options, bool /* disable_prefix_seek */, + IndexBlockIter* iter, GetContext* get_context, + BlockCacheLookupContext* lookup_context) { + const bool no_io = (read_options.read_tier == kBlockCacheTier); + CachableEntry<Block> index_block; + const Status s = + GetOrReadIndexBlock(no_io, read_options.rate_limiter_priority, + get_context, lookup_context, &index_block); + if (!s.ok()) { + if (iter != nullptr) { + iter->Invalidate(s); + return iter; + } + + return NewErrorInternalIterator<IndexValue>(s); + } + + const BlockBasedTable::Rep* rep = table()->rep_; + InternalIteratorBase<IndexValue>* it = nullptr; + + Statistics* kNullStats = nullptr; + // Filters are already checked before seeking the index + if (!partition_map_.empty()) { + // We don't return pinned data from index blocks, so no need + // to set `block_contents_pinned`. + it = NewTwoLevelIterator( + new BlockBasedTable::PartitionedIndexIteratorState(table(), + &partition_map_), + index_block.GetValue()->NewIndexIterator( + internal_comparator()->user_comparator(), + rep->get_global_seqno(BlockType::kIndex), nullptr, kNullStats, true, + index_has_first_key(), index_key_includes_seq(), + index_value_is_full())); + } else { + ReadOptions ro; + ro.fill_cache = read_options.fill_cache; + ro.deadline = read_options.deadline; + ro.io_timeout = read_options.io_timeout; + ro.adaptive_readahead = read_options.adaptive_readahead; + ro.async_io = read_options.async_io; + ro.rate_limiter_priority = read_options.rate_limiter_priority; + + // We don't return pinned data from index blocks, so no need + // to set `block_contents_pinned`. + std::unique_ptr<InternalIteratorBase<IndexValue>> index_iter( + index_block.GetValue()->NewIndexIterator( + internal_comparator()->user_comparator(), + rep->get_global_seqno(BlockType::kIndex), nullptr, kNullStats, true, + index_has_first_key(), index_key_includes_seq(), + index_value_is_full())); + + it = new PartitionedIndexIterator( + table(), ro, *internal_comparator(), std::move(index_iter), + lookup_context ? lookup_context->caller + : TableReaderCaller::kUncategorized); + } + + assert(it != nullptr); + index_block.TransferTo(it); + + return it; + + // TODO(myabandeh): Update TwoLevelIterator to be able to make use of + // on-stack BlockIter while the state is on heap. Currentlly it assumes + // the first level iter is always on heap and will attempt to delete it + // in its destructor. +} +Status PartitionIndexReader::CacheDependencies(const ReadOptions& ro, + bool pin) { + // Before read partitions, prefetch them to avoid lots of IOs + BlockCacheLookupContext lookup_context{TableReaderCaller::kPrefetch}; + const BlockBasedTable::Rep* rep = table()->rep_; + IndexBlockIter biter; + BlockHandle handle; + Statistics* kNullStats = nullptr; + + CachableEntry<Block> index_block; + { + Status s = GetOrReadIndexBlock(false /* no_io */, ro.rate_limiter_priority, + nullptr /* get_context */, &lookup_context, + &index_block); + if (!s.ok()) { + return s; + } + } + + // We don't return pinned data from index blocks, so no need + // to set `block_contents_pinned`. + index_block.GetValue()->NewIndexIterator( + internal_comparator()->user_comparator(), + rep->get_global_seqno(BlockType::kIndex), &biter, kNullStats, true, + index_has_first_key(), index_key_includes_seq(), index_value_is_full()); + // Index partitions are assumed to be consecuitive. Prefetch them all. + // Read the first block offset + biter.SeekToFirst(); + if (!biter.Valid()) { + // Empty index. + return biter.status(); + } + handle = biter.value().handle; + uint64_t prefetch_off = handle.offset(); + + // Read the last block's offset + biter.SeekToLast(); + if (!biter.Valid()) { + // Empty index. + return biter.status(); + } + handle = biter.value().handle; + uint64_t last_off = + handle.offset() + BlockBasedTable::BlockSizeWithTrailer(handle); + uint64_t prefetch_len = last_off - prefetch_off; + std::unique_ptr<FilePrefetchBuffer> prefetch_buffer; + rep->CreateFilePrefetchBuffer( + 0, 0, &prefetch_buffer, false /*Implicit auto readahead*/, + 0 /*num_reads_*/, 0 /*num_file_reads_for_auto_readahead*/); + IOOptions opts; + { + Status s = rep->file->PrepareIOOptions(ro, opts); + if (s.ok()) { + s = prefetch_buffer->Prefetch(opts, rep->file.get(), prefetch_off, + static_cast<size_t>(prefetch_len), + ro.rate_limiter_priority); + } + if (!s.ok()) { + return s; + } + } + + // For saving "all or nothing" to partition_map_ + UnorderedMap<uint64_t, CachableEntry<Block>> map_in_progress; + + // After prefetch, read the partitions one by one + biter.SeekToFirst(); + size_t partition_count = 0; + for (; biter.Valid(); biter.Next()) { + handle = biter.value().handle; + CachableEntry<Block> block; + ++partition_count; + // TODO: Support counter batch update for partitioned index and + // filter blocks + Status s = table()->MaybeReadBlockAndLoadToCache( + prefetch_buffer.get(), ro, handle, UncompressionDict::GetEmptyDict(), + /*wait=*/true, /*for_compaction=*/false, &block, BlockType::kIndex, + /*get_context=*/nullptr, &lookup_context, /*contents=*/nullptr, + /*async_read=*/false); + + if (!s.ok()) { + return s; + } + if (block.GetValue() != nullptr) { + // Might need to "pin" some mmap-read blocks (GetOwnValue) if some + // partitions are successfully compressed (cached) and some are not + // compressed (mmap eligible) + if (block.IsCached() || block.GetOwnValue()) { + if (pin) { + map_in_progress[handle.offset()] = std::move(block); + } + } + } + } + Status s = biter.status(); + // Save (pin) them only if everything checks out + if (map_in_progress.size() == partition_count && s.ok()) { + std::swap(partition_map_, map_in_progress); + } + return s; +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/partitioned_index_reader.h b/src/rocksdb/table/block_based/partitioned_index_reader.h new file mode 100644 index 000000000..58a7877ab --- /dev/null +++ b/src/rocksdb/table/block_based/partitioned_index_reader.h @@ -0,0 +1,55 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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 "table/block_based/index_reader_common.h" +#include "util/hash_containers.h" + +namespace ROCKSDB_NAMESPACE { +// Index that allows binary search lookup in a two-level index structure. +class PartitionIndexReader : public BlockBasedTable::IndexReaderCommon { + public: + // Read the partition index from the file and create an instance for + // `PartitionIndexReader`. + // On success, index_reader will be populated; otherwise it will remain + // unmodified. + static Status Create(const BlockBasedTable* table, const ReadOptions& ro, + FilePrefetchBuffer* prefetch_buffer, bool use_cache, + bool prefetch, bool pin, + BlockCacheLookupContext* lookup_context, + std::unique_ptr<IndexReader>* index_reader); + + // return a two-level iterator: first level is on the partition index + InternalIteratorBase<IndexValue>* NewIterator( + const ReadOptions& read_options, bool /* disable_prefix_seek */, + IndexBlockIter* iter, GetContext* get_context, + BlockCacheLookupContext* lookup_context) override; + + Status CacheDependencies(const ReadOptions& ro, bool pin) override; + size_t ApproximateMemoryUsage() const override { + size_t usage = ApproximateIndexBlockMemoryUsage(); +#ifdef ROCKSDB_MALLOC_USABLE_SIZE + usage += malloc_usable_size(const_cast<PartitionIndexReader*>(this)); +#else + usage += sizeof(*this); +#endif // ROCKSDB_MALLOC_USABLE_SIZE + // TODO(myabandeh): more accurate estimate of partition_map_ mem usage + return usage; + } + + private: + PartitionIndexReader(const BlockBasedTable* t, + CachableEntry<Block>&& index_block) + : IndexReaderCommon(t, std::move(index_block)) {} + + // For partition blocks pinned in cache. This is expected to be "all or + // none" so that !partition_map_.empty() can use an iterator expecting + // all partitions to be saved here. + UnorderedMap<uint64_t, CachableEntry<Block>> partition_map_; +}; +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/reader_common.cc b/src/rocksdb/table/block_based/reader_common.cc new file mode 100644 index 000000000..0ff43e9b4 --- /dev/null +++ b/src/rocksdb/table/block_based/reader_common.cc @@ -0,0 +1,52 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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/block_based/reader_common.h" + +#include "monitoring/perf_context_imp.h" +#include "rocksdb/table.h" +#include "table/format.h" +#include "util/coding.h" +#include "util/crc32c.h" +#include "util/string_util.h" + +namespace ROCKSDB_NAMESPACE { +void ForceReleaseCachedEntry(void* arg, void* h) { + Cache* cache = reinterpret_cast<Cache*>(arg); + Cache::Handle* handle = reinterpret_cast<Cache::Handle*>(h); + cache->Release(handle, true /* erase_if_last_ref */); +} + +// WART: this is specific to block-based table +Status VerifyBlockChecksum(ChecksumType type, const char* data, + size_t block_size, const std::string& file_name, + uint64_t offset) { + PERF_TIMER_GUARD(block_checksum_time); + // After block_size bytes is compression type (1 byte), which is part of + // the checksummed section. + size_t len = block_size + 1; + // And then the stored checksum value (4 bytes). + uint32_t stored = DecodeFixed32(data + len); + + uint32_t computed = ComputeBuiltinChecksum(type, data, len); + if (stored == computed) { + return Status::OK(); + } else { + // Unmask for people who might look for reference crc value + if (type == kCRC32c) { + stored = crc32c::Unmask(stored); + computed = crc32c::Unmask(computed); + } + return Status::Corruption( + "block checksum mismatch: stored = " + std::to_string(stored) + + ", computed = " + std::to_string(computed) + + ", type = " + std::to_string(type) + " in " + file_name + " offset " + + std::to_string(offset) + " size " + std::to_string(block_size)); + } +} +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/reader_common.h b/src/rocksdb/table/block_based/reader_common.h new file mode 100644 index 000000000..5bb199f28 --- /dev/null +++ b/src/rocksdb/table/block_based/reader_common.h @@ -0,0 +1,38 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root 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 "rocksdb/cache.h" +#include "rocksdb/table.h" + +namespace ROCKSDB_NAMESPACE { +// Release the cached entry and decrement its ref count. +extern void ForceReleaseCachedEntry(void* arg, void* h); + +inline MemoryAllocator* GetMemoryAllocator( + const BlockBasedTableOptions& table_options) { + return table_options.block_cache.get() + ? table_options.block_cache->memory_allocator() + : nullptr; +} + +inline MemoryAllocator* GetMemoryAllocatorForCompressedBlock( + const BlockBasedTableOptions& table_options) { + return table_options.block_cache_compressed.get() + ? table_options.block_cache_compressed->memory_allocator() + : nullptr; +} + +// Assumes block has a trailer as in format.h. file_name and offset provided +// for generating a diagnostic message in returned status. +extern Status VerifyBlockChecksum(ChecksumType type, const char* data, + size_t block_size, + const std::string& file_name, + uint64_t offset); +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/uncompression_dict_reader.cc b/src/rocksdb/table/block_based/uncompression_dict_reader.cc new file mode 100644 index 000000000..dc9a47ec7 --- /dev/null +++ b/src/rocksdb/table/block_based/uncompression_dict_reader.cc @@ -0,0 +1,124 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// + +#include "table/block_based/uncompression_dict_reader.h" + +#include "logging/logging.h" +#include "monitoring/perf_context_imp.h" +#include "table/block_based/block_based_table_reader.h" +#include "util/compression.h" + +namespace ROCKSDB_NAMESPACE { + +Status UncompressionDictReader::Create( + const BlockBasedTable* table, const ReadOptions& ro, + FilePrefetchBuffer* prefetch_buffer, bool use_cache, bool prefetch, + bool pin, BlockCacheLookupContext* lookup_context, + std::unique_ptr<UncompressionDictReader>* uncompression_dict_reader) { + assert(table); + assert(table->get_rep()); + assert(!pin || prefetch); + assert(uncompression_dict_reader); + + CachableEntry<UncompressionDict> uncompression_dict; + if (prefetch || !use_cache) { + const Status s = ReadUncompressionDictionary( + table, prefetch_buffer, ro, use_cache, nullptr /* get_context */, + lookup_context, &uncompression_dict); + if (!s.ok()) { + return s; + } + + if (use_cache && !pin) { + uncompression_dict.Reset(); + } + } + + uncompression_dict_reader->reset( + new UncompressionDictReader(table, std::move(uncompression_dict))); + + return Status::OK(); +} + +Status UncompressionDictReader::ReadUncompressionDictionary( + const BlockBasedTable* table, FilePrefetchBuffer* prefetch_buffer, + const ReadOptions& read_options, bool use_cache, GetContext* get_context, + BlockCacheLookupContext* lookup_context, + CachableEntry<UncompressionDict>* uncompression_dict) { + // TODO: add perf counter for compression dictionary read time + + assert(table); + assert(uncompression_dict); + assert(uncompression_dict->IsEmpty()); + + const BlockBasedTable::Rep* const rep = table->get_rep(); + assert(rep); + assert(!rep->compression_dict_handle.IsNull()); + + const Status s = table->RetrieveBlock( + prefetch_buffer, read_options, rep->compression_dict_handle, + UncompressionDict::GetEmptyDict(), uncompression_dict, + BlockType::kCompressionDictionary, get_context, lookup_context, + /* for_compaction */ false, use_cache, /* wait_for_cache */ true, + /* async_read */ false); + + if (!s.ok()) { + ROCKS_LOG_WARN( + rep->ioptions.logger, + "Encountered error while reading data from compression dictionary " + "block %s", + s.ToString().c_str()); + } + + return s; +} + +Status UncompressionDictReader::GetOrReadUncompressionDictionary( + FilePrefetchBuffer* prefetch_buffer, bool no_io, bool verify_checksums, + GetContext* get_context, BlockCacheLookupContext* lookup_context, + CachableEntry<UncompressionDict>* uncompression_dict) const { + assert(uncompression_dict); + + if (!uncompression_dict_.IsEmpty()) { + uncompression_dict->SetUnownedValue(uncompression_dict_.GetValue()); + return Status::OK(); + } + + ReadOptions read_options; + if (no_io) { + read_options.read_tier = kBlockCacheTier; + } + read_options.verify_checksums = verify_checksums; + + return ReadUncompressionDictionary(table_, prefetch_buffer, read_options, + cache_dictionary_blocks(), get_context, + lookup_context, uncompression_dict); +} + +size_t UncompressionDictReader::ApproximateMemoryUsage() const { + assert(!uncompression_dict_.GetOwnValue() || + uncompression_dict_.GetValue() != nullptr); + size_t usage = uncompression_dict_.GetOwnValue() + ? uncompression_dict_.GetValue()->ApproximateMemoryUsage() + : 0; + +#ifdef ROCKSDB_MALLOC_USABLE_SIZE + usage += malloc_usable_size(const_cast<UncompressionDictReader*>(this)); +#else + usage += sizeof(*this); +#endif // ROCKSDB_MALLOC_USABLE_SIZE + + return usage; +} + +bool UncompressionDictReader::cache_dictionary_blocks() const { + assert(table_); + assert(table_->get_rep()); + + return table_->get_rep()->table_options.cache_index_and_filter_blocks; +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/rocksdb/table/block_based/uncompression_dict_reader.h b/src/rocksdb/table/block_based/uncompression_dict_reader.h new file mode 100644 index 000000000..416d25e2d --- /dev/null +++ b/src/rocksdb/table/block_based/uncompression_dict_reader.h @@ -0,0 +1,60 @@ +// Copyright (c) 2011-present, Facebook, Inc. All rights reserved. +// This source code is licensed under both the GPLv2 (found in the +// COPYING file in the root directory) and Apache 2.0 License +// (found in the LICENSE.Apache file in the root directory). +// + +#pragma once + +#include <cassert> + +#include "table/block_based/cachable_entry.h" +#include "table/format.h" + +namespace ROCKSDB_NAMESPACE { + +class BlockBasedTable; +struct BlockCacheLookupContext; +class FilePrefetchBuffer; +class GetContext; +struct ReadOptions; +struct UncompressionDict; + +// Provides access to the uncompression dictionary regardless of whether +// it is owned by the reader or stored in the cache, or whether it is pinned +// in the cache or not. +class UncompressionDictReader { + public: + static Status Create( + const BlockBasedTable* table, const ReadOptions& ro, + FilePrefetchBuffer* prefetch_buffer, bool use_cache, bool prefetch, + bool pin, BlockCacheLookupContext* lookup_context, + std::unique_ptr<UncompressionDictReader>* uncompression_dict_reader); + + Status GetOrReadUncompressionDictionary( + FilePrefetchBuffer* prefetch_buffer, bool no_io, bool verify_checksums, + GetContext* get_context, BlockCacheLookupContext* lookup_context, + CachableEntry<UncompressionDict>* uncompression_dict) const; + + size_t ApproximateMemoryUsage() const; + + private: + UncompressionDictReader(const BlockBasedTable* t, + CachableEntry<UncompressionDict>&& uncompression_dict) + : table_(t), uncompression_dict_(std::move(uncompression_dict)) { + assert(table_); + } + + bool cache_dictionary_blocks() const; + + static Status ReadUncompressionDictionary( + const BlockBasedTable* table, FilePrefetchBuffer* prefetch_buffer, + const ReadOptions& read_options, bool use_cache, GetContext* get_context, + BlockCacheLookupContext* lookup_context, + CachableEntry<UncompressionDict>* uncompression_dict); + + const BlockBasedTable* table_; + CachableEntry<UncompressionDict> uncompression_dict_; +}; + +} // namespace ROCKSDB_NAMESPACE |