mirror of
https://github.com/facebook/rocksdb.git
synced 2024-11-26 07:30:54 +00:00
bf98dcf9a8
Summary: The original goal is to propagate failures from `GetContext::SaveValue()` -> `GetContext::GetBlobValue()` -> `BlobFetcher::FetchBlob()` up to the user. This call sequence happens when a merge chain ends with a base value in a blob file. There's also fixes for bugs encountered along the way where non-ok statuses were ignored/overwritten, and a bit of plumbing work for functions that had no capability to return a status. Pull Request resolved: https://github.com/facebook/rocksdb/pull/12462 Test Plan: A repro command ``` db=/dev/shm/dbstress_db ; exp=/dev/shm/dbstress_exp ; rm -rf $db $exp ; mkdir -p $db $exp ./db_stress \ --clear_column_family_one_in=0 \ --test_batches_snapshots=0 \ --write_fault_one_in=0 \ --use_put_entity_one_in=0 \ --prefixpercent=0 \ --read_fault_one_in=0 \ --readpercent=0 \ --reopen=0 \ --set_options_one_in=10000 \ --delpercent=0 \ --delrangepercent=0 \ --open_metadata_write_fault_one_in=0 \ --open_read_fault_one_in=0 \ --open_write_fault_one_in=0 \ --destroy_db_initially=0 \ --ingest_external_file_one_in=0 \ --iterpercent=0 \ --nooverwritepercent=0 \ --db=$db \ --enable_blob_files=1 \ --expected_values_dir=$exp \ --max_background_compactions=20 \ --max_bytes_for_level_base=2097152 \ --max_key=100000 \ --min_blob_size=0 \ --open_files=-1 \ --ops_per_thread=100000000 \ --prefix_size=-1 \ --target_file_size_base=524288 \ --use_merge=1 \ --value_size_mult=32 \ --write_buffer_size=524288 \ --writepercent=100 ``` It used to fail like: ``` ... frame https://github.com/facebook/rocksdb/issues/9: 0x00007fc63903bc93 libc.so.6`__GI___assert_fail(assertion="HasDefaultColumn(columns)", file="fbcode/internal_repo_rocksdb/repo/db/wide/wide_columns_helper.h", line=33, function="static const rocksdb::Slice &rocksdb::WideColumnsHelper::GetDefaultColumn(const rocksdb::WideColumns &)") at assert.c:101:3 frame https://github.com/facebook/rocksdb/issues/10: 0x00000000006f7e92 db_stress`rocksdb::Version::Get(rocksdb::ReadOptions const&, rocksdb::LookupKey const&, rocksdb::PinnableSlice*, rocksdb::PinnableWideColumns*, std::__cxx11::basic_string<char, std::char_traits<char>, std::allocator<char>>*, rocksdb::Status*, rocksdb::MergeContext*, unsigned long*, rocksdb::PinnedIteratorsManager*, bool*, bool*, unsigned long*, rocksdb::ReadCallback*, bool*, bool) [inlined] rocksdb::WideColumnsHelper::GetDefaultColumn(columns=size=0) at wide_columns_helper.h:33 frame https://github.com/facebook/rocksdb/issues/11: 0x00000000006f7e76 db_stress`rocksdb::Version::Get(this=0x00007fc5ec763000, read_options=<unavailable>, k=<unavailable>, value=0x0000000000000000, columns=0x00007fc6035fd1d8, timestamp=<unavailable>, status=0x00007fc6035fd250, merge_context=0x00007fc6035fce40, max_covering_tombstone_seq=0x00007fc6035fce90, pinned_iters_mgr=0x00007fc6035fcdf0, value_found=0x0000000000000000, key_exists=0x0000000000000000, seq=0x0000000000000000, callback=0x0000000000000000, is_blob=0x0000000000000000, do_merge=<unavailable>) at version_set.cc:2492 frame https://github.com/facebook/rocksdb/issues/12: 0x000000000051e245 db_stress`rocksdb::DBImpl::GetImpl(this=0x00007fc637a86000, read_options=0x00007fc6035fcf60, key=<unavailable>, get_impl_options=0x00007fc6035fd000) at db_impl.cc:2408 frame https://github.com/facebook/rocksdb/issues/13: 0x000000000050cec2 db_stress`rocksdb::DBImpl::GetEntity(this=0x00007fc637a86000, _read_options=<unavailable>, column_family=<unavailable>, key=0x00007fc6035fd3c8, columns=0x00007fc6035fd1d8) at db_impl.cc:2109 frame https://github.com/facebook/rocksdb/issues/14: 0x000000000074f688 db_stress`rocksdb::(anonymous namespace)::MemTableInserter::MergeCF(this=0x00007fc6035fd450, column_family_id=2, key=0x00007fc6035fd3c8, value=0x00007fc6035fd3a0) at write_batch.cc:2656 frame https://github.com/facebook/rocksdb/issues/15: 0x00000000007476fc db_stress`rocksdb::WriteBatchInternal::Iterate(wb=0x00007fc6035fe698, handler=0x00007fc6035fd450, begin=12, end=<unavailable>) at write_batch.cc:607 frame https://github.com/facebook/rocksdb/issues/16: 0x000000000074d7dd db_stress`rocksdb::WriteBatchInternal::InsertInto(rocksdb::WriteThread::WriteGroup&, unsigned long, rocksdb::ColumnFamilyMemTables*, rocksdb::FlushScheduler*, rocksdb::TrimHistoryScheduler*, bool, unsigned long, rocksdb::DB*, bool, bool, bool) [inlined] rocksdb::WriteBatch::Iterate(this=<unavailable>, handler=0x00007fc6035fd450) const at write_batch.cc:505 frame https://github.com/facebook/rocksdb/issues/17: 0x000000000074d77b db_stress`rocksdb::WriteBatchInternal::InsertInto(write_group=<unavailable>, sequence=<unavailable>, memtables=<unavailable>, flush_scheduler=<unavailable>, trim_history_scheduler=<unavailable>, ignore_missing_column_families=<unavailable>, recovery_log_number=0, db=0x00007fc637a86000, concurrent_memtable_writes=<unavailable>, seq_per_batch=false, batch_per_txn=<unavailable>) at write_batch.cc:3084 frame https://github.com/facebook/rocksdb/issues/18: 0x0000000000631d77 db_stress`rocksdb::DBImpl::PipelinedWriteImpl(this=0x00007fc637a86000, write_options=<unavailable>, my_batch=0x00007fc6035fe698, callback=0x0000000000000000, log_used=<unavailable>, log_ref=0, disable_memtable=<unavailable>, seq_used=0x0000000000000000) at db_impl_write.cc:807 frame https://github.com/facebook/rocksdb/issues/19: 0x000000000062ceeb db_stress`rocksdb::DBImpl::WriteImpl(this=<unavailable>, write_options=<unavailable>, my_batch=0x00007fc6035fe698, callback=0x0000000000000000, log_used=<unavailable>, log_ref=0, disable_memtable=<unavailable>, seq_used=0x0000000000000000, batch_cnt=0, pre_release_callback=0x0000000000000000, post_memtable_callback=0x0000000000000000) at db_impl_write.cc:312 frame https://github.com/facebook/rocksdb/issues/20: 0x000000000062c8ec db_stress`rocksdb::DBImpl::Write(this=0x00007fc637a86000, write_options=0x00007fc6035feca8, my_batch=0x00007fc6035fe698) at db_impl_write.cc:157 frame https://github.com/facebook/rocksdb/issues/21: 0x000000000062b847 db_stress`rocksdb::DB::Merge(this=0x00007fc637a86000, opt=0x00007fc6035feca8, column_family=0x00007fc6370bf140, key=0x00007fc6035fe8d8, value=0x00007fc6035fe830) at db_impl_write.cc:2544 frame https://github.com/facebook/rocksdb/issues/22: 0x000000000062b6ef db_stress`rocksdb::DBImpl::Merge(this=0x00007fc637a86000, o=<unavailable>, column_family=0x00007fc6370bf140, key=0x00007fc6035fe8d8, val=0x00007fc6035fe830) at db_impl_write.cc:72 frame https://github.com/facebook/rocksdb/issues/23: 0x00000000004d6397 db_stress`rocksdb::NonBatchedOpsStressTest::TestPut(this=0x00007fc637041000, thread=0x00007fc6370dbc00, write_opts=0x00007fc6035feca8, read_opts=0x00007fc6035fe9c8, rand_column_families=<unavailable>, rand_keys=size=1, value={P\xe9_\x03\xc6\x7f\0\0}) at no_batched_ops_stress.cc:1317 frame https://github.com/facebook/rocksdb/issues/24: 0x000000000049361d db_stress`rocksdb::StressTest::OperateDb(this=0x00007fc637041000, thread=0x00007fc6370dbc00) at db_stress_test_base.cc:1148 ... ``` Reviewed By: ltamasi Differential Revision: D55157795 Pulled By: ajkr fbshipit-source-id: 5f7c1380ead5794c29d41680028e34b839744764
356 lines
12 KiB
C++
356 lines
12 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).
|
|
|
|
#include "table/mock_table.h"
|
|
|
|
#include "db/dbformat.h"
|
|
#include "env/composite_env_wrapper.h"
|
|
#include "file/random_access_file_reader.h"
|
|
#include "port/port.h"
|
|
#include "rocksdb/table_properties.h"
|
|
#include "table/get_context.h"
|
|
#include "util/coding.h"
|
|
|
|
namespace ROCKSDB_NAMESPACE::mock {
|
|
|
|
KVVector MakeMockFile(std::initializer_list<KVPair> l) { return KVVector(l); }
|
|
|
|
void SortKVVector(KVVector* kv_vector, const Comparator* ucmp) {
|
|
InternalKeyComparator icmp(ucmp);
|
|
std::sort(kv_vector->begin(), kv_vector->end(),
|
|
[icmp](KVPair a, KVPair b) -> bool {
|
|
return icmp.Compare(a.first, b.first) < 0;
|
|
});
|
|
}
|
|
|
|
class MockTableReader : public TableReader {
|
|
public:
|
|
explicit MockTableReader(const KVVector& table) : table_(table) {}
|
|
|
|
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;
|
|
|
|
Status Get(const ReadOptions& readOptions, const Slice& key,
|
|
GetContext* get_context, const SliceTransform* prefix_extractor,
|
|
bool skip_filters = false) override;
|
|
|
|
uint64_t ApproximateOffsetOf(const ReadOptions& /*read_options*/,
|
|
const Slice& /*key*/,
|
|
TableReaderCaller /*caller*/) override {
|
|
return 0;
|
|
}
|
|
|
|
uint64_t ApproximateSize(const ReadOptions& /*read_options*/,
|
|
const Slice& /*start*/, const Slice& /*end*/,
|
|
TableReaderCaller /*caller*/) override {
|
|
return 0;
|
|
}
|
|
|
|
size_t ApproximateMemoryUsage() const override { return 0; }
|
|
|
|
void SetupForCompaction() override {}
|
|
|
|
std::shared_ptr<const TableProperties> GetTableProperties() const override;
|
|
|
|
~MockTableReader() = default;
|
|
|
|
private:
|
|
const KVVector& table_;
|
|
};
|
|
|
|
class MockTableIterator : public InternalIterator {
|
|
public:
|
|
explicit MockTableIterator(const KVVector& table) : table_(table) {
|
|
itr_ = table_.end();
|
|
}
|
|
|
|
bool Valid() const override { return itr_ != table_.end(); }
|
|
|
|
void SeekToFirst() override { itr_ = table_.begin(); }
|
|
|
|
void SeekToLast() override {
|
|
itr_ = table_.end();
|
|
--itr_;
|
|
}
|
|
|
|
void Seek(const Slice& target) override {
|
|
KVPair target_pair(target.ToString(), "");
|
|
InternalKeyComparator icmp(BytewiseComparator());
|
|
itr_ = std::lower_bound(table_.begin(), table_.end(), target_pair,
|
|
[icmp](KVPair a, KVPair b) -> bool {
|
|
return icmp.Compare(a.first, b.first) < 0;
|
|
});
|
|
}
|
|
|
|
void SeekForPrev(const Slice& target) override {
|
|
KVPair target_pair(target.ToString(), "");
|
|
InternalKeyComparator icmp(BytewiseComparator());
|
|
itr_ = std::upper_bound(table_.begin(), table_.end(), target_pair,
|
|
[icmp](KVPair a, KVPair b) -> bool {
|
|
return icmp.Compare(a.first, b.first) < 0;
|
|
});
|
|
Prev();
|
|
}
|
|
|
|
void Next() override { ++itr_; }
|
|
|
|
void Prev() override {
|
|
if (itr_ == table_.begin()) {
|
|
itr_ = table_.end();
|
|
} else {
|
|
--itr_;
|
|
}
|
|
}
|
|
|
|
Slice key() const override { return Slice(itr_->first); }
|
|
|
|
Slice value() const override { return Slice(itr_->second); }
|
|
|
|
Status status() const override { return Status::OK(); }
|
|
|
|
private:
|
|
const KVVector& table_;
|
|
KVVector::const_iterator itr_;
|
|
};
|
|
|
|
class MockTableBuilder : public TableBuilder {
|
|
public:
|
|
MockTableBuilder(uint32_t id, MockTableFileSystem* file_system,
|
|
MockTableFactory::MockCorruptionMode corrupt_mode =
|
|
MockTableFactory::kCorruptNone,
|
|
size_t key_value_size = 1)
|
|
: id_(id),
|
|
file_system_(file_system),
|
|
corrupt_mode_(corrupt_mode),
|
|
key_value_size_(key_value_size) {
|
|
table_ = MakeMockFile({});
|
|
}
|
|
|
|
// REQUIRES: Either Finish() or Abandon() has been called.
|
|
~MockTableBuilder() = default;
|
|
|
|
// 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 {
|
|
if (corrupt_mode_ == MockTableFactory::kCorruptValue) {
|
|
// Corrupt the value
|
|
table_.push_back({key.ToString(), value.ToString() + " "});
|
|
corrupt_mode_ = MockTableFactory::kCorruptNone;
|
|
} else if (corrupt_mode_ == MockTableFactory::kCorruptKey) {
|
|
table_.push_back({key.ToString() + " ", value.ToString()});
|
|
corrupt_mode_ = MockTableFactory::kCorruptNone;
|
|
} else if (corrupt_mode_ == MockTableFactory::kCorruptReorderKey) {
|
|
if (prev_key_.empty()) {
|
|
prev_key_ = key.ToString();
|
|
prev_value_ = value.ToString();
|
|
} else {
|
|
table_.push_back({key.ToString(), value.ToString()});
|
|
table_.push_back({prev_key_, prev_value_});
|
|
corrupt_mode_ = MockTableFactory::kCorruptNone;
|
|
}
|
|
} else {
|
|
table_.push_back({key.ToString(), value.ToString()});
|
|
}
|
|
}
|
|
|
|
// Return non-ok iff some error has been detected.
|
|
Status status() const override { return Status::OK(); }
|
|
|
|
// Return non-ok iff some error happens during IO.
|
|
IOStatus io_status() const override { return IOStatus::OK(); }
|
|
|
|
Status Finish() override {
|
|
MutexLock lock_guard(&file_system_->mutex);
|
|
file_system_->files.insert({id_, table_});
|
|
return Status::OK();
|
|
}
|
|
|
|
void Abandon() override {}
|
|
|
|
uint64_t NumEntries() const override { return table_.size(); }
|
|
|
|
uint64_t FileSize() const override { return table_.size() * key_value_size_; }
|
|
|
|
TableProperties GetTableProperties() const override {
|
|
return TableProperties();
|
|
}
|
|
|
|
// Get file checksum
|
|
std::string GetFileChecksum() const override { return kUnknownFileChecksum; }
|
|
// Get file checksum function name
|
|
const char* GetFileChecksumFuncName() const override {
|
|
return kUnknownFileChecksumFuncName;
|
|
}
|
|
|
|
private:
|
|
uint32_t id_;
|
|
std::string prev_key_;
|
|
std::string prev_value_;
|
|
MockTableFileSystem* file_system_;
|
|
int corrupt_mode_;
|
|
KVVector table_;
|
|
size_t key_value_size_;
|
|
};
|
|
|
|
InternalIterator* MockTableReader::NewIterator(
|
|
const ReadOptions&, const SliceTransform* /* prefix_extractor */,
|
|
Arena* /*arena*/, bool /*skip_filters*/, TableReaderCaller /*caller*/,
|
|
size_t /*compaction_readahead_size*/, bool /* allow_unprepared_value */) {
|
|
return new MockTableIterator(table_);
|
|
}
|
|
|
|
Status MockTableReader::Get(const ReadOptions&, const Slice& key,
|
|
GetContext* get_context,
|
|
const SliceTransform* /*prefix_extractor*/,
|
|
bool /*skip_filters*/) {
|
|
std::unique_ptr<MockTableIterator> iter(new MockTableIterator(table_));
|
|
for (iter->Seek(key); iter->Valid(); iter->Next()) {
|
|
ParsedInternalKey parsed_key;
|
|
Status pik_status =
|
|
ParseInternalKey(iter->key(), &parsed_key, true /* log_err_key */);
|
|
if (!pik_status.ok()) {
|
|
return pik_status;
|
|
}
|
|
|
|
bool dont_care __attribute__((__unused__));
|
|
Status read_status;
|
|
bool ret = get_context->SaveValue(parsed_key, iter->value(), &dont_care,
|
|
&read_status);
|
|
if (!read_status.ok()) {
|
|
return read_status;
|
|
}
|
|
if (!ret) {
|
|
break;
|
|
}
|
|
}
|
|
return Status::OK();
|
|
}
|
|
|
|
std::shared_ptr<const TableProperties> MockTableReader::GetTableProperties()
|
|
const {
|
|
TableProperties* tp = new TableProperties();
|
|
tp->num_entries = table_.size();
|
|
tp->num_range_deletions = 0;
|
|
tp->raw_key_size = 1;
|
|
tp->raw_value_size = 1;
|
|
|
|
return std::shared_ptr<const TableProperties>(tp);
|
|
}
|
|
|
|
MockTableFactory::MockTableFactory()
|
|
: next_id_(1), corrupt_mode_(MockTableFactory::kCorruptNone) {}
|
|
|
|
Status MockTableFactory::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 {
|
|
uint32_t id;
|
|
Status s = GetIDFromFile(file.get(), &id);
|
|
if (!s.ok()) {
|
|
return s;
|
|
}
|
|
|
|
MutexLock lock_guard(&file_system_.mutex);
|
|
|
|
auto it = file_system_.files.find(id);
|
|
if (it == file_system_.files.end()) {
|
|
return Status::IOError("Mock file not found");
|
|
}
|
|
|
|
table_reader->reset(new MockTableReader(it->second));
|
|
|
|
return Status::OK();
|
|
}
|
|
|
|
TableBuilder* MockTableFactory::NewTableBuilder(
|
|
const TableBuilderOptions& /*table_builder_options*/,
|
|
WritableFileWriter* file) const {
|
|
uint32_t id;
|
|
Status s = GetAndWriteNextID(file, &id);
|
|
assert(s.ok());
|
|
|
|
return new MockTableBuilder(id, &file_system_, corrupt_mode_,
|
|
key_value_size_);
|
|
}
|
|
|
|
Status MockTableFactory::CreateMockTable(Env* env, const std::string& fname,
|
|
KVVector file_contents) {
|
|
std::unique_ptr<WritableFileWriter> file_writer;
|
|
Status s = WritableFileWriter::Create(env->GetFileSystem(), fname,
|
|
FileOptions(), &file_writer, nullptr);
|
|
if (!s.ok()) {
|
|
return s;
|
|
}
|
|
uint32_t id;
|
|
s = GetAndWriteNextID(file_writer.get(), &id);
|
|
if (s.ok()) {
|
|
file_system_.files.insert({id, std::move(file_contents)});
|
|
}
|
|
return s;
|
|
}
|
|
|
|
Status MockTableFactory::GetAndWriteNextID(WritableFileWriter* file,
|
|
uint32_t* next_id) const {
|
|
*next_id = next_id_.fetch_add(1);
|
|
char buf[4];
|
|
EncodeFixed32(buf, *next_id);
|
|
return file->Append(IOOptions(), Slice(buf, 4));
|
|
}
|
|
|
|
Status MockTableFactory::GetIDFromFile(RandomAccessFileReader* file,
|
|
uint32_t* id) const {
|
|
char buf[4];
|
|
Slice result;
|
|
Status s = file->Read(IOOptions(), 0, 4, &result, buf, nullptr);
|
|
assert(result.size() == 4);
|
|
*id = DecodeFixed32(buf);
|
|
return s;
|
|
}
|
|
|
|
void MockTableFactory::AssertSingleFile(const KVVector& file_contents) {
|
|
ASSERT_EQ(file_system_.files.size(), 1U);
|
|
ASSERT_EQ(file_contents, file_system_.files.begin()->second);
|
|
}
|
|
|
|
void MockTableFactory::AssertLatestFiles(
|
|
const std::vector<KVVector>& files_contents) {
|
|
ASSERT_GE(file_system_.files.size(), files_contents.size());
|
|
auto it = file_system_.files.rbegin();
|
|
for (auto expect = files_contents.rbegin(); expect != files_contents.rend();
|
|
expect++, it++) {
|
|
ASSERT_TRUE(it != file_system_.files.rend());
|
|
if (*expect != it->second) {
|
|
std::cout << "Wrong content! Content of file, expect:" << std::endl;
|
|
for (const auto& kv : *expect) {
|
|
ParsedInternalKey ikey;
|
|
std::string key, value;
|
|
std::tie(key, value) = kv;
|
|
ASSERT_OK(ParseInternalKey(Slice(key), &ikey, true /* log_err_key */));
|
|
std::cout << ikey.DebugString(true, false) << " -> " << value
|
|
<< std::endl;
|
|
}
|
|
std::cout << "actual:" << std::endl;
|
|
for (const auto& kv : it->second) {
|
|
ParsedInternalKey ikey;
|
|
std::string key, value;
|
|
std::tie(key, value) = kv;
|
|
ASSERT_OK(ParseInternalKey(Slice(key), &ikey, true /* log_err_key */));
|
|
std::cout << ikey.DebugString(true, false) << " -> " << value
|
|
<< std::endl;
|
|
}
|
|
FAIL();
|
|
}
|
|
}
|
|
}
|
|
|
|
} // namespace ROCKSDB_NAMESPACE::mock
|