mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Merge branch 'clang-19-2' into fix-bad-numbers-pretty-printing
This commit is contained in:
commit
2758bb3883
@ -19,7 +19,9 @@ namespace
|
|||||||
AggregateFunctionPtr createAggregateFunctionCount(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
AggregateFunctionPtr createAggregateFunctionCount(const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *)
|
||||||
{
|
{
|
||||||
assertNoParameters(name, parameters);
|
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);
|
return std::make_shared<AggregateFunctionCount>(argument_types);
|
||||||
}
|
}
|
||||||
|
@ -116,8 +116,8 @@ template <template <typename, bool> class Function>
|
|||||||
AggregateFunctionPtr createAggregateFunctionQuantile(
|
AggregateFunctionPtr createAggregateFunctionQuantile(
|
||||||
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
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.
|
if (argument_types.empty())
|
||||||
Function<void, true>::assertSecondArg(argument_types);
|
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name);
|
||||||
|
|
||||||
const DataTypePtr & argument_type = argument_types[0];
|
const DataTypePtr & argument_type = argument_types[0];
|
||||||
WhichDataType which(argument_type);
|
WhichDataType which(argument_type);
|
||||||
|
@ -91,6 +91,21 @@ public:
|
|||||||
if (!returns_many && levels.size() > 1)
|
if (!returns_many && levels.size() > 1)
|
||||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires one level parameter or less", getName());
|
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 constexpr (is_quantile_ddsketch)
|
||||||
{
|
{
|
||||||
if (params.empty())
|
if (params.empty())
|
||||||
@ -272,22 +287,6 @@ public:
|
|||||||
static_cast<ColVecType &>(to).getData().push_back(data.get(level));
|
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"; };
|
struct NameQuantile { static constexpr auto name = "quantile"; };
|
||||||
|
@ -26,9 +26,6 @@ template <template <typename, bool> class Function>
|
|||||||
AggregateFunctionPtr createAggregateFunctionQuantile(
|
AggregateFunctionPtr createAggregateFunctionQuantile(
|
||||||
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
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);
|
|
||||||
|
|
||||||
const DataTypePtr & argument_type = argument_types[0];
|
const DataTypePtr & argument_type = argument_types[0];
|
||||||
WhichDataType which(argument_type);
|
WhichDataType which(argument_type);
|
||||||
|
|
||||||
|
@ -26,9 +26,6 @@ template <template <typename, bool> class Function>
|
|||||||
AggregateFunctionPtr createAggregateFunctionQuantile(
|
AggregateFunctionPtr createAggregateFunctionQuantile(
|
||||||
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
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);
|
|
||||||
|
|
||||||
const DataTypePtr & argument_type = argument_types[0];
|
const DataTypePtr & argument_type = argument_types[0];
|
||||||
WhichDataType which(argument_type);
|
WhichDataType which(argument_type);
|
||||||
|
|
||||||
|
@ -27,9 +27,6 @@ template <template <typename, bool> class Function>
|
|||||||
AggregateFunctionPtr createAggregateFunctionQuantile(
|
AggregateFunctionPtr createAggregateFunctionQuantile(
|
||||||
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
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);
|
|
||||||
|
|
||||||
const DataTypePtr & argument_type = argument_types[0];
|
const DataTypePtr & argument_type = argument_types[0];
|
||||||
WhichDataType which(argument_type);
|
WhichDataType which(argument_type);
|
||||||
|
|
||||||
|
@ -116,9 +116,6 @@ template <template <typename, bool> class Function>
|
|||||||
AggregateFunctionPtr createAggregateFunctionQuantile(
|
AggregateFunctionPtr createAggregateFunctionQuantile(
|
||||||
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
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);
|
|
||||||
|
|
||||||
const DataTypePtr & argument_type = argument_types[0];
|
const DataTypePtr & argument_type = argument_types[0];
|
||||||
WhichDataType which(argument_type);
|
WhichDataType which(argument_type);
|
||||||
|
|
||||||
|
@ -27,9 +27,6 @@ template <template <typename, bool> class Function>
|
|||||||
AggregateFunctionPtr createAggregateFunctionQuantile(
|
AggregateFunctionPtr createAggregateFunctionQuantile(
|
||||||
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
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);
|
|
||||||
|
|
||||||
const DataTypePtr & argument_type = argument_types[0];
|
const DataTypePtr & argument_type = argument_types[0];
|
||||||
WhichDataType which(argument_type);
|
WhichDataType which(argument_type);
|
||||||
|
|
||||||
|
@ -26,9 +26,6 @@ template <template <typename, bool> class Function>
|
|||||||
AggregateFunctionPtr createAggregateFunctionQuantile(
|
AggregateFunctionPtr createAggregateFunctionQuantile(
|
||||||
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
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);
|
|
||||||
|
|
||||||
const DataTypePtr & argument_type = argument_types[0];
|
const DataTypePtr & argument_type = argument_types[0];
|
||||||
WhichDataType which(argument_type);
|
WhichDataType which(argument_type);
|
||||||
|
|
||||||
|
@ -27,9 +27,6 @@ template <template <typename, bool> class Function>
|
|||||||
AggregateFunctionPtr createAggregateFunctionQuantile(
|
AggregateFunctionPtr createAggregateFunctionQuantile(
|
||||||
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
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);
|
|
||||||
|
|
||||||
const DataTypePtr & argument_type = argument_types[0];
|
const DataTypePtr & argument_type = argument_types[0];
|
||||||
WhichDataType which(argument_type);
|
WhichDataType which(argument_type);
|
||||||
|
|
||||||
@ -39,12 +36,12 @@ AggregateFunctionPtr createAggregateFunctionQuantile(
|
|||||||
#undef DISPATCH
|
#undef DISPATCH
|
||||||
if (which.idx == TypeIndex::Date) return std::make_shared<Function<DataTypeDate::FieldType, false>>(argument_types, params);
|
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::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::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::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::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::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 {}",
|
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}",
|
||||||
argument_type->getName(), name);
|
argument_type->getName(), name);
|
||||||
|
@ -26,9 +26,6 @@ template <template <typename, bool> class Function>
|
|||||||
AggregateFunctionPtr createAggregateFunctionQuantile(
|
AggregateFunctionPtr createAggregateFunctionQuantile(
|
||||||
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
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);
|
|
||||||
|
|
||||||
const DataTypePtr & argument_type = argument_types[0];
|
const DataTypePtr & argument_type = argument_types[0];
|
||||||
WhichDataType which(argument_type);
|
WhichDataType which(argument_type);
|
||||||
|
|
||||||
|
@ -26,9 +26,6 @@ template <template <typename, bool> class Function>
|
|||||||
AggregateFunctionPtr createAggregateFunctionQuantile(
|
AggregateFunctionPtr createAggregateFunctionQuantile(
|
||||||
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
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);
|
|
||||||
|
|
||||||
const DataTypePtr & argument_type = argument_types[0];
|
const DataTypePtr & argument_type = argument_types[0];
|
||||||
WhichDataType which(argument_type);
|
WhichDataType which(argument_type);
|
||||||
|
|
||||||
|
@ -216,9 +216,6 @@ template <template <typename, bool> class Function>
|
|||||||
AggregateFunctionPtr createAggregateFunctionQuantile(
|
AggregateFunctionPtr createAggregateFunctionQuantile(
|
||||||
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
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);
|
|
||||||
|
|
||||||
const DataTypePtr & argument_type = argument_types[0];
|
const DataTypePtr & argument_type = argument_types[0];
|
||||||
WhichDataType which(argument_type);
|
WhichDataType which(argument_type);
|
||||||
|
|
||||||
|
@ -503,9 +503,6 @@ template <template <typename, bool> class Function>
|
|||||||
AggregateFunctionPtr createAggregateFunctionQuantile(
|
AggregateFunctionPtr createAggregateFunctionQuantile(
|
||||||
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
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);
|
|
||||||
|
|
||||||
const DataTypePtr & argument_type = argument_types[0];
|
const DataTypePtr & argument_type = argument_types[0];
|
||||||
WhichDataType which(argument_type);
|
WhichDataType which(argument_type);
|
||||||
|
|
||||||
|
@ -320,9 +320,6 @@ template <template <typename, bool> class Function>
|
|||||||
AggregateFunctionPtr createAggregateFunctionQuantile(
|
AggregateFunctionPtr createAggregateFunctionQuantile(
|
||||||
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
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);
|
|
||||||
|
|
||||||
const DataTypePtr & argument_type = argument_types[0];
|
const DataTypePtr & argument_type = argument_types[0];
|
||||||
WhichDataType which(argument_type);
|
WhichDataType which(argument_type);
|
||||||
|
|
||||||
|
@ -26,9 +26,6 @@ template <template <typename, bool> class Function>
|
|||||||
AggregateFunctionPtr createAggregateFunctionQuantile(
|
AggregateFunctionPtr createAggregateFunctionQuantile(
|
||||||
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
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);
|
|
||||||
|
|
||||||
const DataTypePtr & argument_type = argument_types[0];
|
const DataTypePtr & argument_type = argument_types[0];
|
||||||
WhichDataType which(argument_type);
|
WhichDataType which(argument_type);
|
||||||
|
|
||||||
|
@ -26,9 +26,6 @@ template <template <typename, bool> class Function>
|
|||||||
AggregateFunctionPtr createAggregateFunctionQuantile(
|
AggregateFunctionPtr createAggregateFunctionQuantile(
|
||||||
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
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);
|
|
||||||
|
|
||||||
const DataTypePtr & argument_type = argument_types[0];
|
const DataTypePtr & argument_type = argument_types[0];
|
||||||
WhichDataType which(argument_type);
|
WhichDataType which(argument_type);
|
||||||
|
|
||||||
|
@ -26,9 +26,6 @@ template <template <typename, bool> class Function>
|
|||||||
AggregateFunctionPtr createAggregateFunctionQuantile(
|
AggregateFunctionPtr createAggregateFunctionQuantile(
|
||||||
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
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);
|
|
||||||
|
|
||||||
const DataTypePtr & argument_type = argument_types[0];
|
const DataTypePtr & argument_type = argument_types[0];
|
||||||
WhichDataType which(argument_type);
|
WhichDataType which(argument_type);
|
||||||
|
|
||||||
|
@ -26,9 +26,6 @@ template <template <typename, bool> class Function>
|
|||||||
AggregateFunctionPtr createAggregateFunctionQuantile(
|
AggregateFunctionPtr createAggregateFunctionQuantile(
|
||||||
const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *)
|
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);
|
|
||||||
|
|
||||||
const DataTypePtr & argument_type = argument_types[0];
|
const DataTypePtr & argument_type = argument_types[0];
|
||||||
WhichDataType which(argument_type);
|
WhichDataType which(argument_type);
|
||||||
|
|
||||||
|
@ -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);
|
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);
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -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);
|
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
|
SetPtr FutureSetFromSubquery::get() const
|
||||||
{
|
{
|
||||||
if (set_and_key->set != nullptr && set_and_key->set->isCreated())
|
if (set_and_key->set != nullptr && set_and_key->set->isCreated())
|
||||||
|
@ -108,6 +108,8 @@ public:
|
|||||||
QueryTreeNodePtr query_tree_,
|
QueryTreeNodePtr query_tree_,
|
||||||
const Settings & settings);
|
const Settings & settings);
|
||||||
|
|
||||||
|
~FutureSetFromSubquery() override;
|
||||||
|
|
||||||
SetPtr get() const override;
|
SetPtr get() const override;
|
||||||
DataTypes getTypes() const override;
|
DataTypes getTypes() const override;
|
||||||
SetPtr buildOrderedSetInplace(const ContextPtr & context) override;
|
SetPtr buildOrderedSetInplace(const ContextPtr & context) override;
|
||||||
|
@ -192,6 +192,8 @@ MySQLHandler::MySQLHandler(
|
|||||||
settings_replacements.emplace("NET_READ_TIMEOUT", "receive_timeout");
|
settings_replacements.emplace("NET_READ_TIMEOUT", "receive_timeout");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
MySQLHandler::~MySQLHandler() = default;
|
||||||
|
|
||||||
void MySQLHandler::run()
|
void MySQLHandler::run()
|
||||||
{
|
{
|
||||||
setThreadName("MySQLHandler");
|
setThreadName("MySQLHandler");
|
||||||
|
@ -46,6 +46,8 @@ public:
|
|||||||
const ProfileEvents::Event & read_event_ = ProfileEvents::end(),
|
const ProfileEvents::Event & read_event_ = ProfileEvents::end(),
|
||||||
const ProfileEvents::Event & write_event_ = ProfileEvents::end());
|
const ProfileEvents::Event & write_event_ = ProfileEvents::end());
|
||||||
|
|
||||||
|
~MySQLHandler() override;
|
||||||
|
|
||||||
void run() final;
|
void run() final;
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
|
@ -624,6 +624,8 @@ HDFSSource::HDFSSource(
|
|||||||
initialize();
|
initialize();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
HDFSSource::~HDFSSource() = default;
|
||||||
|
|
||||||
bool HDFSSource::initialize()
|
bool HDFSSource::initialize()
|
||||||
{
|
{
|
||||||
bool skip_empty_files = getContext()->getSettingsRef().hdfs_skip_empty_files;
|
bool skip_empty_files = getContext()->getSettingsRef().hdfs_skip_empty_files;
|
||||||
|
@ -153,6 +153,8 @@ public:
|
|||||||
std::shared_ptr<IteratorWrapper> file_iterator_,
|
std::shared_ptr<IteratorWrapper> file_iterator_,
|
||||||
bool need_only_count_);
|
bool need_only_count_);
|
||||||
|
|
||||||
|
~HDFSSource() override;
|
||||||
|
|
||||||
String getName() const override;
|
String getName() const override;
|
||||||
|
|
||||||
Chunk generate() override;
|
Chunk generate() override;
|
||||||
|
@ -386,6 +386,8 @@ StorageKafka::StorageKafka(
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
StorageKafka::~StorageKafka() = default;
|
||||||
|
|
||||||
VirtualColumnsDescription StorageKafka::createVirtuals(StreamingHandleErrorMode handle_error_mode)
|
VirtualColumnsDescription StorageKafka::createVirtuals(StreamingHandleErrorMode handle_error_mode)
|
||||||
{
|
{
|
||||||
VirtualColumnsDescription desc;
|
VirtualColumnsDescription desc;
|
||||||
|
@ -42,6 +42,8 @@ public:
|
|||||||
std::unique_ptr<KafkaSettings> kafka_settings_,
|
std::unique_ptr<KafkaSettings> kafka_settings_,
|
||||||
const String & collection_name_);
|
const String & collection_name_);
|
||||||
|
|
||||||
|
~StorageKafka() override;
|
||||||
|
|
||||||
std::string getName() const override { return "Kafka"; }
|
std::string getName() const override { return "Kafka"; }
|
||||||
|
|
||||||
bool noPushingToViews() const override { return true; }
|
bool noPushingToViews() const override { return true; }
|
||||||
|
@ -197,6 +197,8 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_,
|
|||||||
initDB();
|
initDB();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
StorageEmbeddedRocksDB::~StorageEmbeddedRocksDB() = default;
|
||||||
|
|
||||||
void StorageEmbeddedRocksDB::truncate(const ASTPtr &, const StorageMetadataPtr & , ContextPtr, TableExclusiveLockHolder &)
|
void StorageEmbeddedRocksDB::truncate(const ASTPtr &, const StorageMetadataPtr & , ContextPtr, TableExclusiveLockHolder &)
|
||||||
{
|
{
|
||||||
std::lock_guard lock(rocksdb_ptr_mx);
|
std::lock_guard lock(rocksdb_ptr_mx);
|
||||||
|
@ -39,6 +39,8 @@ public:
|
|||||||
String rocksdb_dir_ = "",
|
String rocksdb_dir_ = "",
|
||||||
bool read_only_ = false);
|
bool read_only_ = false);
|
||||||
|
|
||||||
|
~StorageEmbeddedRocksDB() override;
|
||||||
|
|
||||||
std::string getName() const override { return "EmbeddedRocksDB"; }
|
std::string getName() const override { return "EmbeddedRocksDB"; }
|
||||||
|
|
||||||
void read(
|
void read(
|
||||||
|
@ -387,6 +387,8 @@ StorageURLSource::StorageURLSource(
|
|||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
StorageURLSource::~StorageURLSource() = default;
|
||||||
|
|
||||||
Chunk StorageURLSource::generate()
|
Chunk StorageURLSource::generate()
|
||||||
{
|
{
|
||||||
while (true)
|
while (true)
|
||||||
|
@ -181,6 +181,8 @@ public:
|
|||||||
bool glob_url = false,
|
bool glob_url = false,
|
||||||
bool need_only_count_ = false);
|
bool need_only_count_ = false);
|
||||||
|
|
||||||
|
~StorageURLSource() override;
|
||||||
|
|
||||||
String getName() const override { return name; }
|
String getName() const override { return name; }
|
||||||
|
|
||||||
void setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) override
|
void setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) override
|
||||||
|
Loading…
Reference in New Issue
Block a user