ClickHouse/dbms/src/Columns/ColumnAggregateFunction.cpp

585 lines
18 KiB
C++
Raw Normal View History

#include <Columns/ColumnAggregateFunction.h>
2019-02-10 17:40:52 +00:00
#include <Columns/ColumnsCommon.h>
#include <Common/assert_cast.h>
#include <AggregateFunctions/AggregateFunctionState.h>
#include <DataStreams/ColumnGathererStream.h>
#include <IO/WriteBufferFromArena.h>
2019-02-11 13:11:52 +00:00
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <Common/SipHash.h>
#include <Common/AlignedBuffer.h>
2017-07-13 20:58:19 +00:00
#include <Common/typeid_cast.h>
#include <Common/Arena.h>
2019-01-22 22:18:07 +00:00
#include <AggregateFunctions/AggregateFunctionMLMethod.h>
namespace DB
{
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int LOGICAL_ERROR;
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;
}
ColumnAggregateFunction::~ColumnAggregateFunction()
{
if (!func->hasTrivialDestructor() && !src)
for (auto val : data)
func->destroy(val);
}
void ColumnAggregateFunction::addArena(ConstArenaPtr arena_)
{
foreign_arenas.push_back(arena_);
}
MutableColumnPtr ColumnAggregateFunction::convertToValues() const
{
/** 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.
*
* 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`.
*/
if (const AggregateFunctionState *function_state = typeid_cast<const AggregateFunctionState *>(func.get()))
{
auto res = createView();
res->set(function_state->getNestedFunction());
res->data.assign(data.begin(), data.end());
return res;
}
MutableColumnPtr res = func->getReturnType()->createColumn();
res->reserve(data.size());
for (auto val : data)
func->insertResultInto(val, *res);
return res;
}
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
{
MutableColumnPtr res = func->getReturnTypeToPredict()->createColumn();
res->reserve(data.size());
2019-01-22 21:07:05 +00:00
2019-04-20 23:22:42 +00:00
auto ML_function = func.get();
if (ML_function)
2019-01-22 21:07:05 +00:00
{
if (data.size() == 1)
2019-02-10 22:07:47 +00:00
{
/// Case for const column. Predict using single model.
ML_function->predictValues(data[0], *res, block, 0, block.rows(), arguments, context);
}
else
{
/// Case for non-constant column. Use different aggregate function for each row.
size_t row_num = 0;
for (auto val : data)
{
ML_function->predictValues(val, *res, block, row_num, 1, arguments, context);
++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
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();
}
}
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)
{
const ColumnAggregateFunction & from_concrete = assert_cast<const ColumnAggregateFunction &>(from);
if (start + length > from_concrete.data.size())
throw Exception("Parameters start = " + toString(start) + ", length = " + toString(length)
+ " are out of bound in ColumnAggregateFunction::insertRangeFrom method"
" (data.size() = "
+ toString(from_concrete.data.size())
+ ").",
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.
src = from_concrete.getPtr();
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-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
{
size_t size = data.size();
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
if (size == 0)
return cloneEmpty();
2017-01-20 05:00:04 +00:00
auto res = createView();
auto & res_data = res->data;
2017-01-20 05:00:04 +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
for (size_t i = 0; i < size; ++i)
if (filter[i])
res_data.push_back(data[i]);
2017-01-20 05:00:04 +00:00
/// To save RAM in case of too strong filtering.
if (res_data.size() * 2 < res_data.capacity())
res_data = Container(res_data.cbegin(), res_data.cend());
2017-01-20 05:00:04 +00:00
return res;
2017-01-20 05:00:04 +00:00
}
ColumnPtr ColumnAggregateFunction::permute(const Permutation & perm, size_t limit) const
2017-01-20 05:00:04 +00:00
{
size_t size = data.size();
2017-01-20 05:00:04 +00:00
if (limit == 0)
limit = size;
else
limit = std::min(size, limit);
2017-01-20 05:00:04 +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
auto res = createView();
2017-01-20 05:00:04 +00:00
res->data.resize(limit);
for (size_t i = 0; i < limit; ++i)
res->data[i] = data[perm[i]];
2017-01-20 05:00:04 +00:00
return res;
2017-01-20 05:00:04 +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>
ColumnPtr ColumnAggregateFunction::indexImpl(const PaddedPODArray<Type> & indexes, size_t limit) const
2018-04-18 21:00:47 +00:00
{
auto res = createView();
res->data.resize(limit);
2018-04-18 21:00:47 +00:00
for (size_t i = 0; i < limit; ++i)
res->data[i] = data[indexes[i]];
2018-04-18 21:00:47 +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
/// 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;
func->serialize(data[n], wbuf);
2017-07-31 21:39:24 +00:00
hash.update(wbuf.str().c_str(), wbuf.str().size());
}
/// 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.
size_t ColumnAggregateFunction::byteSize() const
{
return data.size() * sizeof(data[0])
+ (my_arena ? my_arena->size() : 0);
}
/// Like in byteSize(), the size is underestimated.
size_t ColumnAggregateFunction::allocatedBytes() const
{
return data.allocated_bytes()
+ (my_arena ? my_arena->size() : 0);
}
2017-06-16 16:23:23 +00:00
void ColumnAggregateFunction::protect()
{
data.protect();
}
MutableColumnPtr ColumnAggregateFunction::cloneEmpty() const
{
return create(func);
}
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();
}
Field ColumnAggregateFunction::operator[](size_t n) const
{
Field field = AggregateFunctionStateData();
2019-02-11 13:11:52 +00:00
field.get<AggregateFunctionStateData &>().name = getTypeString();
{
2019-02-11 13:11:52 +00:00
WriteBufferFromString buffer(field.get<AggregateFunctionStateData &>().data);
func->serialize(data[n], buffer);
}
return field;
}
2017-06-16 16:23:23 +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();
{
2019-02-11 13:11:52 +00:00
WriteBufferFromString buffer(res.get<AggregateFunctionStateData &>().data);
func->serialize(data[n], buffer);
}
}
2017-06-16 16:23:23 +00:00
StringRef ColumnAggregateFunction::getDataAt(size_t n) const
{
return StringRef(reinterpret_cast<const char *>(&data[n]), sizeof(data[n]));
}
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*/)
{
ensureOwnership();
data.push_back(*reinterpret_cast<const AggregateDataPtr *>(pos));
}
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)
{
/// 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).
ensureOwnership();
insertDefault();
2018-08-27 18:16:32 +00:00
insertMergeFrom(from, n);
}
2017-06-16 16:23:23 +00:00
void ColumnAggregateFunction::insertFrom(ConstAggregateDataPtr place)
{
ensureOwnership();
insertDefault();
insertMergeFrom(place);
}
2017-06-16 16:23:23 +00:00
void ColumnAggregateFunction::insertMergeFrom(ConstAggregateDataPtr place)
{
func->merge(data.back(), place, &createOrGetArena());
}
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)
{
insertMergeFrom(assert_cast<const ColumnAggregateFunction &>(from).data[n]);
}
2017-06-16 16:23:23 +00:00
Arena & ColumnAggregateFunction::createOrGetArena()
{
if (unlikely(!my_arena))
my_arena = std::make_shared<Arena>();
return *my_arena.get();
}
2017-06-16 16:23:23 +00:00
static void pushBackAndCreateState(ColumnAggregateFunction::Container & data, Arena & arena, IAggregateFunction * func)
{
data.push_back(arena.alignedAlloc(func->sizeOfData(), func->alignOfData()));
try
{
func->create(data.back());
}
catch (...)
{
data.pop_back();
throw;
}
}
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);
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);
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);
func->deserialize(data.back(), read_buffer, &arena);
}
2017-06-16 16:23:23 +00:00
void ColumnAggregateFunction::insertDefault()
{
ensureOwnership();
Arena & arena = createOrGetArena();
pushBackAndCreateState(data, arena, func.get());
}
2017-06-16 16:23:23 +00:00
StringRef ColumnAggregateFunction::serializeValueIntoArena(size_t n, Arena & dst, const char *& begin) const
{
WriteBufferFromArena out(dst, begin);
func->serialize(data[n], out);
return out.finish();
}
2017-06-16 16:23:23 +00:00
const char * ColumnAggregateFunction::deserializeAndInsertFromArena(const char * src_arena)
{
ensureOwnership();
/** 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();
pushBackAndCreateState(data, dst_arena, func.get());
/** 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.
* 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);
func->deserialize(data.back(), read_buffer, &dst_arena);
return read_buffer.position();
}
2017-06-16 16:23:23 +00:00
void ColumnAggregateFunction::popBack(size_t n)
{
size_t size = data.size();
size_t new_size = size - n;
if (!src)
for (size_t i = new_size; i < size; ++i)
func->destroy(data[i]);
data.resize_assume_reserved(new_size);
}
2017-06-16 16:23:23 +00:00
2018-03-20 14:17:09 +00:00
ColumnPtr ColumnAggregateFunction::replicate(const IColumn::Offsets & offsets) const
{
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
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)
return cloneEmpty();
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
auto res = createView();
auto & res_data = res->data;
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
res_data.reserve(offsets.back());
IColumn::Offset prev_offset = 0;
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
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]);
}
return res;
}
2017-06-16 16:23:23 +00:00
MutableColumns ColumnAggregateFunction::scatter(IColumn::ColumnIndex num_columns, const IColumn::Selector & selector) const
{
/// Columns with scattered values will point to this column as the owner of values.
MutableColumns columns(num_columns);
for (auto & column : columns)
column = createView();
size_t num_rows = size();
{
size_t reserve_size = num_rows / num_columns * 1.1; /// 1.1 is just a guess. Better to use n-sigma rule.
if (reserve_size > 1)
for (auto & column : columns)
column->reserve(reserve_size);
}
for (size_t i = 0; i < num_rows; ++i)
assert_cast<ColumnAggregateFunction &>(*columns[selector[i]]).data.push_back(data[i]);
return columns;
}
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
{
size_t s = data.size();
res.resize(s);
for (size_t i = 0; i < s; ++i)
res[i] = i;
}
2017-06-16 16:23:23 +00:00
void ColumnAggregateFunction::gather(ColumnGathererStream & gatherer)
{
gatherer.gather(*this);
}
void ColumnAggregateFunction::getExtremes(Field & min, Field & max) const
{
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
/// Place serialized default values into min/max.
AlignedBuffer place_buffer(func->sizeOfData(), func->alignOfData());
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
AggregateDataPtr place = place_buffer.data();
AggregateFunctionStateData serialized;
2019-02-11 13:11:52 +00:00
serialized.name = getTypeString();
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
func->create(place);
try
{
2019-02-11 13:11:52 +00:00
WriteBufferFromString buffer(serialized.data);
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
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-06-16 16:23:23 +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_)
: foreign_arenas(concatArenas(src_.foreign_arenas, src_.my_arena)),
func(src_.func), src(src_.getPtr()), data(src_.data.begin(), src_.data.end())
{
}
}