Merge pull request #36320 from ClickHouse/fix-clang-tidy-14

Fix clang-tidy-14 (part 1)
This commit is contained in:
Alexey Milovidov 2022-04-18 07:02:10 +03:00 committed by GitHub
commit 36595e4206
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
29 changed files with 46 additions and 45 deletions

2
contrib/sysroot vendored

@ -1 +1 @@
Subproject commit bbcac834526d90d1e764164b861be426891d1743
Subproject commit e9fb375d0a1e5ebfd74c043f088f2342552103f8

View File

@ -327,7 +327,8 @@ void DiskAccessStorage::scheduleWriteLists(AccessEntityType type)
/// Create the 'need_rebuild_lists.mark' file.
/// This file will be used later to find out if writing lists is successful or not.
std::ofstream{getNeedRebuildListsMarkFilePath(directory_path)};
std::ofstream out{getNeedRebuildListsMarkFilePath(directory_path)};
out.close();
lists_writing_thread = ThreadFromGlobalPool{&DiskAccessStorage::listsWritingThreadFunc, this};
lists_writing_thread_is_waiting = true;

View File

@ -39,7 +39,7 @@ bool allowTypes(const DataTypePtr& left, const DataTypePtr& right) noexcept
}
template <class First, class ... TArgs>
static IAggregateFunction * create(const IDataType & second_type, TArgs && ... args)
IAggregateFunction * create(const IDataType & second_type, TArgs && ... args)
{
const WhichDataType which(second_type);
@ -51,7 +51,7 @@ static IAggregateFunction * create(const IDataType & second_type, TArgs && ... a
// Not using helper functions because there are no templates for binary decimal/numeric function.
template <class... TArgs>
static IAggregateFunction * create(const IDataType & first_type, const IDataType & second_type, TArgs && ... args)
IAggregateFunction * create(const IDataType & first_type, const IDataType & second_type, TArgs && ... args)
{
const WhichDataType which(first_type);

View File

@ -30,7 +30,7 @@ AggregateFunctionPtr createAggregateFunctionDeltaSum(
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
DataTypePtr data_type = arguments[0];
const DataTypePtr & data_type = arguments[0];
if (isInteger(data_type) || isFloat(data_type))
return AggregateFunctionPtr(createWithNumericType<AggregationFunctionDeltaSum>(

View File

@ -20,7 +20,7 @@ namespace
{
template <template <typename, typename> class AggregateFunctionTemplate, typename Data, typename ... TArgs>
static IAggregateFunction * createWithNumericOrTimeType(const IDataType & argument_type, TArgs && ... args)
IAggregateFunction * createWithNumericOrTimeType(const IDataType & argument_type, TArgs && ... args)
{
WhichDataType which(argument_type);
if (which.idx == TypeIndex::Date) return new AggregateFunctionTemplate<UInt16, Data>(std::forward<TArgs>(args)...);

View File

@ -19,7 +19,7 @@ namespace ErrorCodes
namespace
{
template <template <typename, typename> class AggregateFunctionTemplate, template <typename> typename Data, typename... TArgs>
static IAggregateFunction * createWithIntegerType(const IDataType & argument_type, TArgs &&... args)
IAggregateFunction * createWithIntegerType(const IDataType & argument_type, TArgs &&... args)
{
WhichDataType which(argument_type);
if (which.idx == TypeIndex::UInt8) return new AggregateFunctionTemplate<UInt8, Data<UInt8>>(std::forward<TArgs>(args)...);

View File

@ -40,7 +40,7 @@ public:
};
template <typename HasLimit, typename ... TArgs>
static IAggregateFunction * createWithExtraTypes(const DataTypePtr & argument_type, TArgs && ... args)
IAggregateFunction * createWithExtraTypes(const DataTypePtr & argument_type, TArgs && ... args)
{
WhichDataType which(argument_type);
if (which.idx == TypeIndex::Date) return new AggregateFunctionGroupUniqArrayDate<HasLimit>(argument_type, std::forward<TArgs>(args)...);

View File

@ -60,7 +60,7 @@ template <typename Value, bool float_return> using FuncQuantileBFloat16Weighted
template <typename Value, bool float_return> using FuncQuantilesBFloat16Weighted = AggregateFunctionQuantile<Value, QuantileBFloat16Histogram<Value>, NameQuantilesBFloat16Weighted, true, std::conditional_t<float_return, Float64, void>, true>;
template <template <typename, bool> class Function>
static constexpr bool supportDecimal()
constexpr bool supportDecimal()
{
return std::is_same_v<Function<Float32, false>, FuncQuantile<Float32, false>> ||
std::is_same_v<Function<Float32, false>, FuncQuantiles<Float32, false>> ||
@ -75,7 +75,7 @@ static constexpr bool supportDecimal()
}
template <template <typename, bool> class Function>
static constexpr bool supportBigInt()
constexpr bool supportBigInt()
{
return std::is_same_v<Function<Float32, false>, FuncQuantile<Float32, false>> ||
std::is_same_v<Function<Float32, false>, FuncQuantiles<Float32, false>> ||

View File

@ -19,7 +19,7 @@ namespace
{
template <template <typename, typename> class AggregateFunctionTemplate, typename Data, typename ... TArgs>
static IAggregateFunction * createWithUIntegerOrTimeType(const std::string & name, const IDataType & argument_type, TArgs && ... args)
IAggregateFunction * createWithUIntegerOrTimeType(const std::string & name, const IDataType & argument_type, TArgs && ... args)
{
WhichDataType which(argument_type);
if (which.idx == TypeIndex::Date || which.idx == TypeIndex::UInt16) return new AggregateFunctionTemplate<UInt16, Data>(std::forward<TArgs>(args)...);

View File

@ -24,7 +24,7 @@ AggregateFunctionPtr createAggregateFunctionStatisticsUnary(
assertUnary(name, argument_types);
AggregateFunctionPtr res;
DataTypePtr data_type = argument_types[0];
const DataTypePtr & data_type = argument_types[0];
if (isDecimal(data_type))
res.reset(createWithDecimalType<FunctionTemplate>(*data_type, *data_type, argument_types));
else

View File

@ -56,7 +56,7 @@ AggregateFunctionPtr createAggregateFunctionSum(const std::string & name, const
assertUnary(name, argument_types);
AggregateFunctionPtr res;
DataTypePtr data_type = argument_types[0];
const DataTypePtr & data_type = argument_types[0];
if (isDecimal(data_type))
res.reset(createWithDecimalType<Function>(*data_type, *data_type, argument_types));
else

View File

@ -29,7 +29,7 @@ createAggregateFunctionSumCount(const std::string & name, const DataTypes & argu
assertUnary(name, argument_types);
AggregateFunctionPtr res;
DataTypePtr data_type = argument_types[0];
const DataTypePtr & data_type = argument_types[0];
if (!allowType(data_type))
throw Exception("Illegal type " + data_type->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

View File

@ -44,7 +44,7 @@ class AggregateFunctionTopKDateTime : public AggregateFunctionTopK<DataTypeDateT
template <bool is_weighted>
static IAggregateFunction * createWithExtraTypes(const DataTypes & argument_types, UInt64 threshold, UInt64 load_factor, const Array & params)
IAggregateFunction * createWithExtraTypes(const DataTypes & argument_types, UInt64 threshold, UInt64 load_factor, const Array & params)
{
if (argument_types.empty())
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Got empty arguments list");

View File

@ -100,7 +100,7 @@ enum ComputeWidthMode
};
template <ComputeWidthMode mode>
static size_t computeWidthImpl(const UInt8 * data, size_t size, size_t prefix, size_t limit) noexcept
size_t computeWidthImpl(const UInt8 * data, size_t size, size_t prefix, size_t limit) noexcept
{
UTF8Decoder decoder;
size_t width = 0;

View File

@ -288,8 +288,8 @@ ColumnUInt8::Ptr IPAddressDictionary::hasKeys(const Columns & key_columns, const
{
validateKeyTypes(key_types);
const auto first_column = key_columns.front();
const auto rows = first_column->size();
const auto & first_column = key_columns.front();
const size_t rows = first_column->size();
auto result = ColumnUInt8::create(rows);
auto & out = result->getData();
@ -613,8 +613,8 @@ void IPAddressDictionary::getItemsByTwoKeyColumnsImpl(
ValueSetter && set_value,
DefaultValueExtractor & default_value_extractor) const
{
const auto first_column = key_columns.front();
const auto rows = first_column->size();
const auto & first_column = key_columns.front();
const size_t rows = first_column->size();
auto & vec = std::get<ContainerType<AttributeType>>(attribute.maps);
if (const auto * ipv4_col = std::get_if<IPv4Container>(&ip_column))
@ -694,8 +694,8 @@ void IPAddressDictionary::getItemsImpl(
ValueSetter && set_value,
DefaultValueExtractor & default_value_extractor) const
{
const auto first_column = key_columns.front();
const auto rows = first_column->size();
const auto & first_column = key_columns.front();
const size_t rows = first_column->size();
// special case for getBlockInputStream
if (unlikely(key_columns.size() == 2))

View File

@ -128,7 +128,7 @@ void validateArgumentsImpl(const IFunction & func,
}
const auto & arg = arguments[i + argument_offset];
const auto descriptor = descriptors[i];
const auto & descriptor = descriptors[i];
if (int error_code = descriptor.isValid(arg.type, arg.column); error_code != 0)
throw Exception("Illegal type of argument #" + std::to_string(argument_offset + i + 1) // +1 is for human-friendly 1-based indexing
+ (descriptor.argument_name ? " '" + std::string(descriptor.argument_name) + "'" : String{})

View File

@ -95,7 +95,7 @@ void convertAnyColumnToBool(const IColumn * column, UInt8Container & res)
template <class Op, typename Func>
static bool extractConstColumns(ColumnRawPtrs & in, UInt8 & res, Func && func)
bool extractConstColumns(ColumnRawPtrs & in, UInt8 & res, Func && func)
{
bool has_res = false;
@ -345,7 +345,7 @@ struct OperationApplier<Op, OperationApplierImpl, 0>
template <class Op>
static ColumnPtr executeForTernaryLogicImpl(ColumnRawPtrs arguments, const DataTypePtr & result_type, size_t input_rows_count)
ColumnPtr executeForTernaryLogicImpl(ColumnRawPtrs arguments, const DataTypePtr & result_type, size_t input_rows_count)
{
/// Combine all constant columns into a single constant value.
UInt8 const_3v_value = 0;
@ -420,7 +420,7 @@ struct TypedExecutorInvoker<Op>
/// Types of all of the arguments are guaranteed to be non-nullable here
template <class Op>
static ColumnPtr basicExecuteImpl(ColumnRawPtrs arguments, size_t input_rows_count)
ColumnPtr basicExecuteImpl(ColumnRawPtrs arguments, size_t input_rows_count)
{
/// Combine all constant columns into a single constant value.
UInt8 const_val = 0;

View File

@ -146,7 +146,7 @@ public:
void update(size_t from)
{
sink_null_map[index] = bool(src_null_map && src_null_map[from]);
sink_null_map[index] = src_null_map && src_null_map[from];
++index;
}

View File

@ -45,7 +45,7 @@ using namespace GatherUtils;
*/
template <typename ArrayCond, typename ArrayA, typename ArrayB, typename ArrayResult, typename ResultType>
static inline void fillVectorVector(const ArrayCond & cond, const ArrayA & a, const ArrayB & b, ArrayResult & res)
inline void fillVectorVector(const ArrayCond & cond, const ArrayA & a, const ArrayB & b, ArrayResult & res)
{
size_t size = cond.size();
bool a_is_short = a.size() < size;
@ -77,7 +77,7 @@ static inline void fillVectorVector(const ArrayCond & cond, const ArrayA & a, co
}
template <typename ArrayCond, typename ArrayA, typename B, typename ArrayResult, typename ResultType>
static inline void fillVectorConstant(const ArrayCond & cond, const ArrayA & a, B b, ArrayResult & res)
inline void fillVectorConstant(const ArrayCond & cond, const ArrayA & a, B b, ArrayResult & res)
{
size_t size = cond.size();
bool a_is_short = a.size() < size;
@ -95,7 +95,7 @@ static inline void fillVectorConstant(const ArrayCond & cond, const ArrayA & a,
}
template <typename ArrayCond, typename A, typename ArrayB, typename ArrayResult, typename ResultType>
static inline void fillConstantVector(const ArrayCond & cond, A a, const ArrayB & b, ArrayResult & res)
inline void fillConstantVector(const ArrayCond & cond, A a, const ArrayB & b, ArrayResult & res)
{
size_t size = cond.size();
bool b_is_short = b.size() < size;

View File

@ -35,7 +35,6 @@ public:
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & res_type, size_t /*input_rows_count*/) const override
{
auto arg_num = arguments[0];
const auto & arg = arguments[0];
if (arg.type->lowCardinality())

View File

@ -201,7 +201,7 @@ static void compileFunction(llvm::Module & module, const IFunctionBase & functio
for (size_t i = 0; i < arg_types.size(); ++i)
{
auto & column = columns[i];
auto type = arg_types[i];
const auto & type = arg_types[i];
auto * value = b.CreateLoad(toNativeType(b, removeNullable(type)), column.data);
if (!type->isNullable())

View File

@ -52,7 +52,7 @@ namespace
{
template <typename From, typename To>
static Field convertNumericTypeImpl(const Field & from)
Field convertNumericTypeImpl(const Field & from)
{
To result;
if (!accurate::convertNumeric(from.get<From>(), result))
@ -61,7 +61,7 @@ static Field convertNumericTypeImpl(const Field & from)
}
template <typename To>
static Field convertNumericType(const Field & from, const IDataType & type)
Field convertNumericType(const Field & from, const IDataType & type)
{
if (from.getType() == Field::Types::UInt64 || from.getType() == Field::Types::Bool)
return convertNumericTypeImpl<UInt64, To>(from);
@ -84,7 +84,7 @@ static Field convertNumericType(const Field & from, const IDataType & type)
template <typename From, typename T>
static Field convertIntToDecimalType(const Field & from, const DataTypeDecimal<T> & type)
Field convertIntToDecimalType(const Field & from, const DataTypeDecimal<T> & type)
{
From value = from.get<From>();
if (!type.canStoreWhole(value))
@ -96,7 +96,7 @@ static Field convertIntToDecimalType(const Field & from, const DataTypeDecimal<T
template <typename T>
static Field convertStringToDecimalType(const Field & from, const DataTypeDecimal<T> & type)
Field convertStringToDecimalType(const Field & from, const DataTypeDecimal<T> & type)
{
const String & str_value = from.get<String>();
T value = type.parseFromString(str_value);
@ -104,7 +104,7 @@ static Field convertStringToDecimalType(const Field & from, const DataTypeDecima
}
template <typename From, typename T>
static Field convertDecimalToDecimalType(const Field & from, const DataTypeDecimal<T> & type)
Field convertDecimalToDecimalType(const Field & from, const DataTypeDecimal<T> & type)
{
auto field = from.get<DecimalField<From>>();
T value = convertDecimals<DataTypeDecimal<From>, DataTypeDecimal<T>>(field.getValue(), field.getScale(), type.getScale());
@ -112,7 +112,7 @@ static Field convertDecimalToDecimalType(const Field & from, const DataTypeDecim
}
template <typename To>
static Field convertDecimalType(const Field & from, const To & type)
Field convertDecimalType(const Field & from, const To & type)
{
if (from.getType() == Field::Types::UInt64)
return convertIntToDecimalType<UInt64>(from, type);

View File

@ -1,6 +1,7 @@
#include <Parsers/Access/ASTShowAccessEntitiesQuery.h>
#include <Common/quoteString.h>
#include <IO/Operators.h>
#include <fmt/format.h>
namespace DB
@ -20,7 +21,7 @@ String ASTShowAccessEntitiesQuery::getKeyword() const
String ASTShowAccessEntitiesQuery::getID(char) const
{
return "SHOW " + String(getKeyword()) + " query";
return fmt::format("SHOW {} query", getKeyword());
}
void ASTShowAccessEntitiesQuery::formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const

View File

@ -763,7 +763,7 @@ void registerStorageFileLog(StorageFactory & factory)
if (!num_threads) /// Default
{
num_threads = std::max(unsigned(1), physical_cpu_cores / 4);
num_threads = std::max(1U, physical_cpu_cores / 4);
filelog_settings->set("max_threads", num_threads);
}
else if (num_threads > physical_cpu_cores)

View File

@ -987,7 +987,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
}
};
size_t num_threads = std::min(size_t(num_streams), parts.size());
size_t num_threads = std::min<size_t>(num_streams, parts.size());
if (num_threads <= 1)
{

View File

@ -180,7 +180,7 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(
{
Row partition(partition_columns.size());
for (size_t i = 0; i < partition_columns.size(); ++i)
partition[i] = Field((*partition_columns[i])[partition_num_to_first_row[num]]);
partition[i] = (*partition_columns[i])[partition_num_to_first_row[num]];
return partition;
};

View File

@ -103,7 +103,7 @@ MergeTreeIndexPtr bloomFilterIndexCreatorNew(
if (!index.arguments.empty())
{
const auto & argument = index.arguments[0];
max_conflict_probability = std::min(Float64(1), std::max(argument.safeGet<Float64>(), Float64(0)));
max_conflict_probability = std::min<Float64>(1.0, std::max<Float64>(argument.safeGet<Float64>(), 0.0));
}
const auto & bits_per_row_and_size_of_hash_functions = BloomFilterHash::calculationBestPractices(max_conflict_probability);

View File

@ -22,7 +22,7 @@ static void assertGranuleBlocksStructure(const Blocks & granule_index_blocks)
Block prev_block;
for (size_t index = 0; index < granule_index_blocks.size(); ++index)
{
Block granule_index_block = granule_index_blocks[index];
const Block & granule_index_block = granule_index_blocks[index];
if (index != 0)
assertBlocksHaveEqualStructure(prev_block, granule_index_block, "Granule blocks of bloom filter has difference structure.");

View File

@ -464,7 +464,7 @@ Pipe StorageEmbeddedRocksDB::read(
Pipes pipes;
size_t num_keys = keys->size();
size_t num_threads = std::min(size_t(num_streams), keys->size());
size_t num_threads = std::min<size_t>(num_streams, keys->size());
assert(num_keys <= std::numeric_limits<uint32_t>::max());
assert(num_threads <= std::numeric_limits<uint32_t>::max());