Enable clang-tidy readability-misleading-indentation

Official docs:

  Correct indentation helps to understand code. Mismatch of the
  syntactical structure and the indentation of the code may hide serious
  problems.
This commit is contained in:
Robert Schulze 2022-05-05 22:31:59 +02:00
parent 3d3977bad3
commit 61cbcbf073
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
7 changed files with 164 additions and 165 deletions

View File

@ -130,7 +130,6 @@ Checks: '*,
-readability-implicit-bool-conversion,
-readability-isolate-declaration,
-readability-magic-numbers,
-readability-misleading-indentation,
-readability-named-parameter,
-readability-qualified-auto,
-readability-redundant-declaration,

View File

@ -627,27 +627,27 @@ bool DiskLocal::setup()
/// Try to create a new checker file. The disk status can be either broken or readonly.
if (disk_checker_magic_number == -1)
try
{
pcg32_fast rng(randomSeed());
UInt32 magic_number = rng();
try
{
auto buf = writeFile(disk_checker_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, {});
writeIntBinary(magic_number, *buf);
pcg32_fast rng(randomSeed());
UInt32 magic_number = rng();
{
auto buf = writeFile(disk_checker_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, {});
writeIntBinary(magic_number, *buf);
}
disk_checker_magic_number = magic_number;
}
catch (...)
{
LOG_WARNING(
logger,
"Cannot create/write to {0}. Disk {1} is either readonly or broken. Without setting up disk checker file, DiskLocalCheckThread "
"will not be started. Disk is assumed to be RW. Try manually fix the disk and do `SYSTEM RESTART DISK {1}`",
disk_checker_path,
name);
disk_checker_can_check_read = false;
return true;
}
disk_checker_magic_number = magic_number;
}
catch (...)
{
LOG_WARNING(
logger,
"Cannot create/write to {0}. Disk {1} is either readonly or broken. Without setting up disk checker file, DiskLocalCheckThread "
"will not be started. Disk is assumed to be RW. Try manually fix the disk and do `SYSTEM RESTART DISK {1}`",
disk_checker_path,
name);
disk_checker_can_check_read = false;
return true;
}
if (disk_checker_magic_number == -1)
throw Exception("disk_checker_magic_number is not initialized. It's a bug", ErrorCodes::LOGICAL_ERROR);

View File

@ -700,8 +700,8 @@ ColumnPtr FunctionArrayElement::executeArgument(
|| (res = executeConst<IndexType>(arguments, result_type, index_data, builder, input_rows_count))
|| (res = executeString<IndexType>(arguments, index_data, builder))
|| (res = executeGeneric<IndexType>(arguments, index_data, builder))))
throw Exception("Illegal column " + arguments[0].column->getName()
+ " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
throw Exception("Illegal column " + arguments[0].column->getName()
+ " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
return res;
}

View File

@ -151,21 +151,21 @@ private:
}
return dst;
}
else if (const_cnt == 2)
{
const auto * col_const_x = assert_cast<const ColumnConst *> (col_x);
const auto * col_const_y = assert_cast<const ColumnConst *> (col_y);
size_t start_index = 0;
UInt8 res = isPointInEllipses(col_const_x->getValue<Float64>(), col_const_y->getValue<Float64>(), ellipses.data(), ellipses_count, start_index);
return DataTypeUInt8().createColumnConst(size, res);
}
else
{
throw Exception(
"Illegal types " + col_x->getName() + ", " + col_y->getName() + " of arguments 1, 2 of function " + getName() + ". Both must be either const or vector",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
else if (const_cnt == 2)
{
const auto * col_const_x = assert_cast<const ColumnConst *> (col_x);
const auto * col_const_y = assert_cast<const ColumnConst *> (col_y);
size_t start_index = 0;
UInt8 res = isPointInEllipses(col_const_x->getValue<Float64>(), col_const_y->getValue<Float64>(), ellipses.data(), ellipses_count, start_index);
return DataTypeUInt8().createColumnConst(size, res);
}
else
{
throw Exception(
"Illegal types " + col_x->getName() + ", " + col_y->getName() + " of arguments 1, 2 of function " + getName() + ". Both must be either const or vector",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
static bool isPointInEllipses(Float64 x, Float64 y, const Ellipse * ellipses, size_t ellipses_count, size_t & start_index)

View File

@ -230,7 +230,7 @@ void SessionLog::addLoginSuccess(const UUID & auth_id, std::optional<String> ses
log_entry.roles = roles_info->getCurrentRolesNames();
if (const auto profile_info = access->getDefaultProfileInfo())
log_entry.profiles = profile_info->getProfileNames();
log_entry.profiles = profile_info->getProfileNames();
for (const auto & s : settings.allChanged())
log_entry.settings.emplace_back(s.getName(), s.getValueString());

View File

@ -6490,153 +6490,153 @@ ReservationPtr MergeTreeData::balancedReservation(
ReservationPtr reserved_space;
auto min_bytes_to_rebalance_partition_over_jbod = getSettings()->min_bytes_to_rebalance_partition_over_jbod;
if (tagger_ptr && min_bytes_to_rebalance_partition_over_jbod > 0 && part_size >= min_bytes_to_rebalance_partition_over_jbod)
try
{
const auto & disks = getStoragePolicy()->getVolume(max_volume_index)->getDisks();
std::map<String, size_t> disk_occupation;
std::map<String, std::vector<String>> disk_parts_for_logging;
for (const auto & disk : disks)
disk_occupation.emplace(disk->getName(), 0);
std::set<String> committed_big_parts_from_partition;
std::set<String> submerging_big_parts_from_partition;
std::lock_guard lock(currently_submerging_emerging_mutex);
for (const auto & part : currently_submerging_big_parts)
try
{
if (part_info.partition_id == part->info.partition_id)
submerging_big_parts_from_partition.insert(part->name);
}
const auto & disks = getStoragePolicy()->getVolume(max_volume_index)->getDisks();
std::map<String, size_t> disk_occupation;
std::map<String, std::vector<String>> disk_parts_for_logging;
for (const auto & disk : disks)
disk_occupation.emplace(disk->getName(), 0);
{
auto lock_parts = lockParts();
if (covered_parts.empty())
std::set<String> committed_big_parts_from_partition;
std::set<String> submerging_big_parts_from_partition;
std::lock_guard lock(currently_submerging_emerging_mutex);
for (const auto & part : currently_submerging_big_parts)
{
// It's a part fetch. Calculate `covered_parts` here.
MergeTreeData::DataPartPtr covering_part;
covered_parts = getActivePartsToReplace(part_info, part_name, covering_part, lock_parts);
if (part_info.partition_id == part->info.partition_id)
submerging_big_parts_from_partition.insert(part->name);
}
// Remove irrelevant parts.
std::erase_if(covered_parts,
[min_bytes_to_rebalance_partition_over_jbod](const auto & part)
{
return !(part->isStoredOnDisk() && part->getBytesOnDisk() >= min_bytes_to_rebalance_partition_over_jbod);
});
// Include current submerging big parts which are not yet in `currently_submerging_big_parts`
for (const auto & part : covered_parts)
submerging_big_parts_from_partition.insert(part->name);
for (const auto & part : getDataPartsStateRange(MergeTreeData::DataPartState::Active))
{
if (part->isStoredOnDisk() && part->getBytesOnDisk() >= min_bytes_to_rebalance_partition_over_jbod
&& part_info.partition_id == part->info.partition_id)
auto lock_parts = lockParts();
if (covered_parts.empty())
{
auto name = part->volume->getDisk()->getName();
auto it = disk_occupation.find(name);
if (it != disk_occupation.end())
{
if (submerging_big_parts_from_partition.find(part->name) == submerging_big_parts_from_partition.end())
// It's a part fetch. Calculate `covered_parts` here.
MergeTreeData::DataPartPtr covering_part;
covered_parts = getActivePartsToReplace(part_info, part_name, covering_part, lock_parts);
}
// Remove irrelevant parts.
std::erase_if(covered_parts,
[min_bytes_to_rebalance_partition_over_jbod](const auto & part)
{
it->second += part->getBytesOnDisk();
disk_parts_for_logging[name].push_back(formatReadableSizeWithBinarySuffix(part->getBytesOnDisk()));
committed_big_parts_from_partition.insert(part->name);
return !(part->isStoredOnDisk() && part->getBytesOnDisk() >= min_bytes_to_rebalance_partition_over_jbod);
});
// Include current submerging big parts which are not yet in `currently_submerging_big_parts`
for (const auto & part : covered_parts)
submerging_big_parts_from_partition.insert(part->name);
for (const auto & part : getDataPartsStateRange(MergeTreeData::DataPartState::Active))
{
if (part->isStoredOnDisk() && part->getBytesOnDisk() >= min_bytes_to_rebalance_partition_over_jbod
&& part_info.partition_id == part->info.partition_id)
{
auto name = part->volume->getDisk()->getName();
auto it = disk_occupation.find(name);
if (it != disk_occupation.end())
{
if (submerging_big_parts_from_partition.find(part->name) == submerging_big_parts_from_partition.end())
{
it->second += part->getBytesOnDisk();
disk_parts_for_logging[name].push_back(formatReadableSizeWithBinarySuffix(part->getBytesOnDisk()));
committed_big_parts_from_partition.insert(part->name);
}
else
{
disk_parts_for_logging[name].push_back(formatReadableSizeWithBinarySuffix(part->getBytesOnDisk()) + " (submerging)");
}
}
else
{
disk_parts_for_logging[name].push_back(formatReadableSizeWithBinarySuffix(part->getBytesOnDisk()) + " (submerging)");
// Part is on different volume. Ignore it.
}
}
}
}
for (const auto & [name, emerging_part] : currently_emerging_big_parts)
{
// It's possible that the emerging big parts are committed and get added twice. Thus a set is used to deduplicate.
if (committed_big_parts_from_partition.find(name) == committed_big_parts_from_partition.end()
&& part_info.partition_id == emerging_part.partition_id)
{
auto it = disk_occupation.find(emerging_part.disk_name);
if (it != disk_occupation.end())
{
it->second += emerging_part.estimate_bytes;
disk_parts_for_logging[emerging_part.disk_name].push_back(
formatReadableSizeWithBinarySuffix(emerging_part.estimate_bytes) + " (emerging)");
}
else
{
// Part is on different volume. Ignore it.
}
}
}
}
for (const auto & [name, emerging_part] : currently_emerging_big_parts)
{
// It's possible that the emerging big parts are committed and get added twice. Thus a set is used to deduplicate.
if (committed_big_parts_from_partition.find(name) == committed_big_parts_from_partition.end()
&& part_info.partition_id == emerging_part.partition_id)
size_t min_occupation_size = std::numeric_limits<size_t>::max();
std::vector<String> candidates;
for (const auto & [disk_name, size] : disk_occupation)
{
auto it = disk_occupation.find(emerging_part.disk_name);
if (it != disk_occupation.end())
if (size < min_occupation_size)
{
it->second += emerging_part.estimate_bytes;
disk_parts_for_logging[emerging_part.disk_name].push_back(
formatReadableSizeWithBinarySuffix(emerging_part.estimate_bytes) + " (emerging)");
min_occupation_size = size;
candidates = {disk_name};
}
else if (size == min_occupation_size)
{
candidates.push_back(disk_name);
}
}
if (!candidates.empty())
{
// Random pick one disk from best candidates
std::shuffle(candidates.begin(), candidates.end(), thread_local_rng);
String selected_disk_name = candidates.front();
WriteBufferFromOwnString log_str;
writeCString("\nbalancer: \n", log_str);
for (const auto & [disk_name, per_disk_parts] : disk_parts_for_logging)
writeString(fmt::format(" {}: [{}]\n", disk_name, fmt::join(per_disk_parts, ", ")), log_str);
LOG_DEBUG(log, fmt::runtime(log_str.str()));
if (ttl_infos)
reserved_space = tryReserveSpacePreferringTTLRules(
metadata_snapshot,
part_size,
*ttl_infos,
time(nullptr),
max_volume_index,
is_insert,
getStoragePolicy()->getDiskByName(selected_disk_name));
else
reserved_space = tryReserveSpace(part_size, getStoragePolicy()->getDiskByName(selected_disk_name));
if (reserved_space)
{
// Part is on different volume. Ignore it.
currently_emerging_big_parts.emplace(
part_name, EmergingPartInfo{reserved_space->getDisk(0)->getName(), part_info.partition_id, part_size});
for (const auto & part : covered_parts)
{
if (currently_submerging_big_parts.contains(part))
LOG_WARNING(log, "currently_submerging_big_parts contains duplicates. JBOD might lose balance");
else
currently_submerging_big_parts.insert(part);
}
// Record submerging big parts in the tagger to clean them up.
tagger_ptr->emplace(*this, part_name, std::move(covered_parts), log);
}
}
}
size_t min_occupation_size = std::numeric_limits<size_t>::max();
std::vector<String> candidates;
for (const auto & [disk_name, size] : disk_occupation)
catch (...)
{
if (size < min_occupation_size)
{
min_occupation_size = size;
candidates = {disk_name};
}
else if (size == min_occupation_size)
{
candidates.push_back(disk_name);
}
LOG_DEBUG(log, "JBOD balancer encounters an error. Fallback to random disk selection");
tryLogCurrentException(log);
}
if (!candidates.empty())
{
// Random pick one disk from best candidates
std::shuffle(candidates.begin(), candidates.end(), thread_local_rng);
String selected_disk_name = candidates.front();
WriteBufferFromOwnString log_str;
writeCString("\nbalancer: \n", log_str);
for (const auto & [disk_name, per_disk_parts] : disk_parts_for_logging)
writeString(fmt::format(" {}: [{}]\n", disk_name, fmt::join(per_disk_parts, ", ")), log_str);
LOG_DEBUG(log, fmt::runtime(log_str.str()));
if (ttl_infos)
reserved_space = tryReserveSpacePreferringTTLRules(
metadata_snapshot,
part_size,
*ttl_infos,
time(nullptr),
max_volume_index,
is_insert,
getStoragePolicy()->getDiskByName(selected_disk_name));
else
reserved_space = tryReserveSpace(part_size, getStoragePolicy()->getDiskByName(selected_disk_name));
if (reserved_space)
{
currently_emerging_big_parts.emplace(
part_name, EmergingPartInfo{reserved_space->getDisk(0)->getName(), part_info.partition_id, part_size});
for (const auto & part : covered_parts)
{
if (currently_submerging_big_parts.contains(part))
LOG_WARNING(log, "currently_submerging_big_parts contains duplicates. JBOD might lose balance");
else
currently_submerging_big_parts.insert(part);
}
// Record submerging big parts in the tagger to clean them up.
tagger_ptr->emplace(*this, part_name, std::move(covered_parts), log);
}
}
}
catch (...)
{
LOG_DEBUG(log, "JBOD balancer encounters an error. Fallback to random disk selection");
tryLogCurrentException(log);
}
return reserved_space;
}

View File

@ -70,16 +70,16 @@ void StorageMongoDB::connectIfNotConnected()
auto auth_db = database_name;
if (auth_source != query_params.end())
auth_db = auth_source->second;
# if POCO_VERSION >= 0x01070800
if (!username.empty() && !password.empty())
{
Poco::MongoDB::Database poco_db(auth_db);
if (!poco_db.authenticate(*connection, username, password, Poco::MongoDB::Database::AUTH_SCRAM_SHA1))
throw Exception("Cannot authenticate in MongoDB, incorrect user or password", ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
}
# else
authenticate(*connection, database_name, username, password);
# endif
#if POCO_VERSION >= 0x01070800
if (!username.empty() && !password.empty())
{
Poco::MongoDB::Database poco_db(auth_db);
if (!poco_db.authenticate(*connection, username, password, Poco::MongoDB::Database::AUTH_SCRAM_SHA1))
throw Exception("Cannot authenticate in MongoDB, incorrect user or password", ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
}
#else
authenticate(*connection, database_name, username, password);
#endif
authenticated = true;
}
}