mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 00:22:29 +00:00
Merge
This commit is contained in:
commit
d8c3cdda56
@ -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; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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();
|
||||
}
|
||||
}
|
||||
};
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
@ -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]);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -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]);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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; }
|
||||
};
|
||||
|
||||
|
||||
|
@ -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; }
|
||||
};
|
||||
|
||||
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
|
||||
|
@ -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; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
*/
|
||||
|
@ -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);
|
||||
|
52
dbms/include/DB/AggregateFunctions/QuantilesCommon.h
Normal file
52
dbms/include/DB/AggregateFunctions/QuantilesCommon.h
Normal 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]; });
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
}
|
@ -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
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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_;
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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
|
||||
|
22
dbms/include/DB/Columns/ColumnsCommon.h
Normal file
22
dbms/include/DB/Columns/ColumnsCommon.h
Normal 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);
|
||||
|
||||
}
|
@ -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);
|
||||
|
||||
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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();
|
||||
}
|
||||
|
256
dbms/include/DB/Common/RadixSort.h
Normal file
256
dbms/include/DB/Common/RadixSort.h
Normal 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);
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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"; } };
|
||||
|
||||
}
|
||||
|
@ -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");
|
||||
|
||||
|
@ -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()};
|
||||
|
||||
|
@ -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()};
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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)
|
||||
|
@ -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);
|
||||
|
@ -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();
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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()
|
||||
{
|
||||
|
@ -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 {};
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/// Оптимизации для целочисленного деления на константу.
|
||||
|
||||
|
@ -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;
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
|
@ -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{
|
||||
|
@ -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 {};
|
||||
}
|
||||
|
@ -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() {}
|
||||
};
|
||||
|
||||
|
@ -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()
|
||||
|
@ -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
|
||||
|
@ -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) \
|
||||
|
@ -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);
|
||||
|
||||
}
|
||||
|
43
dbms/include/DB/Parsers/ASTSampleRatio.h
Normal file
43
dbms/include/DB/Parsers/ASTSampleRatio.h
Normal 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);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
19
dbms/include/DB/Parsers/ParserSampleRatio.h
Normal file
19
dbms/include/DB/Parsers/ParserSampleRatio.h
Normal 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);
|
||||
};
|
||||
|
||||
}
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
};
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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>);
|
||||
}
|
||||
|
||||
}
|
@ -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);
|
||||
|
173
dbms/src/Columns/ColumnsCommon.cpp
Normal file
173
dbms/src/Columns/ColumnsCommon.cpp
Normal 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 &);
|
||||
|
||||
}
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
116
dbms/src/Common/tests/radix_sort.cpp
Normal file
116
dbms/src/Common/tests/radix_sort.cpp
Normal 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;
|
||||
}
|
@ -1,4 +1,5 @@
|
||||
#include <DB/Columns/ColumnsNumber.h>
|
||||
#include <DB/Columns/ColumnsCommon.h>
|
||||
|
||||
#include <DB/DataStreams/FilterBlockInputStream.h>
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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 {};
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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",
|
||||
|
@ -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) {}
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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)
|
||||
|
40
dbms/src/Parsers/ASTSampleRatio.cpp
Normal file
40
dbms/src/Parsers/ASTSampleRatio.cpp
Normal 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);
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
}
|
@ -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",
|
||||
|
128
dbms/src/Parsers/ParserSampleRatio.cpp
Normal file
128
dbms/src/Parsers/ParserSampleRatio.cpp
Normal 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;
|
||||
}
|
||||
|
||||
}
|
@ -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);
|
||||
|
@ -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)
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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());
|
||||
|
@ -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)
|
||||
|
@ -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:
|
||||
|
@ -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.");
|
||||
}
|
||||
|
@ -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,
|
||||
|
@ -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 (...)
|
||||
{
|
||||
|
@ -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);
|
||||
|
28
dbms/tests/queries/0_stateless/00276_sample.reference
Normal file
28
dbms/tests/queries/0_stateless/00276_sample.reference
Normal 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
|
@ -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
|
||||
|
@ -0,0 +1,2 @@
|
||||
2000
|
||||
2000
|
2
dbms/tests/queries/0_stateless/00277_array_filter.sql
Normal file
2
dbms/tests/queries/0_stateless/00277_array_filter.sql
Normal 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);
|
@ -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
|
@ -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;
|
@ -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]
|
@ -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);
|
@ -0,0 +1 @@
|
||||
0 Р
|
@ -0,0 +1 @@
|
||||
SELECT '\x30 \xD0\xA0';
|
@ -0,0 +1,2 @@
|
||||
1 Hello
|
||||
2 Hello
|
@ -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;
|
394
dbms/tests/queries/0_stateless/00282_merging.reference
Normal file
394
dbms/tests/queries/0_stateless/00282_merging.reference
Normal 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
|
99
dbms/tests/queries/0_stateless/00282_merging.sql
Normal file
99
dbms/tests/queries/0_stateless/00282_merging.sql
Normal 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
Loading…
Reference in New Issue
Block a user