2017-09-15 11:14:19 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <IO/WriteHelpers.h>
|
|
|
|
#include <IO/ReadHelpers.h>
|
|
|
|
|
|
|
|
#include <DataTypes/DataTypeArray.h>
|
2017-09-19 12:35:25 +00:00
|
|
|
#include <DataTypes/DataTypeTuple.h>
|
2017-09-15 11:14:19 +00:00
|
|
|
|
|
|
|
#include <Columns/ColumnArray.h>
|
2017-09-19 12:35:25 +00:00
|
|
|
#include <Columns/ColumnTuple.h>
|
2017-09-15 11:14:19 +00:00
|
|
|
|
2017-09-18 15:41:07 +00:00
|
|
|
#include <Core/FieldVisitors.h>
|
2017-09-15 11:14:19 +00:00
|
|
|
#include <AggregateFunctions/IBinaryAggregateFunction.h>
|
|
|
|
#include <Functions/FunctionHelpers.h>
|
2017-10-12 04:13:33 +00:00
|
|
|
#include <Common/HashTable/HashMap.h>
|
2017-09-15 11:14:19 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
|
|
|
}
|
|
|
|
|
2017-10-12 04:13:33 +00:00
|
|
|
template <typename T>
|
2017-09-15 11:14:19 +00:00
|
|
|
struct AggregateFunctionSumMapData
|
|
|
|
{
|
2017-10-12 04:13:33 +00:00
|
|
|
// Map needs to be ordered to maintain function properties
|
|
|
|
std::map<T, Array> merged_maps;
|
2017-09-15 11:14:19 +00:00
|
|
|
};
|
|
|
|
|
2017-10-12 04:13:33 +00:00
|
|
|
/** Aggregate function, that takes at keast two arguments: keys and values, and as a result, builds a tuple of of at least 2 arrays -
|
|
|
|
* ordered keys and variable number of argument values summed up by corresponding keys.
|
2017-09-15 11:14:19 +00:00
|
|
|
*
|
|
|
|
* This function is the most useful when using SummingMergeTree to sum Nested columns, which name ends in "Map".
|
|
|
|
*
|
2017-10-12 04:13:33 +00:00
|
|
|
* Example: sumMap(k, v...) of:
|
2017-09-15 11:14:19 +00:00
|
|
|
* 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:
|
2017-09-19 12:35:25 +00:00
|
|
|
* ([1,2,3,4,5,6,7,8,9,10],[10,10,45,20,35,20,15,30,20,20])
|
2017-09-15 11:14:19 +00:00
|
|
|
*/
|
2017-10-12 04:13:33 +00:00
|
|
|
|
|
|
|
template <typename T>
|
|
|
|
class AggregateFunctionSumMap final : public IAggregateFunctionHelper<AggregateFunctionSumMapData<typename NearestFieldType<T>::Type>>
|
2017-09-15 11:14:19 +00:00
|
|
|
{
|
|
|
|
private:
|
2017-09-18 17:11:16 +00:00
|
|
|
DataTypePtr keys_type;
|
2017-10-12 04:13:33 +00:00
|
|
|
std::vector<DataTypePtr> values_types;
|
2017-09-15 11:14:19 +00:00
|
|
|
|
|
|
|
public:
|
|
|
|
String getName() const override { return "sumMap"; }
|
|
|
|
|
|
|
|
DataTypePtr getReturnType() const override
|
|
|
|
{
|
2017-09-19 12:35:25 +00:00
|
|
|
DataTypes types;
|
|
|
|
types.emplace_back(std::make_shared<DataTypeArray>(keys_type));
|
2017-10-12 04:13:33 +00:00
|
|
|
|
|
|
|
for (const auto & value_type : values_types)
|
|
|
|
types.emplace_back(std::make_shared<DataTypeArray>(value_type));
|
2017-09-19 12:35:25 +00:00
|
|
|
|
|
|
|
return std::make_shared<DataTypeTuple>(types);
|
2017-09-15 11:14:19 +00:00
|
|
|
}
|
|
|
|
|
2017-10-12 04:13:33 +00:00
|
|
|
void setArguments(const DataTypes & arguments) override
|
2017-09-15 11:14:19 +00:00
|
|
|
{
|
2017-10-12 04:13:33 +00:00
|
|
|
if (arguments.size() < 2)
|
|
|
|
throw Exception("Aggregate function " + getName() + "require at leasat two arguments of array type.",
|
2017-09-15 11:14:19 +00:00
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
const auto * array_type = checkAndGetDataType<DataTypeArray>(arguments[0].get());
|
|
|
|
if (!array_type)
|
|
|
|
throw Exception("First argument for function " + getName() + " must be an array.",
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2017-09-18 17:11:16 +00:00
|
|
|
keys_type = array_type->getNestedType();
|
2017-09-15 11:14:19 +00:00
|
|
|
|
2017-10-12 04:13:33 +00:00
|
|
|
for (size_t i = 1; i < arguments.size(); ++i)
|
|
|
|
{
|
|
|
|
array_type = checkAndGetDataType<DataTypeArray>(arguments[i].get());
|
|
|
|
if (!array_type)
|
|
|
|
throw Exception("Argument " + std::to_string(i) + " for function " + getName() + " must be an array.",
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
values_types.push_back(array_type->getNestedType());
|
|
|
|
}
|
2017-09-15 11:14:19 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void setParameters(const Array & params) override
|
|
|
|
{
|
|
|
|
if (!params.empty())
|
|
|
|
throw Exception("This instantiation of " + getName() + "aggregate function doesn't accept any parameters.",
|
|
|
|
ErrorCodes::LOGICAL_ERROR);
|
|
|
|
}
|
|
|
|
|
2017-10-12 04:13:33 +00:00
|
|
|
void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num, Arena * arena) const override final
|
2017-09-15 11:14:19 +00:00
|
|
|
{
|
2017-10-12 04:13:33 +00:00
|
|
|
// Column 0 contains array of keys of known type
|
|
|
|
const ColumnArray & array_column = static_cast<const ColumnArray &>(*columns[0]);
|
|
|
|
const IColumn::Offsets_t & offsets = array_column.getOffsets();
|
|
|
|
const auto & keys_vec = static_cast<const ColumnVector<T> &>(array_column.getData());
|
|
|
|
const size_t keys_vec_offset = row_num == 0 ? 0 : offsets[row_num - 1];
|
|
|
|
const size_t keys_vec_size = (offsets[row_num] - keys_vec_offset);
|
|
|
|
|
|
|
|
// Columns 1..n contain arrays of numeric values to sum
|
2017-09-15 11:14:19 +00:00
|
|
|
auto & merged_maps = this->data(place).merged_maps;
|
2017-10-12 04:13:33 +00:00
|
|
|
for (size_t col = 0; col < values_types.size(); ++col)
|
2017-09-15 11:14:19 +00:00
|
|
|
{
|
2017-10-12 04:13:33 +00:00
|
|
|
Field value;
|
|
|
|
const ColumnArray & array_column = static_cast<const ColumnArray &>(*columns[col + 1]);
|
|
|
|
const IColumn::Offsets_t & offsets = array_column.getOffsets();
|
|
|
|
const size_t values_vec_offset = row_num == 0 ? 0 : 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)
|
|
|
|
throw Exception("Sizes of keys and values arrays do not match", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
// Insert column values for all keys
|
|
|
|
for (size_t i = 0; i < keys_vec_size; ++i)
|
|
|
|
{
|
|
|
|
array_column.getData().get(values_vec_offset + i, value);
|
|
|
|
const auto & key = keys_vec.getData()[keys_vec_offset + i];
|
|
|
|
const auto & it = merged_maps.find(key);
|
|
|
|
|
|
|
|
if (it != merged_maps.end())
|
|
|
|
applyVisitor(FieldVisitorSum(value), it->second[col]);
|
|
|
|
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)
|
|
|
|
new_values[k] = (k == col) ? value : values_types[k]->getDefault();
|
|
|
|
|
|
|
|
merged_maps[key] = std::move(new_values);
|
|
|
|
}
|
|
|
|
}
|
2017-09-15 11:14:19 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override
|
|
|
|
{
|
|
|
|
auto & merged_maps = this->data(place).merged_maps;
|
|
|
|
const auto & rhs_maps = this->data(rhs).merged_maps;
|
|
|
|
|
2017-10-12 04:13:33 +00:00
|
|
|
for (const auto & elem : rhs_maps)
|
2017-09-15 11:14:19 +00:00
|
|
|
{
|
2017-10-12 04:13:33 +00:00
|
|
|
const auto & it = merged_maps.find(elem.first);
|
|
|
|
if (it != merged_maps.end())
|
|
|
|
{
|
|
|
|
for (size_t col = 0; col < values_types.size(); ++col)
|
|
|
|
applyVisitor(FieldVisitorSum(elem.second[col]), it->second[col]);
|
|
|
|
}
|
2017-09-15 11:14:19 +00:00
|
|
|
else
|
2017-10-12 04:13:33 +00:00
|
|
|
merged_maps[elem.first] = elem.second;
|
2017-09-15 11:14:19 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
|
|
|
|
{
|
|
|
|
const auto & merged_maps = this->data(place).merged_maps;
|
|
|
|
size_t size = merged_maps.size();
|
2017-09-18 17:11:16 +00:00
|
|
|
writeVarUInt(size, buf);
|
2017-09-15 11:14:19 +00:00
|
|
|
|
2017-10-12 04:13:33 +00:00
|
|
|
for (const auto & elem : merged_maps)
|
2017-09-15 11:14:19 +00:00
|
|
|
{
|
2017-10-12 04:13:33 +00:00
|
|
|
keys_type->serializeBinary(elem.first, buf);
|
|
|
|
for (size_t col = 0; col < values_types.size(); ++col)
|
|
|
|
values_types[col]->serializeBinary(elem.second[col], buf);
|
2017-09-15 11:14:19 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
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)
|
|
|
|
{
|
2017-10-12 04:13:33 +00:00
|
|
|
Field key;
|
2017-09-18 17:11:16 +00:00
|
|
|
keys_type->deserializeBinary(key, buf);
|
2017-10-12 04:13:33 +00:00
|
|
|
|
|
|
|
Array values;
|
|
|
|
values.resize(values_types.size());
|
|
|
|
for (size_t col = 0; col < values_types.size(); ++col)
|
|
|
|
values_types[col]->deserializeBinary(values[col], buf);
|
|
|
|
|
|
|
|
merged_maps[key.get<T>()] = values;
|
2017-09-15 11:14:19 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
|
|
|
|
{
|
2017-10-12 04:13:33 +00:00
|
|
|
const auto & merged_maps = this->data(place).merged_maps;
|
|
|
|
size_t size = merged_maps.size();
|
2017-09-19 12:35:25 +00:00
|
|
|
|
2017-10-12 04:13:33 +00:00
|
|
|
auto & to_cols = static_cast<ColumnTuple &>(to).getColumns();
|
2017-09-19 12:35:25 +00:00
|
|
|
auto & to_keys_arr = static_cast<ColumnArray &>(*to_cols[0]);
|
|
|
|
auto & to_keys_col = to_keys_arr.getData();
|
|
|
|
|
2017-10-12 04:13:33 +00:00
|
|
|
// Advance column offsets
|
|
|
|
auto & to_keys_offsets = to_keys_arr.getOffsets();
|
|
|
|
to_keys_offsets.push_back((to_keys_offsets.empty() ? 0 : to_keys_offsets.back()) + size);
|
|
|
|
to_keys_col.reserve(size);
|
2017-09-15 11:14:19 +00:00
|
|
|
|
2017-10-12 04:13:33 +00:00
|
|
|
for (size_t col = 0; col < values_types.size(); ++col)
|
|
|
|
{
|
|
|
|
auto & to_values_arr = static_cast<ColumnArray &>(*to_cols[col + 1]);
|
|
|
|
auto & to_values_offsets = to_values_arr.getOffsets();
|
|
|
|
to_values_offsets.push_back((to_values_offsets.empty() ? 0 : to_values_offsets.back()) + size);
|
|
|
|
to_values_arr.getData().reserve(size);
|
|
|
|
}
|
2017-09-15 11:14:19 +00:00
|
|
|
|
2017-10-12 04:13:33 +00:00
|
|
|
// Write arrays of keys and values
|
|
|
|
for (const auto & elem : merged_maps)
|
2017-09-15 11:14:19 +00:00
|
|
|
{
|
2017-10-12 04:13:33 +00:00
|
|
|
// 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 = static_cast<ColumnArray &>(*to_cols[col + 1]).getData();
|
|
|
|
to_values_col.insert(elem.second[col]);
|
|
|
|
}
|
2017-09-15 11:14:19 +00:00
|
|
|
}
|
2017-10-12 04:13:33 +00:00
|
|
|
}
|
2017-09-15 11:14:19 +00:00
|
|
|
|
2017-10-12 04:13:33 +00:00
|
|
|
static void addFree(const IAggregateFunction * that, AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena)
|
|
|
|
{
|
|
|
|
static_cast<const AggregateFunctionSumMap &>(*that).add(place, columns, row_num, arena);
|
2017-09-15 11:14:19 +00:00
|
|
|
}
|
2017-09-18 18:37:44 +00:00
|
|
|
|
2017-10-12 04:13:33 +00:00
|
|
|
IAggregateFunction::AddFunc getAddressOfAddFunction() const override final { return &addFree; }
|
|
|
|
|
2017-09-18 18:37:44 +00:00
|
|
|
const char * getHeaderFilePath() const override { return __FILE__; }
|
2017-09-15 11:14:19 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|