Merge pull request #61798 from ClickHouse/clang-19-2

Updated clang-19 to master.
This commit is contained in:
Alexey Milovidov 2024-03-23 18:42:15 +03:00 committed by GitHub
commit b2ebaa1d6c
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
32 changed files with 100 additions and 70 deletions

View File

@ -5,6 +5,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
struct Settings;
AggregateFunctionPtr AggregateFunctionCount::getOwnNullAdapter(
@ -19,7 +25,9 @@ namespace
AggregateFunctionPtr createAggregateFunctionCount(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
{
assertNoParameters(name, parameters);
assertArityAtMost<1>(name, argument_types);
if (argument_types.size() > 1)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires zero or one argument", name);
return std::make_shared<AggregateFunctionCount>(argument_types);
}

View File

@ -14,6 +14,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NOT_IMPLEMENTED;
}
@ -116,8 +117,8 @@ template <template <typename, bool> class Function>
AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
/// Second argument type check doesn't depend on the type of the first one.
Function<void, true>::assertSecondArg(argument_types);
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);

View File

@ -91,6 +91,21 @@ public:
if (!returns_many && levels.size() > 1)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires one level parameter or less", getName());
if constexpr (has_second_arg)
{
assertBinary(Name::name, argument_types_);
if (!isUInt(argument_types_[1]))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Second argument (weight) for function {} must be unsigned integer, but it has type {}",
Name::name,
argument_types_[1]->getName());
}
else
{
assertUnary(Name::name, argument_types_);
}
if constexpr (is_quantile_ddsketch)
{
if (params.empty())
@ -272,22 +287,6 @@ public:
static_cast<ColVecType &>(to).getData().push_back(data.get(level));
}
}
static void assertSecondArg(const DataTypes & types)
{
if constexpr (has_second_arg)
{
assertBinary(Name::name, types);
if (!isUInt(types[1]))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Second argument (weight) for function {} must be unsigned integer, but it has type {}",
Name::name,
types[1]->getName());
}
else
assertUnary(Name::name, types);
}
};
struct NameQuantile { static constexpr auto name = "quantile"; };

View File

@ -13,6 +13,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -26,8 +27,8 @@ template <template <typename, bool> class Function>
AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
/// Second argument type check doesn't depend on the type of the first one.
Function<void, true>::assertSecondArg(argument_types);
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);

View File

@ -1,7 +1,6 @@
#include <AggregateFunctions/AggregateFunctionQuantile.h>
#include <AggregateFunctions/QuantileBFloat16Histogram.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/Helpers.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <Core/Field.h>
@ -13,6 +12,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -26,8 +26,8 @@ template <template <typename, bool> class Function>
AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
/// Second argument type check doesn't depend on the type of the first one.
Function<void, true>::assertSecondArg(argument_types);
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);

View File

@ -13,6 +13,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -27,8 +28,8 @@ template <template <typename, bool> class Function>
AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
/// Second argument type check doesn't depend on the type of the first one.
Function<void, true>::assertSecondArg(argument_types);
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);

View File

@ -14,6 +14,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NOT_IMPLEMENTED;
}
@ -116,8 +117,8 @@ template <template <typename, bool> class Function>
AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
/// Second argument type check doesn't depend on the type of the first one.
Function<void, true>::assertSecondArg(argument_types);
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);

View File

@ -13,6 +13,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -27,8 +28,8 @@ template <template <typename, bool> class Function>
AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
/// Second argument type check doesn't depend on the type of the first one.
Function<void, true>::assertSecondArg(argument_types);
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);

View File

@ -13,6 +13,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -26,8 +27,8 @@ template <template <typename, bool> class Function>
AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
/// Second argument type check doesn't depend on the type of the first one.
Function<void, true>::assertSecondArg(argument_types);
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);

View File

@ -13,6 +13,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -27,8 +28,8 @@ template <template <typename, bool> class Function>
AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
/// Second argument type check doesn't depend on the type of the first one.
Function<void, true>::assertSecondArg(argument_types);
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);
@ -39,12 +40,12 @@ AggregateFunctionPtr createAggregateFunctionQuantile(
#undef DISPATCH
if (which.idx == TypeIndex::Date) return std::make_shared<Function<DataTypeDate::FieldType, false>>(argument_types, params);
if (which.idx == TypeIndex::DateTime) return std::make_shared<Function<DataTypeDateTime::FieldType, false>>(argument_types, params);
if (which.idx == TypeIndex::DateTime64) return std::make_shared<Function<DateTime64, false>>(argument_types, params);
if (which.idx == TypeIndex::Decimal32) return std::make_shared<Function<Decimal32, false>>(argument_types, params);
if (which.idx == TypeIndex::Decimal64) return std::make_shared<Function<Decimal64, false>>(argument_types, params);
if (which.idx == TypeIndex::Decimal128) return std::make_shared<Function<Decimal128, false>>(argument_types, params);
if (which.idx == TypeIndex::Decimal256) return std::make_shared<Function<Decimal256, false>>(argument_types, params);
if (which.idx == TypeIndex::DateTime64) return std::make_shared<Function<DateTime64, false>>(argument_types, params);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}",
argument_type->getName(), name);

View File

@ -13,6 +13,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -26,8 +27,8 @@ template <template <typename, bool> class Function>
AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
/// Second argument type check doesn't depend on the type of the first one.
Function<void, true>::assertSecondArg(argument_types);
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);

View File

@ -13,6 +13,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -26,8 +27,8 @@ template <template <typename, bool> class Function>
AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
/// Second argument type check doesn't depend on the type of the first one.
Function<void, true>::assertSecondArg(argument_types);
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);

View File

@ -16,6 +16,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NOT_IMPLEMENTED;
}
@ -216,8 +217,8 @@ template <template <typename, bool> class Function>
AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
/// Second argument type check doesn't depend on the type of the first one.
Function<void, true>::assertSecondArg(argument_types);
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);

View File

@ -16,6 +16,7 @@ namespace DB
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int INCORRECT_DATA;
extern const int LOGICAL_ERROR;
@ -503,8 +504,8 @@ template <template <typename, bool> class Function>
AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
/// Second argument type check doesn't depend on the type of the first one.
Function<void, true>::assertSecondArg(argument_types);
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);

View File

@ -15,6 +15,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NOT_IMPLEMENTED;
}
@ -320,8 +321,8 @@ template <template <typename, bool> class Function>
AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
/// Second argument type check doesn't depend on the type of the first one.
Function<void, true>::assertSecondArg(argument_types);
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);

View File

@ -13,6 +13,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -26,8 +27,8 @@ template <template <typename, bool> class Function>
AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
/// Second argument type check doesn't depend on the type of the first one.
Function<void, true>::assertSecondArg(argument_types);
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);

View File

@ -13,6 +13,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -26,8 +27,8 @@ template <template <typename, bool> class Function>
AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
/// Second argument type check doesn't depend on the type of the first one.
Function<void, true>::assertSecondArg(argument_types);
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);

View File

@ -13,6 +13,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -26,8 +27,8 @@ template <template <typename, bool> class Function>
AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
/// Second argument type check doesn't depend on the type of the first one.
Function<void, true>::assertSecondArg(argument_types);
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);

View File

@ -13,6 +13,7 @@ struct Settings;
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -26,8 +27,8 @@ template <template <typename, bool> class Function>
AggregateFunctionPtr createAggregateFunctionQuantile(
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
{
/// Second argument type check doesn't depend on the type of the first one.
Function<void, true>::assertSecondArg(argument_types);
if (argument_types.empty())
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
const DataTypePtr & argument_type = argument_types[0];
WhichDataType which(argument_type);

View File

@ -33,21 +33,4 @@ inline void assertBinary(const std::string & name, const DataTypes & argument_ty
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires two arguments", name);
}
template<std::size_t maximal_arity>
inline void assertArityAtMost(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() <= maximal_arity)
return;
if constexpr (maximal_arity == 0)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} cannot have arguments", name);
if constexpr (maximal_arity == 1)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires zero or one argument",
name);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at most {} arguments",
name, maximal_arity);
}
}

View File

@ -122,6 +122,8 @@ FutureSetFromSubquery::FutureSetFromSubquery(
set_and_key->set = std::make_shared<Set>(size_limits, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in);
}
FutureSetFromSubquery::~FutureSetFromSubquery() = default;
SetPtr FutureSetFromSubquery::get() const
{
if (set_and_key->set != nullptr && set_and_key->set->isCreated())

View File

@ -108,6 +108,8 @@ public:
QueryTreeNodePtr query_tree_,
const Settings & settings);
~FutureSetFromSubquery() override;
SetPtr get() const override;
DataTypes getTypes() const override;
SetPtr buildOrderedSetInplace(const ContextPtr & context) override;

View File

@ -192,6 +192,8 @@ MySQLHandler::MySQLHandler(
settings_replacements.emplace("NET_READ_TIMEOUT", "receive_timeout");
}
MySQLHandler::~MySQLHandler() = default;
void MySQLHandler::run()
{
setThreadName("MySQLHandler");

View File

@ -46,6 +46,8 @@ public:
const ProfileEvents::Event & read_event_ = ProfileEvents::end(),
const ProfileEvents::Event & write_event_ = ProfileEvents::end());
~MySQLHandler() override;
void run() final;
protected:

View File

@ -624,6 +624,8 @@ HDFSSource::HDFSSource(
initialize();
}
HDFSSource::~HDFSSource() = default;
bool HDFSSource::initialize()
{
bool skip_empty_files = getContext()->getSettingsRef().hdfs_skip_empty_files;

View File

@ -153,6 +153,8 @@ public:
std::shared_ptr<IteratorWrapper> file_iterator_,
bool need_only_count_);
~HDFSSource() override;
String getName() const override;
Chunk generate() override;

View File

@ -386,6 +386,8 @@ StorageKafka::StorageKafka(
});
}
StorageKafka::~StorageKafka() = default;
VirtualColumnsDescription StorageKafka::createVirtuals(StreamingHandleErrorMode handle_error_mode)
{
VirtualColumnsDescription desc;

View File

@ -42,6 +42,8 @@ public:
std::unique_ptr<KafkaSettings> kafka_settings_,
const String & collection_name_);
~StorageKafka() override;
std::string getName() const override { return "Kafka"; }
bool noPushingToViews() const override { return true; }

View File

@ -197,6 +197,8 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_,
initDB();
}
StorageEmbeddedRocksDB::~StorageEmbeddedRocksDB() = default;
void StorageEmbeddedRocksDB::truncate(const ASTPtr &, const StorageMetadataPtr & , ContextPtr, TableExclusiveLockHolder &)
{
std::lock_guard lock(rocksdb_ptr_mx);

View File

@ -39,6 +39,8 @@ public:
String rocksdb_dir_ = "",
bool read_only_ = false);
~StorageEmbeddedRocksDB() override;
std::string getName() const override { return "EmbeddedRocksDB"; }
void read(

View File

@ -387,6 +387,8 @@ StorageURLSource::StorageURLSource(
};
}
StorageURLSource::~StorageURLSource() = default;
Chunk StorageURLSource::generate()
{
while (true)

View File

@ -181,6 +181,8 @@ public:
bool glob_url = false,
bool need_only_count_ = false);
~StorageURLSource() override;
String getName() const override { return name; }
void setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) override