mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-04 13:32:13 +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).
|
* 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_)
|
||||||
|
@ -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())
|
||||||
|
@ -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:
|
||||||
|
@ -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:
|
||||||
|
@ -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:
|
||||||
|
@ -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;
|
||||||
|
@ -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;
|
||||||
|
@ -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)
|
||||||
|
@ -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,
|
||||||
|
@ -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
|
||||||
|
@ -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
|
||||||
|
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