This commit is contained in:
Alexey Vasiliev 2015-11-30 16:42:21 +03:00
commit d8c3cdda56
103 changed files with 4585 additions and 1077 deletions

View File

@ -115,6 +115,13 @@ public:
{
nested_func->insertResultInto(place, to);
}
static void addFree(const IAggregateFunction * that, AggregateDataPtr place, const IColumn ** columns, size_t row_num)
{
return static_cast<const AggregateFunctionArray &>(*that).add(place, columns, row_num);
}
IAggregateFunction::AddFunc getAddressOfAddFunction() const override final { return &addFree; }
};
}

View File

@ -102,6 +102,13 @@ public:
{
nested_func->insertResultInto(place, to);
}
static void addFree(const IAggregateFunction * that, AggregateDataPtr place, const IColumn ** columns, size_t row_num)
{
return static_cast<const AggregateFunctionIf &>(*that).add(place, columns, row_num);
}
IAggregateFunction::AddFunc getAddressOfAddFunction() const override final { return &addFree; }
};
}

View File

@ -103,6 +103,13 @@ public:
{
nested_func->insertResultInto(place, to);
}
static void addFree(const IAggregateFunction * that, AggregateDataPtr place, const IColumn ** columns, size_t row_num)
{
return static_cast<const AggregateFunctionMerge &>(*that).add(place, columns, row_num);
}
IAggregateFunction::AddFunc getAddressOfAddFunction() const override final { return &addFree; }
};
}

View File

@ -2,8 +2,6 @@
#include <DB/Common/PODArray.h>
#include <DB/Core/FieldVisitors.h>
#include <DB/IO/WriteHelpers.h>
#include <DB/IO/ReadHelpers.h>
@ -11,6 +9,7 @@
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
#include <DB/AggregateFunctions/QuantilesCommon.h>
#include <DB/Columns/ColumnArray.h>
@ -131,8 +130,7 @@ class AggregateFunctionQuantilesExact final
: public IUnaryAggregateFunction<AggregateFunctionQuantileExactData<T>, AggregateFunctionQuantilesExact<T>>
{
private:
using Levels = std::vector<double>;
Levels levels;
QuantileLevels<double> levels;
DataTypePtr type;
public:
@ -150,14 +148,7 @@ public:
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]);
levels.set(params);
}
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const
@ -202,26 +193,30 @@ public:
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();
size_t old_size = data_to.size();
data_to.resize(old_size + num_levels);
if (!array.empty())
{
size_t prev_n = 0;
for (const auto & level : levels)
for (auto level_index : levels.permutation)
{
auto level = levels.levels[level_index];
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]);
data_to[old_size + level_index] = array[n];
prev_n = n;
}
}
else
{
for (size_t i = 0; i < num_levels; ++i)
data_to.push_back(T());
data_to[old_size + i] = T();
}
}
};

View File

@ -2,12 +2,11 @@
#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/AggregateFunctions/QuantilesCommon.h>
#include <DB/Columns/ColumnArray.h>
@ -165,8 +164,7 @@ class AggregateFunctionQuantilesExactWeighted final
AggregateFunctionQuantilesExactWeighted<ValueType, WeightType>>
{
private:
using Levels = std::vector<double>;
Levels levels;
QuantileLevels<double> levels;
DataTypePtr type;
public:
@ -184,14 +182,7 @@ public:
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]);
levels.set(params);
}
void addImpl(AggregateDataPtr place, const IColumn & column_value, const IColumn & column_weight, size_t row_num) const
@ -240,10 +231,13 @@ public:
typename ColumnVector<ValueType>::Container_t & data_to = static_cast<ColumnVector<ValueType> &>(arr_to.getData()).getData();
size_t old_size = data_to.size();
data_to.resize(old_size + num_levels);
if (0 == size)
{
for (size_t i = 0; i < num_levels; ++i)
data_to.push_back(ValueType());
data_to[old_size + i] = ValueType();
return;
}
@ -268,9 +262,9 @@ public:
const Pair * it = array;
const Pair * end = array + size;
for (const auto & level : levels)
for (auto level_index : levels.permutation)
{
UInt64 threshold = sum_weight * level;
UInt64 threshold = sum_weight * levels.levels[level_index];
while (it < end && accumulated < threshold)
{
@ -278,7 +272,7 @@ public:
++it;
}
data_to.push_back(it < end ? it->first : it[-1].first);
data_to[old_size + level_index] = it < end ? it->first : it[-1].first;
}
}
};

View File

@ -0,0 +1,643 @@
#pragma once
#include <cmath>
#include <cstdint>
#include <cassert>
#include <vector>
#include <algorithm>
#include <DB/Common/RadixSort.h>
#include <DB/Common/PODArray.h>
#include <DB/Columns/ColumnArray.h>
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
#include <DB/AggregateFunctions/IBinaryAggregateFunction.h>
#include <DB/AggregateFunctions/QuantilesCommon.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeArray.h>
/** Алгоритм реализовал Алексей Борзенков https://███████████.yandex-team.ru/snaury
* Ему принадлежит авторство кода и половины комментариев в данном namespace,
* за исключением слияния, сериализации и сортировки, а также выбора типов и других изменений.
* Мы благодарим Алексея Борзенкова за написание изначального кода.
*/
namespace tdigest
{
/**
* Центроид хранит вес точек вокруг их среднего значения
*/
template <typename Value, typename Count>
struct Centroid
{
Value mean;
Count count;
Centroid() = default;
explicit Centroid(Value mean, Count count = 1)
: mean(mean)
, count(count)
{}
Centroid & operator+=(const Centroid & other)
{
count += other.count;
mean += other.count * (other.mean - mean) / count;
return *this;
}
bool operator<(const Centroid & other) const
{
return mean < other.mean;
}
};
/** :param epsilon: значение \delta из статьи - погрешность в районе
* квантиля 0.5 (по-умолчанию 0.01, т.е. 1%)
* :param max_unmerged: при накоплении кол-ва новых точек сверх этого
* значения запускается компрессия центроидов
* (по-умолчанию 2048, чем выше значение - тем
* больше требуется памяти, но повышается
* амортизация времени выполнения)
*/
template <typename Value>
struct Params
{
Value epsilon = 0.01;
size_t max_unmerged = 2048;
};
/** Реализация алгоритма t-digest (https://github.com/tdunning/t-digest).
* Этот вариант очень похож на MergingDigest на java, однако решение об
* объединении принимается на основе оригинального условия из статьи
* (через ограничение на размер, используя апроксимацию квантиля каждого
* центроида, а не расстояние на кривой положения их границ). MergingDigest
* на java даёт значительно меньше центроидов, чем данный вариант, что
* негативно влияет на точность при том же факторе компрессии, но даёт
* гарантии размера. Сам автор на предложение об этом варианте сказал, что
* размер дайжеста растёт как O(log(n)), в то время как вариант на java
* не зависит от предполагаемого кол-ва точек. Кроме того вариант на java
* использует asin, чем немного замедляет алгоритм.
*/
template <typename Value, typename CentroidCount, typename TotalCount>
class MergingDigest
{
using Params = tdigest::Params<Value>;
using Centroid = tdigest::Centroid<Value, CentroidCount>;
/// Сразу будет выделена память на несколько элементов так, чтобы состояние занимало 64 байта.
static constexpr size_t bytes_in_arena = 64 - sizeof(DB::PODArray<Centroid>) - sizeof(TotalCount) - sizeof(uint32_t);
using Summary = DB::PODArray<Centroid, bytes_in_arena / sizeof(Centroid), AllocatorWithStackMemory<Allocator<false>, bytes_in_arena>>;
Summary summary;
TotalCount count = 0;
uint32_t unmerged = 0;
/** Линейная интерполяция в точке x на прямой (x1, y1)..(x2, y2)
*/
static Value interpolate(Value x, Value x1, Value y1, Value x2, Value y2)
{
double k = (x - x1) / (x2 - x1);
return y1 + k * (y2 - y1);
}
struct RadixSortTraits
{
using Element = Centroid;
using Key = Value;
using CountType = uint32_t;
using KeyBits = uint32_t;
static constexpr size_t PART_SIZE_BITS = 8;
using Transform = RadixSortFloatTransform<KeyBits>;
using Allocator = RadixSortMallocAllocator;
/// Функция получения ключа из элемента массива.
static Key & extractKey(Element & elem) { return elem.mean; }
};
public:
/** Добавляет к дайджесту изменение x с весом cnt (по-умолчанию 1)
*/
void add(const Params & params, Value x, CentroidCount cnt = 1)
{
add(params, Centroid(x, cnt));
}
/** Добавляет к дайджесту центроид c
*/
void add(const Params & params, const Centroid & c)
{
summary.push_back(c);
count += c.count;
++unmerged;
if (unmerged >= params.max_unmerged)
compress(params);
}
/** Выполняет компрессию накопленных центроидов
* При объединении сохраняется инвариант на максимальный размер каждого
* центроида, не превышающий 4 q (1 - q) \delta N.
*/
void compress(const Params & params)
{
if (unmerged > 0)
{
RadixSort<RadixSortTraits>::execute(&summary[0], summary.size());
if (summary.size() > 3)
{
/// Пара подряд идущих столбиков гистограммы.
auto l = summary.begin();
auto r = std::next(l);
TotalCount sum = 0;
while (r != summary.end())
{
// we use quantile which gives us the smallest error
/// Отношение части гистограммы до l, включая половинку l ко всей гистограмме. То есть, какого уровня квантиль в позиции l.
Value ql = (sum + l->count * 0.5) / count;
Value err = ql * (1 - ql);
/// Отношение части гистограммы до l, включая l и половинку r ко всей гистограмме. То есть, какого уровня квантиль в позиции r.
Value qr = (sum + l->count + r->count * 0.5) / count;
Value err2 = qr * (1 - qr);
if (err > err2)
err = err2;
Value k = 4 * count * err * params.epsilon;
/** Отношение веса склеенной пары столбиков ко всем значениям не больше,
* чем epsilon умножить на некий квадратичный коэффициент, который в медиане равен 1 (4 * 1/2 * 1/2),
* а по краям убывает и примерно равен расстоянию до края * 4.
*/
if (l->count + r->count <= k)
{
// it is possible to merge left and right
/// Левый столбик "съедает" правый.
*l += *r;
}
else
{
// not enough capacity, check the next pair
sum += l->count;
++l;
/// Пропускаем все "съеденные" ранее значения.
if (l != r)
*l = *r;
}
++r;
}
/// По окончании цикла, все значения правее l были "съедены".
summary.resize(l - summary.begin() + 1);
}
unmerged = 0;
}
}
/** Вычисляет квантиль q [0, 1] на основе дайджеста
* Для пустого дайджеста возвращает NaN.
*/
Value getQuantile(const Params & params, Value q)
{
if (summary.empty())
return NAN;
compress(params);
if (summary.size() == 1)
return summary.front().mean;
Value x = q * count;
TotalCount sum = 0;
Value prev_mean = summary.front().mean;
Value prev_x = 0;
for (const auto & c : summary)
{
Value current_x = sum + c.count * 0.5;
if (current_x >= x)
return interpolate(x, prev_x, prev_mean, current_x, c.mean);
sum += c.count;
prev_mean = c.mean;
prev_x = current_x;
}
return summary.back().mean;
}
/** Получить несколько квантилей (size штук).
* levels - массив уровней нужных квантилей. Они идут в произвольном порядке.
* levels_permutation - массив-перестановка уровней. На i-ой позиции будет лежать индекс i-го по возрастанию уровня в массиве levels.
* result - массив, куда сложить результаты, в порядке levels,
*/
template <typename ResultType>
void getManyQuantiles(const Params & params, const Value * levels, const size_t * levels_permutation, size_t size, ResultType * result)
{
if (summary.empty())
{
for (size_t result_num = 0; result_num < size; ++result_num)
result[result_num] = std::is_floating_point<ResultType>::value ? NAN : 0;
return;
}
compress(params);
if (summary.size() == 1)
{
for (size_t result_num = 0; result_num < size; ++result_num)
result[result_num] = summary.front().mean;
return;
}
Value x = levels[levels_permutation[0]] * count;
TotalCount sum = 0;
Value prev_mean = summary.front().mean;
Value prev_x = 0;
size_t result_num = 0;
for (const auto & c : summary)
{
Value current_x = sum + c.count * 0.5;
while (current_x >= x)
{
result[levels_permutation[result_num]] = interpolate(x, prev_x, prev_mean, current_x, c.mean);
++result_num;
if (result_num >= size)
return;
x = levels[levels_permutation[result_num]] * count;
}
sum += c.count;
prev_mean = c.mean;
prev_x = current_x;
}
auto rest_of_results = summary.back().mean;
for (; result_num < size; ++result_num)
result[levels_permutation[result_num]] = rest_of_results;
}
/** Объединить с другим состоянием.
*/
void merge(const Params & params, const MergingDigest & other)
{
for (const auto & c : other.summary)
add(params, c);
}
/** Записать в поток.
*/
void write(const Params & params, DB::WriteBuffer & buf)
{
compress(params);
DB::writeVarUInt(summary.size(), buf);
buf.write(reinterpret_cast<const char *>(&summary[0]), summary.size() * sizeof(summary[0]));
}
/** Прочитать из потока и объединить с текущим состоянием.
*/
void readAndMerge(const Params & params, DB::ReadBuffer & buf)
{
size_t size = 0;
DB::readVarUInt(size, buf);
if (size > params.max_unmerged)
throw DB::Exception("Too large t-digest summary size", DB::ErrorCodes::TOO_LARGE_ARRAY_SIZE);
for (size_t i = 0; i < size; ++i)
{
Centroid c;
DB::readPODBinary(c, buf);
add(params, c);
}
}
};
}
namespace DB
{
struct AggregateFunctionQuantileTDigestData
{
tdigest::MergingDigest<Float32, Float32, Float32> digest;
};
template <typename T, bool returns_float = true>
class AggregateFunctionQuantileTDigest final
: public IUnaryAggregateFunction<AggregateFunctionQuantileTDigestData, AggregateFunctionQuantileTDigest<T>>
{
private:
Float32 level;
tdigest::Params<Float32> params;
DataTypePtr type;
public:
AggregateFunctionQuantileTDigest(double level_ = 0.5) : level(level_) {}
String getName() const override { return "quantileTDigest"; }
DataTypePtr getReturnType() const override
{
return type;
}
void setArgument(const DataTypePtr & argument)
{
if (returns_float)
type = new DataTypeFloat32;
else
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<Float32>(), params[0]);
}
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const
{
this->data(place).digest.add(params, static_cast<const ColumnVector<T> &>(column).getData()[row_num]);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).digest.merge(params, this->data(rhs).digest);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(const_cast<AggregateDataPtr>(place)).digest.write(params, buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
this->data(place).digest.readAndMerge(params, buf);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
auto quantile = this->data(const_cast<AggregateDataPtr>(place)).digest.getQuantile(params, level);
if (returns_float)
static_cast<ColumnFloat32 &>(to).getData().push_back(quantile);
else
static_cast<ColumnVector<T> &>(to).getData().push_back(quantile);
}
};
template <typename T, typename Weight, bool returns_float = true>
class AggregateFunctionQuantileTDigestWeighted final
: public IBinaryAggregateFunction<AggregateFunctionQuantileTDigestData, AggregateFunctionQuantileTDigestWeighted<T, Weight, returns_float>>
{
private:
Float32 level;
tdigest::Params<Float32> params;
DataTypePtr type;
public:
AggregateFunctionQuantileTDigestWeighted(double level_ = 0.5) : level(level_) {}
String getName() const override { return "quantileTDigestWeighted"; }
DataTypePtr getReturnType() const override
{
return type;
}
void setArgumentsImpl(const DataTypes & arguments)
{
if (returns_float)
type = new DataTypeFloat32;
else
type = arguments.at(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<Float32>(), params[0]);
}
void addImpl(AggregateDataPtr place, const IColumn & column_value, const IColumn & column_weight, size_t row_num) const
{
this->data(place).digest.add(params,
static_cast<const ColumnVector<T> &>(column_value).getData()[row_num],
static_cast<const ColumnVector<Weight> &>(column_weight).getData()[row_num]);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).digest.merge(params, this->data(rhs).digest);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(const_cast<AggregateDataPtr>(place)).digest.write(params, buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
this->data(place).digest.readAndMerge(params, buf);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
auto quantile = this->data(const_cast<AggregateDataPtr>(place)).digest.getQuantile(params, level);
if (returns_float)
static_cast<ColumnFloat32 &>(to).getData().push_back(quantile);
else
static_cast<ColumnVector<T> &>(to).getData().push_back(quantile);
}
};
template <typename T, bool returns_float = true>
class AggregateFunctionQuantilesTDigest final
: public IUnaryAggregateFunction<AggregateFunctionQuantileTDigestData, AggregateFunctionQuantilesTDigest<T>>
{
private:
QuantileLevels<Float32> levels;
tdigest::Params<Float32> params;
DataTypePtr type;
public:
String getName() const override { return "quantilesTDigest"; }
DataTypePtr getReturnType() const override
{
return new DataTypeArray(type);
}
void setArgument(const DataTypePtr & argument)
{
if (returns_float)
type = new DataTypeFloat32;
else
type = argument;
}
void setParameters(const Array & params) override
{
levels.set(params);
}
void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const
{
this->data(place).digest.add(params, static_cast<const ColumnVector<T> &>(column).getData()[row_num]);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).digest.merge(params, this->data(rhs).digest);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(const_cast<AggregateDataPtr>(place)).digest.write(params, buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
this->data(place).digest.readAndMerge(params, buf);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
ColumnArray & arr_to = static_cast<ColumnArray &>(to);
ColumnArray::Offsets_t & offsets_to = arr_to.getOffsets();
size_t size = levels.size();
offsets_to.push_back((offsets_to.size() == 0 ? 0 : offsets_to.back()) + size);
if (returns_float)
{
typename ColumnFloat32::Container_t & data_to = static_cast<ColumnFloat32 &>(arr_to.getData()).getData();
size_t old_size = data_to.size();
data_to.resize(data_to.size() + size);
this->data(const_cast<AggregateDataPtr>(place)).digest.getManyQuantiles(
params, &levels.levels[0], &levels.permutation[0], size, &data_to[old_size]);
}
else
{
typename ColumnVector<T>::Container_t & data_to = static_cast<ColumnVector<T> &>(arr_to.getData()).getData();
size_t old_size = data_to.size();
data_to.resize(data_to.size() + size);
this->data(const_cast<AggregateDataPtr>(place)).digest.getManyQuantiles(
params, &levels.levels[0], &levels.permutation[0], size, &data_to[old_size]);
}
}
};
template <typename T, typename Weight, bool returns_float = true>
class AggregateFunctionQuantilesTDigestWeighted final
: public IBinaryAggregateFunction<AggregateFunctionQuantileTDigestData, AggregateFunctionQuantilesTDigestWeighted<T, Weight, returns_float>>
{
private:
QuantileLevels<Float32> levels;
tdigest::Params<Float32> params;
DataTypePtr type;
public:
String getName() const override { return "quantilesTDigest"; }
DataTypePtr getReturnType() const override
{
return new DataTypeArray(type);
}
void setArgumentsImpl(const DataTypes & arguments)
{
if (returns_float)
type = new DataTypeFloat32;
else
type = arguments.at(0);
}
void setParameters(const Array & params) override
{
levels.set(params);
}
void addImpl(AggregateDataPtr place, const IColumn & column_value, const IColumn & column_weight, size_t row_num) const
{
this->data(place).digest.add(params,
static_cast<const ColumnVector<T> &>(column_value).getData()[row_num],
static_cast<const ColumnVector<Weight> &>(column_weight).getData()[row_num]);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override
{
this->data(place).digest.merge(params, this->data(rhs).digest);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
this->data(const_cast<AggregateDataPtr>(place)).digest.write(params, buf);
}
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const override
{
this->data(place).digest.readAndMerge(params, buf);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
ColumnArray & arr_to = static_cast<ColumnArray &>(to);
ColumnArray::Offsets_t & offsets_to = arr_to.getOffsets();
size_t size = levels.size();
offsets_to.push_back((offsets_to.size() == 0 ? 0 : offsets_to.back()) + size);
if (returns_float)
{
typename ColumnFloat32::Container_t & data_to = static_cast<ColumnFloat32 &>(arr_to.getData()).getData();
size_t old_size = data_to.size();
data_to.resize(data_to.size() + size);
this->data(const_cast<AggregateDataPtr>(place)).digest.getManyQuantiles(
params, &levels.levels[0], &levels.permutation[0], size, &data_to[old_size]);
}
else
{
typename ColumnVector<T>::Container_t & data_to = static_cast<ColumnVector<T> &>(arr_to.getData()).getData();
size_t old_size = data_to.size();
data_to.resize(data_to.size() + size);
this->data(const_cast<AggregateDataPtr>(place)).digest.getManyQuantiles(
params, &levels.levels[0], &levels.permutation[0], size, &data_to[old_size]);
}
}
};
}

View File

@ -5,8 +5,6 @@
#include <DB/Common/MemoryTracker.h>
#include <DB/Common/HashTable/Hash.h>
#include <DB/Core/FieldVisitors.h>
#include <DB/IO/WriteHelpers.h>
#include <DB/IO/ReadHelpers.h>
@ -15,6 +13,7 @@
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
#include <DB/AggregateFunctions/IBinaryAggregateFunction.h>
#include <DB/AggregateFunctions/QuantilesCommon.h>
#include <DB/Columns/ColumnArray.h>
@ -235,15 +234,10 @@ namespace detail
}
/// Получить значения size квантилей уровней levels. Записать size результатов начиная с адреса result.
/// indices - массив индексов levels такой, что соответствующие элементы будут идти в порядке по возрастанию.
template <typename ResultType>
void getMany(const double * levels, size_t size, ResultType * result) const
void getMany(const double * levels, const size_t * indices, size_t size, ResultType * result) const
{
std::size_t indices[size];
std::copy(ext::range_iterator<size_t>{}, ext::make_range_iterator(size), indices);
std::sort(indices, indices + size, [levels] (auto i1, auto i2) {
return levels[i1] < levels[i2];
});
const auto indices_end = indices + size;
auto index = indices;
@ -311,10 +305,10 @@ namespace detail
: std::numeric_limits<float>::quiet_NaN();
}
void getManyFloat(const double * levels, size_t size, float * result) const
void getManyFloat(const double * levels, const size_t * levels_permutation, size_t size, float * result) const
{
if (count)
getMany(levels, size, result);
getMany(levels, levels_permutation, size, result);
else
for (size_t i = 0; i < size; ++i)
result[i] = std::numeric_limits<float>::quiet_NaN();
@ -503,11 +497,11 @@ public:
/// Получить значения size квантилей уровней levels. Записать size результатов начиная с адреса result.
template <typename ResultType>
void getMany(const double * levels, size_t size, ResultType * result) const
void getMany(const double * levels, const size_t * levels_permutation, size_t size, ResultType * result) const
{
if (isLarge())
{
return large->getMany(levels, size, result);
return large->getMany(levels, levels_permutation, size, result);
}
else
{
@ -524,10 +518,10 @@ public:
: std::numeric_limits<float>::quiet_NaN();
}
void getManyFloat(const double * levels, size_t size, float * result) const
void getManyFloat(const double * levels, const size_t * levels_permutation, size_t size, float * result) const
{
if (tiny.count)
getMany(levels, size, result);
getMany(levels, levels_permutation, size, result);
else
for (size_t i = 0; i < size; ++i)
result[i] = std::numeric_limits<float>::quiet_NaN();
@ -600,7 +594,8 @@ public:
/** То же самое, но с двумя аргументами. Второй аргумент - "вес" (целое число) - сколько раз учитывать значение.
*/
template <typename ArgumentFieldType, typename WeightFieldType>
class AggregateFunctionQuantileTimingWeighted final : public IAggregateFunctionHelper<QuantileTiming>
class AggregateFunctionQuantileTimingWeighted final
: public IBinaryAggregateFunction<QuantileTiming, AggregateFunctionQuantileTimingWeighted<ArgumentFieldType, WeightFieldType>>
{
private:
double level;
@ -615,7 +610,7 @@ public:
return new DataTypeFloat32;
}
void setArguments(const DataTypes & arguments) override
void setArgumentsImpl(const DataTypes & arguments)
{
}
@ -627,12 +622,11 @@ public:
level = apply_visitor(FieldVisitorConvertToNumber<Float64>(), params[0]);
}
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const override
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 override
@ -665,8 +659,7 @@ template <typename ArgumentFieldType>
class AggregateFunctionQuantilesTiming final : public IUnaryAggregateFunction<QuantileTiming, AggregateFunctionQuantilesTiming<ArgumentFieldType> >
{
private:
typedef std::vector<double> Levels;
Levels levels;
QuantileLevels<double> levels;
public:
String getName() const override { return "quantilesTiming"; }
@ -682,14 +675,7 @@ public:
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]);
levels.set(params);
}
@ -725,7 +711,7 @@ public:
size_t old_size = data_to.size();
data_to.resize(data_to.size() + size);
this->data(place).getManyFloat(&levels[0], size, &data_to[old_size]);
this->data(place).getManyFloat(&levels.levels[0], &levels.permutation[0], size, &data_to[old_size]);
}
};
@ -735,8 +721,7 @@ class AggregateFunctionQuantilesTimingWeighted final
: public IBinaryAggregateFunction<QuantileTiming, AggregateFunctionQuantilesTimingWeighted<ArgumentFieldType, WeightFieldType>>
{
private:
using Levels = std::vector<double>;
Levels levels;
QuantileLevels<double> levels;
public:
String getName() const override { return "quantilesTimingWeighted"; }
@ -752,14 +737,7 @@ public:
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]);
levels.set(params);
}
void addImpl(AggregateDataPtr place, const IColumn & column_value, const IColumn & column_weight, size_t row_num) const
@ -796,7 +774,7 @@ public:
size_t old_size = data_to.size();
data_to.resize(data_to.size() + size);
this->data(place).getManyFloat(&levels[0], size, &data_to[old_size]);
this->data(place).getManyFloat(&levels.levels[0], &levels.permutation[0], size, &data_to[old_size]);
}
};

View File

@ -229,6 +229,13 @@ public:
static_cast<ColumnUInt8 &>(to).getData().push_back(match(events_it, events_end));
}
static void addFree(const IAggregateFunction * that, AggregateDataPtr place, const IColumn ** columns, size_t row_num)
{
return static_cast<const AggregateFunctionSequenceMatch &>(*that).add(place, columns, row_num);
}
IAggregateFunction::AddFunc getAddressOfAddFunction() const override final { return &addFree; }
private:
enum class PatternActionType
{

View File

@ -101,6 +101,13 @@ public:
bool isState() const override { return true; }
AggregateFunctionPtr getNestedFunction() const { return nested_func_owner; }
static void addFree(const IAggregateFunction * that, AggregateDataPtr place, const IColumn ** columns, size_t row_num)
{
return static_cast<const AggregateFunctionState &>(*that).add(place, columns, row_num);
}
IAggregateFunction::AddFunc getAddressOfAddFunction() const override final { return &addFree; }
};
}

View File

@ -420,6 +420,13 @@ public:
{
static_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).set.size());
}
static void addFree(const IAggregateFunction * that, AggregateDataPtr place, const IColumn ** columns, size_t row_num)
{
return static_cast<const AggregateFunctionUniqVariadic &>(*that).add(place, columns, row_num);
}
IAggregateFunction::AddFunc getAddressOfAddFunction() const override final { return &addFree; }
};

View File

@ -260,6 +260,13 @@ public:
{
static_cast<ColumnUInt64 &>(to).getData().push_back(this->data(place).size());
}
static void addFree(const IAggregateFunction * that, AggregateDataPtr place, const IColumn ** columns, size_t row_num)
{
return static_cast<const AggregateFunctionUniqUpToVariadic &>(*that).add(place, columns, row_num);
}
IAggregateFunction::AddFunc getAddressOfAddFunction() const override final { return &addFree; }
};

View File

@ -1,6 +1,7 @@
#pragma once
#include <DB/AggregateFunctions/AggregateFunctionsMinMaxAny.h>
#include <DB/AggregateFunctions/IBinaryAggregateFunction.h>
namespace DB
@ -20,7 +21,7 @@ struct AggregateFunctionsArgMinMaxData
/// Возвращает первое попавшееся значение arg для минимального/максимального value. Пример: argMax(arg, value).
template <typename Data>
class AggregateFunctionsArgMinMax final : public IAggregateFunctionHelper<Data>
class AggregateFunctionsArgMinMax final : public IBinaryAggregateFunction<Data, AggregateFunctionsArgMinMax<Data>>
{
private:
DataTypePtr type_res;
@ -34,19 +35,16 @@ public:
return type_res;
}
void setArguments(const DataTypes & arguments) override
void setArgumentsImpl(const DataTypes & arguments)
{
if (arguments.size() != 2)
throw Exception("Aggregate function " + getName() + " requires exactly two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
type_res = arguments[0];
type_val = arguments[1];
}
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num) const override
void addImpl(AggregateDataPtr place, const IColumn & column_arg, const IColumn & column_max, size_t row_num) const
{
if (this->data(place).value.changeIfBetter(*columns[1], row_num))
this->data(place).result.change(*columns[0], row_num);
if (this->data(place).value.changeIfBetter(column_max, row_num))
this->data(place).result.change(column_arg, row_num);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const override

View File

@ -139,7 +139,7 @@ struct SingleValueDataFixed
/** Для строк. Короткие строки хранятся в самой структуре, а длинные выделяются отдельно.
* NOTE Могло бы подойти также для массивов чисел.
*/
struct __attribute__((__packed__)) SingleValueDataString
struct __attribute__((__packed__, __aligned__(1))) SingleValueDataString
{
typedef SingleValueDataString Self;
@ -148,10 +148,10 @@ struct __attribute__((__packed__)) SingleValueDataString
static constexpr Int32 AUTOMATIC_STORAGE_SIZE = 64;
static constexpr Int32 MAX_SMALL_STRING_SIZE = AUTOMATIC_STORAGE_SIZE - sizeof(size);
union __attribute__((__aligned__(1)))
union __attribute__((__packed__, __aligned__(1)))
{
char small_data[MAX_SMALL_STRING_SIZE]; /// Включая завершающий ноль.
char * __attribute__((__aligned__(1))) large_data;
char * __attribute__((__packed__, __aligned__(1))) large_data;
};
~SingleValueDataString()
@ -336,6 +336,10 @@ struct __attribute__((__packed__)) SingleValueDataString
}
};
static_assert(
sizeof(SingleValueDataString) == SingleValueDataString::AUTOMATIC_STORAGE_SIZE,
"Incorrect size of SingleValueDataString struct");
/// Для любых других типов значений.
struct SingleValueDataGeneric

View File

@ -90,6 +90,16 @@ public:
* Они выполняются как другие агрегатные функции, но не финализируются (возвращают состояние агрегации, которое может быть объединено с другим).
*/
virtual bool isState() const { return false; }
/** Внутренний цикл, использующий указатель на функцию, получается лучше, чем использующий виртуальную функцию.
* Причина в том, что в случае виртуальных функций, GCC 5.1.2 генерирует код,
* который на каждой итерации цикла заново грузит из памяти в регистр адрес функции (значение по смещению в таблице виртуальных функций).
* Это даёт падение производительности на простых запросах в районе 12%.
* После появления более хороших компиляторов, код можно будет убрать.
*/
using AddFunc = void (*)(const IAggregateFunction *, AggregateDataPtr, const IColumn **, size_t);
virtual AddFunc getAddressOfAddFunction() const = 0;
};

View File

@ -28,6 +28,13 @@ public:
{
getDerived().addImpl(place, *columns[0], *columns[1], row_num);
}
static void addFree(const IAggregateFunction * that, AggregateDataPtr place, const IColumn ** columns, size_t row_num)
{
return static_cast<const Derived &>(*that).addImpl(place, *columns[0], *columns[1], row_num);
}
IAggregateFunction::AddFunc getAddressOfAddFunction() const override final { return &addFree; }
};
}

View File

@ -30,6 +30,13 @@ public:
getDerived().addImpl(place);
}
static void addFree(const IAggregateFunction * that, AggregateDataPtr place, const IColumn ** columns, size_t row_num)
{
return static_cast<const Derived &>(*that).addImpl(place);
}
IAggregateFunction::AddFunc getAddressOfAddFunction() const override final { return &addFree; }
/** Реализуйте это в классе-наследнике:
* void addImpl(AggregateDataPtr place) const;
*/

View File

@ -31,6 +31,13 @@ public:
getDerived().addImpl(place, *columns[0], row_num);
}
static void addFree(const IAggregateFunction * that, AggregateDataPtr place, const IColumn ** columns, size_t row_num)
{
return static_cast<const Derived &>(*that).addImpl(place, *columns[0], row_num);
}
IAggregateFunction::AddFunc getAddressOfAddFunction() const override final { return &addFree; }
/** Реализуйте это в классе-наследнике:
* void addImpl(AggregateDataPtr place, const IColumn & column, size_t row_num) const;
* void setArgument(const DataTypePtr & argument);

View File

@ -0,0 +1,52 @@
#pragma once
#include <vector>
#include <DB/Core/Field.h>
#include <DB/Core/FieldVisitors.h>
namespace DB
{
/** Параметры разных функций quantilesSomething.
* - список уровней квантилей.
* Также необходимо вычислить массив индексов уровней, идущих по возрастанию.
*
* Пример: quantiles(0.5, 0.99, 0.95)(x).
* levels: 0.5, 0.99, 0.95
* levels_permutation: 0, 2, 1
*/
template <typename T> /// float или double
struct QuantileLevels
{
using Levels = std::vector<T>;
using Permutation = std::vector<size_t>;
Levels levels;
Permutation permutation; /// Индекс i-го по величине уровня в массиве levels.
size_t size() const { return levels.size(); }
void set(const Array & params)
{
if (params.empty())
throw Exception("Aggregate function quantiles requires at least one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
size_t size = params.size();
levels.resize(size);
permutation.resize(size);
for (size_t i = 0; i < size; ++i)
{
levels[i] = apply_visitor(FieldVisitorConvertToNumber<Float64>(), params[i]);
permutation[i] = i;
}
std::sort(permutation.begin(), permutation.end(), [this] (size_t a, size_t b) { return levels[a] < levels[b]; });
}
};
}

View File

@ -194,22 +194,21 @@ public:
return getData().size() * sizeof(getData()[0]);
}
ColumnPtr cut(size_t start, size_t length) const override
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override
{
if (start + length > getData().size())
const ColumnAggregateFunction & src_concrete = static_cast<const ColumnAggregateFunction &>(src);
if (start + length > src_concrete.getData().size())
throw Exception("Parameters start = "
+ toString(start) + ", length = "
+ toString(length) + " are out of bound in ColumnAggregateFunction::cut() method"
" (data.size() = " + toString(getData().size()) + ").",
+ toString(length) + " are out of bound in ColumnAggregateFunction::insertRangeFrom method"
" (data.size() = " + toString(src_concrete.getData().size()) + ").",
ErrorCodes::PARAMETER_OUT_OF_BOUND);
ColumnAggregateFunction * res_ = new ColumnAggregateFunction(*this);
ColumnPtr res = res_;
res_->getData().resize(length);
for (size_t i = 0; i < length; ++i)
res_->getData()[i] = getData()[start + i];
return res;
auto & data = getData();
size_t old_size = data.size();
data.resize(old_size + length);
memcpy(&data[old_size], &src_concrete.getData()[start], length * sizeof(data[0]));
}
ColumnPtr filter(const Filter & filter) const override

View File

@ -147,7 +147,7 @@ public:
return pos;
}
ColumnPtr cut(size_t start, size_t length) const override;
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
void insert(const Field & x) override
{
@ -293,9 +293,9 @@ private:
size_t ALWAYS_INLINE sizeAt(size_t i) const { return i == 0 ? getOffsets()[0] : (getOffsets()[i] - getOffsets()[i - 1]); }
/// Размножить значения, если вложенный столбец - ColumnArray<T>.
/// Размножить значения, если вложенный столбец - ColumnVector<T>.
template <typename T>
ColumnPtr replicate(const Offsets_t & replicate_offsets) const;
ColumnPtr replicateNumber(const Offsets_t & replicate_offsets) const;
/// Размножить значения, если вложенный столбец - ColumnString. Код слишком сложный.
ColumnPtr replicateString(const Offsets_t & replicate_offsets) const;
@ -306,6 +306,14 @@ private:
* Только ради неё сделана реализация метода replicate для ColumnArray(ColumnConst).
*/
ColumnPtr replicateConst(const Offsets_t & replicate_offsets) const;
/// Специализации для функции filter.
template <typename T>
ColumnPtr filterNumber(const Filter & filt) const;
ColumnPtr filterString(const Filter & filt) const;
ColumnPtr filterGeneric(const Filter & filt) const;
};

View File

@ -7,6 +7,7 @@
#include <DB/Core/ErrorCodes.h>
#include <DB/Columns/ColumnVector.h>
#include <DB/Columns/IColumn.h>
#include <DB/Columns/ColumnsCommon.h>
#include <DB/DataTypes/IDataType.h>
@ -62,9 +63,13 @@ public:
Field operator[](size_t n) const override { return FieldType(getDataFromHolder()); }
void get(size_t n, Field & res) const override { res = FieldType(getDataFromHolder()); }
ColumnPtr cut(size_t start, size_t length) const override
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override
{
return new Derived(length, data, data_type);
if (getDataFromHolder() != static_cast<const Derived &>(src).getDataFromHolder())
throw Exception("Cannot insert different element into constant column " + getName(),
ErrorCodes::CANNOT_INSERT_ELEMENT_INTO_CONSTANT_COLUMN);
s += length;
}
void insert(const Field & x) override

View File

@ -5,6 +5,7 @@
#include <DB/Common/PODArray.h>
#include <DB/Common/Arena.h>
#include <DB/Columns/IColumn.h>
#include <DB/IO/ReadHelpers.h>
namespace DB
@ -172,13 +173,20 @@ public:
}
}
ColumnPtr cut(size_t start, size_t length) const override
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override
{
ColumnFixedString * res_ = new ColumnFixedString(n);
ColumnPtr res = res_;
res_->chars.resize(n * length);
memcpy(&res_->chars[0], &chars[n * start], n * length);
return res;
const ColumnFixedString & src_concrete = static_cast<const ColumnFixedString &>(src);
if (start + length > src_concrete.size())
throw Exception("Parameters start = "
+ toString(start) + ", length = "
+ toString(length) + " are out of bound in ColumnFixedString::insertRangeFrom method"
" (size() = " + toString(src_concrete.size()) + ").",
ErrorCodes::PARAMETER_OUT_OF_BOUND);
size_t old_size = chars.size();
chars.resize(old_size + length * n);
memcpy(&chars[old_size], &src_concrete.chars[start * n], length * n);
}
ColumnPtr filter(const IColumn::Filter & filt) const override

View File

@ -5,6 +5,7 @@
#include <DB/Core/Defines.h>
#include <DB/Columns/IColumn.h>
#include <DB/Columns/ColumnsCommon.h>
#include <DB/Common/Collator.h>
#include <DB/Common/PODArray.h>
#include <DB/Common/Arena.h>
@ -144,48 +145,42 @@ public:
return pos + string_size;
}
ColumnPtr cut(size_t start, size_t length) const override
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override
{
if (length == 0)
return new ColumnString;
return;
if (start + length > offsets.size())
throw Exception("Parameter out of bound in IColumnString::cut() method.",
const ColumnString & src_concrete = static_cast<const ColumnString &>(src);
if (start + length > src_concrete.offsets.size())
throw Exception("Parameter out of bound in IColumnString::insertRangeFrom method.",
ErrorCodes::PARAMETER_OUT_OF_BOUND);
size_t nested_offset = offsetAt(start);
size_t nested_length = offsets[start + length - 1] - nested_offset;
size_t nested_offset = src_concrete.offsetAt(start);
size_t nested_length = src_concrete.offsets[start + length - 1] - nested_offset;
ColumnString * res_ = new ColumnString;
ColumnPtr res = res_;
size_t old_chars_size = chars.size();
chars.resize(old_chars_size + nested_length);
memcpy(&chars[old_chars_size], &src_concrete.chars[nested_offset], nested_length);
res_->chars.resize(nested_length);
memcpy(&res_->chars[0], &chars[nested_offset], nested_length);
Offsets_t & res_offsets = res_->offsets;
if (start == 0)
if (start == 0 && offsets.empty())
{
res_offsets.assign(offsets.begin(), offsets.begin() + length);
offsets.assign(src_concrete.offsets.begin(), src_concrete.offsets.begin() + length);
}
else
{
res_offsets.resize(length);
size_t old_size = offsets.size();
size_t prev_max_offset = old_size ? offsets.back() : 0;
offsets.resize(old_size + length);
for (size_t i = 0; i < length; ++i)
res_offsets[i] = offsets[start + i] - nested_offset;
offsets[old_size + i] = src_concrete.offsets[start + i] - nested_offset + prev_max_offset;
}
return res;
}
ColumnPtr filter(const Filter & filt) const override
{
const size_t size = offsets.size();
if (size != filt.size())
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
if (size == 0)
if (offsets.size() == 0)
return new ColumnString;
auto res = new ColumnString;
@ -193,96 +188,8 @@ public:
Chars_t & res_chars = res->chars;
Offsets_t & res_offsets = res->offsets;
res_chars.reserve(chars.size());
res_offsets.reserve(size);
Offset_t current_offset = 0;
const UInt8 * filt_pos = &filt[0];
const auto filt_end = filt_pos + size;
const auto filt_end_aligned = filt_pos + size / 16 * 16;
auto offsets_pos = &offsets[0];
const auto offsets_begin = offsets_pos;
const __m128i zero16 = _mm_setzero_si128();
/// copy string ending at *end_offset_ptr
const auto copy_string = [&] (const Offset_t * offset_ptr) {
const auto offset = offset_ptr == offsets_begin ? 0 : offset_ptr[-1];
const auto size = *offset_ptr - offset;
current_offset += size;
res_offsets.push_back(current_offset);
const auto chars_size_old = res_chars.size();
res_chars.resize_assume_reserved(chars_size_old + size);
memcpy(&res_chars[chars_size_old], &chars[offset], size);
};
while (filt_pos < filt_end_aligned)
{
const auto mask = _mm_movemask_epi8(_mm_cmpgt_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(filt_pos)),
zero16));
if (mask == 0)
{
/// 16 consecutive rows do not pass the filter
}
else if (mask == 0xffff)
{
/// 16 consecutive rows pass the filter
const auto first = offsets_pos == offsets_begin;
const auto chunk_offset = first ? 0 : offsets_pos[-1];
const auto chunk_size = offsets_pos[16 - 1] - chunk_offset;
const auto offsets_size_old = res_offsets.size();
res_offsets.resize(offsets_size_old + 16);
memcpy(&res_offsets[offsets_size_old], offsets_pos, 16 * sizeof(Offset_t));
if (!first)
{
/// difference between current and actual offset
const auto diff_offset = chunk_offset - current_offset;
if (diff_offset > 0)
{
const auto res_offsets_pos = &res_offsets[offsets_size_old];
/// adjust offsets
for (size_t i = 0; i < 16; ++i)
res_offsets_pos[i] -= diff_offset;
}
}
current_offset += chunk_size;
/// copy characters for 16 strings at once
const auto chars_size_old = res_chars.size();
res_chars.resize(chars_size_old + chunk_size);
memcpy(&res_chars[chars_size_old], &chars[chunk_offset], chunk_size);
}
else
{
for (size_t i = 0; i < 16; ++i)
if (filt_pos[i])
copy_string(offsets_pos + i);
}
filt_pos += 16;
offsets_pos += 16;
}
while (filt_pos < filt_end)
{
if (*filt_pos)
copy_string(offsets_pos);
++filt_pos;
++offsets_pos;
}
filterArraysImpl<UInt8>(chars, offsets, res_chars, res_offsets, filt);
return res_;
}

View File

@ -23,7 +23,7 @@ public:
size_t size = data.columns();
columns.resize(size);
for (size_t i = 0; i < size; ++i)
columns[i] = data.getByPosition(i).column;
columns[i] = data.unsafeGetByPosition(i).column;
}
std::string getName() const override { return "Tuple"; }
@ -115,15 +115,12 @@ public:
return pos;
}
ColumnPtr cut(size_t start, size_t length) const override
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override
{
Block res_block = data.cloneEmpty();
for (size_t i = 0; i < columns.size(); ++i)
res_block.getByPosition(i).column = data.getByPosition(i).column->cut(start, length);
return new ColumnTuple(res_block);
data.unsafeGetByPosition(i).column->insertRangeFrom(
*static_cast<const ColumnTuple &>(src).data.unsafeGetByPosition(i).column.get(),
start, length);
}
ColumnPtr filter(const Filter & filt) const override
@ -131,7 +128,7 @@ public:
Block res_block = data.cloneEmpty();
for (size_t i = 0; i < columns.size(); ++i)
res_block.getByPosition(i).column = data.getByPosition(i).column->filter(filt);
res_block.unsafeGetByPosition(i).column = data.unsafeGetByPosition(i).column->filter(filt);
return new ColumnTuple(res_block);
}
@ -141,7 +138,7 @@ public:
Block res_block = data.cloneEmpty();
for (size_t i = 0; i < columns.size(); ++i)
res_block.getByPosition(i).column = data.getByPosition(i).column->permute(perm, limit);
res_block.unsafeGetByPosition(i).column = data.unsafeGetByPosition(i).column->permute(perm, limit);
return new ColumnTuple(res_block);
}
@ -151,7 +148,7 @@ public:
Block res_block = data.cloneEmpty();
for (size_t i = 0; i < columns.size(); ++i)
res_block.getByPosition(i).column = data.getByPosition(i).column->replicate(offsets);
res_block.unsafeGetByPosition(i).column = data.unsafeGetByPosition(i).column->replicate(offsets);
return new ColumnTuple(res_block);
}

View File

@ -254,18 +254,20 @@ public:
data.push_back(DB::get<typename NearestFieldType<T>::Type>(x));
}
ColumnPtr cut(size_t start, size_t length) const override
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override
{
if (start + length > data.size())
const ColumnVector & src_vec = static_cast<const ColumnVector &>(src);
if (start + length > src_vec.data.size())
throw Exception("Parameters start = "
+ toString(start) + ", length = "
+ toString(length) + " are out of bound in IColumnVector<T>::cut() method"
" (data.size() = " + toString(data.size()) + ").",
+ toString(length) + " are out of bound in ColumnVector::insertRangeFrom method"
" (data.size() = " + toString(src_vec.data.size()) + ").",
ErrorCodes::PARAMETER_OUT_OF_BOUND);
Self * res = new Self(length);
memcpy(&res->getData()[0], &data[start], length * sizeof(data[0]));
return res;
size_t old_size = data.size();
data.resize(old_size + length);
memcpy(&data[old_size], &src_vec.data[start], length * sizeof(data[0]));
}
ColumnPtr filter(const IColumn::Filter & filt) const override

View File

@ -0,0 +1,22 @@
#pragma once
#include <DB/Columns/IColumn.h>
/// Общие вспомогательные методы для реализации разных столбцов.
namespace DB
{
/// Считает, сколько байт в filt больше нуля.
size_t countBytesInFilter(const IColumn::Filter & filt);
/// Общая реализация функции filter для ColumnArray и ColumnString.
template <typename T>
void filterArraysImpl(
const PODArray<T> & src_elems, const IColumn::Offsets_t & src_offsets,
PODArray<T> & res_elems, IColumn::Offsets_t & res_offsets,
const IColumn::Filter & filt);
}

View File

@ -111,7 +111,12 @@ public:
/** Удалить всё кроме диапазона элементов.
* Используется, например, для операции LIMIT.
*/
virtual SharedPtr<IColumn> cut(size_t start, size_t length) const = 0;
virtual SharedPtr<IColumn> cut(size_t start, size_t length) const
{
SharedPtr<IColumn> res = cloneEmpty();
res.get()->insertRangeFrom(*this, start, length);
return res;
}
/** Вставить значение в конец столбца (количество значений увеличится на 1).
* Используется для преобразования из строк в блоки (например, при чтении значений из текстового дампа)
@ -123,6 +128,11 @@ public:
*/
virtual void insertFrom(const IColumn & src, size_t n) { insert(src[n]); }
/** Вставить в конец столбца диапазон элементов из другого столбца.
* Может использоваться для склейки столбцов.
*/
virtual void insertRangeFrom(const IColumn & src, size_t start, size_t length) = 0;
/** Вставить данные, расположенные в указанном куске памяти, если возможно.
* (если не реализуемо - кидает исключение)
* Используется для оптимизации некоторых вычислений (например, агрегации).
@ -227,8 +237,4 @@ public:
};
/// Считает, сколько байт в filt больше нуля.
size_t countBytesInFilter(const IColumn::Filter & filt);
}

View File

@ -1,6 +1,7 @@
#pragma once
#include <DB/Columns/IColumn.h>
#include <DB/Columns/ColumnsCommon.h>
namespace DB
@ -44,9 +45,9 @@ public:
throw Exception("Method getExtremes is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
ColumnPtr cut(size_t start, size_t length) const override
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override
{
return cloneDummy(length);
s += length;
}
ColumnPtr filter(const Filter & filt) const override

View File

@ -1,6 +1,7 @@
#pragma once
#include <DB/Common/Arena.h>
#include <ext/bit_cast.hpp>
#include <ext/size.hpp>
#include <cstdlib>
#include <memory>
@ -27,7 +28,13 @@ private:
return sizes;
}
static auto getMinBucketNum() { return 3; }
static auto sizeToPreviousPowerOfTwo(const int size) { return _bit_scan_reverse(size - 1); }
static auto getMinBucketNum()
{
static const auto val = sizeToPreviousPowerOfTwo(getSizes().front());
return val;
}
static auto getMaxFixedBlockSize() { return getSizes().back(); }
Arena pool;
@ -36,7 +43,7 @@ private:
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);
const auto bucket_num = sizeToPreviousPowerOfTwo(size);
return std::max(bucket_num, getMinBucketNum()) - getMinBucketNum();
}

View File

@ -0,0 +1,256 @@
#pragma once
#include <string.h>
#include <malloc.h>
#include <cstdint>
#include <type_traits>
#include <ext/bit_cast.hpp>
#include <DB/Core/Defines.h>
/** Поразрядная сортировка, обладает следующей функциональностью:
* Может сортировать unsigned, signed числа, а также float-ы.
* Может сортировать массив элементов фиксированной длины, которые содержат что-то ещё кроме ключа.
* Настраиваемый размер разряда.
*
* LSB, stable.
* NOTE Для некоторых приложений имеет смысл добавить MSB-radix-sort,
* а также алгоритмы radix-select, radix-partial-sort, radix-get-permutation на его основе.
*/
/** Используется в качестве параметра шаблона. См. ниже.
*/
struct RadixSortMallocAllocator
{
void * allocate(size_t size)
{
return malloc(size);
}
void deallocate(void * ptr, size_t size)
{
return free(ptr);
}
};
/** Преобразование, которое переводит битовое представление ключа в такое целое беззнаковое число,
* что отношение порядка над ключами будет соответствовать отношению порядка над полученными беззнаковыми числами.
* Для float-ов это преобразование делает следующее:
* если выставлен знаковый бит, то переворачивает все остальные биты.
*/
template <typename KeyBits>
struct RadixSortFloatTransform
{
/// Стоит ли записывать результат в память, или лучше делать его каждый раз заново?
static constexpr bool transform_is_simple = false;
static KeyBits forward(KeyBits x)
{
return x ^ (-((x >> (sizeof(KeyBits) * 8 - 1) | (KeyBits(1) << (sizeof(KeyBits) * 8 - 1)))));
}
static KeyBits backward(KeyBits x)
{
return x ^ (((x >> (sizeof(KeyBits) * 8 - 1)) - 1) | (KeyBits(1) << (sizeof(KeyBits) * 8 - 1)));
}
};
template <typename Float>
struct RadixSortFloatTraits
{
using Element = Float; /// Тип элемента. Это может быть структура с ключём и ещё каким-то payload-ом. Либо просто ключ.
using Key = Float; /// Ключ, по которому нужно сортировать.
using CountType = uint32_t; /// Тип для подсчёта гистограмм. В случае заведомо маленького количества элементов, может быть меньше чем size_t.
/// Тип, в который переводится ключ, чтобы делать битовые операции. Это UInt такого же размера, как ключ.
using KeyBits = typename std::conditional<sizeof(Float) == 8, uint64_t, uint32_t>::type;
static constexpr size_t PART_SIZE_BITS = 8; /// Какими кусочками ключа в количестве бит делать один проход - перестановку массива.
/// Преобразования ключа в KeyBits такое, что отношение порядка над ключём соответствует отношению порядка над KeyBits.
using Transform = RadixSortFloatTransform<KeyBits>;
/// Объект с функциями allocate и deallocate.
/// Может быть использован, например, чтобы выделить память для временного массива на стеке.
/// Для этого сам аллокатор создаётся на стеке.
using Allocator = RadixSortMallocAllocator;
/// Функция получения ключа из элемента массива.
static Key & extractKey(Element & elem) { return elem; }
};
template <typename KeyBits>
struct RadixSortIdentityTransform
{
static constexpr bool transform_is_simple = true;
static KeyBits forward(KeyBits x) { return x; }
static KeyBits backward(KeyBits x) { return x; }
};
template <typename KeyBits>
struct RadixSortSignedTransform
{
static constexpr bool transform_is_simple = true;
static KeyBits forward(KeyBits x) { return x ^ (KeyBits(1) << (sizeof(KeyBits) * 8 - 1)); }
static KeyBits backward(KeyBits x) { return x ^ (KeyBits(1) << (sizeof(KeyBits) * 8 - 1)); }
};
template <typename UInt>
struct RadixSortUIntTraits
{
using Element = UInt;
using Key = UInt;
using CountType = uint32_t;
using KeyBits = UInt;
static constexpr size_t PART_SIZE_BITS = 8;
using Transform = RadixSortIdentityTransform<KeyBits>;
using Allocator = RadixSortMallocAllocator;
/// Функция получения ключа из элемента массива.
static Key & extractKey(Element & elem) { return elem; }
};
template <typename Int>
struct RadixSortIntTraits
{
using Element = Int;
using Key = Int;
using CountType = uint32_t;
using KeyBits = typename std::make_unsigned<Int>::type;
static constexpr size_t PART_SIZE_BITS = 8;
using Transform = RadixSortSignedTransform<KeyBits>;
using Allocator = RadixSortMallocAllocator;
/// Функция получения ключа из элемента массива.
static Key & extractKey(Element & elem) { return elem; }
};
template <typename Traits>
struct RadixSort
{
private:
using Element = typename Traits::Element;
using Key = typename Traits::Key;
using CountType = typename Traits::CountType;
using KeyBits = typename Traits::KeyBits;
static constexpr size_t HISTOGRAM_SIZE = 1 << Traits::PART_SIZE_BITS;
static constexpr size_t PART_BITMASK = HISTOGRAM_SIZE - 1;
static constexpr size_t KEY_BITS = sizeof(Key) * 8;
static constexpr size_t NUM_PASSES = (KEY_BITS + (Traits::PART_SIZE_BITS - 1)) / Traits::PART_SIZE_BITS;
static ALWAYS_INLINE KeyBits getPart(size_t N, KeyBits x)
{
if (Traits::Transform::transform_is_simple)
x = Traits::Transform::forward(x);
return (x >> (N * Traits::PART_SIZE_BITS)) & PART_BITMASK;
}
static KeyBits keyToBits(Key x) { return ext::bit_cast<KeyBits>(x); }
static Key bitsToKey(KeyBits x) { return ext::bit_cast<Key>(x); }
public:
static void execute(Element * arr, size_t size)
{
/// Если массив имеет размер меньше 256, то лучше использовать другой алгоритм.
/// Здесь есть циклы по NUM_PASSES. Очень важно, что они разворачиваются в compile-time.
/// Для каждого из NUM_PASSES кусков бит ключа, считаем, сколько раз каждое значение этого куска встретилось.
CountType histograms[HISTOGRAM_SIZE * NUM_PASSES] = {0};
typename Traits::Allocator allocator;
/// Будем делать несколько проходов по массиву. На каждом проходе, данные перекладываются в другой массив. Выделим этот временный массив.
Element * swap_buffer = reinterpret_cast<Element *>(allocator.allocate(size * sizeof(Element)));
/// Трансформируем массив и вычисляем гистограмму.
for (size_t i = 0; i < size; ++i)
{
if (!Traits::Transform::transform_is_simple)
Traits::extractKey(arr[i]) = bitsToKey(Traits::Transform::forward(keyToBits(Traits::extractKey(arr[i]))));
for (size_t j = 0; j < NUM_PASSES; ++j)
++histograms[j * HISTOGRAM_SIZE + getPart(j, keyToBits(Traits::extractKey(arr[i])))];
}
{
/// Заменяем гистограммы на суммы с накоплением: значение в позиции i равно сумме в предыдущих позициях минус один.
size_t sums[NUM_PASSES] = {0};
for (size_t i = 0; i < HISTOGRAM_SIZE; ++i)
{
for (size_t j = 0; j < NUM_PASSES; ++j)
{
size_t tmp = histograms[j * HISTOGRAM_SIZE + i] + sums[j];
histograms[j * HISTOGRAM_SIZE + i] = sums[j] - 1;
sums[j] = tmp;
}
}
}
/// Перекладываем элементы в порядке начиная от младшего куска бит, и далее делаем несколько проходов по количеству кусков.
for (size_t j = 0; j < NUM_PASSES; ++j)
{
Element * writer = j % 2 ? arr : swap_buffer;
Element * reader = j % 2 ? swap_buffer : arr;
for (size_t i = 0; i < size; ++i)
{
size_t pos = getPart(j, keyToBits(Traits::extractKey(reader[i])));
/// Размещаем элемент на следующей свободной позиции.
auto & dest = writer[++histograms[j * HISTOGRAM_SIZE + pos]];
dest = reader[i];
/// На последнем перекладывании, делаем обратную трансформацию.
if (!Traits::Transform::transform_is_simple && j == NUM_PASSES - 1)
Traits::extractKey(dest) = bitsToKey(Traits::Transform::backward(keyToBits(Traits::extractKey(reader[i]))));
}
}
/// Если число проходов нечётное, то результирующий массив находится во временном буфере. Скопируем его на место исходного массива.
if (NUM_PASSES % 2)
memcpy(arr, swap_buffer, size * sizeof(Element));
allocator.deallocate(swap_buffer, size * sizeof(Element));
}
};
template <typename T>
typename std::enable_if<std::is_unsigned<T>::value && std::is_integral<T>::value, void>::type
radixSort(T * arr, size_t size)
{
return RadixSort<RadixSortUIntTraits<T>>::execute(arr, size);
}
template <typename T>
typename std::enable_if<std::is_signed<T>::value && std::is_integral<T>::value, void>::type
radixSort(T * arr, size_t size)
{
return RadixSort<RadixSortIntTraits<T>>::execute(arr, size);
}
template <typename T>
typename std::enable_if<std::is_floating_point<T>::value, void>::type
radixSort(T * arr, size_t size)
{
return RadixSort<RadixSortFloatTraits<T>>::execute(arr, size);
}

View File

@ -1,8 +1,12 @@
#pragma once
#include <string>
#include <DB/IO/ReadHelpers.h>
namespace DB
{
inline std::string escapeForFileName(const std::string & s)
{
std::string res;
@ -30,25 +34,13 @@ inline std::string escapeForFileName(const std::string & s)
return res;
}
inline char unhex(char c)
{
switch (c)
{
case '0' ... '9':
return c - '0';
case 'A' ... 'F':
return c - 'A' + 10;
default:
return 0;
}
}
inline std::string unescapeForFileName(const std::string & s)
{
std::string res;
const char * pos = s.data();
const char * end = pos + s.size();
while (pos != end)
{
if (*pos != '%')
@ -57,17 +49,19 @@ inline std::string unescapeForFileName(const std::string & s)
{
/// пропустим '%'
if (++pos == end) break;
char val = unhex(*pos) * 16;
if (++pos == end) break;
val += unhex(*pos);
res += val;
}
++pos;
}
return res;
}
}

View File

@ -104,6 +104,7 @@ struct SortCursorImpl
rows = all_columns[0]->size();
}
bool isFirst() const { return pos == 0; }
bool isLast() const { return pos + 1 >= rows; }
void next() { ++pos; }
};
@ -118,13 +119,13 @@ struct SortCursor
SortCursorImpl * operator-> () { return impl; }
const SortCursorImpl * operator-> () const { return impl; }
/// Инвертировано, чтобы из priority queue элементы вынимались в нужном порядке.
bool operator< (const SortCursor & rhs) const
/// Указанная строка данного курсора больше указанной строки другого курсора.
bool greaterAt(const SortCursor & rhs, size_t lhs_pos, size_t rhs_pos) const
{
for (size_t i = 0; i < impl->sort_columns_size; ++i)
{
int direction = impl->desc[i].direction;
int res = direction * impl->sort_columns[i]->compareAt(impl->pos, rhs.impl->pos, *(rhs.impl->sort_columns[i]), direction);
int res = direction * impl->sort_columns[i]->compareAt(lhs_pos, rhs_pos, *(rhs.impl->sort_columns[i]), direction);
if (res > 0)
return true;
if (res < 0)
@ -132,6 +133,27 @@ struct SortCursor
}
return impl->order > rhs.impl->order;
}
/// Проверяет, что все строки в текущем блоке данного курсора меньше или равны, чем все строки текущего блока другого курсора.
bool totallyLessOrEquals(const SortCursor & rhs) const
{
if (impl->rows == 0 || rhs.impl->rows == 0)
return false;
/// Последняя строка данного курсора не больше первой строки другого.
return !greaterAt(rhs, impl->rows - 1, 0);
}
bool greater(const SortCursor & rhs) const
{
return greaterAt(rhs, impl->pos, rhs.impl->pos);
}
/// Инвертировано, чтобы из priority queue элементы вынимались в порядке по возрастанию.
bool operator< (const SortCursor & rhs) const
{
return greater(rhs);
}
};
@ -144,8 +166,7 @@ struct SortCursorWithCollation
SortCursorImpl * operator-> () { return impl; }
const SortCursorImpl * operator-> () const { return impl; }
/// Инвертировано, чтобы из priority queue элементы вынимались в нужном порядке.
bool operator< (const SortCursorWithCollation & rhs) const
bool greaterAt(const SortCursorWithCollation & rhs, size_t lhs_pos, size_t rhs_pos) const
{
for (size_t i = 0; i < impl->sort_columns_size; ++i)
{
@ -154,10 +175,10 @@ struct SortCursorWithCollation
if (impl->need_collation[i])
{
const ColumnString & column_string = typeid_cast<const ColumnString &>(*impl->sort_columns[i]);
res = column_string.compareAtWithCollation(impl->pos, rhs.impl->pos, *(rhs.impl->sort_columns[i]), *impl->desc[i].collator);
res = column_string.compareAtWithCollation(lhs_pos, rhs_pos, *(rhs.impl->sort_columns[i]), *impl->desc[i].collator);
}
else
res = impl->sort_columns[i]->compareAt(impl->pos, rhs.impl->pos, *(rhs.impl->sort_columns[i]), direction);
res = impl->sort_columns[i]->compareAt(lhs_pos, rhs_pos, *(rhs.impl->sort_columns[i]), direction);
res *= direction;
if (res > 0)
@ -167,6 +188,25 @@ struct SortCursorWithCollation
}
return impl->order > rhs.impl->order;
}
bool totallyLessOrEquals(const SortCursorWithCollation & rhs) const
{
if (impl->rows == 0 || rhs.impl->rows == 0)
return false;
/// Последняя строка данного курсора не больше первой строки другого.
return !greaterAt(rhs, impl->rows - 1, 0);
}
bool greater(const SortCursorWithCollation & rhs) const
{
return greaterAt(rhs, impl->pos, rhs.impl->pos);
}
bool operator< (const SortCursorWithCollation & rhs) const
{
return greater(rhs);
}
};
}

View File

@ -60,7 +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"; } };
/// Этот тип не поддерживается СУБД, но используется в некоторых внутренних преобразованиях.
template <> struct TypeName<long double>{ static std::string get() { return "long double"; } };
}

View File

@ -136,7 +136,7 @@ private:
void initQueue(std::priority_queue<TSortCursor> & queue);
template <typename TSortCursor>
void merge(ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue);
void merge(Block & merged_block, ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue);
Logger * log = &Logger::get("MergingSortedBlockInputStream");

View File

@ -4,9 +4,12 @@
#include <DB/Dictionaries/IDictionarySource.h>
#include <DB/Dictionaries/DictionaryStructure.h>
#include <DB/Common/HashTable/HashMap.h>
#include <DB/Common/ArenaWithFreeLists.h>
#include <DB/Columns/ColumnString.h>
#include <ext/scope_guard.hpp>
#include <ext/bit_cast.hpp>
#include <ext/range.hpp>
#include <ext/size.hpp>
#include <ext/map.hpp>
#include <Poco/RWLock.h>
#include <cmath>
@ -16,6 +19,7 @@
#include <map>
#include <tuple>
namespace DB
{
@ -49,7 +53,7 @@ public:
std::string getTypeName() const override { return "Cache"; }
std::size_t getBytesAllocated() const override { return bytes_allocated; }
std::size_t getBytesAllocated() const override { return bytes_allocated + (string_arena ? string_arena->size() : 0); }
std::size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); }
@ -90,11 +94,13 @@ 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, out, [&] (const std::size_t) { return null_value; });
}
#define DECLARE_MULTIPLE_GETTER(TYPE)\
void get##TYPE(const std::string & attribute_name, const PODArray<id_t> & ids, PODArray<TYPE> & out) const override\
#define DECLARE(TYPE)\
void get##TYPE(const std::string & attribute_name, const PODArray<id_t> & ids, PODArray<TYPE> & out) const\
{\
auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeUnderlyingType::TYPE)\
@ -103,20 +109,22 @@ public:
ErrorCodes::TYPE_MISMATCH\
};\
\
getItems<TYPE>(attribute, ids, out);\
const auto null_value = std::get<TYPE>(attribute.null_values);\
\
getItems<TYPE>(attribute, ids, out, [&] (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 PODArray<id_t> & ids, ColumnString * out) const override
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
DECLARE(UInt64)
DECLARE(Int8)
DECLARE(Int16)
DECLARE(Int32)
DECLARE(Int64)
DECLARE(Float32)
DECLARE(Float64)
#undef DECLARE
void getString(const std::string & attribute_name, const PODArray<id_t> & ids, ColumnString * out) const
{
auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeUnderlyingType::String)
@ -125,13 +133,15 @@ public:
ErrorCodes::TYPE_MISMATCH
};
getItems(attribute, ids, out);
const auto null_value = StringRef{std::get<String>(attribute.null_values)};
getItems(attribute, ids, out, [&] (const std::size_t) { return null_value; });
}
#define DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(TYPE)\
#define DECLARE(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const PODArray<id_t> & ids, const PODArray<TYPE> & def,\
PODArray<TYPE> & out) const override\
PODArray<TYPE> & out) const\
{\
auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeUnderlyingType::TYPE)\
@ -140,22 +150,22 @@ public:
ErrorCodes::TYPE_MISMATCH\
};\
\
getItems<TYPE>(attribute, ids, out, &def);\
getItems<TYPE>(attribute, ids, out, [&] (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
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
DECLARE(UInt64)
DECLARE(Int8)
DECLARE(Int16)
DECLARE(Int32)
DECLARE(Int64)
DECLARE(Float32)
DECLARE(Float64)
#undef DECLARE
void getString(
const std::string & attribute_name, const PODArray<id_t> & ids, const ColumnString * const def,
ColumnString * const out) const override
ColumnString * const out) const
{
auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeUnderlyingType::String)
@ -164,7 +174,45 @@ public:
ErrorCodes::TYPE_MISMATCH
};
getItems(attribute, ids, out, def);
getItems(attribute, ids, out, [&] (const std::size_t row) { return def->getDataAt(row); });
}
#define DECLARE(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const PODArray<id_t> & ids, const TYPE def, PODArray<TYPE> & out) 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, out, [&] (const std::size_t) { return def; });\
}
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
DECLARE(UInt64)
DECLARE(Int8)
DECLARE(Int16)
DECLARE(Int32)
DECLARE(Int64)
DECLARE(Float32)
DECLARE(Float64)
#undef DECLARE
void getString(
const std::string & attribute_name, const PODArray<id_t> & ids, const String & def,
ColumnString * const out) 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(attribute, ids, out, [&] (const std::size_t) { return StringRef{def}; });
}
void has(const PODArray<id_t> & ids, PODArray<UInt8> & out) const override
@ -178,9 +226,9 @@ public:
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))
for (const auto row : ext::range(0, rows))
{
const auto id = ids[i];
const auto id = ids[row];
const auto cell_idx = getCellIdx(id);
const auto & cell = cells[cell_idx];
@ -189,9 +237,9 @@ public:
* 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);
outdated_ids[id].push_back(row);
else
out[i] = !cell.isDefault();
out[row] = !cell.isDefault();
}
}
@ -207,11 +255,11 @@ public:
/// request new values
update(required_ids, [&] (const auto id, const auto) {
for (const auto out_idx : outdated_ids[id])
out[out_idx] = true;
for (const auto row : outdated_ids[id])
out[row] = true;
}, [&] (const auto id, const auto) {
for (const auto out_idx : outdated_ids[id])
out[out_idx] = false;
for (const auto row : outdated_ids[id])
out[row] = false;
});
}
@ -342,16 +390,17 @@ private:
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);
if (!string_arena)
string_arena = std::make_unique<ArenaWithFreeLists>();
break;
}
return attr;
}
template <typename T>
template <typename T, typename DefaultGetter>
void getItems(
attribute_t & attribute, const PODArray<id_t> & ids, PODArray<T> & out,
const PODArray<T> * const def = nullptr) const
attribute_t & attribute, const PODArray<id_t> & ids, PODArray<T> & out, DefaultGetter && get_default) const
{
/// Mapping: <id> -> { all indices `i` of `ids` such that `ids[i]` = <id> }
MapType<std::vector<std::size_t>> outdated_ids;
@ -363,9 +412,9 @@ private:
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))
for (const auto row : ext::range(0, rows))
{
const auto id = ids[i];
const auto id = ids[row];
const auto cell_idx = getCellIdx(id);
const auto & cell = cells[cell_idx];
@ -374,9 +423,9 @@ private:
* 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);
outdated_ids[id].push_back(row);
else
out[i] = def && cell.isDefault() ? (*def)[i] : attribute_array[cell_idx];
out[row] = cell.isDefault() ? get_default(row) : attribute_array[cell_idx];
}
}
@ -394,21 +443,17 @@ private:
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;
for (const auto row : outdated_ids[id])
out[row] = 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;
for (const auto row : outdated_ids[id])
out[row] = get_default(row);
});
}
template <typename DefaultGetter>
void getItems(
attribute_t & attribute, const PODArray<id_t> & ids, ColumnString * out,
const ColumnString * const def = nullptr) const
attribute_t & attribute, const PODArray<id_t> & ids, ColumnString * out, DefaultGetter && get_default) const
{
const auto rows = ext::size(ids);
@ -425,9 +470,9 @@ private:
const auto now = std::chrono::system_clock::now();
/// fetch up-to-date values, discard on fail
for (const auto i : ext::range(0, rows))
for (const auto row : ext::range(0, rows))
{
const auto id = ids[i];
const auto id = ids[row];
const auto cell_idx = getCellIdx(id);
const auto & cell = cells[cell_idx];
@ -438,7 +483,7 @@ private:
}
else
{
const auto string_ref = def && cell.isDefault() ? def->getDataAt(i) : attribute_array[cell_idx];
const auto string_ref = cell.isDefault() ? get_default(row) : attribute_array[cell_idx];
out->insertData(string_ref.data, string_ref.size);
}
}
@ -466,17 +511,17 @@ private:
const Poco::ScopedReadRWLock read_lock{rw_lock};
const auto now = std::chrono::system_clock::now();
for (const auto i : ext::range(0, ids.size()))
for (const auto row : ext::range(0, ids.size()))
{
const auto id = ids[i];
const auto id = ids[row];
const auto cell_idx = getCellIdx(id);
const auto & cell = cells[cell_idx];
if (cell.id != id || cell.expiresAt() < now)
outdated_ids[id].push_back(i);
outdated_ids[id].push_back(row);
else
{
const auto string_ref = def && cell.isDefault() ? def->getDataAt(i) : attribute_array[cell_idx];
const auto string_ref = cell.isDefault() ? get_default(row) : attribute_array[cell_idx];
map[id] = String{string_ref};
total_length += string_ref.size + 1;
}
@ -499,22 +544,22 @@ private:
map[id] = String{attribute_value};
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();
for (const auto row : outdated_ids[id])
total_length += get_default(row).size + 1;
});
}
out->getChars().reserve(total_length);
const auto & null_value = std::get<String>(attribute.null_values);
// const auto & null_value = std::get<String>(attribute.null_values);
for (const auto id : ids)
for (const auto row : ext::range(0, ext::size(ids)))
{
const auto id = ids[row];
const auto it = map.find(id);
/// @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());
const auto string_ref = it != std::end(map) ? StringRef{it->second} : get_default(row);
out->insertData(string_ref.data, string_ref.size);
}
}
@ -642,14 +687,14 @@ private:
{
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};
if (string_ref.data != null_value_ref.data())
{
if (string_ref.data)
string_arena->free(string_ref.data, string_ref.size);
string_ref = StringRef{null_value_ref};
string_ref = StringRef{null_value_ref};
}
break;
}
@ -675,21 +720,17 @@ private:
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};
}
/// free memory unless it points to a null_value
if (string_ref.data && string_ref.data != null_value_ref.data())
string_arena->free(string_ref.data, string_ref.size);
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;
auto string_ptr = string_arena->alloc(size + 1);
std::copy(string.data(), string.data() + size + 1, string_ptr);
string_ref = StringRef{string_ptr, size};
}
else
string_ref = {};
@ -744,6 +785,7 @@ private:
mutable std::vector<attribute_t> attributes;
mutable std::vector<cell_metadata_t> cells;
attribute_t * hierarchical_attribute = nullptr;
std::unique_ptr<ArenaWithFreeLists> string_arena;
mutable std::mt19937_64 rnd_engine{getSeed()};

View File

@ -9,6 +9,7 @@
#include <DB/Common/HashTable/HashMap.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Core/StringRef.h>
#include <ext/enumerate.hpp>
#include <ext/scope_guard.hpp>
#include <ext/bit_cast.hpp>
#include <ext/range.hpp>
@ -20,7 +21,6 @@
#include <vector>
#include <map>
#include <tuple>
#include <DB/DataStreams/NullBlockInputStream.h>
namespace DB
@ -58,7 +58,8 @@ public:
std::size_t getBytesAllocated() const override
{
return bytes_allocated + key_size_is_fixed ? fixed_size_keys_pool->size() : keys_pool->size();
return bytes_allocated + (key_size_is_fixed ? fixed_size_keys_pool->size() : keys_pool->size()) +
(string_arena ? string_arena->size() : 0);
}
std::size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); }
@ -96,12 +97,12 @@ public:
return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective;
}
#define DECLARE_MULTIPLE_GETTER(TYPE)\
#define DECLARE(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
PODArray<TYPE> & out) const\
{\
validateKeyTypes(key_types);\
dict_struct.validateKeyTypes(key_types);\
\
auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeUnderlyingType::TYPE)\
@ -110,24 +111,26 @@ public:
ErrorCodes::TYPE_MISMATCH\
};\
\
getItems<TYPE>(attribute, key_columns, out);\
const auto null_value = std::get<TYPE>(attribute.null_values);\
\
getItems<TYPE>(attribute, key_columns, out, [&] (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
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
DECLARE(UInt64)
DECLARE(Int8)
DECLARE(Int16)
DECLARE(Int32)
DECLARE(Int64)
DECLARE(Float32)
DECLARE(Float64)
#undef DECLARE
void getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
ColumnString * out) const
{
validateKeyTypes(key_types);
dict_struct.validateKeyTypes(key_types);
auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeUnderlyingType::String)
@ -136,15 +139,17 @@ public:
ErrorCodes::TYPE_MISMATCH
};
getItems(attribute, key_columns, out);
const auto null_value = StringRef{std::get<String>(attribute.null_values)};
getItems(attribute, key_columns, out, [&] (const std::size_t) { return null_value; });
}
#define DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(TYPE)\
#define DECLARE(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);\
dict_struct.validateKeyTypes(key_types);\
\
auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeUnderlyingType::TYPE)\
@ -153,24 +158,24 @@ public:
ErrorCodes::TYPE_MISMATCH\
};\
\
getItems<TYPE>(attribute, key_columns, out, &def);\
getItems<TYPE>(attribute, key_columns, out, [&] (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
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
DECLARE(UInt64)
DECLARE(Int8)
DECLARE(Int16)
DECLARE(Int32)
DECLARE(Int64)
DECLARE(Float32)
DECLARE(Float64)
#undef DECLARE
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);
dict_struct.validateKeyTypes(key_types);
auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeUnderlyingType::String)
@ -179,12 +184,55 @@ public:
ErrorCodes::TYPE_MISMATCH
};
getItems(attribute, key_columns, out, def);
getItems(attribute, key_columns, out, [&] (const std::size_t row) { return def->getDataAt(row); });
}
#define DECLARE(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const TYPE def, PODArray<TYPE> & out) const\
{\
dict_struct.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, [&] (const std::size_t) { return def; });\
}
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
DECLARE(UInt64)
DECLARE(Int8)
DECLARE(Int16)
DECLARE(Int32)
DECLARE(Int64)
DECLARE(Float32)
DECLARE(Float64)
#undef DECLARE
void getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const String & def, ColumnString * const out) const
{
dict_struct.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, [&] (const std::size_t) { return StringRef{def}; });
}
void has(const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types, PODArray<UInt8> & out) const
{
validateKeyTypes(key_types);
dict_struct.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;
@ -362,60 +410,18 @@ private:
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);
if (!string_arena)
string_arena = std::make_unique<ArenaWithFreeLists>();
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>
template <typename T, typename DefaultGetter>
void getItems(
attribute_t & attribute, const ConstColumnPlainPtrs & key_columns, PODArray<T> & out,
const PODArray<T> * const def = nullptr) const
DefaultGetter && get_default) const
{
/// Mapping: <key> -> { all indices `i` of `key_columns` such that `key_columns[i]` = <key> }
MapType<std::vector<std::size_t>> outdated_keys;
@ -447,7 +453,7 @@ private:
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];
out[row] = cell.isDefault() ? get_default(row) : attribute_array[cell_idx];
}
}
@ -463,21 +469,18 @@ private:
/// 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;
for (const auto row : outdated_keys[key])
out[row] = attribute_array[cell_idx];
}, [&] (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;
for (const auto row : outdated_keys[key])
out[row] = get_default(row);
});
}
template <typename DefaultGetter>
void getItems(
attribute_t & attribute, const ConstColumnPlainPtrs & key_columns, ColumnString * out,
const ColumnString * const def = nullptr) const
DefaultGetter && get_default) const
{
const auto rows = key_columns.front()->size();
/// save on some allocations
@ -512,7 +515,7 @@ private:
}
else
{
const auto string_ref = def && cell.isDefault() ? def->getDataAt(row) : attribute_array[cell_idx];
const auto string_ref = cell.isDefault() ? get_default(row) : attribute_array[cell_idx];
out->insertData(string_ref.data, string_ref.size);
}
}
@ -553,7 +556,7 @@ private:
outdated_keys[key].push_back(row);
else
{
const auto string_ref = def && cell.isDefault() ? def->getDataAt(row) : attribute_array[cell_idx];
const auto string_ref = cell.isDefault() ? get_default(row) : attribute_array[cell_idx];
map[key] = String{string_ref};
total_length += string_ref.size + 1;
}
@ -576,22 +579,19 @@ private:
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();
for (const auto row : outdated_keys[key])
total_length += get_default(row).size + 1;
});
}
out->getChars().reserve(total_length);
const auto & null_value = std::get<String>(attribute.null_values);
for (const auto key : keys_array)
for (const auto row : ext::range(0, ext::size(keys_array)))
{
const auto key = keys_array[row];
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());
const auto string_ref = it != std::end(map) ? StringRef{it->second} : get_default(row);
out->insertData(string_ref.data, string_ref.size);
}
}
@ -604,9 +604,9 @@ private:
auto stream = source_ptr->loadKeys(in_key_columns, in_requested_rows);
stream->readPrefix();
MapType<UInt8> remaining_keys{in_requested_rows.size()};
MapType<bool> remaining_keys{in_requested_rows.size()};
for (const auto row : in_requested_rows)
remaining_keys.insert({ in_keys[row], 0 });
remaining_keys.insert({ in_keys[row], false });
std::uniform_int_distribution<std::uint64_t> distribution{
dict_lifetime.min_sec,
@ -679,7 +679,7 @@ private:
/// inform caller
on_cell_updated(key, cell_idx);
/// mark corresponding id as found
remaining_keys[key] = 1;
remaining_keys[key] = true;
}
}
@ -755,14 +755,14 @@ private:
{
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};
if (string_ref.data != null_value_ref.data())
{
if (string_ref.data)
string_arena->free(string_ref.data, string_ref.size);
string_ref = StringRef{null_value_ref};
string_ref = StringRef{null_value_ref};
}
break;
}
@ -788,21 +788,17 @@ private:
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};
}
/// free memory unless it points to a null_value
if (string_ref.data && string_ref.data != null_value_ref.data())
string_arena->free(string_ref.data, string_ref.size);
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;
auto string_ptr = string_arena->alloc(size + 1);
std::copy(string.data(), string.data() + size + 1, string_ptr);
string_ref = StringRef{string_ptr, size};
}
else
string_ref = {};
@ -913,7 +909,7 @@ private:
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;
const std::string key_description{createKeyDescription(dict_struct)};
const std::string key_description{dict_struct.getKeyDescription()};
mutable Poco::RWLock rw_lock;
const std::size_t size;
@ -927,6 +923,7 @@ private:
std::make_unique<ArenaWithFreeLists>();
std::unique_ptr<SmallObjectPool> fixed_size_keys_pool = key_size_is_fixed ?
std::make_unique<SmallObjectPool>(key_size) : nullptr;
std::unique_ptr<ArenaWithFreeLists> string_arena;
mutable std::mt19937_64 rnd_engine{getSeed()};

View File

@ -23,7 +23,7 @@ public:
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)}
require_nonempty(require_nonempty)
{
createAttributes();
@ -82,12 +82,12 @@ public:
return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective;
}
#define DECLARE_MULTIPLE_GETTER(TYPE)\
#define DECLARE(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
PODArray<TYPE> & out) const\
{\
validateKeyTypes(key_types);\
dict_struct.validateKeyTypes(key_types);\
\
const auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeUnderlyingType::TYPE)\
@ -102,22 +102,22 @@ public:
[&] (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
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
DECLARE(UInt64)
DECLARE(Int8)
DECLARE(Int16)
DECLARE(Int32)
DECLARE(Int64)
DECLARE(Float32)
DECLARE(Float64)
#undef DECLARE
void getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
ColumnString * out) const
{
validateKeyTypes(key_types);
dict_struct.validateKeyTypes(key_types);
const auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeUnderlyingType::String)
@ -133,12 +133,12 @@ public:
[&] (const std::size_t) { return null_value; });
}
#define DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(TYPE)\
#define DECLARE(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);\
dict_struct.validateKeyTypes(key_types);\
\
const auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeUnderlyingType::TYPE)\
@ -151,22 +151,22 @@ public:
[&] (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
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
DECLARE(UInt64)
DECLARE(Int8)
DECLARE(Int16)
DECLARE(Int32)
DECLARE(Int64)
DECLARE(Float32)
DECLARE(Float64)
#undef DECLARE
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);
dict_struct.validateKeyTypes(key_types);
const auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeUnderlyingType::String)
@ -180,9 +180,56 @@ public:
[&] (const std::size_t row) { return def->getDataAt(row); });
}
#define DECLARE(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\
const TYPE def, PODArray<TYPE> & out) const\
{\
dict_struct.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) { return def; });\
}
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
DECLARE(UInt64)
DECLARE(Int8)
DECLARE(Int16)
DECLARE(Int32)
DECLARE(Int64)
DECLARE(Float32)
DECLARE(Float64)
#undef DECLARE
void getString(
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,
const String & def, ColumnString * const out) const
{
dict_struct.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) { return StringRef{def}; });
}
void has(const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types, PODArray<UInt8> & out) const
{
validateKeyTypes(key_types);
dict_struct.validateKeyTypes(key_types);
const auto & attribute = attributes.front();
@ -372,50 +419,6 @@ private:
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,
@ -542,7 +545,7 @@ private:
const DictionarySourcePtr source_ptr;
const DictionaryLifetime dict_lifetime;
const bool require_nonempty;
const std::string key_description;
const std::string key_description{dict_struct.getKeyDescription()};
std::map<std::string, std::size_t> attribute_index_by_name;
std::vector<attribute_t> attributes;

View File

@ -6,6 +6,7 @@
#include <DB/IO/WriteBuffer.h>
#include <DB/IO/WriteHelpers.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <ext/range.hpp>
#include <vector>
#include <string>
#include <map>
@ -184,6 +185,53 @@ struct DictionaryStructure final
throw Exception{"Dictionary has no attributes defined", ErrorCodes::BAD_ARGUMENTS};
}
void validateKeyTypes(const DataTypes & key_types) const
{
if (key_types.size() != key->size())
throw Exception{
"Key structure does not match, expected " + getKeyDescription(),
ErrorCodes::TYPE_MISMATCH
};
for (const auto i : ext::range(0, key_types.size()))
{
const auto & expected_type = (*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
};
}
}
std::string getKeyDescription() const
{
if (id)
return "UInt64";
std::ostringstream out;
out << '(';
auto first = true;
for (const auto & key_i : *key)
{
if (!first)
out << ", ";
first = false;
out << key_i.type->getName();
}
out << ')';
return out.str();
}
bool isKeySizeFixed() const
{
if (!key)

View File

@ -93,8 +93,8 @@ public:
[&] (const std::size_t) { return null_value; });
}
#define DECLARE_MULTIPLE_GETTER(TYPE)\
void get##TYPE(const std::string & attribute_name, const PODArray<id_t> & ids, PODArray<TYPE> & out) const override\
#define DECLARE(TYPE)\
void get##TYPE(const std::string & attribute_name, const PODArray<id_t> & ids, PODArray<TYPE> & out) const\
{\
const auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeUnderlyingType::TYPE)\
@ -109,18 +109,18 @@ public:
[&] (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 PODArray<id_t> & ids, ColumnString * out) const override
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
DECLARE(UInt64)
DECLARE(Int8)
DECLARE(Int16)
DECLARE(Int32)
DECLARE(Int64)
DECLARE(Float32)
DECLARE(Float64)
#undef DECLARE
void getString(const std::string & attribute_name, const PODArray<id_t> & ids, ColumnString * out) const
{
const auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeUnderlyingType::String)
@ -136,10 +136,10 @@ public:
[&] (const std::size_t) { return null_value; });
}
#define DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(TYPE)\
#define DECLARE(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const PODArray<id_t> & ids, const PODArray<TYPE> & def,\
PODArray<TYPE> & out) const override\
PODArray<TYPE> & out) const\
{\
const auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeUnderlyingType::TYPE)\
@ -152,20 +152,20 @@ public:
[&] (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
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
DECLARE(UInt64)
DECLARE(Int8)
DECLARE(Int16)
DECLARE(Int32)
DECLARE(Int64)
DECLARE(Float32)
DECLARE(Float64)
#undef DECLARE
void getString(
const std::string & attribute_name, const PODArray<id_t> & ids, const ColumnString * const def,
ColumnString * const out) const override
ColumnString * const out) const
{
const auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeUnderlyingType::String)
@ -179,6 +179,49 @@ public:
[&] (const std::size_t row) { return def->getDataAt(row); });
}
#define DECLARE(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const PODArray<id_t> & ids, const TYPE def,\
PODArray<TYPE> & out) const\
{\
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) { return def; });\
}
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
DECLARE(UInt64)
DECLARE(Int8)
DECLARE(Int16)
DECLARE(Int32)
DECLARE(Int64)
DECLARE(Float32)
DECLARE(Float64)
#undef DECLARE
void getString(
const std::string & attribute_name, const PODArray<id_t> & ids, const String & def,
ColumnString * const out) const
{
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) { return StringRef{def}; });
}
void has(const PODArray<id_t> & ids, PODArray<UInt8> & out) const override
{
const auto & attribute = attributes.front();
@ -354,11 +397,13 @@ private:
{
const auto & attr = *std::get<ContainerPtrType<T>>(attribute.arrays);
const auto rows = ext::size(ids);
using null_value_type = std::conditional_t<std::is_same<T, StringRef>::value, String, T>;
const auto null_value = std::get<null_value_type>(attribute.null_values);
for (const auto i : ext::range(0, rows))
for (const auto row : ext::range(0, rows))
{
const auto id = ids[i];
set_value(i, id < attr.size() ? attr[id] : get_default(i));
const auto id = ids[row];
set_value(row, id < ext::size(attr) && attr[id] != null_value ? attr[id] : get_default(row));
}
query_count.fetch_add(rows, std::memory_order_relaxed);

View File

@ -88,8 +88,8 @@ public:
[&] (const std::size_t) { return null_value; });
}
#define DECLARE_MULTIPLE_GETTER(TYPE)\
void get##TYPE(const std::string & attribute_name, const PODArray<id_t> & ids, PODArray<TYPE> & out) const override\
#define DECLARE(TYPE)\
void get##TYPE(const std::string & attribute_name, const PODArray<id_t> & ids, PODArray<TYPE> & out) const\
{\
const auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeUnderlyingType::TYPE)\
@ -104,18 +104,18 @@ public:
[&] (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 PODArray<id_t> & ids, ColumnString * out) const override
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
DECLARE(UInt64)
DECLARE(Int8)
DECLARE(Int16)
DECLARE(Int32)
DECLARE(Int64)
DECLARE(Float32)
DECLARE(Float64)
#undef DECLARE
void getString(const std::string & attribute_name, const PODArray<id_t> & ids, ColumnString * out) const
{
const auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeUnderlyingType::String)
@ -131,10 +131,10 @@ public:
[&] (const std::size_t) { return null_value; });
}
#define DECLARE_MULTIPLE_GETTER_WITH_DEFAULT(TYPE)\
#define DECLARE(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const PODArray<id_t> & ids, const PODArray<TYPE> & def,\
PODArray<TYPE> & out) const override\
PODArray<TYPE> & out) const\
{\
const auto & attribute = getAttribute(attribute_name);\
if (attribute.type != AttributeUnderlyingType::TYPE)\
@ -147,20 +147,20 @@ public:
[&] (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
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
DECLARE(UInt64)
DECLARE(Int8)
DECLARE(Int16)
DECLARE(Int32)
DECLARE(Int64)
DECLARE(Float32)
DECLARE(Float64)
#undef DECLARE
void getString(
const std::string & attribute_name, const PODArray<id_t> & ids, const ColumnString * const def,
ColumnString * const out) const override
ColumnString * const out) const
{
const auto & attribute = getAttribute(attribute_name);
if (attribute.type != AttributeUnderlyingType::String)
@ -174,6 +174,48 @@ public:
[&] (const std::size_t row) { return def->getDataAt(row); });
}
#define DECLARE(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const PODArray<id_t> & ids, const TYPE & def, PODArray<TYPE> & out) const\
{\
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) { return def; });\
}
DECLARE(UInt8)
DECLARE(UInt16)
DECLARE(UInt32)
DECLARE(UInt64)
DECLARE(Int8)
DECLARE(Int16)
DECLARE(Int32)
DECLARE(Int64)
DECLARE(Float32)
DECLARE(Float64)
#undef DECLARE
void getString(
const std::string & attribute_name, const PODArray<id_t> & ids, const String & def,
ColumnString * const out) const
{
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) { return StringRef{def}; });
}
void has(const PODArray<id_t> & ids, PODArray<UInt8> & out) const override
{
const auto & attribute = attributes.front();

View File

@ -59,6 +59,10 @@ struct IDictionary : IDictionaryBase
{
virtual bool hasHierarchy() const = 0;
virtual void toParent(const PODArray<id_t> & ids, PODArray<id_t> & out) const = 0;
virtual void has(const PODArray<id_t> & ids, PODArray<UInt8> & out) const = 0;
/// do not call unless you ensure that hasHierarchy() returns true
id_t toParent(id_t id) const
{
@ -70,8 +74,6 @@ struct IDictionary : IDictionaryBase
return out.front();
}
virtual void toParent(const PODArray<id_t> & ids, PODArray<id_t> & out) const = 0;
bool in(id_t child_id, const id_t ancestor_id) const
{
while (child_id != 0 && child_id != ancestor_id)
@ -79,34 +81,6 @@ struct IDictionary : IDictionaryBase
return child_id != 0;
}
/// return mapped values for a collection of identifiers
virtual void getUInt8(const std::string & attr_name, const PODArray<id_t> & ids, PODArray<UInt8> & out) const = 0;
virtual void getUInt16(const std::string & attr_name, const PODArray<id_t> & ids, PODArray<UInt16> & out) const = 0;
virtual void getUInt32(const std::string & attr_name, const PODArray<id_t> & ids, PODArray<UInt32> & out) const = 0;
virtual void getUInt64(const std::string & attr_name, const PODArray<id_t> & ids, PODArray<UInt64> & out) const = 0;
virtual void getInt8(const std::string & attr_name, const PODArray<id_t> & ids, PODArray<Int8> & out) const = 0;
virtual void getInt16(const std::string & attr_name, const PODArray<id_t> & ids, PODArray<Int16> & out) const = 0;
virtual void getInt32(const std::string & attr_name, const PODArray<id_t> & ids, PODArray<Int32> & out) const = 0;
virtual void getInt64(const std::string & attr_name, const PODArray<id_t> & ids, PODArray<Int64> & out) const = 0;
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

@ -5,6 +5,8 @@
#include <Poco/Util/AbstractConfiguration.h>
#include <mongo/client/dbclient.h>
#include <ext/collection_cast.hpp>
#include <ext/enumerate.hpp>
#include <ext/size.hpp>
namespace DB
@ -13,6 +15,8 @@ namespace DB
/// Allows loading dictionaries from a MongoDB collection
class MongoDBDictionarySource final : public IDictionarySource
{
static const auto max_block_size = 8192;
MongoDBDictionarySource(
const DictionaryStructure & dict_struct, const std::string & host, const std::string & port,
const std::string & user, const std::string & password,
@ -90,7 +94,7 @@ public:
{
return new MongoDBBlockInputStream{
connection.query(db + '.' + collection, {}, 0, 0, &fields_to_query),
sample_block, 8192
sample_block, max_block_size
};
}
@ -98,23 +102,63 @@ public:
BlockInputStreamPtr loadIds(const std::vector<std::uint64_t> & ids) override
{
if (dict_struct.key)
throw Exception{"Complex key not supported", ErrorCodes::UNSUPPORTED_METHOD};
if (!dict_struct.id)
throw Exception{"'id' is required for selective loading", ErrorCodes::UNSUPPORTED_METHOD};
/// mongo::BSONObj has shitty design and does not use fixed width integral types
const auto ids_enumeration = BSON(
const auto query = 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),
sample_block, 8192
connection.query(db + '.' + collection, query, 0, 0, &fields_to_query),
sample_block, max_block_size
};
}
BlockInputStreamPtr loadKeys(
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows) override
{
throw Exception{"Method unsupported", ErrorCodes::NOT_IMPLEMENTED};
if (!dict_struct.key)
throw Exception{"'key' is required for selective loading", ErrorCodes::UNSUPPORTED_METHOD};
std::string query_string;
{
WriteBufferFromString out{query_string};
writeString("{$or:[", out);
auto first = true;
for (const auto row : requested_rows)
{
if (!first)
writeChar(',', out);
first = false;
writeChar('{', out);
for (const auto idx_key : ext::enumerate(*dict_struct.key))
{
if (idx_key.first != 0)
writeChar(',', out);
writeString(idx_key.second.name, out);
writeChar(':', out);
idx_key.second.type->serializeTextQuoted((*key_columns[idx_key.first])[row], out);
}
writeChar('}', out);
}
writeString("]}", out);
}
return new MongoDBBlockInputStream{
connection.query(db + '.' + collection, query_string, 0, 0, &fields_to_query),
sample_block, max_block_size
};
}
/// @todo: for MongoDB, modification date can somehow be determined from the `_id` object field

View File

@ -25,7 +25,8 @@ private:
public:
FunctionFactory();
FunctionPtr get(const String & name, const Context & context) const;
FunctionPtr get(const String & name, const Context & context) const; /// Кидает исключение, если не нашлось.
FunctionPtr tryGet(const String & name, const Context & context) const; /// Возвращает nullptr, если не нашлось.
template <typename F> void registerFunction()
{

View File

@ -5,6 +5,7 @@
#include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/Functions/IFunction.h>
#include <DB/Functions/NumberTraits.h>
#include <DB/Core/FieldVisitors.h>
namespace DB
@ -713,6 +714,10 @@ public:
};
template <typename FunctionName>
struct FunctionUnaryArithmeticMonotonicity;
template <template <typename> class Op, typename Name>
class FunctionUnaryArithmetic : public IFunction
{
@ -815,6 +820,16 @@ public:
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
bool hasInformationAboutMonotonicity() const override
{
return FunctionUnaryArithmeticMonotonicity<Name>::has();
}
Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override
{
return FunctionUnaryArithmeticMonotonicity<Name>::get(left, right);
}
};
@ -841,7 +856,7 @@ typedef FunctionBinaryArithmetic<MinusImpl, NameMinus> FunctionMinus;
typedef FunctionBinaryArithmetic<MultiplyImpl, NameMultiply> FunctionMultiply;
typedef FunctionBinaryArithmetic<DivideFloatingImpl, NameDivideFloating> FunctionDivideFloating;
typedef FunctionBinaryArithmetic<DivideIntegralImpl, NameDivideIntegral> FunctionDivideIntegral;
typedef FunctionBinaryArithmetic<DivideIntegralOrZeroImpl, NameDivideIntegralOrZero> FunctionDivideIntegralOrZero;
typedef FunctionBinaryArithmetic<DivideIntegralOrZeroImpl, NameDivideIntegralOrZero> FunctionDivideIntegralOrZero;
typedef FunctionBinaryArithmetic<ModuloImpl, NameModulo> FunctionModulo;
typedef FunctionUnaryArithmetic<NegateImpl, NameNegate> FunctionNegate;
typedef FunctionUnaryArithmetic<AbsImpl, NameAbs> FunctionAbs;
@ -854,6 +869,41 @@ typedef FunctionBinaryArithmetic<BitShiftRightImpl, NameBitShiftRight> Functi
typedef FunctionBinaryArithmetic<LeastImpl, NameLeast> FunctionLeast;
typedef FunctionBinaryArithmetic<GreatestImpl, NameGreatest> FunctionGreatest;
/// Свойства монотонности для некоторых функций.
template <> struct FunctionUnaryArithmeticMonotonicity<NameNegate>
{
static bool has() { return true; }
static IFunction::Monotonicity get(const Field & left, const Field & right)
{
return { true, false };
}
};
template <> struct FunctionUnaryArithmeticMonotonicity<NameAbs>
{
static bool has() { return true; }
static IFunction::Monotonicity get(const Field & left, const Field & right)
{
Float64 left_float = left.isNull() ? -std::numeric_limits<Float64>::infinity() : apply_visitor(FieldVisitorConvertToNumber<Float64>(), left);
Float64 right_float = right.isNull() ? std::numeric_limits<Float64>::infinity() : apply_visitor(FieldVisitorConvertToNumber<Float64>(), right);
if ((left_float < 0 && right_float > 0) || (left_float > 0 && right_float < 0))
return {};
return { true, (left_float > 0) };
}
};
template <> struct FunctionUnaryArithmeticMonotonicity<NameBitNot>
{
static bool has() { return false; }
static IFunction::Monotonicity get(const Field & left, const Field & right)
{
return {};
}
};
/// Оптимизации для целочисленного деления на константу.

View File

@ -11,7 +11,9 @@
#include <DB/Columns/ColumnFixedString.h>
#include <DB/Columns/ColumnConst.h>
#include <DB/Functions/IFunction.h>
#include <DB/Core/FieldVisitors.h>
#include <ext/range.hpp>
#include <type_traits>
namespace DB
@ -996,7 +998,7 @@ struct ConvertImpl<DataTypeFixedString, DataTypeString, Name>
/// Предварительное объявление.
struct NameToDate { static constexpr auto name = "toDate"; };
template <typename ToDataType, typename Name>
template <typename ToDataType, typename Name, typename Monotonic>
class FunctionConvert : public IFunction
{
public:
@ -1039,6 +1041,16 @@ public:
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
bool hasInformationAboutMonotonicity() const override
{
return Monotonic::has();
}
Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override
{
return Monotonic::get(type, left, right);
}
private:
template<typename ToDataType2 = ToDataType, typename Name2 = Name>
DataTypePtr getReturnTypeImpl(const DataTypes & arguments,
@ -1260,6 +1272,113 @@ private:
};
/// Монотонность.
struct PositiveMonotonicity
{
static bool has() { return true; }
static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right)
{
return { true };
}
};
template <typename T>
struct ToIntMonotonicity
{
static bool has() { return true; }
template <typename T2 = T>
static UInt64 divideByRangeOfType(typename std::enable_if_t<sizeof(T2) != sizeof(UInt64), UInt64> x) { return x >> (sizeof(T) * 8); };
template <typename T2 = T>
static UInt64 divideByRangeOfType(typename std::enable_if_t<sizeof(T2) == sizeof(UInt64), UInt64> x) { return 0; };
static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right)
{
size_t size_of_type = type.getSizeOfField();
/// Если тип расширяется, то функция монотонна.
if (sizeof(T) > size_of_type)
return { true };
/// Если тип совпадает - тоже.
if (typeid_cast<const typename DataTypeFromFieldType<T>::Type *>(&type))
return { true };
/// В других случаях, для неограниченного диапазона не знаем, будет ли функция монотонной.
if (left.isNull() || right.isNull())
return {};
/// Если преобразуем из float, то аргументы должны помещаться в тип результата.
if (typeid_cast<const DataTypeFloat32 *>(&type)
|| typeid_cast<const DataTypeFloat64 *>(&type))
{
Float64 left_float = left.get<Float64>();
Float64 right_float = right.get<Float64>();
if (left_float >= std::numeric_limits<T>::min() && left_float <= std::numeric_limits<T>::max()
&& right_float >= std::numeric_limits<T>::min() && right_float <= std::numeric_limits<T>::max())
return { true };
return {};
}
/// Если меняем знаковость типа или преобразуем из даты, даты-времени, то аргумент должен быть из одной половинки.
/// На всякий случай, в остальных случаях тоже будем этого требовать.
if ((left.get<Int64>() >= 0) != (right.get<Int64>() >= 0))
return {};
/// Если уменьшаем тип, то все биты кроме тех, которые в него помещаются, должны совпадать.
if (divideByRangeOfType(left.get<UInt64>()) != divideByRangeOfType(right.get<UInt64>()))
return {};
return { true };
}
};
/** Монотонность для функции toString определяем, в основном, для тестовых целей.
* Всерьёз вряд ли кто-нибудь рассчитывает на оптимизацию запросов с условиями toString(CounterID) = 34.
*/
struct ToStringMonotonicity
{
static bool has() { return true; }
static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right)
{
IFunction::Monotonicity positive = { .is_monotonic = true, .is_positive = true };
IFunction::Monotonicity not_monotonic;
/// Функция toString монотонна, если аргумент - Date или DateTime, или неотрицательные числа с одинаковым количеством знаков.
if (typeid_cast<const DataTypeDate *>(&type)
|| typeid_cast<const DataTypeDateTime *>(&type))
return positive;
if (left.isNull() || right.isNull())
return {};
if (left.getType() == Field::Types::UInt64
&& right.getType() == Field::Types::UInt64)
{
return (left.get<Int64>() == 0 && right.get<Int64>() == 0)
|| (floor(log10(left.get<UInt64>())) == floor(log10(right.get<UInt64>())))
? positive : not_monotonic;
}
if (left.getType() == Field::Types::Int64
&& right.getType() == Field::Types::Int64)
{
return (left.get<Int64>() == 0 && right.get<Int64>() == 0)
|| (left.get<Int64>() > 0 && right.get<Int64>() > 0 && floor(log10(left.get<Int64>())) == floor(log10(right.get<Int64>())))
? positive : not_monotonic;
}
return not_monotonic;
}
};
struct NameToUInt8 { static constexpr auto name = "toUInt8"; };
struct NameToUInt16 { static constexpr auto name = "toUInt16"; };
struct NameToUInt32 { static constexpr auto name = "toUInt32"; };
@ -1273,19 +1392,19 @@ struct NameToFloat64 { static constexpr auto name = "toFloat64"; };
struct NameToDateTime { static constexpr auto name = "toDateTime"; };
struct NameToString { static constexpr auto name = "toString"; };
typedef FunctionConvert<DataTypeUInt8, NameToUInt8> FunctionToUInt8;
typedef FunctionConvert<DataTypeUInt16, NameToUInt16> FunctionToUInt16;
typedef FunctionConvert<DataTypeUInt32, NameToUInt32> FunctionToUInt32;
typedef FunctionConvert<DataTypeUInt64, NameToUInt64> FunctionToUInt64;
typedef FunctionConvert<DataTypeInt8, NameToInt8> FunctionToInt8;
typedef FunctionConvert<DataTypeInt16, NameToInt16> FunctionToInt16;
typedef FunctionConvert<DataTypeInt32, NameToInt32> FunctionToInt32;
typedef FunctionConvert<DataTypeInt64, NameToInt64> FunctionToInt64;
typedef FunctionConvert<DataTypeFloat32, NameToFloat32> FunctionToFloat32;
typedef FunctionConvert<DataTypeFloat64, NameToFloat64> FunctionToFloat64;
typedef FunctionConvert<DataTypeDate, NameToDate> FunctionToDate;
typedef FunctionConvert<DataTypeDateTime, NameToDateTime> FunctionToDateTime;
typedef FunctionConvert<DataTypeString, NameToString> FunctionToString;
typedef FunctionConvert<DataTypeInt32, NameToUnixTimestamp> FunctionToUnixTimestamp;
typedef FunctionConvert<DataTypeUInt8, NameToUInt8, ToIntMonotonicity<UInt8>> FunctionToUInt8;
typedef FunctionConvert<DataTypeUInt16, NameToUInt16, ToIntMonotonicity<UInt16>> FunctionToUInt16;
typedef FunctionConvert<DataTypeUInt32, NameToUInt32, ToIntMonotonicity<UInt32>> FunctionToUInt32;
typedef FunctionConvert<DataTypeUInt64, NameToUInt64, ToIntMonotonicity<UInt64>> FunctionToUInt64;
typedef FunctionConvert<DataTypeInt8, NameToInt8, ToIntMonotonicity<Int8>> FunctionToInt8;
typedef FunctionConvert<DataTypeInt16, NameToInt16, ToIntMonotonicity<Int16>> FunctionToInt16;
typedef FunctionConvert<DataTypeInt32, NameToInt32, ToIntMonotonicity<Int32>> FunctionToInt32;
typedef FunctionConvert<DataTypeInt64, NameToInt64, ToIntMonotonicity<Int64>> FunctionToInt64;
typedef FunctionConvert<DataTypeFloat32, NameToFloat32, PositiveMonotonicity> FunctionToFloat32;
typedef FunctionConvert<DataTypeFloat64, NameToFloat64, PositiveMonotonicity> FunctionToFloat64;
typedef FunctionConvert<DataTypeDate, NameToDate, ToIntMonotonicity<UInt16>> FunctionToDate;
typedef FunctionConvert<DataTypeDateTime, NameToDateTime, ToIntMonotonicity<UInt32>> FunctionToDateTime;
typedef FunctionConvert<DataTypeString, NameToString, ToStringMonotonicity> FunctionToString;
typedef FunctionConvert<DataTypeInt32, NameToUnixTimestamp, ToIntMonotonicity<UInt32>> FunctionToUnixTimestamp;
}

View File

@ -48,80 +48,94 @@ namespace DB
#define TIME_SLOT_SIZE 1800
/** Всевозможные преобразования.
* Представляют собой две функции - от даты-с-временем (UInt32) и от даты (UInt16).
*
* Также для преобразования T определяется "фактор-преобразование" F.
* Это такое преобразование F, что его значение идентифицирует область монотонности T
* (при фиксированном значении F, преобразование T является монотонным).
*
* Или, образно, если T аналогично взятию остатка от деления, то F аналогично делению.
*
* Пример: для преобразования T "получить номер дня в месяце" (2015-02-03 -> 3),
* фактор-преобразованием F является "округлить до месяца" (2015-02-03 -> 2015-02-01).
*/
struct ToYearImpl
/// Это фактор-преобразование будет говорить, что функция монотонна всюду.
struct ZeroTransform
{
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(t); }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(DayNum_t(d)); }
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return 0; }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return 0; }
};
struct ToMonthImpl
struct ToDateImpl
{
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toMonth(t); }
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toMonth(DayNum_t(d)); }
};
struct ToDayOfMonthImpl
{
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfMonth(t); }
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfMonth(DayNum_t(d)); }
};
struct ToDayOfWeekImpl
{
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfWeek(t); }
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toDayOfWeek(DayNum_t(d)); }
};
struct ToHourImpl
{
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toHourInaccurate(t); }
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toDate(t);
}
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
throw Exception("Illegal type Date of argument for function toHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return d;
}
};
struct ToMinuteImpl
{
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toMinuteInaccurate(t); }
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
throw Exception("Illegal type Date of argument for function toMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
};
struct ToSecondImpl
{
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toSecondInaccurate(t); }
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
throw Exception("Illegal type Date of argument for function toSecond", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
using FactorTransform = ZeroTransform;
};
struct ToMondayImpl
{
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfWeek(remote_date_lut.toDayNum(t)); }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfWeek(DayNum_t(d)); }
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toFirstDayNumOfWeek(remote_date_lut.toDayNum(t));
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toFirstDayNumOfWeek(DayNum_t(d));
}
using FactorTransform = ZeroTransform;
};
struct ToStartOfMonthImpl
{
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfMonth(remote_date_lut.toDayNum(t)); }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfMonth(DayNum_t(d)); }
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toFirstDayNumOfMonth(remote_date_lut.toDayNum(t));
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toFirstDayNumOfMonth(DayNum_t(d));
}
using FactorTransform = ZeroTransform;
};
struct ToStartOfQuarterImpl
{
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfQuarter(remote_date_lut.toDayNum(t)); }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfQuarter(DayNum_t(d)); }
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toFirstDayNumOfQuarter(remote_date_lut.toDayNum(t));
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toFirstDayNumOfQuarter(DayNum_t(d));
}
using FactorTransform = ZeroTransform;
};
struct ToStartOfYearImpl
{
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfYear(remote_date_lut.toDayNum(t)); }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toFirstDayNumOfYear(DayNum_t(d)); }
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toFirstDayNumOfYear(remote_date_lut.toDayNum(t));
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toFirstDayNumOfYear(DayNum_t(d));
}
using FactorTransform = ZeroTransform;
};
@ -148,87 +162,250 @@ struct ToTimeImpl
{
throw Exception("Illegal type Date of argument for function toTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
using FactorTransform = ToDateImpl;
};
struct ToStartOfMinuteImpl
{
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toStartOfMinuteInaccurate(t); }
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toStartOfMinuteInaccurate(t);
}
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
throw Exception("Illegal type Date of argument for function toStartOfMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
using FactorTransform = ZeroTransform;
};
struct ToStartOfFiveMinuteImpl
{
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toStartOfFiveMinuteInaccurate(t); }
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toStartOfFiveMinuteInaccurate(t);
}
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
throw Exception("Illegal type Date of argument for function toStartOfFiveMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
using FactorTransform = ZeroTransform;
};
struct ToStartOfHourImpl
{
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toStartOfHourInaccurate(t); }
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toStartOfHourInaccurate(t);
}
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
throw Exception("Illegal type Date of argument for function toStartOfHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
using FactorTransform = ZeroTransform;
};
struct ToYearImpl
{
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toYear(t);
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toYear(DayNum_t(d));
}
using FactorTransform = ZeroTransform;
};
struct ToMonthImpl
{
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toMonth(t);
}
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toMonth(DayNum_t(d));
}
using FactorTransform = ToStartOfYearImpl;
};
struct ToDayOfMonthImpl
{
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toDayOfMonth(t);
}
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toDayOfMonth(DayNum_t(d));
}
using FactorTransform = ToStartOfMonthImpl;
};
struct ToDayOfWeekImpl
{
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toDayOfWeek(t);
}
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toDayOfWeek(DayNum_t(d));
}
using FactorTransform = ToMondayImpl;
};
struct ToHourImpl
{
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toHourInaccurate(t);
}
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
throw Exception("Illegal type Date of argument for function toHour", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
using FactorTransform = ToDateImpl;
};
struct ToMinuteImpl
{
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toMinuteInaccurate(t);
}
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
throw Exception("Illegal type Date of argument for function toMinute", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
using FactorTransform = ToStartOfHourImpl;
};
struct ToSecondImpl
{
static inline UInt8 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toSecondInaccurate(t);
}
static inline UInt8 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
throw Exception("Illegal type Date of argument for function toSecond", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
using FactorTransform = ToStartOfMinuteImpl;
};
struct ToRelativeYearNumImpl
{
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(t); }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toYear(DayNum_t(d)); }
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toYear(t);
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toYear(DayNum_t(d));
}
using FactorTransform = ZeroTransform;
};
struct ToRelativeMonthNumImpl
{
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeMonthNum(t); }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeMonthNum(DayNum_t(d)); }
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toRelativeMonthNum(t);
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toRelativeMonthNum(DayNum_t(d));
}
using FactorTransform = ZeroTransform;
};
struct ToRelativeWeekNumImpl
{
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeWeekNum(t); }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeWeekNum(DayNum_t(d)); }
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toRelativeWeekNum(t);
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toRelativeWeekNum(DayNum_t(d));
}
using FactorTransform = ZeroTransform;
};
struct ToRelativeDayNumImpl
{
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toDayNum(t); }
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return static_cast<DayNum_t>(d); }
static inline UInt16 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toDayNum(t);
}
static inline UInt16 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return static_cast<DayNum_t>(d);
}
using FactorTransform = ZeroTransform;
};
struct ToRelativeHourNumImpl
{
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeHourNum(t); }
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toRelativeHourNum(t);
}
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
throw Exception("Illegal type Date of argument for function toRelativeHourNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
using FactorTransform = ZeroTransform;
};
struct ToRelativeMinuteNumImpl
{
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return remote_date_lut.toRelativeMinuteNum(t); }
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return remote_date_lut.toRelativeMinuteNum(t);
}
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
throw Exception("Illegal type Date of argument for function toRelativeMinuteNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
using FactorTransform = ZeroTransform;
};
struct ToRelativeSecondNumImpl
{
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut) { return t; }
static inline UInt32 execute(UInt32 t, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
return t;
}
static inline UInt32 execute(UInt16 d, const DateLUTImpl & remote_date_lut, const DateLUTImpl & local_date_lut)
{
throw Exception("Illegal type Date of argument for function toRelativeSecondNum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
using FactorTransform = ZeroTransform;
};
template<typename FromType, typename ToType, typename Transform>
struct Transformer
{
@ -441,6 +618,42 @@ public:
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
bool hasInformationAboutMonotonicity() const override
{
return true;
}
Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override
{
IFunction::Monotonicity is_monotonic { true };
IFunction::Monotonicity is_not_monotonic;
if (std::is_same<typename Transform::FactorTransform, ZeroTransform>::value)
return is_monotonic;
/// Этот метод вызывается только если у функции один аргумент. Поэтому, нас пока не волнует не-локальная тайм-зона.
const DateLUTImpl & date_lut = DateLUT::instance();
if (left.isNull() || right.isNull())
return is_not_monotonic;
/// Функция монотонна на отрезке [left, right], если фактор-преобразование возвращает для них одинаковые значения.
if (typeid_cast<const DataTypeDate *>(&type))
{
return Transform::FactorTransform::execute(UInt16(left.get<UInt64>()), date_lut, date_lut)
== Transform::FactorTransform::execute(UInt16(right.get<UInt64>()), date_lut, date_lut)
? is_monotonic : is_not_monotonic;
}
else
{
return Transform::FactorTransform::execute(UInt32(left.get<UInt64>()), date_lut, date_lut)
== Transform::FactorTransform::execute(UInt32(right.get<UInt64>()), date_lut, date_lut)
? is_monotonic : is_not_monotonic;
}
}
};

View File

@ -1317,19 +1317,21 @@ private:
/// vector ids, vector defaults
const auto out = new ColumnString;
block.getByPosition(result).column = out;
dictionary->getString(attr_name, id_col->getData(), default_col, out);
const auto & ids = id_col->getData();
dictionary->getString(attr_name, ids, default_col, out);
}
else if (const auto default_col = typeid_cast<const ColumnConst<String> *>(default_col_untyped))
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();
const auto & ids = id_col->getData();
const auto & def = default_col->getData();
dictionary->getString(attr_name, id_col->getData(),
static_cast<const ColumnString *>(default_col_materialized.get()), out);
dictionary->getString(attr_name, ids, def, out);
}
else
throw Exception{
@ -1361,11 +1363,9 @@ private:
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()});
const auto & def = default_col->getData();
dictionary->getString(attr_name, ids, defs.get(), out.get());
dictionary->getString(attr_name, ids, def, out.get());
block.getByPosition(result).column = new ColumnConst<String>{
id_col->size(), out->getDataAt(0).toString()
@ -1417,11 +1417,8 @@ private:
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);
const auto & def = default_col->getData();
dict->getString(attr_name, key_columns, key_types, def, out);
}
else
throw Exception{
@ -1461,17 +1458,17 @@ template <> struct DictGetTraits<DATA_TYPE>\
{\
dict->get##TYPE(name, ids, dates, out);\
}\
template <typename DictionaryType>\
template <typename DictionaryType, typename DefaultsType>\
static void getOrDefault(\
const DictionaryType * const dict, const std::string & name, const PODArray<UInt64> & ids,\
const PODArray<TYPE> & def, PODArray<TYPE> & out)\
const DefaultsType & def, PODArray<TYPE> & out)\
{\
dict->get##TYPE(name, ids, def, out);\
}\
template <typename DictionaryType>\
template <typename DictionaryType, typename DefaultsType>\
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)\
const DataTypes & key_types, const DefaultsType & def, PODArray<TYPE> & out)\
{\
dict->get##TYPE(name, key_columns, key_types, def, out);\
}\
@ -1974,11 +1971,9 @@ private:
const auto & ids = id_col->getData();
auto & data = out->getData();
const auto def = default_col->getData();
/// @todo avoid materialization
const PODArray<Type> defs(id_col->size(), default_col->getData());
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, defs, data);
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, def, data);
}
else
throw Exception{
@ -2013,8 +2008,9 @@ private:
/// 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);
const auto & def = default_col->getData();
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, def, data);
block.getByPosition(result).column = new ColumnConst<Type>{id_col->size(), data.front()};
}
@ -2072,10 +2068,9 @@ private:
}
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());
const auto def = default_col->getData();
DictGetTraits<DataType>::getOrDefault(dict, attr_name, key_columns, key_types, defs, data);
DictGetTraits<DataType>::getOrDefault(dict, attr_name, key_columns, key_types, def, data);
}
else
throw Exception{

View File

@ -1031,6 +1031,16 @@ namespace
ErrorCodes::ILLEGAL_COLUMN);
}
}
bool hasInformationAboutMonotonicity() const override
{
return true;
}
Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override
{
return { true };
}
};
struct NameRoundToExp2 { static constexpr auto name = "roundToExp2"; };
@ -1046,4 +1056,18 @@ namespace
typedef FunctionRounding<NameRound, _MM_FROUND_NINT> FunctionRound;
typedef FunctionRounding<NameCeil, _MM_FROUND_CEIL> FunctionCeil;
typedef FunctionRounding<NameFloor, _MM_FROUND_FLOOR> FunctionFloor;
struct PositiveMonotonicity
{
static bool has() { return true; }
static IFunction::Monotonicity get(const Field & left, const Field & right)
{
return { true };
}
};
template <> struct FunctionUnaryArithmeticMonotonicity<NameRoundToExp2> : PositiveMonotonicity {};
template <> struct FunctionUnaryArithmeticMonotonicity<NameRoundDuration> : PositiveMonotonicity {};
template <> struct FunctionUnaryArithmeticMonotonicity<NameRoundAge> : PositiveMonotonicity {};
}

View File

@ -85,6 +85,31 @@ public:
execute(block, arguments, result);
}
/** Позволяет узнать, является ли функция монотонной в некотором диапазоне значений.
* Это используется для работы с индексом в сортированном куске данных.
* И позволяет использовать индекс не только, когда написано, например date >= const, но и, например, toMonth(date) >= 11.
* Всё это рассматривается только для функций одного аргумента.
*/
virtual bool hasInformationAboutMonotonicity() const { return false; }
/// Свойство монотонности на некотором диапазоне.
struct Monotonicity
{
bool is_monotonic = false; /// Является ли функция монотонной (неубывающей или невозрастающей).
bool is_positive = true; /// true, если функция неубывающая, false, если невозрастающая. Если is_monotonic = false, то не важно.
Monotonicity(bool is_monotonic_ = false, bool is_positive_ = true)
: is_monotonic(is_monotonic_), is_positive(is_positive_) {}
};
/** Получить информацию о монотонности на отрезке значений. Вызывайте только если hasInformationAboutMonotonicity.
* В качестве одного из аргументов может быть передан NULL. Это значит, что соответствующий диапазон неограничен слева или справа.
*/
virtual Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const
{
throw Exception("Function " + getName() + " has no information about its monotonicity.", ErrorCodes::NOT_IMPLEMENTED);
}
virtual ~IFunction() {}
};

View File

@ -28,7 +28,7 @@ namespace DB
/// Функции-помошники для форматированного чтения
static inline char parseEscapeSequence(char c)
inline char parseEscapeSequence(char c)
{
switch(c)
{
@ -49,6 +49,21 @@ static inline char parseEscapeSequence(char c)
}
}
inline char unhex(char c)
{
switch (c)
{
case '0' ... '9':
return c - '0';
case 'a' ... 'f':
return c - 'a' + 10;
case 'A' ... 'F':
return c - 'A' + 10;
default:
return 0;
}
}
/// Эти функции находятся в VarInt.h
/// inline void throwReadAfterEOF()

View File

@ -837,6 +837,26 @@ protected:
Names key_names;
AggregateDescriptions aggregates;
AggregateFunctionsPlainPtrs aggregate_functions;
/** Данный массив служит для двух целей.
*
* 1. Аргументы функции собраны рядом, и их не нужно собирать из разных мест. Также массив сделан zero-terminated.
* Внутренний цикл (для случая without_key) получается почти в два раза компактнее; прирост производительности около 30%.
*
* 2. Вызов по указателю на функцию лучше, чем виртуальный вызов, потому что в случае виртуального вызова,
* GCC 5.1.2 генерирует код, который на каждой итерации цикла заново грузит из памяти в регистр адрес функции
* (значение по смещению в таблице виртуальных функций).
*/
struct AggregateFunctionInstruction
{
const IAggregateFunction * that;
IAggregateFunction::AddFunc func;
size_t state_offset;
const IColumn ** arguments;
};
using AggregateFunctionInstructions = std::vector<AggregateFunctionInstruction>;
size_t keys_size;
size_t aggregates_size;
/// Нужно ли класть в AggregatedDataVariants::without_key агрегаты для ключей, не попавших в max_rows_to_group_by.
@ -915,7 +935,7 @@ protected:
Arena * aggregates_pool,
size_t rows,
ConstColumnPlainPtrs & key_columns,
AggregateColumns & aggregate_columns,
AggregateFunctionInstruction * aggregate_instructions,
const Sizes & key_sizes,
StringRefs & keys,
bool no_more_keys,
@ -929,7 +949,7 @@ protected:
Arena * aggregates_pool,
size_t rows,
ConstColumnPlainPtrs & key_columns,
AggregateColumns & aggregate_columns,
AggregateFunctionInstruction * aggregate_instructions,
const Sizes & key_sizes,
StringRefs & keys,
AggregateDataPtr overflow_row) const;
@ -938,7 +958,7 @@ protected:
void executeWithoutKeyImpl(
AggregatedDataWithoutKey & res,
size_t rows,
AggregateColumns & aggregate_columns) const;
AggregateFunctionInstruction * aggregate_instructions) const;
public:
/// Шаблоны, инстанцирующиеся путём динамической компиляции кода - см. SpecializedAggregator.h

View File

@ -86,9 +86,6 @@ struct Settings
M(SettingTotalsMode, totals_mode, TotalsMode::AFTER_HAVING_EXCLUSIVE) \
M(SettingFloat, totals_auto_threshold, 0.5) \
\
/** Сэмплирование по умолчанию. Если равно 1, то отключено. */ \
M(SettingFloat, default_sample, 1.0) \
\
/** Включена ли компиляция запросов. */ \
M(SettingBool, compile, false) \
/** Количество одинаковых по структуре запросов перед тем, как инициируется их компиляция. */ \
@ -132,8 +129,9 @@ struct Settings
/** Минимальное количество байт для операций ввода/ввывода минуя кэш страниц. 0 - отключено. */ \
M(SettingUInt64, min_bytes_to_use_direct_io, 0) \
\
/** Кидать исключение, если есть индекс по дате, и он не используется. */ \
/** Кидать исключение, если есть индекс, и он не используется. */ \
M(SettingBool, force_index_by_date, 0) \
M(SettingBool, force_primary_key, 0) \
\
/** В запросе INSERT с указанием столбцов, заполнять значения по-умолчанию только для столбцов с явными DEFAULT-ами. */ \
M(SettingBool, strict_insert_defaults, 0) \

View File

@ -23,4 +23,10 @@ void stableSortBlock(Block & block, const SortDescription & description);
*/
void stableGetPermutation(const Block & block, const SortDescription & description, IColumn::Permutation & out_permutation);
/** Быстро проверить, является ли блок уже отсортированным. Если блок не отсортирован - возвращает false максимально быстро.
* Не поддерживаются collations.
*/
bool isAlreadySorted(const Block & block, const SortDescription & description);
}

View File

@ -0,0 +1,43 @@
#pragma once
#include <DB/Parsers/IAST.h>
namespace DB
{
/** Коэффициент сэмплирования вида 0.1 или 1/10.
* Важно сохранять его как рациональное число без преобразования в IEEE-754.
*/
class ASTSampleRatio : public IAST
{
public:
using BigNum = __uint128_t; /// Должен вмещать в себя результат перемножения двух UInt64.
struct Rational
{
BigNum numerator = 0;
BigNum denominator = 1;
};
Rational ratio;
ASTSampleRatio() = default;
ASTSampleRatio(const StringRange range_) : IAST(range_) {}
ASTSampleRatio(const StringRange range_, Rational & ratio_) : IAST(range_), ratio(ratio_) {}
String getID() const override { return "SampleRatio_" + toString(ratio); }
ASTPtr clone() const override { return new ASTSampleRatio(*this); }
static String toString(BigNum num);
static String toString(Rational ratio);
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
settings.ostr << toString(ratio);
}
};
}

View File

@ -0,0 +1,19 @@
#pragma once
#include <DB/Parsers/IParserBase.h>
namespace DB
{
/** Коэффициент сэмплирования вида 0.1 или 1/10.
* Парсится как рациональное число без преобразования в IEEE-754.
*/
class ParserSampleRatio : public IParserBase
{
protected:
const char * getName() const { return "Sample ratio or offset"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
}

View File

@ -395,7 +395,7 @@ public:
for (size_t i = 0; i < size; ++i)
for (size_t j = 0; j < key_size; ++j)
storage.primary_key_sample.getByPosition(j).type->deserializeBinary(index[i * key_size + j], index_file);
storage.primary_key_data_types[j].get()->deserializeBinary(index[i * key_size + j], index_file);
if (!index_file.eof())
throw Exception("index file " + index_path + " is unexpectedly long", ErrorCodes::EXPECTED_END_OF_FILE);
@ -858,6 +858,7 @@ public:
const ASTPtr primary_expr_ast;
Block primary_key_sample;
DataTypes primary_key_data_types;
private:
bool require_part_metadata;

View File

@ -71,7 +71,7 @@ private:
String & out_column,
const Context & context) const;
MarkRanges markRangesFromPkRange(
MarkRanges markRangesFromPKRange(
const MergeTreeData::DataPart::Index & index,
const PKCondition & key_condition,
const Settings & settings) const;

View File

@ -31,7 +31,7 @@ public:
static void checkDataPart(
String path,
const Settings & settings,
const Block & primary_key_sample, /// Проверять первичный ключ. Если не надо - передайте пустой Block.
const DataTypes & primary_key_data_types, /// Проверять первичный ключ. Если не надо - передайте пустой массив.
MergeTreeData::DataPart::Checksums * out_checksums = nullptr);
};

View File

@ -10,6 +10,7 @@
#include <DB/Parsers/ASTFunction.h>
#include <DB/Parsers/ASTLiteral.h>
#include <DB/Storages/MergeTree/BoolMask.h>
#include <DB/Functions/IFunction.h>
namespace DB
@ -142,6 +143,13 @@ public:
return true;
}
void swapLeftAndRight()
{
std::swap(left, right);
std::swap(left_bounded, right_bounded);
std::swap(left_included, right_included);
}
String toString() const
{
std::stringstream str;
@ -181,15 +189,16 @@ public:
static const AtomMap atom_map;
/// Не учитывает секцию SAMPLE. all_columns - набор всех столбцов таблицы.
PKCondition(ASTPtr query, const Context & context, const NamesAndTypesList & all_columns, const SortDescription & sort_descr);
PKCondition(ASTPtr & query, const Context & context, const NamesAndTypesList & all_columns, const SortDescription & sort_descr);
/// Выполнимо ли условие в диапазоне ключей.
/// left_pk и right_pk должны содержать все поля из sort_descr в соответствующем порядке.
bool mayBeTrueInRange(const Field * left_pk, const Field * right_pk) const;
/// data_types - типы столбцов первичного ключа.
bool mayBeTrueInRange(const Field * left_pk, const Field * right_pk, const DataTypes & data_types) const;
/// Выполнимо ли условие в полубесконечном (не ограниченном справа) диапазоне ключей.
/// left_pk должен содержать все поля из sort_descr в соответствующем порядке.
bool mayBeTrueAfter(const Field * left_pk) const;
bool mayBeTrueAfter(const Field * left_pk, const DataTypes & data_types) const;
/// Проверяет, что индекс не может быть использован.
bool alwaysUnknown() const;
@ -236,18 +245,41 @@ private:
/// Для FUNCTION_IN_RANGE и FUNCTION_NOT_IN_RANGE.
Range range;
size_t key_column;
/// Для FUNCTION_IN_SET
/// Для FUNCTION_IN_SET, FUNCTION_NOT_IN_SET
ASTPtr in_function;
/** Цепочка возможно-монотонных функций.
* Если столбец первичного ключа завёрнут в функции, которые могут быть монотонными в некоторых диапазонах значений
* (например: -toFloat64(toDayOfWeek(date))), то здесь будут расположены функции: toDayOfWeek, toFloat64, negate.
*/
using MonotonicFunctionsChain = std::vector<FunctionPtr>;
mutable MonotonicFunctionsChain monotonic_functions_chain; /// Выполнение функции не нарушает константность.
};
typedef std::vector<RPNElement> RPN;
typedef std::map<String, size_t> ColumnIndices;
bool mayBeTrueInRange(const Field * left_pk, const Field * right_pk, bool right_bounded) const;
bool mayBeTrueInRange(const Field * left_pk, const Field * right_pk, const DataTypes & data_types, bool right_bounded) const;
void traverseAST(ASTPtr & node, Block & block_with_constants);
bool atomFromAST(ASTPtr & node, Block & block_with_constants, RPNElement & out);
bool operatorFromAST(ASTFunction * func, RPNElement & out);
void traverseAST(ASTPtr & node, const Context & context, Block & block_with_constants);
bool atomFromAST(ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out);
bool operatorFromAST(const ASTFunction * func, RPNElement & out);
/** Является ли node столбцом первичного ключа
* или выражением, в котором столбец первичного ключа завёрнут в цепочку функций,
* которые могут быть монотонными на некоторых диапазонах.
* Если да - вернуть номер этого столбца в первичном ключе, а также заполнить цепочку возможно-монотонных функций.
*/
bool isPrimaryKeyPossiblyWrappedByMonotonicFunctions(
const ASTPtr & node,
const Context & context,
size_t & out_primary_key_column_num,
RPNElement::MonotonicFunctionsChain & out_functions_chain);
bool isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl(
const ASTPtr & node,
size_t & out_primary_key_column_num,
std::vector<const ASTFunction *> & out_functions_chain);
RPN rpn;

View File

@ -64,6 +64,7 @@ void registerAggregateFunctionsQuantileExact(AggregateFunctionFactory & factory)
void registerAggregateFunctionsQuantileExactWeighted(AggregateFunctionFactory & factory);
void registerAggregateFunctionsQuantileDeterministic(AggregateFunctionFactory & factory);
void registerAggregateFunctionsQuantileTiming(AggregateFunctionFactory & factory);
void registerAggregateFunctionsQuantileTDigest(AggregateFunctionFactory & factory);
void registerAggregateFunctionsSequenceMatch(AggregateFunctionFactory & factory);
void registerAggregateFunctionsMinMaxAny(AggregateFunctionFactory & factory);
void registerAggregateFunctionsStatistics(AggregateFunctionFactory & factory);
@ -88,6 +89,7 @@ AggregateFunctionFactory::AggregateFunctionFactory()
registerAggregateFunctionsQuantileExactWeighted(*this);
registerAggregateFunctionsQuantileDeterministic(*this);
registerAggregateFunctionsQuantileTiming(*this);
registerAggregateFunctionsQuantileTDigest(*this);
registerAggregateFunctionsSequenceMatch(*this);
registerAggregateFunctionsMinMaxAny(*this);
registerAggregateFunctionsStatistics(*this);

View File

@ -0,0 +1,104 @@
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/AggregateFunctions/Helpers.h>
#include <DB/AggregateFunctions/AggregateFunctionQuantileTDigest.h>
namespace DB
{
namespace
{
template <template <typename, bool> class FunctionTemplate>
AggregateFunctionPtr createAggregateFunctionQuantileTDigest(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const IDataType & argument_type = *argument_types[0];
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new FunctionTemplate<UInt8, true>;
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new FunctionTemplate<UInt16, true>;
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new FunctionTemplate<UInt32, true>;
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new FunctionTemplate<UInt64, true>;
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new FunctionTemplate<Int8, true>;
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new FunctionTemplate<Int16, true>;
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new FunctionTemplate<Int32, true>;
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new FunctionTemplate<Int64, true>;
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new FunctionTemplate<Float32, true>;
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new FunctionTemplate<Float64, true>;
else if (typeid_cast<const DataTypeDate *>(&argument_type)) return new FunctionTemplate<DataTypeDate::FieldType, false>;
else if (typeid_cast<const DataTypeDateTime*>(&argument_type)) return new FunctionTemplate<DataTypeDateTime::FieldType, false>;
else
throw Exception("Illegal type " + argument_type.getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
template <template <typename, typename, bool> class FunctionTemplate, typename T, bool returns_float>
AggregateFunctionPtr createAggregateFunctionQuantileTDigestWeightedImpl(const std::string & name, const DataTypes & argument_types)
{
const IDataType & argument_type = *argument_types[1];
if (typeid_cast<const DataTypeUInt8 *>(&argument_type)) return new FunctionTemplate<T, UInt8, returns_float>;
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new FunctionTemplate<T, UInt16, returns_float>;
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new FunctionTemplate<T, UInt32, returns_float>;
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new FunctionTemplate<T, UInt64, returns_float>;
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new FunctionTemplate<T, Int8, returns_float>;
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new FunctionTemplate<T, Int16, returns_float>;
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new FunctionTemplate<T, Int32, returns_float>;
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new FunctionTemplate<T, Int64, returns_float>;
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new FunctionTemplate<T, Float32, returns_float>;
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new FunctionTemplate<T, Float64, returns_float>;
else
throw Exception("Illegal type " + argument_type.getName() + " of second argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
template <template <typename, typename, bool> class FunctionTemplate>
AggregateFunctionPtr createAggregateFunctionQuantileTDigestWeighted(const std::string & name, const DataTypes & argument_types)
{
if (argument_types.size() != 2)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const IDataType & argument_type = *argument_types[0];
if (typeid_cast<const DataTypeUInt8 *>(&argument_type))
return createAggregateFunctionQuantileTDigestWeightedImpl<FunctionTemplate, UInt8, true>(name, argument_types);
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type))
return createAggregateFunctionQuantileTDigestWeightedImpl<FunctionTemplate, UInt16, true>(name, argument_types);
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type))
return createAggregateFunctionQuantileTDigestWeightedImpl<FunctionTemplate, UInt32, true>(name, argument_types);
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type))
return createAggregateFunctionQuantileTDigestWeightedImpl<FunctionTemplate, UInt64, true>(name, argument_types);
else if (typeid_cast<const DataTypeInt8 *>(&argument_type))
return createAggregateFunctionQuantileTDigestWeightedImpl<FunctionTemplate, Int8, true>(name, argument_types);
else if (typeid_cast<const DataTypeInt16 *>(&argument_type))
return createAggregateFunctionQuantileTDigestWeightedImpl<FunctionTemplate, Int16, true>(name, argument_types);
else if (typeid_cast<const DataTypeInt32 *>(&argument_type))
return createAggregateFunctionQuantileTDigestWeightedImpl<FunctionTemplate, Int32, true>(name, argument_types);
else if (typeid_cast<const DataTypeInt64 *>(&argument_type))
return createAggregateFunctionQuantileTDigestWeightedImpl<FunctionTemplate, Int64, true>(name, argument_types);
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type))
return createAggregateFunctionQuantileTDigestWeightedImpl<FunctionTemplate, Float32, true>(name, argument_types);
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type))
return createAggregateFunctionQuantileTDigestWeightedImpl<FunctionTemplate, Float64, true>(name, argument_types);
else if (typeid_cast<const DataTypeDate *>(&argument_type))
return createAggregateFunctionQuantileTDigestWeightedImpl<FunctionTemplate, DataTypeDate::FieldType, false>(name, argument_types);
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
return createAggregateFunctionQuantileTDigestWeightedImpl<FunctionTemplate, DataTypeDateTime::FieldType, false>(name, argument_types);
else
throw Exception("Illegal type " + argument_type.getName() + " of first argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
void registerAggregateFunctionsQuantileTDigest(AggregateFunctionFactory & factory)
{
factory.registerFunction({"quantileTDigest", "medianTDigest"},
createAggregateFunctionQuantileTDigest<AggregateFunctionQuantileTDigest>);
factory.registerFunction({"quantilesTDigest"},
createAggregateFunctionQuantileTDigest<AggregateFunctionQuantilesTDigest>);
factory.registerFunction({"quantileTDigestWeighted", "medianTDigestWeighted"},
createAggregateFunctionQuantileTDigestWeighted<AggregateFunctionQuantileTDigestWeighted>);
factory.registerFunction({"quantilesTDigestWeighted"},
createAggregateFunctionQuantileTDigestWeighted<AggregateFunctionQuantilesTDigestWeighted>);
}
}

View File

@ -1,45 +1,145 @@
#include <DB/Columns/ColumnArray.h>
#include <DB/Columns/ColumnsCommon.h>
namespace DB
{
ColumnPtr ColumnArray::cut(size_t start, size_t length) const
void ColumnArray::insertRangeFrom(const IColumn & src, size_t start, size_t length)
{
if (length == 0)
return new ColumnArray(data);
return;
if (start + length > getOffsets().size())
throw Exception("Parameter out of bound in IColumnArray::cut() method.",
const ColumnArray & src_concrete = static_cast<const ColumnArray &>(src);
if (start + length > src_concrete.getOffsets().size())
throw Exception("Parameter out of bound in ColumnArray::insertRangeFrom method.",
ErrorCodes::PARAMETER_OUT_OF_BOUND);
size_t nested_offset = offsetAt(start);
size_t nested_length = getOffsets()[start + length - 1] - nested_offset;
size_t nested_offset = src_concrete.offsetAt(start);
size_t nested_length = src_concrete.getOffsets()[start + length - 1] - nested_offset;
ColumnArray * res_ = new ColumnArray(data);
ColumnPtr res = res_;
data->insertRangeFrom(src_concrete.getData(), nested_offset, nested_length);
res_->data = data->cut(nested_offset, nested_length);
Offsets_t & res_offsets = res_->getOffsets();
Offsets_t & cur_offsets = getOffsets();
const Offsets_t & src_offsets = src_concrete.getOffsets();
if (start == 0)
if (start == 0 && cur_offsets.empty())
{
res_offsets.assign(getOffsets().begin(), getOffsets().begin() + length);
cur_offsets.assign(src_offsets.begin(), src_offsets.begin() + length);
}
else
{
res_offsets.resize(length);
size_t old_size = cur_offsets.size();
size_t prev_max_offset = old_size ? cur_offsets.back() : 0;
cur_offsets.resize(old_size + length);
for (size_t i = 0; i < length; ++i)
res_offsets[i] = getOffsets()[start + i] - nested_offset;
cur_offsets[old_size + i] = src_offsets[start + i] - nested_offset + prev_max_offset;
}
return res;
}
ColumnPtr ColumnArray::filter(const Filter & filt) const
{
if (typeid_cast<const ColumnUInt8 *>(data.get())) return filterNumber<UInt8>(filt);
if (typeid_cast<const ColumnUInt16 *>(data.get())) return filterNumber<UInt16>(filt);
if (typeid_cast<const ColumnUInt32 *>(data.get())) return filterNumber<UInt32>(filt);
if (typeid_cast<const ColumnUInt64 *>(data.get())) return filterNumber<UInt64>(filt);
if (typeid_cast<const ColumnInt8 *>(data.get())) return filterNumber<Int8>(filt);
if (typeid_cast<const ColumnInt16 *>(data.get())) return filterNumber<Int16>(filt);
if (typeid_cast<const ColumnInt32 *>(data.get())) return filterNumber<Int32>(filt);
if (typeid_cast<const ColumnInt64 *>(data.get())) return filterNumber<Int64>(filt);
if (typeid_cast<const ColumnFloat32 *>(data.get())) return filterNumber<Float32>(filt);
if (typeid_cast<const ColumnFloat64 *>(data.get())) return filterNumber<Float64>(filt);
if (typeid_cast<const ColumnString *>(data.get())) return filterString(filt);
return filterGeneric(filt);
}
template <typename T>
ColumnPtr ColumnArray::filterNumber(const Filter & filt) const
{
if (getOffsets().size() == 0)
return new ColumnArray(data);
ColumnArray * res = new ColumnArray(data->cloneEmpty());
ColumnPtr res_ = res;
PODArray<T> & res_elems = static_cast<ColumnVector<T> &>(res->getData()).getData();
Offsets_t & res_offsets = res->getOffsets();
filterArraysImpl<T>(static_cast<const ColumnVector<T> &>(*data).getData(), getOffsets(), res_elems, res_offsets, filt);
return res_;
}
ColumnPtr ColumnArray::filterString(const Filter & filt) const
{
size_t col_size = getOffsets().size();
if (col_size != filt.size())
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
if (0 == col_size)
return new ColumnArray(data);
ColumnArray * res = new ColumnArray(data->cloneEmpty());
ColumnPtr res_ = res;
const ColumnString & src_string = typeid_cast<const ColumnString &>(*data);
const ColumnString::Chars_t & src_chars = src_string.getChars();
const Offsets_t & src_string_offsets = src_string.getOffsets();
const Offsets_t & src_offsets = getOffsets();
ColumnString::Chars_t & res_chars = typeid_cast<ColumnString &>(res->getData()).getChars();
Offsets_t & res_string_offsets = typeid_cast<ColumnString &>(res->getData()).getOffsets();
Offsets_t & res_offsets = res->getOffsets();
res_chars.reserve(src_chars.size());
res_string_offsets.reserve(src_string_offsets.size());
res_offsets.reserve(col_size);
Offset_t prev_src_offset = 0;
Offset_t prev_src_string_offset = 0;
Offset_t prev_res_offset = 0;
Offset_t prev_res_string_offset = 0;
for (size_t i = 0; i < col_size; ++i)
{
/// Количество строк в массиве.
size_t array_size = src_offsets[i] - prev_src_offset;
if (filt[i])
{
/// Если массив не пуст - копируем внутренности.
if (array_size)
{
size_t chars_to_copy = src_string_offsets[array_size + prev_src_offset - 1] - prev_src_string_offset;
size_t res_chars_prev_size = res_chars.size();
res_chars.resize(res_chars_prev_size + chars_to_copy);
memcpy(&res_chars[res_chars_prev_size], &src_chars[prev_src_string_offset], chars_to_copy);
for (size_t j = 0; j < array_size; ++j)
res_string_offsets.push_back(src_string_offsets[j + prev_src_offset] + prev_res_string_offset - prev_src_string_offset);
prev_res_string_offset = res_string_offsets.back();
}
prev_res_offset += array_size;
res_offsets.push_back(prev_res_offset);
}
if (array_size)
{
prev_src_offset += array_size;
prev_src_string_offset = src_string_offsets[prev_src_offset - 1];
}
}
return res_;
}
ColumnPtr ColumnArray::filterGeneric(const Filter & filt) const
{
size_t size = getOffsets().size();
if (size != filt.size())
@ -48,7 +148,6 @@ ColumnPtr ColumnArray::filter(const Filter & filt) const
if (size == 0)
return new ColumnArray(data);
/// Не слишком оптимально. Можно сделать специализацию для массивов известных типов.
Filter nested_filt(getOffsets().back());
for (size_t i = 0; i < size; ++i)
{
@ -149,25 +248,25 @@ ColumnPtr ColumnArray::replicate(const Offsets_t & replicate_offsets) const
{
/// Не получается реализовать в общем случае.
if (typeid_cast<const ColumnUInt8 *>(&*data)) return replicate<UInt8>(replicate_offsets);
if (typeid_cast<const ColumnUInt16 *>(&*data)) return replicate<UInt16>(replicate_offsets);
if (typeid_cast<const ColumnUInt32 *>(&*data)) return replicate<UInt32>(replicate_offsets);
if (typeid_cast<const ColumnUInt64 *>(&*data)) return replicate<UInt64>(replicate_offsets);
if (typeid_cast<const ColumnInt8 *>(&*data)) return replicate<Int8>(replicate_offsets);
if (typeid_cast<const ColumnInt16 *>(&*data)) return replicate<Int16>(replicate_offsets);
if (typeid_cast<const ColumnInt32 *>(&*data)) return replicate<Int32>(replicate_offsets);
if (typeid_cast<const ColumnInt64 *>(&*data)) return replicate<Int64>(replicate_offsets);
if (typeid_cast<const ColumnFloat32 *>(&*data)) return replicate<Float32>(replicate_offsets);
if (typeid_cast<const ColumnFloat64 *>(&*data)) return replicate<Float64>(replicate_offsets);
if (typeid_cast<const ColumnString *>(&*data)) return replicateString(replicate_offsets);
if (dynamic_cast<const IColumnConst *>(&*data)) return replicateConst(replicate_offsets);
if (typeid_cast<const ColumnUInt8 *>(data.get())) return replicateNumber<UInt8>(replicate_offsets);
if (typeid_cast<const ColumnUInt16 *>(data.get())) return replicateNumber<UInt16>(replicate_offsets);
if (typeid_cast<const ColumnUInt32 *>(data.get())) return replicateNumber<UInt32>(replicate_offsets);
if (typeid_cast<const ColumnUInt64 *>(data.get())) return replicateNumber<UInt64>(replicate_offsets);
if (typeid_cast<const ColumnInt8 *>(data.get())) return replicateNumber<Int8>(replicate_offsets);
if (typeid_cast<const ColumnInt16 *>(data.get())) return replicateNumber<Int16>(replicate_offsets);
if (typeid_cast<const ColumnInt32 *>(data.get())) return replicateNumber<Int32>(replicate_offsets);
if (typeid_cast<const ColumnInt64 *>(data.get())) return replicateNumber<Int64>(replicate_offsets);
if (typeid_cast<const ColumnFloat32 *>(data.get())) return replicateNumber<Float32>(replicate_offsets);
if (typeid_cast<const ColumnFloat64 *>(data.get())) return replicateNumber<Float64>(replicate_offsets);
if (typeid_cast<const ColumnString *>(data.get())) return replicateString(replicate_offsets);
if (dynamic_cast<const IColumnConst *>(data.get())) return replicateConst(replicate_offsets);
throw Exception("Replication of column " + getName() + " is not implemented.", ErrorCodes::NOT_IMPLEMENTED);
}
template <typename T>
ColumnPtr ColumnArray::replicate(const Offsets_t & replicate_offsets) const
ColumnPtr ColumnArray::replicateNumber(const Offsets_t & replicate_offsets) const
{
size_t col_size = size();
if (col_size != replicate_offsets.size())
@ -180,8 +279,8 @@ ColumnPtr ColumnArray::replicate(const Offsets_t & replicate_offsets) const
ColumnArray & res_ = typeid_cast<ColumnArray &>(*res);
const typename ColumnVector<T>::Container_t & cur_data = typeid_cast<const ColumnVector<T> &>(*data).getData();
const Offsets_t & cur_offsets = getOffsets();
const typename ColumnVector<T>::Container_t & src_data = typeid_cast<const ColumnVector<T> &>(*data).getData();
const Offsets_t & src_offsets = getOffsets();
typename ColumnVector<T>::Container_t & res_data = typeid_cast<ColumnVector<T> &>(res_.getData()).getData();
Offsets_t & res_offsets = res_.getOffsets();
@ -196,7 +295,7 @@ ColumnPtr ColumnArray::replicate(const Offsets_t & replicate_offsets) const
for (size_t i = 0; i < col_size; ++i)
{
size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset;
size_t value_size = cur_offsets[i] - prev_data_offset;
size_t value_size = src_offsets[i] - prev_data_offset;
for (size_t j = 0; j < size_to_replicate; ++j)
{
@ -204,11 +303,11 @@ ColumnPtr ColumnArray::replicate(const Offsets_t & replicate_offsets) const
res_offsets.push_back(current_new_offset);
res_data.resize(res_data.size() + value_size);
memcpy(&res_data[res_data.size() - value_size], &cur_data[prev_data_offset], value_size * sizeof(T));
memcpy(&res_data[res_data.size() - value_size], &src_data[prev_data_offset], value_size * sizeof(T));
}
prev_replicate_offset = replicate_offsets[i];
prev_data_offset = cur_offsets[i];
prev_data_offset = src_offsets[i];
}
return res;
@ -226,25 +325,25 @@ ColumnPtr ColumnArray::replicateString(const Offsets_t & replicate_offsets) cons
if (0 == col_size)
return res;
ColumnArray & res_ = typeid_cast<ColumnArray &>(*res);
ColumnArray & res_ = static_cast<ColumnArray &>(*res);
const ColumnString & cur_string = typeid_cast<const ColumnString &>(*data);
const ColumnString::Chars_t & cur_chars = cur_string.getChars();
const Offsets_t & cur_string_offsets = cur_string.getOffsets();
const Offsets_t & cur_offsets = getOffsets();
const ColumnString & src_string = typeid_cast<const ColumnString &>(*data);
const ColumnString::Chars_t & src_chars = src_string.getChars();
const Offsets_t & src_string_offsets = src_string.getOffsets();
const Offsets_t & src_offsets = getOffsets();
ColumnString::Chars_t & res_chars = typeid_cast<ColumnString &>(res_.getData()).getChars();
Offsets_t & res_string_offsets = typeid_cast<ColumnString &>(res_.getData()).getOffsets();
Offsets_t & res_offsets = res_.getOffsets();
res_chars.reserve(cur_chars.size() / col_size * replicate_offsets.back());
res_string_offsets.reserve(cur_string_offsets.size() / col_size * replicate_offsets.back());
res_chars.reserve(src_chars.size() / col_size * replicate_offsets.back());
res_string_offsets.reserve(src_string_offsets.size() / col_size * replicate_offsets.back());
res_offsets.reserve(replicate_offsets.back());
Offset_t prev_replicate_offset = 0;
Offset_t prev_cur_offset = 0;
Offset_t prev_cur_string_offset = 0;
Offset_t prev_src_offset = 0;
Offset_t prev_src_string_offset = 0;
Offset_t current_res_offset = 0;
Offset_t current_res_string_offset = 0;
@ -254,7 +353,7 @@ ColumnPtr ColumnArray::replicateString(const Offsets_t & replicate_offsets) cons
/// Насколько размножить массив.
size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset;
/// Количество строк в массиве.
size_t value_size = cur_offsets[i] - prev_cur_offset;
size_t value_size = src_offsets[i] - prev_src_offset;
size_t sum_chars_size = 0;
@ -265,27 +364,27 @@ ColumnPtr ColumnArray::replicateString(const Offsets_t & replicate_offsets) cons
sum_chars_size = 0;
size_t prev_cur_string_offset_local = prev_cur_string_offset;
size_t prev_src_string_offset_local = prev_src_string_offset;
for (size_t k = 0; k < value_size; ++k)
{
/// Размер одной строки.
size_t chars_size = cur_string_offsets[k + prev_cur_offset] - prev_cur_string_offset_local;
size_t chars_size = src_string_offsets[k + prev_src_offset] - prev_src_string_offset_local;
current_res_string_offset += chars_size;
res_string_offsets.push_back(current_res_string_offset);
/// Копирование символов одной строки.
res_chars.resize(res_chars.size() + chars_size);
memcpy(&res_chars[res_chars.size() - chars_size], &cur_chars[prev_cur_string_offset_local], chars_size);
memcpy(&res_chars[res_chars.size() - chars_size], &src_chars[prev_src_string_offset_local], chars_size);
sum_chars_size += chars_size;
prev_cur_string_offset_local += chars_size;
prev_src_string_offset_local += chars_size;
}
}
prev_replicate_offset = replicate_offsets[i];
prev_cur_offset = cur_offsets[i];
prev_cur_string_offset += sum_chars_size;
prev_src_offset = src_offsets[i];
prev_src_string_offset += sum_chars_size;
}
return res;
@ -301,7 +400,7 @@ ColumnPtr ColumnArray::replicateConst(const Offsets_t & replicate_offsets) const
if (0 == col_size)
return cloneEmpty();
const Offsets_t & cur_offsets = getOffsets();
const Offsets_t & src_offsets = getOffsets();
ColumnOffsets_t * res_column_offsets = new ColumnOffsets_t;
ColumnPtr res_column_offsets_holder = res_column_offsets;
@ -315,7 +414,7 @@ ColumnPtr ColumnArray::replicateConst(const Offsets_t & replicate_offsets) const
for (size_t i = 0; i < col_size; ++i)
{
size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset;
size_t value_size = cur_offsets[i] - prev_data_offset;
size_t value_size = src_offsets[i] - prev_data_offset;
for (size_t j = 0; j < size_to_replicate; ++j)
{
@ -324,7 +423,7 @@ ColumnPtr ColumnArray::replicateConst(const Offsets_t & replicate_offsets) const
}
prev_replicate_offset = replicate_offsets[i];
prev_data_offset = cur_offsets[i];
prev_data_offset = src_offsets[i];
}
return new ColumnArray(getData().cloneResized(current_new_offset), res_column_offsets_holder);

View File

@ -0,0 +1,173 @@
#include <emmintrin.h>
#include <DB/Columns/IColumn.h>
namespace DB
{
size_t countBytesInFilter(const IColumn::Filter & filt)
{
size_t count = 0;
/** NOTE: По идее, filt должен содержать только нолики и единички.
* Но, на всякий случай, здесь используется условие > 0 (на знаковые байты).
* Лучше было бы использовать != 0, то это не позволяет SSE2.
*/
const __m128i zero16 = _mm_setzero_si128();
const Int8 * pos = reinterpret_cast<const Int8 *>(&filt[0]);
const Int8 * end = pos + filt.size();
const Int8 * end64 = pos + filt.size() / 64 * 64;
for (; pos < end64; pos += 64)
count += __builtin_popcountll(
static_cast<UInt64>(_mm_movemask_epi8(_mm_cmpgt_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(pos)),
zero16)))
| (static_cast<UInt64>(_mm_movemask_epi8(_mm_cmpgt_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(pos + 16)),
zero16))) << 16)
| (static_cast<UInt64>(_mm_movemask_epi8(_mm_cmpgt_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(pos + 32)),
zero16))) << 32)
| (static_cast<UInt64>(_mm_movemask_epi8(_mm_cmpgt_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(pos + 48)),
zero16))) << 48));
for (; pos < end; ++pos)
count += *pos > 0;
return count;
}
template <typename T>
void filterArraysImpl(
const PODArray<T> & src_elems, const IColumn::Offsets_t & src_offsets,
PODArray<T> & res_elems, IColumn::Offsets_t & res_offsets,
const IColumn::Filter & filt)
{
const size_t size = src_offsets.size();
if (size != filt.size())
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
res_elems.reserve(src_elems.size());
res_offsets.reserve(size);
IColumn::Offset_t current_src_offset = 0;
static constexpr size_t SIMD_BYTES = 16;
const UInt8 * filt_pos = &filt[0];
const auto filt_end = filt_pos + size;
const auto filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES;
auto offsets_pos = &src_offsets[0];
const auto offsets_begin = offsets_pos;
const __m128i zero_vec = _mm_setzero_si128();
/// copy array ending at *end_offset_ptr
const auto copy_array = [&] (const IColumn::Offset_t * offset_ptr)
{
const auto offset = offset_ptr == offsets_begin ? 0 : offset_ptr[-1];
const auto size = *offset_ptr - offset;
current_src_offset += size;
res_offsets.push_back(current_src_offset);
const auto elems_size_old = res_elems.size();
res_elems.resize_assume_reserved(elems_size_old + size);
memcpy(&res_elems[elems_size_old], &src_elems[offset], size * sizeof(T));
};
while (filt_pos < filt_end_aligned)
{
const auto mask = _mm_movemask_epi8(_mm_cmpgt_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(filt_pos)),
zero_vec));
if (mask == 0)
{
/// SIMD_BYTES consecutive rows do not pass the filter
}
else if (mask == 0xffff)
{
/// SIMD_BYTES consecutive rows pass the filter
const auto first = offsets_pos == offsets_begin;
const auto chunk_offset = first ? 0 : offsets_pos[-1];
const auto chunk_size = offsets_pos[SIMD_BYTES - 1] - chunk_offset;
const auto offsets_size_old = res_offsets.size();
res_offsets.resize(offsets_size_old + SIMD_BYTES);
memcpy(&res_offsets[offsets_size_old], offsets_pos, SIMD_BYTES * sizeof(IColumn::Offset_t));
if (!first)
{
/// difference between current and actual offset
const auto diff_offset = chunk_offset - current_src_offset;
if (diff_offset > 0)
{
const auto res_offsets_pos = &res_offsets[offsets_size_old];
/// adjust offsets
for (size_t i = 0; i < SIMD_BYTES; ++i)
res_offsets_pos[i] -= diff_offset;
}
}
current_src_offset += chunk_size;
/// copy elements for SIMD_BYTES arrays at once
const auto elems_size_old = res_elems.size();
res_elems.resize(elems_size_old + chunk_size);
memcpy(&res_elems[elems_size_old], &src_elems[chunk_offset], chunk_size * sizeof(T));
}
else
{
for (size_t i = 0; i < SIMD_BYTES; ++i)
if (filt_pos[i])
copy_array(offsets_pos + i);
}
filt_pos += SIMD_BYTES;
offsets_pos += SIMD_BYTES;
}
while (filt_pos < filt_end)
{
if (*filt_pos)
copy_array(offsets_pos);
++filt_pos;
++offsets_pos;
}
}
/// Явные инстанцирования - чтобы не размещать реализацию функции выше в заголовочном файле.
template void filterArraysImpl<UInt8>(
const PODArray<UInt8> &, const IColumn::Offsets_t &, PODArray<UInt8> &, IColumn::Offsets_t &, const IColumn::Filter &);
template void filterArraysImpl<UInt16>(
const PODArray<UInt16> &, const IColumn::Offsets_t &, PODArray<UInt16> &, IColumn::Offsets_t &, const IColumn::Filter &);
template void filterArraysImpl<UInt32>(
const PODArray<UInt32> &, const IColumn::Offsets_t &, PODArray<UInt32> &, IColumn::Offsets_t &, const IColumn::Filter &);
template void filterArraysImpl<UInt64>(
const PODArray<UInt64> &, const IColumn::Offsets_t &, PODArray<UInt64> &, IColumn::Offsets_t &, const IColumn::Filter &);
template void filterArraysImpl<Int8>(
const PODArray<Int8> &, const IColumn::Offsets_t &, PODArray<Int8> &, IColumn::Offsets_t &, const IColumn::Filter &);
template void filterArraysImpl<Int16>(
const PODArray<Int16> &, const IColumn::Offsets_t &, PODArray<Int16> &, IColumn::Offsets_t &, const IColumn::Filter &);
template void filterArraysImpl<Int32>(
const PODArray<Int32> &, const IColumn::Offsets_t &, PODArray<Int32> &, IColumn::Offsets_t &, const IColumn::Filter &);
template void filterArraysImpl<Int64>(
const PODArray<Int64> &, const IColumn::Offsets_t &, PODArray<Int64> &, IColumn::Offsets_t &, const IColumn::Filter &);
template void filterArraysImpl<Float32>(
const PODArray<Float32> &, const IColumn::Offsets_t &, PODArray<Float32> &, IColumn::Offsets_t &, const IColumn::Filter &);
template void filterArraysImpl<Float64>(
const PODArray<Float64> &, const IColumn::Offsets_t &, PODArray<Float64> &, IColumn::Offsets_t &, const IColumn::Filter &);
}

View File

@ -1,45 +0,0 @@
#include <emmintrin.h>
#include <DB/Columns/IColumn.h>
namespace DB
{
size_t countBytesInFilter(const IColumn::Filter & filt)
{
size_t count = 0;
/** NOTE: По идее, filt должен содержать только нолики и единички.
* Но, на всякий случай, здесь используется условие > 0 (на знаковые байты).
* Лучше было бы использовать != 0, то это не позволяет SSE2.
*/
const __m128i zero16 = _mm_set1_epi8(0);
const Int8 * pos = reinterpret_cast<const Int8 *>(&filt[0]);
const Int8 * end = pos + filt.size();
const Int8 * end64 = pos + filt.size() / 64 * 64;
for (; pos < end64; pos += 64)
count += __builtin_popcountll(
static_cast<UInt64>(_mm_movemask_epi8(_mm_cmpgt_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(pos)),
zero16)))
| (static_cast<UInt64>(_mm_movemask_epi8(_mm_cmpgt_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(pos + 16)),
zero16))) << 16)
| (static_cast<UInt64>(_mm_movemask_epi8(_mm_cmpgt_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(pos + 32)),
zero16))) << 32)
| (static_cast<UInt64>(_mm_movemask_epi8(_mm_cmpgt_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(pos + 48)),
zero16))) << 48));
for (; pos < end; ++pos)
count += *pos > 0;
return count;
}
}

View File

@ -0,0 +1,116 @@
#include <malloc.h>
#include <ext/bit_cast.hpp>
#include <DB/Common/RadixSort.h>
#include <DB/Common/Stopwatch.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/Core/Defines.h>
using Key = double;
void NO_INLINE sort1(Key * data, size_t size)
{
std::sort(data, data + size);
}
void NO_INLINE sort2(Key * data, size_t size)
{
radixSort(data, size);
}
void NO_INLINE sort3(Key * data, size_t size)
{
std::sort(data, data + size, [](Key a, Key b)
{
return RadixSortFloatTransform<uint32_t>::forward(ext::bit_cast<uint32_t>(a))
< RadixSortFloatTransform<uint32_t>::forward(ext::bit_cast<uint32_t>(b));
});
}
int main(int argc, char ** argv)
{
size_t n = DB::parse<size_t>(argv[1]);
size_t method = DB::parse<size_t>(argv[2]);
std::vector<Key> data(n);
// srand(time(0));
{
Stopwatch watch;
for (auto & elem : data)
elem = rand();
watch.stop();
double elapsed = watch.elapsedSeconds();
std::cerr
<< "Filled in " << elapsed
<< " (" << n / elapsed << " elem/sec., "
<< n * sizeof(Key) / elapsed / 1048576 << " MB/sec.)"
<< std::endl;
}
if (n <= 100)
{
std::cerr << std::endl;
for (const auto & elem : data)
std::cerr << elem << ' ';
std::cerr << std::endl;
}
{
Stopwatch watch;
if (method == 1) sort1(&data[0], n);
if (method == 2) sort2(&data[0], n);
if (method == 3) sort3(&data[0], n);
watch.stop();
double elapsed = watch.elapsedSeconds();
std::cerr
<< "Sorted in " << elapsed
<< " (" << n / elapsed << " elem/sec., "
<< n * sizeof(Key) / elapsed / 1048576 << " MB/sec.)"
<< std::endl;
}
{
Stopwatch watch;
size_t i = 1;
while (i < n)
{
if (!(data[i - 1] <= data[i]))
break;
++i;
}
watch.stop();
double elapsed = watch.elapsedSeconds();
std::cerr
<< "Checked in " << elapsed
<< " (" << n / elapsed << " elem/sec., "
<< n * sizeof(Key) / elapsed / 1048576 << " MB/sec.)"
<< std::endl
<< "Result: " << (i == n ? "Ok." : "Fail!") << std::endl;
}
if (n <= 1000)
{
std::cerr << std::endl;
std::cerr << data[0] << ' ';
for (size_t i = 1; i < n; ++i)
{
if (!(data[i - 1] <= data[i]))
std::cerr << "*** ";
std::cerr << data[i] << ' ';
}
std::cerr << std::endl;
}
return 0;
}

View File

@ -1,4 +1,5 @@
#include <DB/Columns/ColumnsNumber.h>
#include <DB/Columns/ColumnsCommon.h>
#include <DB/DataStreams/FilterBlockInputStream.h>

View File

@ -111,48 +111,141 @@ Block MergingSortedBlockInputStream::readImpl()
return Block();
if (has_collation)
merge(merged_columns, queue_with_collation);
merge(merged_block, merged_columns, queue_with_collation);
else
merge(merged_columns, queue);
merge(merged_block, merged_columns, queue);
return merged_block;
}
template <typename TSortCursor>
void MergingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue)
void MergingSortedBlockInputStream::merge(Block & merged_block, ColumnPlainPtrs & merged_columns, std::priority_queue<TSortCursor> & queue)
{
size_t merged_rows = 0;
/** Увеличить счётчики строк.
* Вернуть true, если пора закончить формировать текущий блок данных.
*/
auto count_row_and_check_limit = [&, this]()
{
++total_merged_rows;
if (limit && total_merged_rows == limit)
{
// std::cerr << "Limit reached\n";
cancel();
finished = true;
return true;
}
++merged_rows;
if (merged_rows == max_block_size)
{
// std::cerr << "max_block_size reached\n";
return true;
}
return false;
};
/// Вынимаем строки в нужном порядке и кладём в merged_block, пока строк не больше max_block_size
while (!queue.empty())
{
TSortCursor current = queue.top();
queue.pop();
for (size_t i = 0; i < num_columns; ++i)
merged_columns[i]->insertFrom(*current->all_columns[i], current->pos);
while (true)
{
/** А вдруг для текущего курсора блок целиком меньше или равен, чем остальные?
* Или в очереди остался только один источник данных? Тогда можно целиком взять блок текущего курсора.
*/
if (current.impl->isFirst() && (queue.empty() || current.totallyLessOrEquals(queue.top())))
{
// std::cerr << "current block is totally less or equals\n";
if (!current->isLast())
{
current->next();
queue.push(current);
}
else
{
/// Достаём из соответствующего источника следующий блок, если есть.
fetchNextBlock(current, queue);
/// Если в текущем блоке уже есть данные, то сначала вернём его. Мы попадём сюда снова при следующем вызове функции merge.
if (merged_rows != 0)
{
// std::cerr << "merged rows is non-zero\n";
queue.push(current);
return;
}
size_t source_num = 0;
size_t size = cursors.size();
for (; source_num < size; ++source_num)
if (&cursors[source_num] == current.impl)
break;
if (source_num == size)
throw Exception("Logical error in MergingSortedBlockInputStream", ErrorCodes::LOGICAL_ERROR);
for (size_t i = 0; i < num_columns; ++i)
merged_block.unsafeGetByPosition(i).column = source_blocks[source_num].unsafeGetByPosition(i).column;
// std::cerr << "copied columns\n";
size_t merged_rows = merged_block.rows();
if (limit && total_merged_rows + merged_rows > limit)
{
merged_rows = limit - total_merged_rows;
for (size_t i = 0; i < num_columns; ++i)
{
auto & column = merged_block.unsafeGetByPosition(i).column;
column = column->cut(0, merged_rows);
}
cancel();
finished = true;
}
// std::cerr << "fetching next block\n";
total_merged_rows += merged_rows;
fetchNextBlock(current, queue);
return;
}
// std::cerr << "total_merged_rows: " << total_merged_rows << ", merged_rows: " << merged_rows << "\n";
// std::cerr << "Inserting row\n";
for (size_t i = 0; i < num_columns; ++i)
merged_columns[i]->insertFrom(*current->all_columns[i], current->pos);
if (!current->isLast())
{
// std::cerr << "moving to next row\n";
current->next();
if (queue.empty() || !(current.greater(queue.top())))
{
if (count_row_and_check_limit())
{
// std::cerr << "pushing back to queue\n";
queue.push(current);
return;
}
/// Не кладём курсор обратно в очередь, а продолжаем работать с текущим курсором.
// std::cerr << "current is still on top, using current row\n";
continue;
}
else
{
// std::cerr << "next row is not least, pushing back to queue\n";
queue.push(current);
}
}
else
{
/// Достаём из соответствующего источника следующий блок, если есть.
// std::cerr << "It was last row, fetching next block\n";
fetchNextBlock(current, queue);
}
break;
}
++total_merged_rows;
if (limit && total_merged_rows == limit)
{
cancel();
finished = true;
return;
}
++merged_rows;
if (merged_rows == max_block_size)
if (count_row_and_check_limit())
return;
}

View File

@ -63,12 +63,23 @@ FunctionFactory::FunctionFactory()
FunctionPtr FunctionFactory::get(
const String & name,
const Context & context) const
{
auto res = tryGet(name, context);
if (!res)
throw Exception("Unknown function " + name, ErrorCodes::UNKNOWN_FUNCTION);
return res;
}
FunctionPtr FunctionFactory::tryGet(
const String & name,
const Context & context) const
{
auto it = functions.find(name);
if (functions.end() != it)
return it->second(context);
else
throw Exception("Unknown function " + name, ErrorCodes::UNKNOWN_FUNCTION);
return {};
}
}

View File

@ -142,6 +142,34 @@ static inline const char * find_first_tab_lf_or_backslash(const char * begin, co
}
/** Распарсить escape-последовательность, которая может быть простой (один символ после бэкслеша) или более сложной (несколько символов).
* Предполагается, что курсор расположен на символе \
*/
static void parseComplexEscapeSequence(String & s, ReadBuffer & buf)
{
++buf.position();
if (buf.eof())
throw Exception("Cannot parse escape sequence", ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE);
if (*buf.position() == 'x')
{
++buf.position();
/// escape-последовательность вида \xAA
UInt8 c1;
UInt8 c2;
readPODBinary(c1, buf);
readPODBinary(c2, buf);
s += static_cast<char>(unhex(c1) * 16 + unhex(c2));
}
else
{
/// Обычная escape-последовательность из одного символа.
s += parseEscapeSequence(*buf.position());
++buf.position();
}
}
void readEscapedString(DB::String & s, DB::ReadBuffer & buf)
{
s = "";
@ -159,13 +187,7 @@ void readEscapedString(DB::String & s, DB::ReadBuffer & buf)
return;
if (*buf.position() == '\\')
{
++buf.position();
if (buf.eof())
throw Exception("Cannot parse escape sequence", DB::ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE);
s += DB::parseEscapeSequence(*buf.position());
++buf.position();
}
parseComplexEscapeSequence(s, buf);
}
}
@ -233,13 +255,7 @@ static void readAnyQuotedString(String & s, ReadBuffer & buf)
}
if (*buf.position() == '\\')
{
++buf.position();
if (buf.eof())
throw Exception("Cannot parse escape sequence", ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE);
s += parseEscapeSequence(*buf.position());
++buf.position();
}
parseComplexEscapeSequence(s, buf);
}
throw Exception("Cannot parse quoted string: expected closing quote",

View File

@ -432,7 +432,7 @@ void NO_INLINE Aggregator::executeImpl(
Arena * aggregates_pool,
size_t rows,
ConstColumnPlainPtrs & key_columns,
AggregateColumns & aggregate_columns,
AggregateFunctionInstruction * aggregate_instructions,
const Sizes & key_sizes,
StringRefs & keys,
bool no_more_keys,
@ -442,9 +442,9 @@ void NO_INLINE Aggregator::executeImpl(
state.init(key_columns);
if (!no_more_keys)
executeImplCase<false>(method, state, aggregates_pool, rows, key_columns, aggregate_columns, key_sizes, keys, overflow_row);
executeImplCase<false>(method, state, aggregates_pool, rows, key_columns, aggregate_instructions, key_sizes, keys, overflow_row);
else
executeImplCase<true>(method, state, aggregates_pool, rows, key_columns, aggregate_columns, key_sizes, keys, overflow_row);
executeImplCase<true>(method, state, aggregates_pool, rows, key_columns, aggregate_instructions, key_sizes, keys, overflow_row);
}
#ifndef __clang__
@ -459,7 +459,7 @@ void NO_INLINE Aggregator::executeImplCase(
Arena * aggregates_pool,
size_t rows,
ConstColumnPlainPtrs & key_columns,
AggregateColumns & aggregate_columns,
AggregateFunctionInstruction * aggregate_instructions,
const Sizes & key_sizes,
StringRefs & keys,
AggregateDataPtr overflow_row) const
@ -486,8 +486,8 @@ void NO_INLINE Aggregator::executeImplCase(
{
/// Добавляем значения в агрегатные функции.
AggregateDataPtr value = Method::getAggregateData(it->second);
for (size_t j = 0; j < aggregates_size; ++j) /// NOTE: Заменить индекс на два указателя?
aggregate_functions[j]->add(value + offsets_of_aggregate_states[j], &aggregate_columns[j][0], i);
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
(*inst->func)(inst->that, value + inst->state_offset, inst->arguments, i);
method.onExistingKey(key, keys, *aggregates_pool);
continue;
@ -534,8 +534,8 @@ void NO_INLINE Aggregator::executeImplCase(
AggregateDataPtr value = (!no_more_keys || !overflow) ? Method::getAggregateData(it->second) : overflow_row;
/// Добавляем значения в агрегатные функции.
for (size_t j = 0; j < aggregates_size; ++j)
aggregate_functions[j]->add(value + offsets_of_aggregate_states[j], &aggregate_columns[j][0], i);
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
(*inst->func)(inst->that, value + inst->state_offset, inst->arguments, i);
}
}
@ -546,7 +546,7 @@ void NO_INLINE Aggregator::executeImplCase(
void NO_INLINE Aggregator::executeWithoutKeyImpl(
AggregatedDataWithoutKey & res,
size_t rows,
AggregateColumns & aggregate_columns) const
AggregateFunctionInstruction * aggregate_instructions) const
{
/// Оптимизация в случае единственной агрегатной функции count.
AggregateFunctionCount * agg_count = aggregates_size == 1
@ -560,8 +560,8 @@ void NO_INLINE Aggregator::executeWithoutKeyImpl(
for (size_t i = 0; i < rows; ++i)
{
/// Добавляем значения
for (size_t j = 0; j < aggregates_size; ++j)
aggregate_functions[j]->add(res + offsets_of_aggregate_states[j], &aggregate_columns[j][0], i);
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
(*inst->func)(inst->that, res + inst->state_offset, inst->arguments, i);
}
}
}
@ -600,6 +600,9 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
}
}
AggregateFunctionInstructions aggregate_functions_instructions(aggregates_size + 1);
aggregate_functions_instructions[aggregates_size].that = nullptr;
for (size_t i = 0; i < aggregates_size; ++i)
{
for (size_t j = 0; j < aggregate_columns[i].size(); ++j)
@ -612,6 +615,11 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
aggregate_columns[i][j] = materialized_columns.back().get();
}
}
aggregate_functions_instructions[i].that = aggregate_functions[i];
aggregate_functions_instructions[i].func = aggregate_functions[i]->getAddressOfAddFunction();
aggregate_functions_instructions[i].state_offset = offsets_of_aggregate_states[i];
aggregate_functions_instructions[i].arguments = &aggregate_columns[i][0];
}
if (isCancelled())
@ -654,7 +662,7 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
(compiled_data->compiled_method_ptr)(*this, result.without_key, rows, aggregate_columns);
}
else
executeWithoutKeyImpl(result.without_key, rows, aggregate_columns);
executeWithoutKeyImpl(result.without_key, rows, &aggregate_functions_instructions[0]);
}
else
{
@ -700,7 +708,7 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
{
#define M(NAME, IS_TWO_LEVEL) \
else if (result.type == AggregatedDataVariants::Type::NAME) \
executeImpl(*result.NAME, result.aggregates_pool, rows, key_columns, aggregate_columns, \
executeImpl(*result.NAME, result.aggregates_pool, rows, key_columns, &aggregate_functions_instructions[0], \
result.key_sizes, key, no_more_keys, overflow_row_ptr);
if (false) {}

View File

@ -698,12 +698,6 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
interpreter_subquery->ignoreWithTotals();
}
/// если в настройках установлен default_sample != 1, то все запросы выполняем с сэмплингом
/// если таблица не поддерживает сэмплинг получим исключение
/// поэтому запросы типа SHOW TABLES работать с включенном default_sample не будут
if (!query.sample_size && settings.default_sample != 1)
query.sample_size = new ASTLiteral(StringRange(), Float64(settings.default_sample));
if (query.sample_size && (!storage || !storage->supportsSampling()))
throw Exception("Illegal SAMPLE: table doesn't support sampling", ErrorCodes::SAMPLING_NOT_SUPPORTED);

View File

@ -172,6 +172,50 @@ void stableGetPermutation(const Block & block, const SortDescription & descripti
}
bool isAlreadySorted(const Block & block, const SortDescription & description)
{
if (!block)
return true;
size_t rows = block.rows();
ColumnsWithSortDescriptions columns_with_sort_desc;
for (size_t i = 0, size = description.size(); i < size; ++i)
{
const IColumn * column = !description[i].column_name.empty()
? block.getByName(description[i].column_name).column
: block.getByPosition(description[i].column_number).column;
columns_with_sort_desc.push_back(std::make_pair(column, description[i]));
}
PartialSortingLess less(columns_with_sort_desc);
/** Если строк не слишком мало, то предпримем быструю попытку проверить, что блок не сортирован.
* Константы - наугад.
*/
static constexpr size_t num_rows_to_try = 10;
if (rows > num_rows_to_try * 5)
{
for (size_t i = 1; i < num_rows_to_try; ++i)
{
size_t prev_position = rows * (i - 1) / num_rows_to_try;
size_t curr_position = rows * i / num_rows_to_try;
if (less(curr_position, prev_position))
return false;
}
}
for (size_t i = 1; i < rows; ++i)
if (less(i, i - 1))
return false;
return true;
}
void stableSortBlock(Block & block, const SortDescription & description)
{
if (!block)

View File

@ -0,0 +1,40 @@
#include <DB/Parsers/ASTSampleRatio.h>
namespace DB
{
String ASTSampleRatio::toString(BigNum num)
{
if (num == 0)
return "0";
static const size_t MAX_WIDTH = 40;
char tmp[MAX_WIDTH];
char * pos;
for (pos = tmp + MAX_WIDTH - 1; num != 0; --pos)
{
*pos = '0' + num % 10;
num /= 10;
}
++pos;
return String(pos, tmp + MAX_WIDTH - pos);
}
String ASTSampleRatio::toString(Rational ratio)
{
if (ratio.denominator == 1)
return toString(ratio.numerator);
else
return toString(ratio.numerator) + " / " + toString(ratio.denominator);
}
}

View File

@ -375,40 +375,21 @@ bool ParserStringLiteral::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max
return false;
}
++pos;
ReadBuffer in(const_cast<char *>(pos), end - pos, 0);
while (pos != end)
try
{
size_t bytes = 0;
for (; pos + bytes != end; ++bytes)
if (pos[bytes] == '\\' || pos[bytes] == '\'')
break;
s.append(pos, bytes);
pos += bytes;
if (*pos == '\'')
{
++pos;
node = new ASTLiteral(StringRange(begin, pos), s);
return true;
}
if (*pos == '\\')
{
++pos;
if (pos == end)
{
expected = "escape sequence";
return false;
}
s += parseEscapeSequence(*pos);
++pos;
}
readQuotedString(s, in);
}
catch (const Exception & e)
{
expected = "string literal";
return false;
}
expected = "closing single quote";
return false;
pos += in.count();
node = new ASTLiteral(StringRange(begin, pos), s);
return true;
}
@ -497,6 +478,7 @@ const char * ParserAlias::restricted_keywords[] =
"FULL",
"CROSS",
"JOIN",
"GLOBAL",
"ANY",
"ALL",
"ON",

View File

@ -0,0 +1,128 @@
#include <DB/Parsers/CommonParsers.h>
#include <DB/Parsers/ParserSampleRatio.h>
#include <DB/Parsers/ASTSampleRatio.h>
#include <DB/IO/ReadHelpers.h>
namespace DB
{
static bool parseDecimal(IParser::Pos & pos, IParser::Pos end, ASTSampleRatio::Rational & res, IParser::Pos & max_parsed_pos)
{
ParserWhiteSpaceOrComments ws;
ws.ignore(pos, end);
UInt64 num_before = 0;
UInt64 num_after = 0;
Int64 exponent = 0;
IParser::Pos pos_after_first_num = tryReadIntText(num_before, pos, end);
bool has_num_before_point = pos_after_first_num > pos;
pos = pos_after_first_num;
bool has_point = pos < end && *pos == '.';
if (has_point)
++pos;
if (!has_num_before_point && !has_point)
return false;
size_t number_of_digits_after_point = 0;
if (has_point)
{
IParser::Pos pos_after_second_num = tryReadIntText(num_after, pos, end);
number_of_digits_after_point = pos_after_second_num - pos;
pos = pos_after_second_num;
}
bool has_exponent = pos < end && (*pos == 'e' || *pos == 'E');
if (has_exponent)
{
++pos;
IParser::Pos pos_after_exponent = tryReadIntText(exponent, pos, end);
if (pos_after_exponent == pos)
return false;
pos = pos_after_exponent;
}
res.numerator = num_before * exp10(number_of_digits_after_point) + num_after;
res.denominator = exp10(number_of_digits_after_point);
if (exponent > 0)
res.numerator *= exp10(exponent);
if (exponent < 0)
res.denominator *= exp10(-exponent);
/// NOTE Удаление общих степеней десяти из числителя и знаменателя - не нужно.
return true;
}
/** Возможные варианты:
*
* 12345
* - целое число
*
* 0.12345
* .12345
* 0.
* - дробь в обычной десятичной записи
*
* 1.23e-1
* - дробь в инженерной десятичной записи
*
* 123 / 456
* - дробь с произвольным знаменателем
*
* На всякий случай, в числителе и знаменателе дроби, поддерживаем предыдущие случаи.
* Пример:
* 123.0 / 456e0
*/
bool ParserSampleRatio::parseImpl(IParser::Pos & pos, IParser::Pos end, ASTPtr & node, IParser::Pos & max_parsed_pos, Expected & expected)
{
auto begin = pos;
ParserWhiteSpaceOrComments ws;
ASTSampleRatio::Rational numerator;
ASTSampleRatio::Rational denominator;
ASTSampleRatio::Rational res;
ws.ignore(pos, end);
if (!parseDecimal(pos, end, numerator, max_parsed_pos))
return false;
ws.ignore(pos, end);
bool has_slash = pos < end && *pos == '/';
if (has_slash)
{
++pos;
ws.ignore(pos, end);
if (!parseDecimal(pos, end, denominator, max_parsed_pos))
return false;
res.numerator = numerator.numerator * denominator.denominator;
res.denominator = numerator.denominator * denominator.numerator;
}
else
{
res = numerator;
}
ws.ignore(pos, end);
node = new ASTSampleRatio(StringRange(begin, pos), res);
return true;
}
}

View File

@ -6,6 +6,7 @@
#include <DB/Parsers/ExpressionListParsers.h>
#include <DB/Parsers/ParserJoin.h>
#include <DB/Parsers/ParserSetQuery.h>
#include <DB/Parsers/ParserSampleRatio.h>
#include <DB/Parsers/ParserSelectQuery.h>
namespace DB
@ -156,9 +157,9 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
{
ws.ignore(pos, end);
ParserNumber num;
ParserSampleRatio ratio;
if (!num.parse(pos, end, select_query->sample_size, max_parsed_pos, expected))
if (!ratio.parse(pos, end, select_query->sample_size, max_parsed_pos, expected))
return false;
ws.ignore(pos, end);
@ -168,9 +169,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
{
ws.ignore(pos, end);
ParserNumber num;
if (!num.parse(pos, end, select_query->sample_offset, max_parsed_pos, expected))
if (!ratio.parse(pos, end, select_query->sample_offset, max_parsed_pos, expected))
return false;
ws.ignore(pos, end);

View File

@ -210,7 +210,7 @@ public:
const MergeTreeSettings & settings = context.getMergeTreeSettings();
bool ok = true;
bool ok = /*true*/false;
std::stringstream message;
for (const auto & db : replicated_tables)

View File

@ -107,6 +107,11 @@ MergeTreeData::MergeTreeData(
ExpressionActionsPtr projected_expr = ExpressionAnalyzer(primary_expr_ast, context, nullptr, getColumnsList()).getActions(true);
primary_key_sample = projected_expr->getSampleBlock();
size_t primary_key_size = primary_key_sample.columns();
primary_key_data_types.resize(primary_key_size);
for (size_t i = 0; i < primary_key_size; ++i)
primary_key_data_types[i] = primary_key_sample.unsafeGetByPosition(i).type;
}
else if (mode != Unsorted)
throw Exception("Primary key could be empty only for UnsortedMergeTree", ErrorCodes::BAD_ARGUMENTS);
@ -1054,7 +1059,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartAndFixMetadata(const St
MergeTreePartChecker::Settings settings;
settings.setIndexGranularity(index_granularity);
settings.setRequireColumnFiles(true);
MergeTreePartChecker::checkDataPart(full_path + relative_path, settings, primary_key_sample, &part->checksums);
MergeTreePartChecker::checkDataPart(full_path + relative_path, settings, primary_key_data_types, &part->checksums);
{
WriteBufferFromFile out(full_path + relative_path + "/checksums.txt.tmp", 4096);

View File

@ -1,9 +1,12 @@
#include <boost/rational.hpp> /// Для вычислений, связанных с коэффициентами сэмплирования.
#include <DB/Core/FieldVisitors.h>
#include <DB/Storages/MergeTree/MergeTreeDataSelectExecutor.h>
#include <DB/Storages/MergeTree/MergeTreeBlockInputStream.h>
#include <DB/Storages/MergeTree/MergeTreeReadPool.h>
#include <DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/ASTSampleRatio.h>
#include <DB/DataStreams/ExpressionBlockInputStream.h>
#include <DB/DataStreams/FilterBlockInputStream.h>
#include <DB/DataStreams/CollapsingFinalBlockInputStream.h>
@ -13,6 +16,7 @@
#include <DB/DataStreams/SummingSortedBlockInputStream.h>
#include <DB/DataStreams/AggregatingSortedBlockInputStream.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeDate.h>
#include <DB/Common/VirtualColumnUtils.h>
@ -50,7 +54,7 @@ size_t MergeTreeDataSelectExecutor::getApproximateTotalRowsToRead(
for (size_t i = 0; i < parts.size(); ++i)
{
const MergeTreeData::DataPartPtr & part = parts[i];
MarkRanges ranges = markRangesFromPkRange(part->index, key_condition, settings);
MarkRanges ranges = markRangesFromPKRange(part->index, key_condition, settings);
/** Для того, чтобы получить оценку снизу количества строк, подходящих под условие на PK,
* учитываем только гарантированно полные засечки.
@ -65,10 +69,13 @@ size_t MergeTreeDataSelectExecutor::getApproximateTotalRowsToRead(
}
/** Пожалуй, наиболее удобный способ разбить диапазон 64-битных целых чисел на интервалы по их относительной величине
* - использовать для этого long double. Это некроссплатформенно. Надо, чтобы long double содержал хотя бы 64 бита мантиссы.
*/
using RelativeSize = long double;
using RelativeSize = boost::rational<ASTSampleRatio::BigNum>;
static std::ostream & operator<<(std::ostream & ostr, const RelativeSize & x)
{
ostr << ASTSampleRatio::toString(x.numerator()) << "/" << ASTSampleRatio::toString(x.denominator());
return ostr;
}
/// Переводит размер сэмпла в приблизительном количестве строк (вида SAMPLE 1000000) в относительную величину (вида SAMPLE 0.1).
@ -77,8 +84,10 @@ static RelativeSize convertAbsoluteSampleSizeToRelative(const ASTPtr & node, siz
if (approx_total_rows == 0)
return 1;
size_t absolute_sample_size = apply_visitor(FieldVisitorConvertToNumber<UInt64>(), typeid_cast<const ASTLiteral &>(*node).value);
return std::min(RelativeSize(1.0), RelativeSize(absolute_sample_size) / approx_total_rows);
const ASTSampleRatio & node_sample = typeid_cast<const ASTSampleRatio &>(*node);
auto absolute_sample_size = node_sample.ratio.numerator / node_sample.ratio.denominator;
return std::min(RelativeSize(1), RelativeSize(absolute_sample_size) / approx_total_rows);
}
@ -126,12 +135,17 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
PKCondition key_condition(query, context, data.getColumnsList(), data.getSortDescription());
PKCondition date_condition(query, context, data.getColumnsList(), SortDescription(1, SortColumnDescription(data.date_column_name, 1)));
if (settings.force_primary_key && key_condition.alwaysUnknown())
throw Exception("Primary key is not used and setting 'force_primary_key' is set.", ErrorCodes::INDEX_NOT_USED);
if (settings.force_index_by_date && date_condition.alwaysUnknown())
throw Exception("Index by date is not used and setting 'force_index_by_date' is set.", ErrorCodes::INDEX_NOT_USED);
/// Выберем куски, в которых могут быть данные, удовлетворяющие date_condition, и которые подходят под условие на _part,
/// а также max_block_number_to_read.
{
const DataTypes data_types_date { new DataTypeDate };
auto prev_parts = parts;
parts.clear();
@ -143,7 +157,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
Field left = static_cast<UInt64>(part->left_date);
Field right = static_cast<UInt64>(part->right_date);
if (!date_condition.mayBeTrueInRange(&left, &right))
if (!date_condition.mayBeTrueInRange(&left, &right, data_types_date))
continue;
if (max_block_number_to_read && part->right > max_block_number_to_read)
@ -166,16 +180,18 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
if (select.sample_size)
{
relative_sample_size = apply_visitor(FieldVisitorConvertToNumber<RelativeSize>(),
typeid_cast<ASTLiteral&>(*select.sample_size).value);
relative_sample_size.assign(
typeid_cast<const ASTSampleRatio &>(*select.sample_size).ratio.numerator,
typeid_cast<const ASTSampleRatio &>(*select.sample_size).ratio.denominator);
if (relative_sample_size < 0)
throw Exception("Negative sample size", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
relative_sample_offset = 0;
if (select.sample_offset)
relative_sample_offset = apply_visitor(FieldVisitorConvertToNumber<RelativeSize>(),
typeid_cast<ASTLiteral&>(*select.sample_offset).value);
relative_sample_offset.assign(
typeid_cast<const ASTSampleRatio &>(*select.sample_offset).ratio.numerator,
typeid_cast<const ASTSampleRatio &>(*select.sample_offset).ratio.denominator);
if (relative_sample_offset < 0)
throw Exception("Negative sample offset", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
@ -234,10 +250,8 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
* <------> - size
* <--><--> - кусочки для разных parallel_replica_offset, выбираем второй.
*
* TODO
* Очень важно, чтобы интервалы для разных parallel_replica_offset покрывали весь диапазон без пропусков и перекрытий.
* Также важно, чтобы весь юнивёрсум можно было покрыть, используя SAMPLE 0.1 OFFSET 0, ... OFFSET 0.9 и похожие десятичные дроби.
* Сейчас это не гарантируется.
*/
bool use_sampling = relative_sample_size > 0 || settings.parallel_replicas_count > 1;
@ -261,6 +275,9 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
if (settings.parallel_replicas_count > 1)
{
if (relative_sample_size == 0)
relative_sample_size = 1;
relative_sample_size /= settings.parallel_replicas_count;
relative_sample_offset += relative_sample_size * settings.parallel_replica_offset;
}
@ -272,23 +289,23 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
bool has_lower_limit = false;
bool has_upper_limit = false;
RelativeSize lower_limit_float = relative_sample_offset * size_of_universum;
RelativeSize upper_limit_float = (relative_sample_offset + relative_sample_size) * size_of_universum;
RelativeSize lower_limit_rational = relative_sample_offset * size_of_universum;
RelativeSize upper_limit_rational = (relative_sample_offset + relative_sample_size) * size_of_universum;
UInt64 lower = lower_limit_float;
UInt64 upper = upper_limit_float;
UInt64 lower = boost::rational_cast<ASTSampleRatio::BigNum>(lower_limit_rational);
UInt64 upper = boost::rational_cast<ASTSampleRatio::BigNum>(upper_limit_rational);
if (lower > 0)
has_lower_limit = true;
if (upper_limit_float <= size_of_universum)
if (upper_limit_rational < size_of_universum)
has_upper_limit = true;
/* std::cerr << std::fixed << std::setprecision(100)
/*std::cerr << std::fixed << std::setprecision(100)
<< "relative_sample_size: " << relative_sample_size << "\n"
<< "relative_sample_offset: " << relative_sample_offset << "\n"
<< "lower_limit_float: " << lower_limit_float << "\n"
<< "upper_limit_float: " << upper_limit_float << "\n"
<< "lower_limit_float: " << lower_limit_rational << "\n"
<< "upper_limit_float: " << upper_limit_rational << "\n"
<< "lower: " << lower << "\n"
<< "upper: " << upper << "\n";*/
@ -400,7 +417,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
RangesInDataPart ranges(part, (*part_index)++);
if (data.mode != MergeTreeData::Unsorted)
ranges.ranges = markRangesFromPkRange(part->index, key_condition, settings);
ranges.ranges = markRangesFromPKRange(part->index, key_condition, settings);
else
ranges.ranges = MarkRanges{MarkRange{0, part->size}};
@ -778,7 +795,7 @@ void MergeTreeDataSelectExecutor::createPositiveSignCondition(ExpressionActionsP
}
/// Получает набор диапазонов засечек, вне которых не могут находиться ключи из заданного диапазона.
MarkRanges MergeTreeDataSelectExecutor::markRangesFromPkRange(
MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange(
const MergeTreeData::DataPart::Index & index, const PKCondition & key_condition, const Settings & settings) const
{
size_t min_marks_for_seek = (settings.merge_tree_min_rows_for_seek + data.index_granularity - 1) / data.index_granularity;
@ -809,9 +826,9 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPkRange(
bool may_be_true;
if (range.end == marks_count)
may_be_true = key_condition.mayBeTrueAfter(&index[range.begin * key_size]);
may_be_true = key_condition.mayBeTrueAfter(&index[range.begin * key_size], data.primary_key_data_types);
else
may_be_true = key_condition.mayBeTrueInRange(&index[range.begin * key_size], &index[range.end * key_size]);
may_be_true = key_condition.mayBeTrueInRange(&index[range.begin * key_size], &index[range.end * key_size], data.primary_key_data_types);
if (!may_be_true)
continue;

View File

@ -109,8 +109,11 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithDa
IColumn::Permutation perm;
if (data.mode != MergeTreeData::Unsorted)
{
stableGetPermutation(block, sort_descr, perm);
perm_ptr = &perm;
if (!isAlreadySorted(block, sort_descr))
{
stableGetPermutation(block, sort_descr, perm);
perm_ptr = &perm;
}
}
NamesAndTypesList columns = data.getColumnsList().filter(block.getColumnsList().getNames());

View File

@ -252,7 +252,7 @@ static size_t checkColumn(const String & path, const String & name, DataTypePtr
void MergeTreePartChecker::checkDataPart(
String path,
const Settings & settings,
const Block & primary_key_sample,
const DataTypes & primary_key_data_types,
MergeTreeData::DataPart::Checksums * out_checksums)
{
if (!path.empty() && path.back() != '/')
@ -284,15 +284,15 @@ void MergeTreePartChecker::checkDataPart(
ReadBufferFromFile file_buf(path + "primary.idx");
HashingReadBuffer hashing_buf(file_buf);
if (primary_key_sample)
if (!primary_key_data_types.empty())
{
Field tmp_field;
size_t key_size = primary_key_sample.columns();
size_t key_size = primary_key_data_types.size();
while (!hashing_buf.eof())
{
++marks_in_primary_key;
for (size_t j = 0; j < key_size; ++j)
primary_key_sample.unsafeGetByPosition(j).type->deserializeBinary(tmp_field, hashing_buf);
primary_key_data_types[j].get()->deserializeBinary(tmp_field, hashing_buf);
}
}
else
@ -363,7 +363,7 @@ void MergeTreePartChecker::checkDataPart(
if (rows == Stream::UNKNOWN)
throw Exception("No columns", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
if (primary_key_sample)
if (!primary_key_data_types.empty())
{
const size_t expected_marks = (rows - 1) / settings.index_granularity + 1;
if (expected_marks != marks_in_primary_key)

View File

@ -4,6 +4,8 @@
#include <DB/Columns/ColumnSet.h>
#include <DB/Columns/ColumnTuple.h>
#include <DB/Parsers/ASTSet.h>
#include <DB/Functions/FunctionFactory.h>
namespace DB
{
@ -84,7 +86,8 @@ inline bool Range::less(const Field & lhs, const Field & rhs) { return apply_vis
Block PKCondition::getBlockWithConstants(
const ASTPtr & query, const Context & context, const NamesAndTypesList & all_columns)
{
Block result{
Block result
{
{ new ColumnConstUInt8{1, 0}, new DataTypeUInt8, "_dummy" }
};
@ -97,7 +100,7 @@ Block PKCondition::getBlockWithConstants(
}
PKCondition::PKCondition(ASTPtr query, const Context & context_, const NamesAndTypesList & all_columns, const SortDescription & sort_descr_)
PKCondition::PKCondition(ASTPtr & query, const Context & context, const NamesAndTypesList & all_columns, const SortDescription & sort_descr_)
: sort_descr(sort_descr_)
{
for (size_t i = 0; i < sort_descr.size(); ++i)
@ -110,23 +113,23 @@ PKCondition::PKCondition(ASTPtr query, const Context & context_, const NamesAndT
/** Вычисление выражений, зависящих только от констант.
* Чтобы индекс мог использоваться, если написано, например WHERE Date = toDate(now()).
*/
Block block_with_constants = getBlockWithConstants(query, context_, all_columns);
Block block_with_constants = getBlockWithConstants(query, context, all_columns);
/// Преобразуем секцию WHERE в обратную польскую строку.
ASTSelectQuery & select = typeid_cast<ASTSelectQuery &>(*query);
if (select.where_expression)
{
traverseAST(select.where_expression, block_with_constants);
traverseAST(select.where_expression, context, block_with_constants);
if (select.prewhere_expression)
{
traverseAST(select.prewhere_expression, block_with_constants);
traverseAST(select.prewhere_expression, context, block_with_constants);
rpn.emplace_back(RPNElement::FUNCTION_AND);
}
}
else if (select.prewhere_expression)
{
traverseAST(select.prewhere_expression, block_with_constants);
traverseAST(select.prewhere_expression, context, block_with_constants);
}
else
{
@ -146,11 +149,11 @@ bool PKCondition::addCondition(const String & column, const Range & range)
/** Получить значение константного выражения.
* Вернуть false, если выражение не константно.
*/
static bool getConstant(ASTPtr & expr, Block & block_with_constants, Field & value)
static bool getConstant(const ASTPtr & expr, Block & block_with_constants, Field & value)
{
String column_name = expr->getColumnName();
if (ASTLiteral * lit = typeid_cast<ASTLiteral *>(&*expr))
if (const ASTLiteral * lit = typeid_cast<const ASTLiteral *>(&*expr))
{
/// литерал
value = lit->value;
@ -166,7 +169,7 @@ static bool getConstant(ASTPtr & expr, Block & block_with_constants, Field & val
return false;
}
void PKCondition::traverseAST(ASTPtr & node, Block & block_with_constants)
void PKCondition::traverseAST(ASTPtr & node, const Context & context, Block & block_with_constants)
{
RPNElement element;
@ -174,10 +177,10 @@ void PKCondition::traverseAST(ASTPtr & node, Block & block_with_constants)
{
if (operatorFromAST(func, element))
{
ASTs & args = typeid_cast<ASTExpressionList &>(*func->arguments).children;
for (size_t i = 0; i < args.size(); ++i)
auto & args = typeid_cast<ASTExpressionList &>(*func->arguments).children;
for (size_t i = 0, size = args.size(); i < size; ++i)
{
traverseAST(args[i], block_with_constants);
traverseAST(args[i], context, block_with_constants);
/** Первая часть условия - для корректной поддержки функций and и or произвольной арности
* - в этом случае добавляется n - 1 элементов (где n - количество аргументов).
@ -190,7 +193,7 @@ void PKCondition::traverseAST(ASTPtr & node, Block & block_with_constants)
}
}
if (!atomFromAST(node, block_with_constants, element))
if (!atomFromAST(node, context, block_with_constants, element))
{
element.function = RPNElement::FUNCTION_UNKNOWN;
}
@ -198,12 +201,74 @@ void PKCondition::traverseAST(ASTPtr & node, Block & block_with_constants)
rpn.push_back(element);
}
bool PKCondition::atomFromAST(ASTPtr & node, Block & block_with_constants, RPNElement & out)
bool PKCondition::isPrimaryKeyPossiblyWrappedByMonotonicFunctions(
const ASTPtr & node,
const Context & context,
size_t & out_primary_key_column_num,
RPNElement::MonotonicFunctionsChain & out_functions_chain)
{
/// Фнукции < > = != <= >= in notIn, у которых один агрумент константа, другой - один из столбцов первичного ключа.
if (ASTFunction * func = typeid_cast<ASTFunction *>(&*node))
std::vector<const ASTFunction *> chain_not_tested_for_monotonicity;
if (!isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl(node, out_primary_key_column_num, chain_not_tested_for_monotonicity))
return false;
for (auto it = chain_not_tested_for_monotonicity.rbegin(); it != chain_not_tested_for_monotonicity.rend(); ++it)
{
ASTs & args = typeid_cast<ASTExpressionList &>(*func->arguments).children;
FunctionPtr func = FunctionFactory::instance().tryGet((*it)->name, context);
if (!func || !func->hasInformationAboutMonotonicity())
return false;
out_functions_chain.push_back(func);
}
return true;
}
bool PKCondition::isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl(
const ASTPtr & node,
size_t & out_primary_key_column_num,
std::vector<const ASTFunction *> & out_functions_chain)
{
/** Сам по себе, столбец первичного ключа может быть функциональным выражением. Например, intHash32(UserID).
* Поэтому, используем полное имя выражения для поиска.
*/
String name = node->getColumnName();
auto it = pk_columns.find(name);
if (pk_columns.end() != it)
{
out_primary_key_column_num = it->second;
return true;
}
if (const ASTFunction * func = typeid_cast<const ASTFunction *>(node.get()))
{
const auto & args = func->arguments->children;
if (args.size() != 1)
return false;
out_functions_chain.push_back(func);
if (!isPrimaryKeyPossiblyWrappedByMonotonicFunctionsImpl(args[0], out_primary_key_column_num, out_functions_chain))
return false;
return true;
}
return false;
}
bool PKCondition::atomFromAST(ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out)
{
/** Функции < > = != <= >= in notIn, у которых один агрумент константа, другой - один из столбцов первичного ключа,
* либо он же, завёрнутый в цепочку возможно-монотонных функций.
*/
if (const ASTFunction * func = typeid_cast<const ASTFunction *>(&*node))
{
const ASTs & args = typeid_cast<const ASTExpressionList &>(*func->arguments).children;
if (args.size() != 2)
return false;
@ -212,21 +277,19 @@ bool PKCondition::atomFromAST(ASTPtr & node, Block & block_with_constants, RPNEl
bool inverted;
size_t column;
Field value;
RPNElement::MonotonicFunctionsChain chain;
if (pk_columns.count(args[0]->getColumnName()) && getConstant(args[1], block_with_constants, value))
if (getConstant(args[1], block_with_constants, value) && isPrimaryKeyPossiblyWrappedByMonotonicFunctions(args[0], context, column, chain))
{
inverted = false;
column = pk_columns[args[0]->getColumnName()];
}
else if (pk_columns.count(args[1]->getColumnName()) && getConstant(args[0], block_with_constants, value))
else if (getConstant(args[0], block_with_constants, value) && isPrimaryKeyPossiblyWrappedByMonotonicFunctions(args[1], context, column, chain))
{
inverted = true;
column = pk_columns[args[1]->getColumnName()];
}
else if (pk_columns.count(args[0]->getColumnName()) && typeid_cast<ASTSet *>(args[1].get()))
else if (typeid_cast<const ASTSet *>(args[1].get()) && isPrimaryKeyPossiblyWrappedByMonotonicFunctions(args[0], context, column, chain))
{
inverted = false;
column = pk_columns[args[0]->getColumnName()];
}
else
return false;
@ -252,6 +315,7 @@ bool PKCondition::atomFromAST(ASTPtr & node, Block & block_with_constants, RPNEl
}
out.key_column = column;
out.monotonic_functions_chain = std::move(chain);
const auto atom_it = atom_map.find(func_name);
if (atom_it == std::end(atom_map))
@ -265,10 +329,10 @@ bool PKCondition::atomFromAST(ASTPtr & node, Block & block_with_constants, RPNEl
return false;
}
bool PKCondition::operatorFromAST(ASTFunction * func, RPNElement & out)
bool PKCondition::operatorFromAST(const ASTFunction * func, RPNElement & out)
{
/// Функции AND, OR, NOT.
ASTs & args = typeid_cast<ASTExpressionList &>(*func->arguments).children;
const ASTs & args = typeid_cast<const ASTExpressionList &>(*func->arguments).children;
if (func->name == "not")
{
@ -302,7 +366,27 @@ String PKCondition::toString() const
return res;
}
bool PKCondition::mayBeTrueInRange(const Field * left_pk, const Field * right_pk, bool right_bounded) const
static void applyFunction(
FunctionPtr & func,
const DataTypePtr & arg_type, const Field & arg_value,
DataTypePtr & res_type, Field & res_value)
{
res_type = func->getReturnType({arg_type});
Block block
{
{ arg_type->createConstColumn(1, arg_value), arg_type, "x" },
{ nullptr, res_type, "y" }
};
func->execute(block, {0}, 1);
block.getByPosition(1).column->get(0, res_value);
}
bool PKCondition::mayBeTrueInRange(const Field * left_pk, const Field * right_pk, const DataTypes & data_types, bool right_bounded) const
{
/// Найдем диапазоны элементов ключа.
std::vector<Range> key_ranges(sort_descr.size(), Range());
@ -335,32 +419,84 @@ bool PKCondition::mayBeTrueInRange(const Field * left_pk, const Field * right_pk
{
rpn_stack.emplace_back(true, true);
}
else if (element.function == RPNElement::FUNCTION_NOT_IN_RANGE || element.function == RPNElement::FUNCTION_IN_RANGE)
else if (element.function == RPNElement::FUNCTION_IN_RANGE
|| element.function == RPNElement::FUNCTION_NOT_IN_RANGE
|| element.function == RPNElement::FUNCTION_IN_SET
|| element.function == RPNElement::FUNCTION_NOT_IN_SET)
{
const Range & key_range = key_ranges[element.key_column];
bool intersects = element.range.intersectsRange(key_range);
bool contains = element.range.containsRange(key_range);
const Range * key_range = &key_ranges[element.key_column];
rpn_stack.emplace_back(intersects, !contains);
if (element.function == RPNElement::FUNCTION_NOT_IN_RANGE)
rpn_stack.back() = !rpn_stack.back();
}
else if (element.function == RPNElement::FUNCTION_IN_SET || element.function == RPNElement::FUNCTION_NOT_IN_SET)
{
auto in_func = typeid_cast<const ASTFunction *>(element.in_function.get());
const ASTs & args = typeid_cast<const ASTExpressionList &>(*in_func->arguments).children;
auto ast_set = typeid_cast<const ASTSet *>(args[1].get());
if (in_func && ast_set)
/// Случай, когда столбец обёрнут в цепочку возможно-монотонных функций.
Range key_range_transformed;
bool evaluation_is_not_possible = false;
if (!element.monotonic_functions_chain.empty())
{
const Range & key_range = key_ranges[element.key_column];
key_range_transformed = *key_range;
DataTypePtr current_type = data_types[element.key_column];
for (auto & func : element.monotonic_functions_chain)
{
/// Проверяем монотонность каждой функции на конкретном диапазоне.
IFunction::Monotonicity monotonicity = func->getMonotonicityForRange(
*current_type.get(), key_range_transformed.left, key_range_transformed.right);
rpn_stack.push_back(ast_set->set->mayBeTrueInRange(key_range));
if (element.function == RPNElement::FUNCTION_NOT_IN_SET)
/* std::cerr << "Function " << func->getName() << " is " << (monotonicity.is_monotonic ? "" : "not ")
<< "monotonic " << (monotonicity.is_monotonic ? (monotonicity.is_positive ? "(positive) " : "(negative) ") : "")
<< "in range "
<< "[" << apply_visitor(FieldVisitorToString(), key_range_transformed.left)
<< ", " << apply_visitor(FieldVisitorToString(), key_range_transformed.right) << "]\n";*/
if (!monotonicity.is_monotonic)
{
evaluation_is_not_possible = true;
break;
}
/// Вычисляем функцию.
DataTypePtr new_type;
if (!key_range_transformed.left.isNull())
applyFunction(func, current_type, key_range_transformed.left, new_type, key_range_transformed.left);
if (!key_range_transformed.right.isNull())
applyFunction(func, current_type, key_range_transformed.right, new_type, key_range_transformed.right);
current_type.swap(new_type);
if (!monotonicity.is_positive)
key_range_transformed.swapLeftAndRight();
}
if (evaluation_is_not_possible)
{
rpn_stack.emplace_back(true, true);
continue;
}
key_range = &key_range_transformed;
}
if (element.function == RPNElement::FUNCTION_IN_RANGE
|| element.function == RPNElement::FUNCTION_NOT_IN_RANGE)
{
bool intersects = element.range.intersectsRange(*key_range);
bool contains = element.range.containsRange(*key_range);
rpn_stack.emplace_back(intersects, !contains);
if (element.function == RPNElement::FUNCTION_NOT_IN_RANGE)
rpn_stack.back() = !rpn_stack.back();
}
else
else /// Set
{
throw DB::Exception("Set for IN is not created yet!", ErrorCodes::LOGICAL_ERROR);
auto in_func = typeid_cast<const ASTFunction *>(element.in_function.get());
const ASTs & args = typeid_cast<const ASTExpressionList &>(*in_func->arguments).children;
auto ast_set = typeid_cast<const ASTSet *>(args[1].get());
if (in_func && ast_set)
{
rpn_stack.push_back(ast_set->set->mayBeTrueInRange(*key_range));
if (element.function == RPNElement::FUNCTION_NOT_IN_SET)
rpn_stack.back() = !rpn_stack.back();
}
else
{
throw DB::Exception("Set for IN is not created yet!", ErrorCodes::LOGICAL_ERROR);
}
}
}
else if (element.function == RPNElement::FUNCTION_NOT)
@ -391,14 +527,14 @@ bool PKCondition::mayBeTrueInRange(const Field * left_pk, const Field * right_pk
return rpn_stack[0].can_be_true;
}
bool PKCondition::mayBeTrueInRange(const Field * left_pk, const Field * right_pk) const
bool PKCondition::mayBeTrueInRange(const Field * left_pk, const Field * right_pk, const DataTypes & data_types) const
{
return mayBeTrueInRange(left_pk, right_pk, true);
return mayBeTrueInRange(left_pk, right_pk, data_types, true);
}
bool PKCondition::mayBeTrueAfter(const Field * left_pk) const
bool PKCondition::mayBeTrueAfter(const Field * left_pk, const DataTypes & data_types) const
{
return mayBeTrueInRange(left_pk, nullptr, false);
return mayBeTrueInRange(left_pk, nullptr, data_types, false);
}
static const ASTSet & inFunctionToSet(const ASTPtr & in_function)
@ -411,6 +547,17 @@ static const ASTSet & inFunctionToSet(const ASTPtr & in_function)
String PKCondition::RPNElement::toString() const
{
auto print_wrapped_column = [this](std::ostringstream & ss)
{
for (auto it = monotonic_functions_chain.rbegin(); it != monotonic_functions_chain.rend(); ++it)
ss << (*it)->getName() << "(";
ss << "column " << key_column;
for (auto it = monotonic_functions_chain.rbegin(); it != monotonic_functions_chain.rend(); ++it)
ss << ")";
};
std::ostringstream ss;
switch (function)
{
@ -425,14 +572,19 @@ String PKCondition::RPNElement::toString() const
case FUNCTION_NOT_IN_SET:
case FUNCTION_IN_SET:
{
ss << "(column " << key_column << (function == FUNCTION_IN_SET ? " in " : " notIn ")
<< inFunctionToSet(in_function).set->describe() << ")";
ss << "(";
print_wrapped_column(ss);
ss << (function == FUNCTION_IN_SET ? " in " : " notIn ") << inFunctionToSet(in_function).set->describe();
ss << ")";
return ss.str();
}
case FUNCTION_IN_RANGE:
case FUNCTION_NOT_IN_RANGE:
{
ss << "(column " << key_column << (function == FUNCTION_NOT_IN_RANGE ? " not" : "") << " in " << range.toString() << ")";
ss << "(";
print_wrapped_column(ss);
ss << (function == FUNCTION_NOT_IN_RANGE ? " not" : "") << " in " << range.toString();
ss << ")";
return ss.str();
}
default:

View File

@ -2231,7 +2231,7 @@ void StorageReplicatedMergeTree::checkPart(const String & part_name)
settings.setRequireChecksums(true);
settings.setRequireColumnFiles(true);
MergeTreePartChecker::checkDataPart(
data.getFullPath() + part_name, settings, data.primary_key_sample);
data.getFullPath() + part_name, settings, data.primary_key_data_types);
LOG_INFO(log, "Checker: Part " << part_name << " looks good.");
}

View File

@ -23,6 +23,7 @@ StorageSystemDictionaries::StorageSystemDictionaries(const std::string & name)
{ "name", new DataTypeString },
{ "origin", new DataTypeString },
{ "type", new DataTypeString },
{ "key", new DataTypeString },
{ "attribute.names", new DataTypeArray{new DataTypeString} },
{ "attribute.types", new DataTypeArray{new DataTypeString} },
{ "bytes_allocated", new DataTypeUInt64 },
@ -57,6 +58,7 @@ BlockInputStreams StorageSystemDictionaries::read(
ColumnWithTypeAndName col_name{new ColumnString, new DataTypeString, "name"};
ColumnWithTypeAndName col_origin{new ColumnString, new DataTypeString, "origin"};
ColumnWithTypeAndName col_type{new ColumnString, new DataTypeString, "type"};
ColumnWithTypeAndName col_key{new ColumnString, new DataTypeString, "key"};
ColumnWithTypeAndName col_attribute_names{
new ColumnArray{new ColumnString},
new DataTypeArray{new DataTypeString},
@ -92,6 +94,8 @@ BlockInputStreams StorageSystemDictionaries::read(
col_type.column->insert(dict_ptr->getTypeName());
const auto & dict_struct = dict_ptr->getStructure();
col_key.column->insert(dict_struct.getKeyDescription());
col_attribute_names.column->insert(ext::map<Array>(dict_struct.attributes, [] (auto & attr) -> decltype(auto) {
return attr.name;
}));
@ -109,6 +113,7 @@ BlockInputStreams StorageSystemDictionaries::read(
else
{
col_type.column->insertDefault();
col_key.column->insertDefault();
col_attribute_names.column->insertDefault();
col_attribute_types.column->insertDefault();
col_bytes_allocated.column->insertDefault();
@ -152,6 +157,7 @@ BlockInputStreams StorageSystemDictionaries::read(
col_name,
col_origin,
col_type,
col_key,
col_attribute_names,
col_attribute_types,
col_bytes_allocated,

View File

@ -25,7 +25,7 @@ int main(int argc, char ** argv)
settings.setRequireColumnFiles(argv[2][0] == '1');
settings.setVerbose(true);
MergeTreePartChecker::checkDataPart(argv[1], settings, Block());
MergeTreePartChecker::checkDataPart(argv[1], settings, {});
}
catch (...)
{

View File

@ -12,7 +12,7 @@ void check(UInt64 left, UInt64 right, bool can_be_true)
{
Field fleft = left;
Field fright = right;
if (cond->mayBeTrueInRange(&fleft, &fright) != can_be_true)
if (cond->mayBeTrueInRange(&fleft, &fright, {}) != can_be_true)
{
std::cout << "failed range [" << left << ", " << right << "]" << std::endl;
exit(2);

View File

@ -0,0 +1,28 @@
256 0 255 32640 256
256 0 255 32640 256
25 0 24 300 25
25 0 24 300 25
25 0 24 300 25
25 0 24 300 25
25 0 24 300 25
5 0 4 10 5
26 25 50 975 26
26 230 255 6305 26
13 243 255 3237 13
1
256 0 255 32640 256
128 0 127 8128 128
128 128 255 24512 128
42 0 41 861 42
42 0 41 861 42
43 42 84 2709 43
43 85 127 4558 43
42 128 169 6237 42
42 128 169 6237 42
43 170 212 8213 43
43 213 255 10062 43
256 0 255 32640 256
13 89 101 1235 13
13 102 114 1404 13
256
65536

View File

@ -1,13 +1,39 @@
DROP TABLE IF EXISTS test.sample;
SET max_block_size = 10;
CREATE TABLE test.sample (d Date DEFAULT '2000-01-01', x UInt8) ENGINE = MergeTree(d, x, x, 10);
INSERT INTO test.sample (x) SELECT toUInt8(number) AS x FROM system.numbers LIMIT 256;
SELECT x FROM test.sample;
SELECT x FROM test.sample SAMPLE 1;
SELECT x FROM test.sample SAMPLE 0.1;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 0.1;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/10;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/1e1;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1e1/1e2;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1e-1;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 2e-2;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/10 OFFSET 1/10;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/10 OFFSET 9/10;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/10 OFFSET 10/10;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/10 OFFSET 19/20;
SELECT x FROM
SELECT count() >= 100 FROM test.sample SAMPLE 100;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1000;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/2;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/2 OFFSET 1/2;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/2 SETTINGS parallel_replicas_count = 3;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/2 SETTINGS parallel_replicas_count = 3, parallel_replica_offset = 0;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/2 SETTINGS parallel_replicas_count = 3, parallel_replica_offset = 1;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/2 SETTINGS parallel_replicas_count = 3, parallel_replica_offset = 2;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/2 OFFSET 1/2 SETTINGS parallel_replicas_count = 3;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/2 OFFSET 1/2 SETTINGS parallel_replicas_count = 3, parallel_replica_offset = 0;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/2 OFFSET 1/2 SETTINGS parallel_replicas_count = 3, parallel_replica_offset = 1;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 1/2 OFFSET 1/2 SETTINGS parallel_replicas_count = 3, parallel_replica_offset = 2;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM
(
SELECT x FROM test.sample SAMPLE 0.1 OFFSET 0.0
UNION ALL SELECT x FROM test.sample SAMPLE 0.1 OFFSET 0.1
@ -19,11 +45,131 @@ UNION ALL SELECT x FROM test.sample SAMPLE 0.1 OFFSET 0.6
UNION ALL SELECT x FROM test.sample SAMPLE 0.1 OFFSET 0.7
UNION ALL SELECT x FROM test.sample SAMPLE 0.1 OFFSET 0.8
UNION ALL SELECT x FROM test.sample SAMPLE 0.1 OFFSET 0.9
)
ORDER BY x;
);
SELECT x FROM test.sample SAMPLE 0.05 OFFSET 0.35;
SELECT x FROM test.sample SAMPLE 0.05 OFFSET 0.4;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 0.05 OFFSET 0.35;
SELECT count(), min(x), max(x), sum(x), uniqExact(x) FROM test.sample SAMPLE 0.05 OFFSET 0.4;
SELECT count()
FROM
(
SELECT
x,
count() AS c
FROM
(
SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.00
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.01
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.02
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.03
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.04
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.05
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.06
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.07
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.08
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.09
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.10
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.11
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.12
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.13
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.14
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.15
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.16
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.17
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.18
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.19
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.20
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.21
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.22
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.23
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.24
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.25
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.26
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.27
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.28
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.29
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.30
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.31
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.32
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.33
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.34
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.35
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.36
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.37
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.38
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.39
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.40
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.41
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.42
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.43
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.44
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.45
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.46
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.47
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.48
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.49
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.50
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.51
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.52
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.53
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.54
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.55
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.56
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.57
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.58
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.59
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.60
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.61
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.62
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.63
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.64
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.65
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.66
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.67
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.68
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.69
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.70
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.71
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.72
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.73
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.74
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.75
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.76
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.77
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.78
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.79
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.80
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.81
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.82
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.83
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.84
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.85
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.86
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.87
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.88
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.89
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.90
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.91
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.92
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.93
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.94
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.95
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.96
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.97
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.98
UNION ALL SELECT * FROM test.sample SAMPLE 0.01 OFFSET 0.99
)
GROUP BY x
HAVING c = 1
ORDER BY x ASC
);
DROP TABLE test.sample;
SET max_block_size = 8192;
CREATE TABLE test.sample (d Date DEFAULT '2000-01-01', x UInt16) ENGINE = MergeTree(d, x, x, 10);
INSERT INTO test.sample (x) SELECT toUInt16(number) AS x FROM system.numbers LIMIT 65536;
SELECT count()
FROM

View File

@ -0,0 +1,2 @@
2000
2000

View File

@ -0,0 +1,2 @@
SELECT sum(length(arr)) FROM (SELECT arrayMap(x -> toString(x), range(number % 10)) AS arr FROM (SELECT * FROM system.numbers LIMIT 1000) WHERE length(arr) % 2 = 0);
SELECT sum(length(arr)) FROM (SELECT range(number % 10) AS arr FROM (SELECT * FROM system.numbers LIMIT 1000) WHERE length(arr) % 2 = 0);

View File

@ -0,0 +1,23 @@
1000000
0
1
2
3
4
5
6
7
8
9
2000000
0
1
2
3
4
5
6
7
8
9
999

View File

@ -0,0 +1,16 @@
DROP TABLE IF EXISTS test.sorted;
CREATE TABLE test.sorted (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 8192);
INSERT INTO test.sorted (x) SELECT intDiv(number, 100000) AS x FROM system.numbers LIMIT 1000000;
SET max_threads = 1;
SELECT count() FROM test.sorted;
SELECT DISTINCT x FROM test.sorted;
INSERT INTO test.sorted (x) SELECT (intHash64(number) % 1000 = 0 ? 999 : intDiv(number, 100000)) AS x FROM system.numbers LIMIT 1000000;
SELECT count() FROM test.sorted;
SELECT DISTINCT x FROM test.sorted;
DROP TABLE test.sorted;

View File

@ -0,0 +1,5 @@
[1000,1,10,50,900,200,300,600,500,400,700,800,100,950,990,999,0,500,300,400]
[1000,1,10,50,900,200,300,600,500,400,700,800,100,950,990,999,0,500,300,400]
[1001,1,10,50,900,200,300,600,500,400,700,800,100,950,990,999,0,500,300,400]
[1000,1,10,50,900,200,300,600,500,400,700,800,100,950,990,999,0,500,300,400]
[1000,1,10,50,900,200,300,600,500,400,700,800,100,950,990,999,0,500,300,400]

View File

@ -0,0 +1,7 @@
SELECT quantilesExact(1, 0.001, 0.01, 0.05, 0.9, 0.2, 0.3, 0.6, 0.5, 0.4, 0.7, 0.8, 0.1, 0.95, 0.99, 0.999, 0, 0.5, 0.3, 0.4)(x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001);
SELECT quantilesExactWeighted(1, 0.001, 0.01, 0.05, 0.9, 0.2, 0.3, 0.6, 0.5, 0.4, 0.7, 0.8, 0.1, 0.95, 0.99, 0.999, 0, 0.5, 0.3, 0.4)(x, 1) FROM (SELECT number AS x FROM system.numbers LIMIT 1001);
SELECT quantilesTiming(1, 0.001, 0.01, 0.05, 0.9, 0.2, 0.3, 0.6, 0.5, 0.4, 0.7, 0.8, 0.1, 0.95, 0.99, 0.999, 0, 0.5, 0.3, 0.4)(x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001);
-- SELECT quantilesTDigest(1, 0.001, 0.01, 0.05, 0.9, 0.2, 0.3, 0.6, 0.5, 0.4, 0.7, 0.8, 0.1, 0.95, 0.99, 0.999, 0, 0.5, 0.3, 0.4)(x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001);
-- SELECT quantilesTDigestWeighted(1, 0.001, 0.01, 0.05, 0.9, 0.2, 0.3, 0.6, 0.5, 0.4, 0.7, 0.8, 0.1, 0.95, 0.99, 0.999, 0, 0.5, 0.3, 0.4)(x, 1) FROM (SELECT number AS x FROM system.numbers LIMIT 1001);
SELECT quantiles(1, 0.001, 0.01, 0.05, 0.9, 0.2, 0.3, 0.6, 0.5, 0.4, 0.7, 0.8, 0.1, 0.95, 0.99, 0.999, 0, 0.5, 0.3, 0.4)(x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001);
SELECT quantilesDeterministic(1, 0.001, 0.01, 0.05, 0.9, 0.2, 0.3, 0.6, 0.5, 0.4, 0.7, 0.8, 0.1, 0.95, 0.99, 0.999, 0, 0.5, 0.3, 0.4)(x, x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001);

View File

@ -0,0 +1 @@
0 Р

View File

@ -0,0 +1 @@
SELECT '\x30 \xD0\xA0';

View File

@ -0,0 +1,2 @@
1 Hello
2 Hello

View File

@ -0,0 +1,2 @@
SET compile = 1, min_count_to_compile = 0, max_threads = 1;
SELECT arrayJoin([1, 2, 1]) AS UserID, argMax('Hello', today()) AS res GROUP BY UserID;

View File

@ -0,0 +1,394 @@
2000-01-01 1
2000-01-01 2
2000-01-01 3
2000-01-01 4
2000-01-01 5
2000-01-01 6
2000-01-01 1
2000-01-01 2
2000-01-01 3
2000-01-01 4
2000-01-01 5
2000-01-01 6
2000-01-01 0
2000-01-01 1
2000-01-01 2
2000-01-01 3
2000-01-01 4
2000-01-01 5
2000-01-01 6
2000-01-01 7
2000-01-01 8
2000-01-01 9
2000-01-01 10
2000-01-01 11
2000-01-01 12
2000-01-01 13
2000-01-01 14
2000-01-01 15
2000-01-01 16
2000-01-01 17
2000-01-01 18
2000-01-01 19
2000-01-01 0
2000-01-01 1
2000-01-01 2
2000-01-01 3
2000-01-01 4
2000-01-01 5
2000-01-01 6
2000-01-01 7
2000-01-01 8
2000-01-01 9
2000-01-01 10
2000-01-01 11
2000-01-01 12
2000-01-01 13
2000-01-01 14
2000-01-01 15
2000-01-01 16
2000-01-01 17
2000-01-01 18
2000-01-01 19
2000-01-01 5
2000-01-01 6
2000-01-01 7
2000-01-01 8
2000-01-01 9
2000-01-01 10
2000-01-01 11
2000-01-01 12
2000-01-01 13
2000-01-01 14
2000-01-01 0
2000-01-01 1
2000-01-01 2
2000-01-01 3
2000-01-01 4
2000-01-01 5
2000-01-01 6
2000-01-01 7
2000-01-01 8
2000-01-01 9
2000-01-01 0
2000-01-01 1
2000-01-01 2
2000-01-01 3
2000-01-01 4
2000-01-01 5
2000-01-01 5
2000-01-01 6
2000-01-01 6
2000-01-01 7
2000-01-01 7
2000-01-01 8
2000-01-01 8
2000-01-01 9
2000-01-01 9
2000-01-01 10
2000-01-01 11
2000-01-01 12
2000-01-01 13
2000-01-01 14
2000-01-01 5
2000-01-01 6
2000-01-01 7
2000-01-01 8
2000-01-01 9
2000-01-01 10
2000-01-01 11
2000-01-01 12
2000-01-01 13
2000-01-01 14
2000-01-01 0
2000-01-01 1
2000-01-01 2
2000-01-01 3
2000-01-01 4
2000-01-01 5
2000-01-01 6
2000-01-01 7
2000-01-01 8
2000-01-01 9
2000-01-01 9
2000-01-01 10
2000-01-01 11
2000-01-01 12
2000-01-01 13
2000-01-01 14
2000-01-01 15
2000-01-01 16
2000-01-01 17
2000-01-01 18
2000-01-01 0
2000-01-01 1
2000-01-01 2
2000-01-01 3
2000-01-01 4
2000-01-01 5
2000-01-01 5
2000-01-01 6
2000-01-01 6
2000-01-01 7
2000-01-01 7
2000-01-01 8
2000-01-01 8
2000-01-01 9
2000-01-01 9
2000-01-01 9
2000-01-01 10
2000-01-01 10
2000-01-01 11
2000-01-01 11
2000-01-01 12
2000-01-01 12
2000-01-01 13
2000-01-01 13
2000-01-01 14
2000-01-01 14
2000-01-01 15
2000-01-01 16
2000-01-01 17
2000-01-01 18
2000-01-01 0
2000-01-01 1
2000-01-01 2
2000-01-01 3
2000-01-01 4
2000-01-01 5
2000-01-01 6
2000-01-01 7
2000-01-01 8
2000-01-01 9
2000-01-01 5
2000-01-01 6
2000-01-01 7
2000-01-01 8
2000-01-01 9
2000-01-01 10
2000-01-01 11
2000-01-01 12
2000-01-01 13
2000-01-01 14
2000-01-01 10
2000-01-01 11
2000-01-01 12
2000-01-01 13
2000-01-01 14
2000-01-01 15
2000-01-01 16
2000-01-01 17
2000-01-01 18
2000-01-01 19
2000-01-01 0
2000-01-01 1
2000-01-01 2
2000-01-01 3
2000-01-01 4
2000-01-01 5
2000-01-01 5
2000-01-01 6
2000-01-01 6
2000-01-01 7
2000-01-01 7
2000-01-01 8
2000-01-01 8
2000-01-01 9
2000-01-01 9
2000-01-01 10
2000-01-01 10
2000-01-01 11
2000-01-01 11
2000-01-01 12
2000-01-01 12
2000-01-01 13
2000-01-01 13
2000-01-01 14
2000-01-01 14
2000-01-01 15
2000-01-01 16
2000-01-01 17
2000-01-01 18
2000-01-01 19
2000-01-01 0
2000-01-01 1
2000-01-01 2
2000-01-01 3
2000-01-01 4
2000-01-01 5
2000-01-01 5
2000-01-01 6
2000-01-01 6
2000-01-01 7
2000-01-01 7
2000-01-01 8
2000-01-01 8
2000-01-01 9
2000-01-01 9
2000-01-01 10
2000-01-01 10
2000-01-01 11
2000-01-01 11
2000-01-01 12
2000-01-01 12
2000-01-01 13
2000-01-01 13
2000-01-01 14
2000-01-01 14
2000-01-01 15
2000-01-01 16
2000-01-01 17
2000-01-01 18
2000-01-01 19
2000-01-01 5
2000-01-01 6
2000-01-01 7
2000-01-01 8
2000-01-01 9
2000-01-01 10
2000-01-01 11
2000-01-01 12
2000-01-01 13
2000-01-01 14
2000-01-01 0
2000-01-01 1
2000-01-01 2
2000-01-01 3
2000-01-01 4
2000-01-01 5
2000-01-01 5
2000-01-01 5
2000-01-01 6
2000-01-01 6
2000-01-01 6
2000-01-01 7
2000-01-01 7
2000-01-01 7
2000-01-01 8
2000-01-01 8
2000-01-01 8
2000-01-01 9
2000-01-01 9
2000-01-01 9
2000-01-01 10
2000-01-01 10
2000-01-01 10
2000-01-01 11
2000-01-01 11
2000-01-01 11
2000-01-01 12
2000-01-01 12
2000-01-01 12
2000-01-01 13
2000-01-01 13
2000-01-01 13
2000-01-01 14
2000-01-01 14
2000-01-01 14
2000-01-01 15
2000-01-01 16
2000-01-01 17
2000-01-01 18
2000-01-01 19
2000-01-01 0
2000-01-01 1
2000-01-01 2
2000-01-01 3
2000-01-01 4
2000-01-01 5
2000-01-01 5
2000-01-01 5
2000-01-01 6
2000-01-01 6
2000-01-01 6
2000-01-01 7
2000-01-01 7
2000-01-01 7
2000-01-01 8
2000-01-01 8
2000-01-01 8
2000-01-01 9
2000-01-01 9
2000-01-01 9
2000-01-01 10
2000-01-01 10
2000-01-01 10
2000-01-01 11
2000-01-01 11
2000-01-01 11
2000-01-01 12
2000-01-01 12
2000-01-01 12
2000-01-01 13
2000-01-01 13
2000-01-01 13
2000-01-01 14
2000-01-01 14
2000-01-01 14
2000-01-01 15
2000-01-01 16
2000-01-01 17
2000-01-01 18
2000-01-01 19
2000-01-01 100
2000-01-01 101
2000-01-01 102
2000-01-01 103
2000-01-01 104
2000-01-01 105
2000-01-01 106
2000-01-01 107
2000-01-01 108
2000-01-01 109
2000-01-01 0
2000-01-01 1
2000-01-01 2
2000-01-01 3
2000-01-01 4
2000-01-01 5
2000-01-01 5
2000-01-01 5
2000-01-01 6
2000-01-01 6
2000-01-01 6
2000-01-01 7
2000-01-01 7
2000-01-01 7
2000-01-01 8
2000-01-01 8
2000-01-01 8
2000-01-01 9
2000-01-01 9
2000-01-01 9
2000-01-01 10
2000-01-01 10
2000-01-01 10
2000-01-01 11
2000-01-01 11
2000-01-01 11
2000-01-01 12
2000-01-01 12
2000-01-01 12
2000-01-01 13
2000-01-01 13
2000-01-01 13
2000-01-01 14
2000-01-01 14
2000-01-01 14
2000-01-01 15
2000-01-01 16
2000-01-01 17
2000-01-01 18
2000-01-01 19
2000-01-01 100
2000-01-01 101
2000-01-01 102
2000-01-01 103
2000-01-01 104
2000-01-01 105
2000-01-01 106
2000-01-01 107
2000-01-01 108
2000-01-01 109
16400 8200 0 8199 67231800 4574870733220668470
20000 15000 0 14999 149990000 5561824225056413482

View File

@ -0,0 +1,99 @@
DROP TABLE IF EXISTS test.merge;
CREATE TABLE IF NOT EXISTS test.merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 5);
INSERT INTO test.merge (x) VALUES (1), (2), (3);
INSERT INTO test.merge (x) VALUES (4), (5), (6);
SELECT * FROM test.merge ORDER BY _part_index, x;
OPTIMIZE TABLE test.merge;
SELECT * FROM test.merge ORDER BY _part_index, x;
DROP TABLE test.merge;
CREATE TABLE IF NOT EXISTS test.merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 5);
INSERT INTO test.merge (x) SELECT number AS x FROM system.numbers LIMIT 10;
INSERT INTO test.merge (x) SELECT number + 10 AS x FROM system.numbers LIMIT 10;
SELECT * FROM test.merge ORDER BY _part_index, x;
OPTIMIZE TABLE test.merge;
SELECT * FROM test.merge ORDER BY _part_index, x;
DROP TABLE test.merge;
CREATE TABLE IF NOT EXISTS test.merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 5);
INSERT INTO test.merge (x) SELECT number + 5 AS x FROM system.numbers LIMIT 10;
INSERT INTO test.merge (x) SELECT number AS x FROM system.numbers LIMIT 10;
SELECT * FROM test.merge ORDER BY _part_index, x;
OPTIMIZE TABLE test.merge;
SELECT * FROM test.merge ORDER BY _part_index, x;
DROP TABLE test.merge;
CREATE TABLE IF NOT EXISTS test.merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 5);
INSERT INTO test.merge (x) SELECT number + 5 AS x FROM system.numbers LIMIT 10;
INSERT INTO test.merge (x) SELECT number AS x FROM system.numbers LIMIT 10;
INSERT INTO test.merge (x) SELECT number + 9 AS x FROM system.numbers LIMIT 10;
SELECT * FROM test.merge ORDER BY _part_index, x;
OPTIMIZE TABLE test.merge;
SELECT * FROM test.merge ORDER BY _part_index, x;
DROP TABLE test.merge;
CREATE TABLE IF NOT EXISTS test.merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 5);
INSERT INTO test.merge (x) SELECT number AS x FROM system.numbers LIMIT 10;
INSERT INTO test.merge (x) SELECT number + 5 AS x FROM system.numbers LIMIT 10;
INSERT INTO test.merge (x) SELECT number + 10 AS x FROM system.numbers LIMIT 10;
SELECT * FROM test.merge ORDER BY _part_index, x;
OPTIMIZE TABLE test.merge;
SELECT * FROM test.merge ORDER BY _part_index, x;
INSERT INTO test.merge (x) SELECT number + 5 AS x FROM system.numbers LIMIT 10;
SELECT * FROM test.merge ORDER BY _part_index, x;
OPTIMIZE TABLE test.merge;
SELECT * FROM test.merge ORDER BY _part_index, x;
INSERT INTO test.merge (x) SELECT number + 100 AS x FROM system.numbers LIMIT 10;
SELECT * FROM test.merge ORDER BY _part_index, x;
OPTIMIZE TABLE test.merge;
SELECT * FROM test.merge ORDER BY _part_index, x;
DROP TABLE test.merge;
CREATE TABLE IF NOT EXISTS test.merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 8192);
SET max_block_size = 8200;
INSERT INTO test.merge (x) SELECT number AS x FROM (SELECT * FROM system.numbers LIMIT 8200) ORDER BY rand();
INSERT INTO test.merge (x) SELECT number AS x FROM (SELECT * FROM system.numbers LIMIT 8200) ORDER BY rand();
OPTIMIZE TABLE test.merge;
SELECT count(), uniqExact(x), min(x), max(x), sum(x), sum(cityHash64(x)) FROM test.merge;
DROP TABLE test.merge;
CREATE TABLE IF NOT EXISTS test.merge (d Date DEFAULT '2000-01-01', x UInt64) ENGINE = MergeTree(d, x, 8192);
SET max_block_size = 10000;
INSERT INTO test.merge (x) SELECT number AS x FROM (SELECT number FROM system.numbers LIMIT 10000);
INSERT INTO test.merge (x) SELECT number AS x FROM (SELECT number + 5000 AS number FROM system.numbers LIMIT 10000);
OPTIMIZE TABLE test.merge;
SELECT count(), uniqExact(x), min(x), max(x), sum(x), sum(cityHash64(x)) FROM test.merge;
DROP TABLE test.merge;

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