| 123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613 |
- // 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.
- #ifndef ROCKSDB_LITE
- #include "rocksdb/db.h"
- #include <errno.h>
- #include <fcntl.h>
- #include <sys/stat.h>
- #include <sys/types.h>
- #include <cinttypes>
- #include "db/db_impl/db_impl.h"
- #include "db/db_test_util.h"
- #include "db/log_format.h"
- #include "db/version_set.h"
- #include "env/composite_env_wrapper.h"
- #include "file/filename.h"
- #include "rocksdb/cache.h"
- #include "rocksdb/convenience.h"
- #include "rocksdb/env.h"
- #include "rocksdb/table.h"
- #include "rocksdb/write_batch.h"
- #include "table/block_based/block_based_table_builder.h"
- #include "table/meta_blocks.h"
- #include "test_util/testharness.h"
- #include "test_util/testutil.h"
- #include "util/string_util.h"
- namespace ROCKSDB_NAMESPACE {
- static const int kValueSize = 1000;
- class CorruptionTest : public testing::Test {
- public:
- test::ErrorEnv env_;
- std::string dbname_;
- std::shared_ptr<Cache> tiny_cache_;
- Options options_;
- DB* db_;
- CorruptionTest() {
- // If LRU cache shard bit is smaller than 2 (or -1 which will automatically
- // set it to 0), test SequenceNumberRecovery will fail, likely because of a
- // bug in recovery code. Keep it 4 for now to make the test passes.
- tiny_cache_ = NewLRUCache(100, 4);
- options_.wal_recovery_mode = WALRecoveryMode::kTolerateCorruptedTailRecords;
- options_.env = &env_;
- dbname_ = test::PerThreadDBPath("corruption_test");
- DestroyDB(dbname_, options_);
- db_ = nullptr;
- options_.create_if_missing = true;
- BlockBasedTableOptions table_options;
- table_options.block_size_deviation = 0; // make unit test pass for now
- options_.table_factory.reset(NewBlockBasedTableFactory(table_options));
- Reopen();
- options_.create_if_missing = false;
- }
- ~CorruptionTest() override {
- delete db_;
- DestroyDB(dbname_, Options());
- }
- void CloseDb() {
- delete db_;
- db_ = nullptr;
- }
- Status TryReopen(Options* options = nullptr) {
- delete db_;
- db_ = nullptr;
- Options opt = (options ? *options : options_);
- if (opt.env == Options().env) {
- // If env is not overridden, replace it with ErrorEnv.
- // Otherwise, the test already uses a non-default Env.
- opt.env = &env_;
- }
- opt.arena_block_size = 4096;
- BlockBasedTableOptions table_options;
- table_options.block_cache = tiny_cache_;
- table_options.block_size_deviation = 0;
- opt.table_factory.reset(NewBlockBasedTableFactory(table_options));
- return DB::Open(opt, dbname_, &db_);
- }
- void Reopen(Options* options = nullptr) {
- ASSERT_OK(TryReopen(options));
- }
- void RepairDB() {
- delete db_;
- db_ = nullptr;
- ASSERT_OK(::ROCKSDB_NAMESPACE::RepairDB(dbname_, options_));
- }
- void Build(int n, int flush_every = 0) {
- std::string key_space, value_space;
- WriteBatch batch;
- for (int i = 0; i < n; i++) {
- if (flush_every != 0 && i != 0 && i % flush_every == 0) {
- DBImpl* dbi = reinterpret_cast<DBImpl*>(db_);
- dbi->TEST_FlushMemTable();
- }
- //if ((i % 100) == 0) fprintf(stderr, "@ %d of %d\n", i, n);
- Slice key = Key(i, &key_space);
- batch.Clear();
- batch.Put(key, Value(i, &value_space));
- ASSERT_OK(db_->Write(WriteOptions(), &batch));
- }
- }
- void Check(int min_expected, int max_expected) {
- uint64_t next_expected = 0;
- uint64_t missed = 0;
- int bad_keys = 0;
- int bad_values = 0;
- int correct = 0;
- std::string value_space;
- // Do not verify checksums. If we verify checksums then the
- // db itself will raise errors because data is corrupted.
- // Instead, we want the reads to be successful and this test
- // will detect whether the appropriate corruptions have
- // occurred.
- Iterator* iter = db_->NewIterator(ReadOptions(false, true));
- for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
- uint64_t key;
- Slice in(iter->key());
- if (!ConsumeDecimalNumber(&in, &key) ||
- !in.empty() ||
- key < next_expected) {
- bad_keys++;
- continue;
- }
- missed += (key - next_expected);
- next_expected = key + 1;
- if (iter->value() != Value(static_cast<int>(key), &value_space)) {
- bad_values++;
- } else {
- correct++;
- }
- }
- delete iter;
- fprintf(stderr,
- "expected=%d..%d; got=%d; bad_keys=%d; bad_values=%d; missed=%llu\n",
- min_expected, max_expected, correct, bad_keys, bad_values,
- static_cast<unsigned long long>(missed));
- ASSERT_LE(min_expected, correct);
- ASSERT_GE(max_expected, correct);
- }
- void CorruptFile(const std::string& fname, int offset, int bytes_to_corrupt) {
- struct stat sbuf;
- if (stat(fname.c_str(), &sbuf) != 0) {
- const char* msg = strerror(errno);
- FAIL() << fname << ": " << msg;
- }
- if (offset < 0) {
- // Relative to end of file; make it absolute
- if (-offset > sbuf.st_size) {
- offset = 0;
- } else {
- offset = static_cast<int>(sbuf.st_size + offset);
- }
- }
- if (offset > sbuf.st_size) {
- offset = static_cast<int>(sbuf.st_size);
- }
- if (offset + bytes_to_corrupt > sbuf.st_size) {
- bytes_to_corrupt = static_cast<int>(sbuf.st_size - offset);
- }
- // Do it
- std::string contents;
- Status s = ReadFileToString(Env::Default(), fname, &contents);
- ASSERT_TRUE(s.ok()) << s.ToString();
- for (int i = 0; i < bytes_to_corrupt; i++) {
- contents[i + offset] ^= 0x80;
- }
- s = WriteStringToFile(Env::Default(), contents, fname);
- ASSERT_TRUE(s.ok()) << s.ToString();
- Options options;
- EnvOptions env_options;
- options.file_system.reset(new LegacyFileSystemWrapper(options.env));
- ASSERT_NOK(VerifySstFileChecksum(options, env_options, fname));
- }
- void Corrupt(FileType filetype, int offset, int bytes_to_corrupt) {
- // Pick file to corrupt
- std::vector<std::string> filenames;
- ASSERT_OK(env_.GetChildren(dbname_, &filenames));
- uint64_t number;
- FileType type;
- std::string fname;
- int picked_number = -1;
- for (size_t i = 0; i < filenames.size(); i++) {
- if (ParseFileName(filenames[i], &number, &type) &&
- type == filetype &&
- static_cast<int>(number) > picked_number) { // Pick latest file
- fname = dbname_ + "/" + filenames[i];
- picked_number = static_cast<int>(number);
- }
- }
- ASSERT_TRUE(!fname.empty()) << filetype;
- CorruptFile(fname, offset, bytes_to_corrupt);
- }
- // corrupts exactly one file at level `level`. if no file found at level,
- // asserts
- void CorruptTableFileAtLevel(int level, int offset, int bytes_to_corrupt) {
- std::vector<LiveFileMetaData> metadata;
- db_->GetLiveFilesMetaData(&metadata);
- for (const auto& m : metadata) {
- if (m.level == level) {
- CorruptFile(dbname_ + "/" + m.name, offset, bytes_to_corrupt);
- return;
- }
- }
- FAIL() << "no file found at level";
- }
- int Property(const std::string& name) {
- std::string property;
- int result;
- if (db_->GetProperty(name, &property) &&
- sscanf(property.c_str(), "%d", &result) == 1) {
- return result;
- } else {
- return -1;
- }
- }
- // Return the ith key
- Slice Key(int i, std::string* storage) {
- char buf[100];
- snprintf(buf, sizeof(buf), "%016d", i);
- storage->assign(buf, strlen(buf));
- return Slice(*storage);
- }
- // Return the value to associate with the specified key
- Slice Value(int k, std::string* storage) {
- if (k == 0) {
- // Ugh. Random seed of 0 used to produce no entropy. This code
- // preserves the implementation that was in place when all of the
- // magic values in this file were picked.
- *storage = std::string(kValueSize, ' ');
- return Slice(*storage);
- } else {
- Random r(k);
- return test::RandomString(&r, kValueSize, storage);
- }
- }
- };
- TEST_F(CorruptionTest, Recovery) {
- Build(100);
- Check(100, 100);
- #ifdef OS_WIN
- // On Wndows OS Disk cache does not behave properly
- // We do not call FlushBuffers on every Flush. If we do not close
- // the log file prior to the corruption we end up with the first
- // block not corrupted but only the second. However, under the debugger
- // things work just fine but never pass when running normally
- // For that reason people may want to run with unbuffered I/O. That option
- // is not available for WAL though.
- CloseDb();
- #endif
- Corrupt(kLogFile, 19, 1); // WriteBatch tag for first record
- Corrupt(kLogFile, log::kBlockSize + 1000, 1); // Somewhere in second block
- ASSERT_TRUE(!TryReopen().ok());
- options_.paranoid_checks = false;
- Reopen(&options_);
- // The 64 records in the first two log blocks are completely lost.
- Check(36, 36);
- }
- TEST_F(CorruptionTest, RecoverWriteError) {
- env_.writable_file_error_ = true;
- Status s = TryReopen();
- ASSERT_TRUE(!s.ok());
- }
- TEST_F(CorruptionTest, NewFileErrorDuringWrite) {
- // Do enough writing to force minor compaction
- env_.writable_file_error_ = true;
- const int num =
- static_cast<int>(3 + (Options().write_buffer_size / kValueSize));
- std::string value_storage;
- Status s;
- bool failed = false;
- for (int i = 0; i < num; i++) {
- WriteBatch batch;
- batch.Put("a", Value(100, &value_storage));
- s = db_->Write(WriteOptions(), &batch);
- if (!s.ok()) {
- failed = true;
- }
- ASSERT_TRUE(!failed || !s.ok());
- }
- ASSERT_TRUE(!s.ok());
- ASSERT_GE(env_.num_writable_file_errors_, 1);
- env_.writable_file_error_ = false;
- Reopen();
- }
- TEST_F(CorruptionTest, TableFile) {
- Build(100);
- DBImpl* dbi = reinterpret_cast<DBImpl*>(db_);
- dbi->TEST_FlushMemTable();
- dbi->TEST_CompactRange(0, nullptr, nullptr);
- dbi->TEST_CompactRange(1, nullptr, nullptr);
- Corrupt(kTableFile, 100, 1);
- Check(99, 99);
- ASSERT_NOK(dbi->VerifyChecksum());
- }
- TEST_F(CorruptionTest, VerifyChecksumReadahead) {
- Options options;
- SpecialEnv senv(Env::Default());
- options.env = &senv;
- // Disable block cache as we are going to check checksum for
- // the same file twice and measure number of reads.
- BlockBasedTableOptions table_options_no_bc;
- table_options_no_bc.no_block_cache = true;
- options.table_factory.reset(NewBlockBasedTableFactory(table_options_no_bc));
- Reopen(&options);
- Build(10000);
- DBImpl* dbi = reinterpret_cast<DBImpl*>(db_);
- dbi->TEST_FlushMemTable();
- dbi->TEST_CompactRange(0, nullptr, nullptr);
- dbi->TEST_CompactRange(1, nullptr, nullptr);
- senv.count_random_reads_ = true;
- senv.random_read_counter_.Reset();
- ASSERT_OK(dbi->VerifyChecksum());
- // Make sure the counter is enabled.
- ASSERT_GT(senv.random_read_counter_.Read(), 0);
- // The SST file is about 10MB. Default readahead size is 256KB.
- // Give a conservative 20 reads for metadata blocks, The number
- // of random reads should be within 10 MB / 256KB + 20 = 60.
- ASSERT_LT(senv.random_read_counter_.Read(), 60);
- senv.random_read_bytes_counter_ = 0;
- ReadOptions ro;
- ro.readahead_size = size_t{32 * 1024};
- ASSERT_OK(dbi->VerifyChecksum(ro));
- // The SST file is about 10MB. We set readahead size to 32KB.
- // Give 0 to 20 reads for metadata blocks, and allow real read
- // to range from 24KB to 48KB. The lower bound would be:
- // 10MB / 48KB + 0 = 213
- // The higher bound is
- // 10MB / 24KB + 20 = 447.
- ASSERT_GE(senv.random_read_counter_.Read(), 213);
- ASSERT_LE(senv.random_read_counter_.Read(), 447);
- // Test readahead shouldn't break mmap mode (where it should be
- // disabled).
- options.allow_mmap_reads = true;
- Reopen(&options);
- dbi = static_cast<DBImpl*>(db_);
- ASSERT_OK(dbi->VerifyChecksum(ro));
- CloseDb();
- }
- TEST_F(CorruptionTest, TableFileIndexData) {
- Options options;
- // very big, we'll trigger flushes manually
- options.write_buffer_size = 100 * 1024 * 1024;
- Reopen(&options);
- // build 2 tables, flush at 5000
- Build(10000, 5000);
- DBImpl* dbi = reinterpret_cast<DBImpl*>(db_);
- dbi->TEST_FlushMemTable();
- // corrupt an index block of an entire file
- Corrupt(kTableFile, -2000, 500);
- options.paranoid_checks = false;
- Reopen(&options);
- dbi = reinterpret_cast<DBImpl*>(db_);
- // one full file may be readable, since only one was corrupted
- // the other file should be fully non-readable, since index was corrupted
- Check(0, 5000);
- ASSERT_NOK(dbi->VerifyChecksum());
- // In paranoid mode, the db cannot be opened due to the corrupted file.
- ASSERT_TRUE(TryReopen().IsCorruption());
- }
- TEST_F(CorruptionTest, MissingDescriptor) {
- Build(1000);
- RepairDB();
- Reopen();
- Check(1000, 1000);
- }
- TEST_F(CorruptionTest, SequenceNumberRecovery) {
- ASSERT_OK(db_->Put(WriteOptions(), "foo", "v1"));
- ASSERT_OK(db_->Put(WriteOptions(), "foo", "v2"));
- ASSERT_OK(db_->Put(WriteOptions(), "foo", "v3"));
- ASSERT_OK(db_->Put(WriteOptions(), "foo", "v4"));
- ASSERT_OK(db_->Put(WriteOptions(), "foo", "v5"));
- RepairDB();
- Reopen();
- std::string v;
- ASSERT_OK(db_->Get(ReadOptions(), "foo", &v));
- ASSERT_EQ("v5", v);
- // Write something. If sequence number was not recovered properly,
- // it will be hidden by an earlier write.
- ASSERT_OK(db_->Put(WriteOptions(), "foo", "v6"));
- ASSERT_OK(db_->Get(ReadOptions(), "foo", &v));
- ASSERT_EQ("v6", v);
- Reopen();
- ASSERT_OK(db_->Get(ReadOptions(), "foo", &v));
- ASSERT_EQ("v6", v);
- }
- TEST_F(CorruptionTest, CorruptedDescriptor) {
- ASSERT_OK(db_->Put(WriteOptions(), "foo", "hello"));
- DBImpl* dbi = reinterpret_cast<DBImpl*>(db_);
- dbi->TEST_FlushMemTable();
- dbi->TEST_CompactRange(0, nullptr, nullptr);
- Corrupt(kDescriptorFile, 0, 1000);
- Status s = TryReopen();
- ASSERT_TRUE(!s.ok());
- RepairDB();
- Reopen();
- std::string v;
- ASSERT_OK(db_->Get(ReadOptions(), "foo", &v));
- ASSERT_EQ("hello", v);
- }
- TEST_F(CorruptionTest, CompactionInputError) {
- Options options;
- Reopen(&options);
- Build(10);
- DBImpl* dbi = reinterpret_cast<DBImpl*>(db_);
- dbi->TEST_FlushMemTable();
- dbi->TEST_CompactRange(0, nullptr, nullptr);
- dbi->TEST_CompactRange(1, nullptr, nullptr);
- ASSERT_EQ(1, Property("rocksdb.num-files-at-level2"));
- Corrupt(kTableFile, 100, 1);
- Check(9, 9);
- ASSERT_NOK(dbi->VerifyChecksum());
- // Force compactions by writing lots of values
- Build(10000);
- Check(10000, 10000);
- ASSERT_NOK(dbi->VerifyChecksum());
- }
- TEST_F(CorruptionTest, CompactionInputErrorParanoid) {
- Options options;
- options.paranoid_checks = true;
- options.write_buffer_size = 131072;
- options.max_write_buffer_number = 2;
- Reopen(&options);
- DBImpl* dbi = reinterpret_cast<DBImpl*>(db_);
- // Fill levels >= 1
- for (int level = 1; level < dbi->NumberLevels(); level++) {
- dbi->Put(WriteOptions(), "", "begin");
- dbi->Put(WriteOptions(), "~", "end");
- dbi->TEST_FlushMemTable();
- for (int comp_level = 0; comp_level < dbi->NumberLevels() - level;
- ++comp_level) {
- dbi->TEST_CompactRange(comp_level, nullptr, nullptr);
- }
- }
- Reopen(&options);
- dbi = reinterpret_cast<DBImpl*>(db_);
- Build(10);
- dbi->TEST_FlushMemTable();
- dbi->TEST_WaitForCompact();
- ASSERT_EQ(1, Property("rocksdb.num-files-at-level0"));
- CorruptTableFileAtLevel(0, 100, 1);
- Check(9, 9);
- ASSERT_NOK(dbi->VerifyChecksum());
- // Write must eventually fail because of corrupted table
- Status s;
- std::string tmp1, tmp2;
- bool failed = false;
- for (int i = 0; i < 10000; i++) {
- s = db_->Put(WriteOptions(), Key(i, &tmp1), Value(i, &tmp2));
- if (!s.ok()) {
- failed = true;
- }
- // if one write failed, every subsequent write must fail, too
- ASSERT_TRUE(!failed || !s.ok()) << "write did not fail in a corrupted db";
- }
- ASSERT_TRUE(!s.ok()) << "write did not fail in corrupted paranoid db";
- }
- TEST_F(CorruptionTest, UnrelatedKeys) {
- Build(10);
- DBImpl* dbi = reinterpret_cast<DBImpl*>(db_);
- dbi->TEST_FlushMemTable();
- Corrupt(kTableFile, 100, 1);
- ASSERT_NOK(dbi->VerifyChecksum());
- std::string tmp1, tmp2;
- ASSERT_OK(db_->Put(WriteOptions(), Key(1000, &tmp1), Value(1000, &tmp2)));
- std::string v;
- ASSERT_OK(db_->Get(ReadOptions(), Key(1000, &tmp1), &v));
- ASSERT_EQ(Value(1000, &tmp2).ToString(), v);
- dbi->TEST_FlushMemTable();
- ASSERT_OK(db_->Get(ReadOptions(), Key(1000, &tmp1), &v));
- ASSERT_EQ(Value(1000, &tmp2).ToString(), v);
- }
- TEST_F(CorruptionTest, RangeDeletionCorrupted) {
- ASSERT_OK(
- db_->DeleteRange(WriteOptions(), db_->DefaultColumnFamily(), "a", "b"));
- ASSERT_OK(db_->Flush(FlushOptions()));
- std::vector<LiveFileMetaData> metadata;
- db_->GetLiveFilesMetaData(&metadata);
- ASSERT_EQ(static_cast<size_t>(1), metadata.size());
- std::string filename = dbname_ + metadata[0].name;
- std::unique_ptr<RandomAccessFile> file;
- ASSERT_OK(options_.env->NewRandomAccessFile(filename, &file, EnvOptions()));
- std::unique_ptr<RandomAccessFileReader> file_reader(
- new RandomAccessFileReader(NewLegacyRandomAccessFileWrapper(file),
- filename));
- uint64_t file_size;
- ASSERT_OK(options_.env->GetFileSize(filename, &file_size));
- BlockHandle range_del_handle;
- ASSERT_OK(FindMetaBlock(
- file_reader.get(), file_size, kBlockBasedTableMagicNumber,
- ImmutableCFOptions(options_), kRangeDelBlock, &range_del_handle));
- ASSERT_OK(TryReopen());
- CorruptFile(filename, static_cast<int>(range_del_handle.offset()), 1);
- ASSERT_TRUE(TryReopen().IsCorruption());
- }
- TEST_F(CorruptionTest, FileSystemStateCorrupted) {
- for (int iter = 0; iter < 2; ++iter) {
- Options options;
- options.paranoid_checks = true;
- options.create_if_missing = true;
- Reopen(&options);
- Build(10);
- ASSERT_OK(db_->Flush(FlushOptions()));
- DBImpl* dbi = reinterpret_cast<DBImpl*>(db_);
- std::vector<LiveFileMetaData> metadata;
- dbi->GetLiveFilesMetaData(&metadata);
- ASSERT_GT(metadata.size(), size_t(0));
- std::string filename = dbname_ + metadata[0].name;
- delete db_;
- db_ = nullptr;
- if (iter == 0) { // corrupt file size
- std::unique_ptr<WritableFile> file;
- env_.NewWritableFile(filename, &file, EnvOptions());
- file->Append(Slice("corrupted sst"));
- file.reset();
- Status x = TryReopen(&options);
- ASSERT_TRUE(x.IsCorruption());
- } else { // delete the file
- env_.DeleteFile(filename);
- Status x = TryReopen(&options);
- ASSERT_TRUE(x.IsPathNotFound());
- }
- DestroyDB(dbname_, options_);
- }
- }
- } // namespace ROCKSDB_NAMESPACE
- int main(int argc, char** argv) {
- ::testing::InitGoogleTest(&argc, argv);
- return RUN_ALL_TESTS();
- }
- #else
- #include <stdio.h>
- int main(int /*argc*/, char** /*argv*/) {
- fprintf(stderr, "SKIPPED as RepairDB() is not supported in ROCKSDB_LITE\n");
- return 0;
- }
- #endif // !ROCKSDB_LITE
|