internal_repo_rocksdb (4372117296613874540) (#12117)

Summary: Pull Request resolved: https://github.com/facebook/rocksdb/pull/12117

Reviewed By: ajkr

Differential Revision: D51745846

Pulled By: jowlyzhang

fbshipit-source-id: 51c806a484b3b43d174b06d2cfe9499191d09914
This commit is contained in:
Yu Zhang 2023-12-04 11:17:32 -08:00 committed by Facebook GitHub Bot
parent dce3ca5ab8
commit ba8fa0f546
48 changed files with 193 additions and 156 deletions

View File

@ -326,7 +326,7 @@ TEST_F(ImportColumnFamilyTest, ImportSSTFileWriterFilesWithRangeTombstone) {
const SstFileMetaData* file_meta = nullptr; const SstFileMetaData* file_meta = nullptr;
for (const auto& level_meta : import_cf_meta.levels) { for (const auto& level_meta : import_cf_meta.levels) {
if (!level_meta.files.empty()) { if (!level_meta.files.empty()) {
file_meta = &(level_meta.files[0]); file_meta = level_meta.files.data();
break; break;
} }
} }
@ -389,7 +389,7 @@ TEST_F(ImportColumnFamilyTest, ImportExportedSSTFromAnotherCF) {
*metadata_ptr_, &import_cfh2_)); *metadata_ptr_, &import_cfh2_));
ASSERT_NE(import_cfh2_, nullptr); ASSERT_NE(import_cfh2_, nullptr);
delete metadata_ptr_; delete metadata_ptr_;
metadata_ptr_ = NULL; metadata_ptr_ = nullptr;
std::string value1, value2; std::string value1, value2;

View File

@ -132,8 +132,8 @@ class TestCompactionListener : public EventListener {
ASSERT_EQ(db->GetEnv()->GetThreadID(), ci.thread_id); ASSERT_EQ(db->GetEnv()->GetThreadID(), ci.thread_id);
ASSERT_GT(ci.thread_id, 0U); ASSERT_GT(ci.thread_id, 0U);
for (auto fl : {ci.input_files, ci.output_files}) { for (const auto& fl : {ci.input_files, ci.output_files}) {
for (auto fn : fl) { for (const auto& fn : fl) {
auto it = ci.table_properties.find(fn); auto it = ci.table_properties.find(fn);
ASSERT_NE(it, ci.table_properties.end()); ASSERT_NE(it, ci.table_properties.end());
auto tp = it->second; auto tp = it->second;
@ -237,7 +237,7 @@ class TestFlushListener : public EventListener {
std::vector<ThreadStatus> thread_list; std::vector<ThreadStatus> thread_list;
ASSERT_OK(env_->GetThreadList(&thread_list)); ASSERT_OK(env_->GetThreadList(&thread_list));
bool found_match = false; bool found_match = false;
for (auto thread_status : thread_list) { for (const auto& thread_status : thread_list) {
if (thread_status.operation_type == ThreadStatus::OP_FLUSH || if (thread_status.operation_type == ThreadStatus::OP_FLUSH ||
thread_status.operation_type == ThreadStatus::OP_COMPACTION) { thread_status.operation_type == ThreadStatus::OP_COMPACTION) {
if (thread_id == thread_status.thread_id) { if (thread_id == thread_status.thread_id) {
@ -893,7 +893,7 @@ class MemTableSealedListener : public EventListener {
SequenceNumber latest_seq_number_; SequenceNumber latest_seq_number_;
public: public:
MemTableSealedListener() {} MemTableSealedListener() = default;
void OnMemTableSealed(const MemTableInfo& info) override { void OnMemTableSealed(const MemTableInfo& info) override {
latest_seq_number_ = info.first_seqno; latest_seq_number_ = info.first_seqno;
} }

View File

@ -9,7 +9,7 @@
#include "db/log_reader.h" #include "db/log_reader.h"
#include <stdio.h> #include <cstdio>
#include "file/sequence_file_reader.h" #include "file/sequence_file_reader.h"
#include "port/lang.h" #include "port/lang.h"
@ -21,7 +21,7 @@
namespace ROCKSDB_NAMESPACE { namespace ROCKSDB_NAMESPACE {
namespace log { namespace log {
Reader::Reporter::~Reporter() {} Reader::Reporter::~Reporter() = default;
Reader::Reader(std::shared_ptr<Logger> info_log, Reader::Reader(std::shared_ptr<Logger> info_log,
std::unique_ptr<SequentialFileReader>&& _file, std::unique_ptr<SequentialFileReader>&& _file,

View File

@ -1167,7 +1167,7 @@ TEST_P(StreamingCompressionTest, Basic) {
} }
allocator->Deallocate((void*)output_buffer); allocator->Deallocate((void*)output_buffer);
} while (remaining > 0); } while (remaining > 0);
std::string uncompressed_buffer = ""; std::string uncompressed_buffer;
int ret_val = 0; int ret_val = 0;
size_t output_pos; size_t output_pos;
char* uncompressed_output_buffer = (char*)allocator->Allocate(kBlockSize); char* uncompressed_output_buffer = (char*)allocator->Allocate(kBlockSize);

View File

@ -9,7 +9,7 @@
#include "db/log_writer.h" #include "db/log_writer.h"
#include <stdint.h> #include <cstdint>
#include "file/writable_file_writer.h" #include "file/writable_file_writer.h"
#include "rocksdb/env.h" #include "rocksdb/env.h"

View File

@ -9,8 +9,7 @@
#include "db/malloc_stats.h" #include "db/malloc_stats.h"
#include <string.h> #include <cstring>
#include <memory> #include <memory>
#include "port/jemalloc_helper.h" #include "port/jemalloc_helper.h"

View File

@ -58,7 +58,7 @@ class ManualCompactionTest : public testing::Test {
class DestroyAllCompactionFilter : public CompactionFilter { class DestroyAllCompactionFilter : public CompactionFilter {
public: public:
DestroyAllCompactionFilter() {} DestroyAllCompactionFilter() = default;
bool Filter(int /*level*/, const Slice& /*key*/, const Slice& existing_value, bool Filter(int /*level*/, const Slice& /*key*/, const Slice& existing_value,
std::string* /*new_value*/, std::string* /*new_value*/,

View File

@ -1362,7 +1362,7 @@ void MemTable::MultiGet(const ReadOptions& read_options, MultiGetRange* range,
range_indexes[num_keys++] = iter.index(); range_indexes[num_keys++] = iter.index();
} }
} }
bloom_filter_->MayContain(num_keys, &bloom_keys[0], &may_match[0]); bloom_filter_->MayContain(num_keys, bloom_keys.data(), may_match.data());
for (int i = 0; i < num_keys; ++i) { for (int i = 0; i < num_keys; ++i) {
if (!may_match[i]) { if (!may_match[i]) {
temp_range.SkipIndex(range_indexes[i]); temp_range.SkipIndex(range_indexes[i]);

View File

@ -428,7 +428,9 @@ Status MergeHelper::MergeUntil(InternalIterator* iter,
Status s = ParseInternalKey(original_key, &orig_ikey, allow_data_in_errors); Status s = ParseInternalKey(original_key, &orig_ikey, allow_data_in_errors);
assert(s.ok()); assert(s.ok());
if (!s.ok()) return s; if (!s.ok()) {
return s;
}
assert(kTypeMerge == orig_ikey.type); assert(kTypeMerge == orig_ikey.type);

View File

@ -3,8 +3,7 @@
// COPYING file in the root directory) and Apache 2.0 License // COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory). // (found in the LICENSE.Apache file in the root directory).
// //
#include <assert.h> #include <cassert>
#include <iostream> #include <iostream>
#include <memory> #include <memory>
@ -87,7 +86,9 @@ class EnvMergeTest : public EnvWrapper {
static std::unique_ptr<EnvMergeTest> singleton_; static std::unique_ptr<EnvMergeTest> singleton_;
static EnvMergeTest* GetInstance() { static EnvMergeTest* GetInstance() {
if (nullptr == singleton_) singleton_.reset(new EnvMergeTest); if (nullptr == singleton_) {
singleton_.reset(new EnvMergeTest);
}
return singleton_.get(); return singleton_.get();
} }
}; };
@ -145,7 +146,7 @@ class Counters {
assert(db_); assert(db_);
} }
virtual ~Counters() {} virtual ~Counters() = default;
// public interface of Counters. // public interface of Counters.
// All four functions return false // All four functions return false
@ -194,7 +195,7 @@ class Counters {
std::cerr << "value corruption\n"; std::cerr << "value corruption\n";
return false; return false;
} }
*value = DecodeFixed64(&str[0]); *value = DecodeFixed64(str.data());
return true; return true;
} else { } else {
std::cerr << s.ToString() << std::endl; std::cerr << s.ToString() << std::endl;
@ -220,14 +221,18 @@ class Counters {
uint64_t value = default_; uint64_t value = default_;
int result = get(key, &value); int result = get(key, &value);
assert(result); assert(result);
if (result == 0) exit(1); // Disable unused variable warning. if (result == 0) {
exit(1); // Disable unused variable warning.
}
return value; return value;
} }
void assert_add(const std::string& key, uint64_t value) { void assert_add(const std::string& key, uint64_t value) {
int result = add(key, value); int result = add(key, value);
assert(result); assert(result);
if (result == 0) exit(1); // Disable unused variable warning. if (result == 0) {
exit(1); // Disable unused variable warning.
}
} }
}; };
@ -496,7 +501,7 @@ void testSingleBatchSuccessiveMerge(DB* db, size_t max_num_merges,
std::string get_value_str; std::string get_value_str;
ASSERT_OK(db->Get(ReadOptions(), key, &get_value_str)); ASSERT_OK(db->Get(ReadOptions(), key, &get_value_str));
assert(get_value_str.size() == sizeof(uint64_t)); assert(get_value_str.size() == sizeof(uint64_t));
uint64_t get_value = DecodeFixed64(&get_value_str[0]); uint64_t get_value = DecodeFixed64(get_value_str.data());
ASSERT_EQ(get_value, num_merges * merge_value); ASSERT_EQ(get_value, num_merges * merge_value);
ASSERT_EQ(num_merge_operator_calls, ASSERT_EQ(num_merge_operator_calls,
static_cast<size_t>((num_merges % (max_num_merges + 1)))); static_cast<size_t>((num_merges % (max_num_merges + 1))));

View File

@ -7,10 +7,8 @@
// Use of this source code is governed by a BSD-style license that can be // Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors. // found in the LICENSE file. See the AUTHORS file for names of contributors.
#include <stdlib.h>
#include <algorithm> #include <algorithm>
#include <cstdlib>
#include <map> #include <map>
#include <string> #include <string>
#include <vector> #include <vector>
@ -68,7 +66,7 @@ class ObsoleteFilesTest : public DBTestBase {
int log_cnt = 0; int log_cnt = 0;
int sst_cnt = 0; int sst_cnt = 0;
int manifest_cnt = 0; int manifest_cnt = 0;
for (auto file : filenames) { for (const auto& file : filenames) {
uint64_t number; uint64_t number;
FileType type; FileType type;
if (ParseFileName(file, &number, &type)) { if (ParseFileName(file, &number, &type)) {

View File

@ -28,7 +28,7 @@ void UpdateOptionsFiles(DB* db,
uint64_t number; uint64_t number;
FileType type; FileType type;
*options_files_count = 0; *options_files_count = 0;
for (auto filename : filenames) { for (const auto& filename : filenames) {
if (ParseFileName(filename, &number, &type) && type == kOptionsFile) { if (ParseFileName(filename, &number, &type) && type == kOptionsFile) {
filename_history->insert(filename); filename_history->insert(filename);
(*options_files_count)++; (*options_files_count)++;
@ -44,16 +44,16 @@ void VerifyOptionsFileName(
EXPECT_OK(db->GetEnv()->GetChildren(db->GetName(), &filenames)); EXPECT_OK(db->GetEnv()->GetChildren(db->GetName(), &filenames));
uint64_t number; uint64_t number;
FileType type; FileType type;
for (auto filename : filenames) { for (const auto& filename : filenames) {
if (ParseFileName(filename, &number, &type) && type == kOptionsFile) { if (ParseFileName(filename, &number, &type) && type == kOptionsFile) {
current_filenames.insert(filename); current_filenames.insert(filename);
} }
} }
for (auto past_filename : past_filenames) { for (const auto& past_filename : past_filenames) {
if (current_filenames.find(past_filename) != current_filenames.end()) { if (current_filenames.find(past_filename) != current_filenames.end()) {
continue; continue;
} }
for (auto filename : current_filenames) { for (const auto& filename : current_filenames) {
ASSERT_GT(filename, past_filename); ASSERT_GT(filename, past_filename);
} }
} }

View File

@ -1050,7 +1050,7 @@ TEST_F(PerfContextTest, MergeOperandCount) {
std::vector<Status> statuses(num_keys); std::vector<Status> statuses(num_keys);
db->MultiGet(ReadOptions(), db->DefaultColumnFamily(), num_keys, db->MultiGet(ReadOptions(), db->DefaultColumnFamily(), num_keys,
&key_slices[0], &results[0], &statuses[0]); key_slices.data(), results.data(), statuses.data());
for (size_t i = 0; i < num_keys; ++i) { for (size_t i = 0; i < num_keys; ++i) {
ASSERT_OK(statuses[i]); ASSERT_OK(statuses[i]);
@ -1068,7 +1068,7 @@ TEST_F(PerfContextTest, MergeOperandCount) {
std::vector<Status> statuses(num_keys); std::vector<Status> statuses(num_keys);
db->MultiGetEntity(ReadOptions(), db->DefaultColumnFamily(), num_keys, db->MultiGetEntity(ReadOptions(), db->DefaultColumnFamily(), num_keys,
&key_slices[0], &results[0], &statuses[0]); key_slices.data(), results.data(), statuses.data());
for (size_t i = 0; i < num_keys; ++i) { for (size_t i = 0; i < num_keys; ++i) {
ASSERT_OK(statuses[i]); ASSERT_OK(statuses[i]);

View File

@ -292,7 +292,7 @@ class TestPlainTableReader : public PlainTableReader {
table_properties_ = std::move(props); table_properties_ = std::move(props);
} }
~TestPlainTableReader() override {} ~TestPlainTableReader() override = default;
private: private:
bool MatchBloom(uint32_t hash) const override { bool MatchBloom(uint32_t hash) const override {

View File

@ -89,8 +89,12 @@ class TestKeyComparator : public Comparator {
const TestKey* key_a = &kkey_a; const TestKey* key_a = &kkey_a;
const TestKey* key_b = &kkey_b; const TestKey* key_b = &kkey_b;
if (key_a->prefix != key_b->prefix) { if (key_a->prefix != key_b->prefix) {
if (key_a->prefix < key_b->prefix) return -1; if (key_a->prefix < key_b->prefix) {
if (key_a->prefix > key_b->prefix) return 1; return -1;
}
if (key_a->prefix > key_b->prefix) {
return 1;
}
} else { } else {
EXPECT_TRUE(key_a->prefix == key_b->prefix); EXPECT_TRUE(key_a->prefix == key_b->prefix);
// note, both a and b could be prefix only // note, both a and b could be prefix only
@ -99,8 +103,12 @@ class TestKeyComparator : public Comparator {
EXPECT_TRUE( EXPECT_TRUE(
(a.size() == sizeof(uint64_t) && b.size() == sizeof(TestKey)) || (a.size() == sizeof(uint64_t) && b.size() == sizeof(TestKey)) ||
(b.size() == sizeof(uint64_t) && a.size() == sizeof(TestKey))); (b.size() == sizeof(uint64_t) && a.size() == sizeof(TestKey)));
if (a.size() < b.size()) return -1; if (a.size() < b.size()) {
if (a.size() > b.size()) return 1; return -1;
}
if (a.size() > b.size()) {
return 1;
}
} else { } else {
// both a and b are prefix // both a and b are prefix
if (a.size() == sizeof(uint64_t)) { if (a.size() == sizeof(uint64_t)) {
@ -109,9 +117,15 @@ class TestKeyComparator : public Comparator {
// both a and b are whole key // both a and b are whole key
EXPECT_TRUE(a.size() == sizeof(TestKey) && b.size() == sizeof(TestKey)); EXPECT_TRUE(a.size() == sizeof(TestKey) && b.size() == sizeof(TestKey));
if (key_a->sorted < key_b->sorted) return -1; if (key_a->sorted < key_b->sorted) {
if (key_a->sorted > key_b->sorted) return 1; return -1;
if (key_a->sorted == key_b->sorted) return 0; }
if (key_a->sorted > key_b->sorted) {
return 1;
}
if (key_a->sorted == key_b->sorted) {
return 0;
}
} }
} }
return 0; return 0;
@ -892,4 +906,3 @@ int main(int argc, char** argv) {
} }
#endif // GFLAGS #endif // GFLAGS

View File

@ -8,7 +8,6 @@
#include "db/compaction/compaction_iteration_stats.h" #include "db/compaction/compaction_iteration_stats.h"
#include "db/dbformat.h" #include "db/dbformat.h"
#include "db/pinned_iterators_manager.h" #include "db/pinned_iterators_manager.h"
#include "db/range_del_aggregator.h"
#include "db/range_tombstone_fragmenter.h" #include "db/range_tombstone_fragmenter.h"
#include "db/version_edit.h" #include "db/version_edit.h"
#include "rocksdb/comparator.h" #include "rocksdb/comparator.h"

View File

@ -378,8 +378,7 @@ TEST_P(RepairTestWithTimestamp, UnflushedSst) {
ColumnFamilyOptions cf_options(options); ColumnFamilyOptions cf_options(options);
std::vector<ColumnFamilyDescriptor> column_families; std::vector<ColumnFamilyDescriptor> column_families;
column_families.push_back( column_families.emplace_back(kDefaultColumnFamilyName, cf_options);
ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options));
ASSERT_OK(DB::Open(options, dbname_, column_families, &handles_, &db_)); ASSERT_OK(DB::Open(options, dbname_, column_families, &handles_, &db_));

View File

@ -85,7 +85,7 @@ TableCache::TableCache(const ImmutableOptions& ioptions,
} }
} }
TableCache::~TableCache() {} TableCache::~TableCache() = default;
Status TableCache::GetTableReader( Status TableCache::GetTableReader(
const ReadOptions& ro, const FileOptions& file_options, const ReadOptions& ro, const FileOptions& file_options,

View File

@ -180,7 +180,6 @@ class RegularKeysStartWithAInternal : public IntTblPropCollector {
uint64_t /* block_compressed_bytes_fast */, uint64_t /* block_compressed_bytes_fast */,
uint64_t /* block_compressed_bytes_slow */) override { uint64_t /* block_compressed_bytes_slow */) override {
// Nothing to do. // Nothing to do.
return;
} }
UserCollectedProperties GetReadableProperties() const override { UserCollectedProperties GetReadableProperties() const override {
@ -227,7 +226,7 @@ class FlushBlockEveryThreePolicy : public FlushBlockPolicy {
class FlushBlockEveryThreePolicyFactory : public FlushBlockPolicyFactory { class FlushBlockEveryThreePolicyFactory : public FlushBlockPolicyFactory {
public: public:
explicit FlushBlockEveryThreePolicyFactory() {} explicit FlushBlockEveryThreePolicyFactory() = default;
const char* Name() const override { const char* Name() const override {
return "FlushBlockEveryThreePolicyFactory"; return "FlushBlockEveryThreePolicyFactory";

View File

@ -484,7 +484,6 @@ void VersionEdit::EncodeFileBoundaries(std::string* dst,
StripTimestampFromInternalKey(&largest_buf, meta.largest.Encode(), ts_sz); StripTimestampFromInternalKey(&largest_buf, meta.largest.Encode(), ts_sz);
PutLengthPrefixedSlice(dst, smallest_buf); PutLengthPrefixedSlice(dst, smallest_buf);
PutLengthPrefixedSlice(dst, largest_buf); PutLengthPrefixedSlice(dst, largest_buf);
return;
}; };
Status VersionEdit::DecodeFrom(const Slice& src) { Status VersionEdit::DecodeFrom(const Slice& src) {

View File

@ -562,7 +562,9 @@ TEST_F(VersionEditTest, AddWalDebug) {
std::stringstream ss; std::stringstream ss;
ss << "{\"LogNumber\": " << kLogNumbers[i] << ", " ss << "{\"LogNumber\": " << kLogNumbers[i] << ", "
<< "\"SyncedSizeInBytes\": " << kSizeInBytes[i] << "}"; << "\"SyncedSizeInBytes\": " << kSizeInBytes[i] << "}";
if (i < n - 1) ss << ", "; if (i < n - 1) {
ss << ", ";
}
expected_json += ss.str(); expected_json += ss.str();
} }
expected_json += "], \"ColumnFamily\": 0}"; expected_json += "], \"ColumnFamily\": 0}";

View File

@ -122,7 +122,9 @@ Status OverlapWithIterator(const Comparator* ucmp,
ParsedInternalKey seek_result; ParsedInternalKey seek_result;
Status s = ParseInternalKey(iter->key(), &seek_result, Status s = ParseInternalKey(iter->key(), &seek_result,
false /* log_err_key */); // TODO false /* log_err_key */); // TODO
if (!s.ok()) return s; if (!s.ok()) {
return s;
}
if (ucmp->CompareWithoutTimestamp(seek_result.user_key, largest_user_key) <= if (ucmp->CompareWithoutTimestamp(seek_result.user_key, largest_user_key) <=
0) { 0) {
@ -835,9 +837,9 @@ Version::~Version() {
assert(cfd_ != nullptr); assert(cfd_ != nullptr);
uint32_t path_id = f->fd.GetPathId(); uint32_t path_id = f->fd.GetPathId();
assert(path_id < cfd_->ioptions()->cf_paths.size()); assert(path_id < cfd_->ioptions()->cf_paths.size());
vset_->obsolete_files_.push_back( vset_->obsolete_files_.emplace_back(
ObsoleteFileInfo(f, cfd_->ioptions()->cf_paths[path_id].path, f, cfd_->ioptions()->cf_paths[path_id].path,
cfd_->GetFileMetadataCacheReservationManager())); cfd_->GetFileMetadataCacheReservationManager());
} }
} }
} }
@ -3101,7 +3103,9 @@ bool Version::MaybeInitializeFileMetaData(const ReadOptions& read_options,
file_meta->fd.GetNumber(), s.ToString().c_str()); file_meta->fd.GetNumber(), s.ToString().c_str());
return false; return false;
} }
if (tp.get() == nullptr) return false; if (tp.get() == nullptr) {
return false;
}
file_meta->num_entries = tp->num_entries; file_meta->num_entries = tp->num_entries;
file_meta->num_deletions = tp->num_deletions; file_meta->num_deletions = tp->num_deletions;
file_meta->raw_value_size = tp->raw_value_size; file_meta->raw_value_size = tp->raw_value_size;
@ -4515,7 +4519,9 @@ const char* VersionStorageInfo::LevelSummary(
for (int i = 0; i < num_levels(); i++) { for (int i = 0; i < num_levels(); i++) {
int sz = sizeof(scratch->buffer) - len; int sz = sizeof(scratch->buffer) - len;
int ret = snprintf(scratch->buffer + len, sz, "%d ", int(files_[i].size())); int ret = snprintf(scratch->buffer + len, sz, "%d ", int(files_[i].size()));
if (ret < 0 || ret >= sz) break; if (ret < 0 || ret >= sz) {
break;
}
len += ret; len += ret;
} }
if (len > 0) { if (len > 0) {
@ -4545,7 +4551,9 @@ const char* VersionStorageInfo::LevelFileSummary(FileSummaryStorage* scratch,
"#%" PRIu64 "(seq=%" PRIu64 ",sz=%s,%d) ", "#%" PRIu64 "(seq=%" PRIu64 ",sz=%s,%d) ",
f->fd.GetNumber(), f->fd.smallest_seqno, sztxt, f->fd.GetNumber(), f->fd.smallest_seqno, sztxt,
static_cast<int>(f->being_compacted)); static_cast<int>(f->being_compacted));
if (ret < 0 || ret >= sz) break; if (ret < 0 || ret >= sz) {
break;
}
len += ret; len += ret;
} }
// overwrite the last space (only if files_[level].size() is non-zero) // overwrite the last space (only if files_[level].size() is non-zero)
@ -5384,9 +5392,9 @@ Status VersionSet::ProcessManifestWrites(
} }
for (const auto* cfd : *column_family_set_) { for (const auto* cfd : *column_family_set_) {
assert(curr_state.find(cfd->GetID()) == curr_state.end()); assert(curr_state.find(cfd->GetID()) == curr_state.end());
curr_state.emplace(std::make_pair( curr_state.emplace(
cfd->GetID(), cfd->GetID(),
MutableCFState(cfd->GetLogNumber(), cfd->GetFullHistoryTsLow()))); MutableCFState(cfd->GetLogNumber(), cfd->GetFullHistoryTsLow()));
} }
for (const auto& wal : wals_.GetWals()) { for (const auto& wal : wals_.GetWals()) {
@ -7277,7 +7285,7 @@ ReactiveVersionSet::ReactiveVersionSet(
/*db_session_id*/ "", /*daily_offpeak_time_utc*/ "", /*db_session_id*/ "", /*daily_offpeak_time_utc*/ "",
/*error_handler=*/nullptr) {} /*error_handler=*/nullptr) {}
ReactiveVersionSet::~ReactiveVersionSet() {} ReactiveVersionSet::~ReactiveVersionSet() = default;
Status ReactiveVersionSet::Recover( Status ReactiveVersionSet::Recover(
const std::vector<ColumnFamilyDescriptor>& column_families, const std::vector<ColumnFamilyDescriptor>& column_families,

View File

@ -34,7 +34,7 @@ class GenerateLevelFilesBriefTest : public testing::Test {
LevelFilesBrief file_level_; LevelFilesBrief file_level_;
Arena arena_; Arena arena_;
GenerateLevelFilesBriefTest() {} GenerateLevelFilesBriefTest() = default;
~GenerateLevelFilesBriefTest() override { ~GenerateLevelFilesBriefTest() override {
for (size_t i = 0; i < files_.size(); i++) { for (size_t i = 0; i < files_.size(); i++) {
@ -213,7 +213,7 @@ class VersionStorageInfoTest : public VersionStorageInfoTestBase {
public: public:
VersionStorageInfoTest() : VersionStorageInfoTestBase(BytewiseComparator()) {} VersionStorageInfoTest() : VersionStorageInfoTestBase(BytewiseComparator()) {}
~VersionStorageInfoTest() override {} ~VersionStorageInfoTest() override = default;
}; };
TEST_F(VersionStorageInfoTest, MaxBytesForLevelStatic) { TEST_F(VersionStorageInfoTest, MaxBytesForLevelStatic) {
@ -928,7 +928,7 @@ class VersionStorageInfoTimestampTest : public VersionStorageInfoTestBase {
VersionStorageInfoTimestampTest() VersionStorageInfoTimestampTest()
: VersionStorageInfoTestBase(test::BytewiseComparatorWithU64TsWrapper()) { : VersionStorageInfoTestBase(test::BytewiseComparatorWithU64TsWrapper()) {
} }
~VersionStorageInfoTimestampTest() override {} ~VersionStorageInfoTimestampTest() override = default;
std::string Timestamp(uint64_t ts) const { std::string Timestamp(uint64_t ts) const {
std::string ret; std::string ret;
PutFixed64(&ret, ts); PutFixed64(&ret, ts);
@ -982,7 +982,7 @@ class FindLevelFileTest : public testing::Test {
FindLevelFileTest() : disjoint_sorted_files_(true) {} FindLevelFileTest() : disjoint_sorted_files_(true) {}
~FindLevelFileTest() override {} ~FindLevelFileTest() override = default;
void LevelFileInit(size_t num = 0) { void LevelFileInit(size_t num = 0) {
char* mem = arena_.AllocateAligned(num * sizeof(FdWithKeyRange)); char* mem = arena_.AllocateAligned(num * sizeof(FdWithKeyRange));
@ -3662,7 +3662,7 @@ class ChargeFileMetadataTestWithParam
: public ChargeFileMetadataTest, : public ChargeFileMetadataTest,
public testing::WithParamInterface<CacheEntryRoleOptions::Decision> { public testing::WithParamInterface<CacheEntryRoleOptions::Decision> {
public: public:
ChargeFileMetadataTestWithParam() {} ChargeFileMetadataTestWithParam() = default;
}; };
INSTANTIATE_TEST_CASE_P( INSTANTIATE_TEST_CASE_P(

View File

@ -94,7 +94,7 @@ TEST_F(DBWideBasicTest, PutEntity) {
std::array<Status, num_keys> statuses; std::array<Status, num_keys> statuses;
db_->MultiGet(ReadOptions(), db_->DefaultColumnFamily(), num_keys, db_->MultiGet(ReadOptions(), db_->DefaultColumnFamily(), num_keys,
&keys[0], &values[0], &statuses[0]); keys.data(), values.data(), statuses.data());
ASSERT_OK(statuses[0]); ASSERT_OK(statuses[0]);
ASSERT_EQ(values[0], first_value_of_default_column); ASSERT_EQ(values[0], first_value_of_default_column);
@ -114,7 +114,7 @@ TEST_F(DBWideBasicTest, PutEntity) {
std::array<Status, num_keys> statuses; std::array<Status, num_keys> statuses;
db_->MultiGetEntity(ReadOptions(), db_->DefaultColumnFamily(), num_keys, db_->MultiGetEntity(ReadOptions(), db_->DefaultColumnFamily(), num_keys,
&keys[0], &results[0], &statuses[0]); keys.data(), results.data(), statuses.data());
ASSERT_OK(statuses[0]); ASSERT_OK(statuses[0]);
ASSERT_EQ(results[0].columns(), first_columns); ASSERT_EQ(results[0].columns(), first_columns);
@ -398,8 +398,8 @@ TEST_F(DBWideBasicTest, MultiCFMultiGetEntity) {
std::array<PinnableWideColumns, num_keys> results; std::array<PinnableWideColumns, num_keys> results;
std::array<Status, num_keys> statuses; std::array<Status, num_keys> statuses;
db_->MultiGetEntity(ReadOptions(), num_keys, &column_families[0], &keys[0], db_->MultiGetEntity(ReadOptions(), num_keys, column_families.data(),
&results[0], &statuses[0]); keys.data(), results.data(), statuses.data());
ASSERT_OK(statuses[0]); ASSERT_OK(statuses[0]);
ASSERT_EQ(results[0].columns(), first_columns); ASSERT_EQ(results[0].columns(), first_columns);
@ -642,7 +642,7 @@ TEST_F(DBWideBasicTest, MergePlainKeyValue) {
std::array<Status, num_keys> statuses; std::array<Status, num_keys> statuses;
db_->MultiGetEntity(ReadOptions(), db_->DefaultColumnFamily(), num_keys, db_->MultiGetEntity(ReadOptions(), db_->DefaultColumnFamily(), num_keys,
&keys[0], &results[0], &statuses[0]); keys.data(), results.data(), statuses.data());
ASSERT_OK(statuses[0]); ASSERT_OK(statuses[0]);
ASSERT_EQ(results[0].columns(), expected_first_columns); ASSERT_EQ(results[0].columns(), expected_first_columns);
@ -822,7 +822,7 @@ TEST_F(DBWideBasicTest, MergeEntity) {
std::array<Status, num_keys> statuses; std::array<Status, num_keys> statuses;
db_->MultiGet(ReadOptions(), db_->DefaultColumnFamily(), num_keys, db_->MultiGet(ReadOptions(), db_->DefaultColumnFamily(), num_keys,
&keys[0], &values[0], &statuses[0]); keys.data(), values.data(), statuses.data());
ASSERT_EQ(values[0], first_expected_default); ASSERT_EQ(values[0], first_expected_default);
ASSERT_OK(statuses[0]); ASSERT_OK(statuses[0]);
@ -839,7 +839,7 @@ TEST_F(DBWideBasicTest, MergeEntity) {
std::array<Status, num_keys> statuses; std::array<Status, num_keys> statuses;
db_->MultiGetEntity(ReadOptions(), db_->DefaultColumnFamily(), num_keys, db_->MultiGetEntity(ReadOptions(), db_->DefaultColumnFamily(), num_keys,
&keys[0], &results[0], &statuses[0]); keys.data(), results.data(), statuses.data());
ASSERT_OK(statuses[0]); ASSERT_OK(statuses[0]);
ASSERT_EQ(results[0].columns(), first_expected_columns); ASSERT_EQ(results[0].columns(), first_expected_columns);
@ -900,7 +900,7 @@ TEST_F(DBWideBasicTest, MergeEntity) {
int number_of_operands = 0; int number_of_operands = 0;
ASSERT_OK(db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(), ASSERT_OK(db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(),
first_key, &merge_operands[0], first_key, merge_operands.data(),
&get_merge_opts, &number_of_operands)); &get_merge_opts, &number_of_operands));
ASSERT_EQ(number_of_operands, num_merge_operands); ASSERT_EQ(number_of_operands, num_merge_operands);
@ -913,7 +913,7 @@ TEST_F(DBWideBasicTest, MergeEntity) {
int number_of_operands = 0; int number_of_operands = 0;
ASSERT_OK(db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(), ASSERT_OK(db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(),
second_key, &merge_operands[0], second_key, merge_operands.data(),
&get_merge_opts, &number_of_operands)); &get_merge_opts, &number_of_operands));
ASSERT_EQ(number_of_operands, num_merge_operands); ASSERT_EQ(number_of_operands, num_merge_operands);
@ -933,7 +933,7 @@ TEST_F(DBWideBasicTest, MergeEntity) {
int number_of_operands = 0; int number_of_operands = 0;
ASSERT_OK(db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(), ASSERT_OK(db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(),
first_key, &merge_operands[0], first_key, merge_operands.data(),
&get_merge_opts, &number_of_operands)); &get_merge_opts, &number_of_operands));
ASSERT_EQ(number_of_operands, num_merge_operands); ASSERT_EQ(number_of_operands, num_merge_operands);
@ -945,7 +945,7 @@ TEST_F(DBWideBasicTest, MergeEntity) {
int number_of_operands = 0; int number_of_operands = 0;
ASSERT_OK(db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(), ASSERT_OK(db_->GetMergeOperands(ReadOptions(), db_->DefaultColumnFamily(),
second_key, &merge_operands[0], second_key, merge_operands.data(),
&get_merge_opts, &number_of_operands)); &get_merge_opts, &number_of_operands));
ASSERT_EQ(number_of_operands, num_merge_operands); ASSERT_EQ(number_of_operands, num_merge_operands);

View File

@ -233,9 +233,9 @@ WriteBatch& WriteBatch::operator=(WriteBatch&& src) {
return *this; return *this;
} }
WriteBatch::~WriteBatch() {} WriteBatch::~WriteBatch() = default;
WriteBatch::Handler::~Handler() {} WriteBatch::Handler::~Handler() = default;
void WriteBatch::Handler::LogData(const Slice& /*blob*/) { void WriteBatch::Handler::LogData(const Slice& /*blob*/) {
// If the user has not specified something to do with blobs, then we ignore // If the user has not specified something to do with blobs, then we ignore
@ -741,7 +741,7 @@ SequenceNumber WriteBatchInternal::Sequence(const WriteBatch* b) {
} }
void WriteBatchInternal::SetSequence(WriteBatch* b, SequenceNumber seq) { void WriteBatchInternal::SetSequence(WriteBatch* b, SequenceNumber seq) {
EncodeFixed64(&b->rep_[0], seq); EncodeFixed64(b->rep_.data(), seq);
} }
size_t WriteBatchInternal::GetFirstOffset(WriteBatch* /*b*/) { size_t WriteBatchInternal::GetFirstOffset(WriteBatch* /*b*/) {
@ -1856,7 +1856,9 @@ class MemTableInserter : public WriteBatch::Handler {
} }
void DecrementProtectionInfoIdxForTryAgain() { void DecrementProtectionInfoIdxForTryAgain() {
if (prot_info_ != nullptr) --prot_info_idx_; if (prot_info_ != nullptr) {
--prot_info_idx_;
}
} }
void ResetProtectionInfo() { void ResetProtectionInfo() {
@ -3018,7 +3020,7 @@ class ProtectionInfoUpdater : public WriteBatch::Handler {
explicit ProtectionInfoUpdater(WriteBatch::ProtectionInfo* prot_info) explicit ProtectionInfoUpdater(WriteBatch::ProtectionInfo* prot_info)
: prot_info_(prot_info) {} : prot_info_(prot_info) {}
~ProtectionInfoUpdater() override {} ~ProtectionInfoUpdater() override = default;
Status PutCF(uint32_t cf, const Slice& key, const Slice& val) override { Status PutCF(uint32_t cf, const Slice& key, const Slice& val) override {
return UpdateProtInfo(cf, key, val, kTypeValue); return UpdateProtInfo(cf, key, val, kTypeValue);

View File

@ -64,7 +64,7 @@ class MockWriteCallback : public WriteCallback {
bool allow_batching_ = false; bool allow_batching_ = false;
std::atomic<bool> was_called_{false}; std::atomic<bool> was_called_{false};
MockWriteCallback() {} MockWriteCallback() = default;
MockWriteCallback(const MockWriteCallback& other) { MockWriteCallback(const MockWriteCallback& other) {
should_fail_ = other.should_fail_; should_fail_ = other.should_fail_;
@ -111,7 +111,7 @@ TEST_P(WriteCallbackPTest, WriteWithCallbackTest) {
WriteOP(bool should_fail = false) { callback_.should_fail_ = should_fail; } WriteOP(bool should_fail = false) { callback_.should_fail_ = should_fail; }
void Put(const string& key, const string& val) { void Put(const string& key, const string& val) {
kvs_.push_back(std::make_pair(key, val)); kvs_.emplace_back(key, val);
ASSERT_OK(write_batch_.Put(key, val)); ASSERT_OK(write_batch_.Put(key, val));
} }
@ -176,8 +176,7 @@ TEST_P(WriteCallbackPTest, WriteWithCallbackTest) {
DBOptions db_options(options); DBOptions db_options(options);
ColumnFamilyOptions cf_options(options); ColumnFamilyOptions cf_options(options);
std::vector<ColumnFamilyDescriptor> column_families; std::vector<ColumnFamilyDescriptor> column_families;
column_families.push_back( column_families.emplace_back(kDefaultColumnFamilyName, cf_options);
ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options));
std::vector<ColumnFamilyHandle*> handles; std::vector<ColumnFamilyHandle*> handles;
auto open_s = DBImpl::Open(db_options, dbname, column_families, &handles, auto open_s = DBImpl::Open(db_options, dbname, column_families, &handles,
&db, seq_per_batch_, true /* batch_per_txn */); &db, seq_per_batch_, true /* batch_per_txn */);

View File

@ -13,8 +13,8 @@
namespace ROCKSDB_NAMESPACE { namespace ROCKSDB_NAMESPACE {
class BatchedOpsStressTest : public StressTest { class BatchedOpsStressTest : public StressTest {
public: public:
BatchedOpsStressTest() {} BatchedOpsStressTest() = default;
virtual ~BatchedOpsStressTest() {} virtual ~BatchedOpsStressTest() = default;
bool IsStateTracked() const override { return false; } bool IsStateTracked() const override { return false; }

View File

@ -16,7 +16,7 @@ class CfConsistencyStressTest : public StressTest {
public: public:
CfConsistencyStressTest() : batch_id_(0) {} CfConsistencyStressTest() : batch_id_(0) {}
~CfConsistencyStressTest() override {} ~CfConsistencyStressTest() override = default;
bool IsStateTracked() const override { return false; } bool IsStateTracked() const override { return false; }
@ -232,7 +232,7 @@ class CfConsistencyStressTest : public StressTest {
} }
db_->MultiGet(readoptionscopy, cfh, num_keys, keys.data(), values.data(), db_->MultiGet(readoptionscopy, cfh, num_keys, keys.data(), values.data(),
statuses.data()); statuses.data());
for (auto s : statuses) { for (const auto& s : statuses) {
if (s.ok()) { if (s.ok()) {
// found case // found case
thread->stats.AddGets(1, 1); thread->stats.AddGets(1, 1);

View File

@ -67,7 +67,7 @@ UniqueIdVerifier::UniqueIdVerifier(const std::string& db_name, Env* env)
std::string id(24U, '\0'); std::string id(24U, '\0');
Slice result; Slice result;
for (;;) { for (;;) {
s = reader->Read(id.size(), opts, &result, &id[0], /*dbg*/ nullptr); s = reader->Read(id.size(), opts, &result, id.data(), /*dbg*/ nullptr);
if (!s.ok()) { if (!s.ok()) {
fprintf(stderr, "Error reading unique id file: %s\n", fprintf(stderr, "Error reading unique id file: %s\n",
s.ToString().c_str()); s.ToString().c_str());

View File

@ -1784,7 +1784,7 @@ Status StressTest::TestBackupRestore(
// `ListColumnFamilies` to get names because it won't necessarily give // `ListColumnFamilies` to get names because it won't necessarily give
// the same order as `column_family_names_`. // the same order as `column_family_names_`.
assert(FLAGS_clear_column_family_one_in == 0); assert(FLAGS_clear_column_family_one_in == 0);
for (auto name : column_family_names_) { for (const auto& name : column_family_names_) {
cf_descriptors.emplace_back(name, ColumnFamilyOptions(restore_options)); cf_descriptors.emplace_back(name, ColumnFamilyOptions(restore_options));
} }
if (inplace_not_restore) { if (inplace_not_restore) {
@ -2762,12 +2762,12 @@ void StressTest::Open(SharedState* shared, bool reopen) {
if (sorted_cfn != existing_column_families) { if (sorted_cfn != existing_column_families) {
fprintf(stderr, "Expected column families differ from the existing:\n"); fprintf(stderr, "Expected column families differ from the existing:\n");
fprintf(stderr, "Expected: {"); fprintf(stderr, "Expected: {");
for (auto cf : sorted_cfn) { for (const auto& cf : sorted_cfn) {
fprintf(stderr, "%s ", cf.c_str()); fprintf(stderr, "%s ", cf.c_str());
} }
fprintf(stderr, "}\n"); fprintf(stderr, "}\n");
fprintf(stderr, "Existing: {"); fprintf(stderr, "Existing: {");
for (auto cf : existing_column_families) { for (const auto& cf : existing_column_families) {
fprintf(stderr, "%s ", cf.c_str()); fprintf(stderr, "%s ", cf.c_str());
} }
fprintf(stderr, "}\n"); fprintf(stderr, "}\n");
@ -2775,7 +2775,7 @@ void StressTest::Open(SharedState* shared, bool reopen) {
assert(sorted_cfn == existing_column_families); assert(sorted_cfn == existing_column_families);
} }
std::vector<ColumnFamilyDescriptor> cf_descriptors; std::vector<ColumnFamilyDescriptor> cf_descriptors;
for (auto name : column_family_names_) { for (const auto& name : column_family_names_) {
if (name != kDefaultColumnFamilyName) { if (name != kDefaultColumnFamilyName) {
new_column_family_name_ = new_column_family_name_ =
std::max(new_column_family_name_.load(), std::stoi(name) + 1); std::max(new_column_family_name_.load(), std::stoi(name) + 1);

View File

@ -185,7 +185,7 @@ ExpectedStateManager::ExpectedStateManager(size_t max_key,
num_column_families_(num_column_families), num_column_families_(num_column_families),
latest_(nullptr) {} latest_(nullptr) {}
ExpectedStateManager::~ExpectedStateManager() {} ExpectedStateManager::~ExpectedStateManager() = default;
const std::string FileExpectedStateManager::kLatestBasename = "LATEST"; const std::string FileExpectedStateManager::kLatestBasename = "LATEST";
const std::string FileExpectedStateManager::kStateFilenameSuffix = ".state"; const std::string FileExpectedStateManager::kStateFilenameSuffix = ".state";

View File

@ -150,7 +150,7 @@ std::string MultiOpsTxnsStressTest::Record::EncodePrimaryKey(uint32_t a) {
PutFixed32(&ret, kPrimaryIndexId); PutFixed32(&ret, kPrimaryIndexId);
PutFixed32(&ret, a); PutFixed32(&ret, a);
char* const buf = &ret[0]; char* const buf = ret.data();
std::reverse(buf, buf + sizeof(kPrimaryIndexId)); std::reverse(buf, buf + sizeof(kPrimaryIndexId));
std::reverse(buf + sizeof(kPrimaryIndexId), std::reverse(buf + sizeof(kPrimaryIndexId),
buf + sizeof(kPrimaryIndexId) + sizeof(a)); buf + sizeof(kPrimaryIndexId) + sizeof(a));
@ -162,7 +162,7 @@ std::string MultiOpsTxnsStressTest::Record::EncodeSecondaryKey(uint32_t c) {
PutFixed32(&ret, kSecondaryIndexId); PutFixed32(&ret, kSecondaryIndexId);
PutFixed32(&ret, c); PutFixed32(&ret, c);
char* const buf = &ret[0]; char* const buf = ret.data();
std::reverse(buf, buf + sizeof(kSecondaryIndexId)); std::reverse(buf, buf + sizeof(kSecondaryIndexId));
std::reverse(buf + sizeof(kSecondaryIndexId), std::reverse(buf + sizeof(kSecondaryIndexId),
buf + sizeof(kSecondaryIndexId) + sizeof(c)); buf + sizeof(kSecondaryIndexId) + sizeof(c));
@ -176,7 +176,7 @@ std::string MultiOpsTxnsStressTest::Record::EncodeSecondaryKey(uint32_t c,
PutFixed32(&ret, c); PutFixed32(&ret, c);
PutFixed32(&ret, a); PutFixed32(&ret, a);
char* const buf = &ret[0]; char* const buf = ret.data();
std::reverse(buf, buf + sizeof(kSecondaryIndexId)); std::reverse(buf, buf + sizeof(kSecondaryIndexId));
std::reverse(buf + sizeof(kSecondaryIndexId), std::reverse(buf + sizeof(kSecondaryIndexId),
buf + sizeof(kSecondaryIndexId) + sizeof(c)); buf + sizeof(kSecondaryIndexId) + sizeof(c));

View File

@ -17,9 +17,9 @@
namespace ROCKSDB_NAMESPACE { namespace ROCKSDB_NAMESPACE {
class NonBatchedOpsStressTest : public StressTest { class NonBatchedOpsStressTest : public StressTest {
public: public:
NonBatchedOpsStressTest() {} NonBatchedOpsStressTest() = default;
virtual ~NonBatchedOpsStressTest() {} virtual ~NonBatchedOpsStressTest() = default;
void VerifyDb(ThreadState* thread) const override { void VerifyDb(ThreadState* thread) const override {
// This `ReadOptions` is for validation purposes. Ignore // This `ReadOptions` is for validation purposes. Ignore
@ -624,7 +624,7 @@ class NonBatchedOpsStressTest : public StressTest {
if (!shared->AllowsOverwrite(rand_key) && if (!shared->AllowsOverwrite(rand_key) &&
shared->Exists(column_family, rand_key)) { shared->Exists(column_family, rand_key)) {
// Just do read your write checks for keys that allow overwrites. // Just do read your write checks for keys that allow overwrites.
ryw_expected_values.push_back(std::nullopt); ryw_expected_values.emplace_back(std::nullopt);
continue; continue;
} }
// With a 1 in 10 probability, insert the just added key in the batch // With a 1 in 10 probability, insert the just added key in the batch
@ -667,7 +667,7 @@ class NonBatchedOpsStressTest : public StressTest {
thread->shared->SafeTerminate(); thread->shared->SafeTerminate();
} }
} else { } else {
ryw_expected_values.push_back(std::nullopt); ryw_expected_values.emplace_back(std::nullopt);
} }
} }
} }

View File

@ -504,7 +504,7 @@ EnvWrapper::EnvWrapper(const std::shared_ptr<Env>& t) : target_(t) {
RegisterOptions("", &target_, &env_wrapper_type_info); RegisterOptions("", &target_, &env_wrapper_type_info);
} }
EnvWrapper::~EnvWrapper() {} EnvWrapper::~EnvWrapper() = default;
Status EnvWrapper::PrepareOptions(const ConfigOptions& options) { Status EnvWrapper::PrepareOptions(const ConfigOptions& options) {
target_.Prepare(); target_.Prepare();

18
env/env.cc vendored
View File

@ -359,7 +359,7 @@ class LegacyFileSystemWrapper : public FileSystem {
public: public:
// Initialize an EnvWrapper that delegates all calls to *t // Initialize an EnvWrapper that delegates all calls to *t
explicit LegacyFileSystemWrapper(Env* t) : target_(t) {} explicit LegacyFileSystemWrapper(Env* t) : target_(t) {}
~LegacyFileSystemWrapper() override {} ~LegacyFileSystemWrapper() override = default;
static const char* kClassName() { return "LegacyFileSystem"; } static const char* kClassName() { return "LegacyFileSystem"; }
const char* Name() const override { return kClassName(); } const char* Name() const override { return kClassName(); }
@ -624,7 +624,7 @@ Env::Env(const std::shared_ptr<FileSystem>& fs,
const std::shared_ptr<SystemClock>& clock) const std::shared_ptr<SystemClock>& clock)
: thread_status_updater_(nullptr), file_system_(fs), system_clock_(clock) {} : thread_status_updater_(nullptr), file_system_(fs), system_clock_(clock) {}
Env::~Env() {} Env::~Env() = default;
Status Env::NewLogger(const std::string& fname, Status Env::NewLogger(const std::string& fname,
std::shared_ptr<Logger>* result) { std::shared_ptr<Logger>* result) {
@ -797,7 +797,7 @@ std::string Env::GenerateUniqueId() {
// Use 36 character format of RFC 4122 // Use 36 character format of RFC 4122
result.resize(36U); result.resize(36U);
char* buf = &result[0]; char* buf = result.data();
PutBaseChars<16>(&buf, 8, upper >> 32, /*!uppercase*/ false); PutBaseChars<16>(&buf, 8, upper >> 32, /*!uppercase*/ false);
*(buf++) = '-'; *(buf++) = '-';
PutBaseChars<16>(&buf, 4, upper >> 16, /*!uppercase*/ false); PutBaseChars<16>(&buf, 4, upper >> 16, /*!uppercase*/ false);
@ -817,15 +817,15 @@ std::string Env::GenerateUniqueId() {
return result; return result;
} }
SequentialFile::~SequentialFile() {} SequentialFile::~SequentialFile() = default;
RandomAccessFile::~RandomAccessFile() {} RandomAccessFile::~RandomAccessFile() = default;
WritableFile::~WritableFile() {} WritableFile::~WritableFile() = default;
MemoryMappedFileBuffer::~MemoryMappedFileBuffer() {} MemoryMappedFileBuffer::~MemoryMappedFileBuffer() = default;
Logger::~Logger() {} Logger::~Logger() = default;
Status Logger::Close() { Status Logger::Close() {
if (!closed_) { if (!closed_) {
@ -838,7 +838,7 @@ Status Logger::Close() {
Status Logger::CloseImpl() { return Status::NotSupported(); } Status Logger::CloseImpl() { return Status::NotSupported(); }
FileLock::~FileLock() {} FileLock::~FileLock() = default;
void LogFlush(Logger* info_log) { void LogFlush(Logger* info_log) {
if (info_log) { if (info_log) {

View File

@ -342,7 +342,7 @@ TEST_P(EnvMoreTestWithParam, GetChildren) {
ASSERT_OK(env_->GetChildrenFileAttributes(test_dir_, &childAttr)); ASSERT_OK(env_->GetChildrenFileAttributes(test_dir_, &childAttr));
ASSERT_EQ(3U, children.size()); ASSERT_EQ(3U, children.size());
ASSERT_EQ(3U, childAttr.size()); ASSERT_EQ(3U, childAttr.size());
for (auto each : children) { for (const auto& each : children) {
env_->DeleteDir(test_dir_ + "/" + each).PermitUncheckedError(); env_->DeleteDir(test_dir_ + "/" + each).PermitUncheckedError();
} // necessary for default POSIX env } // necessary for default POSIX env

5
env/env_chroot.cc vendored
View File

@ -7,10 +7,11 @@
#include "env/env_chroot.h" #include "env/env_chroot.h"
#include <errno.h> // errno
#include <stdlib.h> // realpath, free
#include <unistd.h> // geteuid #include <unistd.h> // geteuid
#include <cerrno> // errno
#include <cstdlib> // realpath, free
#include "env/composite_env_wrapper.h" #include "env/composite_env_wrapper.h"
#include "env/fs_remap.h" #include "env/fs_remap.h"
#include "rocksdb/utilities/options_type.h" #include "rocksdb/utilities/options_type.h"

View File

@ -827,7 +827,7 @@ Status BlockAccessCipherStream::Encrypt(uint64_t fileOffset, char* data,
AllocateScratch(scratch); AllocateScratch(scratch);
// Encrypt individual blocks. // Encrypt individual blocks.
while (1) { while (true) {
char* block = data; char* block = data;
size_t n = std::min(dataSize, blockSize - blockOffset); size_t n = std::min(dataSize, blockSize - blockOffset);
if (n != blockSize) { if (n != blockSize) {
@ -871,7 +871,7 @@ Status BlockAccessCipherStream::Decrypt(uint64_t fileOffset, char* data,
AllocateScratch(scratch); AllocateScratch(scratch);
// Decrypt individual blocks. // Decrypt individual blocks.
while (1) { while (true) {
char* block = data; char* block = data;
size_t n = std::min(dataSize, blockSize - blockOffset); size_t n = std::min(dataSize, blockSize - blockOffset);
if (n != blockSize) { if (n != blockSize) {

16
env/env_posix.cc vendored
View File

@ -14,19 +14,21 @@
#ifndef ROCKSDB_NO_DYNAMIC_EXTENSION #ifndef ROCKSDB_NO_DYNAMIC_EXTENSION
#include <dlfcn.h> #include <dlfcn.h>
#endif #endif
#include <errno.h>
#include <fcntl.h> #include <fcntl.h>
#include <cerrno>
#if defined(ROCKSDB_IOURING_PRESENT) #if defined(ROCKSDB_IOURING_PRESENT)
#include <liburing.h> #include <liburing.h>
#endif #endif
#include <pthread.h> #include <pthread.h>
#include <signal.h>
#include <stdio.h>
#include <stdlib.h>
#include <string.h>
#include <sys/mman.h> #include <sys/mman.h>
#include <sys/stat.h> #include <sys/stat.h>
#include <csignal>
#include <cstdio>
#include <cstdlib>
#include <cstring>
#if defined(OS_LINUX) || defined(OS_SOLARIS) || defined(OS_ANDROID) #if defined(OS_LINUX) || defined(OS_SOLARIS) || defined(OS_ANDROID)
#include <sys/statfs.h> #include <sys/statfs.h>
#endif #endif
@ -36,10 +38,10 @@
#if defined(ROCKSDB_IOURING_PRESENT) #if defined(ROCKSDB_IOURING_PRESENT)
#include <sys/uio.h> #include <sys/uio.h>
#endif #endif
#include <time.h>
#include <unistd.h> #include <unistd.h>
#include <algorithm> #include <algorithm>
#include <ctime>
// Get nano time includes // Get nano time includes
#if defined(OS_LINUX) || defined(OS_FREEBSD) || defined(OS_GNU_KFREEBSD) #if defined(OS_LINUX) || defined(OS_FREEBSD) || defined(OS_GNU_KFREEBSD)
#elif defined(__MACH__) #elif defined(__MACH__)
@ -199,7 +201,7 @@ class PosixClock : public SystemClock {
std::string dummy; std::string dummy;
dummy.reserve(maxsize); dummy.reserve(maxsize);
dummy.resize(maxsize); dummy.resize(maxsize);
char* p = &dummy[0]; char* p = dummy.data();
port::LocalTimeR(&seconds, &t); port::LocalTimeR(&seconds, &t);
snprintf(p, maxsize, "%04d/%02d/%02d-%02d:%02d:%02d ", t.tm_year + 1900, snprintf(p, maxsize, "%04d/%02d/%02d-%02d:%02d:%02d ", t.tm_year + 1900,
t.tm_mon + 1, t.tm_mday, t.tm_hour, t.tm_min, t.tm_sec); t.tm_mon + 1, t.tm_mday, t.tm_hour, t.tm_min, t.tm_sec);

7
env/env_test.cc vendored
View File

@ -26,13 +26,14 @@
#ifdef OS_LINUX #ifdef OS_LINUX
#include <fcntl.h> #include <fcntl.h>
#include <linux/fs.h> #include <linux/fs.h>
#include <stdlib.h>
#include <sys/stat.h> #include <sys/stat.h>
#include <unistd.h> #include <unistd.h>
#include <cstdlib>
#endif #endif
#ifdef ROCKSDB_FALLOCATE_PRESENT #ifdef ROCKSDB_FALLOCATE_PRESENT
#include <errno.h> #include <cerrno>
#endif #endif
#include "db/db_impl/db_impl.h" #include "db/db_impl/db_impl.h"
@ -2955,7 +2956,7 @@ struct NoDuplicateMiniStressTest {
NoDuplicateMiniStressTest() { env = Env::Default(); } NoDuplicateMiniStressTest() { env = Env::Default(); }
virtual ~NoDuplicateMiniStressTest() {} virtual ~NoDuplicateMiniStressTest() = default;
void Run() { void Run() {
std::array<std::thread, kThreads> threads; std::array<std::thread, kThreads> threads;

4
env/file_system.cc vendored
View File

@ -22,9 +22,9 @@
namespace ROCKSDB_NAMESPACE { namespace ROCKSDB_NAMESPACE {
FileSystem::FileSystem() {} FileSystem::FileSystem() = default;
FileSystem::~FileSystem() {} FileSystem::~FileSystem() = default;
static int RegisterBuiltinFileSystems(ObjectLibrary& library, static int RegisterBuiltinFileSystems(ObjectLibrary& library,
const std::string& /*arg*/) { const std::string& /*arg*/) {

15
env/fs_posix.cc vendored
View File

@ -13,16 +13,17 @@
#ifndef ROCKSDB_NO_DYNAMIC_EXTENSION #ifndef ROCKSDB_NO_DYNAMIC_EXTENSION
#include <dlfcn.h> #include <dlfcn.h>
#endif #endif
#include <errno.h>
#include <fcntl.h> #include <fcntl.h>
#include <pthread.h> #include <pthread.h>
#include <signal.h>
#include <stdio.h>
#include <stdlib.h>
#include <string.h>
#include <sys/ioctl.h> #include <sys/ioctl.h>
#include <sys/mman.h> #include <sys/mman.h>
#include <sys/stat.h> #include <sys/stat.h>
#include <cerrno>
#include <csignal>
#include <cstdio>
#include <cstdlib>
#include <cstring>
#if defined(OS_LINUX) || defined(OS_SOLARIS) || defined(OS_ANDROID) #if defined(OS_LINUX) || defined(OS_SOLARIS) || defined(OS_ANDROID)
#include <sys/statfs.h> #include <sys/statfs.h>
#include <sys/sysmacros.h> #include <sys/sysmacros.h>
@ -30,9 +31,9 @@
#include <sys/statvfs.h> #include <sys/statvfs.h>
#include <sys/time.h> #include <sys/time.h>
#include <sys/types.h> #include <sys/types.h>
#include <time.h>
#include <algorithm> #include <algorithm>
#include <ctime>
// Get nano time includes // Get nano time includes
#if defined(OS_LINUX) || defined(OS_FREEBSD) #if defined(OS_LINUX) || defined(OS_FREEBSD)
#elif defined(__MACH__) #elif defined(__MACH__)
@ -143,7 +144,7 @@ class PosixFileSystem : public FileSystem {
const char* Name() const override { return kClassName(); } const char* Name() const override { return kClassName(); }
const char* NickName() const override { return kDefaultName(); } const char* NickName() const override { return kDefaultName(); }
~PosixFileSystem() override {} ~PosixFileSystem() override = default;
bool IsInstanceOf(const std::string& name) const override { bool IsInstanceOf(const std::string& name) const override {
if (name == "posix") { if (name == "posix") {
return true; return true;

15
env/io_posix.cc vendored
View File

@ -10,23 +10,24 @@
#ifdef ROCKSDB_LIB_IO_POSIX #ifdef ROCKSDB_LIB_IO_POSIX
#include "env/io_posix.h" #include "env/io_posix.h"
#include <errno.h>
#include <fcntl.h> #include <fcntl.h>
#include <algorithm> #include <algorithm>
#include <cerrno>
#if defined(OS_LINUX) #if defined(OS_LINUX)
#include <linux/fs.h> #include <linux/fs.h>
#ifndef FALLOC_FL_KEEP_SIZE #ifndef FALLOC_FL_KEEP_SIZE
#include <linux/falloc.h> #include <linux/falloc.h>
#endif #endif
#endif #endif
#include <stdio.h>
#include <stdlib.h>
#include <string.h>
#include <sys/ioctl.h> #include <sys/ioctl.h>
#include <sys/mman.h> #include <sys/mman.h>
#include <sys/stat.h> #include <sys/stat.h>
#include <sys/types.h> #include <sys/types.h>
#include <cstdio>
#include <cstdlib>
#include <cstring>
#ifdef OS_LINUX #ifdef OS_LINUX
#include <sys/statfs.h> #include <sys/statfs.h>
#include <sys/sysmacros.h> #include <sys/sysmacros.h>
@ -437,7 +438,7 @@ void LogicalBlockSizeCache::UnrefAndTryRemoveCachedLogicalBlockSize(
size_t LogicalBlockSizeCache::GetLogicalBlockSize(const std::string& fname, size_t LogicalBlockSizeCache::GetLogicalBlockSize(const std::string& fname,
int fd) { int fd) {
std::string dir = fname.substr(0, fname.find_last_of("/")); std::string dir = fname.substr(0, fname.find_last_of('/'));
if (dir.empty()) { if (dir.empty()) {
dir = "/"; dir = "/";
} }
@ -654,7 +655,9 @@ IOStatus PosixRandomAccessFile::MultiRead(FSReadRequest* reqs, size_t num_reqs,
size_t this_reqs = (num_reqs - reqs_off) + incomplete_rq_list.size(); size_t this_reqs = (num_reqs - reqs_off) + incomplete_rq_list.size();
// If requests exceed depth, split it into batches // If requests exceed depth, split it into batches
if (this_reqs > kIoUringDepth) this_reqs = kIoUringDepth; if (this_reqs > kIoUringDepth) {
this_reqs = kIoUringDepth;
}
assert(incomplete_rq_list.size() <= this_reqs); assert(incomplete_rq_list.size() <= this_reqs);
for (size_t i = 0; i < this_reqs; i++) { for (size_t i = 0; i < this_reqs; i++) {

2
env/mock_env.cc vendored
View File

@ -483,7 +483,7 @@ class TestMemLogger : public Logger {
options_(options), options_(options),
dbg_(dbg), dbg_(dbg),
flush_pending_(false) {} flush_pending_(false) {}
~TestMemLogger() override {} ~TestMemLogger() override = default;
void Flush() override { void Flush() override {
if (flush_pending_) { if (flush_pending_) {

12
env/mock_env_test.cc vendored
View File

@ -37,28 +37,28 @@ TEST_F(MockEnvTest, Corrupt) {
Slice result; Slice result;
std::unique_ptr<RandomAccessFile> rand_file; std::unique_ptr<RandomAccessFile> rand_file;
ASSERT_OK(env_->NewRandomAccessFile(kFileName, &rand_file, soptions_)); ASSERT_OK(env_->NewRandomAccessFile(kFileName, &rand_file, soptions_));
ASSERT_OK(rand_file->Read(0, kGood.size(), &result, &(scratch[0]))); ASSERT_OK(rand_file->Read(0, kGood.size(), &result, scratch.data()));
ASSERT_EQ(result.compare(kGood), 0); ASSERT_EQ(result.compare(kGood), 0);
// Sync + corrupt => no change // Sync + corrupt => no change
ASSERT_OK(writable_file->Fsync()); ASSERT_OK(writable_file->Fsync());
ASSERT_OK(dynamic_cast<MockEnv*>(env_)->CorruptBuffer(kFileName)); ASSERT_OK(dynamic_cast<MockEnv*>(env_)->CorruptBuffer(kFileName));
result.clear(); result.clear();
ASSERT_OK(rand_file->Read(0, kGood.size(), &result, &(scratch[0]))); ASSERT_OK(rand_file->Read(0, kGood.size(), &result, scratch.data()));
ASSERT_EQ(result.compare(kGood), 0); ASSERT_EQ(result.compare(kGood), 0);
// Add new data and corrupt it // Add new data and corrupt it
ASSERT_OK(writable_file->Append(kCorrupted)); ASSERT_OK(writable_file->Append(kCorrupted));
ASSERT_TRUE(writable_file->GetFileSize() == kGood.size() + kCorrupted.size()); ASSERT_TRUE(writable_file->GetFileSize() == kGood.size() + kCorrupted.size());
result.clear(); result.clear();
ASSERT_OK( ASSERT_OK(rand_file->Read(kGood.size(), kCorrupted.size(), &result,
rand_file->Read(kGood.size(), kCorrupted.size(), &result, &(scratch[0]))); scratch.data()));
ASSERT_EQ(result.compare(kCorrupted), 0); ASSERT_EQ(result.compare(kCorrupted), 0);
// Corrupted // Corrupted
ASSERT_OK(dynamic_cast<MockEnv*>(env_)->CorruptBuffer(kFileName)); ASSERT_OK(dynamic_cast<MockEnv*>(env_)->CorruptBuffer(kFileName));
result.clear(); result.clear();
ASSERT_OK( ASSERT_OK(rand_file->Read(kGood.size(), kCorrupted.size(), &result,
rand_file->Read(kGood.size(), kCorrupted.size(), &result, &(scratch[0]))); scratch.data()));
ASSERT_NE(result.compare(kCorrupted), 0); ASSERT_NE(result.compare(kCorrupted), 0);
} }

View File

@ -177,7 +177,7 @@ Status DeleteScheduler::CleanupDirectory(Env* env, SstFileManagerImpl* sfm,
Status DeleteScheduler::MarkAsTrash(const std::string& file_path, Status DeleteScheduler::MarkAsTrash(const std::string& file_path,
std::string* trash_file) { std::string* trash_file) {
// Sanity check of the path // Sanity check of the path
size_t idx = file_path.rfind("/"); size_t idx = file_path.rfind('/');
if (idx == std::string::npos || idx == file_path.size() - 1) { if (idx == std::string::npos || idx == file_path.size() - 1) {
return Status::InvalidArgument("file_path is corrupted"); return Status::InvalidArgument("file_path is corrupted");
} }

View File

@ -8,10 +8,9 @@
// found in the LICENSE file. See the AUTHORS file for names of contributors. // found in the LICENSE file. See the AUTHORS file for names of contributors.
#include "file/filename.h" #include "file/filename.h"
#include <ctype.h> #include <cctype>
#include <stdio.h>
#include <cinttypes> #include <cinttypes>
#include <cstdio>
#include <vector> #include <vector>
#include "file/writable_file_writer.h" #include "file/writable_file_writer.h"

View File

@ -425,7 +425,9 @@ TEST(FSReadRequest, TryMerge) {
src.scratch = nullptr; src.scratch = nullptr;
ASSERT_OK(src.status); ASSERT_OK(src.status);
if (reverse) std::swap(dest, src); if (reverse) {
std::swap(dest, src);
}
ASSERT_TRUE(TryMerge(&dest, src)); ASSERT_TRUE(TryMerge(&dest, src));
ASSERT_EQ(dest.offset, 0); ASSERT_EQ(dest.offset, 0);
ASSERT_EQ(dest.len, 10); ASSERT_EQ(dest.len, 10);
@ -448,7 +450,9 @@ TEST(FSReadRequest, TryMerge) {
src.scratch = nullptr; src.scratch = nullptr;
ASSERT_OK(src.status); ASSERT_OK(src.status);
if (reverse) std::swap(dest, src); if (reverse) {
std::swap(dest, src);
}
ASSERT_TRUE(TryMerge(&dest, src)); ASSERT_TRUE(TryMerge(&dest, src));
ASSERT_EQ(dest.offset, 0); ASSERT_EQ(dest.offset, 0);
ASSERT_EQ(dest.len, 10); ASSERT_EQ(dest.len, 10);
@ -471,7 +475,9 @@ TEST(FSReadRequest, TryMerge) {
src.scratch = nullptr; src.scratch = nullptr;
ASSERT_OK(src.status); ASSERT_OK(src.status);
if (reverse) std::swap(dest, src); if (reverse) {
std::swap(dest, src);
}
ASSERT_TRUE(TryMerge(&dest, src)); ASSERT_TRUE(TryMerge(&dest, src));
ASSERT_EQ(dest.offset, 0); ASSERT_EQ(dest.offset, 0);
ASSERT_EQ(dest.len, 10); ASSERT_EQ(dest.len, 10);