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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -58,7 +58,7 @@ class ManualCompactionTest : public testing::Test {
class DestroyAllCompactionFilter : public CompactionFilter {
public:
DestroyAllCompactionFilter() {}
DestroyAllCompactionFilter() = default;
bool Filter(int /*level*/, const Slice& /*key*/, const Slice& existing_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();
}
}
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) {
if (!may_match[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);
assert(s.ok());
if (!s.ok()) return s;
if (!s.ok()) {
return s;
}
assert(kTypeMerge == orig_ikey.type);

View File

@ -3,8 +3,7 @@
// COPYING file in the root directory) and Apache 2.0 License
// (found in the LICENSE.Apache file in the root directory).
//
#include <assert.h>
#include <cassert>
#include <iostream>
#include <memory>
@ -87,7 +86,9 @@ class EnvMergeTest : public EnvWrapper {
static std::unique_ptr<EnvMergeTest> singleton_;
static EnvMergeTest* GetInstance() {
if (nullptr == singleton_) singleton_.reset(new EnvMergeTest);
if (nullptr == singleton_) {
singleton_.reset(new EnvMergeTest);
}
return singleton_.get();
}
};
@ -145,7 +146,7 @@ class Counters {
assert(db_);
}
virtual ~Counters() {}
virtual ~Counters() = default;
// public interface of Counters.
// All four functions return false
@ -194,7 +195,7 @@ class Counters {
std::cerr << "value corruption\n";
return false;
}
*value = DecodeFixed64(&str[0]);
*value = DecodeFixed64(str.data());
return true;
} else {
std::cerr << s.ToString() << std::endl;
@ -220,14 +221,18 @@ class Counters {
uint64_t value = default_;
int result = get(key, &value);
assert(result);
if (result == 0) exit(1); // Disable unused variable warning.
if (result == 0) {
exit(1); // Disable unused variable warning.
}
return value;
}
void assert_add(const std::string& key, uint64_t value) {
int result = add(key, value);
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;
ASSERT_OK(db->Get(ReadOptions(), key, &get_value_str));
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(num_merge_operator_calls,
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
// found in the LICENSE file. See the AUTHORS file for names of contributors.
#include <stdlib.h>
#include <algorithm>
#include <cstdlib>
#include <map>
#include <string>
#include <vector>
@ -68,7 +66,7 @@ class ObsoleteFilesTest : public DBTestBase {
int log_cnt = 0;
int sst_cnt = 0;
int manifest_cnt = 0;
for (auto file : filenames) {
for (const auto& file : filenames) {
uint64_t number;
FileType type;
if (ParseFileName(file, &number, &type)) {

View File

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

View File

@ -1050,7 +1050,7 @@ TEST_F(PerfContextTest, MergeOperandCount) {
std::vector<Status> statuses(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) {
ASSERT_OK(statuses[i]);
@ -1068,7 +1068,7 @@ TEST_F(PerfContextTest, MergeOperandCount) {
std::vector<Status> statuses(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) {
ASSERT_OK(statuses[i]);

View File

@ -292,7 +292,7 @@ class TestPlainTableReader : public PlainTableReader {
table_properties_ = std::move(props);
}
~TestPlainTableReader() override {}
~TestPlainTableReader() override = default;
private:
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_b = &kkey_b;
if (key_a->prefix != key_b->prefix) {
if (key_a->prefix < key_b->prefix) return -1;
if (key_a->prefix > key_b->prefix) return 1;
if (key_a->prefix < key_b->prefix) {
return -1;
}
if (key_a->prefix > key_b->prefix) {
return 1;
}
} else {
EXPECT_TRUE(key_a->prefix == key_b->prefix);
// note, both a and b could be prefix only
@ -99,8 +103,12 @@ class TestKeyComparator : public Comparator {
EXPECT_TRUE(
(a.size() == sizeof(uint64_t) && b.size() == sizeof(TestKey)) ||
(b.size() == sizeof(uint64_t) && a.size() == sizeof(TestKey)));
if (a.size() < b.size()) return -1;
if (a.size() > b.size()) return 1;
if (a.size() < b.size()) {
return -1;
}
if (a.size() > b.size()) {
return 1;
}
} else {
// both a and b are prefix
if (a.size() == sizeof(uint64_t)) {
@ -109,9 +117,15 @@ class TestKeyComparator : public Comparator {
// both a and b are whole key
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) 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 1;
}
if (key_a->sorted == key_b->sorted) {
return 0;
}
}
}
return 0;
@ -892,4 +906,3 @@ int main(int argc, char** argv) {
}
#endif // GFLAGS

View File

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

View File

@ -378,8 +378,7 @@ TEST_P(RepairTestWithTimestamp, UnflushedSst) {
ColumnFamilyOptions cf_options(options);
std::vector<ColumnFamilyDescriptor> column_families;
column_families.push_back(
ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options));
column_families.emplace_back(kDefaultColumnFamilyName, cf_options);
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(
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_slow */) override {
// Nothing to do.
return;
}
UserCollectedProperties GetReadableProperties() const override {
@ -227,7 +226,7 @@ class FlushBlockEveryThreePolicy : public FlushBlockPolicy {
class FlushBlockEveryThreePolicyFactory : public FlushBlockPolicyFactory {
public:
explicit FlushBlockEveryThreePolicyFactory() {}
explicit FlushBlockEveryThreePolicyFactory() = default;
const char* Name() const override {
return "FlushBlockEveryThreePolicyFactory";

View File

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

View File

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

View File

@ -122,7 +122,9 @@ Status OverlapWithIterator(const Comparator* ucmp,
ParsedInternalKey seek_result;
Status s = ParseInternalKey(iter->key(), &seek_result,
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) <=
0) {
@ -835,9 +837,9 @@ Version::~Version() {
assert(cfd_ != nullptr);
uint32_t path_id = f->fd.GetPathId();
assert(path_id < cfd_->ioptions()->cf_paths.size());
vset_->obsolete_files_.push_back(
ObsoleteFileInfo(f, cfd_->ioptions()->cf_paths[path_id].path,
cfd_->GetFileMetadataCacheReservationManager()));
vset_->obsolete_files_.emplace_back(
f, cfd_->ioptions()->cf_paths[path_id].path,
cfd_->GetFileMetadataCacheReservationManager());
}
}
}
@ -3101,7 +3103,9 @@ bool Version::MaybeInitializeFileMetaData(const ReadOptions& read_options,
file_meta->fd.GetNumber(), s.ToString().c_str());
return false;
}
if (tp.get() == nullptr) return false;
if (tp.get() == nullptr) {
return false;
}
file_meta->num_entries = tp->num_entries;
file_meta->num_deletions = tp->num_deletions;
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++) {
int sz = sizeof(scratch->buffer) - len;
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;
}
if (len > 0) {
@ -4545,7 +4551,9 @@ const char* VersionStorageInfo::LevelFileSummary(FileSummaryStorage* scratch,
"#%" PRIu64 "(seq=%" PRIu64 ",sz=%s,%d) ",
f->fd.GetNumber(), f->fd.smallest_seqno, sztxt,
static_cast<int>(f->being_compacted));
if (ret < 0 || ret >= sz) break;
if (ret < 0 || ret >= sz) {
break;
}
len += ret;
}
// 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_) {
assert(curr_state.find(cfd->GetID()) == curr_state.end());
curr_state.emplace(std::make_pair(
curr_state.emplace(
cfd->GetID(),
MutableCFState(cfd->GetLogNumber(), cfd->GetFullHistoryTsLow())));
MutableCFState(cfd->GetLogNumber(), cfd->GetFullHistoryTsLow()));
}
for (const auto& wal : wals_.GetWals()) {
@ -7277,7 +7285,7 @@ ReactiveVersionSet::ReactiveVersionSet(
/*db_session_id*/ "", /*daily_offpeak_time_utc*/ "",
/*error_handler=*/nullptr) {}
ReactiveVersionSet::~ReactiveVersionSet() {}
ReactiveVersionSet::~ReactiveVersionSet() = default;
Status ReactiveVersionSet::Recover(
const std::vector<ColumnFamilyDescriptor>& column_families,

View File

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

View File

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

View File

@ -233,9 +233,9 @@ WriteBatch& WriteBatch::operator=(WriteBatch&& src) {
return *this;
}
WriteBatch::~WriteBatch() {}
WriteBatch::~WriteBatch() = default;
WriteBatch::Handler::~Handler() {}
WriteBatch::Handler::~Handler() = default;
void WriteBatch::Handler::LogData(const Slice& /*blob*/) {
// 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) {
EncodeFixed64(&b->rep_[0], seq);
EncodeFixed64(b->rep_.data(), seq);
}
size_t WriteBatchInternal::GetFirstOffset(WriteBatch* /*b*/) {
@ -1856,7 +1856,9 @@ class MemTableInserter : public WriteBatch::Handler {
}
void DecrementProtectionInfoIdxForTryAgain() {
if (prot_info_ != nullptr) --prot_info_idx_;
if (prot_info_ != nullptr) {
--prot_info_idx_;
}
}
void ResetProtectionInfo() {
@ -3018,7 +3020,7 @@ class ProtectionInfoUpdater : public WriteBatch::Handler {
explicit ProtectionInfoUpdater(WriteBatch::ProtectionInfo* prot_info)
: prot_info_(prot_info) {}
~ProtectionInfoUpdater() override {}
~ProtectionInfoUpdater() override = default;
Status PutCF(uint32_t cf, const Slice& key, const Slice& val) override {
return UpdateProtInfo(cf, key, val, kTypeValue);

View File

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

View File

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

View File

@ -16,7 +16,7 @@ class CfConsistencyStressTest : public StressTest {
public:
CfConsistencyStressTest() : batch_id_(0) {}
~CfConsistencyStressTest() override {}
~CfConsistencyStressTest() override = default;
bool IsStateTracked() const override { return false; }
@ -232,7 +232,7 @@ class CfConsistencyStressTest : public StressTest {
}
db_->MultiGet(readoptionscopy, cfh, num_keys, keys.data(), values.data(),
statuses.data());
for (auto s : statuses) {
for (const auto& s : statuses) {
if (s.ok()) {
// found case
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');
Slice result;
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()) {
fprintf(stderr, "Error reading unique id file: %s\n",
s.ToString().c_str());

View File

@ -1784,7 +1784,7 @@ Status StressTest::TestBackupRestore(
// `ListColumnFamilies` to get names because it won't necessarily give
// the same order as `column_family_names_`.
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));
}
if (inplace_not_restore) {
@ -2762,12 +2762,12 @@ void StressTest::Open(SharedState* shared, bool reopen) {
if (sorted_cfn != existing_column_families) {
fprintf(stderr, "Expected column families differ from the existing:\n");
fprintf(stderr, "Expected: {");
for (auto cf : sorted_cfn) {
for (const auto& cf : sorted_cfn) {
fprintf(stderr, "%s ", cf.c_str());
}
fprintf(stderr, "}\n");
fprintf(stderr, "Existing: {");
for (auto cf : existing_column_families) {
for (const auto& cf : existing_column_families) {
fprintf(stderr, "%s ", cf.c_str());
}
fprintf(stderr, "}\n");
@ -2775,7 +2775,7 @@ void StressTest::Open(SharedState* shared, bool reopen) {
assert(sorted_cfn == existing_column_families);
}
std::vector<ColumnFamilyDescriptor> cf_descriptors;
for (auto name : column_family_names_) {
for (const auto& name : column_family_names_) {
if (name != kDefaultColumnFamilyName) {
new_column_family_name_ =
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),
latest_(nullptr) {}
ExpectedStateManager::~ExpectedStateManager() {}
ExpectedStateManager::~ExpectedStateManager() = default;
const std::string FileExpectedStateManager::kLatestBasename = "LATEST";
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, a);
char* const buf = &ret[0];
char* const buf = ret.data();
std::reverse(buf, buf + sizeof(kPrimaryIndexId));
std::reverse(buf + sizeof(kPrimaryIndexId),
buf + sizeof(kPrimaryIndexId) + sizeof(a));
@ -162,7 +162,7 @@ std::string MultiOpsTxnsStressTest::Record::EncodeSecondaryKey(uint32_t c) {
PutFixed32(&ret, kSecondaryIndexId);
PutFixed32(&ret, c);
char* const buf = &ret[0];
char* const buf = ret.data();
std::reverse(buf, buf + sizeof(kSecondaryIndexId));
std::reverse(buf + sizeof(kSecondaryIndexId),
buf + sizeof(kSecondaryIndexId) + sizeof(c));
@ -176,7 +176,7 @@ std::string MultiOpsTxnsStressTest::Record::EncodeSecondaryKey(uint32_t c,
PutFixed32(&ret, c);
PutFixed32(&ret, a);
char* const buf = &ret[0];
char* const buf = ret.data();
std::reverse(buf, buf + sizeof(kSecondaryIndexId));
std::reverse(buf + sizeof(kSecondaryIndexId),
buf + sizeof(kSecondaryIndexId) + sizeof(c));

View File

@ -17,9 +17,9 @@
namespace ROCKSDB_NAMESPACE {
class NonBatchedOpsStressTest : public StressTest {
public:
NonBatchedOpsStressTest() {}
NonBatchedOpsStressTest() = default;
virtual ~NonBatchedOpsStressTest() {}
virtual ~NonBatchedOpsStressTest() = default;
void VerifyDb(ThreadState* thread) const override {
// This `ReadOptions` is for validation purposes. Ignore
@ -624,7 +624,7 @@ class NonBatchedOpsStressTest : public StressTest {
if (!shared->AllowsOverwrite(rand_key) &&
shared->Exists(column_family, rand_key)) {
// 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;
}
// 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();
}
} 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);
}
EnvWrapper::~EnvWrapper() {}
EnvWrapper::~EnvWrapper() = default;
Status EnvWrapper::PrepareOptions(const ConfigOptions& options) {
target_.Prepare();

18
env/env.cc vendored
View File

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

View File

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

5
env/env_chroot.cc vendored
View File

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

View File

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

16
env/env_posix.cc vendored
View File

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

7
env/env_test.cc vendored
View File

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

4
env/file_system.cc vendored
View File

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

15
env/fs_posix.cc vendored
View File

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

15
env/io_posix.cc vendored
View File

@ -10,23 +10,24 @@
#ifdef ROCKSDB_LIB_IO_POSIX
#include "env/io_posix.h"
#include <errno.h>
#include <fcntl.h>
#include <algorithm>
#include <cerrno>
#if defined(OS_LINUX)
#include <linux/fs.h>
#ifndef FALLOC_FL_KEEP_SIZE
#include <linux/falloc.h>
#endif
#endif
#include <stdio.h>
#include <stdlib.h>
#include <string.h>
#include <sys/ioctl.h>
#include <sys/mman.h>
#include <sys/stat.h>
#include <sys/types.h>
#include <cstdio>
#include <cstdlib>
#include <cstring>
#ifdef OS_LINUX
#include <sys/statfs.h>
#include <sys/sysmacros.h>
@ -437,7 +438,7 @@ void LogicalBlockSizeCache::UnrefAndTryRemoveCachedLogicalBlockSize(
size_t LogicalBlockSizeCache::GetLogicalBlockSize(const std::string& fname,
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()) {
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();
// 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);
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),
dbg_(dbg),
flush_pending_(false) {}
~TestMemLogger() override {}
~TestMemLogger() override = default;
void Flush() override {
if (flush_pending_) {

12
env/mock_env_test.cc vendored
View File

@ -37,28 +37,28 @@ TEST_F(MockEnvTest, Corrupt) {
Slice result;
std::unique_ptr<RandomAccessFile> rand_file;
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);
// Sync + corrupt => no change
ASSERT_OK(writable_file->Fsync());
ASSERT_OK(dynamic_cast<MockEnv*>(env_)->CorruptBuffer(kFileName));
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);
// Add new data and corrupt it
ASSERT_OK(writable_file->Append(kCorrupted));
ASSERT_TRUE(writable_file->GetFileSize() == kGood.size() + kCorrupted.size());
result.clear();
ASSERT_OK(
rand_file->Read(kGood.size(), kCorrupted.size(), &result, &(scratch[0])));
ASSERT_OK(rand_file->Read(kGood.size(), kCorrupted.size(), &result,
scratch.data()));
ASSERT_EQ(result.compare(kCorrupted), 0);
// Corrupted
ASSERT_OK(dynamic_cast<MockEnv*>(env_)->CorruptBuffer(kFileName));
result.clear();
ASSERT_OK(
rand_file->Read(kGood.size(), kCorrupted.size(), &result, &(scratch[0])));
ASSERT_OK(rand_file->Read(kGood.size(), kCorrupted.size(), &result,
scratch.data()));
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,
std::string* trash_file) {
// 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) {
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.
#include "file/filename.h"
#include <ctype.h>
#include <stdio.h>
#include <cctype>
#include <cinttypes>
#include <cstdio>
#include <vector>
#include "file/writable_file_writer.h"

View File

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