2017-04-01 09:19:00 +00:00
|
|
|
#include <Columns/ColumnAggregateFunction.h>
|
2019-02-10 17:40:52 +00:00
|
|
|
#include <Columns/ColumnsCommon.h>
|
2019-08-21 02:28:04 +00:00
|
|
|
#include <Common/assert_cast.h>
|
2017-07-06 13:54:55 +00:00
|
|
|
#include <AggregateFunctions/AggregateFunctionState.h>
|
|
|
|
#include <DataStreams/ColumnGathererStream.h>
|
2017-12-06 03:10:32 +00:00
|
|
|
#include <IO/WriteBufferFromArena.h>
|
2019-02-11 13:11:52 +00:00
|
|
|
#include <IO/WriteBufferFromString.h>
|
|
|
|
#include <IO/Operators.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Common/SipHash.h>
|
2018-09-01 04:25:03 +00:00
|
|
|
#include <Common/AlignedBuffer.h>
|
2017-07-13 20:58:19 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2018-09-02 05:23:20 +00:00
|
|
|
#include <Common/Arena.h>
|
2020-03-18 16:03:55 +00:00
|
|
|
#include <Common/WeakHash.h>
|
2020-03-18 16:46:07 +00:00
|
|
|
#include <Common/HashTable/Hash.h>
|
2015-11-11 01:28:34 +00:00
|
|
|
|
2019-01-22 22:18:07 +00:00
|
|
|
#include <AggregateFunctions/AggregateFunctionMLMethod.h>
|
2018-09-01 04:25:03 +00:00
|
|
|
|
2015-11-11 01:28:34 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
2018-09-01 04:25:03 +00:00
|
|
|
|
2017-03-11 00:27:59 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2020-02-25 18:02:41 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int PARAMETER_OUT_OF_BOUND;
|
|
|
|
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
|
2019-01-23 01:29:53 +00:00
|
|
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
ColumnAggregateFunction::~ColumnAggregateFunction()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!func->hasTrivialDestructor() && !src)
|
2020-04-22 06:22:14 +00:00
|
|
|
for (auto * val : data)
|
2017-04-01 07:20:54 +00:00
|
|
|
func->destroy(val);
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
|
|
|
|
2019-07-16 18:29:17 +00:00
|
|
|
void ColumnAggregateFunction::addArena(ConstArenaPtr arena_)
|
2017-03-11 00:27:59 +00:00
|
|
|
{
|
2019-07-16 18:29:17 +00:00
|
|
|
foreign_arenas.push_back(arena_);
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
2015-11-11 01:28:34 +00:00
|
|
|
|
2020-05-14 07:59:14 +00:00
|
|
|
MutableColumnPtr ColumnAggregateFunction::convertToValues(MutableColumnPtr column)
|
2015-11-11 01:28:34 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
/** If the aggregate function returns an unfinalized/unfinished state,
|
|
|
|
* then you just need to copy pointers to it and also shared ownership of data.
|
|
|
|
*
|
|
|
|
* Also replace the aggregate function with the nested function.
|
|
|
|
* That is, if this column is the states of the aggregate function `aggState`,
|
|
|
|
* then we return the same column, but with the states of the aggregate function `agg`.
|
|
|
|
* These are the same states, changing only the function to which they correspond.
|
|
|
|
*
|
|
|
|
* Further is quite difficult to understand.
|
|
|
|
* Example when this happens:
|
|
|
|
*
|
|
|
|
* SELECT k, finalizeAggregation(quantileTimingState(0.5)(x)) FROM ... GROUP BY k WITH TOTALS
|
|
|
|
*
|
|
|
|
* This calculates the aggregate function `quantileTimingState`.
|
|
|
|
* Its return type AggregateFunction(quantileTiming(0.5), UInt64)`.
|
|
|
|
* Due to the presence of WITH TOTALS, during aggregation the states of this aggregate function will be stored
|
|
|
|
* in the ColumnAggregateFunction column of type
|
|
|
|
* AggregateFunction(quantileTimingState(0.5), UInt64).
|
|
|
|
* Then, in `TotalsHavingBlockInputStream`, it will be called `convertToValues` method,
|
|
|
|
* to get the "ready" values.
|
|
|
|
* But it just converts a column of type
|
|
|
|
* `AggregateFunction(quantileTimingState(0.5), UInt64)`
|
|
|
|
* into `AggregateFunction(quantileTiming(0.5), UInt64)`
|
|
|
|
* - in the same states.
|
2020-05-14 07:59:14 +00:00
|
|
|
*column_aggregate_func
|
2017-04-01 07:20:54 +00:00
|
|
|
* Then `finalizeAggregation` function will be calculated, which will call `convertToValues` already on the result.
|
|
|
|
* And this converts a column of type
|
|
|
|
* AggregateFunction(quantileTiming(0.5), UInt64)
|
|
|
|
* into UInt16 - already finished result of `quantileTiming`.
|
|
|
|
*/
|
2020-05-14 07:59:14 +00:00
|
|
|
auto & column_aggregate_func = assert_cast<ColumnAggregateFunction &>(*column);
|
|
|
|
auto & func = column_aggregate_func.func;
|
|
|
|
auto & data = column_aggregate_func.data;
|
|
|
|
|
2019-05-27 20:14:23 +00:00
|
|
|
if (const AggregateFunctionState *function_state = typeid_cast<const AggregateFunctionState *>(func.get()))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-05-14 07:59:14 +00:00
|
|
|
auto res = column_aggregate_func.createView();
|
2019-05-27 20:14:23 +00:00
|
|
|
res->set(function_state->getNestedFunction());
|
|
|
|
res->data.assign(data.begin(), data.end());
|
Get rid of useless std::move to get NRVO
http://eel.is/c++draft/class.copy.elision#:constructor,copy,elision
Some quote:
> Speaking of RVO, return std::move(w); prohibits it. It means "use move constructor or fail to compile", whereas return w; means "use RVO, and if you can't, use move constructor, and if you can't, use copy constructor, and if you can't, fail to compile."
There is one exception to this rule:
```cpp
Block FilterBlockInputStream::removeFilterIfNeed(Block && block)
{
if (block && remove_filter)
block.erase(static_cast<size_t>(filter_column));
return std::move(block);
}
```
because references are not eligible for NRVO, which is another rule "always move rvalue references and forward universal references" that takes precedence.
2018-08-27 14:04:22 +00:00
|
|
|
return res;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2020-05-14 07:59:14 +00:00
|
|
|
column_aggregate_func.ensureOwnership();
|
|
|
|
|
2019-05-27 20:14:23 +00:00
|
|
|
MutableColumnPtr res = func->getReturnType()->createColumn();
|
|
|
|
res->reserve(data.size());
|
|
|
|
|
2020-04-22 06:22:14 +00:00
|
|
|
for (auto * val : data)
|
2018-09-09 01:16:34 +00:00
|
|
|
func->insertResultInto(val, *res);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-15 19:46:24 +00:00
|
|
|
return res;
|
2015-11-11 01:28:34 +00:00
|
|
|
}
|
|
|
|
|
2019-04-21 14:32:42 +00:00
|
|
|
MutableColumnPtr ColumnAggregateFunction::predictValues(Block & block, const ColumnNumbers & arguments, const Context & context) const
|
2019-01-22 21:07:05 +00:00
|
|
|
{
|
2019-05-27 20:14:23 +00:00
|
|
|
MutableColumnPtr res = func->getReturnTypeToPredict()->createColumn();
|
|
|
|
res->reserve(data.size());
|
2019-01-22 21:07:05 +00:00
|
|
|
|
2020-04-22 06:22:14 +00:00
|
|
|
auto * machine_learning_function = func.get();
|
2020-03-23 02:12:31 +00:00
|
|
|
if (machine_learning_function)
|
2019-01-22 21:07:05 +00:00
|
|
|
{
|
2019-06-03 05:11:15 +00:00
|
|
|
if (data.size() == 1)
|
2019-02-10 22:07:47 +00:00
|
|
|
{
|
2019-06-03 05:11:15 +00:00
|
|
|
/// Case for const column. Predict using single model.
|
2020-03-23 02:12:31 +00:00
|
|
|
machine_learning_function->predictValues(data[0], *res, block, 0, block.rows(), arguments, context);
|
2019-06-03 05:11:15 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
/// Case for non-constant column. Use different aggregate function for each row.
|
|
|
|
size_t row_num = 0;
|
2020-04-22 06:22:14 +00:00
|
|
|
for (auto * val : data)
|
2019-06-03 05:11:15 +00:00
|
|
|
{
|
2020-03-23 02:12:31 +00:00
|
|
|
machine_learning_function->predictValues(val, *res, block, row_num, 1, arguments, context);
|
2019-06-03 05:11:15 +00:00
|
|
|
++row_num;
|
|
|
|
}
|
2019-01-22 21:07:05 +00:00
|
|
|
}
|
2019-04-15 00:16:13 +00:00
|
|
|
}
|
|
|
|
else
|
2019-01-28 10:39:57 +00:00
|
|
|
{
|
2019-01-23 01:29:53 +00:00
|
|
|
throw Exception("Illegal aggregate function is passed",
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2019-01-22 21:07:05 +00:00
|
|
|
}
|
|
|
|
return res;
|
|
|
|
}
|
2017-01-20 05:00:04 +00:00
|
|
|
|
2018-09-09 01:16:34 +00:00
|
|
|
void ColumnAggregateFunction::ensureOwnership()
|
|
|
|
{
|
|
|
|
if (src)
|
|
|
|
{
|
|
|
|
/// We must copy all data from src and take ownership.
|
|
|
|
size_t size = data.size();
|
|
|
|
|
|
|
|
Arena & arena = createOrGetArena();
|
|
|
|
size_t size_of_state = func->sizeOfData();
|
|
|
|
size_t align_of_state = func->alignOfData();
|
|
|
|
|
|
|
|
size_t rollback_pos = 0;
|
|
|
|
try
|
|
|
|
{
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
{
|
|
|
|
ConstAggregateDataPtr old_place = data[i];
|
|
|
|
data[i] = arena.alignedAlloc(size_of_state, align_of_state);
|
|
|
|
func->create(data[i]);
|
|
|
|
++rollback_pos;
|
|
|
|
func->merge(data[i], old_place, &arena);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
/// If we failed to take ownership, destroy all temporary data.
|
|
|
|
|
|
|
|
if (!func->hasTrivialDestructor())
|
|
|
|
for (size_t i = 0; i < rollback_pos; ++i)
|
|
|
|
func->destroy(data[i]);
|
|
|
|
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Now we own all data.
|
|
|
|
src.reset();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-02-26 18:32:15 +00:00
|
|
|
bool ColumnAggregateFunction::structureEquals(const IColumn & to) const
|
|
|
|
{
|
|
|
|
const auto * to_concrete = typeid_cast<const ColumnAggregateFunction *>(&to);
|
|
|
|
if (!to_concrete)
|
|
|
|
return false;
|
|
|
|
|
|
|
|
/// AggregateFunctions must be the same.
|
|
|
|
|
|
|
|
const IAggregateFunction & func_this = *func;
|
|
|
|
const IAggregateFunction & func_to = *to_concrete->func;
|
|
|
|
|
|
|
|
return typeid(func_this) == typeid(func_to);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-01-20 05:00:04 +00:00
|
|
|
void ColumnAggregateFunction::insertRangeFrom(const IColumn & from, size_t start, size_t length)
|
|
|
|
{
|
2019-08-21 02:28:04 +00:00
|
|
|
const ColumnAggregateFunction & from_concrete = assert_cast<const ColumnAggregateFunction &>(from);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-09-09 01:16:34 +00:00
|
|
|
if (start + length > from_concrete.data.size())
|
2017-04-01 07:20:54 +00:00
|
|
|
throw Exception("Parameters start = " + toString(start) + ", length = " + toString(length)
|
|
|
|
+ " are out of bound in ColumnAggregateFunction::insertRangeFrom method"
|
|
|
|
" (data.size() = "
|
2018-09-09 01:16:34 +00:00
|
|
|
+ toString(from_concrete.data.size())
|
2017-04-01 07:20:54 +00:00
|
|
|
+ ").",
|
|
|
|
ErrorCodes::PARAMETER_OUT_OF_BOUND);
|
|
|
|
|
|
|
|
if (!empty() && src.get() != &from_concrete)
|
|
|
|
{
|
|
|
|
/// Must create new states of aggregate function and take ownership of it,
|
|
|
|
/// because ownership of states of aggregate function cannot be shared for individual rows,
|
|
|
|
/// (only as a whole).
|
|
|
|
|
|
|
|
size_t end = start + length;
|
|
|
|
for (size_t i = start; i < end; ++i)
|
|
|
|
insertFrom(from, i);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
/// Keep shared ownership of aggregation states.
|
2017-12-15 19:17:15 +00:00
|
|
|
src = from_concrete.getPtr();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
size_t old_size = data.size();
|
|
|
|
data.resize(old_size + length);
|
2019-03-07 20:04:59 +00:00
|
|
|
memcpy(data.data() + old_size, &from_concrete.data[start], length * sizeof(data[0]));
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2017-01-20 05:00:04 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-03-20 14:17:09 +00:00
|
|
|
ColumnPtr ColumnAggregateFunction::filter(const Filter & filter, ssize_t result_size_hint) const
|
2017-01-20 05:00:04 +00:00
|
|
|
{
|
2018-09-09 01:16:34 +00:00
|
|
|
size_t size = data.size();
|
2017-04-01 07:20:54 +00:00
|
|
|
if (size != filter.size())
|
|
|
|
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
2017-01-20 05:00:04 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (size == 0)
|
2017-12-15 19:17:15 +00:00
|
|
|
return cloneEmpty();
|
2017-01-20 05:00:04 +00:00
|
|
|
|
2017-12-15 19:17:15 +00:00
|
|
|
auto res = createView();
|
2018-09-09 01:16:34 +00:00
|
|
|
auto & res_data = res->data;
|
2017-01-20 05:00:04 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (result_size_hint)
|
|
|
|
res_data.reserve(result_size_hint > 0 ? result_size_hint : size);
|
2017-01-20 05:00:04 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
if (filter[i])
|
2018-09-09 01:16:34 +00:00
|
|
|
res_data.push_back(data[i]);
|
2017-01-20 05:00:04 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// To save RAM in case of too strong filtering.
|
|
|
|
if (res_data.size() * 2 < res_data.capacity())
|
2017-12-15 21:32:25 +00:00
|
|
|
res_data = Container(res_data.cbegin(), res_data.cend());
|
2017-01-20 05:00:04 +00:00
|
|
|
|
Get rid of useless std::move to get NRVO
http://eel.is/c++draft/class.copy.elision#:constructor,copy,elision
Some quote:
> Speaking of RVO, return std::move(w); prohibits it. It means "use move constructor or fail to compile", whereas return w; means "use RVO, and if you can't, use move constructor, and if you can't, use copy constructor, and if you can't, fail to compile."
There is one exception to this rule:
```cpp
Block FilterBlockInputStream::removeFilterIfNeed(Block && block)
{
if (block && remove_filter)
block.erase(static_cast<size_t>(filter_column));
return std::move(block);
}
```
because references are not eligible for NRVO, which is another rule "always move rvalue references and forward universal references" that takes precedence.
2018-08-27 14:04:22 +00:00
|
|
|
return res;
|
2017-01-20 05:00:04 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-02-18 19:44:26 +00:00
|
|
|
ColumnPtr ColumnAggregateFunction::permute(const Permutation & perm, size_t limit) const
|
2017-01-20 05:00:04 +00:00
|
|
|
{
|
2018-09-09 01:16:34 +00:00
|
|
|
size_t size = data.size();
|
2017-01-20 05:00:04 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (limit == 0)
|
|
|
|
limit = size;
|
|
|
|
else
|
|
|
|
limit = std::min(size, limit);
|
2017-01-20 05:00:04 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (perm.size() < limit)
|
|
|
|
throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
2017-01-20 05:00:04 +00:00
|
|
|
|
2017-12-13 19:14:12 +00:00
|
|
|
auto res = createView();
|
2017-01-20 05:00:04 +00:00
|
|
|
|
2018-09-09 01:16:34 +00:00
|
|
|
res->data.resize(limit);
|
2017-04-01 07:20:54 +00:00
|
|
|
for (size_t i = 0; i < limit; ++i)
|
2018-09-09 01:16:34 +00:00
|
|
|
res->data[i] = data[perm[i]];
|
2017-01-20 05:00:04 +00:00
|
|
|
|
Get rid of useless std::move to get NRVO
http://eel.is/c++draft/class.copy.elision#:constructor,copy,elision
Some quote:
> Speaking of RVO, return std::move(w); prohibits it. It means "use move constructor or fail to compile", whereas return w; means "use RVO, and if you can't, use move constructor, and if you can't, use copy constructor, and if you can't, fail to compile."
There is one exception to this rule:
```cpp
Block FilterBlockInputStream::removeFilterIfNeed(Block && block)
{
if (block && remove_filter)
block.erase(static_cast<size_t>(filter_column));
return std::move(block);
}
```
because references are not eligible for NRVO, which is another rule "always move rvalue references and forward universal references" that takes precedence.
2018-08-27 14:04:22 +00:00
|
|
|
return res;
|
2017-01-20 05:00:04 +00:00
|
|
|
}
|
|
|
|
|
2019-02-18 17:28:53 +00:00
|
|
|
ColumnPtr ColumnAggregateFunction::index(const IColumn & indexes, size_t limit) const
|
2018-04-18 21:00:47 +00:00
|
|
|
{
|
|
|
|
return selectIndexImpl(*this, indexes, limit);
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename Type>
|
2019-02-18 17:28:53 +00:00
|
|
|
ColumnPtr ColumnAggregateFunction::indexImpl(const PaddedPODArray<Type> & indexes, size_t limit) const
|
2018-04-18 21:00:47 +00:00
|
|
|
{
|
|
|
|
auto res = createView();
|
|
|
|
|
2018-09-09 01:16:34 +00:00
|
|
|
res->data.resize(limit);
|
2018-04-18 21:00:47 +00:00
|
|
|
for (size_t i = 0; i < limit; ++i)
|
2018-09-09 01:16:34 +00:00
|
|
|
res->data[i] = data[indexes[i]];
|
2018-04-18 21:00:47 +00:00
|
|
|
|
Get rid of useless std::move to get NRVO
http://eel.is/c++draft/class.copy.elision#:constructor,copy,elision
Some quote:
> Speaking of RVO, return std::move(w); prohibits it. It means "use move constructor or fail to compile", whereas return w; means "use RVO, and if you can't, use move constructor, and if you can't, use copy constructor, and if you can't, fail to compile."
There is one exception to this rule:
```cpp
Block FilterBlockInputStream::removeFilterIfNeed(Block && block)
{
if (block && remove_filter)
block.erase(static_cast<size_t>(filter_column));
return std::move(block);
}
```
because references are not eligible for NRVO, which is another rule "always move rvalue references and forward universal references" that takes precedence.
2018-08-27 14:04:22 +00:00
|
|
|
return res;
|
2018-04-18 21:00:47 +00:00
|
|
|
}
|
|
|
|
|
2018-08-26 00:44:23 +00:00
|
|
|
INSTANTIATE_INDEX_IMPL(ColumnAggregateFunction)
|
2018-04-18 21:00:47 +00:00
|
|
|
|
2017-02-16 22:05:48 +00:00
|
|
|
/// Is required to support operations with Set
|
|
|
|
void ColumnAggregateFunction::updateHashWithValue(size_t n, SipHash & hash) const
|
|
|
|
{
|
2017-07-31 21:39:24 +00:00
|
|
|
WriteBufferFromOwnString wbuf;
|
2018-09-09 01:16:34 +00:00
|
|
|
func->serialize(data[n], wbuf);
|
2017-07-31 21:39:24 +00:00
|
|
|
hash.update(wbuf.str().c_str(), wbuf.str().size());
|
2017-02-16 22:05:48 +00:00
|
|
|
}
|
2017-01-27 12:38:59 +00:00
|
|
|
|
2020-03-18 16:03:55 +00:00
|
|
|
void ColumnAggregateFunction::updateWeakHash32(WeakHash32 & hash) const
|
|
|
|
{
|
|
|
|
auto s = data.size();
|
|
|
|
if (hash.getData().size() != data.size())
|
|
|
|
throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) +
|
|
|
|
", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
auto & hash_data = hash.getData();
|
|
|
|
|
2020-03-25 11:14:11 +00:00
|
|
|
std::vector<UInt8> v;
|
2020-03-18 16:03:55 +00:00
|
|
|
for (size_t i = 0; i < s; ++i)
|
|
|
|
{
|
2020-03-18 16:46:07 +00:00
|
|
|
WriteBufferFromVector<std::vector<UInt8>> wbuf(v);
|
2020-03-18 16:03:55 +00:00
|
|
|
func->serialize(data[i], wbuf);
|
2020-03-18 16:46:07 +00:00
|
|
|
wbuf.finalize();
|
|
|
|
hash_data[i] = ::updateWeakHash32(v.data(), v.size(), hash_data[i]);
|
2020-03-18 16:03:55 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-07-16 18:29:17 +00:00
|
|
|
/// The returned size is less than real size. The reason is that some parts of
|
|
|
|
/// aggregate function data may be allocated on shared arenas. These arenas are
|
|
|
|
/// used for several blocks, and also may be updated concurrently from other
|
|
|
|
/// threads, so we can't know the size of these data.
|
2017-01-27 12:38:59 +00:00
|
|
|
size_t ColumnAggregateFunction::byteSize() const
|
|
|
|
{
|
2019-07-16 18:29:17 +00:00
|
|
|
return data.size() * sizeof(data[0])
|
|
|
|
+ (my_arena ? my_arena->size() : 0);
|
2017-01-27 12:38:59 +00:00
|
|
|
}
|
|
|
|
|
2019-07-16 18:29:17 +00:00
|
|
|
/// Like in byteSize(), the size is underestimated.
|
2017-07-13 16:49:09 +00:00
|
|
|
size_t ColumnAggregateFunction::allocatedBytes() const
|
2017-01-27 12:38:59 +00:00
|
|
|
{
|
2019-07-16 18:29:17 +00:00
|
|
|
return data.allocated_bytes()
|
|
|
|
+ (my_arena ? my_arena->size() : 0);
|
2017-01-27 12:38:59 +00:00
|
|
|
}
|
2017-06-16 16:23:23 +00:00
|
|
|
|
2019-03-10 03:16:51 +00:00
|
|
|
void ColumnAggregateFunction::protect()
|
|
|
|
{
|
|
|
|
data.protect();
|
|
|
|
}
|
|
|
|
|
2017-12-13 19:14:12 +00:00
|
|
|
MutableColumnPtr ColumnAggregateFunction::cloneEmpty() const
|
2017-03-11 00:27:59 +00:00
|
|
|
{
|
2019-07-16 18:29:17 +00:00
|
|
|
return create(func);
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
2017-06-16 16:23:23 +00:00
|
|
|
|
2019-02-11 13:11:52 +00:00
|
|
|
String ColumnAggregateFunction::getTypeString() const
|
|
|
|
{
|
|
|
|
return DataTypeAggregateFunction(func, func->getArgumentTypes(), func->getParameters()).getName();
|
|
|
|
}
|
|
|
|
|
2017-03-11 00:27:59 +00:00
|
|
|
Field ColumnAggregateFunction::operator[](size_t n) const
|
|
|
|
{
|
2019-02-11 11:19:56 +00:00
|
|
|
Field field = AggregateFunctionStateData();
|
2019-02-11 13:11:52 +00:00
|
|
|
field.get<AggregateFunctionStateData &>().name = getTypeString();
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-02-11 13:11:52 +00:00
|
|
|
WriteBufferFromString buffer(field.get<AggregateFunctionStateData &>().data);
|
2018-09-09 01:16:34 +00:00
|
|
|
func->serialize(data[n], buffer);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
return field;
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
2017-06-16 16:23:23 +00:00
|
|
|
|
2017-03-11 00:27:59 +00:00
|
|
|
void ColumnAggregateFunction::get(size_t n, Field & res) const
|
|
|
|
{
|
2019-02-11 13:11:52 +00:00
|
|
|
res = AggregateFunctionStateData();
|
|
|
|
res.get<AggregateFunctionStateData &>().name = getTypeString();
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-02-11 13:11:52 +00:00
|
|
|
WriteBufferFromString buffer(res.get<AggregateFunctionStateData &>().data);
|
2018-09-09 01:16:34 +00:00
|
|
|
func->serialize(data[n], buffer);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
2017-06-16 16:23:23 +00:00
|
|
|
|
2017-03-11 00:27:59 +00:00
|
|
|
StringRef ColumnAggregateFunction::getDataAt(size_t n) const
|
|
|
|
{
|
2018-09-09 01:16:34 +00:00
|
|
|
return StringRef(reinterpret_cast<const char *>(&data[n]), sizeof(data[n]));
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
2017-06-16 16:23:23 +00:00
|
|
|
|
2017-12-01 21:13:25 +00:00
|
|
|
void ColumnAggregateFunction::insertData(const char * pos, size_t /*length*/)
|
2017-03-11 00:27:59 +00:00
|
|
|
{
|
2018-09-09 01:16:34 +00:00
|
|
|
ensureOwnership();
|
|
|
|
data.push_back(*reinterpret_cast<const AggregateDataPtr *>(pos));
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
2017-06-16 16:23:23 +00:00
|
|
|
|
2018-08-27 18:16:32 +00:00
|
|
|
void ColumnAggregateFunction::insertFrom(const IColumn & from, size_t n)
|
2017-03-11 00:27:59 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Must create new state of aggregate function and take ownership of it,
|
|
|
|
/// because ownership of states of aggregate function cannot be shared for individual rows,
|
|
|
|
/// (only as a whole, see comment above).
|
2018-09-09 01:16:34 +00:00
|
|
|
ensureOwnership();
|
2017-04-01 07:20:54 +00:00
|
|
|
insertDefault();
|
2018-08-27 18:16:32 +00:00
|
|
|
insertMergeFrom(from, n);
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
2017-06-16 16:23:23 +00:00
|
|
|
|
2017-03-11 00:27:59 +00:00
|
|
|
void ColumnAggregateFunction::insertFrom(ConstAggregateDataPtr place)
|
|
|
|
{
|
2018-09-09 01:16:34 +00:00
|
|
|
ensureOwnership();
|
2017-04-01 07:20:54 +00:00
|
|
|
insertDefault();
|
|
|
|
insertMergeFrom(place);
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
2017-06-16 16:23:23 +00:00
|
|
|
|
2017-03-11 00:27:59 +00:00
|
|
|
void ColumnAggregateFunction::insertMergeFrom(ConstAggregateDataPtr place)
|
|
|
|
{
|
2018-09-09 01:16:34 +00:00
|
|
|
func->merge(data.back(), place, &createOrGetArena());
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
2017-06-16 16:23:23 +00:00
|
|
|
|
2018-08-27 18:16:32 +00:00
|
|
|
void ColumnAggregateFunction::insertMergeFrom(const IColumn & from, size_t n)
|
2017-03-11 00:27:59 +00:00
|
|
|
{
|
2019-08-21 02:28:04 +00:00
|
|
|
insertMergeFrom(assert_cast<const ColumnAggregateFunction &>(from).data[n]);
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
2017-06-16 16:23:23 +00:00
|
|
|
|
2017-03-11 00:27:59 +00:00
|
|
|
Arena & ColumnAggregateFunction::createOrGetArena()
|
|
|
|
{
|
2019-07-16 18:29:17 +00:00
|
|
|
if (unlikely(!my_arena))
|
|
|
|
my_arena = std::make_shared<Arena>();
|
|
|
|
|
|
|
|
return *my_arena.get();
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
2017-06-16 16:23:23 +00:00
|
|
|
|
2018-09-09 01:16:34 +00:00
|
|
|
|
|
|
|
static void pushBackAndCreateState(ColumnAggregateFunction::Container & data, Arena & arena, IAggregateFunction * func)
|
2017-03-11 00:27:59 +00:00
|
|
|
{
|
2018-09-09 01:16:34 +00:00
|
|
|
data.push_back(arena.alignedAlloc(func->sizeOfData(), func->alignOfData()));
|
|
|
|
try
|
|
|
|
{
|
|
|
|
func->create(data.back());
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
data.pop_back();
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
2017-03-11 00:27:59 +00:00
|
|
|
|
2018-09-09 01:16:34 +00:00
|
|
|
void ColumnAggregateFunction::insert(const Field & x)
|
|
|
|
{
|
2019-02-11 13:11:52 +00:00
|
|
|
String type_string = getTypeString();
|
2019-02-12 10:09:03 +00:00
|
|
|
|
|
|
|
if (x.getType() != Field::Types::AggregateFunctionState)
|
|
|
|
throw Exception(String("Inserting field of type ") + x.getTypeName() + " into ColumnAggregateFunction. "
|
|
|
|
"Expected " + Field::Types::toString(Field::Types::AggregateFunctionState), ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
2020-04-22 06:22:14 +00:00
|
|
|
const auto & field_name = x.get<const AggregateFunctionStateData &>().name;
|
2019-02-11 13:11:52 +00:00
|
|
|
if (type_string != field_name)
|
|
|
|
throw Exception("Cannot insert filed with type " + field_name + " into column with type " + type_string,
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
2018-09-09 01:16:34 +00:00
|
|
|
ensureOwnership();
|
|
|
|
Arena & arena = createOrGetArena();
|
|
|
|
pushBackAndCreateState(data, arena, func.get());
|
2019-02-11 13:11:52 +00:00
|
|
|
ReadBufferFromString read_buffer(x.get<const AggregateFunctionStateData &>().data);
|
2018-09-09 01:16:34 +00:00
|
|
|
func->deserialize(data.back(), read_buffer, &arena);
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
2017-06-16 16:23:23 +00:00
|
|
|
|
2017-03-11 00:27:59 +00:00
|
|
|
void ColumnAggregateFunction::insertDefault()
|
|
|
|
{
|
2018-09-09 01:16:34 +00:00
|
|
|
ensureOwnership();
|
2017-04-01 07:20:54 +00:00
|
|
|
Arena & arena = createOrGetArena();
|
2018-09-09 01:16:34 +00:00
|
|
|
pushBackAndCreateState(data, arena, func.get());
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
2017-06-16 16:23:23 +00:00
|
|
|
|
2020-03-09 01:03:43 +00:00
|
|
|
StringRef ColumnAggregateFunction::serializeValueIntoArena(size_t n, Arena & arena, const char *& begin) const
|
2017-03-11 00:27:59 +00:00
|
|
|
{
|
2020-03-09 01:03:43 +00:00
|
|
|
WriteBufferFromArena out(arena, begin);
|
2018-09-09 01:16:34 +00:00
|
|
|
func->serialize(data[n], out);
|
2017-12-06 03:10:32 +00:00
|
|
|
return out.finish();
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
2017-06-16 16:23:23 +00:00
|
|
|
|
2017-12-06 03:10:32 +00:00
|
|
|
const char * ColumnAggregateFunction::deserializeAndInsertFromArena(const char * src_arena)
|
2017-03-11 00:27:59 +00:00
|
|
|
{
|
2018-09-09 01:16:34 +00:00
|
|
|
ensureOwnership();
|
2017-12-06 03:10:32 +00:00
|
|
|
|
|
|
|
/** Parameter "src_arena" points to Arena, from which we will deserialize the state.
|
|
|
|
* And "dst_arena" is another Arena, that aggregate function state will use to store its data.
|
|
|
|
*/
|
|
|
|
Arena & dst_arena = createOrGetArena();
|
2018-09-09 01:16:34 +00:00
|
|
|
pushBackAndCreateState(data, dst_arena, func.get());
|
2017-12-06 03:10:32 +00:00
|
|
|
|
|
|
|
/** We will read from src_arena.
|
|
|
|
* There is no limit for reading - it is assumed, that we can read all that we need after src_arena pointer.
|
2018-10-13 14:33:43 +00:00
|
|
|
* Buf ReadBufferFromMemory requires some bound. We will use arbitrary big enough number, that will not overflow pointer.
|
2017-12-06 03:10:32 +00:00
|
|
|
* NOTE Technically, this is not compatible with C++ standard,
|
|
|
|
* as we cannot legally compare pointers after last element + 1 of some valid memory region.
|
|
|
|
* Probably this will not work under UBSan.
|
|
|
|
*/
|
2018-12-27 00:59:52 +00:00
|
|
|
ReadBufferFromMemory read_buffer(src_arena, std::numeric_limits<char *>::max() - src_arena - 1);
|
2018-09-09 01:16:34 +00:00
|
|
|
func->deserialize(data.back(), read_buffer, &dst_arena);
|
2017-12-06 03:10:32 +00:00
|
|
|
|
|
|
|
return read_buffer.position();
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
2017-06-16 16:23:23 +00:00
|
|
|
|
2017-03-11 00:27:59 +00:00
|
|
|
void ColumnAggregateFunction::popBack(size_t n)
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t size = data.size();
|
|
|
|
size_t new_size = size - n;
|
2017-01-27 12:38:59 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!src)
|
|
|
|
for (size_t i = new_size; i < size; ++i)
|
|
|
|
func->destroy(data[i]);
|
2017-01-27 12:38:59 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
data.resize_assume_reserved(new_size);
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
2017-06-16 16:23:23 +00:00
|
|
|
|
2018-03-20 14:17:09 +00:00
|
|
|
ColumnPtr ColumnAggregateFunction::replicate(const IColumn::Offsets & offsets) const
|
2017-03-11 00:27:59 +00:00
|
|
|
{
|
2017-07-21 06:35:58 +00:00
|
|
|
size_t size = data.size();
|
|
|
|
if (size != offsets.size())
|
|
|
|
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
if (size == 0)
|
2017-12-15 19:17:15 +00:00
|
|
|
return cloneEmpty();
|
2017-07-21 06:35:58 +00:00
|
|
|
|
2017-12-15 19:17:15 +00:00
|
|
|
auto res = createView();
|
2018-09-09 01:16:34 +00:00
|
|
|
auto & res_data = res->data;
|
2017-07-21 06:35:58 +00:00
|
|
|
res_data.reserve(offsets.back());
|
|
|
|
|
2017-12-15 21:32:25 +00:00
|
|
|
IColumn::Offset prev_offset = 0;
|
2017-07-21 06:35:58 +00:00
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
{
|
|
|
|
size_t size_to_replicate = offsets[i] - prev_offset;
|
|
|
|
prev_offset = offsets[i];
|
|
|
|
|
|
|
|
for (size_t j = 0; j < size_to_replicate; ++j)
|
|
|
|
res_data.push_back(data[i]);
|
|
|
|
}
|
|
|
|
|
Get rid of useless std::move to get NRVO
http://eel.is/c++draft/class.copy.elision#:constructor,copy,elision
Some quote:
> Speaking of RVO, return std::move(w); prohibits it. It means "use move constructor or fail to compile", whereas return w; means "use RVO, and if you can't, use move constructor, and if you can't, use copy constructor, and if you can't, fail to compile."
There is one exception to this rule:
```cpp
Block FilterBlockInputStream::removeFilterIfNeed(Block && block)
{
if (block && remove_filter)
block.erase(static_cast<size_t>(filter_column));
return std::move(block);
}
```
because references are not eligible for NRVO, which is another rule "always move rvalue references and forward universal references" that takes precedence.
2018-08-27 14:04:22 +00:00
|
|
|
return res;
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
2017-06-16 16:23:23 +00:00
|
|
|
|
2017-12-13 19:14:12 +00:00
|
|
|
MutableColumns ColumnAggregateFunction::scatter(IColumn::ColumnIndex num_columns, const IColumn::Selector & selector) const
|
2017-03-11 00:27:59 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Columns with scattered values will point to this column as the owner of values.
|
2017-12-13 19:14:12 +00:00
|
|
|
MutableColumns columns(num_columns);
|
2017-04-01 07:20:54 +00:00
|
|
|
for (auto & column : columns)
|
2017-12-13 19:14:12 +00:00
|
|
|
column = createView();
|
2017-01-27 12:38:59 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t num_rows = size();
|
2017-03-11 00:27:59 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-03-18 02:02:24 +00:00
|
|
|
size_t reserve_size = double(num_rows) / num_columns * 1.1; /// 1.1 is just a guess. Better to use n-sigma rule.
|
2017-03-11 00:27:59 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (reserve_size > 1)
|
|
|
|
for (auto & column : columns)
|
|
|
|
column->reserve(reserve_size);
|
|
|
|
}
|
2017-03-11 00:27:59 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
for (size_t i = 0; i < num_rows; ++i)
|
2019-08-21 02:28:04 +00:00
|
|
|
assert_cast<ColumnAggregateFunction &>(*columns[selector[i]]).data.push_back(data[i]);
|
2017-03-11 00:27:59 +00:00
|
|
|
|
2017-12-15 19:46:24 +00:00
|
|
|
return columns;
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
2017-06-16 16:23:23 +00:00
|
|
|
|
2017-12-01 21:13:25 +00:00
|
|
|
void ColumnAggregateFunction::getPermutation(bool /*reverse*/, size_t /*limit*/, int /*nan_direction_hint*/, IColumn::Permutation & res) const
|
2017-03-11 00:27:59 +00:00
|
|
|
{
|
2018-09-09 01:16:34 +00:00
|
|
|
size_t s = data.size();
|
2017-04-01 07:20:54 +00:00
|
|
|
res.resize(s);
|
|
|
|
for (size_t i = 0; i < s; ++i)
|
|
|
|
res[i] = i;
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
2017-06-16 16:23:23 +00:00
|
|
|
|
2017-07-06 13:54:55 +00:00
|
|
|
void ColumnAggregateFunction::gather(ColumnGathererStream & gatherer)
|
|
|
|
{
|
|
|
|
gatherer.gather(*this);
|
|
|
|
}
|
|
|
|
|
2017-03-11 00:27:59 +00:00
|
|
|
void ColumnAggregateFunction::getExtremes(Field & min, Field & max) const
|
|
|
|
{
|
2017-07-21 06:35:58 +00:00
|
|
|
/// Place serialized default values into min/max.
|
|
|
|
|
2018-09-01 04:25:03 +00:00
|
|
|
AlignedBuffer place_buffer(func->sizeOfData(), func->alignOfData());
|
2017-07-21 06:35:58 +00:00
|
|
|
AggregateDataPtr place = place_buffer.data();
|
|
|
|
|
2019-02-11 11:19:56 +00:00
|
|
|
AggregateFunctionStateData serialized;
|
2019-02-11 13:11:52 +00:00
|
|
|
serialized.name = getTypeString();
|
2017-07-21 06:35:58 +00:00
|
|
|
|
|
|
|
func->create(place);
|
|
|
|
try
|
|
|
|
{
|
2019-02-11 13:11:52 +00:00
|
|
|
WriteBufferFromString buffer(serialized.data);
|
2017-07-21 06:35:58 +00:00
|
|
|
func->serialize(place, buffer);
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
func->destroy(place);
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
func->destroy(place);
|
|
|
|
|
|
|
|
min = serialized;
|
|
|
|
max = serialized;
|
2017-03-11 00:27:59 +00:00
|
|
|
}
|
2017-06-16 16:23:23 +00:00
|
|
|
|
2019-07-16 18:29:17 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
|
|
|
|
ConstArenas concatArenas(const ConstArenas & array, ConstArenaPtr arena)
|
|
|
|
{
|
|
|
|
ConstArenas result = array;
|
|
|
|
if (arena)
|
|
|
|
result.push_back(std::move(arena));
|
|
|
|
|
|
|
|
return result;
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
ColumnAggregateFunction::MutablePtr ColumnAggregateFunction::createView() const
|
|
|
|
{
|
|
|
|
auto res = create(func, concatArenas(foreign_arenas, my_arena));
|
|
|
|
res->src = getPtr();
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
|
|
|
ColumnAggregateFunction::ColumnAggregateFunction(const ColumnAggregateFunction & src_)
|
2020-03-18 00:57:00 +00:00
|
|
|
: COWHelper<IColumn, ColumnAggregateFunction>(src_),
|
|
|
|
foreign_arenas(concatArenas(src_.foreign_arenas, src_.my_arena)),
|
|
|
|
func(src_.func), src(src_.getPtr()), data(src_.data.begin(), src_.data.end())
|
2019-07-16 18:29:17 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2015-11-11 01:28:34 +00:00
|
|
|
}
|