mirror of https://github.com/facebook/rocksdb.git
Fix Checkpoint hard link of inactive but unsynced WAL (#12731)
Summary: Background: there is one active WAL file but there can be several more WAL files in various states. Those other WALs are always in a "flushed" state but could be on the `logs_` list not yet fully synced. We currently allow any WAL that is not the active WAL to be hard-linked when creating a Checkpoint, as although it might still be open for write, we are not appending any more data to it. The problem is that a created Checkpoint is supposed to be fully synced on return of that function, and a hard-linked WAL in the state described above might not be fully synced. (Through some prudence in https://github.com/facebook/rocksdb/issues/10083, it would synced if using track_and_verify_wals_in_manifest=true.) The fix is a step toward a long term goal of removing the need to query the filesystem to determine WAL files and their state. (I consider it dubious any time we independently read from or query metadata from a file we have open for writing, as this makes us more susceptible to FileSystem deficiencies or races.) More specifically: * Detect which WALs might not be fully synced, according to our DBImpl metadata, and prevent hard linking those (with `trim_to_size=true` from `GetLiveFilesStorageInfo()`. And while we're at it, use our known flushed sizes for those WALs. * To avoid a race between that and GetSortedWalFiles(), track a maximum needed WAL number for the Checkpoint/GetLiveFilesStorageInfo. * Because of the level of consistency provided by those two, we no longer need to consider syncing as part of the FlushWAL in GetLiveFilesStorageInfo. (We determine the max WAL number consistent with the manifest file size, while holding DB mutex. Should make track_and_verify_wals_in_manifest happy.) This makes the premise of test PutRaceWithCheckpointTrackedWalSync obsolete (sync point callback no longer hit) so the test is removed, with crash test as backstop for related issues. See https://github.com/facebook/rocksdb/issues/10185 Stacked on https://github.com/facebook/rocksdb/issues/12729 Pull Request resolved: https://github.com/facebook/rocksdb/pull/12731 Test Plan: Expanded an existing test, which now fails before fix. Also long runs of blackbox_crash_test with amplified checkpoint frequency. Reviewed By: cbi42 Differential Revision: D58199629 Pulled By: pdillinger fbshipit-source-id: 376e55f4a2b082cd2adb6408a41209de14422382
This commit is contained in:
parent
a211e06552
commit
98393f0139
|
@ -204,6 +204,7 @@ Status DBImpl::GetLiveFilesStorageInfo(
|
|||
if (opts.wal_size_for_flush == std::numeric_limits<uint64_t>::max()) {
|
||||
flush_memtable = false;
|
||||
} else if (opts.wal_size_for_flush > 0) {
|
||||
// FIXME: avoid querying the filesystem for current WAL state
|
||||
// If the outstanding WAL files are small, we skip the flush.
|
||||
s = GetSortedWalFiles(live_wal_files);
|
||||
|
||||
|
@ -316,6 +317,8 @@ Status DBImpl::GetLiveFilesStorageInfo(
|
|||
const uint64_t options_number = versions_->options_file_number();
|
||||
const uint64_t options_size = versions_->options_file_size_;
|
||||
const uint64_t min_log_num = MinLogNumberToKeep();
|
||||
// Ensure consistency with manifest for track_and_verify_wals_in_manifest
|
||||
const uint64_t max_log_num = logfile_number_;
|
||||
|
||||
mutex_.Unlock();
|
||||
|
||||
|
@ -379,10 +382,13 @@ Status DBImpl::GetLiveFilesStorageInfo(
|
|||
TEST_SYNC_POINT("CheckpointImpl::CreateCheckpoint:SavedLiveFiles2");
|
||||
|
||||
if (s.ok()) {
|
||||
// To maximize the effectiveness of track_and_verify_wals_in_manifest,
|
||||
// sync WAL when it is enabled.
|
||||
s = FlushWAL(
|
||||
immutable_db_options_.track_and_verify_wals_in_manifest /* sync */);
|
||||
// FlushWAL is required to ensure we can physically copy everything
|
||||
// logically written to the WAL. (Sync not strictly required for
|
||||
// active WAL to be copied rather than hard linked, even when
|
||||
// Checkpoint guarantees that the copied-to file is sync-ed. Plus we can't
|
||||
// help track_and_verify_wals_in_manifest after manifest_size is
|
||||
// already determined.)
|
||||
s = FlushWAL(/*sync=*/false);
|
||||
if (s.IsNotSupported()) { // read-only DB or similar
|
||||
s = Status::OK();
|
||||
}
|
||||
|
@ -391,8 +397,37 @@ Status DBImpl::GetLiveFilesStorageInfo(
|
|||
TEST_SYNC_POINT("CheckpointImpl::CreateCustomCheckpoint:AfterGetLive1");
|
||||
TEST_SYNC_POINT("CheckpointImpl::CreateCustomCheckpoint:AfterGetLive2");
|
||||
|
||||
// If we have more than one column family, we also need to get WAL files.
|
||||
// Even after WAL flush, there could be multiple WALs that are not
|
||||
// fully synced. Although the output DB of a Checkpoint or Backup needs
|
||||
// to be fully synced on return, we don't strictly need to sync this
|
||||
// DB (the input DB). If we allow Checkpoint to hard link an inactive
|
||||
// WAL that isn't fully synced, that could result in an unsufficiently
|
||||
// sync-ed Checkpoint. Here we get the set of WALs that are potentially
|
||||
// unsynced or still being written to, to prevent them from being hard
|
||||
// linked. Enforcing max_log_num from above ensures any new WALs after
|
||||
// GetOpenWalSizes() and before GetSortedWalFiles() are not included in
|
||||
// the results.
|
||||
// NOTE: we might still hard link a file that is open for writing, even
|
||||
// if we don't do any more writes to it.
|
||||
//
|
||||
// In a step toward reducing unnecessary file metadata queries, we also
|
||||
// get and use our known flushed sizes for those WALs.
|
||||
// FIXME: eventually we should not be using filesystem queries at all for
|
||||
// the required set of WAL files.
|
||||
//
|
||||
// However for recycled log files, we just copy the whole file,
|
||||
// for better or worse.
|
||||
//
|
||||
std::map<uint64_t, uint64_t> open_wal_number_to_size;
|
||||
bool recycling_log_files = immutable_db_options_.recycle_log_file_num > 0;
|
||||
if (s.ok() && !recycling_log_files) {
|
||||
s = GetOpenWalSizes(open_wal_number_to_size);
|
||||
}
|
||||
|
||||
// [old comment] If we have more than one column family, we also need to get
|
||||
// WAL files.
|
||||
if (s.ok()) {
|
||||
// FIXME: avoid querying the filesystem for current WAL state
|
||||
s = GetSortedWalFiles(live_wal_files);
|
||||
}
|
||||
if (!s.ok()) {
|
||||
|
@ -405,7 +440,8 @@ Status DBImpl::GetLiveFilesStorageInfo(
|
|||
auto wal_dir = immutable_db_options_.GetWalDir();
|
||||
for (size_t i = 0; s.ok() && i < wal_count; ++i) {
|
||||
if ((live_wal_files[i]->Type() == kAliveLogFile) &&
|
||||
(!flush_memtable || live_wal_files[i]->LogNumber() >= min_log_num)) {
|
||||
(!flush_memtable || live_wal_files[i]->LogNumber() >= min_log_num) &&
|
||||
live_wal_files[i]->LogNumber() <= max_log_num) {
|
||||
results.emplace_back();
|
||||
LiveFileStorageInfo& info = results.back();
|
||||
auto f = live_wal_files[i]->PathName();
|
||||
|
@ -414,12 +450,29 @@ Status DBImpl::GetLiveFilesStorageInfo(
|
|||
info.directory = wal_dir;
|
||||
info.file_number = live_wal_files[i]->LogNumber();
|
||||
info.file_type = kWalFile;
|
||||
info.size = live_wal_files[i]->SizeFileBytes();
|
||||
// Trim the log either if its the last one, or log file recycling is
|
||||
// enabled. In the latter case, a hard link doesn't prevent the file
|
||||
// from being renamed and recycled. So we need to copy it instead.
|
||||
info.trim_to_size = (i + 1 == wal_count) ||
|
||||
(immutable_db_options_.recycle_log_file_num > 0);
|
||||
if (recycling_log_files) {
|
||||
info.size = live_wal_files[i]->SizeFileBytes();
|
||||
// Recyclable WAL files must be copied instead of hard linked
|
||||
info.trim_to_size = true;
|
||||
} else {
|
||||
auto it = open_wal_number_to_size.find(info.file_number);
|
||||
if (it == open_wal_number_to_size.end()) {
|
||||
// Known fully synced and no future writes (in part from
|
||||
// max_log_num check). Ok to hard link
|
||||
info.size = live_wal_files[i]->SizeFileBytes();
|
||||
assert(!info.trim_to_size);
|
||||
} else {
|
||||
// Marked as (possibly) still open -> use our known flushed size
|
||||
// and force file copy instead of hard link
|
||||
info.size = it->second;
|
||||
info.trim_to_size = true;
|
||||
// FIXME: this is needed as long as db_stress uses
|
||||
// SetReadUnsyncedData(false), because it will only be able to
|
||||
// copy the synced portion of the WAL, which under
|
||||
// SetReadUnsyncedData(false) is given by the reported file size.
|
||||
info.size = std::min(info.size, live_wal_files[i]->SizeFileBytes());
|
||||
}
|
||||
}
|
||||
if (opts.include_checksum_info) {
|
||||
info.file_checksum_func_name = kUnknownFileChecksumFuncName;
|
||||
info.file_checksum = kUnknownFileChecksum;
|
||||
|
|
|
@ -1562,6 +1562,17 @@ bool DBImpl::WALBufferIsEmpty() {
|
|||
return res;
|
||||
}
|
||||
|
||||
Status DBImpl::GetOpenWalSizes(std::map<uint64_t, uint64_t>& number_to_size) {
|
||||
InstrumentedMutexLock l(&log_write_mutex_);
|
||||
for (auto& log : logs_) {
|
||||
auto* open_file = log.writer->file();
|
||||
if (open_file) {
|
||||
number_to_size[log.number] = open_file->GetFlushedSize();
|
||||
}
|
||||
}
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status DBImpl::SyncWAL() {
|
||||
TEST_SYNC_POINT("DBImpl::SyncWAL:Begin");
|
||||
WriteOptions write_options;
|
||||
|
|
|
@ -9,6 +9,7 @@
|
|||
#pragma once
|
||||
|
||||
#include <atomic>
|
||||
#include <cstdint>
|
||||
#include <deque>
|
||||
#include <functional>
|
||||
#include <limits>
|
||||
|
@ -507,6 +508,12 @@ class DBImpl : public DB {
|
|||
Status GetLiveFiles(std::vector<std::string>&, uint64_t* manifest_file_size,
|
||||
bool flush_memtable = true) override;
|
||||
Status GetSortedWalFiles(VectorWalPtr& files) override;
|
||||
// Get the known flushed sizes of WALs that might still be written to
|
||||
// or have pending sync.
|
||||
// NOTE: unlike alive_log_files_, this function includes WALs that might
|
||||
// be obsolete (but not obsolete to a pending Checkpoint) and not yet fully
|
||||
// synced.
|
||||
Status GetOpenWalSizes(std::map<uint64_t, uint64_t>& number_to_size);
|
||||
Status GetCurrentWalFile(std::unique_ptr<WalFile>* current_log_file) override;
|
||||
Status GetCreationTimeOfOldestFile(uint64_t* creation_time) override;
|
||||
|
||||
|
@ -2541,8 +2548,10 @@ class DBImpl : public DB {
|
|||
|
||||
bool persistent_stats_cfd_exists_ = true;
|
||||
|
||||
// alive_log_files_ is protected by mutex_ and log_write_mutex_ with details
|
||||
// as follows:
|
||||
// The current WAL file and those that have not been found obsolete from
|
||||
// memtable flushes. A WAL not on this list might still be pending writer
|
||||
// flush and/or sync and close and might still be in logs_. alive_log_files_
|
||||
// is protected by mutex_ and log_write_mutex_ with details as follows:
|
||||
// 1. read by FindObsoleteFiles() which can be called in either application
|
||||
// thread or RocksDB bg threads, both mutex_ and log_write_mutex_ are
|
||||
// held.
|
||||
|
|
|
@ -312,6 +312,9 @@ void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force,
|
|||
// logs_ could have changed while we were waiting.
|
||||
continue;
|
||||
}
|
||||
// This WAL file is not live, so it's OK if we never sync the rest of it
|
||||
// or if we close it *after* removing from `logs_`. If it's already
|
||||
// closed, then it's been fully synced.
|
||||
logs_to_free_.push_back(log.ReleaseWriter());
|
||||
logs_.pop_front();
|
||||
}
|
||||
|
|
|
@ -1891,6 +1891,17 @@ Status StressTest::TestBackupRestore(
|
|||
if (!s.ok()) {
|
||||
from = "BackupEngine::Open";
|
||||
}
|
||||
// FIXME: this is only needed as long as db_stress uses
|
||||
// SetReadUnsyncedData(false), because it will only be able to
|
||||
// copy the synced portion of the WAL. For correctness validation, that
|
||||
// needs to include updates to the locked key.
|
||||
if (s.ok()) {
|
||||
s = db_->SyncWAL();
|
||||
if (!s.ok()) {
|
||||
from = "SyncWAL";
|
||||
}
|
||||
}
|
||||
|
||||
if (s.ok()) {
|
||||
if (backup_opts.schema_version >= 2 && thread->rand.OneIn(2)) {
|
||||
TEST_BackupMetaSchemaOptions test_opts;
|
||||
|
|
|
@ -59,7 +59,10 @@ IOStatus CopyFile(FileSystem* fs, const std::string& source,
|
|||
return io_s;
|
||||
}
|
||||
if (slice.size() == 0) {
|
||||
return IOStatus::Corruption("file too small");
|
||||
return IOStatus::Corruption(
|
||||
"File smaller than expected for copy: " + source + " expecting " +
|
||||
std::to_string(size) + " more bytes after " +
|
||||
std::to_string(dest_writer->GetFileSize()));
|
||||
}
|
||||
|
||||
io_s = dest_writer->Append(opts, slice);
|
||||
|
@ -226,7 +229,10 @@ IOStatus GenerateOneFileChecksum(
|
|||
io_s.ToString());
|
||||
}
|
||||
if (slice.size() == 0) {
|
||||
return IOStatus::Corruption("file too small");
|
||||
return IOStatus::Corruption(
|
||||
"File smaller than expected for checksum: " + file_path +
|
||||
" expecting " + std::to_string(size) + " more bytes after " +
|
||||
std::to_string(offset));
|
||||
}
|
||||
checksum_generator->Update(slice.data(), slice.size());
|
||||
size -= slice.size();
|
||||
|
|
|
@ -0,0 +1 @@
|
|||
* Fix a rare case in which a hard-linked WAL in a Checkpoint is not fully synced (so might lose data on power loss).
|
|
@ -760,18 +760,54 @@ TEST_F(CheckpointTest, CheckpointWithParallelWrites) {
|
|||
thread.join();
|
||||
}
|
||||
|
||||
TEST_F(CheckpointTest, CheckpointWithUnsyncedDataDropped) {
|
||||
class CheckpointTestWithWalParams
|
||||
: public CheckpointTest,
|
||||
public testing::WithParamInterface<std::tuple<uint64_t, bool, bool>> {
|
||||
public:
|
||||
uint64_t GetLogSizeForFlush() { return std::get<0>(GetParam()); }
|
||||
bool GetWalsInManifest() { return std::get<1>(GetParam()); }
|
||||
bool GetManualWalFlush() { return std::get<2>(GetParam()); }
|
||||
};
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(CheckpointTestWithWalParams,
|
||||
CheckpointTestWithWalParams,
|
||||
::testing::Combine(::testing::Values(0U, 100000000U),
|
||||
::testing::Bool(),
|
||||
::testing::Bool()));
|
||||
|
||||
TEST_P(CheckpointTestWithWalParams, CheckpointWithUnsyncedDataDropped) {
|
||||
Options options = CurrentOptions();
|
||||
std::unique_ptr<FaultInjectionTestEnv> env(new FaultInjectionTestEnv(env_));
|
||||
options.env = env.get();
|
||||
options.max_write_buffer_number = 4;
|
||||
options.track_and_verify_wals_in_manifest = GetWalsInManifest();
|
||||
options.manual_wal_flush = GetManualWalFlush();
|
||||
auto fault_fs = std::make_shared<FaultInjectionTestFS>(FileSystem::Default());
|
||||
std::unique_ptr<Env> fault_fs_env(NewCompositeEnv(fault_fs));
|
||||
|
||||
options.env = fault_fs_env.get();
|
||||
Reopen(options);
|
||||
ASSERT_OK(Put("key1", "val1"));
|
||||
if (GetLogSizeForFlush() > 0) {
|
||||
// When not flushing memtable for checkpoint, this is the simplest way
|
||||
// to get
|
||||
// * one inactive WAL, synced
|
||||
// * one inactive WAL, not synced, and
|
||||
// * one active WAL, not synced
|
||||
// with a single thread, so that we have at least one that can be hard
|
||||
// linked, etc.
|
||||
ASSERT_OK(static_cast_with_check<DBImpl>(db_)->PauseBackgroundWork());
|
||||
ASSERT_OK(static_cast_with_check<DBImpl>(db_)->TEST_SwitchMemtable());
|
||||
ASSERT_OK(db_->SyncWAL());
|
||||
}
|
||||
ASSERT_OK(Put("key2", "val2"));
|
||||
if (GetLogSizeForFlush() > 0) {
|
||||
ASSERT_OK(static_cast_with_check<DBImpl>(db_)->TEST_SwitchMemtable());
|
||||
}
|
||||
ASSERT_OK(Put("key3", "val3"));
|
||||
Checkpoint* checkpoint;
|
||||
ASSERT_OK(Checkpoint::Create(db_, &checkpoint));
|
||||
ASSERT_OK(checkpoint->CreateCheckpoint(snapshot_name_));
|
||||
ASSERT_OK(checkpoint->CreateCheckpoint(snapshot_name_, GetLogSizeForFlush()));
|
||||
delete checkpoint;
|
||||
ASSERT_OK(env->DropUnsyncedFileData());
|
||||
|
||||
ASSERT_OK(fault_fs->DropUnsyncedFileData());
|
||||
// make sure it's openable even though whatever data that wasn't synced got
|
||||
// dropped.
|
||||
options.env = env_;
|
||||
|
@ -781,6 +817,10 @@ TEST_F(CheckpointTest, CheckpointWithUnsyncedDataDropped) {
|
|||
std::string get_result;
|
||||
ASSERT_OK(snapshot_db->Get(read_opts, "key1", &get_result));
|
||||
ASSERT_EQ("val1", get_result);
|
||||
ASSERT_OK(snapshot_db->Get(read_opts, "key2", &get_result));
|
||||
ASSERT_EQ("val2", get_result);
|
||||
ASSERT_OK(snapshot_db->Get(read_opts, "key3", &get_result));
|
||||
ASSERT_EQ("val3", get_result);
|
||||
delete snapshot_db;
|
||||
delete db_;
|
||||
db_ = nullptr;
|
||||
|
@ -930,51 +970,6 @@ TEST_F(CheckpointTest, CheckpointWithDbPath) {
|
|||
delete checkpoint;
|
||||
}
|
||||
|
||||
TEST_F(CheckpointTest, PutRaceWithCheckpointTrackedWalSync) {
|
||||
// Repro for a race condition where a user write comes in after the checkpoint
|
||||
// syncs WAL for `track_and_verify_wals_in_manifest` but before the
|
||||
// corresponding MANIFEST update. With the bug, that scenario resulted in an
|
||||
// unopenable DB with error "Corruption: Size mismatch: WAL ...".
|
||||
Options options = CurrentOptions();
|
||||
std::unique_ptr<FaultInjectionTestEnv> fault_env(
|
||||
new FaultInjectionTestEnv(env_));
|
||||
options.env = fault_env.get();
|
||||
options.track_and_verify_wals_in_manifest = true;
|
||||
Reopen(options);
|
||||
|
||||
ASSERT_OK(Put("key1", "val1"));
|
||||
|
||||
SyncPoint::GetInstance()->SetCallBack(
|
||||
"DBImpl::SyncWAL:BeforeMarkLogsSynced:1",
|
||||
[this](void* /* arg */) { ASSERT_OK(Put("key2", "val2")); });
|
||||
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();
|
||||
|
||||
std::unique_ptr<Checkpoint> checkpoint;
|
||||
{
|
||||
Checkpoint* checkpoint_ptr;
|
||||
ASSERT_OK(Checkpoint::Create(db_, &checkpoint_ptr));
|
||||
checkpoint.reset(checkpoint_ptr);
|
||||
}
|
||||
|
||||
ASSERT_OK(checkpoint->CreateCheckpoint(snapshot_name_));
|
||||
|
||||
// Ensure callback ran.
|
||||
ASSERT_EQ("val2", Get("key2"));
|
||||
|
||||
Close();
|
||||
|
||||
// Simulate full loss of unsynced data. This drops "key2" -> "val2" from the
|
||||
// DB WAL.
|
||||
ASSERT_OK(fault_env->DropUnsyncedFileData());
|
||||
|
||||
// Before the bug fix, reopening the DB would fail because the MANIFEST's
|
||||
// AddWal entry indicated the WAL should be synced through "key2" -> "val2".
|
||||
Reopen(options);
|
||||
|
||||
// Need to close before `fault_env` goes out of scope.
|
||||
Close();
|
||||
}
|
||||
|
||||
} // namespace ROCKSDB_NAMESPACE
|
||||
|
||||
int main(int argc, char** argv) {
|
||||
|
|
Loading…
Reference in New Issue