mirror of
https://github.com/facebook/rocksdb.git
synced 2024-11-25 14:31:35 +00:00
39455974cb
Summary: Not sure where or how it happens, but using a recent CircleCI failure I got a reliable db_stress reproducer. Using std::unique_ptr appropriately for managing them has apparently (and unsurprisingly) fixed the problem without needing to know exactly where the problem was. Suggested follow-up: * Three or even four levels of pointers is very confusing to work with. Surely this part can be cleaned up to be simpler. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12805 Test Plan: Reproducer passes, plus ASAN test and crash test runs. I don't think it's worth the extra work to track down the details and create a careful unit test. ``` ./db_stress --WAL_size_limit_MB=1 --WAL_ttl_seconds=60 --acquire_snapshot_one_in=10000 --adaptive_readahead=1 --adm_policy=2 --advise_random_on_open=1 --allow_data_in_errors=True --allow_fallocate=1 --async_io=0 --auto_readahead_size=1 --avoid_flush_during_recovery=0 --avoid_flush_during_shutdown=1 --avoid_unnecessary_blocking_io=1 --backup_max_size=104857600 --backup_one_in=100000 --batch_protection_bytes_per_key=0 --bgerror_resume_retry_interval=1000000 --block_align=1 --block_protection_bytes_per_key=4 --block_size=16384 --bloom_before_level=2147483646 --bloom_bits=15 --bottommost_compression_type=none --bottommost_file_compaction_delay=3600 --bytes_per_sync=262144 --cache_index_and_filter_blocks=0 --cache_index_and_filter_blocks_with_high_priority=0 --cache_size=33554432 --cache_type=tiered_lru_cache --charge_compression_dictionary_building_buffer=0 --charge_file_metadata=1 --charge_filter_construction=0 --charge_table_reader=0 --check_multiget_consistency=1 --check_multiget_entity_consistency=1 --checkpoint_one_in=10000 --checksum_type=kxxHash --clear_column_family_one_in=0 --compact_files_one_in=1000000 --compact_range_one_in=1000 --compaction_pri=0 --compaction_readahead_size=0 --compaction_ttl=0 --compress_format_version=2 --compressed_secondary_cache_ratio=0.2 --compressed_secondary_cache_size=0 --compression_checksum=0 --compression_max_dict_buffer_bytes=0 --compression_max_dict_bytes=0 --compression_parallel_threads=1 --compression_type=none --compression_use_zstd_dict_trainer=0 --compression_zstd_max_train_bytes=0 --continuous_verification_interval=0 --daily_offpeak_time_utc= --data_block_index_type=0 --db=/dev/shm/rocksdb.gpxs/rocksdb_crashtest_blackbox --db_write_buffer_size=0 --default_temperature=kWarm --default_write_temperature=kCold --delete_obsolete_files_period_micros=21600000000 --delpercent=4 --delrangepercent=1 --destroy_db_initially=0 --detect_filter_construct_corruption=0 --disable_file_deletions_one_in=10000 --disable_manual_compaction_one_in=1000000 --disable_wal=0 --dump_malloc_stats=1 --enable_checksum_handoff=1 --enable_compaction_filter=0 --enable_custom_split_merge=0 --enable_do_not_compress_roles=0 --enable_index_compression=0 --enable_memtable_insert_with_hint_prefix_extractor=0 --enable_pipelined_write=1 --enable_sst_partitioner_factory=0 --enable_thread_tracking=1 --enable_write_thread_adaptive_yield=0 --error_recovery_with_no_fault_injection=0 --expected_values_dir=/dev/shm/rocksdb.gpxs/rocksdb_crashtest_expected --fail_if_options_file_error=0 --fifo_allow_compaction=0 --file_checksum_impl=none --fill_cache=1 --flush_one_in=1000000 --format_version=3 --get_all_column_family_metadata_one_in=1000000 --get_current_wal_file_one_in=0 --get_live_files_apis_one_in=10000 --get_properties_of_all_tables_one_in=100000 --get_property_one_in=100000 --get_sorted_wal_files_one_in=0 --hard_pending_compaction_bytes_limit=274877906944 --high_pri_pool_ratio=0 --index_block_restart_interval=4 --index_shortening=0 --index_type=0 --ingest_external_file_one_in=0 --initial_auto_readahead_size=16384 --inplace_update_support=0 --iterpercent=10 --key_len_percent_dist=1,30,69 --key_may_exist_one_in=100 --last_level_temperature=kHot --level_compaction_dynamic_level_bytes=0 --lock_wal_one_in=1000000 --log_file_time_to_roll=0 --log_readahead_size=0 --long_running_snapshots=1 --low_pri_pool_ratio=0 --lowest_used_cache_tier=2 --manifest_preallocation_size=5120 --manual_wal_flush_one_in=1000 --mark_for_compaction_one_file_in=10 --max_auto_readahead_size=16384 --max_background_compactions=20 --max_bytes_for_level_base=10485760 --max_key=2500000 --max_key_len=3 --max_log_file_size=0 --max_manifest_file_size=1073741824 --max_sequential_skip_in_iterations=1 --max_total_wal_size=0 --max_write_batch_group_size_bytes=16 --max_write_buffer_number=3 --max_write_buffer_size_to_maintain=0 --memtable_insert_hint_per_batch=1 --memtable_max_range_deletions=100 --memtable_prefix_bloom_size_ratio=0 --memtable_protection_bytes_per_key=4 --memtable_whole_key_filtering=0 --memtablerep=skip_list --metadata_charge_policy=0 --metadata_read_fault_one_in=32 --metadata_write_fault_one_in=0 --min_write_buffer_number_to_merge=2 --mmap_read=1 --mock_direct_io=False --nooverwritepercent=1 --num_file_reads_for_auto_readahead=0 --open_files=100 --open_metadata_read_fault_one_in=0 --open_metadata_write_fault_one_in=8 --open_read_fault_one_in=0 --open_write_fault_one_in=16 --ops_per_thread=100000000 --optimize_filters_for_hits=1 --optimize_filters_for_memory=0 --optimize_multiget_for_io=1 --paranoid_file_checks=1 --partition_filters=0 --partition_pinning=1 --pause_background_one_in=1000000 --periodic_compaction_seconds=0 --prefix_size=-1 --prefixpercent=0 --prepopulate_block_cache=1 --preserve_internal_time_seconds=60 --progress_reports=0 --promote_l0_one_in=0 --read_amp_bytes_per_bit=0 --read_fault_one_in=32 --readahead_size=524288 --readpercent=50 --recycle_log_file_num=1 --reopen=0 --report_bg_io_stats=1 --reset_stats_one_in=10000 --sample_for_compression=5 --secondary_cache_fault_one_in=32 --secondary_cache_uri= --set_options_one_in=10000 --skip_stats_update_on_db_open=0 --snapshot_hold_ops=100000 --soft_pending_compaction_bytes_limit=68719476736 --sqfc_name=bar --sqfc_version=1 --sst_file_manager_bytes_per_sec=104857600 --sst_file_manager_bytes_per_truncate=0 --stats_dump_period_sec=0 --stats_history_buffer_size=1048576 --strict_bytes_per_sync=1 --subcompactions=3 --sync=0 --sync_fault_injection=1 --table_cache_numshardbits=0 --target_file_size_base=524288 --target_file_size_multiplier=2 --test_batches_snapshots=0 --test_cf_consistency=1 --top_level_index_pinning=1 --uncache_aggressiveness=5 --universal_max_read_amp=-1 --unpartitioned_pinning=2 --use_adaptive_mutex=0 --use_adaptive_mutex_lru=0 --use_attribute_group=1 --use_delta_encoding=1 --use_direct_io_for_flush_and_compaction=0 --use_direct_reads=0 --use_full_merge_v1=0 --use_get_entity=0 --use_merge=0 --use_multi_cf_iterator=0 --use_multi_get_entity=0 --use_multiget=1 --use_put_entity_one_in=1 --use_sqfc_for_range_queries=1 --use_timed_put_one_in=0 --use_write_buffer_manager=0 --user_timestamp_size=0 --value_size_mult=32 --verification_only=0 --verify_checksum=1 --verify_checksum_one_in=1000000 --verify_compression=1 --verify_db_one_in=100000 --verify_file_checksums_one_in=0 --verify_iterator_with_expected_state_one_in=0 --verify_sst_unique_id_in_manifest=1 --wal_bytes_per_sync=0 --wal_compression=none --write_buffer_size=1048576 --write_dbid_to_manifest=1 --write_fault_one_in=0 --writepercent=35 ``` Reviewed By: cbi42 Differential Revision: D58958390 Pulled By: pdillinger fbshipit-source-id: 1271cfdcc3c574f78cd59f3c68148f7ed4a19c47
299 lines
14 KiB
C++
299 lines
14 KiB
C++
// 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.
|
|
//
|
|
// Thread-safe (provides internal synchronization)
|
|
|
|
#pragma once
|
|
#include <cstdint>
|
|
#include <string>
|
|
#include <vector>
|
|
|
|
#include "cache/typed_cache.h"
|
|
#include "db/dbformat.h"
|
|
#include "db/range_del_aggregator.h"
|
|
#include "options/cf_options.h"
|
|
#include "port/port.h"
|
|
#include "rocksdb/cache.h"
|
|
#include "rocksdb/env.h"
|
|
#include "rocksdb/options.h"
|
|
#include "rocksdb/table.h"
|
|
#include "table/table_reader.h"
|
|
#include "trace_replay/block_cache_tracer.h"
|
|
#include "util/coro_utils.h"
|
|
|
|
namespace ROCKSDB_NAMESPACE {
|
|
|
|
class Env;
|
|
class Arena;
|
|
struct FileDescriptor;
|
|
class GetContext;
|
|
class HistogramImpl;
|
|
|
|
// Manages caching for TableReader objects for a column family. The actual
|
|
// cache is allocated separately and passed to the constructor. TableCache
|
|
// wraps around the underlying SST file readers by providing Get(),
|
|
// MultiGet() and NewIterator() methods that hide the instantiation,
|
|
// caching and access to the TableReader. The main purpose of this is
|
|
// performance - by caching the TableReader, it avoids unnecessary file opens
|
|
// and object allocation and instantiation. One exception is compaction, where
|
|
// a new TableReader may be instantiated - see NewIterator() comments
|
|
//
|
|
// Another service provided by TableCache is managing the row cache - if the
|
|
// DB is configured with a row cache, and the lookup key is present in the row
|
|
// cache, lookup is very fast. The row cache is obtained from
|
|
// ioptions.row_cache
|
|
class TableCache {
|
|
public:
|
|
TableCache(const ImmutableOptions& ioptions,
|
|
const FileOptions* storage_options, Cache* cache,
|
|
BlockCacheTracer* const block_cache_tracer,
|
|
const std::shared_ptr<IOTracer>& io_tracer,
|
|
const std::string& db_session_id);
|
|
~TableCache();
|
|
|
|
// Cache interface for table cache
|
|
using CacheInterface =
|
|
BasicTypedCacheInterface<TableReader, CacheEntryRole::kMisc>;
|
|
using TypedHandle = CacheInterface::TypedHandle;
|
|
|
|
// Cache interface for row cache
|
|
using RowCacheInterface =
|
|
BasicTypedCacheInterface<std::string, CacheEntryRole::kMisc>;
|
|
using RowHandle = RowCacheInterface::TypedHandle;
|
|
|
|
// Return an iterator for the specified file number (the corresponding
|
|
// file length must be exactly "file_size" bytes). If "table_reader_ptr"
|
|
// is non-nullptr, also sets "*table_reader_ptr" to point to the Table object
|
|
// underlying the returned iterator, or nullptr if no Table object underlies
|
|
// the returned iterator. The returned "*table_reader_ptr" object is owned
|
|
// by the cache and should not be deleted, and is valid for as long as the
|
|
// returned iterator is live.
|
|
// If !options.ignore_range_deletions, and range_del_iter is non-nullptr,
|
|
// then range_del_iter is set to a TruncatedRangeDelIterator for range
|
|
// tombstones in the SST file corresponding to the specified file number. The
|
|
// upper/lower bounds for the TruncatedRangeDelIterator are set to the SST
|
|
// file's boundary.
|
|
// @param options Must outlive the returned iterator.
|
|
// @param range_del_agg If non-nullptr, adds range deletions to the
|
|
// aggregator. If an error occurs, returns it in a NewErrorInternalIterator
|
|
// @param for_compaction If true, a new TableReader may be allocated (but
|
|
// not cached), depending on the CF options
|
|
// @param skip_filters Disables loading/accessing the filter block
|
|
// @param level The level this table is at, -1 for "not set / don't know"
|
|
// @param range_del_read_seqno If non-nullptr, will be used to create
|
|
// *range_del_iter.
|
|
InternalIterator* NewIterator(
|
|
const ReadOptions& options, const FileOptions& toptions,
|
|
const InternalKeyComparator& internal_comparator,
|
|
const FileMetaData& file_meta, RangeDelAggregator* range_del_agg,
|
|
const std::shared_ptr<const SliceTransform>& prefix_extractor,
|
|
TableReader** table_reader_ptr, HistogramImpl* file_read_hist,
|
|
TableReaderCaller caller, Arena* arena, bool skip_filters, int level,
|
|
size_t max_file_size_for_l0_meta_pin,
|
|
const InternalKey* smallest_compaction_key,
|
|
const InternalKey* largest_compaction_key, bool allow_unprepared_value,
|
|
uint8_t protection_bytes_per_key,
|
|
const SequenceNumber* range_del_read_seqno = nullptr,
|
|
std::unique_ptr<TruncatedRangeDelIterator>* range_del_iter = nullptr);
|
|
|
|
// If a seek to internal key "k" in specified file finds an entry,
|
|
// call get_context->SaveValue() repeatedly until
|
|
// it returns false. As a side effect, it will insert the TableReader
|
|
// into the cache and potentially evict another entry
|
|
// @param get_context Context for get operation. The result of the lookup
|
|
// can be retrieved by calling get_context->State()
|
|
// @param file_read_hist If non-nullptr, the file reader statistics are
|
|
// recorded
|
|
// @param skip_filters Disables loading/accessing the filter block
|
|
// @param level The level this table is at, -1 for "not set / don't know"
|
|
Status Get(
|
|
const ReadOptions& options,
|
|
const InternalKeyComparator& internal_comparator,
|
|
const FileMetaData& file_meta, const Slice& k, GetContext* get_context,
|
|
uint8_t block_protection_bytes_per_key,
|
|
const std::shared_ptr<const SliceTransform>& prefix_extractor = nullptr,
|
|
HistogramImpl* file_read_hist = nullptr, bool skip_filters = false,
|
|
int level = -1, size_t max_file_size_for_l0_meta_pin = 0);
|
|
|
|
// Return the range delete tombstone iterator of the file specified by
|
|
// `file_meta`.
|
|
Status GetRangeTombstoneIterator(
|
|
const ReadOptions& options,
|
|
const InternalKeyComparator& internal_comparator,
|
|
const FileMetaData& file_meta, uint8_t block_protection_bytes_per_key,
|
|
std::unique_ptr<FragmentedRangeTombstoneIterator>* out_iter);
|
|
|
|
// Call table reader's MultiGetFilter to use the bloom filter to filter out
|
|
// keys. Returns Status::NotSupported() if row cache needs to be checked.
|
|
// If the table cache is looked up to get the table reader, the cache handle
|
|
// is returned in table_handle. This handle should be passed back to
|
|
// MultiGet() so it can be released.
|
|
Status MultiGetFilter(
|
|
const ReadOptions& options,
|
|
const InternalKeyComparator& internal_comparator,
|
|
const FileMetaData& file_meta,
|
|
const std::shared_ptr<const SliceTransform>& prefix_extractor,
|
|
HistogramImpl* file_read_hist, int level,
|
|
MultiGetContext::Range* mget_range, TypedHandle** table_handle,
|
|
uint8_t block_protection_bytes_per_key);
|
|
|
|
// If a seek to internal key "k" in specified file finds an entry,
|
|
// call get_context->SaveValue() repeatedly until
|
|
// it returns false. As a side effect, it will insert the TableReader
|
|
// into the cache and potentially evict another entry
|
|
// @param mget_range Pointer to the structure describing a batch of keys to
|
|
// be looked up in this table file. The result is stored
|
|
// in the embedded GetContext
|
|
// @param skip_filters Disables loading/accessing the filter block
|
|
// @param level The level this table is at, -1 for "not set / don't know"
|
|
DECLARE_SYNC_AND_ASYNC(
|
|
Status, MultiGet, const ReadOptions& options,
|
|
const InternalKeyComparator& internal_comparator,
|
|
const FileMetaData& file_meta, const MultiGetContext::Range* mget_range,
|
|
uint8_t block_protection_bytes_per_key,
|
|
const std::shared_ptr<const SliceTransform>& prefix_extractor = nullptr,
|
|
HistogramImpl* file_read_hist = nullptr, bool skip_filters = false,
|
|
bool skip_range_deletions = false, int level = -1,
|
|
TypedHandle* table_handle = nullptr);
|
|
|
|
// Evict any entry for the specified file number
|
|
static void Evict(Cache* cache, uint64_t file_number);
|
|
|
|
// Handles releasing, erasing, etc. of what should be the last reference
|
|
// to an obsolete file.
|
|
static void ReleaseObsolete(Cache* cache, Cache::Handle* handle,
|
|
uint32_t uncache_aggressiveness);
|
|
|
|
// Return handle to an existing cache entry if there is one
|
|
static Cache::Handle* Lookup(Cache* cache, uint64_t file_number);
|
|
|
|
// Find table reader
|
|
// @param skip_filters Disables loading/accessing the filter block
|
|
// @param level == -1 means not specified
|
|
Status FindTable(
|
|
const ReadOptions& ro, const FileOptions& toptions,
|
|
const InternalKeyComparator& internal_comparator,
|
|
const FileMetaData& file_meta, TypedHandle**,
|
|
uint8_t block_protection_bytes_per_key,
|
|
const std::shared_ptr<const SliceTransform>& prefix_extractor = nullptr,
|
|
const bool no_io = false, HistogramImpl* file_read_hist = nullptr,
|
|
bool skip_filters = false, int level = -1,
|
|
bool prefetch_index_and_filter_in_cache = true,
|
|
size_t max_file_size_for_l0_meta_pin = 0,
|
|
Temperature file_temperature = Temperature::kUnknown);
|
|
|
|
// Get the table properties of a given table.
|
|
// @no_io: indicates if we should load table to the cache if it is not present
|
|
// in table cache yet.
|
|
// @returns: `properties` will be reset on success. Please note that we will
|
|
// return Status::Incomplete() if table is not present in cache and
|
|
// we set `no_io` to be true.
|
|
Status GetTableProperties(
|
|
const FileOptions& toptions, const ReadOptions& read_options,
|
|
const InternalKeyComparator& internal_comparator,
|
|
const FileMetaData& file_meta,
|
|
std::shared_ptr<const TableProperties>* properties,
|
|
uint8_t block_protection_bytes_per_key,
|
|
const std::shared_ptr<const SliceTransform>& prefix_extractor = nullptr,
|
|
bool no_io = false);
|
|
|
|
Status ApproximateKeyAnchors(const ReadOptions& ro,
|
|
const InternalKeyComparator& internal_comparator,
|
|
const FileMetaData& file_meta,
|
|
uint8_t block_protection_bytes_per_key,
|
|
std::vector<TableReader::Anchor>& anchors);
|
|
|
|
// Return total memory usage of the table reader of the file.
|
|
// 0 if table reader of the file is not loaded.
|
|
size_t GetMemoryUsageByTableReader(
|
|
const FileOptions& toptions, const ReadOptions& read_options,
|
|
const InternalKeyComparator& internal_comparator,
|
|
const FileMetaData& file_meta, uint8_t block_protection_bytes_per_key,
|
|
const std::shared_ptr<const SliceTransform>& prefix_extractor = nullptr);
|
|
|
|
// Returns approximated offset of a key in a file represented by fd.
|
|
uint64_t ApproximateOffsetOf(
|
|
const ReadOptions& read_options, const Slice& key,
|
|
const FileMetaData& file_meta, TableReaderCaller caller,
|
|
const InternalKeyComparator& internal_comparator,
|
|
uint8_t block_protection_bytes_per_key,
|
|
const std::shared_ptr<const SliceTransform>& prefix_extractor = nullptr);
|
|
|
|
// Returns approximated data size between start and end keys in a file
|
|
// represented by fd (the start key must not be greater than the end key).
|
|
uint64_t ApproximateSize(
|
|
const ReadOptions& read_options, const Slice& start, const Slice& end,
|
|
const FileMetaData& file_meta, TableReaderCaller caller,
|
|
const InternalKeyComparator& internal_comparator,
|
|
uint8_t block_protection_bytes_per_key,
|
|
const std::shared_ptr<const SliceTransform>& prefix_extractor = nullptr);
|
|
|
|
CacheInterface& get_cache() { return cache_; }
|
|
|
|
// Capacity of the backing Cache that indicates infinite TableCache capacity.
|
|
// For example when max_open_files is -1 we set the backing Cache to this.
|
|
static const int kInfiniteCapacity = 0x400000;
|
|
|
|
// The tables opened with this TableCache will be immortal, i.e., their
|
|
// lifetime is as long as that of the DB.
|
|
void SetTablesAreImmortal() {
|
|
if (cache_.get()->GetCapacity() >= kInfiniteCapacity) {
|
|
immortal_tables_ = true;
|
|
}
|
|
}
|
|
|
|
private:
|
|
// Build a table reader
|
|
Status GetTableReader(
|
|
const ReadOptions& ro, const FileOptions& file_options,
|
|
const InternalKeyComparator& internal_comparator,
|
|
const FileMetaData& file_meta, bool sequential_mode,
|
|
uint8_t block_protection_bytes_per_key, HistogramImpl* file_read_hist,
|
|
std::unique_ptr<TableReader>* table_reader,
|
|
const std::shared_ptr<const SliceTransform>& prefix_extractor = nullptr,
|
|
bool skip_filters = false, int level = -1,
|
|
bool prefetch_index_and_filter_in_cache = true,
|
|
size_t max_file_size_for_l0_meta_pin = 0,
|
|
Temperature file_temperature = Temperature::kUnknown);
|
|
|
|
// Update the max_covering_tombstone_seq in the GetContext for each key based
|
|
// on the range deletions in the table
|
|
void UpdateRangeTombstoneSeqnums(const ReadOptions& options, TableReader* t,
|
|
MultiGetContext::Range& table_range);
|
|
|
|
// Create a key prefix for looking up the row cache. The prefix is of the
|
|
// format row_cache_id + fd_number + seq_no. Later, the user key can be
|
|
// appended to form the full key
|
|
// Return the sequence number that determines the visibility of row_cache_key
|
|
uint64_t CreateRowCacheKeyPrefix(const ReadOptions& options,
|
|
const FileDescriptor& fd,
|
|
const Slice& internal_key,
|
|
GetContext* get_context,
|
|
IterKey& row_cache_key);
|
|
|
|
// Helper function to lookup the row cache for a key. It appends the
|
|
// user key to row_cache_key at offset prefix_size
|
|
bool GetFromRowCache(const Slice& user_key, IterKey& row_cache_key,
|
|
size_t prefix_size, GetContext* get_context,
|
|
Status* read_status,
|
|
SequenceNumber seq_no = kMaxSequenceNumber);
|
|
|
|
const ImmutableOptions& ioptions_;
|
|
const FileOptions& file_options_;
|
|
CacheInterface cache_;
|
|
std::string row_cache_id_;
|
|
bool immortal_tables_;
|
|
BlockCacheTracer* const block_cache_tracer_;
|
|
Striped<CacheAlignedWrapper<port::Mutex>> loader_mutex_;
|
|
std::shared_ptr<IOTracer> io_tracer_;
|
|
std::string db_session_id_;
|
|
};
|
|
|
|
} // namespace ROCKSDB_NAMESPACE
|