Fix clang-tidy warnings in Common folder

This commit is contained in:
Maksim Kita 2022-03-11 21:47:28 +00:00
parent 07b0845110
commit e7772ed434
99 changed files with 396 additions and 402 deletions

View File

@ -21,9 +21,9 @@ private:
void dealloc();
public:
AlignedBuffer() {}
AlignedBuffer() = default;
AlignedBuffer(size_t size, size_t alignment);
AlignedBuffer(AlignedBuffer && old) { std::swap(buf, old.buf); }
AlignedBuffer(AlignedBuffer && old) noexcept { std::swap(buf, old.buf); }
~AlignedBuffer();
void reset(size_t size, size_t alignment);

View File

@ -15,12 +15,12 @@
template <typename T>
struct AllocatorWithMemoryTracking
{
typedef T value_type;
using value_type = T;
AllocatorWithMemoryTracking() = default;
template <typename U>
constexpr AllocatorWithMemoryTracking(const AllocatorWithMemoryTracking<U> &) noexcept
constexpr explicit AllocatorWithMemoryTracking(const AllocatorWithMemoryTracking<U> &) noexcept
{
}

View File

@ -69,8 +69,7 @@ private:
Allocator<false>::free(begin, size());
if (prev)
delete prev;
delete prev;
}
size_t size() const { return end + pad_right - begin; }

View File

@ -11,10 +11,10 @@
template <typename Arena>
inline StringRef copyStringInArena(Arena & arena, StringRef value)
{
size_t key_size = value.size;
char * place_for_key = arena.alloc(key_size);
memcpy(reinterpret_cast<void *>(place_for_key), reinterpret_cast<const void *>(value.data), key_size);
StringRef result{place_for_key, key_size};
size_t value_size = value.size;
char * place_for_key = arena.alloc(value_size);
memcpy(reinterpret_cast<void *>(place_for_key), reinterpret_cast<const void *>(value.data), value_size);
StringRef result{place_for_key, value_size};
return result;
}

View File

@ -49,7 +49,7 @@ private:
Block * free_lists[16] {};
public:
ArenaWithFreeLists(
explicit ArenaWithFreeLists(
const size_t initial_size = 4096, const size_t growth_factor = 2,
const size_t linear_growth_threshold = 128 * 1024 * 1024)
: pool{initial_size, growth_factor, linear_growth_threshold}
@ -74,7 +74,7 @@ public:
ASAN_UNPOISON_MEMORY_REGION(free_block_ptr,
std::max(size, sizeof(Block)));
const auto res = free_block_ptr->data;
auto * const res = free_block_ptr->data;
free_block_ptr = free_block_ptr->next;
return res;
}
@ -93,7 +93,7 @@ public:
/// Insert the released block into the head of the list.
auto & free_block_ptr = free_lists[list_idx];
const auto old_head = free_block_ptr;
auto * const old_head = free_block_ptr;
free_block_ptr = reinterpret_cast<Block *>(ptr);
free_block_ptr->next = old_head;

View File

@ -188,7 +188,7 @@ private:
DB::throwFromErrno(fmt::format("Allocator: Cannot munmap {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_MUNMAP);
}
Chunk(Chunk && other) : ptr(other.ptr), size(other.size)
Chunk(Chunk && other) noexcept : ptr(other.ptr), size(other.size)
{
other.ptr = nullptr;
}
@ -261,7 +261,7 @@ private:
/// Represents pending insertion attempt.
struct InsertToken
{
InsertToken(ArrayCache & cache_) : cache(cache_) {}
explicit InsertToken(ArrayCache & cache_) : cache(cache_) {}
std::mutex mutex;
bool cleaned_up = false; /// Protected by the token mutex
@ -535,7 +535,7 @@ private:
public:
ArrayCache(size_t max_total_size_) : max_total_size(max_total_size_)
explicit ArrayCache(size_t max_total_size_) : max_total_size(max_total_size_)
{
}

View File

@ -39,7 +39,7 @@ inline size_t getLeadingZeroBitsUnsafe(T x)
{
return __builtin_clz(x);
}
else if constexpr (sizeof(T) <= sizeof(unsigned long int))
else if constexpr (sizeof(T) <= sizeof(unsigned long int)) /// NOLINT
{
return __builtin_clzl(x);
}
@ -79,7 +79,7 @@ inline size_t getTrailingZeroBitsUnsafe(T x)
{
return __builtin_ctz(x);
}
else if constexpr (sizeof(T) <= sizeof(unsigned long int))
else if constexpr (sizeof(T) <= sizeof(unsigned long int)) /// NOLINT
{
return __builtin_ctzl(x);
}

View File

@ -81,7 +81,7 @@ private:
protected:
template <typename T>
class mutable_ptr : public boost::intrusive_ptr<T>
class mutable_ptr : public boost::intrusive_ptr<T> /// NOLINT
{
private:
using Base = boost::intrusive_ptr<T>;
@ -96,16 +96,16 @@ protected:
mutable_ptr(const mutable_ptr &) = delete;
/// Move: ok.
mutable_ptr(mutable_ptr &&) = default;
mutable_ptr & operator=(mutable_ptr &&) = default;
mutable_ptr(mutable_ptr &&) = default; /// NOLINT
mutable_ptr & operator=(mutable_ptr &&) = default; /// NOLINT
/// Initializing from temporary of compatible type.
template <typename U>
mutable_ptr(mutable_ptr<U> && other) : Base(std::move(other)) {}
mutable_ptr(mutable_ptr<U> && other) : Base(std::move(other)) {} /// NOLINT
mutable_ptr() = default;
mutable_ptr(std::nullptr_t) {}
mutable_ptr(std::nullptr_t) {} /// NOLINT
};
public:
@ -113,7 +113,7 @@ public:
protected:
template <typename T>
class immutable_ptr : public boost::intrusive_ptr<const T>
class immutable_ptr : public boost::intrusive_ptr<const T> /// NOLINT
{
private:
using Base = boost::intrusive_ptr<const T>;
@ -129,19 +129,19 @@ protected:
immutable_ptr & operator=(const immutable_ptr &) = default;
template <typename U>
immutable_ptr(const immutable_ptr<U> & other) : Base(other) {}
immutable_ptr(const immutable_ptr<U> & other) : Base(other) {} /// NOLINT
/// Move: ok.
immutable_ptr(immutable_ptr &&) = default;
immutable_ptr & operator=(immutable_ptr &&) = default;
immutable_ptr(immutable_ptr &&) = default; /// NOLINT
immutable_ptr & operator=(immutable_ptr &&) = default; /// NOLINT
/// Initializing from temporary of compatible type.
template <typename U>
immutable_ptr(immutable_ptr<U> && other) : Base(std::move(other)) {}
immutable_ptr(immutable_ptr<U> && other) : Base(std::move(other)) {} /// NOLINT
/// Move from mutable ptr: ok.
template <typename U>
immutable_ptr(mutable_ptr<U> && other) : Base(std::move(other)) {}
immutable_ptr(mutable_ptr<U> && other) : Base(std::move(other)) {} /// NOLINT
/// Copy from mutable ptr: not possible.
template <typename U>
@ -149,7 +149,7 @@ protected:
immutable_ptr() = default;
immutable_ptr(std::nullptr_t) {}
immutable_ptr(std::nullptr_t) {} /// NOLINT
};
public:
@ -161,7 +161,6 @@ public:
template <typename T>
static MutablePtr create(std::initializer_list<T> && arg) { return create(std::forward<std::initializer_list<T>>(arg)); }
public:
Ptr getPtr() const { return static_cast<Ptr>(derived()); }
MutablePtr getPtr() { return static_cast<MutablePtr>(derived()); }
@ -193,14 +192,14 @@ public:
protected:
/// It works as immutable_ptr if it is const and as mutable_ptr if it is non const.
template <typename T>
class chameleon_ptr
class chameleon_ptr /// NOLINT
{
private:
immutable_ptr<T> value;
public:
template <typename... Args>
chameleon_ptr(Args &&... args) : value(std::forward<Args>(args)...) {}
chameleon_ptr(Args &&... args) : value(std::forward<Args>(args)...) {} /// NOLINT
template <typename U>
chameleon_ptr(std::initializer_list<U> && arg) : value(std::forward<std::initializer_list<U>>(arg)) {}
@ -214,13 +213,13 @@ protected:
const T & operator*() const { return *value; }
T & operator*() { return value->assumeMutableRef(); }
operator const immutable_ptr<T> & () const { return value; }
operator immutable_ptr<T> & () { return value; }
operator const immutable_ptr<T> & () const { return value; } /// NOLINT
operator immutable_ptr<T> & () { return value; } /// NOLINT
/// Get internal immutable ptr. Does not change internal use counter.
immutable_ptr<T> detach() && { return std::move(value); }
operator bool() const { return value != nullptr; }
operator bool() const { return value != nullptr; } /// NOLINT
bool operator! () const { return value == nullptr; }
bool operator== (const chameleon_ptr & rhs) const { return value == rhs.value; }

View File

@ -44,7 +44,7 @@ struct HashMethodOneNumber
vec = key_columns[0]->getRawData().data;
}
HashMethodOneNumber(const IColumn * column)
explicit HashMethodOneNumber(const IColumn * column)
{
vec = column->getRawData().data;
}
@ -233,7 +233,7 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod
static const ColumnLowCardinality & getLowCardinalityColumn(const IColumn * column)
{
auto low_cardinality_column = typeid_cast<const ColumnLowCardinality *>(column);
const auto * low_cardinality_column = typeid_cast<const ColumnLowCardinality *>(column);
if (!low_cardinality_column)
throw Exception("Invalid aggregation key type for HashMethodSingleLowCardinalityColumn method. "
"Excepted LowCardinality, got " + column->getName(), ErrorCodes::LOGICAL_ERROR);
@ -244,7 +244,7 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod
const ColumnRawPtrs & key_columns_low_cardinality, const Sizes & key_sizes, const HashMethodContextPtr & context)
: Base({getLowCardinalityColumn(key_columns_low_cardinality[0]).getDictionary().getNestedNotNullableColumn().get()}, key_sizes, context)
{
auto column = &getLowCardinalityColumn(key_columns_low_cardinality[0]);
const auto * column = &getLowCardinalityColumn(key_columns_low_cardinality[0]);
if (!context)
throw Exception("Cache wasn't created for HashMethodSingleLowCardinalityColumn",
@ -262,7 +262,7 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod
}
}
auto * dict = column->getDictionary().getNestedNotNullableColumn().get();
const auto * dict = column->getDictionary().getNestedNotNullableColumn().get();
is_nullable = column->getDictionary().nestedColumnIsNullable();
key_columns = {dict};
bool is_shared_dict = column->isSharedDictionary();
@ -504,7 +504,7 @@ struct HashMethodKeysFixed
}
HashMethodKeysFixed(const ColumnRawPtrs & key_columns, const Sizes & key_sizes_, const HashMethodContextPtr &)
: Base(key_columns), key_sizes(std::move(key_sizes_)), keys_size(key_columns.size())
: Base(key_columns), key_sizes(key_sizes_), keys_size(key_columns.size())
{
if constexpr (has_low_cardinality)
{
@ -513,7 +513,7 @@ struct HashMethodKeysFixed
low_cardinality_keys.position_sizes.resize(key_columns.size());
for (size_t i = 0; i < key_columns.size(); ++i)
{
if (auto * low_cardinality_col = typeid_cast<const ColumnLowCardinality *>(key_columns[i]))
if (const auto * low_cardinality_col = typeid_cast<const ColumnLowCardinality *>(key_columns[i]))
{
low_cardinality_keys.nested_columns[i] = low_cardinality_col->getDictionary().getNestedColumn().get();
low_cardinality_keys.positions[i] = &low_cardinality_col->getIndexes();

View File

@ -312,14 +312,14 @@ template <typename Key>
class BaseStateKeysFixed<Key, true>
{
protected:
BaseStateKeysFixed(const ColumnRawPtrs & key_columns)
explicit BaseStateKeysFixed(const ColumnRawPtrs & key_columns)
{
null_maps.reserve(key_columns.size());
actual_columns.reserve(key_columns.size());
for (const auto & col : key_columns)
{
if (auto * nullable_col = checkAndGetColumn<ColumnNullable>(col))
if (const auto * nullable_col = checkAndGetColumn<ColumnNullable>(col))
{
actual_columns.push_back(&nullable_col->getNestedColumn());
null_maps.push_back(&nullable_col->getNullMapColumn());
@ -373,7 +373,7 @@ template <typename Key>
class BaseStateKeysFixed<Key, false>
{
protected:
BaseStateKeysFixed(const ColumnRawPtrs & columns) : actual_columns(columns) {}
explicit BaseStateKeysFixed(const ColumnRawPtrs & columns) : actual_columns(columns) {}
const ColumnRawPtrs & getActualColumns() const { return actual_columns; }

View File

@ -314,7 +314,6 @@ private:
address &= mask;
}
private:
Small small;
union
{

View File

@ -28,7 +28,6 @@ public:
class Reader;
class Locus;
public:
CompactArray() = default;
UInt8 ALWAYS_INLINE operator[](BucketIndex bucket_index) const
@ -67,7 +66,7 @@ template <typename BucketIndex, UInt8 content_width, size_t bucket_count>
class CompactArray<BucketIndex, content_width, bucket_count>::Reader final
{
public:
Reader(ReadBuffer & in_)
explicit Reader(ReadBuffer & in_)
: in(in_)
{
}
@ -160,7 +159,7 @@ class CompactArray<BucketIndex, content_width, bucket_count>::Locus final
friend class CompactArray::Reader;
public:
ALWAYS_INLINE operator UInt8() const
ALWAYS_INLINE operator UInt8() const /// NOLINT
{
if (content_l == content_r)
return read(*content_l);
@ -194,7 +193,7 @@ public:
private:
Locus() = default;
Locus(BucketIndex bucket_index)
explicit Locus(BucketIndex bucket_index)
{
init(bucket_index);
}
@ -230,7 +229,6 @@ private:
| ((value_r & ((1 << offset_r) - 1)) << (8 - offset_l));
}
private:
size_t index_l;
size_t offset_l;
size_t index_r;

View File

@ -98,7 +98,6 @@ public:
/// Set path of main config.xml. It will be cut from all configs placed to preprocessed_configs/
static void setConfigPath(const std::string & config_path);
public:
using Files = std::vector<std::string>;
static Files getConfigMergeFiles(const std::string & config_path);
@ -122,7 +121,6 @@ private:
Poco::AutoPtr<Poco::XML::NamePool> name_pool;
Poco::XML::DOMParser dom_parser;
private:
using NodePtr = Poco::AutoPtr<Poco::XML::Node>;
void mergeRecursive(XMLDocumentPtr config, Poco::XML::Node * config_root, const Poco::XML::Node * with_root);

View File

@ -69,8 +69,6 @@ private:
FilesChangesTracker getNewFileList() const;
private:
static constexpr auto reload_interval = std::chrono::seconds(2);
Poco::Logger * log = &Poco::Logger::get("ConfigReloader");

View File

@ -6,6 +6,7 @@
#include <string>
#include <iostream>
#include <mutex>
#include <filesystem>
#include <Poco/Exception.h>
@ -28,6 +29,7 @@ namespace DB
}
}
namespace fs = std::filesystem;
/** Stores a number in the file.
* Designed for rare calls (not designed for performance).
@ -39,7 +41,7 @@ private:
public:
/// path - the name of the file, including the path
CounterInFile(const std::string & path_) : path(path_) {}
explicit CounterInFile(const std::string & path_) : path(path_) {}
/** Add `delta` to the number in the file and return the new value.
* If the `create_if_need` parameter is not set to true, then

View File

@ -28,7 +28,7 @@ inline UInt64 our_xgetbv(UInt32 xcr) noexcept
}
#endif
inline bool cpuid(UInt32 op, UInt32 sub_op, UInt32 * res) noexcept
inline bool cpuid(UInt32 op, UInt32 sub_op, UInt32 * res) noexcept /// NOLINT
{
#if defined(__x86_64__) || defined(__i386__)
__cpuid_count(op, sub_op, res[0], res[1], res[2], res[3]);
@ -43,7 +43,7 @@ inline bool cpuid(UInt32 op, UInt32 sub_op, UInt32 * res) noexcept
#endif
}
inline bool cpuid(UInt32 op, UInt32 * res) noexcept
inline bool cpuid(UInt32 op, UInt32 * res) noexcept /// NOLINT
{
#if defined(__x86_64__) || defined(__i386__)
__cpuid(op, res[0], res[1], res[2], res[3]);
@ -106,7 +106,7 @@ union CpuInfo
UInt32 edx;
} registers;
inline CpuInfo(UInt32 op) noexcept { cpuid(op, info); }
inline explicit CpuInfo(UInt32 op) noexcept { cpuid(op, info); }
inline CpuInfo(UInt32 op, UInt32 sub_op) noexcept { cpuid(op, sub_op, info); }
};

View File

@ -72,7 +72,7 @@ namespace CurrentMetrics
}
public:
Increment(Metric metric, Value amount_ = 1)
explicit Increment(Metric metric, Value amount_ = 1)
: Increment(&values[metric], amount_) {}
~Increment()
@ -81,12 +81,12 @@ namespace CurrentMetrics
what->fetch_sub(amount, std::memory_order_relaxed);
}
Increment(Increment && old)
Increment(Increment && old) noexcept
{
*this = std::move(old);
}
Increment & operator= (Increment && old)
Increment & operator=(Increment && old) noexcept
{
what = old.what;
amount = old.amount;

View File

@ -17,7 +17,7 @@ namespace DB
class DNSResolver : private boost::noncopyable
{
public:
typedef std::vector<Poco::Net::IPAddress> IPAddresses;
using IPAddresses = std::vector<Poco::Net::IPAddress>;
static DNSResolver & instance();

View File

@ -52,10 +52,10 @@ private:
friend class DateLUT;
explicit DateLUTImpl(const std::string & time_zone);
DateLUTImpl(const DateLUTImpl &) = delete;
DateLUTImpl & operator=(const DateLUTImpl &) = delete;
DateLUTImpl(const DateLUTImpl &&) = delete;
DateLUTImpl & operator=(const DateLUTImpl &&) = delete;
DateLUTImpl(const DateLUTImpl &) = delete; /// NOLINT
DateLUTImpl & operator=(const DateLUTImpl &) = delete; /// NOLINT
DateLUTImpl(const DateLUTImpl &&) = delete; /// NOLINT
DateLUTImpl & operator=(const DateLUTImpl &&) = delete; /// NOLINT
// Normalized and bound-checked index of element in lut,
// has to be a separate type to support overloading
@ -149,12 +149,12 @@ public:
Int8 amount_of_offset_change_value; /// Usually -4 or 4, but look at Lord Howe Island. Multiply by OffsetChangeFactor
UInt8 time_at_offset_change_value; /// In seconds from beginning of the day. Multiply by OffsetChangeFactor
inline Int32 amount_of_offset_change() const
inline Int32 amount_of_offset_change() const /// NOLINT
{
return static_cast<Int32>(amount_of_offset_change_value) * OffsetChangeFactor;
}
inline UInt32 time_at_offset_change() const
inline UInt32 time_at_offset_change() const /// NOLINT
{
return static_cast<UInt32>(time_at_offset_change_value) * OffsetChangeFactor;
}
@ -230,12 +230,12 @@ private:
return LUTIndex(guess ? guess - 1 : 0);
}
inline LUTIndex toLUTIndex(DayNum d) const
static inline LUTIndex toLUTIndex(DayNum d)
{
return LUTIndex{(d + daynum_offset_epoch) & date_lut_mask};
}
inline LUTIndex toLUTIndex(ExtendedDayNum d) const
static inline LUTIndex toLUTIndex(ExtendedDayNum d)
{
return LUTIndex{static_cast<UInt32>(d + daynum_offset_epoch) & date_lut_mask};
}
@ -245,7 +245,7 @@ private:
return findIndex(t);
}
inline LUTIndex toLUTIndex(LUTIndex i) const
static inline LUTIndex toLUTIndex(LUTIndex i)
{
return i;
}
@ -793,7 +793,7 @@ public:
}
/// Check and change mode to effective.
inline UInt8 check_week_mode(UInt8 mode) const
inline UInt8 check_week_mode(UInt8 mode) const /// NOLINT
{
UInt8 week_format = (mode & 7);
if (!(week_format & static_cast<UInt8>(WeekModeFlag::MONDAY_FIRST)))
@ -805,7 +805,7 @@ public:
* Returns 0 for monday, 1 for tuesday...
*/
template <typename DateOrTime>
inline unsigned calc_weekday(DateOrTime v, bool sunday_first_day_of_week) const
inline unsigned calc_weekday(DateOrTime v, bool sunday_first_day_of_week) const /// NOLINT
{
const LUTIndex i = toLUTIndex(v);
if (!sunday_first_day_of_week)
@ -815,7 +815,7 @@ public:
}
/// Calculate days in one year.
inline unsigned calc_days_in_year(Int32 year) const
inline unsigned calc_days_in_year(Int32 year) const /// NOLINT
{
return ((year & 3) == 0 && (year % 100 || (year % 400 == 0 && year)) ? 366 : 365);
}
@ -852,7 +852,7 @@ public:
return toRelativeHourNum(lut[toLUTIndex(v)].date);
}
inline Time toRelativeMinuteNum(Time t) const
inline Time toRelativeMinuteNum(Time t) const /// NOLINT
{
return (t + DATE_LUT_ADD) / 60 - (DATE_LUT_ADD / 60);
}
@ -1057,12 +1057,12 @@ public:
return values.year * 10000 + values.month * 100 + values.day_of_month;
}
inline Time YYYYMMDDToDate(UInt32 num) const
inline Time YYYYMMDDToDate(UInt32 num) const /// NOLINT
{
return makeDate(num / 10000, num / 100 % 100, num % 100);
}
inline ExtendedDayNum YYYYMMDDToDayNum(UInt32 num) const
inline ExtendedDayNum YYYYMMDDToDayNum(UInt32 num) const /// NOLINT
{
return makeDayNum(num / 10000, num / 100 % 100, num % 100);
}
@ -1143,7 +1143,7 @@ public:
+ UInt64(components.date.year) * 10000000000;
}
inline Time YYYYMMDDhhmmssToTime(UInt64 num) const
inline Time YYYYMMDDhhmmssToTime(UInt64 num) const /// NOLINT
{
return makeDateTime(
num / 10000000000,

View File

@ -103,7 +103,7 @@ Dwarf::Dwarf(const std::shared_ptr<Elf> & elf) : elf_(elf)
init();
}
Dwarf::Section::Section(std::string_view d) : is64Bit_(false), data_(d)
Dwarf::Section::Section(std::string_view d) : is64_bit(false), data(d)
{
}
@ -343,7 +343,7 @@ void Dwarf::Path::toString(std::string & dest) const
// Next chunk in section
bool Dwarf::Section::next(std::string_view & chunk)
{
chunk = data_;
chunk = data;
if (chunk.empty())
return false;
@ -351,11 +351,11 @@ bool Dwarf::Section::next(std::string_view & chunk)
// a 96-bit value (0xffffffff followed by the 64-bit length) for a 64-bit
// section.
auto initial_length = read<uint32_t>(chunk);
is64Bit_ = (initial_length == uint32_t(-1));
auto length = is64Bit_ ? read<uint64_t>(chunk) : initial_length;
is64_bit = (initial_length == uint32_t(-1));
auto length = is64_bit ? read<uint64_t>(chunk) : initial_length;
SAFE_CHECK(length <= chunk.size(), "invalid DWARF section");
chunk = std::string_view(chunk.data(), length);
data_ = std::string_view(chunk.end(), data_.end() - chunk.end());
data = std::string_view(chunk.end(), data.end() - chunk.end());
return true;
}

View File

@ -112,9 +112,9 @@ public:
// TODO(tudorb): Implement operator==, operator!=; not as easy as it
// seems as the same path can be represented in multiple ways
private:
std::string_view baseDir_;
std::string_view subDir_;
std::string_view file_;
std::string_view baseDir_; /// NOLINT
std::string_view subDir_; /// NOLINT
std::string_view file_; /// NOLINT
};
// Indicates inline function `name` is called at `line@file`.
@ -173,7 +173,7 @@ private:
void init();
std::shared_ptr<const Elf> elf_;
std::shared_ptr<const Elf> elf_; /// NOLINT
// DWARF section made up of chunks, each prefixed with a length header.
// The length indicates whether the chunk is DWARF-32 or DWARF-64, which
@ -182,7 +182,7 @@ private:
class Section
{
public:
Section() : is64Bit_(false) {}
Section() : is64_bit(false) {}
explicit Section(std::string_view d);
@ -191,12 +191,12 @@ private:
bool next(std::string_view & chunk);
// Is the current chunk 64 bit?
bool is64Bit() const { return is64Bit_; }
bool is64Bit() const { return is64_bit; }
private:
// Yes, 32- and 64- bit sections may coexist. Yikes!
bool is64Bit_;
std::string_view data_;
bool is64_bit;
std::string_view data;
};
// Abbreviation for a Debugging Information Entry.
@ -215,7 +215,7 @@ private:
// provide a description of a corresponding entity in the source program.
struct Die
{
bool is64Bit;
bool is64Bit; /// NOLINT
// Offset from start to first attribute
uint8_t attr_offset;
// Offset within debug info.
@ -241,7 +241,7 @@ private:
struct CompilationUnit
{
bool is64Bit;
bool is64Bit; /// NOLINT
uint8_t version;
uint8_t addr_size;
// Offset in .debug_info of this compilation unit.
@ -311,10 +311,10 @@ private:
struct FileName
{
std::string_view relativeName;
std::string_view relativeName; /// NOLINT
// 0 = current compilation directory
// otherwise, 1-based index in the list of include directories
uint64_t directoryIndex;
uint64_t directoryIndex; /// NOLINT
};
// Read one FileName object, remove_prefix program
static bool readFileName(std::string_view & program, FileName & fn);
@ -332,37 +332,37 @@ private:
bool nextDefineFile(std::string_view & program, FileName & fn) const;
// Initialization
bool is64Bit_;
std::string_view data_;
std::string_view compilationDirectory_;
bool is64Bit_; /// NOLINT
std::string_view data_; /// NOLINT
std::string_view compilationDirectory_; /// NOLINT
// Header
uint16_t version_;
uint8_t minLength_;
bool defaultIsStmt_;
int8_t lineBase_;
uint8_t lineRange_;
uint8_t opcodeBase_;
const uint8_t * standardOpcodeLengths_;
uint16_t version_; /// NOLINT
uint8_t minLength_; /// NOLINT
bool defaultIsStmt_; /// NOLINT
int8_t lineBase_; /// NOLINT
uint8_t lineRange_; /// NOLINT
uint8_t opcodeBase_; /// NOLINT
const uint8_t * standardOpcodeLengths_; /// NOLINT
std::string_view includeDirectories_;
size_t includeDirectoryCount_;
std::string_view includeDirectories_; /// NOLINT
size_t includeDirectoryCount_; /// NOLINT
std::string_view fileNames_;
size_t fileNameCount_;
std::string_view fileNames_; /// NOLINT
size_t fileNameCount_; /// NOLINT
// State machine registers
uint64_t address_;
uint64_t file_;
uint64_t line_;
uint64_t column_;
bool isStmt_;
bool basicBlock_;
bool endSequence_;
bool prologueEnd_;
bool epilogueBegin_;
uint64_t isa_;
uint64_t discriminator_;
uint64_t address_; /// NOLINT
uint64_t file_; /// NOLINT
uint64_t line_; /// NOLINT
uint64_t column_; /// NOLINT
bool isStmt_; /// NOLINT
bool basicBlock_; /// NOLINT
bool endSequence_; /// NOLINT
bool prologueEnd_; /// NOLINT
bool epilogueBegin_; /// NOLINT
uint64_t isa_; /// NOLINT
uint64_t discriminator_; /// NOLINT
};
/**
@ -439,12 +439,12 @@ private:
// Finds the Compilation Unit starting at offset.
static CompilationUnit findCompilationUnit(std::string_view info, uint64_t targetOffset);
std::string_view info_; // .debug_info
std::string_view abbrev_; // .debug_abbrev
std::string_view aranges_; // .debug_aranges
std::string_view line_; // .debug_line
std::string_view strings_; // .debug_str
std::string_view ranges_; // .debug_ranges
std::string_view info_; // .debug_info /// NOLINT
std::string_view abbrev_; // .debug_abbrev /// NOLINT
std::string_view aranges_; // .debug_aranges /// NOLINT
std::string_view line_; // .debug_line /// NOLINT
std::string_view strings_; // .debug_str /// NOLINT
std::string_view ranges_; // .debug_ranges /// NOLINT
};
}

View File

@ -555,19 +555,19 @@ std::string ParsingException::displayText() const
{
try
{
if (line_number_ == -1)
formatted_message_ = message();
if (line_number == -1)
formatted_message = message();
else
formatted_message_ = message() + fmt::format(": (at row {})\n", line_number_);
formatted_message = message() + fmt::format(": (at row {})\n", line_number);
}
catch (...)
{}
if (!formatted_message_.empty())
if (!formatted_message.empty())
{
std::string result = name();
result.append(": ");
result.append(formatted_message_);
result.append(formatted_message);
return result;
}
else

View File

@ -96,7 +96,7 @@ public:
void rethrow() const override { throw *this; }
int getErrno() const { return saved_errno; }
const std::optional<std::string> getPath() const { return path; }
std::optional<std::string> getPath() const { return path; }
private:
int saved_errno;
@ -129,12 +129,12 @@ public:
#endif
;
int getLineNumber() { return line_number_; }
void setLineNumber(int line_number) { line_number_ = line_number;}
int getLineNumber() const { return line_number; }
void setLineNumber(int line_number_) { line_number = line_number_;}
private:
ssize_t line_number_{-1};
mutable std::string formatted_message_;
ssize_t line_number{-1};
mutable std::string formatted_message;
const char * name() const throw() override { return "DB::ParsingException"; }
const char * className() const throw() override { return "DB::ParsingException"; }
@ -209,7 +209,7 @@ std::enable_if_t<std::is_pointer_v<T>, T> exception_cast(std::exception_ptr e)
{
try
{
std::rethrow_exception(std::move(e));
std::rethrow_exception(e);
}
catch (std::remove_pointer_t<T> & concrete)
{

View File

@ -43,9 +43,7 @@ struct ExponentiallySmoothedAverage
double time = 0;
ExponentiallySmoothedAverage()
{
}
ExponentiallySmoothedAverage() = default;
ExponentiallySmoothedAverage(double current_value, double current_time)
: value(current_value), time(current_time)

View File

@ -39,7 +39,7 @@ public:
page_size = getPageSize();
}
boost::context::stack_context allocate()
boost::context::stack_context allocate() const
{
size_t num_pages = 1 + (stack_size - 1) / page_size;
size_t num_bytes = (num_pages + 1) * page_size; /// Add one page at bottom that will be used as guard-page
@ -68,7 +68,7 @@ public:
return sctx;
}
void deallocate(boost::context::stack_context & sctx)
void deallocate(boost::context::stack_context & sctx) const
{
#if defined(BOOST_USE_VALGRIND)
VALGRIND_STACK_DEREGISTER(sctx.valgrind_stack_id);

View File

@ -13,7 +13,7 @@ class FieldVisitorHash : public StaticVisitor<>
private:
SipHash & hash;
public:
FieldVisitorHash(SipHash & hash_);
explicit FieldVisitorHash(SipHash & hash_);
void operator() (const Null & x) const;
void operator() (const UInt64 & x) const;

View File

@ -5,7 +5,6 @@
#include <filesystem>
#include <Common/filesystemHelpers.h>
namespace fs = std::filesystem;
class FileUpdatesTracker
{
@ -14,7 +13,7 @@ private:
Poco::Timestamp known_time;
public:
FileUpdatesTracker(const std::string & path_)
explicit FileUpdatesTracker(const std::string & path_)
: path(path_)
, known_time(0)
{}

View File

@ -44,7 +44,7 @@ struct ClearableHashTableCell : public BaseCell
/// Do I need to store the zero key separately (that is, can a zero key be inserted into the hash table).
static constexpr bool need_zero_value_storage = false;
ClearableHashTableCell() {} //-V730
ClearableHashTableCell() {} //-V730 /// NOLINT
ClearableHashTableCell(const Key & key_, const State & state) : BaseCell(key_, state), version(state.version) {}
};

View File

@ -16,11 +16,11 @@ struct FixedClearableHashMapCell
UInt32 version;
Mapped mapped;
FixedClearableHashMapCell() {}
FixedClearableHashMapCell() {} /// NOLINT
FixedClearableHashMapCell(const Key &, const State & state) : version(state.version) {}
FixedClearableHashMapCell(const value_type & value_, const State & state) : version(state.version), mapped(value_.second) {}
const VoidKey getKey() const { return {}; }
const VoidKey getKey() const { return {}; } /// NOLINT
Mapped & getMapped() { return mapped; }
const Mapped & getMapped() const { return mapped; }
@ -29,7 +29,7 @@ struct FixedClearableHashMapCell
struct CellExt
{
CellExt() {}
CellExt() {} /// NOLINT
CellExt(Key && key_, FixedClearableHashMapCell * ptr_) : key(key_), ptr(ptr_) {}
void update(Key && key_, FixedClearableHashMapCell * ptr_)
{
@ -41,7 +41,7 @@ struct FixedClearableHashMapCell
const Key & getKey() const { return key; }
Mapped & getMapped() { return ptr->mapped; }
const Mapped & getMapped() const { return *ptr->mapped; }
const value_type getValue() const { return {key, *ptr->mapped}; }
const value_type getValue() const { return {key, *ptr->mapped}; } /// NOLINT
};
};

View File

@ -13,10 +13,10 @@ struct FixedClearableHashTableCell
using mapped_type = VoidMapped;
UInt32 version;
FixedClearableHashTableCell() {} //-V730
FixedClearableHashTableCell() {} //-V730 /// NOLINT
FixedClearableHashTableCell(const Key &, const State & state) : version(state.version) {}
const VoidKey getKey() const { return {}; }
const VoidKey getKey() const { return {}; } /// NOLINT
VoidMapped getMapped() const { return {}; }
bool isZero(const State & state) const { return version != state.version; }
@ -25,7 +25,7 @@ struct FixedClearableHashTableCell
struct CellExt
{
Key key;
const VoidKey getKey() const { return {}; }
const VoidKey getKey() const { return {}; } /// NOLINT
VoidMapped getMapped() const { return {}; }
const value_type & getValue() const { return key; }
void update(Key && key_, FixedClearableHashTableCell *) { key = key_; }

View File

@ -16,11 +16,11 @@ struct FixedHashMapCell
bool full;
Mapped mapped;
FixedHashMapCell() {} //-V730
FixedHashMapCell() {} //-V730 /// NOLINT
FixedHashMapCell(const Key &, const State &) : full(true) {}
FixedHashMapCell(const value_type & value_, const State &) : full(true), mapped(value_.second) {}
const VoidKey getKey() const { return {}; }
const VoidKey getKey() const { return {}; } /// NOLINT
Mapped & getMapped() { return mapped; }
const Mapped & getMapped() const { return mapped; }
@ -31,7 +31,7 @@ struct FixedHashMapCell
/// Note that we have to assemble a continuous layout for the value_type on each call of getValue().
struct CellExt
{
CellExt() {} //-V730
CellExt() {} //-V730 /// NOLINT
CellExt(Key && key_, const FixedHashMapCell * ptr_) : key(key_), ptr(const_cast<FixedHashMapCell *>(ptr_)) {}
void update(Key && key_, const FixedHashMapCell * ptr_)
{
@ -44,7 +44,7 @@ struct FixedHashMapCell
const Key & getKey() const { return key; }
Mapped & getMapped() { return ptr->mapped; }
const Mapped & getMapped() const { return ptr->mapped; }
const value_type getValue() const { return {key, ptr->mapped}; }
const value_type getValue() const { return {key, ptr->mapped}; } /// NOLINT
};
};
@ -61,11 +61,11 @@ struct FixedHashMapImplicitZeroCell
Mapped mapped;
FixedHashMapImplicitZeroCell() {}
FixedHashMapImplicitZeroCell() {} /// NOLINT
FixedHashMapImplicitZeroCell(const Key &, const State &) {}
FixedHashMapImplicitZeroCell(const value_type & value_, const State &) : mapped(value_.second) {}
const VoidKey getKey() const { return {}; }
const VoidKey getKey() const { return {}; } /// NOLINT
Mapped & getMapped() { return mapped; }
const Mapped & getMapped() const { return mapped; }
@ -76,7 +76,7 @@ struct FixedHashMapImplicitZeroCell
/// Note that we have to assemble a continuous layout for the value_type on each call of getValue().
struct CellExt
{
CellExt() {} //-V730
CellExt() {} //-V730 /// NOLINT
CellExt(Key && key_, const FixedHashMapImplicitZeroCell * ptr_) : key(key_), ptr(const_cast<FixedHashMapImplicitZeroCell *>(ptr_)) {}
void update(Key && key_, const FixedHashMapImplicitZeroCell * ptr_)
{
@ -89,7 +89,7 @@ struct FixedHashMapImplicitZeroCell
const Key & getKey() const { return key; }
Mapped & getMapped() { return ptr->mapped; }
const Mapped & getMapped() const { return ptr->mapped; }
const value_type getValue() const { return {key, ptr->mapped}; }
const value_type getValue() const { return {key, ptr->mapped}; } /// NOLINT
};
};

View File

@ -19,10 +19,10 @@ struct FixedHashTableCell
using mapped_type = VoidMapped;
bool full;
FixedHashTableCell() {} //-V730
FixedHashTableCell() {} //-V730 /// NOLINT
FixedHashTableCell(const Key &, const State &) : full(true) {}
const VoidKey getKey() const { return {}; }
const VoidKey getKey() const { return {}; } /// NOLINT
VoidMapped getMapped() const { return {}; }
bool isZero(const State &) const { return !full; }
@ -39,7 +39,7 @@ struct FixedHashTableCell
{
Key key;
const VoidKey getKey() const { return {}; }
const VoidKey getKey() const { return {}; } /// NOLINT
VoidMapped getMapped() const { return {}; }
const value_type & getValue() const { return key; }
void update(Key && key_, FixedHashTableCell *) { key = key_; }
@ -138,7 +138,7 @@ protected:
template <typename Derived, bool is_const>
class iterator_base
class iterator_base /// NOLINT
{
using Container = std::conditional_t<is_const, const Self, Self>;
using cell_type = std::conditional_t<is_const, const Cell, Cell>;
@ -149,7 +149,7 @@ protected:
friend class FixedHashTable;
public:
iterator_base() {}
iterator_base() {} /// NOLINT
iterator_base(Container * container_, cell_type * ptr_) : container(container_), ptr(ptr_)
{
cell.update(ptr - container->buf, ptr);
@ -163,7 +163,7 @@ protected:
++ptr;
/// Skip empty cells in the main buffer.
auto buf_end = container->buf + container->NUM_CELLS;
const auto * buf_end = container->buf + container->NUM_CELLS;
while (ptr < buf_end && ptr->isZero(*container))
++ptr;
@ -204,7 +204,7 @@ public:
FixedHashTable() { alloc(); }
FixedHashTable(FixedHashTable && rhs) : buf(nullptr) { *this = std::move(rhs); }
FixedHashTable(FixedHashTable && rhs) noexcept : buf(nullptr) { *this = std::move(rhs); } /// NOLINT
~FixedHashTable()
{
@ -212,7 +212,7 @@ public:
free();
}
FixedHashTable & operator=(FixedHashTable && rhs)
FixedHashTable & operator=(FixedHashTable && rhs) noexcept
{
destroyElements();
free();
@ -229,7 +229,7 @@ public:
class Reader final : private Cell::State
{
public:
Reader(DB::ReadBuffer & in_) : in(in_) {}
explicit Reader(DB::ReadBuffer & in_) : in(in_) {}
Reader(const Reader &) = delete;
Reader & operator=(const Reader &) = delete;
@ -273,13 +273,13 @@ public:
};
class iterator : public iterator_base<iterator, false>
class iterator : public iterator_base<iterator, false> /// NOLINT
{
public:
using iterator_base<iterator, false>::iterator_base;
};
class const_iterator : public iterator_base<const_iterator, true>
class const_iterator : public iterator_base<const_iterator, true> /// NOLINT
{
public:
using iterator_base<const_iterator, true>::iterator_base;
@ -331,7 +331,6 @@ public:
}
public:
/// The last parameter is unused but exists for compatibility with HashTable interface.
void ALWAYS_INLINE emplace(const Key & x, LookupResult & it, bool & inserted, size_t /* hash */ = 0)
{

View File

@ -157,7 +157,7 @@ struct HashTableCell
Key key;
HashTableCell() {}
HashTableCell() {} /// NOLINT
/// Create a cell with the given key / key and value.
HashTableCell(const Key & key_, const State &) : key(key_) {}
@ -602,7 +602,7 @@ protected:
template <typename Derived, bool is_const>
class iterator_base
class iterator_base /// NOLINT
{
using Container = std::conditional_t<is_const, const Self, Self>;
using cell_type = std::conditional_t<is_const, const Cell, Cell>;
@ -613,7 +613,7 @@ protected:
friend class HashTable;
public:
iterator_base() {}
iterator_base() {} /// NOLINT
iterator_base(Container * container_, cell_type * ptr_) : container(container_), ptr(ptr_) {}
bool operator== (const iterator_base & rhs) const { return ptr == rhs.ptr; }
@ -628,7 +628,7 @@ protected:
++ptr;
/// Skip empty cells in the main buffer.
auto buf_end = container->buf + container->grower.bufSize();
auto * buf_end = container->buf + container->grower.bufSize();
while (ptr < buf_end && ptr->isZero(*container))
++ptr;
@ -661,7 +661,7 @@ protected:
* compatibility with std find(). Unfortunately, now is not the time to
* do this.
*/
operator Cell * () const { return nullptr; }
operator Cell * () const { return nullptr; } /// NOLINT
};
@ -684,7 +684,7 @@ public:
alloc(grower);
}
HashTable(size_t reserve_for_num_elements)
HashTable(size_t reserve_for_num_elements) /// NOLINT
{
if (Cell::need_zero_value_storage)
this->zeroValue()->setZero();
@ -692,7 +692,7 @@ public:
alloc(grower);
}
HashTable(HashTable && rhs)
HashTable(HashTable && rhs) noexcept
: buf(nullptr)
{
*this = std::move(rhs);
@ -704,7 +704,7 @@ public:
free();
}
HashTable & operator= (HashTable && rhs)
HashTable & operator=(HashTable && rhs) noexcept
{
destroyElements();
free();
@ -713,10 +713,10 @@ public:
std::swap(m_size, rhs.m_size);
std::swap(grower, rhs.grower);
Hash::operator=(std::move(rhs));
Allocator::operator=(std::move(rhs));
Cell::State::operator=(std::move(rhs));
ZeroValueStorage<Cell::need_zero_value_storage, Cell>::operator=(std::move(rhs));
Hash::operator=(std::move(rhs)); ///NOLINT
Allocator::operator=(std::move(rhs)); ///NOLINT
Cell::State::operator=(std::move(rhs)); ///NOLINT
ZeroValueStorage<Cell::need_zero_value_storage, Cell>::operator=(std::move(rhs)); ///NOLINT
return *this;
}
@ -724,7 +724,7 @@ public:
class Reader final : private Cell::State
{
public:
Reader(DB::ReadBuffer & in_)
explicit Reader(DB::ReadBuffer & in_)
: in(in_)
{
}
@ -771,13 +771,13 @@ public:
};
class iterator : public iterator_base<iterator, false>
class iterator : public iterator_base<iterator, false> /// NOLINT
{
public:
using iterator_base<iterator, false>::iterator_base;
};
class const_iterator : public iterator_base<const_iterator, true>
class const_iterator : public iterator_base<const_iterator, true> /// NOLINT
{
public:
using iterator_base<const_iterator, true>::iterator_base;
@ -811,7 +811,7 @@ public:
return iteratorToZero();
Cell * ptr = buf;
auto buf_end = buf + grower.bufSize();
auto * buf_end = buf + grower.bufSize();
while (ptr < buf_end && ptr->isZero(*this))
++ptr;

View File

@ -1,5 +1,7 @@
#pragma once
#include <base/StringRef.h>
#include <Common/Arena.h>
/**

View File

@ -71,10 +71,10 @@ struct LRUHashMapCellNodeTraits
using node_ptr = LRUHashMapCell<Key, Value, Hash, save_hash_in_cell> *;
using const_node_ptr = const LRUHashMapCell<Key, Value, Hash, save_hash_in_cell> *;
static node * get_next(const node * ptr) { return ptr->next; }
static void set_next(node * __restrict ptr, node * __restrict next) { ptr->next = next; }
static node * get_previous(const node * ptr) { return ptr->prev; }
static void set_previous(node * __restrict ptr, node * __restrict prev) { ptr->prev = prev; }
static node * get_next(const node * ptr) { return ptr->next; } /// NOLINT
static void set_next(node * __restrict ptr, node * __restrict next) { ptr->next = next; } /// NOLINT
static node * get_previous(const node * ptr) { return ptr->prev; } /// NOLINT
static void set_previous(node * __restrict ptr, node * __restrict prev) { ptr->prev = prev; } /// NOLINT
};
template <typename TKey, typename TValue, typename Disposer, typename Hash, bool save_hash_in_cells>

View File

@ -79,7 +79,7 @@ public:
class Reader final : private Cell::State
{
public:
Reader(DB::ReadBuffer & in_)
explicit Reader(DB::ReadBuffer & in_)
: in(in_)
{
}
@ -129,7 +129,7 @@ public:
bool is_initialized = false;
};
class iterator
class iterator /// NOLINT
{
Self * container = nullptr;
Cell * ptr = nullptr;
@ -137,7 +137,7 @@ public:
friend class SmallTable;
public:
iterator() {}
iterator() {} /// NOLINT
iterator(Self * container_, Cell * ptr_) : container(container_), ptr(ptr_) {}
bool operator== (const iterator & rhs) const { return ptr == rhs.ptr; }
@ -156,7 +156,7 @@ public:
};
class const_iterator
class const_iterator /// NOLINT
{
const Self * container = nullptr;
const Cell * ptr = nullptr;
@ -164,9 +164,9 @@ public:
friend class SmallTable;
public:
const_iterator() {}
const_iterator(const Self * container_, const Cell * ptr_) : container(container_), ptr(ptr_) {}
const_iterator(const iterator & rhs) : container(rhs.container), ptr(rhs.ptr) {}
const_iterator() = default;
const_iterator(const Self * container_, const Cell * ptr_) : container(container_), ptr(ptr_) {} /// NOLINT
const_iterator(const iterator & rhs) : container(rhs.container), ptr(rhs.ptr) {} /// NOLINT
bool operator== (const const_iterator & rhs) const { return ptr == rhs.ptr; }
bool operator!= (const const_iterator & rhs) const { return ptr != rhs.ptr; }

View File

@ -12,7 +12,7 @@ struct StringHashMapCell : public HashMapCell<Key, TMapped, StringHashTableHash,
using Base::Base;
static constexpr bool need_zero_value_storage = false;
// external
const StringRef getKey() const { return toStringRef(this->value.first); }
const StringRef getKey() const { return toStringRef(this->value.first); } /// NOLINT
// internal
static const Key & getKey(const value_type & value_) { return value_.first; }
};
@ -32,7 +32,7 @@ struct StringHashMapCell<StringKey16, TMapped> : public HashMapCell<StringKey16,
void setZero() { this->value.first.items[1] = 0; }
// external
const StringRef getKey() const { return toStringRef(this->value.first); }
const StringRef getKey() const { return toStringRef(this->value.first); } /// NOLINT
// internal
static const StringKey16 & getKey(const value_type & value_) { return value_.first; }
};
@ -53,7 +53,7 @@ struct StringHashMapCell<StringKey24, TMapped> : public HashMapCell<StringKey24,
void setZero() { this->value.first.c = 0; }
// external
const StringRef getKey() const { return toStringRef(this->value.first); }
const StringRef getKey() const { return toStringRef(this->value.first); } /// NOLINT
// internal
static const StringKey24 & getKey(const value_type & value_) { return value_.first; }
};

View File

@ -160,16 +160,16 @@ template <typename Mapped>
struct StringHashTableLookupResult
{
Mapped * mapped_ptr;
StringHashTableLookupResult() {}
StringHashTableLookupResult(Mapped * mapped_ptr_) : mapped_ptr(mapped_ptr_) {}
StringHashTableLookupResult(std::nullptr_t) {}
const VoidKey getKey() const { return {}; }
StringHashTableLookupResult() {} /// NOLINT
StringHashTableLookupResult(Mapped * mapped_ptr_) : mapped_ptr(mapped_ptr_) {} /// NOLINT
StringHashTableLookupResult(std::nullptr_t) {} /// NOLINT
const VoidKey getKey() const { return {}; } /// NOLINT
auto & getMapped() { return *mapped_ptr; }
auto & operator*() { return *this; }
auto & operator*() const { return *this; }
auto * operator->() { return this; }
auto * operator->() const { return this; }
operator bool() const { return mapped_ptr; }
operator bool() const { return mapped_ptr; } /// NOLINT
friend bool operator==(const StringHashTableLookupResult & a, const std::nullptr_t &) { return !a.mapped_ptr; }
friend bool operator==(const std::nullptr_t &, const StringHashTableLookupResult & b) { return !b.mapped_ptr; }
friend bool operator!=(const StringHashTableLookupResult & a, const std::nullptr_t &) { return a.mapped_ptr; }
@ -214,7 +214,7 @@ public:
StringHashTable() = default;
StringHashTable(size_t reserve_for_num_elements)
explicit StringHashTable(size_t reserve_for_num_elements)
: m1{reserve_for_num_elements / 4}
, m2{reserve_for_num_elements / 4}
, m3{reserve_for_num_elements / 4}
@ -222,7 +222,7 @@ public:
{
}
StringHashTable(StringHashTable && rhs)
StringHashTable(StringHashTable && rhs) noexcept
: m1(std::move(rhs.m1))
, m2(std::move(rhs.m2))
, m3(std::move(rhs.m3))
@ -232,7 +232,6 @@ public:
~StringHashTable() = default;
public:
// Dispatch is written in a way that maximizes the performance:
// 1. Always memcpy 8 times bytes
// 2. Use switch case extension to generate fast dispatching table

View File

@ -92,11 +92,11 @@ public:
Impl impls[NUM_BUCKETS];
TwoLevelHashTable() {}
TwoLevelHashTable() = default;
/// Copy the data from another (normal) hash table. It should have the same hash function.
template <typename Source>
TwoLevelHashTable(const Source & src)
explicit TwoLevelHashTable(const Source & src)
{
typename Source::const_iterator it = src.begin();
@ -117,7 +117,7 @@ public:
}
class iterator
class iterator /// NOLINT
{
Self * container{};
size_t bucket{};
@ -129,7 +129,7 @@ public:
: container(container_), bucket(bucket_), current_it(current_it_) {}
public:
iterator() {}
iterator() = default;
bool operator== (const iterator & rhs) const { return bucket == rhs.bucket && current_it == rhs.current_it; }
bool operator!= (const iterator & rhs) const { return !(*this == rhs); }
@ -154,7 +154,7 @@ public:
};
class const_iterator
class const_iterator /// NOLINT
{
Self * container{};
size_t bucket{};
@ -166,8 +166,8 @@ public:
: container(container_), bucket(bucket_), current_it(current_it_) {}
public:
const_iterator() {}
const_iterator(const iterator & rhs) : container(rhs.container), bucket(rhs.bucket), current_it(rhs.current_it) {}
const_iterator() = default;
const_iterator(const iterator & rhs) : container(rhs.container), bucket(rhs.bucket), current_it(rhs.current_it) {} /// NOLINT
bool operator== (const const_iterator & rhs) const { return bucket == rhs.bucket && current_it == rhs.current_it; }
bool operator!= (const const_iterator & rhs) const { return !(*this == rhs); }

View File

@ -27,7 +27,6 @@ public:
/// NOTE Bad for hash tables with more than 2^32 cells.
static size_t getBucketFromHash(size_t hash_value) { return (hash_value >> (32 - BITS_FOR_BUCKET)) & MAX_BUCKET; }
public:
using key_type = typename Impl::key_type;
using mapped_type = typename Impl::mapped_type;
using value_type = typename Impl::value_type;
@ -38,10 +37,10 @@ public:
Impl impls[NUM_BUCKETS];
TwoLevelStringHashTable() {}
TwoLevelStringHashTable() = default;
template <typename Source>
TwoLevelStringHashTable(const Source & src)
explicit TwoLevelStringHashTable(const Source & src)
{
if (src.m0.hasZero())
impls[0].m0.setHasZero(*src.m0.zeroValue());

View File

@ -123,12 +123,11 @@ private:
using T = typename IntermediateDenominator<HashValueType, DenominatorType, denominator_mode>::Type;
public:
Denominator(DenominatorType initial_value)
Denominator(DenominatorType initial_value) /// NOLINT
: denominator(initial_value)
{
}
public:
inline void update(UInt8 cur_rank, UInt8 new_rank)
{
denominator -= static_cast<T>(1.0) / (1ULL << cur_rank);
@ -164,7 +163,7 @@ class __attribute__((__packed__)) Denominator<precision, max_rank, HashValueType
std::enable_if_t<details::isBigRankStore(precision) && denominator_mode == DenominatorMode::StableIfBig>>
{
public:
Denominator(DenominatorType initial_value)
Denominator(DenominatorType initial_value) /// NOLINT
{
rank_count[0] = initial_value;
}
@ -321,7 +320,7 @@ public:
double final_estimate = fixRawEstimate(raw_estimate);
return static_cast<UInt64>(final_estimate + 0.5);
return static_cast<UInt64>(final_estimate + 0.5); /// NOLINT
}
void merge(const HyperLogLogCounter & rhs)
@ -513,7 +512,6 @@ private:
return fixed_estimate;
}
private:
static constexpr int max_rank = sizeof(HashValueType) * 8 - precision + 1;
RankStore rank_store;

View File

@ -125,7 +125,7 @@ public:
return name;
}
virtual ~IFactoryWithAliases() override {}
virtual ~IFactoryWithAliases() override = default;
private:
using InnerMap = std::unordered_map<String, Value>; // name -> creator

View File

@ -10,7 +10,7 @@ class Increment
{
public:
/// path - the name of the file, including the path
Increment(const std::string & path_) : counter(path_) {}
explicit Increment(const std::string & path_) : counter(path_) {}
/** Get the next number.
* If the `create_if_need` parameter is not set to true, then

View File

@ -21,8 +21,8 @@ struct IntervalKind
};
Kind kind = Second;
IntervalKind(Kind kind_ = Second) : kind(kind_) {}
operator Kind() const { return kind; }
IntervalKind(Kind kind_ = Second) : kind(kind_) {} /// NOLINT
operator Kind() const { return kind; } /// NOLINT
constexpr std::string_view toString() const { return magic_enum::enum_name(kind); }
@ -59,6 +59,7 @@ struct IntervalKind
static bool tryParseString(const std::string & kind, IntervalKind::Kind & result);
};
/// NOLINTNEXTLINE
#define FOR_EACH_INTERVAL_KIND(M) \
M(Second) \
M(Minute) \

View File

@ -33,7 +33,7 @@ using ItemPtr = std::unique_ptr<IItem>;
class JSONString : public IItem
{
public:
JSONString(std::string_view value_) : value(value_) {}
explicit JSONString(std::string_view value_) : value(value_) {}
void format(const FormatSettings & settings, FormatContext & context) override;
private:

View File

@ -39,7 +39,7 @@ public:
/** Initialize LRUCache with max_size and max_elements_size.
* max_elements_size == 0 means no elements size restrictions.
*/
LRUCache(size_t max_size_, size_t max_elements_size_ = 0)
explicit LRUCache(size_t max_size_, size_t max_elements_size_ = 0)
: max_size(std::max(static_cast<size_t>(1), max_size_))
, max_elements_size(max_elements_size_)
{}
@ -174,7 +174,7 @@ public:
misses = 0;
}
virtual ~LRUCache() {}
virtual ~LRUCache() = default;
protected:
using LRUQueue = std::list<Key>;

View File

@ -20,7 +20,7 @@
class LocalDate
{
private:
unsigned short m_year;
unsigned short m_year; /// NOLINT
unsigned char m_month;
unsigned char m_day;
@ -61,7 +61,7 @@ public:
init(time);
}
LocalDate(DayNum day_num)
LocalDate(DayNum day_num) /// NOLINT
{
const auto & values = DateLUT::instance().getValues(day_num);
m_year = values.year;
@ -77,7 +77,7 @@ public:
m_day = values.day_of_month;
}
LocalDate(unsigned short year_, unsigned char month_, unsigned char day_)
LocalDate(unsigned short year_, unsigned char month_, unsigned char day_) /// NOLINT
: m_year(year_), m_month(month_), m_day(day_)
{
}
@ -111,16 +111,16 @@ public:
return ExtendedDayNum (lut.makeDayNum(m_year, m_month, m_day).toUnderType());
}
operator DayNum() const
operator DayNum() const /// NOLINT
{
return getDayNum();
}
unsigned short year() const { return m_year; }
unsigned short year() const { return m_year; } /// NOLINT
unsigned char month() const { return m_month; }
unsigned char day() const { return m_day; }
void year(unsigned short x) { m_year = x; }
void year(unsigned short x) { m_year = x; } /// NOLINT
void month(unsigned char x) { m_month = x; }
void day(unsigned char x) { m_day = x; }

View File

@ -18,7 +18,7 @@
class LocalDateTime
{
private:
unsigned short m_year;
unsigned short m_year; /// NOLINT
unsigned char m_month;
unsigned char m_day;
unsigned char m_hour;
@ -65,7 +65,7 @@ public:
init(time, time_zone);
}
LocalDateTime(unsigned short year_, unsigned char month_, unsigned char day_,
LocalDateTime(unsigned short year_, unsigned char month_, unsigned char day_, /// NOLINT
unsigned char hour_, unsigned char minute_, unsigned char second_)
: m_year(year_), m_month(month_), m_day(day_), m_hour(hour_), m_minute(minute_), m_second(second_)
{
@ -91,14 +91,14 @@ public:
LocalDateTime(const LocalDateTime &) noexcept = default;
LocalDateTime & operator= (const LocalDateTime &) noexcept = default;
unsigned short year() const { return m_year; }
unsigned short year() const { return m_year; } /// NOLINT
unsigned char month() const { return m_month; }
unsigned char day() const { return m_day; }
unsigned char hour() const { return m_hour; }
unsigned char minute() const { return m_minute; }
unsigned char second() const { return m_second; }
void year(unsigned short x) { m_year = x; }
void year(unsigned short x) { m_year = x; } /// NOLINT
void month(unsigned char x) { m_month = x; }
void day(unsigned char x) { m_day = x; }
void hour(unsigned char x) { m_hour = x; }
@ -108,7 +108,7 @@ public:
LocalDate toDate() const { return LocalDate(m_year, m_month, m_day); }
LocalDateTime toStartOfDate() const { return LocalDateTime(m_year, m_month, m_day, 0, 0, 0); }
time_t to_time_t(const DateLUTImpl & time_zone = DateLUT::instance()) const
time_t to_time_t(const DateLUTImpl & time_zone = DateLUT::instance()) const /// NOLINT
{
return time_zone.makeDateTime(m_year, m_month, m_day, m_hour, m_minute, m_second);
}

View File

@ -63,7 +63,7 @@ public:
Names expand(const Names & source_names, size_t level = 0) const;
using MacroMap = std::map<String, String>;
const MacroMap getMacroMap() const { return macros; }
MacroMap getMacroMap() const { return macros; }
String getValue(const String & key) const;

View File

@ -12,10 +12,10 @@
#undef __msan_print_shadow
#undef __msan_unpoison_string
#define __msan_unpoison(X, Y)
#define __msan_test_shadow(X, Y) (false)
#define __msan_print_shadow(X, Y)
#define __msan_unpoison_string(X)
#define __msan_unpoison(X, Y) /// NOLINT
#define __msan_test_shadow(X, Y) (false) /// NOLINT
#define __msan_print_shadow(X, Y) /// NOLINT
#define __msan_unpoison_string(X) /// NOLINT
#if defined(ch_has_feature)
# if ch_has_feature(memory_sanitizer)

View File

@ -16,12 +16,20 @@
#ifdef MEMORY_TRACKER_DEBUG_CHECKS
#include <base/scope_guard.h>
extern thread_local bool memory_tracker_always_throw_logical_error_on_allocation;
/// NOLINTNEXTLINE
#define ALLOCATIONS_IN_SCOPE_IMPL_CONCAT(n, val) \
bool _allocations_flag_prev_val##n = memory_tracker_always_throw_logical_error_on_allocation; \
memory_tracker_always_throw_logical_error_on_allocation = val; \
SCOPE_EXIT({ memory_tracker_always_throw_logical_error_on_allocation = _allocations_flag_prev_val##n; })
/// NOLINTNEXTLINE
#define ALLOCATIONS_IN_SCOPE_IMPL(n, val) ALLOCATIONS_IN_SCOPE_IMPL_CONCAT(n, val)
/// NOLINTNEXTLINE
#define DENY_ALLOCATIONS_IN_SCOPE ALLOCATIONS_IN_SCOPE_IMPL(__LINE__, true)
/// NOLINTNEXTLINE
#define ALLOW_ALLOCATIONS_IN_SCOPE ALLOCATIONS_IN_SCOPE_IMPL(__LINE__, false)
#else
#define DENY_ALLOCATIONS_IN_SCOPE static_assert(true)

View File

@ -30,7 +30,7 @@ public:
/// Default initialization - by nullptr.
MultiVersion() = default;
MultiVersion(std::unique_ptr<const T> && value)
explicit MultiVersion(std::unique_ptr<const T> && value)
{
set(std::move(value));
}

View File

@ -105,9 +105,9 @@ public:
IHints() = default;
IHints(const IHints &) = default;
IHints(IHints &&) = default;
IHints(IHints &&) noexcept = default;
IHints & operator=(const IHints &) = default;
IHints & operator=(IHints &&) = default;
IHints & operator=(IHints &&) noexcept = default;
virtual ~IHints() = default;

View File

@ -34,7 +34,7 @@ protected:
{
SimpleObjectPool<T> * parent;
Deleter(SimpleObjectPool<T> * parent_ = nullptr) : parent{parent_} {}
Deleter(SimpleObjectPool<T> * parent_ = nullptr) : parent{parent_} {} /// NOLINT
void operator()(T * owning_ptr) const
{

View File

@ -1,5 +1,8 @@
#pragma once
#include <base/types.h>
#include <base/UUID.h>
namespace DB
{

View File

@ -55,7 +55,7 @@ public:
using RegexType = std::conditional_t<thread_safe, re2::RE2, re2_st::RE2>;
using StringPieceType = std::conditional_t<thread_safe, re2::StringPiece, re2_st::StringPiece>;
OptimizedRegularExpressionImpl(const std::string & regexp_, int options = 0);
OptimizedRegularExpressionImpl(const std::string & regexp_, int options = 0); /// NOLINT
bool match(const std::string & subject) const
{

View File

@ -106,7 +106,7 @@ protected:
char * c_end_of_storage = null; /// Does not include pad_right.
/// The amount of memory occupied by the num_elements of the elements.
static size_t byte_size(size_t num_elements)
static size_t byte_size(size_t num_elements) /// NOLINT
{
size_t amount;
if (__builtin_mul_overflow(num_elements, ELEMENT_SIZE, &amount))
@ -115,9 +115,9 @@ protected:
}
/// Minimum amount of memory to allocate for num_elements, including padding.
static size_t minimum_memory_for_elements(size_t num_elements) { return byte_size(num_elements) + pad_right + pad_left; }
static size_t minimum_memory_for_elements(size_t num_elements) { return byte_size(num_elements) + pad_right + pad_left; } /// NOLINT
void alloc_for_num_elements(size_t num_elements)
void alloc_for_num_elements(size_t num_elements) /// NOLINT
{
alloc(minimum_memory_for_elements(num_elements));
}
@ -220,7 +220,7 @@ public:
size_t capacity() const { return (c_end_of_storage - c_start) / ELEMENT_SIZE; }
/// This method is safe to use only for information about memory usage.
size_t allocated_bytes() const { return c_end_of_storage - c_start + pad_right + pad_left; }
size_t allocated_bytes() const { return c_end_of_storage - c_start + pad_right + pad_left; } /// NOLINT
void clear() { c_end = c_start; }
@ -235,7 +235,7 @@ public:
}
template <typename ... TAllocatorParams>
void reserve_exact(size_t n, TAllocatorParams &&... allocator_params)
void reserve_exact(size_t n, TAllocatorParams &&... allocator_params) /// NOLINT
{
if (n > capacity())
realloc(minimum_memory_for_elements(n), std::forward<TAllocatorParams>(allocator_params)...);
@ -249,24 +249,24 @@ public:
}
template <typename ... TAllocatorParams>
void resize_exact(size_t n, TAllocatorParams &&... allocator_params)
void resize_exact(size_t n, TAllocatorParams &&... allocator_params) /// NOLINT
{
reserve_exact(n, std::forward<TAllocatorParams>(allocator_params)...);
resize_assume_reserved(n);
}
void resize_assume_reserved(const size_t n)
void resize_assume_reserved(const size_t n) /// NOLINT
{
c_end = c_start + byte_size(n);
}
const char * raw_data() const
const char * raw_data() const /// NOLINT
{
return c_start;
}
template <typename ... TAllocatorParams>
void push_back_raw(const void * ptr, TAllocatorParams &&... allocator_params)
void push_back_raw(const void * ptr, TAllocatorParams &&... allocator_params) /// NOLINT
{
size_t required_capacity = size() + ELEMENT_SIZE;
if (unlikely(required_capacity > capacity()))
@ -317,11 +317,11 @@ class PODArray : public PODArrayBase<sizeof(T), initial_bytes, TAllocator, pad_r
protected:
using Base = PODArrayBase<sizeof(T), initial_bytes, TAllocator, pad_right_, pad_left_>;
T * t_start() { return reinterpret_cast<T *>(this->c_start); }
T * t_end() { return reinterpret_cast<T *>(this->c_end); }
T * t_start() { return reinterpret_cast<T *>(this->c_start); } /// NOLINT
T * t_end() { return reinterpret_cast<T *>(this->c_end); } /// NOLINT
const T * t_start() const { return reinterpret_cast<const T *>(this->c_start); }
const T * t_end() const { return reinterpret_cast<const T *>(this->c_end); }
const T * t_start() const { return reinterpret_cast<const T *>(this->c_start); } /// NOLINT
const T * t_end() const { return reinterpret_cast<const T *>(this->c_end); } /// NOLINT
public:
using value_type = T;
@ -335,7 +335,7 @@ public:
PODArray() = default;
PODArray(size_t n)
explicit PODArray(size_t n)
{
this->alloc_for_num_elements(n);
this->c_end += this->byte_size(n);
@ -363,12 +363,12 @@ public:
}
}
PODArray(PODArray && other)
PODArray(PODArray && other) noexcept
{
this->swap(other);
}
PODArray & operator=(PODArray && other)
PODArray & operator=(PODArray && other) noexcept
{
this->swap(other);
return *this;
@ -404,7 +404,7 @@ public:
const_iterator cend() const { return t_end(); }
/// Same as resize, but zeroes new elements.
void resize_fill(size_t n)
void resize_fill(size_t n) /// NOLINT
{
size_t old_size = this->size();
if (n > old_size)
@ -415,7 +415,7 @@ public:
this->c_end = this->c_start + this->byte_size(n);
}
void resize_fill(size_t n, const T & value)
void resize_fill(size_t n, const T & value) /// NOLINT
{
size_t old_size = this->size();
if (n > old_size)
@ -427,7 +427,7 @@ public:
}
template <typename U, typename ... TAllocatorParams>
void push_back(U && x, TAllocatorParams &&... allocator_params)
void push_back(U && x, TAllocatorParams &&... allocator_params) /// NOLINT
{
if (unlikely(this->c_end + sizeof(T) > this->c_end_of_storage))
this->reserveForNextSize(std::forward<TAllocatorParams>(allocator_params)...);
@ -440,7 +440,7 @@ public:
* and it couldn't be used if Allocator requires custom parameters.
*/
template <typename... Args>
void emplace_back(Args &&... args)
void emplace_back(Args &&... args) /// NOLINT
{
if (unlikely(this->c_end + sizeof(T) > this->c_end_of_storage))
this->reserveForNextSize();
@ -449,7 +449,7 @@ public:
this->c_end += this->byte_size(1);
}
void pop_back()
void pop_back() /// NOLINT
{
this->c_end -= this->byte_size(1);
}
@ -554,7 +554,7 @@ public:
}
template <typename It1, typename It2>
void insert_assume_reserved(It1 from_begin, It2 from_end)
void insert_assume_reserved(It1 from_begin, It2 from_end) /// NOLINT
{
static_assert(memcpy_can_be_used_for_assignment<std::decay_t<T>, std::decay_t<decltype(*from_begin)>>);
this->assertNotIntersects(from_begin, from_end);

View File

@ -395,8 +395,8 @@ void PoolWithFailoverBase<TNestedPool>::updateErrorCounts(PoolWithFailoverBase<T
if (delta >= 0)
{
const UInt64 MAX_BITS = sizeof(UInt64) * CHAR_BIT;
size_t shift_amount = MAX_BITS;
const UInt64 max_bits = sizeof(UInt64) * CHAR_BIT;
size_t shift_amount = max_bits;
/// Divide error counts by 2 every decrease_error_period seconds.
if (decrease_error_period)
shift_amount = delta / decrease_error_period;
@ -405,7 +405,7 @@ void PoolWithFailoverBase<TNestedPool>::updateErrorCounts(PoolWithFailoverBase<T
if (shift_amount)
last_decrease_time = current_time;
if (shift_amount >= MAX_BITS)
if (shift_amount >= max_bits)
{
for (auto & state : states)
{

View File

@ -92,7 +92,7 @@ bool ProcfsMetricsProvider::isAvailable() noexcept
}
ProcfsMetricsProvider::ProcfsMetricsProvider(const pid_t /*tid*/)
ProcfsMetricsProvider::ProcfsMetricsProvider(pid_t /*tid*/)
{
thread_schedstat_fd = ::open(thread_schedstat, O_RDONLY | O_CLOEXEC);
if (-1 == thread_schedstat_fd)

View File

@ -13,7 +13,7 @@ namespace DB
class ProcfsMetricsProvider : private boost::noncopyable
{
public:
ProcfsMetricsProvider(const pid_t /*tid*/);
explicit ProcfsMetricsProvider(pid_t /*tid*/);
~ProcfsMetricsProvider();
/// Updates only a part of taskstats struct's fields:
@ -31,13 +31,12 @@ private:
void readParseAndSetThreadBlkIOStat(::taskstats & out_stats, char *, size_t) const;
void readParseAndSetThreadIOStat(::taskstats & out_stats, char *, size_t) const;
private:
int thread_schedstat_fd = -1;
int thread_stat_fd = -1;
int thread_io_fd = -1;
/// This field is used for compatibility with TasksStatsCounters::incrementProfileEvents()
unsigned short stats_version = 1;
unsigned short stats_version = 1; /// NOLINT
};
}

View File

@ -35,10 +35,10 @@ namespace ProfileEvents
VariableContext level = VariableContext::Thread;
/// By default, any instance have to increment global counters
Counters(VariableContext level_ = VariableContext::Thread, Counters * parent_ = &global_counters);
explicit Counters(VariableContext level_ = VariableContext::Thread, Counters * parent_ = &global_counters);
/// Global level static initializer
Counters(Counter * allocated_counters) noexcept
explicit Counters(Counter * allocated_counters) noexcept
: counters(allocated_counters), parent(nullptr), level(VariableContext::Global) {}
Counter & operator[] (Event event)

View File

@ -86,7 +86,7 @@ namespace ErrorCodes
}
template <typename ProfilerImpl>
QueryProfilerBase<ProfilerImpl>::QueryProfilerBase(const UInt64 thread_id, const int clock_type, UInt32 period, const int pause_signal_)
QueryProfilerBase<ProfilerImpl>::QueryProfilerBase(UInt64 thread_id, int clock_type, UInt32 period, int pause_signal_)
: log(&Poco::Logger::get("QueryProfiler"))
, pause_signal(pause_signal_)
{
@ -199,7 +199,7 @@ void QueryProfilerBase<ProfilerImpl>::tryCleanup()
template class QueryProfilerBase<QueryProfilerReal>;
template class QueryProfilerBase<QueryProfilerCPU>;
QueryProfilerReal::QueryProfilerReal(const UInt64 thread_id, const UInt32 period)
QueryProfilerReal::QueryProfilerReal(UInt64 thread_id, UInt32 period)
: QueryProfilerBase(thread_id, CLOCK_MONOTONIC, period, SIGUSR1)
{}
@ -212,7 +212,7 @@ void QueryProfilerReal::signalHandler(int sig, siginfo_t * info, void * context)
writeTraceInfo(TraceType::Real, sig, info, context);
}
QueryProfilerCPU::QueryProfilerCPU(const UInt64 thread_id, const UInt32 period)
QueryProfilerCPU::QueryProfilerCPU(UInt64 thread_id, UInt32 period)
: QueryProfilerBase(thread_id, CLOCK_THREAD_CPUTIME_ID, period, SIGUSR2)
{}

View File

@ -31,7 +31,7 @@ template <typename ProfilerImpl>
class QueryProfilerBase
{
public:
QueryProfilerBase(const UInt64 thread_id, const int clock_type, UInt32 period, const int pause_signal_);
QueryProfilerBase(UInt64 thread_id, int clock_type, UInt32 period, int pause_signal_);
~QueryProfilerBase();
private:
@ -52,7 +52,7 @@ private:
class QueryProfilerReal : public QueryProfilerBase<QueryProfilerReal>
{
public:
QueryProfilerReal(const UInt64 thread_id, const UInt32 period);
QueryProfilerReal(UInt64 thread_id, UInt32 period); /// NOLINT
static void signalHandler(int sig, siginfo_t * info, void * context);
};
@ -61,7 +61,7 @@ public:
class QueryProfilerCPU : public QueryProfilerBase<QueryProfilerCPU>
{
public:
QueryProfilerCPU(const UInt64 thread_id, const UInt32 period);
QueryProfilerCPU(UInt64 thread_id, UInt32 period); /// NOLINT
static void signalHandler(int sig, siginfo_t * info, void * context);
};

View File

@ -37,12 +37,12 @@
*/
struct RadixSortAllocator
{
void * allocate(size_t size)
static void * allocate(size_t size)
{
return ::operator new(size);
}
void deallocate(void * ptr, size_t size)
static void deallocate(void * ptr, size_t size)
{
::operator delete(ptr, size);
}

View File

@ -13,7 +13,7 @@ struct SettingChange
String name;
Field value;
SettingChange() {}
SettingChange() = default;
SettingChange(const std::string_view & name_, const Field & value_) : name(name_), value(value_) {}
SettingChange(const std::string_view & name_, Field && value_) : name(name_), value(std::move(value_)) {}

View File

@ -24,7 +24,7 @@ namespace detail
ColumnRawPtrs all_columns;
ColumnRawPtrs sort_columns;
SharedBlock(Block && block) : Block(std::move(block)) {}
explicit SharedBlock(Block && block) : Block(std::move(block)) {}
};
}

View File

@ -45,11 +45,11 @@ public:
struct Config
{
Config(const std::string & command_)
Config(const std::string & command_) /// NOLINT
: command(command_)
{}
Config(const char * command_)
Config(const char * command_) /// NOLINT
: command(command_)
{}

View File

@ -71,7 +71,7 @@ private:
public:
/// Arguments - seed.
SipHash(UInt64 k0 = 0, UInt64 k1 = 0)
explicit SipHash(UInt64 k0 = 0, UInt64 k1 = 0)
{
/// Initialize the state with some random bytes and seed.
v0 = 0x736f6d6570736575ULL ^ k0;
@ -139,7 +139,7 @@ public:
template <typename T>
void update(const T & x)
{
update(reinterpret_cast<const char *>(&x), sizeof(x));
update(reinterpret_cast<const char *>(&x), sizeof(x)); /// NOLINT
}
void update(const std::string & x)

View File

@ -18,7 +18,7 @@ private:
char * free_list = nullptr;
public:
SmallObjectPool(size_t object_size_)
explicit SmallObjectPool(size_t object_size_)
: object_size{std::max(object_size_, sizeof(char *))}
{
}

View File

@ -9,6 +9,7 @@
#include <Common/AllocatorWithMemoryTracking.h>
#include <Common/ArenaWithFreeLists.h>
#include <Common/ArenaUtils.h>
#include <Common/HashTable/Hash.h>
#include <Common/HashTable/HashMap.h>
@ -35,8 +36,8 @@ namespace DB
template <typename TKey>
struct SpaceSavingArena
{
SpaceSavingArena() {}
const TKey emplace(const TKey & key) { return key; }
SpaceSavingArena() = default;
TKey emplace(const TKey & key) { return key; }
void free(const TKey & /*key*/) {}
};
@ -48,11 +49,9 @@ struct SpaceSavingArena
template <>
struct SpaceSavingArena<StringRef>
{
const StringRef emplace(const StringRef & key)
StringRef emplace(const StringRef & key)
{
auto ptr = arena.alloc(key.size);
std::copy(key.data, key.data + key.size, ptr);
return StringRef{ptr, key.size};
return copyStringInArena(arena, key);
}
void free(const StringRef & key)
@ -78,8 +77,8 @@ private:
// Round to nearest power of 2 for cheaper binning without modulo
constexpr uint64_t nextAlphaSize(uint64_t x)
{
constexpr uint64_t ALPHA_MAP_ELEMENTS_PER_COUNTER = 6;
return 1ULL << (sizeof(uint64_t) * 8 - __builtin_clzll(x * ALPHA_MAP_ELEMENTS_PER_COUNTER));
constexpr uint64_t alpha_map_elements_per_counter = 6;
return 1ULL << (sizeof(uint64_t) * 8 - __builtin_clzll(x * alpha_map_elements_per_counter));
}
public:
@ -89,7 +88,7 @@ public:
{
Counter() = default; //-V730
Counter(const TKey & k, UInt64 c = 0, UInt64 e = 0, size_t h = 0)
explicit Counter(const TKey & k, UInt64 c = 0, UInt64 e = 0, size_t h = 0)
: key(k), slot(0), hash(h), count(c), error(e) {}
void write(WriteBuffer & wb) const
@ -119,7 +118,7 @@ public:
UInt64 error;
};
SpaceSaving(size_t c = 10) : alpha_map(nextAlphaSize(c)), m_capacity(c) {}
explicit SpaceSaving(size_t c = 10) : alpha_map(nextAlphaSize(c)), m_capacity(c) {}
~SpaceSaving() { destroyElements(); }

View File

@ -137,7 +137,7 @@ public:
patu = _mm_set1_epi8(u);
/// lower and uppercase vectors of first 16 octets of `needle`
auto needle_pos = needle;
const auto * needle_pos = needle;
for (size_t i = 0; i < n;)
{
@ -251,7 +251,7 @@ public:
if (*pos == l || *pos == u)
{
pos += first_needle_symbol_is_ascii;
auto needle_pos = needle + first_needle_symbol_is_ascii;
const auto * needle_pos = needle + first_needle_symbol_is_ascii;
if (compareTrivial(pos, haystack_end, needle_pos))
return true;
@ -325,7 +325,7 @@ public:
if (*haystack == l || *haystack == u)
{
auto haystack_pos = haystack + first_needle_symbol_is_ascii;
auto needle_pos = needle + first_needle_symbol_is_ascii;
const auto * needle_pos = needle + first_needle_symbol_is_ascii;
if (compareTrivial(haystack_pos, haystack_end, needle_pos))
return haystack;
@ -381,7 +381,7 @@ public:
patl = _mm_set1_epi8(l);
patu = _mm_set1_epi8(u);
auto needle_pos = needle;
const auto * needle_pos = needle;
for (const auto i : collections::range(0, n))
{
@ -416,7 +416,7 @@ public:
if (mask == cachemask)
{
pos += n;
auto needle_pos = needle + n;
const auto * needle_pos = needle + n;
while (needle_pos < needle_end && std::tolower(*pos) == std::tolower(*needle_pos))
{
@ -438,7 +438,7 @@ public:
if (*pos == l || *pos == u)
{
++pos;
auto needle_pos = needle + 1;
const auto * needle_pos = needle + 1;
while (needle_pos < needle_end && std::tolower(*pos) == std::tolower(*needle_pos))
{
@ -492,8 +492,8 @@ public:
{
if (mask_offset == cachemask)
{
auto haystack_pos = haystack + n;
auto needle_pos = needle + n;
const auto * haystack_pos = haystack + n;
const auto * needle_pos = needle + n;
while (haystack_pos < haystack_end && needle_pos < needle_end &&
std::tolower(*haystack_pos) == std::tolower(*needle_pos))
@ -520,8 +520,8 @@ public:
if (*haystack == l || *haystack == u)
{
auto haystack_pos = haystack + 1;
auto needle_pos = needle + 1;
const auto * haystack_pos = haystack + 1;
const auto * needle_pos = needle + 1;
while (haystack_pos < haystack_end && needle_pos < needle_end &&
std::tolower(*haystack_pos) == std::tolower(*needle_pos))
@ -580,7 +580,7 @@ public:
#ifdef __SSE4_1__
pattern = _mm_set1_epi8(first);
auto needle_pos = needle;
const auto * needle_pos = needle;
for (const auto i : collections::range(0, n))
{
@ -611,7 +611,7 @@ public:
if (mask == cachemask)
{
pos += n;
auto needle_pos = needle + n;
const auto * needle_pos = needle + n;
while (needle_pos < needle_end && *pos == *needle_pos)
++pos, ++needle_pos;
@ -630,7 +630,7 @@ public:
if (*pos == first)
{
++pos;
auto needle_pos = needle + 1;
const auto * needle_pos = needle + 1;
while (needle_pos < needle_end && *pos == *needle_pos)
++pos, ++needle_pos;
@ -680,8 +680,8 @@ public:
{
if (mask_offset == cachemask)
{
auto haystack_pos = haystack + n;
auto needle_pos = needle + n;
const auto * haystack_pos = haystack + n;
const auto * needle_pos = needle + n;
while (haystack_pos < haystack_end && needle_pos < needle_end &&
*haystack_pos == *needle_pos)
@ -705,8 +705,8 @@ public:
if (*haystack == first)
{
auto haystack_pos = haystack + 1;
auto needle_pos = needle + 1;
const auto * haystack_pos = haystack + 1;
const auto * needle_pos = needle + 1;
while (haystack_pos < haystack_end && needle_pos < needle_end &&
*haystack_pos == *needle_pos)
@ -796,10 +796,7 @@ public:
ALWAYS_INLINE static bool isTokenSeparator(const uint8_t c)
{
if (isAlphaNumericASCII(c) || !isASCII(c))
return false;
return true;
return !(isAlphaNumericASCII(c) || !isASCII(c));
}
};
@ -829,7 +826,7 @@ struct LibCASCIICaseSensitiveStringSearcher : public StringSearcherBase
template <typename CharT, typename = std::enable_if_t<sizeof(CharT) == 1>>
const CharT * search(const CharT * haystack, const CharT * const haystack_end) const
{
auto res = strstr(reinterpret_cast<const char *>(haystack), reinterpret_cast<const char *>(needle));
const auto * res = strstr(reinterpret_cast<const char *>(haystack), reinterpret_cast<const char *>(needle));
if (!res)
return haystack_end;
return reinterpret_cast<const CharT *>(res);
@ -853,7 +850,7 @@ struct LibCASCIICaseInsensitiveStringSearcher : public StringSearcherBase
template <typename CharT, typename = std::enable_if_t<sizeof(CharT) == 1>>
const CharT * search(const CharT * haystack, const CharT * const haystack_end) const
{
auto res = strcasestr(reinterpret_cast<const char *>(haystack), reinterpret_cast<const char *>(needle));
const auto * res = strcasestr(reinterpret_cast<const char *>(haystack), reinterpret_cast<const char *>(needle));
if (!res)
return haystack_end;
return reinterpret_cast<const CharT *>(res);

View File

@ -45,7 +45,7 @@ class ISystemLog
public:
virtual String getName() = 0;
//// force -- force table creation (used for SYSTEM FLUSH LOGS)
virtual void flush(bool force = false) = 0;
virtual void flush(bool force = false) = 0; /// NOLINT
virtual void prepareTable() = 0;
/// Start the background thread.

View File

@ -20,7 +20,7 @@ class TLDList
public:
using Container = StringHashSet<>;
TLDList(size_t size);
explicit TLDList(size_t size);
/// Return true if the tld_container does not contains such element.
bool insert(const StringRef & host);

View File

@ -12,5 +12,5 @@ uint16_t getTerminalWidth();
/** Creates po::options_description with name and an appropriate size for option displaying
* when program is called with option --help
* */
po::options_description createOptionsDescription(const std::string &caption, unsigned short terminal_width);
po::options_description createOptionsDescription(const std::string &caption, unsigned short terminal_width); /// NOLINT

View File

@ -189,12 +189,12 @@ public:
});
}
ThreadFromGlobalPool(ThreadFromGlobalPool && rhs)
ThreadFromGlobalPool(ThreadFromGlobalPool && rhs) noexcept
{
*this = std::move(rhs);
}
ThreadFromGlobalPool & operator=(ThreadFromGlobalPool && rhs)
ThreadFromGlobalPool & operator=(ThreadFromGlobalPool && rhs) noexcept
{
if (joinable())
abort();

View File

@ -71,7 +71,7 @@ bool TasksStatsCounters::checkIfAvailable()
return findBestAvailableProvider() != MetricsProvider::None;
}
std::unique_ptr<TasksStatsCounters> TasksStatsCounters::create(const UInt64 tid)
std::unique_ptr<TasksStatsCounters> TasksStatsCounters::create(UInt64 tid)
{
std::unique_ptr<TasksStatsCounters> instance;
if (checkIfAvailable())

View File

@ -177,7 +177,7 @@ class TasksStatsCounters
{
public:
static bool checkIfAvailable();
static std::unique_ptr<TasksStatsCounters> create(const UInt64 tid);
static std::unique_ptr<TasksStatsCounters> create(UInt64 tid);
void reset();
void updateCounters(ProfileEvents::Counters & profile_events);
@ -193,8 +193,7 @@ private:
Netlink
};
private:
explicit TasksStatsCounters(const UInt64 tid, const MetricsProvider provider);
explicit TasksStatsCounters(UInt64 tid, MetricsProvider provider);
static MetricsProvider findBestAvailableProvider();
static void incrementProfileEvents(const ::taskstats & prev, const ::taskstats & curr, ProfileEvents::Counters & profile_events);

View File

@ -18,7 +18,7 @@ namespace DB
class Throttler
{
public:
Throttler(size_t max_speed_, const std::shared_ptr<Throttler> & parent_ = nullptr)
explicit Throttler(size_t max_speed_, const std::shared_ptr<Throttler> & parent_ = nullptr)
: max_speed(max_speed_), limit_exceeded_exception_message(""), parent(parent_) {}
Throttler(size_t max_speed_, size_t limit_, const char * limit_exceeded_exception_message_,

View File

@ -4,11 +4,12 @@
#include <base/types.h>
class StackTrace;
class TraceCollector;
namespace DB
{
class TraceCollector;
enum class TraceType : uint8_t
{
Real,

View File

@ -55,11 +55,11 @@ inline size_t seqLength(const UInt8 first_octet)
inline size_t countCodePoints(const UInt8 * data, size_t size)
{
size_t res = 0;
const auto end = data + size;
const auto * end = data + size;
#ifdef __SSE2__
constexpr auto bytes_sse = sizeof(__m128i);
const auto src_end_sse = data + size / bytes_sse * bytes_sse;
const auto * src_end_sse = data + size / bytes_sse * bytes_sse;
const auto threshold = _mm_set1_epi8(0xBF);

View File

@ -19,15 +19,15 @@ struct VersionNumber
VersionNumber(const std::initializer_list<Int64> & init)
: components(init)
{}
VersionNumber(Int64 major, Int64 minor = 0, Int64 patch = 0)
VersionNumber(Int64 major, Int64 minor = 0, Int64 patch = 0) /// NOLINT
: components{major, minor, patch}
{}
VersionNumber(const std::vector<Int64> & components_)
VersionNumber(const std::vector<Int64> & components_) /// NOLINT
: components(components_)
{}
/// Parse version number from string.
VersionNumber(std::string version);
explicit VersionNumber(std::string version);
/// NOTE: operator<=> can be used once libc++ will be upgraded.
bool operator<(const VersionNumber & rhs) const { return compare(rhs.components) < 0; }

View File

@ -3,6 +3,7 @@
#include <base/demangle.h>
#include <base/TypeList.h>
#include <Common/Exception.h>
#include <typeinfo>
/* Generic utils which are intended for visitor pattern implementation.
* The original purpose is to provide possibility to get concrete template specialisation for type in list.

View File

@ -155,7 +155,7 @@ namespace VolnitskyTraits
if (UTF8::isContinuationOctet(chars.c1))
{
/// ngram is inside a sequence
auto seq_pos = pos;
const auto * seq_pos = pos;
UTF8::syncBackward(seq_pos, begin);
auto u32 = UTF8::convertUTF8ToCodePoint(seq_pos, end - seq_pos);
@ -205,7 +205,7 @@ namespace VolnitskyTraits
{
/// ngram is on the boundary of two sequences
/// first sequence may start before u_pos if it is not ASCII
auto first_seq_pos = pos;
const auto * first_seq_pos = pos;
UTF8::syncBackward(first_seq_pos, begin);
/// where is the given ngram in respect to the start of first UTF-8 sequence?
size_t seq_ngram_offset = pos - first_seq_pos;
@ -221,7 +221,7 @@ namespace VolnitskyTraits
}
/// second sequence always start immediately after u_pos
auto second_seq_pos = pos + 1;
const auto * second_seq_pos = pos + 1;
auto second_u32 = UTF8::convertUTF8ToCodePoint(second_seq_pos, end - second_seq_pos);
int second_l_u32 = 0;
@ -427,7 +427,7 @@ public:
if (needle_size == 0)
return haystack;
const auto haystack_end = haystack + haystack_size;
const auto * haystack_end = haystack + haystack_size;
if (fallback || haystack_size <= needle_size || fallback_searcher.force_fallback)
return fallback_searcher.search(haystack, haystack_end);
@ -441,7 +441,7 @@ public:
cell_num = (cell_num + 1) % VolnitskyTraits::hash_size)
{
/// When found - compare bytewise, using the offset from the hash table.
const auto res = pos - (hash[cell_num] - 1);
const auto * res = pos - (hash[cell_num] - 1);
/// pointer in the code is always padded array so we can use pagesafe semantics
if (fallback_searcher.compare(haystack, haystack_end, res))
@ -502,7 +502,7 @@ private:
static constexpr size_t small_limit = VolnitskyTraits::hash_size / 8;
public:
MultiVolnitskyBase(const std::vector<StringRef> & needles_) : needles{needles_}, step{0}, last{0}
explicit MultiVolnitskyBase(const std::vector<StringRef> & needles_) : needles{needles_}, step{0}, last{0}
{
fallback_searchers.reserve(needles.size());
hash = std::unique_ptr<OffsetId[]>(new OffsetId[VolnitskyTraits::hash_size]); /// No zero initialization, it will be done later.

View File

@ -47,9 +47,9 @@ struct Stat
int32_t version;
int32_t cversion;
int32_t aversion;
int64_t ephemeralOwner;
int32_t dataLength;
int32_t numChildren;
int64_t ephemeralOwner; /// NOLINT
int32_t dataLength; /// NOLINT
int32_t numChildren; /// NOLINT
int64_t pzxid;
};
@ -387,12 +387,12 @@ class Exception : public DB::Exception
{
private:
/// Delegate constructor, used to minimize repetition; last parameter used for overload resolution.
Exception(const std::string & msg, const Error code_, int);
Exception(const std::string & msg, const Error code_, int); /// NOLINT
public:
explicit Exception(const Error code_);
Exception(const std::string & msg, const Error code_);
Exception(const Error code_, const std::string & path);
explicit Exception(const Error code_); /// NOLINT
Exception(const std::string & msg, const Error code_); /// NOLINT
Exception(const Error code_, const std::string & path); /// NOLINT
Exception(const Exception & exc);
const char * name() const throw() override { return "Coordination::Exception"; }

View File

@ -53,14 +53,14 @@ public:
using Ptr = std::shared_ptr<ZooKeeper>;
/// hosts_string -- comma separated [secure://]host:port list
ZooKeeper(const std::string & hosts_string, const std::string & identity_ = "",
explicit ZooKeeper(const std::string & hosts_string, const std::string & identity_ = "",
int32_t session_timeout_ms_ = Coordination::DEFAULT_SESSION_TIMEOUT_MS,
int32_t operation_timeout_ms_ = Coordination::DEFAULT_OPERATION_TIMEOUT_MS,
const std::string & chroot_ = "",
const std::string & implementation_ = "zookeeper",
std::shared_ptr<DB::ZooKeeperLog> zk_log_ = nullptr);
ZooKeeper(const Strings & hosts_, const std::string & identity_ = "",
explicit ZooKeeper(const Strings & hosts_, const std::string & identity_ = "",
int32_t session_timeout_ms_ = Coordination::DEFAULT_SESSION_TIMEOUT_MS,
int32_t operation_timeout_ms_ = Coordination::DEFAULT_OPERATION_TIMEOUT_MS,
const std::string & chroot_ = "",

View File

@ -524,7 +524,6 @@ public:
private:
OpNumToRequest op_num_to_request;
private:
ZooKeeperRequestFactory();
};

View File

@ -31,7 +31,7 @@ namespace zkutil
class ZooKeeperNodeCache
{
public:
ZooKeeperNodeCache(GetZooKeeper get_zookeeper);
explicit ZooKeeperNodeCache(GetZooKeeper get_zookeeper);
ZooKeeperNodeCache(const ZooKeeperNodeCache &) = delete;
ZooKeeperNodeCache(ZooKeeperNodeCache &&) = default;

View File

@ -36,10 +36,10 @@ struct ReadableSize
template <>
struct fmt::formatter<ReadableSize>
{
constexpr auto parse(format_parse_context & ctx)
constexpr static auto parse(format_parse_context & ctx)
{
auto it = ctx.begin();
auto end = ctx.end();
const auto * it = ctx.begin();
const auto * end = ctx.end();
/// Only support {}.
if (it != end && *it != '}')

View File

@ -83,7 +83,7 @@ public:
/// Creates connection. Can be used if Poco::Util::Application is using.
/// All settings will be got from config_name section of configuration.
Connection(const std::string & config_name);
explicit Connection(const std::string & config_name);
~Connection();

View File

@ -10,7 +10,7 @@ namespace mysqlxx
/// Common exception class for MySQL library. Functions code() and errnum() return error numbers from MySQL, for details see mysqld_error.h
struct Exception : public Poco::Exception
{
Exception(const std::string & msg, int code = 0) : Poco::Exception(msg, code) {}
explicit Exception(const std::string & msg, int code = 0) : Poco::Exception(msg, code) {}
int errnum() const { return code(); }
const char * name() const throw() override { return "mysqlxx::Exception"; }
const char * className() const throw() override { return "mysqlxx::Exception"; }
@ -20,7 +20,7 @@ struct Exception : public Poco::Exception
/// Cannot connect to MySQL server
struct ConnectionFailed : public Exception
{
ConnectionFailed(const std::string & msg, int code = 0) : Exception(msg, code) {}
explicit ConnectionFailed(const std::string & msg, int code = 0) : Exception(msg, code) {}
const char * name() const throw() override { return "mysqlxx::ConnectionFailed"; }
const char * className() const throw() override { return "mysqlxx::ConnectionFailed"; }
};
@ -29,7 +29,7 @@ struct ConnectionFailed : public Exception
/// Connection to MySQL server was lost
struct ConnectionLost : public Exception
{
ConnectionLost(const std::string & msg, int code = 0) : Exception(msg, code) {}
explicit ConnectionLost(const std::string & msg, int code = 0) : Exception(msg, code) {}
const char * name() const throw() override { return "mysqlxx::ConnectionLost"; }
const char * className() const throw() override { return "mysqlxx::ConnectionLost"; }
};
@ -38,7 +38,7 @@ struct ConnectionLost : public Exception
/// Erroneous query.
struct BadQuery : public Exception
{
BadQuery(const std::string & msg, int code = 0) : Exception(msg, code) {}
explicit BadQuery(const std::string & msg, int code = 0) : Exception(msg, code) {}
const char * name() const throw() override { return "mysqlxx::BadQuery"; }
const char * className() const throw() override { return "mysqlxx::BadQuery"; }
};
@ -47,7 +47,7 @@ struct BadQuery : public Exception
/// Value parsing failure
struct CannotParseValue : public Exception
{
CannotParseValue(const std::string & msg, int code = 0) : Exception(msg, code) {}
explicit CannotParseValue(const std::string & msg, int code = 0) : Exception(msg, code) {}
const char * name() const throw() override { return "mysqlxx::CannotParseValue"; }
const char * className() const throw() override { return "mysqlxx::CannotParseValue"; }
};

View File

@ -27,17 +27,17 @@ public:
Null() : is_null(true) {}
Null(const Null<T> &) = default;
Null(Null<T> &&) noexcept = default;
Null(NullType) : is_null(true) {}
Null(NullType) : is_null(true) {} /// NOLINT
explicit Null(const T & data_) : data(data_), is_null(false) {}
operator T & ()
operator T & () /// NOLINT
{
if (is_null)
throw Exception("Value is NULL");
return data;
}
operator const T & () const
operator const T & () const /// NOLINT
{
if (is_null)
throw Exception("Value is NULL");

View File

@ -48,7 +48,7 @@ public:
class Entry
{
public:
Entry() {}
Entry() {} /// NOLINT
Entry(const Entry & src)
: data(src.data), pool(src.pool)
@ -61,7 +61,7 @@ public:
decrementRefCount();
}
Entry & operator= (const Entry & src)
Entry & operator= (const Entry & src) /// NOLINT
{
pool = src.pool;
if (data)
@ -77,13 +77,13 @@ public:
return data == nullptr;
}
operator mysqlxx::Connection & () &
operator mysqlxx::Connection & () & /// NOLINT
{
forceConnected();
return data->conn;
}
operator const mysqlxx::Connection & () const &
operator const mysqlxx::Connection & () const & /// NOLINT
{
forceConnected();
return data->conn;
@ -136,7 +136,7 @@ public:
};
Pool(const std::string & config_name,
explicit Pool(const std::string & config_name,
unsigned default_connections_ = MYSQLXX_POOL_DEFAULT_START_CONNECTIONS,
unsigned max_connections_ = MYSQLXX_POOL_DEFAULT_MAX_CONNECTIONS,
const char * parent_config_name_ = nullptr)

View File

@ -101,7 +101,7 @@ namespace mysqlxx
* max_tries_ Max number of connection tries.
* wait_timeout_ Timeout for waiting free connection.
*/
PoolWithFailover(
explicit PoolWithFailover(
const std::string & config_name_,
unsigned default_connections_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS,
unsigned max_connections_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS,

View File

@ -27,7 +27,7 @@ public:
Connection * getConnection() { return conn; }
MYSQL_FIELDS getFields() { return fields; }
unsigned getNumFields() { return num_fields; }
unsigned getNumFields() const { return num_fields; }
MYSQL_RES * getRes() { return res; }
const Query * getQuery() const { return query; }

View File

@ -27,11 +27,11 @@ private:
* @see http://www.artima.com/cppsource/safebool.html
* Взято из mysql++.
*/
typedef MYSQL_ROW Row::*private_bool_type;
typedef MYSQL_ROW Row::*private_bool_type; /// NOLINT
public:
/** Для возможности отложенной инициализации. */
Row()
Row() /// NOLINT
{
}
@ -77,7 +77,7 @@ public:
/** Преобразование в bool.
* (Точнее - в тип, который преобразуется в bool, и с которым больше почти ничего нельзя сделать.)
*/
operator private_bool_type() const { return row == nullptr ? nullptr : &Row::row; }
operator private_bool_type() const { return row == nullptr ? nullptr : &Row::row; } /// NOLINT
enum enum_field_types getFieldType(size_t i);

View File

@ -14,7 +14,7 @@ namespace mysqlxx
class Transaction : private boost::noncopyable
{
public:
Transaction(Connection & conn_)
explicit Transaction(Connection & conn_)
: conn(conn_), finished(false)
{
conn.query("START TRANSACTION").execute();

View File

@ -16,7 +16,7 @@ using MYSQL_ROW = char**;
struct st_mysql_field;
using MYSQL_FIELD = st_mysql_field;
enum struct enum_field_types { MYSQL_TYPE_DECIMAL, MYSQL_TYPE_TINY,
enum struct enum_field_types { MYSQL_TYPE_DECIMAL, MYSQL_TYPE_TINY, /// NOLINT
MYSQL_TYPE_SHORT, MYSQL_TYPE_LONG,
MYSQL_TYPE_FLOAT, MYSQL_TYPE_DOUBLE,
MYSQL_TYPE_NULL, MYSQL_TYPE_TIMESTAMP,

View File

@ -30,7 +30,7 @@ public:
Row fetch();
/// Для совместимости
Row fetch_row() { return fetch(); }
Row fetch_row() { return fetch(); } /// NOLINT
};
}

View File

@ -115,7 +115,7 @@ public:
}
/// Для совместимости (используйте вместо этого метод isNull())
bool is_null() const { return isNull(); }
bool is_null() const { return isNull(); } /// NOLINT
/** Получить любой поддерживаемый тип (для шаблонного кода).
* Поддерживаются основные типы, а также любые типы с конструктором от Value (для удобства расширения).
@ -123,7 +123,7 @@ public:
template <typename T> T get() const;
/// Для совместимости. Не рекомендуется к использованию, так как неудобен (часто возникают неоднозначности).
template <typename T> operator T() const { return get<T>(); }
template <typename T> operator T() const { return get<T>(); } /// NOLINT
const char * data() const { return m_data; }
size_t length() const { return m_length; }
@ -166,7 +166,7 @@ private:
else
throwException("Cannot parse DateTime");
return 0; /// avoid warning.
return 0; /// avoid warning. /// NOLINT
}
@ -184,7 +184,7 @@ private:
else
throwException("Cannot parse Date");
return 0; /// avoid warning.
return 0; /// avoid warning. /// NOLINT
}
@ -240,14 +240,14 @@ template <> inline char Value::get<char >() cons
template <> inline signed char Value::get<signed char >() const { return getInt(); }
template <> inline unsigned char Value::get<unsigned char >() const { return getUInt(); }
template <> inline char8_t Value::get<char8_t >() const { return getUInt(); }
template <> inline short Value::get<short >() const { return getInt(); }
template <> inline unsigned short Value::get<unsigned short >() const { return getUInt(); }
template <> inline short Value::get<short >() const { return getInt(); } /// NOLINT
template <> inline unsigned short Value::get<unsigned short >() const { return getUInt(); } /// NOLINT
template <> inline int Value::get<int >() const { return getInt(); }
template <> inline unsigned int Value::get<unsigned int >() const { return getUInt(); }
template <> inline long Value::get<long >() const { return getInt(); }
template <> inline unsigned long Value::get<unsigned long >() const { return getUInt(); }
template <> inline long long Value::get<long long >() const { return getInt(); }
template <> inline unsigned long long Value::get<unsigned long long >() const { return getUInt(); }
template <> inline long Value::get<long >() const { return getInt(); } /// NOLINT
template <> inline unsigned long Value::get<unsigned long >() const { return getUInt(); } /// NOLINT
template <> inline long long Value::get<long long >() const { return getInt(); } /// NOLINT
template <> inline unsigned long long Value::get<unsigned long long >() const { return getUInt(); } /// NOLINT
template <> inline float Value::get<float >() const { return getDouble(); }
template <> inline double Value::get<double >() const { return getDouble(); }
template <> inline std::string Value::get<std::string >() const { return getString(); }