Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Ensure writes to WAL tail during FlushWAL(true /* sync */) will be … #357

Merged
merged 1 commit into from
Mar 26, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 2 additions & 0 deletions HISTORY.md
Original file line number Diff line number Diff line change
@@ -1,6 +1,8 @@
# Rocksdb Change Log

## Unreleased
### Bug Fixes
* Fixed bug where `FlushWAL(true /* sync */)` (used by `GetLiveFilesStorageInfo()`, which is used by checkpoint and backup) could cause parallel writes at the tail of a WAL file to never be synced.

### Bug Fixes

Expand Down
18 changes: 13 additions & 5 deletions db/db_impl/db_impl.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1488,20 +1488,28 @@ void DBImpl::MarkLogsSynced(uint64_t up_to, bool synced_dir,
auto& wal = *it;
assert(wal.IsSyncing());

if (logs_.size() > 1) {
if (wal.number < logs_.back().number) {
// Inactive WAL
if (immutable_db_options_.track_and_verify_wals_in_manifest &&
wal.GetPreSyncSize() > 0) {
synced_wals->AddWal(wal.number, WalMetadata(wal.GetPreSyncSize()));
}
logs_to_free_.push_back(wal.ReleaseWriter());
it = logs_.erase(it);
if (wal.GetPreSyncSize() == wal.writer->file()->GetFlushedSize()) {
// Fully synced
logs_to_free_.push_back(wal.ReleaseWriter());
it = logs_.erase(it);
} else {
assert(wal.GetPreSyncSize() < wal.writer->file()->GetFlushedSize());
wal.FinishSync();
++it;
}
} else {
assert(wal.number == logs_.back().number);
// Active WAL
wal.FinishSync();
++it;
}
}
assert(logs_.empty() || logs_[0].number > up_to ||
(logs_.size() == 1 && !logs_[0].IsSyncing()));
log_sync_cv_.SignalAll();
}

Expand Down
45 changes: 45 additions & 0 deletions db/db_write_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -386,6 +386,51 @@ TEST_P(DBWriteTest, UnflushedPutRaceWithTrackedWalSync) {
Close();
}

TEST_P(DBWriteTest, InactiveWalFullySyncedBeforeUntracked) {
// Repro bug where a WAL is appended and switched after
// `FlushWAL(true /* sync */)`'s sync finishes and before it untracks fully
// synced inactive logs. Previously such a WAL would be wrongly untracked
// so the final append would never be synced.
Options options = GetOptions();
std::unique_ptr<FaultInjectionTestEnv> fault_env(
new FaultInjectionTestEnv(env_));
options.env = fault_env.get();
Reopen(options);

ASSERT_OK(Put("key1", "val1"));

SyncPoint::GetInstance()->SetCallBack(
"DBImpl::SyncWAL:BeforeMarkLogsSynced:1", [this](void* /* arg */) {
ASSERT_OK(Put("key2", "val2"));
ASSERT_OK(dbfull()->TEST_SwitchMemtable());
});
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->EnableProcessing();

ASSERT_OK(db_->FlushWAL(true /* sync */));

ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->DisableProcessing();
ROCKSDB_NAMESPACE::SyncPoint::GetInstance()->ClearAllCallBacks();

ASSERT_OK(Put("key3", "val3"));

ASSERT_OK(db_->FlushWAL(true /* sync */));

Close();

// Simulate full loss of unsynced data. This should drop nothing since we did
// `FlushWAL(true /* sync */)` before `Close()`.
fault_env->DropUnsyncedFileData();

Reopen(options);

ASSERT_EQ("val1", Get("key1"));
ASSERT_EQ("val2", Get("key2"));
ASSERT_EQ("val3", Get("key3"));

// Need to close before `fault_env` goes out of scope.
Close();
}

TEST_P(DBWriteTest, IOErrorOnWALWriteTriggersReadOnlyMode) {
std::unique_ptr<FaultInjectionTestEnv> mock_env(
new FaultInjectionTestEnv(env_));
Expand Down
Loading