mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-04 05:22:17 +00:00
Merge pull request #72556 from ClickHouse/devirtualize_serialize_call
Devirtualize serialize call in AggregateFunction
This commit is contained in:
commit
4f596388a2
@ -192,7 +192,7 @@ struct AggregateFunctionGroupArrayIntersectGenericData
|
||||
* For such columns GroupArrayIntersect() can be implemented more efficiently (especially for small numeric arrays).
|
||||
*/
|
||||
template <bool is_plain_column = false>
|
||||
class AggregateFunctionGroupArrayIntersectGeneric
|
||||
class AggregateFunctionGroupArrayIntersectGeneric final
|
||||
: public IAggregateFunctionDataHelper<AggregateFunctionGroupArrayIntersectGenericData,
|
||||
AggregateFunctionGroupArrayIntersectGeneric<is_plain_column>>
|
||||
{
|
||||
@ -353,7 +353,7 @@ namespace
|
||||
{
|
||||
|
||||
/// Substitute return type for Date and DateTime
|
||||
class AggregateFunctionGroupArrayIntersectDate : public AggregateFunctionGroupArrayIntersect<DataTypeDate::FieldType>
|
||||
class AggregateFunctionGroupArrayIntersectDate final : public AggregateFunctionGroupArrayIntersect<DataTypeDate::FieldType>
|
||||
{
|
||||
public:
|
||||
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>()); }
|
||||
};
|
||||
|
||||
class AggregateFunctionGroupArrayIntersectDateTime : public AggregateFunctionGroupArrayIntersect<DataTypeDateTime::FieldType>
|
||||
class AggregateFunctionGroupArrayIntersectDateTime final : public AggregateFunctionGroupArrayIntersect<DataTypeDateTime::FieldType>
|
||||
{
|
||||
public:
|
||||
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>()); }
|
||||
};
|
||||
|
||||
class AggregateFunctionGroupArrayIntersectDate32 : public AggregateFunctionGroupArrayIntersect<DataTypeDate32::FieldType>
|
||||
class AggregateFunctionGroupArrayIntersectDate32 final : public AggregateFunctionGroupArrayIntersect<DataTypeDate32::FieldType>
|
||||
{
|
||||
public:
|
||||
explicit AggregateFunctionGroupArrayIntersectDate32(const DataTypePtr & argument_type, const Array & parameters_)
|
||||
|
@ -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 <bool is_plain_column = false, typename LimitNumElems = std::false_type>
|
||||
class AggregateFunctionGroupUniqArrayGeneric
|
||||
class AggregateFunctionGroupUniqArrayGeneric final
|
||||
: public IAggregateFunctionDataHelper<AggregateFunctionGroupUniqArrayGenericData,
|
||||
AggregateFunctionGroupUniqArrayGeneric<is_plain_column, LimitNumElems>>
|
||||
{
|
||||
@ -245,7 +245,7 @@ public:
|
||||
|
||||
/// Substitute return type for Date and DateTime
|
||||
template <typename HasLimit>
|
||||
class AggregateFunctionGroupUniqArrayDate : public AggregateFunctionGroupUniqArray<DataTypeDate::FieldType, HasLimit>
|
||||
class AggregateFunctionGroupUniqArrayDate final : public AggregateFunctionGroupUniqArray<DataTypeDate::FieldType, HasLimit>
|
||||
{
|
||||
public:
|
||||
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>
|
||||
class AggregateFunctionGroupUniqArrayDateTime : public AggregateFunctionGroupUniqArray<DataTypeDateTime::FieldType, HasLimit>
|
||||
class AggregateFunctionGroupUniqArrayDateTime final : public AggregateFunctionGroupUniqArray<DataTypeDateTime::FieldType, HasLimit>
|
||||
{
|
||||
public:
|
||||
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>
|
||||
class AggregateFunctionGroupUniqArrayIPv4 : public AggregateFunctionGroupUniqArray<DataTypeIPv4::FieldType, HasLimit>
|
||||
class AggregateFunctionGroupUniqArrayIPv4 final : public AggregateFunctionGroupUniqArray<DataTypeIPv4::FieldType, HasLimit>
|
||||
{
|
||||
public:
|
||||
explicit AggregateFunctionGroupUniqArrayIPv4(const DataTypePtr & argument_type, const Array & parameters_, UInt64 max_elems_ = std::numeric_limits<UInt64>::max())
|
||||
|
@ -28,7 +28,7 @@ namespace
|
||||
|
||||
/// Returns tuple of (z-statistic, p-value, confidence-interval-low, confidence-interval-high)
|
||||
template <typename Data>
|
||||
class AggregateFunctionMeanZTest :
|
||||
class AggregateFunctionMeanZTest final:
|
||||
public IAggregateFunctionDataHelper<Data, AggregateFunctionMeanZTest<Data>>
|
||||
{
|
||||
private:
|
||||
|
@ -47,7 +47,7 @@ struct RankCorrelationData : public StatisticalSample<Float64, Float64>
|
||||
}
|
||||
};
|
||||
|
||||
class AggregateFunctionRankCorrelation :
|
||||
class AggregateFunctionRankCorrelation final :
|
||||
public IAggregateFunctionDataHelper<RankCorrelationData, AggregateFunctionRankCorrelation>
|
||||
{
|
||||
public:
|
||||
|
@ -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 <typename Data>
|
||||
class AggregateFunctionTTest :
|
||||
class AggregateFunctionTTest final:
|
||||
public IAggregateFunctionDataHelper<Data, AggregateFunctionTTest<Data>>
|
||||
{
|
||||
private:
|
||||
|
@ -203,7 +203,7 @@ struct AggregateFunctionTopKGenericData
|
||||
* For such columns topK() can be implemented more efficiently (especially for small numeric arrays).
|
||||
*/
|
||||
template <bool is_plain_column, bool is_weighted>
|
||||
class AggregateFunctionTopKGeneric
|
||||
class AggregateFunctionTopKGeneric final
|
||||
: public IAggregateFunctionDataHelper<AggregateFunctionTopKGenericData, AggregateFunctionTopKGeneric<is_plain_column, is_weighted>>
|
||||
{
|
||||
private:
|
||||
@ -367,7 +367,7 @@ public:
|
||||
|
||||
/// Substitute return type for Date and DateTime
|
||||
template <bool is_weighted>
|
||||
class AggregateFunctionTopKDate : public AggregateFunctionTopK<DataTypeDate::FieldType, is_weighted>
|
||||
class AggregateFunctionTopKDate final : public AggregateFunctionTopK<DataTypeDate::FieldType, is_weighted>
|
||||
{
|
||||
public:
|
||||
using AggregateFunctionTopK<DataTypeDate::FieldType, is_weighted>::AggregateFunctionTopK;
|
||||
@ -384,7 +384,7 @@ public:
|
||||
};
|
||||
|
||||
template <bool is_weighted>
|
||||
class AggregateFunctionTopKDateTime : public AggregateFunctionTopK<DataTypeDateTime::FieldType, is_weighted>
|
||||
class AggregateFunctionTopKDateTime final : public AggregateFunctionTopK<DataTypeDateTime::FieldType, is_weighted>
|
||||
{
|
||||
public:
|
||||
using AggregateFunctionTopK<DataTypeDateTime::FieldType, is_weighted>::AggregateFunctionTopK;
|
||||
@ -401,7 +401,7 @@ public:
|
||||
};
|
||||
|
||||
template <bool is_weighted>
|
||||
class AggregateFunctionTopKIPv4 : public AggregateFunctionTopK<DataTypeIPv4::FieldType, is_weighted>
|
||||
class AggregateFunctionTopKIPv4 final : public AggregateFunctionTopK<DataTypeIPv4::FieldType, is_weighted>
|
||||
{
|
||||
public:
|
||||
using AggregateFunctionTopK<DataTypeIPv4::FieldType, is_weighted>::AggregateFunctionTopK;
|
||||
|
@ -150,7 +150,7 @@ struct AggregateFunctionDistinctMultipleGenericData : public AggregateFunctionDi
|
||||
* Adding -Distinct suffix to aggregate function
|
||||
**/
|
||||
template <typename Data>
|
||||
class AggregateFunctionDistinct : public IAggregateFunctionDataHelper<Data, AggregateFunctionDistinct<Data>>
|
||||
class AggregateFunctionDistinct final : public IAggregateFunctionDataHelper<Data, AggregateFunctionDistinct<Data>>
|
||||
{
|
||||
private:
|
||||
AggregateFunctionPtr nested_func;
|
||||
|
@ -114,7 +114,7 @@ struct CrossTabData
|
||||
|
||||
|
||||
template <typename Data>
|
||||
class AggregateFunctionCrossTab : public IAggregateFunctionDataHelper<Data, AggregateFunctionCrossTab<Data>>
|
||||
class AggregateFunctionCrossTab final : public IAggregateFunctionDataHelper<Data, AggregateFunctionCrossTab<Data>>
|
||||
{
|
||||
public:
|
||||
explicit AggregateFunctionCrossTab(const DataTypes & arguments)
|
||||
|
@ -176,6 +176,8 @@ public:
|
||||
/// 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 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.
|
||||
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(
|
||||
size_t row_begin,
|
||||
size_t row_end,
|
||||
|
@ -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
|
||||
|
@ -60,12 +60,11 @@ void SerializationAggregateFunction::serializeBinaryBulk(const IColumn & column,
|
||||
const ColumnAggregateFunction & real_column = typeid_cast<const ColumnAggregateFunction &>(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
|
||||
|
10
tests/performance/avg_serialization.xml
Normal file
10
tests/performance/avg_serialization.xml
Normal 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>
|
Loading…
Reference in New Issue
Block a user