rocksdb/table/plain/plain_table_reader.h

243 lines
8.8 KiB
C
Raw Permalink Normal View History

// Copyright (c) Facebook, Inc. and its affiliates. All Rights Reserved.
// 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
#ifndef ROCKSDB_LITE
#include <unordered_map>
#include <memory>
#include <vector>
#include <string>
#include <stdint.h>
#include "file/random_access_file_reader.h"
#include "memory/arena.h"
#include "rocksdb/env.h"
#include "rocksdb/iterator.h"
#include "rocksdb/slice_transform.h"
#include "rocksdb/table.h"
#include "rocksdb/table_properties.h"
#include "table/plain/plain_table_bloom.h"
#include "table/plain/plain_table_factory.h"
#include "table/plain/plain_table_index.h"
#include "table/table_reader.h"
namespace ROCKSDB_NAMESPACE {
class Block;
2014-07-20 16:56:40 +02:00
struct BlockContents;
class BlockHandle;
class Footer;
struct Options;
class RandomAccessFile;
struct ReadOptions;
class TableCache;
class TableReader;
class InternalKeyComparator;
class PlainTableKeyDecoder;
class GetContext;
extern const uint32_t kPlainTableVariableLength;
struct PlainTableReaderFileInfo {
bool is_mmap_mode;
Slice file_data;
uint32_t data_end_offset;
std::unique_ptr<RandomAccessFileReader> file;
PlainTableReaderFileInfo(std::unique_ptr<RandomAccessFileReader>&& _file,
const EnvOptions& storage_options,
uint32_t _data_size_offset)
: is_mmap_mode(storage_options.use_mmap_reads),
data_end_offset(_data_size_offset),
file(std::move(_file)) {}
};
// The reader class of PlainTable. For description of PlainTable format
// See comments of class PlainTableFactory, where instances of
// PlainTableReader are created.
class PlainTableReader: public TableReader {
public:
// Based on following output file format shown in plain_table_factory.h
// When opening the output file, PlainTableReader creates a hash table
// from key prefixes to offset of the output file. PlainTable will decide
// whether it points to the data offset of the first key with the key prefix
// or the offset of it. If there are too many keys share this prefix, it will
// create a binary search-able index from the suffix to offset on disk.
static Status Open(const ImmutableOptions& ioptions,
const EnvOptions& env_options,
const InternalKeyComparator& internal_comparator,
std::unique_ptr<RandomAccessFileReader>&& file,
uint64_t file_size, std::unique_ptr<TableReader>* table,
const int bloom_bits_per_key, double hash_table_ratio,
size_t index_sparseness, size_t huge_page_tlb_size,
bool full_scan_mode, const bool immortal_table = false,
const SliceTransform* prefix_extractor = nullptr);
// Returns new iterator over table contents
// compaction_readahead_size: its value will only be used if for_compaction =
// true
InternalIterator* NewIterator(const ReadOptions&,
const SliceTransform* prefix_extractor,
Arena* arena, bool skip_filters,
TableReaderCaller caller,
Properly report IO errors when IndexType::kBinarySearchWithFirstKey is used (#6621) Summary: Context: Index type `kBinarySearchWithFirstKey` added the ability for sst file iterator to sometimes report a key from index without reading the corresponding data block. This is useful when sst blocks are cut at some meaningful boundaries (e.g. one block per key prefix), and many seeks land between blocks (e.g. for each prefix, the ranges of keys in different sst files are nearly disjoint, so a typical seek needs to read a data block from only one file even if all files have the prefix). But this added a new error condition, which rocksdb code was really not equipped to deal with: `InternalIterator::value()` may fail with an IO error or Status::Incomplete, but it's just a method returning a Slice, with no way to report error instead. Before this PR, this type of error wasn't handled at all (an empty slice was returned), and kBinarySearchWithFirstKey implementation was considered a prototype. Now that we (LogDevice) have experimented with kBinarySearchWithFirstKey for a while and confirmed that it's really useful, this PR is adding the missing error handling. It's a pretty inconvenient situation implementation-wise. The error needs to be reported from InternalIterator when trying to access value. But there are ~700 call sites of `InternalIterator::value()`, most of which either can't hit the error condition (because the iterator is reading from memtable or from index or something) or wouldn't benefit from the deferred loading of the value (e.g. compaction iterator that reads all values anyway). Adding error handling to all these call sites would needlessly bloat the code. So instead I made the deferred value loading optional: only the call sites that may use deferred loading have to call the new method `PrepareValue()` before calling `value()`. The feature is enabled with a new bool argument `allow_unprepared_value` to a bunch of methods that create iterators (it wouldn't make sense to put it in ReadOptions because it's completely internal to iterators, with virtually no user-visible effect). Lmk if you have better ideas. Note that the deferred value loading only happens for *internal* iterators. The user-visible iterator (DBIter) always prepares the value before returning from Seek/Next/etc. We could go further and add an API to defer that value loading too, but that's most likely not useful for LogDevice, so it doesn't seem worth the complexity for now. Pull Request resolved: https://github.com/facebook/rocksdb/pull/6621 Test Plan: make -j5 check . Will also deploy to some logdevice test clusters and look at stats. Reviewed By: siying Differential Revision: D20786930 Pulled By: al13n321 fbshipit-source-id: 6da77d918bad3780522e918f17f4d5513d3e99ee
2020-04-16 02:37:23 +02:00
size_t compaction_readahead_size = 0,
bool allow_unprepared_value = false) override;
void Prepare(const Slice& target) override;
Status Get(const ReadOptions& readOptions, const Slice& key,
GetContext* get_context, const SliceTransform* prefix_extractor,
bool skip_filters = false) override;
Create a BlockCacheLookupContext to enable fine-grained block cache tracing. (#5421) Summary: BlockCacheLookupContext only contains the caller for now. We will trace block accesses at five places: 1. BlockBasedTable::GetFilter. 2. BlockBasedTable::GetUncompressedDict. 3. BlockBasedTable::MaybeReadAndLoadToCache. (To trace access on data, index, and range deletion block.) 4. BlockBasedTable::Get. (To trace the referenced key and whether the referenced key exists in a fetched data block.) 5. BlockBasedTable::MultiGet. (To trace the referenced key and whether the referenced key exists in a fetched data block.) We create the context at: 1. BlockBasedTable::Get. (kUserGet) 2. BlockBasedTable::MultiGet. (kUserMGet) 3. BlockBasedTable::NewIterator. (either kUserIterator, kCompaction, or external SST ingestion calls this function.) 4. BlockBasedTable::Open. (kPrefetch) 5. Index/Filter::CacheDependencies. (kPrefetch) 6. BlockBasedTable::ApproximateOffsetOf. (kCompaction or kUserApproximateSize). I loaded 1 million key-value pairs into the database and ran the readrandom benchmark with a single thread. I gave the block cache 10 GB to make sure all reads hit the block cache after warmup. The throughput is comparable. Throughput of this PR: 231334 ops/s. Throughput of the master branch: 238428 ops/s. Experiment setup: RocksDB: version 6.2 Date: Mon Jun 10 10:42:51 2019 CPU: 24 * Intel Core Processor (Skylake) CPUCache: 16384 KB Keys: 20 bytes each Values: 100 bytes each (100 bytes after compression) Entries: 1000000 Prefix: 20 bytes Keys per prefix: 0 RawSize: 114.4 MB (estimated) FileSize: 114.4 MB (estimated) Write rate: 0 bytes/second Read rate: 0 ops/second Compression: NoCompression Compression sampling rate: 0 Memtablerep: skip_list Perf Level: 1 Load command: ./db_bench --benchmarks="fillseq" --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 Run command: ./db_bench --benchmarks="readrandom,stats" --use_existing_db --threads=1 --duration=120 --key_size=20 --prefix_size=20 --keys_per_prefix=0 --value_size=100 --statistics --cache_index_and_filter_blocks --cache_size=10737418240 --disable_auto_compactions=1 --disable_wal=1 --compression_type=none --min_level_to_compress=-1 --compression_ratio=1 --num=1000000 --duration=120 TODOs: 1. Create a caller for external SST file ingestion and differentiate the callers for iterator. 2. Integrate tracer to trace block cache accesses. Pull Request resolved: https://github.com/facebook/rocksdb/pull/5421 Differential Revision: D15704258 Pulled By: HaoyuHuang fbshipit-source-id: 4aa8a55f8cb1576ffb367bfa3186a91d8f06d93a
2019-06-11 00:30:05 +02:00
uint64_t ApproximateOffsetOf(const Slice& key,
TableReaderCaller caller) override;
uint64_t ApproximateSize(const Slice& start, const Slice& end,
TableReaderCaller caller) override;
uint32_t GetIndexSize() const { return index_.GetIndexSize(); }
void SetupForCompaction() override;
std::shared_ptr<const TableProperties> GetTableProperties() const override {
return table_properties_;
}
virtual size_t ApproximateMemoryUsage() const override {
return arena_.MemoryAllocatedBytes();
}
PlainTableReader(const ImmutableOptions& ioptions,
std::unique_ptr<RandomAccessFileReader>&& file,
const EnvOptions& env_options,
const InternalKeyComparator& internal_comparator,
EncodingType encoding_type, uint64_t file_size,
const TableProperties* table_properties,
const SliceTransform* prefix_extractor);
virtual ~PlainTableReader();
protected:
// Check bloom filter to see whether it might contain this prefix.
// The hash of the prefix is given, since it can be reused for index lookup
// too.
virtual bool MatchBloom(uint32_t hash) const;
// PopulateIndex() builds index of keys. It must be called before any query
// to the table.
//
// props: the table properties object that need to be stored. Ownership of
// the object will be passed.
//
Status PopulateIndex(TableProperties* props, int bloom_bits_per_key,
double hash_table_ratio, size_t index_sparseness,
size_t huge_page_tlb_size);
Status MmapDataIfNeeded();
private:
const InternalKeyComparator internal_comparator_;
EncodingType encoding_type_;
// represents plain table's current status.
Status status_;
PlainTableIndex index_;
bool full_scan_mode_;
// data_start_offset_ and data_end_offset_ defines the range of the
// sst file that stores data.
const uint32_t data_start_offset_ = 0;
const uint32_t user_key_len_;
const SliceTransform* prefix_extractor_;
static const size_t kNumInternalBytes = 8;
// Bloom filter is used to rule out non-existent key
bool enable_bloom_;
PlainTableBloomV1 bloom_;
PlainTableReaderFileInfo file_info_;
Arena arena_;
CacheAllocationPtr index_block_alloc_;
CacheAllocationPtr bloom_block_alloc_;
const ImmutableOptions& ioptions_;
std::unique_ptr<Cleanable> dummy_cleanable_;
uint64_t file_size_;
protected: // for testing
std::shared_ptr<const TableProperties> table_properties_;
private:
bool IsFixedLength() const {
return user_key_len_ != kPlainTableVariableLength;
}
size_t GetFixedInternalKeyLength() const {
return user_key_len_ + kNumInternalBytes;
}
Slice GetPrefix(const Slice& target) const {
assert(target.size() >= 8); // target is internal key
return GetPrefixFromUserKey(ExtractUserKey(target));
}
Slice GetPrefix(const ParsedInternalKey& target) const {
return GetPrefixFromUserKey(target.user_key);
}
Slice GetPrefixFromUserKey(const Slice& user_key) const {
if (!IsTotalOrderMode()) {
return prefix_extractor_->Transform(user_key);
} else {
// Use empty slice as prefix if prefix_extractor is not set.
// In that case,
// it falls back to pure binary search and
// total iterator seek is supported.
return Slice();
}
}
friend class TableCache;
friend class PlainTableIterator;
// Internal helper function to generate an IndexRecordList object from all
// the rows, which contains index records as a list.
// If bloom_ is not null, all the keys' full-key hash will be added to the
// bloom filter.
Status PopulateIndexRecordList(PlainTableIndexBuilder* index_builder,
std::vector<uint32_t>* prefix_hashes);
// Internal helper function to allocate memory for bloom filter
void AllocateBloom(int bloom_bits_per_key, int num_prefixes,
size_t huge_page_tlb_size);
void FillBloom(const std::vector<uint32_t>& prefix_hashes);
// Read the key and value at `offset` to parameters for keys, the and
// `seekable`.
// On success, `offset` will be updated as the offset for the next key.
// `parsed_key` will be key in parsed format.
// if `internal_key` is not empty, it will be filled with key with slice
// format.
// if `seekable` is not null, it will return whether we can directly read
// data using this offset.
Status Next(PlainTableKeyDecoder* decoder, uint32_t* offset,
ParsedInternalKey* parsed_key, Slice* internal_key, Slice* value,
bool* seekable = nullptr) const;
// Get file offset for key target.
// return value prefix_matched is set to true if the offset is confirmed
// for a key with the same prefix as target.
Status GetOffset(PlainTableKeyDecoder* decoder, const Slice& target,
const Slice& prefix, uint32_t prefix_hash,
bool& prefix_matched, uint32_t* offset) const;
bool IsTotalOrderMode() const { return (prefix_extractor_ == nullptr); }
// No copying allowed
explicit PlainTableReader(const TableReader&) = delete;
void operator=(const TableReader&) = delete;
};
} // namespace ROCKSDB_NAMESPACE
#endif // ROCKSDB_LITE