mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Merge
This commit is contained in:
commit
f1b94e8235
@ -1,97 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/IO/WriteHelpers.h>
|
||||
#include <DB/IO/ReadHelpers.h>
|
||||
|
||||
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
struct AggregateFunctionAnyData
|
||||
{
|
||||
Field value;
|
||||
};
|
||||
|
||||
|
||||
/// Берёт первое попавшееся значение
|
||||
class AggregateFunctionAny final : public IUnaryAggregateFunction<AggregateFunctionAnyData, AggregateFunctionAny>
|
||||
{
|
||||
private:
|
||||
DataTypePtr type;
|
||||
|
||||
public:
|
||||
String getName() const { return "any"; }
|
||||
|
||||
DataTypePtr getReturnType() const
|
||||
{
|
||||
return type;
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
type = argument;
|
||||
}
|
||||
|
||||
|
||||
void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const
|
||||
{
|
||||
Data & d = data(place);
|
||||
|
||||
if (!d.value.isNull())
|
||||
return;
|
||||
column.get(row_num, d.value);
|
||||
}
|
||||
|
||||
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
|
||||
{
|
||||
Data & d = data(place);
|
||||
|
||||
if (d.value.isNull())
|
||||
d.value = data(rhs).value;
|
||||
}
|
||||
|
||||
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
|
||||
{
|
||||
const Data & d = data(place);
|
||||
|
||||
if (unlikely(d.value.isNull()))
|
||||
{
|
||||
writeBinary(false, buf);
|
||||
}
|
||||
else
|
||||
{
|
||||
writeBinary(true, buf);
|
||||
type->serializeBinary(data(place).value, buf);
|
||||
}
|
||||
}
|
||||
|
||||
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
|
||||
{
|
||||
Data & d = data(place);
|
||||
|
||||
bool is_not_null = false;
|
||||
readBinary(is_not_null, buf);
|
||||
|
||||
if (is_not_null)
|
||||
{
|
||||
Field tmp;
|
||||
type->deserializeBinary(tmp, buf);
|
||||
|
||||
if (d.value.isNull())
|
||||
d.value = tmp;
|
||||
}
|
||||
}
|
||||
|
||||
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
|
||||
{
|
||||
if (unlikely(data(place).value.isNull()))
|
||||
to.insertDefault();
|
||||
else
|
||||
to.insert(data(place).value);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -1,83 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/IO/WriteHelpers.h>
|
||||
#include <DB/IO/ReadHelpers.h>
|
||||
|
||||
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
struct AggregateFunctionAnyLastData
|
||||
{
|
||||
Field value;
|
||||
};
|
||||
|
||||
|
||||
/// Берёт последнее попавшееся значение
|
||||
class AggregateFunctionAnyLast final : public IUnaryAggregateFunction<AggregateFunctionAnyLastData, AggregateFunctionAnyLast>
|
||||
{
|
||||
private:
|
||||
DataTypePtr type;
|
||||
|
||||
public:
|
||||
String getName() const { return "anyLast"; }
|
||||
|
||||
DataTypePtr getReturnType() const
|
||||
{
|
||||
return type;
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
type = argument;
|
||||
}
|
||||
|
||||
|
||||
void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const
|
||||
{
|
||||
column.get(row_num, data(place).value);
|
||||
}
|
||||
|
||||
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
|
||||
{
|
||||
if (!data(rhs).value.isNull())
|
||||
data(place).value = data(rhs).value;
|
||||
}
|
||||
|
||||
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
|
||||
{
|
||||
const Data & d = data(place);
|
||||
|
||||
if (unlikely(d.value.isNull()))
|
||||
{
|
||||
writeBinary(false, buf);
|
||||
}
|
||||
else
|
||||
{
|
||||
writeBinary(true, buf);
|
||||
type->serializeBinary(data(place).value, buf);
|
||||
}
|
||||
}
|
||||
|
||||
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
|
||||
{
|
||||
bool is_not_null = false;
|
||||
readBinary(is_not_null, buf);
|
||||
|
||||
if (is_not_null)
|
||||
type->deserializeBinary(data(place).value, buf);
|
||||
}
|
||||
|
||||
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
|
||||
{
|
||||
if (unlikely(data(place).value.isNull()))
|
||||
to.insertDefault();
|
||||
else
|
||||
to.insert(data(place).value);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -19,8 +19,15 @@ namespace DB
|
||||
template <typename T>
|
||||
struct __attribute__((__packed__)) AggregateFunctionUniqUpToData
|
||||
{
|
||||
/** Если count == threshold + 1 - это значит, что "переполнилось" (значений больше threshold).
|
||||
* В этом случае (например, после вызова функции merge), массив data не обязательно содержит инициализированные значения
|
||||
* - пример: объединяем состояние, в котором мало значений, с другим состоянием, которое переполнилось;
|
||||
* тогда выставляем count в threshold + 1, а значения из другого состояния не копируем.
|
||||
*/
|
||||
UInt8 count = 0;
|
||||
T data[0]; /// Данные идут после конца структуры. При вставке, делается линейный поиск.
|
||||
|
||||
/// Данные идут после конца структуры. При вставке, делается линейный поиск.
|
||||
T data[0];
|
||||
|
||||
|
||||
size_t size() const
|
||||
@ -31,17 +38,20 @@ struct __attribute__((__packed__)) AggregateFunctionUniqUpToData
|
||||
/// threshold - для скольки элементов есть место в data.
|
||||
void insert(T x, UInt8 threshold)
|
||||
{
|
||||
/// Состояние уже переполнено - ничего делать не нужно.
|
||||
if (count > threshold)
|
||||
return;
|
||||
|
||||
size_t limit = std::min(count, threshold);
|
||||
for (size_t i = 0; i < limit; ++i)
|
||||
/// Линейный поиск совпадающего элемента.
|
||||
for (size_t i = 0; i < count; ++i)
|
||||
if (data[i] == x)
|
||||
return;
|
||||
|
||||
/// Не нашли совпадающий элемент. Если есть место ещё для одного элемента - вставляем его.
|
||||
if (count < threshold)
|
||||
data[count] = x;
|
||||
|
||||
/// После увеличения count, состояние может оказаться переполненным.
|
||||
++count;
|
||||
}
|
||||
|
||||
@ -52,19 +62,22 @@ struct __attribute__((__packed__)) AggregateFunctionUniqUpToData
|
||||
|
||||
if (rhs.count > threshold)
|
||||
{
|
||||
/// Если rhs переполнено, то выставляем у текущего состояния count тоже переполненным.
|
||||
count = rhs.count;
|
||||
return;
|
||||
}
|
||||
|
||||
size_t limit = std::min(rhs.count, threshold);
|
||||
for (size_t i = 0; i < limit; ++i)
|
||||
for (size_t i = 0; i < rhs.count; ++i)
|
||||
insert(rhs.data[i], threshold);
|
||||
}
|
||||
|
||||
void write(WriteBuffer & wb, UInt8 threshold) const
|
||||
{
|
||||
size_t limit = std::min(count, threshold);
|
||||
wb.write(reinterpret_cast<const char *>(this), sizeof(*this) + limit * sizeof(data[0]));
|
||||
writeBinary(count, wb);
|
||||
|
||||
/// Пишем значения, только если состояние не переполнено. Иначе они не нужны, а важен только факт того, что состояние переполнено.
|
||||
if (count <= threshold)
|
||||
wb.write(reinterpret_cast<const char *>(this), count * sizeof(data[0]));
|
||||
}
|
||||
|
||||
void readAndMerge(ReadBuffer & rb, UInt8 threshold)
|
||||
@ -72,11 +85,14 @@ struct __attribute__((__packed__)) AggregateFunctionUniqUpToData
|
||||
UInt8 rhs_count;
|
||||
readBinary(rhs_count, rb);
|
||||
|
||||
if (rhs_count > threshold + 1)
|
||||
throw Poco::Exception("Cannot read AggregateFunctionUniqUpToData: too large count.");
|
||||
if (rhs_count > threshold)
|
||||
{
|
||||
/// Если rhs переполнено, то выставляем у текущего состояния count тоже переполненным.
|
||||
count = rhs_count;
|
||||
return;
|
||||
}
|
||||
|
||||
size_t limit = std::min(rhs_count, threshold);
|
||||
for (size_t i = 0; i < limit; ++i)
|
||||
for (size_t i = 0; i < rhs_count; ++i)
|
||||
{
|
||||
T x;
|
||||
readBinary(x, rb);
|
||||
@ -135,11 +151,13 @@ public:
|
||||
if (params.size() != 1)
|
||||
throw Exception("Aggregate function " + getName() + " requires exactly one parameter.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
threshold = apply_visitor(FieldVisitorConvertToNumber<UInt64>(), params[0]);
|
||||
UInt64 threshold_param = apply_visitor(FieldVisitorConvertToNumber<UInt64>(), params[0]);
|
||||
|
||||
if (threshold > uniq_upto_max_threshold)
|
||||
if (threshold_param > uniq_upto_max_threshold)
|
||||
throw Exception("Too large parameter for aggregate function " + getName() + ". Maximum: " + toString(uniq_upto_max_threshold),
|
||||
ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
threshold = threshold_param;
|
||||
}
|
||||
|
||||
void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const
|
||||
|
@ -1,132 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/IO/WriteHelpers.h>
|
||||
#include <DB/IO/ReadHelpers.h>
|
||||
|
||||
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
struct AggregateFunctionMinTraits
|
||||
{
|
||||
static bool better(const Field & lhs, const Field & rhs) { return lhs < rhs; }
|
||||
static String name() { return "min"; }
|
||||
};
|
||||
|
||||
struct AggregateFunctionMaxTraits
|
||||
{
|
||||
static bool better(const Field & lhs, const Field & rhs) { return lhs > rhs; }
|
||||
static String name() { return "max"; }
|
||||
};
|
||||
|
||||
|
||||
struct AggregateFunctionsMinMaxData
|
||||
{
|
||||
Field value;
|
||||
};
|
||||
|
||||
|
||||
/// Берёт минимальное (или максимальное) значение. Если таких много - то первое попавшееся из них.
|
||||
template <typename Traits>
|
||||
class AggregateFunctionsMinMax final : public IUnaryAggregateFunction<AggregateFunctionsMinMaxData, AggregateFunctionsMinMax<Traits> >
|
||||
{
|
||||
private:
|
||||
typedef typename IAggregateFunctionHelper<AggregateFunctionsMinMaxData>::Data Data;
|
||||
DataTypePtr type;
|
||||
|
||||
public:
|
||||
String getName() const { return Traits::name(); }
|
||||
|
||||
DataTypePtr getReturnType() const
|
||||
{
|
||||
return type;
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
type = argument;
|
||||
}
|
||||
|
||||
|
||||
void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const
|
||||
{
|
||||
Field value;
|
||||
column.get(row_num, value);
|
||||
Data & d = this->data(place);
|
||||
|
||||
if (!d.value.isNull())
|
||||
{
|
||||
if (Traits::better(value, d.value))
|
||||
d.value = value;
|
||||
}
|
||||
else
|
||||
d.value = value;
|
||||
}
|
||||
|
||||
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
|
||||
{
|
||||
Data & d = this->data(place);
|
||||
const Data & d_rhs = this->data(rhs);
|
||||
|
||||
if (!d.value.isNull())
|
||||
{
|
||||
if (Traits::better(d_rhs.value, d.value))
|
||||
d.value = d_rhs.value;
|
||||
}
|
||||
else
|
||||
d.value = d_rhs.value;
|
||||
}
|
||||
|
||||
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
|
||||
{
|
||||
const Data & d = this->data(place);
|
||||
|
||||
if (unlikely(d.value.isNull()))
|
||||
{
|
||||
writeBinary(false, buf);
|
||||
}
|
||||
else
|
||||
{
|
||||
writeBinary(true, buf);
|
||||
type->serializeBinary(this->data(place).value, buf);
|
||||
}
|
||||
}
|
||||
|
||||
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
|
||||
{
|
||||
Data & d = this->data(place);
|
||||
|
||||
bool is_not_null = false;
|
||||
readBinary(is_not_null, buf);
|
||||
|
||||
if (is_not_null)
|
||||
{
|
||||
if (!d.value.isNull())
|
||||
{
|
||||
Field value_;
|
||||
type->deserializeBinary(value_, buf);
|
||||
if (Traits::better(value_, d.value))
|
||||
d.value = value_;
|
||||
}
|
||||
else
|
||||
type->deserializeBinary(d.value, buf);
|
||||
}
|
||||
}
|
||||
|
||||
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
|
||||
{
|
||||
if (unlikely(this->data(place).value.isNull()))
|
||||
to.insertDefault();
|
||||
else
|
||||
to.insert(this->data(place).value);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
typedef AggregateFunctionsMinMax<AggregateFunctionMinTraits> AggregateFunctionMin;
|
||||
typedef AggregateFunctionsMinMax<AggregateFunctionMaxTraits> AggregateFunctionMax;
|
||||
|
||||
}
|
480
dbms/include/DB/AggregateFunctions/AggregateFunctionsMinMaxAny.h
Normal file
480
dbms/include/DB/AggregateFunctions/AggregateFunctionsMinMaxAny.h
Normal file
@ -0,0 +1,480 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/IO/WriteHelpers.h>
|
||||
#include <DB/IO/ReadHelpers.h>
|
||||
|
||||
#include <DB/Columns/ColumnVector.h>
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
|
||||
#include <DB/AggregateFunctions/IUnaryAggregateFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Агрегатные функции, запоминающие одно какое-либо переданное значение.
|
||||
* Например, min, max, any, anyLast.
|
||||
*/
|
||||
|
||||
|
||||
/// Для числовых значений.
|
||||
template <typename T>
|
||||
struct SingleValueDataFixed
|
||||
{
|
||||
typedef SingleValueDataFixed<T> Self;
|
||||
|
||||
bool has_value = false; /// Надо запомнить, было ли передано хотя бы одно значение. Это нужно для AggregateFunctionIf.
|
||||
T value;
|
||||
|
||||
|
||||
bool has() const
|
||||
{
|
||||
return has_value;
|
||||
}
|
||||
|
||||
void insertResultInto(IColumn & to) const
|
||||
{
|
||||
if (has())
|
||||
static_cast<ColumnVector<T> &>(to).getData().push_back(value);
|
||||
else
|
||||
static_cast<ColumnVector<T> &>(to).insertDefault();
|
||||
}
|
||||
|
||||
void write(WriteBuffer & buf, const IDataType & data_type) const
|
||||
{
|
||||
writeBinary(has(), buf);
|
||||
if (has())
|
||||
writeBinary(value, buf);
|
||||
}
|
||||
|
||||
void read(ReadBuffer & buf, const IDataType & data_type)
|
||||
{
|
||||
readBinary(has_value, buf);
|
||||
if (has())
|
||||
readBinary(value, buf);
|
||||
}
|
||||
|
||||
|
||||
void change(const IColumn & column, size_t row_num)
|
||||
{
|
||||
has_value = true;
|
||||
value = static_cast<const ColumnVector<T> &>(column).getData()[row_num];
|
||||
}
|
||||
|
||||
void change(const Self & to)
|
||||
{
|
||||
has_value = true;
|
||||
value = to.value;
|
||||
}
|
||||
|
||||
void changeFirstTime(const IColumn & column, size_t row_num)
|
||||
{
|
||||
if (!has())
|
||||
change(column, row_num);
|
||||
}
|
||||
|
||||
void changeFirstTime(const Self & to)
|
||||
{
|
||||
if (!has())
|
||||
change(to);
|
||||
}
|
||||
|
||||
void changeIfLess(const IColumn & column, size_t row_num)
|
||||
{
|
||||
if (!has() || static_cast<const ColumnVector<T> &>(column).getData()[row_num] < value)
|
||||
change(column, row_num);
|
||||
}
|
||||
|
||||
void changeIfLess(const Self & to)
|
||||
{
|
||||
if (!has() || to.value < value)
|
||||
change(to);
|
||||
}
|
||||
|
||||
void changeIfGreater(const IColumn & column, size_t row_num)
|
||||
{
|
||||
if (!has() || static_cast<const ColumnVector<T> &>(column).getData()[row_num] > value)
|
||||
change(column, row_num);
|
||||
}
|
||||
|
||||
void changeIfGreater(const Self & to)
|
||||
{
|
||||
if (!has() || to.value > value)
|
||||
change(to);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/** Для строк. Короткие строки хранятся в самой структуре, а длинные выделяются отдельно.
|
||||
* NOTE Могло бы подойти также для массивов чисел.
|
||||
*/
|
||||
struct __attribute__((__packed__)) SingleValueDataString
|
||||
{
|
||||
typedef SingleValueDataString Self;
|
||||
|
||||
Int32 size = -1; /// -1 обозначает, что значения нет.
|
||||
|
||||
static constexpr Int32 AUTOMATIC_STORAGE_SIZE = 64;
|
||||
static constexpr Int32 MAX_SMALL_STRING_SIZE = AUTOMATIC_STORAGE_SIZE - sizeof(size);
|
||||
|
||||
union
|
||||
{
|
||||
char small_data[MAX_SMALL_STRING_SIZE]; /// Включая завершающий ноль.
|
||||
char * large_data;
|
||||
};
|
||||
|
||||
~SingleValueDataString()
|
||||
{
|
||||
if (size > MAX_SMALL_STRING_SIZE)
|
||||
free(large_data);
|
||||
}
|
||||
|
||||
bool has() const
|
||||
{
|
||||
return size >= 0;
|
||||
}
|
||||
|
||||
const char * getData() const
|
||||
{
|
||||
return size <= MAX_SMALL_STRING_SIZE ? small_data : large_data;
|
||||
}
|
||||
|
||||
StringRef getStringRef() const
|
||||
{
|
||||
return StringRef(getData(), size);
|
||||
}
|
||||
|
||||
void insertResultInto(IColumn & to) const
|
||||
{
|
||||
if (has())
|
||||
static_cast<ColumnString &>(to).insertDataWithTerminatingZero(getData(), size);
|
||||
else
|
||||
static_cast<ColumnString &>(to).insertDefault();
|
||||
}
|
||||
|
||||
void write(WriteBuffer & buf, const IDataType & data_type) const
|
||||
{
|
||||
writeBinary(size, buf);
|
||||
if (has())
|
||||
buf.write(getData(), size);
|
||||
}
|
||||
|
||||
void read(ReadBuffer & buf, const IDataType & data_type)
|
||||
{
|
||||
Int32 rhs_size;
|
||||
readBinary(rhs_size, buf);
|
||||
|
||||
if (rhs_size >= 0)
|
||||
{
|
||||
if (rhs_size <= MAX_SMALL_STRING_SIZE)
|
||||
{
|
||||
if (size > MAX_SMALL_STRING_SIZE)
|
||||
free(large_data);
|
||||
|
||||
size = rhs_size;
|
||||
|
||||
if (size > 0)
|
||||
buf.read(small_data, size);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (size < rhs_size)
|
||||
{
|
||||
if (size > MAX_SMALL_STRING_SIZE)
|
||||
free(large_data);
|
||||
|
||||
large_data = reinterpret_cast<char *>(malloc(rhs_size));
|
||||
}
|
||||
|
||||
size = rhs_size;
|
||||
buf.read(large_data, size);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
if (size > MAX_SMALL_STRING_SIZE)
|
||||
free(large_data);
|
||||
size = rhs_size;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void changeImpl(StringRef value)
|
||||
{
|
||||
Int32 value_size = value.size;
|
||||
|
||||
if (value_size <= MAX_SMALL_STRING_SIZE)
|
||||
{
|
||||
if (size > MAX_SMALL_STRING_SIZE)
|
||||
free(large_data);
|
||||
|
||||
size = value_size;
|
||||
|
||||
if (size > 0)
|
||||
memcpy(small_data, value.data, size);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (size < value_size)
|
||||
{
|
||||
if (size > MAX_SMALL_STRING_SIZE)
|
||||
free(large_data);
|
||||
|
||||
large_data = reinterpret_cast<char *>(malloc(value.size));
|
||||
}
|
||||
|
||||
size = value_size;
|
||||
memcpy(large_data, value.data, size);
|
||||
}
|
||||
}
|
||||
|
||||
void change(const IColumn & column, size_t row_num)
|
||||
{
|
||||
changeImpl(static_cast<const ColumnString &>(column).getDataAtWithTerminatingZero(row_num));
|
||||
}
|
||||
|
||||
void change(const Self & to)
|
||||
{
|
||||
changeImpl(to.getStringRef());
|
||||
}
|
||||
|
||||
void changeFirstTime(const IColumn & column, size_t row_num)
|
||||
{
|
||||
if (!has())
|
||||
change(column, row_num);
|
||||
}
|
||||
|
||||
void changeFirstTime(const Self & to)
|
||||
{
|
||||
if (!has())
|
||||
change(to);
|
||||
}
|
||||
|
||||
void changeIfLess(const IColumn & column, size_t row_num)
|
||||
{
|
||||
if (!has() || static_cast<const ColumnString &>(column).getDataAtWithTerminatingZero(row_num) < getStringRef())
|
||||
change(column, row_num);
|
||||
}
|
||||
|
||||
void changeIfLess(const Self & to)
|
||||
{
|
||||
if (!has() || to.getStringRef() < getStringRef())
|
||||
change(to);
|
||||
}
|
||||
|
||||
void changeIfGreater(const IColumn & column, size_t row_num)
|
||||
{
|
||||
if (!has() || static_cast<const ColumnString &>(column).getDataAtWithTerminatingZero(row_num) > getStringRef())
|
||||
change(column, row_num);
|
||||
}
|
||||
|
||||
void changeIfGreater(const Self & to)
|
||||
{
|
||||
if (!has() || to.getStringRef() > getStringRef())
|
||||
change(to);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/// Для любых других типов значений.
|
||||
struct SingleValueDataGeneric
|
||||
{
|
||||
typedef SingleValueDataGeneric Self;
|
||||
|
||||
Field value;
|
||||
|
||||
bool has() const
|
||||
{
|
||||
return !value.isNull();
|
||||
}
|
||||
|
||||
void insertResultInto(IColumn & to) const
|
||||
{
|
||||
if (has())
|
||||
to.insert(value);
|
||||
else
|
||||
to.insertDefault();
|
||||
}
|
||||
|
||||
void write(WriteBuffer & buf, const IDataType & data_type) const
|
||||
{
|
||||
if (!value.isNull())
|
||||
{
|
||||
writeBinary(true, buf);
|
||||
data_type.serializeBinary(value, buf);
|
||||
}
|
||||
else
|
||||
writeBinary(false, buf);
|
||||
}
|
||||
|
||||
void read(ReadBuffer & buf, const IDataType & data_type)
|
||||
{
|
||||
bool is_not_null;
|
||||
readBinary(is_not_null, buf);
|
||||
|
||||
if (is_not_null)
|
||||
data_type.deserializeBinary(value, buf);
|
||||
}
|
||||
|
||||
void change(const IColumn & column, size_t row_num)
|
||||
{
|
||||
column.get(row_num, value);
|
||||
}
|
||||
|
||||
void change(const Self & to)
|
||||
{
|
||||
value = to.value;
|
||||
}
|
||||
|
||||
void changeFirstTime(const IColumn & column, size_t row_num)
|
||||
{
|
||||
if (!has())
|
||||
change(column, row_num);
|
||||
}
|
||||
|
||||
void changeFirstTime(const Self & to)
|
||||
{
|
||||
if (!has())
|
||||
change(to);
|
||||
}
|
||||
|
||||
void changeIfLess(const IColumn & column, size_t row_num)
|
||||
{
|
||||
if (!has())
|
||||
change(column, row_num);
|
||||
else
|
||||
{
|
||||
Field new_value;
|
||||
column.get(row_num, new_value);
|
||||
if (new_value < value)
|
||||
value = new_value;
|
||||
}
|
||||
}
|
||||
|
||||
void changeIfLess(const Self & to)
|
||||
{
|
||||
if (!has() || to.value < value)
|
||||
change(to);
|
||||
}
|
||||
|
||||
void changeIfGreater(const IColumn & column, size_t row_num)
|
||||
{
|
||||
if (!has())
|
||||
change(column, row_num);
|
||||
else
|
||||
{
|
||||
Field new_value;
|
||||
column.get(row_num, new_value);
|
||||
if (new_value > value)
|
||||
value = new_value;
|
||||
}
|
||||
}
|
||||
|
||||
void changeIfGreater(const Self & to)
|
||||
{
|
||||
if (!has() || to.value > value)
|
||||
change(to);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/** То, чем отличаются друг от другая агрегатные функции min, max, any, anyLast
|
||||
* (условием, при котором сохранённое значение заменяется на новое,
|
||||
* а также, конечно, именем).
|
||||
*/
|
||||
|
||||
template <typename Data>
|
||||
struct AggregateFunctionMinData : Data
|
||||
{
|
||||
typedef AggregateFunctionMinData<Data> Self;
|
||||
|
||||
void changeIfBetter(const IColumn & column, size_t row_num) { this->changeIfLess(column, row_num); }
|
||||
void changeIfBetter(const Self & to) { this->changeIfLess(to); }
|
||||
|
||||
static const char * name() { return "min"; }
|
||||
};
|
||||
|
||||
template <typename Data>
|
||||
struct AggregateFunctionMaxData : Data
|
||||
{
|
||||
typedef AggregateFunctionMaxData<Data> Self;
|
||||
|
||||
void changeIfBetter(const IColumn & column, size_t row_num) { this->changeIfGreater(column, row_num); }
|
||||
void changeIfBetter(const Self & to) { this->changeIfGreater(to); }
|
||||
|
||||
static const char * name() { return "max"; }
|
||||
};
|
||||
|
||||
template <typename Data>
|
||||
struct AggregateFunctionAnyData : Data
|
||||
{
|
||||
typedef AggregateFunctionAnyData<Data> Self;
|
||||
|
||||
void changeIfBetter(const IColumn & column, size_t row_num) { this->changeFirstTime(column, row_num); }
|
||||
void changeIfBetter(const Self & to) { this->changeFirstTime(to); }
|
||||
|
||||
static const char * name() { return "any"; }
|
||||
};
|
||||
|
||||
template <typename Data>
|
||||
struct AggregateFunctionAnyLastData : Data
|
||||
{
|
||||
typedef AggregateFunctionAnyLastData<Data> Self;
|
||||
|
||||
void changeIfBetter(const IColumn & column, size_t row_num) { this->change(column, row_num); }
|
||||
void changeIfBetter(const Self & to) { this->change(to); }
|
||||
|
||||
static const char * name() { return "anyLast"; }
|
||||
};
|
||||
|
||||
|
||||
template <typename Data>
|
||||
class AggregateFunctionsSingleValue final : public IUnaryAggregateFunction<Data, AggregateFunctionsSingleValue<Data> >
|
||||
{
|
||||
private:
|
||||
DataTypePtr type;
|
||||
|
||||
public:
|
||||
String getName() const { return Data::name(); }
|
||||
|
||||
DataTypePtr getReturnType() const
|
||||
{
|
||||
return type;
|
||||
}
|
||||
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
type = argument;
|
||||
}
|
||||
|
||||
|
||||
void addOne(AggregateDataPtr place, const IColumn & column, size_t row_num) const
|
||||
{
|
||||
this->data(place).changeIfBetter(column, row_num);
|
||||
}
|
||||
|
||||
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs) const
|
||||
{
|
||||
this->data(place).changeIfBetter(this->data(rhs));
|
||||
}
|
||||
|
||||
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const
|
||||
{
|
||||
this->data(place).write(buf, *type.get());
|
||||
}
|
||||
|
||||
void deserializeMerge(AggregateDataPtr place, ReadBuffer & buf) const
|
||||
{
|
||||
Data rhs; /// Для строчек не очень оптимально, так как может делаться одна лишняя аллокация.
|
||||
rhs.read(buf, *type.get());
|
||||
|
||||
this->data(place).changeIfBetter(rhs);
|
||||
}
|
||||
|
||||
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const
|
||||
{
|
||||
this->data(place).insertResultInto(to);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
@ -97,7 +97,8 @@ public:
|
||||
void sendExternalTablesData(ExternalTablesData & data);
|
||||
|
||||
/// Отправить блок данных, который уже был заранее сериализован (и, если надо, сжат), который следует прочитать из input-а.
|
||||
void sendPreparedData(ReadBuffer & input, const String & name = "");
|
||||
/// можно передать размер сериализованного/сжатого блока.
|
||||
void sendPreparedData(ReadBuffer & input, size_t size, const String & name = "");
|
||||
|
||||
/// Проверить, есть ли данные, которые можно прочитать.
|
||||
bool poll(size_t timeout_microseconds = 0);
|
||||
|
@ -161,8 +161,7 @@ public:
|
||||
|
||||
void insertDefault()
|
||||
{
|
||||
data->insertDefault();
|
||||
getOffsets().push_back(getOffsets().size() == 0 ? 1 : (getOffsets().back() + 1));
|
||||
getOffsets().push_back(getOffsets().size() == 0 ? 0 : getOffsets().back());
|
||||
}
|
||||
|
||||
ColumnPtr filter(const Filter & filt) const
|
||||
@ -387,6 +386,10 @@ private:
|
||||
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
||||
ColumnPtr res = cloneEmpty();
|
||||
|
||||
if (0 == col_size)
|
||||
return res;
|
||||
|
||||
ColumnArray & res_ = typeid_cast<ColumnArray &>(*res);
|
||||
|
||||
const typename ColumnVector<T>::Container_t & cur_data = typeid_cast<const ColumnVector<T> &>(*data).getData();
|
||||
@ -431,6 +434,10 @@ private:
|
||||
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
||||
ColumnPtr res = cloneEmpty();
|
||||
|
||||
if (0 == col_size)
|
||||
return res;
|
||||
|
||||
ColumnArray & res_ = typeid_cast<ColumnArray &>(*res);
|
||||
|
||||
const ColumnString & cur_string = typeid_cast<const ColumnString &>(*data);
|
||||
|
@ -32,11 +32,11 @@ class ColumnConst final : public IColumnConst
|
||||
public:
|
||||
typedef T Type;
|
||||
typedef typename NearestFieldType<T>::Type FieldType;
|
||||
|
||||
|
||||
/// Для ColumnConst<Array> data_type_ должен быть ненулевым.
|
||||
/// Для ColumnConst<String> data_type_ должен быть ненулевым, если тип данных FixedString.
|
||||
ColumnConst(size_t s_, const T & data_, DataTypePtr data_type_ = DataTypePtr()) : s(s_), data(data_), data_type(data_type_) {}
|
||||
|
||||
|
||||
std::string getName() const { return "ColumnConst<" + TypeName<T>::get() + ">"; }
|
||||
bool isNumeric() const { return IsNumber<T>::value; }
|
||||
bool isFixed() const { return IsNumber<T>::value; }
|
||||
@ -50,7 +50,7 @@ public:
|
||||
{
|
||||
return new ColumnConst<T>(length, data, data_type);
|
||||
}
|
||||
|
||||
|
||||
void insert(const Field & x)
|
||||
{
|
||||
if (x.get<FieldType>() != FieldType(data))
|
||||
@ -71,20 +71,15 @@ public:
|
||||
ErrorCodes::CANNOT_INSERT_ELEMENT_INTO_CONSTANT_COLUMN);
|
||||
++s;
|
||||
}
|
||||
|
||||
|
||||
void insertDefault() { ++s; }
|
||||
|
||||
ColumnPtr filter(const Filter & filt) const
|
||||
{
|
||||
if (s != filt.size())
|
||||
throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
||||
size_t new_size = 0;
|
||||
for (Filter::const_iterator it = filt.begin(); it != filt.end(); ++it)
|
||||
if (*it)
|
||||
++new_size;
|
||||
|
||||
return new ColumnConst<T>(new_size, data, data_type);
|
||||
|
||||
return new ColumnConst<T>(countBytesInFilter(filt), data, data_type);
|
||||
}
|
||||
|
||||
ColumnPtr replicate(const Offsets_t & offsets) const
|
||||
@ -92,7 +87,8 @@ public:
|
||||
if (s != offsets.size())
|
||||
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
||||
return new ColumnConst<T>(offsets.back(), data, data_type);
|
||||
size_t replicated_size = 0 == s ? 0 : offsets.back();
|
||||
return new ColumnConst<T>(replicated_size, data, data_type);
|
||||
}
|
||||
|
||||
size_t byteSize() const { return sizeof(data) + sizeof(s); }
|
||||
|
@ -48,12 +48,12 @@ public:
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
size_t byteSize() const
|
||||
{
|
||||
return chars.size() + sizeof(n);
|
||||
}
|
||||
|
||||
|
||||
Field operator[](size_t index) const
|
||||
{
|
||||
return String(reinterpret_cast<const char *>(&chars[n * index]), n);
|
||||
@ -75,7 +75,7 @@ public:
|
||||
|
||||
if (s.size() > n)
|
||||
throw Exception("Too large string '" + s + "' for FixedString column", ErrorCodes::TOO_LARGE_STRING_SIZE);
|
||||
|
||||
|
||||
size_t old_size = chars.size();
|
||||
chars.resize_fill(old_size + n);
|
||||
memcpy(&chars[old_size], s.data(), s.size());
|
||||
@ -222,7 +222,10 @@ public:
|
||||
|
||||
ColumnFixedString * res_ = new ColumnFixedString(n);
|
||||
ColumnPtr res = res_;
|
||||
|
||||
|
||||
if (0 == col_size)
|
||||
return res;
|
||||
|
||||
Chars_t & res_chars = res_->chars;
|
||||
res_chars.reserve(n * offsets.back());
|
||||
|
||||
|
@ -31,8 +31,8 @@ private:
|
||||
|
||||
/// Размер, включая завершающий нулевой байт.
|
||||
size_t __attribute__((__always_inline__)) sizeAt(size_t i) const { return i == 0 ? offsets[0] : (offsets[i] - offsets[i - 1]); }
|
||||
|
||||
public:
|
||||
|
||||
public:
|
||||
/** Создать пустой столбец строк */
|
||||
ColumnString() {}
|
||||
|
||||
@ -78,7 +78,7 @@ public:
|
||||
const String & s = DB::get<const String &>(x);
|
||||
size_t old_size = chars.size();
|
||||
size_t size_to_append = s.size() + 1;
|
||||
|
||||
|
||||
chars.resize(old_size + size_to_append);
|
||||
memcpy(&chars[old_size], s.c_str(), size_to_append);
|
||||
offsets.push_back((offsets.size() == 0 ? 0 : offsets.back()) + size_to_append);
|
||||
@ -90,7 +90,7 @@ public:
|
||||
size_t old_size = chars.size();
|
||||
size_t size_to_append = src.sizeAt(n);
|
||||
size_t offset = src.offsetAt(n);
|
||||
|
||||
|
||||
chars.resize(old_size + size_to_append);
|
||||
memcpy(&chars[old_size], &src.chars[offset], size_to_append);
|
||||
offsets.push_back((offsets.size() == 0 ? 0 : offsets.back()) + size_to_append);
|
||||
@ -132,7 +132,7 @@ public:
|
||||
|
||||
res_->chars.resize(nested_length);
|
||||
memcpy(&res_->chars[0], &chars[nested_offset], nested_length);
|
||||
|
||||
|
||||
Offsets_t & res_offsets = res_->offsets;
|
||||
|
||||
if (start == 0)
|
||||
@ -152,36 +152,105 @@ public:
|
||||
|
||||
ColumnPtr filter(const Filter & filt) const
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
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)
|
||||
return new ColumnString;
|
||||
|
||||
ColumnString * res_ = new ColumnString;
|
||||
ColumnPtr res = res_;
|
||||
auto res = new ColumnString;
|
||||
|
||||
Chars_t & res_chars = res_->chars;
|
||||
Offsets_t & res_offsets = res_->offsets;
|
||||
Chars_t & res_chars = res->chars;
|
||||
Offsets_t & res_offsets = res->offsets;
|
||||
res_chars.reserve(chars.size());
|
||||
res_offsets.reserve(size);
|
||||
|
||||
Offset_t current_new_offset = 0;
|
||||
Offset_t current_offset = 0;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
auto 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 auto zero16 = _mm_set1_epi8(0);
|
||||
|
||||
/// 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(chars_size_old + size);
|
||||
memcpy(&res_chars[chars_size_old], &chars[offset], size);
|
||||
};
|
||||
|
||||
while (filt_pos < filt_end_aligned)
|
||||
{
|
||||
if (!filt[i])
|
||||
continue;
|
||||
|
||||
size_t string_offset = i == 0 ? 0 : offsets[i - 1];
|
||||
size_t string_size = offsets[i] - string_offset;
|
||||
const auto mask = _mm_movemask_epi8(_mm_cmpgt_epi8(
|
||||
_mm_loadu_si128(reinterpret_cast<const __m128i *>(filt_pos)),
|
||||
zero16));
|
||||
|
||||
current_new_offset += string_size;
|
||||
res_offsets.push_back(current_new_offset);
|
||||
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;
|
||||
|
||||
res_chars.resize(res_chars.size() + string_size);
|
||||
memcpy(&res_chars[res_chars.size() - string_size], &chars[string_offset], string_size);
|
||||
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;
|
||||
}
|
||||
|
||||
return res;
|
||||
@ -248,12 +317,12 @@ public:
|
||||
reinterpret_cast<const char *>(&chars[offsetAt(n)]),
|
||||
reinterpret_cast<const char *>(&rhs.chars[rhs.offsetAt(m)]));
|
||||
}
|
||||
|
||||
|
||||
/// Версия compareAt для locale-sensitive сравнения строк
|
||||
int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs_, const Collator & collator) const
|
||||
{
|
||||
const ColumnString & rhs = static_cast<const ColumnString &>(rhs_);
|
||||
|
||||
|
||||
return collator.compare(
|
||||
reinterpret_cast<const char *>(&chars[offsetAt(n)]), sizeAt(n),
|
||||
reinterpret_cast<const char *>(&rhs.chars[rhs.offsetAt(m)]), rhs.sizeAt(m));
|
||||
@ -305,9 +374,9 @@ public:
|
||||
{
|
||||
const ColumnString & parent;
|
||||
const Collator & collator;
|
||||
|
||||
|
||||
lessWithCollation(const ColumnString & parent_, const Collator & collator_) : parent(parent_), collator(collator_) {}
|
||||
|
||||
|
||||
bool operator()(size_t lhs, size_t rhs) const
|
||||
{
|
||||
int res = collator.compare(
|
||||
@ -354,6 +423,9 @@ public:
|
||||
ColumnString * res_ = new ColumnString;
|
||||
ColumnPtr res = res_;
|
||||
|
||||
if (0 == col_size)
|
||||
return res;
|
||||
|
||||
Chars_t & res_chars = res_->chars;
|
||||
Offsets_t & res_offsets = res_->offsets;
|
||||
res_chars.reserve(chars.size() / col_size * replicate_offsets.back());
|
||||
|
@ -222,11 +222,51 @@ public:
|
||||
typename Self::Container_t & res_data = res_->getData();
|
||||
res_data.reserve(size);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
if (filt[i])
|
||||
res_data.push_back(data[i]);
|
||||
/** Чуть более оптимизированная версия.
|
||||
* Исходит из допущения, что часто куски последовательно идущих значений
|
||||
* полностью проходят или полностью не проходят фильтр.
|
||||
* Поэтому, будем оптимистично проверять куски по 16 значений.
|
||||
*/
|
||||
const UInt8 * filt_pos = &filt[0];
|
||||
const UInt8 * filt_end = filt_pos + size;
|
||||
const UInt8 * filt_end_sse = filt_pos + size / 16 * 16;
|
||||
const T * data_pos = &data[0];
|
||||
|
||||
return res;
|
||||
const __m128i zero16 = _mm_set1_epi8(0);
|
||||
|
||||
while (filt_pos < filt_end_sse)
|
||||
{
|
||||
int mask = _mm_movemask_epi8(_mm_cmpgt_epi8(_mm_loadu_si128(reinterpret_cast<const __m128i *>(filt_pos)), zero16));
|
||||
|
||||
if (0 == mask)
|
||||
{
|
||||
/// Ничего не вставляем.
|
||||
}
|
||||
else if (0xFFFF == mask)
|
||||
{
|
||||
res_data.insert_assume_reserved(data_pos, data_pos + 16);
|
||||
}
|
||||
else
|
||||
{
|
||||
for (size_t i = 0; i < 16; ++i)
|
||||
if (filt_pos[i])
|
||||
res_data.push_back(data_pos[i]);
|
||||
}
|
||||
|
||||
filt_pos += 16;
|
||||
data_pos += 16;
|
||||
}
|
||||
|
||||
while (filt_pos < filt_end)
|
||||
{
|
||||
if (*filt_pos)
|
||||
res_data.push_back(*data_pos);
|
||||
|
||||
++filt_pos;
|
||||
++data_pos;
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
ColumnPtr permute(const IColumn::Permutation & perm, size_t limit) const
|
||||
@ -256,6 +296,9 @@ public:
|
||||
if (size != offsets.size())
|
||||
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
||||
if (0 == size)
|
||||
return new Self;
|
||||
|
||||
Self * res_ = new Self;
|
||||
ColumnPtr res = res_;
|
||||
typename Self::Container_t & res_data = res_->getData();
|
||||
|
@ -198,4 +198,8 @@ public:
|
||||
};
|
||||
|
||||
|
||||
/// Считает, сколько байт в filt больше нуля.
|
||||
size_t countBytesInFilter(const IColumn::Filter & filt);
|
||||
|
||||
|
||||
}
|
||||
|
@ -6,7 +6,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
/** Базовый класс для столбцов-констант, содержащих значение, не входящее в Field.
|
||||
* Не является полноценым столбцом и используется особым образом.
|
||||
*/
|
||||
@ -14,16 +14,16 @@ class IColumnDummy : public IColumn
|
||||
{
|
||||
public:
|
||||
IColumnDummy(size_t s_) : s(s_) {}
|
||||
|
||||
|
||||
virtual ColumnPtr cloneDummy(size_t s_) const = 0;
|
||||
|
||||
|
||||
ColumnPtr cloneResized(size_t s_) const { return cloneDummy(s_); }
|
||||
bool isConst() const { return true; }
|
||||
size_t size() const { return s; }
|
||||
void insertDefault() { ++s; }
|
||||
size_t byteSize() const { return 0; }
|
||||
int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const { return 0; }
|
||||
|
||||
|
||||
Field operator[](size_t n) const { throw Exception("Cannot get value from " + getName(), ErrorCodes::NOT_IMPLEMENTED); }
|
||||
void get(size_t n, Field & res) const { throw Exception("Cannot get value from " + getName(), ErrorCodes::NOT_IMPLEMENTED); };
|
||||
void insert(const Field & x) { throw Exception("Cannot insert element into " + getName(), ErrorCodes::NOT_IMPLEMENTED); }
|
||||
@ -39,42 +39,42 @@ public:
|
||||
{
|
||||
return cloneDummy(length);
|
||||
}
|
||||
|
||||
|
||||
ColumnPtr filter(const Filter & filt) const
|
||||
{
|
||||
size_t new_size = 0;
|
||||
for (Filter::const_iterator it = filt.begin(); it != filt.end(); ++it)
|
||||
if (*it)
|
||||
++new_size;
|
||||
|
||||
|
||||
return cloneDummy(new_size);
|
||||
}
|
||||
|
||||
|
||||
ColumnPtr permute(const Permutation & perm, size_t limit) const
|
||||
{
|
||||
if (s != perm.size())
|
||||
throw Exception("Size of permutation doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
||||
|
||||
return cloneDummy(limit ? std::min(s, limit) : s);
|
||||
}
|
||||
|
||||
|
||||
void getPermutation(bool reverse, size_t limit, Permutation & res) const
|
||||
{
|
||||
res.resize(s);
|
||||
for (size_t i = 0; i < s; ++i)
|
||||
res[i] = i;
|
||||
}
|
||||
|
||||
|
||||
ColumnPtr replicate(const Offsets_t & offsets) const
|
||||
{
|
||||
if (s != offsets.size())
|
||||
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
||||
return cloneDummy(offsets.back());
|
||||
|
||||
return cloneDummy(s == 0 ? 0 : offsets.back());
|
||||
}
|
||||
|
||||
|
||||
private:
|
||||
size_t s;
|
||||
};
|
||||
|
||||
|
||||
}
|
||||
|
95
dbms/include/DB/Common/FileChecker.h
Normal file
95
dbms/include/DB/Common/FileChecker.h
Normal file
@ -0,0 +1,95 @@
|
||||
#pragma once
|
||||
|
||||
#include <Yandex/logger_useful.h>
|
||||
#include <DB/Columns/IColumn.h>
|
||||
#include <Poco/AutoPtr.h>
|
||||
#include <Poco/Util/XMLConfiguration.h>
|
||||
#include <string>
|
||||
#include <Poco/File.h>
|
||||
#include <DB/Common/escapeForFileName.h>
|
||||
#include <jsonxx.h>
|
||||
#include <fstream>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// хранит размеры всех столбцов, и может проверять не побились ли столбцы
|
||||
template <class Storage>
|
||||
class FileChecker
|
||||
{
|
||||
public:
|
||||
FileChecker(const std::string &file_info_path_, Storage & storage_) :
|
||||
files_info_path(file_info_path_), storage(storage_), log(&Logger::get("FileChecker"))
|
||||
{
|
||||
std::ifstream istr(files_info_path);
|
||||
files_info.parse(istr);
|
||||
}
|
||||
|
||||
void setPath(const std::string & file_info_path_)
|
||||
{
|
||||
files_info_path = file_info_path_;
|
||||
}
|
||||
|
||||
using Files = std::vector<Poco::File>;
|
||||
|
||||
void update(const Poco::File & file)
|
||||
{
|
||||
updateTree(file);
|
||||
saveTree();
|
||||
}
|
||||
|
||||
void update(const Files::iterator & begin, const Files::iterator & end)
|
||||
{
|
||||
for (auto it = begin; it != end; ++it)
|
||||
updateTree(*it);
|
||||
saveTree();
|
||||
}
|
||||
|
||||
/// Проверяем файлы, параметры которых указаны в sizes.json
|
||||
bool check() const
|
||||
{
|
||||
bool correct = true;
|
||||
for (auto & node : files_info.kv_map())
|
||||
{
|
||||
std::string filename = unescapeForFileName(node.first);
|
||||
size_t expected_size = std::stoull(node.second->get<jsonxx::Object>().get<std::string>("size"));
|
||||
|
||||
Poco::File file(Poco::Path(files_info_path).parent().toString() + "/" + filename);
|
||||
if (!file.exists())
|
||||
{
|
||||
LOG_ERROR(log, "File " << file.path() << " doesn't exists");
|
||||
correct = false;
|
||||
continue;
|
||||
}
|
||||
|
||||
size_t real_size = file.getSize();
|
||||
if (real_size != expected_size)
|
||||
{
|
||||
LOG_ERROR(log, "Size of " << file.path() << " is wrong. Size is " << real_size << " but should be " << expected_size);
|
||||
correct = false;
|
||||
}
|
||||
}
|
||||
return correct;
|
||||
}
|
||||
|
||||
private:
|
||||
void updateTree(const Poco::File & file)
|
||||
{
|
||||
files_info.import(escapeForFileName(Poco::Path(file.path()).getFileName()),
|
||||
jsonxx::Object("size", std::to_string(file.getSize())));
|
||||
}
|
||||
|
||||
void saveTree()
|
||||
{
|
||||
std::ofstream file(files_info_path, std::ofstream::trunc);
|
||||
file << files_info.write(jsonxx::JSON);
|
||||
}
|
||||
|
||||
std::string files_info_path;
|
||||
|
||||
jsonxx::Object files_info;
|
||||
|
||||
Storage & storage;
|
||||
Logger * log;
|
||||
};
|
||||
}
|
27
dbms/include/DB/Common/Macros.h
Normal file
27
dbms/include/DB/Common/Macros.h
Normal file
@ -0,0 +1,27 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Core/Types.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <map>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Раскрывает в строке макросы из конфига.
|
||||
*/
|
||||
class Macros
|
||||
{
|
||||
public:
|
||||
Macros();
|
||||
Macros(const Poco::Util::AbstractConfiguration & config, const String & key);
|
||||
|
||||
/// Заменить в строке подстроки вида {macro_name} на значение для macro_name, полученное из конфига.
|
||||
String expand(const String & s) const;
|
||||
|
||||
private:
|
||||
typedef std::map<String, String> MacroMap;
|
||||
|
||||
MacroMap macros;
|
||||
};
|
||||
|
||||
}
|
@ -248,6 +248,12 @@ public:
|
||||
if (required_capacity > capacity())
|
||||
reserve(round_up_to_power_of_two(required_capacity));
|
||||
|
||||
insert_assume_reserved(from_begin, from_end);
|
||||
}
|
||||
|
||||
template <typename It1, typename It2>
|
||||
void insert_assume_reserved(It1 from_begin, It2 from_end)
|
||||
{
|
||||
size_t bytes_to_copy = byte_size(from_end - from_begin);
|
||||
memcpy(c_end, reinterpret_cast<const void *>(&*from_begin), bytes_to_copy);
|
||||
c_end += bytes_to_copy;
|
||||
|
@ -21,7 +21,7 @@
|
||||
* - ищем её в хэш-таблице, если нашли - достаём смещение из хэш-таблицы и сравниваем строку побайтово;
|
||||
* - если сравнить не получилось - проверяем следующую ячейку хэш-таблицы из цепочки разрешения коллизий;
|
||||
* - если не нашли, пропускаем в haystack почти размер needle байт;
|
||||
*
|
||||
*
|
||||
* Используется невыровненный доступ к памяти.
|
||||
*/
|
||||
class Volnitsky
|
||||
@ -35,7 +35,7 @@ private:
|
||||
const char * needle_end;
|
||||
size_t step; /// Насколько двигаемся, если n-грамма из haystack не нашлась в хэш-таблице.
|
||||
|
||||
static const size_t hash_size = 64 * 1024; /// Обычно помещается в L1-кэш, хотя занимает его целиком.
|
||||
static const size_t hash_size = 64 * 1024; /// Помещается в L2-кэш.
|
||||
offset_t hash[hash_size]; /// Хэш-таблица.
|
||||
|
||||
bool fallback; /// Нужно ли использовать fallback алгоритм.
|
||||
@ -57,7 +57,7 @@ public:
|
||||
}
|
||||
else
|
||||
fallback = false;
|
||||
|
||||
|
||||
memset(hash, 0, hash_size * sizeof(hash[0]));
|
||||
|
||||
for (int i = needle_size - sizeof(ngram_t); i >= 0; --i)
|
||||
|
@ -64,3 +64,5 @@
|
||||
#define DBMS_MIN_REVISION_WITH_TOTALS_EXTREMES 35265
|
||||
#define DBMS_MIN_REVISION_WITH_STRING_QUERY_ID 39002
|
||||
#define DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES 50264
|
||||
|
||||
#define DBMS_DISTRIBUTED_DIRECTORY_MONITOR_SLEEP_TIME_MS 100
|
||||
|
@ -255,6 +255,10 @@ namespace ErrorCodes
|
||||
INVALID_NESTED_NAME,
|
||||
CORRUPTED_DATA,
|
||||
INCORRECT_MARK,
|
||||
INVALID_PARTITION_NAME,
|
||||
NOT_LEADER,
|
||||
NOT_ENOUGH_BLOCK_NUMBERS,
|
||||
NO_SUCH_REPLICA,
|
||||
|
||||
POCO_EXCEPTION = 1000,
|
||||
STD_EXCEPTION,
|
||||
|
@ -582,11 +582,39 @@ private:
|
||||
writeQuoted(x, wb);
|
||||
return res;
|
||||
}
|
||||
|
||||
/** В отличие от writeFloatText (и writeQuoted), если число после форматирования выглядит целым, всё равно добавляет десятичную точку.
|
||||
* - для того, чтобы это число могло обратно распарситься как Float64 парсером запроса (иначе распарсится как целое).
|
||||
*
|
||||
* При этом, не оставляет завершающие нули справа.
|
||||
*
|
||||
* NOTE: При таком roundtrip-е, точность может теряться.
|
||||
*/
|
||||
static inline String formatFloat(Float64 x)
|
||||
{
|
||||
char tmp[24];
|
||||
int res = std::snprintf(tmp, 23, "%.*g", WRITE_HELPERS_DEFAULT_FLOAT_PRECISION, x);
|
||||
|
||||
if (res >= 23 || res <= 0)
|
||||
throw Exception("Cannot print float or double number", ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER);
|
||||
|
||||
size_t string_size = res;
|
||||
|
||||
tmp[23] = '\0';
|
||||
if (string_size == strspn(tmp, "-0123456789"))
|
||||
{
|
||||
tmp[string_size] = '.';
|
||||
++string_size;
|
||||
}
|
||||
|
||||
return {tmp, string_size};
|
||||
}
|
||||
|
||||
public:
|
||||
String operator() (const Null & x) const { return "NULL"; }
|
||||
String operator() (const UInt64 & x) const { return formatQuoted(x); }
|
||||
String operator() (const Int64 & x) const { return formatQuoted(x); }
|
||||
String operator() (const Float64 & x) const { return formatQuoted(x); }
|
||||
String operator() (const Float64 & x) const { return formatFloat(x); }
|
||||
String operator() (const String & x) const { return formatQuoted(x); }
|
||||
|
||||
String operator() (const Array & x) const
|
||||
@ -689,7 +717,7 @@ namespace DB
|
||||
{
|
||||
class ReadBuffer;
|
||||
class WriteBuffer;
|
||||
|
||||
|
||||
/// Предполагается что у всех элементов массива одинаковый тип.
|
||||
inline void readBinary(Array & x, ReadBuffer & buf)
|
||||
{
|
||||
@ -745,7 +773,7 @@ namespace DB
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
inline void readText(Array & x, ReadBuffer & buf) { throw Exception("Cannot read Array.", ErrorCodes::NOT_IMPLEMENTED); }
|
||||
inline void readQuoted(Array & x, ReadBuffer & buf) { throw Exception("Cannot read Array.", ErrorCodes::NOT_IMPLEMENTED); }
|
||||
|
||||
@ -758,7 +786,7 @@ namespace DB
|
||||
type = x.front().getType();
|
||||
DB::writeBinary(type, buf);
|
||||
DB::writeBinary(size, buf);
|
||||
|
||||
|
||||
for (Array::const_iterator it = x.begin(); it != x.end(); ++it)
|
||||
{
|
||||
switch (type)
|
||||
@ -792,13 +820,13 @@ namespace DB
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
inline void writeText(const Array & x, WriteBuffer & buf)
|
||||
{
|
||||
DB::String res = apply_visitor(DB::FieldVisitorToString(), DB::Field(x));
|
||||
buf.write(res.data(), res.size());
|
||||
}
|
||||
|
||||
|
||||
inline void writeQuoted(const Array & x, WriteBuffer & buf) { throw Exception("Cannot write Array quoted.", ErrorCodes::NOT_IMPLEMENTED); }
|
||||
}
|
||||
|
||||
|
@ -26,7 +26,7 @@ struct StringRef
|
||||
|
||||
typedef std::vector<StringRef> StringRefs;
|
||||
|
||||
inline bool operator==(StringRef lhs, StringRef rhs)
|
||||
inline bool operator== (StringRef lhs, StringRef rhs)
|
||||
{
|
||||
/// Так почему-то быстрее, чем return lhs.size == rhs.size && 0 == memcmp(lhs.data, rhs.data, lhs.size);
|
||||
|
||||
@ -40,18 +40,21 @@ inline bool operator==(StringRef lhs, StringRef rhs)
|
||||
return true;
|
||||
}
|
||||
|
||||
inline bool operator!=(StringRef lhs, StringRef rhs)
|
||||
inline bool operator!= (StringRef lhs, StringRef rhs)
|
||||
{
|
||||
return !(lhs == rhs);
|
||||
}
|
||||
|
||||
inline bool operator<(StringRef lhs, StringRef rhs)
|
||||
inline bool operator< (StringRef lhs, StringRef rhs)
|
||||
{
|
||||
int cmp = memcmp(lhs.data, rhs.data, std::min(lhs.size, rhs.size));
|
||||
if (cmp == 0)
|
||||
return lhs.size < rhs.size;
|
||||
else
|
||||
return cmp < 0;
|
||||
return cmp < 0 || (cmp == 0 && lhs.size < rhs.size);
|
||||
}
|
||||
|
||||
inline bool operator> (StringRef lhs, StringRef rhs)
|
||||
{
|
||||
int cmp = memcmp(lhs.data, rhs.data, std::min(lhs.size, rhs.size));
|
||||
return cmp > 0 || (cmp == 0 && lhs.size > rhs.size);
|
||||
}
|
||||
|
||||
|
||||
|
@ -33,6 +33,8 @@ public:
|
||||
output->write(res);
|
||||
}
|
||||
|
||||
void flush() { output->flush(); }
|
||||
|
||||
private:
|
||||
BlockOutputStreamPtr output;
|
||||
NamesAndTypesListPtr required_columns;
|
||||
|
@ -20,6 +20,8 @@ public:
|
||||
void writeField(const Field & field);
|
||||
void writeRowEndDelimiter();
|
||||
|
||||
void flush() { ostr.next(); }
|
||||
|
||||
protected:
|
||||
WriteBuffer & ostr;
|
||||
const Block sample;
|
||||
|
@ -17,6 +17,8 @@ public:
|
||||
void write(const Block & block);
|
||||
void writePrefix() { row_output->writePrefix(); }
|
||||
void writeSuffix() { row_output->writeSuffix(); }
|
||||
|
||||
void flush() { row_output->flush(); }
|
||||
|
||||
void setRowsBeforeLimit(size_t rows_before_limit);
|
||||
void setTotals(const Block & totals);
|
||||
|
@ -63,7 +63,6 @@ public:
|
||||
BlockInputStreams & getChildren() { return children; }
|
||||
|
||||
void dumpTree(std::ostream & ostr, size_t indent = 0, size_t multiplier = 1);
|
||||
void dumpTreeWithProfile(std::ostream & ostr, size_t indent = 0);
|
||||
|
||||
/// Получить листовые источники (не считая этот).
|
||||
BlockInputStreams getLeaves();
|
||||
|
@ -31,6 +31,10 @@ public:
|
||||
*/
|
||||
virtual void writePrefix() {}
|
||||
virtual void writeSuffix() {}
|
||||
|
||||
/** Сбросить имеющиеся буферы для записи.
|
||||
*/
|
||||
virtual void flush() {}
|
||||
|
||||
/** Методы для установки дополнительной информации для вывода в поддерживающих её форматах.
|
||||
*/
|
||||
|
@ -21,8 +21,7 @@ namespace DB
|
||||
struct BlockStreamProfileInfo
|
||||
{
|
||||
bool started = false;
|
||||
Stopwatch work_stopwatch; /// Время вычислений (выполнения функции read())
|
||||
Stopwatch total_stopwatch; /// Время с учётом ожидания
|
||||
Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; /// Время с учётом ожидания
|
||||
|
||||
String stream_name; /// Короткое имя потока, для которого собирается информация
|
||||
|
||||
@ -48,7 +47,6 @@ struct BlockStreamProfileInfo
|
||||
bool hasAppliedLimit() const;
|
||||
|
||||
void update(Block & block);
|
||||
void print(std::ostream & ostr) const;
|
||||
|
||||
/// Методы для бинарной [де]сериализации
|
||||
void read(ReadBuffer & in);
|
||||
|
@ -32,6 +32,9 @@ public:
|
||||
virtual void writePrefix() {}; /// разделитель перед началом результата
|
||||
virtual void writeSuffix() {}; /// разделитель после конца результата
|
||||
|
||||
/** Сбросить имеющиеся буферы для записи. */
|
||||
virtual void flush() {}
|
||||
|
||||
/** Методы для установки дополнительной информации для вывода в поддерживающих её форматах.
|
||||
*/
|
||||
virtual void setRowsBeforeLimit(size_t rows_before_limit) {}
|
||||
|
@ -25,6 +25,8 @@ public:
|
||||
void writeRowEndDelimiter();
|
||||
void writePrefix();
|
||||
void writeSuffix();
|
||||
|
||||
void flush() { ostr.next(); dst_ostr.next(); }
|
||||
|
||||
void setRowsBeforeLimit(size_t rows_before_limit_)
|
||||
{
|
||||
@ -41,7 +43,8 @@ protected:
|
||||
virtual void writeTotals();
|
||||
virtual void writeExtremes();
|
||||
|
||||
WriteBufferValidUTF8 ostr;
|
||||
WriteBuffer & dst_ostr;
|
||||
WriteBufferValidUTF8 ostr; /// Валидирует и пишет в dst_ostr.
|
||||
size_t field_number;
|
||||
size_t row_count;
|
||||
bool applied_limit;
|
||||
|
@ -15,6 +15,8 @@ public:
|
||||
NativeBlockOutputStream(WriteBuffer & ostr_) : ostr(ostr_) {}
|
||||
void write(const Block & block);
|
||||
|
||||
void flush() { ostr.next(); }
|
||||
|
||||
private:
|
||||
WriteBuffer & ostr;
|
||||
};
|
||||
|
@ -18,6 +18,8 @@ public:
|
||||
void write(const Block & block);
|
||||
void writeSuffix();
|
||||
|
||||
void flush() { ostr.next(); }
|
||||
|
||||
void setTotals(const Block & totals_) { totals = totals_; }
|
||||
void setExtremes(const Block & extremes_) { extremes = extremes_; }
|
||||
|
||||
|
@ -18,8 +18,8 @@ namespace DB
|
||||
class PushingToViewsBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
PushingToViewsBlockOutputStream(String database_, String table_, const Context &context_, ASTPtr query_ptr_)
|
||||
:database(database_), table(table_), context(context_), query_ptr(query_ptr_)
|
||||
PushingToViewsBlockOutputStream(String database_, String table_, const Context & context_, ASTPtr query_ptr_)
|
||||
: database(database_), table(table_), context(context_), query_ptr(query_ptr_)
|
||||
{
|
||||
if (database.empty())
|
||||
database = context.getCurrentDatabase();
|
||||
|
@ -14,8 +14,8 @@ namespace DB
|
||||
class RemoteBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
RemoteBlockOutputStream(Connection & connection_, const String & query_)
|
||||
: connection(connection_), query(query_)
|
||||
RemoteBlockOutputStream(Connection & connection_, const String & query_, Settings * settings_ = nullptr)
|
||||
: connection(connection_), query(query_), settings(settings_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -26,7 +26,7 @@ public:
|
||||
*/
|
||||
Block sendQueryAndGetSampleBlock()
|
||||
{
|
||||
connection.sendQuery(query);
|
||||
connection.sendQuery(query, "", QueryProcessingStage::Complete, settings);
|
||||
sent_query = true;
|
||||
|
||||
Connection::Packet packet = connection.receivePacket();
|
||||
@ -64,12 +64,12 @@ public:
|
||||
|
||||
|
||||
/// Отправить блок данных, который уже был заранее сериализован (и, если надо, сжат), который следует прочитать из input-а.
|
||||
void writePrepared(ReadBuffer & input)
|
||||
void writePrepared(ReadBuffer & input, size_t size = 0)
|
||||
{
|
||||
if (!sent_query)
|
||||
sendQueryAndGetSampleBlock(); /// Никак не можем использовать sample_block.
|
||||
|
||||
connection.sendPreparedData(input);
|
||||
connection.sendPreparedData(input, size);
|
||||
}
|
||||
|
||||
|
||||
@ -95,6 +95,7 @@ public:
|
||||
private:
|
||||
Connection & connection;
|
||||
String query;
|
||||
Settings * settings;
|
||||
Block sample_block;
|
||||
|
||||
bool sent_query = false;
|
||||
|
@ -16,6 +16,8 @@ public:
|
||||
TabSeparatedBlockOutputStream(WriteBuffer & ostr_) : ostr(ostr_) {}
|
||||
void write(const Block & block);
|
||||
|
||||
void flush() { ostr.next(); }
|
||||
|
||||
private:
|
||||
WriteBuffer & ostr;
|
||||
};
|
||||
|
@ -26,6 +26,8 @@ public:
|
||||
void writePrefix();
|
||||
void writeSuffix();
|
||||
|
||||
void flush() { ostr.next(); }
|
||||
|
||||
void setTotals(const Block & totals_) { totals = totals_; }
|
||||
void setExtremes(const Block & extremes_) { extremes = extremes_; }
|
||||
|
||||
|
@ -26,6 +26,8 @@ public:
|
||||
void writeRowEndDelimiter();
|
||||
void writeRowBetweenDelimiter();
|
||||
|
||||
void flush() { ostr.next(); }
|
||||
|
||||
private:
|
||||
WriteBuffer & ostr;
|
||||
const Block sample;
|
||||
|
@ -25,6 +25,8 @@ public:
|
||||
void writeRowStartDelimiter();
|
||||
void writeRowBetweenDelimiter();
|
||||
|
||||
void flush() { ostr.next(); }
|
||||
|
||||
private:
|
||||
WriteBuffer & ostr;
|
||||
const Block sample;
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Functions/IFunction.h>
|
||||
|
||||
|
||||
@ -17,10 +16,19 @@ class Context;
|
||||
*/
|
||||
class FunctionFactory
|
||||
{
|
||||
private:
|
||||
typedef IFunction* (*Creator)(const Context & context); /// Не std::function, так как меньше indirection и размер объекта.
|
||||
std::unordered_map<String, Creator> functions;
|
||||
|
||||
public:
|
||||
FunctionPtr get(
|
||||
const String & name,
|
||||
const Context & context) const;
|
||||
FunctionFactory();
|
||||
|
||||
FunctionPtr get(const String & name, const Context & context) const;
|
||||
|
||||
void registerFunction(const String & name, Creator creator)
|
||||
{
|
||||
functions[name] = creator;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -14,7 +14,7 @@ namespace DB
|
||||
*/
|
||||
|
||||
template<typename A, typename B, typename Op>
|
||||
struct BinaryOperationImpl
|
||||
struct BinaryOperationImplBase
|
||||
{
|
||||
typedef typename Op::ResultType ResultType;
|
||||
|
||||
@ -45,6 +45,11 @@ struct BinaryOperationImpl
|
||||
}
|
||||
};
|
||||
|
||||
template<typename A, typename B, typename Op>
|
||||
struct BinaryOperationImpl : BinaryOperationImplBase<A, B, Op>
|
||||
{
|
||||
};
|
||||
|
||||
template<typename A, typename Op>
|
||||
struct UnaryOperationImpl
|
||||
{
|
||||
@ -221,7 +226,7 @@ struct NegateImpl
|
||||
|
||||
static inline ResultType apply(A a)
|
||||
{
|
||||
return -a;
|
||||
return -static_cast<ResultType>(a);
|
||||
}
|
||||
};
|
||||
|
||||
@ -570,4 +575,140 @@ typedef FunctionBinaryArithmetic<BitShiftRightImpl, NameBitShiftRight> Functi
|
||||
|
||||
|
||||
|
||||
/// Оптимизации для целочисленного деления на константу.
|
||||
|
||||
#define LIBDIVIDE_USE_SSE2 1
|
||||
#include <libdivide.h>
|
||||
|
||||
|
||||
template <typename A, typename B>
|
||||
struct DivideIntegralByConstantImpl
|
||||
: BinaryOperationImplBase<A, B, DivideIntegralImpl<A, B>>
|
||||
{
|
||||
typedef typename DivideIntegralImpl<A, B>::ResultType ResultType;
|
||||
|
||||
static void vector_constant(const PODArray<A> & a, B b, PODArray<ResultType> & c)
|
||||
{
|
||||
if (unlikely(b == 0))
|
||||
throw Exception("Division by zero", ErrorCodes::ILLEGAL_DIVISION);
|
||||
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wsign-compare"
|
||||
|
||||
if (unlikely(std::is_signed<B>::value && b == -1))
|
||||
{
|
||||
size_t size = a.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = -c[i];
|
||||
return;
|
||||
}
|
||||
|
||||
#pragma GCC diagnostic pop
|
||||
|
||||
libdivide::divider<A> divider(b);
|
||||
|
||||
size_t size = a.size();
|
||||
const A * a_pos = &a[0];
|
||||
const A * a_end = a_pos + size;
|
||||
ResultType * c_pos = &c[0];
|
||||
static constexpr size_t values_per_sse_register = 16 / sizeof(A);
|
||||
const A * a_end_sse = a_pos + size / values_per_sse_register * values_per_sse_register;
|
||||
|
||||
while (a_pos < a_end_sse)
|
||||
{
|
||||
_mm_storeu_si128(reinterpret_cast<__m128i *>(c_pos),
|
||||
_mm_loadu_si128(reinterpret_cast<const __m128i *>(a_pos)) / divider);
|
||||
|
||||
a_pos += values_per_sse_register;
|
||||
c_pos += values_per_sse_register;
|
||||
}
|
||||
|
||||
while (a_pos < a_end)
|
||||
{
|
||||
*c_pos = *a_pos / divider;
|
||||
++a_pos;
|
||||
++c_pos;
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
template <typename A, typename B>
|
||||
struct ModuloByConstantImpl
|
||||
: BinaryOperationImplBase<A, B, ModuloImpl<A, B>>
|
||||
{
|
||||
typedef typename ModuloImpl<A, B>::ResultType ResultType;
|
||||
|
||||
static void vector_constant(const PODArray<A> & a, B b, PODArray<ResultType> & c)
|
||||
{
|
||||
if (unlikely(b == 0))
|
||||
throw Exception("Division by zero", ErrorCodes::ILLEGAL_DIVISION);
|
||||
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wsign-compare"
|
||||
|
||||
if (unlikely((std::is_signed<B>::value && b == -1) || b == 1))
|
||||
{
|
||||
size_t size = a.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = 0;
|
||||
return;
|
||||
}
|
||||
|
||||
#pragma GCC diagnostic pop
|
||||
|
||||
libdivide::divider<A> divider(b);
|
||||
|
||||
/// Тут не удалось сделать так, чтобы SSE вариант из libdivide давал преимущество.
|
||||
size_t size = a.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = a[i] - (a[i] / divider) * b; /// NOTE: возможно, не сохраняется семантика деления с остатком отрицательных чисел.
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/** Прописаны специализации для деления чисел типа UInt64 и UInt32 на числа той же знаковости.
|
||||
* Можно дополнить до всех возможных комбинаций, но потребуется больше кода.
|
||||
*/
|
||||
|
||||
template <> struct BinaryOperationImpl<UInt64, UInt8, DivideIntegralImpl<UInt64, UInt8>> : DivideIntegralByConstantImpl<UInt64, UInt8> {};
|
||||
template <> struct BinaryOperationImpl<UInt64, UInt16, DivideIntegralImpl<UInt64, UInt16>> : DivideIntegralByConstantImpl<UInt64, UInt16> {};
|
||||
template <> struct BinaryOperationImpl<UInt64, UInt32, DivideIntegralImpl<UInt64, UInt32>> : DivideIntegralByConstantImpl<UInt64, UInt32> {};
|
||||
template <> struct BinaryOperationImpl<UInt64, UInt64, DivideIntegralImpl<UInt64, UInt64>> : DivideIntegralByConstantImpl<UInt64, UInt64> {};
|
||||
|
||||
template <> struct BinaryOperationImpl<UInt32, UInt8, DivideIntegralImpl<UInt32, UInt8>> : DivideIntegralByConstantImpl<UInt32, UInt8> {};
|
||||
template <> struct BinaryOperationImpl<UInt32, UInt16, DivideIntegralImpl<UInt32, UInt16>> : DivideIntegralByConstantImpl<UInt32, UInt16> {};
|
||||
template <> struct BinaryOperationImpl<UInt32, UInt32, DivideIntegralImpl<UInt32, UInt32>> : DivideIntegralByConstantImpl<UInt32, UInt32> {};
|
||||
template <> struct BinaryOperationImpl<UInt32, UInt64, DivideIntegralImpl<UInt32, UInt64>> : DivideIntegralByConstantImpl<UInt32, UInt64> {};
|
||||
|
||||
template <> struct BinaryOperationImpl<Int64, Int8, DivideIntegralImpl<Int64, Int8>> : DivideIntegralByConstantImpl<Int64, Int8> {};
|
||||
template <> struct BinaryOperationImpl<Int64, Int16, DivideIntegralImpl<Int64, Int16>> : DivideIntegralByConstantImpl<Int64, Int16> {};
|
||||
template <> struct BinaryOperationImpl<Int64, Int32, DivideIntegralImpl<Int64, Int32>> : DivideIntegralByConstantImpl<Int64, Int32> {};
|
||||
template <> struct BinaryOperationImpl<Int64, Int64, DivideIntegralImpl<Int64, Int64>> : DivideIntegralByConstantImpl<Int64, Int64> {};
|
||||
|
||||
template <> struct BinaryOperationImpl<Int32, Int8, DivideIntegralImpl<Int32, Int8>> : DivideIntegralByConstantImpl<Int32, Int8> {};
|
||||
template <> struct BinaryOperationImpl<Int32, Int16, DivideIntegralImpl<Int32, Int16>> : DivideIntegralByConstantImpl<Int32, Int16> {};
|
||||
template <> struct BinaryOperationImpl<Int32, Int32, DivideIntegralImpl<Int32, Int32>> : DivideIntegralByConstantImpl<Int32, Int32> {};
|
||||
template <> struct BinaryOperationImpl<Int32, Int64, DivideIntegralImpl<Int32, Int64>> : DivideIntegralByConstantImpl<Int32, Int64> {};
|
||||
|
||||
|
||||
template <> struct BinaryOperationImpl<UInt64, UInt8, ModuloImpl<UInt64, UInt8>> : ModuloByConstantImpl<UInt64, UInt8> {};
|
||||
template <> struct BinaryOperationImpl<UInt64, UInt16, ModuloImpl<UInt64, UInt16>> : ModuloByConstantImpl<UInt64, UInt16> {};
|
||||
template <> struct BinaryOperationImpl<UInt64, UInt32, ModuloImpl<UInt64, UInt32>> : ModuloByConstantImpl<UInt64, UInt32> {};
|
||||
template <> struct BinaryOperationImpl<UInt64, UInt64, ModuloImpl<UInt64, UInt64>> : ModuloByConstantImpl<UInt64, UInt64> {};
|
||||
|
||||
template <> struct BinaryOperationImpl<UInt32, UInt8, ModuloImpl<UInt32, UInt8>> : ModuloByConstantImpl<UInt32, UInt8> {};
|
||||
template <> struct BinaryOperationImpl<UInt32, UInt16, ModuloImpl<UInt32, UInt16>> : ModuloByConstantImpl<UInt32, UInt16> {};
|
||||
template <> struct BinaryOperationImpl<UInt32, UInt32, ModuloImpl<UInt32, UInt32>> : ModuloByConstantImpl<UInt32, UInt32> {};
|
||||
template <> struct BinaryOperationImpl<UInt32, UInt64, ModuloImpl<UInt32, UInt64>> : ModuloByConstantImpl<UInt32, UInt64> {};
|
||||
|
||||
template <> struct BinaryOperationImpl<Int64, Int8, ModuloImpl<Int64, Int8>> : ModuloByConstantImpl<Int64, Int8> {};
|
||||
template <> struct BinaryOperationImpl<Int64, Int16, ModuloImpl<Int64, Int16>> : ModuloByConstantImpl<Int64, Int16> {};
|
||||
template <> struct BinaryOperationImpl<Int64, Int32, ModuloImpl<Int64, Int32>> : ModuloByConstantImpl<Int64, Int32> {};
|
||||
template <> struct BinaryOperationImpl<Int64, Int64, ModuloImpl<Int64, Int64>> : ModuloByConstantImpl<Int64, Int64> {};
|
||||
|
||||
template <> struct BinaryOperationImpl<Int32, Int8, ModuloImpl<Int32, Int8>> : ModuloByConstantImpl<Int32, Int8> {};
|
||||
template <> struct BinaryOperationImpl<Int32, Int16, ModuloImpl<Int32, Int16>> : ModuloByConstantImpl<Int32, Int16> {};
|
||||
template <> struct BinaryOperationImpl<Int32, Int32, ModuloImpl<Int32, Int32>> : ModuloByConstantImpl<Int32, Int32> {};
|
||||
template <> struct BinaryOperationImpl<Int32, Int64, ModuloImpl<Int32, Int64>> : ModuloByConstantImpl<Int32, Int64> {};
|
||||
|
||||
}
|
||||
|
@ -4,10 +4,12 @@
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/DataTypes/DataTypeFixedString.h>
|
||||
#include <DB/DataTypes/DataTypeArray.h>
|
||||
#include <DB/DataTypes/DataTypeDate.h>
|
||||
#include <DB/DataTypes/DataTypeDateTime.h>
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
#include <DB/Columns/ColumnFixedString.h>
|
||||
#include <DB/Columns/ColumnArray.h>
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
#include <DB/Functions/IFunction.h>
|
||||
|
||||
@ -375,7 +377,7 @@ public:
|
||||
prev_offset = new_offset;
|
||||
}
|
||||
|
||||
if (out_offsets.back() != out_vec.size())
|
||||
if (!out_offsets.empty() && out_offsets.back() != out_vec.size())
|
||||
throw Exception("Column size mismatch (internal logical error)", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return true;
|
||||
@ -436,7 +438,7 @@ public:
|
||||
prev_offset = new_offset;
|
||||
}
|
||||
|
||||
if (out_offsets.back() != out_vec.size())
|
||||
if (!out_offsets.empty() && out_offsets.back() != out_vec.size())
|
||||
throw Exception("Column size mismatch (internal logical error)", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return true;
|
||||
@ -742,7 +744,7 @@ public:
|
||||
}
|
||||
out_vec.resize(pos - begin);
|
||||
|
||||
if (out_offsets.back() != out_vec.size())
|
||||
if (!out_offsets.empty() && out_offsets.back() != out_vec.size())
|
||||
throw Exception("Column size mismatch (internal logical error)", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return true;
|
||||
@ -797,7 +799,7 @@ public:
|
||||
}
|
||||
out_vec.resize(pos - begin);
|
||||
|
||||
if (out_offsets.back() != out_vec.size())
|
||||
if (!out_offsets.empty() && out_offsets.back() != out_vec.size())
|
||||
throw Exception("Column size mismatch (internal logical error)", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return true;
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -1,6 +1,6 @@
|
||||
#include <DB/Functions/IFunction.h>
|
||||
#include "NumberTraits.h"
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/IO/WriteBufferFromVector.h>
|
||||
#include <DB/IO/WriteBufferFromString.h>
|
||||
|
@ -50,6 +50,8 @@ namespace DB
|
||||
* не предназначена для пользователя, а используется только как prerequisites для функций высшего порядка.
|
||||
*
|
||||
* sleep(n) - спит n секунд каждый блок.
|
||||
*
|
||||
* bar(x, min, max, width) - рисует полосу из количества символов, пропорционального (x - min) и равного width при x == max.
|
||||
*/
|
||||
|
||||
|
||||
@ -274,7 +276,7 @@ public:
|
||||
{
|
||||
const IColumn & argument = *block.getByPosition(arguments[0]).column;
|
||||
if (!argument.isConst())
|
||||
throw Exception("Argument for function 'materialize' must be constant.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
throw Exception("Argument for function " + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
block.getByPosition(result).column = dynamic_cast<const IColumnConst &>(argument).convertToFullColumn();
|
||||
}
|
||||
@ -354,7 +356,7 @@ public:
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() < 2)
|
||||
throw Exception("Function tuple requires at least two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
throw Exception("Function " + getName() + " requires at least two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
return new DataTypeTuple(arguments);
|
||||
}
|
||||
@ -386,18 +388,18 @@ public:
|
||||
ExpressionActions::Actions & out_prerequisites)
|
||||
{
|
||||
if (arguments.size() != 2)
|
||||
throw Exception("Function tupleElement requires exactly two arguments: tuple and element index.",
|
||||
throw Exception("Function " + getName() + " requires exactly two arguments: tuple and element index.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const ColumnConstUInt8 * index_col = typeid_cast<const ColumnConstUInt8 *>(&*arguments[1].column);
|
||||
if (!index_col)
|
||||
throw Exception("Second argument to tupleElement must be a constant UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception("Second argument to " + getName() + " must be a constant UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
size_t index = index_col->getData();
|
||||
|
||||
const DataTypeTuple * tuple = typeid_cast<const DataTypeTuple *>(&*arguments[0].type);
|
||||
if (!tuple)
|
||||
throw Exception("First argument for function tupleElement must be tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception("First argument for function " + getName() + " must be tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (index == 0)
|
||||
throw Exception("Indices in tuples are 1-based.", ErrorCodes::ILLEGAL_INDEX);
|
||||
@ -417,10 +419,10 @@ public:
|
||||
const ColumnConstUInt8 * index_col = typeid_cast<const ColumnConstUInt8 *>(&*block.getByPosition(arguments[1]).column);
|
||||
|
||||
if (!tuple_col)
|
||||
throw Exception("First argument for function tupleElement must be tuple.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
throw Exception("First argument for function " + getName() + " must be tuple.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
if (!index_col)
|
||||
throw Exception("Second argument for function tupleElement must be UInt8 constant literal.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
throw Exception("Second argument for function " + getName() + " must be UInt8 constant literal.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
size_t index = index_col->getData();
|
||||
if (index == 0)
|
||||
@ -472,11 +474,11 @@ public:
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Function arrayJoin requires exactly one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
throw Exception("Function " + getName() + " requires exactly one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const DataTypeArray * arr = typeid_cast<const DataTypeArray *>(&*arguments[0]);
|
||||
if (!arr)
|
||||
throw Exception("Argument for function arrayJoin must be Array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception("Argument for function " + getName() + " must be Array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return arr->getNestedType()->clone();
|
||||
}
|
||||
@ -484,7 +486,7 @@ public:
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
throw Exception("Function arrayJoin must not be executed directly.", ErrorCodes::FUNCTION_IS_SPECIAL);
|
||||
throw Exception("Function " + getName() + " must not be executed directly.", ErrorCodes::FUNCTION_IS_SPECIAL);
|
||||
}
|
||||
};
|
||||
|
||||
@ -539,4 +541,202 @@ class FunctionReplicate : public IFunction
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
class FunctionBar : public IFunction
|
||||
{
|
||||
public:
|
||||
/// Получить имя функции.
|
||||
String getName() const
|
||||
{
|
||||
return "bar";
|
||||
}
|
||||
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() != 3 && arguments.size() != 4)
|
||||
throw Exception("Function " + getName() + " requires from 3 or 4 parameters: value, min_value, max_value, [max_width_of_bar = 80]. Passed "
|
||||
+ toString(arguments.size()) + ".",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!arguments[0]->isNumeric() || !arguments[1]->isNumeric() || !arguments[2]->isNumeric()
|
||||
|| (arguments.size() == 4 && !arguments[3]->isNumeric()))
|
||||
throw Exception("All arguments for function " + getName() + " must be numeric.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return new DataTypeString;
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
Int64 min = extractConstant<Int64>(block, arguments, 1, "Second"); /// Уровень значения, при котором полоска имеет нулевую длину.
|
||||
Int64 max = extractConstant<Int64>(block, arguments, 2, "Third"); /// Уровень значения, при котором полоска имеет максимальную длину.
|
||||
|
||||
/// Максимальная ширина полоски в символах, по-умолчанию.
|
||||
Float64 max_width = arguments.size() == 4
|
||||
? extractConstant<Float64>(block, arguments, 3, "Fourth")
|
||||
: 80;
|
||||
|
||||
if (max_width < 1)
|
||||
throw Exception("Max_width argument must be >= 1.", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
if (max_width > 1000)
|
||||
throw Exception("Too large max_width.", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
const auto & src = *block.getByPosition(arguments[0]).column;
|
||||
|
||||
if (src.isConst())
|
||||
{
|
||||
auto res_column = new ColumnConstString(block.rowsInFirstColumn(), "");
|
||||
block.getByPosition(result).column = res_column;
|
||||
|
||||
if ( executeConstNumber<UInt8> (src, *res_column, min, max, max_width)
|
||||
|| executeConstNumber<UInt16> (src, *res_column, min, max, max_width)
|
||||
|| executeConstNumber<UInt32> (src, *res_column, min, max, max_width)
|
||||
|| executeConstNumber<UInt64> (src, *res_column, min, max, max_width)
|
||||
|| executeConstNumber<Int8> (src, *res_column, min, max, max_width)
|
||||
|| executeConstNumber<Int16> (src, *res_column, min, max, max_width)
|
||||
|| executeConstNumber<Int32> (src, *res_column, min, max, max_width)
|
||||
|| executeConstNumber<Int64> (src, *res_column, min, max, max_width)
|
||||
|| executeConstNumber<Float32> (src, *res_column, min, max, max_width)
|
||||
|| executeConstNumber<Float64> (src, *res_column, min, max, max_width))
|
||||
{
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto res_column = new ColumnString;
|
||||
block.getByPosition(result).column = res_column;
|
||||
|
||||
if ( executeNumber<UInt8> (src, *res_column, min, max, max_width)
|
||||
|| executeNumber<UInt16> (src, *res_column, min, max, max_width)
|
||||
|| executeNumber<UInt32> (src, *res_column, min, max, max_width)
|
||||
|| executeNumber<UInt64> (src, *res_column, min, max, max_width)
|
||||
|| executeNumber<Int8> (src, *res_column, min, max, max_width)
|
||||
|| executeNumber<Int16> (src, *res_column, min, max, max_width)
|
||||
|| executeNumber<Int32> (src, *res_column, min, max, max_width)
|
||||
|| executeNumber<Int64> (src, *res_column, min, max, max_width)
|
||||
|| executeNumber<Float32> (src, *res_column, min, max, max_width)
|
||||
|| executeNumber<Float64> (src, *res_column, min, max, max_width))
|
||||
{
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename T>
|
||||
T extractConstant(Block & block, const ColumnNumbers & arguments, size_t argument_pos, const char * which_argument) const
|
||||
{
|
||||
const auto & column = *block.getByPosition(arguments[argument_pos]).column;
|
||||
|
||||
if (!column.isConst())
|
||||
throw Exception(which_argument + String(" argument for function ") + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
return apply_visitor(FieldVisitorConvertToNumber<T>(), column[0]);
|
||||
}
|
||||
|
||||
static constexpr size_t BAR_CHAR_SIZE = strlen("█");
|
||||
|
||||
template <typename T>
|
||||
static Float64 barWidth(T x, Int64 min, Int64 max, Float64 max_width)
|
||||
{
|
||||
if (x <= min)
|
||||
return 0;
|
||||
|
||||
if (x >= max)
|
||||
return max_width;
|
||||
|
||||
return (x - min) * max_width / (max - min);
|
||||
}
|
||||
|
||||
static size_t barWidthInBytes(Float64 width)
|
||||
{
|
||||
return ceil(width - 1.0 / 8) * BAR_CHAR_SIZE;
|
||||
}
|
||||
|
||||
/// В dst должно быть место для barWidthInBytes(width) символов и завершающего нуля.
|
||||
static void renderBar(Float64 width, char * dst)
|
||||
{
|
||||
size_t floor_width = floor(width);
|
||||
|
||||
for (size_t i = 0; i < floor_width; ++i)
|
||||
{
|
||||
memcpy(dst, "█", BAR_CHAR_SIZE);
|
||||
dst += BAR_CHAR_SIZE;
|
||||
}
|
||||
|
||||
size_t remainder = floor((width - floor_width) * 8);
|
||||
|
||||
if (remainder)
|
||||
{
|
||||
memcpy(dst, &"▏▎▍▌▋▋▊▉"[(remainder - 1) * BAR_CHAR_SIZE], BAR_CHAR_SIZE);
|
||||
dst += BAR_CHAR_SIZE;
|
||||
}
|
||||
|
||||
*dst = 0;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static void fill(const PODArray<T> & src, ColumnString::Chars_t & dst_chars, ColumnString::Offsets_t & dst_offsets,
|
||||
Int64 min, Int64 max, Float64 max_width)
|
||||
{
|
||||
size_t size = src.size();
|
||||
size_t current_offset = 0;
|
||||
|
||||
dst_offsets.resize(size);
|
||||
dst_chars.reserve(size * (barWidthInBytes(max_width) + 1)); /// строки 0-terminated.
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
Float64 width = barWidth(src[i], min, max, max_width);
|
||||
size_t next_size = current_offset + barWidthInBytes(width) + 1;
|
||||
dst_chars.resize(next_size);
|
||||
renderBar(width, reinterpret_cast<char *>(&dst_chars[current_offset]));
|
||||
current_offset = next_size;
|
||||
dst_offsets[i] = current_offset;
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static void fill(T src, String & dst_chars,
|
||||
Int64 min, Int64 max, Float64 max_width)
|
||||
{
|
||||
Float64 width = barWidth(src, min, max, max_width);
|
||||
dst_chars.resize(barWidthInBytes(width));
|
||||
renderBar(width, &dst_chars[0]);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static bool executeNumber(const IColumn & src, ColumnString & dst, Int64 min, Int64 max, Float64 max_width)
|
||||
{
|
||||
if (const ColumnVector<T> * col = typeid_cast<const ColumnVector<T> *>(&src))
|
||||
{
|
||||
fill(col->getData(), dst.getChars(), dst.getOffsets(), min, max, max_width);
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static bool executeConstNumber(const IColumn & src, ColumnConstString & dst, Int64 min, Int64 max, Float64 max_width)
|
||||
{
|
||||
if (const ColumnConst<T> * col = typeid_cast<const ColumnConst<T> *>(&src))
|
||||
{
|
||||
fill(col->getData(), dst.getData(), min, max, max_width);
|
||||
return true;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -4,6 +4,7 @@
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/Functions/IFunction.h>
|
||||
#include <DB/Common/HashTable/Hash.h>
|
||||
#include <stats/IntHash.h>
|
||||
|
||||
|
||||
@ -13,14 +14,14 @@ namespace DB
|
||||
/** Функции генерации псевдослучайных чисел.
|
||||
* Функция может быть вызвана без аргументов или с одним аргументом.
|
||||
* Аргумент игнорируется и служит лишь для того, чтобы несколько вызовов одной функции считались разными и не склеивались.
|
||||
*
|
||||
*
|
||||
* Пример:
|
||||
* SELECT rand(), rand() - выдаст два одинаковых столбца.
|
||||
* SELECT rand(1), rand(2) - выдаст два разных столбца.
|
||||
*
|
||||
* Некриптографические генераторы:
|
||||
*
|
||||
* rand - linear congruental generator 0 .. 2^31 - 1.
|
||||
*
|
||||
* rand - linear congruental generator 0 .. 2^32 - 1.
|
||||
* rand64 - комбинирует несколько значений rand, чтобы получить значения из диапазона 0 .. 2^64 - 1.
|
||||
*
|
||||
* В качестве затравки используют время.
|
||||
@ -30,31 +31,74 @@ namespace DB
|
||||
|
||||
namespace detail
|
||||
{
|
||||
void seed(drand48_data & rand_state, intptr_t additional_seed)
|
||||
struct LinearCongruentialGenerator
|
||||
{
|
||||
/// Константы из man lrand48_r.
|
||||
static constexpr UInt64 a = 0x5DEECE66D;
|
||||
static constexpr UInt64 c = 0xB;
|
||||
|
||||
/// А эта - из head -c8 /dev/urandom | xxd -p
|
||||
UInt64 current = 0x09826f4a081cee35ULL;
|
||||
|
||||
LinearCongruentialGenerator() {}
|
||||
LinearCongruentialGenerator(UInt64 value) : current(value) {}
|
||||
|
||||
void seed(UInt64 value)
|
||||
{
|
||||
current = value;
|
||||
}
|
||||
|
||||
UInt32 next()
|
||||
{
|
||||
current = current * a + c;
|
||||
return current >> 16;
|
||||
}
|
||||
};
|
||||
|
||||
void seed(LinearCongruentialGenerator & generator, intptr_t additional_seed)
|
||||
{
|
||||
struct timespec times;
|
||||
if (clock_gettime(CLOCK_THREAD_CPUTIME_ID, ×))
|
||||
throwFromErrno("Cannot clock_gettime.", ErrorCodes::CANNOT_CLOCK_GETTIME);
|
||||
|
||||
srand48_r(intHash32<0>(times.tv_nsec ^ intHash32<0>(additional_seed)), &rand_state);
|
||||
generator.seed(intHash64(times.tv_nsec ^ intHash64(additional_seed)));
|
||||
}
|
||||
}
|
||||
|
||||
struct RandImpl
|
||||
{
|
||||
typedef UInt32 ReturnType;
|
||||
|
||||
|
||||
static void execute(PODArray<ReturnType> & res)
|
||||
{
|
||||
drand48_data rand_state;
|
||||
detail::seed(rand_state, reinterpret_cast<intptr_t>(&res[0]));
|
||||
|
||||
detail::LinearCongruentialGenerator generator0;
|
||||
detail::LinearCongruentialGenerator generator1;
|
||||
detail::LinearCongruentialGenerator generator2;
|
||||
detail::LinearCongruentialGenerator generator3;
|
||||
|
||||
detail::seed(generator0, 0xfb4121280b2ab902ULL + reinterpret_cast<intptr_t>(&res[0]));
|
||||
detail::seed(generator1, 0x0121cf76df39c673ULL + reinterpret_cast<intptr_t>(&res[0]));
|
||||
detail::seed(generator2, 0x17ae86e3a19a602fULL + reinterpret_cast<intptr_t>(&res[0]));
|
||||
detail::seed(generator3, 0x8b6e16da7e06d622ULL + reinterpret_cast<intptr_t>(&res[0]));
|
||||
|
||||
size_t size = res.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
ReturnType * pos = &res[0];
|
||||
ReturnType * end = pos + size;
|
||||
ReturnType * end4 = pos + size / 4 * 4;
|
||||
|
||||
while (pos < end4)
|
||||
{
|
||||
long rand_res;
|
||||
lrand48_r(&rand_state, &rand_res);
|
||||
res[i] = rand_res;
|
||||
pos[0] = generator0.next();
|
||||
pos[1] = generator1.next();
|
||||
pos[2] = generator2.next();
|
||||
pos[3] = generator3.next();
|
||||
pos += 4;
|
||||
}
|
||||
|
||||
while (pos < end)
|
||||
{
|
||||
pos[0] = generator0.next();
|
||||
++pos;
|
||||
}
|
||||
}
|
||||
};
|
||||
@ -65,21 +109,32 @@ struct Rand64Impl
|
||||
|
||||
static void execute(PODArray<ReturnType> & res)
|
||||
{
|
||||
drand48_data rand_state;
|
||||
detail::seed(rand_state, reinterpret_cast<intptr_t>(&res[0]));
|
||||
detail::LinearCongruentialGenerator generator0;
|
||||
detail::LinearCongruentialGenerator generator1;
|
||||
detail::LinearCongruentialGenerator generator2;
|
||||
detail::LinearCongruentialGenerator generator3;
|
||||
|
||||
detail::seed(generator0, 0xfb4121280b2ab902ULL + reinterpret_cast<intptr_t>(&res[0]));
|
||||
detail::seed(generator1, 0x0121cf76df39c673ULL + reinterpret_cast<intptr_t>(&res[0]));
|
||||
detail::seed(generator2, 0x17ae86e3a19a602fULL + reinterpret_cast<intptr_t>(&res[0]));
|
||||
detail::seed(generator3, 0x8b6e16da7e06d622ULL + reinterpret_cast<intptr_t>(&res[0]));
|
||||
|
||||
size_t size = res.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
ReturnType * pos = &res[0];
|
||||
ReturnType * end = pos + size;
|
||||
ReturnType * end2 = pos + size / 2 * 2;
|
||||
|
||||
while (pos < end2)
|
||||
{
|
||||
long rand_res1;
|
||||
long rand_res2;
|
||||
long rand_res3;
|
||||
|
||||
lrand48_r(&rand_state, &rand_res1);
|
||||
lrand48_r(&rand_state, &rand_res2);
|
||||
lrand48_r(&rand_state, &rand_res3);
|
||||
|
||||
res[i] = rand_res1 ^ (rand_res2 << 18) ^ (rand_res3 << 33);
|
||||
pos[0] = (static_cast<UInt64>(generator0.next()) << 32) | generator1.next();
|
||||
pos[1] = (static_cast<UInt64>(generator2.next()) << 32) | generator3.next();
|
||||
pos += 2;
|
||||
}
|
||||
|
||||
while (pos < end)
|
||||
{
|
||||
pos[0] = (static_cast<UInt64>(generator0.next()) << 32) | generator1.next();
|
||||
++pos;
|
||||
}
|
||||
}
|
||||
};
|
||||
@ -90,7 +145,7 @@ class FunctionRandom : public IFunction
|
||||
{
|
||||
private:
|
||||
typedef typename Impl::ReturnType ToType;
|
||||
|
||||
|
||||
public:
|
||||
/// Получить имя функции.
|
||||
String getName() const
|
||||
|
@ -1,8 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/DataTypes/DataTypeArray.h>
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
#include <DB/Columns/ColumnFixedString.h>
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
#include <DB/Columns/ColumnArray.h>
|
||||
#include <DB/Functions/IFunction.h>
|
||||
|
@ -7,7 +7,9 @@
|
||||
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/DataTypes/DataTypeFixedString.h>
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
#include <DB/Columns/ColumnFixedString.h>
|
||||
#include <DB/Columns/ColumnConst.h>
|
||||
#include <DB/Common/Volnitsky.h>
|
||||
#include <DB/Functions/IFunction.h>
|
||||
|
@ -149,7 +149,7 @@ inline void writeJSONString(const char * begin, const char * end, WriteBuffer &
|
||||
case '\t':
|
||||
writeChar('\\', buf);
|
||||
writeChar('t', buf);
|
||||
break;
|
||||
break;
|
||||
case '\\':
|
||||
writeChar('\\', buf);
|
||||
writeChar('\\', buf);
|
||||
@ -167,10 +167,10 @@ inline void writeJSONString(const char * begin, const char * end, WriteBuffer &
|
||||
{
|
||||
char higher_half = (*it) >> 4;
|
||||
char lower_half = (*it) & 0xF;
|
||||
|
||||
|
||||
writeCString("\\u00", buf);
|
||||
writeChar('0' + higher_half, buf);
|
||||
|
||||
|
||||
if (0 <= lower_half && lower_half <= 9)
|
||||
writeChar('0' + lower_half, buf);
|
||||
else
|
||||
@ -282,7 +282,6 @@ inline void writeQuotedString(const String & s, WriteBuffer & buf)
|
||||
writeAnyQuotedString<'\''>(s, buf);
|
||||
}
|
||||
|
||||
/// Совместимо с JSON.
|
||||
inline void writeDoubleQuotedString(const String & s, WriteBuffer & buf)
|
||||
{
|
||||
writeAnyQuotedString<'"'>(s, buf);
|
||||
@ -335,7 +334,7 @@ inline void writeDateText(DayNum_t date, WriteBuffer & buf)
|
||||
s[6] += values.month % 10;
|
||||
s[8] += values.day_of_month / 10;
|
||||
s[9] += values.day_of_month % 10;
|
||||
|
||||
|
||||
buf.write(s, 10);
|
||||
}
|
||||
|
||||
|
@ -91,7 +91,7 @@ struct AggregationMethodKey64
|
||||
|
||||
/** Разместить дополнительные данные, если это необходимо, в случае, когда в хэш-таблицу был вставлен новый ключ.
|
||||
*/
|
||||
void onNewKey(iterator & it, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
|
||||
static void onNewKey(iterator & it, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
|
||||
{
|
||||
}
|
||||
|
||||
@ -139,7 +139,7 @@ struct AggregationMethodString
|
||||
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
|
||||
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
|
||||
|
||||
void onNewKey(iterator & it, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
|
||||
static void onNewKey(iterator & it, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
|
||||
{
|
||||
it->first.data = pool.insert(it->first.data, it->first.size);
|
||||
}
|
||||
@ -186,7 +186,7 @@ struct AggregationMethodFixedString
|
||||
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
|
||||
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
|
||||
|
||||
void onNewKey(iterator & it, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
|
||||
static void onNewKey(iterator & it, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
|
||||
{
|
||||
it->first.data = pool.insert(it->first.data, it->first.size);
|
||||
}
|
||||
@ -226,7 +226,7 @@ struct AggregationMethodKeys128
|
||||
static AggregateDataPtr & getAggregateData(Mapped & value) { return value; }
|
||||
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; }
|
||||
|
||||
void onNewKey(iterator & it, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
|
||||
static void onNewKey(iterator & it, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
|
||||
{
|
||||
}
|
||||
|
||||
@ -271,7 +271,7 @@ struct AggregationMethodHashed
|
||||
static AggregateDataPtr & getAggregateData(Mapped & value) { return value.second; }
|
||||
static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value.second; }
|
||||
|
||||
void onNewKey(iterator & it, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
|
||||
static void onNewKey(iterator & it, size_t keys_size, size_t i, StringRefs & keys, Arena & pool)
|
||||
{
|
||||
it->second.first = placeKeysInPool(i, keys_size, keys, pool);
|
||||
}
|
||||
@ -308,7 +308,7 @@ struct AggregatedDataVariants : private boost::noncopyable
|
||||
|
||||
size_t keys_size; /// Количество ключей NOTE нужно ли это поле?
|
||||
Sizes key_sizes; /// Размеры ключей, если ключи фиксированной длины
|
||||
|
||||
|
||||
/// Пулы для состояний агрегатных функций. Владение потом будет передано в ColumnAggregateFunction.
|
||||
Arenas aggregates_pools;
|
||||
Arena * aggregates_pool; /// Пул, который сейчас используется для аллокации.
|
||||
@ -322,7 +322,7 @@ struct AggregatedDataVariants : private boost::noncopyable
|
||||
std::unique_ptr<AggregationMethodFixedString> key_fixed_string;
|
||||
std::unique_ptr<AggregationMethodKeys128> keys128;
|
||||
std::unique_ptr<AggregationMethodHashed> hashed;
|
||||
|
||||
|
||||
enum Type
|
||||
{
|
||||
EMPTY = 0,
|
||||
@ -465,7 +465,7 @@ public:
|
||||
|
||||
protected:
|
||||
friend struct AggregatedDataVariants;
|
||||
|
||||
|
||||
ColumnNumbers keys;
|
||||
Names key_names;
|
||||
AggregateDescriptions aggregates;
|
||||
|
@ -29,10 +29,19 @@ public:
|
||||
/// Соединения с удалёнными серверами.
|
||||
ConnectionPools pools;
|
||||
|
||||
struct ShardInfo
|
||||
{
|
||||
/// contains names of directories for asynchronous write to StorageDistributed
|
||||
std::vector<std::string> dir_names;
|
||||
int weight;
|
||||
size_t num_local_nodes;
|
||||
};
|
||||
std::vector<ShardInfo> shard_info_vec;
|
||||
std::vector<size_t> slot_to_shard;
|
||||
|
||||
/// используеться для выставления ограничения на размер таймаута
|
||||
static Poco::Timespan saturate(const Poco::Timespan & v, const Poco::Timespan & limit);
|
||||
|
||||
private:
|
||||
struct Address
|
||||
{
|
||||
/** В конфиге адреса либо находятся в узлах <node>:
|
||||
@ -59,6 +68,7 @@ private:
|
||||
Address(const String & host_port_, const String & user_, const String & password_);
|
||||
};
|
||||
|
||||
private:
|
||||
static bool isLocal(const Address & address);
|
||||
|
||||
/// Массив шардов. Каждый шард - адреса одного сервера.
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Yandex/logger_useful.h>
|
||||
|
||||
#include <DB/Core/NamesAndTypes.h>
|
||||
#include <DB/Common/Macros.h>
|
||||
#include <DB/IO/UncompressedCache.h>
|
||||
#include <DB/Storages/MarkCache.h>
|
||||
#include <DB/DataStreams/FormatFactory.h>
|
||||
@ -56,8 +57,8 @@ typedef std::vector<DatabaseAndTableName> Dependencies;
|
||||
*/
|
||||
struct ContextShared
|
||||
{
|
||||
Logger * log; /// Логгер.
|
||||
|
||||
Logger * log = &Logger::get("Context"); /// Логгер.
|
||||
|
||||
struct AfterDestroy
|
||||
{
|
||||
Logger * log;
|
||||
@ -69,7 +70,7 @@ struct ContextShared
|
||||
LOG_INFO(log, "Uninitialized shared context.");
|
||||
#endif
|
||||
}
|
||||
} after_destroy;
|
||||
} after_destroy {log};
|
||||
|
||||
mutable Poco::Mutex mutex; /// Для доступа и модификации разделяемых объектов.
|
||||
|
||||
@ -95,8 +96,8 @@ struct ContextShared
|
||||
ViewDependencies view_dependencies; /// Текущие зависимости
|
||||
ConfigurationPtr users_config; /// Конфиг с секциями users, profiles и quotas.
|
||||
InterserverIOHandler interserver_io_handler; /// Обработчик для межсерверной передачи данных.
|
||||
String default_replica_name; /// Имя реплики из конфига.
|
||||
BackgroundProcessingPoolPtr background_pool; /// Пул потоков для фоновой работы, выполняемой таблицами.
|
||||
Macros macros; /// Подстановки из конфига.
|
||||
|
||||
/// Кластеры для distributed таблиц
|
||||
/// Создаются при создании Distributed таблиц, так как нужно дождаться пока будут выставлены Settings
|
||||
@ -105,8 +106,6 @@ struct ContextShared
|
||||
bool shutdown_called = false;
|
||||
|
||||
|
||||
ContextShared() : log(&Logger::get("Context")), after_destroy(log) {};
|
||||
|
||||
~ContextShared()
|
||||
{
|
||||
#ifndef DBMS_CLIENT
|
||||
@ -227,7 +226,7 @@ public:
|
||||
|
||||
/// Возвращает отцепленную таблицу.
|
||||
StoragePtr detachTable(const String & database_name, const String & table_name);
|
||||
|
||||
|
||||
void detachDatabase(const String & database_name);
|
||||
|
||||
String getCurrentDatabase() const;
|
||||
@ -238,9 +237,8 @@ public:
|
||||
String getDefaultFormat() const; /// Если default_format не задан - возвращается некоторый глобальный формат по-умолчанию.
|
||||
void setDefaultFormat(const String & name);
|
||||
|
||||
/// Имя этой реплики из конфига.
|
||||
String getDefaultReplicaName() const;
|
||||
void setDefaultReplicaName(const String & name);
|
||||
const Macros & getMacros() const;
|
||||
void setMacros(Macros && macros);
|
||||
|
||||
Settings getSettings() const;
|
||||
void setSettings(const Settings & settings_);
|
||||
|
@ -22,11 +22,41 @@ public:
|
||||
/** Изменяет список столбцов в метаданных таблицы на диске. Нужно вызывать под TableStructureLock соответствующей таблицы.
|
||||
*/
|
||||
static void updateMetadata(const String & database, const String & table, const NamesAndTypesList & columns, Context & context);
|
||||
|
||||
static AlterCommands parseAlter(const ASTAlterQuery::ParameterContainer & params, const DataTypeFactory & data_type_factory);
|
||||
private:
|
||||
struct PartitionCommand
|
||||
{
|
||||
enum Type
|
||||
{
|
||||
DROP_PARTITION,
|
||||
ATTACH_PARTITION,
|
||||
};
|
||||
|
||||
Type type;
|
||||
|
||||
Field partition;
|
||||
bool detach; /// true для DETACH PARTITION.
|
||||
|
||||
bool unreplicated;
|
||||
bool part;
|
||||
|
||||
static PartitionCommand dropPartition(const Field & partition, bool detach)
|
||||
{
|
||||
return {DROP_PARTITION, partition, detach};
|
||||
}
|
||||
|
||||
static PartitionCommand attachPartition(const Field & partition, bool unreplicated, bool part)
|
||||
{
|
||||
return {ATTACH_PARTITION, partition, false, unreplicated, part};
|
||||
}
|
||||
};
|
||||
|
||||
typedef std::vector<PartitionCommand> PartitionCommands;
|
||||
|
||||
ASTPtr query_ptr;
|
||||
|
||||
Context context;
|
||||
|
||||
static void parseAlter(const ASTAlterQuery::ParameterContainer & params, const DataTypeFactory & data_type_factory,
|
||||
AlterCommands & out_alter_commands, PartitionCommands & out_partition_commands);
|
||||
};
|
||||
}
|
||||
|
22
dbms/include/DB/Interpreters/InterpreterCheckQuery.h
Normal file
22
dbms/include/DB/Interpreters/InterpreterCheckQuery.h
Normal file
@ -0,0 +1,22 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Interpreters/Context.h>
|
||||
#include <DB/Parsers/ASTIdentifier.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class InterpreterCheckQuery
|
||||
{
|
||||
public:
|
||||
InterpreterCheckQuery(ASTPtr query_ptr_, Context & context_);
|
||||
BlockInputStreamPtr execute();
|
||||
DB::Block getSampleBlock();
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
Context context;
|
||||
DB::Block result;
|
||||
};
|
||||
|
||||
}
|
@ -1,7 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <statdaemons/Stopwatch.h>
|
||||
|
||||
#include <Yandex/logger_useful.h>
|
||||
|
||||
#include <DB/Parsers/ASTJoin.h>
|
||||
|
@ -2,8 +2,6 @@
|
||||
|
||||
#include <set>
|
||||
|
||||
#include <statdaemons/Stopwatch.h>
|
||||
|
||||
#include <Yandex/logger_useful.h>
|
||||
|
||||
#include <DB/Core/ColumnNumbers.h>
|
||||
|
@ -70,6 +70,9 @@ struct Settings
|
||||
* TODO: Сейчас применяется только при запуске сервера. Можно сделать изменяемым динамически. */ \
|
||||
M(SettingUInt64, background_pool_size, DBMS_DEFAULT_BACKGROUND_POOL_SIZE) \
|
||||
\
|
||||
/** Sleep time for StorageDistributed DirectoryMonitors in case there is no work or exception has been thrown */ \
|
||||
M(SettingMilliseconds, distributed_directory_monitor_sleep_time_ms, DBMS_DISTRIBUTED_DIRECTORY_MONITOR_SLEEP_TIME_MS) \
|
||||
\
|
||||
M(SettingLoadBalancing, load_balancing, LoadBalancing::RANDOM) \
|
||||
\
|
||||
M(SettingTotalsMode, totals_mode, TotalsMode::BEFORE_HAVING) \
|
||||
|
@ -9,7 +9,8 @@ namespace DB
|
||||
* ALTER TABLE [db.]name_type
|
||||
* ADD COLUMN col_name type [AFTER col_after],
|
||||
* DROP COLUMN col_drop,
|
||||
* MODIFY COLUMN col_name type
|
||||
* MODIFY COLUMN col_name type,
|
||||
* DROP PARTITION partition
|
||||
* ...
|
||||
*/
|
||||
|
||||
@ -18,16 +19,18 @@ class ASTAlterQuery : public IAST
|
||||
public:
|
||||
enum ParameterType
|
||||
{
|
||||
ADD,
|
||||
DROP,
|
||||
MODIFY,
|
||||
ADD_COLUMN,
|
||||
DROP_COLUMN,
|
||||
MODIFY_COLUMN,
|
||||
DROP_PARTITION,
|
||||
ATTACH_PARTITION,
|
||||
NO_TYPE
|
||||
};
|
||||
|
||||
struct Parameters
|
||||
{
|
||||
Parameters() : type(NO_TYPE) {}
|
||||
int type;
|
||||
int type = NO_TYPE;
|
||||
|
||||
/** В запросе ADD COLUMN здесь хранится имя и тип добавляемого столбца
|
||||
* В запросе DROP это поле не используется
|
||||
@ -40,12 +43,21 @@ public:
|
||||
*/
|
||||
ASTPtr column;
|
||||
|
||||
/** В запросе DROP PARTITION здесь хранится имя partition'а.
|
||||
*/
|
||||
ASTPtr partition;
|
||||
bool detach = false; /// true для DETACH PARTITION.
|
||||
|
||||
bool part = false; /// true для ATTACH [UNREPLICATED] PART
|
||||
bool unreplicated = false; /// true для ATTACH UNREPLICATED ...
|
||||
|
||||
/// deep copy
|
||||
void clone(Parameters & p) const
|
||||
{
|
||||
p.type = type;
|
||||
p.column = column->clone();
|
||||
p = *this;
|
||||
p.name_type = name_type->clone();
|
||||
p.column = column->clone();
|
||||
p.partition = partition->clone();
|
||||
}
|
||||
};
|
||||
typedef std::vector<Parameters> ParameterContainer;
|
||||
@ -54,6 +66,18 @@ public:
|
||||
String table;
|
||||
|
||||
|
||||
void addParameters(const Parameters & params)
|
||||
{
|
||||
parameters.push_back(params);
|
||||
if (params.name_type)
|
||||
children.push_back(params.name_type);
|
||||
if (params.column)
|
||||
children.push_back(params.column);
|
||||
if (params.partition)
|
||||
children.push_back(params.partition);
|
||||
}
|
||||
|
||||
|
||||
ASTAlterQuery(StringRange range_ = StringRange()) : IAST(range_) {};
|
||||
|
||||
/** Получить текст, который идентифицирует этот элемент. */
|
||||
|
25
dbms/include/DB/Parsers/ASTCheckQuery.h
Normal file
25
dbms/include/DB/Parsers/ASTCheckQuery.h
Normal file
@ -0,0 +1,25 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Parsers/IAST.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct ASTCheckQuery : public IAST
|
||||
{
|
||||
ASTCheckQuery(StringRange range_ = StringRange()) : IAST(range_) {};
|
||||
ASTCheckQuery(const ASTCheckQuery & ast) = default;
|
||||
|
||||
/** Получить текст, который идентифицирует этот элемент. */
|
||||
String getID() const { return ("CheckQuery_" + database + "_" + table); };
|
||||
|
||||
ASTPtr clone() const
|
||||
{
|
||||
return new ASTCheckQuery(*this);
|
||||
}
|
||||
|
||||
std::string database;
|
||||
std::string table;
|
||||
};
|
||||
|
||||
}
|
@ -10,6 +10,7 @@ namespace DB
|
||||
* [ADD COLUMN col_name type [AFTER col_after],]
|
||||
* [DROP COLUMN col_drop, ...]
|
||||
* [MODIFY COLUMN col_modify type, ...]
|
||||
* [DROP PARTITION partition, ...]
|
||||
*/
|
||||
class ParserAlterQuery : public IParserBase
|
||||
{
|
||||
|
17
dbms/include/DB/Parsers/ParserCheckQuery.h
Normal file
17
dbms/include/DB/Parsers/ParserCheckQuery.h
Normal file
@ -0,0 +1,17 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Parsers/IParserBase.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/** Запрос вида
|
||||
* CHECK [TABLE] [database.]table
|
||||
*/
|
||||
class ParserCheckQuery : public IParserBase
|
||||
{
|
||||
protected:
|
||||
const char * getName() const { return "ALTER query"; }
|
||||
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
|
||||
};
|
||||
|
||||
}
|
@ -24,6 +24,7 @@
|
||||
#include <DB/Parsers/ASTAlterQuery.h>
|
||||
#include <DB/Parsers/ASTShowProcesslistQuery.h>
|
||||
#include <DB/Parsers/ASTJoin.h>
|
||||
#include <DB/Parsers/ASTCheckQuery.h>
|
||||
//#include <DB/Parsers/ASTMultiQuery.h>
|
||||
|
||||
|
||||
@ -57,6 +58,7 @@ void formatAST(const ASTOrderByElement & ast, std::ostream & s, size_t indent =
|
||||
void formatAST(const ASTSubquery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
|
||||
void formatAST(const ASTAlterQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
|
||||
void formatAST(const ASTJoin & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
|
||||
void formatAST(const ASTCheckQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
|
||||
//void formatAST(const ASTMultiQuery & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false, bool need_parens = false);
|
||||
|
||||
void formatAST(const ASTQueryWithTableAndOutput & ast, std::string name, std::ostream & s,
|
||||
|
@ -6,7 +6,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Операция из запроса ALTER. Добавление столбцов типа Nested не развернуто в добавление отдельных столбцов.
|
||||
/// Операция из запроса ALTER (кроме DROP PARTITION). Добавление столбцов типа Nested не развернуто в добавление отдельных столбцов.
|
||||
struct AlterCommand
|
||||
{
|
||||
enum Type
|
||||
|
207
dbms/include/DB/Storages/Distributed/DirectoryMonitor.h
Normal file
207
dbms/include/DB/Storages/Distributed/DirectoryMonitor.h
Normal file
@ -0,0 +1,207 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/DataStreams/RemoteBlockOutputStream.h>
|
||||
#include <DB/Common/escapeForFileName.h>
|
||||
#include <DB/Storages/StorageDistributed.h>
|
||||
#include <boost/algorithm/string/find_iterator.hpp>
|
||||
#include <boost/algorithm/string/finder.hpp>
|
||||
#include <thread>
|
||||
#include <mutex>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
template <typename PoolFactory>
|
||||
ConnectionPools createPoolsForAddresses(const std::string & name, PoolFactory && factory)
|
||||
{
|
||||
ConnectionPools pools;
|
||||
|
||||
for (auto it = boost::make_split_iterator(name, boost::first_finder(",")); it != decltype(it){}; ++it)
|
||||
{
|
||||
const auto address = boost::copy_range<std::string>(*it);
|
||||
|
||||
const auto user_pw_end = strchr(address.data(), '@');
|
||||
const auto colon = strchr(address.data(), ':');
|
||||
if (!user_pw_end || !colon)
|
||||
throw Exception{
|
||||
"Shard address '" + address + "' does not match to 'user[:password]@host:port' pattern",
|
||||
ErrorCodes::INCORRECT_FILE_NAME
|
||||
};
|
||||
|
||||
const auto has_pw = colon < user_pw_end;
|
||||
const auto host_end = has_pw ? strchr(user_pw_end + 1, ':') : colon;
|
||||
if (!host_end)
|
||||
throw Exception{
|
||||
"Shard address '" + address + "' does not contain port",
|
||||
ErrorCodes::INCORRECT_FILE_NAME
|
||||
};
|
||||
|
||||
const auto user = unescapeForFileName({address.data(), has_pw ? colon : user_pw_end});
|
||||
const auto password = has_pw ? unescapeForFileName({colon + 1, user_pw_end}) : std::string{};
|
||||
const auto host = unescapeForFileName({user_pw_end + 1, host_end});
|
||||
const auto port = parse<UInt16>(host_end + 1);
|
||||
|
||||
pools.emplace_back(factory(host, port, user, password));
|
||||
}
|
||||
|
||||
return pools;
|
||||
}
|
||||
}
|
||||
|
||||
/** Implementation for StorageDistributed::DirectoryMonitor nested class.
|
||||
* This type is not designed for standalone use. */
|
||||
class StorageDistributed::DirectoryMonitor
|
||||
{
|
||||
public:
|
||||
DirectoryMonitor(StorageDistributed & storage, const std::string & name)
|
||||
: storage(storage), pool{createPool(name)}, path{storage.path + name + '/'}
|
||||
, sleep_time{storage.context.getSettingsRef().distributed_directory_monitor_sleep_time_ms.totalMilliseconds()}
|
||||
, log{&Logger::get(getLoggerName())}
|
||||
{
|
||||
}
|
||||
|
||||
~DirectoryMonitor()
|
||||
{
|
||||
{
|
||||
quit = true;
|
||||
std::lock_guard<std::mutex> lock{mutex};
|
||||
}
|
||||
cond.notify_one();
|
||||
thread.join();
|
||||
}
|
||||
|
||||
private:
|
||||
void run()
|
||||
{
|
||||
std::unique_lock<std::mutex> lock{mutex};
|
||||
|
||||
const auto quit_requested = [this] { return quit; };
|
||||
|
||||
while (!quit_requested())
|
||||
{
|
||||
auto do_sleep = true;
|
||||
|
||||
try
|
||||
{
|
||||
do_sleep = !findFiles();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
do_sleep = true;
|
||||
tryLogCurrentException(getLoggerName().data());
|
||||
}
|
||||
|
||||
if (do_sleep)
|
||||
cond.wait_for(lock, sleep_time, quit_requested);
|
||||
}
|
||||
}
|
||||
|
||||
ConnectionPoolPtr createPool(const std::string & name)
|
||||
{
|
||||
const auto pool_factory = [this, &name] (const std::string & host, const UInt16 port,
|
||||
const std::string & user, const std::string & password) {
|
||||
return new ConnectionPool{
|
||||
1, host, port, "",
|
||||
user, password, storage.context.getDataTypeFactory(),
|
||||
storage.getName() + '_' + name};
|
||||
};
|
||||
|
||||
auto pools = createPoolsForAddresses(name, pool_factory);
|
||||
|
||||
return pools.size() == 1 ? pools.front() : new ConnectionPoolWithFailover(pools, LoadBalancing::RANDOM);
|
||||
}
|
||||
|
||||
bool findFiles()
|
||||
{
|
||||
std::map<UInt64, std::string> files;
|
||||
|
||||
Poco::DirectoryIterator end;
|
||||
for (Poco::DirectoryIterator it{path}; it != end; ++it)
|
||||
{
|
||||
const auto & file_path_str = it->path();
|
||||
Poco::Path file_path{file_path_str};
|
||||
|
||||
if (!it->isDirectory() && 0 == strncmp(file_path.getExtension().data(), "bin", strlen("bin")))
|
||||
files[parse<UInt64>(file_path.getBaseName())] = file_path_str;
|
||||
}
|
||||
|
||||
if (files.empty())
|
||||
return false;
|
||||
|
||||
for (const auto & file : files)
|
||||
{
|
||||
if (quit)
|
||||
return true;
|
||||
|
||||
processFile(file.second);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
void processFile(const std::string & file_path)
|
||||
{
|
||||
LOG_TRACE(log, "Started processing `" << file_path << '`');
|
||||
auto connection = pool->get();
|
||||
|
||||
try
|
||||
{
|
||||
ReadBufferFromFile in{file_path};
|
||||
|
||||
std::string insert_query;
|
||||
readStringBinary(insert_query, in);
|
||||
|
||||
RemoteBlockOutputStream remote{*connection, insert_query};
|
||||
|
||||
remote.writePrefix();
|
||||
remote.writePrepared(in);
|
||||
remote.writeSuffix();
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
const auto code = e.code();
|
||||
|
||||
/// mark file as broken if necessary
|
||||
if (code == ErrorCodes::CHECKSUM_DOESNT_MATCH ||
|
||||
code == ErrorCodes::TOO_LARGE_SIZE_COMPRESSED ||
|
||||
code == ErrorCodes::CANNOT_READ_ALL_DATA)
|
||||
{
|
||||
const auto last_path_separator_pos = file_path.rfind('/');
|
||||
const auto & path = file_path.substr(0, last_path_separator_pos + 1);
|
||||
const auto & file_name = file_path.substr(last_path_separator_pos + 1);
|
||||
const auto & broken_path = path + "broken/";
|
||||
const auto & broken_file_path = broken_path + file_name;
|
||||
|
||||
Poco::File{broken_path}.createDirectory();
|
||||
Poco::File{file_path}.renameTo(broken_file_path);
|
||||
|
||||
LOG_ERROR(log, "Renamed `" << file_path << "` to `" << broken_file_path << '`');
|
||||
}
|
||||
|
||||
throw;
|
||||
}
|
||||
|
||||
Poco::File{file_path}.remove();
|
||||
|
||||
LOG_TRACE(log, "Finished processing `" << file_path << '`');
|
||||
}
|
||||
|
||||
std::string getLoggerName() const
|
||||
{
|
||||
return storage.name + '.' + storage.getName() + ".DirectoryMonitor";
|
||||
}
|
||||
|
||||
StorageDistributed & storage;
|
||||
ConnectionPoolPtr pool;
|
||||
std::string path;
|
||||
std::chrono::milliseconds sleep_time;
|
||||
bool quit{false};
|
||||
std::mutex mutex;
|
||||
std::condition_variable cond;
|
||||
Logger * log;
|
||||
std::thread thread{&DirectoryMonitor::run, this};
|
||||
};
|
||||
|
||||
}
|
@ -0,0 +1,217 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Storages/StorageDistributed.h>
|
||||
#include <DB/Storages/Distributed/queryToString.h>
|
||||
|
||||
#include <DB/IO/WriteBufferFromFile.h>
|
||||
#include <DB/IO/CompressedWriteBuffer.h>
|
||||
#include <DB/DataStreams/NativeBlockOutputStream.h>
|
||||
#include <DB/Interpreters/InterpreterInsertQuery.h>
|
||||
|
||||
#include <statdaemons/Increment.h>
|
||||
#include <statdaemons/stdext.h>
|
||||
|
||||
#include <iostream>
|
||||
#include <type_traits>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Запись асинхронная - данные сначала записываются на локальную файловую систему, а потом отправляются на удалённые серверы.
|
||||
* Если Distributed таблица использует более одного шарда, то для того, чтобы поддерживалась запись,
|
||||
* при создании таблицы должен быть указан дополнительный параметр у ENGINE - ключ шардирования.
|
||||
* Ключ шардирования - произвольное выражение от столбцов. Например, rand() или UserID.
|
||||
* При записи блок данных разбивается по остатку от деления ключа шардирования на суммарный вес шардов,
|
||||
* и полученные блоки пишутся в сжатом Native формате в отдельные директории для отправки.
|
||||
* Для каждого адреса назначения (каждой директории с данными для отправки), в StorageDistributed создаётся отдельный поток,
|
||||
* который следит за директорией и отправляет данные. */
|
||||
class DistributedBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
DistributedBlockOutputStream(StorageDistributed & storage, const ASTPtr & query_ast)
|
||||
: storage(storage), query_ast(query_ast)
|
||||
{
|
||||
}
|
||||
|
||||
void write(const Block & block) override
|
||||
{
|
||||
if (storage.getShardingKeyExpr() && storage.cluster.shard_info_vec.size() > 1)
|
||||
return writeSplit(block);
|
||||
|
||||
writeImpl(block);
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename T>
|
||||
static std::vector<IColumn::Filter> createFiltersImpl(const size_t num_rows, const IColumn * column, const Cluster & cluster)
|
||||
{
|
||||
const auto total_weight = cluster.slot_to_shard.size();
|
||||
const auto num_shards = cluster.shard_info_vec.size();
|
||||
std::vector<IColumn::Filter> filters(num_shards);
|
||||
|
||||
/** Деление отрицательного числа с остатком на положительное, в C++ даёт отрицательный остаток.
|
||||
* Для данной задачи это не подходит. Поэтому, будем обрабатывать знаковые типы как беззнаковые.
|
||||
* Это даёт уже что-то совсем не похожее на деление с остатком, но подходящее для данной задачи.
|
||||
*/
|
||||
using UnsignedT = typename std::make_unsigned<T>::type;
|
||||
|
||||
/// const columns contain only one value, therefore we do not need to read it at every iteration
|
||||
if (column->isConst())
|
||||
{
|
||||
const auto data = typeid_cast<const ColumnConst<T> *>(column)->getData();
|
||||
const auto shard_num = cluster.slot_to_shard[static_cast<UnsignedT>(data) % total_weight];
|
||||
|
||||
for (size_t i = 0; i < num_shards; ++i)
|
||||
filters[i].assign(num_rows, static_cast<UInt8>(shard_num == i));
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto & data = typeid_cast<const ColumnVector<T> *>(column)->getData();
|
||||
|
||||
for (size_t i = 0; i < num_shards; ++i)
|
||||
{
|
||||
filters[i].resize(num_rows);
|
||||
for (size_t j = 0; j < num_rows; ++j)
|
||||
filters[i][j] = cluster.slot_to_shard[static_cast<UnsignedT>(data[j]) % total_weight] == i;
|
||||
}
|
||||
}
|
||||
|
||||
return filters;
|
||||
}
|
||||
|
||||
std::vector<IColumn::Filter> createFilters(Block block)
|
||||
{
|
||||
using create_filters_sig = std::vector<IColumn::Filter>(size_t, const IColumn *, const Cluster &);
|
||||
/// hashmap of pointers to functions corresponding to each integral type
|
||||
static std::unordered_map<std::string, create_filters_sig *> creators{
|
||||
{ TypeName<UInt8>::get(), &createFiltersImpl<UInt8> },
|
||||
{ TypeName<UInt16>::get(), &createFiltersImpl<UInt16> },
|
||||
{ TypeName<UInt32>::get(), &createFiltersImpl<UInt32> },
|
||||
{ TypeName<UInt64>::get(), &createFiltersImpl<UInt64> },
|
||||
{ TypeName<Int8>::get(), &createFiltersImpl<Int8> },
|
||||
{ TypeName<Int16>::get(), &createFiltersImpl<Int16> },
|
||||
{ TypeName<Int32>::get(), &createFiltersImpl<Int32> },
|
||||
{ TypeName<Int64>::get(), &createFiltersImpl<Int64> },
|
||||
};
|
||||
|
||||
storage.getShardingKeyExpr()->execute(block);
|
||||
|
||||
const auto & key_column = block.getByName(storage.getShardingKeyColumnName());
|
||||
|
||||
/// check that key column has valid type
|
||||
const auto it = creators.find(key_column.type->getName());
|
||||
|
||||
return it != std::end(creators)
|
||||
? (*it->second)(block.rowsInFirstColumn(), key_column.column.get(), storage.cluster)
|
||||
: throw Exception{
|
||||
"Sharding key expression does not evaluate to an integer type",
|
||||
ErrorCodes::TYPE_MISMATCH
|
||||
};
|
||||
}
|
||||
|
||||
void writeSplit(const Block & block)
|
||||
{
|
||||
const auto num_cols = block.columns();
|
||||
/// cache column pointers for later reuse
|
||||
std::vector<const IColumn*> columns(num_cols);
|
||||
for (size_t i = 0; i < columns.size(); ++i)
|
||||
columns[i] = block.getByPosition(i).column;
|
||||
|
||||
auto filters = createFilters(block);
|
||||
|
||||
const auto num_shards = storage.cluster.shard_info_vec.size();
|
||||
for (size_t i = 0; i < num_shards; ++i)
|
||||
{
|
||||
auto target_block = block.cloneEmpty();
|
||||
|
||||
for (size_t col = 0; col < num_cols; ++col)
|
||||
target_block.getByPosition(col).column = columns[col]->filter(filters[i]);
|
||||
|
||||
if (target_block.rowsInFirstColumn())
|
||||
writeImpl(target_block, i);
|
||||
}
|
||||
}
|
||||
|
||||
void writeImpl(const Block & block, const size_t shard_id = 0)
|
||||
{
|
||||
const auto & shard_info = storage.cluster.shard_info_vec[shard_id];
|
||||
if (shard_info.num_local_nodes)
|
||||
writeToLocal(block, shard_info.num_local_nodes);
|
||||
|
||||
/// dir_names is empty if shard has only local addresses
|
||||
if (!shard_info.dir_names.empty())
|
||||
writeToShard(block, shard_info.dir_names);
|
||||
}
|
||||
|
||||
void writeToLocal(const Block & block, const size_t repeats)
|
||||
{
|
||||
InterpreterInsertQuery interp{query_ast, storage.context};
|
||||
|
||||
auto block_io = interp.execute();
|
||||
block_io.out->writePrefix();
|
||||
|
||||
for (size_t i = 0; i < repeats; ++i)
|
||||
block_io.out->write(block);
|
||||
|
||||
block_io.out->writeSuffix();
|
||||
}
|
||||
|
||||
void writeToShard(const Block & block, const std::vector<std::string> & dir_names)
|
||||
{
|
||||
/** tmp directory is used to ensure atomicity of transactions
|
||||
* and keep monitor thread out from reading incomplete data
|
||||
*/
|
||||
std::string first_file_tmp_path{};
|
||||
|
||||
auto first = true;
|
||||
const auto & query_string = queryToString(query_ast);
|
||||
|
||||
/// write first file, hardlink the others
|
||||
for (const auto & dir_name : dir_names)
|
||||
{
|
||||
const auto & path = storage.getPath() + dir_name + '/';
|
||||
|
||||
/// ensure shard subdirectory creation and notify storage
|
||||
if (Poco::File(path).createDirectory())
|
||||
storage.requireDirectoryMonitor(dir_name);
|
||||
|
||||
const auto & file_name = toString(Increment{path + "increment.txt"}.get(true)) + ".bin";
|
||||
const auto & block_file_path = path + file_name;
|
||||
|
||||
/** on first iteration write block to a temporary directory for subsequent hardlinking to ensure
|
||||
* the inode is not freed until we're done */
|
||||
if (first)
|
||||
{
|
||||
first = false;
|
||||
|
||||
const auto & tmp_path = path + "tmp/";
|
||||
Poco::File(tmp_path).createDirectory();
|
||||
const auto & block_file_tmp_path = tmp_path + file_name;
|
||||
|
||||
first_file_tmp_path = block_file_tmp_path;
|
||||
|
||||
WriteBufferFromFile out{block_file_tmp_path};
|
||||
CompressedWriteBuffer compress{out};
|
||||
NativeBlockOutputStream stream{compress};
|
||||
|
||||
writeStringBinary(query_string, out);
|
||||
|
||||
stream.writePrefix();
|
||||
stream.write(block);
|
||||
stream.writeSuffix();
|
||||
}
|
||||
|
||||
if (link(first_file_tmp_path.data(), block_file_path.data()))
|
||||
throwFromErrno("Could not link " + block_file_path + " to " + first_file_tmp_path);
|
||||
}
|
||||
|
||||
/** remove the temporary file, enabling the OS to reclaim inode after all threads
|
||||
* have removed their corresponding files */
|
||||
Poco::File(first_file_tmp_path).remove();
|
||||
}
|
||||
|
||||
StorageDistributed & storage;
|
||||
ASTPtr query_ast;
|
||||
};
|
||||
|
||||
}
|
14
dbms/include/DB/Storages/Distributed/queryToString.h
Normal file
14
dbms/include/DB/Storages/Distributed/queryToString.h
Normal file
@ -0,0 +1,14 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Parsers/formatAST.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
inline std::string queryToString(const ASTPtr & query)
|
||||
{
|
||||
std::ostringstream s;
|
||||
formatAST(*query, s, 0, false, true);
|
||||
|
||||
return s.str();
|
||||
}
|
||||
}
|
@ -205,6 +205,20 @@ public:
|
||||
throw Exception("Method alter is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
/** Выполнить запрос (DROP|DETACH) PARTITION.
|
||||
*/
|
||||
virtual void dropPartition(const Field & partition, bool detach)
|
||||
{
|
||||
throw Exception("Method dropPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
/** Выполнить запрос ATTACH [UNREPLICATED] (PART|PARTITION).
|
||||
*/
|
||||
virtual void attachPartition(const Field & partition, bool unreplicated, bool part)
|
||||
{
|
||||
throw Exception("Method attachPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
/** Выполнить какую-либо фоновую работу. Например, объединение кусков в таблице типа MergeTree.
|
||||
* Возвращает - была ли выполнена какая-либо работа.
|
||||
*/
|
||||
@ -248,6 +262,9 @@ public:
|
||||
/// Поддерживается ли индекс в секции IN
|
||||
virtual bool supportsIndexForIn() const { return false; };
|
||||
|
||||
/// проверяет валидность данных
|
||||
virtual bool checkData() const { throw DB::Exception("Check query is not supported for " + getName() + " storage"); }
|
||||
|
||||
protected:
|
||||
IStorage() : is_dropped(false) {}
|
||||
|
||||
|
@ -13,7 +13,7 @@ namespace DB
|
||||
* При вызове деструктора или завершении сессии в ZooKeeper, переходит в состояние ABANDONED.
|
||||
* (В том числе при падении программы)
|
||||
*/
|
||||
class AbandonableLockInZooKeeper
|
||||
class AbandonableLockInZooKeeper : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
enum State
|
||||
@ -34,6 +34,14 @@ public:
|
||||
path = zookeeper.create(path_prefix, holder_path, zkutil::CreateMode::PersistentSequential);
|
||||
}
|
||||
|
||||
AbandonableLockInZooKeeper(AbandonableLockInZooKeeper && rhs)
|
||||
: zookeeper(rhs.zookeeper)
|
||||
{
|
||||
std::swap(path_prefix, rhs.path_prefix);
|
||||
std::swap(path, rhs.path);
|
||||
std::swap(holder_path, rhs.holder_path);
|
||||
}
|
||||
|
||||
String getPath()
|
||||
{
|
||||
return path;
|
||||
@ -49,6 +57,7 @@ public:
|
||||
{
|
||||
zookeeper.remove(path);
|
||||
zookeeper.remove(holder_path);
|
||||
holder_path = "";
|
||||
}
|
||||
|
||||
/// Добавляет в список действия, эквивалентные unlock().
|
||||
@ -60,6 +69,9 @@ public:
|
||||
|
||||
~AbandonableLockInZooKeeper()
|
||||
{
|
||||
if (holder_path.empty())
|
||||
return;
|
||||
|
||||
try
|
||||
{
|
||||
zookeeper.tryRemove(holder_path);
|
||||
|
@ -63,12 +63,14 @@ public:
|
||||
void add(const String & name);
|
||||
String getContainingPart(const String & name) const;
|
||||
|
||||
Strings getParts() const;
|
||||
Strings getParts() const; /// В порядке возрастания месяца и номера блока.
|
||||
|
||||
size_t size() const;
|
||||
|
||||
static String getPartName(DayNum_t left_date, DayNum_t right_date, UInt64 left_id, UInt64 right_id, UInt64 level);
|
||||
|
||||
/// Возвращает true если имя директории совпадает с форматом имени директории кусочков
|
||||
static bool isPartDirectory(const String & dir_name, Poco::RegularExpression::MatchVec & matches);
|
||||
static bool isPartDirectory(const String & dir_name, Poco::RegularExpression::MatchVec * out_matches = nullptr);
|
||||
|
||||
/// Кладет в DataPart данные из имени кусочка.
|
||||
static void parsePartName(const String & file_name, Part & part, const Poco::RegularExpression::MatchVec * matches = nullptr);
|
||||
|
@ -16,15 +16,21 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** В нескольких потоках в бесконечном цикле выполняет указанные функции.
|
||||
/** Используя фиксированное количество потоков, выполнять произвольное количество задач в бесконечном цикле.
|
||||
* Предназначена для задач, выполняющих постоянную фоновую работу (например, слияния).
|
||||
* Задача - функция, возвращающая bool - сделала ли она какую-либо работу.
|
||||
* Если сделала - надо выполнить ещё раз. Если нет - надо подождать несколько секунд, или до события wake, и выполнить ещё раз.
|
||||
*
|
||||
* Также, задача во время выполнения может временно увеличить какой-либо счётчик, относящийся ко всем задачам
|
||||
* - например, число одновременно идующих слияний.
|
||||
*/
|
||||
class BackgroundProcessingPool
|
||||
{
|
||||
public:
|
||||
typedef std::map<String, int> Counters;
|
||||
|
||||
/** Используется изнутри таски. Позволяет инкрементировать какие-нибудь счетчики.
|
||||
* После завершения таски, все изменения откатятся.
|
||||
/** Используется изнутри задачи. Позволяет инкрементировать какие-нибудь счетчики.
|
||||
* После завершения задачи, все изменения откатятся.
|
||||
* Например, чтобы можно было узнавать количество потоков, выполняющих большое слияние,
|
||||
* можно в таске, выполняющей большое слияние, инкрементировать счетчик. Декрементировать обратно его не нужно.
|
||||
*/
|
||||
@ -57,10 +63,14 @@ public:
|
||||
/// Переставить таск в начало очереди и разбудить какой-нибудь поток.
|
||||
void wake()
|
||||
{
|
||||
Poco::ScopedReadRWLock rlock(rwlock);
|
||||
if (removed)
|
||||
return;
|
||||
|
||||
std::unique_lock<std::mutex> lock(pool.mutex);
|
||||
pool.tasks.splice(pool.tasks.begin(), pool.tasks, iterator);
|
||||
|
||||
/// Не очень надежно: если все потоки сейчас выполняют работу, этот вызов никого не разбудит,
|
||||
/// Не очень надёжно: если все потоки сейчас выполняют работу, этот вызов никого не разбудит,
|
||||
/// и все будут спать в конце итерации.
|
||||
pool.wake_event.notify_one();
|
||||
}
|
||||
@ -70,50 +80,32 @@ public:
|
||||
|
||||
BackgroundProcessingPool & pool;
|
||||
Task function;
|
||||
Poco::RWLock lock;
|
||||
volatile bool removed;
|
||||
|
||||
/// При выполнении задачи, держится read lock. Переменная removed меняется под write lock-ом.
|
||||
Poco::RWLock rwlock;
|
||||
volatile bool removed = false;
|
||||
|
||||
std::list<std::shared_ptr<TaskInfo>>::iterator iterator;
|
||||
|
||||
TaskInfo(BackgroundProcessingPool & pool_, const Task & function_) : pool(pool_), function(function_), removed(false) {}
|
||||
TaskInfo(BackgroundProcessingPool & pool_, const Task & function_) : pool(pool_), function(function_) {}
|
||||
};
|
||||
|
||||
typedef std::shared_ptr<TaskInfo> TaskHandle;
|
||||
|
||||
|
||||
BackgroundProcessingPool(int size_) : size(size_), sleep_seconds(10), shutdown(false) {}
|
||||
|
||||
void setNumberOfThreads(int size_)
|
||||
BackgroundProcessingPool(int size_) : size(size_)
|
||||
{
|
||||
if (size_ <= 0)
|
||||
throw Exception("Invalid number of threads: " + toString(size_), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
std::unique_lock<std::mutex> tlock(threads_mutex);
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
|
||||
if (size_ == size)
|
||||
return;
|
||||
|
||||
if (threads.empty())
|
||||
{
|
||||
size = size_;
|
||||
return;
|
||||
}
|
||||
|
||||
throw Exception("setNumberOfThreads is not implemented for non-empty pool", ErrorCodes::NOT_IMPLEMENTED);
|
||||
threads.resize(size);
|
||||
for (auto & thread : threads)
|
||||
thread = std::thread([this] { threadFunction(); });
|
||||
}
|
||||
|
||||
int getNumberOfThreads()
|
||||
|
||||
int getNumberOfThreads() const
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
return size;
|
||||
}
|
||||
|
||||
void setSleepTime(double seconds)
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
sleep_seconds = seconds;
|
||||
}
|
||||
|
||||
int getCounter(const String & name)
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
@ -122,8 +114,6 @@ public:
|
||||
|
||||
TaskHandle addTask(const Task & task)
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(threads_mutex);
|
||||
|
||||
TaskHandle res(new TaskInfo(*this, task));
|
||||
|
||||
{
|
||||
@ -132,44 +122,22 @@ public:
|
||||
res->iterator = --tasks.end();
|
||||
}
|
||||
|
||||
if (threads.empty())
|
||||
{
|
||||
shutdown = false;
|
||||
counters.clear();
|
||||
threads.resize(size);
|
||||
for (std::thread & thread : threads)
|
||||
thread = std::thread(std::bind(&BackgroundProcessingPool::threadFunction, this));
|
||||
}
|
||||
wake_event.notify_all();
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
void removeTask(const TaskHandle & task)
|
||||
{
|
||||
std::unique_lock<std::mutex> tlock(threads_mutex);
|
||||
|
||||
/// Дождемся завершения всех выполнений этой задачи.
|
||||
/// Дождёмся завершения всех выполнений этой задачи.
|
||||
{
|
||||
Poco::ScopedWriteRWLock wlock(task->lock);
|
||||
Poco::ScopedWriteRWLock wlock(task->rwlock);
|
||||
task->removed = true;
|
||||
}
|
||||
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
auto it = std::find(tasks.begin(), tasks.end(), task);
|
||||
if (it == tasks.end())
|
||||
throw Exception("Task not found", ErrorCodes::LOGICAL_ERROR);
|
||||
tasks.erase(it);
|
||||
}
|
||||
|
||||
if (tasks.empty())
|
||||
{
|
||||
shutdown = true;
|
||||
wake_event.notify_all();
|
||||
for (std::thread & thread : threads)
|
||||
thread.join();
|
||||
threads.clear();
|
||||
counters.clear();
|
||||
tasks.erase(task->iterator);
|
||||
}
|
||||
}
|
||||
|
||||
@ -177,15 +145,10 @@ public:
|
||||
{
|
||||
try
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(threads_mutex);
|
||||
if (!threads.empty())
|
||||
{
|
||||
LOG_ERROR(&Logger::get("~BackgroundProcessingPool"), "Destroying non-empty BackgroundProcessingPool");
|
||||
shutdown = true;
|
||||
wake_event.notify_all();
|
||||
for (std::thread & thread : threads)
|
||||
thread.join();
|
||||
}
|
||||
shutdown = true;
|
||||
wake_event.notify_all();
|
||||
for (std::thread & thread : threads)
|
||||
thread.join();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -197,24 +160,25 @@ private:
|
||||
typedef std::list<TaskHandle> Tasks;
|
||||
typedef std::vector<std::thread> Threads;
|
||||
|
||||
std::mutex threads_mutex;
|
||||
std::mutex mutex;
|
||||
int size;
|
||||
Tasks tasks; /// Таски в порядке, в котором мы планируем их выполнять.
|
||||
Threads threads;
|
||||
Counters counters;
|
||||
double sleep_seconds;
|
||||
const size_t size;
|
||||
static constexpr double sleep_seconds = 10;
|
||||
|
||||
volatile bool shutdown;
|
||||
Tasks tasks; /// Задачи в порядке, в котором мы планируем их выполнять.
|
||||
Counters counters;
|
||||
std::mutex mutex; /// Для работы со списком tasks, а также с counters (когда threads не пустой).
|
||||
|
||||
Threads threads;
|
||||
|
||||
volatile bool shutdown = false;
|
||||
std::condition_variable wake_event;
|
||||
|
||||
|
||||
void threadFunction()
|
||||
{
|
||||
while (!shutdown)
|
||||
{
|
||||
Counters counters_diff;
|
||||
bool need_sleep = false;
|
||||
size_t tasks_count = 1;
|
||||
|
||||
try
|
||||
{
|
||||
@ -236,11 +200,12 @@ private:
|
||||
|
||||
if (!task)
|
||||
{
|
||||
std::this_thread::sleep_for(std::chrono::duration<double>(sleep_seconds));
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
wake_event.wait_for(lock, std::chrono::duration<double>(sleep_seconds));
|
||||
continue;
|
||||
}
|
||||
|
||||
Poco::ScopedReadRWLock rlock(task->lock);
|
||||
Poco::ScopedReadRWLock rlock(task->rwlock);
|
||||
if (task->removed)
|
||||
continue;
|
||||
|
||||
@ -248,15 +213,11 @@ private:
|
||||
|
||||
if (task->function(context))
|
||||
{
|
||||
/// Если у таска получилось выполнить какую-то работу, запустим его снова без паузы.
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
/// Если у задачи получилось выполнить какую-то работу, запустим её снова без паузы.
|
||||
need_sleep = false;
|
||||
|
||||
auto it = std::find(tasks.begin(), tasks.end(), task);
|
||||
if (it != tasks.end())
|
||||
{
|
||||
need_sleep = false;
|
||||
tasks.splice(tasks.begin(), tasks, it);
|
||||
}
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
tasks.splice(tasks.begin(), tasks, task->iterator);
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
@ -265,14 +226,12 @@ private:
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
|
||||
/// Вычтем все счетчики обратно.
|
||||
/// Вычтем все счётчики обратно.
|
||||
if (!counters_diff.empty())
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
for (const auto & it : counters_diff)
|
||||
{
|
||||
counters[it.first] -= it.second;
|
||||
}
|
||||
}
|
||||
|
||||
if (shutdown)
|
||||
@ -281,7 +240,7 @@ private:
|
||||
if (need_sleep)
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
wake_event.wait_for(lock, std::chrono::duration<double>(sleep_seconds / tasks_count));
|
||||
wake_event.wait_for(lock, std::chrono::duration<double>(sleep_seconds));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -26,27 +26,51 @@ public:
|
||||
{
|
||||
try
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(DiskSpaceMonitor::reserved_bytes_mutex);
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(DiskSpaceMonitor::mutex);
|
||||
if (DiskSpaceMonitor::reserved_bytes < size)
|
||||
{
|
||||
DiskSpaceMonitor::reserved_bytes = 0;
|
||||
LOG_ERROR(&Logger::get("DiskSpaceMonitor"), "Unbalanced reservations; it's a bug");
|
||||
LOG_ERROR(&Logger::get("DiskSpaceMonitor"), "Unbalanced reservations size; it's a bug");
|
||||
}
|
||||
else
|
||||
{
|
||||
DiskSpaceMonitor::reserved_bytes -= size;
|
||||
}
|
||||
|
||||
if (DiskSpaceMonitor::reservation_count == 0)
|
||||
{
|
||||
LOG_ERROR(&Logger::get("DiskSpaceMonitor"), "Unbalanced reservation count; it's a bug");
|
||||
}
|
||||
else
|
||||
{
|
||||
--DiskSpaceMonitor::reservation_count;
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException("~DiskSpaceMonitor");
|
||||
}
|
||||
}
|
||||
|
||||
/// Изменить количество зарезервированного места. При увеличении не делается проверка, что места достаточно.
|
||||
void update(size_t new_size)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(DiskSpaceMonitor::mutex);
|
||||
DiskSpaceMonitor::reserved_bytes -= size;
|
||||
size = new_size;
|
||||
DiskSpaceMonitor::reserved_bytes += size;
|
||||
}
|
||||
|
||||
size_t getSize() const
|
||||
{
|
||||
return size;
|
||||
}
|
||||
private:
|
||||
Reservation(size_t size_) : size(size_)
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(DiskSpaceMonitor::reserved_bytes_mutex);
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(DiskSpaceMonitor::mutex);
|
||||
DiskSpaceMonitor::reserved_bytes += size;
|
||||
++DiskSpaceMonitor::reservation_count;
|
||||
}
|
||||
size_t size;
|
||||
};
|
||||
@ -65,7 +89,7 @@ public:
|
||||
/// Зарезервируем дополнительно 30 МБ. Когда я тестировал, statvfs показывал на несколько мегабайт больше свободного места, чем df.
|
||||
res -= std::min(res, 30 * (1ul << 20));
|
||||
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(reserved_bytes_mutex);
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
|
||||
if (reserved_bytes > res)
|
||||
res = 0;
|
||||
@ -75,6 +99,18 @@ public:
|
||||
return res;
|
||||
}
|
||||
|
||||
static size_t getReservedSpace()
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
return reserved_bytes;
|
||||
}
|
||||
|
||||
static size_t getReservationCount()
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(mutex);
|
||||
return reservation_count;
|
||||
}
|
||||
|
||||
/// Если места (приблизительно) недостаточно, бросает исключение.
|
||||
static ReservationPtr reserve(const std::string & path, size_t size)
|
||||
{
|
||||
@ -87,7 +123,8 @@ public:
|
||||
|
||||
private:
|
||||
static size_t reserved_bytes;
|
||||
static Poco::FastMutex reserved_bytes_mutex;
|
||||
static size_t reservation_count;
|
||||
static Poco::FastMutex mutex;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -100,7 +100,7 @@ struct MergeTreeSettings
|
||||
size_t max_rows_to_use_cache = 1024 * 1024;
|
||||
|
||||
/// Через сколько секунд удалять ненужные куски.
|
||||
time_t old_parts_lifetime = 5 * 60;
|
||||
time_t old_parts_lifetime = 8 * 60;
|
||||
|
||||
/// Если в таблице хотя бы столько активных кусков, искусственно замедлять вставки в таблицу.
|
||||
size_t parts_to_delay_insert = 150;
|
||||
@ -110,11 +110,17 @@ struct MergeTreeSettings
|
||||
double insert_delay_step = 1.1;
|
||||
|
||||
/// Для скольки последних блоков хранить хеши в ZooKeeper.
|
||||
size_t replicated_deduplication_window = 1000;
|
||||
size_t replicated_deduplication_window = 100;
|
||||
|
||||
/// Хранить примерно столько последних записей в логе в ZooKeeper, даже если они никому уже не нужны.
|
||||
/// Не влияет на работу таблиц; используется только чтобы успеть посмотреть на лог в ZooKeeper глазами прежде, чем его очистят.
|
||||
size_t replicated_logs_to_keep = 100;
|
||||
|
||||
/// Максимальное количество ошибок при загрузке кусков, при котором ReplicatedMergeTree соглашается запускаться.
|
||||
size_t replicated_max_unexpected_parts = 3;
|
||||
size_t replicated_max_unexpectedly_merged_parts = 2;
|
||||
size_t replicated_max_missing_obsolete_parts = 5;
|
||||
size_t replicated_max_missing_active_parts = 20;
|
||||
};
|
||||
|
||||
class MergeTreeData : public ITableDeclaration
|
||||
@ -307,17 +313,22 @@ public:
|
||||
Poco::File(to).remove(true);
|
||||
}
|
||||
|
||||
/// Переименовывает кусок, дописав к имени префикс.
|
||||
void renameAddPrefix(const String & prefix) const
|
||||
void renameTo(const String & new_name) const
|
||||
{
|
||||
String from = storage.full_path + name + "/";
|
||||
String to = storage.full_path + prefix + name + "/";
|
||||
String to = storage.full_path + new_name + "/";
|
||||
|
||||
Poco::File f(from);
|
||||
f.setLastModified(Poco::Timestamp::fromEpochTime(time(0)));
|
||||
f.renameTo(to);
|
||||
}
|
||||
|
||||
/// Переименовывает кусок, дописав к имени префикс.
|
||||
void renameAddPrefix(const String & prefix) const
|
||||
{
|
||||
renameTo(prefix + name);
|
||||
}
|
||||
|
||||
/// Загрузить индекс и вычислить размер. Если size=0, вычислить его тоже.
|
||||
void loadIndex()
|
||||
{
|
||||
@ -344,12 +355,12 @@ public:
|
||||
}
|
||||
|
||||
/// Прочитать контрольные суммы, если есть.
|
||||
void loadChecksums()
|
||||
void loadChecksums(bool require)
|
||||
{
|
||||
String path = storage.full_path + name + "/checksums.txt";
|
||||
if (!Poco::File(path).exists())
|
||||
{
|
||||
if (storage.require_part_metadata)
|
||||
if (require)
|
||||
throw Exception("No checksums.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
|
||||
return;
|
||||
@ -359,16 +370,21 @@ public:
|
||||
assertEOF(file);
|
||||
}
|
||||
|
||||
void loadColumns()
|
||||
void loadColumns(bool require)
|
||||
{
|
||||
String path = storage.full_path + name + "/columns.txt";
|
||||
if (!Poco::File(path).exists())
|
||||
{
|
||||
if (storage.require_part_metadata)
|
||||
if (require)
|
||||
throw Exception("No columns.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
columns = *storage.columns;
|
||||
|
||||
/// Если нет файла со списком столбцов, запишем его.
|
||||
for (const NameAndTypePair & column : *storage.columns)
|
||||
{
|
||||
if (Poco::File(storage.full_path + name + "/" + escapeForFileName(column.name) + ".bin").exists())
|
||||
columns.push_back(column);
|
||||
}
|
||||
|
||||
{
|
||||
WriteBufferFromFile out(path + ".tmp", 4096);
|
||||
columns.writeText(out);
|
||||
@ -382,7 +398,7 @@ public:
|
||||
columns.readText(file, storage.context.getDataTypeFactory());
|
||||
}
|
||||
|
||||
void checkNotBroken()
|
||||
void checkNotBroken(bool require_part_metadata)
|
||||
{
|
||||
String path = storage.full_path + name;
|
||||
|
||||
@ -391,7 +407,7 @@ public:
|
||||
if (!checksums.files.count("primary.idx"))
|
||||
throw Exception("No checksum for primary.idx", ErrorCodes::NO_FILE_IN_DATA_PART);
|
||||
|
||||
if (storage.require_part_metadata)
|
||||
if (require_part_metadata)
|
||||
{
|
||||
for (const NameAndTypePair & it : columns)
|
||||
{
|
||||
@ -560,6 +576,9 @@ public:
|
||||
bool require_part_metadata_,
|
||||
BrokenPartCallback broken_part_callback_ = &MergeTreeData::doNothing);
|
||||
|
||||
/// Загрузить множество кусков с данными с диска. Вызывается один раз - сразу после создания объекта.
|
||||
void loadDataParts(bool skip_sanity_checks);
|
||||
|
||||
std::string getModePrefix() const;
|
||||
|
||||
bool supportsSampling() const { return !!sampling_expression; }
|
||||
@ -625,15 +644,23 @@ public:
|
||||
*/
|
||||
DataPartsVector renameTempPartAndReplace(MutableDataPartPtr part, Increment * increment = nullptr, Transaction * out_transaction = nullptr);
|
||||
|
||||
/** Убирает из рабочего набора куски remove и добавляет куски add.
|
||||
/** Убирает из рабочего набора куски remove и добавляет куски add. add должны уже быть в all_data_parts.
|
||||
* Если clear_without_timeout, данные будут удалены при следующем clearOldParts, игнорируя old_parts_lifetime.
|
||||
*/
|
||||
void replaceParts(const DataPartsVector & remove, const DataPartsVector & add, bool clear_without_timeout);
|
||||
|
||||
/** Переименовывает кусок в prefix_кусок и убирает его из рабочего набора.
|
||||
/** Добавляет новый кусок в список известных кусков и в рабочий набор.
|
||||
*/
|
||||
void attachPart(DataPartPtr part);
|
||||
|
||||
/** Переименовывает кусок в detached/prefix_кусок и забывает про него. Данные не будут удалены в clearOldParts.
|
||||
* Если restore_covered, добавляет в рабочий набор неактивные куски, слиянием которых получен удаляемый кусок.
|
||||
*/
|
||||
void renameAndDetachPart(DataPartPtr part, const String & prefix, bool restore_covered = false);
|
||||
void renameAndDetachPart(DataPartPtr part, const String & prefix = "", bool restore_covered = false, bool move_to_detached = true);
|
||||
|
||||
/** Убирает кусок из списка кусков (включая all_data_parts), но не перемещщает директорию.
|
||||
*/
|
||||
void detachPartInPlace(DataPartPtr part);
|
||||
|
||||
/** Возвращает старые неактуальные куски, которые можно удалить. Одновременно удаляет их из списка кусков, но не с диска.
|
||||
*/
|
||||
@ -685,6 +712,9 @@ public:
|
||||
ExpressionActionsPtr getPrimaryExpression() const { return primary_expr; }
|
||||
SortDescription getSortDescription() const { return sort_descr; }
|
||||
|
||||
/// Проверить, что кусок не сломан и посчитать для него чексуммы, если их нет.
|
||||
MutableDataPartPtr loadPartAndFixMetadata(const String & relative_path);
|
||||
|
||||
const Context & context;
|
||||
const String date_column_name;
|
||||
const ASTPtr sampling_expression;
|
||||
@ -726,9 +756,6 @@ private:
|
||||
DataParts all_data_parts;
|
||||
Poco::FastMutex all_data_parts_mutex;
|
||||
|
||||
/// Загрузить множество кусков с данными с диска. Вызывается один раз - при создании объекта.
|
||||
void loadDataParts();
|
||||
|
||||
/** Выражение, преобразующее типы столбцов.
|
||||
* Если преобразований типов нет, out_expression=nullptr.
|
||||
* out_rename_map отображает файлы-столбцы на выходе выражения в новые файлы таблицы.
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Storages/MergeTree/MergeTreeData.h>
|
||||
#include <DB/Storages/MergeTree/DiskSpaceMonitor.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -34,9 +35,13 @@ public:
|
||||
bool only_small,
|
||||
const AllowedMergingPredicate & can_merge);
|
||||
|
||||
/// Сливает куски.
|
||||
/** Сливает куски.
|
||||
* Если reservation != nullptr, то и дело уменьшает размер зарезервированного места
|
||||
* приблизительно пропорционально количеству уже выписанных данных.
|
||||
*/
|
||||
MergeTreeData::DataPartPtr mergeParts(
|
||||
const MergeTreeData::DataPartsVector & parts, const String & merged_name, MergeTreeData::Transaction * out_transaction = nullptr);
|
||||
const MergeTreeData::DataPartsVector & parts, const String & merged_name,
|
||||
MergeTreeData::Transaction * out_transaction = nullptr, DiskSpaceMonitor::Reservation * disk_reservation = nullptr);
|
||||
|
||||
/// Примерное количество места на диске, нужное для мерджа. С запасом.
|
||||
size_t estimateDiskSpaceForMerge(const MergeTreeData::DataPartsVector & parts);
|
||||
|
@ -9,16 +9,27 @@ namespace DB
|
||||
class MergeTreePartChecker
|
||||
{
|
||||
public:
|
||||
struct Settings
|
||||
{
|
||||
bool verbose = false; /// Пишет в stderr прогресс и ошибки, и не останавливается при первой ошибке.
|
||||
bool require_checksums = false; /// Требует, чтобы был columns.txt.
|
||||
bool require_column_files = false; /// Требует, чтобы для всех столбцов из columns.txt были файлы.
|
||||
size_t index_granularity = 8192;
|
||||
|
||||
Settings & setVerbose(bool verbose_) { verbose = verbose_; return *this; }
|
||||
Settings & setRequireChecksums(bool require_checksums_) { require_checksums = require_checksums_; return *this; }
|
||||
Settings & setRequireColumnFiles(bool require_column_files_) { require_column_files = require_column_files_; return *this; }
|
||||
Settings & setIndexGranularity(size_t index_granularity_) { index_granularity = index_granularity_; return *this; }
|
||||
};
|
||||
|
||||
/** Полностью проверяет данные кусочка:
|
||||
* - Вычисляет контрольные суммы и сравнивает с checksums.txt.
|
||||
* - Для массивов и строк проверяет соответствие размеров и количества данных.
|
||||
* - Проверяет правильность засечек.
|
||||
* Бросает исключение, если кусок испорчен или если проверить не получилось (TODO: можно попробовать разделить эти случаи).
|
||||
* Если strict, требует, чтобы для всех столбцов из columns.txt были файлы.
|
||||
* Если verbose, пишет в stderr прогресс и ошибки, и не останавливается при первой ошибке.
|
||||
*/
|
||||
static void checkDataPart(String path, size_t index_granularity, bool strict, const DataTypeFactory & data_type_factory,
|
||||
bool verbose = false);
|
||||
static void checkDataPart(String path, const Settings & settings, const DataTypeFactory & data_type_factory,
|
||||
MergeTreeData::DataPart::Checksums * out_checksums = nullptr);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -28,30 +28,12 @@ public:
|
||||
time_t min_date_time = DateLUT::instance().fromDayNum(DayNum_t(current_block.min_date));
|
||||
String month_name = toString(Date2OrderedIdentifier(min_date_time) / 100);
|
||||
|
||||
String month_path = storage.zookeeper_path + "/block_numbers/" + month_name;
|
||||
if (!storage.zookeeper->exists(month_path))
|
||||
{
|
||||
/// Создадим в block_numbers ноду для месяца и пропустим в ней 200 значений инкремента.
|
||||
/// Нужно, чтобы в будущем при необходимости можно было добавить данные в начало.
|
||||
zkutil::Ops ops;
|
||||
auto acl = storage.zookeeper->getDefaultACL();
|
||||
ops.push_back(new zkutil::Op::Create(month_path, "", acl, zkutil::CreateMode::Persistent));
|
||||
for (size_t i = 0; i < 200; ++i)
|
||||
{
|
||||
ops.push_back(new zkutil::Op::Create(month_path + "/skip_increment", "", acl, zkutil::CreateMode::Persistent));
|
||||
ops.push_back(new zkutil::Op::Remove(month_path + "/skip_increment", -1));
|
||||
}
|
||||
/// Игнорируем ошибки - не получиться могло только если кто-то еще выполнил эту строчку раньше нас.
|
||||
storage.zookeeper->tryMulti(ops);
|
||||
}
|
||||
|
||||
AbandonableLockInZooKeeper block_number_lock(
|
||||
storage.zookeeper_path + "/block_numbers/" + month_name + "/block-",
|
||||
storage.zookeeper_path + "/temp", *storage.zookeeper);
|
||||
AbandonableLockInZooKeeper block_number_lock = storage.allocateBlockNumber(month_name);
|
||||
|
||||
UInt64 part_number = block_number_lock.getNumber();
|
||||
|
||||
MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, part_number);
|
||||
String part_name = ActiveDataPartSet::getPartName(part->left_date, part->right_date, part->left, part->right, part->level);
|
||||
|
||||
/// Если в запросе не указан ID, возьмем в качестве ID хеш от данных. То есть, не вставляем одинаковые данные дважды.
|
||||
/// NOTE: Если такая дедупликация не нужна, можно вместо этого оставлять block_id пустым.
|
||||
@ -61,13 +43,10 @@ public:
|
||||
|
||||
LOG_DEBUG(log, "Wrote block " << part_number << " with ID " << block_id << ", " << current_block.block.rows() << " rows");
|
||||
|
||||
MergeTreeData::Transaction transaction; /// Если не получится добавить кусок в ZK, снова уберем его из рабочего набора.
|
||||
storage.data.renameTempPartAndAdd(part, nullptr, &transaction);
|
||||
|
||||
StorageReplicatedMergeTree::LogEntry log_entry;
|
||||
log_entry.type = StorageReplicatedMergeTree::LogEntry::GET_PART;
|
||||
log_entry.source_replica = storage.replica_name;
|
||||
log_entry.new_part_name = part->name;
|
||||
log_entry.new_part_name = part_name;
|
||||
|
||||
/// Одновременно добавим информацию о куске во все нужные места в ZooKeeper и снимем block_number_lock.
|
||||
zkutil::Ops ops;
|
||||
@ -94,7 +73,7 @@ public:
|
||||
storage.zookeeper->getDefaultACL(),
|
||||
zkutil::CreateMode::Persistent));
|
||||
}
|
||||
storage.checkPartAndAddToZooKeeper(part, ops);
|
||||
storage.checkPartAndAddToZooKeeper(part, ops, part_name);
|
||||
ops.push_back(new zkutil::Op::Create(
|
||||
storage.zookeeper_path + "/log/log-",
|
||||
log_entry.toString(),
|
||||
@ -102,6 +81,9 @@ public:
|
||||
zkutil::CreateMode::PersistentSequential));
|
||||
block_number_lock.getUnlockOps(ops);
|
||||
|
||||
MergeTreeData::Transaction transaction; /// Если не получится добавить кусок в ZK, снова уберем его из рабочего набора.
|
||||
storage.data.renameTempPartAndAdd(part, nullptr, &transaction);
|
||||
|
||||
try
|
||||
{
|
||||
auto code = storage.zookeeper->tryMulti(ops);
|
||||
|
@ -36,6 +36,8 @@ public:
|
||||
|
||||
String source_database_name;
|
||||
String source_table_name;
|
||||
|
||||
bool checkData() const override;
|
||||
|
||||
private:
|
||||
String name;
|
||||
|
@ -18,6 +18,9 @@ namespace DB
|
||||
*/
|
||||
class StorageDistributed : public IStorage
|
||||
{
|
||||
friend class DistributedBlockOutputStream;
|
||||
friend class DirectoryMonitor;
|
||||
|
||||
public:
|
||||
static StoragePtr create(
|
||||
const std::string & name_, /// Имя таблицы.
|
||||
@ -25,7 +28,9 @@ public:
|
||||
const String & remote_database_, /// БД на удалённых серверах.
|
||||
const String & remote_table_, /// Имя таблицы на удалённых серверах.
|
||||
const String & cluster_name,
|
||||
Context & context_);
|
||||
Context & context_,
|
||||
const ASTPtr & sharding_key_,
|
||||
const String & data_path_);
|
||||
|
||||
static StoragePtr create(
|
||||
const std::string & name_, /// Имя таблицы.
|
||||
@ -57,12 +62,21 @@ public:
|
||||
size_t max_block_size = DEFAULT_BLOCK_SIZE,
|
||||
unsigned threads = 1);
|
||||
|
||||
BlockOutputStreamPtr write(ASTPtr query) override;
|
||||
|
||||
void drop() override {}
|
||||
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) { name = new_table_name; }
|
||||
/// в подтаблицах добавлять и удалять столбы нужно вручную
|
||||
/// структура подтаблиц не проверяется
|
||||
void alter(const AlterCommands & params, const String & database_name, const String & table_name, Context & context);
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
const ExpressionActionsPtr & getShardingKeyExpr() const { return sharding_key_expr; }
|
||||
const String & getShardingKeyColumnName() const { return sharding_key_column_name; }
|
||||
const String & getPath() const { return path; }
|
||||
|
||||
|
||||
private:
|
||||
StorageDistributed(
|
||||
const std::string & name_,
|
||||
@ -70,17 +84,24 @@ private:
|
||||
const String & remote_database_,
|
||||
const String & remote_table_,
|
||||
Cluster & cluster_,
|
||||
const Context & context_);
|
||||
Context & context_,
|
||||
const ASTPtr & sharding_key_ = nullptr,
|
||||
const String & data_path_ = String{});
|
||||
|
||||
/// Создает копию запроса, меняет имена базы данных и таблицы.
|
||||
ASTPtr rewriteQuery(ASTPtr query);
|
||||
|
||||
/// create directory monitor thread by subdirectory name
|
||||
void createDirectoryMonitor(const std::string & name);
|
||||
/// create directory monitors for each existing subdirectory
|
||||
void createDirectoryMonitors();
|
||||
/// ensure directory monitor creation
|
||||
void requireDirectoryMonitor(const std::string & name);
|
||||
|
||||
String name;
|
||||
NamesAndTypesListPtr columns;
|
||||
String remote_database;
|
||||
String remote_table;
|
||||
|
||||
const Context & context;
|
||||
Context & context;
|
||||
|
||||
/// Временные таблицы, которые необходимо отправить на сервер. Переменная очищается после каждого вызова метода read
|
||||
/// Для подготовки к отправке нужно использовтаь метод storeExternalTables
|
||||
@ -91,6 +112,14 @@ private:
|
||||
|
||||
/// Соединения с удалёнными серверами.
|
||||
Cluster & cluster;
|
||||
|
||||
ExpressionActionsPtr sharding_key_expr;
|
||||
String sharding_key_column_name;
|
||||
bool write_enabled;
|
||||
String path;
|
||||
|
||||
class DirectoryMonitor;
|
||||
std::unordered_map<std::string, std::unique_ptr<DirectoryMonitor>> directory_monitors;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -13,6 +13,7 @@
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||
#include <DB/Common/FileChecker.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -80,6 +81,8 @@ class LogBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
LogBlockOutputStream(StorageLog & storage_);
|
||||
~LogBlockOutputStream() { writeSuffix(); }
|
||||
|
||||
void write(const Block & block);
|
||||
void writeSuffix();
|
||||
private:
|
||||
@ -156,6 +159,22 @@ public:
|
||||
|
||||
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name);
|
||||
|
||||
/// Данные столбца
|
||||
struct ColumnData
|
||||
{
|
||||
/// Задает номер столбца в файле с засечками.
|
||||
/// Не обязательно совпадает с номером столбца среди столбцов таблицы: здесь нумеруются также столбцы с длинами массивов.
|
||||
size_t column_index;
|
||||
|
||||
Poco::File data_file;
|
||||
Marks marks;
|
||||
};
|
||||
typedef std::map<String, ColumnData> Files_t;
|
||||
|
||||
Files_t & getFiles() { return files; }
|
||||
|
||||
bool checkData() const override;
|
||||
|
||||
protected:
|
||||
String path;
|
||||
String name;
|
||||
@ -195,18 +214,8 @@ protected:
|
||||
unsigned threads = 1);
|
||||
|
||||
private:
|
||||
/// Данные столбца
|
||||
struct ColumnData
|
||||
{
|
||||
/// Задает номер столбца в файле с засечками.
|
||||
/// Не обязательно совпадает с номером столбца среди столбцов таблицы: здесь нумеруются также столбцы с длинами массивов.
|
||||
size_t column_index;
|
||||
|
||||
Poco::File data_file;
|
||||
Marks marks;
|
||||
};
|
||||
typedef std::map<String, ColumnData> Files_t;
|
||||
Files_t files; /// name -> data
|
||||
|
||||
Names column_names; /// column_index -> name
|
||||
|
||||
Poco::File marks_file;
|
||||
@ -218,6 +227,10 @@ private:
|
||||
|
||||
size_t max_compress_block_size;
|
||||
|
||||
protected:
|
||||
FileChecker<StorageLog> file_checker;
|
||||
|
||||
private:
|
||||
/** Для обычных столбцов, в засечках указано количество строчек в блоке.
|
||||
* Для столбцов-массивов и вложенных структур, есть более одной группы засечек, соответствующих разным файлам:
|
||||
* - для внутренностей (файла name.bin) - указано суммарное количество элементов массивов в блоке,
|
||||
|
@ -15,6 +15,9 @@ public:
|
||||
std::string getName() const { return "MaterializedView"; }
|
||||
std::string getInnerTableName() const { return ".inner." + table_name; }
|
||||
|
||||
NameAndTypePair getColumn(const String &column_name) const;
|
||||
bool hasColumn(const String &column_name) const;
|
||||
|
||||
BlockOutputStreamPtr write(ASTPtr query);
|
||||
void drop() override;
|
||||
bool optimize();
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <DB/Storages/MergeTree/MergeTreeDataWriter.h>
|
||||
#include <DB/Storages/MergeTree/MergeTreeDataSelectExecutor.h>
|
||||
#include <DB/Storages/MergeTree/ReplicatedMergeTreePartsExchange.h>
|
||||
#include "MergeTree/AbandonableLockInZooKeeper.h"
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <zkutil/ZooKeeper.h>
|
||||
#include <zkutil/LeaderElection.h>
|
||||
@ -77,6 +78,9 @@ public:
|
||||
|
||||
void alter(const AlterCommands & params, const String & database_name, const String & table_name, Context & context) override;
|
||||
|
||||
void dropPartition(const Field & partition, bool detach) override;
|
||||
void attachPartition(const Field & partition, bool unreplicated, bool part) override;
|
||||
|
||||
/** Удаляет реплику из ZooKeeper. Если других реплик нет, удаляет всю таблицу из ZooKeeper.
|
||||
*/
|
||||
void drop() override;
|
||||
@ -111,7 +115,7 @@ private:
|
||||
{
|
||||
try
|
||||
{
|
||||
Poco::ScopedLock<Poco::FastMutex> lock(storage.queue_mutex);
|
||||
std::unique_lock<std::mutex> lock(storage.queue_mutex);
|
||||
if (!storage.future_parts.erase(part))
|
||||
throw Exception("Untagging already untagged future part " + part + ". This is a bug.", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
@ -126,30 +130,48 @@ private:
|
||||
|
||||
struct LogEntry
|
||||
{
|
||||
typedef Poco::SharedPtr<LogEntry> Ptr;
|
||||
|
||||
enum Type
|
||||
{
|
||||
GET_PART,
|
||||
MERGE_PARTS,
|
||||
GET_PART, /// Получить кусок с другой реплики.
|
||||
MERGE_PARTS, /// Слить куски.
|
||||
DROP_RANGE, /// Удалить куски в указанном месяце в указанном диапазоне номеров.
|
||||
ATTACH_PART, /// Перенести кусок из директории detached или unreplicated.
|
||||
};
|
||||
|
||||
String znode_name;
|
||||
|
||||
Type type;
|
||||
String source_replica; /// Пустая строка значит, что эта запись была добавлена сразу в очередь, а не скопирована из лога.
|
||||
|
||||
/// Имя куска, получающегося в результате.
|
||||
/// Для DROP_RANGE имя несуществующего куска. Нужно удалить все куски, покрытые им.
|
||||
String new_part_name;
|
||||
|
||||
Strings parts_to_merge;
|
||||
|
||||
/// Для DROP_RANGE, true значит, что куски нужно не удалить, а перенести в директорию detached.
|
||||
bool detach = false;
|
||||
|
||||
/// Для ATTACH_PART имя куска в директории detached или unreplicated.
|
||||
String source_part_name;
|
||||
/// Нужно переносить из директории unreplicated, а не detached.
|
||||
bool attach_unreplicated;
|
||||
|
||||
FuturePartTaggerPtr future_part_tagger;
|
||||
bool currently_executing = false; /// Доступ под queue_mutex.
|
||||
std::condition_variable execution_complete; /// Пробуждается когда currently_executing становится false.
|
||||
|
||||
void addResultToVirtualParts(StorageReplicatedMergeTree & storage)
|
||||
{
|
||||
if (type == MERGE_PARTS || type == GET_PART)
|
||||
if (type == MERGE_PARTS || type == GET_PART || type == DROP_RANGE || type == ATTACH_PART)
|
||||
storage.virtual_parts.add(new_part_name);
|
||||
}
|
||||
|
||||
void tagPartAsFuture(StorageReplicatedMergeTree & storage)
|
||||
{
|
||||
if (type == MERGE_PARTS || type == GET_PART)
|
||||
if (type == MERGE_PARTS || type == GET_PART || type == ATTACH_PART)
|
||||
future_part_tagger = new FuturePartTagger(new_part_name, storage);
|
||||
}
|
||||
|
||||
@ -166,17 +188,19 @@ private:
|
||||
return s;
|
||||
}
|
||||
|
||||
static LogEntry parse(const String & s)
|
||||
static Ptr parse(const String & s)
|
||||
{
|
||||
ReadBufferFromString in(s);
|
||||
LogEntry res;
|
||||
res.readText(in);
|
||||
Ptr res = new LogEntry;
|
||||
res->readText(in);
|
||||
assertEOF(in);
|
||||
return res;
|
||||
}
|
||||
};
|
||||
|
||||
typedef std::list<LogEntry> LogEntries;
|
||||
typedef LogEntry::Ptr LogEntryPtr;
|
||||
|
||||
typedef std::list<LogEntryPtr> LogEntries;
|
||||
|
||||
typedef std::set<String> StringSet;
|
||||
typedef std::list<String> StringList;
|
||||
@ -194,7 +218,7 @@ private:
|
||||
* В ZK записи в хронологическом порядке. Здесь - не обязательно.
|
||||
*/
|
||||
LogEntries queue;
|
||||
Poco::FastMutex queue_mutex;
|
||||
std::mutex queue_mutex;
|
||||
|
||||
/** Куски, которые появятся в результате действий, выполняемых прямо сейчас фоновыми потоками (этих действий нет в очереди).
|
||||
* Использовать под залоченным queue_mutex.
|
||||
@ -248,6 +272,7 @@ private:
|
||||
std::unique_ptr<MergeTreeData> unreplicated_data;
|
||||
std::unique_ptr<MergeTreeDataSelectExecutor> unreplicated_reader;
|
||||
std::unique_ptr<MergeTreeDataMerger> unreplicated_merger;
|
||||
Poco::FastMutex unreplicated_mutex; /// Для мерджей и удаления нереплицируемых кусков.
|
||||
|
||||
/// Потоки:
|
||||
|
||||
@ -261,12 +286,14 @@ private:
|
||||
/// Поток, выбирающий куски для слияния.
|
||||
std::thread merge_selecting_thread;
|
||||
Poco::Event merge_selecting_event;
|
||||
std::mutex merge_selecting_mutex; /// Берется на каждую итерацию выбора кусков для слияния.
|
||||
|
||||
/// Поток, удаляющий старые куски, записи в логе и блоки.
|
||||
std::thread cleanup_thread;
|
||||
|
||||
/// Поток, обрабатывающий переподключение к ZooKeeper при истечении сессии (очень маловероятное событие).
|
||||
std::thread restarting_thread;
|
||||
Poco::Event restarting_event;
|
||||
|
||||
/// Поток, следящий за изменениями списка столбцов в ZooKeeper и обновляющий куски в соответствии с этими изменениями.
|
||||
std::thread alter_thread;
|
||||
@ -285,7 +312,6 @@ private:
|
||||
Poco::Event shutdown_event;
|
||||
/// Нужно ли завершить restarting_thread.
|
||||
volatile bool permanent_shutdown_called = false;
|
||||
Poco::Event permanent_shutdown_event;
|
||||
|
||||
StorageReplicatedMergeTree(
|
||||
const String & zookeeper_path_,
|
||||
@ -306,7 +332,7 @@ private:
|
||||
|
||||
/** Создает минимальный набор нод в ZooKeeper.
|
||||
*/
|
||||
void createTable();
|
||||
void createTableIfNotExists();
|
||||
|
||||
/** Создает реплику в ZooKeeper и добавляет в очередь все, что нужно, чтобы догнать остальные реплики.
|
||||
*/
|
||||
@ -319,7 +345,7 @@ private:
|
||||
/** Проверить, что список столбцов и настройки таблицы совпадают с указанными в ZK (/metadata).
|
||||
* Если нет - бросить исключение.
|
||||
*/
|
||||
void checkTableStructure(bool skip_sanity_checks);
|
||||
void checkTableStructure(bool skip_sanity_checks, bool allow_alter);
|
||||
|
||||
/** Проверить, что множество кусков соответствует тому, что в ZK (/replicas/me/parts/).
|
||||
* Если каких-то кусков, описанных в ZK нет локально, бросить исключение.
|
||||
@ -332,11 +358,11 @@ private:
|
||||
void initVirtualParts();
|
||||
|
||||
/// Запустить или остановить фоновые потоки. Используется для частичной переинициализации при пересоздании сессии в ZooKeeper.
|
||||
void startup();
|
||||
bool tryStartup(); /// Возвращает false, если недоступен ZooKeeper.
|
||||
void partialShutdown();
|
||||
|
||||
/// Запретить запись в таблицу и завершить все фоновые потоки.
|
||||
void goReadOnly();
|
||||
void goReadOnlyPermanently();
|
||||
|
||||
|
||||
/** Проверить, что чексумма куска совпадает с чексуммой того же куска на какой-нибудь другой реплике.
|
||||
@ -345,7 +371,7 @@ private:
|
||||
* Кладет в ops действия, добавляющие данные о куске в ZooKeeper.
|
||||
* Вызывать под TableStructureLock.
|
||||
*/
|
||||
void checkPartAndAddToZooKeeper(MergeTreeData::DataPartPtr part, zkutil::Ops & ops);
|
||||
void checkPartAndAddToZooKeeper(MergeTreeData::DataPartPtr part, zkutil::Ops & ops, String name_override = "");
|
||||
|
||||
/// Убирает кусок из ZooKeeper и добавляет в очередь задание скачать его. Предполагается это делать с битыми кусками.
|
||||
void removePartAndEnqueueFetch(const String & part_name);
|
||||
@ -379,6 +405,9 @@ private:
|
||||
*/
|
||||
bool executeLogEntry(const LogEntry & entry, BackgroundProcessingPool::Context & pool_context);
|
||||
|
||||
void executeDropRange(const LogEntry & entry);
|
||||
bool executeAttachPart(const LogEntry & entry); /// Возвращает false, если куска нет, и его нужно забрать с другой реплики.
|
||||
|
||||
/** Обновляет очередь.
|
||||
*/
|
||||
void queueUpdatingThread();
|
||||
@ -423,6 +452,15 @@ private:
|
||||
/** Скачать указанный кусок с указанной реплики.
|
||||
*/
|
||||
void fetchPart(const String & part_name, const String & replica_name);
|
||||
|
||||
///
|
||||
|
||||
AbandonableLockInZooKeeper allocateBlockNumber(const String & month_name);
|
||||
|
||||
/** Дождаться, пока все реплики, включая эту, выполнят указанное действие из лога.
|
||||
* Если одновременно с этим добавляются реплики, может не дождаться добавленную реплику.
|
||||
*/
|
||||
void waitForAllReplicasToProcessLogEntry(const LogEntry & entry);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -12,20 +11,6 @@ namespace DB
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
class NumbersBlockInputStream : public IProfilingBlockInputStream
|
||||
{
|
||||
public:
|
||||
NumbersBlockInputStream(size_t block_size_);
|
||||
String getName() const { return "NumbersBlockInputStream"; }
|
||||
String getID() const { return "Numbers"; }
|
||||
protected:
|
||||
Block readImpl();
|
||||
private:
|
||||
size_t block_size;
|
||||
UInt64 next;
|
||||
};
|
||||
|
||||
|
||||
/** Реализует хранилище для системной таблицы Numbers.
|
||||
* Таблица содержит единственный столбец number UInt64.
|
||||
* Из этой таблицы можно прочитать все натуральные числа, начиная с 0 (до 2^64 - 1, а потом заново).
|
||||
@ -33,7 +18,7 @@ private:
|
||||
class StorageSystemNumbers : public IStorage
|
||||
{
|
||||
public:
|
||||
static StoragePtr create(const std::string & name_);
|
||||
static StoragePtr create(const std::string & name_, bool multithreaded_ = false);
|
||||
|
||||
std::string getName() const { return "SystemNumbers"; }
|
||||
std::string getTableName() const { return name; }
|
||||
@ -51,8 +36,9 @@ public:
|
||||
private:
|
||||
const std::string name;
|
||||
NamesAndTypesList columns;
|
||||
bool multithreaded;
|
||||
|
||||
StorageSystemNumbers(const std::string & name_);
|
||||
StorageSystemNumbers(const std::string & name_, bool multithreaded_);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -12,6 +12,8 @@
|
||||
#include <DB/Storages/IStorage.h>
|
||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||
#include <DB/DataStreams/IBlockOutputStream.h>
|
||||
#include <DB/Common/FileChecker.h>
|
||||
#include <Poco/Util/XMLConfiguration.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -55,11 +57,13 @@ private:
|
||||
void readData(const String & name, const IDataType & type, IColumn & column, size_t limit, size_t level = 0, bool read_offsets = true);
|
||||
};
|
||||
|
||||
|
||||
class TinyLogBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
TinyLogBlockOutputStream(StorageTinyLog & storage_);
|
||||
|
||||
~TinyLogBlockOutputStream();
|
||||
|
||||
void write(const Block & block);
|
||||
void writeSuffix();
|
||||
private:
|
||||
@ -129,12 +133,7 @@ public:
|
||||
|
||||
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name);
|
||||
|
||||
private:
|
||||
String path;
|
||||
String name;
|
||||
NamesAndTypesListPtr columns;
|
||||
|
||||
size_t max_compress_block_size;
|
||||
bool checkData() const override;
|
||||
|
||||
/// Данные столбца
|
||||
struct ColumnData
|
||||
@ -142,8 +141,22 @@ private:
|
||||
Poco::File data_file;
|
||||
};
|
||||
typedef std::map<String, ColumnData> Files_t;
|
||||
|
||||
Files_t & getFiles();
|
||||
|
||||
private:
|
||||
String path;
|
||||
String name;
|
||||
NamesAndTypesListPtr columns;
|
||||
|
||||
size_t max_compress_block_size;
|
||||
|
||||
Files_t files;
|
||||
|
||||
FileChecker<StorageTinyLog> file_checker;
|
||||
|
||||
Logger * log;
|
||||
|
||||
StorageTinyLog(const std::string & path_, const std::string & name_, NamesAndTypesListPtr columns_, bool attach, size_t max_compress_block_size_);
|
||||
|
||||
void addFile(const String & column_name, const IDataType & type, size_t level = 0);
|
||||
|
@ -1,9 +1,7 @@
|
||||
#include <DB/AggregateFunctions/AggregateFunctionCount.h>
|
||||
#include <DB/AggregateFunctions/AggregateFunctionSum.h>
|
||||
#include <DB/AggregateFunctions/AggregateFunctionAvg.h>
|
||||
#include <DB/AggregateFunctions/AggregateFunctionAny.h>
|
||||
#include <DB/AggregateFunctions/AggregateFunctionAnyLast.h>
|
||||
#include <DB/AggregateFunctions/AggregateFunctionsMinMax.h>
|
||||
#include <DB/AggregateFunctions/AggregateFunctionsMinMaxAny.h>
|
||||
#include <DB/AggregateFunctions/AggregateFunctionsArgMinMax.h>
|
||||
#include <DB/AggregateFunctions/AggregateFunctionUniq.h>
|
||||
#include <DB/AggregateFunctions/AggregateFunctionUniqUpTo.h>
|
||||
@ -69,6 +67,7 @@ static IAggregateFunction * createWithNumericType(const IDataType & argument_typ
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
|
||||
template<template <typename, typename> class AggregateFunctionTemplate, template <typename> class Data>
|
||||
static IAggregateFunction * createWithNumericType(const IDataType & argument_type)
|
||||
{
|
||||
@ -87,18 +86,48 @@ static IAggregateFunction * createWithNumericType(const IDataType & argument_typ
|
||||
}
|
||||
|
||||
|
||||
/// min, max, any, anyLast
|
||||
template<template <typename> class AggregateFunctionTemplate, template <typename> class Data>
|
||||
static IAggregateFunction * createAggregateFunctionSingleValue(const 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 AggregateFunctionTemplate<Data<SingleValueDataFixed<UInt8>>>;
|
||||
else if (typeid_cast<const DataTypeUInt16 *>(&argument_type)) return new AggregateFunctionTemplate<Data<SingleValueDataFixed<UInt16>>>;
|
||||
else if (typeid_cast<const DataTypeUInt32 *>(&argument_type)) return new AggregateFunctionTemplate<Data<SingleValueDataFixed<UInt32>>>;
|
||||
else if (typeid_cast<const DataTypeUInt64 *>(&argument_type)) return new AggregateFunctionTemplate<Data<SingleValueDataFixed<UInt64>>>;
|
||||
else if (typeid_cast<const DataTypeInt8 *>(&argument_type)) return new AggregateFunctionTemplate<Data<SingleValueDataFixed<Int8>>>;
|
||||
else if (typeid_cast<const DataTypeInt16 *>(&argument_type)) return new AggregateFunctionTemplate<Data<SingleValueDataFixed<Int16>>>;
|
||||
else if (typeid_cast<const DataTypeInt32 *>(&argument_type)) return new AggregateFunctionTemplate<Data<SingleValueDataFixed<Int32>>>;
|
||||
else if (typeid_cast<const DataTypeInt64 *>(&argument_type)) return new AggregateFunctionTemplate<Data<SingleValueDataFixed<Int64>>>;
|
||||
else if (typeid_cast<const DataTypeFloat32 *>(&argument_type)) return new AggregateFunctionTemplate<Data<SingleValueDataFixed<Float32>>>;
|
||||
else if (typeid_cast<const DataTypeFloat64 *>(&argument_type)) return new AggregateFunctionTemplate<Data<SingleValueDataFixed<Float64>>>;
|
||||
else if (typeid_cast<const DataTypeDate *>(&argument_type))
|
||||
return new AggregateFunctionTemplate<Data<SingleValueDataFixed<DataTypeDate::FieldType>>>;
|
||||
else if (typeid_cast<const DataTypeDateTime*>(&argument_type))
|
||||
return new AggregateFunctionTemplate<Data<SingleValueDataFixed<DataTypeDateTime::FieldType>>>;
|
||||
else if (typeid_cast<const DataTypeString*>(&argument_type))
|
||||
return new AggregateFunctionTemplate<Data<SingleValueDataString>>;
|
||||
else
|
||||
return new AggregateFunctionTemplate<Data<SingleValueDataGeneric>>;
|
||||
}
|
||||
|
||||
|
||||
AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const DataTypes & argument_types, int recursion_level) const
|
||||
{
|
||||
if (name == "count")
|
||||
return new AggregateFunctionCount;
|
||||
else if (name == "any")
|
||||
return new AggregateFunctionAny;
|
||||
return createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyData>(name, argument_types);
|
||||
else if (name == "anyLast")
|
||||
return new AggregateFunctionAnyLast;
|
||||
return createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionAnyLastData>(name, argument_types);
|
||||
else if (name == "min")
|
||||
return new AggregateFunctionMin;
|
||||
return createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMinData>(name, argument_types);
|
||||
else if (name == "max")
|
||||
return new AggregateFunctionMax;
|
||||
return createAggregateFunctionSingleValue<AggregateFunctionsSingleValue, AggregateFunctionMaxData>(name, argument_types);
|
||||
else if (name == "argMin")
|
||||
return new AggregateFunctionArgMin;
|
||||
else if (name == "argMax")
|
||||
|
@ -49,6 +49,16 @@
|
||||
|
||||
#include <DB/Common/ExternalTable.h>
|
||||
|
||||
|
||||
/// http://en.wikipedia.org/wiki/ANSI_escape_code
|
||||
#define SAVE_CURSOR_POSITION "\033[s"
|
||||
#define RESTORE_CURSOR_POSITION "\033[u"
|
||||
#define CLEAR_TO_END_OF_LINE "\033[K"
|
||||
/// Эти коды, возможно, поддерживаются не везде.
|
||||
#define DISABLE_LINE_WRAPPING "\033[?7l"
|
||||
#define ENABLE_LINE_WRAPPING "\033[?7h"
|
||||
|
||||
|
||||
/** Клиент командной строки СУБД ClickHouse.
|
||||
*/
|
||||
|
||||
@ -61,11 +71,7 @@ using Poco::SharedPtr;
|
||||
class Client : public Poco::Util::Application
|
||||
{
|
||||
public:
|
||||
Client() : is_interactive(true), stdin_is_not_tty(false),
|
||||
format_max_block_size(0), std_in(STDIN_FILENO), std_out(STDOUT_FILENO), processed_rows(0),
|
||||
rows_read_on_server(0), bytes_read_on_server(0), written_progress_chars(0), written_first_block(false)
|
||||
{
|
||||
}
|
||||
Client() {}
|
||||
|
||||
private:
|
||||
typedef std::unordered_set<String> StringSet;
|
||||
@ -77,24 +83,24 @@ private:
|
||||
"q", "й", "\\q", "\\Q", "\\й", "\\Й", ":q", "Жй"
|
||||
};
|
||||
|
||||
bool is_interactive; /// Использовать readline интерфейс или batch режим.
|
||||
bool stdin_is_not_tty; /// stdin - не терминал.
|
||||
bool is_interactive = true; /// Использовать readline интерфейс или batch режим.
|
||||
bool stdin_is_not_tty = false; /// stdin - не терминал.
|
||||
|
||||
SharedPtr<Connection> connection; /// Соединение с БД.
|
||||
String query; /// Текущий запрос.
|
||||
|
||||
String format; /// Формат вывода результата в консоль.
|
||||
size_t format_max_block_size; /// Максимальный размер блока при выводе в консоль.
|
||||
size_t format_max_block_size = 0; /// Максимальный размер блока при выводе в консоль.
|
||||
String insert_format; /// Формат данных для INSERT-а при чтении их из stdin в batch режиме
|
||||
size_t insert_format_max_block_size; /// Максимальный размер блока при чтении данных INSERT-а.
|
||||
size_t insert_format_max_block_size = 0; /// Максимальный размер блока при чтении данных INSERT-а.
|
||||
|
||||
Context context;
|
||||
|
||||
/// Чтение из stdin для batch режима
|
||||
ReadBufferFromFileDescriptor std_in;
|
||||
ReadBufferFromFileDescriptor std_in {STDIN_FILENO};
|
||||
|
||||
/// Вывод в консоль
|
||||
WriteBufferFromFileDescriptor std_out;
|
||||
WriteBufferFromFileDescriptor std_out {STDOUT_FILENO};
|
||||
BlockOutputStreamPtr block_std_out;
|
||||
|
||||
String home_path;
|
||||
@ -105,7 +111,7 @@ private:
|
||||
String history_file;
|
||||
|
||||
/// Строк прочитано или записано.
|
||||
size_t processed_rows;
|
||||
size_t processed_rows = 0;
|
||||
|
||||
/// Распарсенный запрос. Оттуда берутся некоторые настройки (формат).
|
||||
ASTPtr parsed_query;
|
||||
@ -115,10 +121,10 @@ private:
|
||||
|
||||
Stopwatch watch;
|
||||
|
||||
size_t rows_read_on_server;
|
||||
size_t bytes_read_on_server;
|
||||
size_t written_progress_chars;
|
||||
bool written_first_block;
|
||||
size_t rows_read_on_server = 0;
|
||||
size_t bytes_read_on_server = 0;
|
||||
size_t written_progress_chars = 0;
|
||||
bool written_first_block = false;
|
||||
|
||||
/// Информация о внешних таблицах
|
||||
std::list<ExternalTable> external_tables;
|
||||
@ -441,7 +447,7 @@ private:
|
||||
if (exit_strings.end() != exit_strings.find(line))
|
||||
return false;
|
||||
|
||||
block_std_out = nullptr;
|
||||
resetOutput();
|
||||
|
||||
watch.restart();
|
||||
|
||||
@ -642,6 +648,14 @@ private:
|
||||
}
|
||||
|
||||
|
||||
/** Сбросить все данные, что ещё остались в буферах. */
|
||||
void resetOutput()
|
||||
{
|
||||
block_std_out = nullptr;
|
||||
std_out.next();
|
||||
}
|
||||
|
||||
|
||||
/** Получает и обрабатывает пакеты из сервера.
|
||||
* Также следит, не требуется ли прервать выполнение запроса.
|
||||
*/
|
||||
@ -747,12 +761,7 @@ private:
|
||||
void onData(Block & block)
|
||||
{
|
||||
if (written_progress_chars)
|
||||
{
|
||||
for (size_t i = 0; i < written_progress_chars; ++i)
|
||||
std::cerr << "\b \b";
|
||||
|
||||
written_progress_chars = 0;
|
||||
}
|
||||
clearProgress();
|
||||
|
||||
if (!block)
|
||||
return;
|
||||
@ -780,7 +789,8 @@ private:
|
||||
written_first_block = true;
|
||||
}
|
||||
|
||||
std_out.next();
|
||||
/// Полученный блок данных сразу выводится клиенту.
|
||||
block_std_out->flush();
|
||||
}
|
||||
|
||||
|
||||
@ -804,8 +814,18 @@ private:
|
||||
}
|
||||
|
||||
|
||||
void clearProgress()
|
||||
{
|
||||
std::cerr << RESTORE_CURSOR_POSITION CLEAR_TO_END_OF_LINE;
|
||||
written_progress_chars = 0;
|
||||
}
|
||||
|
||||
|
||||
void writeProgress()
|
||||
{
|
||||
if (!is_interactive)
|
||||
return;
|
||||
|
||||
static size_t increment = 0;
|
||||
static const char * indicators[8] =
|
||||
{
|
||||
@ -816,30 +836,30 @@ private:
|
||||
"\033[1;34m←\033[0m",
|
||||
"\033[1;35m↖\033[0m",
|
||||
"\033[1;36m↑\033[0m",
|
||||
"\033[1;37m↗\033[0m",
|
||||
"\033[1m↗\033[0m",
|
||||
};
|
||||
|
||||
if (is_interactive)
|
||||
{
|
||||
std::cerr << std::string(written_progress_chars, '\b');
|
||||
if (written_progress_chars)
|
||||
clearProgress();
|
||||
else
|
||||
std::cerr << SAVE_CURSOR_POSITION;
|
||||
|
||||
std::stringstream message;
|
||||
message << indicators[increment % 8]
|
||||
<< std::fixed << std::setprecision(3)
|
||||
<< " Progress: " << rows_read_on_server << " rows, " << bytes_read_on_server / 1000000.0 << " MB";
|
||||
std::stringstream message;
|
||||
message << indicators[increment % 8]
|
||||
<< std::fixed << std::setprecision(3)
|
||||
<< " Progress: " << rows_read_on_server << " rows, " << bytes_read_on_server / 1000000.0 << " MB";
|
||||
|
||||
size_t elapsed_ns = watch.elapsed();
|
||||
if (elapsed_ns)
|
||||
message << " ("
|
||||
<< rows_read_on_server * 1000000000.0 / elapsed_ns << " rows/s., "
|
||||
<< bytes_read_on_server * 1000.0 / elapsed_ns << " MB/s.) ";
|
||||
else
|
||||
message << ". ";
|
||||
size_t elapsed_ns = watch.elapsed();
|
||||
if (elapsed_ns)
|
||||
message << " ("
|
||||
<< rows_read_on_server * 1000000000.0 / elapsed_ns << " rows/s., "
|
||||
<< bytes_read_on_server * 1000.0 / elapsed_ns << " MB/s.) ";
|
||||
else
|
||||
message << ". ";
|
||||
|
||||
written_progress_chars = message.str().size() - 13;
|
||||
std::cerr << message.rdbuf();
|
||||
++increment;
|
||||
}
|
||||
written_progress_chars = message.str().size() - 13;
|
||||
std::cerr << DISABLE_LINE_WRAPPING << message.rdbuf() << ENABLE_LINE_WRAPPING;
|
||||
++increment;
|
||||
}
|
||||
|
||||
|
||||
@ -859,6 +879,8 @@ private:
|
||||
|
||||
void onException(const Exception & e)
|
||||
{
|
||||
resetOutput();
|
||||
|
||||
std::cerr << "Received exception from server:" << std::endl
|
||||
<< "Code: " << e.code() << ". " << e.displayText();
|
||||
}
|
||||
@ -876,7 +898,7 @@ private:
|
||||
if (block_std_out)
|
||||
block_std_out->writeSuffix();
|
||||
|
||||
std_out.next();
|
||||
resetOutput();
|
||||
|
||||
if (is_interactive && !written_first_block)
|
||||
std::cout << "Ok." << std::endl;
|
||||
|
@ -284,14 +284,17 @@ void Connection::sendData(const Block & block, const String & name)
|
||||
}
|
||||
|
||||
|
||||
void Connection::sendPreparedData(ReadBuffer & input, const String & name)
|
||||
void Connection::sendPreparedData(ReadBuffer & input, size_t size, const String & name)
|
||||
{
|
||||
writeVarUInt(Protocol::Client::Data, *out);
|
||||
|
||||
if (server_revision >= DBMS_MIN_REVISION_WITH_TEMPORARY_TABLES)
|
||||
writeStringBinary(name, *out);
|
||||
|
||||
copyData(input, *out);
|
||||
if (0 == size)
|
||||
copyData(input, *out);
|
||||
else
|
||||
copyData(input, *out, size);
|
||||
out->next();
|
||||
}
|
||||
|
||||
|
45
dbms/src/Columns/IColumn.cpp
Normal file
45
dbms/src/Columns/IColumn.cpp
Normal file
@ -0,0 +1,45 @@
|
||||
#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;
|
||||
}
|
||||
|
||||
}
|
60
dbms/src/Common/Macros.cpp
Normal file
60
dbms/src/Common/Macros.cpp
Normal file
@ -0,0 +1,60 @@
|
||||
#include <DB/Common/Macros.h>
|
||||
#include <DB/Core/Exception.h>
|
||||
#include <DB/Core/ErrorCodes.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
Macros::Macros() {}
|
||||
|
||||
Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String & root_key)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys keys;
|
||||
config.keys(root_key, keys);
|
||||
for (const String & key : keys)
|
||||
{
|
||||
macros[key] = config.getString(root_key + "." + key);
|
||||
}
|
||||
}
|
||||
|
||||
String Macros::expand(const String & s) const
|
||||
{
|
||||
if (s.find('{') == String::npos)
|
||||
return s;
|
||||
|
||||
String res;
|
||||
size_t pos = 0;
|
||||
while (true)
|
||||
{
|
||||
size_t begin = s.find('{', pos);
|
||||
|
||||
if (begin == String::npos)
|
||||
{
|
||||
res.append(s, pos, String::npos);
|
||||
break;
|
||||
}
|
||||
else
|
||||
{
|
||||
res.append(s, pos, begin - pos);
|
||||
}
|
||||
|
||||
++begin;
|
||||
size_t end = s.find('}', begin);
|
||||
if (end == String::npos)
|
||||
throw Exception("Unbalanced { and } in string with macros: \"" + s + "\"", ErrorCodes::SYNTAX_ERROR);
|
||||
|
||||
String macro_name = s.substr(begin, end - begin);
|
||||
|
||||
auto it = macros.find(macro_name);
|
||||
if (it == macros.end())
|
||||
throw Exception("No macro " + macro_name + " in config", ErrorCodes::SYNTAX_ERROR);
|
||||
|
||||
res += it->second;
|
||||
|
||||
pos = end + 1;
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
}
|
@ -108,18 +108,15 @@ static bool isValidFunction(ASTPtr expression, const NameSet & columns)
|
||||
/// Извлечь все подфункции главной конъюнкции, но зависящие только от заданных столбцов
|
||||
static void extractFunctions(ASTPtr expression, const NameSet & columns, std::vector<ASTPtr> & result)
|
||||
{
|
||||
if (const ASTFunction * function = typeid_cast<const ASTFunction *>(&* expression))
|
||||
const ASTFunction * function = typeid_cast<const ASTFunction *>(&* expression);
|
||||
if (function && function->name == "and")
|
||||
{
|
||||
if (function->name == "and")
|
||||
{
|
||||
for (size_t i = 0; i < function->arguments->children.size(); ++i)
|
||||
extractFunctions(function->arguments->children[i], columns, result);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (isValidFunction(expression, columns))
|
||||
result.push_back(expression->clone());
|
||||
}
|
||||
for (size_t i = 0; i < function->arguments->children.size(); ++i)
|
||||
extractFunctions(function->arguments->children[i], columns, result);
|
||||
}
|
||||
else if (isValidFunction(expression, columns))
|
||||
{
|
||||
result.push_back(expression->clone());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -111,6 +111,8 @@ void CreatingSetsBlockInputStream::create(SubqueryForSet & subquery)
|
||||
size_t rows = 0;
|
||||
size_t bytes = 0;
|
||||
|
||||
watch.stop();
|
||||
|
||||
subquery.source->getLeafRowsBytes(rows, bytes);
|
||||
|
||||
size_t head_rows = 0;
|
||||
|
@ -41,7 +41,7 @@ Block FilterBlockInputStream::readImpl()
|
||||
/** Если фильтр - константа (например, написано WHERE 1),
|
||||
* то либо вернём пустой блок, либо вернём блок без изменений.
|
||||
*/
|
||||
ColumnConstUInt8 * column_const = typeid_cast<ColumnConstUInt8 *>(&*column);
|
||||
const ColumnConstUInt8 * column_const = typeid_cast<const ColumnConstUInt8 *>(&*column);
|
||||
if (column_const)
|
||||
{
|
||||
if (!column_const->getData())
|
||||
@ -50,52 +50,73 @@ Block FilterBlockInputStream::readImpl()
|
||||
return res;
|
||||
}
|
||||
|
||||
ColumnUInt8 * column_vec = typeid_cast<ColumnUInt8 *>(&*column);
|
||||
const ColumnUInt8 * column_vec = typeid_cast<const ColumnUInt8 *>(&*column);
|
||||
if (!column_vec)
|
||||
throw Exception("Illegal type " + column->getName() + " of column for filter. Must be ColumnUInt8 or ColumnConstUInt8.", ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER);
|
||||
|
||||
IColumn::Filter & filter = column_vec->getData();
|
||||
const IColumn::Filter & filter = column_vec->getData();
|
||||
|
||||
/// Если кроме столбца с фильтром ничего нет.
|
||||
if (columns == 1)
|
||||
{
|
||||
/// То посчитаем в нём количество единичек.
|
||||
size_t filtered_rows = 0;
|
||||
for (size_t i = 0, size = filter.size(); i < size; ++i)
|
||||
if (filter[i])
|
||||
++filtered_rows;
|
||||
|
||||
/// Если текущий блок полностью отфильтровался - перейдём к следующему.
|
||||
if (filtered_rows == 0)
|
||||
continue;
|
||||
|
||||
/// Заменяем этот столбец на столбец с константой 1, нужного размера.
|
||||
res.getByPosition(filter_column).column = new ColumnConstUInt8(filtered_rows, 1);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
/// Общий случай - фильтруем остальные столбцы.
|
||||
/** Выясним, сколько строк будет в результате.
|
||||
* Для этого отфильтруем первый попавшийся неконстантный столбец
|
||||
* или же посчитаем количество выставленных байт в фильтре.
|
||||
*/
|
||||
size_t first_non_constant_column = 0;
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
{
|
||||
if (i != static_cast<size_t>(filter_column))
|
||||
if (!res.getByPosition(i).column->isConst())
|
||||
{
|
||||
ColumnWithNameAndType & current_column = res.getByPosition(i);
|
||||
current_column.column = current_column.column->filter(filter);
|
||||
if (current_column.column->empty())
|
||||
first_non_constant_column = i;
|
||||
|
||||
if (first_non_constant_column != static_cast<size_t>(filter_column))
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
/// Любой столбец - не являющийся фильтром.
|
||||
IColumn & any_not_filter_column = *res.getByPosition(filter_column == 0 ? 1 : 0).column;
|
||||
size_t filtered_rows = 0;
|
||||
if (first_non_constant_column != static_cast<size_t>(filter_column))
|
||||
{
|
||||
ColumnWithNameAndType & current_column = res.getByPosition(first_non_constant_column);
|
||||
current_column.column = current_column.column->filter(filter);
|
||||
filtered_rows = current_column.column->size();
|
||||
}
|
||||
else
|
||||
{
|
||||
filtered_rows = countBytesInFilter(filter);
|
||||
}
|
||||
|
||||
/// Если текущий блок полностью отфильтровался - перейдём к следующему.
|
||||
if (any_not_filter_column.empty())
|
||||
if (filtered_rows == 0)
|
||||
continue;
|
||||
|
||||
/// Сам столбец с фильтром заменяем на столбец с константой 1, так как после фильтрации в нём ничего другого не останется.
|
||||
res.getByPosition(filter_column).column = new ColumnConstUInt8(any_not_filter_column.size(), 1);
|
||||
/// Если через фильтр проходят все строчки.
|
||||
if (filtered_rows == filter.size())
|
||||
{
|
||||
/// Заменим столбец с фильтром на константу.
|
||||
res.getByPosition(filter_column).column = new ColumnConstUInt8(filtered_rows, 1);
|
||||
/// Остальные столбцы трогать не нужно.
|
||||
return res;
|
||||
}
|
||||
|
||||
/// Фильтруем остальные столбцы.
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
{
|
||||
ColumnWithNameAndType & current_column = res.getByPosition(i);
|
||||
|
||||
if (i == static_cast<size_t>(filter_column))
|
||||
{
|
||||
/// Сам столбец с фильтром заменяем на столбец с константой 1, так как после фильтрации в нём ничего другого не останется.
|
||||
current_column.column = new ColumnConstUInt8(filtered_rows, 1);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (i == first_non_constant_column)
|
||||
continue;
|
||||
|
||||
if (current_column.column->isConst())
|
||||
current_column.column = current_column.column->cut(0, filtered_rows);
|
||||
else
|
||||
current_column.column = current_column.column->filter(filter);
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
@ -91,31 +91,6 @@ void IBlockInputStream::dumpTree(std::ostream & ostr, size_t indent, size_t mult
|
||||
}
|
||||
|
||||
|
||||
void IBlockInputStream::dumpTreeWithProfile(std::ostream & ostr, size_t indent)
|
||||
{
|
||||
ostr << indent + 1 << ". " << getShortName() << "." << std::endl;
|
||||
|
||||
/// Для красоты
|
||||
size_t width = log10(indent + 1) + 4 + getShortName().size();
|
||||
for (size_t i = 0; i < width; ++i)
|
||||
ostr << "─";
|
||||
ostr << std::endl;
|
||||
|
||||
/// Информация профайлинга, если есть
|
||||
if (IProfilingBlockInputStream * profiling = dynamic_cast<IProfilingBlockInputStream *>(this))
|
||||
{
|
||||
if (profiling->getInfo().blocks != 0)
|
||||
{
|
||||
profiling->getInfo().print(ostr);
|
||||
ostr << std::endl;
|
||||
}
|
||||
}
|
||||
|
||||
for (BlockInputStreams::iterator it = children.begin(); it != children.end(); ++it)
|
||||
(*it)->dumpTreeWithProfile(ostr, indent + 1);
|
||||
}
|
||||
|
||||
|
||||
String IBlockInputStream::getShortName() const
|
||||
{
|
||||
String res = getName();
|
||||
|
@ -18,7 +18,7 @@ void BlockStreamProfileInfo::read(ReadBuffer & in)
|
||||
readVarUInt(bytes, in);
|
||||
readBinary(applied_limit, in);
|
||||
readVarUInt(rows_before_limit, in);
|
||||
readBinary(calculated_rows_before_limit, in);
|
||||
readBinary(calculated_rows_before_limit, in);
|
||||
}
|
||||
|
||||
|
||||
@ -46,13 +46,13 @@ bool BlockStreamProfileInfo::hasAppliedLimit() const
|
||||
if (!calculated_rows_before_limit)
|
||||
calculateRowsBeforeLimit();
|
||||
return applied_limit;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void BlockStreamProfileInfo::update(Block & block)
|
||||
{
|
||||
++blocks;
|
||||
rows += block.rows();
|
||||
rows += block.rowsInFirstColumn();
|
||||
bytes += block.bytes();
|
||||
|
||||
if (column_names.empty())
|
||||
@ -76,7 +76,7 @@ void BlockStreamProfileInfo::collectInfosForStreamsWithName(const String & name,
|
||||
void BlockStreamProfileInfo::calculateRowsBeforeLimit() const
|
||||
{
|
||||
calculated_rows_before_limit = true;
|
||||
|
||||
|
||||
/// есть ли Limit?
|
||||
BlockStreamProfileInfos limits;
|
||||
collectInfosForStreamsWithName("Limit", limits);
|
||||
@ -99,63 +99,6 @@ void BlockStreamProfileInfo::calculateRowsBeforeLimit() const
|
||||
}
|
||||
|
||||
|
||||
void BlockStreamProfileInfo::print(std::ostream & ostr) const
|
||||
{
|
||||
UInt64 elapsed = work_stopwatch.elapsed();
|
||||
UInt64 nested_elapsed = 0;
|
||||
double elapsed_seconds = work_stopwatch.elapsedSeconds();
|
||||
double nested_elapsed_seconds = 0;
|
||||
|
||||
UInt64 nested_rows = 0;
|
||||
UInt64 nested_blocks = 0;
|
||||
UInt64 nested_bytes = 0;
|
||||
|
||||
if (!nested_infos.empty())
|
||||
{
|
||||
for (BlockStreamProfileInfos::const_iterator it = nested_infos.begin(); it != nested_infos.end(); ++it)
|
||||
{
|
||||
if ((*it)->work_stopwatch.elapsed() > nested_elapsed)
|
||||
{
|
||||
nested_elapsed = (*it)->work_stopwatch.elapsed();
|
||||
nested_elapsed_seconds = (*it)->work_stopwatch.elapsedSeconds();
|
||||
}
|
||||
|
||||
nested_rows += (*it)->rows;
|
||||
nested_blocks += (*it)->blocks;
|
||||
nested_bytes += (*it)->bytes;
|
||||
}
|
||||
}
|
||||
|
||||
ostr << std::fixed << std::setprecision(2)
|
||||
<< "Columns: " << column_names << std::endl
|
||||
<< "Elapsed: " << elapsed_seconds << " sec. "
|
||||
<< "(" << elapsed * 100.0 / total_stopwatch.elapsed() << "%), " << std::endl;
|
||||
|
||||
if (!nested_infos.empty())
|
||||
{
|
||||
double self_percents = (elapsed - nested_elapsed) * 100.0 / total_stopwatch.elapsed();
|
||||
|
||||
ostr<< "Elapsed (self): " << (elapsed_seconds - nested_elapsed_seconds) << " sec. "
|
||||
<< "(" << (self_percents >= 50 ? "\033[1;31m" : (self_percents >= 10 ? "\033[1;33m" : "")) /// Раскраска больших значений
|
||||
<< self_percents << "%"
|
||||
<< (self_percents >= 10 ? "\033[0m" : "") << "), " << std::endl
|
||||
<< "Rows (in): " << nested_rows << ", per second: " << nested_rows / elapsed_seconds << ", " << std::endl
|
||||
<< "Blocks (in): " << nested_blocks << ", per second: " << nested_blocks / elapsed_seconds << ", " << std::endl
|
||||
<< " " << nested_bytes / 1000000.0 << " MB (memory), "
|
||||
<< nested_bytes * 1000 / elapsed << " MB/s (memory), " << std::endl;
|
||||
|
||||
if (self_percents > 0.1)
|
||||
ostr << "Rows per second (in, self): " << (nested_rows / (elapsed_seconds - nested_elapsed_seconds))
|
||||
<< ", " << (elapsed - nested_elapsed) / nested_rows << " ns/row, " << std::endl;
|
||||
}
|
||||
|
||||
ostr << "Rows (out): " << rows << ", per second: " << rows / elapsed_seconds << ", " << std::endl
|
||||
<< "Blocks (out): " << blocks << ", per second: " << blocks / elapsed_seconds << ", " << std::endl
|
||||
<< " " << bytes / 1000000.0 << " MB (memory), " << bytes * 1000 / elapsed << " MB/s (memory), " << std::endl
|
||||
<< "Average block size (out): " << rows / blocks << "." << std::endl;
|
||||
}
|
||||
|
||||
|
||||
Block IProfilingBlockInputStream::read()
|
||||
{
|
||||
if (!info.started)
|
||||
@ -166,7 +109,7 @@ Block IProfilingBlockInputStream::read()
|
||||
for (BlockInputStreams::const_iterator it = children.begin(); it != children.end(); ++it)
|
||||
if (const IProfilingBlockInputStream * child = dynamic_cast<const IProfilingBlockInputStream *>(&**it))
|
||||
info.nested_infos.push_back(&child->info);
|
||||
|
||||
|
||||
info.started = true;
|
||||
}
|
||||
|
||||
@ -175,9 +118,7 @@ Block IProfilingBlockInputStream::read()
|
||||
if (is_cancelled)
|
||||
return res;
|
||||
|
||||
info.work_stopwatch.start();
|
||||
res = readImpl();
|
||||
info.work_stopwatch.stop();
|
||||
|
||||
/* if (res)
|
||||
{
|
||||
@ -194,7 +135,7 @@ Block IProfilingBlockInputStream::read()
|
||||
std::cerr << ", ";
|
||||
std::cerr << res.getByPosition(i).name << " (" << res.getByPosition(i).column->size() << ")";
|
||||
}
|
||||
|
||||
|
||||
std::cerr << std::endl;
|
||||
}*/
|
||||
|
||||
@ -225,7 +166,7 @@ Block IProfilingBlockInputStream::read()
|
||||
cancel();
|
||||
}
|
||||
|
||||
progress(res.rows(), res.bytes());
|
||||
progress(res.rowsInFirstColumn(), res.bytes());
|
||||
|
||||
return res;
|
||||
}
|
||||
@ -269,7 +210,7 @@ void IProfilingBlockInputStream::updateExtremes(Block & block)
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
{
|
||||
ColumnPtr & column = extremes.getByPosition(i).column;
|
||||
|
||||
|
||||
Field min_value = (*column)[0];
|
||||
Field max_value = (*column)[1];
|
||||
|
||||
@ -330,9 +271,6 @@ bool IProfilingBlockInputStream::checkLimits()
|
||||
|
||||
void IProfilingBlockInputStream::checkQuota(Block & block)
|
||||
{
|
||||
time_t current_time = time(0);
|
||||
double total_elapsed = info.total_stopwatch.elapsedSeconds();
|
||||
|
||||
switch (limits.mode)
|
||||
{
|
||||
case LIMITS_TOTAL:
|
||||
@ -340,15 +278,20 @@ void IProfilingBlockInputStream::checkQuota(Block & block)
|
||||
break;
|
||||
|
||||
case LIMITS_CURRENT:
|
||||
quota->checkAndAddResultRowsBytes(current_time, block.rows(), block.bytes());
|
||||
{
|
||||
time_t current_time = time(0);
|
||||
double total_elapsed = info.total_stopwatch.elapsedSeconds();
|
||||
|
||||
quota->checkAndAddResultRowsBytes(current_time, block.rowsInFirstColumn(), block.bytes());
|
||||
quota->checkAndAddExecutionTime(current_time, Poco::Timespan((total_elapsed - prev_elapsed) * 1000000.0));
|
||||
|
||||
prev_elapsed = total_elapsed;
|
||||
break;
|
||||
}
|
||||
|
||||
default:
|
||||
throw Exception("Logical error: unknown limits mode.", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
prev_elapsed = total_elapsed;
|
||||
}
|
||||
|
||||
|
||||
@ -366,10 +309,9 @@ void IProfilingBlockInputStream::progressImpl(size_t rows, size_t bytes)
|
||||
cancel();
|
||||
|
||||
/// Общее количество данных, обработанных во всех листовых источниках, возможно, на удалённых серверах.
|
||||
|
||||
|
||||
size_t total_rows = process_list_elem->rows_processed;
|
||||
size_t total_bytes = process_list_elem->bytes_processed;
|
||||
double total_elapsed = info.total_stopwatch.elapsedSeconds();
|
||||
|
||||
/** Проверяем ограничения на объём данных для чтения, скорость выполнения запроса, квоту на объём данных для чтения.
|
||||
* NOTE: Может быть, имеет смысл сделать, чтобы они проверялись прямо в ProcessList?
|
||||
@ -389,13 +331,17 @@ void IProfilingBlockInputStream::progressImpl(size_t rows, size_t bytes)
|
||||
throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
if (limits.min_execution_speed
|
||||
&& total_elapsed > limits.timeout_before_checking_execution_speed.totalMicroseconds() / 1000000.0
|
||||
&& total_rows / total_elapsed < limits.min_execution_speed)
|
||||
if (limits.min_execution_speed)
|
||||
{
|
||||
throw Exception("Query is executing too slow: " + toString(total_rows / total_elapsed)
|
||||
+ " rows/sec., minimum: " + toString(limits.min_execution_speed),
|
||||
ErrorCodes::TOO_SLOW);
|
||||
double total_elapsed = info.total_stopwatch.elapsedSeconds();
|
||||
|
||||
if (total_elapsed > limits.timeout_before_checking_execution_speed.totalMicroseconds() / 1000000.0
|
||||
&& total_rows / total_elapsed < limits.min_execution_speed)
|
||||
{
|
||||
throw Exception("Query is executing too slow: " + toString(total_rows / total_elapsed)
|
||||
+ " rows/sec., minimum: " + toString(limits.min_execution_speed),
|
||||
ErrorCodes::TOO_SLOW);
|
||||
}
|
||||
}
|
||||
|
||||
if (quota != nullptr && limits.mode == LIMITS_TOTAL)
|
||||
@ -405,7 +351,7 @@ void IProfilingBlockInputStream::progressImpl(size_t rows, size_t bytes)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
const BlockStreamProfileInfo & IProfilingBlockInputStream::getInfo() const
|
||||
{
|
||||
@ -427,7 +373,7 @@ void IProfilingBlockInputStream::cancel()
|
||||
void IProfilingBlockInputStream::setProgressCallback(ProgressCallback callback)
|
||||
{
|
||||
progress_callback = callback;
|
||||
|
||||
|
||||
for (BlockInputStreams::iterator it = children.begin(); it != children.end(); ++it)
|
||||
if (IProfilingBlockInputStream * child = dynamic_cast<IProfilingBlockInputStream *>(&**it))
|
||||
child->setProgressCallback(callback);
|
||||
|
@ -10,7 +10,7 @@ using Poco::SharedPtr;
|
||||
|
||||
|
||||
JSONRowOutputStream::JSONRowOutputStream(WriteBuffer & ostr_, const Block & sample_)
|
||||
: ostr(ostr_), field_number(0), row_count(0), applied_limit(false), rows_before_limit(0)
|
||||
: dst_ostr(ostr_), ostr(dst_ostr), field_number(0), row_count(0), applied_limit(false), rows_before_limit(0)
|
||||
{
|
||||
NamesAndTypesList columns(sample_.getColumnsList());
|
||||
fields.assign(columns.begin(), columns.end());
|
||||
@ -44,8 +44,6 @@ void JSONRowOutputStream::writePrefix()
|
||||
writeChar('\n', ostr);
|
||||
writeCString("\t\"data\":\n", ostr);
|
||||
writeCString("\t[\n", ostr);
|
||||
|
||||
ostr.next();
|
||||
}
|
||||
|
||||
|
||||
|
@ -18,7 +18,7 @@ Block MergeSortingBlockInputStream::readImpl()
|
||||
|
||||
if (has_been_read)
|
||||
return Block();
|
||||
|
||||
|
||||
has_been_read = true;
|
||||
|
||||
Blocks blocks;
|
||||
@ -38,15 +38,15 @@ Block MergeSortingBlockInputStream::merge(Blocks & blocks)
|
||||
|
||||
if (blocks.size() == 1)
|
||||
return blocks[0];
|
||||
|
||||
|
||||
Stopwatch watch;
|
||||
|
||||
LOG_DEBUG(log, "Merge sorting");
|
||||
|
||||
|
||||
CursorImpls cursors(blocks.size());
|
||||
|
||||
bool has_collation = false;
|
||||
|
||||
|
||||
size_t i = 0;
|
||||
for (Blocks::const_iterator it = blocks.begin(); it != blocks.end(); ++it, ++i)
|
||||
{
|
||||
@ -56,20 +56,22 @@ Block MergeSortingBlockInputStream::merge(Blocks & blocks)
|
||||
cursors[i] = SortCursorImpl(*it, description);
|
||||
has_collation |= cursors[i].has_collation;
|
||||
}
|
||||
|
||||
|
||||
Block merged;
|
||||
|
||||
|
||||
if (has_collation)
|
||||
merged = mergeImpl<SortCursorWithCollation>(blocks, cursors);
|
||||
else
|
||||
merged = mergeImpl<SortCursor>(blocks, cursors);
|
||||
|
||||
|
||||
watch.stop();
|
||||
|
||||
LOG_DEBUG(log, std::fixed << std::setprecision(2)
|
||||
<< "Merge sorted " << blocks.size() << " blocks, " << merged.rows() << " rows"
|
||||
<< " in " << watch.elapsedSeconds() << " sec., "
|
||||
<< merged.rows() / watch.elapsedSeconds() << " rows/sec., "
|
||||
<< merged.bytes() / 1000000.0 / watch.elapsedSeconds() << " MiB/sec.");
|
||||
|
||||
|
||||
return merged;
|
||||
}
|
||||
|
||||
@ -78,13 +80,13 @@ Block MergeSortingBlockInputStream::mergeImpl(Blocks & blocks, CursorImpls & cur
|
||||
{
|
||||
Block merged = blocks[0].cloneEmpty();
|
||||
size_t num_columns = blocks[0].columns();
|
||||
|
||||
|
||||
typedef std::priority_queue<TSortCursor> Queue;
|
||||
Queue queue;
|
||||
|
||||
|
||||
for (size_t i = 0; i < cursors.size(); ++i)
|
||||
queue.push(TSortCursor(&cursors[i]));
|
||||
|
||||
|
||||
ColumnPlainPtrs merged_columns;
|
||||
for (size_t i = 0; i < num_columns; ++i) /// TODO: reserve
|
||||
merged_columns.push_back(&*merged.getByPosition(i).column);
|
||||
|
@ -184,7 +184,7 @@ void MergingSortedBlockInputStream::fetchNextBlock(const TSortCursor & current,
|
||||
void MergingSortedBlockInputStream::readSuffixImpl()
|
||||
{
|
||||
const BlockStreamProfileInfo & profile_info = getInfo();
|
||||
double seconds = profile_info.work_stopwatch.elapsedSeconds();
|
||||
double seconds = profile_info.total_stopwatch.elapsedSeconds();
|
||||
LOG_DEBUG(log, std::fixed << std::setprecision(2)
|
||||
<< "Merge sorted " << profile_info.blocks << " blocks, " << profile_info.rows << " rows"
|
||||
<< " in " << seconds << " sec., "
|
||||
|
@ -140,7 +140,7 @@ void PrettyBlockOutputStream::write(const Block & block_)
|
||||
const ColumnWithNameAndType & col = block.getByPosition(i);
|
||||
|
||||
if (!no_escapes)
|
||||
writeCString("\033[1;37m", ostr);
|
||||
writeCString("\033[1m", ostr);
|
||||
|
||||
if (col.type->isNumeric())
|
||||
{
|
||||
|
@ -29,7 +29,7 @@ void PrettyCompactBlockOutputStream::writeHeader(
|
||||
writeCString("─", ostr);
|
||||
|
||||
if (!no_escapes)
|
||||
writeCString("\033[1;37m", ostr);
|
||||
writeCString("\033[1m", ostr);
|
||||
writeEscapedString(col.name, ostr);
|
||||
if (!no_escapes)
|
||||
writeCString("\033[0m", ostr);
|
||||
@ -37,7 +37,7 @@ void PrettyCompactBlockOutputStream::writeHeader(
|
||||
else
|
||||
{
|
||||
if (!no_escapes)
|
||||
writeCString("\033[1;37m", ostr);
|
||||
writeCString("\033[1m", ostr);
|
||||
writeEscapedString(col.name, ostr);
|
||||
if (!no_escapes)
|
||||
writeCString("\033[0m", ostr);
|
||||
@ -75,7 +75,7 @@ void PrettyCompactBlockOutputStream::writeRow(
|
||||
const Widths_t & name_widths)
|
||||
{
|
||||
size_t columns = max_widths.size();
|
||||
|
||||
|
||||
writeCString("│ ", ostr);
|
||||
|
||||
for (size_t j = 0; j < columns; ++j)
|
||||
@ -90,7 +90,7 @@ void PrettyCompactBlockOutputStream::writeRow(
|
||||
size_t width = get<UInt64>((*block.getByPosition(columns + j).column)[row_id]);
|
||||
for (size_t k = 0; k < max_widths[j] - width; ++k)
|
||||
writeChar(' ', ostr);
|
||||
|
||||
|
||||
col.type->serializeTextEscaped((*col.column)[row_id], ostr);
|
||||
}
|
||||
else
|
||||
@ -113,16 +113,16 @@ void PrettyCompactBlockOutputStream::write(const Block & block_)
|
||||
total_rows += block_.rows();
|
||||
return;
|
||||
}
|
||||
|
||||
|
||||
/// Будем вставлять сюда столбцы с вычисленными значениями видимых длин.
|
||||
Block block = block_;
|
||||
|
||||
|
||||
size_t rows = block.rows();
|
||||
|
||||
Widths_t max_widths;
|
||||
Widths_t name_widths;
|
||||
calculateWidths(block, max_widths, name_widths);
|
||||
|
||||
|
||||
writeHeader(block, max_widths, name_widths);
|
||||
|
||||
for (size_t i = 0; i < rows && total_rows + i < max_rows; ++i)
|
||||
|
@ -17,10 +17,10 @@ void PrettySpaceBlockOutputStream::write(const Block & block_)
|
||||
total_rows += block_.rows();
|
||||
return;
|
||||
}
|
||||
|
||||
|
||||
/// Будем вставлять суда столбцы с вычисленными значениями видимых длин.
|
||||
Block block = block_;
|
||||
|
||||
|
||||
size_t rows = block.rows();
|
||||
size_t columns = block.columns();
|
||||
|
||||
@ -48,7 +48,7 @@ void PrettySpaceBlockOutputStream::write(const Block & block_)
|
||||
writeChar(' ', ostr);
|
||||
|
||||
if (!no_escapes)
|
||||
writeCString("\033[1;37m", ostr);
|
||||
writeCString("\033[1m", ostr);
|
||||
writeEscapedString(col.name, ostr);
|
||||
if (!no_escapes)
|
||||
writeCString("\033[0m", ostr);
|
||||
@ -56,7 +56,7 @@ void PrettySpaceBlockOutputStream::write(const Block & block_)
|
||||
else
|
||||
{
|
||||
if (!no_escapes)
|
||||
writeCString("\033[1;37m", ostr);
|
||||
writeCString("\033[1m", ostr);
|
||||
writeEscapedString(col.name, ostr);
|
||||
if (!no_escapes)
|
||||
writeCString("\033[0m", ostr);
|
||||
@ -81,7 +81,7 @@ void PrettySpaceBlockOutputStream::write(const Block & block_)
|
||||
size_t width = get<UInt64>((*block.getByPosition(columns + j).column)[i]);
|
||||
for (ssize_t k = 0; k < std::max(0L, static_cast<ssize_t>(max_widths[j] - width)); ++k)
|
||||
writeChar(' ', ostr);
|
||||
|
||||
|
||||
col.type->serializeTextEscaped((*col.column)[i], ostr);
|
||||
}
|
||||
else
|
||||
|
@ -1,247 +1,69 @@
|
||||
#include <DB/Functions/FunctionsArithmetic.h>
|
||||
#include <DB/Functions/FunctionsComparison.h>
|
||||
#include <DB/Functions/FunctionsLogical.h>
|
||||
#include <DB/Functions/FunctionsString.h>
|
||||
#include <DB/Functions/FunctionsConversion.h>
|
||||
#include <DB/Functions/FunctionsDateTime.h>
|
||||
#include <DB/Functions/FunctionsStringSearch.h>
|
||||
#include <DB/Functions/FunctionsHashing.h>
|
||||
#include <DB/Functions/FunctionsRandom.h>
|
||||
#include <DB/Functions/FunctionsURL.h>
|
||||
#include <DB/Functions/FunctionsArray.h>
|
||||
#include <DB/Functions/FunctionsStringArray.h>
|
||||
#include <DB/Functions/FunctionsConditional.h>
|
||||
#include <DB/Functions/FunctionsDictionaries.h>
|
||||
#include <DB/Functions/FunctionsMiscellaneous.h>
|
||||
#include <DB/Functions/FunctionsRound.h>
|
||||
#include <DB/Functions/FunctionsReinterpret.h>
|
||||
#include <DB/Functions/FunctionsFormatting.h>
|
||||
#include <DB/Functions/FunctionsCoding.h>
|
||||
#include <DB/Functions/FunctionsHigherOrder.h>
|
||||
#include <DB/Functions/FunctionsVisitParam.h>
|
||||
|
||||
#include <DB/Functions/FunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Эти функции определены в отдельных translation unit-ах.
|
||||
* Это сделано для того, чтобы уменьшить потребление оперативки при сборке, и ускорить параллельную сборку.
|
||||
*/
|
||||
void registerFunctionsArithmetic(FunctionFactory &);
|
||||
void registerFunctionsArray(FunctionFactory &);
|
||||
void registerFunctionsCoding(FunctionFactory &);
|
||||
void registerFunctionsComparison(FunctionFactory &);
|
||||
void registerFunctionsConditional(FunctionFactory &);
|
||||
void registerFunctionsConversion(FunctionFactory &);
|
||||
void registerFunctionsDateTime(FunctionFactory &);
|
||||
void registerFunctionsDictionaries(FunctionFactory &);
|
||||
void registerFunctionsFormatting(FunctionFactory &);
|
||||
void registerFunctionsHashing(FunctionFactory &);
|
||||
void registerFunctionsHigherOrder(FunctionFactory &);
|
||||
void registerFunctionsLogical(FunctionFactory &);
|
||||
void registerFunctionsMiscellaneous(FunctionFactory &);
|
||||
void registerFunctionsRandom(FunctionFactory &);
|
||||
void registerFunctionsReinterpret(FunctionFactory &);
|
||||
void registerFunctionsRound(FunctionFactory &);
|
||||
void registerFunctionsString(FunctionFactory &);
|
||||
void registerFunctionsStringArray(FunctionFactory &);
|
||||
void registerFunctionsStringSearch(FunctionFactory &);
|
||||
void registerFunctionsURL(FunctionFactory &);
|
||||
void registerFunctionsVisitParam(FunctionFactory &);
|
||||
|
||||
|
||||
FunctionFactory::FunctionFactory()
|
||||
{
|
||||
registerFunctionsArithmetic(*this);
|
||||
registerFunctionsArray(*this);
|
||||
registerFunctionsCoding(*this);
|
||||
registerFunctionsComparison(*this);
|
||||
registerFunctionsConditional(*this);
|
||||
registerFunctionsConversion(*this);
|
||||
registerFunctionsDateTime(*this);
|
||||
registerFunctionsDictionaries(*this);
|
||||
registerFunctionsFormatting(*this);
|
||||
registerFunctionsHashing(*this);
|
||||
registerFunctionsHigherOrder(*this);
|
||||
registerFunctionsLogical(*this);
|
||||
registerFunctionsMiscellaneous(*this);
|
||||
registerFunctionsRandom(*this);
|
||||
registerFunctionsReinterpret(*this);
|
||||
registerFunctionsRound(*this);
|
||||
registerFunctionsString(*this);
|
||||
registerFunctionsStringArray(*this);
|
||||
registerFunctionsStringSearch(*this);
|
||||
registerFunctionsURL(*this);
|
||||
registerFunctionsVisitParam(*this);
|
||||
}
|
||||
|
||||
|
||||
|
||||
FunctionPtr FunctionFactory::get(
|
||||
const String & name,
|
||||
const Context & context) const
|
||||
{
|
||||
/// Немного неоптимально.
|
||||
|
||||
if (name == "plus") return new FunctionPlus;
|
||||
else if (name == "minus") return new FunctionMinus;
|
||||
else if (name == "multiply") return new FunctionMultiply;
|
||||
else if (name == "divide") return new FunctionDivideFloating;
|
||||
else if (name == "intDiv") return new FunctionDivideIntegral;
|
||||
else if (name == "modulo") return new FunctionModulo;
|
||||
else if (name == "negate") return new FunctionNegate;
|
||||
else if (name == "bitAnd") return new FunctionBitAnd;
|
||||
else if (name == "bitOr") return new FunctionBitOr;
|
||||
else if (name == "bitXor") return new FunctionBitXor;
|
||||
else if (name == "bitNot") return new FunctionBitNot;
|
||||
else if (name == "bitShiftLeft") return new FunctionBitShiftLeft;
|
||||
else if (name == "bitShiftRight") return new FunctionBitShiftRight;
|
||||
|
||||
else if (name == "equals") return new FunctionEquals;
|
||||
else if (name == "notEquals") return new FunctionNotEquals;
|
||||
else if (name == "less") return new FunctionLess;
|
||||
else if (name == "greater") return new FunctionGreater;
|
||||
else if (name == "lessOrEquals") return new FunctionLessOrEquals;
|
||||
else if (name == "greaterOrEquals") return new FunctionGreaterOrEquals;
|
||||
|
||||
else if (name == "and") return new FunctionAnd;
|
||||
else if (name == "or") return new FunctionOr;
|
||||
else if (name == "xor") return new FunctionXor;
|
||||
else if (name == "not") return new FunctionNot;
|
||||
|
||||
else if (name == "roundToExp2") return new FunctionRoundToExp2;
|
||||
else if (name == "roundDuration") return new FunctionRoundDuration;
|
||||
else if (name == "roundAge") return new FunctionRoundAge;
|
||||
|
||||
else if (name == "empty") return new FunctionEmpty;
|
||||
else if (name == "notEmpty") return new FunctionNotEmpty;
|
||||
else if (name == "length") return new FunctionLength;
|
||||
else if (name == "lengthUTF8") return new FunctionLengthUTF8;
|
||||
else if (name == "lower") return new FunctionLower;
|
||||
else if (name == "upper") return new FunctionUpper;
|
||||
else if (name == "lowerUTF8") return new FunctionLowerUTF8;
|
||||
else if (name == "upperUTF8") return new FunctionUpperUTF8;
|
||||
else if (name == "reverse") return new FunctionReverse;
|
||||
else if (name == "reverseUTF8") return new FunctionReverseUTF8;
|
||||
else if (name == "concat") return new FunctionConcat;
|
||||
else if (name == "substring") return new FunctionSubstring;
|
||||
else if (name == "replaceOne") return new FunctionReplaceOne;
|
||||
else if (name == "replaceAll") return new FunctionReplaceAll;
|
||||
else if (name == "replaceRegexpOne") return new FunctionReplaceRegexpOne;
|
||||
else if (name == "replaceRegexpAll") return new FunctionReplaceRegexpAll;
|
||||
else if (name == "substringUTF8") return new FunctionSubstringUTF8;
|
||||
|
||||
else if (name == "toUInt8") return new FunctionToUInt8;
|
||||
else if (name == "toUInt16") return new FunctionToUInt16;
|
||||
else if (name == "toUInt32") return new FunctionToUInt32;
|
||||
else if (name == "toUInt64") return new FunctionToUInt64;
|
||||
else if (name == "toInt8") return new FunctionToInt8;
|
||||
else if (name == "toInt16") return new FunctionToInt16;
|
||||
else if (name == "toInt32") return new FunctionToInt32;
|
||||
else if (name == "toInt64") return new FunctionToInt64;
|
||||
else if (name == "toFloat32") return new FunctionToFloat32;
|
||||
else if (name == "toFloat64") return new FunctionToFloat64;
|
||||
else if (name == "toDate") return new FunctionToDate;
|
||||
else if (name == "toDateTime") return new FunctionToDateTime;
|
||||
else if (name == "toString") return new FunctionToString;
|
||||
else if (name == "toFixedString") return new FunctionToFixedString;
|
||||
else if (name == "toStringCutToZero") return new FunctionToStringCutToZero;
|
||||
|
||||
else if (name == "reinterpretAsUInt8") return new FunctionReinterpretAsUInt8;
|
||||
else if (name == "reinterpretAsUInt16") return new FunctionReinterpretAsUInt16;
|
||||
else if (name == "reinterpretAsUInt32") return new FunctionReinterpretAsUInt32;
|
||||
else if (name == "reinterpretAsUInt64") return new FunctionReinterpretAsUInt64;
|
||||
else if (name == "reinterpretAsInt8") return new FunctionReinterpretAsInt8;
|
||||
else if (name == "reinterpretAsInt16") return new FunctionReinterpretAsInt16;
|
||||
else if (name == "reinterpretAsInt32") return new FunctionReinterpretAsInt32;
|
||||
else if (name == "reinterpretAsInt64") return new FunctionReinterpretAsInt64;
|
||||
else if (name == "reinterpretAsFloat32") return new FunctionReinterpretAsFloat32;
|
||||
else if (name == "reinterpretAsFloat64") return new FunctionReinterpretAsFloat64;
|
||||
else if (name == "reinterpretAsDate") return new FunctionReinterpretAsDate;
|
||||
else if (name == "reinterpretAsDateTime") return new FunctionReinterpretAsDateTime;
|
||||
else if (name == "reinterpretAsString") return new FunctionReinterpretAsString;
|
||||
|
||||
else if (name == "toYear") return new FunctionToYear;
|
||||
else if (name == "toMonth") return new FunctionToMonth;
|
||||
else if (name == "toDayOfMonth") return new FunctionToDayOfMonth;
|
||||
else if (name == "toDayOfWeek") return new FunctionToDayOfWeek;
|
||||
else if (name == "toHour") return new FunctionToHour;
|
||||
else if (name == "toMinute") return new FunctionToMinute;
|
||||
else if (name == "toSecond") return new FunctionToSecond;
|
||||
else if (name == "toMonday") return new FunctionToMonday;
|
||||
else if (name == "toStartOfMonth") return new FunctionToStartOfMonth;
|
||||
else if (name == "toStartOfQuarter") return new FunctionToStartOfQuarter;
|
||||
else if (name == "toStartOfYear") return new FunctionToStartOfYear;
|
||||
else if (name == "toStartOfMinute") return new FunctionToStartOfMinute;
|
||||
else if (name == "toStartOfHour") return new FunctionToStartOfHour;
|
||||
else if (name == "toRelativeYearNum") return new FunctionToRelativeYearNum;
|
||||
else if (name == "toRelativeMonthNum") return new FunctionToRelativeMonthNum;
|
||||
else if (name == "toRelativeWeekNum") return new FunctionToRelativeWeekNum;
|
||||
else if (name == "toRelativeDayNum") return new FunctionToRelativeDayNum;
|
||||
else if (name == "toRelativeHourNum") return new FunctionToRelativeHourNum;
|
||||
else if (name == "toRelativeMinuteNum") return new FunctionToRelativeMinuteNum;
|
||||
else if (name == "toRelativeSecondNum") return new FunctionToRelativeSecondNum;
|
||||
else if (name == "toTime") return new FunctionToTime;
|
||||
else if (name == "now") return new FunctionNow;
|
||||
else if (name == "timeSlot") return new FunctionTimeSlot;
|
||||
else if (name == "timeSlots") return new FunctionTimeSlots;
|
||||
|
||||
else if (name == "position") return new FunctionPosition;
|
||||
else if (name == "positionUTF8") return new FunctionPositionUTF8;
|
||||
else if (name == "match") return new FunctionMatch;
|
||||
else if (name == "like") return new FunctionLike;
|
||||
else if (name == "notLike") return new FunctionNotLike;
|
||||
else if (name == "extract") return new FunctionExtract;
|
||||
else if (name == "extractAll") return new FunctionExtractAll;
|
||||
|
||||
else if (name == "halfMD5") return new FunctionHalfMD5;
|
||||
else if (name == "sipHash64") return new FunctionSipHash64;
|
||||
else if (name == "cityHash64") return new FunctionCityHash64;
|
||||
else if (name == "intHash32") return new FunctionIntHash32;
|
||||
else if (name == "intHash64") return new FunctionIntHash64;
|
||||
|
||||
else if (name == "IPv4NumToString") return new FunctionIPv4NumToString;
|
||||
else if (name == "IPv4StringToNum") return new FunctionIPv4StringToNum;
|
||||
else if (name == "hex") return new FunctionHex;
|
||||
else if (name == "unhex") return new FunctionUnhex;
|
||||
else if (name == "bitmaskToList") return new FunctionBitmaskToList;
|
||||
else if (name == "bitmaskToArray") return new FunctionBitmaskToArray;
|
||||
|
||||
else if (name == "rand") return new FunctionRand;
|
||||
else if (name == "rand64") return new FunctionRand64;
|
||||
|
||||
else if (name == "protocol") return new FunctionProtocol;
|
||||
else if (name == "domain") return new FunctionDomain;
|
||||
else if (name == "domainWithoutWWW") return new FunctionDomainWithoutWWW;
|
||||
else if (name == "topLevelDomain") return new FunctionTopLevelDomain;
|
||||
else if (name == "path") return new FunctionPath;
|
||||
else if (name == "queryString") return new FunctionQueryString;
|
||||
else if (name == "fragment") return new FunctionFragment;
|
||||
else if (name == "queryStringAndFragment") return new FunctionQueryStringAndFragment;
|
||||
else if (name == "extractURLParameter") return new FunctionExtractURLParameter;
|
||||
else if (name == "extractURLParameters") return new FunctionExtractURLParameters;
|
||||
else if (name == "extractURLParameterNames") return new FunctionExtractURLParameterNames;
|
||||
else if (name == "URLHierarchy") return new FunctionURLHierarchy;
|
||||
else if (name == "URLPathHierarchy") return new FunctionURLPathHierarchy;
|
||||
else if (name == "cutWWW") return new FunctionCutWWW;
|
||||
else if (name == "cutQueryString") return new FunctionCutQueryString;
|
||||
else if (name == "cutFragment") return new FunctionCutFragment;
|
||||
else if (name == "cutQueryStringAndFragment") return new FunctionCutQueryStringAndFragment;
|
||||
else if (name == "cutURLParameter") return new FunctionCutURLParameter;
|
||||
|
||||
else if (name == "hostName") return new FunctionHostName;
|
||||
else if (name == "visibleWidth") return new FunctionVisibleWidth;
|
||||
else if (name == "toTypeName") return new FunctionToTypeName;
|
||||
else if (name == "blockSize") return new FunctionBlockSize;
|
||||
else if (name == "sleep") return new FunctionSleep;
|
||||
else if (name == "materialize") return new FunctionMaterialize;
|
||||
else if (name == "ignore") return new FunctionIgnore;
|
||||
else if (name == "arrayJoin") return new FunctionArrayJoin;
|
||||
|
||||
else if (name == "tuple") return new FunctionTuple;
|
||||
else if (name == "tupleElement") return new FunctionTupleElement;
|
||||
else if (name == "in") return new FunctionIn(false, false);
|
||||
else if (name == "notIn") return new FunctionIn(true, false);
|
||||
else if (name == "globalIn") return new FunctionIn(false, true);
|
||||
else if (name == "globalNotIn") return new FunctionIn(true, true);
|
||||
|
||||
else if (name == "array") return new FunctionArray;
|
||||
else if (name == "arrayElement") return new FunctionArrayElement;
|
||||
else if (name == "has") return new FunctionHas;
|
||||
else if (name == "indexOf") return new FunctionIndexOf;
|
||||
else if (name == "countEqual") return new FunctionCountEqual;
|
||||
else if (name == "arrayEnumerate") return new FunctionArrayEnumerate;
|
||||
else if (name == "arrayEnumerateUniq") return new FunctionArrayEnumerateUniq;
|
||||
|
||||
else if (name == "arrayMap") return new FunctionArrayMap;
|
||||
else if (name == "arrayFilter") return new FunctionArrayFilter;
|
||||
else if (name == "arrayCount") return new FunctionArrayCount;
|
||||
else if (name == "arrayExists") return new FunctionArrayExists;
|
||||
else if (name == "arrayAll") return new FunctionArrayAll;
|
||||
else if (name == "arraySum") return new FunctionArraySum;
|
||||
|
||||
else if (name == "alphaTokens") return new FunctionAlphaTokens;
|
||||
else if (name == "splitByChar") return new FunctionSplitByChar;
|
||||
else if (name == "splitByString") return new FunctionSplitByString;
|
||||
|
||||
else if (name == "if") return new FunctionIf;
|
||||
|
||||
else if (name == "regionToCity") return new FunctionRegionToCity(context.getDictionaries().getRegionsHierarchies());
|
||||
else if (name == "regionToArea") return new FunctionRegionToArea(context.getDictionaries().getRegionsHierarchies());
|
||||
else if (name == "regionToCountry") return new FunctionRegionToCountry(context.getDictionaries().getRegionsHierarchies());
|
||||
else if (name == "regionToContinent") return new FunctionRegionToContinent(context.getDictionaries().getRegionsHierarchies());
|
||||
else if (name == "OSToRoot") return new FunctionOSToRoot(context.getDictionaries().getTechDataHierarchy());
|
||||
else if (name == "SEToRoot") return new FunctionSEToRoot(context.getDictionaries().getTechDataHierarchy());
|
||||
else if (name == "categoryToRoot") return new FunctionCategoryToRoot(context.getDictionaries().getCategoriesHierarchy());
|
||||
else if (name == "categoryToSecondLevel") return new FunctionCategoryToSecondLevel(context.getDictionaries().getCategoriesHierarchy());
|
||||
else if (name == "regionIn") return new FunctionRegionIn(context.getDictionaries().getRegionsHierarchies());
|
||||
else if (name == "OSIn") return new FunctionOSIn(context.getDictionaries().getTechDataHierarchy());
|
||||
else if (name == "SEIn") return new FunctionSEIn(context.getDictionaries().getTechDataHierarchy());
|
||||
else if (name == "categoryIn") return new FunctionCategoryIn(context.getDictionaries().getCategoriesHierarchy());
|
||||
else if (name == "regionHierarchy") return new FunctionRegionHierarchy(context.getDictionaries().getRegionsHierarchies());
|
||||
else if (name == "OSHierarchy") return new FunctionOSHierarchy(context.getDictionaries().getTechDataHierarchy());
|
||||
else if (name == "SEHierarchy") return new FunctionSEHierarchy(context.getDictionaries().getTechDataHierarchy());
|
||||
else if (name == "categoryHierarchy") return new FunctionCategoryHierarchy(context.getDictionaries().getCategoriesHierarchy());
|
||||
else if (name == "regionToName") return new FunctionRegionToName(context.getDictionaries().getRegionsNames());
|
||||
|
||||
else if (name == "visitParamHas") return new FunctionVisitParamHas;
|
||||
else if (name == "visitParamExtractUInt") return new FunctionVisitParamExtractUInt;
|
||||
else if (name == "visitParamExtractInt") return new FunctionVisitParamExtractInt;
|
||||
else if (name == "visitParamExtractFloat") return new FunctionVisitParamExtractFloat;
|
||||
else if (name == "visitParamExtractBool") return new FunctionVisitParamExtractBool;
|
||||
else if (name == "visitParamExtractRaw") return new FunctionVisitParamExtractRaw;
|
||||
else if (name == "visitParamExtractString") return new FunctionVisitParamExtractString;
|
||||
|
||||
auto it = functions.find(name);
|
||||
if (functions.end() != it)
|
||||
return it->second(context);
|
||||
else
|
||||
throw Exception("Unknown function " + name, ErrorCodes::UNKNOWN_FUNCTION);
|
||||
}
|
||||
|
28
dbms/src/Functions/FunctionsArithmetic.cpp
Normal file
28
dbms/src/Functions/FunctionsArithmetic.cpp
Normal file
@ -0,0 +1,28 @@
|
||||
#include <DB/Functions/FunctionFactory.h>
|
||||
#include <DB/Functions/FunctionsArithmetic.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void registerFunctionsArithmetic(FunctionFactory & factory)
|
||||
{
|
||||
#define F [](const Context & context) -> IFunction*
|
||||
|
||||
factory.registerFunction("plus", F { return new FunctionPlus; });
|
||||
factory.registerFunction("minus", F { return new FunctionMinus; });
|
||||
factory.registerFunction("multiply", F { return new FunctionMultiply; });
|
||||
factory.registerFunction("divide", F { return new FunctionDivideFloating; });
|
||||
factory.registerFunction("intDiv", F { return new FunctionDivideIntegral; });
|
||||
factory.registerFunction("modulo", F { return new FunctionModulo; });
|
||||
factory.registerFunction("negate", F { return new FunctionNegate; });
|
||||
factory.registerFunction("bitAnd", F { return new FunctionBitAnd; });
|
||||
factory.registerFunction("bitOr", F { return new FunctionBitOr; });
|
||||
factory.registerFunction("bitXor", F { return new FunctionBitXor; });
|
||||
factory.registerFunction("bitNot", F { return new FunctionBitNot; });
|
||||
factory.registerFunction("bitShiftLeft", F { return new FunctionBitShiftLeft; });
|
||||
factory.registerFunction("bitShiftRight", F { return new FunctionBitShiftRight; });
|
||||
|
||||
#undef F
|
||||
}
|
||||
|
||||
}
|
22
dbms/src/Functions/FunctionsArray.cpp
Normal file
22
dbms/src/Functions/FunctionsArray.cpp
Normal file
@ -0,0 +1,22 @@
|
||||
#include <DB/Functions/FunctionFactory.h>
|
||||
#include <DB/Functions/FunctionsArray.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void registerFunctionsArray(FunctionFactory & factory)
|
||||
{
|
||||
#define F [](const Context & context) -> IFunction*
|
||||
|
||||
factory.registerFunction("array", F { return new FunctionArray; });
|
||||
factory.registerFunction("arrayElement", F { return new FunctionArrayElement; });
|
||||
factory.registerFunction("has", F { return new FunctionHas; });
|
||||
factory.registerFunction("indexOf", F { return new FunctionIndexOf; });
|
||||
factory.registerFunction("countEqual", F { return new FunctionCountEqual; });
|
||||
factory.registerFunction("arrayEnumerate", F { return new FunctionArrayEnumerate; });
|
||||
factory.registerFunction("arrayEnumerateUniq", F { return new FunctionArrayEnumerateUniq; });
|
||||
|
||||
#undef F
|
||||
}
|
||||
|
||||
}
|
21
dbms/src/Functions/FunctionsCoding.cpp
Normal file
21
dbms/src/Functions/FunctionsCoding.cpp
Normal file
@ -0,0 +1,21 @@
|
||||
#include <DB/Functions/FunctionFactory.h>
|
||||
#include <DB/Functions/FunctionsCoding.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void registerFunctionsCoding(FunctionFactory & factory)
|
||||
{
|
||||
#define F [](const Context & context) -> IFunction*
|
||||
|
||||
factory.registerFunction("toStringCutToZero", F { return new FunctionToStringCutToZero; });
|
||||
factory.registerFunction("IPv4NumToString", F { return new FunctionIPv4NumToString; });
|
||||
factory.registerFunction("IPv4StringToNum", F { return new FunctionIPv4StringToNum; });
|
||||
factory.registerFunction("hex", F { return new FunctionHex; });
|
||||
factory.registerFunction("unhex", F { return new FunctionUnhex; });
|
||||
factory.registerFunction("bitmaskToArray", F { return new FunctionBitmaskToArray; });
|
||||
|
||||
#undef F
|
||||
}
|
||||
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user