Track SST unique id in MANIFEST and verify

Summary:
Start tracking SST unique id in MANIFEST, which is used to verify with
SST properties to make sure the SST file is not overwritten or
misplaced. A DB option `try_verify_sst_unique_id` is introduced to
enable/disable the verification, default is false, but might be changed
to true in the future.

Test Plan: unittests and format-compatible test
This commit is contained in:
Jay Zhuang 2022-05-09 16:02:04 -07:00
parent b11ff347b4
commit 9b2fae59d9
46 changed files with 829 additions and 101 deletions

View File

@ -10,6 +10,7 @@
### New Features ### New Features
* DB::GetLiveFilesStorageInfo is ready for production use. * DB::GetLiveFilesStorageInfo is ready for production use.
* Add new stats PREFETCHED_BYTES_DISCARDED which records number of prefetched bytes discarded by RocksDB FilePrefetchBuffer on destruction and POLL_WAIT_MICROS records wait time for FS::Poll API completion. * Add new stats PREFETCHED_BYTES_DISCARDED which records number of prefetched bytes discarded by RocksDB FilePrefetchBuffer on destruction and POLL_WAIT_MICROS records wait time for FS::Poll API completion.
* Start tracking SST unique id in MANIFEST, which will be used to verify with SST properties during DB open to make sure the SST file is not overwritten or misplaced. A db option `verify_sst_unique_id_in_manifest` is introduced to enable/disable the verification, it's disabled by default for now, but plan to be enabled by default later.
### Public API changes ### Public API changes
* Add rollback_deletion_type_callback to TransactionDBOptions so that write-prepared transactions know whether to issue a Delete or SingleDelete to cancel a previous key written during prior prepare phase. The PR aims to prevent mixing SingleDeletes and Deletes for the same key that can lead to undefined behaviors for write-prepared transactions. * Add rollback_deletion_type_callback to TransactionDBOptions so that write-prepared transactions know whether to issue a Delete or SingleDelete to cancel a previous key written during prior prepare phase. The PR aims to prevent mixing SingleDeletes and Deletes for the same key that can lead to undefined behaviors for write-prepared transactions.

View File

@ -37,6 +37,7 @@
#include "table/block_based/block_based_table_builder.h" #include "table/block_based/block_based_table_builder.h"
#include "table/format.h" #include "table/format.h"
#include "table/internal_iterator.h" #include "table/internal_iterator.h"
#include "table/unique_id_impl.h"
#include "test_util/sync_point.h" #include "test_util/sync_point.h"
#include "util/stop_watch.h" #include "util/stop_watch.h"
@ -310,6 +311,11 @@ Status BuildTable(
meta->file_checksum_func_name = file_writer->GetFileChecksumFuncName(); meta->file_checksum_func_name = file_writer->GetFileChecksumFuncName();
file_checksum = meta->file_checksum; file_checksum = meta->file_checksum;
file_checksum_func_name = meta->file_checksum_func_name; file_checksum_func_name = meta->file_checksum_func_name;
// Set unique_id
if (!tboptions.db_id.empty() && !tboptions.db_session_id.empty()) {
s = GetSstInternalUniqueId(tboptions.db_id, tboptions.db_session_id,
meta->fd.GetNumber(), &meta->unique_id);
}
} }
if (s.ok()) { if (s.ok()) {

View File

@ -1060,6 +1060,10 @@ uint64_t ColumnFamilyData::GetLiveSstFilesSize() const {
return current_->GetSstFilesSize(); return current_->GetSstFilesSize();
} }
Status ColumnFamilyData::TryVerifySstUniqueIds() const {
return current_->TryVerifySstUniqueIds();
}
MemTable* ColumnFamilyData::ConstructNewMemtable( MemTable* ColumnFamilyData::ConstructNewMemtable(
const MutableCFOptions& mutable_cf_options, SequenceNumber earliest_seq) { const MutableCFOptions& mutable_cf_options, SequenceNumber earliest_seq) {
return new MemTable(internal_comparator_, ioptions_, mutable_cf_options, return new MemTable(internal_comparator_, ioptions_, mutable_cf_options,

View File

@ -359,6 +359,7 @@ class ColumnFamilyData {
uint64_t GetTotalSstFilesSize() const; // REQUIRE: DB mutex held uint64_t GetTotalSstFilesSize() const; // REQUIRE: DB mutex held
uint64_t GetLiveSstFilesSize() const; // REQUIRE: DB mutex held uint64_t GetLiveSstFilesSize() const; // REQUIRE: DB mutex held
uint64_t GetTotalBlobFileSize() const; // REQUIRE: DB mutex held uint64_t GetTotalBlobFileSize() const; // REQUIRE: DB mutex held
Status TryVerifySstUniqueIds() const; // REQUIRE: DB mutex held
void SetMemtable(MemTable* new_mem) { void SetMemtable(MemTable* new_mem) {
uint64_t memtable_id = last_memtable_id_.fetch_add(1) + 1; uint64_t memtable_id = last_memtable_id_.fetch_add(1) + 1;
new_mem->SetID(memtable_id); new_mem->SetID(memtable_id);

View File

@ -64,6 +64,7 @@
#include "table/block_based/block_based_table_factory.h" #include "table/block_based/block_based_table_factory.h"
#include "table/merging_iterator.h" #include "table/merging_iterator.h"
#include "table/table_builder.h" #include "table/table_builder.h"
#include "table/unique_id_impl.h"
#include "test_util/sync_point.h" #include "test_util/sync_point.h"
#include "util/coding.h" #include "util/coding.h"
#include "util/hash.h" #include "util/hash.h"
@ -1047,6 +1048,7 @@ CompactionJob::ProcessKeyValueCompactionWithCompactionService(
const Compaction* compaction = sub_compact->compaction; const Compaction* compaction = sub_compact->compaction;
CompactionServiceInput compaction_input; CompactionServiceInput compaction_input;
compaction_input.output_level = compaction->output_level(); compaction_input.output_level = compaction->output_level();
compaction_input.db_id = db_id_;
const std::vector<CompactionInputFiles>& inputs = const std::vector<CompactionInputFiles>& inputs =
*(compact_->compaction->inputs()); *(compact_->compaction->inputs());
@ -1208,6 +1210,7 @@ CompactionJob::ProcessKeyValueCompactionWithCompactionService(
meta.oldest_ancester_time = file.oldest_ancester_time; meta.oldest_ancester_time = file.oldest_ancester_time;
meta.file_creation_time = file.file_creation_time; meta.file_creation_time = file.file_creation_time;
meta.marked_for_compaction = file.marked_for_compaction; meta.marked_for_compaction = file.marked_for_compaction;
meta.unique_id = file.unique_id;
auto cfd = compaction->column_family_data(); auto cfd = compaction->column_family_data();
sub_compact->outputs.emplace_back(std::move(meta), sub_compact->outputs.emplace_back(std::move(meta),
@ -2277,6 +2280,18 @@ Status CompactionJob::OpenCompactionOutputFile(
meta.oldest_ancester_time = oldest_ancester_time; meta.oldest_ancester_time = oldest_ancester_time;
meta.file_creation_time = current_time; meta.file_creation_time = current_time;
meta.temperature = temperature; meta.temperature = temperature;
assert(!db_id_.empty());
assert(!db_session_id_.empty());
s = GetSstInternalUniqueId(db_id_, db_session_id_, meta.fd.GetNumber(),
&meta.unique_id);
if (!s.ok()) {
ROCKS_LOG_ERROR(db_options_.info_log,
"[%s] [JOB %d] file #%" PRIu64
" failed to generate unique id: %s.",
cfd->GetName().c_str(), job_id_, meta.fd.GetNumber(),
s.ToString().c_str());
return s;
}
sub_compact->outputs.emplace_back( sub_compact->outputs.emplace_back(
std::move(meta), cfd->internal_comparator(), std::move(meta), cfd->internal_comparator(),
/*enable_order_check=*/ /*enable_order_check=*/
@ -2596,7 +2611,7 @@ Status CompactionServiceCompactionJob::Run() {
meta.fd.largest_seqno, meta.smallest.Encode().ToString(), meta.fd.largest_seqno, meta.smallest.Encode().ToString(),
meta.largest.Encode().ToString(), meta.oldest_ancester_time, meta.largest.Encode().ToString(), meta.oldest_ancester_time,
meta.file_creation_time, output_file.validator.GetHash(), meta.file_creation_time, output_file.validator.GetHash(),
meta.marked_for_compaction); meta.marked_for_compaction, meta.unique_id);
} }
compaction_result_->num_output_records = sub_compact->num_output_records; compaction_result_->num_output_records = sub_compact->num_output_records;
compaction_result_->total_bytes = sub_compact->total_bytes; compaction_result_->total_bytes = sub_compact->total_bytes;
@ -2700,6 +2715,9 @@ static std::unordered_map<std::string, OptionTypeInfo> cs_input_type_info = {
{"output_level", {"output_level",
{offsetof(struct CompactionServiceInput, output_level), OptionType::kInt, {offsetof(struct CompactionServiceInput, output_level), OptionType::kInt,
OptionVerificationType::kNormal, OptionTypeFlags::kNone}}, OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
{"db_id",
{offsetof(struct CompactionServiceInput, db_id),
OptionType::kEncodedString}},
{"has_begin", {"has_begin",
{offsetof(struct CompactionServiceInput, has_begin), OptionType::kBoolean, {offsetof(struct CompactionServiceInput, has_begin), OptionType::kBoolean,
OptionVerificationType::kNormal, OptionTypeFlags::kNone}}, OptionVerificationType::kNormal, OptionTypeFlags::kNone}},
@ -2757,6 +2775,11 @@ static std::unordered_map<std::string, OptionTypeInfo>
{offsetof(struct CompactionServiceOutputFile, marked_for_compaction), {offsetof(struct CompactionServiceOutputFile, marked_for_compaction),
OptionType::kBoolean, OptionVerificationType::kNormal, OptionType::kBoolean, OptionVerificationType::kNormal,
OptionTypeFlags::kNone}}, OptionTypeFlags::kNone}},
{"unique_id",
OptionTypeInfo::Array<uint64_t, 2>(
offsetof(struct CompactionServiceOutputFile, unique_id),
OptionVerificationType::kNormal, OptionTypeFlags::kNone,
{0, OptionType::kUInt64T})},
}; };
static std::unordered_map<std::string, OptionTypeInfo> static std::unordered_map<std::string, OptionTypeInfo>

View File

@ -253,6 +253,9 @@ struct CompactionServiceInput {
std::vector<std::string> input_files; std::vector<std::string> input_files;
int output_level; int output_level;
// db_id which will be used to generate unique id of sst
std::string db_id;
// information for subcompaction // information for subcompaction
bool has_begin = false; bool has_begin = false;
std::string begin; std::string begin;
@ -284,13 +287,15 @@ struct CompactionServiceOutputFile {
uint64_t file_creation_time; uint64_t file_creation_time;
uint64_t paranoid_hash; uint64_t paranoid_hash;
bool marked_for_compaction; bool marked_for_compaction;
UniqueId64x2 unique_id;
CompactionServiceOutputFile() = default; CompactionServiceOutputFile() = default;
CompactionServiceOutputFile( CompactionServiceOutputFile(
const std::string& name, SequenceNumber smallest, SequenceNumber largest, const std::string& name, SequenceNumber smallest, SequenceNumber largest,
std::string _smallest_internal_key, std::string _largest_internal_key, std::string _smallest_internal_key, std::string _largest_internal_key,
uint64_t _oldest_ancester_time, uint64_t _file_creation_time, uint64_t _oldest_ancester_time, uint64_t _file_creation_time,
uint64_t _paranoid_hash, bool _marked_for_compaction) uint64_t _paranoid_hash, bool _marked_for_compaction,
UniqueId64x2 _unique_id)
: file_name(name), : file_name(name),
smallest_seqno(smallest), smallest_seqno(smallest),
largest_seqno(largest), largest_seqno(largest),
@ -299,7 +304,8 @@ struct CompactionServiceOutputFile {
oldest_ancester_time(_oldest_ancester_time), oldest_ancester_time(_oldest_ancester_time),
file_creation_time(_file_creation_time), file_creation_time(_file_creation_time),
paranoid_hash(_paranoid_hash), paranoid_hash(_paranoid_hash),
marked_for_compaction(_marked_for_compaction) {} marked_for_compaction(_marked_for_compaction),
unique_id(std::move(_unique_id)) {}
}; };
// CompactionServiceResult contains the compaction result from a different db // CompactionServiceResult contains the compaction result from a different db

View File

@ -206,7 +206,7 @@ class CompactionJobTestBase : public testing::Test {
oldest_blob_file_number, kUnknownOldestAncesterTime, oldest_blob_file_number, kUnknownOldestAncesterTime,
kUnknownFileCreationTime, kUnknownFileChecksum, kUnknownFileCreationTime, kUnknownFileChecksum,
kUnknownFileChecksumFuncName, kDisableUserTimestamp, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
kDisableUserTimestamp); kDisableUserTimestamp, {});
mutex_.Lock(); mutex_.Lock();
EXPECT_OK( EXPECT_OK(
@ -359,8 +359,8 @@ class CompactionJobTestBase : public testing::Test {
table_cache_, &event_logger, false, false, dbname_, table_cache_, &event_logger, false, false, dbname_,
&compaction_job_stats_, Env::Priority::USER, nullptr /* IOTracer */, &compaction_job_stats_, Env::Priority::USER, nullptr /* IOTracer */,
/*manual_compaction_paused=*/nullptr, /*manual_compaction_paused=*/nullptr,
/*manual_compaction_canceled=*/nullptr, /*db_id=*/"", /*manual_compaction_canceled=*/nullptr, env_->GenerateUniqueId(),
/*db_session_id=*/"", full_history_ts_low_); DBImpl::GenerateDbSessionId(nullptr), full_history_ts_low_);
VerifyInitializationOfCompactionJobStats(compaction_job_stats_); VerifyInitializationOfCompactionJobStats(compaction_job_stats_);
compaction_job.Prepare(); compaction_job.Prepare();
@ -1227,13 +1227,14 @@ TEST_F(CompactionJobTest, ResultSerialization) {
result.status = result.status =
status_list.at(rnd.Uniform(static_cast<int>(status_list.size()))); status_list.at(rnd.Uniform(static_cast<int>(status_list.size())));
while (!rnd.OneIn(10)) { while (!rnd.OneIn(10)) {
UniqueId64x2 id{rnd64.Uniform(UINT64_MAX), rnd64.Uniform(UINT64_MAX)};
result.output_files.emplace_back( result.output_files.emplace_back(
rnd.RandomString(rnd.Uniform(kStrMaxLen)), rnd64.Uniform(UINT64_MAX), rnd.RandomString(rnd.Uniform(kStrMaxLen)), rnd64.Uniform(UINT64_MAX),
rnd64.Uniform(UINT64_MAX), rnd64.Uniform(UINT64_MAX),
rnd.RandomBinaryString(rnd.Uniform(kStrMaxLen)), rnd.RandomBinaryString(rnd.Uniform(kStrMaxLen)),
rnd.RandomBinaryString(rnd.Uniform(kStrMaxLen)), rnd.RandomBinaryString(rnd.Uniform(kStrMaxLen)),
rnd64.Uniform(UINT64_MAX), rnd64.Uniform(UINT64_MAX), rnd64.Uniform(UINT64_MAX), rnd64.Uniform(UINT64_MAX),
rnd64.Uniform(UINT64_MAX), rnd.OneIn(2)); rnd64.Uniform(UINT64_MAX), rnd.OneIn(2), id);
} }
result.output_level = rnd.Uniform(10); result.output_level = rnd.Uniform(10);
result.output_path = rnd.RandomString(rnd.Uniform(kStrMaxLen)); result.output_path = rnd.RandomString(rnd.Uniform(kStrMaxLen));
@ -1261,6 +1262,15 @@ TEST_F(CompactionJobTest, ResultSerialization) {
ASSERT_FALSE(deserialized1.TEST_Equals(&result, &mismatch)); ASSERT_FALSE(deserialized1.TEST_Equals(&result, &mismatch));
ASSERT_EQ(mismatch, "stats.num_input_files"); ASSERT_EQ(mismatch, "stats.num_input_files");
// Test unique id mismatch
if (!result.output_files.empty()) {
CompactionServiceResult deserialized_tmp;
ASSERT_OK(CompactionServiceResult::Read(output, &deserialized_tmp));
deserialized_tmp.output_files[0].unique_id[0] += 1;
ASSERT_FALSE(deserialized_tmp.TEST_Equals(&result, &mismatch));
ASSERT_EQ(mismatch, "output_files.unique_id");
}
// Test unknown field // Test unknown field
CompactionServiceResult deserialized2; CompactionServiceResult deserialized2;
output.clear(); output.clear();

View File

@ -115,7 +115,7 @@ class CompactionPickerTest : public testing::Test {
largest_seq, marked_for_compact, temperature, kInvalidBlobFileNumber, largest_seq, marked_for_compact, temperature, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
f->compensated_file_size = f->compensated_file_size =
(compensated_file_size != 0) ? compensated_file_size : file_size; (compensated_file_size != 0) ? compensated_file_size : file_size;
f->oldest_ancester_time = oldest_ancestor_time; f->oldest_ancester_time = oldest_ancestor_time;

View File

@ -274,6 +274,16 @@ TEST_F(CompactionServiceTest, BasicCompactions) {
auto s = static_cast<Status*>(status); auto s = static_cast<Status*>(status);
*s = Status::Aborted("MyTestCompactionService failed to compact!"); *s = Status::Aborted("MyTestCompactionService failed to compact!");
}); });
// tracking success unique id verification
std::atomic_int verify_passed{0};
SyncPoint::GetInstance()->SetCallBack(
"Version::TryVerifySstUniqueIds::Passed", [&](void* arg) {
// override job status
auto id = static_cast<std::string*>(arg);
assert(!id->empty());
verify_passed++;
});
SyncPoint::GetInstance()->EnableProcessing(); SyncPoint::GetInstance()->EnableProcessing();
Status s; Status s;
@ -298,6 +308,12 @@ TEST_F(CompactionServiceTest, BasicCompactions) {
} }
} }
ASSERT_TRUE(s.IsAborted()); ASSERT_TRUE(s.IsAborted());
// Test verification
ASSERT_EQ(verify_passed, 0);
options.verify_sst_unique_id_in_manifest = true;
Reopen(options);
ASSERT_GT(verify_passed, 0);
} }
TEST_F(CompactionServiceTest, ManualCompaction) { TEST_F(CompactionServiceTest, ManualCompaction) {

View File

@ -1651,7 +1651,8 @@ Status DBImpl::ReFitLevel(ColumnFamilyData* cfd, int level, int target_level) {
f->smallest, f->largest, f->fd.smallest_seqno, f->fd.largest_seqno, f->smallest, f->largest, f->fd.smallest_seqno, f->fd.largest_seqno,
f->marked_for_compaction, f->temperature, f->oldest_blob_file_number, f->marked_for_compaction, f->temperature, f->oldest_blob_file_number,
f->oldest_ancester_time, f->file_creation_time, f->file_checksum, f->oldest_ancester_time, f->file_creation_time, f->file_checksum,
f->file_checksum_func_name, f->min_timestamp, f->max_timestamp); f->file_checksum_func_name, f->min_timestamp, f->max_timestamp,
f->unique_id);
} }
ROCKS_LOG_DEBUG(immutable_db_options_.info_log, ROCKS_LOG_DEBUG(immutable_db_options_.info_log,
"[%s] Apply version edit:\n%s", cfd->GetName().c_str(), "[%s] Apply version edit:\n%s", cfd->GetName().c_str(),
@ -3276,7 +3277,7 @@ Status DBImpl::BackgroundCompaction(bool* made_progress,
f->fd.largest_seqno, f->marked_for_compaction, f->temperature, f->fd.largest_seqno, f->marked_for_compaction, f->temperature,
f->oldest_blob_file_number, f->oldest_ancester_time, f->oldest_blob_file_number, f->oldest_ancester_time,
f->file_creation_time, f->file_checksum, f->file_checksum_func_name, f->file_creation_time, f->file_checksum, f->file_checksum_func_name,
f->min_timestamp, f->max_timestamp); f->min_timestamp, f->max_timestamp, f->unique_id);
ROCKS_LOG_BUFFER( ROCKS_LOG_BUFFER(
log_buffer, log_buffer,

View File

@ -137,7 +137,7 @@ Status DBImpl::PromoteL0(ColumnFamilyHandle* column_family, int target_level) {
f->fd.largest_seqno, f->marked_for_compaction, f->temperature, f->fd.largest_seqno, f->marked_for_compaction, f->temperature,
f->oldest_blob_file_number, f->oldest_ancester_time, f->oldest_blob_file_number, f->oldest_ancester_time,
f->file_creation_time, f->file_checksum, f->file_checksum_func_name, f->file_creation_time, f->file_checksum, f->file_checksum_func_name,
f->min_timestamp, f->max_timestamp); f->min_timestamp, f->max_timestamp, f->unique_id);
} }
status = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(), status = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),

View File

@ -518,6 +518,23 @@ Status DBImpl::Recover(
if (!s.ok()) { if (!s.ok()) {
return s; return s;
} }
if (immutable_db_options_.verify_sst_unique_id_in_manifest) {
if (mutable_db_options_.max_open_files != -1) {
ROCKS_LOG_WARN(
immutable_db_options_.info_log,
"verify_sst_unique_id_in_manifest is set to true, but as "
"max_open_files is "
"not -1, only opened SST during DB-open will be verified.");
}
for (auto cfd : *versions_->GetColumnFamilySet()) {
if (!cfd->IsDropped()) {
s = cfd->TryVerifySstUniqueIds();
if (!s.ok()) {
return s;
}
}
}
}
s = SetDBId(read_only); s = SetDBId(read_only);
if (s.ok() && !read_only) { if (s.ok() && !read_only) {
s = DeleteUnreferencedSstFiles(); s = DeleteUnreferencedSstFiles();
@ -1498,13 +1515,14 @@ Status DBImpl::WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd,
constexpr int level = 0; constexpr int level = 0;
if (s.ok() && has_output) { if (s.ok() && has_output) {
edit->AddFile( edit->AddFile(level, meta.fd.GetNumber(), meta.fd.GetPathId(),
level, meta.fd.GetNumber(), meta.fd.GetPathId(), meta.fd.GetFileSize(), meta.fd.GetFileSize(), meta.smallest, meta.largest,
meta.smallest, meta.largest, meta.fd.smallest_seqno, meta.fd.smallest_seqno, meta.fd.largest_seqno,
meta.fd.largest_seqno, meta.marked_for_compaction, meta.temperature, meta.marked_for_compaction, meta.temperature,
meta.oldest_blob_file_number, meta.oldest_ancester_time, meta.oldest_blob_file_number, meta.oldest_ancester_time,
meta.file_creation_time, meta.file_checksum, meta.file_creation_time, meta.file_checksum,
meta.file_checksum_func_name, meta.min_timestamp, meta.max_timestamp); meta.file_checksum_func_name, meta.min_timestamp,
meta.max_timestamp, meta.unique_id);
for (const auto& blob : blob_file_additions) { for (const auto& blob : blob_file_additions) {
edit->AddBlobFile(blob); edit->AddBlobFile(blob);

View File

@ -772,12 +772,15 @@ Status DBImplSecondary::CompactWithoutInstallation(
const int job_id = next_job_id_.fetch_add(1); const int job_id = next_job_id_.fetch_add(1);
// use primary host's db_id for running the compaction, but db_session_id is
// using the local one. No special reason, only because local db_id is empty.
CompactionServiceCompactionJob compaction_job( CompactionServiceCompactionJob compaction_job(
job_id, c.get(), immutable_db_options_, mutable_db_options_, job_id, c.get(), immutable_db_options_, mutable_db_options_,
file_options_for_compaction_, versions_.get(), &shutting_down_, file_options_for_compaction_, versions_.get(), &shutting_down_,
&log_buffer, output_dir.get(), stats_, &mutex_, &error_handler_, &log_buffer, output_dir.get(), stats_, &mutex_, &error_handler_,
input.snapshots, table_cache_, &event_logger_, dbname_, io_tracer_, input.snapshots, table_cache_, &event_logger_, dbname_, io_tracer_,
options.canceled, db_id_, db_session_id_, secondary_path_, input, result); options.canceled, input.db_id, db_session_id_, secondary_path_, input,
result);
mutex_.Unlock(); mutex_.Unlock();
s = compaction_job.Run(); s = compaction_job.Run();

View File

@ -181,6 +181,7 @@ TEST_F(DBSecondaryTest, SimpleInternalCompaction) {
ASSERT_EQ(input.input_files.size(), 3); ASSERT_EQ(input.input_files.size(), 3);
input.output_level = 1; input.output_level = 1;
ASSERT_OK(db_->GetDbIdentity(input.db_id));
Close(); Close();
options.max_open_files = -1; options.max_open_files = -1;
@ -241,6 +242,7 @@ TEST_F(DBSecondaryTest, InternalCompactionMultiLevels) {
input1.input_files.push_back(meta.levels[1].files[2].name); input1.input_files.push_back(meta.levels[1].files[2].name);
input1.output_level = 1; input1.output_level = 1;
ASSERT_OK(db_->GetDbIdentity(input1.db_id));
options.max_open_files = -1; options.max_open_files = -1;
Close(); Close();
@ -261,6 +263,7 @@ TEST_F(DBSecondaryTest, InternalCompactionMultiLevels) {
} }
input2.output_level = 2; input2.output_level = 2;
input2.db_id = input1.db_id;
ASSERT_OK(db_secondary_full()->TEST_CompactWithoutInstallation( ASSERT_OK(db_secondary_full()->TEST_CompactWithoutInstallation(
OpenAndCompactOptions(), cfh, input2, &result)); OpenAndCompactOptions(), cfh, input2, &result));
ASSERT_OK(result.status); ASSERT_OK(result.status);
@ -305,6 +308,7 @@ TEST_F(DBSecondaryTest, InternalCompactionCompactedFiles) {
ASSERT_EQ(input.input_files.size(), 3); ASSERT_EQ(input.input_files.size(), 3);
input.output_level = 1; input.output_level = 1;
ASSERT_OK(db_->GetDbIdentity(input.db_id));
// trigger compaction to delete the files for secondary instance compaction // trigger compaction to delete the files for secondary instance compaction
ASSERT_OK(Put("foo", "foo_value" + std::to_string(3))); ASSERT_OK(Put("foo", "foo_value" + std::to_string(3)));
@ -346,6 +350,7 @@ TEST_F(DBSecondaryTest, InternalCompactionMissingFiles) {
ASSERT_EQ(input.input_files.size(), 3); ASSERT_EQ(input.input_files.size(), 3);
input.output_level = 1; input.output_level = 1;
ASSERT_OK(db_->GetDbIdentity(input.db_id));
Close(); Close();

View File

@ -7145,6 +7145,113 @@ TEST_F(DBTest2, RenameDirectory) {
dbname_ = old_dbname; dbname_ = old_dbname;
} }
TEST_F(DBTest2, SstUniqueIdVerifyBackwardCompatible) {
const int kNumSst = 3;
const int kLevel0Trigger = 4;
auto options = CurrentOptions();
options.level0_file_num_compaction_trigger = kLevel0Trigger;
options.statistics = CreateDBStatistics();
// Existing manifest doesn't have unique id
SyncPoint::GetInstance()->SetCallBack(
"VersionEdit::EncodeTo:UniqueId", [&](void* arg) {
auto unique_id = static_cast<UniqueId64x2*>(arg);
// remove id before writing it to manifest
(*unique_id)[0] = 0;
(*unique_id)[1] = 0;
});
SyncPoint::GetInstance()->EnableProcessing();
// generate a few SSTs
for (int i = 0; i < kNumSst; i++) {
for (int j = 0; j < 100; j++) {
ASSERT_OK(Put(Key(i * 10 + j), "value"));
}
ASSERT_OK(Flush());
}
// Reopen without verification
Reopen(options);
// Reopen with verification, but it's skipped because manifest doesn't have id
options.verify_sst_unique_id_in_manifest = true;
Reopen(options);
ASSERT_EQ(options.statistics->getTickerCount(
NUMBER_SST_UNIQUE_ID_VERIFICATION_SKIP),
kNumSst);
// test compaction generated Sst
for (int i = kNumSst; i < kLevel0Trigger; i++) {
for (int j = 0; j < 100; j++) {
ASSERT_OK(Put(Key(i * 10 + j), "value"));
}
ASSERT_OK(Flush());
}
ASSERT_OK(dbfull()->TEST_WaitForCompact());
#ifndef ROCKSDB_LITE
ASSERT_EQ("0,1", FilesPerLevel(0));
#endif // ROCKSDB_LITE
// Reopen without verification should fail
options.verify_sst_unique_id_in_manifest = true;
ASSERT_OK(options.statistics->Reset());
Reopen(options);
ASSERT_EQ(options.statistics->getTickerCount(
NUMBER_SST_UNIQUE_ID_VERIFICATION_SKIP),
1);
}
TEST_F(DBTest2, SstUniqueIdVerify) {
const int kNumSst = 3;
const int kLevel0Trigger = 4;
auto options = CurrentOptions();
options.level0_file_num_compaction_trigger = kLevel0Trigger;
SyncPoint::GetInstance()->SetCallBack(
"PropertyBlockBuilder::AddTableProperty:Start", [&](void* props_vs) {
auto props = static_cast<TableProperties*>(props_vs);
// update table property session_id to a different one
props->db_session_id = DBImpl::GenerateDbSessionId(nullptr);
});
SyncPoint::GetInstance()->EnableProcessing();
// generate a few SSTs
for (int i = 0; i < kNumSst; i++) {
for (int j = 0; j < 100; j++) {
ASSERT_OK(Put(Key(i * 10 + j), "value"));
}
ASSERT_OK(Flush());
}
// Reopen with verification should report corruption
options.verify_sst_unique_id_in_manifest = true;
auto s = TryReopen(options);
ASSERT_TRUE(s.IsCorruption());
// Reopen without verification should be fine
options.verify_sst_unique_id_in_manifest = false;
Reopen(options);
// test compaction generated Sst
for (int i = kNumSst; i < kLevel0Trigger; i++) {
for (int j = 0; j < 100; j++) {
ASSERT_OK(Put(Key(i * 10 + j), "value"));
}
ASSERT_OK(Flush());
}
ASSERT_OK(dbfull()->TEST_WaitForCompact());
#ifndef ROCKSDB_LITE
ASSERT_EQ("0,1", FilesPerLevel(0));
#endif // ROCKSDB_LITE
// Reopen without verification should fail
options.verify_sst_unique_id_in_manifest = true;
s = TryReopen(options);
ASSERT_TRUE(s.IsCorruption());
}
#ifndef ROCKSDB_LITE #ifndef ROCKSDB_LITE
TEST_F(DBTest2, GetLatestSeqAndTsForKey) { TEST_F(DBTest2, GetLatestSeqAndTsForKey) {
Destroy(last_options_); Destroy(last_options_);

View File

@ -112,7 +112,7 @@ Status UpdateManifestForFilesState(
lf->oldest_blob_file_number, lf->oldest_blob_file_number,
lf->oldest_ancester_time, lf->file_creation_time, lf->oldest_ancester_time, lf->file_creation_time,
lf->file_checksum, lf->file_checksum_func_name, lf->file_checksum, lf->file_checksum_func_name,
lf->min_timestamp, lf->max_timestamp); lf->min_timestamp, lf->max_timestamp, lf->unique_id);
} }
} }
} else { } else {

View File

@ -1866,6 +1866,60 @@ TEST_F(ExternalSSTFileBasicTest, VerifyChecksum) {
ASSERT_OK(db_->VerifyChecksum()); ASSERT_OK(db_->VerifyChecksum());
} }
TEST_F(ExternalSSTFileBasicTest, VerifySstUniqueId) {
const std::string kPutVal = "put_val";
const std::string kIngestedVal = "ingested_val";
ASSERT_OK(Put("k", kPutVal, WriteOptions()));
ASSERT_OK(Flush());
std::string external_file = sst_files_dir_ + "/file_to_ingest.sst";
{
SstFileWriter sst_file_writer{EnvOptions(), CurrentOptions()};
ASSERT_OK(sst_file_writer.Open(external_file));
ASSERT_OK(sst_file_writer.Put("k", kIngestedVal));
ASSERT_OK(sst_file_writer.Finish());
}
ASSERT_OK(db_->IngestExternalFile(db_->DefaultColumnFamily(), {external_file},
IngestExternalFileOptions()));
auto options = CurrentOptions();
options.verify_sst_unique_id_in_manifest = true;
Reopen(options);
// Test ingest file without session_id and db_id (for example generated by an
// older version of sst_writer)
SyncPoint::GetInstance()->SetCallBack(
"PropertyBlockBuilder::AddTableProperty:Start", [&](void* props_vs) {
auto props = static_cast<TableProperties*>(props_vs);
// update table property session_id to a different one
props->db_session_id = "";
props->db_id = "";
});
SyncPoint::GetInstance()->EnableProcessing();
external_file = sst_files_dir_ + "/file_to_ingest2.sst";
{
SstFileWriter sst_file_writer{EnvOptions(), CurrentOptions()};
ASSERT_OK(sst_file_writer.Open(external_file));
ASSERT_OK(sst_file_writer.Put("k", kIngestedVal));
ASSERT_OK(sst_file_writer.Finish());
}
ASSERT_OK(db_->IngestExternalFile(db_->DefaultColumnFamily(), {external_file},
IngestExternalFileOptions()));
options.statistics = CreateDBStatistics();
options.verify_sst_unique_id_in_manifest = true;
Reopen(options);
// only one sst file is not verified because of missing unique_id
ASSERT_EQ(options.statistics->getTickerCount(
NUMBER_SST_UNIQUE_ID_VERIFICATION_SKIP),
1);
}
INSTANTIATE_TEST_CASE_P(ExternalSSTFileBasicTest, ExternalSSTFileBasicTest, INSTANTIATE_TEST_CASE_P(ExternalSSTFileBasicTest, ExternalSSTFileBasicTest,
testing::Values(std::make_tuple(true, true), testing::Values(std::make_tuple(true, true),
std::make_tuple(true, false), std::make_tuple(true, false),

View File

@ -22,6 +22,7 @@
#include "table/scoped_arena_iterator.h" #include "table/scoped_arena_iterator.h"
#include "table/sst_file_writer_collectors.h" #include "table/sst_file_writer_collectors.h"
#include "table/table_builder.h" #include "table/table_builder.h"
#include "table/unique_id_impl.h"
#include "test_util/sync_point.h" #include "test_util/sync_point.h"
#include "util/stop_watch.h" #include "util/stop_watch.h"
@ -446,8 +447,8 @@ Status ExternalSstFileIngestionJob::Run() {
f.smallest_internal_key, f.largest_internal_key, f.assigned_seqno, f.smallest_internal_key, f.largest_internal_key, f.assigned_seqno,
f.assigned_seqno, false, f.file_temperature, kInvalidBlobFileNumber, f.assigned_seqno, false, f.file_temperature, kInvalidBlobFileNumber,
oldest_ancester_time, current_time, f.file_checksum, oldest_ancester_time, current_time, f.file_checksum,
f.file_checksum_func_name, kDisableUserTimestamp, f.file_checksum_func_name, kDisableUserTimestamp, kDisableUserTimestamp,
kDisableUserTimestamp); f.unique_id);
f_metadata.temperature = f.file_temperature; f_metadata.temperature = f.file_temperature;
edit_.AddFile(f.picked_level, f_metadata); edit_.AddFile(f.picked_level, f_metadata);
} }
@ -727,6 +728,15 @@ Status ExternalSstFileIngestionJob::GetIngestedFileInfo(
file_to_ingest->table_properties = *props; file_to_ingest->table_properties = *props;
auto s = GetSstInternalUniqueId(props->db_id, props->db_session_id,
props->orig_file_number,
&(file_to_ingest->unique_id));
if (!s.ok()) {
ROCKS_LOG_WARN(db_options_.info_log,
"Failed to get SST unique id for file %s",
file_to_ingest->internal_file_path.c_str());
}
return status; return status;
} }

View File

@ -70,6 +70,8 @@ struct IngestedFileInfo {
std::string file_checksum_func_name; std::string file_checksum_func_name;
// The temperature of the file to be ingested // The temperature of the file to be ingested
Temperature file_temperature = Temperature::kUnknown; Temperature file_temperature = Temperature::kUnknown;
// Unique id of the file to be ingested
UniqueId64x2 unique_id;
}; };
class ExternalSstFileIngestionJob { class ExternalSstFileIngestionJob {

View File

@ -9,9 +9,8 @@
#include "db/flush_job.h" #include "db/flush_job.h"
#include <cinttypes>
#include <algorithm> #include <algorithm>
#include <cinttypes>
#include <vector> #include <vector>
#include "db/builder.h" #include "db/builder.h"
@ -988,7 +987,7 @@ Status FlushJob::WriteLevel0Table() {
meta_.oldest_blob_file_number, meta_.oldest_ancester_time, meta_.oldest_blob_file_number, meta_.oldest_ancester_time,
meta_.file_creation_time, meta_.file_checksum, meta_.file_creation_time, meta_.file_checksum,
meta_.file_checksum_func_name, meta_.min_timestamp, meta_.file_checksum_func_name, meta_.min_timestamp,
meta_.max_timestamp); meta_.max_timestamp, meta_.unique_id);
edit_->SetBlobFileAdditions(std::move(blob_file_additions)); edit_->SetBlobFileAdditions(std::move(blob_file_additions));
} }

View File

@ -15,6 +15,7 @@
#include "table/scoped_arena_iterator.h" #include "table/scoped_arena_iterator.h"
#include "table/sst_file_writer_collectors.h" #include "table/sst_file_writer_collectors.h"
#include "table/table_builder.h" #include "table/table_builder.h"
#include "table/unique_id_impl.h"
#include "util/stop_watch.h" #include "util/stop_watch.h"
namespace ROCKSDB_NAMESPACE { namespace ROCKSDB_NAMESPACE {
@ -156,7 +157,7 @@ Status ImportColumnFamilyJob::Run() {
file_metadata.largest_seqno, false, file_metadata.temperature, file_metadata.largest_seqno, false, file_metadata.temperature,
kInvalidBlobFileNumber, oldest_ancester_time, current_time, kInvalidBlobFileNumber, oldest_ancester_time, current_time,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, f.unique_id);
// If incoming sequence number is higher, update local sequence number. // If incoming sequence number is higher, update local sequence number.
if (file_metadata.largest_seqno > versions_->LastSequence()) { if (file_metadata.largest_seqno > versions_->LastSequence()) {
@ -285,6 +286,15 @@ Status ImportColumnFamilyJob::GetIngestedFileInfo(
file_to_import->table_properties = *props; file_to_import->table_properties = *props;
auto s = GetSstInternalUniqueId(props->db_id, props->db_session_id,
props->orig_file_number,
&(file_to_import->unique_id));
if (!s.ok()) {
ROCKS_LOG_WARN(db_options_.info_log,
"Failed to get SST unique id for file %s",
file_to_import->internal_file_path.c_str());
}
return status; return status;
} }

View File

@ -130,6 +130,12 @@ TEST_F(ImportColumnFamilyTest, ImportSSTFileWriterFiles) {
ASSERT_OK(db_->Get(ReadOptions(), import_cfh_, "K4", &value)); ASSERT_OK(db_->Get(ReadOptions(), import_cfh_, "K4", &value));
ASSERT_EQ(value, "V2"); ASSERT_EQ(value, "V2");
} }
EXPECT_OK(db_->DestroyColumnFamilyHandle(import_cfh_));
import_cfh_ = nullptr;
// verify sst unique id during reopen
options.verify_sst_unique_id_in_manifest = true;
ReopenWithColumnFamilies({"default", "koko", "yoyo"}, options);
} }
TEST_F(ImportColumnFamilyTest, ImportSSTFileWriterFilesWithOverlap) { TEST_F(ImportColumnFamilyTest, ImportSSTFileWriterFilesWithOverlap) {

View File

@ -82,6 +82,7 @@
#include "rocksdb/options.h" #include "rocksdb/options.h"
#include "rocksdb/write_buffer_manager.h" #include "rocksdb/write_buffer_manager.h"
#include "table/scoped_arena_iterator.h" #include "table/scoped_arena_iterator.h"
#include "table/unique_id_impl.h"
#include "util/string_util.h" #include "util/string_util.h"
namespace ROCKSDB_NAMESPACE { namespace ROCKSDB_NAMESPACE {
@ -505,6 +506,15 @@ class Repairer {
t->meta.fd, &props); t->meta.fd, &props);
} }
if (status.ok()) { if (status.ok()) {
auto s =
GetSstInternalUniqueId(props->db_id, props->db_session_id,
props->orig_file_number, &t->meta.unique_id);
if (!s.ok()) {
ROCKS_LOG_WARN(db_options_.info_log,
"Table #%" PRIu64
": unable to get unique id, default to Unknown.",
t->meta.fd.GetNumber());
}
t->column_family_id = static_cast<uint32_t>(props->column_family_id); t->column_family_id = static_cast<uint32_t>(props->column_family_id);
if (t->column_family_id == if (t->column_family_id ==
TablePropertiesCollectorFactory::Context::kUnknownColumnFamily) { TablePropertiesCollectorFactory::Context::kUnknownColumnFamily) {
@ -639,7 +649,8 @@ class Repairer {
table->meta.temperature, table->meta.oldest_blob_file_number, table->meta.temperature, table->meta.oldest_blob_file_number,
table->meta.oldest_ancester_time, table->meta.file_creation_time, table->meta.oldest_ancester_time, table->meta.file_creation_time,
table->meta.file_checksum, table->meta.file_checksum_func_name, table->meta.file_checksum, table->meta.file_checksum_func_name,
table->meta.min_timestamp, table->meta.max_timestamp); table->meta.min_timestamp, table->meta.max_timestamp,
table->meta.unique_id);
} }
assert(next_file_number_ > 0); assert(next_file_number_ > 0);
vset_.MarkFileNumberUsed(next_file_number_ - 1); vset_.MarkFileNumberUsed(next_file_number_ - 1);

View File

@ -43,6 +43,23 @@ class RepairTest : public DBTestBase {
} }
return s; return s;
} }
void ReopenWithSstIdVerify() {
std::atomic_int verify_passed{0};
SyncPoint::GetInstance()->SetCallBack(
"Version::TryVerifySstUniqueIds::Passed", [&](void* arg) {
// override job status
auto id = static_cast<std::string*>(arg);
assert(!id->empty());
verify_passed++;
});
SyncPoint::GetInstance()->EnableProcessing();
auto options = CurrentOptions();
options.verify_sst_unique_id_in_manifest = true;
Reopen(options);
ASSERT_GT(verify_passed, 0);
}
}; };
TEST_F(RepairTest, LostManifest) { TEST_F(RepairTest, LostManifest) {
@ -61,7 +78,7 @@ TEST_F(RepairTest, LostManifest) {
ASSERT_OK(env_->FileExists(manifest_path)); ASSERT_OK(env_->FileExists(manifest_path));
ASSERT_OK(env_->DeleteFile(manifest_path)); ASSERT_OK(env_->DeleteFile(manifest_path));
ASSERT_OK(RepairDB(dbname_, CurrentOptions())); ASSERT_OK(RepairDB(dbname_, CurrentOptions()));
Reopen(CurrentOptions()); ReopenWithSstIdVerify();
ASSERT_EQ(Get("key"), "val"); ASSERT_EQ(Get("key"), "val");
ASSERT_EQ(Get("key2"), "val2"); ASSERT_EQ(Get("key2"), "val2");
@ -88,7 +105,9 @@ TEST_F(RepairTest, LostManifestMoreDbFeatures) {
ASSERT_OK(env_->FileExists(manifest_path)); ASSERT_OK(env_->FileExists(manifest_path));
ASSERT_OK(env_->DeleteFile(manifest_path)); ASSERT_OK(env_->DeleteFile(manifest_path));
ASSERT_OK(RepairDB(dbname_, CurrentOptions())); ASSERT_OK(RepairDB(dbname_, CurrentOptions()));
Reopen(CurrentOptions());
// repair from sst should work with unique_id verification
ReopenWithSstIdVerify();
ASSERT_EQ(Get("key"), "val"); ASSERT_EQ(Get("key"), "val");
ASSERT_EQ(Get("key2"), "NOT_FOUND"); ASSERT_EQ(Get("key2"), "NOT_FOUND");
@ -113,7 +132,8 @@ TEST_F(RepairTest, CorruptManifest) {
ASSERT_OK(CreateFile(env_->GetFileSystem(), manifest_path, "blah", ASSERT_OK(CreateFile(env_->GetFileSystem(), manifest_path, "blah",
false /* use_fsync */)); false /* use_fsync */));
ASSERT_OK(RepairDB(dbname_, CurrentOptions())); ASSERT_OK(RepairDB(dbname_, CurrentOptions()));
Reopen(CurrentOptions());
ReopenWithSstIdVerify();
ASSERT_EQ(Get("key"), "val"); ASSERT_EQ(Get("key"), "val");
ASSERT_EQ(Get("key2"), "val2"); ASSERT_EQ(Get("key2"), "val2");
@ -139,7 +159,8 @@ TEST_F(RepairTest, IncompleteManifest) {
// Replace the manifest with one that is only aware of the first SST file. // Replace the manifest with one that is only aware of the first SST file.
CopyFile(orig_manifest_path + ".tmp", new_manifest_path); CopyFile(orig_manifest_path + ".tmp", new_manifest_path);
ASSERT_OK(RepairDB(dbname_, CurrentOptions())); ASSERT_OK(RepairDB(dbname_, CurrentOptions()));
Reopen(CurrentOptions());
ReopenWithSstIdVerify();
ASSERT_EQ(Get("key"), "val"); ASSERT_EQ(Get("key"), "val");
ASSERT_EQ(Get("key2"), "val2"); ASSERT_EQ(Get("key2"), "val2");
@ -157,7 +178,8 @@ TEST_F(RepairTest, PostRepairSstFileNumbering) {
ASSERT_OK(RepairDB(dbname_, CurrentOptions())); ASSERT_OK(RepairDB(dbname_, CurrentOptions()));
Reopen(CurrentOptions()); ReopenWithSstIdVerify();
uint64_t post_repair_file_num = dbfull()->TEST_Current_Next_FileNo(); uint64_t post_repair_file_num = dbfull()->TEST_Current_Next_FileNo();
ASSERT_GE(post_repair_file_num, pre_repair_file_num); ASSERT_GE(post_repair_file_num, pre_repair_file_num);
} }
@ -176,7 +198,7 @@ TEST_F(RepairTest, LostSst) {
Close(); Close();
ASSERT_OK(RepairDB(dbname_, CurrentOptions())); ASSERT_OK(RepairDB(dbname_, CurrentOptions()));
Reopen(CurrentOptions()); ReopenWithSstIdVerify();
// Exactly one of the key-value pairs should be in the DB now. // Exactly one of the key-value pairs should be in the DB now.
ASSERT_TRUE((Get("key") == "val") != (Get("key2") == "val2")); ASSERT_TRUE((Get("key") == "val") != (Get("key2") == "val2"));
@ -198,7 +220,7 @@ TEST_F(RepairTest, CorruptSst) {
Close(); Close();
ASSERT_OK(RepairDB(dbname_, CurrentOptions())); ASSERT_OK(RepairDB(dbname_, CurrentOptions()));
Reopen(CurrentOptions()); ReopenWithSstIdVerify();
// Exactly one of the key-value pairs should be in the DB now. // Exactly one of the key-value pairs should be in the DB now.
ASSERT_TRUE((Get("key") == "val") != (Get("key2") == "val2")); ASSERT_TRUE((Get("key") == "val") != (Get("key2") == "val2"));
@ -226,7 +248,7 @@ TEST_F(RepairTest, UnflushedSst) {
ASSERT_OK(env_->FileExists(manifest_path)); ASSERT_OK(env_->FileExists(manifest_path));
ASSERT_OK(env_->DeleteFile(manifest_path)); ASSERT_OK(env_->DeleteFile(manifest_path));
ASSERT_OK(RepairDB(dbname_, CurrentOptions())); ASSERT_OK(RepairDB(dbname_, CurrentOptions()));
Reopen(CurrentOptions()); ReopenWithSstIdVerify();
ASSERT_OK(dbfull()->GetSortedWalFiles(wal_files)); ASSERT_OK(dbfull()->GetSortedWalFiles(wal_files));
ASSERT_EQ(wal_files.size(), 0); ASSERT_EQ(wal_files.size(), 0);
@ -265,7 +287,7 @@ TEST_F(RepairTest, SeparateWalDir) {
// make sure that all WALs are converted to SSTables. // make sure that all WALs are converted to SSTables.
options.wal_dir = ""; options.wal_dir = "";
Reopen(options); ReopenWithSstIdVerify();
ASSERT_OK(dbfull()->GetSortedWalFiles(wal_files)); ASSERT_OK(dbfull()->GetSortedWalFiles(wal_files));
ASSERT_EQ(wal_files.size(), 0); ASSERT_EQ(wal_files.size(), 0);
{ {
@ -398,7 +420,7 @@ TEST_F(RepairTest, DbNameContainsTrailingSlash) {
Close(); Close();
ASSERT_OK(RepairDB(dbname_ + "/", CurrentOptions())); ASSERT_OK(RepairDB(dbname_ + "/", CurrentOptions()));
Reopen(CurrentOptions()); ReopenWithSstIdVerify();
ASSERT_EQ(Get("key"), "val"); ASSERT_EQ(Get("key"), "val");
} }
#endif // ROCKSDB_LITE #endif // ROCKSDB_LITE

View File

@ -72,7 +72,7 @@ class VersionBuilderTest : public testing::Test {
oldest_blob_file_number, kUnknownOldestAncesterTime, oldest_blob_file_number, kUnknownOldestAncesterTime,
kUnknownFileCreationTime, kUnknownFileChecksum, kUnknownFileCreationTime, kUnknownFileChecksum,
kUnknownFileChecksumFuncName, kDisableUserTimestamp, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
kDisableUserTimestamp); kDisableUserTimestamp, {});
f->compensated_file_size = file_size; f->compensated_file_size = file_size;
f->num_entries = num_entries; f->num_entries = num_entries;
f->num_deletions = num_deletions; f->num_deletions = num_deletions;
@ -134,7 +134,7 @@ class VersionBuilderTest : public testing::Test {
Temperature::kUnknown, blob_file_number, Temperature::kUnknown, blob_file_number,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
} }
void UpdateVersionStorageInfo(VersionStorageInfo* vstorage) { void UpdateVersionStorageInfo(VersionStorageInfo* vstorage) {
@ -180,7 +180,7 @@ TEST_F(VersionBuilderTest, ApplyAndSaveTo) {
Temperature::kUnknown, kInvalidBlobFileNumber, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
version_edit.DeleteFile(3, 27U); version_edit.DeleteFile(3, 27U);
EnvOptions env_options; EnvOptions env_options;
@ -224,7 +224,7 @@ TEST_F(VersionBuilderTest, ApplyAndSaveToDynamic) {
Temperature::kUnknown, kInvalidBlobFileNumber, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
version_edit.DeleteFile(0, 1U); version_edit.DeleteFile(0, 1U);
version_edit.DeleteFile(0, 88U); version_edit.DeleteFile(0, 88U);
@ -271,7 +271,7 @@ TEST_F(VersionBuilderTest, ApplyAndSaveToDynamic2) {
Temperature::kUnknown, kInvalidBlobFileNumber, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
version_edit.DeleteFile(0, 1U); version_edit.DeleteFile(0, 1U);
version_edit.DeleteFile(0, 88U); version_edit.DeleteFile(0, 88U);
version_edit.DeleteFile(4, 6U); version_edit.DeleteFile(4, 6U);
@ -308,31 +308,31 @@ TEST_F(VersionBuilderTest, ApplyMultipleAndSaveTo) {
Temperature::kUnknown, kInvalidBlobFileNumber, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
version_edit.AddFile(2, 676, 0, 100U, GetInternalKey("401"), version_edit.AddFile(2, 676, 0, 100U, GetInternalKey("401"),
GetInternalKey("450"), 200, 200, false, GetInternalKey("450"), 200, 200, false,
Temperature::kUnknown, kInvalidBlobFileNumber, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
version_edit.AddFile(2, 636, 0, 100U, GetInternalKey("601"), version_edit.AddFile(2, 636, 0, 100U, GetInternalKey("601"),
GetInternalKey("650"), 200, 200, false, GetInternalKey("650"), 200, 200, false,
Temperature::kUnknown, kInvalidBlobFileNumber, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
version_edit.AddFile(2, 616, 0, 100U, GetInternalKey("501"), version_edit.AddFile(2, 616, 0, 100U, GetInternalKey("501"),
GetInternalKey("550"), 200, 200, false, GetInternalKey("550"), 200, 200, false,
Temperature::kUnknown, kInvalidBlobFileNumber, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
version_edit.AddFile(2, 606, 0, 100U, GetInternalKey("701"), version_edit.AddFile(2, 606, 0, 100U, GetInternalKey("701"),
GetInternalKey("750"), 200, 200, false, GetInternalKey("750"), 200, 200, false,
Temperature::kUnknown, kInvalidBlobFileNumber, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
EnvOptions env_options; EnvOptions env_options;
constexpr TableCache* table_cache = nullptr; constexpr TableCache* table_cache = nullptr;
@ -372,31 +372,31 @@ TEST_F(VersionBuilderTest, ApplyDeleteAndSaveTo) {
Temperature::kUnknown, kInvalidBlobFileNumber, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
version_edit.AddFile(2, 676, 0, 100U, GetInternalKey("401"), version_edit.AddFile(2, 676, 0, 100U, GetInternalKey("401"),
GetInternalKey("450"), 200, 200, false, GetInternalKey("450"), 200, 200, false,
Temperature::kUnknown, kInvalidBlobFileNumber, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
version_edit.AddFile(2, 636, 0, 100U, GetInternalKey("601"), version_edit.AddFile(2, 636, 0, 100U, GetInternalKey("601"),
GetInternalKey("650"), 200, 200, false, GetInternalKey("650"), 200, 200, false,
Temperature::kUnknown, kInvalidBlobFileNumber, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
version_edit.AddFile(2, 616, 0, 100U, GetInternalKey("501"), version_edit.AddFile(2, 616, 0, 100U, GetInternalKey("501"),
GetInternalKey("550"), 200, 200, false, GetInternalKey("550"), 200, 200, false,
Temperature::kUnknown, kInvalidBlobFileNumber, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
version_edit.AddFile(2, 606, 0, 100U, GetInternalKey("701"), version_edit.AddFile(2, 606, 0, 100U, GetInternalKey("701"),
GetInternalKey("750"), 200, 200, false, GetInternalKey("750"), 200, 200, false,
Temperature::kUnknown, kInvalidBlobFileNumber, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
ASSERT_OK(version_builder.Apply(&version_edit)); ASSERT_OK(version_builder.Apply(&version_edit));
VersionEdit version_edit2; VersionEdit version_edit2;
@ -405,7 +405,7 @@ TEST_F(VersionBuilderTest, ApplyDeleteAndSaveTo) {
Temperature::kUnknown, kInvalidBlobFileNumber, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
version_edit2.DeleteFile(2, 616); version_edit2.DeleteFile(2, 616);
version_edit2.DeleteFile(2, 636); version_edit2.DeleteFile(2, 636);
version_edit.AddFile(2, 806, 0, 100U, GetInternalKey("801"), version_edit.AddFile(2, 806, 0, 100U, GetInternalKey("801"),
@ -413,7 +413,7 @@ TEST_F(VersionBuilderTest, ApplyDeleteAndSaveTo) {
Temperature::kUnknown, kInvalidBlobFileNumber, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
ASSERT_OK(version_builder.Apply(&version_edit2)); ASSERT_OK(version_builder.Apply(&version_edit2));
ASSERT_OK(version_builder.SaveTo(&new_vstorage)); ASSERT_OK(version_builder.SaveTo(&new_vstorage));
@ -525,7 +525,7 @@ TEST_F(VersionBuilderTest, ApplyFileDeletionAndAddition) {
kInvalidBlobFileNumber, kUnknownOldestAncesterTime, kInvalidBlobFileNumber, kUnknownOldestAncesterTime,
kUnknownFileCreationTime, kUnknownFileChecksum, kUnknownFileCreationTime, kUnknownFileChecksum,
kUnknownFileChecksumFuncName, kDisableUserTimestamp, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
kDisableUserTimestamp); kDisableUserTimestamp, {});
ASSERT_OK(builder.Apply(&addition)); ASSERT_OK(builder.Apply(&addition));
@ -575,7 +575,7 @@ TEST_F(VersionBuilderTest, ApplyFileAdditionAlreadyInBase) {
Temperature::kUnknown, kInvalidBlobFileNumber, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
const Status s = builder.Apply(&edit); const Status s = builder.Apply(&edit);
ASSERT_TRUE(s.IsCorruption()); ASSERT_TRUE(s.IsCorruption());
@ -612,7 +612,7 @@ TEST_F(VersionBuilderTest, ApplyFileAdditionAlreadyApplied) {
kInvalidBlobFileNumber, kUnknownOldestAncesterTime, kInvalidBlobFileNumber, kUnknownOldestAncesterTime,
kUnknownFileCreationTime, kUnknownFileChecksum, kUnknownFileCreationTime, kUnknownFileChecksum,
kUnknownFileChecksumFuncName, kDisableUserTimestamp, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
kDisableUserTimestamp); kDisableUserTimestamp, {});
ASSERT_OK(builder.Apply(&edit)); ASSERT_OK(builder.Apply(&edit));
@ -626,7 +626,7 @@ TEST_F(VersionBuilderTest, ApplyFileAdditionAlreadyApplied) {
Temperature::kUnknown, kInvalidBlobFileNumber, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
const Status s = builder.Apply(&other_edit); const Status s = builder.Apply(&other_edit);
ASSERT_TRUE(s.IsCorruption()); ASSERT_TRUE(s.IsCorruption());
@ -663,7 +663,7 @@ TEST_F(VersionBuilderTest, ApplyFileAdditionAndDeletion) {
Temperature::kUnknown, kInvalidBlobFileNumber, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
ASSERT_OK(builder.Apply(&addition)); ASSERT_OK(builder.Apply(&addition));
@ -1232,7 +1232,7 @@ TEST_F(VersionBuilderTest, SaveBlobFilesToConcurrentJobs) {
GetInternalKey(largest), smallest_seqno, largest_seqno, GetInternalKey(largest), smallest_seqno, largest_seqno,
marked_for_compaction, Temperature::kUnknown, blob_file_number, marked_for_compaction, Temperature::kUnknown, blob_file_number,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, checksum_value, kUnknownOldestAncesterTime, kUnknownFileCreationTime, checksum_value,
checksum_method, kDisableUserTimestamp, kDisableUserTimestamp); checksum_method, kDisableUserTimestamp, kDisableUserTimestamp, {});
edit.AddBlobFile(blob_file_number, total_blob_count, total_blob_bytes, edit.AddBlobFile(blob_file_number, total_blob_count, total_blob_bytes,
checksum_method, checksum_value); checksum_method, checksum_value);
@ -1320,7 +1320,7 @@ TEST_F(VersionBuilderTest, CheckConsistencyForBlobFiles) {
/* oldest_blob_file_number */ 16, kUnknownOldestAncesterTime, /* oldest_blob_file_number */ 16, kUnknownOldestAncesterTime,
kUnknownFileCreationTime, kUnknownFileChecksum, kUnknownFileCreationTime, kUnknownFileChecksum,
kUnknownFileChecksumFuncName, kDisableUserTimestamp, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
kDisableUserTimestamp); kDisableUserTimestamp, {});
edit.AddFile(/* level */ 1, /* file_number */ 700, /* path_id */ 0, edit.AddFile(/* level */ 1, /* file_number */ 700, /* path_id */ 0,
/* file_size */ 100, /* smallest */ GetInternalKey("801"), /* file_size */ 100, /* smallest */ GetInternalKey("801"),
@ -1330,7 +1330,7 @@ TEST_F(VersionBuilderTest, CheckConsistencyForBlobFiles) {
/* oldest_blob_file_number */ 1000, kUnknownOldestAncesterTime, /* oldest_blob_file_number */ 1000, kUnknownOldestAncesterTime,
kUnknownFileCreationTime, kUnknownFileChecksum, kUnknownFileCreationTime, kUnknownFileChecksum,
kUnknownFileChecksumFuncName, kDisableUserTimestamp, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
kDisableUserTimestamp); kDisableUserTimestamp, {});
edit.AddBlobFile(/* blob_file_number */ 1000, /* total_blob_count */ 2000, edit.AddBlobFile(/* blob_file_number */ 1000, /* total_blob_count */ 2000,
/* total_blob_bytes */ 200000, /* total_blob_bytes */ 200000,
/* checksum_method */ std::string(), /* checksum_method */ std::string(),
@ -1552,7 +1552,7 @@ TEST_F(VersionBuilderTest, MaintainLinkedSstsForBlobFiles) {
/* oldest_blob_file_number */ 1, kUnknownOldestAncesterTime, /* oldest_blob_file_number */ 1, kUnknownOldestAncesterTime,
kUnknownFileCreationTime, kUnknownFileChecksum, kUnknownFileCreationTime, kUnknownFileChecksum,
kUnknownFileChecksumFuncName, kDisableUserTimestamp, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
kDisableUserTimestamp); kDisableUserTimestamp, {});
// Add an SST that does not reference any blob files. // Add an SST that does not reference any blob files.
edit.AddFile( edit.AddFile(
@ -1563,7 +1563,7 @@ TEST_F(VersionBuilderTest, MaintainLinkedSstsForBlobFiles) {
Temperature::kUnknown, kInvalidBlobFileNumber, kUnknownOldestAncesterTime, Temperature::kUnknown, kInvalidBlobFileNumber, kUnknownOldestAncesterTime,
kUnknownFileCreationTime, kUnknownFileChecksum, kUnknownFileCreationTime, kUnknownFileChecksum,
kUnknownFileChecksumFuncName, kDisableUserTimestamp, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
kDisableUserTimestamp); kDisableUserTimestamp, {});
// Delete a file that references a blob file. // Delete a file that references a blob file.
edit.DeleteFile(/* level */ 1, /* file_number */ 6); edit.DeleteFile(/* level */ 1, /* file_number */ 6);
@ -1586,7 +1586,7 @@ TEST_F(VersionBuilderTest, MaintainLinkedSstsForBlobFiles) {
/* oldest_blob_file_number */ 3, kUnknownOldestAncesterTime, /* oldest_blob_file_number */ 3, kUnknownOldestAncesterTime,
kUnknownFileCreationTime, kUnknownFileChecksum, kUnknownFileCreationTime, kUnknownFileChecksum,
kUnknownFileChecksumFuncName, kDisableUserTimestamp, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
kDisableUserTimestamp); kDisableUserTimestamp, {});
// Trivially move a file that does not reference any blob files. // Trivially move a file that does not reference any blob files.
edit.DeleteFile(/* level */ 1, /* file_number */ 13); edit.DeleteFile(/* level */ 1, /* file_number */ 13);
@ -1598,7 +1598,7 @@ TEST_F(VersionBuilderTest, MaintainLinkedSstsForBlobFiles) {
Temperature::kUnknown, kInvalidBlobFileNumber, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
// Add one more SST file that references a blob file, then promptly // Add one more SST file that references a blob file, then promptly
// delete it in a second version edit before the new version gets saved. // delete it in a second version edit before the new version gets saved.
@ -1612,7 +1612,7 @@ TEST_F(VersionBuilderTest, MaintainLinkedSstsForBlobFiles) {
/* oldest_blob_file_number */ 5, kUnknownOldestAncesterTime, /* oldest_blob_file_number */ 5, kUnknownOldestAncesterTime,
kUnknownFileCreationTime, kUnknownFileChecksum, kUnknownFileCreationTime, kUnknownFileChecksum,
kUnknownFileChecksumFuncName, kDisableUserTimestamp, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
kDisableUserTimestamp); kDisableUserTimestamp, {});
VersionEdit edit2; VersionEdit edit2;

View File

@ -13,6 +13,7 @@
#include "db/version_set.h" #include "db/version_set.h"
#include "logging/event_logger.h" #include "logging/event_logger.h"
#include "rocksdb/slice.h" #include "rocksdb/slice.h"
#include "table/unique_id_impl.h"
#include "test_util/sync_point.h" #include "test_util/sync_point.h"
#include "util/coding.h" #include "util/coding.h"
#include "util/string_util.h" #include "util/string_util.h"
@ -221,6 +222,14 @@ bool VersionEdit::EncodeTo(std::string* dst) const {
PutVarint64(&oldest_blob_file_number, f.oldest_blob_file_number); PutVarint64(&oldest_blob_file_number, f.oldest_blob_file_number);
PutLengthPrefixedSlice(dst, Slice(oldest_blob_file_number)); PutLengthPrefixedSlice(dst, Slice(oldest_blob_file_number));
} }
UniqueId64x2 unique_id = f.unique_id;
TEST_SYNC_POINT_CALLBACK("VersionEdit::EncodeTo:UniqueId", &unique_id);
if (!IsUniqueIdUnknown(unique_id)) {
PutVarint32(dst, NewFileCustomTag::kUniqueId);
std::string unique_id_str = EncodeUniqueIdBytes(unique_id);
PutLengthPrefixedSlice(dst, Slice(unique_id_str));
}
TEST_SYNC_POINT_CALLBACK("VersionEdit::EncodeTo:NewFile4:CustomizeFields", TEST_SYNC_POINT_CALLBACK("VersionEdit::EncodeTo:NewFile4:CustomizeFields",
dst); dst);
@ -392,6 +401,11 @@ const char* VersionEdit::DecodeNewFile4From(Slice* input) {
case kMaxTimestamp: case kMaxTimestamp:
f.max_timestamp = field.ToString(); f.max_timestamp = field.ToString();
break; break;
case kUniqueId:
if (!DecodeUniqueIdBytes(field.ToString(), &f.unique_id).ok()) {
return "invalid unique id";
}
break;
default: default:
if ((custom_tag & kCustomTagNonSafeIgnoreMask) != 0) { if ((custom_tag & kCustomTagNonSafeIgnoreMask) != 0) {
// Should not proceed if cannot understand it // Should not proceed if cannot understand it
@ -819,6 +833,10 @@ std::string VersionEdit::DebugString(bool hex_key) const {
// permanent // permanent
r.append(std::to_string(static_cast<int>(f.temperature))); r.append(std::to_string(static_cast<int>(f.temperature)));
} }
if (!f.unique_id.empty()) {
r.append(" unique_id(internal): ");
r.append(UniqueIdToHumanString(f.unique_id));
}
} }
for (const auto& blob_file_addition : blob_file_additions_) { for (const auto& blob_file_addition : blob_file_additions_) {

View File

@ -85,6 +85,7 @@ enum NewFileCustomTag : uint32_t {
kTemperature = 9, kTemperature = 9,
kMinTimestamp = 10, kMinTimestamp = 10,
kMaxTimestamp = 11, kMaxTimestamp = 11,
kUniqueId = 12,
// If this bit for the custom tag is set, opening DB should fail if // If this bit for the custom tag is set, opening DB should fail if
// we don't know this field. // we don't know this field.
@ -102,6 +103,8 @@ constexpr uint64_t kUnknownFileCreationTime = 0;
extern uint64_t PackFileNumberAndPathId(uint64_t number, uint64_t path_id); extern uint64_t PackFileNumberAndPathId(uint64_t number, uint64_t path_id);
using UniqueId64x2 = std::array<uint64_t, 2>;
// A copyable structure contains information needed to read data from an SST // A copyable structure contains information needed to read data from an SST
// file. It can contain a pointer to a table reader opened for the file, or // file. It can contain a pointer to a table reader opened for the file, or
// file number and size, which can be used to create a new table reader for it. // file number and size, which can be used to create a new table reader for it.
@ -217,6 +220,9 @@ struct FileMetaData {
// Max (newest) timestamp of keys in this file // Max (newest) timestamp of keys in this file
std::string max_timestamp; std::string max_timestamp;
// SST unique id
UniqueId64x2 unique_id{};
FileMetaData() = default; FileMetaData() = default;
FileMetaData(uint64_t file, uint32_t file_path_id, uint64_t file_size, FileMetaData(uint64_t file, uint32_t file_path_id, uint64_t file_size,
@ -227,7 +233,8 @@ struct FileMetaData {
uint64_t _oldest_ancester_time, uint64_t _file_creation_time, uint64_t _oldest_ancester_time, uint64_t _file_creation_time,
const std::string& _file_checksum, const std::string& _file_checksum,
const std::string& _file_checksum_func_name, const std::string& _file_checksum_func_name,
std::string _min_timestamp, std::string _max_timestamp) std::string _min_timestamp, std::string _max_timestamp,
UniqueId64x2 _unique_id)
: fd(file, file_path_id, file_size, smallest_seq, largest_seq), : fd(file, file_path_id, file_size, smallest_seq, largest_seq),
smallest(smallest_key), smallest(smallest_key),
largest(largest_key), largest(largest_key),
@ -239,7 +246,8 @@ struct FileMetaData {
file_checksum(_file_checksum), file_checksum(_file_checksum),
file_checksum_func_name(_file_checksum_func_name), file_checksum_func_name(_file_checksum_func_name),
min_timestamp(std::move(_min_timestamp)), min_timestamp(std::move(_min_timestamp)),
max_timestamp(std::move(_max_timestamp)) { max_timestamp(std::move(_max_timestamp)),
unique_id(std::move(_unique_id)) {
TEST_SYNC_POINT_CALLBACK("FileMetaData::FileMetaData", this); TEST_SYNC_POINT_CALLBACK("FileMetaData::FileMetaData", this);
} }
@ -408,7 +416,8 @@ class VersionEdit {
const std::string& file_checksum, const std::string& file_checksum,
const std::string& file_checksum_func_name, const std::string& file_checksum_func_name,
const std::string& min_timestamp, const std::string& min_timestamp,
const std::string& max_timestamp) { const std::string& max_timestamp,
const UniqueId64x2& unique_id) {
assert(smallest_seqno <= largest_seqno); assert(smallest_seqno <= largest_seqno);
new_files_.emplace_back( new_files_.emplace_back(
level, level,
@ -416,7 +425,7 @@ class VersionEdit {
smallest_seqno, largest_seqno, marked_for_compaction, smallest_seqno, largest_seqno, marked_for_compaction,
temperature, oldest_blob_file_number, oldest_ancester_time, temperature, oldest_blob_file_number, oldest_ancester_time,
file_creation_time, file_checksum, file_checksum_func_name, file_creation_time, file_checksum, file_checksum_func_name,
min_timestamp, max_timestamp)); min_timestamp, max_timestamp, unique_id));
if (!HasLastSequence() || largest_seqno > GetLastSequence()) { if (!HasLastSequence() || largest_seqno > GetLastSequence()) {
SetLastSequence(largest_seqno); SetLastSequence(largest_seqno);
} }

View File

@ -43,7 +43,7 @@ TEST_F(VersionEditTest, EncodeDecode) {
InternalKey("zoo", kBig + 600 + i, kTypeDeletion), InternalKey("zoo", kBig + 600 + i, kTypeDeletion),
kBig + 500 + i, kBig + 600 + i, false, Temperature::kUnknown, kBig + 500 + i, kBig + 600 + i, false, Temperature::kUnknown,
kInvalidBlobFileNumber, 888, 678, "234", "crc32c", "123", kInvalidBlobFileNumber, 888, 678, "234", "crc32c", "123",
"345"); "345", {});
edit.DeleteFile(4, kBig + 700 + i); edit.DeleteFile(4, kBig + 700 + i);
} }
@ -62,25 +62,25 @@ TEST_F(VersionEditTest, EncodeDecodeNewFile4) {
InternalKey("zoo", kBig + 600, kTypeDeletion), kBig + 500, InternalKey("zoo", kBig + 600, kTypeDeletion), kBig + 500,
kBig + 600, true, Temperature::kUnknown, kInvalidBlobFileNumber, kBig + 600, true, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, "123", kUnknownFileChecksum, kUnknownFileChecksumFuncName, "123", "234",
"234"); {});
edit.AddFile(4, 301, 3, 100, InternalKey("foo", kBig + 501, kTypeValue), edit.AddFile(4, 301, 3, 100, InternalKey("foo", kBig + 501, kTypeValue),
InternalKey("zoo", kBig + 601, kTypeDeletion), kBig + 501, InternalKey("zoo", kBig + 601, kTypeDeletion), kBig + 501,
kBig + 601, false, Temperature::kUnknown, kInvalidBlobFileNumber, kBig + 601, false, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, "345", kUnknownFileChecksum, kUnknownFileChecksumFuncName, "345", "543",
"543"); {});
edit.AddFile(5, 302, 0, 100, InternalKey("foo", kBig + 502, kTypeValue), edit.AddFile(5, 302, 0, 100, InternalKey("foo", kBig + 502, kTypeValue),
InternalKey("zoo", kBig + 602, kTypeDeletion), kBig + 502, InternalKey("zoo", kBig + 602, kTypeDeletion), kBig + 502,
kBig + 602, true, Temperature::kUnknown, kInvalidBlobFileNumber, kBig + 602, true, Temperature::kUnknown, kInvalidBlobFileNumber,
666, 888, kUnknownFileChecksum, kUnknownFileChecksumFuncName, 666, 888, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
"456", "567"); "456", "567", {});
edit.AddFile(5, 303, 0, 100, InternalKey("foo", kBig + 503, kTypeBlobIndex), edit.AddFile(5, 303, 0, 100, InternalKey("foo", kBig + 503, kTypeBlobIndex),
InternalKey("zoo", kBig + 603, kTypeBlobIndex), kBig + 503, InternalKey("zoo", kBig + 603, kTypeBlobIndex), kBig + 503,
kBig + 603, true, Temperature::kUnknown, 1001, kBig + 603, true, Temperature::kUnknown, 1001,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, "678", kUnknownFileChecksum, kUnknownFileChecksumFuncName, "678", "789",
"789"); {});
; ;
edit.DeleteFile(4, 700); edit.DeleteFile(4, 700);
@ -129,13 +129,13 @@ TEST_F(VersionEditTest, ForwardCompatibleNewFile4) {
InternalKey("zoo", kBig + 600, kTypeDeletion), kBig + 500, InternalKey("zoo", kBig + 600, kTypeDeletion), kBig + 500,
kBig + 600, true, Temperature::kUnknown, kInvalidBlobFileNumber, kBig + 600, true, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, "123", kUnknownFileChecksum, kUnknownFileChecksumFuncName, "123", "234",
"234"); {});
edit.AddFile(4, 301, 3, 100, InternalKey("foo", kBig + 501, kTypeValue), edit.AddFile(4, 301, 3, 100, InternalKey("foo", kBig + 501, kTypeValue),
InternalKey("zoo", kBig + 601, kTypeDeletion), kBig + 501, InternalKey("zoo", kBig + 601, kTypeDeletion), kBig + 501,
kBig + 601, false, Temperature::kUnknown, kInvalidBlobFileNumber, kBig + 601, false, Temperature::kUnknown, kInvalidBlobFileNumber,
686, 868, "234", "crc32c", kDisableUserTimestamp, 686, 868, "234", "crc32c", kDisableUserTimestamp,
kDisableUserTimestamp); kDisableUserTimestamp, {});
edit.DeleteFile(4, 700); edit.DeleteFile(4, 700);
edit.SetComparatorName("foo"); edit.SetComparatorName("foo");
@ -188,7 +188,7 @@ TEST_F(VersionEditTest, NewFile4NotSupportedField) {
kBig + 600, true, Temperature::kUnknown, kInvalidBlobFileNumber, kBig + 600, true, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
edit.SetComparatorName("foo"); edit.SetComparatorName("foo");
edit.SetLogNumber(kBig + 100); edit.SetLogNumber(kBig + 100);
@ -219,7 +219,7 @@ TEST_F(VersionEditTest, EncodeEmptyFile) {
Temperature::kUnknown, kInvalidBlobFileNumber, Temperature::kUnknown, kInvalidBlobFileNumber,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
std::string buffer; std::string buffer;
ASSERT_TRUE(!edit.EncodeTo(&buffer)); ASSERT_TRUE(!edit.EncodeTo(&buffer));
} }

View File

@ -59,6 +59,7 @@
#include "table/plain/plain_table_factory.h" #include "table/plain/plain_table_factory.h"
#include "table/table_reader.h" #include "table/table_reader.h"
#include "table/two_level_iterator.h" #include "table/two_level_iterator.h"
#include "table/unique_id_impl.h"
#include "test_util/sync_point.h" #include "test_util/sync_point.h"
#include "util/cast_util.h" #include "util/cast_util.h"
#include "util/coding.h" #include "util/coding.h"
@ -1534,6 +1535,37 @@ void Version::GetCreationTimeOfOldestFile(uint64_t* creation_time) {
*creation_time = oldest_time; *creation_time = oldest_time;
} }
Status Version::TryVerifySstUniqueIds() {
for (int level = 0; level < storage_info_.num_non_empty_levels_; level++) {
for (FileMetaData* meta : storage_info_.LevelFiles(level)) {
// if table is not opened or manifest doesn't have unique_id, skip
// verifying
if (meta->fd.table_reader != nullptr &&
!IsUniqueIdUnknown(meta->unique_id)) {
auto props = meta->fd.table_reader->GetTableProperties();
UniqueId64x2 id;
auto s = GetSstInternalUniqueId(props->db_id, props->db_session_id,
props->orig_file_number, &id);
if (!s.ok() || id != meta->unique_id) {
std::ostringstream oss;
oss << "SST #" << meta->fd.GetNumber() << " unique ID mismatch. ";
oss << "Manifest: " << UniqueIdToHumanString(meta->unique_id) << ", ";
if (s.ok()) {
oss << "Table Properties: " << UniqueIdToHumanString(id);
} else {
oss << "Failed to get Table Properties: " << s.ToString();
}
return Status::Corruption("VersionSet", oss.str());
}
TEST_SYNC_POINT_CALLBACK("Version::TryVerifySstUniqueIds::Passed", &id);
} else {
RecordTick(db_statistics_, NUMBER_SST_UNIQUE_ID_VERIFICATION_SKIP);
}
}
}
return Status::OK();
}
uint64_t VersionStorageInfo::GetEstimatedActiveKeys() const { uint64_t VersionStorageInfo::GetEstimatedActiveKeys() const {
// Estimation will be inaccurate when: // Estimation will be inaccurate when:
// (1) there exist merge keys // (1) there exist merge keys
@ -5492,13 +5524,14 @@ Status VersionSet::WriteCurrentStateToManifest(
for (const auto& f : level_files) { for (const auto& f : level_files) {
assert(f); assert(f);
edit.AddFile( edit.AddFile(level, f->fd.GetNumber(), f->fd.GetPathId(),
level, f->fd.GetNumber(), f->fd.GetPathId(), f->fd.GetFileSize(), f->fd.GetFileSize(), f->smallest, f->largest,
f->smallest, f->largest, f->fd.smallest_seqno, f->fd.smallest_seqno, f->fd.largest_seqno,
f->fd.largest_seqno, f->marked_for_compaction, f->temperature, f->marked_for_compaction, f->temperature,
f->oldest_blob_file_number, f->oldest_ancester_time, f->oldest_blob_file_number, f->oldest_ancester_time,
f->file_creation_time, f->file_checksum, f->file_creation_time, f->file_checksum,
f->file_checksum_func_name, f->min_timestamp, f->max_timestamp); f->file_checksum_func_name, f->min_timestamp,
f->max_timestamp, f->unique_id);
} }
} }

View File

@ -851,6 +851,8 @@ class Version {
const MutableCFOptions& GetMutableCFOptions() { return mutable_cf_options_; } const MutableCFOptions& GetMutableCFOptions() { return mutable_cf_options_; }
Status TryVerifySstUniqueIds();
private: private:
Env* env_; Env* env_;
SystemClock* clock_; SystemClock* clock_;

View File

@ -49,7 +49,7 @@ class GenerateLevelFilesBriefTest : public testing::Test {
kInvalidBlobFileNumber, kUnknownOldestAncesterTime, kInvalidBlobFileNumber, kUnknownOldestAncesterTime,
kUnknownFileCreationTime, kUnknownFileChecksum, kUnknownFileCreationTime, kUnknownFileChecksum,
kUnknownFileChecksumFuncName, kDisableUserTimestamp, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
kDisableUserTimestamp); kDisableUserTimestamp, {});
files_.push_back(f); files_.push_back(f);
} }
@ -158,7 +158,7 @@ class VersionStorageInfoTestBase : public testing::Test {
Temperature::kUnknown, oldest_blob_file_number, Temperature::kUnknown, oldest_blob_file_number,
kUnknownOldestAncesterTime, kUnknownFileCreationTime, kUnknownOldestAncesterTime, kUnknownFileCreationTime,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
f->compensated_file_size = file_size; f->compensated_file_size = file_size;
vstorage_.AddFile(level, f); vstorage_.AddFile(level, f);
} }
@ -3222,11 +3222,12 @@ class VersionSetTestMissingFiles : public VersionSetTestBase,
s = fs_->GetFileSize(fname, IOOptions(), &file_size, nullptr); s = fs_->GetFileSize(fname, IOOptions(), &file_size, nullptr);
ASSERT_OK(s); ASSERT_OK(s);
ASSERT_NE(0, file_size); ASSERT_NE(0, file_size);
file_metas->emplace_back(file_num, /*file_path_id=*/0, file_size, ikey, UniqueId64x2 id{};
ikey, 0, 0, false, Temperature::kUnknown, 0, 0, file_metas->emplace_back(
0, kUnknownFileChecksum, file_num, /*file_path_id=*/0, file_size, ikey, ikey, 0, 0, false,
kUnknownFileChecksumFuncName, Temperature::kUnknown, 0, 0, 0, kUnknownFileChecksum,
kDisableUserTimestamp, kDisableUserTimestamp); kUnknownFileChecksumFuncName, kDisableUserTimestamp,
kDisableUserTimestamp, id);
} }
} }
@ -3282,7 +3283,7 @@ TEST_F(VersionSetTestMissingFiles, ManifestFarBehindSst) {
file_num, /*file_path_id=*/0, /*file_size=*/12, smallest_ikey, file_num, /*file_path_id=*/0, /*file_size=*/12, smallest_ikey,
largest_ikey, 0, 0, false, Temperature::kUnknown, 0, 0, 0, largest_ikey, 0, 0, false, Temperature::kUnknown, 0, 0, 0,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
added_files.emplace_back(0, meta); added_files.emplace_back(0, meta);
} }
WriteFileAdditionAndDeletionToManifest( WriteFileAdditionAndDeletionToManifest(
@ -3338,7 +3339,7 @@ TEST_F(VersionSetTestMissingFiles, ManifestAheadofSst) {
file_num, /*file_path_id=*/0, /*file_size=*/12, smallest_ikey, file_num, /*file_path_id=*/0, /*file_size=*/12, smallest_ikey,
largest_ikey, 0, 0, false, Temperature::kUnknown, 0, 0, 0, largest_ikey, 0, 0, false, Temperature::kUnknown, 0, 0, 0,
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kUnknownFileChecksum, kUnknownFileChecksumFuncName,
kDisableUserTimestamp, kDisableUserTimestamp); kDisableUserTimestamp, kDisableUserTimestamp, {});
added_files.emplace_back(0, meta); added_files.emplace_back(0, meta);
} }
WriteFileAdditionAndDeletionToManifest( WriteFileAdditionAndDeletionToManifest(

View File

@ -289,6 +289,7 @@ DECLARE_uint64(wp_commit_cache_bits);
DECLARE_bool(adaptive_readahead); DECLARE_bool(adaptive_readahead);
DECLARE_bool(async_io); DECLARE_bool(async_io);
DECLARE_string(wal_compression); DECLARE_string(wal_compression);
DECLARE_bool(verify_sst_unique_id_in_manifest);
constexpr long KB = 1024; constexpr long KB = 1024;
constexpr int kRandomValueMaxFactor = 3; constexpr int kRandomValueMaxFactor = 3;

View File

@ -936,4 +936,10 @@ DEFINE_bool(
DEFINE_string(wal_compression, "none", DEFINE_string(wal_compression, "none",
"Algorithm to use for WAL compression. none to disable."); "Algorithm to use for WAL compression. none to disable.");
DEFINE_bool(
verify_sst_unique_id_in_manifest, false,
"Enable DB options `verify_sst_unique_id_in_manifest`, if true, during "
"DB-open try verifying the SST unique id between MANIFEST and SST "
"properties.");
#endif // GFLAGS #endif // GFLAGS

View File

@ -2314,6 +2314,8 @@ void StressTest::PrintEnv() const {
static_cast<int>(FLAGS_user_timestamp_size)); static_cast<int>(FLAGS_user_timestamp_size));
fprintf(stdout, "WAL compression : %s\n", fprintf(stdout, "WAL compression : %s\n",
FLAGS_wal_compression.c_str()); FLAGS_wal_compression.c_str());
fprintf(stdout, "Try verify sst unique id : %d\n",
static_cast<int>(FLAGS_verify_sst_unique_id_in_manifest));
fprintf(stdout, "------------------------------------------------\n"); fprintf(stdout, "------------------------------------------------\n");
} }

View File

@ -492,6 +492,26 @@ struct DBOptions {
// Default: false // Default: false
bool track_and_verify_wals_in_manifest = false; bool track_and_verify_wals_in_manifest = false;
// EXPERIMENTAL: This API/behavior is subject to change
// If true, during DB-open it verifies the SST unique id between MANIFEST
// and SST properties, which is to make sure the SST is not overwritten or
// misplaced. A corruption error will be reported if mismatch detected, but
// only when MANIFEST tracks the unique id, which starts from version 7.3.
// The unique id is an internal unique id and subject to change.
// The feature is disabled by default for now and planed to be enabled by
// default in a future release.
//
// Note:
// 1. the option should be used with `max_open_files=-1` to check all SST
// files. Otherwise, only the opened files during DB-open are checked.
// 2. existing SST files won't have its unique_id tracked in MANIFEST, then
// verification will be skipped.
// To check how many SST verifications are skipped, a statistic ticker
// `NUMBER_SST_UNIQUE_ID_VERIFICATION_SKIP` is introduced for that.
//
// Default: false
bool verify_sst_unique_id_in_manifest = false;
// Use the specified object to interact with the environment, // Use the specified object to interact with the environment,
// e.g. to read/write files, schedule background work, etc. In the near // e.g. to read/write files, schedule background work, etc. In the near
// future, support for doing storage operations such as read/write files // future, support for doing storage operations such as read/write files

View File

@ -433,6 +433,9 @@ enum Tickers : uint32_t {
BLOCK_CHECKSUM_COMPUTE_COUNT, BLOCK_CHECKSUM_COMPUTE_COUNT,
// Number of sst files which skipped unique id verification during DB open
NUMBER_SST_UNIQUE_ID_VERIFICATION_SKIP,
TICKER_ENUM_MAX TICKER_ENUM_MAX
}; };

View File

@ -56,6 +56,7 @@ enum class OptionType {
kCustomizable, kCustomizable,
kEncodedString, kEncodedString,
kTemperature, kTemperature,
kArray,
kUnknown, kUnknown,
}; };
@ -155,6 +156,24 @@ bool SerializeEnum(const std::unordered_map<std::string, T>& type_map,
return false; return false;
} }
template <typename T, size_t kSize>
Status ParseArray(const ConfigOptions& config_options,
const OptionTypeInfo& elem_info, char separator,
const std::string& name, const std::string& value,
std::array<T, kSize>* result);
template <typename T, size_t kSize>
Status SerializeArray(const ConfigOptions& config_options,
const OptionTypeInfo& elem_info, char separator,
const std::string& name, const std::array<T, kSize>& vec,
std::string* value);
template <typename T, size_t kSize>
bool ArraysAreEqual(const ConfigOptions& config_options,
const OptionTypeInfo& elem_info, const std::string& name,
const std::array<T, kSize>& array1,
const std::array<T, kSize>& array2, std::string* mismatch);
template <typename T> template <typename T>
Status ParseVector(const ConfigOptions& config_options, Status ParseVector(const ConfigOptions& config_options,
const OptionTypeInfo& elem_info, char separator, const OptionTypeInfo& elem_info, char separator,
@ -388,6 +407,38 @@ class OptionTypeInfo {
return info.SetParseFunc(parse_func); return info.SetParseFunc(parse_func);
} }
template <typename T, size_t kSize>
static OptionTypeInfo Array(int _offset, OptionVerificationType _verification,
OptionTypeFlags _flags,
const OptionTypeInfo& elem_info,
char separator = ':') {
OptionTypeInfo info(_offset, OptionType::kArray, _verification, _flags);
info.SetParseFunc([elem_info, separator](
const ConfigOptions& opts, const std::string& name,
const std::string& value, void* addr) {
auto result = static_cast<std::array<T, kSize>*>(addr);
return ParseArray<T, kSize>(opts, elem_info, separator, name, value,
result);
});
info.SetSerializeFunc([elem_info, separator](const ConfigOptions& opts,
const std::string& name,
const void* addr,
std::string* value) {
const auto& array = *(static_cast<const std::array<T, kSize>*>(addr));
return SerializeArray<T, kSize>(opts, elem_info, separator, name, array,
value);
});
info.SetEqualsFunc([elem_info](const ConfigOptions& opts,
const std::string& name, const void* addr1,
const void* addr2, std::string* mismatch) {
const auto& array1 = *(static_cast<const std::array<T, kSize>*>(addr1));
const auto& array2 = *(static_cast<const std::array<T, kSize>*>(addr2));
return ArraysAreEqual<T, kSize>(opts, elem_info, name, array1, array2,
mismatch);
});
return info;
}
template <typename T> template <typename T>
static OptionTypeInfo Vector(int _offset, static OptionTypeInfo Vector(int _offset,
OptionVerificationType _verification, OptionVerificationType _verification,
@ -893,6 +944,144 @@ class OptionTypeInfo {
OptionTypeFlags flags_; OptionTypeFlags flags_;
}; };
// Parses the input value into elements of the result array, which has fixed
// array size. For example, if the value=1:2:3 and elem_info parses integers,
// the result array will be {1,2,3}. Array size is defined in the OptionTypeInfo
// the input value has to match with that.
// @param config_options Controls how the option value is parsed.
// @param elem_info Controls how individual tokens in value are parsed
// @param separator Character separating tokens in values (':' in the above
// example)
// @param name The name associated with this array option
// @param value The input string to parse into tokens
// @param result Returns the results of parsing value into its elements.
// @return OK if the value was successfully parse
// @return InvalidArgument if the value is improperly formed or element number
// doesn't match array size defined in OptionTypeInfo
// or if the token could not be parsed
// @return NotFound If the tokenized value contains unknown options for
// its type
template <typename T, size_t kSize>
Status ParseArray(const ConfigOptions& config_options,
const OptionTypeInfo& elem_info, char separator,
const std::string& name, const std::string& value,
std::array<T, kSize>* result) {
Status status;
ConfigOptions copy = config_options;
copy.ignore_unsupported_options = false;
size_t i = 0, start = 0, end = 0;
for (; status.ok() && i < kSize && start < value.size() &&
end != std::string::npos;
i++, start = end + 1) {
std::string token;
status = OptionTypeInfo::NextToken(value, separator, start, &end, &token);
if (status.ok()) {
status = elem_info.Parse(copy, name, token, &((*result)[i]));
if (config_options.ignore_unsupported_options &&
status.IsNotSupported()) {
// If we were ignoring unsupported options and this one should be
// ignored, ignore it by setting the status to OK
status = Status::OK();
}
}
}
if (!status.ok()) {
return status;
}
// make sure the element number matches the array size
if (i < kSize) {
return Status::InvalidArgument(
"Serialized value has less elements than array size", name);
}
if (start < value.size() && end != std::string::npos) {
return Status::InvalidArgument(
"Serialized value has more elements than array size", name);
}
return status;
}
// Serializes the fixed size input array into its output value. Elements are
// separated by the separator character. This element will convert all of the
// elements in array into their serialized form, using elem_info to perform the
// serialization.
// For example, if the array contains the integers 1,2,3 and elem_info
// serializes the output would be 1:2:3 for separator ":".
// @param config_options Controls how the option value is serialized.
// @param elem_info Controls how individual tokens in value are serialized
// @param separator Character separating tokens in value (':' in the above
// example)
// @param name The name associated with this array option
// @param array The input array to serialize
// @param value The output string of serialized options
// @return OK if the value was successfully parse
// @return InvalidArgument if the value is improperly formed or if the token
// could not be parsed
// @return NotFound If the tokenized value contains unknown options for
// its type
template <typename T, size_t kSize>
Status SerializeArray(const ConfigOptions& config_options,
const OptionTypeInfo& elem_info, char separator,
const std::string& name,
const std::array<T, kSize>& array, std::string* value) {
std::string result;
ConfigOptions embedded = config_options;
embedded.delimiter = ";";
int printed = 0;
for (const auto& elem : array) {
std::string elem_str;
Status s = elem_info.Serialize(embedded, name, &elem, &elem_str);
if (!s.ok()) {
return s;
} else if (!elem_str.empty()) {
if (printed++ > 0) {
result += separator;
}
// If the element contains embedded separators, put it inside of brackets
if (elem_str.find(separator) != std::string::npos) {
result += "{" + elem_str + "}";
} else {
result += elem_str;
}
}
}
if (result.find("=") != std::string::npos) {
*value = "{" + result + "}";
} else if (printed > 1 && result.at(0) == '{') {
*value = "{" + result + "}";
} else {
*value = result;
}
return Status::OK();
}
// Compares the input arrays array1 and array2 for equality
// Elements of the array are compared one by one using elem_info to perform the
// comparison.
//
// @param config_options Controls how the arrays are compared.
// @param elem_info Controls how individual elements in the arrays are compared
// @param name The name associated with this array option
// @param array1,array2 The arrays to compare.
// @param mismatch If the arrays are not equivalent, mismatch will point to
// the first element of the comparison that did not match.
// @return true If vec1 and vec2 are "equal", false otherwise
template <typename T, size_t kSize>
bool ArraysAreEqual(const ConfigOptions& config_options,
const OptionTypeInfo& elem_info, const std::string& name,
const std::array<T, kSize>& array1,
const std::array<T, kSize>& array2, std::string* mismatch) {
assert(array1.size() == kSize);
assert(array2.size() == kSize);
for (size_t i = 0; i < kSize; ++i) {
if (!elem_info.AreEqual(config_options, name, &array1[i], &array2[i],
mismatch)) {
return false;
}
}
return true;
}
// Parses the input value into elements of the result vector. This method // Parses the input value into elements of the result vector. This method
// will break the input value into the individual tokens (based on the // will break the input value into the individual tokens (based on the
// separator), where each of those tokens will be parsed based on the rules of // separator), where each of those tokens will be parsed based on the rules of

View File

@ -5088,6 +5088,8 @@ class TickerTypeJni {
return -0x2D; return -0x2D;
case ROCKSDB_NAMESPACE::Tickers::BLOCK_CHECKSUM_COMPUTE_COUNT: case ROCKSDB_NAMESPACE::Tickers::BLOCK_CHECKSUM_COMPUTE_COUNT:
return -0x2E; return -0x2E;
case ROCKSDB_NAMESPACE::Tickers::NUMBER_SST_UNIQUE_ID_VERIFICATION_SKIP:
return -0x2F;
case ROCKSDB_NAMESPACE::Tickers::TICKER_ENUM_MAX: case ROCKSDB_NAMESPACE::Tickers::TICKER_ENUM_MAX:
// 0x5F was the max value in the initial copy of tickers to Java. // 0x5F was the max value in the initial copy of tickers to Java.
// Since these values are exposed directly to Java clients, we keep // Since these values are exposed directly to Java clients, we keep
@ -5459,6 +5461,9 @@ class TickerTypeJni {
return ROCKSDB_NAMESPACE::Tickers::NON_LAST_LEVEL_READ_COUNT; return ROCKSDB_NAMESPACE::Tickers::NON_LAST_LEVEL_READ_COUNT;
case -0x2E: case -0x2E:
return ROCKSDB_NAMESPACE::Tickers::BLOCK_CHECKSUM_COMPUTE_COUNT; return ROCKSDB_NAMESPACE::Tickers::BLOCK_CHECKSUM_COMPUTE_COUNT;
case -0x2F:
return ROCKSDB_NAMESPACE::Tickers::
NUMBER_SST_UNIQUE_ID_VERIFICATION_SKIP;
case 0x5F: case 0x5F:
// 0x5F was the max value in the initial copy of tickers to Java. // 0x5F was the max value in the initial copy of tickers to Java.
// Since these values are exposed directly to Java clients, we keep // Since these values are exposed directly to Java clients, we keep

View File

@ -804,6 +804,11 @@ public enum TickerType {
NON_LAST_LEVEL_READ_BYTES((byte) -0x2C), NON_LAST_LEVEL_READ_BYTES((byte) -0x2C),
NON_LAST_LEVEL_READ_COUNT((byte) -0x2D), NON_LAST_LEVEL_READ_COUNT((byte) -0x2D),
/**
* Number of sst files which skipped unique id verification in DB open
*/
NUMBER_SST_UNIQUE_ID_VERIFICATION_SKIP((byte) -0x2F),
BLOCK_CHECKSUM_COMPUTE_COUNT((byte) -0x2E), BLOCK_CHECKSUM_COMPUTE_COUNT((byte) -0x2E),
TICKER_ENUM_MAX((byte) 0x5F); TICKER_ENUM_MAX((byte) 0x5F);

View File

@ -226,7 +226,9 @@ const std::vector<std::pair<Tickers, std::string>> TickersNameMap = {
{LAST_LEVEL_READ_COUNT, "rocksdb.last.level.read.count"}, {LAST_LEVEL_READ_COUNT, "rocksdb.last.level.read.count"},
{NON_LAST_LEVEL_READ_BYTES, "rocksdb.non.last.level.read.bytes"}, {NON_LAST_LEVEL_READ_BYTES, "rocksdb.non.last.level.read.bytes"},
{NON_LAST_LEVEL_READ_COUNT, "rocksdb.non.last.level.read.count"}, {NON_LAST_LEVEL_READ_COUNT, "rocksdb.non.last.level.read.count"},
{BLOCK_CHECKSUM_COMPUTE_COUNT, "rocksdb.block.checksum.compute.count"}}; {BLOCK_CHECKSUM_COMPUTE_COUNT, "rocksdb.block.checksum.compute.count"},
{NUMBER_SST_UNIQUE_ID_VERIFICATION_SKIP,
"rocksdb.number.sst.uniqueid.verification.skip"}};
const std::vector<std::pair<Histograms, std::string>> HistogramsNameMap = { const std::vector<std::pair<Histograms, std::string>> HistogramsNameMap = {
{DB_GET, "rocksdb.db.get.micros"}, {DB_GET, "rocksdb.db.get.micros"},

View File

@ -680,6 +680,8 @@ ImmutableDBOptions::ImmutableDBOptions(const DBOptions& options)
flush_verify_memtable_count(options.flush_verify_memtable_count), flush_verify_memtable_count(options.flush_verify_memtable_count),
track_and_verify_wals_in_manifest( track_and_verify_wals_in_manifest(
options.track_and_verify_wals_in_manifest), options.track_and_verify_wals_in_manifest),
verify_sst_unique_id_in_manifest(
options.verify_sst_unique_id_in_manifest),
env(options.env), env(options.env),
rate_limiter(options.rate_limiter), rate_limiter(options.rate_limiter),
sst_file_manager(options.sst_file_manager), sst_file_manager(options.sst_file_manager),

View File

@ -26,6 +26,7 @@ struct ImmutableDBOptions {
bool paranoid_checks; bool paranoid_checks;
bool flush_verify_memtable_count; bool flush_verify_memtable_count;
bool track_and_verify_wals_in_manifest; bool track_and_verify_wals_in_manifest;
bool verify_sst_unique_id_in_manifest;
Env* env; Env* env;
std::shared_ptr<RateLimiter> rate_limiter; std::shared_ptr<RateLimiter> rate_limiter;
std::shared_ptr<SstFileManager> sst_file_manager; std::shared_ptr<SstFileManager> sst_file_manager;

View File

@ -4603,6 +4603,68 @@ TEST_F(OptionTypeInfoTest, TestStruct) {
ASSERT_EQ(e1.b.s, "66"); ASSERT_EQ(e1.b.s, "66");
} }
TEST_F(OptionTypeInfoTest, TestArrayType) {
OptionTypeInfo array_info = OptionTypeInfo::Array<std::string, 4>(
0, OptionVerificationType::kNormal, OptionTypeFlags::kNone,
{0, OptionType::kString});
std::array<std::string, 4> array1, array2;
std::string mismatch;
ConfigOptions config_options;
TestParseAndCompareOption(config_options, array_info, "v", "a:b:c:d", &array1,
&array2);
ASSERT_EQ(array1.size(), 4);
ASSERT_EQ(array1[0], "a");
ASSERT_EQ(array1[1], "b");
ASSERT_EQ(array1[2], "c");
ASSERT_EQ(array1[3], "d");
array1[3] = "e";
ASSERT_FALSE(
array_info.AreEqual(config_options, "v", &array1, &array2, &mismatch));
ASSERT_EQ(mismatch, "v");
// Test vectors with inner brackets
TestParseAndCompareOption(config_options, array_info, "v", "a:{b}:c:d",
&array1, &array2);
ASSERT_EQ(array1.size(), 4);
ASSERT_EQ(array1[0], "a");
ASSERT_EQ(array1[1], "b");
ASSERT_EQ(array1[2], "c");
ASSERT_EQ(array1[3], "d");
std::array<std::string, 3> array3, array4;
OptionTypeInfo bar_info = OptionTypeInfo::Array<std::string, 3>(
0, OptionVerificationType::kNormal, OptionTypeFlags::kNone,
{0, OptionType::kString}, '|');
TestParseAndCompareOption(config_options, bar_info, "v", "x|y|z", &array3,
&array4);
// Test arrays with inner array
TestParseAndCompareOption(config_options, bar_info, "v",
"a|{b1|b2}|{c1|c2|{d1|d2}}", &array3, &array4,
false);
ASSERT_EQ(array3.size(), 3);
ASSERT_EQ(array3[0], "a");
ASSERT_EQ(array3[1], "b1|b2");
ASSERT_EQ(array3[2], "c1|c2|{d1|d2}");
TestParseAndCompareOption(config_options, bar_info, "v",
"{a1|a2}|{b1|{c1|c2}}|d1", &array3, &array4, true);
ASSERT_EQ(array3.size(), 3);
ASSERT_EQ(array3[0], "a1|a2");
ASSERT_EQ(array3[1], "b1|{c1|c2}");
ASSERT_EQ(array3[2], "d1");
// Test invalid input: less element than requested
auto s = bar_info.Parse(config_options, "opt_name1", "a1|a2", &array3);
ASSERT_TRUE(s.IsInvalidArgument());
// Test invalid input: more element than requested
s = bar_info.Parse(config_options, "opt_name2", "a1|b|c1|d3", &array3);
ASSERT_TRUE(s.IsInvalidArgument());
}
TEST_F(OptionTypeInfoTest, TestVectorType) { TEST_F(OptionTypeInfoTest, TestVectorType) {
OptionTypeInfo vec_info = OptionTypeInfo::Vector<std::string>( OptionTypeInfo vec_info = OptionTypeInfo::Vector<std::string>(
0, OptionVerificationType::kNormal, OptionTypeFlags::kNone, 0, OptionVerificationType::kNormal, OptionTypeFlags::kNone,

View File

@ -105,6 +105,20 @@ Status GetSstInternalUniqueId(const std::string &db_id,
return Status::OK(); return Status::OK();
} }
Status GetSstInternalUniqueId(const std::string &db_id,
const std::string &db_session_id,
uint64_t file_number, UniqueId64x2 *out) {
UniqueId64x3 tmp{};
Status s = GetSstInternalUniqueId(db_id, db_session_id, file_number, &tmp);
if (s.ok()) {
(*out)[0] = tmp[0];
(*out)[1] = tmp[1];
} else {
*out = {0, 0};
}
return s;
}
namespace { namespace {
// For InternalUniqueIdToExternal / ExternalUniqueIdToInternal we want all // For InternalUniqueIdToExternal / ExternalUniqueIdToInternal we want all
// zeros in first 128 bits to map to itself, so that excluding zero in // zeros in first 128 bits to map to itself, so that excluding zero in
@ -140,6 +154,27 @@ std::string EncodeUniqueIdBytes(const UniqueId64x3 &in) {
return ret; return ret;
} }
std::string EncodeUniqueIdBytes(const UniqueId64x2 &in) {
std::string ret(16U, '\0');
EncodeFixed64(&ret[0], in[0]);
EncodeFixed64(&ret[8], in[1]);
return ret;
}
Status DecodeUniqueIdBytes(const std::string &unique_id, UniqueId64x2 *out) {
if (unique_id.size() != 16) {
return Status::NotSupported("Not a valid unique_id");
}
const char *buf = &unique_id.front();
(*out)[0] = DecodeFixed64(buf);
(*out)[1] = DecodeFixed64(buf + 8);
return Status::OK();
}
bool IsUniqueIdUnknown(const UniqueId64x2 &in) {
return in[0] == 0 && in[1] == 0;
}
Status GetUniqueIdFromTableProperties(const TableProperties &props, Status GetUniqueIdFromTableProperties(const TableProperties &props,
std::string *out_id) { std::string *out_id) {
UniqueId64x3 tmp{}; UniqueId64x3 tmp{};
@ -163,4 +198,8 @@ std::string UniqueIdToHumanString(const std::string &id) {
return str; return str;
} }
std::string UniqueIdToHumanString(const UniqueId64x2 &id) {
return UniqueIdToHumanString(EncodeUniqueIdBytes(id));
}
} // namespace ROCKSDB_NAMESPACE } // namespace ROCKSDB_NAMESPACE

View File

@ -12,6 +12,7 @@
namespace ROCKSDB_NAMESPACE { namespace ROCKSDB_NAMESPACE {
using UniqueId64x3 = std::array<uint64_t, 3>; using UniqueId64x3 = std::array<uint64_t, 3>;
using UniqueId64x2 = std::array<uint64_t, 2>;
// Helper for GetUniqueIdFromTableProperties. This function can also be used // Helper for GetUniqueIdFromTableProperties. This function can also be used
// for temporary ids for files without sufficient information in table // for temporary ids for files without sufficient information in table
@ -23,6 +24,12 @@ Status GetSstInternalUniqueId(const std::string &db_id,
const std::string &db_session_id, const std::string &db_session_id,
uint64_t file_number, UniqueId64x3 *out); uint64_t file_number, UniqueId64x3 *out);
Status GetSstInternalUniqueId(const std::string &db_id,
const std::string &db_session_id,
uint64_t file_number, UniqueId64x2 *out);
std::string UniqueIdToHumanString(const UniqueId64x2 &id);
// Helper for GetUniqueIdFromTableProperties. External unique ids go through // Helper for GetUniqueIdFromTableProperties. External unique ids go through
// this extra hashing layer so that prefixes of the unique id have predictable // this extra hashing layer so that prefixes of the unique id have predictable
// "full" entropy. This hashing layer is 1-to-1 on the first 128 bits and on // "full" entropy. This hashing layer is 1-to-1 on the first 128 bits and on
@ -37,6 +44,11 @@ void ExternalUniqueIdToInternal(UniqueId64x3 *in_out);
// Convert numerical format to byte format for public API // Convert numerical format to byte format for public API
std::string EncodeUniqueIdBytes(const UniqueId64x3 &in); std::string EncodeUniqueIdBytes(const UniqueId64x3 &in);
std::string EncodeUniqueIdBytes(const UniqueId64x2 &in);
Status DecodeUniqueIdBytes(const std::string &unique_id, UniqueId64x2 *out);
bool IsUniqueIdUnknown(const UniqueId64x2 &in);
// Reformat a random value down to our "DB session id" format, // Reformat a random value down to our "DB session id" format,
// which is intended to be compact and friendly for use in file names. // which is intended to be compact and friendly for use in file names.

View File

@ -171,6 +171,7 @@ default_params = {
"adaptive_readahead": lambda: random.choice([0, 1]), "adaptive_readahead": lambda: random.choice([0, 1]),
"async_io": lambda: random.choice([0, 1]), "async_io": lambda: random.choice([0, 1]),
"wal_compression": lambda: random.choice(["none", "zstd"]), "wal_compression": lambda: random.choice(["none", "zstd"]),
"verify_sst_unique_id_in_manifest": 1, # always verify for now
} }
_TEST_DIR_ENV_VAR = 'TEST_TMPDIR' _TEST_DIR_ENV_VAR = 'TEST_TMPDIR'