This commit is contained in:
Andrey Urusov 2015-11-26 21:12:02 +03:00
commit c527c7269c
255 changed files with 13923 additions and 3936 deletions

View File

@ -24,17 +24,17 @@ private:
public:
AggregateFunctionArray(AggregateFunctionPtr nested_) : nested_func_owner(nested_), nested_func(nested_func_owner.get()) {}
String getName() const
String getName() const override
{
return nested_func->getName() + "Array";
}
DataTypePtr getReturnType() const
DataTypePtr getReturnType() const override
{
return nested_func->getReturnType();
}
void setArguments(const DataTypes & arguments)
void setArguments(const DataTypes & arguments) override
{
num_agruments = arguments.size();
@ -49,37 +49,37 @@ public:
nested_func->setArguments(nested_arguments);
}
void setParameters(const Array & params)
void setParameters(const Array & params) override
{
nested_func->setParameters(params);
}
void create(AggregateDataPtr place) const
void create(AggregateDataPtr place) const override
{
nested_func->create(place);
}
void destroy(AggregateDataPtr place) const noexcept
void destroy(AggregateDataPtr place) const noexcept override
{
nested_func->destroy(place);
}
bool hasTrivialDestructor() const
bool hasTrivialDestructor() const override
{
return nested_func->hasTrivialDestructor();
}
size_t sizeOfData() const
size_t sizeOfData() const override
{
return nested_func->sizeOfData();
}
size_t alignOfData() const
size_t alignOfData() const override
{
return nested_func->alignOfData();
}
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const override
{
const IColumn * nested[num_agruments];
@ -96,22 +96,22 @@ public:
nested_func->add(place, nested, i);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
nested_func->merge(place, rhs);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
nested_func->serialize(place, buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
nested_func->deserializeMerge(place, buf);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
nested_func->insertResultInto(place, to);
}

View File

@ -27,9 +27,9 @@ template <typename T>
class AggregateFunctionAvg final : public IUnaryAggregateFunction<AggregateFunctionAvgData<typename NearestFieldType<T>::Type>, AggregateFunctionAvg<T> >
{
public:
String getName() const { return "avg"; }
String getName() const override { return "avg"; }
DataTypePtr getReturnType() const
DataTypePtr getReturnType() const override
{
return new DataTypeFloat64;
}
@ -42,25 +42,25 @@ public:
}
void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const
{
this->data(place).sum += static_cast<const ColumnVector<T> &>(column).getData()[row_num];
++this->data(place).count;
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).sum += this->data(rhs).sum;
this->data(place).count += this->data(rhs).count;
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
writeBinary(this->data(place).sum, buf);
writeVarUInt(this->data(place).count, buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
typename NearestFieldType<T>::Type tmp_sum = 0;
UInt64 tmp_count = 0;
@ -70,7 +70,7 @@ public:
this->data(place).count += tmp_count;
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
static_cast<ColumnFloat64 &>(to).getData().push_back(
static_cast<Float64>(this->data(place).sum) / this->data(place).count);

View File

@ -23,37 +23,37 @@ struct AggregateFunctionCountData
class AggregateFunctionCount final : public INullaryAggregateFunction<AggregateFunctionCountData, AggregateFunctionCount>
{
public:
String getName() const { return "count"; }
String getName() const override { return "count"; }
DataTypePtr getReturnType() const
DataTypePtr getReturnType() const override
{
return new DataTypeUInt64;
}
void addZero(AggregateDataPtr place) const
void addImpl(AggregateDataPtr place) const
{
++data(place).count;
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
data(place).count += data(rhs).count;
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
writeVarUInt(data(place).count, buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
UInt64 tmp;
readVarUInt(tmp, buf);
data(place).count += tmp;
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
static_cast<ColumnUInt64 &>(to).getData().push_back(data(place).count);
}

View File

@ -4,6 +4,10 @@
#include <DB/IO/ReadHelpers.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/Columns/ColumnVector.h>
#include <DB/Columns/ColumnArray.h>
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
@ -13,22 +17,102 @@
namespace DB
{
struct AggregateFunctionGroupArrayData
/// Частный случай - реализация для числовых типов.
template <typename T>
struct AggregateFunctionGroupArrayDataNumeric
{
/// Сразу будет выделена память на несколько элементов так, чтобы состояние занимало 64 байта.
static constexpr size_t bytes_in_arena = 64 - sizeof(PODArray<T>);
using Array = PODArray<T, bytes_in_arena / sizeof(T), AllocatorWithStackMemory<Allocator<false>, bytes_in_arena>>;
Array value;
};
/// Общий случай (неэффективно). NOTE Можно ещё реализовать частный случай для строк.
struct AggregateFunctionGroupArrayDataGeneric
{
Array value; /// TODO Добавить MemoryTracker
};
/// Складывает все значения в массив. Реализовано неэффективно.
class AggregateFunctionGroupArray final : public IUnaryAggregateFunction<AggregateFunctionGroupArrayData, AggregateFunctionGroupArray>
template <typename T>
class AggregateFunctionGroupArrayNumeric final
: public IUnaryAggregateFunction<AggregateFunctionGroupArrayDataNumeric<T>, AggregateFunctionGroupArrayNumeric<T>>
{
public:
String getName() const override { return "groupArray"; }
DataTypePtr getReturnType() const override
{
return new DataTypeArray(new typename DataTypeFromFieldType<T>::Type);
}
void setArgument(const DataTypePtr & argument)
{
}
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const
{
this->data(place).value.push_back(static_cast<const ColumnVector<T> &>(column).getData()[row_num]);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).value.insert(this->data(rhs).value.begin(), this->data(rhs).value.end());
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
const auto & value = this->data(place).value;
size_t size = value.size();
writeVarUInt(size, buf);
buf.write(reinterpret_cast<const char *>(&value[0]), size * sizeof(value[0]));
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
size_t size = 0;
readVarUInt(size, buf);
if (size > AGGREGATE_FUNCTION_GROUP_ARRAY_MAX_ARRAY_SIZE)
throw Exception("Too large array size", ErrorCodes::TOO_LARGE_ARRAY_SIZE);
auto & value = this->data(place).value;
size_t old_size = value.size();
value.resize(old_size + size);
buf.read(reinterpret_cast<char *>(&value[old_size]), size * sizeof(value[0]));
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
const auto & value = this->data(place).value;
size_t size = value.size();
ColumnArray & arr_to = static_cast<ColumnArray &>(to);
ColumnArray::Offsets_t & offsets_to = arr_to.getOffsets();
offsets_to.push_back((offsets_to.size() == 0 ? 0 : offsets_to.back()) + size);
typename ColumnVector<T>::Container_t & data_to = static_cast<ColumnVector<T> &>(arr_to.getData()).getData();
data_to.insert(this->data(place).value.begin(), this->data(place).value.end());
}
};
/// Складывает все значения в массив, общий случай. Реализовано неэффективно.
class AggregateFunctionGroupArrayGeneric final
: public IUnaryAggregateFunction<AggregateFunctionGroupArrayDataGeneric, AggregateFunctionGroupArrayGeneric>
{
private:
DataTypePtr type;
public:
String getName() const { return "groupArray"; }
String getName() const override { return "groupArray"; }
DataTypePtr getReturnType() const
DataTypePtr getReturnType() const override
{
return new DataTypeArray(type);
}
@ -39,18 +123,18 @@ public:
}
void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const
{
data(place).value.push_back(Array::value_type());
column.get(row_num, data(place).value.back());
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
data(place).value.insert(data(place).value.end(), data(rhs).value.begin(), data(rhs).value.end());
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
const Array & value = data(place).value;
size_t size = value.size();
@ -59,7 +143,7 @@ public:
type->serializeBinary(value[i], buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
size_t size = 0;
readVarUInt(size, buf);
@ -75,7 +159,7 @@ public:
type->deserializeBinary(value[old_size + i], buf);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
to.insert(data(place).value);
}

View File

@ -40,11 +40,11 @@ class AggregateFunctionGroupUniqArray : public IUnaryAggregateFunction<Aggregate
{
private:
typedef AggregateFunctionGroupUniqArrayData<T> State;
public:
String getName() const { return "groupUniqArray"; }
DataTypePtr getReturnType() const
public:
String getName() const override { return "groupUniqArray"; }
DataTypePtr getReturnType() const override
{
return new DataTypeArray(new typename DataTypeFromFieldType<T>::Type);
}
@ -54,17 +54,17 @@ public:
}
void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const
{
this->data(place).value.insert(static_cast<const ColumnVector<T> &>(column).getData()[row_num]);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).value.merge(this->data(rhs).value);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
const typename State::Set & set = this->data(place).value;
size_t size = set.size();
@ -73,12 +73,12 @@ public:
writeIntBinary(*it, buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
this->data(place).value.readAndMerge(buf);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
ColumnArray & arr_to = static_cast<ColumnArray &>(to);
ColumnArray::Offsets_t & offsets_to = arr_to.getOffsets();
@ -106,7 +106,7 @@ template <typename T>
class AggregateFunctionGroupUniqArrays final : public AggregateFunctionGroupUniqArray<T>
{
public:
void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const
{
const ColumnArray & arr = static_cast<const ColumnArray &>(column);
const ColumnArray::Offsets_t & offsets = arr.getOffsets();

View File

@ -23,17 +23,17 @@ private:
public:
AggregateFunctionIf(AggregateFunctionPtr nested_) : nested_func_owner(nested_), nested_func(nested_func_owner.get()) {}
String getName() const
String getName() const override
{
return nested_func->getName() + "If";
}
DataTypePtr getReturnType() const
DataTypePtr getReturnType() const override
{
return nested_func->getReturnType();
}
void setArguments(const DataTypes & arguments)
void setArguments(const DataTypes & arguments) override
{
num_agruments = arguments.size();
@ -47,58 +47,58 @@ public:
nested_func->setArguments(nested_arguments);
}
void setParameters(const Array & params)
void setParameters(const Array & params) override
{
nested_func->setParameters(params);
}
void create(AggregateDataPtr place) const
void create(AggregateDataPtr place) const override
{
nested_func->create(place);
}
void destroy(AggregateDataPtr place) const noexcept
void destroy(AggregateDataPtr place) const noexcept override
{
nested_func->destroy(place);
}
bool hasTrivialDestructor() const
bool hasTrivialDestructor() const override
{
return nested_func->hasTrivialDestructor();
}
size_t sizeOfData() const
size_t sizeOfData() const override
{
return nested_func->sizeOfData();
}
size_t alignOfData() const
size_t alignOfData() const override
{
return nested_func->alignOfData();
}
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const override
{
if (static_cast<const ColumnUInt8 &>(*columns[num_agruments - 1]).getData()[row_num])
nested_func->add(place, columns, row_num);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
nested_func->merge(place, rhs);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
nested_func->serialize(place, buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
nested_func->deserializeMerge(place, buf);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
nested_func->insertResultInto(place, to);
}

View File

@ -24,17 +24,17 @@ private:
public:
AggregateFunctionMerge(AggregateFunctionPtr nested_) : nested_func_owner(nested_), nested_func(nested_func_owner.get()) {}
String getName() const
String getName() const override
{
return nested_func->getName() + "Merge";
}
DataTypePtr getReturnType() const
DataTypePtr getReturnType() const override
{
return nested_func->getReturnType();
}
void setArguments(const DataTypes & arguments)
void setArguments(const DataTypes & arguments) override
{
if (arguments.size() != 1)
throw Exception("Passed " + toString(arguments.size()) + " arguments to unary aggregate function " + this->getName(),
@ -49,57 +49,57 @@ public:
nested_func->setArguments(data_type->getArgumentsDataTypes());
}
void setParameters(const Array & params)
void setParameters(const Array & params) override
{
nested_func->setParameters(params);
}
void create(AggregateDataPtr place) const
void create(AggregateDataPtr place) const override
{
nested_func->create(place);
}
void destroy(AggregateDataPtr place) const noexcept
void destroy(AggregateDataPtr place) const noexcept override
{
nested_func->destroy(place);
}
bool hasTrivialDestructor() const
bool hasTrivialDestructor() const override
{
return nested_func->hasTrivialDestructor();
}
size_t sizeOfData() const
size_t sizeOfData() const override
{
return nested_func->sizeOfData();
}
size_t alignOfData() const
size_t alignOfData() const override
{
return nested_func->alignOfData();
}
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const override
{
nested_func->merge(place, static_cast<const ColumnAggregateFunction &>(*columns[0]).getData()[row_num]);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
nested_func->merge(place, rhs);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
nested_func->serialize(place, buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
nested_func->deserializeMerge(place, buf);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
nested_func->insertResultInto(place, to);
}

View File

@ -1,6 +1,6 @@
#pragma once
#include <stats/ReservoirSampler.h>
#include <DB/AggregateFunctions/ReservoirSampler.h>
#include <DB/Core/FieldVisitors.h>
@ -32,7 +32,8 @@ struct AggregateFunctionQuantileData
* Для дат и дат-с-временем returns_float следует задавать равным false.
*/
template <typename ArgumentFieldType, bool returns_float = true>
class AggregateFunctionQuantile final : public IUnaryAggregateFunction<AggregateFunctionQuantileData<ArgumentFieldType>, AggregateFunctionQuantile<ArgumentFieldType, returns_float> >
class AggregateFunctionQuantile final
: public IUnaryAggregateFunction<AggregateFunctionQuantileData<ArgumentFieldType>, AggregateFunctionQuantile<ArgumentFieldType, returns_float> >
{
private:
using Sample = typename AggregateFunctionQuantileData<ArgumentFieldType>::Sample;
@ -43,9 +44,9 @@ private:
public:
AggregateFunctionQuantile(double level_ = 0.5) : level(level_) {}
String getName() const { return "quantile"; }
String getName() const override { return "quantile"; }
DataTypePtr getReturnType() const
DataTypePtr getReturnType() const override
{
return type;
}
@ -58,7 +59,7 @@ public:
type = argument;
}
void setParameters(const Array & params)
void setParameters(const Array & params) override
{
if (params.size() != 1)
throw Exception("Aggregate function " + getName() + " requires exactly one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
@ -67,29 +68,29 @@ public:
}
void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const
{
this->data(place).sample.insert(static_cast<const ColumnVector<ArgumentFieldType> &>(column).getData()[row_num]);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).sample.merge(this->data(rhs).sample);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(place).sample.write(buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
Sample tmp_sample;
tmp_sample.read(buf);
this->data(place).sample.merge(tmp_sample);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
/// Sample может отсортироваться при получении квантиля, но в этом контексте можно не считать это нарушением константности.
Sample & sample = const_cast<Sample &>(this->data(place).sample);
@ -107,19 +108,20 @@ public:
* Возвращает массив результатов.
*/
template <typename ArgumentFieldType, bool returns_float = true>
class AggregateFunctionQuantiles final : public IUnaryAggregateFunction<AggregateFunctionQuantileData<ArgumentFieldType>, AggregateFunctionQuantiles<ArgumentFieldType, returns_float> >
class AggregateFunctionQuantiles final
: public IUnaryAggregateFunction<AggregateFunctionQuantileData<ArgumentFieldType>, AggregateFunctionQuantiles<ArgumentFieldType, returns_float> >
{
private:
using Sample = typename AggregateFunctionQuantileData<ArgumentFieldType>::Sample;
typedef std::vector<double> Levels;
using Levels = std::vector<double>;
Levels levels;
DataTypePtr type;
public:
String getName() const { return "quantiles"; }
String getName() const override { return "quantiles"; }
DataTypePtr getReturnType() const
DataTypePtr getReturnType() const override
{
return new DataTypeArray(type);
}
@ -132,7 +134,7 @@ public:
type = argument;
}
void setParameters(const Array & params)
void setParameters(const Array & params) override
{
if (params.empty())
throw Exception("Aggregate function " + getName() + " requires at least one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
@ -145,29 +147,29 @@ public:
}
void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const
{
this->data(place).sample.insert(static_cast<const ColumnVector<ArgumentFieldType> &>(column).getData()[row_num]);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).sample.merge(this->data(rhs).sample);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(place).sample.write(buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
Sample tmp_sample;
tmp_sample.read(buf);
this->data(place).sample.merge(tmp_sample);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
/// Sample может отсортироваться при получении квантиля, но в этом контексте можно не считать это нарушением константности.
Sample & sample = const_cast<Sample &>(this->data(place).sample);
@ -183,14 +185,14 @@ public:
ColumnFloat64::Container_t & data_to = static_cast<ColumnFloat64 &>(arr_to.getData()).getData();
for (size_t i = 0; i < size; ++i)
data_to.push_back(sample.quantileInterpolated(levels[i]));
data_to.push_back(sample.quantileInterpolated(levels[i]));
}
else
{
typename ColumnVector<ArgumentFieldType>::Container_t & data_to = static_cast<ColumnVector<ArgumentFieldType> &>(arr_to.getData()).getData();
for (size_t i = 0; i < size; ++i)
data_to.push_back(sample.quantileInterpolated(levels[i]));
data_to.push_back(sample.quantileInterpolated(levels[i]));
}
}
};

View File

@ -46,9 +46,9 @@ private:
public:
AggregateFunctionQuantileDeterministic(double level_ = 0.5) : level(level_) {}
String getName() const { return "quantileDeterministic"; }
String getName() const override { return "quantileDeterministic"; }
DataTypePtr getReturnType() const
DataTypePtr getReturnType() const override
{
return type;
}
@ -65,7 +65,7 @@ public:
};
}
void setParameters(const Array & params)
void setParameters(const Array & params) override
{
if (params.size() != 1)
throw Exception("Aggregate function " + getName() + " requires exactly one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
@ -74,30 +74,30 @@ public:
}
void addOne(AggregateDataPtr place, const IColumn & column, const IColumn & determinator, size_t row_num) const
void addImpl(AggregateDataPtr place, const IColumn & column, const IColumn & determinator, size_t row_num) const
{
this->data(place).sample.insert(static_cast<const ColumnVector<ArgumentFieldType> &>(column).getData()[row_num],
determinator.get64(row_num));
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).sample.merge(this->data(rhs).sample);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(place).sample.write(buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
Sample tmp_sample;
tmp_sample.read(buf);
this->data(place).sample.merge(tmp_sample);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
/// Sample может отсортироваться при получении квантиля, но в этом контексте можно не считать это нарушением константности.
Sample & sample = const_cast<Sample &>(this->data(place).sample);
@ -123,14 +123,14 @@ class AggregateFunctionQuantilesDeterministic final
private:
using Sample = typename AggregateFunctionQuantileDeterministicData<ArgumentFieldType>::Sample;
typedef std::vector<double> Levels;
using Levels = std::vector<double>;
Levels levels;
DataTypePtr type;
public:
String getName() const { return "quantilesDeterministic"; }
String getName() const override { return "quantilesDeterministic"; }
DataTypePtr getReturnType() const
DataTypePtr getReturnType() const override
{
return new DataTypeArray(type);
}
@ -147,7 +147,7 @@ public:
};
}
void setParameters(const Array & params)
void setParameters(const Array & params) override
{
if (params.empty())
throw Exception("Aggregate function " + getName() + " requires at least one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
@ -160,30 +160,30 @@ public:
}
void addOne(AggregateDataPtr place, const IColumn & column, const IColumn & determinator, size_t row_num) const
void addImpl(AggregateDataPtr place, const IColumn & column, const IColumn & determinator, size_t row_num) const
{
this->data(place).sample.insert(static_cast<const ColumnVector<ArgumentFieldType> &>(column).getData()[row_num],
determinator.get64(row_num));
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).sample.merge(this->data(rhs).sample);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(place).sample.write(buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
Sample tmp_sample;
tmp_sample.read(buf);
this->data(place).sample.merge(tmp_sample);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
/// Sample может отсортироваться при получении квантиля, но в этом контексте можно не считать это нарушением константности.
Sample & sample = const_cast<Sample &>(this->data(place).sample);
@ -199,14 +199,14 @@ public:
ColumnFloat64::Container_t & data_to = static_cast<ColumnFloat64 &>(arr_to.getData()).getData();
for (size_t i = 0; i < size; ++i)
data_to.push_back(sample.quantileInterpolated(levels[i]));
data_to.push_back(sample.quantileInterpolated(levels[i]));
}
else
{
typename ColumnVector<ArgumentFieldType>::Container_t & data_to = static_cast<ColumnVector<ArgumentFieldType> &>(arr_to.getData()).getData();
for (size_t i = 0; i < size; ++i)
data_to.push_back(sample.quantileInterpolated(levels[i]));
data_to.push_back(sample.quantileInterpolated(levels[i]));
}
}
};

View File

@ -0,0 +1,229 @@
#pragma once
#include <DB/Common/PODArray.h>
#include <DB/Core/FieldVisitors.h>
#include <DB/IO/WriteHelpers.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
#include <DB/Columns/ColumnArray.h>
namespace DB
{
/** В качестве состояния используется массив, в который складываются все значения.
* NOTE Если различных значений мало, то это не оптимально.
* Для 8 и 16-битных значений возможно, было бы лучше использовать lookup-таблицу.
*/
template <typename T>
struct AggregateFunctionQuantileExactData
{
/// Сразу будет выделена память на несколько элементов так, чтобы состояние занимало 64 байта.
static constexpr size_t bytes_in_arena = 64 - sizeof(PODArray<T>);
using Array = PODArray<T, bytes_in_arena / sizeof(T), AllocatorWithStackMemory<Allocator<false>, bytes_in_arena>>;
Array array;
};
/** Точно вычисляет квантиль.
* В качестве типа аргумента может быть только числовой тип (в том числе, дата и дата-с-временем).
* Тип результата совпадает с типом аргумента.
*/
template <typename T>
class AggregateFunctionQuantileExact final
: public IUnaryAggregateFunction<AggregateFunctionQuantileExactData<T>, AggregateFunctionQuantileExact<T>>
{
private:
double level;
DataTypePtr type;
public:
AggregateFunctionQuantileExact(double level_ = 0.5) : level(level_) {}
String getName() const override { return "quantileExact"; }
DataTypePtr getReturnType() const override
{
return type;
}
void setArgument(const DataTypePtr & argument)
{
type = argument;
}
void setParameters(const Array & params) override
{
if (params.size() != 1)
throw Exception("Aggregate function " + getName() + " requires exactly one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
level = apply_visitor(FieldVisitorConvertToNumber<Float64>(), params[0]);
}
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const
{
this->data(place).array.push_back(static_cast<const ColumnVector<T> &>(column).getData()[row_num]);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).array.insert(this->data(rhs).array.begin(), this->data(rhs).array.end());
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
const auto & array = this->data(place).array;
size_t size = array.size();
writeVarUInt(size, buf);
buf.write(reinterpret_cast<const char *>(&array[0]), size * sizeof(array[0]));
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
auto & array = this->data(place).array;
size_t size = 0;
readVarUInt(size, buf);
size_t old_size = array.size();
array.resize(old_size + size);
buf.read(reinterpret_cast<char *>(&array[old_size]), size * sizeof(array[0]));
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
/// Сортировка массива не будет считаться нарушением константности.
auto & array = const_cast<typename AggregateFunctionQuantileExactData<T>::Array &>(this->data(place).array);
T quantile = T();
if (!array.empty())
{
size_t n = level < 1
? level * array.size()
: (array.size() - 1);
std::nth_element(array.begin(), array.begin() + n, array.end()); /// NOTE Можно придумать алгоритм radix-select.
quantile = array[n];
}
static_cast<ColumnVector<T> &>(to).getData().push_back(quantile);
}
};
/** То же самое, но позволяет вычислить сразу несколько квантилей.
* Для этого, принимает в качестве параметров несколько уровней. Пример: quantilesExact(0.5, 0.8, 0.9, 0.95)(ConnectTiming).
* Возвращает массив результатов.
*/
template <typename T>
class AggregateFunctionQuantilesExact final
: public IUnaryAggregateFunction<AggregateFunctionQuantileExactData<T>, AggregateFunctionQuantilesExact<T>>
{
private:
using Levels = std::vector<double>;
Levels levels;
DataTypePtr type;
public:
String getName() const override { return "quantilesExact"; }
DataTypePtr getReturnType() const override
{
return new DataTypeArray(type);
}
void setArgument(const DataTypePtr & argument)
{
type = argument;
}
void setParameters(const Array & params) override
{
if (params.empty())
throw Exception("Aggregate function " + getName() + " requires at least one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
size_t size = params.size();
levels.resize(size);
for (size_t i = 0; i < size; ++i)
levels[i] = apply_visitor(FieldVisitorConvertToNumber<Float64>(), params[i]);
}
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const
{
this->data(place).array.push_back(static_cast<const ColumnVector<T> &>(column).getData()[row_num]);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).array.insert(this->data(rhs).array.begin(), this->data(rhs).array.end());
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
const auto & array = this->data(place).array;
size_t size = array.size();
writeVarUInt(size, buf);
buf.write(reinterpret_cast<const char *>(&array[0]), size * sizeof(array[0]));
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
auto & array = this->data(place).array;
size_t size = 0;
readVarUInt(size, buf);
size_t old_size = array.size();
array.resize(old_size + size);
buf.read(reinterpret_cast<char *>(&array[old_size]), size * sizeof(array[0]));
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
/// Сортировка массива не будет считаться нарушением константности.
auto & array = const_cast<typename AggregateFunctionQuantileExactData<T>::Array &>(this->data(place).array);
ColumnArray & arr_to = static_cast<ColumnArray &>(to);
ColumnArray::Offsets_t & offsets_to = arr_to.getOffsets();
size_t num_levels = levels.size();
offsets_to.push_back((offsets_to.size() == 0 ? 0 : offsets_to.back()) + num_levels);
typename ColumnVector<T>::Container_t & data_to = static_cast<ColumnVector<T> &>(arr_to.getData()).getData();
if (!array.empty())
{
size_t prev_n = 0;
for (const auto & level : levels)
{
size_t n = level < 1
? level * array.size()
: (array.size() - 1);
std::nth_element(array.begin() + prev_n, array.begin() + n, array.end());
data_to.push_back(array[n]);
prev_n = n;
}
}
else
{
for (size_t i = 0; i < num_levels; ++i)
data_to.push_back(T());
}
}
};
}

View File

@ -0,0 +1,286 @@
#pragma once
#include <DB/Common/HashTable/HashMap.h>
#include <DB/Core/FieldVisitors.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/AggregateFunctions/IBinaryAggregateFunction.h>
#include <DB/Columns/ColumnArray.h>
namespace DB
{
/** В качестве состояния используется хэш-таблица вида: значение -> сколько раз встретилось.
*/
template <typename T>
struct AggregateFunctionQuantileExactWeightedData
{
using Key = T;
using Weight = UInt64;
/// При создании, хэш-таблица должна быть небольшой.
using Map = HashMap<
Key, Weight,
HashCRC32<Key>,
HashTableGrower<4>,
HashTableAllocatorWithStackMemory<sizeof(std::pair<Key, Weight>) * (1 << 3)>
>;
Map map;
};
/** Точно вычисляет квантиль по множеству значений, для каждого из которых задан вес - сколько раз значение встречалось.
* Можно рассматривать набор пар value, weight - как набор гистограмм,
* в которых value - значение, округлённое до середины столбика, а weight - высота столбика.
* В качестве типа аргумента может быть только числовой тип (в том числе, дата и дата-с-временем).
* Тип результата совпадает с типом аргумента.
*/
template <typename ValueType, typename WeightType>
class AggregateFunctionQuantileExactWeighted final
: public IBinaryAggregateFunction<
AggregateFunctionQuantileExactWeightedData<ValueType>,
AggregateFunctionQuantileExactWeighted<ValueType, WeightType>>
{
private:
double level;
DataTypePtr type;
public:
AggregateFunctionQuantileExactWeighted(double level_ = 0.5) : level(level_) {}
String getName() const override { return "quantileExactWeighted"; }
DataTypePtr getReturnType() const override
{
return type;
}
void setArgumentsImpl(const DataTypes & arguments)
{
type = arguments[0];
}
void setParameters(const Array & params) override
{
if (params.size() != 1)
throw Exception("Aggregate function " + getName() + " requires exactly one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
level = apply_visitor(FieldVisitorConvertToNumber<Float64>(), params[0]);
}
void addImpl(AggregateDataPtr place, const IColumn & column_value, const IColumn & column_weight, size_t row_num) const
{
this->data(place)
.map[static_cast<const ColumnVector<ValueType> &>(column_value).getData()[row_num]]
+= static_cast<const ColumnVector<WeightType> &>(column_weight).getData()[row_num];
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
auto & map = this->data(place).map;
const auto & rhs_map = this->data(rhs).map;
for (const auto & pair : rhs_map)
map[pair.first] += pair.second;
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(place).map.write(buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
typename AggregateFunctionQuantileExactWeightedData<ValueType>::Map::Reader reader(buf);
auto & map = this->data(place).map;
while (reader.next())
{
const auto & pair = reader.get();
map[pair.first] += pair.second;
}
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
auto & map = this->data(place).map;
size_t size = map.size();
if (0 == size)
{
static_cast<ColumnVector<ValueType> &>(to).getData().push_back(ValueType());
return;
}
/// Копируем данные во временный массив, чтобы получить нужный по порядку элемент.
using Pair = typename AggregateFunctionQuantileExactWeightedData<ValueType>::Map::value_type;
std::unique_ptr<Pair[]> array_holder(new Pair[size]);
Pair * array = array_holder.get();
size_t i = 0;
UInt64 sum_weight = 0;
for (const auto & pair : map)
{
sum_weight += pair.second;
array[i] = pair;
++i;
}
std::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.first < b.first; });
UInt64 threshold = sum_weight * level;
UInt64 accumulated = 0;
const Pair * it = array;
const Pair * end = array + size;
while (it < end && accumulated < threshold)
{
accumulated += it->second;
++it;
}
if (it == end)
--it;
static_cast<ColumnVector<ValueType> &>(to).getData().push_back(it->first);
}
};
/** То же самое, но позволяет вычислить сразу несколько квантилей.
* Для этого, принимает в качестве параметров несколько уровней. Пример: quantilesExactWeighted(0.5, 0.8, 0.9, 0.95)(ConnectTiming, Weight).
* Возвращает массив результатов.
*/
template <typename ValueType, typename WeightType>
class AggregateFunctionQuantilesExactWeighted final
: public IBinaryAggregateFunction<
AggregateFunctionQuantileExactWeightedData<ValueType>,
AggregateFunctionQuantilesExactWeighted<ValueType, WeightType>>
{
private:
using Levels = std::vector<double>;
Levels levels;
DataTypePtr type;
public:
String getName() const override { return "quantilesExactWeighted"; }
DataTypePtr getReturnType() const override
{
return new DataTypeArray(type);
}
void setArgumentsImpl(const DataTypes & arguments)
{
type = arguments[0];
}
void setParameters(const Array & params) override
{
if (params.empty())
throw Exception("Aggregate function " + getName() + " requires at least one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
size_t size = params.size();
levels.resize(size);
for (size_t i = 0; i < size; ++i)
levels[i] = apply_visitor(FieldVisitorConvertToNumber<Float64>(), params[i]);
}
void addImpl(AggregateDataPtr place, const IColumn & column_value, const IColumn & column_weight, size_t row_num) const
{
this->data(place)
.map[static_cast<const ColumnVector<ValueType> &>(column_value).getData()[row_num]]
+= static_cast<const ColumnVector<WeightType> &>(column_weight).getData()[row_num];
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
auto & map = this->data(place).map;
const auto & rhs_map = this->data(rhs).map;
for (const auto & pair : rhs_map)
map[pair.first] += pair.second;
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(place).map.write(buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
typename AggregateFunctionQuantileExactWeightedData<ValueType>::Map::Reader reader(buf);
auto & map = this->data(place).map;
while (reader.next())
{
const auto & pair = reader.get();
map[pair.first] += pair.second;
}
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
auto & map = this->data(place).map;
size_t size = map.size();
ColumnArray & arr_to = static_cast<ColumnArray &>(to);
ColumnArray::Offsets_t & offsets_to = arr_to.getOffsets();
size_t num_levels = levels.size();
offsets_to.push_back((offsets_to.size() == 0 ? 0 : offsets_to.back()) + num_levels);
typename ColumnVector<ValueType>::Container_t & data_to = static_cast<ColumnVector<ValueType> &>(arr_to.getData()).getData();
if (0 == size)
{
for (size_t i = 0; i < num_levels; ++i)
data_to.push_back(ValueType());
return;
}
/// Копируем данные во временный массив, чтобы получить нужный по порядку элемент.
using Pair = typename AggregateFunctionQuantileExactWeightedData<ValueType>::Map::value_type;
std::unique_ptr<Pair[]> array_holder(new Pair[size]);
Pair * array = array_holder.get();
size_t i = 0;
UInt64 sum_weight = 0;
for (const auto & pair : map)
{
sum_weight += pair.second;
array[i] = pair;
++i;
}
std::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.first < b.first; });
UInt64 accumulated = 0;
const Pair * it = array;
const Pair * end = array + size;
for (const auto & level : levels)
{
UInt64 threshold = sum_weight * level;
while (it < end && accumulated < threshold)
{
accumulated += it->second;
++it;
}
data_to.push_back(it < end ? it->first : it[-1].first);
}
}
};
}

View File

@ -3,6 +3,7 @@
#include <limits>
#include <DB/Common/MemoryTracker.h>
#include <DB/Common/HashTable/Hash.h>
#include <DB/Core/FieldVisitors.h>
@ -13,10 +14,10 @@
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
#include <DB/AggregateFunctions/IBinaryAggregateFunction.h>
#include <DB/Columns/ColumnArray.h>
#include <stats/IntHash.h>
#include <ext/range.hpp>
@ -549,9 +550,9 @@ private:
public:
AggregateFunctionQuantileTiming(double level_ = 0.5) : level(level_) {}
String getName() const { return "quantileTiming"; }
String getName() const override { return "quantileTiming"; }
DataTypePtr getReturnType() const
DataTypePtr getReturnType() const override
{
return new DataTypeFloat32;
}
@ -560,7 +561,7 @@ public:
{
}
void setParameters(const Array & params)
void setParameters(const Array & params) override
{
if (params.size() != 1)
throw Exception("Aggregate function " + getName() + " requires exactly one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
@ -569,27 +570,27 @@ public:
}
void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const
{
this->data(place).insert(static_cast<const ColumnVector<ArgumentFieldType> &>(column).getData()[row_num]);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(place).serialize(buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
this->data(place).deserializeMerge(buf);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
static_cast<ColumnFloat32 &>(to).getData().push_back(this->data(place).getFloat(level));
}
@ -607,18 +608,18 @@ private:
public:
AggregateFunctionQuantileTimingWeighted(double level_ = 0.5) : level(level_) {}
String getName() const { return "quantileTimingWeighted"; }
String getName() const override { return "quantileTimingWeighted"; }
DataTypePtr getReturnType() const
DataTypePtr getReturnType() const override
{
return new DataTypeFloat32;
}
void setArguments(const DataTypes & arguments)
void setArguments(const DataTypes & arguments) override
{
}
void setParameters(const Array & params)
void setParameters(const Array & params) override
{
if (params.size() != 1)
throw Exception("Aggregate function " + getName() + " requires exactly one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
@ -627,29 +628,29 @@ public:
}
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const override
{
this->data(place).insertWeighted(
static_cast<const ColumnVector<ArgumentFieldType> &>(*columns[0]).getData()[row_num],
static_cast<const ColumnVector<WeightFieldType> &>(*columns[1]).getData()[row_num]);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(place).serialize(buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
this->data(place).deserializeMerge(buf);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
static_cast<ColumnFloat32 &>(to).getData().push_back(this->data(place).getFloat(level));
}
@ -668,9 +669,9 @@ private:
Levels levels;
public:
String getName() const { return "quantilesTiming"; }
String getName() const override { return "quantilesTiming"; }
DataTypePtr getReturnType() const
DataTypePtr getReturnType() const override
{
return new DataTypeArray(new DataTypeFloat32);
}
@ -679,7 +680,7 @@ public:
{
}
void setParameters(const Array & params)
void setParameters(const Array & params) override
{
if (params.empty())
throw Exception("Aggregate function " + getName() + " requires at least one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
@ -692,27 +693,27 @@ public:
}
void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const
{
this->data(place).insert(static_cast<const ColumnVector<ArgumentFieldType> &>(column).getData()[row_num]);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(place).serialize(buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
this->data(place).deserializeMerge(buf);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
ColumnArray & arr_to = static_cast<ColumnArray &>(to);
ColumnArray::Offsets_t & offsets_to = arr_to.getOffsets();
@ -730,25 +731,26 @@ public:
template <typename ArgumentFieldType, typename WeightFieldType>
class AggregateFunctionQuantilesTimingWeighted final : public IAggregateFunctionHelper<QuantileTiming>
class AggregateFunctionQuantilesTimingWeighted final
: public IBinaryAggregateFunction<QuantileTiming, AggregateFunctionQuantilesTimingWeighted<ArgumentFieldType, WeightFieldType>>
{
private:
typedef std::vector<double> Levels;
using Levels = std::vector<double>;
Levels levels;
public:
String getName() const { return "quantilesTimingWeighted"; }
String getName() const override { return "quantilesTimingWeighted"; }
DataTypePtr getReturnType() const
DataTypePtr getReturnType() const override
{
return new DataTypeArray(new DataTypeFloat32);
}
void setArguments(const DataTypes & arguments)
void setArgumentsImpl(const DataTypes & arguments)
{
}
void setParameters(const Array & params)
void setParameters(const Array & params) override
{
if (params.empty())
throw Exception("Aggregate function " + getName() + " requires at least one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
@ -760,29 +762,29 @@ public:
levels[i] = apply_visitor(FieldVisitorConvertToNumber<Float64>(), params[i]);
}
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const
void addImpl(AggregateDataPtr place, const IColumn & column_value, const IColumn & column_weight, size_t row_num) const
{
this->data(place).insertWeighted(
static_cast<const ColumnVector<ArgumentFieldType> &>(*columns[0]).getData()[row_num],
static_cast<const ColumnVector<WeightFieldType> &>(*columns[1]).getData()[row_num]);
static_cast<const ColumnVector<ArgumentFieldType> &>(column_value).getData()[row_num],
static_cast<const ColumnVector<WeightFieldType> &>(column_weight).getData()[row_num]);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).merge(this->data(rhs));
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(place).serialize(buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
this->data(place).deserializeMerge(buf);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
ColumnArray & arr_to = static_cast<ColumnArray &>(to);
ColumnArray::Offsets_t & offsets_to = arr_to.getOffsets();

View File

@ -25,81 +25,82 @@ private:
public:
AggregateFunctionState(AggregateFunctionPtr nested_) : nested_func_owner(nested_), nested_func(nested_func_owner.get()) {}
String getName() const
String getName() const override
{
return nested_func->getName() + "State";
}
DataTypePtr getReturnType() const
DataTypePtr getReturnType() const override
{
return new DataTypeAggregateFunction(nested_func_owner, arguments, params);
}
void setArguments(const DataTypes & arguments_)
void setArguments(const DataTypes & arguments_) override
{
arguments = arguments_;
nested_func->setArguments(arguments);
}
void setParameters(const Array & params_)
void setParameters(const Array & params_) override
{
params = params_;
nested_func->setParameters(params);
}
void create(AggregateDataPtr place) const
void create(AggregateDataPtr place) const override
{
nested_func->create(place);
}
void destroy(AggregateDataPtr place) const noexcept
void destroy(AggregateDataPtr place) const noexcept override
{
nested_func->destroy(place);
}
bool hasTrivialDestructor() const
bool hasTrivialDestructor() const override
{
return nested_func->hasTrivialDestructor();
}
size_t sizeOfData() const
size_t sizeOfData() const override
{
return nested_func->sizeOfData();
}
size_t alignOfData() const
size_t alignOfData() const override
{
return nested_func->alignOfData();
}
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const override
{
nested_func->add(place, columns, row_num);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
nested_func->merge(place, rhs);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
nested_func->serialize(place, buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
nested_func->deserializeMerge(place, buf);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
static_cast<ColumnAggregateFunction &>(to).getData().push_back(const_cast<AggregateDataPtr>(place));
}
/// Аггрегатная функция или состояние аггрегатной функции.
bool isState() const { return true; }
bool isState() const override { return true; }
AggregateFunctionPtr getNestedFunction() const { return nested_func_owner; }
};
}

View File

@ -19,15 +19,15 @@ struct AggregateFunctionSumData
AggregateFunctionSumData() : sum(0) {}
};
/// Считает сумму чисел.
template <typename T>
class AggregateFunctionSum final : public IUnaryAggregateFunction<AggregateFunctionSumData<typename NearestFieldType<T>::Type>, AggregateFunctionSum<T> >
{
public:
String getName() const { return "sum"; }
String getName() const override { return "sum"; }
DataTypePtr getReturnType() const
DataTypePtr getReturnType() const override
{
return new typename DataTypeFromFieldType<typename NearestFieldType<T>::Type>::Type;
}
@ -40,29 +40,29 @@ public:
}
void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const
{
this->data(place).sum += static_cast<const ColumnVector<T> &>(column).getData()[row_num];
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).sum += this->data(rhs).sum;
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
writeBinary(this->data(place).sum, buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
typename NearestFieldType<T>::Type tmp;
readBinary(tmp, buf);
this->data(place).sum += tmp;
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
static_cast<ColumnVector<typename NearestFieldType<T>::Type> &>(to).getData().push_back(this->data(place).sum);
}

View File

@ -3,7 +3,7 @@
#include <city.h>
#include <type_traits>
#include <stats/UniquesHashSet.h>
#include <DB/AggregateFunctions/UniquesHashSet.h>
#include <DB/IO/WriteHelpers.h>
#include <DB/IO/ReadHelpers.h>
@ -11,6 +11,7 @@
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeTuple.h>
#include <DB/Interpreters/AggregationCommon.h>
#include <DB/Common/HashTable/HashSet.h>
@ -18,9 +19,11 @@
#include <DB/Common/CombinedCardinalityEstimator.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Columns/ColumnTuple.h>
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
#include <DB/AggregateFunctions/UniqCombinedBiasData.h>
#include <DB/AggregateFunctions/UniqVariadicHash.h>
namespace DB
@ -30,12 +33,22 @@ namespace DB
struct AggregateFunctionUniqUniquesHashSetData
{
typedef UniquesHashSet<DefaultHash<UInt64> > Set;
typedef UniquesHashSet<DefaultHash<UInt64>> Set;
Set set;
static String getName() { return "uniq"; }
};
/// Для функции, принимающей несколько аргументов. Такая функция сама заранее их хэширует, поэтому здесь используется TrivialHash.
struct AggregateFunctionUniqUniquesHashSetDataForVariadic
{
typedef UniquesHashSet<TrivialHash> Set;
Set set;
static String getName() { return "uniq"; }
};
/// uniqHLL12
template <typename T>
@ -56,6 +69,15 @@ struct AggregateFunctionUniqHLL12Data<String>
static String getName() { return "uniqHLL12"; }
};
struct AggregateFunctionUniqHLL12DataForVariadic
{
typedef HyperLogLogWithSmallSetOptimization<UInt64, 16, 12, TrivialHash> Set;
Set set;
static String getName() { return "uniqHLL12"; }
};
/// uniqExact
template <typename T>
@ -66,7 +88,7 @@ struct AggregateFunctionUniqExactData
/// При создании, хэш-таблица должна быть небольшой.
typedef HashSet<
Key,
DefaultHash<Key>,
HashCRC32<Key>,
HashTableGrower<4>,
HashTableAllocatorWithStackMemory<sizeof(Key) * (1 << 4)>
> Set;
@ -234,7 +256,7 @@ struct OneAdder<T, Data, typename std::enable_if<
std::is_same<Data, AggregateFunctionUniqHLL12Data<T> >::value>::type>
{
template <typename T2 = T>
static void addOne(Data & data, const IColumn & column, size_t row_num,
static void addImpl(Data & data, const IColumn & column, size_t row_num,
typename std::enable_if<!std::is_same<T2, String>::value>::type * = nullptr)
{
const auto & value = static_cast<const ColumnVector<T2> &>(column).getData()[row_num];
@ -242,7 +264,7 @@ struct OneAdder<T, Data, typename std::enable_if<
}
template <typename T2 = T>
static void addOne(Data & data, const IColumn & column, size_t row_num,
static void addImpl(Data & data, const IColumn & column, size_t row_num,
typename std::enable_if<std::is_same<T2, String>::value>::type * = nullptr)
{
StringRef value = column.getDataAt(row_num);
@ -258,7 +280,7 @@ struct OneAdder<T, Data, typename std::enable_if<
std::is_same<Data, AggregateFunctionUniqCombinedData<T> >::value>::type>
{
template <typename T2 = T>
static void addOne(Data & data, const IColumn & column, size_t row_num,
static void addImpl(Data & data, const IColumn & column, size_t row_num,
typename std::enable_if<!std::is_same<T2, String>::value>::type * = nullptr)
{
const auto & value = static_cast<const ColumnVector<T2> &>(column).getData()[row_num];
@ -266,7 +288,7 @@ struct OneAdder<T, Data, typename std::enable_if<
}
template <typename T2 = T>
static void addOne(Data & data, const IColumn & column, size_t row_num,
static void addImpl(Data & data, const IColumn & column, size_t row_num,
typename std::enable_if<std::is_same<T2, String>::value>::type * = nullptr)
{
StringRef value = column.getDataAt(row_num);
@ -279,14 +301,14 @@ struct OneAdder<T, Data, typename std::enable_if<
std::is_same<Data, AggregateFunctionUniqExactData<T> >::value>::type>
{
template <typename T2 = T>
static void addOne(Data & data, const IColumn & column, size_t row_num,
static void addImpl(Data & data, const IColumn & column, size_t row_num,
typename std::enable_if<!std::is_same<T2, String>::value>::type * = nullptr)
{
data.set.insert(static_cast<const ColumnVector<T2> &>(column).getData()[row_num]);
}
template <typename T2 = T>
static void addOne(Data & data, const IColumn & column, size_t row_num,
static void addImpl(Data & data, const IColumn & column, size_t row_num,
typename std::enable_if<std::is_same<T2, String>::value>::type * = nullptr)
{
StringRef value = column.getDataAt(row_num);
@ -303,14 +325,14 @@ struct OneAdder<T, Data, typename std::enable_if<
}
/// Приближённо вычисляет количество различных значений.
/// Приближённо или точно вычисляет количество различных значений.
template <typename T, typename Data>
class AggregateFunctionUniq final : public IUnaryAggregateFunction<Data, AggregateFunctionUniq<T, Data> >
{
public:
String getName() const { return Data::getName(); }
String getName() const override { return Data::getName(); }
DataTypePtr getReturnType() const
DataTypePtr getReturnType() const override
{
return new DataTypeUInt64;
}
@ -319,27 +341,82 @@ public:
{
}
void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const
{
detail::OneAdder<T, Data>::addOne(this->data(place), column, row_num);
detail::OneAdder<T, Data>::addImpl(this->data(place), column, row_num);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).set.merge(this->data(rhs).set);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(place).set.write(buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
this->data(place).set.readAndMerge(buf);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
static_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).set.size());
}
};
/** Для нескольких аргументов. Для вычисления, хэширует их.
* Можно передать несколько аргументов как есть; также можно передать один аргумент - кортеж.
* Но (для возможности эффективной реализации), нельзя передать несколько аргументов, среди которых есть кортежи.
*/
template <typename Data, bool argument_is_tuple>
class AggregateFunctionUniqVariadic final : public IAggregateFunctionHelper<Data>
{
private:
static constexpr bool is_exact = std::is_same<Data, AggregateFunctionUniqExactData<String>>::value;
size_t num_args = 0;
public:
String getName() const override { return Data::getName(); }
DataTypePtr getReturnType() const override
{
return new DataTypeUInt64;
}
void setArguments(const DataTypes & arguments) override
{
if (argument_is_tuple)
num_args = typeid_cast<const DataTypeTuple &>(*arguments[0]).getElements().size();
else
num_args = arguments.size();
}
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const override
{
this->data(place).set.insert(UniqVariadicHash<is_exact, argument_is_tuple>::apply(num_args, columns, row_num));
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).set.merge(this->data(rhs).set);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(place).set.write(buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
this->data(place).set.readAndMerge(buf);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
static_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).set.size());
}

View File

@ -2,7 +2,9 @@
#include <DB/Core/FieldVisitors.h>
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
#include <DB/AggregateFunctions/UniqVariadicHash.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeTuple.h>
namespace DB
@ -102,7 +104,7 @@ struct __attribute__((__packed__)) AggregateFunctionUniqUpToData
}
void addOne(const IColumn & column, size_t row_num, UInt8 threshold)
void addImpl(const IColumn & column, size_t row_num, UInt8 threshold)
{
insert(static_cast<const ColumnVector<T> &>(column).getData()[row_num], threshold);
}
@ -113,7 +115,7 @@ struct __attribute__((__packed__)) AggregateFunctionUniqUpToData
template <>
struct AggregateFunctionUniqUpToData<String> : AggregateFunctionUniqUpToData<UInt64>
{
void addOne(const IColumn & column, size_t row_num, UInt8 threshold)
void addImpl(const IColumn & column, size_t row_num, UInt8 threshold)
{
/// Имейте ввиду, что вычисление приближённое.
StringRef value = column.getDataAt(row_num);
@ -131,14 +133,14 @@ private:
UInt8 threshold = 5; /// Значение по-умолчанию, если параметр не указан.
public:
size_t sizeOfData() const
size_t sizeOfData() const override
{
return sizeof(AggregateFunctionUniqUpToData<T>) + sizeof(T) * threshold;
}
String getName() const { return "uniqUpTo"; }
String getName() const override { return "uniqUpTo"; }
DataTypePtr getReturnType() const
DataTypePtr getReturnType() const override
{
return new DataTypeUInt64;
}
@ -147,7 +149,7 @@ public:
{
}
void setParameters(const Array & params)
void setParameters(const Array & params) override
{
if (params.size() != 1)
throw Exception("Aggregate function " + getName() + " requires exactly one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
@ -161,27 +163,100 @@ public:
threshold = threshold_param;
}
void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const
{
this->data(place).addOne(column, row_num, threshold);
this->data(place).addImpl(column, row_num, threshold);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).merge(this->data(rhs), threshold);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(place).write(buf, threshold);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
this->data(place).readAndMerge(buf, threshold);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
static_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).size());
}
};
/** Для нескольких аргументов. Для вычисления, хэширует их.
* Можно передать несколько аргументов как есть; также можно передать один аргумент - кортеж.
* Но (для возможности эффективной реализации), нельзя передать несколько аргументов, среди которых есть кортежи.
*/
template <bool argument_is_tuple>
class AggregateFunctionUniqUpToVariadic final : public IAggregateFunctionHelper<AggregateFunctionUniqUpToData<UInt64>>
{
private:
size_t num_args = 0;
UInt8 threshold = 5; /// Значение по-умолчанию, если параметр не указан.
public:
size_t sizeOfData() const override
{
return sizeof(AggregateFunctionUniqUpToData<UInt64>) + sizeof(UInt64) * threshold;
}
String getName() const override { return "uniqUpTo"; }
DataTypePtr getReturnType() const override
{
return new DataTypeUInt64;
}
void setArguments(const DataTypes & arguments) override
{
if (argument_is_tuple)
num_args = typeid_cast<const DataTypeTuple &>(*arguments[0]).getElements().size();
else
num_args = arguments.size();
}
void setParameters(const Array & params) override
{
if (params.size() != 1)
throw Exception("Aggregate function " + getName() + " requires exactly one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
UInt64 threshold_param = apply_visitor(FieldVisitorConvertToNumber<UInt64>(), params[0]);
if (threshold_param > uniq_upto_max_threshold)
throw Exception("Too large parameter for aggregate function " + getName() + ". Maximum: " + toString(uniq_upto_max_threshold),
ErrorCodes::ARGUMENT_OUT_OF_BOUND);
threshold = threshold_param;
}
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const override
{
this->data(place).insert(UniqVariadicHash<false, argument_is_tuple>::apply(num_args, columns, row_num), threshold);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).merge(this->data(rhs), threshold);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(place).write(buf, threshold);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
this->data(place).readAndMerge(buf, threshold);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
static_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).size());
}

View File

@ -27,14 +27,14 @@ private:
DataTypePtr type_val;
public:
String getName() const { return (0 == strcmp(Data::ValueData_t::name(), "min")) ? "argMin" : "argMax"; }
String getName() const override { return (0 == strcmp(Data::ValueData_t::name(), "min")) ? "argMin" : "argMax"; }
DataTypePtr getReturnType() const
DataTypePtr getReturnType() const override
{
return type_res;
}
void setArguments(const DataTypes & arguments)
void setArguments(const DataTypes & arguments) override
{
if (arguments.size() != 2)
throw Exception("Aggregate function " + getName() + " requires exactly two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
@ -43,25 +43,25 @@ public:
type_val = arguments[1];
}
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const override
{
if (this->data(place).value.changeIfBetter(*columns[1], row_num))
this->data(place).result.change(*columns[0], row_num);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
if (this->data(place).value.changeIfBetter(this->data(rhs).value))
this->data(place).result.change(this->data(rhs).result);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(place).result.write(buf, *type_res.get());
this->data(place).value.write(buf, *type_val.get());
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
Data rhs; /// Для строчек не очень оптимально, так как может делаться одна лишняя аллокация.
@ -72,7 +72,7 @@ public:
this->data(place).result.change(rhs.result);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
this->data(place).result.insertResultInto(to);
}

View File

@ -5,6 +5,7 @@
#include <DB/Columns/ColumnVector.h>
#include <DB/Columns/ColumnString.h>
#include <DB/DataTypes/DataTypeAggregateFunction.h>
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
@ -531,9 +532,9 @@ private:
DataTypePtr type;
public:
String getName() const { return Data::name(); }
String getName() const override { return Data::name(); }
DataTypePtr getReturnType() const
DataTypePtr getReturnType() const override
{
return type;
}
@ -541,25 +542,28 @@ public:
void setArgument(const DataTypePtr & argument)
{
type = argument;
if (typeid_cast<const DataTypeAggregateFunction *>(type.get()))
throw Exception("Illegal type " + type->getName() + " of argument of aggregate function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const
{
this->data(place).changeIfBetter(column, row_num);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).changeIfBetter(this->data(rhs));
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(place).write(buf, *type.get());
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
Data rhs; /// Для строчек не очень оптимально, так как может делаться одна лишняя аллокация.
rhs.read(buf, *type.get());
@ -567,7 +571,7 @@ public:
this->data(place).changeIfBetter(rhs);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
this->data(place).insertResultInto(to);
}

View File

@ -122,14 +122,14 @@ public:
return new DataTypeFloat64;
}
void setArgument(const DataTypePtr & argument) override
void setArgument(const DataTypePtr & argument)
{
if (!argument->behavesAsNumber())
throw Exception("Illegal type " + argument->getName() + " of argument for aggregate function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const
{
this->data(place).update(column, row_num);
}
@ -400,7 +400,7 @@ public:
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
void addOne(AggregateDataPtr place, const IColumn & column_left, const IColumn & column_right, size_t row_num) const
void addImpl(AggregateDataPtr place, const IColumn & column_left, const IColumn & column_right, size_t row_num) const
{
this->data(place).update(column_left, column_right, row_num);
}

View File

@ -104,28 +104,28 @@ protected:
static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast<const Data*>(place); }
public:
void create(AggregateDataPtr place) const
void create(AggregateDataPtr place) const override
{
new (place) Data;
}
void destroy(AggregateDataPtr place) const noexcept
void destroy(AggregateDataPtr place) const noexcept override
{
data(place).~Data();
}
bool hasTrivialDestructor() const
bool hasTrivialDestructor() const override
{
return __has_trivial_destructor(Data);
}
size_t sizeOfData() const
size_t sizeOfData() const override
{
return sizeof(Data);
}
/// NOTE: Сейчас не используется (структуры с состоянием агрегации кладутся без выравнивания).
size_t alignOfData() const
size_t alignOfData() const override
{
return __alignof__(Data);
}

View File

@ -8,11 +8,12 @@ namespace DB
template <typename T, typename Derived>
class IBinaryAggregateFunction : public IAggregateFunctionHelper<T>
{
private:
Derived & getDerived() { return static_cast<Derived &>(*this); }
const Derived & getDerived() const { return static_cast<const Derived &>(*this); }
public:
void setArguments(const DataTypes & arguments)
void setArguments(const DataTypes & arguments) override final
{
if (arguments.size() != 2)
throw Exception{
@ -23,9 +24,9 @@ public:
getDerived().setArgumentsImpl(arguments);
}
void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num) const
void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num) const override final
{
getDerived().addOne(place, *columns[0], *columns[1], row_num);
getDerived().addImpl(place, *columns[0], *columns[1], row_num);
}
};

View File

@ -11,9 +11,13 @@ namespace DB
template <typename T, typename Derived>
class INullaryAggregateFunction : public IAggregateFunctionHelper<T>
{
private:
Derived & getDerived() { return static_cast<Derived &>(*this); }
const Derived & getDerived() const { return static_cast<const Derived &>(*this); }
public:
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
void setArguments(const DataTypes & arguments)
void setArguments(const DataTypes & arguments) override final
{
if (arguments.size() != 0)
throw Exception("Passed " + toString(arguments.size()) + " arguments to nullary aggregate function " + this->getName(),
@ -21,13 +25,13 @@ public:
}
/// Добавить значение.
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const override final
{
static_cast<const Derived &>(*this).addZero(place);
getDerived().addImpl(place);
}
/** Реализуйте это в классе-наследнике:
* void addZero(AggregateDataPtr place) const;
* void addImpl(AggregateDataPtr place) const;
*/
};

View File

@ -11,25 +11,29 @@ namespace DB
template <typename T, typename Derived>
class IUnaryAggregateFunction : public IAggregateFunctionHelper<T>
{
private:
Derived & getDerived() { return static_cast<Derived &>(*this); }
const Derived & getDerived() const { return static_cast<const Derived &>(*this); }
public:
void setArguments(const DataTypes & arguments)
void setArguments(const DataTypes & arguments) override final
{
if (arguments.size() != 1)
throw Exception("Passed " + toString(arguments.size()) + " arguments to unary aggregate function " + this->getName(),
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
setArgument(arguments[0]);
getDerived().setArgument(arguments[0]);
}
virtual void setArgument(const DataTypePtr & argument) = 0;
/// Добавить значение.
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const override final
{
static_cast<const Derived &>(*this).addOne(place, *columns[0], row_num);
getDerived().addImpl(place, *columns[0], row_num);
}
/** Реализуйте это в классе-наследнике:
* void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const;
* void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const;
* void setArgument(const DataTypePtr & argument);
*/
};

View File

@ -0,0 +1,237 @@
#pragma once
#include <limits>
#include <algorithm>
#include <climits>
#include <sstream>
#include <common/Common.h>
#include <DB/IO/ReadBuffer.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/IO/WriteHelpers.h>
#include <DB/Common/PODArray.h>
#include <Poco/Exception.h>
#include <boost/random.hpp>
/// Реализация алгоритма Reservoir Sampling. Инкрементально выбирает из добавленных объектов случайное подмножество размера sample_count.
/// Умеет приближенно получать квантили.
/// Вызов quantile занимает O(sample_count log sample_count), если после предыдущего вызова quantile был хотя бы один вызов insert. Иначе, O(1).
/// То есть, имеет смысл сначала добавлять, потом получать квантили, не добавляя.
const size_t DEFAULT_SAMPLE_COUNT = 8192;
/// Что делать, если нет ни одного значения - кинуть исключение, или вернуть 0 или NaN в случае double?
namespace ReservoirSamplerOnEmpty
{
enum Enum
{
THROW,
RETURN_NAN_OR_ZERO,
};
}
template<typename ResultType, bool IsFloatingPoint>
struct NanLikeValueConstructor
{
static ResultType getValue()
{
return std::numeric_limits<ResultType>::quiet_NaN();
}
};
template<typename ResultType>
struct NanLikeValueConstructor<ResultType, false>
{
static ResultType getValue()
{
return ResultType();
}
};
template<typename T, ReservoirSamplerOnEmpty::Enum OnEmpty = ReservoirSamplerOnEmpty::THROW, typename Comparer = std::less<T> >
class ReservoirSampler
{
public:
ReservoirSampler(size_t sample_count_ = DEFAULT_SAMPLE_COUNT)
: sample_count(sample_count_)
{
rng.seed(123456);
}
void clear()
{
samples.clear();
sorted = false;
total_values = 0;
rng.seed(123456);
}
void insert(const T & v)
{
sorted = false;
++total_values;
if (samples.size() < sample_count)
{
samples.push_back(v);
}
else
{
UInt64 rnd = genRandom(total_values);
if (rnd < sample_count)
samples[rnd] = v;
}
}
size_t size() const
{
return total_values;
}
T quantileNearest(double level)
{
if (samples.empty())
return onEmpty<T>();
sortIfNeeded();
double index = level * (samples.size() - 1);
size_t int_index = static_cast<size_t>(index + 0.5);
int_index = std::max(0LU, std::min(samples.size() - 1, int_index));
return samples[int_index];
}
/** Если T не числовой тип, использование этого метода вызывает ошибку компиляции,
* но использование класса ошибки не вызывает. SFINAE.
*/
double quantileInterpolated(double level)
{
if (samples.empty())
return onEmpty<double>();
sortIfNeeded();
double index = std::max(0., std::min(samples.size() - 1., level * (samples.size() - 1)));
/// Чтобы получить значение по дробному индексу линейно интерполируем между соседними значениями.
size_t left_index = static_cast<size_t>(index);
size_t right_index = left_index + 1;
if (right_index == samples.size())
return samples[left_index];
double left_coef = right_index - index;
double right_coef = index - left_index;
return samples[left_index] * left_coef + samples[right_index] * right_coef;
}
void merge(const ReservoirSampler<T, OnEmpty> & b)
{
if (sample_count != b.sample_count)
throw Poco::Exception("Cannot merge ReservoirSampler's with different sample_count");
sorted = false;
if (b.total_values <= sample_count)
{
for (size_t i = 0; i < b.samples.size(); ++i)
insert(b.samples[i]);
}
else if (total_values <= sample_count)
{
Array from = std::move(samples);
samples.assign(b.samples.begin(), b.samples.end());
total_values = b.total_values;
for (size_t i = 0; i < from.size(); ++i)
insert(from[i]);
}
else
{
randomShuffle(samples);
total_values += b.total_values;
for (size_t i = 0; i < sample_count; ++i)
{
UInt64 rnd = genRandom(total_values);
if (rnd < b.total_values)
samples[i] = b.samples[i];
}
}
}
void read(DB::ReadBuffer & buf)
{
DB::readIntBinary<size_t>(sample_count, buf);
DB::readIntBinary<size_t>(total_values, buf);
samples.resize(std::min(total_values, sample_count));
std::string rng_string;
DB::readStringBinary(rng_string, buf);
std::istringstream rng_stream(rng_string);
rng_stream >> rng;
for (size_t i = 0; i < samples.size(); ++i)
DB::readBinary(samples[i], buf);
sorted = false;
}
void write(DB::WriteBuffer & buf) const
{
DB::writeIntBinary<size_t>(sample_count, buf);
DB::writeIntBinary<size_t>(total_values, buf);
std::ostringstream rng_stream;
rng_stream << rng;
DB::writeStringBinary(rng_stream.str(), buf);
for (size_t i = 0; i < std::min(sample_count, total_values); ++i)
DB::writeBinary(samples[i], buf);
}
private:
friend void qdigest_test(int normal_size, UInt64 value_limit, const std::vector<UInt64> & values, int queries_count, bool verbose);
friend void rs_perf_test();
/// Будем выделять немного памяти на стеке - чтобы избежать аллокаций, когда есть много объектов с маленьким количеством элементов.
static constexpr size_t bytes_on_stack = 64;
using Array = DB::PODArray<T, bytes_on_stack / sizeof(T), AllocatorWithStackMemory<Allocator<false>, bytes_on_stack>>;
size_t sample_count;
size_t total_values = 0;
Array samples;
boost::taus88 rng;
bool sorted = false;
UInt64 genRandom(size_t lim)
{
/// При большом количестве значений будем генерировать случайные числа в несколько раз медленнее.
if (lim <= static_cast<UInt64>(rng.max()))
return static_cast<UInt32>(rng()) % static_cast<UInt32>(lim);
else
return (static_cast<UInt64>(rng()) * (static_cast<UInt64>(rng.max()) + 1ULL) + static_cast<UInt64>(rng())) % lim;
}
void randomShuffle(Array & v)
{
for (size_t i = 1; i < v.size(); ++i)
{
size_t j = genRandom(i + 1);
std::swap(v[i], v[j]);
}
}
void sortIfNeeded()
{
if (sorted)
return;
sorted = true;
std::sort(samples.begin(), samples.end(), Comparer());
}
template <typename ResultType>
ResultType onEmpty() const
{
if (OnEmpty == ReservoirSamplerOnEmpty::THROW)
throw Poco::Exception("Quantile of empty ReservoirSampler");
else
return NanLikeValueConstructor<ResultType, std::is_floating_point<ResultType>::value>::getValue();
}
};

View File

@ -1,16 +1,16 @@
#pragma once
#include <limits>
#include <vector>
#include <algorithm>
#include <climits>
#include <sstream>
#include <stats/ReservoirSampler.h>
#include <DB/AggregateFunctions/ReservoirSampler.h>
#include <common/Common.h>
#include <DB/Common/HashTable/Hash.h>
#include <DB/IO/ReadBuffer.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/IO/WriteHelpers.h>
#include <DB/Common/PODArray.h>
#include <Poco/Exception.h>
#include <boost/random.hpp>
@ -150,13 +150,15 @@ public:
}
private:
friend void rs_perf_test();
friend void qdigest_test(int, UInt64, const std::vector<UInt64> &, int, bool);
/// Будем выделять немного памяти на стеке - чтобы избежать аллокаций, когда есть много объектов с маленьким количеством элементов.
static constexpr size_t bytes_on_stack = 64;
using Element = std::pair<T, UInt32>;
using Array = DB::PODArray<Element, bytes_on_stack / sizeof(Element), AllocatorWithStackMemory<Allocator<false>, bytes_on_stack>>;
size_t sample_count;
size_t total_values{};
bool sorted{};
std::vector<std::pair<T, UInt32>> samples;
Array samples;
UInt8 skip_degree{};
void insertImpl(const T & v, const UInt32 hash)

View File

@ -0,0 +1,136 @@
#pragma once
#include <city.h>
#include <DB/Core/Defines.h>
#include <DB/Common/SipHash.h>
#include <DB/Common/UInt128.h>
#include <DB/Columns/ColumnTuple.h>
namespace DB
{
/** Хэширует набор аргументов агрегатной функции
* для вычисления количества уникальных значений
* и добавляет их в множество.
*
* Четыре варианта (2 x 2):
*
* - для приближённого вычисления, использует некриптографическую 64-битную хэш-функцию;
* - для точного вычисления, использует криптографическую 128-битную хэш-функцию;
*
* - для нескольких аргументов, переданных обычным способом;
* - для одного аргумента-кортежа.
*/
template <bool exact, bool for_tuple>
struct UniqVariadicHash;
template <>
struct UniqVariadicHash<false, false>
{
static inline UInt64 apply(size_t num_args, const IColumn ** columns, size_t row_num)
{
UInt64 hash;
const IColumn ** column = columns;
const IColumn ** columns_end = column + num_args;
{
StringRef value = (*column)->getDataAt(row_num);
hash = CityHash64(value.data, value.size);
++column;
}
while (column < columns_end)
{
StringRef value = (*column)->getDataAt(row_num);
hash = Hash128to64(uint128(CityHash64(value.data, value.size), hash));
++column;
}
return hash;
}
};
template <>
struct UniqVariadicHash<false, true>
{
static inline UInt64 apply(size_t num_args, const IColumn ** columns, size_t row_num)
{
UInt64 hash;
const Columns & tuple_columns = static_cast<const ColumnTuple *>(columns[0])->getColumns();
const ColumnPtr * column = tuple_columns.data();
const ColumnPtr * columns_end = column + num_args;
{
StringRef value = column->get()->getDataAt(row_num);
hash = CityHash64(value.data, value.size);
++column;
}
while (column < columns_end)
{
StringRef value = column->get()->getDataAt(row_num);
hash = Hash128to64(uint128(CityHash64(value.data, value.size), hash));
++column;
}
return hash;
}
};
template <>
struct UniqVariadicHash<true, false>
{
static inline UInt128 apply(size_t num_args, const IColumn ** columns, size_t row_num)
{
SipHash hash;
const IColumn ** column = columns;
const IColumn ** columns_end = column + num_args;
while (column < columns_end)
{
StringRef value = (*column)->getDataAt(row_num);
hash.update(reinterpret_cast<const char *>(&value.size), sizeof(value.size));
hash.update(value.data, value.size);
++column;
}
UInt128 key;
hash.get128(key.first, key.second);
return key;
}
};
template <>
struct UniqVariadicHash<true, true>
{
static inline UInt128 apply(size_t num_args, const IColumn ** columns, size_t row_num)
{
SipHash hash;
const Columns & tuple_columns = static_cast<const ColumnTuple *>(columns[0])->getColumns();
const ColumnPtr * column = tuple_columns.data();
const ColumnPtr * columns_end = column + num_args;
while (column < columns_end)
{
StringRef value = column->get()->getDataAt(row_num);
hash.update(reinterpret_cast<const char *>(&value.size), sizeof(value.size));
hash.update(value.data, value.size);
++column;
}
UInt128 key;
hash.get128(key.first, key.second);
return key;
}
};
}

View File

@ -0,0 +1,540 @@
#pragma once
#include <math.h>
#include <common/Common.h>
#include <DB/IO/WriteBuffer.h>
#include <DB/IO/WriteHelpers.h>
#include <DB/IO/ReadBuffer.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/IO/VarInt.h>
#include <DB/Common/HashTable/HashTableAllocator.h>
#include <DB/Common/HashTable/Hash.h>
/** Приближённый рассчёт чего-угодно, как правило, построен по следующей схеме:
* - для рассчёта значения X используется некоторая структура данных;
* - в структуру данных добавляются не все значения, а только избранные (согласно некоторому критерию избранности);
* - после обработки всех элементов, структура данных находится в некотором состоянии S;
* - в качестве приближённого значения X возвращается значание, посчитанное по принципу максимального правдоподобия:
* при каком реальном значении X, вероятность нахождения структуры данных в полученном состоянии S максимальна.
*/
/** В частности, то, что описано ниже, можно найти по названию BJKST algorithm.
*/
/** Очень простое хэш-множество для приближённого подсчёта количества уникальных значений.
* Работает так:
* - вставлять можно UInt64;
* - перед вставкой, сначала вычисляется хэш-функция UInt64 -> UInt32;
* - исходное значение не сохраняется (теряется);
* - далее все операции производятся с этими хэшами;
* - хэш таблица построена по схеме:
* - open addressing (один буфер, позиция в буфере вычисляется взятием остатка от деления на его размер);
* - linear probing (если в ячейке уже есть значение, то берётся ячейка, следующая за ней и т. д.);
* - отсутствующее значение кодируется нулём; чтобы запомнить наличие в множестве нуля, используется отдельная переменная типа bool;
* - рост буфера в 2 раза при заполнении более чем на 50%;
* - если в множестве больше UNIQUES_HASH_MAX_SIZE элементов, то из множества удаляются все элементы,
* не делящиеся на 2, и затем все элементы, которые не делятся на 2, не вставляются в множество;
* - если ситуация повторяется, то берутся только элементы делящиеся на 4 и т. п.
* - метод size() возвращает приблизительное количество элементов, которые были вставлены в множество;
* - есть методы для быстрого чтения и записи в бинарный и текстовый вид.
*/
/// Максимальная степень размера буфера перед тем, как значения будут выкидываться
#define UNIQUES_HASH_MAX_SIZE_DEGREE 17
/// Максимальное количество элементов перед тем, как значения будут выкидываться
#define UNIQUES_HASH_MAX_SIZE (1 << (UNIQUES_HASH_MAX_SIZE_DEGREE - 1))
/** Количество младших бит, использующихся для прореживания. Оставшиеся старшие биты используются для определения позиции в хэш-таблице.
* (старшие биты берутся потому что младшие будут постоянными после выкидывания части значений)
*/
#define UNIQUES_HASH_BITS_FOR_SKIP (32 - UNIQUES_HASH_MAX_SIZE_DEGREE)
/// Начальная степень размера буфера
#define UNIQUES_HASH_SET_INITIAL_SIZE_DEGREE 4
/** Эта хэш-функция не самая оптимальная, но состояния UniquesHashSet, посчитанные с ней,
* хранятся много где на дисках (в Метраже), поэтому она продолжает использоваться.
*/
struct UniquesHashSetDefaultHash
{
size_t operator() (UInt64 x) const
{
return intHash32<0>(x);
}
};
template <typename Hash = UniquesHashSetDefaultHash>
class UniquesHashSet : private HashTableAllocatorWithStackMemory<(1 << UNIQUES_HASH_SET_INITIAL_SIZE_DEGREE) * sizeof(UInt32)>
{
private:
typedef UInt64 Value_t;
typedef UInt32 HashValue_t;
typedef HashTableAllocatorWithStackMemory<(1 << UNIQUES_HASH_SET_INITIAL_SIZE_DEGREE) * sizeof(UInt32)> Allocator;
UInt32 m_size; /// Количество элементов
UInt8 size_degree; /// Размер таблицы в виде степени двух
UInt8 skip_degree; /// Пропускать элементы не делящиеся на 2 ^ skip_degree
bool has_zero; /// Хэш-таблица содержит элемент со значением хэш-функции = 0.
HashValue_t * buf;
#ifdef UNIQUES_HASH_SET_COUNT_COLLISIONS
/// Для профилирования.
mutable size_t collisions;
#endif
void alloc(UInt8 new_size_degree)
{
buf = reinterpret_cast<HashValue_t *>(Allocator::alloc((1 << new_size_degree) * sizeof(buf[0])));
size_degree = new_size_degree;
}
void free()
{
if (buf)
{
Allocator::free(buf, buf_size() * sizeof(buf[0]));
buf = nullptr;
}
}
inline size_t buf_size() const { return 1 << size_degree; }
inline size_t max_fill() const { return 1 << (size_degree - 1); }
inline size_t mask() const { return buf_size() - 1; }
inline size_t place(HashValue_t x) const { return (x >> UNIQUES_HASH_BITS_FOR_SKIP) & mask(); }
/// Значение делится на 2 ^ skip_degree
inline bool good(HashValue_t hash) const
{
return hash == ((hash >> skip_degree) << skip_degree);
}
HashValue_t hash(Value_t key) const
{
return Hash()(key);
}
/// Удалить все значения, хэши которых не делятся на 2 ^ skip_degree
void rehash()
{
for (size_t i = 0; i < buf_size(); ++i)
{
if (buf[i] && !good(buf[i]))
{
buf[i] = 0;
--m_size;
}
}
/** После удаления элементов, возможно, освободилось место для элементов,
* которые были помещены дальше, чем нужно, из-за коллизии.
* Надо переместить их.
*/
for (size_t i = 0; i < buf_size(); ++i)
{
if (unlikely(buf[i] && i != place(buf[i])))
{
HashValue_t x = buf[i];
buf[i] = 0;
reinsertImpl(x);
}
}
}
/// Увеличить размер буфера в 2 раза или до new_size_degree, если указана ненулевая.
void resize(size_t new_size_degree = 0)
{
size_t old_size = buf_size();
if (!new_size_degree)
new_size_degree = size_degree + 1;
/// Расширим пространство.
buf = reinterpret_cast<HashValue_t *>(Allocator::realloc(buf, old_size * sizeof(buf[0]), (1 << new_size_degree) * sizeof(buf[0])));
size_degree = new_size_degree;
/** Теперь некоторые элементы может потребоваться переместить на новое место.
* Элемент может остаться на месте, или переместиться в новое место "справа",
* или переместиться левее по цепочке разрешения коллизий, из-за того, что элементы левее него были перемещены в новое место "справа".
* Также имеется особый случай:
* если элемент должен был быть в конце старого буфера, [ x]
* но находится в начале из-за цепочки разрешения коллизий, [o x]
* то после ресайза, он сначала снова окажется не на своём месте, [ xo ]
* и для того, чтобы перенести его куда надо,
* надо будет после переноса всех элементов из старой половинки [ o x ]
* обработать ещё хвостик из цепочки разрешения коллизий сразу после неё [ o x ]
* Именно для этого написано || buf[i] ниже.
*/
for (size_t i = 0; i < old_size || buf[i]; ++i)
{
HashValue_t x = buf[i];
if (!x)
continue;
size_t place_value = place(x);
/// Элемент на своём месте.
if (place_value == i)
continue;
while (buf[place_value] && buf[place_value] != x)
{
++place_value;
place_value &= mask();
#ifdef UNIQUES_HASH_SET_COUNT_COLLISIONS
++collisions;
#endif
}
/// Элемент остался на своём месте.
if (buf[place_value] == x)
continue;
buf[place_value] = x;
buf[i] = 0;
}
}
/// Вставить значение.
void insertImpl(HashValue_t x)
{
if (x == 0)
{
m_size += !has_zero;
has_zero = true;
return;
}
size_t place_value = place(x);
while (buf[place_value] && buf[place_value] != x)
{
++place_value;
place_value &= mask();
#ifdef UNIQUES_HASH_SET_COUNT_COLLISIONS
++collisions;
#endif
}
if (buf[place_value] == x)
return;
buf[place_value] = x;
++m_size;
}
/** Вставить в новый буфер значение, которое было в старом буфере.
* Используется при увеличении размера буфера, а также при чтении из файла.
*/
void reinsertImpl(HashValue_t x)
{
size_t place_value = place(x);
while (buf[place_value])
{
++place_value;
place_value &= mask();
#ifdef UNIQUES_HASH_SET_COUNT_COLLISIONS
++collisions;
#endif
}
buf[place_value] = x;
}
/** Если хэш-таблица достаточно заполнена, то сделать resize.
* Если элементов слишком много - то выкидывать половину, пока их не станет достаточно мало.
*/
void shrinkIfNeed()
{
if (unlikely(m_size > max_fill()))
{
if (m_size > UNIQUES_HASH_MAX_SIZE)
{
while (m_size > UNIQUES_HASH_MAX_SIZE)
{
++skip_degree;
rehash();
}
}
else
resize();
}
}
public:
UniquesHashSet() :
m_size(0),
skip_degree(0),
has_zero(false)
{
alloc(UNIQUES_HASH_SET_INITIAL_SIZE_DEGREE);
#ifdef UNIQUES_HASH_SET_COUNT_COLLISIONS
collisions = 0;
#endif
}
UniquesHashSet(const UniquesHashSet & rhs)
: m_size(rhs.m_size), skip_degree(rhs.skip_degree), has_zero(rhs.has_zero)
{
alloc(rhs.size_degree);
memcpy(buf, rhs.buf, buf_size() * sizeof(buf[0]));
}
UniquesHashSet & operator= (const UniquesHashSet & rhs)
{
if (size_degree != rhs.size_degree)
{
free();
alloc(rhs.size_degree);
}
m_size = rhs.m_size;
skip_degree = rhs.skip_degree;
has_zero = rhs.has_zero;
memcpy(buf, rhs.buf, buf_size() * sizeof(buf[0]));
return *this;
}
~UniquesHashSet()
{
free();
}
void insert(Value_t x)
{
HashValue_t hash_value = hash(x);
if (!good(hash_value))
return;
insertImpl(hash_value);
shrinkIfNeed();
}
size_t size() const
{
if (0 == skip_degree)
return m_size;
size_t res = m_size * (1 << skip_degree);
/** Псевдослучайный остаток - для того, чтобы не было видно,
* что количество делится на степень двух.
*/
res += (intHashCRC32(m_size) & ((1 << skip_degree) - 1));
/** Коррекция систематической погрешности из-за коллизий при хэшировании в UInt32.
* Формула fixed_res(res)
* - при каком количестве разных элементов fixed_res,
* при их случайном разбрасывании по 2^32 корзинам,
* получается в среднем res заполненных корзин.
*/
size_t p32 = 1ULL << 32;
size_t fixed_res = round(p32 * (log(p32) - log(p32 - res)));
return fixed_res;
}
void merge(const UniquesHashSet & rhs)
{
if (rhs.skip_degree > skip_degree)
{
skip_degree = rhs.skip_degree;
rehash();
}
if (!has_zero && rhs.has_zero)
{
has_zero = true;
++m_size;
shrinkIfNeed();
}
for (size_t i = 0; i < rhs.buf_size(); ++i)
{
if (rhs.buf[i] && good(rhs.buf[i]))
{
insertImpl(rhs.buf[i]);
shrinkIfNeed();
}
}
}
void write(DB::WriteBuffer & wb) const
{
if (m_size > UNIQUES_HASH_MAX_SIZE)
throw Poco::Exception("Cannot write UniquesHashSet: too large size_degree.");
DB::writeIntBinary(skip_degree, wb);
DB::writeVarUInt(m_size, wb);
if (has_zero)
{
HashValue_t x = 0;
DB::writeIntBinary(x, wb);
}
for (size_t i = 0; i < buf_size(); ++i)
if (buf[i])
DB::writeIntBinary(buf[i], wb);
}
void read(DB::ReadBuffer & rb)
{
has_zero = false;
DB::readIntBinary(skip_degree, rb);
DB::readVarUInt(m_size, rb);
if (m_size > UNIQUES_HASH_MAX_SIZE)
throw Poco::Exception("Cannot read UniquesHashSet: too large size_degree.");
free();
UInt8 new_size_degree = m_size <= 1
? UNIQUES_HASH_SET_INITIAL_SIZE_DEGREE
: std::max(UNIQUES_HASH_SET_INITIAL_SIZE_DEGREE, static_cast<int>(log2(m_size - 1)) + 2);
alloc(new_size_degree);
for (size_t i = 0; i < m_size; ++i)
{
HashValue_t x = 0;
DB::readIntBinary(x, rb);
if (x == 0)
has_zero = true;
else
reinsertImpl(x);
}
}
void readAndMerge(DB::ReadBuffer & rb)
{
UInt8 rhs_skip_degree = 0;
DB::readIntBinary(rhs_skip_degree, rb);
if (rhs_skip_degree > skip_degree)
{
skip_degree = rhs_skip_degree;
rehash();
}
size_t rhs_size = 0;
DB::readVarUInt(rhs_size, rb);
if (rhs_size > UNIQUES_HASH_MAX_SIZE)
throw Poco::Exception("Cannot read UniquesHashSet: too large size_degree.");
if ((1U << size_degree) < rhs_size)
{
UInt8 new_size_degree = std::max(UNIQUES_HASH_SET_INITIAL_SIZE_DEGREE, static_cast<int>(log2(rhs_size - 1)) + 2);
resize(new_size_degree);
}
for (size_t i = 0; i < rhs_size; ++i)
{
HashValue_t x = 0;
DB::readIntBinary(x, rb);
insertHash(x);
}
}
static void skip(DB::ReadBuffer & rb)
{
size_t size = 0;
rb.ignore();
DB::readVarUInt(size, rb);
if (size > UNIQUES_HASH_MAX_SIZE)
throw Poco::Exception("Cannot read UniquesHashSet: too large size_degree.");
rb.ignore(sizeof(HashValue_t) * size);
}
void writeText(DB::WriteBuffer & wb) const
{
if (m_size > UNIQUES_HASH_MAX_SIZE)
throw Poco::Exception("Cannot write UniquesHashSet: too large size_degree.");
DB::writeIntText(skip_degree, wb);
wb.write(",", 1);
DB::writeIntText(m_size, wb);
if (has_zero)
wb.write(",0", 2);
for (size_t i = 0; i < buf_size(); ++i)
{
if (buf[i])
{
wb.write(",", 1);
DB::writeIntText(buf[i], wb);
}
}
}
void readText(DB::ReadBuffer & rb)
{
has_zero = false;
DB::readIntText(skip_degree, rb);
DB::assertString(",", rb);
DB::readIntText(m_size, rb);
if (m_size > UNIQUES_HASH_MAX_SIZE)
throw Poco::Exception("Cannot read UniquesHashSet: too large size_degree.");
free();
UInt8 new_size_degree = m_size <= 1
? UNIQUES_HASH_SET_INITIAL_SIZE_DEGREE
: std::max(UNIQUES_HASH_SET_INITIAL_SIZE_DEGREE, static_cast<int>(log2(m_size - 1)) + 2);
alloc(new_size_degree);
for (size_t i = 0; i < m_size; ++i)
{
HashValue_t x = 0;
DB::assertString(",", rb);
DB::readIntText(x, rb);
if (x == 0)
has_zero = true;
else
reinsertImpl(x);
}
}
void insertHash(HashValue_t hash_value)
{
if (!good(hash_value))
return;
insertImpl(hash_value);
shrinkIfNeed();
}
#ifdef UNIQUES_HASH_SET_COUNT_COLLISIONS
size_t getCollisions() const
{
return collisions;
}
#endif
};
#undef UNIQUES_HASH_MAX_SIZE_DEGREE
#undef UNIQUES_HASH_MAX_SIZE
#undef UNIQUES_HASH_BITS_FOR_SKIP
#undef UNIQUES_HASH_SET_INITIAL_SIZE_DEGREE

View File

@ -26,8 +26,6 @@ namespace DB
using Poco::SharedPtr;
class ParallelReplicas;
/// Поток блоков читающих из таблицы и ее имя
typedef std::pair<BlockInputStreamPtr, std::string> ExternalTableData;
/// Вектор пар, описывающих таблицы
@ -44,6 +42,7 @@ typedef std::vector<ExternalTableData> ExternalTablesData;
class Connection : private boost::noncopyable
{
friend class ParallelReplicas;
friend class MultiplexedConnections;
public:
Connection(const String & host_, UInt16 port_, const String & default_database_,

View File

@ -57,7 +57,7 @@ protected:
typedef SharedPtr<IConnectionPool> ConnectionPoolPtr;
typedef std::vector<ConnectionPoolPtr> ConnectionPools;
typedef SharedPtr<ConnectionPools> ConnectionPoolsPtr;
/** Обычный пул соединений, без отказоустойчивости.

View File

@ -6,30 +6,38 @@
#include <Poco/ScopedLock.h>
#include <Poco/Mutex.h>
namespace DB
{
/** Для получения данных сразу из нескольких реплик (соединений) в рамках одного потока.
* В качестве вырожденного случая, может также работать с одним соединением.
/** Для получения данных сразу из нескольких реплик (соединений) из одного или нексольких шардов
* в рамках одного потока. В качестве вырожденного случая, может также работать с одним соединением.
* Предполагается, что все функции кроме sendCancel всегда выполняются в одном потоке.
*
* Интерфейс почти совпадает с Connection.
*/
class ParallelReplicas final : private boost::noncopyable
class MultiplexedConnections final : private boost::noncopyable
{
public:
/// Принимает готовое соединение.
ParallelReplicas(Connection * connection_, const Settings * settings_, ThrottlerPtr throttler_);
MultiplexedConnections(Connection * connection_, const Settings * settings_, ThrottlerPtr throttler_);
/** Принимает пул, из которого нужно будет достать одно или несколько соединений.
* Если флаг append_extra_info установлен, к каждому полученному блоку прилагается
* дополнительная информация.
* Если флаг get_all_replicas установлен, достаются все соединения.
*/
ParallelReplicas(IConnectionPool * pool_, const Settings * settings_, ThrottlerPtr throttler_,
bool append_extra_info = false, bool get_all_replicas = false);
MultiplexedConnections(IConnectionPool * pool_, const Settings * settings_, ThrottlerPtr throttler_,
bool append_extra_info = false, bool do_broadcast = false);
/** Принимает пулы, один для каждого шарда, из которих нужно будет достать одно или несколько
* соединений.
* Если флаг append_extra_info установлен, к каждому полученному блоку прилагается
* дополнительная информация.
* Если флаг do_broadcast установлен, достаются все соединения.
*/
MultiplexedConnections(ConnectionPools & pools_, const Settings * settings_, ThrottlerPtr throttler_,
bool append_extra_info = false, bool do_broadcast = false);
/// Отправить на реплики всё содержимое внешних таблиц.
void sendExternalTablesData(std::vector<ExternalTablesData> & data);
@ -65,15 +73,44 @@ public:
/// Проверить, есть ли действительные реплики.
/// Без блокировки, потому что sendCancel() не меняет состояние реплик.
bool hasActiveReplicas() const { return active_replica_count > 0; }
bool hasActiveConnections() const { return active_connection_total_count > 0; }
private:
/// Реплики хэшированные по id сокета
using ReplicaMap = std::unordered_map<int, Connection *>;
/// Соединения 1-го шарда, затем соединения 2-го шарда, и т.д.
using Connections = std::vector<Connection *>;
/// Состояние соединений одного шарда.
struct ShardState
{
/// Количество выделенных соединений, т.е. реплик, для этого шарда.
size_t allocated_connection_count;
/// Текущее количество действительных соединений к репликам этого шарда.
size_t active_connection_count;
};
/// Описание одной реплики.
struct ReplicaState
{
/// Индекс соединения.
size_t connection_index;
/// Владелец этой реплики.
ShardState * shard_state;
};
/// Реплики хэшированные по id сокета.
using ReplicaMap = std::unordered_map<int, ReplicaState>;
/// Состояние каждого шарда.
using ShardStates = std::vector<ShardState>;
private:
/// Зарегистрировать реплику.
void registerReplica(Connection * connection);
void initFromShard(IConnectionPool * pool);
/// Зарегистрировать шарды.
void registerShards();
/// Зарегистрировать реплики одного шарда.
void registerReplicas(size_t index_begin, size_t index_end, ShardState & shard_state);
/// Внутренняя версия функции receivePacket без блокировки.
Connection::Packet receivePacketUnlocked();
@ -94,13 +131,15 @@ private:
private:
const Settings * settings;
Connections connections;
ReplicaMap replica_map;
ShardStates shard_states;
/// Если не nullptr, то используется, чтобы ограничить сетевой трафик.
ThrottlerPtr throttler;
std::vector<ConnectionPool::Entry> pool_entries;
ConnectionPool::Entry pool_entry;
/// Соединение, c которого был получен последний блок.
Connection * current_connection;
@ -108,7 +147,7 @@ private:
std::unique_ptr<BlockExtraInfo> block_extra_info;
/// Текущее количество действительных соединений к репликам.
size_t active_replica_count;
size_t active_connection_total_count = 0;
/// Запрос выполняется параллельно на нескольких репликах.
bool supports_parallel_execution;
/// Отправили запрос
@ -116,6 +155,8 @@ private:
/// Отменили запрос
bool cancelled = false;
bool do_broadcast = false;
/// Мьютекс для того, чтобы функция sendCancel могла выполняться безопасно
/// в отдельном потоке.
mutable Poco::FastMutex cancel_mutex;

View File

@ -103,18 +103,9 @@ public:
arenas.push_back(arena_);
}
ColumnPtr convertToValues() const
{
const IAggregateFunction * function = holder->func;
ColumnPtr res = function->getReturnType()->createColumn();
IColumn & column = *res;
res->reserve(getData().size());
for (auto val : getData())
function->insertResultInto(val, column);
return res;
}
/** Преобразовать столбец состояний агрегатной функции в столбец с готовыми значениями результатов.
*/
ColumnPtr convertToValues() const;
std::string getName() const override { return "ColumnAggregateFunction"; }
@ -174,6 +165,9 @@ public:
{
IAggregateFunction * function = holder.get()->func;
if (unlikely(arenas.empty()))
arenas.emplace_back(new Arena);
getData().push_back(arenas.back().get()->alloc(function->sizeOfData()));
function->create(getData().back());
ReadBufferFromString read_buffer(x.get<const String &>());

View File

@ -266,6 +266,25 @@ public:
ColumnPtr replicate(const Offsets_t & replicate_offsets) const override;
ColumnPtr convertToFullColumnIfConst() const override
{
ColumnPtr new_data;
ColumnPtr new_offsets;
if (auto full_column = data->convertToFullColumnIfConst())
new_data = full_column;
else
new_data = data;
if (auto full_column = offsets->convertToFullColumnIfConst())
new_offsets = full_column;
else
new_offsets = offsets;
return new ColumnArray(new_data, new_offsets);
}
private:
ColumnPtr data;
ColumnPtr offsets; /// Смещения могут быть разделяемыми для нескольких столбцов - для реализации вложенных структур данных.

View File

@ -21,6 +21,7 @@ class IColumnConst : public IColumn
public:
bool isConst() const override { return true; }
virtual ColumnPtr convertToFullColumn() const = 0;
ColumnPtr convertToFullColumnIfConst() const override { return convertToFullColumn(); }
};

View File

@ -14,7 +14,7 @@ namespace DB
class ColumnSet final : public IColumnDummy
{
public:
ColumnSet(size_t s_, SetPtr data_) : IColumnDummy(s_), data(data_) {}
ColumnSet(size_t s_, ConstSetPtr data_) : IColumnDummy(s_), data(data_) {}
/// Столбец не константный. Иначе столбец будет использоваться в вычислениях в ExpressionActions::prepare, когда множество из подзапроса ещё не готово.
bool isConst() const override { return false; }
@ -22,11 +22,10 @@ public:
std::string getName() const override { return "ColumnSet"; }
ColumnPtr cloneDummy(size_t s_) const override { return new ColumnSet(s_, data); }
SetPtr & getData() { return data; }
const SetPtr & getData() const { return data; }
ConstSetPtr getData() const { return data; }
private:
SetPtr data;
ConstSetPtr data;
};
}

View File

@ -242,9 +242,22 @@ public:
columns[i]->getExtremes(min.get<Array &>()[i], max.get<Array &>()[i]);
}
ColumnPtr convertToFullColumnIfConst() const override
{
Block materialized = data;
for (size_t i = 0, size = materialized.columns(); i < size; ++i)
if (auto converted = materialized.unsafeGetByPosition(i).column->convertToFullColumnIfConst())
materialized.unsafeGetByPosition(i).column = converted;
return new ColumnTuple(materialized);
}
const Block & getData() const { return data; }
Block & getData() { return data; }
const Columns & getColumns() const { return columns; }
Columns & getColumns() { return columns; }
};

View File

@ -42,6 +42,15 @@ public:
*/
virtual bool isConst() const { return false; }
/** Если столбец не константа - возвращает nullptr (либо может вернуть самого себя).
* Если столбец константа, то превращает его в полноценный столбец (если тип столбца предполагает такую возможность) и возвращает его.
* Отдельный случай:
* Если столбец состоит из нескольких других столбцов (пример: кортеж),
* и он может содержать как константные, так и полноценные столбцы,
* то превратить в нём все константные столбцы в полноценные, и вернуть результат.
*/
virtual SharedPtr<IColumn> convertToFullColumnIfConst() const { return {}; }
/** Значения имеют фиксированную длину.
*/
virtual bool isFixed() const { return false; }

View File

@ -1,19 +1,26 @@
#pragma once
#include <linux/aio_abi.h>
#include <unistd.h>
#include <sys/syscall.h>
#include <boost/noncopyable.hpp>
#include <DB/Core/ErrorCodes.h>
#include <DB/Common/Exception.h>
#include <common/logger_useful.h>
#include <common/singleton.h>
#include <Poco/Logger.h>
#include <boost/range/iterator_range.hpp>
#include <boost/noncopyable.hpp>
#include <condition_variable>
#include <future>
#include <mutex>
#include <map>
#include <linux/aio_abi.h>
#include <sys/syscall.h>
#include <unistd.h>
/** Небольшие обёртки для асинхронного ввода-вывода.
*/
inline int io_setup(unsigned nr, aio_context_t *ctxp)
inline int io_setup(unsigned nr, aio_context_t * ctxp)
{
return syscall(__NR_io_setup, nr, ctxp);
}
@ -23,12 +30,13 @@ inline int io_destroy(aio_context_t ctx)
return syscall(__NR_io_destroy, ctx);
}
inline int io_submit(aio_context_t ctx, long nr, struct iocb **iocbpp)
/// last argument is an array of pointers technically speaking
inline int io_submit(aio_context_t ctx, long nr, struct iocb * iocbpp[])
{
return syscall(__NR_io_submit, ctx, nr, iocbpp);
}
inline int io_getevents(aio_context_t ctx, long min_nr, long max_nr, io_event *events, struct timespec *timeout)
inline int io_getevents(aio_context_t ctx, long min_nr, long max_nr, io_event *events, struct timespec * timeout)
{
return syscall(__NR_io_getevents, ctx, min_nr, max_nr, events, timeout);
}
@ -50,3 +58,159 @@ struct AIOContext : private boost::noncopyable
io_destroy(ctx);
}
};
namespace DB
{
class AIOContextPool : public Singleton<AIOContextPool>
{
friend class Singleton<AIOContextPool>;
static const auto max_concurrent_events = 128;
static const auto timeout_sec = 1;
AIOContext aio_context{max_concurrent_events};
using id_t = size_t;
using bytes_read_t = ssize_t;
/// Autoincremental id used to identify completed requests
id_t id{};
mutable std::mutex mutex;
mutable std::condition_variable have_resources;
std::map<id_t, std::promise<bytes_read_t>> promises;
std::atomic<bool> cancelled{false};
std::thread io_completion_monitor{&AIOContextPool::doMonitor, this};
~AIOContextPool()
{
cancelled.store(true, std::memory_order_relaxed);
io_completion_monitor.join();
}
void doMonitor()
{
/// continue checking for events unless cancelled
while (!cancelled.load(std::memory_order_relaxed))
waitForCompletion();
/// wait until all requests have been completed
while (!promises.empty())
waitForCompletion();
}
void waitForCompletion()
{
/// array to hold completion events
io_event events[max_concurrent_events];
try
{
const auto num_events = getCompletionEvents(events, max_concurrent_events);
fulfillPromises(events, num_events);
notifyProducers(num_events);
}
catch (...)
{
/// there was an error, log it, return to any producer and continue
reportExceptionToAnyProducer();
tryLogCurrentException("AIOContextPool::waitForCompletion()");
}
}
int getCompletionEvents(io_event events[], const int max_events)
{
timespec timeout{timeout_sec};
auto num_events = 0;
/// request 1 to `max_events` events
while ((num_events = io_getevents(aio_context.ctx, 1, max_events, events, &timeout)) < 0)
if (errno != EINTR)
throwFromErrno("io_getevents: Failed to wait for asynchronous IO completion",
ErrorCodes::AIO_COMPLETION_ERROR, errno);
return num_events;
}
void fulfillPromises(const io_event events[], const int num_events)
{
if (num_events == 0)
return;
const std::lock_guard<std::mutex> lock{mutex};
/// look at returned events and find corresponding promise, set result and erase promise from map
for (const auto & event : boost::make_iterator_range(events, events + num_events))
{
/// get id from event
const auto id = event.data;
/// set value via promise and release it
const auto it = promises.find(id);
if (it == std::end(promises))
{
LOG_CRITICAL(&Poco::Logger::get("AIOcontextPool"), "Found io_event with unknown id " << id);
continue;
}
it->second.set_value(event.res);
promises.erase(it);
}
}
void notifyProducers(const int num_producers) const
{
if (num_producers == 0)
return;
if (num_producers > 1)
have_resources.notify_all();
else
have_resources.notify_one();
}
void reportExceptionToAnyProducer()
{
const std::lock_guard<std::mutex> lock{mutex};
const auto any_promise_it = std::begin(promises);
any_promise_it->second.set_exception(std::current_exception());
}
public:
/// Request AIO read operation for iocb, returns a future with number of bytes read
std::future<bytes_read_t> post(struct iocb & iocb)
{
std::unique_lock<std::mutex> lock{mutex};
/// get current id and increment it by one
const auto request_id = id++;
/// create a promise and put request in "queue"
promises.emplace(request_id, std::promise<bytes_read_t>{});
/// store id in AIO request for further identification
iocb.aio_data = request_id;
auto num_requests = 0;
struct iocb * requests[] { &iocb };
/// submit a request
while ((num_requests = io_submit(aio_context.ctx, 1, requests)) < 0)
{
if (errno == EAGAIN)
/// wait until at least one event has been completed (or a spurious wakeup) and try again
have_resources.wait(lock);
else if (errno != EINTR)
throwFromErrno("io_submit: Failed to submit a request for asynchronous IO",
ErrorCodes::AIO_SUBMIT_ERROR, errno);
}
return promises[request_id].get_future();
}
};
}

View File

@ -9,19 +9,42 @@
#include <DB/Core/ErrorCodes.h>
/** При использовании AllocatorWithStackMemory, размещённом на стеке,
* GCC 4.9 ошибочно делает предположение, что мы можем вызывать free от указателя на стек.
* На самом деле, комбинация условий внутри AllocatorWithStackMemory этого не допускает.
*/
#if !__clang__
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wfree-nonheap-object"
#endif
/** Отвечает за выделение/освобождение памяти. Используется, например, в PODArray, Arena.
* Также используется в хэш-таблицах.
* Интерфейс отличается от std::allocator
* - наличием метода realloc, который для больших кусков памяти использует mremap;
* - передачей размера в метод free;
* - наличием аргумента alignment;
* - возможностью зануления памяти (используется в хэш-таблицах);
*/
template <bool clear_memory_>
class Allocator
{
protected:
static constexpr bool clear_memory = clear_memory_;
private:
/** См. комментарий в HashTableAllocator.h
/** Многие современные аллокаторы (например, tcmalloc) не умеют делать mremap для realloc,
* даже в случае достаточно больших кусков памяти.
* Хотя это позволяет увеличить производительность и уменьшить потребление памяти во время realloc-а.
* Чтобы это исправить, делаем mremap самостоятельно, если кусок памяти достаточно большой.
* Порог (64 МБ) выбран достаточно большим, так как изменение адресного пространства
* довольно сильно тормозит, особенно в случае наличия большого количества потоков.
* Рассчитываем, что набор операций mmap/что-то сделать/mremap может выполняться всего лишь около 1000 раз в секунду.
*
* PS. Также это требуется, потому что tcmalloc не может выделить кусок памяти больше 16 GB.
*/
static constexpr size_t MMAP_THRESHOLD = 64 * (1 << 20);
static constexpr size_t HUGE_PAGE_SIZE = 2 * (1 << 20);
static constexpr size_t MMAP_MIN_ALIGNMENT = 4096;
static constexpr size_t MALLOC_MIN_ALIGNMENT = 8;
@ -43,15 +66,16 @@ public:
if (MAP_FAILED == buf)
DB::throwFromErrno("Allocator: Cannot mmap.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
/// См. комментарий в HashTableAllocator.h
if (size >= HUGE_PAGE_SIZE && 0 != madvise(buf, size, MADV_HUGEPAGE))
DB::throwFromErrno("HashTableAllocator: Cannot madvise with MADV_HUGEPAGE.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
/// Заполнение нулями не нужно - mmap сам это делает.
}
else
{
if (alignment <= MALLOC_MIN_ALIGNMENT)
{
buf = ::malloc(size);
if (clear_memory)
buf = ::calloc(size, 1);
else
buf = ::malloc(size);
if (nullptr == buf)
DB::throwFromErrno("Allocator: Cannot malloc.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
@ -63,6 +87,9 @@ public:
if (0 != res)
DB::throwFromErrno("Cannot allocate memory (posix_memalign)", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res);
if (clear_memory)
memset(buf, 0, size);
}
}
@ -101,6 +128,9 @@ public:
if (nullptr == buf)
DB::throwFromErrno("Allocator: Cannot realloc.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
if (clear_memory)
memset(reinterpret_cast<char *>(buf) + old_size, 0, new_size - old_size);
}
else if (old_size >= MMAP_THRESHOLD && new_size >= MMAP_THRESHOLD)
{
@ -110,6 +140,8 @@ public:
buf = mremap(buf, old_size, new_size, MREMAP_MAYMOVE);
if (MAP_FAILED == buf)
DB::throwFromErrno("Allocator: Cannot mremap.", DB::ErrorCodes::CANNOT_MREMAP);
/// Заполнение нулями не нужно.
}
else
{
@ -122,3 +154,53 @@ public:
return buf;
}
};
/** Аллокатор с оптимизацией для маленьких кусков памяти.
*/
template <typename Base, size_t N = 64>
class AllocatorWithStackMemory : private Base
{
private:
char stack_memory[N];
public:
void * alloc(size_t size)
{
if (size <= N)
{
if (Base::clear_memory)
memset(stack_memory, 0, N);
return stack_memory;
}
return Base::alloc(size);
}
void free(void * buf, size_t size)
{
if (size > N)
Base::free(buf, size);
}
void * realloc(void * buf, size_t old_size, size_t new_size)
{
/// Было в stack_memory, там и останется.
if (new_size <= N)
return buf;
/// Уже не помещалось в stack_memory.
if (old_size > N)
return Base::realloc(buf, old_size, new_size);
/// Было в stack_memory, но теперь не помещается.
void * new_buf = Base::alloc(new_size);
memcpy(new_buf, buf, old_size);
return new_buf;
}
};
#if !__clang__
#pragma GCC diagnostic pop
#endif

View File

@ -26,7 +26,7 @@ class Arena
{
private:
/// Непрерывный кусок памяти и указатель на свободное место в нём. Односвязный список.
struct Chunk : private Allocator /// empty base optimization
struct Chunk : private Allocator<false> /// empty base optimization
{
char * begin;
char * pos;

View File

@ -0,0 +1,93 @@
#pragma once
#include <DB/Common/Arena.h>
#include <ext/size.hpp>
#include <cstdlib>
#include <memory>
#include <array>
namespace DB
{
class ArenaWithFreeLists : private Allocator<false>
{
private:
struct Block { Block * next; };
static const std::array<std::size_t, 14> & getSizes()
{
static constexpr std::array<std::size_t, 14> sizes{
8, 16, 32, 64, 128, 256, 512, 1024, 2048, 4096, 8192, 16384, 32768, 65536
};
static_assert(sizes.front() >= sizeof(Block), "Can't make allocations smaller than sizeof(Block)");
return sizes;
}
static auto getMinBucketNum() { return 3; }
static auto getMaxFixedBlockSize() { return getSizes().back(); }
Arena pool;
const std::unique_ptr<Block * []> free_lists = std::make_unique<Block * []>(ext::size(getSizes()));
static std::size_t findFreeListIndex(const std::size_t size)
{
/// shift powers of two into previous bucket by subtracting 1
const auto bucket_num = _bit_scan_reverse(size - 1);
return std::max(bucket_num, getMinBucketNum()) - getMinBucketNum();
}
public:
ArenaWithFreeLists(
const std::size_t initial_size = 4096, const std::size_t growth_factor = 2,
const std::size_t linear_growth_threshold = 128 * 1024 * 1024)
: pool{initial_size, growth_factor, linear_growth_threshold}
{
}
char * alloc(const std::size_t size)
{
if (size > getMaxFixedBlockSize())
return static_cast<char *>(Allocator::alloc(size));
/// find list of required size
const auto list_idx = findFreeListIndex(size);
if (auto & block = free_lists[list_idx])
{
const auto res = ext::bit_cast<char *>(block);
block = block->next;
return res;
}
/// no block of corresponding size, allocate a new one
return pool.alloc(getSizes()[list_idx]);
}
void free(const void * ptr, const std::size_t size)
{
if (size > getMaxFixedBlockSize())
return Allocator::free(const_cast<void *>(ptr), size);
/// find list of required size
const auto list_idx = findFreeListIndex(size);
auto & block = free_lists[list_idx];
const auto old = block;
block = ext::bit_cast<Block *>(ptr);
block->next = old;
}
/// Размер выделенного пула в байтах
size_t size() const
{
return pool.size();
}
};
}

View File

@ -10,11 +10,11 @@
namespace DB
{
/** Компактный массив для хранения данных, размер L, в битах, которых составляет
/** Компактный массив для хранения данных, размер content_width, в битах, которых составляет
* меньше одного байта. Вместо того, чтобы хранить каждое значение в отдельный
* байт, что приводит к растрате 37.5% пространства для L=5, CompactArray хранит
* смежные L-битные значения в массиве байтов, т.е. фактически CompactArray
* симулирует массив L-битных значений.
* байт, что приводит к растрате 37.5% пространства для content_width=5, CompactArray хранит
* смежные content_width-битные значения в массиве байтов, т.е. фактически CompactArray
* симулирует массив content_width-битных значений.
*/
template <typename BucketIndex, UInt8 content_width, size_t bucket_count>
class __attribute__ ((packed)) CompactArray final

View File

@ -121,3 +121,42 @@ struct TrivialHash
return key;
}
};
/** Сравнительно неплохая некриптографическая хэш функция из UInt64 в UInt32.
* Но хуже (и по качеству и по скорости), чем просто срезка intHash64.
* Взята отсюда: http://www.concentric.net/~ttwang/tech/inthash.htm
*
* Немного изменена по сравнению с функцией по ссылке: сдвиги вправо случайно заменены на цикличесвие сдвиги вправо.
* Это изменение никак не повлияло на результаты тестов smhasher.
*
* Рекомендуется для разных задач использовать разные salt.
* А то был случай, что в БД значения сортировались по хэшу (для некачественного псевдослучайного разбрасывания),
* а в другом месте, в агрегатной функции, в хэш таблице использовался такой же хэш,
* в результате чего, эта агрегатная функция чудовищно тормозила из-за коллизий.
*/
template <DB::UInt64 salt>
inline DB::UInt32 intHash32(DB::UInt64 key)
{
key ^= salt;
key = (~key) + (key << 18);
key = key ^ ((key >> 31) | (key << 33));
key = key * 21;
key = key ^ ((key >> 11) | (key << 53));
key = key + (key << 6);
key = key ^ ((key >> 22) | (key << 42));
return key;
}
/// Для контейнеров.
template <typename T, DB::UInt64 salt = 0>
struct IntHash32
{
size_t operator() (const T & key) const
{
return intHash32<salt>(key);
}
};

View File

@ -11,8 +11,6 @@
#include <common/likely.h>
#include <stats/IntHash.h>
#include <DB/Core/Defines.h>
#include <DB/Core/Types.h>
#include <DB/Common/Exception.h>

View File

@ -1,184 +1,9 @@
#pragma once
#include <malloc.h>
#include <string.h>
#include <sys/mman.h>
#include <DB/Common/MemoryTracker.h>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
/** При использовании HashTableAllocatorWithStackMemory, размещённом на стеке,
* GCC 4.9 ошибочно делает предположение, что мы можем вызывать free от указателя на стек.
* На самом деле, комбинация условий внутри HashTableAllocatorWithStackMemory этого не допускает.
*/
#if !__clang__
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wfree-nonheap-object"
#endif
#include <DB/Common/Allocator.h>
/** Общая часть разных хэш-таблиц, отвечающая за выделение/освобождение памяти.
* Отличается от Allocator тем, что зануляет память.
* Используется в качестве параметра шаблона (есть несколько реализаций с таким же интерфейсом).
*/
class HashTableAllocator
{
private:
/** Многие современные аллокаторы (например, tcmalloc) не умеют делать mremap для realloc,
* даже в случае достаточно больших кусков памяти.
* Хотя это позволяет увеличить производительность и уменьшить потребление памяти во время realloc-а.
* Чтобы это исправить, делаем mremap самостоятельно, если кусок памяти достаточно большой.
* Порог (64 МБ) выбран достаточно большим, так как изменение адресного пространства
* довольно сильно тормозит, особенно в случае наличия большого количества потоков.
* Рассчитываем, что набор операций mmap/что-то сделать/mremap может выполняться всего лишь около 1000 раз в секунду.
*
* PS. Также это требуется, потому что tcmalloc не может выделить кусок памяти больше 16 GB.
*/
static constexpr size_t MMAP_THRESHOLD = 64 * (1 << 20);
static constexpr size_t HUGE_PAGE_SIZE = 2 * (1 << 20);
using HashTableAllocator = Allocator<true>;
public:
/// Выделить кусок памяти и заполнить его нулями.
void * alloc(size_t size)
{
if (current_memory_tracker)
current_memory_tracker->alloc(size);
void * buf;
if (size >= MMAP_THRESHOLD)
{
buf = mmap(NULL, size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
if (MAP_FAILED == buf)
DB::throwFromErrno("HashTableAllocator: Cannot mmap.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
/** Использование huge pages позволяет увеличить производительность более чем в три раза
* в запросе SELECT number % 1000000 AS k, count() FROM system.numbers GROUP BY k,
* (хэш-таблица на 1 000 000 элементов)
* и примерно на 15% в случае хэш-таблицы на 100 000 000 элементов.
*/
if (size >= HUGE_PAGE_SIZE && 0 != madvise(buf, size, MADV_HUGEPAGE))
DB::throwFromErrno("HashTableAllocator: Cannot madvise with MADV_HUGEPAGE.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
/// Заполнение нулями не нужно - mmap сам это делает.
}
else
{
buf = ::calloc(size, 1);
if (nullptr == buf)
DB::throwFromErrno("HashTableAllocator: Cannot calloc.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
}
return buf;
}
/// Освободить память.
void free(void * buf, size_t size)
{
if (size >= MMAP_THRESHOLD)
{
if (0 != munmap(buf, size))
DB::throwFromErrno("HashTableAllocator: Cannot munmap.", DB::ErrorCodes::CANNOT_MUNMAP);
}
else
{
::free(buf);
}
if (current_memory_tracker)
current_memory_tracker->free(size);
}
/** Увеличить размер куска памяти.
* Содержимое старого куска памяти переезжает в начало нового.
* Оставшаяся часть заполняется нулями.
* Положение куска памяти может измениться.
*/
void * realloc(void * buf, size_t old_size, size_t new_size)
{
if (old_size < MMAP_THRESHOLD && new_size < MMAP_THRESHOLD)
{
if (current_memory_tracker)
current_memory_tracker->realloc(old_size, new_size);
buf = ::realloc(buf, new_size);
if (nullptr == buf)
DB::throwFromErrno("HashTableAllocator: Cannot realloc.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
memset(reinterpret_cast<char *>(buf) + old_size, 0, new_size - old_size);
}
else if (old_size >= MMAP_THRESHOLD && new_size >= MMAP_THRESHOLD)
{
if (current_memory_tracker)
current_memory_tracker->realloc(old_size, new_size);
buf = mremap(buf, old_size, new_size, MREMAP_MAYMOVE);
if (MAP_FAILED == buf)
DB::throwFromErrno("HashTableAllocator: Cannot mremap.", DB::ErrorCodes::CANNOT_MREMAP);
/** Здесь не получается сделать madvise с MADV_HUGEPAGE.
* Похоже, что при mremap, huge pages сами расширяются на новую область.
*/
/// Заполнение нулями не нужно.
}
else
{
void * new_buf = alloc(new_size);
memcpy(new_buf, buf, old_size);
free(buf, old_size);
buf = new_buf;
}
return buf;
}
};
/** Аллокатор с оптимизацией для маленьких кусков памяти.
*/
template <size_t N = 64>
class HashTableAllocatorWithStackMemory : private HashTableAllocator
{
private:
char stack_memory[N];
public:
void * alloc(size_t size)
{
if (size <= N)
{
memset(stack_memory, 0, N);
return stack_memory;
}
return HashTableAllocator::alloc(size);
}
void free(void * buf, size_t size)
{
if (size > N)
HashTableAllocator::free(buf, size);
}
void * realloc(void * buf, size_t old_size, size_t new_size)
{
/// Было в stack_memory, там и останется.
if (new_size <= N)
return buf;
/// Уже не помещалось в stack_memory.
if (old_size > N)
return HashTableAllocator::realloc(buf, old_size, new_size);
/// Было в stack_memory, но теперь не помещается.
void * new_buf = HashTableAllocator::alloc(new_size);
memcpy(new_buf, buf, old_size);
return new_buf;
}
};
#if !__clang__
#pragma GCC diagnostic pop
#endif
using HashTableAllocatorWithStackMemory = AllocatorWithStackMemory<HashTableAllocator, N>;

View File

@ -1,9 +1,9 @@
#pragma once
#include <common/Common.h>
#include <stats/IntHash.h>
#include <DB/Common/HyperLogLogBiasEstimator.h>
#include <DB/Common/CompactArray.h>
#include <DB/Common/HashTable/Hash.h>
#include <DB/IO/ReadBuffer.h>
#include <DB/IO/WriteBuffer.h>

View File

@ -9,7 +9,11 @@
#define MAX_SUBPATTERNS 5
template <bool b>
void OptimizedRegularExpressionImpl<b>::analyze(const std::string & regexp, std::string & required_substring, bool & is_trivial, bool & required_substring_is_prefix)
void OptimizedRegularExpressionImpl<b>::analyze(
const std::string & regexp,
std::string & required_substring,
bool & is_trivial,
bool & required_substring_is_prefix)
{
/** Выражение тривиально, если в нём все метасимволы эскейплены.
* Безальтернативная строка - это
@ -44,6 +48,7 @@ void OptimizedRegularExpressionImpl<b>::analyze(const std::string & regexp, std:
case '\0':
pos = end;
break;
case '\\':
{
++pos;
@ -74,6 +79,7 @@ void OptimizedRegularExpressionImpl<b>::analyze(const std::string & regexp, std:
++pos;
break;
}
case '|':
if (depth == 0)
has_alternative_on_depth_0 = true;
@ -85,6 +91,7 @@ void OptimizedRegularExpressionImpl<b>::analyze(const std::string & regexp, std:
}
++pos;
break;
case '(':
if (!in_square_braces)
{
@ -98,6 +105,7 @@ void OptimizedRegularExpressionImpl<b>::analyze(const std::string & regexp, std:
}
++pos;
break;
case '[':
in_square_braces = true;
++depth;
@ -109,7 +117,11 @@ void OptimizedRegularExpressionImpl<b>::analyze(const std::string & regexp, std:
}
++pos;
break;
case ']':
if (!in_square_braces)
goto ordinary;
in_square_braces = false;
--depth;
is_trivial = false;
@ -120,6 +132,7 @@ void OptimizedRegularExpressionImpl<b>::analyze(const std::string & regexp, std:
}
++pos;
break;
case ')':
if (!in_square_braces)
{
@ -133,6 +146,7 @@ void OptimizedRegularExpressionImpl<b>::analyze(const std::string & regexp, std:
}
++pos;
break;
case '^': case '$': case '.': case '+':
is_trivial = false;
if (!last_substring->first.empty() && !in_square_braces)
@ -142,6 +156,7 @@ void OptimizedRegularExpressionImpl<b>::analyze(const std::string & regexp, std:
}
++pos;
break;
/// Квантификаторы, допускающие нулевое количество.
case '{':
in_curly_braces = true;
@ -155,10 +170,16 @@ void OptimizedRegularExpressionImpl<b>::analyze(const std::string & regexp, std:
}
++pos;
break;
case '}':
if (!in_curly_braces)
goto ordinary;
in_curly_braces = false;
++pos;
break;
ordinary: /// Обычный, не заэскейпленный символ.
default:
if (depth == 0 && !in_curly_braces && !in_square_braces)
{

View File

@ -29,14 +29,13 @@ namespace DB
* Поддерживается только часть интерфейса std::vector.
*
* Конструктор по-умолчанию создаёт пустой объект, который не выделяет память.
* Затем выделяется память минимум под POD_ARRAY_INITIAL_SIZE элементов.
* Затем выделяется память минимум под INITIAL_SIZE элементов.
*
* Если вставлять элементы push_back-ом, не делая reserve, то PODArray примерно в 2.5 раза быстрее std::vector.
*/
#define POD_ARRAY_INITIAL_SIZE 4096UL
template <typename T>
class PODArray : private boost::noncopyable, private Allocator /// empty base optimization
template <typename T, size_t INITIAL_SIZE = 4096, typename TAllocator = Allocator<false>>
class PODArray : private boost::noncopyable, private TAllocator /// empty base optimization
{
private:
char * c_start;
@ -79,7 +78,7 @@ private:
size_t bytes_to_alloc = to_size(n);
c_start = c_end = reinterpret_cast<char *>(Allocator::alloc(bytes_to_alloc));
c_start = c_end = reinterpret_cast<char *>(TAllocator::alloc(bytes_to_alloc));
c_end_of_storage = c_start + bytes_to_alloc;
}
@ -88,7 +87,7 @@ private:
if (c_start == nullptr)
return;
Allocator::free(c_start, storage_size());
TAllocator::free(c_start, storage_size());
}
void realloc(size_t n)
@ -102,7 +101,7 @@ private:
ptrdiff_t end_diff = c_end - c_start;
size_t bytes_to_alloc = to_size(n);
c_start = reinterpret_cast<char *>(Allocator::realloc(c_start, storage_size(), bytes_to_alloc));
c_start = reinterpret_cast<char *>(TAllocator::realloc(c_start, storage_size(), bytes_to_alloc));
c_end = c_start + end_diff;
c_end_of_storage = c_start + bytes_to_alloc;
@ -133,7 +132,17 @@ public:
PODArray(const_iterator from_begin, const_iterator from_end) { alloc(from_end - from_begin); insert(from_begin, from_end); }
~PODArray() { dealloc(); }
PODArray(PODArray && other) { *this = std::move(other); }
PODArray(PODArray && other)
{
c_start = other.c_start;
c_end = other.c_end;
c_end_of_storage = other.c_end_of_storage;
other.c_start = nullptr;
other.c_end = nullptr;
other.c_end_of_storage = nullptr;
}
PODArray & operator=(PODArray && other)
{
std::swap(c_start, other.c_start);
@ -174,7 +183,7 @@ public:
void reserve()
{
if (size() == 0)
realloc(POD_ARRAY_INITIAL_SIZE);
realloc(INITIAL_SIZE);
else
realloc(size() * 2);
}
@ -227,6 +236,16 @@ public:
c_end += byte_size(1);
}
template <typename... Args>
void emplace_back(Args &&... args)
{
if (unlikely(c_end == c_end_of_storage))
reserve();
new (t_end()) T(std::forward<Args>(args)...);
c_end += byte_size(1);
}
/// Не вставляйте в массив кусок самого себя. Потому что при ресайзе, итераторы на самого себя могут инвалидироваться.
template <typename It1, typename It2>
void insert(It1 from_begin, It2 from_end)
@ -246,7 +265,7 @@ public:
c_end += bytes_to_copy;
}
void swap(PODArray<T> & rhs)
void swap(PODArray & rhs)
{
std::swap(c_start, rhs.c_start);
std::swap(c_end, rhs.c_end);
@ -271,13 +290,13 @@ public:
c_end = c_start + bytes_to_copy;
}
void assign(const PODArray<T> & from)
void assign(const PODArray & from)
{
assign(from.begin(), from.end());
}
bool operator== (const PODArray<T> & other) const
bool operator== (const PODArray & other) const
{
if (size() != other.size())
return false;
@ -297,7 +316,7 @@ public:
return true;
}
bool operator!= (const PODArray<T> & other) const
bool operator!= (const PODArray & other) const
{
return !operator==(other);
}

View File

@ -0,0 +1,88 @@
#pragma once
#include <DB/Core/ErrorCodes.h>
#include <DB/Common/Arena.h>
#include <common/likely.h>
#include <ext/range.hpp>
#include <ext/size.hpp>
#include <ext/bit_cast.hpp>
#include <cstdlib>
#include <memory>
namespace DB
{
class SmallObjectPool
{
private:
struct Block { Block * next; };
const std::size_t object_size;
Arena pool;
Block * free_list{};
public:
SmallObjectPool(
const std::size_t object_size, const std::size_t initial_size = 4096, const std::size_t growth_factor = 2,
const std::size_t linear_growth_threshold = 128 * 1024 * 1024)
: object_size{object_size}, pool{initial_size, growth_factor, linear_growth_threshold}
{
if (object_size < sizeof(Block))
throw Exception{
"Can't make allocations smaller than sizeof(Block) = " + std::to_string(sizeof(Block)),
ErrorCodes::LOGICAL_ERROR
};
if (pool.size() < object_size)
return;
const auto num_objects = pool.size() / object_size;
auto head = free_list = ext::bit_cast<Block *>(pool.alloc(num_objects * object_size));
for (const auto i : ext::range(0, num_objects - 1))
{
(void) i;
head->next = ext::bit_cast<Block *>(ext::bit_cast<char *>(head) + object_size);
head = head->next;
}
head->next = nullptr;
}
char * alloc()
{
if (free_list)
{
const auto res = reinterpret_cast<char *>(free_list);
free_list = free_list->next;
return res;
}
return pool.alloc(object_size);
}
void free(const void * ptr)
{
union {
const void * p_v;
Block * block;
};
p_v = ptr;
block->next = free_list;
free_list = block;
}
/// Размер выделенного пула в байтах
size_t size() const
{
return pool.size();
}
};
}

View File

@ -60,4 +60,7 @@ template <> struct TypeName<Float32> { static std::string get() { return "Float
template <> struct TypeName<Float64> { static std::string get() { return "Float64"; } };
template <> struct TypeName<String> { static std::string get() { return "String"; } };
/// Эти типы не поддерживаются СУБД. Но используются в других местах.
template <> struct TypeName<long double>{ static std::string get() { return "long double"; } };
}

View File

@ -30,11 +30,6 @@ public:
children.push_back(input_);
}
AddingDefaultBlockInputStream(BlockInputStreamPtr input_, NamesAndTypesListPtr required_columns_, const Context & context_)
: AddingDefaultBlockInputStream{input_, required_columns_, ColumnDefaults{}, context_}
{
}
String getName() const override { return "AddingDefault"; }
String getID() const override
@ -65,7 +60,7 @@ protected:
private:
NamesAndTypesListPtr required_columns;
const ColumnDefaults & column_defaults;
const ColumnDefaults column_defaults;
Context context;
};

View File

@ -58,7 +58,7 @@ public:
private:
BlockOutputStreamPtr output;
NamesAndTypesListPtr required_columns;
const ColumnDefaults & column_defaults;
const ColumnDefaults column_defaults;
Context context;
bool only_explicit_column_defaults;
};

View File

@ -22,6 +22,8 @@ public:
void flush() override { ostr.next(); }
String getContentType() const override { return "application/octet-stream"; }
protected:
WriteBuffer & ostr;
const Block sample;

View File

@ -24,6 +24,8 @@ public:
void setTotals(const Block & totals) override;
void setExtremes(const Block & extremes) override;
String getContentType() const override { return row_output->getContentType(); }
private:
RowOutputStreamPtr row_output;
bool first_row;

View File

@ -42,6 +42,10 @@ public:
virtual void setTotals(const Block & totals) {}
virtual void setExtremes(const Block & extremes) {}
/** Выставлять такой Content-Type при отдаче по HTTP.
*/
virtual String getContentType() const { return "text/plain; charset=UTF-8"; }
virtual ~IBlockOutputStream() {}
/** Не давать изменить таблицу, пока жив поток блоков.

View File

@ -169,6 +169,9 @@ protected:
/// Информация о приблизительном общем количестве строк собрана в родительском источнике.
bool collected_total_rows_approx = false;
/// Превышено ограничение на количество строк/байт, и нужно прекратить выполнение на следующем вызове read, как будто поток иссяк.
bool limit_exceeded_need_break = false;
/// Ограничения и квоты.
LocalLimits limits;

View File

@ -41,6 +41,9 @@ public:
virtual void setTotals(const Block & totals) {}
virtual void setExtremes(const Block & extremes) {}
/** Выставлять такой Content-Type при отдаче по HTTP. */
virtual String getContentType() const { return "text/plain; charset=UTF-8"; }
virtual ~IRowOutputStream() {}
};

View File

@ -42,6 +42,8 @@ public:
void setTotals(const Block & totals_) override { totals = totals_; }
void setExtremes(const Block & extremes_) override { extremes = extremes_; }
String getContentType() const override { return "application/json; charset=UTF-8"; }
protected:
void writeRowsBeforeLimitAtLeast();

View File

@ -38,9 +38,10 @@ protected:
size_t columns = res.columns();
for (size_t i = 0; i < columns; ++i)
{
ColumnPtr col = res.getByPosition(i).column;
if (col->isConst())
res.getByPosition(i).column = dynamic_cast<IColumnConst &>(*col).convertToFullColumn();
auto & src = res.getByPosition(i).column;
ColumnPtr converted = src->convertToFullColumnIfConst();
if (converted)
src = converted;
}
return res;

View File

@ -23,9 +23,10 @@ public:
for (const auto i : ext::range(0, block.columns()))
{
ColumnPtr & col = block.getByPosition(i).column;
if (col->isConst())
col = dynamic_cast<IColumnConst &>(*col).convertToFullColumn();
auto & src = block.getByPosition(i).column;
ColumnPtr converted = src->convertToFullColumnIfConst();
if (converted)
src = converted;
}
output->write(block);

View File

@ -30,6 +30,8 @@ public:
static void writeData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit);
String getContentType() const override { return "application/octet-stream"; }
private:
WriteBuffer & ostr;
UInt64 client_revision;

View File

@ -0,0 +1,30 @@
#pragma once
#include <DB/DataStreams/IBlockOutputStream.h>
namespace DB
{
/** Формат данных, предназначенный для упрощения реализации ODBC драйвера.
* ODBC драйвер предназначен для сборки под разные платформы без зависимостей от основного кода,
* поэтому формат сделан так, чтобы в нём можно было как можно проще его распарсить.
* Выводится заголовок с нужной информацией.
* Затем данные выводятся в порядке строк. Каждое значение выводится так: длина в формате VarUInt, затем данные в текстовом виде.
*/
class ODBCBlockOutputStream : public IBlockOutputStream
{
public:
ODBCBlockOutputStream(WriteBuffer & out_);
void write(const Block & block) override;
void flush() override { out.next(); }
String getContentType() const override { return "application/octet-stream"; }
private:
bool is_first = true;
WriteBuffer & out;
};
}

View File

@ -3,67 +3,63 @@
#include <common/logger_useful.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Common/VirtualColumnUtils.h>
#include <DB/Common/Throttler.h>
#include <DB/Interpreters/Context.h>
#include <DB/Client/ConnectionPool.h>
#include <DB/Client/ParallelReplicas.h>
#include <DB/Client/MultiplexedConnections.h>
namespace DB
{
/** Позволяет выполнить запрос (SELECT) на удалённых репликах одного шарда и получить результат.
/** Позволяет выполнить запрос на удалённых репликах одного шарда и получить результат.
*/
class RemoteBlockInputStream : public IProfilingBlockInputStream
{
private:
void init(const Settings * settings_)
{
if (settings_)
{
send_settings = true;
settings = *settings_;
}
else
send_settings = false;
}
public:
/// Принимает готовое соединение.
RemoteBlockInputStream(Connection & connection_, const String & query_, const Settings * settings_, ThrottlerPtr throttler_ = nullptr,
const Tables & external_tables_ = Tables(), QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete,
const Context & context = getDefaultContext())
: connection(&connection_), query(query_), throttler(throttler_), external_tables(external_tables_), stage(stage_), context(context)
{
init(settings_);
}
RemoteBlockInputStream(Connection & connection_, const String & query_, const Settings * settings_,
ThrottlerPtr throttler_ = nullptr, const Tables & external_tables_ = Tables(),
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete,
const Context & context_ = getDefaultContext());
/// Принимает готовое соединение. Захватывает владение соединением из пула.
RemoteBlockInputStream(ConnectionPool::Entry & pool_entry_, const String & query_, const Settings * settings_, ThrottlerPtr throttler_ = nullptr,
const Tables & external_tables_ = Tables(), QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete,
const Context & context = getDefaultContext())
: pool_entry(pool_entry_), connection(&*pool_entry_), query(query_), throttler(throttler_),
external_tables(external_tables_), stage(stage_), context(context)
{
init(settings_);
}
RemoteBlockInputStream(ConnectionPool::Entry & pool_entry_, const String & query_, const Settings * settings_,
ThrottlerPtr throttler_ = nullptr, const Tables & external_tables_ = Tables(),
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete,
const Context & context_ = getDefaultContext());
/// Принимает пул, из которого нужно будет достать одно или несколько соединений.
RemoteBlockInputStream(IConnectionPool * pool_, const String & query_, const Settings * settings_, ThrottlerPtr throttler_ = nullptr,
const Tables & external_tables_ = Tables(), QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete,
const Context & context = getDefaultContext())
: pool(pool_), query(query_), throttler(throttler_), external_tables(external_tables_), stage(stage_), context(context)
{
init(settings_);
}
RemoteBlockInputStream(IConnectionPool * pool_, const String & query_, const Settings * settings_,
ThrottlerPtr throttler_ = nullptr, const Tables & external_tables_ = Tables(),
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete,
const Context & context_ = getDefaultContext());
/// Принимает пулы - один для каждого шарда, из которых нужно будет достать одно или несколько соединений.
RemoteBlockInputStream(ConnectionPoolsPtr & pools_, const String & query_, const Settings * settings_,
ThrottlerPtr throttler_ = nullptr, const Tables & external_tables_ = Tables(),
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete,
const Context & context_ = getDefaultContext());
~RemoteBlockInputStream() override;
/// Отправить запрос на все существующие реплики.
void doBroadcast();
/// Кроме блоков, получить информацию о блоках.
void appendExtraInfo();
/// Отправляет запрос (инициирует вычисления) раньше, чем read.
void readPrefix() override;
/** Отменяем умолчальное уведомление о прогрессе,
* так как колбэк прогресса вызывается самостоятельно.
*/
void progress(const Progress & value) override {}
void cancel() override;
String getName() const override { return "Remote"; }
String getID() const override
{
std::stringstream res;
@ -71,249 +67,35 @@ public:
return res.str();
}
/** Отменяем умолчальное уведомление о прогрессе,
* так как колбэк прогресса вызывается самостоятельно.
*/
void progress(const Progress & value) override {}
void cancel() override
{
bool old_val = false;
if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed))
return;
{
std::lock_guard<std::mutex> lock(external_tables_mutex);
/// Останавливаем отправку внешних данных.
for (auto & vec : external_tables_data)
for (auto & elem : vec)
if (IProfilingBlockInputStream * stream = dynamic_cast<IProfilingBlockInputStream *>(elem.first.get()))
stream->cancel();
}
if (!isQueryPending() || hasThrownException())
return;
tryCancel("Cancelling query");
}
~RemoteBlockInputStream() override
{
/** Если прервались в середине цикла общения с репликами, то прервываем
* все соединения, затем читаем и пропускаем оставшиеся пакеты чтобы
* эти соединения не остались висеть в рассихронизированном состоянии.
*/
if (established || isQueryPending())
parallel_replicas->disconnect();
}
/// Отправляет запрос (инициирует вычисления) раньше, чем read.
void readPrefix() override
{
if (!sent_query)
sendQuery();
}
/// Отправить запрос на все существующие реплики.
void reachAllReplicas()
{
reach_all_replicas = true;
}
/// Кроме блоков, получить информацию о блоках.
void appendExtraInfo()
{
append_extra_info = true;
}
BlockExtraInfo getBlockExtraInfo() const override
{
return parallel_replicas->getBlockExtraInfo();
return multiplexed_connections->getBlockExtraInfo();
}
protected:
/// Отправить на удаленные серверы все временные таблицы.
void sendExternalTables()
{
size_t count = parallel_replicas->size();
void sendExternalTables();
{
std::lock_guard<std::mutex> lock(external_tables_mutex);
Block readImpl() override;
external_tables_data.reserve(count);
for (size_t i = 0; i < count; ++i)
{
ExternalTablesData res;
for (const auto & table : external_tables)
{
StoragePtr cur = table.second;
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete;
DB::BlockInputStreams input = cur->read(cur->getColumnNamesList(), ASTPtr(), context, settings,
stage, DEFAULT_BLOCK_SIZE, 1);
if (input.size() == 0)
res.push_back(std::make_pair(new OneBlockInputStream(cur->getSampleBlock()), table.first));
else
res.push_back(std::make_pair(input[0], table.first));
}
external_tables_data.push_back(std::move(res));
}
}
parallel_replicas->sendExternalTablesData(external_tables_data);
}
Block readImpl() override
{
if (!sent_query)
{
sendQuery();
if (settings.skip_unavailable_shards && 0 == parallel_replicas->size())
return {};
}
while (true)
{
if (isCancelled())
return Block();
Connection::Packet packet = parallel_replicas->receivePacket();
switch (packet.type)
{
case Protocol::Server::Data:
/// Если блок не пуст и не является заголовочным блоком
if (packet.block && packet.block.rows() > 0)
return packet.block;
break; /// Если блок пустой - получим другие пакеты до EndOfStream.
case Protocol::Server::Exception:
got_exception_from_replica = true;
packet.exception->rethrow();
break;
case Protocol::Server::EndOfStream:
if (!parallel_replicas->hasActiveReplicas())
{
finished = true;
return Block();
}
break;
case Protocol::Server::Progress:
/** Используем прогресс с удалённого сервера.
* В том числе, запишем его в ProcessList,
* и будем использовать его для проверки
* ограничений (например, минимальная скорость выполнения запроса)
* и квот (например, на количество строчек для чтения).
*/
progressImpl(packet.progress);
break;
case Protocol::Server::ProfileInfo:
info = packet.profile_info;
break;
case Protocol::Server::Totals:
totals = packet.block;
break;
case Protocol::Server::Extremes:
extremes = packet.block;
break;
default:
got_unknown_packet_from_replica = true;
throw Exception("Unknown packet from server", ErrorCodes::UNKNOWN_PACKET_FROM_SERVER);
}
}
}
void readSuffixImpl() override
{
/** Если одно из:
* - ничего не начинали делать;
* - получили все пакеты до EndOfStream;
* - получили с одной реплики эксепшен;
* - получили с одной реплики неизвестный пакет;
* - то больше читать ничего не нужно.
*/
if (!isQueryPending() || hasThrownException())
return;
/** Если ещё прочитали не все данные, но они больше не нужны.
* Это может быть из-за того, что данных достаточно (например, при использовании LIMIT).
*/
/// Отправим просьбу прервать выполнение запроса, если ещё не отправляли.
tryCancel("Cancelling query because enough data has been read");
/// Получим оставшиеся пакеты, чтобы не было рассинхронизации в соединениях с репликами.
Connection::Packet packet = parallel_replicas->drain();
switch (packet.type)
{
case Protocol::Server::EndOfStream:
finished = true;
break;
case Protocol::Server::Exception:
got_exception_from_replica = true;
packet.exception->rethrow();
break;
default:
got_unknown_packet_from_replica = true;
throw Exception("Unknown packet from server", ErrorCodes::UNKNOWN_PACKET_FROM_SERVER);
}
}
void readSuffixImpl() override;
/// Создать объект для общения с репликами одного шарда, на которых должен выполниться запрос.
void createParallelReplicas()
{
Settings * parallel_replicas_settings = send_settings ? &settings : nullptr;
if (connection != nullptr)
parallel_replicas = std::make_unique<ParallelReplicas>(connection, parallel_replicas_settings, throttler);
else
parallel_replicas = std::make_unique<ParallelReplicas>(pool, parallel_replicas_settings, throttler,
append_extra_info, reach_all_replicas);
}
void createMultiplexedConnections();
/// Возвращает true, если запрос отправлен.
bool isQueryPending() const
{
return sent_query && !finished;
}
bool isQueryPending() const;
/// Возвращает true, если исключение было выкинуто.
bool hasThrownException() const
{
return got_exception_from_replica || got_unknown_packet_from_replica;
}
bool hasThrownException() const;
private:
void sendQuery()
{
createParallelReplicas();
void init(const Settings * settings_);
if (settings.skip_unavailable_shards && 0 == parallel_replicas->size())
return;
void sendQuery();
established = true;
parallel_replicas->sendQuery(query, "", stage, true);
established = false;
sent_query = true;
sendExternalTables();
}
/// Отправить запрос на отмену всех соединений к репликам, если такой запрос ещё не был отправлен.
void tryCancel(const char * reason);
/// ITable::read requires a Context, therefore we should create one if the user can't supply it
static Context & getDefaultContext()
@ -322,23 +104,18 @@ private:
return instance;
}
/// Отправить запрос на отмену всех соединений к репликам, если такой запрос ещё не был отправлен.
void tryCancel(const char * reason)
{
bool old_val = false;
if (!was_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed))
return;
LOG_TRACE(log, "(" << parallel_replicas->dumpAddresses() << ") " << reason);
parallel_replicas->sendCancel();
}
private:
IConnectionPool * pool = nullptr;
/// Готовое соединение.
ConnectionPool::Entry pool_entry;
Connection * connection = nullptr;
std::unique_ptr<ParallelReplicas> parallel_replicas;
/// Пул соединений одного шарда.
IConnectionPool * pool = nullptr;
/// Пулы соединений одного или нескольких шардов.
ConnectionPoolsPtr pools;
std::unique_ptr<MultiplexedConnections> multiplexed_connections;
const String query;
bool send_settings;
@ -384,7 +161,7 @@ private:
std::atomic<bool> got_unknown_packet_from_replica { false };
bool append_extra_info = false;
bool reach_all_replicas = false;
bool do_broadcast = false;
Logger * log = &Logger::get("RemoteBlockInputStream");
};

View File

@ -99,7 +99,7 @@ private:
/** Делаем поддержку двух разных курсоров - с Collation и без.
* Шаблоны используем вместо полиморфных SortCursor'ов и вызовов виртуальных функций.
*/
template<class TSortCursor>
template <class TSortCursor>
void merge(ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue);
/// Вставить в результат просуммированную строку для текущей группы.
@ -128,155 +128,14 @@ private:
/** Для вложенных Map выполняется слияние по ключу с выбрасыванием строк вложенных массивов, в которых
* все элементы - нулевые.
*/
template<class TSortCursor>
bool mergeMaps(Row & row, TSortCursor & cursor)
{
auto non_empty_map_present = false;
/// merge nested maps
for (const auto & map : maps_to_sum)
{
const auto val_count = map.val_col_nums.size();
/// fetch key array reference from accumulator-row
auto & key_array_lhs = row[map.key_col_num].get<Array>();
/// returns a Field for pos-th item of val_index-th value
const auto val_getter_lhs = [&] (const auto val_index, const auto pos) -> decltype(auto) {
return row[map.val_col_nums[val_index]].get<Array>()[pos];
};
/// we will be sorting key positions, not the entire rows, to minimize actions
std::vector<std::size_t> key_pos_lhs(ext::range_iterator<std::size_t>{0},
ext::range_iterator<std::size_t>{key_array_lhs.size()});
std::sort(std::begin(key_pos_lhs), std::end(key_pos_lhs), [&] (const auto pos1, const auto pos2) {
return key_array_lhs[pos1] < key_array_lhs[pos2];
});
/// copy key field from current row under cursor
const auto key_field_rhs = (*cursor->all_columns[map.key_col_num])[cursor->pos];
/// for each element of `map.val_col_nums` copy corresponding array under cursor into vector
const auto val_fields_rhs = ext::map<std::vector>(map.val_col_nums,
[&] (const auto col_num) -> decltype(auto) {
return (*cursor->all_columns[col_num])[cursor->pos];
});
/// fetch key array reference from row under cursor
const auto & key_array_rhs = key_field_rhs.get<Array>();
/// returns a Field for pos-th item of val_index-th value
const auto val_getter_rhs = [&] (const auto val_index, const auto pos) -> decltype(auto) {
return val_fields_rhs[val_index].get<Array>()[pos];
};
std::vector<std::size_t> key_pos_rhs(ext::range_iterator<std::size_t>{0},
ext::range_iterator<std::size_t>{key_array_rhs.size()});
std::sort(std::begin(key_pos_rhs), std::end(key_pos_rhs), [&] (const auto pos1, const auto pos2) {
return key_array_rhs[pos1] < key_array_rhs[pos2];
});
/// max size after merge estimation
const auto max_size = key_pos_lhs.size() + key_pos_rhs.size();
/// create arrays with a single element (it will be overwritten on first iteration)
Array key_array_result(1);
key_array_result.reserve(max_size);
std::vector<Array> val_arrays_result(val_count, Array(1));
for (auto & val_array_result : val_arrays_result)
val_array_result.reserve(max_size);
/// discard first element
auto discard_prev = true;
/// either insert or merge new element
const auto insert_or_sum = [&] (std::size_t & index, const std::vector<std::size_t> & key_pos,
const auto & key_array, auto && val_getter) {
const auto pos = key_pos[index++];
const auto & key = key_array[pos];
if (discard_prev)
{
discard_prev = false;
key_array_result.back() = key;
for (const auto val_index : ext::range(0, val_count))
val_arrays_result[val_index].back() = val_getter(val_index, pos);
}
else if (key_array_result.back() == key)
{
/// merge with same key
auto should_discard = true;
for (const auto val_index : ext::range(0, val_count))
if (apply_visitor(FieldVisitorSum{val_getter(val_index, pos)},
val_arrays_result[val_index].back()))
should_discard = false;
discard_prev = should_discard;
}
else
{
/// append new key
key_array_result.emplace_back(key);
for (const auto val_index : ext::range(0, val_count))
val_arrays_result[val_index].emplace_back(val_getter(val_index, pos));
}
};
std::size_t index_lhs = 0;
std::size_t index_rhs = 0;
/// perform 2-way merge
while (true)
if (index_lhs < key_pos_lhs.size() && index_rhs == key_pos_rhs.size())
insert_or_sum(index_lhs, key_pos_lhs, key_array_lhs, val_getter_lhs);
else if (index_lhs == key_pos_lhs.size() && index_rhs < key_pos_rhs.size())
insert_or_sum(index_rhs, key_pos_rhs, key_array_rhs, val_getter_rhs);
else if (index_lhs < key_pos_lhs.size() && index_rhs < key_pos_rhs.size())
if (key_array_lhs[key_pos_lhs[index_lhs]] < key_array_rhs[key_pos_rhs[index_rhs]])
insert_or_sum(index_lhs, key_pos_lhs, key_array_lhs, val_getter_lhs);
else
insert_or_sum(index_rhs, key_pos_rhs, key_array_rhs, val_getter_rhs);
else
break;
/// discard last row if necessary
if (discard_prev)
key_array_result.pop_back();
/// store results into accumulator-row
key_array_lhs = std::move(key_array_result);
for (const auto val_col_index : ext::range(0, val_count))
{
/// discard last row if necessary
if (discard_prev)
val_arrays_result[val_col_index].pop_back();
row[map.val_col_nums[val_col_index]].get<Array>() = std::move(val_arrays_result[val_col_index]);
}
if (!key_array_lhs.empty())
non_empty_map_present = true;
}
return non_empty_map_present;
}
template <class TSortCursor>
bool mergeMaps(Row & row, TSortCursor & cursor);
/** Прибавить строчку под курсором к row.
* Возвращает false, если результат получился нулевым.
*/
template<class TSortCursor>
bool addRow(Row & row, TSortCursor & cursor)
{
bool res = mergeMaps(row, cursor); /// Есть ли хотя бы одно ненулевое число или непустой массив
for (size_t i = 0, size = column_numbers_to_sum.size(); i < size; ++i)
{
size_t j = column_numbers_to_sum[i];
if (apply_visitor(FieldVisitorSum((*cursor->all_columns[j])[cursor->pos]), row[j]))
res = true;
}
return res;
}
template <class TSortCursor>
bool addRow(Row & row, TSortCursor & cursor);
};
}

View File

@ -31,6 +31,9 @@ public:
void setTotals(const Block & totals_) override { totals = totals_; }
void setExtremes(const Block & extremes_) override { extremes = extremes_; }
/// https://www.iana.org/assignments/media-types/text/tab-separated-values
String getContentType() const override { return "text/tab-separated-values; charset=UTF-8"; }
protected:
void writeTotals();
void writeExtremes();

View File

@ -5,8 +5,9 @@
#include <DB/Dictionaries/DictionaryStructure.h>
#include <DB/Common/HashTable/HashMap.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Common/HashTable/HashMap.h>
#include <ext/scope_guard.hpp>
#include <ext/bit_cast.hpp>
#include <ext/map.hpp>
#include <Poco/RWLock.h>
#include <cmath>
#include <atomic>
@ -127,31 +128,132 @@ public:
getItems(attribute, ids, out);
}
#define DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const PODArray<id_t> & ids, const PODArray<TYPE> & def,\
PODArray<TYPE> & out) const override\
{\
auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeUnderlyingType::TYPE)\
throw Exception{\
name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\
ErrorCodes::TYPE_MISMATCH\
};\
\
getItems<TYPE>(attribute, ids, out, &def);\
}
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(UInt8)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(UInt16)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(UInt32)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(UInt64)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Int8)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Int16)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Int32)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Int64)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Float32)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Float64)
#undef DECLARE_MULTIPLE_GETTER_WITH_DEFAULT
void getString(
const std::string & attribute_name, const PODArray<id_t> & ids, const ColumnString * const def,
ColumnString * const out) const override
{
auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeUnderlyingType::String)
throw Exception{
name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH
};
getItems(attribute, ids, out, def);
}
void has(const PODArray<id_t> & ids, PODArray<UInt8> & out) const override
{
/// Mapping: <id> -> { all indices `i` of `ids` such that `ids[i]` = <id> }
MapType<std::vector<std::size_t>> outdated_ids;
const auto rows = ext::size(ids);
{
const Poco::ScopedReadRWLock read_lock{rw_lock};
const auto now = std::chrono::system_clock::now();
/// fetch up-to-date values, decide which ones require update
for (const auto i : ext::range(0, rows))
{
const auto id = ids[i];
const auto cell_idx = getCellIdx(id);
const auto & cell = cells[cell_idx];
/** cell should be updated if either:
* 1. ids do not match,
* 2. cell has expired,
* 3. explicit defaults were specified and cell was set default. */
if (cell.id != id || cell.expiresAt() < now)
outdated_ids[id].push_back(i);
else
out[i] = !cell.isDefault();
}
}
query_count.fetch_add(rows, std::memory_order_relaxed);
hit_count.fetch_add(rows - outdated_ids.size(), std::memory_order_release);
if (outdated_ids.empty())
return;
std::vector<id_t> required_ids(outdated_ids.size());
std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids),
[] (auto & pair) { return pair.first; });
/// request new values
update(required_ids, [&] (const auto id, const auto) {
for (const auto out_idx : outdated_ids[id])
out[out_idx] = true;
}, [&] (const auto id, const auto) {
for (const auto out_idx : outdated_ids[id])
out[out_idx] = false;
});
}
private:
template <typename Value> using MapType = HashMap<id_t, Value>;
template <typename Value> using ContainerType = Value[];
template <typename Value> using ContainerPtrType = std::unique_ptr<ContainerType<Value>>;
struct cell_metadata_t final
{
using time_point_t = std::chrono::system_clock::time_point;
using time_point_rep_t = time_point_t::rep;
using time_point_urep_t = std::make_unsigned_t<time_point_rep_t>;
static constexpr std::uint64_t EXPIRES_AT_MASK = std::numeric_limits<time_point_rep_t>::max();
static constexpr std::uint64_t IS_DEFAULT_MASK = ~EXPIRES_AT_MASK;
std::uint64_t id;
std::chrono::system_clock::time_point expires_at;
/// Stores both expiration time and `is_default` flag in the most significant bit
time_point_urep_t data;
/// Sets expiration time, resets `is_default` flag to false
time_point_t expiresAt() const { return ext::safe_bit_cast<time_point_t>(data & EXPIRES_AT_MASK); }
void setExpiresAt(const time_point_t & t) { data = ext::safe_bit_cast<time_point_urep_t>(t); }
bool isDefault() const { return (data & IS_DEFAULT_MASK) == IS_DEFAULT_MASK; }
void setDefault() { data |= IS_DEFAULT_MASK; }
};
struct attribute_t final
{
AttributeUnderlyingType type;
std::tuple<UInt8, UInt16, UInt32, UInt64,
std::tuple<
UInt8, UInt16, UInt32, UInt64,
Int8, Int16, Int32, Int64,
Float32, Float64,
String> null_values;
std::tuple<std::unique_ptr<UInt8[]>,
std::unique_ptr<UInt16[]>,
std::unique_ptr<UInt32[]>,
std::unique_ptr<UInt64[]>,
std::unique_ptr<Int8[]>,
std::unique_ptr<Int16[]>,
std::unique_ptr<Int32[]>,
std::unique_ptr<Int64[]>,
std::unique_ptr<Float32[]>,
std::unique_ptr<Float64[]>,
std::unique_ptr<StringRef[]>> arrays;
std::tuple<
ContainerPtrType<UInt8>, ContainerPtrType<UInt16>, ContainerPtrType<UInt32>, ContainerPtrType<UInt64>,
ContainerPtrType<Int8>, ContainerPtrType<Int16>, ContainerPtrType<Int32>, ContainerPtrType<Int64>,
ContainerPtrType<Float32>, ContainerPtrType<Float64>,
ContainerPtrType<StringRef>> arrays;
};
void createAttributes()
@ -188,57 +290,57 @@ private:
{
case AttributeUnderlyingType::UInt8:
std::get<UInt8>(attr.null_values) = null_value.get<UInt64>();
std::get<std::unique_ptr<UInt8[]>>(attr.arrays) = std::make_unique<UInt8[]>(size);
std::get<ContainerPtrType<UInt8>>(attr.arrays) = std::make_unique<ContainerType<UInt8>>(size);
bytes_allocated += size * sizeof(UInt8);
break;
case AttributeUnderlyingType::UInt16:
std::get<UInt16>(attr.null_values) = null_value.get<UInt64>();
std::get<std::unique_ptr<UInt16[]>>(attr.arrays) = std::make_unique<UInt16[]>(size);
std::get<ContainerPtrType<UInt16>>(attr.arrays) = std::make_unique<ContainerType<UInt16>>(size);
bytes_allocated += size * sizeof(UInt16);
break;
case AttributeUnderlyingType::UInt32:
std::get<UInt32>(attr.null_values) = null_value.get<UInt64>();
std::get<std::unique_ptr<UInt32[]>>(attr.arrays) = std::make_unique<UInt32[]>(size);
std::get<ContainerPtrType<UInt32>>(attr.arrays) = std::make_unique<ContainerType<UInt32>>(size);
bytes_allocated += size * sizeof(UInt32);
break;
case AttributeUnderlyingType::UInt64:
std::get<UInt64>(attr.null_values) = null_value.get<UInt64>();
std::get<std::unique_ptr<UInt64[]>>(attr.arrays) = std::make_unique<UInt64[]>(size);
std::get<ContainerPtrType<UInt64>>(attr.arrays) = std::make_unique<ContainerType<UInt64>>(size);
bytes_allocated += size * sizeof(UInt64);
break;
case AttributeUnderlyingType::Int8:
std::get<Int8>(attr.null_values) = null_value.get<Int64>();
std::get<std::unique_ptr<Int8[]>>(attr.arrays) = std::make_unique<Int8[]>(size);
std::get<ContainerPtrType<Int8>>(attr.arrays) = std::make_unique<ContainerType<Int8>>(size);
bytes_allocated += size * sizeof(Int8);
break;
case AttributeUnderlyingType::Int16:
std::get<Int16>(attr.null_values) = null_value.get<Int64>();
std::get<std::unique_ptr<Int16[]>>(attr.arrays) = std::make_unique<Int16[]>(size);
std::get<ContainerPtrType<Int16>>(attr.arrays) = std::make_unique<ContainerType<Int16>>(size);
bytes_allocated += size * sizeof(Int16);
break;
case AttributeUnderlyingType::Int32:
std::get<Int32>(attr.null_values) = null_value.get<Int64>();
std::get<std::unique_ptr<Int32[]>>(attr.arrays) = std::make_unique<Int32[]>(size);
std::get<ContainerPtrType<Int32>>(attr.arrays) = std::make_unique<ContainerType<Int32>>(size);
bytes_allocated += size * sizeof(Int32);
break;
case AttributeUnderlyingType::Int64:
std::get<Int64>(attr.null_values) = null_value.get<Int64>();
std::get<std::unique_ptr<Int64[]>>(attr.arrays) = std::make_unique<Int64[]>(size);
std::get<ContainerPtrType<Int64>>(attr.arrays) = std::make_unique<ContainerType<Int64>>(size);
bytes_allocated += size * sizeof(Int64);
break;
case AttributeUnderlyingType::Float32:
std::get<Float32>(attr.null_values) = null_value.get<Float64>();
std::get<std::unique_ptr<Float32[]>>(attr.arrays) = std::make_unique<Float32[]>(size);
std::get<ContainerPtrType<Float32>>(attr.arrays) = std::make_unique<ContainerType<Float32>>(size);
bytes_allocated += size * sizeof(Float32);
break;
case AttributeUnderlyingType::Float64:
std::get<Float64>(attr.null_values) = null_value.get<Float64>();
std::get<std::unique_ptr<Float64[]>>(attr.arrays) = std::make_unique<Float64[]>(size);
std::get<ContainerPtrType<Float64>>(attr.arrays) = std::make_unique<ContainerType<Float64>>(size);
bytes_allocated += size * sizeof(Float64);
break;
case AttributeUnderlyingType::String:
std::get<String>(attr.null_values) = null_value.get<String>();
std::get<std::unique_ptr<StringRef[]>>(attr.arrays) = std::make_unique<StringRef[]>(size);
std::get<ContainerPtrType<StringRef>>(attr.arrays) = std::make_unique<ContainerType<StringRef>>(size);
bytes_allocated += size * sizeof(StringRef);
break;
}
@ -247,64 +349,73 @@ private:
}
template <typename T>
void getItems(attribute_t & attribute, const PODArray<id_t> & ids, PODArray<T> & out) const
void getItems(
attribute_t & attribute, const PODArray<id_t> & ids, PODArray<T> & out,
const PODArray<T> * const def = nullptr) const
{
HashMap<id_t, std::vector<std::size_t>> outdated_ids;
auto & attribute_array = std::get<std::unique_ptr<T[]>>(attribute.arrays);
/// Mapping: <id> -> { all indices `i` of `ids` such that `ids[i]` = <id> }
MapType<std::vector<std::size_t>> outdated_ids;
auto & attribute_array = std::get<ContainerPtrType<T>>(attribute.arrays);
const auto rows = ext::size(ids);
{
const Poco::ScopedReadRWLock read_lock{rw_lock};
const auto now = std::chrono::system_clock::now();
/// fetch up-to-date values, decide which ones require update
for (const auto i : ext::range(0, ids.size()))
for (const auto i : ext::range(0, rows))
{
const auto id = ids[i];
if (id == 0)
{
out[i] = std::get<T>(attribute.null_values);
continue;
}
const auto cell_idx = getCellIdx(id);
const auto & cell = cells[cell_idx];
if (cell.id != id || cell.expires_at < now)
{
out[i] = std::get<T>(attribute.null_values);
/** cell should be updated if either:
* 1. ids do not match,
* 2. cell has expired,
* 3. explicit defaults were specified and cell was set default. */
if (cell.id != id || cell.expiresAt() < now)
outdated_ids[id].push_back(i);
}
else
out[i] = attribute_array[cell_idx];
out[i] = def && cell.isDefault() ? (*def)[i] : attribute_array[cell_idx];
}
}
query_count.fetch_add(ids.size(), std::memory_order_relaxed);
hit_count.fetch_add(ids.size() - outdated_ids.size(), std::memory_order_release);
query_count.fetch_add(rows, std::memory_order_relaxed);
hit_count.fetch_add(rows - outdated_ids.size(), std::memory_order_release);
if (outdated_ids.empty())
return;
/// request new values
std::vector<id_t> required_ids(outdated_ids.size());
std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids),
[] (auto & pair) { return pair.first; });
/// request new values
update(required_ids, [&] (const auto id, const auto cell_idx) {
const auto attribute_value = attribute_array[cell_idx];
/// set missing values to out
for (const auto out_idx : outdated_ids[id])
out[out_idx] = attribute_value;
}, [&] (const auto id, const auto cell_idx) {
const auto attribute_value = !def ? attribute_array[cell_idx] : (*def)[outdated_ids[id].front()];
/// set missing values to out
for (const auto out_idx : outdated_ids[id])
out[out_idx] = attribute_value;
});
}
void getItems(attribute_t & attribute, const PODArray<id_t> & ids, ColumnString * out) const
void getItems(
attribute_t & attribute, const PODArray<id_t> & ids, ColumnString * out,
const ColumnString * const def = nullptr) const
{
/// save on some allocations
out->getOffsets().reserve(ids.size());
const auto rows = ext::size(ids);
auto & attribute_array = std::get<std::unique_ptr<StringRef[]>>(attribute.arrays);
/// save on some allocations
out->getOffsets().reserve(rows);
auto & attribute_array = std::get<ContainerPtrType<StringRef>>(attribute.arrays);
auto found_outdated_values = false;
@ -314,20 +425,20 @@ private:
const auto now = std::chrono::system_clock::now();
/// fetch up-to-date values, discard on fail
for (const auto i : ext::range(0, ids.size()))
for (const auto i : ext::range(0, rows))
{
const auto id = ids[i];
const auto cell_idx = getCellIdx(id);
const auto & cell = cells[cell_idx];
if (cell.id != id || cell.expires_at < now)
if (cell.id != id || cell.expiresAt() < now)
{
found_outdated_values = true;
break;
}
else
{
const auto string_ref = attribute_array[cell_idx];
const auto string_ref = def && cell.isDefault() ? def->getDataAt(i) : attribute_array[cell_idx];
out->insertData(string_ref.data, string_ref.size);
}
}
@ -336,19 +447,19 @@ private:
/// optimistic code completed successfully
if (!found_outdated_values)
{
query_count.fetch_add(ids.size(), std::memory_order_relaxed);
hit_count.fetch_add(ids.size(), std::memory_order_release);
query_count.fetch_add(rows, std::memory_order_relaxed);
hit_count.fetch_add(rows, std::memory_order_release);
return;
}
/// now onto the pessimistic one, discard possibly partial results from the optimistic path
/// now onto the pessimistic one, discard possible partial results from the optimistic path
out->getChars().resize_assume_reserved(0);
out->getOffsets().resize_assume_reserved(0);
/// outdated ids joined number of times they've been requested
HashMap<id_t, std::size_t> outdated_ids;
/// Mapping: <id> -> { all indices `i` of `ids` such that `ids[i]` = <id> }
MapType<std::vector<std::size_t>> outdated_ids;
/// we are going to store every string separately
HashMap<id_t, String> map;
MapType<String> map;
std::size_t total_length = 0;
{
@ -361,19 +472,19 @@ private:
const auto cell_idx = getCellIdx(id);
const auto & cell = cells[cell_idx];
if (cell.id != id || cell.expires_at < now)
outdated_ids[id] += 1;
if (cell.id != id || cell.expiresAt() < now)
outdated_ids[id].push_back(i);
else
{
const auto string_ref = attribute_array[cell_idx];
const auto string_ref = def && cell.isDefault() ? def->getDataAt(i) : attribute_array[cell_idx];
map[id] = String{string_ref};
total_length += string_ref.size + 1;
}
}
}
query_count.fetch_add(ids.size(), std::memory_order_relaxed);
hit_count.fetch_add(ids.size() - outdated_ids.size(), std::memory_order_release);
query_count.fetch_add(rows, std::memory_order_relaxed);
hit_count.fetch_add(rows - outdated_ids.size(), std::memory_order_release);
/// request new values
if (!outdated_ids.empty())
@ -386,28 +497,37 @@ private:
const auto attribute_value = attribute_array[cell_idx];
map[id] = String{attribute_value};
total_length += (attribute_value.size + 1) * outdated_ids[id];
total_length += (attribute_value.size + 1) * outdated_ids[id].size();
}, [&] (const auto id, const auto cell_idx) {
auto attribute_value = def ? def->getDataAt(outdated_ids[id].front()) : attribute_array[cell_idx];
map[id] = String{attribute_value};
total_length += (attribute_value.size + 1) * outdated_ids[id].size();
});
}
out->getChars().reserve(total_length);
const auto & null_value = std::get<String>(attribute.null_values);
for (const auto id : ids)
{
const auto it = map.find(id);
const auto string = it != map.end() ? it->second : std::get<String>(attribute.null_values);
/// @note check seems redundant, null_values are explicitly stored in the `map`
const auto & string = it != map.end() ? it->second : null_value;
out->insertData(string.data(), string.size());
}
}
template <typename F>
void update(const std::vector<id_t> ids, F && on_cell_updated) const
template <typename PresentIdHandler, typename AbsentIdHandler>
void update(
const std::vector<id_t> & requested_ids, PresentIdHandler && on_cell_updated,
AbsentIdHandler && on_id_not_found) const
{
auto stream = source_ptr->loadIds(ids);
auto stream = source_ptr->loadIds(requested_ids);
stream->readPrefix();
HashMap<UInt64, UInt8> remaining_ids{ids.size()};
for (const auto id : ids)
MapType<UInt8> remaining_ids{requested_ids.size()};
for (const auto id : requested_ids)
remaining_ids.insert({ id, 0 });
std::uniform_int_distribution<std::uint64_t> distribution{
@ -429,9 +549,9 @@ private:
const auto & ids = id_column->getData();
/// cache column pointers
std::vector<const IColumn *> column_ptrs(attributes.size());
for (const auto i : ext::range(0, attributes.size()))
column_ptrs[i] = block.getByPosition(i + 1).column.get();
const auto column_ptrs = ext::map<std::vector>(ext::range(0, attributes.size()), [&block] (const auto & i) {
return block.getByPosition(i + 1).column.get();
});
for (const auto i : ext::range(0, ids.size()))
{
@ -453,17 +573,20 @@ private:
cell.id = id;
if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0)
cell.expires_at = std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)};
cell.setExpiresAt(std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)});
else
cell.expires_at = std::chrono::time_point<std::chrono::system_clock>::max();
cell.setExpiresAt(std::chrono::time_point<std::chrono::system_clock>::max());
/// inform caller
on_cell_updated(id, cell_idx);
/// mark corresponding id as found
remaining_ids[id] = 1;
}
}
stream->readSuffix();
/// Check which ids have not been found and require setting null_value
for (const auto id_found_pair : remaining_ids)
{
if (id_found_pair.second)
@ -473,19 +596,24 @@ private:
const auto cell_idx = getCellIdx(id);
auto & cell = cells[cell_idx];
/// Set null_value for each attribute
for (auto & attribute : attributes)
setDefaultAttributeValue(attribute, cell_idx);
/// Check if cell had not been occupied before and increment element counter if it hadn't
if (cell.id == 0 && cell_idx != zero_cell_idx)
element_count.fetch_add(1, std::memory_order_relaxed);
cell.id = id;
if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0)
cell.expires_at = std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)};
cell.setExpiresAt(std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)});
else
cell.expires_at = std::chrono::time_point<std::chrono::system_clock>::max();
cell.setExpiresAt(std::chrono::time_point<std::chrono::system_clock>::max());
on_cell_updated(id, cell_idx);
cell.setDefault();
/// inform caller that the cell has not been found
on_id_not_found(id, cell_idx);
}
}
@ -500,20 +628,20 @@ private:
{
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: std::get<std::unique_ptr<UInt8[]>>(attribute.arrays)[idx] = std::get<UInt8>(attribute.null_values); break;
case AttributeUnderlyingType::UInt16: std::get<std::unique_ptr<UInt16[]>>(attribute.arrays)[idx] = std::get<UInt16>(attribute.null_values); break;
case AttributeUnderlyingType::UInt32: std::get<std::unique_ptr<UInt32[]>>(attribute.arrays)[idx] = std::get<UInt32>(attribute.null_values); break;
case AttributeUnderlyingType::UInt64: std::get<std::unique_ptr<UInt64[]>>(attribute.arrays)[idx] = std::get<UInt64>(attribute.null_values); break;
case AttributeUnderlyingType::Int8: std::get<std::unique_ptr<Int8[]>>(attribute.arrays)[idx] = std::get<Int8>(attribute.null_values); break;
case AttributeUnderlyingType::Int16: std::get<std::unique_ptr<Int16[]>>(attribute.arrays)[idx] = std::get<Int16>(attribute.null_values); break;
case AttributeUnderlyingType::Int32: std::get<std::unique_ptr<Int32[]>>(attribute.arrays)[idx] = std::get<Int32>(attribute.null_values); break;
case AttributeUnderlyingType::Int64: std::get<std::unique_ptr<Int64[]>>(attribute.arrays)[idx] = std::get<Int64>(attribute.null_values); break;
case AttributeUnderlyingType::Float32: std::get<std::unique_ptr<Float32[]>>(attribute.arrays)[idx] = std::get<Float32>(attribute.null_values); break;
case AttributeUnderlyingType::Float64: std::get<std::unique_ptr<Float64[]>>(attribute.arrays)[idx] = std::get<Float64>(attribute.null_values); break;
case AttributeUnderlyingType::UInt8: std::get<ContainerPtrType<UInt8>>(attribute.arrays)[idx] = std::get<UInt8>(attribute.null_values); break;
case AttributeUnderlyingType::UInt16: std::get<ContainerPtrType<UInt16>>(attribute.arrays)[idx] = std::get<UInt16>(attribute.null_values); break;
case AttributeUnderlyingType::UInt32: std::get<ContainerPtrType<UInt32>>(attribute.arrays)[idx] = std::get<UInt32>(attribute.null_values); break;
case AttributeUnderlyingType::UInt64: std::get<ContainerPtrType<UInt64>>(attribute.arrays)[idx] = std::get<UInt64>(attribute.null_values); break;
case AttributeUnderlyingType::Int8: std::get<ContainerPtrType<Int8>>(attribute.arrays)[idx] = std::get<Int8>(attribute.null_values); break;
case AttributeUnderlyingType::Int16: std::get<ContainerPtrType<Int16>>(attribute.arrays)[idx] = std::get<Int16>(attribute.null_values); break;
case AttributeUnderlyingType::Int32: std::get<ContainerPtrType<Int32>>(attribute.arrays)[idx] = std::get<Int32>(attribute.null_values); break;
case AttributeUnderlyingType::Int64: std::get<ContainerPtrType<Int64>>(attribute.arrays)[idx] = std::get<Int64>(attribute.null_values); break;
case AttributeUnderlyingType::Float32: std::get<ContainerPtrType<Float32>>(attribute.arrays)[idx] = std::get<Float32>(attribute.null_values); break;
case AttributeUnderlyingType::Float64: std::get<ContainerPtrType<Float64>>(attribute.arrays)[idx] = std::get<Float64>(attribute.null_values); break;
case AttributeUnderlyingType::String:
{
const auto & null_value_ref = std::get<String>(attribute.null_values);
auto & string_ref = std::get<std::unique_ptr<StringRef[]>>(attribute.arrays)[idx];
auto & string_ref = std::get<ContainerPtrType<StringRef>>(attribute.arrays)[idx];
if (string_ref.data == null_value_ref.data())
return;
@ -532,20 +660,20 @@ private:
{
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: std::get<std::unique_ptr<UInt8[]>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::UInt16: std::get<std::unique_ptr<UInt16[]>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::UInt32: std::get<std::unique_ptr<UInt32[]>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::UInt64: std::get<std::unique_ptr<UInt64[]>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::Int8: std::get<std::unique_ptr<Int8[]>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Int16: std::get<std::unique_ptr<Int16[]>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Int32: std::get<std::unique_ptr<Int32[]>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Int64: std::get<std::unique_ptr<Int64[]>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Float32: std::get<std::unique_ptr<Float32[]>>(attribute.arrays)[idx] = value.get<Float64>(); break;
case AttributeUnderlyingType::Float64: std::get<std::unique_ptr<Float64[]>>(attribute.arrays)[idx] = value.get<Float64>(); break;
case AttributeUnderlyingType::UInt8: std::get<ContainerPtrType<UInt8>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::UInt16: std::get<ContainerPtrType<UInt16>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::UInt32: std::get<ContainerPtrType<UInt32>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::UInt64: std::get<ContainerPtrType<UInt64>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::Int8: std::get<ContainerPtrType<Int8>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Int16: std::get<ContainerPtrType<Int16>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Int32: std::get<ContainerPtrType<Int32>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Int64: std::get<ContainerPtrType<Int64>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Float32: std::get<ContainerPtrType<Float32>>(attribute.arrays)[idx] = value.get<Float64>(); break;
case AttributeUnderlyingType::Float64: std::get<ContainerPtrType<Float64>>(attribute.arrays)[idx] = value.get<Float64>(); break;
case AttributeUnderlyingType::String:
{
const auto & string = value.get<String>();
auto & string_ref = std::get<std::unique_ptr<StringRef[]>>(attribute.arrays)[idx];
auto & string_ref = std::get<ContainerPtrType<StringRef>>(attribute.arrays)[idx];
const auto & null_value_ref = std::get<String>(attribute.null_values);
if (string_ref.data != null_value_ref.data())
{

View File

@ -1,6 +1,7 @@
#pragma once
#include <DB/Dictionaries/IDictionarySource.h>
#include <DB/Dictionaries/DictionaryStructure.h>
#include <DB/Client/ConnectionPool.h>
#include <DB/DataStreams/RemoteBlockInputStream.h>
#include <DB/Interpreters/executeQuery.h>
@ -40,7 +41,8 @@ public:
max_connections, host, port, db, user, password,
"ClickHouseDictionarySource")
},
load_all_query{composeLoadAllQuery()}
load_all_query{composeLoadAllQuery()},
key_tuple_definition{dict_struct.key ? composeKeyTupleDefinition() : std::string{}}
{}
/// copy-constructor is provided in order to support cloneability
@ -69,11 +71,13 @@ public:
BlockInputStreamPtr loadIds(const std::vector<std::uint64_t> & ids) override
{
const auto query = composeLoadIdsQuery(ids);
return createStreamForSelectiveLoad(composeLoadIdsQuery(ids));
}
if (is_local)
return executeQuery(query, context, true).in;
return new RemoteBlockInputStream{pool.get(), query, nullptr};
BlockInputStreamPtr loadKeys(
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows) override
{
return createStreamForSelectiveLoad(composeLoadKeysQuery(key_columns, requested_rows));
}
bool isModified() const override { return true; }
@ -95,35 +99,57 @@ private:
WriteBufferFromString out{query};
writeString("SELECT ", out);
if (!dict_struct.id.expression.empty())
if (dict_struct.id)
{
writeParenthesisedString(dict_struct.id.expression, out);
writeString(" AS ", out);
if (!dict_struct.id->expression.empty())
{
writeParenthesisedString(dict_struct.id->expression, out);
writeString(" AS ", out);
}
writeProbablyBackQuotedString(dict_struct.id->name, out);
if (dict_struct.range_min && dict_struct.range_max)
{
writeString(", ", out);
if (!dict_struct.range_min->expression.empty())
{
writeParenthesisedString(dict_struct.range_min->expression, out);
writeString(" AS ", out);
}
writeProbablyBackQuotedString(dict_struct.range_min->name, out);
writeString(", ", out);
if (!dict_struct.range_max->expression.empty())
{
writeParenthesisedString(dict_struct.range_max->expression, out);
writeString(" AS ", out);
}
writeProbablyBackQuotedString(dict_struct.range_max->name, out);
}
}
writeProbablyBackQuotedString(dict_struct.id.name, out);
if (dict_struct.range_min && dict_struct.range_max)
else if (dict_struct.key)
{
writeString(", ", out);
if (!dict_struct.range_min->expression.empty())
auto first = true;
for (const auto & key : *dict_struct.key)
{
writeParenthesisedString(dict_struct.range_min->expression, out);
writeString(" AS ", out);
if (!first)
writeString(", ", out);
first = false;
if (!key.expression.empty())
{
writeParenthesisedString(key.expression, out);
writeString(" AS ", out);
}
writeProbablyBackQuotedString(key.name, out);
}
writeProbablyBackQuotedString(dict_struct.range_min->name, out);
writeString(", ", out);
if (!dict_struct.range_max->expression.empty())
{
writeParenthesisedString(dict_struct.range_max->expression, out);
writeString(" AS ", out);
}
writeProbablyBackQuotedString(dict_struct.range_max->name, out);
}
for (const auto & attr : dict_struct.attributes)
@ -161,19 +187,22 @@ private:
std::string composeLoadIdsQuery(const std::vector<std::uint64_t> ids)
{
if (!dict_struct.id)
throw Exception{"Simple key required for method", ErrorCodes::UNSUPPORTED_METHOD};
std::string query;
{
WriteBufferFromString out{query};
writeString("SELECT ", out);
if (!dict_struct.id.expression.empty())
if (!dict_struct.id->expression.empty())
{
writeParenthesisedString(dict_struct.id.expression, out);
writeParenthesisedString(dict_struct.id->expression, out);
writeString(" AS ", out);
}
writeProbablyBackQuotedString(dict_struct.id.name, out);
writeProbablyBackQuotedString(dict_struct.id->name, out);
for (const auto & attr : dict_struct.attributes)
{
@ -204,7 +233,7 @@ private:
writeString(" AND ", out);
}
writeProbablyBackQuotedString(dict_struct.id.name, out);
writeProbablyBackQuotedString(dict_struct.id->name, out);
writeString(" IN (", out);
auto first = true;
@ -223,6 +252,118 @@ private:
return query;
}
std::string composeLoadKeysQuery(
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows)
{
if (!dict_struct.key)
throw Exception{"Composite key required for method", ErrorCodes::UNSUPPORTED_METHOD};
std::string query;
{
WriteBufferFromString out{query};
writeString("SELECT ", out);
auto first = true;
for (const auto & key_or_attribute : boost::join(*dict_struct.key, dict_struct.attributes))
{
if (!first)
writeString(", ", out);
first = false;
if (!key_or_attribute.expression.empty())
{
writeParenthesisedString(key_or_attribute.expression, out);
writeString(" AS ", out);
}
writeProbablyBackQuotedString(key_or_attribute.name, out);
}
writeString(" FROM ", out);
if (!db.empty())
{
writeProbablyBackQuotedString(db, out);
writeChar('.', out);
}
writeProbablyBackQuotedString(table, out);
writeString(" WHERE ", out);
if (!where.empty())
{
writeString(where, out);
writeString(" AND ", out);
}
writeString(key_tuple_definition, out);
writeString(" IN (", out);
first = true;
for (const auto row : requested_rows)
{
if (!first)
writeString(", ", out);
first = false;
composeKeyTuple(key_columns, row, out);
}
writeString(");", out);
}
return query;
}
std::string composeKeyTupleDefinition() const
{
if (!dict_struct.key)
throw Exception{"Composite key required for method", ErrorCodes::UNSUPPORTED_METHOD};
std::string result{"("};
auto first = true;
for (const auto & key : *dict_struct.key)
{
if (!first)
result += ", ";
first = false;
result += key.name;
}
result += ")";
return result;
}
void composeKeyTuple(const ConstColumnPlainPtrs & key_columns, const std::size_t row, WriteBuffer & out) const
{
writeString("(", out);
const auto keys_size = key_columns.size();
auto first = true;
for (const auto i : ext::range(0, keys_size))
{
if (!first)
writeString(", ", out);
first = false;
const auto & value = (*key_columns[i])[row];
(*dict_struct.key)[i].type->serializeTextQuoted(value, out);
}
writeString(")", out);
}
BlockInputStreamPtr createStreamForSelectiveLoad(const std::string query)
{
if (is_local)
return executeQuery(query, context, true).in;
return new RemoteBlockInputStream{pool.get(), query, nullptr};
}
const DictionaryStructure dict_struct;
const std::string host;
const UInt16 port;
@ -236,6 +377,7 @@ private:
const bool is_local;
std::unique_ptr<ConnectionPool> pool;
const std::string load_all_query;
const std::string key_tuple_definition;
};
}

View File

@ -0,0 +1,941 @@
#pragma once
#include <DB/Dictionaries/IDictionary.h>
#include <DB/Dictionaries/IDictionarySource.h>
#include <DB/Dictionaries/DictionaryStructure.h>
#include <DB/Common/Arena.h>
#include <DB/Common/ArenaWithFreeLists.h>
#include <DB/Common/SmallObjectPool.h>
#include <DB/Common/HashTable/HashMap.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Core/StringRef.h>
#include <ext/scope_guard.hpp>
#include <ext/bit_cast.hpp>
#include <ext/range.hpp>
#include <ext/map.hpp>
#include <Poco/RWLock.h>
#include <cmath>
#include <atomic>
#include <chrono>
#include <vector>
#include <map>
#include <tuple>
#include <DB/DataStreams/NullBlockInputStream.h>
namespace DB
{
class ComplexKeyCacheDictionary final : public IDictionaryBase
{
public:
ComplexKeyCacheDictionary(const std::string & name, const DictionaryStructure & dict_struct,
DictionarySourcePtr source_ptr, const DictionaryLifetime dict_lifetime,
const std::size_t size)
: name{name}, dict_struct(dict_struct), source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime),
size{round_up_to_power_of_two(size)}
{
if (!this->source_ptr->supportsSelectiveLoad())
throw Exception{
name + ": source cannot be used with ComplexKeyCacheDictionary",
ErrorCodes::UNSUPPORTED_METHOD
};
createAttributes();
}
ComplexKeyCacheDictionary(const ComplexKeyCacheDictionary & other)
: ComplexKeyCacheDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.size}
{}
std::string getKeyDescription() const { return key_description; };
std::exception_ptr getCreationException() const override { return {}; }
std::string getName() const override { return name; }
std::string getTypeName() const override { return "ComplexKeyCache"; }
std::size_t getBytesAllocated() const override
{
return bytes_allocated + key_size_is_fixed ? fixed_size_keys_pool->size() : keys_pool->size();
}
std::size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); }
double getHitRate() const override
{
return static_cast<double>(hit_count.load(std::memory_order_acquire)) /
query_count.load(std::memory_order_relaxed);
}
std::size_t getElementCount() const override { return element_count.load(std::memory_order_relaxed); }
double getLoadFactor() const override
{
return static_cast<double>(element_count.load(std::memory_order_relaxed)) / size;
}
bool isCached() const override { return true; }
DictionaryPtr clone() const override { return std::make_unique<ComplexKeyCacheDictionary>(*this); }
const IDictionarySource * getSource() const override { return source_ptr.get(); }
const DictionaryLifetime & getLifetime() const override { return dict_lifetime; }
const DictionaryStructure & getStructure() const override { return dict_struct; }
std::chrono::time_point<std::chrono::system_clock> getCreationTime() const override
{
return creation_time;
}
bool isInjective(const std::string & attribute_name) const override
{
return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective;
}
#define DECLARE_MULTIPLE_GETTER(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
PODArray<TYPE> & out) const\
{\
validateKeyTypes(key_types);\
\
auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeUnderlyingType::TYPE)\
throw Exception{\
name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\
ErrorCodes::TYPE_MISMATCH\
};\
\
getItems<TYPE>(attribute, key_columns, out);\
}
DECLARE_MULTIPLE_GETTER(UInt8)
DECLARE_MULTIPLE_GETTER(UInt16)
DECLARE_MULTIPLE_GETTER(UInt32)
DECLARE_MULTIPLE_GETTER(UInt64)
DECLARE_MULTIPLE_GETTER(Int8)
DECLARE_MULTIPLE_GETTER(Int16)
DECLARE_MULTIPLE_GETTER(Int32)
DECLARE_MULTIPLE_GETTER(Int64)
DECLARE_MULTIPLE_GETTER(Float32)
DECLARE_MULTIPLE_GETTER(Float64)
#undef DECLARE_MULTIPLE_GETTER
void getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
ColumnString * out) const
{
validateKeyTypes(key_types);
auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeUnderlyingType::String)
throw Exception{
name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH
};
getItems(attribute, key_columns, out);
}
#define DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const PODArray<TYPE> & def, PODArray<TYPE> & out) const\
{\
validateKeyTypes(key_types);\
\
auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeUnderlyingType::TYPE)\
throw Exception{\
name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\
ErrorCodes::TYPE_MISMATCH\
};\
\
getItems<TYPE>(attribute, key_columns, out, &def);\
}
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(UInt8)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(UInt16)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(UInt32)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(UInt64)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Int8)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Int16)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Int32)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Int64)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Float32)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Float64)
#undef DECLARE_MULTIPLE_GETTER_WITH_DEFAULT
void getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const ColumnString * const def, ColumnString * const out) const
{
validateKeyTypes(key_types);
auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeUnderlyingType::String)
throw Exception{
name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH
};
getItems(attribute, key_columns, out, def);
}
void has(const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types, PODArray<UInt8> & out) const
{
validateKeyTypes(key_types);
/// Mapping: <key> -> { all indices `i` of `key_columns` such that `key_columns[i]` = <key> }
MapType<std::vector<std::size_t>> outdated_keys;
const auto rows = key_columns.front()->size();
const auto keys_size = dict_struct.key->size();
StringRefs keys(keys_size);
Arena temporary_keys_pool;
PODArray<StringRef> keys_array(rows);
{
const Poco::ScopedReadRWLock read_lock{rw_lock};
const auto now = std::chrono::system_clock::now();
/// fetch up-to-date values, decide which ones require update
for (const auto row : ext::range(0, rows))
{
const auto key = placeKeysInPool(row, key_columns, keys, temporary_keys_pool);
keys_array[row] = key;
const auto hash = StringRefHash{}(key);
const auto cell_idx = hash & (size - 1);
const auto & cell = cells[cell_idx];
/** cell should be updated if either:
* 1. keys (or hash) do not match,
* 2. cell has expired,
* 3. explicit defaults were specified and cell was set default. */
if (cell.hash != hash || cell.key != key || cell.expiresAt() < now)
outdated_keys[key].push_back(row);
else
out[row] = !cell.isDefault();
}
}
query_count.fetch_add(rows, std::memory_order_relaxed);
hit_count.fetch_add(rows - outdated_keys.size(), std::memory_order_release);
if (outdated_keys.empty())
return;
std::vector<std::size_t> required_rows(outdated_keys.size());
std::transform(std::begin(outdated_keys), std::end(outdated_keys), std::begin(required_rows),
[] (auto & pair) { return pair.second.front(); });
/// request new values
update(key_columns, keys_array, required_rows, [&] (const auto key, const auto) {
for (const auto out_idx : outdated_keys[key])
out[out_idx] = true;
}, [&] (const auto key, const auto) {
for (const auto out_idx : outdated_keys[key])
out[out_idx] = false;
});
}
private:
template <typename Value> using MapType = HashMapWithSavedHash<StringRef, Value, StringRefHash>;
template <typename Value> using ContainerType = Value[];
template <typename Value> using ContainerPtrType = std::unique_ptr<ContainerType<Value>>;
struct cell_metadata_t final
{
using time_point_t = std::chrono::system_clock::time_point;
using time_point_rep_t = time_point_t::rep;
using time_point_urep_t = std::make_unsigned_t<time_point_rep_t>;
static constexpr std::uint64_t EXPIRES_AT_MASK = std::numeric_limits<time_point_rep_t>::max();
static constexpr std::uint64_t IS_DEFAULT_MASK = ~EXPIRES_AT_MASK;
StringRef key;
decltype(StringRefHash{}(key)) hash;
/// Stores both expiration time and `is_default` flag in the most significant bit
time_point_urep_t data;
/// Sets expiration time, resets `is_default` flag to false
time_point_t expiresAt() const { return ext::safe_bit_cast<time_point_t>(data & EXPIRES_AT_MASK); }
void setExpiresAt(const time_point_t & t) { data = ext::safe_bit_cast<time_point_urep_t>(t); }
bool isDefault() const { return (data & IS_DEFAULT_MASK) == IS_DEFAULT_MASK; }
void setDefault() { data |= IS_DEFAULT_MASK; }
};
struct attribute_t final
{
AttributeUnderlyingType type;
std::tuple<
UInt8, UInt16, UInt32, UInt64,
Int8, Int16, Int32, Int64,
Float32, Float64,
String> null_values;
std::tuple<
ContainerPtrType<UInt8>, ContainerPtrType<UInt16>, ContainerPtrType<UInt32>, ContainerPtrType<UInt64>,
ContainerPtrType<Int8>, ContainerPtrType<Int16>, ContainerPtrType<Int32>, ContainerPtrType<Int64>,
ContainerPtrType<Float32>, ContainerPtrType<Float64>,
ContainerPtrType<StringRef>> arrays;
};
void createAttributes()
{
const auto size = dict_struct.attributes.size();
attributes.reserve(size);
bytes_allocated += size * sizeof(cell_metadata_t);
bytes_allocated += size * sizeof(attributes.front());
for (const auto & attribute : dict_struct.attributes)
{
attribute_index_by_name.emplace(attribute.name, attributes.size());
attributes.push_back(createAttributeWithType(attribute.underlying_type, attribute.null_value));
if (attribute.hierarchical)
throw Exception{
name + ": hierarchical attributes not supported for dictionary of type " + getTypeName(),
ErrorCodes::TYPE_MISMATCH
};
}
}
attribute_t createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value)
{
attribute_t attr{type};
switch (type)
{
case AttributeUnderlyingType::UInt8:
std::get<UInt8>(attr.null_values) = null_value.get<UInt64>();
std::get<ContainerPtrType<UInt8>>(attr.arrays) = std::make_unique<ContainerType<UInt8>>(size);
bytes_allocated += size * sizeof(UInt8);
break;
case AttributeUnderlyingType::UInt16:
std::get<UInt16>(attr.null_values) = null_value.get<UInt64>();
std::get<ContainerPtrType<UInt16>>(attr.arrays) = std::make_unique<ContainerType<UInt16>>(size);
bytes_allocated += size * sizeof(UInt16);
break;
case AttributeUnderlyingType::UInt32:
std::get<UInt32>(attr.null_values) = null_value.get<UInt64>();
std::get<ContainerPtrType<UInt32>>(attr.arrays) = std::make_unique<ContainerType<UInt32>>(size);
bytes_allocated += size * sizeof(UInt32);
break;
case AttributeUnderlyingType::UInt64:
std::get<UInt64>(attr.null_values) = null_value.get<UInt64>();
std::get<ContainerPtrType<UInt64>>(attr.arrays) = std::make_unique<ContainerType<UInt64>>(size);
bytes_allocated += size * sizeof(UInt64);
break;
case AttributeUnderlyingType::Int8:
std::get<Int8>(attr.null_values) = null_value.get<Int64>();
std::get<ContainerPtrType<Int8>>(attr.arrays) = std::make_unique<ContainerType<Int8>>(size);
bytes_allocated += size * sizeof(Int8);
break;
case AttributeUnderlyingType::Int16:
std::get<Int16>(attr.null_values) = null_value.get<Int64>();
std::get<ContainerPtrType<Int16>>(attr.arrays) = std::make_unique<ContainerType<Int16>>(size);
bytes_allocated += size * sizeof(Int16);
break;
case AttributeUnderlyingType::Int32:
std::get<Int32>(attr.null_values) = null_value.get<Int64>();
std::get<ContainerPtrType<Int32>>(attr.arrays) = std::make_unique<ContainerType<Int32>>(size);
bytes_allocated += size * sizeof(Int32);
break;
case AttributeUnderlyingType::Int64:
std::get<Int64>(attr.null_values) = null_value.get<Int64>();
std::get<ContainerPtrType<Int64>>(attr.arrays) = std::make_unique<ContainerType<Int64>>(size);
bytes_allocated += size * sizeof(Int64);
break;
case AttributeUnderlyingType::Float32:
std::get<Float32>(attr.null_values) = null_value.get<Float64>();
std::get<ContainerPtrType<Float32>>(attr.arrays) = std::make_unique<ContainerType<Float32>>(size);
bytes_allocated += size * sizeof(Float32);
break;
case AttributeUnderlyingType::Float64:
std::get<Float64>(attr.null_values) = null_value.get<Float64>();
std::get<ContainerPtrType<Float64>>(attr.arrays) = std::make_unique<ContainerType<Float64>>(size);
bytes_allocated += size * sizeof(Float64);
break;
case AttributeUnderlyingType::String:
std::get<String>(attr.null_values) = null_value.get<String>();
std::get<ContainerPtrType<StringRef>>(attr.arrays) = std::make_unique<ContainerType<StringRef>>(size);
bytes_allocated += size * sizeof(StringRef);
break;
}
return attr;
}
static std::string createKeyDescription(const DictionaryStructure & dict_struct)
{
std::ostringstream out;
out << '(';
auto first = true;
for (const auto & key : *dict_struct.key)
{
if (!first)
out << ", ";
first = false;
out << key.type->getName();
}
out << ')';
return out.str();
}
void validateKeyTypes(const DataTypes & key_types) const
{
if (key_types.size() != dict_struct.key->size())
throw Exception{
"Key structure does not match, expected " + key_description,
ErrorCodes::TYPE_MISMATCH
};
for (const auto i : ext::range(0, key_types.size()))
{
const auto & expected_type = (*dict_struct.key)[i].type->getName();
const auto & actual_type = key_types[i]->getName();
if (expected_type != actual_type)
throw Exception{
"Key type at position " + std::to_string(i) + " does not match, expected " + expected_type +
", found " + actual_type,
ErrorCodes::TYPE_MISMATCH
};
}
}
template <typename T>
void getItems(
attribute_t & attribute, const ConstColumnPlainPtrs & key_columns, PODArray<T> & out,
const PODArray<T> * const def = nullptr) const
{
/// Mapping: <key> -> { all indices `i` of `key_columns` such that `key_columns[i]` = <key> }
MapType<std::vector<std::size_t>> outdated_keys;
auto & attribute_array = std::get<ContainerPtrType<T>>(attribute.arrays);
const auto rows = key_columns.front()->size();
const auto keys_size = dict_struct.key->size();
StringRefs keys(keys_size);
Arena temporary_keys_pool;
PODArray<StringRef> keys_array(rows);
{
const Poco::ScopedReadRWLock read_lock{rw_lock};
const auto now = std::chrono::system_clock::now();
/// fetch up-to-date values, decide which ones require update
for (const auto row : ext::range(0, rows))
{
const auto key = placeKeysInPool(row, key_columns, keys, temporary_keys_pool);
keys_array[row] = key;
const auto hash = StringRefHash{}(key);
const auto cell_idx = hash & (size - 1);
const auto & cell = cells[cell_idx];
/** cell should be updated if either:
* 1. keys (or hash) do not match,
* 2. cell has expired,
* 3. explicit defaults were specified and cell was set default. */
if (cell.hash != hash || cell.key != key || cell.expiresAt() < now)
outdated_keys[key].push_back(row);
else
out[row] = def && cell.isDefault() ? (*def)[row] : attribute_array[cell_idx];
}
}
query_count.fetch_add(rows, std::memory_order_relaxed);
hit_count.fetch_add(rows - outdated_keys.size(), std::memory_order_release);
if (outdated_keys.empty())
return;
std::vector<std::size_t> required_rows(outdated_keys.size());
std::transform(std::begin(outdated_keys), std::end(outdated_keys), std::begin(required_rows),
[] (auto & pair) { return pair.second.front(); });
/// request new values
update(key_columns, keys_array, required_rows, [&] (const auto key, const auto cell_idx) {
const auto attribute_value = attribute_array[cell_idx];
for (const auto out_idx : outdated_keys[key])
out[out_idx] = attribute_value;
}, [&] (const auto key, const auto cell_idx) {
const auto attribute_value = !def ? attribute_array[cell_idx] : (*def)[outdated_keys[key].front()];
for (const auto out_idx : outdated_keys[key])
out[out_idx] = attribute_value;
});
}
void getItems(
attribute_t & attribute, const ConstColumnPlainPtrs & key_columns, ColumnString * out,
const ColumnString * const def = nullptr) const
{
const auto rows = key_columns.front()->size();
/// save on some allocations
out->getOffsets().reserve(rows);
const auto keys_size = dict_struct.key->size();
StringRefs keys(keys_size);
Arena temporary_keys_pool;
auto & attribute_array = std::get<ContainerPtrType<StringRef>>(attribute.arrays);
auto found_outdated_values = false;
/// perform optimistic version, fallback to pessimistic if failed
{
const Poco::ScopedReadRWLock read_lock{rw_lock};
const auto now = std::chrono::system_clock::now();
/// fetch up-to-date values, discard on fail
for (const auto row : ext::range(0, rows))
{
const auto key = placeKeysInPool(row, key_columns, keys, temporary_keys_pool);
SCOPE_EXIT(temporary_keys_pool.rollback(key.size));
const auto hash = StringRefHash{}(key);
const auto cell_idx = hash & (size - 1);
const auto & cell = cells[cell_idx];
if (cell.hash != hash || cell.key != key || cell.expiresAt() < now)
{
found_outdated_values = true;
break;
}
else
{
const auto string_ref = def && cell.isDefault() ? def->getDataAt(row) : attribute_array[cell_idx];
out->insertData(string_ref.data, string_ref.size);
}
}
}
/// optimistic code completed successfully
if (!found_outdated_values)
{
query_count.fetch_add(rows, std::memory_order_relaxed);
hit_count.fetch_add(rows, std::memory_order_release);
return;
}
/// now onto the pessimistic one, discard possible partial results from the optimistic path
out->getChars().resize_assume_reserved(0);
out->getOffsets().resize_assume_reserved(0);
/// Mapping: <key> -> { all indices `i` of `key_columns` such that `key_columns[i]` = <key> }
MapType<std::vector<std::size_t>> outdated_keys;
/// we are going to store every string separately
MapType<String> map;
PODArray<StringRef> keys_array(rows);
std::size_t total_length = 0;
{
const Poco::ScopedReadRWLock read_lock{rw_lock};
const auto now = std::chrono::system_clock::now();
for (const auto row : ext::range(0, rows))
{
const auto key = placeKeysInPool(row, key_columns, keys, temporary_keys_pool);
keys_array[row] = key;
const auto hash = StringRefHash{}(key);
const auto cell_idx = hash & (size - 1);
const auto & cell = cells[cell_idx];
if (cell.hash != hash || cell.key != key || cell.expiresAt() < now)
outdated_keys[key].push_back(row);
else
{
const auto string_ref = def && cell.isDefault() ? def->getDataAt(row) : attribute_array[cell_idx];
map[key] = String{string_ref};
total_length += string_ref.size + 1;
}
}
}
query_count.fetch_add(rows, std::memory_order_relaxed);
hit_count.fetch_add(rows - outdated_keys.size(), std::memory_order_release);
/// request new values
if (!outdated_keys.empty())
{
std::vector<std::size_t> required_rows(outdated_keys.size());
std::transform(std::begin(outdated_keys), std::end(outdated_keys), std::begin(required_rows),
[] (auto & pair) { return pair.second.front(); });
update(key_columns, keys_array, required_rows, [&] (const auto key, const auto cell_idx) {
const auto attribute_value = attribute_array[cell_idx];
map[key] = String{attribute_value};
total_length += (attribute_value.size + 1) * outdated_keys[key].size();
}, [&] (const auto key, const auto cell_idx) {
auto attribute_value = def ? def->getDataAt(outdated_keys[key].front()) : attribute_array[cell_idx];
map[key] = String{attribute_value};
total_length += (attribute_value.size + 1) * outdated_keys[key].size();
});
}
out->getChars().reserve(total_length);
const auto & null_value = std::get<String>(attribute.null_values);
for (const auto key : keys_array)
{
const auto it = map.find(key);
/// @note check seems redundant, null_values are explicitly stored in the `map`
const auto & string = it != map.end() ? it->second : null_value;
out->insertData(string.data(), string.size());
}
}
template <typename PresentKeyHandler, typename AbsentKeyHandler>
void update(
const ConstColumnPlainPtrs & in_key_columns, const PODArray<StringRef> & in_keys,
const std::vector<std::size_t> & in_requested_rows, PresentKeyHandler && on_cell_updated,
AbsentKeyHandler && on_key_not_found) const
{
auto stream = source_ptr->loadKeys(in_key_columns, in_requested_rows);
stream->readPrefix();
MapType<UInt8> remaining_keys{in_requested_rows.size()};
for (const auto row : in_requested_rows)
remaining_keys.insert({ in_keys[row], 0 });
std::uniform_int_distribution<std::uint64_t> distribution{
dict_lifetime.min_sec,
dict_lifetime.max_sec
};
const Poco::ScopedWriteRWLock write_lock{rw_lock};
const auto keys_size = dict_struct.key->size();
StringRefs keys(keys_size);
const auto attributes_size = attributes.size();
while (const auto block = stream->read())
{
/// cache column pointers
const auto key_columns = ext::map<ConstColumnPlainPtrs>(ext::range(0, keys_size),
[&] (const std::size_t attribute_idx) {
return block.getByPosition(attribute_idx).column.get();
});
const auto attribute_columns = ext::map<ConstColumnPlainPtrs>(ext::range(0, attributes_size),
[&] (const std::size_t attribute_idx) {
return block.getByPosition(keys_size + attribute_idx).column.get();
});
const auto rows = block.rowsInFirstColumn();
for (const auto row : ext::range(0, rows))
{
auto key = allocKey(row, key_columns, keys);
const auto hash = StringRefHash{}(key);
const auto cell_idx = hash & (size - 1);
auto & cell = cells[cell_idx];
for (const auto attribute_idx : ext::range(0, attributes.size()))
{
const auto & attribute_column = *attribute_columns[attribute_idx];
auto & attribute = attributes[attribute_idx];
setAttributeValue(attribute, cell_idx, attribute_column[row]);
}
/// if cell id is zero and zero does not map to this cell, then the cell is unused
if (cell.key == StringRef{} && cell_idx != zero_cell_idx)
element_count.fetch_add(1, std::memory_order_relaxed);
/// handle memory allocated for old key
if (key == cell.key)
{
freeKey(key);
key = cell.key;
}
else
{
/// new key is different from the old one
if (cell.key.data)
freeKey(cell.key);
cell.key = key;
}
cell.hash = hash;
if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0)
cell.setExpiresAt(std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)});
else
cell.setExpiresAt(std::chrono::time_point<std::chrono::system_clock>::max());
/// inform caller
on_cell_updated(key, cell_idx);
/// mark corresponding id as found
remaining_keys[key] = 1;
}
}
stream->readSuffix();
/// Check which ids have not been found and require setting null_value
for (const auto key_found_pair : remaining_keys)
{
if (key_found_pair.second)
continue;
auto key = key_found_pair.first;
const auto hash = StringRefHash{}(key);
const auto cell_idx = hash & (size - 1);
auto & cell = cells[cell_idx];
/// Set null_value for each attribute
for (auto & attribute : attributes)
setDefaultAttributeValue(attribute, cell_idx);
/// Check if cell had not been occupied before and increment element counter if it hadn't
if (cell.key == StringRef{} && cell_idx != zero_cell_idx)
element_count.fetch_add(1, std::memory_order_relaxed);
if (key == cell.key)
key = cell.key;
else
{
if (cell.key.data)
freeKey(cell.key);
/// copy key from temporary pool
key = copyKey(key);
cell.key = key;
}
cell.hash = hash;
if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0)
cell.setExpiresAt(std::chrono::system_clock::now() + std::chrono::seconds{distribution(rnd_engine)});
else
cell.setExpiresAt(std::chrono::time_point<std::chrono::system_clock>::max());
cell.setDefault();
/// inform caller that the cell has not been found
on_key_not_found(key, cell_idx);
}
}
std::uint64_t getCellIdx(const StringRef key) const
{
const auto hash = StringRefHash{}(key);
const auto idx = hash & (size - 1);
return idx;
}
void setDefaultAttributeValue(attribute_t & attribute, const std::size_t idx) const
{
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: std::get<ContainerPtrType<UInt8>>(attribute.arrays)[idx] = std::get<UInt8>(attribute.null_values); break;
case AttributeUnderlyingType::UInt16: std::get<ContainerPtrType<UInt16>>(attribute.arrays)[idx] = std::get<UInt16>(attribute.null_values); break;
case AttributeUnderlyingType::UInt32: std::get<ContainerPtrType<UInt32>>(attribute.arrays)[idx] = std::get<UInt32>(attribute.null_values); break;
case AttributeUnderlyingType::UInt64: std::get<ContainerPtrType<UInt64>>(attribute.arrays)[idx] = std::get<UInt64>(attribute.null_values); break;
case AttributeUnderlyingType::Int8: std::get<ContainerPtrType<Int8>>(attribute.arrays)[idx] = std::get<Int8>(attribute.null_values); break;
case AttributeUnderlyingType::Int16: std::get<ContainerPtrType<Int16>>(attribute.arrays)[idx] = std::get<Int16>(attribute.null_values); break;
case AttributeUnderlyingType::Int32: std::get<ContainerPtrType<Int32>>(attribute.arrays)[idx] = std::get<Int32>(attribute.null_values); break;
case AttributeUnderlyingType::Int64: std::get<ContainerPtrType<Int64>>(attribute.arrays)[idx] = std::get<Int64>(attribute.null_values); break;
case AttributeUnderlyingType::Float32: std::get<ContainerPtrType<Float32>>(attribute.arrays)[idx] = std::get<Float32>(attribute.null_values); break;
case AttributeUnderlyingType::Float64: std::get<ContainerPtrType<Float64>>(attribute.arrays)[idx] = std::get<Float64>(attribute.null_values); break;
case AttributeUnderlyingType::String:
{
const auto & null_value_ref = std::get<String>(attribute.null_values);
auto & string_ref = std::get<ContainerPtrType<StringRef>>(attribute.arrays)[idx];
if (string_ref.data == null_value_ref.data())
return;
if (string_ref.size != 0)
bytes_allocated -= string_ref.size + 1;
const std::unique_ptr<const char[]> deleter{string_ref.data};
string_ref = StringRef{null_value_ref};
break;
}
}
}
void setAttributeValue(attribute_t & attribute, const std::size_t idx, const Field & value) const
{
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: std::get<ContainerPtrType<UInt8>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::UInt16: std::get<ContainerPtrType<UInt16>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::UInt32: std::get<ContainerPtrType<UInt32>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::UInt64: std::get<ContainerPtrType<UInt64>>(attribute.arrays)[idx] = value.get<UInt64>(); break;
case AttributeUnderlyingType::Int8: std::get<ContainerPtrType<Int8>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Int16: std::get<ContainerPtrType<Int16>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Int32: std::get<ContainerPtrType<Int32>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Int64: std::get<ContainerPtrType<Int64>>(attribute.arrays)[idx] = value.get<Int64>(); break;
case AttributeUnderlyingType::Float32: std::get<ContainerPtrType<Float32>>(attribute.arrays)[idx] = value.get<Float64>(); break;
case AttributeUnderlyingType::Float64: std::get<ContainerPtrType<Float64>>(attribute.arrays)[idx] = value.get<Float64>(); break;
case AttributeUnderlyingType::String:
{
const auto & string = value.get<String>();
auto & string_ref = std::get<ContainerPtrType<StringRef>>(attribute.arrays)[idx];
const auto & null_value_ref = std::get<String>(attribute.null_values);
if (string_ref.data != null_value_ref.data())
{
if (string_ref.size != 0)
bytes_allocated -= string_ref.size + 1;
/// avoid explicit delete, let unique_ptr handle it
const std::unique_ptr<const char[]> deleter{string_ref.data};
}
const auto size = string.size();
if (size != 0)
{
auto string_ptr = std::make_unique<char[]>(size + 1);
std::copy(string.data(), string.data() + size + 1, string_ptr.get());
string_ref = StringRef{string_ptr.release(), size};
bytes_allocated += size + 1;
}
else
string_ref = {};
break;
}
}
}
attribute_t & getAttribute(const std::string & attribute_name) const
{
const auto it = attribute_index_by_name.find(attribute_name);
if (it == std::end(attribute_index_by_name))
throw Exception{
name + ": no such attribute '" + attribute_name + "'",
ErrorCodes::BAD_ARGUMENTS
};
return attributes[it->second];
}
StringRef allocKey(const std::size_t row, const ConstColumnPlainPtrs & key_columns, StringRefs & keys) const
{
if (key_size_is_fixed)
return placeKeysInFixedSizePool(row, key_columns);
return placeKeysInPool(row, key_columns, keys, *keys_pool);
}
void freeKey(const StringRef key) const
{
if (key_size_is_fixed)
fixed_size_keys_pool->free(key.data);
else
keys_pool->free(key.data, key.size);
}
static std::size_t round_up_to_power_of_two(std::size_t n)
{
--n;
n |= n >> 1;
n |= n >> 2;
n |= n >> 4;
n |= n >> 8;
n |= n >> 16;
n |= n >> 32;
++n;
return n;
}
static std::uint64_t getSeed()
{
timespec ts;
clock_gettime(CLOCK_MONOTONIC, &ts);
return ts.tv_nsec ^ getpid();
}
template <typename Arena>
static StringRef placeKeysInPool(
const std::size_t row, const ConstColumnPlainPtrs & key_columns, StringRefs & keys, Arena & pool)
{
const auto keys_size = key_columns.size();
size_t sum_keys_size{};
for (const auto i : ext::range(0, keys_size))
{
keys[i] = key_columns[i]->getDataAtWithTerminatingZero(row);
sum_keys_size += keys[i].size;
}
const auto res = pool.alloc(sum_keys_size);
auto place = res;
for (size_t j = 0; j < keys_size; ++j)
{
memcpy(place, keys[j].data, keys[j].size);
place += keys[j].size;
}
return { res, sum_keys_size };
}
StringRef placeKeysInFixedSizePool(
const std::size_t row, const ConstColumnPlainPtrs & key_columns) const
{
const auto res = fixed_size_keys_pool->alloc();
auto place = res;
for (const auto & key_column : key_columns)
{
const auto key = key_column->getDataAt(row);
memcpy(place, key.data, key.size);
place += key.size;
}
return { res, key_size };
}
StringRef copyKey(const StringRef key) const
{
const auto res = key_size_is_fixed ? fixed_size_keys_pool->alloc() : keys_pool->alloc(key.size);
memcpy(res, key.data, key.size);
return { res, key.size };
}
const std::string name;
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;
const std::string key_description{createKeyDescription(dict_struct)};
mutable Poco::RWLock rw_lock;
const std::size_t size;
const std::uint64_t zero_cell_idx{getCellIdx(StringRef{})};
std::map<std::string, std::size_t> attribute_index_by_name;
mutable std::vector<attribute_t> attributes;
mutable std::vector<cell_metadata_t> cells{size};
const bool key_size_is_fixed{dict_struct.isKeySizeFixed()};
std::size_t key_size{key_size_is_fixed ? dict_struct.getKeySize() : 0};
std::unique_ptr<ArenaWithFreeLists> keys_pool = key_size_is_fixed ? nullptr :
std::make_unique<ArenaWithFreeLists>();
std::unique_ptr<SmallObjectPool> fixed_size_keys_pool = key_size_is_fixed ?
std::make_unique<SmallObjectPool>(key_size) : nullptr;
mutable std::mt19937_64 rnd_engine{getSeed()};
mutable std::size_t bytes_allocated = 0;
mutable std::atomic<std::size_t> element_count{0};
mutable std::atomic<std::size_t> hit_count{0};
mutable std::atomic<std::size_t> query_count{0};
const std::chrono::time_point<std::chrono::system_clock> creation_time = std::chrono::system_clock::now();
};
}

View File

@ -0,0 +1,562 @@
#pragma once
#include <DB/Dictionaries/IDictionary.h>
#include <DB/Dictionaries/IDictionarySource.h>
#include <DB/Dictionaries/DictionaryStructure.h>
#include <DB/Core/StringRef.h>
#include <DB/Common/HashTable/HashMap.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Common/Arena.h>
#include <ext/range.hpp>
#include <atomic>
#include <memory>
#include <tuple>
namespace DB
{
class ComplexKeyHashedDictionary final : public IDictionaryBase
{
public:
ComplexKeyHashedDictionary(
const std::string & name, const DictionaryStructure & dict_struct, DictionarySourcePtr source_ptr,
const DictionaryLifetime dict_lifetime, bool require_nonempty)
: name{name}, dict_struct(dict_struct), source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime),
require_nonempty(require_nonempty), key_description{createKeyDescription(dict_struct)}
{
createAttributes();
try
{
loadData();
calculateBytesAllocated();
}
catch (...)
{
creation_exception = std::current_exception();
}
creation_time = std::chrono::system_clock::now();
}
ComplexKeyHashedDictionary(const ComplexKeyHashedDictionary & other)
: ComplexKeyHashedDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty}
{}
std::string getKeyDescription() const { return key_description; };
std::exception_ptr getCreationException() const override { return creation_exception; }
std::string getName() const override { return name; }
std::string getTypeName() const override { return "ComplexKeyHashed"; }
std::size_t getBytesAllocated() const override { return bytes_allocated; }
std::size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); }
double getHitRate() const override { return 1.0; }
std::size_t getElementCount() const override { return element_count; }
double getLoadFactor() const override { return static_cast<double>(element_count) / bucket_count; }
bool isCached() const override { return false; }
DictionaryPtr clone() const override { return std::make_unique<ComplexKeyHashedDictionary>(*this); }
const IDictionarySource * getSource() const override { return source_ptr.get(); }
const DictionaryLifetime & getLifetime() const override { return dict_lifetime; }
const DictionaryStructure & getStructure() const override { return dict_struct; }
std::chrono::time_point<std::chrono::system_clock> getCreationTime() const override
{
return creation_time;
}
bool isInjective(const std::string & attribute_name) const override
{
return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective;
}
#define DECLARE_MULTIPLE_GETTER(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
PODArray<TYPE> & out) const\
{\
validateKeyTypes(key_types);\
\
const auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeUnderlyingType::TYPE)\
throw Exception{\
name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\
ErrorCodes::TYPE_MISMATCH\
};\
\
const auto null_value = std::get<TYPE>(attribute.null_values);\
\
getItems<TYPE>(attribute, key_columns,\
[&] (const std::size_t row, const auto value) { out[row] = value; },\
[&] (const std::size_t) { return null_value; });\
}
DECLARE_MULTIPLE_GETTER(UInt8)
DECLARE_MULTIPLE_GETTER(UInt16)
DECLARE_MULTIPLE_GETTER(UInt32)
DECLARE_MULTIPLE_GETTER(UInt64)
DECLARE_MULTIPLE_GETTER(Int8)
DECLARE_MULTIPLE_GETTER(Int16)
DECLARE_MULTIPLE_GETTER(Int32)
DECLARE_MULTIPLE_GETTER(Int64)
DECLARE_MULTIPLE_GETTER(Float32)
DECLARE_MULTIPLE_GETTER(Float64)
#undef DECLARE_MULTIPLE_GETTER
void getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
ColumnString * out) const
{
validateKeyTypes(key_types);
const auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeUnderlyingType::String)
throw Exception{
name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH
};
const auto & null_value = StringRef{std::get<String>(attribute.null_values)};
getItems<StringRef>(attribute, key_columns,
[&] (const std::size_t row, const StringRef value) { out->insertData(value.data, value.size); },
[&] (const std::size_t) { return null_value; });
}
#define DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const PODArray<TYPE> & def, PODArray<TYPE> & out) const\
{\
validateKeyTypes(key_types);\
\
const auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeUnderlyingType::TYPE)\
throw Exception{\
name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\
ErrorCodes::TYPE_MISMATCH\
};\
\
getItems<TYPE>(attribute, key_columns,\
[&] (const std::size_t row, const auto value) { out[row] = value; },\
[&] (const std::size_t row) { return def[row]; });\
}
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(UInt8)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(UInt16)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(UInt32)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(UInt64)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Int8)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Int16)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Int32)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Int64)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Float32)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Float64)
#undef DECLARE_MULTIPLE_GETTER_WITH_DEFAULT
void getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const ColumnString * const def, ColumnString * const out) const
{
validateKeyTypes(key_types);
const auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeUnderlyingType::String)
throw Exception{
name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH
};
getItems<StringRef>(attribute, key_columns,
[&] (const std::size_t row, const StringRef value) { out->insertData(value.data, value.size); },
[&] (const std::size_t row) { return def->getDataAt(row); });
}
void has(const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types, PODArray<UInt8> & out) const
{
validateKeyTypes(key_types);
const auto & attribute = attributes.front();
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: has<UInt8>(attribute, key_columns, out); break;
case AttributeUnderlyingType::UInt16: has<UInt16>(attribute, key_columns, out); break;
case AttributeUnderlyingType::UInt32: has<UInt32>(attribute, key_columns, out); break;
case AttributeUnderlyingType::UInt64: has<UInt64>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Int8: has<Int8>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Int16: has<Int16>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Int32: has<Int32>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Int64: has<Int64>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Float32: has<Float32>(attribute, key_columns, out); break;
case AttributeUnderlyingType::Float64: has<Float64>(attribute, key_columns, out); break;
case AttributeUnderlyingType::String: has<StringRef>(attribute, key_columns, out); break;
}
}
private:
template <typename Value> using ContainerType = HashMapWithSavedHash<StringRef, Value, StringRefHash>;
template <typename Value> using ContainerPtrType = std::unique_ptr<ContainerType<Value>>;
struct attribute_t final
{
AttributeUnderlyingType type;
std::tuple<
UInt8, UInt16, UInt32, UInt64,
Int8, Int16, Int32, Int64,
Float32, Float64,
String> null_values;
std::tuple<
ContainerPtrType<UInt8>, ContainerPtrType<UInt16>, ContainerPtrType<UInt32>, ContainerPtrType<UInt64>,
ContainerPtrType<Int8>, ContainerPtrType<Int16>, ContainerPtrType<Int32>, ContainerPtrType<Int64>,
ContainerPtrType<Float32>, ContainerPtrType<Float64>,
ContainerPtrType<StringRef>> maps;
std::unique_ptr<Arena> string_arena;
};
void createAttributes()
{
const auto size = dict_struct.attributes.size();
attributes.reserve(size);
for (const auto & attribute : dict_struct.attributes)
{
attribute_index_by_name.emplace(attribute.name, attributes.size());
attributes.push_back(createAttributeWithType(attribute.underlying_type, attribute.null_value));
if (attribute.hierarchical)
throw Exception{
name + ": hierarchical attributes not supported for dictionary of type " + getTypeName(),
ErrorCodes::TYPE_MISMATCH
};
}
}
void loadData()
{
auto stream = source_ptr->loadAll();
stream->readPrefix();
/// created upfront to avoid excess allocations
const auto keys_size = dict_struct.key->size();
StringRefs keys(keys_size);
const auto attributes_size = attributes.size();
while (const auto block = stream->read())
{
const auto rows = block.rowsInFirstColumn();
element_count += rows;
const auto key_column_ptrs = ext::map<ConstColumnPlainPtrs>(ext::range(0, keys_size),
[&] (const std::size_t attribute_idx) {
return block.getByPosition(attribute_idx).column.get();
});
const auto attribute_column_ptrs = ext::map<ConstColumnPlainPtrs>(ext::range(0, attributes_size),
[&] (const std::size_t attribute_idx) {
return block.getByPosition(keys_size + attribute_idx).column.get();
});
for (const auto row_idx : ext::range(0, rows))
{
/// calculate key once per row
const auto key = placeKeysInPool(row_idx, key_column_ptrs, keys, keys_pool);
auto should_rollback = false;
for (const auto attribute_idx : ext::range(0, attributes_size))
{
const auto & attribute_column = *attribute_column_ptrs[attribute_idx];
auto & attribute = attributes[attribute_idx];
const auto inserted = setAttributeValue(attribute, key, attribute_column[row_idx]);
if (!inserted)
should_rollback = true;
}
/// @note on multiple equal keys the mapped value for the first one is stored
if (should_rollback)
keys_pool.rollback(key.size);
}
}
stream->readSuffix();
if (require_nonempty && 0 == element_count)
throw Exception{
name + ": dictionary source is empty and 'require_nonempty' property is set.",
ErrorCodes::DICTIONARY_IS_EMPTY
};
}
template <typename T>
void addAttributeSize(const attribute_t & attribute)
{
const auto & map_ref = std::get<ContainerPtrType<T>>(attribute.maps);
bytes_allocated += sizeof(ContainerType<T>) + map_ref->getBufferSizeInBytes();
bucket_count = map_ref->getBufferSizeInCells();
}
void calculateBytesAllocated()
{
bytes_allocated += attributes.size() * sizeof(attributes.front());
for (const auto & attribute : attributes)
{
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: addAttributeSize<UInt8>(attribute); break;
case AttributeUnderlyingType::UInt16: addAttributeSize<UInt16>(attribute); break;
case AttributeUnderlyingType::UInt32: addAttributeSize<UInt32>(attribute); break;
case AttributeUnderlyingType::UInt64: addAttributeSize<UInt64>(attribute); break;
case AttributeUnderlyingType::Int8: addAttributeSize<Int8>(attribute); break;
case AttributeUnderlyingType::Int16: addAttributeSize<Int16>(attribute); break;
case AttributeUnderlyingType::Int32: addAttributeSize<Int32>(attribute); break;
case AttributeUnderlyingType::Int64: addAttributeSize<Int64>(attribute); break;
case AttributeUnderlyingType::Float32: addAttributeSize<Float32>(attribute); break;
case AttributeUnderlyingType::Float64: addAttributeSize<Float64>(attribute); break;
case AttributeUnderlyingType::String:
{
addAttributeSize<StringRef>(attribute);
bytes_allocated += sizeof(Arena) + attribute.string_arena->size();
break;
}
}
}
bytes_allocated += keys_pool.size();
}
template <typename T>
void createAttributeImpl(attribute_t & attribute, const Field & null_value)
{
std::get<T>(attribute.null_values) = null_value.get<typename NearestFieldType<T>::Type>();
std::get<ContainerPtrType<T>>(attribute.maps) = std::make_unique<ContainerType<T>>();
}
attribute_t createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value)
{
attribute_t attr{type};
switch (type)
{
case AttributeUnderlyingType::UInt8: createAttributeImpl<UInt8>(attr, null_value); break;
case AttributeUnderlyingType::UInt16: createAttributeImpl<UInt16>(attr, null_value); break;
case AttributeUnderlyingType::UInt32: createAttributeImpl<UInt32>(attr, null_value); break;
case AttributeUnderlyingType::UInt64: createAttributeImpl<UInt64>(attr, null_value); break;
case AttributeUnderlyingType::Int8: createAttributeImpl<Int8>(attr, null_value); break;
case AttributeUnderlyingType::Int16: createAttributeImpl<Int16>(attr, null_value); break;
case AttributeUnderlyingType::Int32: createAttributeImpl<Int32>(attr, null_value); break;
case AttributeUnderlyingType::Int64: createAttributeImpl<Int64>(attr, null_value); break;
case AttributeUnderlyingType::Float32: createAttributeImpl<Float32>(attr, null_value); break;
case AttributeUnderlyingType::Float64: createAttributeImpl<Float64>(attr, null_value); break;
case AttributeUnderlyingType::String:
{
std::get<String>(attr.null_values) = null_value.get<String>();
std::get<ContainerPtrType<StringRef>>(attr.maps) = std::make_unique<ContainerType<StringRef>>();
attr.string_arena = std::make_unique<Arena>();
break;
}
}
return attr;
}
static std::string createKeyDescription(const DictionaryStructure & dict_struct)
{
std::ostringstream out;
out << '(';
auto first = true;
for (const auto & key : *dict_struct.key)
{
if (!first)
out << ", ";
first = false;
out << key.type->getName();
}
out << ')';
return out.str();
}
void validateKeyTypes(const DataTypes & key_types) const
{
if (key_types.size() != dict_struct.key->size())
throw Exception{
"Key structure does not match, expected " + key_description,
ErrorCodes::TYPE_MISMATCH
};
for (const auto i : ext::range(0, key_types.size()))
{
const auto & expected_type = (*dict_struct.key)[i].type->getName();
const auto & actual_type = key_types[i]->getName();
if (expected_type != actual_type)
throw Exception{
"Key type at position " + std::to_string(i) + " does not match, expected " + expected_type +
", found " + actual_type,
ErrorCodes::TYPE_MISMATCH
};
}
}
template <typename T, typename ValueSetter, typename DefaultGetter>
void getItems(
const attribute_t & attribute, const ConstColumnPlainPtrs & key_columns, ValueSetter && set_value,
DefaultGetter && get_default) const
{
const auto & attr = *std::get<ContainerPtrType<T>>(attribute.maps);
const auto keys_size = key_columns.size();
StringRefs keys(keys_size);
Arena temporary_keys_pool;
const auto rows = key_columns.front()->size();
for (const auto i : ext::range(0, rows))
{
/// copy key data to arena so it is contiguous and return StringRef to it
const auto key = placeKeysInPool(i, key_columns, keys, temporary_keys_pool);
const auto it = attr.find(key);
set_value(i, it != attr.end() ? it->second : get_default(i));
/// free memory allocated for the key
temporary_keys_pool.rollback(key.size);
}
query_count.fetch_add(rows, std::memory_order_relaxed);
}
template <typename T>
bool setAttributeValueImpl(attribute_t & attribute, const StringRef key, const T value)
{
auto & map = *std::get<ContainerPtrType<T>>(attribute.maps);
const auto pair = map.insert({ key, value });
return pair.second;
}
bool setAttributeValue(attribute_t & attribute, const StringRef key, const Field & value)
{
switch (attribute.type)
{
case AttributeUnderlyingType::UInt8: return setAttributeValueImpl<UInt8>(attribute, key, value.get<UInt64>());
case AttributeUnderlyingType::UInt16: return setAttributeValueImpl<UInt16>(attribute, key, value.get<UInt64>());
case AttributeUnderlyingType::UInt32: return setAttributeValueImpl<UInt32>(attribute, key, value.get<UInt64>());
case AttributeUnderlyingType::UInt64: return setAttributeValueImpl<UInt64>(attribute, key, value.get<UInt64>());
case AttributeUnderlyingType::Int8: return setAttributeValueImpl<Int8>(attribute, key, value.get<Int64>());
case AttributeUnderlyingType::Int16: return setAttributeValueImpl<Int16>(attribute, key, value.get<Int64>());
case AttributeUnderlyingType::Int32: return setAttributeValueImpl<Int32>(attribute, key, value.get<Int64>());
case AttributeUnderlyingType::Int64: return setAttributeValueImpl<Int64>(attribute, key, value.get<Int64>());
case AttributeUnderlyingType::Float32: return setAttributeValueImpl<Float32>(attribute, key, value.get<Float64>());
case AttributeUnderlyingType::Float64: return setAttributeValueImpl<Float64>(attribute, key, value.get<Float64>());
case AttributeUnderlyingType::String:
{
auto & map = *std::get<ContainerPtrType<StringRef>>(attribute.maps);
const auto & string = value.get<String>();
const auto string_in_arena = attribute.string_arena->insert(string.data(), string.size());
const auto pair = map.insert({ key, StringRef{string_in_arena, string.size()} });
return pair.second;
}
}
return {};
}
const attribute_t & getAttribute(const std::string & attribute_name) const
{
const auto it = attribute_index_by_name.find(attribute_name);
if (it == std::end(attribute_index_by_name))
throw Exception{
name + ": no such attribute '" + attribute_name + "'",
ErrorCodes::BAD_ARGUMENTS
};
return attributes[it->second];
}
static StringRef placeKeysInPool(
const std::size_t row, const ConstColumnPlainPtrs & key_columns, StringRefs & keys, Arena & pool)
{
const auto keys_size = key_columns.size();
size_t sum_keys_size{};
for (const auto i : ext::range(0, keys_size))
{
keys[i] = key_columns[i]->getDataAtWithTerminatingZero(row);
sum_keys_size += keys[i].size;
}
const auto res = pool.alloc(sum_keys_size);
auto place = res;
for (size_t j = 0; j < keys_size; ++j)
{
memcpy(place, keys[j].data, keys[j].size);
place += keys[j].size;
}
return { res, sum_keys_size };
}
template <typename T>
void has(const attribute_t & attribute, const ConstColumnPlainPtrs & key_columns, PODArray<UInt8> & out) const
{
const auto & attr = *std::get<ContainerPtrType<T>>(attribute.maps);
const auto keys_size = key_columns.size();
StringRefs keys(keys_size);
Arena temporary_keys_pool;
const auto rows = key_columns.front()->size();
for (const auto i : ext::range(0, rows))
{
/// copy key data to arena so it is contiguous and return StringRef to it
const auto key = placeKeysInPool(i, key_columns, keys, temporary_keys_pool);
const auto it = attr.find(key);
out[i] = it != attr.end();
/// free memory allocated for the key
temporary_keys_pool.rollback(key.size);
}
query_count.fetch_add(rows, std::memory_order_relaxed);
}
const std::string name;
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;
const bool require_nonempty;
const std::string key_description;
std::map<std::string, std::size_t> attribute_index_by_name;
std::vector<attribute_t> attributes;
Arena keys_pool;
std::size_t bytes_allocated = 0;
std::size_t element_count = 0;
std::size_t bucket_count = 0;
mutable std::atomic<std::size_t> query_count{0};
std::chrono::time_point<std::chrono::system_clock> creation_time;
std::exception_ptr creation_exception;
};
}

View File

@ -20,14 +20,29 @@ namespace
Block createSampleBlock(const DictionaryStructure & dict_struct)
{
Block block{
ColumnWithTypeAndName{new ColumnUInt64{1}, new DataTypeUInt64, dict_struct.id.name}
};
Block block;
if (dict_struct.id)
block.insert(ColumnWithTypeAndName{
new ColumnUInt64{1}, new DataTypeUInt64, dict_struct.id->name
});
if (dict_struct.key)
{
for (const auto & attribute : *dict_struct.key)
{
auto column = attribute.type->createColumn();
column->insertDefault();
block.insert(ColumnWithTypeAndName{column, attribute.type, attribute.name});
}
}
if (dict_struct.range_min)
for (const auto & attribute : { dict_struct.range_min, dict_struct.range_max })
block.insert(
ColumnWithTypeAndName{new ColumnUInt16{1}, new DataTypeDate, attribute->name});
block.insert(ColumnWithTypeAndName{
new ColumnUInt16{1}, new DataTypeDate, attribute->name
});
for (const auto & attribute : dict_struct.attributes)
{

View File

@ -136,35 +136,83 @@ struct DictionarySpecialAttribute final
/// Name of identifier plus list of attributes
struct DictionaryStructure final
{
DictionarySpecialAttribute id;
std::experimental::optional<DictionarySpecialAttribute> id;
std::experimental::optional<std::vector<DictionaryAttribute>> key;
std::vector<DictionaryAttribute> attributes;
std::experimental::optional<DictionarySpecialAttribute> range_min;
std::experimental::optional<DictionarySpecialAttribute> range_max;
bool has_expressions = false;
DictionaryStructure(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
: id{config, config_prefix + ".id"}
{
if (id.name.empty())
throw Exception{
"No 'id' specified for dictionary",
ErrorCodes::BAD_ARGUMENTS
};
const auto has_id = config.has(config_prefix + ".id");
const auto has_key = config.has(config_prefix + ".key");
if (config.has(config_prefix + ".range_min"))
range_min.emplace(config, config_prefix + ".range_min");
if (has_key && has_id)
throw Exception{"Only one of 'id' and 'key' should be specified", ErrorCodes::BAD_ARGUMENTS};
if (config.has(config_prefix + ".range_max"))
range_max.emplace(config, config_prefix + ".range_max");
if (has_id)
id.emplace(config, config_prefix + ".id");
else if (has_key)
{
key.emplace(getAttributes(config, config_prefix + ".key", false, false));
if (key->empty())
throw Exception{"Empty 'key' supplied", ErrorCodes::BAD_ARGUMENTS};
}
else
throw Exception{"Dictionary structure should specify either 'id' or 'key'", ErrorCodes::BAD_ARGUMENTS};
if (!id.expression.empty() ||
(range_min && !range_min->expression.empty()) || (range_max && !range_max->expression.empty()))
has_expressions = true;
if (id)
{
if (id->name.empty())
throw Exception{"'id' cannot be empty", ErrorCodes::BAD_ARGUMENTS};
if (config.has(config_prefix + ".range_min"))
range_min.emplace(config, config_prefix + ".range_min");
if (config.has(config_prefix + ".range_max"))
range_max.emplace(config, config_prefix + ".range_max");
if (!id->expression.empty() ||
(range_min && !range_min->expression.empty()) ||
(range_max && !range_max->expression.empty()))
has_expressions = true;
}
attributes = getAttributes(config, config_prefix);
if (attributes.empty())
throw Exception{"Dictionary has no attributes defined", ErrorCodes::BAD_ARGUMENTS};
}
bool isKeySizeFixed() const
{
if (!key)
return true;
for (const auto key_i : * key)
if (key_i.underlying_type == AttributeUnderlyingType::String)
return false;
return true;
}
std::size_t getKeySize() const
{
return std::accumulate(std::begin(*key), std::end(*key), std::size_t{},
[] (const auto running_size, const auto & key_i) {return running_size + key_i.type->getSizeOfField(); });
}
private:
std::vector<DictionaryAttribute> getAttributes(
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
const bool hierarchy_allowed = true, const bool allow_null_values = true)
{
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_prefix, keys);
auto has_hierarchy = false;
std::vector<DictionaryAttribute> attributes;
for (const auto & key : keys)
{
if (0 != strncmp(key.data(), "attribute", strlen("attribute")))
@ -181,19 +229,22 @@ struct DictionaryStructure final
if (!expression.empty())
has_expressions = true;
const auto null_value_string = config.getString(prefix + "null_value");
Field null_value;
try
if (allow_null_values)
{
ReadBufferFromString null_value_buffer{null_value_string};
type->deserializeText(null_value, null_value_buffer);
}
catch (const std::exception & e)
{
throw Exception{
std::string{"Error parsing null_value: "} + e.what(),
ErrorCodes::BAD_ARGUMENTS
};
const auto null_value_string = config.getString(prefix + "null_value");
try
{
ReadBufferFromString null_value_buffer{null_value_string};
type->deserializeText(null_value, null_value_buffer);
}
catch (const std::exception & e)
{
throw Exception{
std::string{"Error parsing null_value: "} + e.what(),
ErrorCodes::BAD_ARGUMENTS
};
}
}
const auto hierarchical = config.getBool(prefix + "hierarchical", false);
@ -204,6 +255,12 @@ struct DictionaryStructure final
ErrorCodes::BAD_ARGUMENTS
};
if (has_hierarchy && !hierarchy_allowed)
throw Exception{
"Hierarchy not allowed in '" + prefix,
ErrorCodes::BAD_ARGUMENTS
};
if (has_hierarchy && hierarchical)
throw Exception{
"Only one hierarchical attribute supported",
@ -217,11 +274,7 @@ struct DictionaryStructure final
});
}
if (attributes.empty())
throw Exception{
"Dictionary has no attributes defined",
ErrorCodes::BAD_ARGUMENTS
};
return attributes;
}
};

View File

@ -41,10 +41,13 @@ public:
BlockInputStreamPtr loadIds(const std::vector<std::uint64_t> & ids) override
{
throw Exception{
"Method unsupported",
ErrorCodes::NOT_IMPLEMENTED
};
throw Exception{"Method unsupported", ErrorCodes::NOT_IMPLEMENTED};
}
BlockInputStreamPtr loadKeys(
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows) override
{
throw Exception{"Method unsupported", ErrorCodes::NOT_IMPLEMENTED};
}
bool isModified() const override { return getLastModification() > last_modification; }

View File

@ -6,10 +6,12 @@
#include <DB/Columns/ColumnString.h>
#include <DB/Common/Arena.h>
#include <ext/range.hpp>
#include <ext/size.hpp>
#include <atomic>
#include <vector>
#include <tuple>
namespace DB
{
@ -84,7 +86,11 @@ public:
void toParent(const PODArray<id_t> & ids, PODArray<id_t> & out) const override
{
getItems<UInt64>(*hierarchical_attribute, ids, out);
const auto null_value = std::get<UInt64>(hierarchical_attribute->null_values);
getItems<UInt64>(*hierarchical_attribute, ids,
[&] (const std::size_t row, const UInt64 value) { out[row] = value; },
[&] (const std::size_t) { return null_value; });
}
#define DECLARE_MULTIPLE_GETTER(TYPE)\
@ -97,7 +103,11 @@ public:
ErrorCodes::TYPE_MISMATCH\
};\
\
getItems<TYPE>(attribute, ids, out);\
const auto null_value = std::get<TYPE>(attribute.null_values);\
\
getItems<TYPE>(attribute, ids,\
[&] (const std::size_t row, const auto value) { out[row] = value; },\
[&] (const std::size_t) { return null_value; });\
}
DECLARE_MULTIPLE_GETTER(UInt8)
DECLARE_MULTIPLE_GETTER(UInt16)
@ -119,38 +129,93 @@ public:
ErrorCodes::TYPE_MISMATCH
};
const auto & attr = *std::get<std::unique_ptr<PODArray<StringRef>>>(attribute.arrays);
const auto & null_value = std::get<String>(attribute.null_values);
const auto & null_value = StringRef{std::get<String>(attribute.null_values)};
for (const auto i : ext::range(0, ids.size()))
getItems<StringRef>(attribute, ids,
[&] (const std::size_t row, const StringRef value) { out->insertData(value.data, value.size); },
[&] (const std::size_t) { return null_value; });
}
#define DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const PODArray<id_t> & ids, const PODArray<TYPE> & def,\
PODArray<TYPE> & out) const override\
{\
const auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeUnderlyingType::TYPE)\
throw Exception{\
name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\
ErrorCodes::TYPE_MISMATCH\
};\
\
getItems<TYPE>(attribute, ids,\
[&] (const std::size_t row, const auto value) { out[row] = value; },\
[&] (const std::size_t row) { return def[row]; });\
}
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(UInt8)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(UInt16)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(UInt32)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(UInt64)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Int8)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Int16)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Int32)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Int64)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Float32)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Float64)
#undef DECLARE_MULTIPLE_GETTER_WITH_DEFAULT
void getString(
const std::string & attribute_name, const PODArray<id_t> & ids, const ColumnString * const def,
ColumnString * const out) const override
{
const auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeUnderlyingType::String)
throw Exception{
name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH
};
getItems<StringRef>(attribute, ids,
[&] (const std::size_t row, const StringRef value) { out->insertData(value.data, value.size); },
[&] (const std::size_t row) { return def->getDataAt(row); });
}
void has(const PODArray<id_t> & ids, PODArray<UInt8> & out) const override
{
const auto & attribute = attributes.front();
switch (attribute.type)
{
const auto id = ids[i];
const auto string_ref = id < attr.size() ? attr[id] : StringRef{null_value};
out->insertData(string_ref.data, string_ref.size);
case AttributeUnderlyingType::UInt8: has<UInt8>(attribute, ids, out); break;
case AttributeUnderlyingType::UInt16: has<UInt16>(attribute, ids, out); break;
case AttributeUnderlyingType::UInt32: has<UInt32>(attribute, ids, out); break;
case AttributeUnderlyingType::UInt64: has<UInt64>(attribute, ids, out); break;
case AttributeUnderlyingType::Int8: has<Int8>(attribute, ids, out); break;
case AttributeUnderlyingType::Int16: has<Int16>(attribute, ids, out); break;
case AttributeUnderlyingType::Int32: has<Int32>(attribute, ids, out); break;
case AttributeUnderlyingType::Int64: has<Int64>(attribute, ids, out); break;
case AttributeUnderlyingType::Float32: has<Float32>(attribute, ids, out); break;
case AttributeUnderlyingType::Float64: has<Float64>(attribute, ids, out); break;
case AttributeUnderlyingType::String: has<String>(attribute, ids, out); break;
}
query_count.fetch_add(ids.size(), std::memory_order_relaxed);
}
private:
template <typename Value> using ContainerType = PODArray<Value>;
template <typename Value> using ContainerPtrType = std::unique_ptr<ContainerType<Value>>;
struct attribute_t final
{
AttributeUnderlyingType type;
std::tuple<UInt8, UInt16, UInt32, UInt64,
std::tuple<
UInt8, UInt16, UInt32, UInt64,
Int8, Int16, Int32, Int64,
Float32, Float64,
String> null_values;
std::tuple<std::unique_ptr<PODArray<UInt8>>,
std::unique_ptr<PODArray<UInt16>>,
std::unique_ptr<PODArray<UInt32>>,
std::unique_ptr<PODArray<UInt64>>,
std::unique_ptr<PODArray<Int8>>,
std::unique_ptr<PODArray<Int16>>,
std::unique_ptr<PODArray<Int32>>,
std::unique_ptr<PODArray<Int64>>,
std::unique_ptr<PODArray<Float32>>,
std::unique_ptr<PODArray<Float64>>,
std::unique_ptr<PODArray<StringRef>>> arrays;
std::tuple<
ContainerPtrType<UInt8>, ContainerPtrType<UInt16>, ContainerPtrType<UInt32>, ContainerPtrType<UInt64>,
ContainerPtrType<Int8>, ContainerPtrType<Int16>, ContainerPtrType<Int32>, ContainerPtrType<Int64>,
ContainerPtrType<Float32>, ContainerPtrType<Float64>,
ContainerPtrType<StringRef>> arrays;
std::unique_ptr<Arena> string_arena;
};
@ -210,7 +275,7 @@ private:
template <typename T>
void addAttributeSize(const attribute_t & attribute)
{
const auto & array_ref = std::get<std::unique_ptr<PODArray<T>>>(attribute.arrays);
const auto & array_ref = std::get<ContainerPtrType<T>>(attribute.arrays);
bytes_allocated += sizeof(PODArray<T>) + array_ref->storage_size();
bucket_count = array_ref->capacity();
}
@ -249,8 +314,8 @@ private:
{
const auto & null_value_ref = std::get<T>(attribute.null_values) =
null_value.get<typename NearestFieldType<T>::Type>();
std::get<std::unique_ptr<PODArray<T>>>(attribute.arrays) =
std::make_unique<PODArray<T>>(initial_array_size, null_value_ref);
std::get<ContainerPtrType<T>>(attribute.arrays) =
std::make_unique<ContainerType<T>>(initial_array_size, null_value_ref);
}
attribute_t createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value)
@ -272,8 +337,8 @@ private:
case AttributeUnderlyingType::String:
{
const auto & null_value_ref = std::get<String>(attr.null_values) = null_value.get<String>();
std::get<std::unique_ptr<PODArray<StringRef>>>(attr.arrays) =
std::make_unique<PODArray<StringRef>>(initial_array_size, null_value_ref);
std::get<ContainerPtrType<StringRef>>(attr.arrays) =
std::make_unique<ContainerType<StringRef>>(initial_array_size, StringRef{null_value_ref});
attr.string_arena = std::make_unique<Arena>();
break;
}
@ -282,25 +347,27 @@ private:
return attr;
}
template <typename T>
void getItems(const attribute_t & attribute, const PODArray<id_t> & ids, PODArray<T> & out) const
template <typename T, typename ValueSetter, typename DefaultGetter>
void getItems(
const attribute_t & attribute, const PODArray<id_t> & ids, ValueSetter && set_value,
DefaultGetter && get_default) const
{
const auto & attr = *std::get<std::unique_ptr<PODArray<T>>>(attribute.arrays);
const auto null_value = std::get<T>(attribute.null_values);
const auto & attr = *std::get<ContainerPtrType<T>>(attribute.arrays);
const auto rows = ext::size(ids);
for (const auto i : ext::range(0, ids.size()))
for (const auto i : ext::range(0, rows))
{
const auto id = ids[i];
out[i] = id < attr.size() ? attr[id] : null_value;
set_value(i, id < attr.size() ? attr[id] : get_default(i));
}
query_count.fetch_add(ids.size(), std::memory_order_relaxed);
query_count.fetch_add(rows, std::memory_order_relaxed);
}
template <typename T>
void setAttributeValueImpl(attribute_t & attribute, const id_t id, const T value)
{
auto & array = *std::get<std::unique_ptr<PODArray<T>>>(attribute.arrays);
auto & array = *std::get<ContainerPtrType<T>>(attribute.arrays);
if (id >= array.size())
array.resize_fill(id + 1, std::get<T>(attribute.null_values));
array[id] = value;
@ -328,9 +395,9 @@ private:
case AttributeUnderlyingType::Float64: setAttributeValueImpl<Float64>(attribute, id, value.get<Float64>()); break;
case AttributeUnderlyingType::String:
{
auto & array = *std::get<std::unique_ptr<PODArray<StringRef>>>(attribute.arrays);
auto & array = *std::get<ContainerPtrType<StringRef>>(attribute.arrays);
if (id >= array.size())
array.resize_fill(id + 1, std::get<String>(attribute.null_values));
array.resize_fill(id + 1, StringRef{std::get<String>(attribute.null_values)});
const auto & string = value.get<String>();
const auto string_in_arena = attribute.string_arena->insert(string.data(), string.size());
array[id] = StringRef{string_in_arena, string.size()};
@ -351,6 +418,23 @@ private:
return attributes[it->second];
}
template <typename T>
void has(const attribute_t & attribute, const PODArray<id_t> & ids, PODArray<UInt8> & out) const
{
using stored_type = std::conditional_t<std::is_same<T, String>::value, StringRef, T>;
const auto & attr = *std::get<ContainerPtrType<stored_type>>(attribute.arrays);
const auto & null_value = std::get<T>(attribute.null_values);
const auto rows = ext::size(ids);
for (const auto i : ext::range(0, rows))
{
const auto id = ids[i];
out[i] = id < ext::size(attr) && attr[id] != null_value;
}
query_count.fetch_add(rows, std::memory_order_relaxed);
}
const std::string name;
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;

View File

@ -19,8 +19,7 @@ class HashedDictionary final : public IDictionary
public:
HashedDictionary(const std::string & name, const DictionaryStructure & dict_struct,
DictionarySourcePtr source_ptr, const DictionaryLifetime dict_lifetime, bool require_nonempty)
: name{name}, dict_struct(dict_struct),
source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime),
: name{name}, dict_struct(dict_struct), source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime),
require_nonempty(require_nonempty)
{
createAttributes();
@ -82,7 +81,11 @@ public:
void toParent(const PODArray<id_t> & ids, PODArray<id_t> & out) const override
{
getItems<UInt64>(*hierarchical_attribute, ids, out);
const auto null_value = std::get<UInt64>(hierarchical_attribute->null_values);
getItems<UInt64>(*hierarchical_attribute, ids,
[&] (const std::size_t row, const UInt64 value) { out[row] = value; },
[&] (const std::size_t) { return null_value; });
}
#define DECLARE_MULTIPLE_GETTER(TYPE)\
@ -95,7 +98,11 @@ public:
ErrorCodes::TYPE_MISMATCH\
};\
\
getItems<TYPE>(attribute, ids, out);\
const auto null_value = std::get<TYPE>(attribute.null_values);\
\
getItems<TYPE>(attribute, ids,\
[&] (const std::size_t row, const auto value) { out[row] = value; },\
[&] (const std::size_t) { return null_value; });\
}
DECLARE_MULTIPLE_GETTER(UInt8)
DECLARE_MULTIPLE_GETTER(UInt16)
@ -117,38 +124,93 @@ public:
ErrorCodes::TYPE_MISMATCH
};
const auto & attr = *std::get<std::unique_ptr<HashMap<UInt64, StringRef>>>(attribute.maps);
const auto & null_value = std::get<String>(attribute.null_values);
const auto & null_value = StringRef{std::get<String>(attribute.null_values)};
for (const auto i : ext::range(0, ids.size()))
getItems<StringRef>(attribute, ids,
[&] (const std::size_t row, const StringRef value) { out->insertData(value.data, value.size); },
[&] (const std::size_t) { return null_value; });
}
#define DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const PODArray<id_t> & ids, const PODArray<TYPE> & def,\
PODArray<TYPE> & out) const override\
{\
const auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeUnderlyingType::TYPE)\
throw Exception{\
name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\
ErrorCodes::TYPE_MISMATCH\
};\
\
getItems<TYPE>(attribute, ids,\
[&] (const std::size_t row, const auto value) { out[row] = value; },\
[&] (const std::size_t row) { return def[row]; });\
}
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(UInt8)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(UInt16)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(UInt32)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(UInt64)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Int8)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Int16)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Int32)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Int64)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Float32)
DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(Float64)
#undef DECLARE_MULTIPLE_GETTER_WITH_DEFAULT
void getString(
const std::string & attribute_name, const PODArray<id_t> & ids, const ColumnString * const def,
ColumnString * const out) const override
{
const auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeUnderlyingType::String)
throw Exception{
name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),
ErrorCodes::TYPE_MISMATCH
};
getItems<StringRef>(attribute, ids,
[&] (const std::size_t row, const StringRef value) { out->insertData(value.data, value.size); },
[&] (const std::size_t row) { return def->getDataAt(row); });
}
void has(const PODArray<id_t> & ids, PODArray<UInt8> & out) const override
{
const auto & attribute = attributes.front();
switch (attribute.type)
{
const auto it = attr.find(ids[i]);
const auto string_ref = it != attr.end() ? it->second : StringRef{null_value};
out->insertData(string_ref.data, string_ref.size);
case AttributeUnderlyingType::UInt8: has<UInt8>(attribute, ids, out); break;
case AttributeUnderlyingType::UInt16: has<UInt16>(attribute, ids, out); break;
case AttributeUnderlyingType::UInt32: has<UInt32>(attribute, ids, out); break;
case AttributeUnderlyingType::UInt64: has<UInt64>(attribute, ids, out); break;
case AttributeUnderlyingType::Int8: has<Int8>(attribute, ids, out); break;
case AttributeUnderlyingType::Int16: has<Int16>(attribute, ids, out); break;
case AttributeUnderlyingType::Int32: has<Int32>(attribute, ids, out); break;
case AttributeUnderlyingType::Int64: has<Int64>(attribute, ids, out); break;
case AttributeUnderlyingType::Float32: has<Float32>(attribute, ids, out); break;
case AttributeUnderlyingType::Float64: has<Float64>(attribute, ids, out); break;
case AttributeUnderlyingType::String: has<StringRef>(attribute, ids, out); break;
}
query_count.fetch_add(ids.size(), std::memory_order_relaxed);
}
private:
template <typename Value> using CollectionType = HashMap<UInt64, Value>;
template <typename Value> using CollectionPtrType = std::unique_ptr<CollectionType<Value>>;
struct attribute_t final
{
AttributeUnderlyingType type;
std::tuple<UInt8, UInt16, UInt32, UInt64,
std::tuple<
UInt8, UInt16, UInt32, UInt64,
Int8, Int16, Int32, Int64,
Float32, Float64,
String> null_values;
std::tuple<std::unique_ptr<HashMap<UInt64, UInt8>>,
std::unique_ptr<HashMap<UInt64, UInt16>>,
std::unique_ptr<HashMap<UInt64, UInt32>>,
std::unique_ptr<HashMap<UInt64, UInt64>>,
std::unique_ptr<HashMap<UInt64, Int8>>,
std::unique_ptr<HashMap<UInt64, Int16>>,
std::unique_ptr<HashMap<UInt64, Int32>>,
std::unique_ptr<HashMap<UInt64, Int64>>,
std::unique_ptr<HashMap<UInt64, Float32>>,
std::unique_ptr<HashMap<UInt64, Float64>>,
std::unique_ptr<HashMap<UInt64, StringRef>>> maps;
std::tuple<
CollectionPtrType<UInt8>, CollectionPtrType<UInt16>, CollectionPtrType<UInt32>, CollectionPtrType<UInt64>,
CollectionPtrType<Int8>, CollectionPtrType<Int16>, CollectionPtrType<Int32>, CollectionPtrType<Int64>,
CollectionPtrType<Float32>, CollectionPtrType<Float64>,
CollectionPtrType<StringRef>> maps;
std::unique_ptr<Arena> string_arena;
};
@ -208,8 +270,8 @@ private:
template <typename T>
void addAttributeSize(const attribute_t & attribute)
{
const auto & map_ref = std::get<std::unique_ptr<HashMap<UInt64, T>>>(attribute.maps);
bytes_allocated += sizeof(HashMap<UInt64, T>) + map_ref->getBufferSizeInBytes();
const auto & map_ref = std::get<CollectionPtrType<T>>(attribute.maps);
bytes_allocated += sizeof(CollectionType<T>) + map_ref->getBufferSizeInBytes();
bucket_count = map_ref->getBufferSizeInCells();
}
@ -246,7 +308,7 @@ private:
void createAttributeImpl(attribute_t & attribute, const Field & null_value)
{
std::get<T>(attribute.null_values) = null_value.get<typename NearestFieldType<T>::Type>();
std::get<std::unique_ptr<HashMap<UInt64, T>>>(attribute.maps) = std::make_unique<HashMap<UInt64, T>>();
std::get<CollectionPtrType<T>>(attribute.maps) = std::make_unique<CollectionType<T>>();
}
attribute_t createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value)
@ -268,8 +330,7 @@ private:
case AttributeUnderlyingType::String:
{
std::get<String>(attr.null_values) = null_value.get<String>();
std::get<std::unique_ptr<HashMap<UInt64, StringRef>>>(attr.maps) =
std::make_unique<HashMap<UInt64, StringRef>>();
std::get<CollectionPtrType<StringRef>>(attr.maps) = std::make_unique<CollectionType<StringRef>>();
attr.string_arena = std::make_unique<Arena>();
break;
}
@ -278,25 +339,27 @@ private:
return attr;
}
template <typename T>
void getItems(const attribute_t & attribute, const PODArray<id_t> & ids, PODArray<T> & out) const
template <typename T, typename ValueSetter, typename DefaultGetter>
void getItems(
const attribute_t & attribute, const PODArray<id_t> & ids, ValueSetter && set_value,
DefaultGetter && get_default) const
{
const auto & attr = *std::get<std::unique_ptr<HashMap<UInt64, T>>>(attribute.maps);
const auto null_value = std::get<T>(attribute.null_values);
const auto & attr = *std::get<CollectionPtrType<T>>(attribute.maps);
const auto rows = ext::size(ids);
for (const auto i : ext::range(0, ids.size()))
for (const auto i : ext::range(0, rows))
{
const auto it = attr.find(ids[i]);
out[i] = it != attr.end() ? it->second : null_value;
set_value(i, it != attr.end() ? it->second : get_default(i));
}
query_count.fetch_add(ids.size(), std::memory_order_relaxed);
query_count.fetch_add(rows, std::memory_order_relaxed);
}
template <typename T>
void setAttributeValueImpl(attribute_t & attribute, const id_t id, const T value)
{
auto & map = *std::get<std::unique_ptr<HashMap<UInt64, T>>>(attribute.maps);
auto & map = *std::get<CollectionPtrType<T>>(attribute.maps);
map.insert({ id, value });
}
@ -316,7 +379,7 @@ private:
case AttributeUnderlyingType::Float64: setAttributeValueImpl<Float64>(attribute, id, value.get<Float64>()); break;
case AttributeUnderlyingType::String:
{
auto & map = *std::get<std::unique_ptr<HashMap<UInt64, StringRef>>>(attribute.maps);
auto & map = *std::get<CollectionPtrType<StringRef>>(attribute.maps);
const auto & string = value.get<String>();
const auto string_in_arena = attribute.string_arena->insert(string.data(), string.size());
map.insert({ id, StringRef{string_in_arena, string.size()} });
@ -337,6 +400,18 @@ private:
return attributes[it->second];
}
template <typename T>
void has(const attribute_t & attribute, const PODArray<id_t> & ids, PODArray<UInt8> & out) const
{
const auto & attr = *std::get<CollectionPtrType<T>>(attribute.maps);
const auto rows = ext::size(ids);
for (const auto i : ext::range(0, rows))
out[i] = attr.find(ids[i]) != std::end(attr);
query_count.fetch_add(rows, std::memory_order_relaxed);
}
const std::string name;
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;

View File

@ -92,6 +92,21 @@ struct IDictionary : IDictionaryBase
virtual void getFloat32(const std::string & attr_name, const PODArray<id_t> & ids, PODArray<Float32> & out) const = 0;
virtual void getFloat64(const std::string & attr_name, const PODArray<id_t> & ids, PODArray<Float64> & out) const = 0;
virtual void getString(const std::string & attr_name, const PODArray<id_t> & ids, ColumnString * out) const = 0;
/// return mapped values for a collection of identifiers with explicit default values
virtual void getUInt8(const std::string & attr_name, const PODArray<id_t> & ids, const PODArray<UInt8> & def, PODArray<UInt8> & out) const = 0;
virtual void getUInt16(const std::string & attr_name, const PODArray<id_t> & ids, const PODArray<UInt16> & def, PODArray<UInt16> & out) const = 0;
virtual void getUInt32(const std::string & attr_name, const PODArray<id_t> & ids, const PODArray<UInt32> & def, PODArray<UInt32> & out) const = 0;
virtual void getUInt64(const std::string & attr_name, const PODArray<id_t> & ids, const PODArray<UInt64> & def, PODArray<UInt64> & out) const = 0;
virtual void getInt8(const std::string & attr_name, const PODArray<id_t> & ids, const PODArray<Int8> & def, PODArray<Int8> & out) const = 0;
virtual void getInt16(const std::string & attr_name, const PODArray<id_t> & ids, const PODArray<Int16> & def, PODArray<Int16> & out) const = 0;
virtual void getInt32(const std::string & attr_name, const PODArray<id_t> & ids, const PODArray<Int32> & def, PODArray<Int32> & out) const = 0;
virtual void getInt64(const std::string & attr_name, const PODArray<id_t> & ids, const PODArray<Int64> & def, PODArray<Int64> & out) const = 0;
virtual void getFloat32(const std::string & attr_name, const PODArray<id_t> & ids, const PODArray<Float32> & def, PODArray<Float32> & out) const = 0;
virtual void getFloat64(const std::string & attr_name, const PODArray<id_t> & ids, const PODArray<Float64> & def, PODArray<Float64> & out) const = 0;
virtual void getString(const std::string & attr_name, const PODArray<id_t> & ids, const ColumnString * def, ColumnString * out) const = 0;
virtual void has(const PODArray<id_t> & ids, PODArray<UInt8> & out) const = 0;
};
}

View File

@ -27,6 +27,11 @@ public:
/// returns an input stream with the data for a collection of identifiers
virtual BlockInputStreamPtr loadIds(const std::vector<std::uint64_t> & ids) = 0;
/** returns an input stream with the data for a collection of composite keys.
* `requested_rows` contains indices of all rows containing unique keys. */
virtual BlockInputStreamPtr loadKeys(
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows) = 0;
/// indicates whether the source has been modified since last load* operation
virtual bool isModified() const = 0;

View File

@ -4,6 +4,7 @@
#include <DB/Dictionaries/MongoDBBlockInputStream.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <mongo/client/dbclient.h>
#include <ext/collection_cast.hpp>
namespace DB
@ -53,8 +54,8 @@ class MongoDBDictionarySource final : public IDictionarySource
if (!mongo_init_status.isOK())
throw DB::Exception{
"mongo::client::initialize() failed: " + mongo_init_status.toString(),
ErrorCodes::MONGODB_INIT_FAILED
"mongo::client::initialize() failed: " + mongo_init_status.toString(),
ErrorCodes::MONGODB_INIT_FAILED
};
LOG_TRACE(&Logger::get("MongoDBDictionarySource"), "mongo::client::initialize() ok");
@ -97,9 +98,12 @@ public:
BlockInputStreamPtr loadIds(const std::vector<std::uint64_t> & ids) override
{
if (dict_struct.key)
throw Exception{"Complex key not supported", ErrorCodes::UNSUPPORTED_METHOD};
/// mongo::BSONObj has shitty design and does not use fixed width integral types
const std::vector<long long int> iids{std::begin(ids), std::end(ids)};
const auto ids_enumeration = BSON(dict_struct.id.name << BSON("$in" << iids));
const auto ids_enumeration = BSON(
dict_struct.id->name << BSON("$in" << ext::collection_cast<std::vector<long long int>>(ids)));
return new MongoDBBlockInputStream{
connection.query(db + '.' + collection, ids_enumeration, 0, 0, &fields_to_query),
@ -107,6 +111,12 @@ public:
};
}
BlockInputStreamPtr loadKeys(
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows) override
{
throw Exception{"Method unsupported", ErrorCodes::NOT_IMPLEMENTED};
}
/// @todo: for MongoDB, modification date can somehow be determined from the `_id` object field
bool isModified() const override { return false; }

View File

@ -58,6 +58,15 @@ public:
return new MySQLBlockInputStream{pool.Get(), query, sample_block, max_block_size};
}
BlockInputStreamPtr loadKeys(
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows) override
{
/// Здесь не логгируем и не обновляем время модификации, так как запрос может быть большим, и часто задаваться.
const auto query = composeLoadKeysQuery(key_columns, requested_rows);
return new MySQLBlockInputStream{pool.Get(), query, sample_block, max_block_size};
}
bool isModified() const override
{
if (dont_check_update_time)
@ -156,35 +165,57 @@ private:
WriteBufferFromString out{query};
writeString("SELECT ", out);
if (!dict_struct.id.expression.empty())
if (dict_struct.id)
{
writeParenthesisedString(dict_struct.id.expression, out);
writeString(" AS ", out);
if (!dict_struct.id->expression.empty())
{
writeParenthesisedString(dict_struct.id->expression, out);
writeString(" AS ", out);
}
writeProbablyBackQuotedString(dict_struct.id->name, out);
if (dict_struct.range_min && dict_struct.range_max)
{
writeString(", ", out);
if (!dict_struct.range_min->expression.empty())
{
writeParenthesisedString(dict_struct.range_min->expression, out);
writeString(" AS ", out);
}
writeProbablyBackQuotedString(dict_struct.range_min->name, out);
writeString(", ", out);
if (!dict_struct.range_max->expression.empty())
{
writeParenthesisedString(dict_struct.range_max->expression, out);
writeString(" AS ", out);
}
writeProbablyBackQuotedString(dict_struct.range_max->name, out);
}
}
writeProbablyBackQuotedString(dict_struct.id.name, out);
if (dict_struct.range_min && dict_struct.range_max)
else if (dict_struct.key)
{
writeString(", ", out);
if (!dict_struct.range_min->expression.empty())
auto first = true;
for (const auto & key : *dict_struct.key)
{
writeParenthesisedString(dict_struct.range_min->expression, out);
writeString(" AS ", out);
if (!first)
writeString(", ", out);
first = false;
if (!key.expression.empty())
{
writeParenthesisedString(key.expression, out);
writeString(" AS ", out);
}
writeProbablyBackQuotedString(key.name, out);
}
writeProbablyBackQuotedString(dict_struct.range_min->name, out);
writeString(", ", out);
if (!dict_struct.range_max->expression.empty())
{
writeParenthesisedString(dict_struct.range_max->expression, out);
writeString(" AS ", out);
}
writeProbablyBackQuotedString(dict_struct.range_max->name, out);
}
for (const auto & attr : dict_struct.attributes)
@ -222,19 +253,22 @@ private:
std::string composeLoadIdsQuery(const std::vector<std::uint64_t> & ids)
{
if (!dict_struct.id)
throw Exception{"Simple key required for method", ErrorCodes::UNSUPPORTED_METHOD};
std::string query;
{
WriteBufferFromString out{query};
writeString("SELECT ", out);
if (!dict_struct.id.expression.empty())
if (!dict_struct.id->expression.empty())
{
writeParenthesisedString(dict_struct.id.expression, out);
writeParenthesisedString(dict_struct.id->expression, out);
writeString(" AS ", out);
}
writeProbablyBackQuotedString(dict_struct.id.name, out);
writeProbablyBackQuotedString(dict_struct.id->name, out);
for (const auto & attr : dict_struct.attributes)
{
@ -265,7 +299,7 @@ private:
writeString(" AND ", out);
}
writeProbablyBackQuotedString(dict_struct.id.name, out);
writeProbablyBackQuotedString(dict_struct.id->name, out);
writeString(" IN (", out);
auto first = true;
@ -284,6 +318,92 @@ private:
return query;
}
std::string composeLoadKeysQuery(
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows)
{
if (!dict_struct.key)
throw Exception{"Composite key required for method", ErrorCodes::UNSUPPORTED_METHOD};
std::string query;
{
WriteBufferFromString out{query};
writeString("SELECT ", out);
auto first = true;
for (const auto & key_or_attribute : boost::join(*dict_struct.key, dict_struct.attributes))
{
if (!first)
writeString(", ", out);
first = false;
if (!key_or_attribute.expression.empty())
{
writeParenthesisedString(key_or_attribute.expression, out);
writeString(" AS ", out);
}
writeProbablyBackQuotedString(key_or_attribute.name, out);
}
writeString(" FROM ", out);
if (!db.empty())
{
writeProbablyBackQuotedString(db, out);
writeChar('.', out);
}
writeProbablyBackQuotedString(table, out);
writeString(" WHERE ", out);
if (!where.empty())
{
writeString(where, out);
writeString(" AND ", out);
}
first = true;
for (const auto row : requested_rows)
{
if (!first)
writeString(" OR ", out);
first = false;
composeKeyCondition(key_columns, row, out);
}
writeString(";", out);
}
return query;
}
void composeKeyCondition(const ConstColumnPlainPtrs & key_columns, const std::size_t row, WriteBuffer & out) const
{
writeString("(", out);
const auto keys_size = key_columns.size();
auto first = true;
for (const auto i : ext::range(0, keys_size))
{
if (!first)
writeString(" AND ", out);
first = false;
const auto & key_description = (*dict_struct.key)[i];
const auto & value = (*key_columns[i])[row];
/// key_i=value_i
writeString(key_description.name, out);
writeString("=", out);
key_description.type->serializeTextQuoted(value, out);
}
writeString(")", out);
}
const DictionaryStructure dict_struct;
const std::string db;
const std::string table;

View File

@ -117,7 +117,7 @@ public:
const auto val_it = std::find_if(std::begin(ranges_and_values), std::end(ranges_and_values),
[date] (const value_t<StringRef> & v) { return v.range.contains(date); });
const auto string_ref = val_it != std::end(ranges_and_values) ? val_it->value : null_value;
const auto string_ref = val_it != std::end(ranges_and_values) ? val_it->value : StringRef{null_value};
out->insertData(string_ref.data, string_ref.size);
}
else

View File

@ -5,6 +5,7 @@
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeDate.h>
#include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/DataTypes/DataTypeTuple.h>
#include <DB/Columns/ColumnVector.h>
#include <DB/Columns/ColumnArray.h>
@ -18,6 +19,8 @@
#include <DB/Dictionaries/FlatDictionary.h>
#include <DB/Dictionaries/HashedDictionary.h>
#include <DB/Dictionaries/CacheDictionary.h>
#include <DB/Dictionaries/ComplexKeyHashedDictionary.h>
#include <DB/Dictionaries/ComplexKeyCacheDictionary.h>
#include <DB/Dictionaries/RangeHashedDictionary.h>
#include <ext/range.hpp>
@ -739,6 +742,156 @@ public:
};
class FunctionDictHas final : public IFunction
{
public:
static constexpr auto name = "dictHas";
static IFunction * create(const Context & context)
{
return new FunctionDictHas{context.getExternalDictionaries()};
}
FunctionDictHas(const ExternalDictionaries & dictionaries) : dictionaries(dictionaries) {}
String getName() const override { return name; }
private:
DataTypePtr getReturnType(const DataTypes & arguments) const override
{
if (arguments.size() != 2)
throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
if (!typeid_cast<const DataTypeString *>(arguments[0].get()))
throw Exception{
"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
if (!typeid_cast<const DataTypeUInt64 *>(arguments[1].get()) &&
!typeid_cast<const DataTypeTuple *>(arguments[1].get()))
throw Exception{
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", must be UInt64 or tuple(...).",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
return new DataTypeUInt8;
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const auto dict_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{
"First argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
auto dict = dictionaries.getDictionary(dict_name_col->getData());
const auto dict_ptr = dict.get();
if (!executeDispatchSimple<FlatDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchSimple<HashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchSimple<CacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr))
throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(),
ErrorCodes::UNKNOWN_TYPE
};
}
template <typename DictionaryType>
bool executeDispatchSimple(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * const dictionary)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
if (!dict)
return false;
if (arguments.size() != 2)
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 2 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
const auto id_col_untyped = block.getByPosition(arguments[1]).column.get();
if (const auto id_col = typeid_cast<const ColumnVector<UInt64> *>(id_col_untyped))
{
const auto & ids = id_col->getData();
const auto out = new ColumnVector<UInt8>(ext::size(ids));
block.getByPosition(result).column = out;
dict->has(ids, out->getData());
}
else if (const auto id_col = typeid_cast<const ColumnConst<UInt64> *>(id_col_untyped))
{
const PODArray<UInt64> ids(1, id_col->getData());
PODArray<UInt8> out(1);
dict->has(ids, out);
block.getByPosition(result).column = new ColumnConst<UInt8>{id_col->size(), out.front()};
}
else
throw Exception{
"Second argument of function " + getName() + " must be UInt64",
ErrorCodes::ILLEGAL_COLUMN
};
return true;
}
template <typename DictionaryType>
bool executeDispatchComplex(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * const dictionary)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
if (!dict)
return false;
if (arguments.size() != 2)
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 2 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
const auto key_col_with_type = block.getByPosition(arguments[1]);
if (const auto key_col = typeid_cast<const ColumnTuple *>(key_col_with_type.column.get()))
{
const auto key_columns = ext::map<ConstColumnPlainPtrs>(key_col->getColumns(), [] (const ColumnPtr & ptr) {
return ptr.get();
});
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
const auto out = new ColumnVector<UInt8>(key_col->size());
block.getByPosition(result).column = out;
dict->has(key_columns, key_types, out->getData());
}
else
throw Exception{
"Second argument of function " + getName() + " must be " + dict->getKeyDescription(),
ErrorCodes::TYPE_MISMATCH
};
return true;
}
const ExternalDictionaries & dictionaries;
};
class FunctionDictGetString final : public IFunction
{
public:
@ -781,11 +934,12 @@ private:
};
}
if (!typeid_cast<const DataTypeUInt64 *>(arguments[2].get()))
if (!typeid_cast<const DataTypeUInt64 *>(arguments[2].get()) &&
!typeid_cast<const DataTypeTuple *>(arguments[2].get()))
{
throw Exception{
"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64.",
+ ", must be UInt64 or tuple(...).",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
}
@ -794,7 +948,7 @@ private:
{
throw Exception{
"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName()
+ ", must be Date.",
+ ", must be Date.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
}
@ -817,6 +971,8 @@ private:
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchRange<RangeHashedDictionary>(block, arguments, result, dict_ptr))
throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(),
@ -876,6 +1032,53 @@ private:
return true;
}
template <typename DictionaryType>
bool executeDispatchComplex(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * const dictionary)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
if (!dict)
return false;
if (arguments.size() != 3)
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 3 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
const auto & attr_name = attr_name_col->getData();
const auto key_col_with_type = block.getByPosition(arguments[2]);
if (const auto key_col = typeid_cast<const ColumnTuple *>(key_col_with_type.column.get()))
{
const auto key_columns = ext::map<ConstColumnPlainPtrs>(key_col->getColumns(), [] (const ColumnPtr & ptr) {
return ptr.get();
});
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
const auto out = new ColumnString;
block.getByPosition(result).column = out;
dict->getString(attr_name, key_columns, key_types, out);
}
else
throw Exception{
"Third argument of function " + getName() + " must be " + dict->getKeyDescription(),
ErrorCodes::TYPE_MISMATCH
};
return true;
}
template <typename DictionaryType>
bool executeDispatchRange(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * const dictionary)
@ -983,6 +1186,256 @@ private:
};
class FunctionDictGetStringOrDefault final : public IFunction
{
public:
static constexpr auto name = "dictGetStringOrDefault";
static IFunction * create(const Context & context)
{
return new FunctionDictGetStringOrDefault{context.getExternalDictionaries()};
}
FunctionDictGetStringOrDefault(const ExternalDictionaries & dictionaries) : dictionaries(dictionaries) {}
String getName() const override { return name; }
private:
DataTypePtr getReturnType(const DataTypes & arguments) const override
{
if (arguments.size() != 4)
throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed " +
toString(arguments.size()) + ", should be 4.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
if (!typeid_cast<const DataTypeString *>(arguments[0].get()))
throw Exception{
"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() +
", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
if (!typeid_cast<const DataTypeString *>(arguments[1].get()))
throw Exception{
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() +
", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
if (!typeid_cast<const DataTypeUInt64 *>(arguments[2].get()) &&
!typeid_cast<const DataTypeTuple *>(arguments[2].get()))
{
throw Exception{
"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64 or tuple(...).",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
}
if (!typeid_cast<const DataTypeString *>(arguments[3].get()))
throw Exception{
"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName() +
", must be String.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
return new DataTypeString;
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result)
{
const auto dict_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{
"First argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
auto dict = dictionaries.getDictionary(dict_name_col->getData());
const auto dict_ptr = dict.get();
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr))
throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(),
ErrorCodes::UNKNOWN_TYPE
};
}
template <typename DictionaryType>
bool executeDispatch(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * const dictionary)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
if (!dict)
return false;
if (arguments.size() != 4)
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 4 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
const auto & attr_name = attr_name_col->getData();
const auto id_col_untyped = block.getByPosition(arguments[2]).column.get();
if (const auto id_col = typeid_cast<const ColumnVector<UInt64> *>(id_col_untyped))
executeDispatch(block, arguments, result, dict, attr_name, id_col);
else if (const auto id_col = typeid_cast<const ColumnConst<UInt64> *>(id_col_untyped))
executeDispatch(block, arguments, result, dict, attr_name, id_col);
else
throw Exception{
"Third argument of function " + getName() + " must be UInt64",
ErrorCodes::ILLEGAL_COLUMN
};
return true;
}
template <typename DictionaryType>
void executeDispatch(
Block & block, const ColumnNumbers & arguments, const size_t result, const DictionaryType * const dictionary,
const std::string & attr_name, const ColumnVector<UInt64> * const id_col)
{
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
if (const auto default_col = typeid_cast<const ColumnString *>(default_col_untyped))
{
/// vector ids, vector defaults
const auto out = new ColumnString;
block.getByPosition(result).column = out;
dictionary->getString(attr_name, id_col->getData(), default_col, out);
}
else if (const auto default_col = typeid_cast<const ColumnConst<String> *>(default_col_untyped))
{
/// vector ids, const defaults
const auto out = new ColumnString;
block.getByPosition(result).column = out;
/// @todo avoid materialization
const auto default_col_materialized = default_col->convertToFullColumn();
dictionary->getString(attr_name, id_col->getData(),
static_cast<const ColumnString *>(default_col_materialized.get()), out);
}
else
throw Exception{
"Fourth argument of function " + getName() + " must be String",
ErrorCodes::ILLEGAL_COLUMN
};
}
template <typename DictionaryType>
void executeDispatch(
Block & block, const ColumnNumbers & arguments, const size_t result, const DictionaryType * const dictionary,
const std::string & attr_name, const ColumnConst<UInt64> * const id_col)
{
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
if (const auto default_col = typeid_cast<const ColumnString *>(default_col_untyped))
{
/// const ids, vector defaults
/// @todo avoid materialization
const PODArray<UInt64> ids(id_col->size(), id_col->getData());
const auto out = new ColumnString;
block.getByPosition(result).column = out;
dictionary->getString(attr_name, ids, default_col, out);
}
else if (const auto default_col = typeid_cast<const ColumnConst<String> *>(default_col_untyped))
{
/// const ids, const defaults
const PODArray<UInt64> ids(1, id_col->getData());
auto out = std::make_unique<ColumnString>();
/// create ColumnString with default
const auto defs = std::make_unique<ColumnString>();
defs->insert(Field{default_col->getData()});
dictionary->getString(attr_name, ids, defs.get(), out.get());
block.getByPosition(result).column = new ColumnConst<String>{
id_col->size(), out->getDataAt(0).toString()
};
}
else
throw Exception{
"Fourth argument of function " + getName() + " must be String",
ErrorCodes::ILLEGAL_COLUMN
};
}
template <typename DictionaryType>
bool executeDispatchComplex(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * const dictionary)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
if (!dict)
return false;
if (arguments.size() != 4)
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 4 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
const auto & attr_name = attr_name_col->getData();
const auto key_col_with_type = block.getByPosition(arguments[2]);
const auto & key_col = typeid_cast<const ColumnTuple &>(*key_col_with_type.column);
const auto key_columns = ext::map<ConstColumnPlainPtrs>(key_col.getColumns(), [] (const ColumnPtr & ptr) {
return ptr.get();
});
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
const auto out = new ColumnString;
block.getByPosition(result).column = out;
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
if (const auto default_col = typeid_cast<const ColumnString *>(default_col_untyped))
dict->getString(attr_name, key_columns, key_types, default_col, out);
else if (const auto default_col = typeid_cast<const ColumnConst<String> *>(default_col_untyped))
{
/// @todo avoid materialization
const auto default_col_materialized = default_col->convertToFullColumn();
dict->getString(attr_name, key_columns, key_types,
static_cast<const ColumnString *>(default_col_materialized.get()), out);
}
else
throw Exception{
"Fourth argument of function " + getName() + " must be String",
ErrorCodes::ILLEGAL_COLUMN
};
return true;
}
const ExternalDictionaries & dictionaries;
};
template <typename DataType> struct DictGetTraits;
#define DECLARE_DICT_GET_TRAITS(TYPE, DATA_TYPE) \
template <> struct DictGetTraits<DATA_TYPE>\
@ -995,12 +1448,33 @@ template <> struct DictGetTraits<DATA_TYPE>\
dict->get##TYPE(name, ids, out);\
}\
template <typename DictionaryType>\
static void get(\
const DictionaryType * const dict, const std::string & name, const ConstColumnPlainPtrs & key_columns,\
const DataTypes & key_types, PODArray<TYPE> & out)\
{\
dict->get##TYPE(name, key_columns, key_types, out);\
}\
template <typename DictionaryType>\
static void get(\
const DictionaryType * const dict, const std::string & name, const PODArray<UInt64> & ids,\
const PODArray<UInt16> & dates, PODArray<TYPE> & out)\
{\
dict->get##TYPE(name, ids, dates, out);\
}\
template <typename DictionaryType>\
static void getOrDefault(\
const DictionaryType * const dict, const std::string & name, const PODArray<UInt64> & ids,\
const PODArray<TYPE> & def, PODArray<TYPE> & out)\
{\
dict->get##TYPE(name, ids, def, out);\
}\
template <typename DictionaryType>\
static void getOrDefault(\
const DictionaryType * const dict, const std::string & name, const ConstColumnPlainPtrs & key_columns,\
const DataTypes & key_types, const PODArray<TYPE> & def, PODArray<TYPE> & out)\
{\
dict->get##TYPE(name, key_columns, key_types, def, out);\
}\
};
DECLARE_DICT_GET_TRAITS(UInt8, DataTypeUInt8)
DECLARE_DICT_GET_TRAITS(UInt16, DataTypeUInt16)
@ -1061,11 +1535,12 @@ private:
};
}
if (!typeid_cast<const DataTypeUInt64 *>(arguments[2].get()))
if (!typeid_cast<const DataTypeUInt64 *>(arguments[2].get()) &&
!typeid_cast<const DataTypeTuple *>(arguments[2].get()))
{
throw Exception{
"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64.",
+ ", must be UInt64 or tuple(...).",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
}
@ -1097,6 +1572,8 @@ private:
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchRange<RangeHashedDictionary>(block, arguments, result, dict_ptr))
throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(),
@ -1131,13 +1608,11 @@ private:
const auto id_col_untyped = block.getByPosition(arguments[2]).column.get();
if (const auto id_col = typeid_cast<const ColumnVector<UInt64> *>(id_col_untyped))
{
const auto out = new ColumnVector<Type>;
const auto out = new ColumnVector<Type>(id_col->size());
block.getByPosition(result).column = out;
const auto & ids = id_col->getData();
auto & data = out->getData();
const auto size = ids.size();
data.resize(size);
DictGetTraits<DataType>::get(dict, attr_name, ids, data);
}
@ -1160,6 +1635,55 @@ private:
return true;
}
template <typename DictionaryType>
bool executeDispatchComplex(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * const dictionary)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
if (!dict)
return false;
if (arguments.size() != 3)
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 3 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
const auto & attr_name = attr_name_col->getData();
const auto key_col_with_type = block.getByPosition(arguments[2]);
if (const auto key_col = typeid_cast<const ColumnTuple *>(key_col_with_type.column.get()))
{
const auto key_columns = ext::map<ConstColumnPlainPtrs>(key_col->getColumns(), [] (const ColumnPtr & ptr) {
return ptr.get();
});
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
const auto out = new ColumnVector<Type>(key_columns.front()->size());
block.getByPosition(result).column = out;
auto & data = out->getData();
DictGetTraits<DataType>::get(dict, attr_name, key_columns, key_types, data);
}
else
throw Exception{
"Third argument of function " + getName() + " must be " + dict->getKeyDescription(),
ErrorCodes::TYPE_MISMATCH
};
return true;
}
template <typename DictionaryType>
bool executeDispatchRange(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * const dictionary)
@ -1295,6 +1819,294 @@ using FunctionDictGetDate = FunctionDictGet<DataTypeDate>;
using FunctionDictGetDateTime = FunctionDictGet<DataTypeDateTime>;
template <typename DataType>
class FunctionDictGetOrDefault final : public IFunction
{
using Type = typename DataType::FieldType;
public:
static const std::string name;
static IFunction * create(const Context & context)
{
return new FunctionDictGetOrDefault{context.getExternalDictionaries()};
}
FunctionDictGetOrDefault(const ExternalDictionaries & dictionaries) : dictionaries(dictionaries) {}
String getName() const override { return name; }
private:
DataTypePtr getReturnType(const DataTypes & arguments) const override
{
if (arguments.size() != 4)
throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 4.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
if (!typeid_cast<const DataTypeString *>(arguments[0].get()))
{
throw Exception{
"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
}
if (!typeid_cast<const DataTypeString *>(arguments[1].get()))
{
throw Exception{
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
}
if (!typeid_cast<const DataTypeUInt64 *>(arguments[2].get()) &&
!typeid_cast<const DataTypeTuple *>(arguments[2].get()))
{
throw Exception{
"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64 or tuple(...).",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
}
if (!typeid_cast<const DataType *>(arguments[3].get()))
{
throw Exception{
"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName()
+ ", must be " + DataType{}.getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
}
return new DataType;
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override
{
const auto dict_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{
"First argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
auto dict = dictionaries.getDictionary(dict_name_col->getData());
const auto dict_ptr = dict.get();
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr))
throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(),
ErrorCodes::UNKNOWN_TYPE
};
}
template <typename DictionaryType>
bool executeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result,
const IDictionaryBase * const dictionary)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
if (!dict)
return false;
if (arguments.size() != 4)
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 4 arguments.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
const auto & attr_name = attr_name_col->getData();
const auto id_col_untyped = block.getByPosition(arguments[2]).column.get();
if (const auto id_col = typeid_cast<const ColumnVector<UInt64> *>(id_col_untyped))
executeDispatch(block, arguments, result, dict, attr_name, id_col);
else if (const auto id_col = typeid_cast<const ColumnConst<UInt64> *>(id_col_untyped))
executeDispatch(block, arguments, result, dict, attr_name, id_col);
else
throw Exception{
"Third argument of function " + getName() + " must be UInt64",
ErrorCodes::ILLEGAL_COLUMN
};
return true;
}
template <typename DictionaryType>
void executeDispatch(
Block & block, const ColumnNumbers & arguments, const size_t result, const DictionaryType * const dictionary,
const std::string & attr_name, const ColumnVector<UInt64> * const id_col)
{
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
if (const auto default_col = typeid_cast<const ColumnVector<Type> *>(default_col_untyped))
{
/// vector ids, vector defaults
const auto out = new ColumnVector<Type>(id_col->size());
block.getByPosition(result).column = out;
const auto & ids = id_col->getData();
auto & data = out->getData();
const auto & defs = default_col->getData();
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, defs, data);
}
else if (const auto default_col = typeid_cast<const ColumnConst<Type> *>(default_col_untyped))
{
/// vector ids, const defaults
const auto out = new ColumnVector<Type>(id_col->size());
block.getByPosition(result).column = out;
const auto & ids = id_col->getData();
auto & data = out->getData();
/// @todo avoid materialization
const PODArray<Type> defs(id_col->size(), default_col->getData());
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, defs, data);
}
else
throw Exception{
"Fourth argument of function " + getName() + " must be " + DataType{}.getName(),
ErrorCodes::ILLEGAL_COLUMN
};
}
template <typename DictionaryType>
void executeDispatch(
Block & block, const ColumnNumbers & arguments, const size_t result, const DictionaryType * const dictionary,
const std::string & attr_name, const ColumnConst<UInt64> * const id_col)
{
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
if (const auto default_col = typeid_cast<const ColumnVector<Type> *>(default_col_untyped))
{
/// const ids, vector defaults
/// @todo avoid materialization
const PODArray<UInt64> ids(id_col->size(), id_col->getData());
const auto out = new ColumnVector<Type>(id_col->size());
block.getByPosition(result).column = out;
auto & data = out->getData();
const auto & defs = default_col->getData();
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, defs, data);
}
else if (const auto default_col = typeid_cast<const ColumnConst<Type> *>(default_col_untyped))
{
/// const ids, const defaults
const PODArray<UInt64> ids(1, id_col->getData());
PODArray<Type> data(1);
const PODArray<Type> defs(1, default_col->getData());
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, defs, data);
block.getByPosition(result).column = new ColumnConst<Type>{id_col->size(), data.front()};
}
else
throw Exception{
"Fourth argument of function " + getName() + " must be " + DataType{}.getName(),
ErrorCodes::ILLEGAL_COLUMN
};
}
template <typename DictionaryType>
bool executeDispatchComplex(
Block & block, const ColumnNumbers & arguments, const size_t result, const IDictionaryBase * const dictionary)
{
const auto dict = typeid_cast<const DictionaryType *>(dictionary);
if (!dict)
return false;
if (arguments.size() != 4)
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 4 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
const auto & attr_name = attr_name_col->getData();
const auto key_col_with_type = block.getByPosition(arguments[2]);
const auto & key_col = typeid_cast<const ColumnTuple &>(*key_col_with_type.column);
const auto key_columns = ext::map<ConstColumnPlainPtrs>(key_col.getColumns(), [] (const ColumnPtr & ptr) {
return ptr.get();
});
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
/// @todo detect when all key columns are constant
const auto rows = key_col.size();
const auto out = new ColumnVector<Type>(rows);
block.getByPosition(result).column = out;
auto & data = out->getData();
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
if (const auto default_col = typeid_cast<const ColumnVector<Type> *>(default_col_untyped))
{
/// const defaults
const auto & defs = default_col->getData();
DictGetTraits<DataType>::getOrDefault(dict, attr_name, key_columns, key_types, defs, data);
}
else if (const auto default_col = typeid_cast<const ColumnConst<Type> *>(default_col_untyped))
{
/// @todo avoid materialization
const PODArray<Type> defs(rows, default_col->getData());
DictGetTraits<DataType>::getOrDefault(dict, attr_name, key_columns, key_types, defs, data);
}
else
throw Exception{
"Fourth argument of function " + getName() + " must be " + DataType{}.getName(),
ErrorCodes::ILLEGAL_COLUMN
};
return true;
}
const ExternalDictionaries & dictionaries;
};
template <typename DataType>
const std::string FunctionDictGetOrDefault<DataType>::name = "dictGet" + DataType{}.getName() + "OrDefault";
using FunctionDictGetUInt8OrDefault = FunctionDictGetOrDefault<DataTypeUInt8>;
using FunctionDictGetUInt16OrDefault = FunctionDictGetOrDefault<DataTypeUInt16>;
using FunctionDictGetUInt32OrDefault = FunctionDictGetOrDefault<DataTypeUInt32>;
using FunctionDictGetUInt64OrDefault = FunctionDictGetOrDefault<DataTypeUInt64>;
using FunctionDictGetInt8OrDefault = FunctionDictGetOrDefault<DataTypeInt8>;
using FunctionDictGetInt16OrDefault = FunctionDictGetOrDefault<DataTypeInt16>;
using FunctionDictGetInt32OrDefault = FunctionDictGetOrDefault<DataTypeInt32>;
using FunctionDictGetInt64OrDefault = FunctionDictGetOrDefault<DataTypeInt64>;
using FunctionDictGetFloat32OrDefault = FunctionDictGetOrDefault<DataTypeFloat32>;
using FunctionDictGetFloat64OrDefault = FunctionDictGetOrDefault<DataTypeFloat64>;
using FunctionDictGetDateOrDefault = FunctionDictGetOrDefault<DataTypeDate>;
using FunctionDictGetDateTimeOrDefault = FunctionDictGetOrDefault<DataTypeDateTime>;
class FunctionDictGetHierarchy final : public IFunction
{
public:

View File

@ -19,13 +19,12 @@
#include <DB/Columns/ColumnConst.h>
#include <DB/Columns/ColumnFixedString.h>
#include <DB/Columns/ColumnArray.h>
#include <DB/Columns/ColumnTuple.h>
#include <DB/Common/HashTable/Hash.h>
#include <DB/Functions/IFunction.h>
#include <ext/range.hpp>
#include <stats/IntHash.h>
namespace DB
{
@ -399,6 +398,9 @@ UInt64 toInteger<Float64>(Float64 x)
}
/** Используются хэш-функции под названием CityHash, FarmHash, MetroHash.
* В связи с этим, этот шаблон назван со словами NeighbourhoodHash.
*/
template <typename Impl>
class FunctionNeighbourhoodHash64 : public IFunction
{
@ -565,6 +567,29 @@ private:
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
void executeForArgument(const IDataType * type, const IColumn * column, ColumnUInt64::Container_t & vec_to, bool & is_first)
{
/// Раскрытие кортежей.
if (const ColumnTuple * tuple = typeid_cast<const ColumnTuple *>(column))
{
const Block & tuple_data = tuple->getData();
for (size_t i = 0, size = tuple_data.columns(); i < size; ++i)
{
const ColumnWithTypeAndName & col = tuple_data.unsafeGetByPosition(i);
executeForArgument(col.type.get(), col.column.get(), vec_to, is_first);
}
}
else
{
if (is_first)
executeAny<true>(type, column, vec_to);
else
executeAny<false>(type, column, vec_to);
}
is_first = false;
}
public:
/// Получить имя функции.
String getName() const override
@ -589,20 +614,17 @@ public:
if (arguments.empty())
{
/// Случайное число из /dev/urandom используется как хеш пустого кортежа.
/// Случайное число из /dev/urandom используется как хеш пустого количества аргументов.
vec_to.assign(rows, 0xe28dbde7fe22e41c);
}
/// Функция поддерживает произвольное количество аргументов всевозможных типов.
bool is_first_argument = true;
for (size_t i = 0; i < arguments.size(); ++i)
{
const ColumnWithTypeAndName & column = block.getByPosition(arguments[i]);
const IDataType * from_type = &*column.type;
const IColumn * icolumn = &*column.column;
if (i == 0)
executeAny<true>(from_type, icolumn, vec_to);
else
executeAny<false>(from_type, icolumn, vec_to);
const ColumnWithTypeAndName & col = block.getByPosition(arguments[i]);
executeForArgument(col.type.get(), col.column.get(), vec_to, is_first_argument);
}
}
};

View File

@ -339,11 +339,11 @@ public:
/// Выполнить функцию над блоком.
void execute(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const IColumn & argument = *block.getByPosition(arguments[0]).column;
if (!argument.isConst())
throw Exception("Argument for function " + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN);
block.getByPosition(result).column = dynamic_cast<const IColumnConst &>(argument).convertToFullColumn();
const auto & src = block.getByPosition(arguments[0]).column;
if (auto converted = src->convertToFullColumnIfConst())
block.getByPosition(result).column = converted;
else
block.getByPosition(result).column = src;
}
};
@ -865,7 +865,7 @@ public:
}
template <typename T>
bool execute(Block & block, const IColumn * in_untyped, const size_t result) override
bool execute(Block & block, const IColumn * in_untyped, const size_t result)
{
if (const auto in = typeid_cast<const ColumnVector<T> *>(in_untyped))
{

View File

@ -5,7 +5,6 @@
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/Functions/IFunction.h>
#include <DB/Common/HashTable/Hash.h>
#include <stats/IntHash.h>
namespace DB

View File

@ -18,7 +18,7 @@ namespace DB
* Отличается тем, что не делает лишний memset. (И почти ничего не делает.)
* Также можно попросить выделять выровненный кусок памяти.
*/
struct Memory : boost::noncopyable, Allocator
struct Memory : boost::noncopyable, Allocator<false>
{
size_t m_capacity = 0;
size_t m_size = 0;

View File

@ -1,6 +1,6 @@
#pragma once
#include <src/double-conversion.h>
#include <double-conversion/double-conversion.h>
namespace DB
{

View File

@ -55,11 +55,8 @@ private:
BufferWithOwnMemory<ReadBuffer> fill_buffer;
/// Описание асинхронного запроса на чтение.
iocb request = { 0 };
std::vector<iocb *> request_ptrs{&request};
std::vector<io_event> events{1};
AIOContext aio_context{1};
iocb request{};
std::future<ssize_t> future_bytes_read;
const std::string filename;

View File

@ -743,6 +743,18 @@ struct AggregatedDataVariants : private boost::noncopyable
typedef SharedPtr<AggregatedDataVariants> AggregatedDataVariantsPtr;
typedef std::vector<AggregatedDataVariantsPtr> ManyAggregatedDataVariants;
/** Как считаются "тотальные" значения при наличии WITH TOTALS?
* (Более подробно смотрите в TotalsHavingBlockInputStream.)
*
* В случае отсутствия group_by_overflow_mode = 'any', данные агрегируются как обычно, но состояния агрегатных функций не финализируются.
* Позже, состояния агрегатных функций для всех строк (прошедших через HAVING) мерджатся в одну - это и будет TOTALS.
*
* В случае наличия group_by_overflow_mode = 'any', данные агрегируются как обычно, кроме ключей, не поместившихся в max_rows_to_group_by.
* Для этих ключей, данные агрегируются в одну дополнительную строку - далее см. под названиями overflow_row, overflows...
* Позже, состояния агрегатных функций для всех строк (прошедших через HAVING) мерджатся в одну,
* а также к ним прибавляется или не прибавляется (в зависимости от настройки totals_mode) также overflow_row - это и будет TOTALS.
*/
/** Агрегирует источник блоков.
*/
@ -1032,7 +1044,7 @@ protected:
size_t rows,
Filler && filler) const;
BlocksList prepareBlocksAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final) const;
BlocksList prepareBlocksAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows) const;
BlocksList prepareBlocksAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const;
BlocksList prepareBlocksAndFillTwoLevel(AggregatedDataVariants & data_variants, bool final, boost::threadpool::pool * thread_pool) const;

View File

@ -69,7 +69,7 @@ public:
bool array_join_is_left;
/// Для JOIN
const Join * join = nullptr;
std::shared_ptr<const Join> join;
NamesAndTypesList columns_added_by_join;
/// Для PROJECT.
@ -134,7 +134,7 @@ public:
return a;
}
static ExpressionAction ordinaryJoin(const Join * join_, const NamesAndTypesList & columns_added_by_join_)
static ExpressionAction ordinaryJoin(std::shared_ptr<const Join> join_, const NamesAndTypesList & columns_added_by_join_)
{
ExpressionAction a;
a.type = JOIN;

View File

@ -208,6 +208,9 @@ private:
*/
void collectJoinedColumns(NameSet & joined_columns, NamesAndTypesList & joined_columns_name_type);
/** Добавляет ALIAS столбцы из storage в aliases, если запрос не является SELECT с ARRAY JOIN. При наличии
* ARRAY JOIN их добавлять нельзя, иначе ломается логика его выполнения.
*/
void addStorageAliases();
/** Создать словарь алиасов.
@ -229,6 +232,9 @@ private:
/// Превратить перечисление значений или подзапрос в ASTSet. node - функция in или notIn.
void makeSet(ASTFunction * node, const Block & sample_block);
/// Добавляет список ALIAS столбцов из таблицы
void addAliasColumns();
/// Замена скалярных подзапросов на значения-константы.
void executeScalarSubqueries();
void executeScalarSubqueriesImpl(ASTPtr & ast);

View File

@ -61,6 +61,21 @@ std::string getNameFromInSubqueryAttributes(IAST::Attributes attributes)
return name;
}
/// Проверить, указана ли таблица в секции FROM.
bool isQueryFromTable(const ASTSelectQuery & query)
{
if (query.table)
{
if (typeid_cast<const ASTSelectQuery *>(query.table.get()) != nullptr)
return false;
else if (typeid_cast<const ASTFunction *>(query.table.get()) != nullptr)
return false;
else
return true;
}
return false;
}
/// Проверить, является ли движок распределённым с количеством шардов более одного.
template <typename TStorageDistributed>
bool isEligibleStorageForInJoinPreprocessing(const StoragePtr & storage)
@ -175,7 +190,8 @@ public:
else if ((node != static_cast<IAST *>(select_query))
&& ((sub_select_query = typeid_cast<ASTSelectQuery *>(node)) != nullptr))
{
++node->select_query_depth;
if (isQueryFromTable(*sub_select_query))
++node->select_query_depth;
if (sub_select_query->enclosing_in_or_join != nullptr)
{
/// Найден подзапрос внутри секции IN или JOIN.

View File

@ -4,7 +4,6 @@
#include <DB/Storages/AlterCommands.h>
#include <DB/Interpreters/Context.h>
#include <DB/Interpreters/IInterpreter.h>
#include <DB/Parsers/ASTIdentifier.h>
namespace DB
{

View File

@ -2,7 +2,6 @@
#include <DB/Interpreters/Context.h>
#include <DB/Interpreters/IInterpreter.h>
#include <DB/Parsers/ASTIdentifier.h>
namespace DB
{

View File

@ -2,7 +2,6 @@
#include <DB/Storages/IStorage.h>
#include <DB/Parsers/TablePropertiesQueriesASTs.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Interpreters/Context.h>
#include <DB/Interpreters/IInterpreter.h>
#include <DB/DataStreams/OneBlockInputStream.h>

View File

@ -2,7 +2,6 @@
#include <DB/Storages/IStorage.h>
#include <DB/Parsers/TablePropertiesQueriesASTs.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Interpreters/Context.h>
#include <DB/Interpreters/IInterpreter.h>
#include <DB/DataStreams/OneBlockInputStream.h>

View File

@ -2,7 +2,6 @@
#include <DB/Storages/IStorage.h>
#include <DB/Parsers/TablePropertiesQueriesASTs.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/formatAST.h>
#include <DB/Interpreters/Context.h>
#include <DB/Interpreters/IInterpreter.h>

View File

@ -260,7 +260,7 @@ private:
void checkTypesOfKeys(const Block & block_left, const Block & block_right) const;
};
typedef Poco::SharedPtr<Join> JoinPtr;
typedef std::shared_ptr<Join> JoinPtr;
typedef std::vector<JoinPtr> Joins;

View File

@ -366,7 +366,8 @@ private:
size_t rows) const;
};
typedef Poco::SharedPtr<Set> SetPtr;
typedef std::shared_ptr<Set> SetPtr;
typedef std::shared_ptr<const Set> ConstSetPtr;
typedef std::vector<SetPtr> Sets;

View File

@ -16,6 +16,10 @@ namespace DB
*/
struct Settings
{
/// Для того, чтобы инициализация из пустого initializer-list была value initialization, а не aggregate initialization в С++14.
/// http://en.cppreference.com/w/cpp/language/aggregate_initialization
Settings() {}
/** Перечисление настроек: тип, имя, значение по-умолчанию.
*
* Это сделано несколько неудобно, чтобы не перечислять настройки во многих разных местах.
@ -168,6 +172,8 @@ struct Settings
M(SettingUInt64, select_sequential_consistency, 0) \
/** Максимальное количество различных шардов и максимальное количество реплик одного шарда в функции remote. */ \
M(SettingUInt64, table_function_remote_max_addresses, 1000) \
/** Маскимальное количество потоков при распределённой обработке одного запроса **/ \
M(SettingUInt64, max_distributed_processing_threads, 8) \
/// Всевозможные ограничения на выполнение запроса.
Limits limits;

View File

@ -44,8 +44,8 @@ inline void evaluateMissingDefaults(Block & block,
/// move evaluated columns to the original block, materializing them at the same time
for (auto & column_name_type : copy_block.getColumns())
{
if (column_name_type.column->isConst())
column_name_type.column = static_cast<const IColumnConst &>(*column_name_type.column).convertToFullColumn();
if (auto converted = column_name_type.column->convertToFullColumnIfConst())
column_name_type.column = converted;
block.insert(std::move(column_name_type));
}

View File

@ -15,8 +15,8 @@ void executeQuery(
WriteBuffer & ostr, /// Куда писать результат
Context & context, /// БД, таблицы, типы данных, движки таблиц, функции, агрегатные функции...
BlockInputStreamPtr & query_plan, /// Сюда может быть записано описание, как выполнялся запрос
bool internal = false, /// Если true - значит запрос порождён из другого запроса, и не нужно его регистировать в ProcessList-е.
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete); /// До какой стадии выполнять SELECT запрос.
std::function<void(const String &)> set_content_type /// Может быть передан колбэк, с помощью которого может быть сообщён Content-Type формата.
);
/** Более низкоуровневая функция для межсерверного взаимодействия.
@ -35,8 +35,9 @@ void executeQuery(
*/
BlockIO executeQuery(
const String & query, /// Текст запроса, без данных INSERT-а (если есть). Данные INSERT-а следует писать в BlockIO::out.
Context & context,
bool internal = false,
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete);
Context & context, /// БД, таблицы, типы данных, движки таблиц, функции, агрегатные функции...
bool internal = false, /// Если true - значит запрос порождён из другого запроса, и не нужно его регистировать в ProcessList-е.
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete /// До какой стадии выполнять SELECT запрос.
);
}

View File

@ -8,7 +8,7 @@
namespace DB
{
/** Идентификатор (столбца или алиас, или именованый элемент кортежа)
/** Идентификатор (столбца или алиас)
*/
class ASTIdentifier : public ASTWithAlias
{
@ -21,7 +21,7 @@ public:
Format,
};
/// имя
/// имя. У составного идентификатора здесь будет конкатенированное имя (вида a.b.c), а отдельные составляюшие будут доступны внутри children.
String name;
/// чего идентифицирует этот идентификатор
@ -44,16 +44,7 @@ public:
}
protected:
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
settings.ostr << (settings.hilite ? hilite_identifier : "");
WriteBufferFromOStream wb(settings.ostr, 32);
writeProbablyBackQuotedString(name, wb);
wb.next();
settings.ostr << (settings.hilite ? hilite_none : "");
}
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
}

Some files were not shown because too many files have changed in this diff Show More