Merge pull request #72556 from ClickHouse/devirtualize_serialize_call

Devirtualize serialize call in AggregateFunction
This commit is contained in:
alesapin 2024-12-01 21:18:38 +00:00 committed by GitHub
commit 4f596388a2
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
12 changed files with 43 additions and 25 deletions

View File

@ -192,7 +192,7 @@ struct AggregateFunctionGroupArrayIntersectGenericData
* For such columns GroupArrayIntersect() can be implemented more efficiently (especially for small numeric arrays). * For such columns GroupArrayIntersect() can be implemented more efficiently (especially for small numeric arrays).
*/ */
template <bool is_plain_column = false> template <bool is_plain_column = false>
class AggregateFunctionGroupArrayIntersectGeneric class AggregateFunctionGroupArrayIntersectGeneric final
: public IAggregateFunctionDataHelper<AggregateFunctionGroupArrayIntersectGenericData, : public IAggregateFunctionDataHelper<AggregateFunctionGroupArrayIntersectGenericData,
AggregateFunctionGroupArrayIntersectGeneric<is_plain_column>> AggregateFunctionGroupArrayIntersectGeneric<is_plain_column>>
{ {
@ -353,7 +353,7 @@ namespace
{ {
/// Substitute return type for Date and DateTime /// Substitute return type for Date and DateTime
class AggregateFunctionGroupArrayIntersectDate : public AggregateFunctionGroupArrayIntersect<DataTypeDate::FieldType> class AggregateFunctionGroupArrayIntersectDate final : public AggregateFunctionGroupArrayIntersect<DataTypeDate::FieldType>
{ {
public: public:
explicit AggregateFunctionGroupArrayIntersectDate(const DataTypePtr & argument_type, const Array & parameters_) explicit AggregateFunctionGroupArrayIntersectDate(const DataTypePtr & argument_type, const Array & parameters_)
@ -361,7 +361,7 @@ public:
static DataTypePtr createResultType() { return std::make_shared<DataTypeArray>(std::make_shared<DataTypeDate>()); } static DataTypePtr createResultType() { return std::make_shared<DataTypeArray>(std::make_shared<DataTypeDate>()); }
}; };
class AggregateFunctionGroupArrayIntersectDateTime : public AggregateFunctionGroupArrayIntersect<DataTypeDateTime::FieldType> class AggregateFunctionGroupArrayIntersectDateTime final : public AggregateFunctionGroupArrayIntersect<DataTypeDateTime::FieldType>
{ {
public: public:
explicit AggregateFunctionGroupArrayIntersectDateTime(const DataTypePtr & argument_type, const Array & parameters_) explicit AggregateFunctionGroupArrayIntersectDateTime(const DataTypePtr & argument_type, const Array & parameters_)
@ -369,7 +369,7 @@ public:
static DataTypePtr createResultType() { return std::make_shared<DataTypeArray>(std::make_shared<DataTypeDateTime>()); } static DataTypePtr createResultType() { return std::make_shared<DataTypeArray>(std::make_shared<DataTypeDateTime>()); }
}; };
class AggregateFunctionGroupArrayIntersectDate32 : public AggregateFunctionGroupArrayIntersect<DataTypeDate32::FieldType> class AggregateFunctionGroupArrayIntersectDate32 final : public AggregateFunctionGroupArrayIntersect<DataTypeDate32::FieldType>
{ {
public: public:
explicit AggregateFunctionGroupArrayIntersectDate32(const DataTypePtr & argument_type, const Array & parameters_) explicit AggregateFunctionGroupArrayIntersectDate32(const DataTypePtr & argument_type, const Array & parameters_)

View File

@ -153,7 +153,7 @@ static void deserializeAndInsertImpl(StringRef str, IColumn & data_to);
* For such columns groupUniqArray() can be implemented more efficiently (especially for small numeric arrays). * For such columns groupUniqArray() can be implemented more efficiently (especially for small numeric arrays).
*/ */
template <bool is_plain_column = false, typename LimitNumElems = std::false_type> template <bool is_plain_column = false, typename LimitNumElems = std::false_type>
class AggregateFunctionGroupUniqArrayGeneric class AggregateFunctionGroupUniqArrayGeneric final
: public IAggregateFunctionDataHelper<AggregateFunctionGroupUniqArrayGenericData, : public IAggregateFunctionDataHelper<AggregateFunctionGroupUniqArrayGenericData,
AggregateFunctionGroupUniqArrayGeneric<is_plain_column, LimitNumElems>> AggregateFunctionGroupUniqArrayGeneric<is_plain_column, LimitNumElems>>
{ {
@ -245,7 +245,7 @@ public:
/// Substitute return type for Date and DateTime /// Substitute return type for Date and DateTime
template <typename HasLimit> template <typename HasLimit>
class AggregateFunctionGroupUniqArrayDate : public AggregateFunctionGroupUniqArray<DataTypeDate::FieldType, HasLimit> class AggregateFunctionGroupUniqArrayDate final : public AggregateFunctionGroupUniqArray<DataTypeDate::FieldType, HasLimit>
{ {
public: public:
explicit AggregateFunctionGroupUniqArrayDate(const DataTypePtr & argument_type, const Array & parameters_, UInt64 max_elems_ = std::numeric_limits<UInt64>::max()) explicit AggregateFunctionGroupUniqArrayDate(const DataTypePtr & argument_type, const Array & parameters_, UInt64 max_elems_ = std::numeric_limits<UInt64>::max())
@ -254,7 +254,7 @@ public:
}; };
template <typename HasLimit> template <typename HasLimit>
class AggregateFunctionGroupUniqArrayDateTime : public AggregateFunctionGroupUniqArray<DataTypeDateTime::FieldType, HasLimit> class AggregateFunctionGroupUniqArrayDateTime final : public AggregateFunctionGroupUniqArray<DataTypeDateTime::FieldType, HasLimit>
{ {
public: public:
explicit AggregateFunctionGroupUniqArrayDateTime(const DataTypePtr & argument_type, const Array & parameters_, UInt64 max_elems_ = std::numeric_limits<UInt64>::max()) explicit AggregateFunctionGroupUniqArrayDateTime(const DataTypePtr & argument_type, const Array & parameters_, UInt64 max_elems_ = std::numeric_limits<UInt64>::max())
@ -263,7 +263,7 @@ public:
}; };
template <typename HasLimit> template <typename HasLimit>
class AggregateFunctionGroupUniqArrayIPv4 : public AggregateFunctionGroupUniqArray<DataTypeIPv4::FieldType, HasLimit> class AggregateFunctionGroupUniqArrayIPv4 final : public AggregateFunctionGroupUniqArray<DataTypeIPv4::FieldType, HasLimit>
{ {
public: public:
explicit AggregateFunctionGroupUniqArrayIPv4(const DataTypePtr & argument_type, const Array & parameters_, UInt64 max_elems_ = std::numeric_limits<UInt64>::max()) explicit AggregateFunctionGroupUniqArrayIPv4(const DataTypePtr & argument_type, const Array & parameters_, UInt64 max_elems_ = std::numeric_limits<UInt64>::max())

View File

@ -28,7 +28,7 @@ namespace
/// Returns tuple of (z-statistic, p-value, confidence-interval-low, confidence-interval-high) /// Returns tuple of (z-statistic, p-value, confidence-interval-low, confidence-interval-high)
template <typename Data> template <typename Data>
class AggregateFunctionMeanZTest : class AggregateFunctionMeanZTest final:
public IAggregateFunctionDataHelper<Data, AggregateFunctionMeanZTest<Data>> public IAggregateFunctionDataHelper<Data, AggregateFunctionMeanZTest<Data>>
{ {
private: private:

View File

@ -47,7 +47,7 @@ struct RankCorrelationData : public StatisticalSample<Float64, Float64>
} }
}; };
class AggregateFunctionRankCorrelation : class AggregateFunctionRankCorrelation final :
public IAggregateFunctionDataHelper<RankCorrelationData, AggregateFunctionRankCorrelation> public IAggregateFunctionDataHelper<RankCorrelationData, AggregateFunctionRankCorrelation>
{ {
public: public:

View File

@ -38,7 +38,7 @@ namespace ErrorCodes
/// Returns tuple of (t-statistic, p-value) /// Returns tuple of (t-statistic, p-value)
/// https://cpb-us-w2.wpmucdn.com/voices.uchicago.edu/dist/9/1193/files/2016/01/05b-TandP.pdf /// https://cpb-us-w2.wpmucdn.com/voices.uchicago.edu/dist/9/1193/files/2016/01/05b-TandP.pdf
template <typename Data> template <typename Data>
class AggregateFunctionTTest : class AggregateFunctionTTest final:
public IAggregateFunctionDataHelper<Data, AggregateFunctionTTest<Data>> public IAggregateFunctionDataHelper<Data, AggregateFunctionTTest<Data>>
{ {
private: private:

View File

@ -203,7 +203,7 @@ struct AggregateFunctionTopKGenericData
* For such columns topK() can be implemented more efficiently (especially for small numeric arrays). * For such columns topK() can be implemented more efficiently (especially for small numeric arrays).
*/ */
template <bool is_plain_column, bool is_weighted> template <bool is_plain_column, bool is_weighted>
class AggregateFunctionTopKGeneric class AggregateFunctionTopKGeneric final
: public IAggregateFunctionDataHelper<AggregateFunctionTopKGenericData, AggregateFunctionTopKGeneric<is_plain_column, is_weighted>> : public IAggregateFunctionDataHelper<AggregateFunctionTopKGenericData, AggregateFunctionTopKGeneric<is_plain_column, is_weighted>>
{ {
private: private:
@ -367,7 +367,7 @@ public:
/// Substitute return type for Date and DateTime /// Substitute return type for Date and DateTime
template <bool is_weighted> template <bool is_weighted>
class AggregateFunctionTopKDate : public AggregateFunctionTopK<DataTypeDate::FieldType, is_weighted> class AggregateFunctionTopKDate final : public AggregateFunctionTopK<DataTypeDate::FieldType, is_weighted>
{ {
public: public:
using AggregateFunctionTopK<DataTypeDate::FieldType, is_weighted>::AggregateFunctionTopK; using AggregateFunctionTopK<DataTypeDate::FieldType, is_weighted>::AggregateFunctionTopK;
@ -384,7 +384,7 @@ public:
}; };
template <bool is_weighted> template <bool is_weighted>
class AggregateFunctionTopKDateTime : public AggregateFunctionTopK<DataTypeDateTime::FieldType, is_weighted> class AggregateFunctionTopKDateTime final : public AggregateFunctionTopK<DataTypeDateTime::FieldType, is_weighted>
{ {
public: public:
using AggregateFunctionTopK<DataTypeDateTime::FieldType, is_weighted>::AggregateFunctionTopK; using AggregateFunctionTopK<DataTypeDateTime::FieldType, is_weighted>::AggregateFunctionTopK;
@ -401,7 +401,7 @@ public:
}; };
template <bool is_weighted> template <bool is_weighted>
class AggregateFunctionTopKIPv4 : public AggregateFunctionTopK<DataTypeIPv4::FieldType, is_weighted> class AggregateFunctionTopKIPv4 final : public AggregateFunctionTopK<DataTypeIPv4::FieldType, is_weighted>
{ {
public: public:
using AggregateFunctionTopK<DataTypeIPv4::FieldType, is_weighted>::AggregateFunctionTopK; using AggregateFunctionTopK<DataTypeIPv4::FieldType, is_weighted>::AggregateFunctionTopK;

View File

@ -150,7 +150,7 @@ struct AggregateFunctionDistinctMultipleGenericData : public AggregateFunctionDi
* Adding -Distinct suffix to aggregate function * Adding -Distinct suffix to aggregate function
**/ **/
template <typename Data> template <typename Data>
class AggregateFunctionDistinct : public IAggregateFunctionDataHelper<Data, AggregateFunctionDistinct<Data>> class AggregateFunctionDistinct final : public IAggregateFunctionDataHelper<Data, AggregateFunctionDistinct<Data>>
{ {
private: private:
AggregateFunctionPtr nested_func; AggregateFunctionPtr nested_func;

View File

@ -114,7 +114,7 @@ struct CrossTabData
template <typename Data> template <typename Data>
class AggregateFunctionCrossTab : public IAggregateFunctionDataHelper<Data, AggregateFunctionCrossTab<Data>> class AggregateFunctionCrossTab final : public IAggregateFunctionDataHelper<Data, AggregateFunctionCrossTab<Data>>
{ {
public: public:
explicit AggregateFunctionCrossTab(const DataTypes & arguments) explicit AggregateFunctionCrossTab(const DataTypes & arguments)

View File

@ -176,6 +176,8 @@ public:
/// Serializes state (to transmit it over the network, for example). /// Serializes state (to transmit it over the network, for example).
virtual void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> version = std::nullopt) const = 0; /// NOLINT virtual void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> version = std::nullopt) const = 0; /// NOLINT
virtual void serializeBatch(const PaddedPODArray<AggregateDataPtr> & data, size_t start, size_t size, WriteBuffer & buf, std::optional<size_t> version = std::nullopt) const = 0; /// NOLINT
/// Deserializes state. This function is called only for empty (just created) states. /// Deserializes state. This function is called only for empty (just created) states.
virtual void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> version = std::nullopt, Arena * arena = nullptr) const = 0; /// NOLINT virtual void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> version = std::nullopt, Arena * arena = nullptr) const = 0; /// NOLINT
@ -471,6 +473,12 @@ public:
} }
} }
void serializeBatch(const PaddedPODArray<AggregateDataPtr> & data, size_t start, size_t size, WriteBuffer & buf, std::optional<size_t> version) const override // NOLINT
{
for (size_t i = start; i < size; ++i)
static_cast<const Derived *>(this)->serialize(data[i], buf, version);
}
void addBatchSparse( void addBatchSparse(
size_t row_begin, size_t row_begin,
size_t row_end, size_t row_end,

View File

@ -418,9 +418,10 @@ WeakHash32 ColumnAggregateFunction::getWeakHash32() const
void ColumnAggregateFunction::updateHashFast(SipHash & hash) const void ColumnAggregateFunction::updateHashFast(SipHash & hash) const
{ {
/// Fallback to per-element hashing, as there is no faster way WriteBufferFromOwnString wbuf;
for (size_t i = 0; i < size(); ++i) const ColumnAggregateFunction::Container & vec = getData();
updateHashWithValue(i, hash); func->serializeBatch(vec, 0, size(), wbuf);
hash.update(wbuf.str().c_str(), wbuf.str().size());
} }
/// The returned size is less than real size. The reason is that some parts of /// The returned size is less than real size. The reason is that some parts of

View File

@ -60,12 +60,11 @@ void SerializationAggregateFunction::serializeBinaryBulk(const IColumn & column,
const ColumnAggregateFunction & real_column = typeid_cast<const ColumnAggregateFunction &>(column); const ColumnAggregateFunction & real_column = typeid_cast<const ColumnAggregateFunction &>(column);
const ColumnAggregateFunction::Container & vec = real_column.getData(); const ColumnAggregateFunction::Container & vec = real_column.getData();
ColumnAggregateFunction::Container::const_iterator it = vec.begin() + offset; size_t end = vec.size();
ColumnAggregateFunction::Container::const_iterator end = limit ? it + limit : vec.end(); if (limit)
end = std::min(end, offset + limit);
end = std::min(end, vec.end()); function->serializeBatch(vec, offset, end, ostr, version);
for (; it != end; ++it)
function->serialize(*it, ostr, version);
} }
void SerializationAggregateFunction::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double /*avg_value_size_hint*/) const void SerializationAggregateFunction::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double /*avg_value_size_hint*/) const

View File

@ -0,0 +1,10 @@
<test>
<create_query>DROP TABLE IF EXISTS test_avg_insert</create_query>
<create_query>
CREATE TABLE test_avg_insert (key UInt64, value AggregateFunction(avg, UInt8)) ENGINE = Memory()
</create_query>
<query>INSERT INTO test_avg_insert with initializeAggregation('avgState', 1) as s select number, s AS value FROM numbers(200000000)</query>
<drop_query>DROP TABLE IF EXISTS test_avg_insert</drop_query>
</test>