diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp b/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp index c7656a98de73..1289e29cac15 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayIntersect.cpp @@ -192,7 +192,7 @@ struct AggregateFunctionGroupArrayIntersectGenericData * For such columns GroupArrayIntersect() can be implemented more efficiently (especially for small numeric arrays). */ template -class AggregateFunctionGroupArrayIntersectGeneric +class AggregateFunctionGroupArrayIntersectGeneric final : public IAggregateFunctionDataHelper> { @@ -353,7 +353,7 @@ namespace { /// Substitute return type for Date and DateTime -class AggregateFunctionGroupArrayIntersectDate : public AggregateFunctionGroupArrayIntersect +class AggregateFunctionGroupArrayIntersectDate final : public AggregateFunctionGroupArrayIntersect { public: explicit AggregateFunctionGroupArrayIntersectDate(const DataTypePtr & argument_type, const Array & parameters_) @@ -361,7 +361,7 @@ class AggregateFunctionGroupArrayIntersectDate : public AggregateFunctionGroupAr static DataTypePtr createResultType() { return std::make_shared(std::make_shared()); } }; -class AggregateFunctionGroupArrayIntersectDateTime : public AggregateFunctionGroupArrayIntersect +class AggregateFunctionGroupArrayIntersectDateTime final : public AggregateFunctionGroupArrayIntersect { public: explicit AggregateFunctionGroupArrayIntersectDateTime(const DataTypePtr & argument_type, const Array & parameters_) @@ -369,7 +369,7 @@ class AggregateFunctionGroupArrayIntersectDateTime : public AggregateFunctionGro static DataTypePtr createResultType() { return std::make_shared(std::make_shared()); } }; -class AggregateFunctionGroupArrayIntersectDate32 : public AggregateFunctionGroupArrayIntersect +class AggregateFunctionGroupArrayIntersectDate32 final : public AggregateFunctionGroupArrayIntersect { public: explicit AggregateFunctionGroupArrayIntersectDate32(const DataTypePtr & argument_type, const Array & parameters_) diff --git a/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp index 88edbc4a0ae1..2ca877fc668c 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp @@ -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). */ template -class AggregateFunctionGroupUniqArrayGeneric +class AggregateFunctionGroupUniqArrayGeneric final : public IAggregateFunctionDataHelper> { @@ -245,7 +245,7 @@ class AggregateFunctionGroupUniqArrayGeneric /// Substitute return type for Date and DateTime template -class AggregateFunctionGroupUniqArrayDate : public AggregateFunctionGroupUniqArray +class AggregateFunctionGroupUniqArrayDate final : public AggregateFunctionGroupUniqArray { public: explicit AggregateFunctionGroupUniqArrayDate(const DataTypePtr & argument_type, const Array & parameters_, UInt64 max_elems_ = std::numeric_limits::max()) @@ -254,7 +254,7 @@ class AggregateFunctionGroupUniqArrayDate : public AggregateFunctionGroupUniqArr }; template -class AggregateFunctionGroupUniqArrayDateTime : public AggregateFunctionGroupUniqArray +class AggregateFunctionGroupUniqArrayDateTime final : public AggregateFunctionGroupUniqArray { public: explicit AggregateFunctionGroupUniqArrayDateTime(const DataTypePtr & argument_type, const Array & parameters_, UInt64 max_elems_ = std::numeric_limits::max()) @@ -263,7 +263,7 @@ class AggregateFunctionGroupUniqArrayDateTime : public AggregateFunctionGroupUni }; template -class AggregateFunctionGroupUniqArrayIPv4 : public AggregateFunctionGroupUniqArray +class AggregateFunctionGroupUniqArrayIPv4 final : public AggregateFunctionGroupUniqArray { public: explicit AggregateFunctionGroupUniqArrayIPv4(const DataTypePtr & argument_type, const Array & parameters_, UInt64 max_elems_ = std::numeric_limits::max()) diff --git a/src/AggregateFunctions/AggregateFunctionMeanZTest.cpp b/src/AggregateFunctions/AggregateFunctionMeanZTest.cpp index 7f4b1d9d6dce..6ede549ccc39 100644 --- a/src/AggregateFunctions/AggregateFunctionMeanZTest.cpp +++ b/src/AggregateFunctions/AggregateFunctionMeanZTest.cpp @@ -28,7 +28,7 @@ namespace /// Returns tuple of (z-statistic, p-value, confidence-interval-low, confidence-interval-high) template -class AggregateFunctionMeanZTest : +class AggregateFunctionMeanZTest final: public IAggregateFunctionDataHelper> { private: diff --git a/src/AggregateFunctions/AggregateFunctionRankCorrelation.cpp b/src/AggregateFunctions/AggregateFunctionRankCorrelation.cpp index 0c4726734ce9..dc0255ef044e 100644 --- a/src/AggregateFunctions/AggregateFunctionRankCorrelation.cpp +++ b/src/AggregateFunctions/AggregateFunctionRankCorrelation.cpp @@ -47,7 +47,7 @@ struct RankCorrelationData : public StatisticalSample } }; -class AggregateFunctionRankCorrelation : +class AggregateFunctionRankCorrelation final : public IAggregateFunctionDataHelper { public: diff --git a/src/AggregateFunctions/AggregateFunctionTTest.h b/src/AggregateFunctions/AggregateFunctionTTest.h index 45407b832067..6b887f0461e9 100644 --- a/src/AggregateFunctions/AggregateFunctionTTest.h +++ b/src/AggregateFunctions/AggregateFunctionTTest.h @@ -38,7 +38,7 @@ namespace ErrorCodes /// 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 template -class AggregateFunctionTTest : +class AggregateFunctionTTest final: public IAggregateFunctionDataHelper> { private: diff --git a/src/AggregateFunctions/AggregateFunctionTopK.cpp b/src/AggregateFunctions/AggregateFunctionTopK.cpp index 26108054e87b..484633a76ff6 100644 --- a/src/AggregateFunctions/AggregateFunctionTopK.cpp +++ b/src/AggregateFunctions/AggregateFunctionTopK.cpp @@ -203,7 +203,7 @@ struct AggregateFunctionTopKGenericData * For such columns topK() can be implemented more efficiently (especially for small numeric arrays). */ template -class AggregateFunctionTopKGeneric +class AggregateFunctionTopKGeneric final : public IAggregateFunctionDataHelper> { private: @@ -367,7 +367,7 @@ class AggregateFunctionTopKGeneric /// Substitute return type for Date and DateTime template -class AggregateFunctionTopKDate : public AggregateFunctionTopK +class AggregateFunctionTopKDate final : public AggregateFunctionTopK { public: using AggregateFunctionTopK::AggregateFunctionTopK; @@ -384,7 +384,7 @@ class AggregateFunctionTopKDate : public AggregateFunctionTopK -class AggregateFunctionTopKDateTime : public AggregateFunctionTopK +class AggregateFunctionTopKDateTime final : public AggregateFunctionTopK { public: using AggregateFunctionTopK::AggregateFunctionTopK; @@ -401,7 +401,7 @@ class AggregateFunctionTopKDateTime : public AggregateFunctionTopK -class AggregateFunctionTopKIPv4 : public AggregateFunctionTopK +class AggregateFunctionTopKIPv4 final : public AggregateFunctionTopK { public: using AggregateFunctionTopK::AggregateFunctionTopK; diff --git a/src/AggregateFunctions/Combinators/AggregateFunctionDistinct.h b/src/AggregateFunctions/Combinators/AggregateFunctionDistinct.h index f532858b3d8d..658d53abb2f1 100644 --- a/src/AggregateFunctions/Combinators/AggregateFunctionDistinct.h +++ b/src/AggregateFunctions/Combinators/AggregateFunctionDistinct.h @@ -150,7 +150,7 @@ struct AggregateFunctionDistinctMultipleGenericData : public AggregateFunctionDi * Adding -Distinct suffix to aggregate function **/ template -class AggregateFunctionDistinct : public IAggregateFunctionDataHelper> +class AggregateFunctionDistinct final : public IAggregateFunctionDataHelper> { private: AggregateFunctionPtr nested_func; diff --git a/src/AggregateFunctions/CrossTab.h b/src/AggregateFunctions/CrossTab.h index a3a357c69757..353b2e7f67cb 100644 --- a/src/AggregateFunctions/CrossTab.h +++ b/src/AggregateFunctions/CrossTab.h @@ -114,7 +114,7 @@ struct CrossTabData template -class AggregateFunctionCrossTab : public IAggregateFunctionDataHelper> +class AggregateFunctionCrossTab final : public IAggregateFunctionDataHelper> { public: explicit AggregateFunctionCrossTab(const DataTypes & arguments) diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index bd1a8f876a40..66330465a399 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -176,6 +176,8 @@ class IAggregateFunction : public std::enable_shared_from_this version = std::nullopt) const = 0; /// NOLINT + virtual void serializeBatch(const PaddedPODArray & data, size_t start, size_t size, WriteBuffer & buf, std::optional version = std::nullopt) const = 0; /// NOLINT + /// Deserializes state. This function is called only for empty (just created) states. virtual void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional version = std::nullopt, Arena * arena = nullptr) const = 0; /// NOLINT @@ -471,6 +473,12 @@ class IAggregateFunctionHelper : public IAggregateFunction } } + void serializeBatch(const PaddedPODArray & data, size_t start, size_t size, WriteBuffer & buf, std::optional version) const override // NOLINT + { + for (size_t i = start; i < size; ++i) + static_cast(this)->serialize(data[i], buf, version); + } + void addBatchSparse( size_t row_begin, size_t row_end, diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 2569d2235a30..902096c788e9 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -418,9 +418,10 @@ WeakHash32 ColumnAggregateFunction::getWeakHash32() const void ColumnAggregateFunction::updateHashFast(SipHash & hash) const { - /// Fallback to per-element hashing, as there is no faster way - for (size_t i = 0; i < size(); ++i) - updateHashWithValue(i, hash); + WriteBufferFromOwnString wbuf; + const ColumnAggregateFunction::Container & vec = getData(); + 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 diff --git a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp index 41b198890e4b..04e61558852b 100644 --- a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp +++ b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp @@ -60,12 +60,11 @@ void SerializationAggregateFunction::serializeBinaryBulk(const IColumn & column, const ColumnAggregateFunction & real_column = typeid_cast(column); const ColumnAggregateFunction::Container & vec = real_column.getData(); - ColumnAggregateFunction::Container::const_iterator it = vec.begin() + offset; - ColumnAggregateFunction::Container::const_iterator end = limit ? it + limit : vec.end(); + size_t end = vec.size(); + if (limit) + end = std::min(end, offset + limit); - end = std::min(end, vec.end()); - for (; it != end; ++it) - function->serialize(*it, ostr, version); + function->serializeBatch(vec, offset, end, ostr, version); } void SerializationAggregateFunction::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double /*avg_value_size_hint*/) const diff --git a/tests/performance/avg_serialization.xml b/tests/performance/avg_serialization.xml new file mode 100644 index 000000000000..f18d632cd9c8 --- /dev/null +++ b/tests/performance/avg_serialization.xml @@ -0,0 +1,10 @@ + + DROP TABLE IF EXISTS test_avg_insert + + CREATE TABLE test_avg_insert (key UInt64, value AggregateFunction(avg, UInt8)) ENGINE = Memory() + + + INSERT INTO test_avg_insert with initializeAggregation('avgState', 1) as s select number, s AS value FROM numbers(200000000) + + DROP TABLE IF EXISTS test_avg_insert +