Track SST unique id in MANIFEST and verify (#9990)
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, if enabled, it opens all SST files during DB-open to read the unique_id from table properties (default is false), so it's recommended to use it with `max_open_files = -1` to pre-open the files. Pull Request resolved: https://github.com/facebook/rocksdb/pull/9990 Test Plan: unittests, format-compatible test, mini-crash Reviewed By: anand1976 Differential Revision: D36381863 Pulled By: jay-zhuang fbshipit-source-id: 89ea2eb6b35ed3e80ead9c724eb096083eaba63f
This commit is contained in:
parent
dde774db64
commit
c6d326d3d7
@ -12,6 +12,7 @@
|
|||||||
* 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.
|
||||||
* RemoteCompaction supports table_properties_collector_factories override on compaction worker.
|
* RemoteCompaction supports table_properties_collector_factories override on compaction worker.
|
||||||
|
* 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, if enabled all SST files will be opened during DB-open to verify the unique id (default is false), so it's recommended to use it with `max_open_files = -1` to pre-open the files.
|
||||||
|
|
||||||
### 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.
|
||||||
|
@ -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,15 @@ 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 only if db_id and db_session_id exist
|
||||||
|
if (!tboptions.db_id.empty() && !tboptions.db_session_id.empty()) {
|
||||||
|
if (!GetSstInternalUniqueId(tboptions.db_id, tboptions.db_session_id,
|
||||||
|
meta->fd.GetNumber(), &(meta->unique_id))
|
||||||
|
.ok()) {
|
||||||
|
// if failed to get unique id, just set it Null
|
||||||
|
meta->unique_id = kNullUniqueId64x2;
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (s.ok()) {
|
if (s.ok()) {
|
||||||
|
@ -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=*/
|
||||||
@ -2609,7 +2624,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;
|
||||||
@ -2713,6 +2728,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}},
|
||||||
@ -2770,6 +2788,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>
|
||||||
|
@ -259,6 +259,9 @@ struct CompactionServiceInput {
|
|||||||
std::vector<std::string> input_files;
|
std::vector<std::string> input_files;
|
||||||
int output_level;
|
int output_level;
|
||||||
|
|
||||||
|
// db_id is used to generate unique id of sst on the remote compactor
|
||||||
|
std::string db_id;
|
||||||
|
|
||||||
// information for subcompaction
|
// information for subcompaction
|
||||||
bool has_begin = false;
|
bool has_begin = false;
|
||||||
std::string begin;
|
std::string begin;
|
||||||
@ -290,13 +293,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),
|
||||||
@ -305,7 +310,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
|
||||||
|
@ -27,6 +27,7 @@
|
|||||||
#include "rocksdb/options.h"
|
#include "rocksdb/options.h"
|
||||||
#include "rocksdb/write_buffer_manager.h"
|
#include "rocksdb/write_buffer_manager.h"
|
||||||
#include "table/mock_table.h"
|
#include "table/mock_table.h"
|
||||||
|
#include "table/unique_id_impl.h"
|
||||||
#include "test_util/testharness.h"
|
#include "test_util/testharness.h"
|
||||||
#include "test_util/testutil.h"
|
#include "test_util/testutil.h"
|
||||||
#include "util/string_util.h"
|
#include "util/string_util.h"
|
||||||
@ -206,7 +207,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, kNullUniqueId64x2);
|
||||||
|
|
||||||
mutex_.Lock();
|
mutex_.Lock();
|
||||||
EXPECT_OK(
|
EXPECT_OK(
|
||||||
@ -360,8 +361,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();
|
||||||
@ -1254,13 +1255,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));
|
||||||
@ -1288,6 +1290,16 @@ 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");
|
||||||
|
deserialized_tmp.status.PermitUncheckedError();
|
||||||
|
}
|
||||||
|
|
||||||
// Test unknown field
|
// Test unknown field
|
||||||
CompactionServiceResult deserialized2;
|
CompactionServiceResult deserialized2;
|
||||||
output.clear();
|
output.clear();
|
||||||
|
@ -12,6 +12,7 @@
|
|||||||
#include "db/compaction/compaction_picker_level.h"
|
#include "db/compaction/compaction_picker_level.h"
|
||||||
#include "db/compaction/compaction_picker_universal.h"
|
#include "db/compaction/compaction_picker_universal.h"
|
||||||
#include "db/compaction/file_pri.h"
|
#include "db/compaction/file_pri.h"
|
||||||
|
#include "table/unique_id_impl.h"
|
||||||
#include "test_util/testharness.h"
|
#include "test_util/testharness.h"
|
||||||
#include "test_util/testutil.h"
|
#include "test_util/testutil.h"
|
||||||
#include "util/string_util.h"
|
#include "util/string_util.h"
|
||||||
@ -115,7 +116,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, kNullUniqueId64x2);
|
||||||
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;
|
||||||
|
@ -288,6 +288,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::VerifySstUniqueIds::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;
|
||||||
@ -312,6 +322,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) {
|
||||||
|
@ -1974,6 +1974,11 @@ class DBImpl : public DB {
|
|||||||
IOStatus CreateWAL(uint64_t log_file_num, uint64_t recycle_log_number,
|
IOStatus CreateWAL(uint64_t log_file_num, uint64_t recycle_log_number,
|
||||||
size_t preallocate_block_size, log::Writer** new_log);
|
size_t preallocate_block_size, log::Writer** new_log);
|
||||||
|
|
||||||
|
// Verify SST file unique id between Manifest and table properties to make
|
||||||
|
// sure they're the same. Currently only used during DB open when
|
||||||
|
// `verify_sst_unique_id_in_manifest = true`.
|
||||||
|
Status VerifySstUniqueIdInManifest();
|
||||||
|
|
||||||
// Validate self-consistency of DB options
|
// Validate self-consistency of DB options
|
||||||
static Status ValidateOptions(const DBOptions& db_options);
|
static Status ValidateOptions(const DBOptions& db_options);
|
||||||
// Validate self-consistency of DB options and its consistency with cf options
|
// Validate self-consistency of DB options and its consistency with cf options
|
||||||
|
@ -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,
|
||||||
|
@ -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(),
|
||||||
|
@ -518,6 +518,12 @@ Status DBImpl::Recover(
|
|||||||
if (!s.ok()) {
|
if (!s.ok()) {
|
||||||
return s;
|
return s;
|
||||||
}
|
}
|
||||||
|
if (immutable_db_options_.verify_sst_unique_id_in_manifest) {
|
||||||
|
s = VerifySstUniqueIdInManifest();
|
||||||
|
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();
|
||||||
@ -698,6 +704,25 @@ Status DBImpl::Recover(
|
|||||||
return s;
|
return s;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Status DBImpl::VerifySstUniqueIdInManifest() {
|
||||||
|
mutex_.AssertHeld();
|
||||||
|
ROCKS_LOG_INFO(
|
||||||
|
immutable_db_options_.info_log,
|
||||||
|
"Verifying SST unique id between MANIFEST and SST file table properties");
|
||||||
|
Status status;
|
||||||
|
for (auto cfd : *versions_->GetColumnFamilySet()) {
|
||||||
|
if (!cfd->IsDropped()) {
|
||||||
|
auto version = cfd->current();
|
||||||
|
version->Ref();
|
||||||
|
mutex_.Unlock();
|
||||||
|
status = version->VerifySstUniqueIds();
|
||||||
|
mutex_.Lock();
|
||||||
|
version->Unref();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return status;
|
||||||
|
}
|
||||||
|
|
||||||
Status DBImpl::PersistentStatsProcessFormatVersion() {
|
Status DBImpl::PersistentStatsProcessFormatVersion() {
|
||||||
mutex_.AssertHeld();
|
mutex_.AssertHeld();
|
||||||
Status s;
|
Status s;
|
||||||
@ -1498,13 +1523,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);
|
||||||
|
@ -772,12 +772,19 @@ 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, which is to make sure the unique id is unique from
|
||||||
|
// the remote compactors. Because the id is generated from db_id,
|
||||||
|
// db_session_id and orig_file_number, unlike the local compaction, remote
|
||||||
|
// compaction cannot guarantee the uniqueness of orig_file_number, the file
|
||||||
|
// number is only assigned when compaction is done.
|
||||||
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();
|
||||||
|
@ -95,6 +95,22 @@ TEST_F(DBOptionsTest, ImmutableTrackAndVerifyWalsInManifest) {
|
|||||||
ASSERT_FALSE(s.ok());
|
ASSERT_FALSE(s.ok());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
TEST_F(DBOptionsTest, ImmutableVerifySstUniqueIdInManifest) {
|
||||||
|
Options options;
|
||||||
|
options.env = env_;
|
||||||
|
options.verify_sst_unique_id_in_manifest = true;
|
||||||
|
|
||||||
|
ImmutableDBOptions db_options(options);
|
||||||
|
ASSERT_TRUE(db_options.verify_sst_unique_id_in_manifest);
|
||||||
|
|
||||||
|
Reopen(options);
|
||||||
|
ASSERT_TRUE(dbfull()->GetDBOptions().verify_sst_unique_id_in_manifest);
|
||||||
|
|
||||||
|
Status s =
|
||||||
|
dbfull()->SetDBOptions({{"verify_sst_unique_id_in_manifest", "false"}});
|
||||||
|
ASSERT_FALSE(s.ok());
|
||||||
|
}
|
||||||
|
|
||||||
// RocksDB lite don't support dynamic options.
|
// RocksDB lite don't support dynamic options.
|
||||||
#ifndef ROCKSDB_LITE
|
#ifndef ROCKSDB_LITE
|
||||||
|
|
||||||
|
@ -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();
|
||||||
|
|
||||||
|
106
db/db_test2.cc
106
db/db_test2.cc
@ -7149,6 +7149,112 @@ 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;
|
||||||
|
});
|
||||||
|
std::atomic_int skipped = 0;
|
||||||
|
SyncPoint::GetInstance()->SetCallBack("Version::VerifySstUniqueIds::Skipped",
|
||||||
|
[&](void* /*arg*/) { skipped++; });
|
||||||
|
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(skipped, 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 with verification should fail
|
||||||
|
options.verify_sst_unique_id_in_manifest = true;
|
||||||
|
skipped = 0;
|
||||||
|
Reopen(options);
|
||||||
|
ASSERT_EQ(skipped, 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 with 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_);
|
||||||
|
@ -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 {
|
||||||
|
@ -1866,6 +1866,63 @@ 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 = "";
|
||||||
|
});
|
||||||
|
std::atomic_int skipped = 0;
|
||||||
|
SyncPoint::GetInstance()->SetCallBack("Version::VerifySstUniqueIds::Skipped",
|
||||||
|
[&](void* /*arg*/) { skipped++; });
|
||||||
|
SyncPoint::GetInstance()->EnableProcessing();
|
||||||
|
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;
|
||||||
|
ASSERT_EQ(skipped, 0);
|
||||||
|
Reopen(options);
|
||||||
|
// only one sst file is not verified because of missing unique_id
|
||||||
|
ASSERT_EQ(skipped, 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),
|
||||||
|
@ -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"
|
||||||
|
|
||||||
@ -449,8 +450,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);
|
||||||
}
|
}
|
||||||
@ -730,6 +731,16 @@ 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());
|
||||||
|
file_to_ingest->unique_id = kNullUniqueId64x2;
|
||||||
|
}
|
||||||
|
|
||||||
return status;
|
return status;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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 {
|
||||||
|
@ -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"
|
||||||
@ -989,7 +988,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));
|
||||||
}
|
}
|
||||||
|
@ -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 {
|
||||||
@ -159,7 +160,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()) {
|
||||||
@ -288,6 +289,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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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) {
|
||||||
|
13
db/repair.cc
13
db/repair.cc
@ -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);
|
||||||
|
@ -43,6 +43,23 @@ class RepairTest : public DBTestBase {
|
|||||||
}
|
}
|
||||||
return s;
|
return s;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void ReopenWithSstIdVerify() {
|
||||||
|
std::atomic_int verify_passed{0};
|
||||||
|
SyncPoint::GetInstance()->SetCallBack(
|
||||||
|
"Version::VerifySstUniqueIds::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
|
||||||
|
@ -12,6 +12,7 @@
|
|||||||
#include "db/version_edit.h"
|
#include "db/version_edit.h"
|
||||||
#include "db/version_set.h"
|
#include "db/version_set.h"
|
||||||
#include "rocksdb/advanced_options.h"
|
#include "rocksdb/advanced_options.h"
|
||||||
|
#include "table/unique_id_impl.h"
|
||||||
#include "test_util/testharness.h"
|
#include "test_util/testharness.h"
|
||||||
#include "test_util/testutil.h"
|
#include "test_util/testutil.h"
|
||||||
#include "util/string_util.h"
|
#include "util/string_util.h"
|
||||||
@ -72,7 +73,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, kNullUniqueId64x2);
|
||||||
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;
|
||||||
@ -128,13 +129,13 @@ class VersionBuilderTest : public testing::Test {
|
|||||||
constexpr SequenceNumber largest_seqno = 300;
|
constexpr SequenceNumber largest_seqno = 300;
|
||||||
constexpr bool marked_for_compaction = false;
|
constexpr bool marked_for_compaction = false;
|
||||||
|
|
||||||
edit->AddFile(level, table_file_number, path_id, file_size,
|
edit->AddFile(
|
||||||
GetInternalKey(smallest), GetInternalKey(largest),
|
level, table_file_number, path_id, file_size, GetInternalKey(smallest),
|
||||||
smallest_seqno, largest_seqno, marked_for_compaction,
|
GetInternalKey(largest), smallest_seqno, largest_seqno,
|
||||||
Temperature::kUnknown, blob_file_number,
|
marked_for_compaction, Temperature::kUnknown, blob_file_number,
|
||||||
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
||||||
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
||||||
kDisableUserTimestamp, kDisableUserTimestamp);
|
kDisableUserTimestamp, kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
}
|
}
|
||||||
|
|
||||||
void UpdateVersionStorageInfo(VersionStorageInfo* vstorage) {
|
void UpdateVersionStorageInfo(VersionStorageInfo* vstorage) {
|
||||||
@ -175,12 +176,12 @@ TEST_F(VersionBuilderTest, ApplyAndSaveTo) {
|
|||||||
UpdateVersionStorageInfo();
|
UpdateVersionStorageInfo();
|
||||||
|
|
||||||
VersionEdit version_edit;
|
VersionEdit version_edit;
|
||||||
version_edit.AddFile(2, 666, 0, 100U, GetInternalKey("301"),
|
version_edit.AddFile(
|
||||||
GetInternalKey("350"), 200, 200, false,
|
2, 666, 0, 100U, GetInternalKey("301"), GetInternalKey("350"), 200, 200,
|
||||||
Temperature::kUnknown, kInvalidBlobFileNumber,
|
false, Temperature::kUnknown, kInvalidBlobFileNumber,
|
||||||
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
||||||
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
||||||
kDisableUserTimestamp, kDisableUserTimestamp);
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
version_edit.DeleteFile(3, 27U);
|
version_edit.DeleteFile(3, 27U);
|
||||||
|
|
||||||
EnvOptions env_options;
|
EnvOptions env_options;
|
||||||
@ -219,12 +220,12 @@ TEST_F(VersionBuilderTest, ApplyAndSaveToDynamic) {
|
|||||||
UpdateVersionStorageInfo();
|
UpdateVersionStorageInfo();
|
||||||
|
|
||||||
VersionEdit version_edit;
|
VersionEdit version_edit;
|
||||||
version_edit.AddFile(3, 666, 0, 100U, GetInternalKey("301"),
|
version_edit.AddFile(
|
||||||
GetInternalKey("350"), 200, 200, false,
|
3, 666, 0, 100U, GetInternalKey("301"), GetInternalKey("350"), 200, 200,
|
||||||
Temperature::kUnknown, kInvalidBlobFileNumber,
|
false, Temperature::kUnknown, kInvalidBlobFileNumber,
|
||||||
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
||||||
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
||||||
kDisableUserTimestamp, kDisableUserTimestamp);
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
version_edit.DeleteFile(0, 1U);
|
version_edit.DeleteFile(0, 1U);
|
||||||
version_edit.DeleteFile(0, 88U);
|
version_edit.DeleteFile(0, 88U);
|
||||||
|
|
||||||
@ -266,12 +267,12 @@ TEST_F(VersionBuilderTest, ApplyAndSaveToDynamic2) {
|
|||||||
UpdateVersionStorageInfo();
|
UpdateVersionStorageInfo();
|
||||||
|
|
||||||
VersionEdit version_edit;
|
VersionEdit version_edit;
|
||||||
version_edit.AddFile(4, 666, 0, 100U, GetInternalKey("301"),
|
version_edit.AddFile(
|
||||||
GetInternalKey("350"), 200, 200, false,
|
4, 666, 0, 100U, GetInternalKey("301"), GetInternalKey("350"), 200, 200,
|
||||||
Temperature::kUnknown, kInvalidBlobFileNumber,
|
false, Temperature::kUnknown, kInvalidBlobFileNumber,
|
||||||
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
||||||
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
||||||
kDisableUserTimestamp, kDisableUserTimestamp);
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
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);
|
||||||
@ -303,36 +304,36 @@ TEST_F(VersionBuilderTest, ApplyMultipleAndSaveTo) {
|
|||||||
UpdateVersionStorageInfo();
|
UpdateVersionStorageInfo();
|
||||||
|
|
||||||
VersionEdit version_edit;
|
VersionEdit version_edit;
|
||||||
version_edit.AddFile(2, 666, 0, 100U, GetInternalKey("301"),
|
version_edit.AddFile(
|
||||||
GetInternalKey("350"), 200, 200, false,
|
2, 666, 0, 100U, GetInternalKey("301"), GetInternalKey("350"), 200, 200,
|
||||||
Temperature::kUnknown, kInvalidBlobFileNumber,
|
false, Temperature::kUnknown, kInvalidBlobFileNumber,
|
||||||
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
||||||
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
||||||
kDisableUserTimestamp, kDisableUserTimestamp);
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
version_edit.AddFile(2, 676, 0, 100U, GetInternalKey("401"),
|
version_edit.AddFile(
|
||||||
GetInternalKey("450"), 200, 200, false,
|
2, 676, 0, 100U, GetInternalKey("401"), GetInternalKey("450"), 200, 200,
|
||||||
Temperature::kUnknown, kInvalidBlobFileNumber,
|
false, Temperature::kUnknown, kInvalidBlobFileNumber,
|
||||||
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
||||||
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
||||||
kDisableUserTimestamp, kDisableUserTimestamp);
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
version_edit.AddFile(2, 636, 0, 100U, GetInternalKey("601"),
|
version_edit.AddFile(
|
||||||
GetInternalKey("650"), 200, 200, false,
|
2, 636, 0, 100U, GetInternalKey("601"), GetInternalKey("650"), 200, 200,
|
||||||
Temperature::kUnknown, kInvalidBlobFileNumber,
|
false, Temperature::kUnknown, kInvalidBlobFileNumber,
|
||||||
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
||||||
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
||||||
kDisableUserTimestamp, kDisableUserTimestamp);
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
version_edit.AddFile(2, 616, 0, 100U, GetInternalKey("501"),
|
version_edit.AddFile(
|
||||||
GetInternalKey("550"), 200, 200, false,
|
2, 616, 0, 100U, GetInternalKey("501"), GetInternalKey("550"), 200, 200,
|
||||||
Temperature::kUnknown, kInvalidBlobFileNumber,
|
false, Temperature::kUnknown, kInvalidBlobFileNumber,
|
||||||
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
||||||
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
||||||
kDisableUserTimestamp, kDisableUserTimestamp);
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
version_edit.AddFile(2, 606, 0, 100U, GetInternalKey("701"),
|
version_edit.AddFile(
|
||||||
GetInternalKey("750"), 200, 200, false,
|
2, 606, 0, 100U, GetInternalKey("701"), GetInternalKey("750"), 200, 200,
|
||||||
Temperature::kUnknown, kInvalidBlobFileNumber,
|
false, Temperature::kUnknown, kInvalidBlobFileNumber,
|
||||||
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
||||||
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
||||||
kDisableUserTimestamp, kDisableUserTimestamp);
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
|
|
||||||
EnvOptions env_options;
|
EnvOptions env_options;
|
||||||
constexpr TableCache* table_cache = nullptr;
|
constexpr TableCache* table_cache = nullptr;
|
||||||
@ -367,53 +368,53 @@ TEST_F(VersionBuilderTest, ApplyDeleteAndSaveTo) {
|
|||||||
kCompactionStyleLevel, nullptr, false);
|
kCompactionStyleLevel, nullptr, false);
|
||||||
|
|
||||||
VersionEdit version_edit;
|
VersionEdit version_edit;
|
||||||
version_edit.AddFile(2, 666, 0, 100U, GetInternalKey("301"),
|
version_edit.AddFile(
|
||||||
GetInternalKey("350"), 200, 200, false,
|
2, 666, 0, 100U, GetInternalKey("301"), GetInternalKey("350"), 200, 200,
|
||||||
Temperature::kUnknown, kInvalidBlobFileNumber,
|
false, Temperature::kUnknown, kInvalidBlobFileNumber,
|
||||||
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
||||||
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
||||||
kDisableUserTimestamp, kDisableUserTimestamp);
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
version_edit.AddFile(2, 676, 0, 100U, GetInternalKey("401"),
|
version_edit.AddFile(
|
||||||
GetInternalKey("450"), 200, 200, false,
|
2, 676, 0, 100U, GetInternalKey("401"), GetInternalKey("450"), 200, 200,
|
||||||
Temperature::kUnknown, kInvalidBlobFileNumber,
|
false, Temperature::kUnknown, kInvalidBlobFileNumber,
|
||||||
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
||||||
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
||||||
kDisableUserTimestamp, kDisableUserTimestamp);
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
version_edit.AddFile(2, 636, 0, 100U, GetInternalKey("601"),
|
version_edit.AddFile(
|
||||||
GetInternalKey("650"), 200, 200, false,
|
2, 636, 0, 100U, GetInternalKey("601"), GetInternalKey("650"), 200, 200,
|
||||||
Temperature::kUnknown, kInvalidBlobFileNumber,
|
false, Temperature::kUnknown, kInvalidBlobFileNumber,
|
||||||
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
||||||
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
||||||
kDisableUserTimestamp, kDisableUserTimestamp);
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
version_edit.AddFile(2, 616, 0, 100U, GetInternalKey("501"),
|
version_edit.AddFile(
|
||||||
GetInternalKey("550"), 200, 200, false,
|
2, 616, 0, 100U, GetInternalKey("501"), GetInternalKey("550"), 200, 200,
|
||||||
Temperature::kUnknown, kInvalidBlobFileNumber,
|
false, Temperature::kUnknown, kInvalidBlobFileNumber,
|
||||||
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
||||||
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
||||||
kDisableUserTimestamp, kDisableUserTimestamp);
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
version_edit.AddFile(2, 606, 0, 100U, GetInternalKey("701"),
|
version_edit.AddFile(
|
||||||
GetInternalKey("750"), 200, 200, false,
|
2, 606, 0, 100U, GetInternalKey("701"), GetInternalKey("750"), 200, 200,
|
||||||
Temperature::kUnknown, kInvalidBlobFileNumber,
|
false, Temperature::kUnknown, kInvalidBlobFileNumber,
|
||||||
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
||||||
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
||||||
kDisableUserTimestamp, kDisableUserTimestamp);
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
ASSERT_OK(version_builder.Apply(&version_edit));
|
ASSERT_OK(version_builder.Apply(&version_edit));
|
||||||
|
|
||||||
VersionEdit version_edit2;
|
VersionEdit version_edit2;
|
||||||
version_edit.AddFile(2, 808, 0, 100U, GetInternalKey("901"),
|
version_edit.AddFile(
|
||||||
GetInternalKey("950"), 200, 200, false,
|
2, 808, 0, 100U, GetInternalKey("901"), GetInternalKey("950"), 200, 200,
|
||||||
Temperature::kUnknown, kInvalidBlobFileNumber,
|
false, Temperature::kUnknown, kInvalidBlobFileNumber,
|
||||||
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
||||||
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
||||||
kDisableUserTimestamp, kDisableUserTimestamp);
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
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(
|
||||||
GetInternalKey("850"), 200, 200, false,
|
2, 806, 0, 100U, GetInternalKey("801"), GetInternalKey("850"), 200, 200,
|
||||||
Temperature::kUnknown, kInvalidBlobFileNumber,
|
false, Temperature::kUnknown, kInvalidBlobFileNumber,
|
||||||
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
||||||
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
||||||
kDisableUserTimestamp, kDisableUserTimestamp);
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
|
|
||||||
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 +526,7 @@ TEST_F(VersionBuilderTest, ApplyFileDeletionAndAddition) {
|
|||||||
kInvalidBlobFileNumber, kUnknownOldestAncesterTime,
|
kInvalidBlobFileNumber, kUnknownOldestAncesterTime,
|
||||||
kUnknownFileCreationTime, kUnknownFileChecksum,
|
kUnknownFileCreationTime, kUnknownFileChecksum,
|
||||||
kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
||||||
kDisableUserTimestamp);
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
|
|
||||||
ASSERT_OK(builder.Apply(&addition));
|
ASSERT_OK(builder.Apply(&addition));
|
||||||
|
|
||||||
@ -575,7 +576,7 @@ TEST_F(VersionBuilderTest, ApplyFileAdditionAlreadyInBase) {
|
|||||||
Temperature::kUnknown, kInvalidBlobFileNumber,
|
Temperature::kUnknown, kInvalidBlobFileNumber,
|
||||||
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
||||||
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
||||||
kDisableUserTimestamp, kDisableUserTimestamp);
|
kDisableUserTimestamp, kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
|
|
||||||
const Status s = builder.Apply(&edit);
|
const Status s = builder.Apply(&edit);
|
||||||
ASSERT_TRUE(s.IsCorruption());
|
ASSERT_TRUE(s.IsCorruption());
|
||||||
@ -612,7 +613,7 @@ TEST_F(VersionBuilderTest, ApplyFileAdditionAlreadyApplied) {
|
|||||||
kInvalidBlobFileNumber, kUnknownOldestAncesterTime,
|
kInvalidBlobFileNumber, kUnknownOldestAncesterTime,
|
||||||
kUnknownFileCreationTime, kUnknownFileChecksum,
|
kUnknownFileCreationTime, kUnknownFileChecksum,
|
||||||
kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
||||||
kDisableUserTimestamp);
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
|
|
||||||
ASSERT_OK(builder.Apply(&edit));
|
ASSERT_OK(builder.Apply(&edit));
|
||||||
|
|
||||||
@ -620,13 +621,13 @@ TEST_F(VersionBuilderTest, ApplyFileAdditionAlreadyApplied) {
|
|||||||
|
|
||||||
constexpr int new_level = 2;
|
constexpr int new_level = 2;
|
||||||
|
|
||||||
other_edit.AddFile(new_level, file_number, path_id, file_size,
|
other_edit.AddFile(
|
||||||
GetInternalKey(smallest), GetInternalKey(largest),
|
new_level, file_number, path_id, file_size, GetInternalKey(smallest),
|
||||||
smallest_seqno, largest_seqno, marked_for_compaction,
|
GetInternalKey(largest), smallest_seqno, largest_seqno,
|
||||||
Temperature::kUnknown, kInvalidBlobFileNumber,
|
marked_for_compaction, Temperature::kUnknown, kInvalidBlobFileNumber,
|
||||||
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
||||||
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
||||||
kDisableUserTimestamp, kDisableUserTimestamp);
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
|
|
||||||
const Status s = builder.Apply(&other_edit);
|
const Status s = builder.Apply(&other_edit);
|
||||||
ASSERT_TRUE(s.IsCorruption());
|
ASSERT_TRUE(s.IsCorruption());
|
||||||
@ -657,13 +658,13 @@ TEST_F(VersionBuilderTest, ApplyFileAdditionAndDeletion) {
|
|||||||
|
|
||||||
VersionEdit addition;
|
VersionEdit addition;
|
||||||
|
|
||||||
addition.AddFile(level, file_number, path_id, file_size,
|
addition.AddFile(
|
||||||
GetInternalKey(smallest), GetInternalKey(largest),
|
level, file_number, path_id, file_size, GetInternalKey(smallest),
|
||||||
smallest_seqno, largest_seqno, marked_for_compaction,
|
GetInternalKey(largest), smallest_seqno, largest_seqno,
|
||||||
Temperature::kUnknown, kInvalidBlobFileNumber,
|
marked_for_compaction, Temperature::kUnknown, kInvalidBlobFileNumber,
|
||||||
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
||||||
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
kUnknownFileChecksum, kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
||||||
kDisableUserTimestamp, kDisableUserTimestamp);
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
|
|
||||||
ASSERT_OK(builder.Apply(&addition));
|
ASSERT_OK(builder.Apply(&addition));
|
||||||
|
|
||||||
@ -1227,12 +1228,13 @@ TEST_F(VersionBuilderTest, SaveBlobFilesToConcurrentJobs) {
|
|||||||
constexpr uint64_t total_blob_count = 234;
|
constexpr uint64_t total_blob_count = 234;
|
||||||
constexpr uint64_t total_blob_bytes = 1 << 22;
|
constexpr uint64_t total_blob_bytes = 1 << 22;
|
||||||
|
|
||||||
edit.AddFile(
|
edit.AddFile(level, table_file_number, path_id, file_size,
|
||||||
level, table_file_number, path_id, file_size, GetInternalKey(smallest),
|
GetInternalKey(smallest), GetInternalKey(largest),
|
||||||
GetInternalKey(largest), smallest_seqno, largest_seqno,
|
smallest_seqno, largest_seqno, marked_for_compaction,
|
||||||
marked_for_compaction, Temperature::kUnknown, blob_file_number,
|
Temperature::kUnknown, blob_file_number,
|
||||||
kUnknownOldestAncesterTime, kUnknownFileCreationTime, checksum_value,
|
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
||||||
checksum_method, kDisableUserTimestamp, kDisableUserTimestamp);
|
checksum_value, checksum_method, kDisableUserTimestamp,
|
||||||
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
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 +1322,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, kNullUniqueId64x2);
|
||||||
|
|
||||||
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 +1332,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, kNullUniqueId64x2);
|
||||||
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 +1554,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, kNullUniqueId64x2);
|
||||||
|
|
||||||
// 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 +1565,7 @@ TEST_F(VersionBuilderTest, MaintainLinkedSstsForBlobFiles) {
|
|||||||
Temperature::kUnknown, kInvalidBlobFileNumber, kUnknownOldestAncesterTime,
|
Temperature::kUnknown, kInvalidBlobFileNumber, kUnknownOldestAncesterTime,
|
||||||
kUnknownFileCreationTime, kUnknownFileChecksum,
|
kUnknownFileCreationTime, kUnknownFileChecksum,
|
||||||
kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
||||||
kDisableUserTimestamp);
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
|
|
||||||
// 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 +1588,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, kNullUniqueId64x2);
|
||||||
|
|
||||||
// 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 +1600,7 @@ TEST_F(VersionBuilderTest, MaintainLinkedSstsForBlobFiles) {
|
|||||||
Temperature::kUnknown, kInvalidBlobFileNumber,
|
Temperature::kUnknown, kInvalidBlobFileNumber,
|
||||||
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
kUnknownOldestAncesterTime, kUnknownFileCreationTime,
|
||||||
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
kUnknownFileChecksum, kUnknownFileChecksumFuncName,
|
||||||
kDisableUserTimestamp, kDisableUserTimestamp);
|
kDisableUserTimestamp, kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
|
|
||||||
// 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 +1614,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, kNullUniqueId64x2);
|
||||||
|
|
||||||
VersionEdit edit2;
|
VersionEdit edit2;
|
||||||
|
|
||||||
|
@ -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 (unique_id != kNullUniqueId64x2) {
|
||||||
|
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,12 @@ 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()) {
|
||||||
|
f.unique_id = kNullUniqueId64x2;
|
||||||
|
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 +834,11 @@ 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 != kNullUniqueId64x2) {
|
||||||
|
r.append(" unique_id(internal): ");
|
||||||
|
UniqueId64x2 id = f.unique_id;
|
||||||
|
r.append(InternalUniqueIdToHumanString(&id));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
for (const auto& blob_file_addition : blob_file_additions_) {
|
for (const auto& blob_file_addition : blob_file_additions_) {
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
|
@ -11,6 +11,7 @@
|
|||||||
|
|
||||||
#include "db/blob/blob_index.h"
|
#include "db/blob/blob_index.h"
|
||||||
#include "rocksdb/advanced_options.h"
|
#include "rocksdb/advanced_options.h"
|
||||||
|
#include "table/unique_id_impl.h"
|
||||||
#include "test_util/sync_point.h"
|
#include "test_util/sync_point.h"
|
||||||
#include "test_util/testharness.h"
|
#include "test_util/testharness.h"
|
||||||
#include "test_util/testutil.h"
|
#include "test_util/testutil.h"
|
||||||
@ -43,7 +44,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", kNullUniqueId64x2);
|
||||||
edit.DeleteFile(4, kBig + 700 + i);
|
edit.DeleteFile(4, kBig + 700 + i);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -62,26 +63,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");
|
kNullUniqueId64x2);
|
||||||
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");
|
kNullUniqueId64x2);
|
||||||
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", kNullUniqueId64x2);
|
||||||
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");
|
kNullUniqueId64x2);
|
||||||
;
|
|
||||||
|
|
||||||
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");
|
kNullUniqueId64x2);
|
||||||
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, kNullUniqueId64x2);
|
||||||
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, kNullUniqueId64x2);
|
||||||
|
|
||||||
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, kNullUniqueId64x2);
|
||||||
std::string buffer;
|
std::string buffer;
|
||||||
ASSERT_TRUE(!edit.EncodeTo(&buffer));
|
ASSERT_TRUE(!edit.EncodeTo(&buffer));
|
||||||
}
|
}
|
||||||
|
@ -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,40 @@ void Version::GetCreationTimeOfOldestFile(uint64_t* creation_time) {
|
|||||||
*creation_time = oldest_time;
|
*creation_time = oldest_time;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Status Version::VerifySstUniqueIds() const {
|
||||||
|
for (int level = 0; level < storage_info_.num_non_empty_levels_; level++) {
|
||||||
|
for (FileMetaData* meta : storage_info_.LevelFiles(level)) {
|
||||||
|
if (meta->unique_id != kNullUniqueId64x2) {
|
||||||
|
std::shared_ptr<const TableProperties> props;
|
||||||
|
Status s =
|
||||||
|
GetTableProperties(&props, meta); // may open the file if it's not
|
||||||
|
if (!s.ok()) {
|
||||||
|
return s;
|
||||||
|
}
|
||||||
|
UniqueId64x2 id;
|
||||||
|
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: "
|
||||||
|
<< InternalUniqueIdToHumanString(&(meta->unique_id)) << ", ";
|
||||||
|
if (s.ok()) {
|
||||||
|
oss << "Table Properties: " << InternalUniqueIdToHumanString(&id);
|
||||||
|
} else {
|
||||||
|
oss << "Failed to get Table Properties: " << s.ToString();
|
||||||
|
}
|
||||||
|
return Status::Corruption("VersionSet", oss.str());
|
||||||
|
}
|
||||||
|
TEST_SYNC_POINT_CALLBACK("Version::VerifySstUniqueIds::Passed", &id);
|
||||||
|
} else {
|
||||||
|
TEST_SYNC_POINT_CALLBACK("Version::VerifySstUniqueIds::Skipped", meta);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
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 +5527,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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -851,6 +851,8 @@ class Version {
|
|||||||
|
|
||||||
const MutableCFOptions& GetMutableCFOptions() { return mutable_cf_options_; }
|
const MutableCFOptions& GetMutableCFOptions() { return mutable_cf_options_; }
|
||||||
|
|
||||||
|
Status VerifySstUniqueIds() const;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
Env* env_;
|
Env* env_;
|
||||||
SystemClock* clock_;
|
SystemClock* clock_;
|
||||||
|
@ -18,6 +18,7 @@
|
|||||||
#include "rocksdb/file_system.h"
|
#include "rocksdb/file_system.h"
|
||||||
#include "table/block_based/block_based_table_factory.h"
|
#include "table/block_based/block_based_table_factory.h"
|
||||||
#include "table/mock_table.h"
|
#include "table/mock_table.h"
|
||||||
|
#include "table/unique_id_impl.h"
|
||||||
#include "test_util/testharness.h"
|
#include "test_util/testharness.h"
|
||||||
#include "test_util/testutil.h"
|
#include "test_util/testutil.h"
|
||||||
#include "util/string_util.h"
|
#include "util/string_util.h"
|
||||||
@ -49,7 +50,7 @@ class GenerateLevelFilesBriefTest : public testing::Test {
|
|||||||
kInvalidBlobFileNumber, kUnknownOldestAncesterTime,
|
kInvalidBlobFileNumber, kUnknownOldestAncesterTime,
|
||||||
kUnknownFileCreationTime, kUnknownFileChecksum,
|
kUnknownFileCreationTime, kUnknownFileChecksum,
|
||||||
kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
||||||
kDisableUserTimestamp);
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
files_.push_back(f);
|
files_.push_back(f);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -158,7 +159,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, kNullUniqueId64x2);
|
||||||
f->compensated_file_size = file_size;
|
f->compensated_file_size = file_size;
|
||||||
vstorage_.AddFile(level, f);
|
vstorage_.AddFile(level, f);
|
||||||
}
|
}
|
||||||
@ -3222,11 +3223,11 @@ 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,
|
file_metas->emplace_back(
|
||||||
ikey, 0, 0, false, Temperature::kUnknown, 0, 0,
|
file_num, /*file_path_id=*/0, file_size, ikey, ikey, 0, 0, false,
|
||||||
0, kUnknownFileChecksum,
|
Temperature::kUnknown, 0, 0, 0, kUnknownFileChecksum,
|
||||||
kUnknownFileChecksumFuncName,
|
kUnknownFileChecksumFuncName, kDisableUserTimestamp,
|
||||||
kDisableUserTimestamp, kDisableUserTimestamp);
|
kDisableUserTimestamp, kNullUniqueId64x2);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -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, kNullUniqueId64x2);
|
||||||
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, kNullUniqueId64x2);
|
||||||
added_files.emplace_back(0, meta);
|
added_files.emplace_back(0, meta);
|
||||||
}
|
}
|
||||||
WriteFileAdditionAndDeletionToManifest(
|
WriteFileAdditionAndDeletionToManifest(
|
||||||
|
@ -292,6 +292,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;
|
||||||
|
@ -952,4 +952,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
|
||||||
|
@ -2315,6 +2315,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");
|
||||||
}
|
}
|
||||||
@ -2941,6 +2943,8 @@ void InitializeOptionsFromFlags(
|
|||||||
options.level_compaction_dynamic_level_bytes =
|
options.level_compaction_dynamic_level_bytes =
|
||||||
FLAGS_level_compaction_dynamic_level_bytes;
|
FLAGS_level_compaction_dynamic_level_bytes;
|
||||||
options.track_and_verify_wals_in_manifest = true;
|
options.track_and_verify_wals_in_manifest = true;
|
||||||
|
options.verify_sst_unique_id_in_manifest =
|
||||||
|
FLAGS_verify_sst_unique_id_in_manifest;
|
||||||
|
|
||||||
// Integrated BlobDB
|
// Integrated BlobDB
|
||||||
options.enable_blob_files = FLAGS_enable_blob_files;
|
options.enable_blob_files = FLAGS_enable_blob_files;
|
||||||
|
@ -492,6 +492,23 @@ 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.
|
||||||
|
//
|
||||||
|
// Note:
|
||||||
|
// 1. if enabled, it opens every SST files during DB open to read the unique
|
||||||
|
// id from SST properties, so it's recommended to have `max_open_files=-1`
|
||||||
|
// to pre-open the SST files before the verification.
|
||||||
|
// 2. existing SST files won't have its unique_id tracked in MANIFEST, then
|
||||||
|
// verification will be skipped.
|
||||||
|
//
|
||||||
|
// 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
|
||||||
|
@ -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
|
||||||
|
@ -228,6 +228,10 @@ static std::unordered_map<std::string, OptionTypeInfo>
|
|||||||
track_and_verify_wals_in_manifest),
|
track_and_verify_wals_in_manifest),
|
||||||
OptionType::kBoolean, OptionVerificationType::kNormal,
|
OptionType::kBoolean, OptionVerificationType::kNormal,
|
||||||
OptionTypeFlags::kNone}},
|
OptionTypeFlags::kNone}},
|
||||||
|
{"verify_sst_unique_id_in_manifest",
|
||||||
|
{offsetof(struct ImmutableDBOptions, verify_sst_unique_id_in_manifest),
|
||||||
|
OptionType::kBoolean, OptionVerificationType::kNormal,
|
||||||
|
OptionTypeFlags::kNone}},
|
||||||
{"skip_log_error_on_recovery",
|
{"skip_log_error_on_recovery",
|
||||||
{0, OptionType::kBoolean, OptionVerificationType::kDeprecated,
|
{0, OptionType::kBoolean, OptionVerificationType::kDeprecated,
|
||||||
OptionTypeFlags::kNone}},
|
OptionTypeFlags::kNone}},
|
||||||
@ -680,6 +684,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),
|
||||||
@ -775,6 +781,8 @@ void ImmutableDBOptions::Dump(Logger* log) const {
|
|||||||
" "
|
" "
|
||||||
"Options.track_and_verify_wals_in_manifest: %d",
|
"Options.track_and_verify_wals_in_manifest: %d",
|
||||||
track_and_verify_wals_in_manifest);
|
track_and_verify_wals_in_manifest);
|
||||||
|
ROCKS_LOG_HEADER(log, " Options.verify_sst_unique_id_in_manifest: %d",
|
||||||
|
verify_sst_unique_id_in_manifest);
|
||||||
ROCKS_LOG_HEADER(log, " Options.env: %p",
|
ROCKS_LOG_HEADER(log, " Options.env: %p",
|
||||||
env);
|
env);
|
||||||
ROCKS_LOG_HEADER(log, " Options.fs: %s",
|
ROCKS_LOG_HEADER(log, " Options.fs: %s",
|
||||||
|
@ -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;
|
||||||
|
@ -70,6 +70,8 @@ DBOptions BuildDBOptions(const ImmutableDBOptions& immutable_db_options,
|
|||||||
immutable_db_options.flush_verify_memtable_count;
|
immutable_db_options.flush_verify_memtable_count;
|
||||||
options.track_and_verify_wals_in_manifest =
|
options.track_and_verify_wals_in_manifest =
|
||||||
immutable_db_options.track_and_verify_wals_in_manifest;
|
immutable_db_options.track_and_verify_wals_in_manifest;
|
||||||
|
options.verify_sst_unique_id_in_manifest =
|
||||||
|
immutable_db_options.verify_sst_unique_id_in_manifest;
|
||||||
options.env = immutable_db_options.env;
|
options.env = immutable_db_options.env;
|
||||||
options.rate_limiter = immutable_db_options.rate_limiter;
|
options.rate_limiter = immutable_db_options.rate_limiter;
|
||||||
options.sst_file_manager = immutable_db_options.sst_file_manager;
|
options.sst_file_manager = immutable_db_options.sst_file_manager;
|
||||||
|
@ -304,6 +304,7 @@ TEST_F(OptionsSettableTest, DBOptionsAllFieldsSettable) {
|
|||||||
"paranoid_checks=true;"
|
"paranoid_checks=true;"
|
||||||
"flush_verify_memtable_count=true;"
|
"flush_verify_memtable_count=true;"
|
||||||
"track_and_verify_wals_in_manifest=true;"
|
"track_and_verify_wals_in_manifest=true;"
|
||||||
|
"verify_sst_unique_id_in_manifest=true;"
|
||||||
"is_fd_close_on_exec=false;"
|
"is_fd_close_on_exec=false;"
|
||||||
"bytes_per_sync=4295013613;"
|
"bytes_per_sync=4295013613;"
|
||||||
"strict_bytes_per_sync=true;"
|
"strict_bytes_per_sync=true;"
|
||||||
|
@ -118,6 +118,7 @@ TEST_F(OptionsTest, GetOptionsFromMapTest) {
|
|||||||
{"error_if_exists", "false"},
|
{"error_if_exists", "false"},
|
||||||
{"paranoid_checks", "true"},
|
{"paranoid_checks", "true"},
|
||||||
{"track_and_verify_wals_in_manifest", "true"},
|
{"track_and_verify_wals_in_manifest", "true"},
|
||||||
|
{"verify_sst_unique_id_in_manifest", "true"},
|
||||||
{"max_open_files", "32"},
|
{"max_open_files", "32"},
|
||||||
{"max_total_wal_size", "33"},
|
{"max_total_wal_size", "33"},
|
||||||
{"use_fsync", "true"},
|
{"use_fsync", "true"},
|
||||||
@ -278,6 +279,7 @@ TEST_F(OptionsTest, GetOptionsFromMapTest) {
|
|||||||
ASSERT_EQ(new_db_opt.error_if_exists, false);
|
ASSERT_EQ(new_db_opt.error_if_exists, false);
|
||||||
ASSERT_EQ(new_db_opt.paranoid_checks, true);
|
ASSERT_EQ(new_db_opt.paranoid_checks, true);
|
||||||
ASSERT_EQ(new_db_opt.track_and_verify_wals_in_manifest, true);
|
ASSERT_EQ(new_db_opt.track_and_verify_wals_in_manifest, true);
|
||||||
|
ASSERT_EQ(new_db_opt.verify_sst_unique_id_in_manifest, true);
|
||||||
ASSERT_EQ(new_db_opt.max_open_files, 32);
|
ASSERT_EQ(new_db_opt.max_open_files, 32);
|
||||||
ASSERT_EQ(new_db_opt.max_total_wal_size, static_cast<uint64_t>(33));
|
ASSERT_EQ(new_db_opt.max_total_wal_size, static_cast<uint64_t>(33));
|
||||||
ASSERT_EQ(new_db_opt.use_fsync, true);
|
ASSERT_EQ(new_db_opt.use_fsync, true);
|
||||||
@ -799,6 +801,7 @@ TEST_F(OptionsTest, OldInterfaceTest) {
|
|||||||
{"error_if_exists", "false"},
|
{"error_if_exists", "false"},
|
||||||
{"paranoid_checks", "true"},
|
{"paranoid_checks", "true"},
|
||||||
{"track_and_verify_wals_in_manifest", "true"},
|
{"track_and_verify_wals_in_manifest", "true"},
|
||||||
|
{"verify_sst_unique_id_in_manifest", "true"},
|
||||||
{"max_open_files", "32"},
|
{"max_open_files", "32"},
|
||||||
};
|
};
|
||||||
ASSERT_OK(GetDBOptionsFromMap(base_db_opt, db_options_map, &new_db_opt));
|
ASSERT_OK(GetDBOptionsFromMap(base_db_opt, db_options_map, &new_db_opt));
|
||||||
@ -807,6 +810,7 @@ TEST_F(OptionsTest, OldInterfaceTest) {
|
|||||||
ASSERT_EQ(new_db_opt.error_if_exists, false);
|
ASSERT_EQ(new_db_opt.error_if_exists, false);
|
||||||
ASSERT_EQ(new_db_opt.paranoid_checks, true);
|
ASSERT_EQ(new_db_opt.paranoid_checks, true);
|
||||||
ASSERT_EQ(new_db_opt.track_and_verify_wals_in_manifest, true);
|
ASSERT_EQ(new_db_opt.track_and_verify_wals_in_manifest, true);
|
||||||
|
ASSERT_EQ(new_db_opt.verify_sst_unique_id_in_manifest, true);
|
||||||
ASSERT_EQ(new_db_opt.max_open_files, 32);
|
ASSERT_EQ(new_db_opt.max_open_files, 32);
|
||||||
db_options_map["unknown_option"] = "1";
|
db_options_map["unknown_option"] = "1";
|
||||||
Status s = GetDBOptionsFromMap(base_db_opt, db_options_map, &new_db_opt);
|
Status s = GetDBOptionsFromMap(base_db_opt, db_options_map, &new_db_opt);
|
||||||
@ -2312,6 +2316,7 @@ TEST_F(OptionsOldApiTest, GetOptionsFromMapTest) {
|
|||||||
{"error_if_exists", "false"},
|
{"error_if_exists", "false"},
|
||||||
{"paranoid_checks", "true"},
|
{"paranoid_checks", "true"},
|
||||||
{"track_and_verify_wals_in_manifest", "true"},
|
{"track_and_verify_wals_in_manifest", "true"},
|
||||||
|
{"verify_sst_unique_id_in_manifest", "true"},
|
||||||
{"max_open_files", "32"},
|
{"max_open_files", "32"},
|
||||||
{"max_total_wal_size", "33"},
|
{"max_total_wal_size", "33"},
|
||||||
{"use_fsync", "true"},
|
{"use_fsync", "true"},
|
||||||
@ -4599,6 +4604,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,
|
||||||
|
@ -107,6 +107,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
|
||||||
@ -148,6 +162,19 @@ std::string EncodeUniqueIdBytes(UniqueIdPtr in) {
|
|||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Status DecodeUniqueIdBytes(const std::string &unique_id, UniqueIdPtr out) {
|
||||||
|
if (unique_id.size() != (out.extended ? 24 : 16)) {
|
||||||
|
return Status::NotSupported("Not a valid unique_id");
|
||||||
|
}
|
||||||
|
const char *buf = &unique_id.front();
|
||||||
|
out.ptr[0] = DecodeFixed64(&buf[0]);
|
||||||
|
out.ptr[1] = DecodeFixed64(&buf[8]);
|
||||||
|
if (out.extended) {
|
||||||
|
out.ptr[2] = DecodeFixed64(&buf[16]);
|
||||||
|
}
|
||||||
|
return Status::OK();
|
||||||
|
}
|
||||||
|
|
||||||
template <typename ID>
|
template <typename ID>
|
||||||
Status GetUniqueIdFromTablePropertiesHelper(const TableProperties &props,
|
Status GetUniqueIdFromTablePropertiesHelper(const TableProperties &props,
|
||||||
std::string *out_id) {
|
std::string *out_id) {
|
||||||
|
@ -64,6 +64,9 @@ void ExternalUniqueIdToInternal(UniqueIdPtr in_out);
|
|||||||
// Convert numerical format to byte format for public API
|
// Convert numerical format to byte format for public API
|
||||||
std::string EncodeUniqueIdBytes(UniqueIdPtr in);
|
std::string EncodeUniqueIdBytes(UniqueIdPtr in);
|
||||||
|
|
||||||
|
// Reverse of EncodeUniqueIdBytes.
|
||||||
|
Status DecodeUniqueIdBytes(const std::string &unique_id, UniqueIdPtr out);
|
||||||
|
|
||||||
// For presenting internal IDs for debugging purposes. Visually distinct from
|
// For presenting internal IDs for debugging purposes. Visually distinct from
|
||||||
// UniqueIdToHumanString for external IDs.
|
// UniqueIdToHumanString for external IDs.
|
||||||
std::string InternalUniqueIdToHumanString(UniqueIdPtr in);
|
std::string InternalUniqueIdToHumanString(UniqueIdPtr in);
|
||||||
|
@ -285,6 +285,7 @@ void RandomInitDBOptions(DBOptions* db_opt, Random* rnd) {
|
|||||||
db_opt->is_fd_close_on_exec = rnd->Uniform(2);
|
db_opt->is_fd_close_on_exec = rnd->Uniform(2);
|
||||||
db_opt->paranoid_checks = rnd->Uniform(2);
|
db_opt->paranoid_checks = rnd->Uniform(2);
|
||||||
db_opt->track_and_verify_wals_in_manifest = rnd->Uniform(2);
|
db_opt->track_and_verify_wals_in_manifest = rnd->Uniform(2);
|
||||||
|
db_opt->verify_sst_unique_id_in_manifest = rnd->Uniform(2);
|
||||||
db_opt->skip_stats_update_on_db_open = rnd->Uniform(2);
|
db_opt->skip_stats_update_on_db_open = rnd->Uniform(2);
|
||||||
db_opt->skip_checking_sst_file_sizes_on_db_open = rnd->Uniform(2);
|
db_opt->skip_checking_sst_file_sizes_on_db_open = rnd->Uniform(2);
|
||||||
db_opt->use_adaptive_mutex = rnd->Uniform(2);
|
db_opt->use_adaptive_mutex = rnd->Uniform(2);
|
||||||
|
@ -173,6 +173,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 do unique_id verification
|
||||||
}
|
}
|
||||||
|
|
||||||
_TEST_DIR_ENV_VAR = 'TEST_TMPDIR'
|
_TEST_DIR_ENV_VAR = 'TEST_TMPDIR'
|
||||||
|
Loading…
x
Reference in New Issue
Block a user