diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionAny.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionAny.h deleted file mode 100644 index 894ba99d39c..00000000000 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionAny.h +++ /dev/null @@ -1,97 +0,0 @@ -#pragma once - -#include -#include - -#include - - -namespace DB -{ - - -struct AggregateFunctionAnyData -{ - Field value; -}; - - -/// Берёт первое попавшееся значение -class AggregateFunctionAny final : public IUnaryAggregateFunction -{ -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); - } -}; - -} diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionAnyLast.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionAnyLast.h deleted file mode 100644 index a0fecbc08e9..00000000000 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionAnyLast.h +++ /dev/null @@ -1,83 +0,0 @@ -#pragma once - -#include -#include - -#include - - -namespace DB -{ - - -struct AggregateFunctionAnyLastData -{ - Field value; -}; - - -/// Берёт последнее попавшееся значение -class AggregateFunctionAnyLast final : public IUnaryAggregateFunction -{ -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); - } -}; - -} diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionUniqUpTo.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionUniqUpTo.h index 9bff8d8207a..79d122255da 100644 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionUniqUpTo.h +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionUniqUpTo.h @@ -19,8 +19,15 @@ namespace DB template 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(this), sizeof(*this) + limit * sizeof(data[0])); + writeBinary(count, wb); + + /// Пишем значения, только если состояние не переполнено. Иначе они не нужны, а важен только факт того, что состояние переполнено. + if (count <= threshold) + wb.write(reinterpret_cast(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(), params[0]); + UInt64 threshold_param = apply_visitor(FieldVisitorConvertToNumber(), 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 diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionsMinMax.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionsMinMax.h deleted file mode 100644 index c9e08b8580d..00000000000 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionsMinMax.h +++ /dev/null @@ -1,132 +0,0 @@ -#pragma once - -#include -#include - -#include - - -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 -class AggregateFunctionsMinMax final : public IUnaryAggregateFunction > -{ -private: - typedef typename IAggregateFunctionHelper::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 AggregateFunctionMin; -typedef AggregateFunctionsMinMax AggregateFunctionMax; - -} diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionsMinMaxAny.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionsMinMaxAny.h new file mode 100644 index 00000000000..6100ef47827 --- /dev/null +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionsMinMaxAny.h @@ -0,0 +1,480 @@ +#pragma once + +#include +#include + +#include +#include + +#include + + +namespace DB +{ + +/** Агрегатные функции, запоминающие одно какое-либо переданное значение. + * Например, min, max, any, anyLast. + */ + + +/// Для числовых значений. +template +struct SingleValueDataFixed +{ + typedef SingleValueDataFixed Self; + + bool has_value = false; /// Надо запомнить, было ли передано хотя бы одно значение. Это нужно для AggregateFunctionIf. + T value; + + + bool has() const + { + return has_value; + } + + void insertResultInto(IColumn & to) const + { + if (has()) + static_cast &>(to).getData().push_back(value); + else + static_cast &>(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 &>(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 &>(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 &>(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(to).insertDataWithTerminatingZero(getData(), size); + else + static_cast(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(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(malloc(value.size)); + } + + size = value_size; + memcpy(large_data, value.data, size); + } + } + + void change(const IColumn & column, size_t row_num) + { + changeImpl(static_cast(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(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(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 +struct AggregateFunctionMinData : Data +{ + typedef AggregateFunctionMinData 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 +struct AggregateFunctionMaxData : Data +{ + typedef AggregateFunctionMaxData 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 +struct AggregateFunctionAnyData : Data +{ + typedef AggregateFunctionAnyData 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 +struct AggregateFunctionAnyLastData : Data +{ + typedef AggregateFunctionAnyLastData 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 +class AggregateFunctionsSingleValue final : public IUnaryAggregateFunction > +{ +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); + } +}; + +} diff --git a/dbms/include/DB/Client/Connection.h b/dbms/include/DB/Client/Connection.h index 4dbfbb17889..9f179382cc5 100644 --- a/dbms/include/DB/Client/Connection.h +++ b/dbms/include/DB/Client/Connection.h @@ -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); diff --git a/dbms/include/DB/Columns/ColumnArray.h b/dbms/include/DB/Columns/ColumnArray.h index 83a51066ee5..3950488d3a7 100644 --- a/dbms/include/DB/Columns/ColumnArray.h +++ b/dbms/include/DB/Columns/ColumnArray.h @@ -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(*res); const typename ColumnVector::Container_t & cur_data = typeid_cast &>(*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(*res); const ColumnString & cur_string = typeid_cast(*data); diff --git a/dbms/include/DB/Columns/ColumnConst.h b/dbms/include/DB/Columns/ColumnConst.h index d6b642839ec..a0031c7070e 100644 --- a/dbms/include/DB/Columns/ColumnConst.h +++ b/dbms/include/DB/Columns/ColumnConst.h @@ -32,11 +32,11 @@ class ColumnConst final : public IColumnConst public: typedef T Type; typedef typename NearestFieldType::Type FieldType; - + /// Для ColumnConst data_type_ должен быть ненулевым. /// Для ColumnConst 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::get() + ">"; } bool isNumeric() const { return IsNumber::value; } bool isFixed() const { return IsNumber::value; } @@ -50,7 +50,7 @@ public: { return new ColumnConst(length, data, data_type); } - + void insert(const Field & x) { if (x.get() != 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(new_size, data, data_type); + + return new ColumnConst(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(offsets.back(), data, data_type); + size_t replicated_size = 0 == s ? 0 : offsets.back(); + return new ColumnConst(replicated_size, data, data_type); } size_t byteSize() const { return sizeof(data) + sizeof(s); } diff --git a/dbms/include/DB/Columns/ColumnFixedString.h b/dbms/include/DB/Columns/ColumnFixedString.h index c9a3ff45d94..c18af6e21da 100644 --- a/dbms/include/DB/Columns/ColumnFixedString.h +++ b/dbms/include/DB/Columns/ColumnFixedString.h @@ -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(&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()); diff --git a/dbms/include/DB/Columns/ColumnString.h b/dbms/include/DB/Columns/ColumnString.h index 1d7253a1479..91689a384f4 100644 --- a/dbms/include/DB/Columns/ColumnString.h +++ b/dbms/include/DB/Columns/ColumnString.h @@ -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(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(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(&chars[offsetAt(n)]), reinterpret_cast(&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(rhs_); - + return collator.compare( reinterpret_cast(&chars[offsetAt(n)]), sizeAt(n), reinterpret_cast(&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()); diff --git a/dbms/include/DB/Columns/ColumnVector.h b/dbms/include/DB/Columns/ColumnVector.h index 1faa5eaf662..e6473d4a6a1 100644 --- a/dbms/include/DB/Columns/ColumnVector.h +++ b/dbms/include/DB/Columns/ColumnVector.h @@ -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(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(); diff --git a/dbms/include/DB/Columns/IColumn.h b/dbms/include/DB/Columns/IColumn.h index cab7bf1a645..8a448e4bb01 100644 --- a/dbms/include/DB/Columns/IColumn.h +++ b/dbms/include/DB/Columns/IColumn.h @@ -198,4 +198,8 @@ public: }; +/// Считает, сколько байт в filt больше нуля. +size_t countBytesInFilter(const IColumn::Filter & filt); + + } diff --git a/dbms/include/DB/Columns/IColumnDummy.h b/dbms/include/DB/Columns/IColumnDummy.h index ffa6a8742c5..1ab49024fc8 100644 --- a/dbms/include/DB/Columns/IColumnDummy.h +++ b/dbms/include/DB/Columns/IColumnDummy.h @@ -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; }; - + } diff --git a/dbms/include/DB/Common/FileChecker.h b/dbms/include/DB/Common/FileChecker.h new file mode 100644 index 00000000000..8d9f1b1a741 --- /dev/null +++ b/dbms/include/DB/Common/FileChecker.h @@ -0,0 +1,95 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +/// хранит размеры всех столбцов, и может проверять не побились ли столбцы +template +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; + + 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().get("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; +}; +} diff --git a/dbms/include/DB/Common/Macros.h b/dbms/include/DB/Common/Macros.h new file mode 100644 index 00000000000..2855eb6b750 --- /dev/null +++ b/dbms/include/DB/Common/Macros.h @@ -0,0 +1,27 @@ +#pragma once + +#include +#include +#include + +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 MacroMap; + + MacroMap macros; +}; + +} diff --git a/dbms/include/DB/Common/PODArray.h b/dbms/include/DB/Common/PODArray.h index 93fe1ad839f..37fabed0a0d 100644 --- a/dbms/include/DB/Common/PODArray.h +++ b/dbms/include/DB/Common/PODArray.h @@ -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 + 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(&*from_begin), bytes_to_copy); c_end += bytes_to_copy; diff --git a/dbms/include/DB/Common/Volnitsky.h b/dbms/include/DB/Common/Volnitsky.h index 77de66e0f39..c6b5fb8ac5c 100644 --- a/dbms/include/DB/Common/Volnitsky.h +++ b/dbms/include/DB/Common/Volnitsky.h @@ -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) diff --git a/dbms/include/DB/Core/Defines.h b/dbms/include/DB/Core/Defines.h index a253d6ea740..7599036d248 100644 --- a/dbms/include/DB/Core/Defines.h +++ b/dbms/include/DB/Core/Defines.h @@ -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 diff --git a/dbms/include/DB/Core/ErrorCodes.h b/dbms/include/DB/Core/ErrorCodes.h index 72b8f4cc309..ac69ce20aaa 100644 --- a/dbms/include/DB/Core/ErrorCodes.h +++ b/dbms/include/DB/Core/ErrorCodes.h @@ -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, diff --git a/dbms/include/DB/Core/Field.h b/dbms/include/DB/Core/Field.h index 48558f9717e..7bf0fe6e288 100644 --- a/dbms/include/DB/Core/Field.h +++ b/dbms/include/DB/Core/Field.h @@ -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); } } diff --git a/dbms/include/DB/Core/StringRef.h b/dbms/include/DB/Core/StringRef.h index 0f0e21eb481..76cfc676805 100644 --- a/dbms/include/DB/Core/StringRef.h +++ b/dbms/include/DB/Core/StringRef.h @@ -26,7 +26,7 @@ struct StringRef typedef std::vector 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); } diff --git a/dbms/include/DB/DataStreams/AddingDefaultBlockOutputStream.h b/dbms/include/DB/DataStreams/AddingDefaultBlockOutputStream.h index 643a99270c4..c3a49cece4c 100644 --- a/dbms/include/DB/DataStreams/AddingDefaultBlockOutputStream.h +++ b/dbms/include/DB/DataStreams/AddingDefaultBlockOutputStream.h @@ -33,6 +33,8 @@ public: output->write(res); } + void flush() { output->flush(); } + private: BlockOutputStreamPtr output; NamesAndTypesListPtr required_columns; diff --git a/dbms/include/DB/DataStreams/BinaryRowOutputStream.h b/dbms/include/DB/DataStreams/BinaryRowOutputStream.h index df8ab4a4a99..09f4383437e 100644 --- a/dbms/include/DB/DataStreams/BinaryRowOutputStream.h +++ b/dbms/include/DB/DataStreams/BinaryRowOutputStream.h @@ -20,6 +20,8 @@ public: void writeField(const Field & field); void writeRowEndDelimiter(); + void flush() { ostr.next(); } + protected: WriteBuffer & ostr; const Block sample; diff --git a/dbms/include/DB/DataStreams/BlockOutputStreamFromRowOutputStream.h b/dbms/include/DB/DataStreams/BlockOutputStreamFromRowOutputStream.h index 70213b5d2d4..e6d15c7b323 100644 --- a/dbms/include/DB/DataStreams/BlockOutputStreamFromRowOutputStream.h +++ b/dbms/include/DB/DataStreams/BlockOutputStreamFromRowOutputStream.h @@ -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); diff --git a/dbms/include/DB/DataStreams/IBlockInputStream.h b/dbms/include/DB/DataStreams/IBlockInputStream.h index 10887a4ee66..a63c3fee017 100644 --- a/dbms/include/DB/DataStreams/IBlockInputStream.h +++ b/dbms/include/DB/DataStreams/IBlockInputStream.h @@ -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(); diff --git a/dbms/include/DB/DataStreams/IBlockOutputStream.h b/dbms/include/DB/DataStreams/IBlockOutputStream.h index 72769711008..ad19e50d4ba 100644 --- a/dbms/include/DB/DataStreams/IBlockOutputStream.h +++ b/dbms/include/DB/DataStreams/IBlockOutputStream.h @@ -31,6 +31,10 @@ public: */ virtual void writePrefix() {} virtual void writeSuffix() {} + + /** Сбросить имеющиеся буферы для записи. + */ + virtual void flush() {} /** Методы для установки дополнительной информации для вывода в поддерживающих её форматах. */ diff --git a/dbms/include/DB/DataStreams/IProfilingBlockInputStream.h b/dbms/include/DB/DataStreams/IProfilingBlockInputStream.h index 0e0d0248551..7487503452c 100644 --- a/dbms/include/DB/DataStreams/IProfilingBlockInputStream.h +++ b/dbms/include/DB/DataStreams/IProfilingBlockInputStream.h @@ -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); diff --git a/dbms/include/DB/DataStreams/IRowOutputStream.h b/dbms/include/DB/DataStreams/IRowOutputStream.h index d58fb50ae64..eec6c7a9201 100644 --- a/dbms/include/DB/DataStreams/IRowOutputStream.h +++ b/dbms/include/DB/DataStreams/IRowOutputStream.h @@ -32,6 +32,9 @@ public: virtual void writePrefix() {}; /// разделитель перед началом результата virtual void writeSuffix() {}; /// разделитель после конца результата + /** Сбросить имеющиеся буферы для записи. */ + virtual void flush() {} + /** Методы для установки дополнительной информации для вывода в поддерживающих её форматах. */ virtual void setRowsBeforeLimit(size_t rows_before_limit) {} diff --git a/dbms/include/DB/DataStreams/JSONRowOutputStream.h b/dbms/include/DB/DataStreams/JSONRowOutputStream.h index 224dbbadf76..b96dde4c3ae 100644 --- a/dbms/include/DB/DataStreams/JSONRowOutputStream.h +++ b/dbms/include/DB/DataStreams/JSONRowOutputStream.h @@ -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; diff --git a/dbms/include/DB/DataStreams/NativeBlockOutputStream.h b/dbms/include/DB/DataStreams/NativeBlockOutputStream.h index eb47f2d88a5..ae11058f664 100644 --- a/dbms/include/DB/DataStreams/NativeBlockOutputStream.h +++ b/dbms/include/DB/DataStreams/NativeBlockOutputStream.h @@ -15,6 +15,8 @@ public: NativeBlockOutputStream(WriteBuffer & ostr_) : ostr(ostr_) {} void write(const Block & block); + void flush() { ostr.next(); } + private: WriteBuffer & ostr; }; diff --git a/dbms/include/DB/DataStreams/PrettyBlockOutputStream.h b/dbms/include/DB/DataStreams/PrettyBlockOutputStream.h index 65c8d03050f..eaa820dc170 100644 --- a/dbms/include/DB/DataStreams/PrettyBlockOutputStream.h +++ b/dbms/include/DB/DataStreams/PrettyBlockOutputStream.h @@ -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_; } diff --git a/dbms/include/DB/DataStreams/PushingToViewsBlockOutputStream.h b/dbms/include/DB/DataStreams/PushingToViewsBlockOutputStream.h index c22a4c258f7..58e9264c91f 100644 --- a/dbms/include/DB/DataStreams/PushingToViewsBlockOutputStream.h +++ b/dbms/include/DB/DataStreams/PushingToViewsBlockOutputStream.h @@ -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(); diff --git a/dbms/include/DB/DataStreams/RemoteBlockOutputStream.h b/dbms/include/DB/DataStreams/RemoteBlockOutputStream.h index 712852b4941..d4087429770 100644 --- a/dbms/include/DB/DataStreams/RemoteBlockOutputStream.h +++ b/dbms/include/DB/DataStreams/RemoteBlockOutputStream.h @@ -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; diff --git a/dbms/include/DB/DataStreams/TabSeparatedBlockOutputStream.h b/dbms/include/DB/DataStreams/TabSeparatedBlockOutputStream.h index 9bba296b7d5..05d801362f9 100644 --- a/dbms/include/DB/DataStreams/TabSeparatedBlockOutputStream.h +++ b/dbms/include/DB/DataStreams/TabSeparatedBlockOutputStream.h @@ -16,6 +16,8 @@ public: TabSeparatedBlockOutputStream(WriteBuffer & ostr_) : ostr(ostr_) {} void write(const Block & block); + void flush() { ostr.next(); } + private: WriteBuffer & ostr; }; diff --git a/dbms/include/DB/DataStreams/TabSeparatedRowOutputStream.h b/dbms/include/DB/DataStreams/TabSeparatedRowOutputStream.h index 46056042d53..60681d56ed1 100644 --- a/dbms/include/DB/DataStreams/TabSeparatedRowOutputStream.h +++ b/dbms/include/DB/DataStreams/TabSeparatedRowOutputStream.h @@ -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_; } diff --git a/dbms/include/DB/DataStreams/ValuesRowOutputStream.h b/dbms/include/DB/DataStreams/ValuesRowOutputStream.h index ba632b62fdb..4cfd5d74c92 100644 --- a/dbms/include/DB/DataStreams/ValuesRowOutputStream.h +++ b/dbms/include/DB/DataStreams/ValuesRowOutputStream.h @@ -26,6 +26,8 @@ public: void writeRowEndDelimiter(); void writeRowBetweenDelimiter(); + void flush() { ostr.next(); } + private: WriteBuffer & ostr; const Block sample; diff --git a/dbms/include/DB/DataStreams/VerticalRowOutputStream.h b/dbms/include/DB/DataStreams/VerticalRowOutputStream.h index dbd9cbb5672..08a9c83145f 100644 --- a/dbms/include/DB/DataStreams/VerticalRowOutputStream.h +++ b/dbms/include/DB/DataStreams/VerticalRowOutputStream.h @@ -25,6 +25,8 @@ public: void writeRowStartDelimiter(); void writeRowBetweenDelimiter(); + void flush() { ostr.next(); } + private: WriteBuffer & ostr; const Block sample; diff --git a/dbms/include/DB/Functions/FunctionFactory.h b/dbms/include/DB/Functions/FunctionFactory.h index c154fe17e6b..5900365995a 100644 --- a/dbms/include/DB/Functions/FunctionFactory.h +++ b/dbms/include/DB/Functions/FunctionFactory.h @@ -1,7 +1,6 @@ #pragma once #include - #include @@ -17,10 +16,19 @@ class Context; */ class FunctionFactory { +private: + typedef IFunction* (*Creator)(const Context & context); /// Не std::function, так как меньше indirection и размер объекта. + std::unordered_map 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; + } }; } diff --git a/dbms/include/DB/Functions/FunctionsArithmetic.h b/dbms/include/DB/Functions/FunctionsArithmetic.h index 7996d8f5818..eed06e4135a 100644 --- a/dbms/include/DB/Functions/FunctionsArithmetic.h +++ b/dbms/include/DB/Functions/FunctionsArithmetic.h @@ -14,7 +14,7 @@ namespace DB */ template -struct BinaryOperationImpl +struct BinaryOperationImplBase { typedef typename Op::ResultType ResultType; @@ -45,6 +45,11 @@ struct BinaryOperationImpl } }; +template +struct BinaryOperationImpl : BinaryOperationImplBase +{ +}; + template struct UnaryOperationImpl { @@ -221,7 +226,7 @@ struct NegateImpl static inline ResultType apply(A a) { - return -a; + return -static_cast(a); } }; @@ -570,4 +575,140 @@ typedef FunctionBinaryArithmetic Functi +/// Оптимизации для целочисленного деления на константу. + +#define LIBDIVIDE_USE_SSE2 1 +#include + + +template +struct DivideIntegralByConstantImpl + : BinaryOperationImplBase> +{ + typedef typename DivideIntegralImpl::ResultType ResultType; + + static void vector_constant(const PODArray & a, B b, PODArray & 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::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 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(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 +struct ModuloByConstantImpl + : BinaryOperationImplBase> +{ + typedef typename ModuloImpl::ResultType ResultType; + + static void vector_constant(const PODArray & a, B b, PODArray & 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::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 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> : DivideIntegralByConstantImpl {}; +template <> struct BinaryOperationImpl> : DivideIntegralByConstantImpl {}; +template <> struct BinaryOperationImpl> : DivideIntegralByConstantImpl {}; +template <> struct BinaryOperationImpl> : DivideIntegralByConstantImpl {}; + +template <> struct BinaryOperationImpl> : DivideIntegralByConstantImpl {}; +template <> struct BinaryOperationImpl> : DivideIntegralByConstantImpl {}; +template <> struct BinaryOperationImpl> : DivideIntegralByConstantImpl {}; +template <> struct BinaryOperationImpl> : DivideIntegralByConstantImpl {}; + +template <> struct BinaryOperationImpl> : DivideIntegralByConstantImpl {}; +template <> struct BinaryOperationImpl> : DivideIntegralByConstantImpl {}; +template <> struct BinaryOperationImpl> : DivideIntegralByConstantImpl {}; +template <> struct BinaryOperationImpl> : DivideIntegralByConstantImpl {}; + +template <> struct BinaryOperationImpl> : DivideIntegralByConstantImpl {}; +template <> struct BinaryOperationImpl> : DivideIntegralByConstantImpl {}; +template <> struct BinaryOperationImpl> : DivideIntegralByConstantImpl {}; +template <> struct BinaryOperationImpl> : DivideIntegralByConstantImpl {}; + + +template <> struct BinaryOperationImpl> : ModuloByConstantImpl {}; +template <> struct BinaryOperationImpl> : ModuloByConstantImpl {}; +template <> struct BinaryOperationImpl> : ModuloByConstantImpl {}; +template <> struct BinaryOperationImpl> : ModuloByConstantImpl {}; + +template <> struct BinaryOperationImpl> : ModuloByConstantImpl {}; +template <> struct BinaryOperationImpl> : ModuloByConstantImpl {}; +template <> struct BinaryOperationImpl> : ModuloByConstantImpl {}; +template <> struct BinaryOperationImpl> : ModuloByConstantImpl {}; + +template <> struct BinaryOperationImpl> : ModuloByConstantImpl {}; +template <> struct BinaryOperationImpl> : ModuloByConstantImpl {}; +template <> struct BinaryOperationImpl> : ModuloByConstantImpl {}; +template <> struct BinaryOperationImpl> : ModuloByConstantImpl {}; + +template <> struct BinaryOperationImpl> : ModuloByConstantImpl {}; +template <> struct BinaryOperationImpl> : ModuloByConstantImpl {}; +template <> struct BinaryOperationImpl> : ModuloByConstantImpl {}; +template <> struct BinaryOperationImpl> : ModuloByConstantImpl {}; + } diff --git a/dbms/include/DB/Functions/FunctionsCoding.h b/dbms/include/DB/Functions/FunctionsCoding.h index 05494c80bab..168f8a10a18 100644 --- a/dbms/include/DB/Functions/FunctionsCoding.h +++ b/dbms/include/DB/Functions/FunctionsCoding.h @@ -4,10 +4,12 @@ #include #include #include +#include #include #include #include #include +#include #include #include @@ -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; diff --git a/dbms/include/DB/Functions/FunctionsComparison.h b/dbms/include/DB/Functions/FunctionsComparison.h index 52fac72f64c..adeee4a6d17 100644 --- a/dbms/include/DB/Functions/FunctionsComparison.h +++ b/dbms/include/DB/Functions/FunctionsComparison.h @@ -34,37 +34,81 @@ namespace DB #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wsign-compare" -template -struct EqualsNumImpl +template struct EqualsOp { static UInt8 apply(A a, B b) { return a == b; } }; +template struct NotEqualsOp { static UInt8 apply(A a, B b) { return a != b; } }; +template struct LessOp { static UInt8 apply(A a, B b) { return a < b; } }; +template struct GreaterOp { static UInt8 apply(A a, B b) { return a > b; } }; +template struct LessOrEqualsOp { static UInt8 apply(A a, B b) { return a <= b; } }; +template struct GreaterOrEqualsOp { static UInt8 apply(A a, B b) { return a >= b; } }; + +#pragma GCC diagnostic pop + + + +template +struct NumComparisonImpl { static void vector_vector(const PODArray & a, const PODArray & b, PODArray & c) { + /** GCC 4.8.2 векторизует цикл только если его записать в такой форме. + * В данном случае, если сделать цикл по индексу массива (код будет выглядеть проще), + * цикл не будет векторизовываться. + */ + size_t size = a.size(); - for (size_t i = 0; i < size; ++i) - c[i] = a[i] == b[i]; + const A * a_pos = &a[0]; + const B * b_pos = &b[0]; + UInt8 * c_pos = &c[0]; + const A * a_end = a_pos + size; + + while (a_pos < a_end) + { + *c_pos = Op::apply(*a_pos, *b_pos); + ++a_pos; + ++b_pos; + ++c_pos; + } } static void vector_constant(const PODArray & a, B b, PODArray & c) { size_t size = a.size(); - for (size_t i = 0; i < size; ++i) - c[i] = a[i] == b; + const A * a_pos = &a[0]; + UInt8 * c_pos = &c[0]; + const A * a_end = a_pos + size; + + while (a_pos < a_end) + { + *c_pos = Op::apply(*a_pos, b); + ++a_pos; + ++c_pos; + } } static void constant_vector(A a, const PODArray & b, PODArray & c) { size_t size = b.size(); - for (size_t i = 0; i < size; ++i) - c[i] = a == b[i]; + const B * b_pos = &b[0]; + UInt8 * c_pos = &c[0]; + const B * b_end = b_pos + size; + + while (b_pos < b_end) + { + *c_pos = Op::apply(a, *b_pos); + ++b_pos; + ++c_pos; + } } static void constant_constant(A a, B b, UInt8 & c) { - c = a == b; + c = Op::apply(a, b); } }; -struct EqualsStringImpl + +template +struct StringComparisonImpl { static void string_vector_string_vector( const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_offsets, @@ -73,10 +117,18 @@ struct EqualsStringImpl { size_t size = a_offsets.size(); for (size_t i = 0; i < size; ++i) - c[i] = (i == 0) - ? (a_offsets[0] == b_offsets[0] && !memcmp(&a_data[0], &b_data[0], a_offsets[0] - 1)) - : (a_offsets[i] - a_offsets[i - 1] == b_offsets[i] - b_offsets[i - 1] - && !memcmp(&a_data[a_offsets[i - 1]], &b_data[b_offsets[i - 1]], a_offsets[i] - a_offsets[i - 1] - 1)); + { + if (i == 0) + { + /// Завершающий ноль в меньшей по длине строке входит в сравнение. + c[i] = Op::apply(memcmp(&a_data[0], &b_data[0], std::min(a_offsets[0], b_offsets[0])), 0); + } + else + { + c[i] = Op::apply(memcmp(&a_data[a_offsets[i - 1]], &b_data[b_offsets[i - 1]], + std::min(a_offsets[i] - a_offsets[i - 1], b_offsets[i] - b_offsets[i - 1])), 0); + } + } } static void string_vector_fixed_string_vector( @@ -86,10 +138,19 @@ struct EqualsStringImpl { size_t size = a_offsets.size(); for (size_t i = 0; i < size; ++i) - c[i] = (i == 0) - ? (a_offsets[0] == b_n + 1 && !memcmp(&a_data[0], &b_data[0], b_n)) - : (a_offsets[i] - a_offsets[i - 1] == b_n + 1 - && !memcmp(&a_data[a_offsets[i - 1]], &b_data[b_n * i], b_n)); + { + if (i == 0) + { + int res = memcmp(&a_data[0], &b_data[0], std::min(a_offsets[0] - 1, b_n)); + c[i] = Op::apply(res, 0) || (res == 0 && Op::apply(a_offsets[0], b_n + 1)); + } + else + { + int res = memcmp(&a_data[a_offsets[i - 1]], &b_data[i * b_n], + std::min(a_offsets[i] - a_offsets[i - 1] - 1, b_n)); + c[i] = Op::apply(res, 0) || (res == 0 && Op::apply(a_offsets[i] - a_offsets[i - 1], b_n + 1)); + } + } } static void string_vector_constant( @@ -101,10 +162,17 @@ struct EqualsStringImpl ColumnString::Offset_t b_n = b.size(); const UInt8 * b_data = reinterpret_cast(b.data()); for (size_t i = 0; i < size; ++i) - c[i] = (i == 0) - ? (a_offsets[0] == b_n + 1 && !memcmp(&a_data[0], b_data, b_n)) - : (a_offsets[i] - a_offsets[i - 1] == b_n + 1 - && !memcmp(&a_data[a_offsets[i - 1]], b_data, b_n)); + { + if (i == 0) + { + c[i] = Op::apply(memcmp(&a_data[0], b_data, std::min(a_offsets[0], b_n + 1)), 0); + } + else + { + c[i] = Op::apply(memcmp(&a_data[a_offsets[i - 1]], b_data, + std::min(a_offsets[i] - a_offsets[i - 1], b_n + 1)), 0); + } + } } static void fixed_string_vector_string_vector( @@ -114,10 +182,19 @@ struct EqualsStringImpl { size_t size = b_offsets.size(); for (size_t i = 0; i < size; ++i) - c[i] = (i == 0) - ? (b_offsets[0] == a_n + 1 && !memcmp(&b_data[0], &a_data[0], a_n)) - : (b_offsets[i] - b_offsets[i - 1] == a_n + 1 - && !memcmp(&b_data[b_offsets[i - 1]], &a_data[a_n * i], a_n)); + { + if (i == 0) + { + int res = memcmp(&a_data[0], &b_data[0], std::min(b_offsets[0] - 1, a_n)); + c[i] = Op::apply(res, 0) || (res == 0 && Op::apply(a_n + 1, b_offsets[0])); + } + else + { + int res = memcmp(&a_data[i * a_n], &b_data[b_offsets[i - 1]], + std::min(b_offsets[i] - b_offsets[i - 1] - 1, a_n)); + c[i] = Op::apply(res, 0) || (res == 0 && Op::apply(a_n + 1, b_offsets[i] - b_offsets[i - 1])); + } + } } static void fixed_string_vector_fixed_string_vector( @@ -127,7 +204,10 @@ struct EqualsStringImpl { size_t size = a_data.size(); for (size_t i = 0, j = 0; i < size; i += a_n, ++j) - c[j] = a_n == b_n && !memcmp(&a_data[i], &b_data[i], a_n); + { + int res = memcmp(&a_data[i], &b_data[i], std::min(a_n, b_n)); + c[j] = Op::apply(res, 0) || (res == 0 && Op::apply(a_n, b_n)); + } } static void fixed_string_vector_constant( @@ -139,7 +219,10 @@ struct EqualsStringImpl const UInt8 * b_data = reinterpret_cast(b.data()); ColumnString::Offset_t b_n = b.size(); for (size_t i = 0, j = 0; i < size; i += a_n, ++j) - c[j] = a_n == b_n && !memcmp(&a_data[i], b_data, a_n); + { + int res = memcmp(&a_data[i], b_data, std::min(a_n, b_n)); + c[j] = Op::apply(res, 0) || (res == 0 && Op::apply(a_n, b_n)); + } } static void constant_string_vector( @@ -151,10 +234,17 @@ struct EqualsStringImpl ColumnString::Offset_t a_n = a.size(); const UInt8 * a_data = reinterpret_cast(a.data()); for (size_t i = 0; i < size; ++i) - c[i] = (i == 0) - ? (b_offsets[0] == a_n + 1 && !memcmp(&b_data[0], a_data, a_n)) - : (b_offsets[i] - b_offsets[i - 1] == a_n + 1 - && !memcmp(&b_data[b_offsets[i - 1]], a_data, a_n)); + { + if (i == 0) + { + c[i] = Op::apply(memcmp(a_data, &b_data[0], std::min(b_offsets[0], a_n + 1)), 0); + } + else + { + c[i] = Op::apply(memcmp(a_data, &b_data[b_offsets[i - 1]], + std::min(b_offsets[i] - b_offsets[i - 1], a_n + 1)), 0); + } + } } static void constant_fixed_string_vector( @@ -166,7 +256,10 @@ struct EqualsStringImpl const UInt8 * a_data = reinterpret_cast(a.data()); ColumnString::Offset_t a_n = a.size(); for (size_t i = 0, j = 0; i < size; i += b_n, ++j) - c[j] = a_n == b_n && !memcmp(&b_data[i], a_data, b_n); + { + int res = memcmp(a_data, &b_data[i], std::min(a_n, b_n)); + c[j] = Op::apply(res, 0) || (res == 0 && Op::apply(b_n, a_n)); + } } static void constant_constant( @@ -174,41 +267,14 @@ struct EqualsStringImpl const std::string & b, UInt8 & c) { - c = a == b; + c = Op::apply(memcmp(a.data(), b.data(), std::min(a.size(), b.size()) + 1), 0); } }; -template -struct NotEqualsNumImpl -{ - static void vector_vector(const PODArray & a, const PODArray & b, PODArray & c) - { - size_t size = a.size(); - for (size_t i = 0; i < size; ++i) - c[i] = a[i] != b[i]; - } - static void vector_constant(const PODArray & a, B b, PODArray & c) - { - size_t size = a.size(); - for (size_t i = 0; i < size; ++i) - c[i] = a[i] != b; - } - - static void constant_vector(A a, const PODArray & b, PODArray & c) - { - size_t size = b.size(); - for (size_t i = 0; i < size; ++i) - c[i] = a != b[i]; - } - - static void constant_constant(A a, B b, UInt8 & c) - { - c = a != b; - } -}; - -struct NotEqualsStringImpl +/// Сравнения на равенство/неравенство реализованы несколько более эффективно. +template +struct StringEqualsImpl { static void string_vector_string_vector( const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_offsets, @@ -217,7 +283,7 @@ struct NotEqualsStringImpl { size_t size = a_offsets.size(); for (size_t i = 0; i < size; ++i) - c[i] = !((i == 0) + c[i] = positive == ((i == 0) ? (a_offsets[0] == b_offsets[0] && !memcmp(&a_data[0], &b_data[0], a_offsets[0] - 1)) : (a_offsets[i] - a_offsets[i - 1] == b_offsets[i] - b_offsets[i - 1] && !memcmp(&a_data[a_offsets[i - 1]], &b_data[b_offsets[i - 1]], a_offsets[i] - a_offsets[i - 1] - 1))); @@ -230,7 +296,7 @@ struct NotEqualsStringImpl { size_t size = a_offsets.size(); for (size_t i = 0; i < size; ++i) - c[i] = !((i == 0) + c[i] = positive == ((i == 0) ? (a_offsets[0] == b_n + 1 && !memcmp(&a_data[0], &b_data[0], b_n)) : (a_offsets[i] - a_offsets[i - 1] == b_n + 1 && !memcmp(&a_data[a_offsets[i - 1]], &b_data[b_n * i], b_n))); @@ -245,25 +311,12 @@ struct NotEqualsStringImpl ColumnString::Offset_t b_n = b.size(); const UInt8 * b_data = reinterpret_cast(b.data()); for (size_t i = 0; i < size; ++i) - c[i] = !((i == 0) + c[i] = positive == ((i == 0) ? (a_offsets[0] == b_n + 1 && !memcmp(&a_data[0], b_data, b_n)) : (a_offsets[i] - a_offsets[i - 1] == b_n + 1 && !memcmp(&a_data[a_offsets[i - 1]], b_data, b_n))); } - static void fixed_string_vector_string_vector( - const ColumnString::Chars_t & a_data, ColumnString::Offset_t a_n, - const ColumnString::Chars_t & b_data, const ColumnString::Offsets_t & b_offsets, - PODArray & c) - { - size_t size = b_offsets.size(); - for (size_t i = 0; i < size; ++i) - c[i] = !((i == 0) - ? (b_offsets[0] == a_n + 1 && !memcmp(&b_data[0], &a_data[0], a_n)) - : (b_offsets[i] - b_offsets[i - 1] == a_n + 1 - && !memcmp(&b_data[b_offsets[i - 1]], &a_data[a_n * i], a_n))); - } - static void fixed_string_vector_fixed_string_vector( const ColumnString::Chars_t & a_data, ColumnString::Offset_t a_n, const ColumnString::Chars_t & b_data, ColumnString::Offset_t b_n, @@ -271,7 +324,7 @@ struct NotEqualsStringImpl { size_t size = a_data.size(); for (size_t i = 0, j = 0; i < size; i += a_n, ++j) - c[j] = !(a_n == b_n && !memcmp(&a_data[i], &b_data[i], a_n)); + c[j] = positive == (a_n == b_n && !memcmp(&a_data[i], &b_data[i], a_n)); } static void fixed_string_vector_constant( @@ -283,34 +336,7 @@ struct NotEqualsStringImpl const UInt8 * b_data = reinterpret_cast(b.data()); ColumnString::Offset_t b_n = b.size(); for (size_t i = 0, j = 0; i < size; i += a_n, ++j) - c[j] = !(a_n == b_n && !memcmp(&a_data[i], b_data, a_n)); - } - - static void constant_string_vector( - const std::string & a, - const ColumnString::Chars_t & b_data, const ColumnString::Offsets_t & b_offsets, - PODArray & c) - { - size_t size = b_offsets.size(); - ColumnString::Offset_t a_n = a.size(); - const UInt8 * a_data = reinterpret_cast(a.data()); - for (size_t i = 0; i < size; ++i) - c[i] = !((i == 0) - ? (b_offsets[0] == a_n + 1 && !memcmp(&b_data[0], a_data, a_n)) - : (b_offsets[i] - b_offsets[i - 1] == a_n + 1 - && !memcmp(&b_data[b_offsets[i - 1]], a_data, a_n))); - } - - static void constant_fixed_string_vector( - const std::string & a, - const ColumnString::Chars_t & b_data, ColumnString::Offset_t b_n, - PODArray & c) - { - size_t size = b_data.size(); - const UInt8 * a_data = reinterpret_cast(a.data()); - ColumnString::Offset_t a_n = a.size(); - for (size_t i = 0, j = 0; i < size; i += b_n, ++j) - c[j] = !(a_n == b_n && !memcmp(&b_data[i], a_data, b_n)); + c[j] = positive == (a_n == b_n && !memcmp(&a_data[i], b_data, a_n)); } static void constant_constant( @@ -318,108 +344,7 @@ struct NotEqualsStringImpl const std::string & b, UInt8 & c) { - c = !(a == b); - } -}; - -template -struct LessNumImpl -{ - static void vector_vector(const PODArray & a, const PODArray & b, PODArray & c) - { - size_t size = a.size(); - for (size_t i = 0; i < size; ++i) - c[i] = a[i] < b[i]; - } - - static void vector_constant(const PODArray & a, B b, PODArray & c) - { - size_t size = a.size(); - for (size_t i = 0; i < size; ++i) - c[i] = a[i] < b; - } - - static void constant_vector(A a, const PODArray & b, PODArray & c) - { - size_t size = b.size(); - for (size_t i = 0; i < size; ++i) - c[i] = a < b[i]; - } - - static void constant_constant(A a, B b, UInt8 & c) - { - c = a < b; - } -}; - -struct LessStringImpl -{ - static void string_vector_string_vector( - const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_offsets, - const ColumnString::Chars_t & b_data, const ColumnString::Offsets_t & b_offsets, - PODArray & c) - { - size_t size = a_offsets.size(); - for (size_t i = 0; i < size; ++i) - { - if (i == 0) - { - int res = memcmp(&a_data[0], &b_data[0], std::min(a_offsets[0], b_offsets[0]) - 1); - c[i] = res < 0 || (res == 0 && a_offsets[0] < b_offsets[0]); - } - else - { - int res = memcmp(&a_data[a_offsets[i - 1]], &b_data[b_offsets[i - 1]], - std::min(a_offsets[i] - a_offsets[i - 1], b_offsets[i] - b_offsets[i - 1]) - 1); - c[i] = res < 0 || (res == 0 && a_offsets[i] - a_offsets[i - 1] < b_offsets[i] - b_offsets[i - 1]); - } - } - } - - static void string_vector_fixed_string_vector( - const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_offsets, - const ColumnString::Chars_t & b_data, ColumnString::Offset_t b_n, - PODArray & c) - { - size_t size = a_offsets.size(); - for (size_t i = 0; i < size; ++i) - { - if (i == 0) - { - int res = memcmp(&a_data[0], &b_data[0], std::min(a_offsets[0] - 1, b_n)); - c[i] = res < 0 || (res == 0 && a_offsets[0] < b_n + 1); - } - else - { - int res = memcmp(&a_data[a_offsets[i - 1]], &b_data[i * b_n], - std::min(a_offsets[i] - a_offsets[i - 1] - 1, b_n)); - c[i] = res < 0 || (res == 0 && a_offsets[i] - a_offsets[i - 1] < b_n + 1); - } - } - } - - static void string_vector_constant( - const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_offsets, - const std::string & b, - PODArray & c) - { - size_t size = a_offsets.size(); - ColumnString::Offset_t b_n = b.size(); - const UInt8 * b_data = reinterpret_cast(b.data()); - for (size_t i = 0; i < size; ++i) - { - if (i == 0) - { - int res = memcmp(&a_data[0], b_data, std::min(a_offsets[0] - 1, b_n)); - c[i] = res < 0 || (res == 0 && a_offsets[0] < b_n + 1); - } - else - { - int res = memcmp(&a_data[a_offsets[i - 1]], b_data, - std::min(a_offsets[i] - a_offsets[i - 1] - 1, b_n)); - c[i] = res < 0 || (res == 0 && a_offsets[i] - a_offsets[i - 1] < b_n + 1); - } - } + c = positive == (a == b); } static void fixed_string_vector_string_vector( @@ -427,49 +352,7 @@ struct LessStringImpl const ColumnString::Chars_t & b_data, const ColumnString::Offsets_t & b_offsets, PODArray & c) { - size_t size = b_offsets.size(); - for (size_t i = 0; i < size; ++i) - { - if (i == 0) - { - int res = memcmp(&a_data[0], &b_data[0], std::min(b_offsets[0] - 1, a_n)); - c[i] = res < 0 || (res == 0 && a_n + 1 < b_offsets[0]); - } - else - { - int res = memcmp(&a_data[i * a_n], &b_data[b_offsets[i - 1]], - std::min(b_offsets[i] - b_offsets[i - 1] - 1, a_n)); - c[i] = res < 0 || (res == 0 && a_n + 1 < b_offsets[i] - b_offsets[i - 1]); - } - } - } - - static void fixed_string_vector_fixed_string_vector( - const ColumnString::Chars_t & a_data, ColumnString::Offset_t a_n, - const ColumnString::Chars_t & b_data, ColumnString::Offset_t b_n, - PODArray & c) - { - size_t size = a_data.size(); - for (size_t i = 0, j = 0; i < size; i += a_n, ++j) - { - int res = memcmp(&a_data[i], &b_data[i], std::min(a_n, b_n)); - c[j] = res < 0 || (res == 0 && a_n < b_n); - } - } - - static void fixed_string_vector_constant( - const ColumnString::Chars_t & a_data, ColumnString::Offset_t a_n, - const std::string & b, - PODArray & c) - { - size_t size = a_data.size(); - const UInt8 * b_data = reinterpret_cast(b.data()); - ColumnString::Offset_t b_n = b.size(); - for (size_t i = 0, j = 0; i < size; i += a_n, ++j) - { - int res = memcmp(&a_data[i], b_data, std::min(a_n, b_n)); - c[j] = res < 0 || (res == 0 && a_n < b_n); - } + string_vector_fixed_string_vector(b_data, b_offsets, a_data, a_n, c); } static void constant_string_vector( @@ -477,23 +360,7 @@ struct LessStringImpl const ColumnString::Chars_t & b_data, const ColumnString::Offsets_t & b_offsets, PODArray & c) { - size_t size = b_offsets.size(); - ColumnString::Offset_t a_n = a.size(); - const UInt8 * a_data = reinterpret_cast(a.data()); - for (size_t i = 0; i < size; ++i) - { - if (i == 0) - { - int res = memcmp(a_data, &b_data[0], std::min(b_offsets[0] - 1, a_n)); - c[i] = res < 0 || (res == 0 && a_n + 1 < b_offsets[0]); - } - else - { - int res = memcmp(a_data, &b_data[b_offsets[i - 1]], - std::min(b_offsets[i] - b_offsets[i - 1] - 1, a_n)); - c[i] = res < 0 || (res == 0 && a_n + 1 < b_offsets[i] - b_offsets[i - 1]); - } - } + string_vector_constant(b_data, b_offsets, a, c); } static void constant_fixed_string_vector( @@ -501,626 +368,20 @@ struct LessStringImpl const ColumnString::Chars_t & b_data, ColumnString::Offset_t b_n, PODArray & c) { - size_t size = b_data.size(); - const UInt8 * a_data = reinterpret_cast(a.data()); - ColumnString::Offset_t a_n = a.size(); - for (size_t i = 0, j = 0; i < size; i += b_n, ++j) - { - int res = memcmp(a_data, &b_data[i], std::min(a_n, b_n)); - c[j] = res < 0 || (res == 0 && b_n < a_n); - } - } - - static void constant_constant( - const std::string & a, - const std::string & b, - UInt8 & c) - { - c = a < b; - } -}; - -template -struct GreaterNumImpl -{ - static void vector_vector(const PODArray & a, const PODArray & b, PODArray & c) - { - size_t size = a.size(); - for (size_t i = 0; i < size; ++i) - c[i] = a[i] > b[i]; - } - - static void vector_constant(const PODArray & a, B b, PODArray & c) - { - size_t size = a.size(); - for (size_t i = 0; i < size; ++i) - c[i] = a[i] > b; - } - - static void constant_vector(A a, const PODArray & b, PODArray & c) - { - size_t size = b.size(); - for (size_t i = 0; i < size; ++i) - c[i] = a > b[i]; - } - - static void constant_constant(A a, B b, UInt8 & c) - { - c = a > b; - } -}; - -struct GreaterStringImpl -{ - static void string_vector_string_vector( - const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_offsets, - const ColumnString::Chars_t & b_data, const ColumnString::Offsets_t & b_offsets, - PODArray & c) - { - size_t size = a_offsets.size(); - for (size_t i = 0; i < size; ++i) - { - if (i == 0) - { - int res = memcmp(&a_data[0], &b_data[0], std::min(a_offsets[0], b_offsets[0]) - 1); - c[i] = res > 0 || (res == 0 && a_offsets[0] > b_offsets[0]); - } - else - { - int res = memcmp(&a_data[a_offsets[i - 1]], &b_data[b_offsets[i - 1]], - std::min(a_offsets[i] - a_offsets[i - 1], b_offsets[i] - b_offsets[i - 1]) - 1); - c[i] = res > 0 || (res == 0 && a_offsets[i] - a_offsets[i - 1] > b_offsets[i] - b_offsets[i - 1]); - } - } - } - - static void string_vector_fixed_string_vector( - const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_offsets, - const ColumnString::Chars_t & b_data, ColumnString::Offset_t b_n, - PODArray & c) - { - size_t size = a_offsets.size(); - for (size_t i = 0; i < size; ++i) - { - if (i == 0) - { - int res = memcmp(&a_data[0], &b_data[0], std::min(a_offsets[0] - 1, b_n)); - c[i] = res > 0 || (res == 0 && a_offsets[0] > b_n + 1); - } - else - { - int res = memcmp(&a_data[a_offsets[i - 1]], &b_data[i * b_n], - std::min(a_offsets[i] - a_offsets[i - 1] - 1, b_n)); - c[i] = res > 0 || (res == 0 && a_offsets[i] - a_offsets[i - 1] > b_n + 1); - } - } - } - - static void string_vector_constant( - const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_offsets, - const std::string & b, - PODArray & c) - { - size_t size = a_offsets.size(); - ColumnString::Offset_t b_n = b.size(); - const UInt8 * b_data = reinterpret_cast(b.data()); - for (size_t i = 0; i < size; ++i) - { - if (i == 0) - { - int res = memcmp(&a_data[0], b_data, std::min(a_offsets[0] - 1, b_n)); - c[i] = res > 0 || (res == 0 && a_offsets[0] > b_n + 1); - } - else - { - int res = memcmp(&a_data[a_offsets[i - 1]], b_data, - std::min(a_offsets[i] - a_offsets[i - 1] - 1, b_n)); - c[i] = res > 0 || (res == 0 && a_offsets[i] - a_offsets[i - 1] > b_n + 1); - } - } - } - - static void fixed_string_vector_string_vector( - const ColumnString::Chars_t & a_data, ColumnString::Offset_t a_n, - const ColumnString::Chars_t & b_data, const ColumnString::Offsets_t & b_offsets, - PODArray & c) - { - size_t size = b_offsets.size(); - for (size_t i = 0; i < size; ++i) - { - if (i == 0) - { - int res = memcmp(&a_data[0], &b_data[0], std::min(b_offsets[0] - 1, a_n)); - c[i] = res > 0 || (res == 0 && a_n + 1 > b_offsets[0]); - } - else - { - int res = memcmp(&a_data[i * a_n], &b_data[b_offsets[i - 1]], - std::min(b_offsets[i] - b_offsets[i - 1] - 1, a_n)); - c[i] = res > 0 || (res == 0 && a_n + 1 > b_offsets[i] - b_offsets[i - 1]); - } - } - } - - static void fixed_string_vector_fixed_string_vector( - const ColumnString::Chars_t & a_data, ColumnString::Offset_t a_n, - const ColumnString::Chars_t & b_data, ColumnString::Offset_t b_n, - PODArray & c) - { - size_t size = a_data.size(); - for (size_t i = 0, j = 0; i < size; i += a_n, ++j) - { - int res = memcmp(&a_data[i], &b_data[i], std::min(a_n, b_n)); - c[j] = res > 0 || (res == 0 && a_n > b_n); - } - } - - static void fixed_string_vector_constant( - const ColumnString::Chars_t & a_data, ColumnString::Offset_t a_n, - const std::string & b, - PODArray & c) - { - size_t size = a_data.size(); - const UInt8 * b_data = reinterpret_cast(b.data()); - ColumnString::Offset_t b_n = b.size(); - for (size_t i = 0, j = 0; i < size; i += a_n, ++j) - { - int res = memcmp(&a_data[i], b_data, std::min(a_n, b_n)); - c[j] = res > 0 || (res == 0 && a_n > b_n); - } - } - - static void constant_string_vector( - const std::string & a, - const ColumnString::Chars_t & b_data, const ColumnString::Offsets_t & b_offsets, - PODArray & c) - { - size_t size = b_offsets.size(); - ColumnString::Offset_t a_n = a.size(); - const UInt8 * a_data = reinterpret_cast(a.data()); - for (size_t i = 0; i < size; ++i) - { - if (i == 0) - { - int res = memcmp(a_data, &b_data[0], std::min(b_offsets[0] - 1, a_n)); - c[i] = res > 0 || (res == 0 && a_n + 1 > b_offsets[0]); - } - else - { - int res = memcmp(a_data, &b_data[b_offsets[i - 1]], - std::min(b_offsets[i] - b_offsets[i - 1] - 1, a_n)); - c[i] = res > 0 || (res == 0 && a_n + 1 > b_offsets[i] - b_offsets[i - 1]); - } - } - } - - static void constant_fixed_string_vector( - const std::string & a, - const ColumnString::Chars_t & b_data, ColumnString::Offset_t b_n, - PODArray & c) - { - size_t size = b_data.size(); - const UInt8 * a_data = reinterpret_cast(a.data()); - ColumnString::Offset_t a_n = a.size(); - for (size_t i = 0, j = 0; i < size; i += b_n, ++j) - { - int res = memcmp(a_data, &b_data[i], std::min(a_n, b_n)); - c[j] = res > 0 || (res == 0 && b_n > a_n); - } - } - - static void constant_constant( - const std::string & a, - const std::string & b, - UInt8 & c) - { - c = a > b; - } -}; - -template -struct LessOrEqualsNumImpl -{ - static void vector_vector(const PODArray & a, const PODArray & b, PODArray & c) - { - size_t size = a.size(); - for (size_t i = 0; i < size; ++i) - c[i] = a[i] <= b[i]; - } - - static void vector_constant(const PODArray & a, B b, PODArray & c) - { - size_t size = a.size(); - for (size_t i = 0; i < size; ++i) - c[i] = a[i] <= b; - } - - static void constant_vector(A a, const PODArray & b, PODArray & c) - { - size_t size = b.size(); - for (size_t i = 0; i < size; ++i) - c[i] = a <= b[i]; - } - - static void constant_constant(A a, B b, UInt8 & c) - { - c = a <= b; - } -}; - -struct LessOrEqualsStringImpl -{ - static void string_vector_string_vector( - const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_offsets, - const ColumnString::Chars_t & b_data, const ColumnString::Offsets_t & b_offsets, - PODArray & c) - { - size_t size = a_offsets.size(); - for (size_t i = 0; i < size; ++i) - { - if (i == 0) - { - int res = memcmp(&a_data[0], &b_data[0], std::min(a_offsets[0], b_offsets[0]) - 1); - c[i] = !(res > 0 || (res == 0 && a_offsets[0] > b_offsets[0])); - } - else - { - int res = memcmp(&a_data[a_offsets[i - 1]], &b_data[b_offsets[i - 1]], - std::min(a_offsets[i] - a_offsets[i - 1], b_offsets[i] - b_offsets[i - 1]) - 1); - c[i] = !(res > 0 || (res == 0 && a_offsets[i] - a_offsets[i - 1] > b_offsets[i] - b_offsets[i - 1])); - } - } - } - - static void string_vector_fixed_string_vector( - const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_offsets, - const ColumnString::Chars_t & b_data, ColumnString::Offset_t b_n, - PODArray & c) - { - size_t size = a_offsets.size(); - for (size_t i = 0; i < size; ++i) - { - if (i == 0) - { - int res = memcmp(&a_data[0], &b_data[0], std::min(a_offsets[0] - 1, b_n)); - c[i] = !(res > 0 || (res == 0 && a_offsets[0] > b_n + 1)); - } - else - { - int res = memcmp(&a_data[a_offsets[i - 1]], &b_data[i * b_n], - std::min(a_offsets[i] - a_offsets[i - 1] - 1, b_n)); - c[i] = !(res > 0 || (res == 0 && a_offsets[i] - a_offsets[i - 1] > b_n + 1)); - } - } - } - - static void string_vector_constant( - const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_offsets, - const std::string & b, - PODArray & c) - { - size_t size = a_offsets.size(); - ColumnString::Offset_t b_n = b.size(); - const UInt8 * b_data = reinterpret_cast(b.data()); - for (size_t i = 0; i < size; ++i) - { - if (i == 0) - { - int res = memcmp(&a_data[0], b_data, std::min(a_offsets[0] - 1, b_n)); - c[i] = !(res > 0 || (res == 0 && a_offsets[0] > b_n + 1)); - } - else - { - int res = memcmp(&a_data[a_offsets[i - 1]], b_data, - std::min(a_offsets[i] - a_offsets[i - 1] - 1, b_n)); - c[i] = !(res > 0 || (res == 0 && a_offsets[i] - a_offsets[i - 1] > b_n + 1)); - } - } - } - - static void fixed_string_vector_string_vector( - const ColumnString::Chars_t & a_data, ColumnString::Offset_t a_n, - const ColumnString::Chars_t & b_data, const ColumnString::Offsets_t & b_offsets, - PODArray & c) - { - size_t size = b_offsets.size(); - for (size_t i = 0; i < size; ++i) - { - if (i == 0) - { - int res = memcmp(&a_data[0], &b_data[0], std::min(b_offsets[0] - 1, a_n)); - c[i] = !(res > 0 || (res == 0 && a_n + 1 > b_offsets[0])); - } - else - { - int res = memcmp(&a_data[i * a_n], &b_data[b_offsets[i - 1]], - std::min(b_offsets[i] - b_offsets[i - 1] - 1, a_n)); - c[i] = !(res > 0 || (res == 0 && a_n + 1 > b_offsets[i] - b_offsets[i - 1])); - } - } - } - - static void fixed_string_vector_fixed_string_vector( - const ColumnString::Chars_t & a_data, ColumnString::Offset_t a_n, - const ColumnString::Chars_t & b_data, ColumnString::Offset_t b_n, - PODArray & c) - { - size_t size = a_data.size(); - for (size_t i = 0, j = 0; i < size; i += a_n, ++j) - { - int res = memcmp(&a_data[i], &b_data[i], std::min(a_n, b_n)); - c[j] = !(res > 0 || (res == 0 && a_n > b_n)); - } - } - - static void fixed_string_vector_constant( - const ColumnString::Chars_t & a_data, ColumnString::Offset_t a_n, - const std::string & b, - PODArray & c) - { - size_t size = a_data.size(); - const UInt8 * b_data = reinterpret_cast(b.data()); - ColumnString::Offset_t b_n = b.size(); - for (size_t i = 0, j = 0; i < size; i += a_n, ++j) - { - int res = memcmp(&a_data[i], b_data, std::min(a_n, b_n)); - c[j] = !(res > 0 || (res == 0 && a_n > b_n)); - } - } - - static void constant_string_vector( - const std::string & a, - const ColumnString::Chars_t & b_data, const ColumnString::Offsets_t & b_offsets, - PODArray & c) - { - size_t size = b_offsets.size(); - ColumnString::Offset_t a_n = a.size(); - const UInt8 * a_data = reinterpret_cast(a.data()); - for (size_t i = 0; i < size; ++i) - { - if (i == 0) - { - int res = memcmp(a_data, &b_data[0], std::min(b_offsets[0] - 1, a_n)); - c[i] = !(res > 0 || (res == 0 && a_n + 1 > b_offsets[0])); - } - else - { - int res = memcmp(a_data, &b_data[b_offsets[i - 1]], - std::min(b_offsets[i] - b_offsets[i - 1] - 1, a_n)); - c[i] = !(res > 0 || (res == 0 && a_n + 1 > b_offsets[i] - b_offsets[i - 1])); - } - } - } - - static void constant_fixed_string_vector( - const std::string & a, - const ColumnString::Chars_t & b_data, ColumnString::Offset_t b_n, - PODArray & c) - { - size_t size = b_data.size(); - const UInt8 * a_data = reinterpret_cast(a.data()); - ColumnString::Offset_t a_n = a.size(); - for (size_t i = 0, j = 0; i < size; i += b_n, ++j) - { - int res = memcmp(a_data, &b_data[i], std::min(a_n, b_n)); - c[j] = !(res > 0 || (res == 0 && b_n > a_n)); - } - } - - static void constant_constant( - const std::string & a, - const std::string & b, - UInt8 & c) - { - c = a <= b; - } -}; - -template -struct GreaterOrEqualsNumImpl -{ - static void vector_vector(const PODArray & a, const PODArray & b, PODArray & c) - { - size_t size = a.size(); - for (size_t i = 0; i < size; ++i) - c[i] = a[i] >= b[i]; - } - - static void vector_constant(const PODArray & a, B b, PODArray & c) - { - size_t size = a.size(); - for (size_t i = 0; i < size; ++i) - c[i] = a[i] >= b; - } - - static void constant_vector(A a, const PODArray & b, PODArray & c) - { - size_t size = b.size(); - for (size_t i = 0; i < size; ++i) - c[i] = a >= b[i]; - } - - static void constant_constant(A a, B b, UInt8 & c) - { - c = a >= b; - } -}; - -struct GreaterOrEqualsStringImpl -{ - static void string_vector_string_vector( - const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_offsets, - const ColumnString::Chars_t & b_data, const ColumnString::Offsets_t & b_offsets, - PODArray & c) - { - size_t size = a_offsets.size(); - for (size_t i = 0; i < size; ++i) - { - if (i == 0) - { - int res = memcmp(&a_data[0], &b_data[0], std::min(a_offsets[0], b_offsets[0]) - 1); - c[i] = !(res < 0 || (res == 0 && a_offsets[0] < b_offsets[0])); - } - else - { - int res = memcmp(&a_data[a_offsets[i - 1]], &b_data[b_offsets[i - 1]], - std::min(a_offsets[i] - a_offsets[i - 1], b_offsets[i] - b_offsets[i - 1]) - 1); - c[i] = !(res < 0 || (res == 0 && a_offsets[i] - a_offsets[i - 1] < b_offsets[i] - b_offsets[i - 1])); - } - } - } - - static void string_vector_fixed_string_vector( - const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_offsets, - const ColumnString::Chars_t & b_data, ColumnString::Offset_t b_n, - PODArray & c) - { - size_t size = a_offsets.size(); - for (size_t i = 0; i < size; ++i) - { - if (i == 0) - { - int res = memcmp(&a_data[0], &b_data[0], std::min(a_offsets[0] - 1, b_n)); - c[i] = !(res < 0 || (res == 0 && a_offsets[0] < b_n + 1)); - } - else - { - int res = memcmp(&a_data[a_offsets[i - 1]], &b_data[i * b_n], - std::min(a_offsets[i] - a_offsets[i - 1] - 1, b_n)); - c[i] = !(res < 0 || (res == 0 && a_offsets[i] - a_offsets[i - 1] < b_n + 1)); - } - } - } - - static void string_vector_constant( - const ColumnString::Chars_t & a_data, const ColumnString::Offsets_t & a_offsets, - const std::string & b, - PODArray & c) - { - size_t size = a_offsets.size(); - ColumnString::Offset_t b_n = b.size(); - const UInt8 * b_data = reinterpret_cast(b.data()); - for (size_t i = 0; i < size; ++i) - { - if (i == 0) - { - int res = memcmp(&a_data[0], b_data, std::min(a_offsets[0] - 1, b_n)); - c[i] = !(res < 0 || (res == 0 && a_offsets[0] < b_n + 1)); - } - else - { - int res = memcmp(&a_data[a_offsets[i - 1]], b_data, - std::min(a_offsets[i] - a_offsets[i - 1] - 1, b_n)); - c[i] = !(res < 0 || (res == 0 && a_offsets[i] - a_offsets[i - 1] < b_n + 1)); - } - } - } - - static void fixed_string_vector_string_vector( - const ColumnString::Chars_t & a_data, ColumnString::Offset_t a_n, - const ColumnString::Chars_t & b_data, const ColumnString::Offsets_t & b_offsets, - PODArray & c) - { - size_t size = b_offsets.size(); - for (size_t i = 0; i < size; ++i) - { - if (i == 0) - { - int res = memcmp(&a_data[0], &b_data[0], std::min(b_offsets[0] - 1, a_n)); - c[i] = !(res < 0 || (res == 0 && a_n + 1 < b_offsets[0])); - } - else - { - int res = memcmp(&a_data[i * a_n], &b_data[b_offsets[i - 1]], - std::min(b_offsets[i] - b_offsets[i - 1] - 1, a_n)); - c[i] = !(res < 0 || (res == 0 && a_n + 1 < b_offsets[i] - b_offsets[i - 1])); - } - } - } - - static void fixed_string_vector_fixed_string_vector( - const ColumnString::Chars_t & a_data, ColumnString::Offset_t a_n, - const ColumnString::Chars_t & b_data, ColumnString::Offset_t b_n, - PODArray & c) - { - size_t size = a_data.size(); - for (size_t i = 0, j = 0; i < size; i += a_n, ++j) - { - int res = memcmp(&a_data[i], &b_data[i], std::min(a_n, b_n)); - c[j] = !(res < 0 || (res == 0 && a_n < b_n)); - } - } - - static void fixed_string_vector_constant( - const ColumnString::Chars_t & a_data, ColumnString::Offset_t a_n, - const std::string & b, - PODArray & c) - { - size_t size = a_data.size(); - const UInt8 * b_data = reinterpret_cast(b.data()); - ColumnString::Offset_t b_n = b.size(); - for (size_t i = 0, j = 0; i < size; i += a_n, ++j) - { - int res = memcmp(&a_data[i], b_data, std::min(a_n, b_n)); - c[j] = !(res < 0 || (res == 0 && a_n < b_n)); - } - } - - static void constant_string_vector( - const std::string & a, - const ColumnString::Chars_t & b_data, const ColumnString::Offsets_t & b_offsets, - PODArray & c) - { - size_t size = b_offsets.size(); - ColumnString::Offset_t a_n = a.size(); - const UInt8 * a_data = reinterpret_cast(a.data()); - for (size_t i = 0; i < size; ++i) - { - if (i == 0) - { - int res = memcmp(a_data, &b_data[0], std::min(b_offsets[0] - 1, a_n)); - c[i] = !(res < 0 || (res == 0 && a_n + 1 < b_offsets[0])); - } - else - { - int res = memcmp(a_data, &b_data[b_offsets[i - 1]], - std::min(b_offsets[i] - b_offsets[i - 1] - 1, a_n)); - c[i] = !(res < 0 || (res == 0 && a_n + 1 < b_offsets[i] - b_offsets[i - 1])); - } - } - } - - static void constant_fixed_string_vector( - const std::string & a, - const ColumnString::Chars_t & b_data, ColumnString::Offset_t b_n, - PODArray & c) - { - size_t size = b_data.size(); - const UInt8 * a_data = reinterpret_cast(a.data()); - ColumnString::Offset_t a_n = a.size(); - for (size_t i = 0, j = 0; i < size; i += b_n, ++j) - { - int res = memcmp(a_data, &b_data[i], std::min(a_n, b_n)); - c[j] = !(res < 0 || (res == 0 && b_n < a_n)); - } - } - - static void constant_constant( - const std::string & a, - const std::string & b, - UInt8 & c) - { - c = a >= b; + fixed_string_vector_constant(b_data, b_n, a, c); } }; -#pragma GCC diagnostic pop +template +struct StringComparisonImpl> : StringEqualsImpl {}; + +template +struct StringComparisonImpl> : StringEqualsImpl {}; template < - template class NumImpl, - typename StringImpl, + template class Op, typename Name> class FunctionComparison : public IFunction { @@ -1136,7 +397,7 @@ private: ColumnUInt8::Container_t & vec_res = col_res->getData(); vec_res.resize(col_left->getData().size()); - NumImpl::vector_vector(col_left->getData(), col_right->getData(), vec_res); + NumComparisonImpl>::vector_vector(col_left->getData(), col_right->getData(), vec_res); return true; } @@ -1147,7 +408,7 @@ private: ColumnUInt8::Container_t & vec_res = col_res->getData(); vec_res.resize(col_left->getData().size()); - NumImpl::vector_constant(col_left->getData(), col_right->getData(), vec_res); + NumComparisonImpl>::vector_constant(col_left->getData(), col_right->getData(), vec_res); return true; } @@ -1165,14 +426,14 @@ private: ColumnUInt8::Container_t & vec_res = col_res->getData(); vec_res.resize(col_left->size()); - NumImpl::constant_vector(col_left->getData(), col_right->getData(), vec_res); + NumComparisonImpl>::constant_vector(col_left->getData(), col_right->getData(), vec_res); return true; } else if (ColumnConst * col_right = typeid_cast *>(&*block.getByPosition(arguments[1]).column)) { UInt8 res = 0; - NumImpl::constant_constant(col_left->getData(), col_right->getData(), res); + NumComparisonImpl>::constant_constant(col_left->getData(), col_right->getData(), res); ColumnConstUInt8 * col_res = new ColumnConstUInt8(col_left->size(), res); block.getByPosition(result).column = col_res; @@ -1238,6 +499,8 @@ private: ColumnConstString * c0_const = typeid_cast(c0); ColumnConstString * c1_const = typeid_cast(c1); + using StringImpl = StringComparisonImpl>; + if (c0_const && c1_const) { ColumnConstUInt8 * c_res = new ColumnConstUInt8(c0_const->size(), 0); @@ -1359,11 +622,11 @@ struct NameGreater { static const char * get() { return "greater"; } }; struct NameLessOrEquals { static const char * get() { return "lessOrEquals"; } }; struct NameGreaterOrEquals { static const char * get() { return "greaterOrEquals"; } }; -typedef FunctionComparison FunctionEquals; -typedef FunctionComparison FunctionNotEquals; -typedef FunctionComparison FunctionLess; -typedef FunctionComparison FunctionGreater; -typedef FunctionComparison FunctionLessOrEquals; -typedef FunctionComparison FunctionGreaterOrEquals; +typedef FunctionComparison FunctionEquals; +typedef FunctionComparison FunctionNotEquals; +typedef FunctionComparison FunctionLess; +typedef FunctionComparison FunctionGreater; +typedef FunctionComparison FunctionLessOrEquals; +typedef FunctionComparison FunctionGreaterOrEquals; } diff --git a/dbms/include/DB/Functions/FunctionsFormatting.h b/dbms/include/DB/Functions/FunctionsFormatting.h index f636b4b1dc2..99f533ac9f9 100644 --- a/dbms/include/DB/Functions/FunctionsFormatting.h +++ b/dbms/include/DB/Functions/FunctionsFormatting.h @@ -1,6 +1,6 @@ #include -#include "NumberTraits.h" #include +#include #include #include #include diff --git a/dbms/include/DB/Functions/FunctionsMiscellaneous.h b/dbms/include/DB/Functions/FunctionsMiscellaneous.h index 29191faadc7..fb3042bc047 100644 --- a/dbms/include/DB/Functions/FunctionsMiscellaneous.h +++ b/dbms/include/DB/Functions/FunctionsMiscellaneous.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(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(&*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(&*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(&*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(&*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(block, arguments, 1, "Second"); /// Уровень значения, при котором полоска имеет нулевую длину. + Int64 max = extractConstant(block, arguments, 2, "Third"); /// Уровень значения, при котором полоска имеет максимальную длину. + + /// Максимальная ширина полоски в символах, по-умолчанию. + Float64 max_width = arguments.size() == 4 + ? extractConstant(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 (src, *res_column, min, max, max_width) + || executeConstNumber (src, *res_column, min, max, max_width) + || executeConstNumber (src, *res_column, min, max, max_width) + || executeConstNumber (src, *res_column, min, max, max_width) + || executeConstNumber (src, *res_column, min, max, max_width) + || executeConstNumber (src, *res_column, min, max, max_width) + || executeConstNumber (src, *res_column, min, max, max_width) + || executeConstNumber (src, *res_column, min, max, max_width) + || executeConstNumber (src, *res_column, min, max, max_width) + || executeConstNumber (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 (src, *res_column, min, max, max_width) + || executeNumber (src, *res_column, min, max, max_width) + || executeNumber (src, *res_column, min, max, max_width) + || executeNumber (src, *res_column, min, max, max_width) + || executeNumber (src, *res_column, min, max, max_width) + || executeNumber (src, *res_column, min, max, max_width) + || executeNumber (src, *res_column, min, max, max_width) + || executeNumber (src, *res_column, min, max, max_width) + || executeNumber (src, *res_column, min, max, max_width) + || executeNumber (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 + 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(), column[0]); + } + + static constexpr size_t BAR_CHAR_SIZE = strlen("█"); + + template + 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 + static void fill(const PODArray & 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(&dst_chars[current_offset])); + current_offset = next_size; + dst_offsets[i] = current_offset; + } + } + + template + 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 + static bool executeNumber(const IColumn & src, ColumnString & dst, Int64 min, Int64 max, Float64 max_width) + { + if (const ColumnVector * col = typeid_cast *>(&src)) + { + fill(col->getData(), dst.getChars(), dst.getOffsets(), min, max, max_width); + return true; + } + else + return false; + } + + template + static bool executeConstNumber(const IColumn & src, ColumnConstString & dst, Int64 min, Int64 max, Float64 max_width) + { + if (const ColumnConst * col = typeid_cast *>(&src)) + { + fill(col->getData(), dst.getData(), min, max, max_width); + return true; + } + else + return false; + } +}; + } diff --git a/dbms/include/DB/Functions/FunctionsRandom.h b/dbms/include/DB/Functions/FunctionsRandom.h index d9ce0121664..48080058a24 100644 --- a/dbms/include/DB/Functions/FunctionsRandom.h +++ b/dbms/include/DB/Functions/FunctionsRandom.h @@ -4,6 +4,7 @@ #include #include +#include #include @@ -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 & res) { - drand48_data rand_state; - detail::seed(rand_state, reinterpret_cast(&res[0])); - + detail::LinearCongruentialGenerator generator0; + detail::LinearCongruentialGenerator generator1; + detail::LinearCongruentialGenerator generator2; + detail::LinearCongruentialGenerator generator3; + + detail::seed(generator0, 0xfb4121280b2ab902ULL + reinterpret_cast(&res[0])); + detail::seed(generator1, 0x0121cf76df39c673ULL + reinterpret_cast(&res[0])); + detail::seed(generator2, 0x17ae86e3a19a602fULL + reinterpret_cast(&res[0])); + detail::seed(generator3, 0x8b6e16da7e06d622ULL + reinterpret_cast(&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 & res) { - drand48_data rand_state; - detail::seed(rand_state, reinterpret_cast(&res[0])); + detail::LinearCongruentialGenerator generator0; + detail::LinearCongruentialGenerator generator1; + detail::LinearCongruentialGenerator generator2; + detail::LinearCongruentialGenerator generator3; + + detail::seed(generator0, 0xfb4121280b2ab902ULL + reinterpret_cast(&res[0])); + detail::seed(generator1, 0x0121cf76df39c673ULL + reinterpret_cast(&res[0])); + detail::seed(generator2, 0x17ae86e3a19a602fULL + reinterpret_cast(&res[0])); + detail::seed(generator3, 0x8b6e16da7e06d622ULL + reinterpret_cast(&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(generator0.next()) << 32) | generator1.next(); + pos[1] = (static_cast(generator2.next()) << 32) | generator3.next(); + pos += 2; + } + + while (pos < end) + { + pos[0] = (static_cast(generator0.next()) << 32) | generator1.next(); + ++pos; } } }; @@ -90,7 +145,7 @@ class FunctionRandom : public IFunction { private: typedef typename Impl::ReturnType ToType; - + public: /// Получить имя функции. String getName() const diff --git a/dbms/include/DB/Functions/FunctionsStringArray.h b/dbms/include/DB/Functions/FunctionsStringArray.h index fce1de2b1e1..312671e08cf 100644 --- a/dbms/include/DB/Functions/FunctionsStringArray.h +++ b/dbms/include/DB/Functions/FunctionsStringArray.h @@ -1,8 +1,8 @@ #pragma once -#include #include #include +#include #include #include #include diff --git a/dbms/include/DB/Functions/FunctionsStringSearch.h b/dbms/include/DB/Functions/FunctionsStringSearch.h index ec872ef61fc..5a7e90b7498 100644 --- a/dbms/include/DB/Functions/FunctionsStringSearch.h +++ b/dbms/include/DB/Functions/FunctionsStringSearch.h @@ -7,7 +7,9 @@ #include #include +#include #include +#include #include #include #include diff --git a/dbms/include/DB/IO/WriteHelpers.h b/dbms/include/DB/IO/WriteHelpers.h index 4b781f61253..42d2ad1ab96 100644 --- a/dbms/include/DB/IO/WriteHelpers.h +++ b/dbms/include/DB/IO/WriteHelpers.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); } diff --git a/dbms/include/DB/Interpreters/Aggregator.h b/dbms/include/DB/Interpreters/Aggregator.h index 3afa1802b4d..1a2afa54eb6 100644 --- a/dbms/include/DB/Interpreters/Aggregator.h +++ b/dbms/include/DB/Interpreters/Aggregator.h @@ -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 key_fixed_string; std::unique_ptr keys128; std::unique_ptr hashed; - + enum Type { EMPTY = 0, @@ -465,7 +465,7 @@ public: protected: friend struct AggregatedDataVariants; - + ColumnNumbers keys; Names key_names; AggregateDescriptions aggregates; diff --git a/dbms/include/DB/Interpreters/Cluster.h b/dbms/include/DB/Interpreters/Cluster.h index 3a21dcc9bb0..9766cadbc0e 100644 --- a/dbms/include/DB/Interpreters/Cluster.h +++ b/dbms/include/DB/Interpreters/Cluster.h @@ -29,10 +29,19 @@ public: /// Соединения с удалёнными серверами. ConnectionPools pools; + struct ShardInfo + { + /// contains names of directories for asynchronous write to StorageDistributed + std::vector dir_names; + int weight; + size_t num_local_nodes; + }; + std::vector shard_info_vec; + std::vector slot_to_shard; + /// используеться для выставления ограничения на размер таймаута static Poco::Timespan saturate(const Poco::Timespan & v, const Poco::Timespan & limit); -private: struct Address { /** В конфиге адреса либо находятся в узлах : @@ -59,6 +68,7 @@ private: Address(const String & host_port_, const String & user_, const String & password_); }; +private: static bool isLocal(const Address & address); /// Массив шардов. Каждый шард - адреса одного сервера. diff --git a/dbms/include/DB/Interpreters/Context.h b/dbms/include/DB/Interpreters/Context.h index b837c7fc673..e1b51266252 100644 --- a/dbms/include/DB/Interpreters/Context.h +++ b/dbms/include/DB/Interpreters/Context.h @@ -9,6 +9,7 @@ #include #include +#include #include #include #include @@ -56,8 +57,8 @@ typedef std::vector 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_); diff --git a/dbms/include/DB/Interpreters/InterpreterAlterQuery.h b/dbms/include/DB/Interpreters/InterpreterAlterQuery.h index 8c2a635d535..e3f44a0efd8 100644 --- a/dbms/include/DB/Interpreters/InterpreterAlterQuery.h +++ b/dbms/include/DB/Interpreters/InterpreterAlterQuery.h @@ -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 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); }; } diff --git a/dbms/include/DB/Interpreters/InterpreterCheckQuery.h b/dbms/include/DB/Interpreters/InterpreterCheckQuery.h new file mode 100644 index 00000000000..48c6092352e --- /dev/null +++ b/dbms/include/DB/Interpreters/InterpreterCheckQuery.h @@ -0,0 +1,22 @@ +#pragma once + +#include +#include + +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; +}; + +} diff --git a/dbms/include/DB/Interpreters/Join.h b/dbms/include/DB/Interpreters/Join.h index 71d24be652e..4a0f562c0e1 100644 --- a/dbms/include/DB/Interpreters/Join.h +++ b/dbms/include/DB/Interpreters/Join.h @@ -1,7 +1,5 @@ #pragma once -#include - #include #include diff --git a/dbms/include/DB/Interpreters/Set.h b/dbms/include/DB/Interpreters/Set.h index ca7f9d6313f..1c8024f490f 100644 --- a/dbms/include/DB/Interpreters/Set.h +++ b/dbms/include/DB/Interpreters/Set.h @@ -2,8 +2,6 @@ #include -#include - #include #include diff --git a/dbms/include/DB/Interpreters/Settings.h b/dbms/include/DB/Interpreters/Settings.h index 2057cf720c9..9eb46a7b102 100644 --- a/dbms/include/DB/Interpreters/Settings.h +++ b/dbms/include/DB/Interpreters/Settings.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) \ diff --git a/dbms/include/DB/Parsers/ASTAlterQuery.h b/dbms/include/DB/Parsers/ASTAlterQuery.h index 3454be99222..897db72a0b9 100644 --- a/dbms/include/DB/Parsers/ASTAlterQuery.h +++ b/dbms/include/DB/Parsers/ASTAlterQuery.h @@ -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 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_) {}; /** Получить текст, который идентифицирует этот элемент. */ diff --git a/dbms/include/DB/Parsers/ASTCheckQuery.h b/dbms/include/DB/Parsers/ASTCheckQuery.h new file mode 100644 index 00000000000..252f077374d --- /dev/null +++ b/dbms/include/DB/Parsers/ASTCheckQuery.h @@ -0,0 +1,25 @@ +#pragma once + +#include + +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; +}; + +} diff --git a/dbms/include/DB/Parsers/ParserAlterQuery.h b/dbms/include/DB/Parsers/ParserAlterQuery.h index d027a976c87..d871a78ce55 100644 --- a/dbms/include/DB/Parsers/ParserAlterQuery.h +++ b/dbms/include/DB/Parsers/ParserAlterQuery.h @@ -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 { diff --git a/dbms/include/DB/Parsers/ParserCheckQuery.h b/dbms/include/DB/Parsers/ParserCheckQuery.h new file mode 100644 index 00000000000..42832ffc5ed --- /dev/null +++ b/dbms/include/DB/Parsers/ParserCheckQuery.h @@ -0,0 +1,17 @@ +#pragma once + +#include + +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); +}; + +} diff --git a/dbms/include/DB/Parsers/formatAST.h b/dbms/include/DB/Parsers/formatAST.h index 92efb23ffb7..a9be893e06c 100644 --- a/dbms/include/DB/Parsers/formatAST.h +++ b/dbms/include/DB/Parsers/formatAST.h @@ -24,6 +24,7 @@ #include #include #include +#include //#include @@ -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, diff --git a/dbms/include/DB/Storages/AlterCommands.h b/dbms/include/DB/Storages/AlterCommands.h index c2dc7485fa2..50650a1bc56 100644 --- a/dbms/include/DB/Storages/AlterCommands.h +++ b/dbms/include/DB/Storages/AlterCommands.h @@ -6,7 +6,7 @@ namespace DB { -/// Операция из запроса ALTER. Добавление столбцов типа Nested не развернуто в добавление отдельных столбцов. +/// Операция из запроса ALTER (кроме DROP PARTITION). Добавление столбцов типа Nested не развернуто в добавление отдельных столбцов. struct AlterCommand { enum Type diff --git a/dbms/include/DB/Storages/Distributed/DirectoryMonitor.h b/dbms/include/DB/Storages/Distributed/DirectoryMonitor.h new file mode 100644 index 00000000000..3438dfd9bb2 --- /dev/null +++ b/dbms/include/DB/Storages/Distributed/DirectoryMonitor.h @@ -0,0 +1,207 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ + template + 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(*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(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 lock{mutex}; + } + cond.notify_one(); + thread.join(); + } + +private: + void run() + { + std::unique_lock 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 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(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}; +}; + +} diff --git a/dbms/include/DB/Storages/Distributed/DistributedBlockOutputStream.h b/dbms/include/DB/Storages/Distributed/DistributedBlockOutputStream.h new file mode 100644 index 00000000000..1bddee6b5cc --- /dev/null +++ b/dbms/include/DB/Storages/Distributed/DistributedBlockOutputStream.h @@ -0,0 +1,217 @@ +#pragma once + +#include +#include + +#include +#include +#include +#include + +#include +#include + +#include +#include + +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 + static std::vector 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 filters(num_shards); + + /** Деление отрицательного числа с остатком на положительное, в C++ даёт отрицательный остаток. + * Для данной задачи это не подходит. Поэтому, будем обрабатывать знаковые типы как беззнаковые. + * Это даёт уже что-то совсем не похожее на деление с остатком, но подходящее для данной задачи. + */ + using UnsignedT = typename std::make_unsigned::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 *>(column)->getData(); + const auto shard_num = cluster.slot_to_shard[static_cast(data) % total_weight]; + + for (size_t i = 0; i < num_shards; ++i) + filters[i].assign(num_rows, static_cast(shard_num == i)); + } + else + { + const auto & data = typeid_cast *>(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(data[j]) % total_weight] == i; + } + } + + return filters; + } + + std::vector createFilters(Block block) + { + using create_filters_sig = std::vector(size_t, const IColumn *, const Cluster &); + /// hashmap of pointers to functions corresponding to each integral type + static std::unordered_map creators{ + { TypeName::get(), &createFiltersImpl }, + { TypeName::get(), &createFiltersImpl }, + { TypeName::get(), &createFiltersImpl }, + { TypeName::get(), &createFiltersImpl }, + { TypeName::get(), &createFiltersImpl }, + { TypeName::get(), &createFiltersImpl }, + { TypeName::get(), &createFiltersImpl }, + { TypeName::get(), &createFiltersImpl }, + }; + + 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 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 & 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; +}; + +} diff --git a/dbms/include/DB/Storages/Distributed/queryToString.h b/dbms/include/DB/Storages/Distributed/queryToString.h new file mode 100644 index 00000000000..9f1b243ed9f --- /dev/null +++ b/dbms/include/DB/Storages/Distributed/queryToString.h @@ -0,0 +1,14 @@ +#pragma once + +#include + +namespace DB +{ + inline std::string queryToString(const ASTPtr & query) + { + std::ostringstream s; + formatAST(*query, s, 0, false, true); + + return s.str(); + } +} diff --git a/dbms/include/DB/Storages/IStorage.h b/dbms/include/DB/Storages/IStorage.h index 97b07831a96..70ed1b158b4 100644 --- a/dbms/include/DB/Storages/IStorage.h +++ b/dbms/include/DB/Storages/IStorage.h @@ -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) {} diff --git a/dbms/include/DB/Storages/MergeTree/AbandonableLockInZooKeeper.h b/dbms/include/DB/Storages/MergeTree/AbandonableLockInZooKeeper.h index 09e7a67c653..6f60cd37b94 100644 --- a/dbms/include/DB/Storages/MergeTree/AbandonableLockInZooKeeper.h +++ b/dbms/include/DB/Storages/MergeTree/AbandonableLockInZooKeeper.h @@ -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); diff --git a/dbms/include/DB/Storages/MergeTree/ActiveDataPartSet.h b/dbms/include/DB/Storages/MergeTree/ActiveDataPartSet.h index 9543aa991e9..807087f7da9 100644 --- a/dbms/include/DB/Storages/MergeTree/ActiveDataPartSet.h +++ b/dbms/include/DB/Storages/MergeTree/ActiveDataPartSet.h @@ -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); diff --git a/dbms/include/DB/Storages/MergeTree/BackgroundProcessingPool.h b/dbms/include/DB/Storages/MergeTree/BackgroundProcessingPool.h index de2b10cf11e..b846a1e037d 100644 --- a/dbms/include/DB/Storages/MergeTree/BackgroundProcessingPool.h +++ b/dbms/include/DB/Storages/MergeTree/BackgroundProcessingPool.h @@ -16,15 +16,21 @@ namespace DB { -/** В нескольких потоках в бесконечном цикле выполняет указанные функции. +/** Используя фиксированное количество потоков, выполнять произвольное количество задач в бесконечном цикле. + * Предназначена для задач, выполняющих постоянную фоновую работу (например, слияния). + * Задача - функция, возвращающая bool - сделала ли она какую-либо работу. + * Если сделала - надо выполнить ещё раз. Если нет - надо подождать несколько секунд, или до события wake, и выполнить ещё раз. + * + * Также, задача во время выполнения может временно увеличить какой-либо счётчик, относящийся ко всем задачам + * - например, число одновременно идующих слияний. */ class BackgroundProcessingPool { public: typedef std::map Counters; - /** Используется изнутри таски. Позволяет инкрементировать какие-нибудь счетчики. - * После завершения таски, все изменения откатятся. + /** Используется изнутри задачи. Позволяет инкрементировать какие-нибудь счетчики. + * После завершения задачи, все изменения откатятся. * Например, чтобы можно было узнавать количество потоков, выполняющих большое слияние, * можно в таске, выполняющей большое слияние, инкрементировать счетчик. Декрементировать обратно его не нужно. */ @@ -57,10 +63,14 @@ public: /// Переставить таск в начало очереди и разбудить какой-нибудь поток. void wake() { + Poco::ScopedReadRWLock rlock(rwlock); + if (removed) + return; + std::unique_lock 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>::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 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 tlock(threads_mutex); - std::unique_lock 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 lock(mutex); return size; } - void setSleepTime(double seconds) - { - std::unique_lock lock(mutex); - sleep_seconds = seconds; - } - int getCounter(const String & name) { std::unique_lock lock(mutex); @@ -122,8 +114,6 @@ public: TaskHandle addTask(const Task & task) { - std::unique_lock 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 tlock(threads_mutex); - - /// Дождемся завершения всех выполнений этой задачи. + /// Дождёмся завершения всех выполнений этой задачи. { - Poco::ScopedWriteRWLock wlock(task->lock); + Poco::ScopedWriteRWLock wlock(task->rwlock); task->removed = true; } { std::unique_lock 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 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 Tasks; typedef std::vector 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(sleep_seconds)); + std::unique_lock lock(mutex); + wake_event.wait_for(lock, std::chrono::duration(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 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 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 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 lock(mutex); - wake_event.wait_for(lock, std::chrono::duration(sleep_seconds / tasks_count)); + wake_event.wait_for(lock, std::chrono::duration(sleep_seconds)); } } } diff --git a/dbms/include/DB/Storages/MergeTree/DiskSpaceMonitor.h b/dbms/include/DB/Storages/MergeTree/DiskSpaceMonitor.h index 5c81a0d4dfd..04a28e996c1 100644 --- a/dbms/include/DB/Storages/MergeTree/DiskSpaceMonitor.h +++ b/dbms/include/DB/Storages/MergeTree/DiskSpaceMonitor.h @@ -26,27 +26,51 @@ public: { try { - Poco::ScopedLock lock(DiskSpaceMonitor::reserved_bytes_mutex); + Poco::ScopedLock 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 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 lock(DiskSpaceMonitor::reserved_bytes_mutex); + Poco::ScopedLock 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 lock(reserved_bytes_mutex); + Poco::ScopedLock lock(mutex); if (reserved_bytes > res) res = 0; @@ -75,6 +99,18 @@ public: return res; } + static size_t getReservedSpace() + { + Poco::ScopedLock lock(mutex); + return reserved_bytes; + } + + static size_t getReservationCount() + { + Poco::ScopedLock 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; }; } diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeData.h b/dbms/include/DB/Storages/MergeTree/MergeTreeData.h index 0aa257df3a5..114158f5c65 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeData.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeData.h @@ -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 отображает файлы-столбцы на выходе выражения в новые файлы таблицы. diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeDataMerger.h b/dbms/include/DB/Storages/MergeTree/MergeTreeDataMerger.h index 39a117d3f06..a0c06ce5dcd 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeDataMerger.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeDataMerger.h @@ -1,6 +1,7 @@ #pragma once #include +#include 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); diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreePartChecker.h b/dbms/include/DB/Storages/MergeTree/MergeTreePartChecker.h index 4490cd9ebdb..bf653f2a7bb 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreePartChecker.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreePartChecker.h @@ -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); }; } diff --git a/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h b/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h index d95de7fce4e..4b16aa00d56 100644 --- a/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h +++ b/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h @@ -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); diff --git a/dbms/include/DB/Storages/StorageChunkRef.h b/dbms/include/DB/Storages/StorageChunkRef.h index d6e63744828..4b54ea61ecb 100644 --- a/dbms/include/DB/Storages/StorageChunkRef.h +++ b/dbms/include/DB/Storages/StorageChunkRef.h @@ -36,6 +36,8 @@ public: String source_database_name; String source_table_name; + + bool checkData() const override; private: String name; diff --git a/dbms/include/DB/Storages/StorageDistributed.h b/dbms/include/DB/Storages/StorageDistributed.h index 14367169ebd..f1a543bcddd 100644 --- a/dbms/include/DB/Storages/StorageDistributed.h +++ b/dbms/include/DB/Storages/StorageDistributed.h @@ -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> directory_monitors; }; } diff --git a/dbms/include/DB/Storages/StorageLog.h b/dbms/include/DB/Storages/StorageLog.h index 25bc55155ac..2cae126ca02 100644 --- a/dbms/include/DB/Storages/StorageLog.h +++ b/dbms/include/DB/Storages/StorageLog.h @@ -13,6 +13,7 @@ #include #include #include +#include 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 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 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 file_checker; + +private: /** Для обычных столбцов, в засечках указано количество строчек в блоке. * Для столбцов-массивов и вложенных структур, есть более одной группы засечек, соответствующих разным файлам: * - для внутренностей (файла name.bin) - указано суммарное количество элементов массивов в блоке, diff --git a/dbms/include/DB/Storages/StorageMaterializedView.h b/dbms/include/DB/Storages/StorageMaterializedView.h index bbe2807270f..4cea9d804e0 100644 --- a/dbms/include/DB/Storages/StorageMaterializedView.h +++ b/dbms/include/DB/Storages/StorageMaterializedView.h @@ -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(); diff --git a/dbms/include/DB/Storages/StorageReplicatedMergeTree.h b/dbms/include/DB/Storages/StorageReplicatedMergeTree.h index 9acc7d1cf35..bd8fd81a525 100644 --- a/dbms/include/DB/Storages/StorageReplicatedMergeTree.h +++ b/dbms/include/DB/Storages/StorageReplicatedMergeTree.h @@ -6,6 +6,7 @@ #include #include #include +#include "MergeTree/AbandonableLockInZooKeeper.h" #include #include #include @@ -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 lock(storage.queue_mutex); + std::unique_lock 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 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 LogEntries; + typedef LogEntry::Ptr LogEntryPtr; + + typedef std::list LogEntries; typedef std::set StringSet; typedef std::list 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 unreplicated_data; std::unique_ptr unreplicated_reader; std::unique_ptr 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); }; } diff --git a/dbms/include/DB/Storages/StorageSystemNumbers.h b/dbms/include/DB/Storages/StorageSystemNumbers.h index 8acedef4872..b6eb8d9d665 100644 --- a/dbms/include/DB/Storages/StorageSystemNumbers.h +++ b/dbms/include/DB/Storages/StorageSystemNumbers.h @@ -3,7 +3,6 @@ #include #include -#include 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_); }; } diff --git a/dbms/include/DB/Storages/StorageTinyLog.h b/dbms/include/DB/Storages/StorageTinyLog.h index 0f00aac036d..d91cc88d9a3 100644 --- a/dbms/include/DB/Storages/StorageTinyLog.h +++ b/dbms/include/DB/Storages/StorageTinyLog.h @@ -12,6 +12,8 @@ #include #include #include +#include +#include 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 Files_t; + + Files_t & getFiles(); + +private: + String path; + String name; + NamesAndTypesListPtr columns; + + size_t max_compress_block_size; + Files_t files; + FileChecker 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); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp index e93f9fb9304..2229437e4a7 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -1,9 +1,7 @@ #include #include #include -#include -#include -#include +#include #include #include #include @@ -69,6 +67,7 @@ static IAggregateFunction * createWithNumericType(const IDataType & argument_typ return nullptr; } + template