ClickHouse/src/AggregateFunctions/AggregateFunctionSumMap.h

530 lines
18 KiB
C++
Raw Normal View History

#pragma once
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
2020-08-02 01:29:52 +00:00
#include <DataTypes/DataTypeNullable.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Common/FieldVisitors.h>
#include <Common/assert_cast.h>
#include <AggregateFunctions/IAggregateFunction.h>
2017-10-12 20:49:26 +00:00
#include <map>
namespace DB
{
namespace ErrorCodes
{
2020-04-28 14:30:45 +00:00
extern const int BAD_ARGUMENTS;
2020-04-29 06:35:02 +00:00
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
2020-06-16 10:44:23 +00:00
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
2020-08-02 14:22:53 +00:00
extern const int LOGICAL_ERROR;
}
template <typename T>
2020-06-15 18:53:54 +00:00
struct AggregateFunctionMapData
{
// Map needs to be ordered to maintain function properties
std::map<T, Array> merged_maps;
};
2017-10-12 20:49:59 +00:00
/** Aggregate function, that takes at least two arguments: keys and values, and as a result, builds a tuple of of at least 2 arrays -
2020-06-11 10:31:37 +00:00
* ordered keys and variable number of argument values aggregated by corresponding keys.
*
2020-06-11 10:31:37 +00:00
* sumMap function is the most useful when using SummingMergeTree to sum Nested columns, which name ends in "Map".
*
* Example: sumMap(k, v...) of:
* k v
* [1,2,3] [10,10,10]
* [3,4,5] [10,10,10]
* [4,5,6] [10,10,10]
* [6,7,8] [10,10,10]
* [7,5,3] [5,15,25]
* [8,9,10] [20,20,20]
* will return:
* ([1,2,3,4,5,6,7,8,9,10],[10,10,45,20,35,20,15,30,20,20])
2020-06-11 10:31:37 +00:00
*
* minMap and maxMap share the same idea, but calculate min and max correspondingly.
*/
template <typename T, typename Derived, typename Visitor, bool overflow, bool tuple_argument, bool compact>
2020-06-15 18:53:54 +00:00
class AggregateFunctionMapBase : public IAggregateFunctionDataHelper<
AggregateFunctionMapData<NearestFieldType<T>>, Derived>
{
private:
DataTypePtr keys_type;
2017-10-12 20:51:12 +00:00
DataTypes values_types;
public:
2020-06-16 10:44:23 +00:00
using Base = IAggregateFunctionDataHelper<
AggregateFunctionMapData<NearestFieldType<T>>, Derived>;
2020-06-16 10:44:23 +00:00
AggregateFunctionMapBase(const DataTypePtr & keys_type_,
const DataTypes & values_types_, const DataTypes & argument_types_)
: Base(argument_types_, {} /* parameters */), keys_type(keys_type_),
values_types(values_types_)
{}
DataTypePtr getReturnType() const override
{
DataTypes types;
types.emplace_back(std::make_shared<DataTypeArray>(keys_type));
for (const auto & value_type : values_types)
2020-04-28 14:30:45 +00:00
{
DataTypePtr result_type;
if constexpr (overflow)
{
// Overflow, meaning that the returned type is the same as
// the input type.
result_type = value_type;
}
else
2020-04-29 06:05:52 +00:00
{
2020-08-02 01:29:52 +00:00
auto value_type_without_nullable = removeNullable(value_type);
2020-04-28 14:30:45 +00:00
// No overflow, meaning we promote the types if necessary.
2020-08-02 01:29:52 +00:00
if (!value_type_without_nullable->canBePromoted())
throw Exception{ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Values for {} are expected to be Numeric, Float or Decimal, passed type {}",
getName(), value_type->getName()};
2020-04-28 14:30:45 +00:00
2020-08-02 01:29:52 +00:00
result_type = value_type_without_nullable->promoteNumericType();
2020-04-28 14:30:45 +00:00
}
types.emplace_back(std::make_shared<DataTypeArray>(result_type));
}
return std::make_shared<DataTypeTuple>(types);
}
2020-04-29 06:05:52 +00:00
static const auto & getArgumentColumns(const IColumn**& columns)
{
if constexpr (tuple_argument)
{
2020-04-29 06:05:52 +00:00
return assert_cast<const ColumnTuple *>(columns[0])->getColumns();
}
else
{
2020-04-29 06:05:52 +00:00
return columns;
}
2020-04-29 06:05:52 +00:00
}
2020-04-29 06:05:52 +00:00
void add(AggregateDataPtr place, const IColumn** _columns, const size_t row_num, Arena *) const override
{
const auto & columns = getArgumentColumns(_columns);
// Column 0 contains array of keys of known type
const ColumnArray & array_column0 = assert_cast<const ColumnArray &>(*columns[0]);
const IColumn::Offsets & offsets0 = array_column0.getOffsets();
const IColumn & key_column = array_column0.getData();
const size_t keys_vec_offset = offsets0[row_num - 1];
const size_t keys_vec_size = (offsets0[row_num] - keys_vec_offset);
// Columns 1..n contain arrays of numeric values to sum
auto & merged_maps = this->data(place).merged_maps;
2017-10-12 21:03:51 +00:00
for (size_t col = 0, size = values_types.size(); col < size; ++col)
{
2020-04-28 14:30:45 +00:00
const auto & array_column = assert_cast<const ColumnArray&>(*columns[col + 1]);
const IColumn & value_column = array_column.getData();
const IColumn::Offsets & offsets = array_column.getOffsets();
2018-12-24 14:26:38 +00:00
const size_t values_vec_offset = offsets[row_num - 1];
const size_t values_vec_size = (offsets[row_num] - values_vec_offset);
// Expect key and value arrays to be of same length
if (keys_vec_size != values_vec_size)
2020-04-28 14:30:45 +00:00
throw Exception("Sizes of keys and values arrays do not match", ErrorCodes::BAD_ARGUMENTS);
// Insert column values for all keys
for (size_t i = 0; i < keys_vec_size; ++i)
{
2020-04-28 14:30:45 +00:00
auto value = value_column.operator[](values_vec_offset + i);
auto key = key_column.operator[](keys_vec_offset + i).get<T>();
2019-01-22 16:47:43 +00:00
if (!keepKey(key))
continue;
2020-08-02 01:29:52 +00:00
if (value.isNull())
continue;
2019-01-22 16:47:43 +00:00
2020-04-28 14:30:45 +00:00
typename std::decay_t<decltype(merged_maps)>::iterator it;
if constexpr (IsDecimalNumber<T>)
{
2020-04-28 14:30:45 +00:00
// FIXME why is storing NearestFieldType not enough, and we
// have to check for decimals again here?
UInt32 scale = static_cast<const ColumnDecimal<T> &>(key_column).getData().getScale();
it = merged_maps.find(DecimalField<T>(key, scale));
}
else
it = merged_maps.find(key);
if (it != merged_maps.end())
2020-04-28 14:30:45 +00:00
{
2020-06-11 10:31:37 +00:00
applyVisitor(Visitor(value), it->second[col]);
2020-04-28 14:30:45 +00:00
}
else
{
// Create a value array for this key
Array new_values;
new_values.resize(values_types.size());
for (size_t k = 0; k < new_values.size(); ++k)
2020-04-28 14:30:45 +00:00
{
new_values[k] = (k == col) ? value : values_types[k]->getDefault();
2020-04-28 14:30:45 +00:00
}
if constexpr (IsDecimalNumber<T>)
{
UInt32 scale = static_cast<const ColumnDecimal<T> &>(key_column).getData().getScale();
merged_maps.emplace(DecimalField<T>(key, scale), std::move(new_values));
}
else
2020-04-28 14:30:45 +00:00
{
merged_maps.emplace(key, std::move(new_values));
2020-04-28 14:30:45 +00:00
}
}
}
}
}
2017-12-01 21:51:50 +00:00
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override
{
auto & merged_maps = this->data(place).merged_maps;
const auto & rhs_maps = this->data(rhs).merged_maps;
for (const auto & elem : rhs_maps)
{
const auto & it = merged_maps.find(elem.first);
if (it != merged_maps.end())
{
for (size_t col = 0; col < values_types.size(); ++col)
2020-06-11 10:31:37 +00:00
applyVisitor(Visitor(elem.second[col]), it->second[col]);
}
else
merged_maps[elem.first] = elem.second;
}
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
const auto & merged_maps = this->data(place).merged_maps;
size_t size = merged_maps.size();
writeVarUInt(size, buf);
for (const auto & elem : merged_maps)
{
keys_type->serializeBinary(elem.first, buf);
for (size_t col = 0; col < values_types.size(); ++col)
values_types[col]->serializeBinary(elem.second[col], buf);
}
}
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override
{
auto & merged_maps = this->data(place).merged_maps;
size_t size = 0;
readVarUInt(size, buf);
for (size_t i = 0; i < size; ++i)
{
Field key;
keys_type->deserializeBinary(key, buf);
Array values;
values.resize(values_types.size());
for (size_t col = 0; col < values_types.size(); ++col)
values_types[col]->deserializeBinary(values[col], buf);
if constexpr (IsDecimalNumber<T>)
merged_maps[key.get<DecimalField<T>>()] = values;
else
merged_maps[key.get<T>()] = values;
}
}
2020-06-17 19:36:27 +00:00
void insertResultInto(AggregateDataPtr place, IColumn & to, Arena *) const override
{
// Final step does compaction of keys that have zero values, this mutates the state
auto & merged_maps = this->data(place).merged_maps;
2020-11-04 14:20:11 +00:00
// Remove keys which are zeros or empty. This should be enabled only for sumMap.
if constexpr (compact)
{
for (auto it = merged_maps.cbegin(); it != merged_maps.cend();)
{
// Key is not compacted if it has at least one non-zero value
bool erase = true;
for (size_t col = 0; col < values_types.size(); ++col)
{
if (it->second[col] != values_types[col]->getDefault())
{
erase = false;
break;
}
}
if (erase)
it = merged_maps.erase(it);
else
++it;
}
}
size_t size = merged_maps.size();
auto & to_tuple = assert_cast<ColumnTuple &>(to);
auto & to_keys_arr = assert_cast<ColumnArray &>(to_tuple.getColumn(0));
auto & to_keys_col = to_keys_arr.getData();
// Advance column offsets
auto & to_keys_offsets = to_keys_arr.getOffsets();
2019-01-08 10:07:33 +00:00
to_keys_offsets.push_back(to_keys_offsets.back() + size);
to_keys_col.reserve(size);
for (size_t col = 0; col < values_types.size(); ++col)
{
auto & to_values_arr = assert_cast<ColumnArray &>(to_tuple.getColumn(col + 1));
auto & to_values_offsets = to_values_arr.getOffsets();
2019-01-08 10:07:33 +00:00
to_values_offsets.push_back(to_values_offsets.back() + size);
to_values_arr.getData().reserve(size);
}
// Write arrays of keys and values
for (const auto & elem : merged_maps)
{
// Write array of keys into column
to_keys_col.insert(elem.first);
// Write 0..n arrays of values
for (size_t col = 0; col < values_types.size(); ++col)
{
auto & to_values_col = assert_cast<ColumnArray &>(to_tuple.getColumn(col + 1)).getData();
to_values_col.insert(elem.second[col]);
}
}
}
bool keepKey(const T & key) const { return static_cast<const Derived &>(*this).keepKey(key); }
2020-06-11 10:31:37 +00:00
String getName() const override { return static_cast<const Derived &>(*this).getName(); }
2019-01-22 16:47:43 +00:00
};
2020-04-28 14:30:45 +00:00
template <typename T, bool overflow, bool tuple_argument>
2019-01-25 19:35:53 +00:00
class AggregateFunctionSumMap final :
public AggregateFunctionMapBase<T, AggregateFunctionSumMap<T, overflow, tuple_argument>, FieldVisitorSum, overflow, tuple_argument, true>
2019-01-22 16:47:43 +00:00
{
2019-01-25 19:35:53 +00:00
private:
2020-04-28 14:30:45 +00:00
using Self = AggregateFunctionSumMap<T, overflow, tuple_argument>;
using Base = AggregateFunctionMapBase<T, Self, FieldVisitorSum, overflow, tuple_argument, true>;
2019-01-25 19:35:53 +00:00
2019-01-22 16:47:43 +00:00
public:
2020-06-16 10:44:23 +00:00
AggregateFunctionSumMap(const DataTypePtr & keys_type_,
DataTypes & values_types_, const DataTypes & argument_types_,
const Array & params_)
: Base{keys_type_, values_types_, argument_types_}
{
// The constructor accepts parameters to have a uniform interface with
// sumMapFiltered, but this function doesn't have any parameters.
assertNoParameters(getName(), params_);
}
2019-01-22 16:47:43 +00:00
String getName() const override { return "sumMap"; }
bool keepKey(const T &) const { return true; }
2019-01-22 16:47:43 +00:00
};
2020-04-28 14:30:45 +00:00
template <typename T, bool overflow, bool tuple_argument>
2019-01-25 19:35:53 +00:00
class AggregateFunctionSumMapFiltered final :
2020-06-15 18:53:54 +00:00
public AggregateFunctionMapBase<T,
2020-04-28 14:30:45 +00:00
AggregateFunctionSumMapFiltered<T, overflow, tuple_argument>,
2020-06-11 10:31:37 +00:00
FieldVisitorSum,
2020-04-28 14:30:45 +00:00
overflow,
tuple_argument,
true>
2019-01-22 16:47:43 +00:00
{
private:
2020-04-28 14:30:45 +00:00
using Self = AggregateFunctionSumMapFiltered<T, overflow, tuple_argument>;
using Base = AggregateFunctionMapBase<T, Self, FieldVisitorSum, overflow, tuple_argument, true>;
2019-01-25 19:35:53 +00:00
/// ARCADIA_BUILD disallow unordered_set for big ints for some reason
static constexpr const bool allow_hash = !OverBigInt<T>;
using ContainerT = std::conditional_t<allow_hash, std::unordered_set<T>, std::set<T>>;
ContainerT keys_to_keep;
2019-01-22 16:47:43 +00:00
public:
2020-06-16 10:44:23 +00:00
AggregateFunctionSumMapFiltered(const DataTypePtr & keys_type_,
const DataTypes & values_types_, const DataTypes & argument_types_,
const Array & params_)
: Base{keys_type_, values_types_, argument_types_}
2019-01-22 16:47:43 +00:00
{
2020-06-16 10:44:23 +00:00
if (params_.size() != 1)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Aggregate function '{}' requires exactly one parameter "
"of Array type", getName());
Array keys_to_keep_;
if (!params_.front().tryGet<Array>(keys_to_keep_))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Aggregate function {} requires an Array as a parameter",
getName());
if constexpr (allow_hash)
keys_to_keep.reserve(keys_to_keep_.size());
for (const Field & f : keys_to_keep_)
2019-01-22 16:47:43 +00:00
{
2019-01-24 12:31:33 +00:00
keys_to_keep.emplace(f.safeGet<NearestFieldType<T>>());
2019-01-22 16:47:43 +00:00
}
}
2020-06-16 10:44:23 +00:00
String getName() const override
{ return overflow ? "sumMapFilteredWithOverflow" : "sumMapFiltered"; }
2019-01-22 16:47:43 +00:00
2019-01-24 12:31:33 +00:00
bool keepKey(const T & key) const { return keys_to_keep.count(key); }
};
2020-08-02 01:29:52 +00:00
/** Implements `Max` operation.
* Returns true if changed
*/
class FieldVisitorMax : public StaticVisitor<bool>
{
private:
const Field & rhs;
public:
explicit FieldVisitorMax(const Field & rhs_) : rhs(rhs_) {}
bool operator() (Null &) const { throw Exception("Cannot compare Nulls", ErrorCodes::LOGICAL_ERROR); }
bool operator() (Array &) const { throw Exception("Cannot compare Arrays", ErrorCodes::LOGICAL_ERROR); }
bool operator() (Tuple &) const { throw Exception("Cannot compare Tuples", ErrorCodes::LOGICAL_ERROR); }
bool operator() (AggregateFunctionStateData &) const { throw Exception("Cannot compare AggregateFunctionStates", ErrorCodes::LOGICAL_ERROR); }
template <typename T>
bool operator() (DecimalField<T> & x) const
{
auto val = get<DecimalField<T>>(rhs);
if (val > x)
{
x = val;
return true;
}
return false;
}
template <typename T>
bool operator() (T & x) const
{
auto val = get<T>(rhs);
if (val > x)
{
x = val;
return true;
}
return false;
}
};
/** Implements `Min` operation.
* Returns true if changed
*/
class FieldVisitorMin : public StaticVisitor<bool>
{
private:
const Field & rhs;
public:
explicit FieldVisitorMin(const Field & rhs_) : rhs(rhs_) {}
bool operator() (Null &) const { throw Exception("Cannot compare Nulls", ErrorCodes::LOGICAL_ERROR); }
bool operator() (Array &) const { throw Exception("Cannot sum Arrays", ErrorCodes::LOGICAL_ERROR); }
bool operator() (Tuple &) const { throw Exception("Cannot sum Tuples", ErrorCodes::LOGICAL_ERROR); }
bool operator() (AggregateFunctionStateData &) const { throw Exception("Cannot sum AggregateFunctionStates", ErrorCodes::LOGICAL_ERROR); }
template <typename T>
bool operator() (DecimalField<T> & x) const
{
auto val = get<DecimalField<T>>(rhs);
if (val < x)
{
x = val;
return true;
}
return false;
}
template <typename T>
bool operator() (T & x) const
{
auto val = get<T>(rhs);
if (val < x)
{
x = val;
return true;
}
return false;
}
};
2020-06-11 10:31:37 +00:00
template <typename T, bool tuple_argument>
class AggregateFunctionMinMap final :
public AggregateFunctionMapBase<T, AggregateFunctionMinMap<T, tuple_argument>, FieldVisitorMin, true, tuple_argument, false>
2020-06-11 10:31:37 +00:00
{
private:
using Self = AggregateFunctionMinMap<T, tuple_argument>;
using Base = AggregateFunctionMapBase<T, Self, FieldVisitorMin, true, tuple_argument, false>;
2020-06-11 10:31:37 +00:00
public:
2020-06-16 10:44:23 +00:00
AggregateFunctionMinMap(const DataTypePtr & keys_type_,
DataTypes & values_types_, const DataTypes & argument_types_,
const Array & params_)
: Base{keys_type_, values_types_, argument_types_}
{
// The constructor accepts parameters to have a uniform interface with
// sumMapFiltered, but this function doesn't have any parameters.
assertNoParameters(getName(), params_);
}
2020-06-11 10:31:37 +00:00
String getName() const override { return "minMap"; }
bool keepKey(const T &) const { return true; }
};
template <typename T, bool tuple_argument>
class AggregateFunctionMaxMap final :
public AggregateFunctionMapBase<T, AggregateFunctionMaxMap<T, tuple_argument>, FieldVisitorMax, true, tuple_argument, false>
2020-06-11 10:31:37 +00:00
{
private:
using Self = AggregateFunctionMaxMap<T, tuple_argument>;
using Base = AggregateFunctionMapBase<T, Self, FieldVisitorMax, true, tuple_argument, false>;
2020-06-11 10:31:37 +00:00
public:
2020-06-16 10:44:23 +00:00
AggregateFunctionMaxMap(const DataTypePtr & keys_type_,
DataTypes & values_types_, const DataTypes & argument_types_,
const Array & params_)
: Base{keys_type_, values_types_, argument_types_}
{
// The constructor accepts parameters to have a uniform interface with
// sumMapFiltered, but this function doesn't have any parameters.
assertNoParameters(getName(), params_);
}
2020-06-11 10:31:37 +00:00
String getName() const override { return "maxMap"; }
bool keepKey(const T &) const { return true; }
};
}