diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h index 86a75c339e0..e61997c5d82 100644 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h @@ -15,7 +15,7 @@ #include #include -#include +#include namespace DB diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionSequenceMatch.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionSequenceMatch.h index 87f86ded3f8..9800a1cfe3b 100644 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionSequenceMatch.h +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionSequenceMatch.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/include/DB/Client/ConnectionPool.h b/dbms/include/DB/Client/ConnectionPool.h index c4dd3d69a60..7b608ca1d8a 100644 --- a/dbms/include/DB/Client/ConnectionPool.h +++ b/dbms/include/DB/Client/ConnectionPool.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include diff --git a/dbms/include/DB/Client/ConnectionPoolWithFailover.h b/dbms/include/DB/Client/ConnectionPoolWithFailover.h index e2f42366341..be81f55001d 100644 --- a/dbms/include/DB/Client/ConnectionPoolWithFailover.h +++ b/dbms/include/DB/Client/ConnectionPoolWithFailover.h @@ -3,7 +3,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/include/DB/Columns/ColumnAggregateFunction.h b/dbms/include/DB/Columns/ColumnAggregateFunction.h index 68bf2970a4f..e9e0522a115 100644 --- a/dbms/include/DB/Columns/ColumnAggregateFunction.h +++ b/dbms/include/DB/Columns/ColumnAggregateFunction.h @@ -182,6 +182,16 @@ public: throw Exception("Method insertDefault is not supported for ColumnAggregateFunction.", ErrorCodes::NOT_IMPLEMENTED); } + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override + { + throw Exception("Method serializeValueIntoArena is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + + const char * deserializeAndInsertFromArena(const char * pos) override + { + throw Exception("Method deserializeAndInsertFromArena is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + size_t byteSize() const override { return getData().size() * sizeof(getData()[0]); diff --git a/dbms/include/DB/Columns/ColumnArray.h b/dbms/include/DB/Columns/ColumnArray.h index 170dafce004..7da55b4c769 100644 --- a/dbms/include/DB/Columns/ColumnArray.h +++ b/dbms/include/DB/Columns/ColumnArray.h @@ -4,8 +4,9 @@ #include -#include +#include #include +#include #include #include @@ -119,6 +120,33 @@ public: getOffsets().push_back((getOffsets().size() == 0 ? 0 : getOffsets().back()) + elems); } + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override + { + size_t array_size = sizeAt(n); + size_t offset = offsetAt(n); + + char * pos = arena.allocContinue(sizeof(array_size), begin); + memcpy(pos, &array_size, sizeof(array_size)); + + size_t values_size = 0; + for (size_t i = 0; i < array_size; ++i) + values_size += data->serializeValueIntoArena(offset + i, arena, begin).size; + + return StringRef(begin, sizeof(array_size) + values_size); + } + + const char * deserializeAndInsertFromArena(const char * pos) override + { + size_t array_size = *reinterpret_cast(pos); + pos += sizeof(array_size); + + for (size_t i = 0; i < array_size; ++i) + pos = data->deserializeAndInsertFromArena(pos); + + getOffsets().push_back((getOffsets().size() == 0 ? 0 : getOffsets().back()) + array_size); + return pos; + } + ColumnPtr cut(size_t start, size_t length) const override; void insert(const Field & x) override diff --git a/dbms/include/DB/Columns/ColumnConst.h b/dbms/include/DB/Columns/ColumnConst.h index fad47e76f96..5099cce3935 100644 --- a/dbms/include/DB/Columns/ColumnConst.h +++ b/dbms/include/DB/Columns/ColumnConst.h @@ -3,7 +3,7 @@ #include #include -#include +#include #include #include #include @@ -89,6 +89,16 @@ public: void insertDefault() override { ++s; } + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override + { + throw Exception("Method serializeValueIntoArena is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + + const char * deserializeAndInsertFromArena(const char * pos) override + { + throw Exception("Method deserializeAndInsertFromArena is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + ColumnPtr filter(const Filter & filt) const override { if (s != filt.size()) diff --git a/dbms/include/DB/Columns/ColumnFixedString.h b/dbms/include/DB/Columns/ColumnFixedString.h index c2ecba68768..cd6c502974d 100644 --- a/dbms/include/DB/Columns/ColumnFixedString.h +++ b/dbms/include/DB/Columns/ColumnFixedString.h @@ -3,6 +3,7 @@ #include // memcpy #include +#include #include @@ -111,6 +112,21 @@ public: chars.resize_fill(chars.size() + n); } + StringRef serializeValueIntoArena(size_t index, Arena & arena, char const *& begin) const override + { + auto pos = arena.allocContinue(n, begin); + memcpy(pos, &chars[n * index], n); + return StringRef(pos, n); + } + + const char * deserializeAndInsertFromArena(const char * pos) override + { + size_t old_size = chars.size(); + chars.resize(old_size + n); + memcpy(&chars[old_size], pos, n); + return pos + n; + } + int compareAt(size_t p1, size_t p2, const IColumn & rhs_, int nan_direction_hint) const override { const ColumnFixedString & rhs = static_cast(rhs_); diff --git a/dbms/include/DB/Columns/ColumnNested.h b/dbms/include/DB/Columns/ColumnNested.h deleted file mode 100644 index 401ac076f3f..00000000000 --- a/dbms/include/DB/Columns/ColumnNested.h +++ /dev/null @@ -1,314 +0,0 @@ -#pragma once - -#include // memcpy - -#include - -#include -#include -#include - -#include -#include - - -namespace DB -{ - -using Poco::SharedPtr; - -/** Cтолбeц значений типа вложенная таблица. - * В памяти это выглядит, как столбцы вложенных типов одинковой длины, равной сумме размеров всех массивов с общим именем, - * и как общий для всех столбцов массив смещений, который позволяет достать каждый элемент. - * - * Не предназначен для возвращения результа в запросах SELECT. Предполагается, что для SELECT'а будут отдаваться - * столбцы вида ColumnArray, ссылающиеся на один массив Offset'ов и соответствующий массив с данными. - * - * Используется для сериализации вложенной таблицы. - */ -class ColumnNested final : public IColumn -{ -public: - /** По индексу i находится смещение до начала i + 1 -го элемента. */ - typedef ColumnVector ColumnOffsets_t; - - /** Создать пустой столбец вложенных таблиц, с типом значений, как в столбце nested_column */ - explicit ColumnNested(Columns nested_columns, ColumnPtr offsets_column = nullptr) - : data(nested_columns), offsets(offsets_column) - { - if (!offsets_column) - { - offsets = new ColumnOffsets_t; - } - else - { - if (!typeid_cast(&*offsets_column)) - throw Exception("offsets_column must be a ColumnVector", ErrorCodes::ILLEGAL_COLUMN); - } - } - - std::string getName() const override - { - std::string res; - { - WriteBufferFromString out(res); - - for (Columns::const_iterator it = data.begin(); it != data.end(); ++it) - { - if (it != data.begin()) - writeCString(", ", out); - writeString((*it)->getName(), out); - } - } - return "ColumnNested(" + res + ")"; - } - - ColumnPtr cloneEmpty() const override - { - Columns res(data.size()); - for (size_t i = 0; i < data.size(); ++i) - res[i] = data[i]->cloneEmpty(); - return new ColumnNested(res); - } - - size_t size() const override - { - return getOffsets().size(); - } - - Field operator[](size_t n) const override - { - throw Exception("Method operator[] is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); - } - - void get(size_t n, Field & res) const override - { - throw Exception("Method get is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); - } - - StringRef getDataAt(size_t n) const override - { - throw Exception("Method getDataAt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); - } - - void insertData(const char * pos, size_t length) override - { - throw Exception("Method insertData is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); - } - - ColumnPtr cut(size_t start, size_t length) const override - { - if (length == 0) - return new ColumnNested(data); - - if (start + length > getOffsets().size()) - throw Exception("Parameter out of bound in ColumnNested::cut() method.", - ErrorCodes::PARAMETER_OUT_OF_BOUND); - - size_t nested_offset = offsetAt(start); - size_t nested_length = getOffsets()[start + length - 1] - nested_offset; - - ColumnNested * res_ = new ColumnNested(data); - ColumnPtr res = res_; - - for (size_t i = 0; i < data.size(); ++i) - res_->data[i] = data[i]->cut(nested_offset, nested_length); - - Offsets_t & res_offsets = res_->getOffsets(); - - if (start == 0) - { - res_offsets.assign(getOffsets().begin(), getOffsets().begin() + length); - } - else - { - res_offsets.resize(length); - - for (size_t i = 0; i < length; ++i) - res_offsets[i] = getOffsets()[start + i] - nested_offset; - } - - return res; - } - - void insert(const Field & x) override - { - throw Exception("Method insert is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); - } - - void insertFrom(const IColumn & src_, size_t n) override - { - const ColumnNested & src = static_cast(src_); - - if (data.size() != src.getData().size()) - throw Exception("Number of columns in nested tables do not match.", ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH); - - size_t size = src.sizeAt(n); - size_t offset = src.offsetAt(n); - - for (size_t i = 0; i < data.size(); ++i) - { - if (data[i]->getName() != src.getData()[i]->getName()) - throw Exception("Types of columns in nested tables do not match.", ErrorCodes::TYPE_MISMATCH); - - for (size_t j = 0; j < size; ++j) - data[i]->insertFrom(*src.getData()[i], offset + j); - } - - getOffsets().push_back((getOffsets().size() == 0 ? 0 : getOffsets().back()) + size); - } - - void insertDefault() override - { - for (size_t i = 0; i < data.size(); ++i) - data[i]->insertDefault(); - getOffsets().push_back(getOffsets().size() == 0 ? 1 : (getOffsets().back() + 1)); - } - - ColumnPtr filter(const Filter & filt) const override - { - size_t size = getOffsets().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 ColumnNested(data); - - /// Не слишком оптимально. Можно сделать специализацию для массивов известных типов. - Filter nested_filt(getOffsets().back()); - for (size_t i = 0; i < size; ++i) - { - if (filt[i]) - memset(&nested_filt[offsetAt(i)], 1, sizeAt(i)); - else - memset(&nested_filt[offsetAt(i)], 0, sizeAt(i)); - } - - ColumnNested * res_ = new ColumnNested(data); - ColumnPtr res = res_; - for (size_t i = 0; i < data.size(); ++i) - res_->data[i] = data[i]->filter(nested_filt); - - Offsets_t & res_offsets = res_->getOffsets(); - res_offsets.reserve(size); - - size_t current_offset = 0; - for (size_t i = 0; i < size; ++i) - { - if (filt[i]) - { - current_offset += sizeAt(i); - res_offsets.push_back(current_offset); - } - } - - return res; - } - - ColumnPtr replicate(const Offsets_t & offsets) const override - { - throw Exception("Replication of ColumnNested is not implemented.", ErrorCodes::NOT_IMPLEMENTED); - } - - ColumnPtr permute(const Permutation & perm, size_t limit) const override - { - size_t size = getOffsets().size(); - if (size != perm.size()) - throw Exception("Size of permutation doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); - - if (limit == 0) - limit = size; - else - limit = std::min(size, limit); - - if (perm.size() < limit) - throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); - - if (limit == 0) - return new ColumnNested(data); - - Permutation nested_perm(getOffsets().back()); - - Columns cloned_columns(data.size()); - for (size_t i = 0; i < data.size(); ++i) - cloned_columns[i] = data[i]->cloneEmpty(); - - ColumnNested * res_ = new ColumnNested(cloned_columns); - ColumnPtr res = res_; - - Offsets_t & res_offsets = res_->getOffsets(); - res_offsets.resize(limit); - size_t current_offset = 0; - - for (size_t i = 0; i < limit; ++i) - { - for (size_t j = 0; j < sizeAt(perm[i]); ++j) - nested_perm[current_offset + j] = offsetAt(perm[i]) + j; - current_offset += sizeAt(perm[i]); - res_offsets[i] = current_offset; - } - - if (current_offset != 0) - for (size_t i = 0; i < data.size(); ++i) - res_->data[i] = data[i]->permute(nested_perm, current_offset); - - return res; - } - - int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override - { - throw Exception("Method compareAt is not supported for ColumnNested.", ErrorCodes::NOT_IMPLEMENTED); - } - - void getPermutation(bool reverse, size_t limit, Permutation & res) const override - { - throw Exception("Method getPermutation is not supported for ColumnNested.", ErrorCodes::NOT_IMPLEMENTED); - } - - void reserve(size_t n) override - { - getOffsets().reserve(n); - for (Columns::iterator it = data.begin(); it != data.end(); ++it) - (*it)->reserve(n); - } - - size_t byteSize() const override - { - size_t size = getOffsets().size() * sizeof(getOffsets()[0]); - for (Columns::const_iterator it = data.begin(); it != data.end(); ++it) - size += (*it)->byteSize(); - return size; - } - - void getExtremes(Field & min, Field & max) const override - { - throw Exception("Method getExtremes is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); - } - - /** Более эффективные методы манипуляции */ - Columns & getData() { return data; } - const Columns & getData() const { return data; } - - Offsets_t & ALWAYS_INLINE getOffsets() - { - return static_cast(*offsets.get()).getData(); - } - - const Offsets_t & ALWAYS_INLINE getOffsets() const - { - return static_cast(*offsets.get()).getData(); - } - - ColumnPtr & getOffsetsColumn() { return offsets; } - const ColumnPtr & getOffsetsColumn() const { return offsets; } - -private: - Columns data; - ColumnPtr offsets; - - size_t ALWAYS_INLINE offsetAt(size_t i) const { return i == 0 ? 0 : getOffsets()[i - 1]; } - size_t ALWAYS_INLINE sizeAt(size_t i) const { return i == 0 ? getOffsets()[0] : (getOffsets()[i] - getOffsets()[i - 1]); } -}; - - -} diff --git a/dbms/include/DB/Columns/ColumnString.h b/dbms/include/DB/Columns/ColumnString.h index 805a4b096ec..635bdcfbc02 100644 --- a/dbms/include/DB/Columns/ColumnString.h +++ b/dbms/include/DB/Columns/ColumnString.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -115,6 +116,34 @@ public: offsets.push_back((offsets.size() == 0 ? 0 : offsets.back()) + length); } + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override + { + size_t string_size = sizeAt(n); + size_t offset = offsetAt(n); + + StringRef res; + res.size = sizeof(string_size) + string_size; + char * pos = arena.allocContinue(res.size, begin); + memcpy(pos, &string_size, sizeof(string_size)); + memcpy(pos + sizeof(string_size), &chars[offset], string_size); + res.data = pos; + + return res; + } + + const char * deserializeAndInsertFromArena(const char * pos) override + { + size_t string_size = *reinterpret_cast(pos); + pos += sizeof(string_size); + + size_t old_size = chars.size(); + chars.resize(old_size + string_size); + memcpy(&chars[old_size], pos, string_size); + + offsets.push_back((offsets.size() == 0 ? 0 : offsets.back()) + string_size); + return pos + string_size; + } + ColumnPtr cut(size_t start, size_t length) const override { if (length == 0) diff --git a/dbms/include/DB/Columns/ColumnTuple.h b/dbms/include/DB/Columns/ColumnTuple.h index 4a85f036180..bfbbc4c4f59 100644 --- a/dbms/include/DB/Columns/ColumnTuple.h +++ b/dbms/include/DB/Columns/ColumnTuple.h @@ -42,8 +42,8 @@ public: { Array res; - for (Columns::const_iterator it = columns.begin(); it != columns.end(); ++it) - res.push_back((**it)[n]); + for (const auto & column : columns) + res.push_back((*column)[n]); return res; } @@ -93,8 +93,26 @@ public: void insertDefault() override { - for (Columns::iterator it = columns.begin(); it != columns.end(); ++it) - (*it)->insertDefault(); + for (auto & column : columns) + column->insertDefault(); + } + + + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override + { + size_t values_size = 0; + for (auto & column : columns) + values_size += column->serializeValueIntoArena(n, arena, begin).size; + + return StringRef(begin, values_size); + } + + const char * deserializeAndInsertFromArena(const char * pos) override + { + for (auto & column : columns) + pos = column->deserializeAndInsertFromArena(pos); + + return pos; } @@ -155,8 +173,8 @@ public: Less(const Columns & columns) { - for (Columns::const_iterator it = columns.begin(); it != columns.end(); ++it) - plain_columns.push_back(&**it); + for (const auto & column : columns) + plain_columns.push_back(column.get()); } bool operator() (size_t a, size_t b) const @@ -201,15 +219,15 @@ public: void reserve(size_t n) override { - for (Columns::iterator it = columns.begin(); it != columns.end(); ++it) - (*it)->reserve(n); + for (auto & column : columns) + column->reserve(n); } size_t byteSize() const override { size_t res = 0; - for (Columns::const_iterator it = columns.begin(); it != columns.end(); ++it) - res += (*it)->byteSize(); + for (const auto & column : columns) + res += column->byteSize(); return res; } diff --git a/dbms/include/DB/Columns/ColumnVector.h b/dbms/include/DB/Columns/ColumnVector.h index 078a474be2d..c7b85aff703 100644 --- a/dbms/include/DB/Columns/ColumnVector.h +++ b/dbms/include/DB/Columns/ColumnVector.h @@ -2,8 +2,9 @@ #include -#include +#include #include +#include #include @@ -150,6 +151,19 @@ public: data.push_back(T()); } + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override + { + auto pos = arena.allocContinue(sizeof(T), begin); + memcpy(pos, &data[n], sizeof(T)); + return StringRef(pos, sizeof(T)); + } + + const char * deserializeAndInsertFromArena(const char * pos) override + { + data.push_back(*reinterpret_cast(pos)); + return pos + sizeof(T); + } + size_t byteSize() const override { return data.size() * sizeof(data[0]); diff --git a/dbms/include/DB/Columns/IColumn.h b/dbms/include/DB/Columns/IColumn.h index 236b6fd49af..19565613140 100644 --- a/dbms/include/DB/Columns/IColumn.h +++ b/dbms/include/DB/Columns/IColumn.h @@ -6,7 +6,7 @@ #include #include -#include +#include #include #include @@ -22,6 +22,8 @@ typedef std::vector Columns; typedef std::vector ColumnPlainPtrs; typedef std::vector ConstColumnPlainPtrs; +class Arena; + /** Интерфейс для хранения столбцов значений в оперативке. */ @@ -134,6 +136,20 @@ public: */ virtual void insertDefault() = 0; + /** Сериализовать значение, расположив его в непрерывном куске памяти в Arena. + * Значение можно будет потом прочитать обратно. Используется для агрегации. + * Метод похож на getDataAt, но может работать для тех случаев, + * когда значение не однозначно соответствует какому-то уже существующему непрерывному куску памяти + * - например, для массива строк, чтобы получить однозначное соответствие, надо укладывать строки вместе с их размерами. + * Параметр begin - см. метод Arena::allocContinue. + */ + virtual StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const = 0; + + /** Десериализовать значение, которое было сериализовано с помощью serializeValueIntoArena. + * Вернуть указатель на позицию после прочитанных данных. + */ + virtual const char * deserializeAndInsertFromArena(const char * pos) = 0; + /** Соединить столбец с одним или несколькими другими. * Используется при склейке маленьких блоков. */ diff --git a/dbms/include/DB/Columns/IColumnDummy.h b/dbms/include/DB/Columns/IColumnDummy.h index d9c559f68f8..8b0a9a4a0ab 100644 --- a/dbms/include/DB/Columns/IColumnDummy.h +++ b/dbms/include/DB/Columns/IColumnDummy.h @@ -29,6 +29,16 @@ public: StringRef getDataAt(size_t n) const override { throw Exception("Method getDataAt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); } void insertData(const char * pos, size_t length) override { throw Exception("Method insertData is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); } + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override + { + throw Exception("Method serializeValueIntoArena is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + + const char * deserializeAndInsertFromArena(const char * pos) override + { + throw Exception("Method deserializeAndInsertFromArena is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + void getExtremes(Field & min, Field & max) const override { throw Exception("Method getExtremes is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); diff --git a/dbms/include/DB/Common/AIO.h b/dbms/include/DB/Common/AIO.h new file mode 100644 index 00000000000..85b9ea4149d --- /dev/null +++ b/dbms/include/DB/Common/AIO.h @@ -0,0 +1,52 @@ +#pragma once + +#include +#include +#include + +#include + +#include + + +/** Небольшие обёртки для асинхронного ввода-вывода. + */ + + +inline int io_setup(unsigned nr, aio_context_t *ctxp) +{ + return syscall(__NR_io_setup, nr, ctxp); +} + +inline int io_destroy(aio_context_t ctx) +{ + return syscall(__NR_io_destroy, ctx); +} + +inline int io_submit(aio_context_t ctx, long nr, struct iocb **iocbpp) +{ + return syscall(__NR_io_submit, ctx, nr, iocbpp); +} + +inline int io_getevents(aio_context_t ctx, long min_nr, long max_nr, io_event *events, struct timespec *timeout) +{ + return syscall(__NR_io_getevents, ctx, min_nr, max_nr, events, timeout); +} + + +struct AIOContext : private boost::noncopyable +{ + aio_context_t ctx; + + AIOContext(unsigned int nr_events = 128) + { + ctx = 0; + if (io_setup(nr_events, &ctx) < 0) + DB::throwFromErrno("io_setup failed"); + } + + ~AIOContext() + { + io_destroy(ctx); + } +}; diff --git a/dbms/include/DB/Common/Allocator.h b/dbms/include/DB/Common/Allocator.h index 524d88f010f..a79ef9ab800 100644 --- a/dbms/include/DB/Common/Allocator.h +++ b/dbms/include/DB/Common/Allocator.h @@ -5,7 +5,7 @@ #include #include -#include +#include #include diff --git a/dbms/include/DB/Common/Arena.h b/dbms/include/DB/Common/Arena.h index 28d40fed635..5ee75c57193 100644 --- a/dbms/include/DB/Common/Arena.h +++ b/dbms/include/DB/Common/Arena.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -84,7 +85,7 @@ private: } /// Добавить следующий непрерывный кусок памяти размера не меньше заданного. - void addChunk(size_t min_size) + void NO_INLINE addChunk(size_t min_size) { head = new Chunk(nextSize(min_size), head); size_in_bytes += head->size(); @@ -127,16 +128,15 @@ public: */ char * allocContinue(size_t size, char const *& begin) { - if (unlikely(head->pos + size > head->end)) + while (unlikely(head->pos + size > head->end)) { char * prev_end = head->pos; addChunk(size); if (begin) - { begin = insert(begin, prev_end - begin); - return allocContinue(size, begin); - } + else + break; } char * res = head->pos; diff --git a/dbms/include/DB/Common/Collator.h b/dbms/include/DB/Common/Collator.h index 012694e932a..c4fd0b3f788 100644 --- a/dbms/include/DB/Common/Collator.h +++ b/dbms/include/DB/Common/Collator.h @@ -3,7 +3,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/include/DB/Common/CombinedCardinalityEstimator.h b/dbms/include/DB/Common/CombinedCardinalityEstimator.h index 57050bbfb99..64502176009 100644 --- a/dbms/include/DB/Common/CombinedCardinalityEstimator.h +++ b/dbms/include/DB/Common/CombinedCardinalityEstimator.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include diff --git a/dbms/include/DB/Common/ConfigProcessor.h b/dbms/include/DB/Common/ConfigProcessor.h new file mode 100644 index 00000000000..439e9d22b82 --- /dev/null +++ b/dbms/include/DB/Common/ConfigProcessor.h @@ -0,0 +1,65 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +typedef Poco::AutoPtr ConfigurationPtr; +typedef Poco::AutoPtr XMLDocumentPtr; + +class ConfigProcessor +{ +public: + using Substitutions = std::vector >; + + /// log_to_console нужно использовать, если система логгирования еще не инициализирована. + ConfigProcessor(bool throw_on_bad_incl = false, bool log_to_console = false, const Substitutions & substitutions = Substitutions()); + + /** Выполняет подстановки в конфиге и возвращает XML-документ. + * + * Пусть в качестве path передана "/path/file.xml" + * 1) Объединяем xml-дерево из /path/file.xml со всеми деревьями из файлов /path/{conf,file}.d/ *.{conf,xml} + * Если у элемента есть атрибут replace, заменяем на него подходящий элемент. + * Если у элемента есть атрибут remove, удаляем подходящий элемент. + * Иначе объединяем детей рекурсивно. + * 2) Берем из конфига путь к файлу, из которого будем делать подстановки: /path2/metrika.xml. + * Если путь не указан, используем /etc/metrika.xml + * 3) Заменяем элементы вида "" на "содержимое элемента yandex.bar из metrika.xml" + * 4) Заменяет "" на "номер слоя из имени хоста" + */ + XMLDocumentPtr processConfig(const std::string & path); + + /** Делает processConfig и создает из результата Poco::Util::XMLConfiguration. + * Еще сохраняет результат в файл по пути, полученному из path приписыванием строки "-preprocessed" к имени файла. + */ + ConfigurationPtr loadConfig(const std::string & path); + +private: + Logger * log; + Poco::AutoPtr channel_ptr; + bool throw_on_bad_incl; + Substitutions substitutions; + + typedef XMLDocumentPtr DocumentPtr; + typedef Poco::AutoPtr NodePtr; + + void mergeRecursive(DocumentPtr config, Poco::XML::Node * config_node, Poco::XML::Node * with_node); + + void merge(DocumentPtr config, DocumentPtr with); + + std::string layerFromHost(); + + void doIncludesRecursive(DocumentPtr config, DocumentPtr include_from, Poco::XML::Node * node); + + void doIncludes(DocumentPtr config, DocumentPtr include_from); +}; diff --git a/dbms/include/DB/Common/CounterInFile.h b/dbms/include/DB/Common/CounterInFile.h new file mode 100644 index 00000000000..23928c31386 --- /dev/null +++ b/dbms/include/DB/Common/CounterInFile.h @@ -0,0 +1,188 @@ +#pragma once + +#include +#include + +#include +#include + +#include +#include +#include +#include + +#include + +#include +#include +#include +#include +#include + +#include + +#define SMALL_READ_WRITE_BUFFER_SIZE 16 + + +/** Хранит в файле число. + * Предназначен для редких вызовов (не рассчитан на производительность). + */ +class CounterInFile +{ +public: + /// path - имя файла, включая путь + CounterInFile(const std::string & path_) : path(path_) {} + + /** Добавить delta к числу в файле и вернуть новое значение. + * Если параметр create_if_need не установлен в true, то + * в файле уже должно быть записано какое-нибудь число (если нет - создайте файл вручную с нулём). + * + * Для защиты от race condition-ов между разными процессами, используются файловые блокировки. + * (Но при первом создании файла race condition возможен, так что лучше создать файл заранее.) + * + * locked_callback вызывается при заблокированном файле со счетчиком. В него передается новое значение. + * locked_callback можно использовать, чтобы делать что-нибудь атомарно с увеличением счетчика (например, переименовывать файлы). + */ + template + Int64 add(Int64 delta, Callback && locked_callback, bool create_if_need = false) + { + Poco::ScopedLock lock(mutex); + + Int64 res = -1; + + bool file_doesnt_exists = !Poco::File(path).exists(); + if (file_doesnt_exists && !create_if_need) + { + throw Poco::Exception("File " + path + " does not exist. " + "You must create it manulally with appropriate value or 0 for first start."); + } + + int fd = open(path.c_str(), O_RDWR | O_CREAT, 0666); + if (-1 == fd) + DB::throwFromErrno("Cannot open file " + path); + + try + { + int flock_ret = flock(fd, LOCK_EX); + if (-1 == flock_ret) + DB::throwFromErrno("Cannot lock file " + path); + + if (!file_doesnt_exists) + { + DB::ReadBufferFromFileDescriptor rb(fd, SMALL_READ_WRITE_BUFFER_SIZE); + try + { + DB::readIntText(res, rb); + } + catch (const DB::Exception & e) + { + /// Более понятное сообщение об ошибке. + if (e.code() == DB::ErrorCodes::CANNOT_READ_ALL_DATA || e.code() == DB::ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF) + throw DB::Exception("File " + path + " is empty. You must fill it manually with appropriate value.", e.code()); + else + throw; + } + } + else + res = 0; + + if (delta || file_doesnt_exists) + { + res += delta; + + DB::WriteBufferFromFileDescriptor wb(fd, SMALL_READ_WRITE_BUFFER_SIZE); + wb.seek(0); + wb.truncate(); + DB::writeIntText(res, wb); + DB::writeChar('\n', wb); + wb.sync(); + } + + locked_callback(res); + } + catch (...) + { + close(fd); + throw; + } + + close(fd); + return res; + } + + Int64 add(Int64 delta, bool create_if_need = false) + { + return add(delta, &CounterInFile::doNothing, create_if_need); + } + + const std::string & getPath() const + { + return path; + } + + /// Изменить путь к файлу. + void setPath(std::string path_) + { + path = path_; + } + + // Не thread-safe и не синхронизирован между процессами. + void fixIfBroken(UInt64 value) + { + bool file_exists = Poco::File(path).exists(); + + int fd = open(path.c_str(), O_RDWR | O_CREAT, 0666); + if (-1 == fd) + DB::throwFromErrno("Cannot open file " + path); + + try + { + bool broken = true; + + if (file_exists) + { + DB::ReadBufferFromFileDescriptor rb(fd, SMALL_READ_WRITE_BUFFER_SIZE); + try + { + UInt64 current_value; + DB::readIntText(current_value, rb); + char c; + DB::readChar(c, rb); + if (rb.count() > 0 && c == '\n' && rb.eof()) + broken = false; + } + catch (const DB::Exception & e) + { + if (e.code() != DB::ErrorCodes::CANNOT_READ_ALL_DATA && e.code() != DB::ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF) + throw; + } + } + + if (broken) + { + DB::WriteBufferFromFileDescriptor wb(fd, SMALL_READ_WRITE_BUFFER_SIZE); + wb.seek(0); + wb.truncate(); + DB::writeIntText(value, wb); + DB::writeChar('\n', wb); + wb.sync(); + } + } + catch (...) + { + close(fd); + throw; + } + + close(fd); + } + +private: + std::string path; + Poco::FastMutex mutex; + + static void doNothing(UInt64 a) {} +}; + + +#undef SMALL_READ_WRITE_BUFFER_SIZE diff --git a/dbms/include/DB/Common/Exception.h b/dbms/include/DB/Common/Exception.h new file mode 100644 index 00000000000..28a1c82ccf2 --- /dev/null +++ b/dbms/include/DB/Common/Exception.h @@ -0,0 +1,113 @@ +#pragma once + +#include +#include + +#include +#include + +#include + +namespace Poco { class Logger; } + + +namespace DB +{ + +class Exception : public Poco::Exception +{ +public: + Exception(int code = 0) : Poco::Exception(code) {} + Exception(const std::string & msg, int code = 0) : Poco::Exception(msg, code) {} + Exception(const std::string & msg, const std::string & arg, int code = 0) : Poco::Exception(msg, arg, code) {} + Exception(const std::string & msg, const Exception & exc, int code = 0) : Poco::Exception(msg, exc, code), trace(exc.trace) {} + Exception(const Exception & exc) : Poco::Exception(exc), trace(exc.trace) {} + explicit Exception(const Poco::Exception & exc) : Poco::Exception(exc.displayText()) {} + ~Exception() throw() override {} + Exception & operator = (const Exception & exc) + { + Poco::Exception::operator=(exc); + trace = exc.trace; + return *this; + } + const char * name() const throw() override { return "DB::Exception"; } + const char * className() const throw() override { return "DB::Exception"; } + DB::Exception * clone() const override { return new DB::Exception(*this); } + void rethrow() const override { throw *this; } + + /// Дописать к существующему сообщению что-нибудь ещё. + void addMessage(const std::string & arg) { extendedMessage(arg); } + + const StackTrace & getStackTrace() const { return trace; } + +private: + StackTrace trace; +}; + + +/// Содержит дополнительный член saved_errno. См. функцию throwFromErrno. +class ErrnoException : public Exception +{ +public: + ErrnoException(int code = 0, int saved_errno_ = 0) + : Exception(code), saved_errno(saved_errno_) {} + ErrnoException(const std::string & msg, int code = 0, int saved_errno_ = 0) + : Exception(msg, code), saved_errno(saved_errno_) {} + ErrnoException(const std::string & msg, const std::string & arg, int code = 0, int saved_errno_ = 0) + : Exception(msg, arg, code), saved_errno(saved_errno_) {} + ErrnoException(const std::string & msg, const Exception & exc, int code = 0, int saved_errno_ = 0) + : Exception(msg, exc, code), saved_errno(saved_errno_) {} + ErrnoException(const ErrnoException & exc) + : Exception(exc), saved_errno(exc.saved_errno) {} + + int getErrno() const { return saved_errno; } + +private: + int saved_errno; +}; + + +typedef std::vector Exceptions; + + +void throwFromErrno(const std::string & s, int code = 0, int the_errno = errno); + + +/** Попробовать записать исключение в лог (и забыть про него). + * Можно использовать в деструкторах в блоке catch (...). + */ +void tryLogCurrentException(const char * log_name, const std::string & start_of_message = ""); +void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message = ""); + +std::string getCurrentExceptionMessage(bool with_stacktrace); + + +void tryLogException(std::exception_ptr e, const char * log_name, const std::string & start_of_message = ""); +void tryLogException(std::exception_ptr e, Poco::Logger * logger, const std::string & start_of_message = ""); + +std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace); + + +void rethrowFirstException(Exceptions & exceptions); + +Poco::SharedPtr convertCurrentException(); + + +template +typename std::enable_if::value, T>::type exception_cast(std::exception_ptr e) +{ + try + { + std::rethrow_exception(e); + } + catch (typename std::remove_pointer::type & concrete) + { + return &concrete; + } + catch (...) + { + return nullptr; + } +} + +} diff --git a/dbms/include/DB/Common/ExternalTable.h b/dbms/include/DB/Common/ExternalTable.h index f73db238821..d3a9006cc6d 100644 --- a/dbms/include/DB/Common/ExternalTable.h +++ b/dbms/include/DB/Common/ExternalTable.h @@ -14,7 +14,7 @@ #include #include #include -#include +#include namespace DB diff --git a/dbms/include/DB/Common/HTMLForm.h b/dbms/include/DB/Common/HTMLForm.h new file mode 100644 index 00000000000..7bf3048f3c1 --- /dev/null +++ b/dbms/include/DB/Common/HTMLForm.h @@ -0,0 +1,26 @@ +#pragma once + +#include +#include +#include +#include + + +/** Почему-то при методе POST, Poco::Net::HTMLForm не считывает параметры из URL, а считывает только из тела. + * Этот помошник позволяет считывать параметры только из URL. + */ +struct HTMLForm : public Poco::Net::HTMLForm +{ + HTMLForm(Poco::Net::HTTPRequest & request) + { + Poco::URI uri(request.getURI()); + std::istringstream istr(uri.getRawQuery()); + readUrl(istr); + } + + HTMLForm(Poco::URI & uri) + { + std::istringstream istr(uri.getRawQuery()); + readUrl(istr); + } +}; diff --git a/dbms/include/DB/Common/HashTable/HashTable.h b/dbms/include/DB/Common/HashTable/HashTable.h index 5d3713e95bc..d7c1058e0f0 100644 --- a/dbms/include/DB/Common/HashTable/HashTable.h +++ b/dbms/include/DB/Common/HashTable/HashTable.h @@ -15,7 +15,7 @@ #include #include -#include +#include #include #include @@ -29,7 +29,7 @@ #ifdef DBMS_HASH_MAP_DEBUG_RESIZES #include #include - #include + #include #endif diff --git a/dbms/include/DB/Common/HashTable/HashTableAllocator.h b/dbms/include/DB/Common/HashTable/HashTableAllocator.h index 5c36857406c..91f501672a5 100644 --- a/dbms/include/DB/Common/HashTable/HashTableAllocator.h +++ b/dbms/include/DB/Common/HashTable/HashTableAllocator.h @@ -5,7 +5,7 @@ #include #include -#include +#include #include /** При использовании HashTableAllocatorWithStackMemory, размещённом на стеке, diff --git a/dbms/include/DB/Common/HyperLogLogBiasEstimator.h b/dbms/include/DB/Common/HyperLogLogBiasEstimator.h new file mode 100644 index 00000000000..c19a1a8c350 --- /dev/null +++ b/dbms/include/DB/Common/HyperLogLogBiasEstimator.h @@ -0,0 +1,109 @@ +#pragma once + +#include +#include + +#include +#include +#include +#include + +/** Этот класс предоставляет способ, чтобы оценить погрешность результата применения алгоритма HyperLogLog. + * Эмирические наблюдения показывают, что большие погрешности возникают при E < 5 * 2^precision, где + * E - возвращаемое значение алгоритмом HyperLogLog, и precision - параметр точности HyperLogLog. + * См. "HyperLogLog in Practice: Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm". + * (S. Heule et al., Proceedings of the EDBT 2013 Conference). + */ +template +class HyperLogLogBiasEstimator +{ +public: + static constexpr bool isTrivial() + { + return false; + } + + /// Предельное количество уникальных значений до которого должна примениться поправка + /// из алгоритма LinearCounting. + static double getThreshold() + { + return BiasData::getThreshold(); + } + + /// Вернуть оценку погрешности. + static double getBias(double raw_estimate) + { + const auto & estimates = BiasData::getRawEstimates(); + const auto & biases = BiasData::getBiases(); + + auto it = std::lower_bound(estimates.begin(), estimates.end(), raw_estimate); + + if (it == estimates.end()) + { + return biases[estimates.size() - 1]; + } + else if (*it == raw_estimate) + { + size_t index = std::distance(estimates.begin(), it); + return biases[index]; + } + else if (it == estimates.begin()) + { + return biases[0]; + } + else + { + /// Получаем оценку погрешности путём линейной интерполяции. + size_t index = std::distance(estimates.begin(), it); + + double estimate1 = estimates[index - 1]; + double estimate2 = estimates[index]; + + double bias1 = biases[index - 1]; + double bias2 = biases[index]; + /// Предполагается, что условие estimate1 < estimate2 всегда выполнено. + double slope = (bias2 - bias1) / (estimate2 - estimate1); + + return bias1 + slope * (raw_estimate - estimate1); + } + } + +private: + /// Статические проверки. + using TRawEstimatesRef = decltype(BiasData::getRawEstimates()); + using TRawEstimates = typename std::remove_reference::type; + + using TBiasDataRef = decltype(BiasData::getBiases()); + using TBiasData = typename std::remove_reference::type; + + static_assert(std::is_same::value, "Bias estimator data have inconsistent types"); + static_assert(std::tuple_size::value > 0, "Bias estimator has no raw estimate data"); + static_assert(std::tuple_size::value > 0, "Bias estimator has no bias data"); + static_assert(std::tuple_size::value == std::tuple_size::value, + "Bias estimator has inconsistent data"); +}; + +/** Тривиальный случай HyperLogLogBiasEstimator: употребляется, если не хотим исправить + * погрешность. Это имеет смысль при маленьких значениях параметра точности, например 5 или 12. + * Тогда применяются поправки из оригинальной версии алгоритма HyperLogLog. + * См. "HyperLogLog: The analysis of a near-optimal cardinality estimation algorithm" + * (P. Flajolet et al., AOFA '07: Proceedings of the 2007 International Conference on Analysis + * of Algorithms) + */ +struct TrivialBiasEstimator +{ + static constexpr bool isTrivial() + { + return true; + } + + static double getThreshold() + { + return 0.0; + } + + static double getBias(double raw_estimate) + { + return 0.0; + } +}; diff --git a/dbms/include/DB/Common/HyperLogLogCounter.h b/dbms/include/DB/Common/HyperLogLogCounter.h new file mode 100644 index 00000000000..d37b9127bf5 --- /dev/null +++ b/dbms/include/DB/Common/HyperLogLogCounter.h @@ -0,0 +1,727 @@ +#pragma once + +#include +#include +#include + +#include +#include +#include +#include +#include + +#include +#include + +namespace details +{ + +/// Look-up table логарифмов от целых чисел для использования в HyperLogLogCounter. +template +struct LogLUT +{ + LogLUT() + { + log_table[0] = 0.0; + for (size_t i = 1; i <= M; ++i) + log_table[i] = log(static_cast(i)); + } + + double getLog(size_t x) const + { + if (x <= M) + return log_table[x]; + else + return log(static_cast(x)); + } + +private: + static constexpr size_t M = 1 << ((static_cast(K) <= 12) ? K : 12); + + double log_table[M + 1]; +}; + +template struct MinCounterTypeHelper; +template<> struct MinCounterTypeHelper<0> { typedef UInt8 Type; }; +template<> struct MinCounterTypeHelper<1> { typedef UInt16 Type; }; +template<> struct MinCounterTypeHelper<2> { typedef UInt32 Type; }; +template<> struct MinCounterTypeHelper<3> { typedef UInt64 Type; }; + +/// Вспомогательная структура для автоматического определения +/// минимального размера типа счетчика в зависимости от максимального значения. +/// Используется там, где нужна максимальная экономия памяти, +/// например, в HyperLogLogCounter +template struct MinCounterType +{ + typedef typename MinCounterTypeHelper< + (MaxValue >= 1 << 8) + + (MaxValue >= 1 << 16) + + (MaxValue >= 1ULL << 32) + >::Type Type; +}; + +/** Компактный массив для хранения данных, размер L, в битах, которых составляет меньше одного байта. + * Вместо того, чтобы хранить каждое значение в 8-битную ячейку памяти, что приводит к растрате + * 37.5% пространства для L=5, CompactArray хранит смежные L-битные значения, именно компактные + * ячейки в массиве байтов, т.е. фактически CompactArray симулирует массив L-битных значений. + */ +template +class __attribute__ ((packed)) CompactArray final +{ +public: + class Locus; + +public: + CompactArray() = default; + + UInt8 ALWAYS_INLINE operator[](BucketIndex bucket_index) const + { + Locus locus(bucket_index); + + if (locus.index_l == locus.index_r) + return locus.read(bitset[locus.index_l]); + else + return locus.read(bitset[locus.index_l], bitset[locus.index_r]); + } + + Locus ALWAYS_INLINE operator[](BucketIndex bucket_index) + { + Locus locus(bucket_index); + + locus.content_l = &bitset[locus.index_l]; + + if (locus.index_l == locus.index_r) + locus.content_r = locus.content_l; + else + locus.content_r = &bitset[locus.index_r]; + + return locus; + } + + void readText(DB::ReadBuffer & in) + { + for (size_t i = 0; i < BITSET_SIZE; ++i) + { + if (i != 0) + DB::assertString(",", in); + DB::readIntText(bitset[i], in); + } + } + + void writeText(DB::WriteBuffer & out) const + { + for (size_t i = 0; i < BITSET_SIZE; ++i) + { + if (i != 0) + writeCString(",", out); + DB::writeIntText(bitset[i], out); + } + } + +private: + /// число байт в битсете + static constexpr size_t BITSET_SIZE = (static_cast(bucket_count) * content_width + 7) / 8; + UInt8 bitset[BITSET_SIZE] = { 0 }; +}; + +/** Структура Locus содержит необходимую информацию, чтобы найти для каждой компактной ячейки + * соответствующие физическую ячейку и смещение, в битах, от начала ячейки. Поскольку в общем + * случае размер одной физической ячейки не делится на размер одной компактной ячейки, возможны + * случаи, когда одна компактная ячейка перекрывает две физические ячейки. Поэтому структура + * Locus содержит две пары (индекс, смещение). + */ +template +class CompactArray::Locus final +{ + friend class CompactArray; + +public: + ALWAYS_INLINE operator UInt8() const + { + if (content_l == content_r) + return read(*content_l); + else + return read(*content_l, *content_r); + } + + Locus ALWAYS_INLINE & operator=(UInt8 content) + { + if ((index_l == index_r) || (index_l == (BITSET_SIZE - 1))) + { + /// Компактная ячейка полностью влезает в одну физическую ячейку. + *content_l &= ~(((1 << content_width) - 1) << offset_l); + *content_l |= content << offset_l; + } + else + { + /// Компактная ячейка перекрывает две физические ячейки. + size_t left = 8 - offset_l; + + *content_l &= ~(((1 << left) - 1) << offset_l); + *content_l |= (content & ((1 << left) - 1)) << offset_l; + + *content_r &= ~((1 << offset_r) - 1); + *content_r |= content >> left; + } + + return *this; + } + +private: + Locus() = default; + + Locus(BucketIndex bucket_index) + { + size_t l = static_cast(bucket_index) * content_width; + index_l = l >> 3; + offset_l = l & 7; + + size_t r = static_cast(bucket_index + 1) * content_width; + index_r = r >> 3; + offset_r = r & 7; + } + + UInt8 ALWAYS_INLINE read(UInt8 value_l) const + { + /// Компактная ячейка полностью влезает в одну физическую ячейку. + return (value_l >> offset_l) & ((1 << content_width) - 1); + } + + UInt8 ALWAYS_INLINE read(UInt8 value_l, UInt8 value_r) const + { + /// Компактная ячейка перекрывает две физические ячейки. + return ((value_l >> offset_l) & ((1 << (8 - offset_l)) - 1)) + | ((value_r & ((1 << offset_r) - 1)) << (8 - offset_l)); + } + +private: + size_t index_l; + size_t offset_l; + size_t index_r; + size_t offset_r; + + UInt8 * content_l; + UInt8 * content_r; + + /// Проверки + static_assert((content_width > 0) && (content_width < 8), "Invalid parameter value"); + static_assert(bucket_count <= (std::numeric_limits::max() / content_width), "Invalid parameter value"); +}; + +/** Знаменатель формулы алгоритма HyperLogLog + */ +template +class __attribute__ ((packed)) Denominator; + +namespace +{ + +/// Возвращает true, если хранилище для рангов большое. +constexpr bool isBigRankStore(UInt8 precision) +{ + return precision >= 12; +} + +} + +/** Тип употребляемый для вычисления знаменателя. + */ +template +struct IntermediateDenominator; + +template <> +struct IntermediateDenominator +{ + using Type = double; +}; + +template <> +struct IntermediateDenominator +{ + using Type = long double; +}; + +/** "Лёгкая" реализация знаменателя формулы HyperLogLog. + * Занимает минимальный объём памяти, зато вычисления могут быть неустойчивы. + * Подходит, когда хранилище для рангов небольшое. + */ +template +class __attribute__ ((packed)) Denominator::type> +{ +private: + using T = typename IntermediateDenominator::Type; + +public: + Denominator(DenominatorType initial_value) + : denominator(initial_value) + { + } + +public: + inline void update(UInt8 cur_rank, UInt8 new_rank) + { + denominator -= static_cast(1.0) / (1ULL << cur_rank); + denominator += static_cast(1.0) / (1ULL << new_rank); + } + + inline void update(UInt8 rank) + { + denominator += static_cast(1.0) / (1ULL << rank); + } + + void clear() + { + denominator = 0; + } + + DenominatorType get() const + { + return denominator; + } + +private: + T denominator; +}; + +/** "Тяжёлая" версия знаменателя формулы HyperLogLog. + * Занимает больший объём памяти, чем лёгкая версия, зато вычисления всегда устойчивы. + * Подходит, когда хранилище для рангов довольно большое. + */ +template +class __attribute__ ((packed)) Denominator::type> +{ +public: + Denominator(DenominatorType initial_value) + { + rank_count[0] = initial_value; + } + + inline void update(UInt8 cur_rank, UInt8 new_rank) + { + --rank_count[cur_rank]; + ++rank_count[new_rank]; + } + + inline void update(UInt8 rank) + { + ++rank_count[rank]; + } + + void clear() + { + memset(rank_count, 0, size * sizeof(UInt32)); + } + + DenominatorType get() const + { + long double val = rank_count[size - 1]; + for (int i = size - 2; i >= 0; --i) + { + val /= 2.0; + val += rank_count[i]; + } + return val; + } + +private: + static constexpr size_t size = max_rank + 1; + UInt32 rank_count[size] = { 0 }; +}; + +/** Число хвостовых (младших) нулей. + */ +template +struct TrailingZerosCounter; + +template <> +struct TrailingZerosCounter +{ + static int apply(UInt32 val) + { + return __builtin_ctz(val); + } +}; + +template <> +struct TrailingZerosCounter +{ + static int apply(UInt64 val) + { + return __builtin_ctzll(val); + } +}; + +/** Размер счётчика ранга в битах. + */ +template +struct RankWidth; + +template <> +struct RankWidth +{ + static constexpr UInt8 get() + { + return 5; + } +}; + +template <> +struct RankWidth +{ + static constexpr UInt8 get() + { + return 6; + } +}; + +} + +/** Поведение класса HyperLogLogCounter. + */ +enum class HyperLogLogMode +{ + Raw, /// Применить алгоритм HyperLogLog без исправления погрешности + LinearCounting, /// Исправить погрешность по алгоритму LinearCounting + BiasCorrected, /// Исправить погрешность по алгоритму HyperLogLog++ + FullFeatured /// Исправить погрешность по алгоритму LinearCounting или HyperLogLog++ +}; + +/** Подсчёт уникальных значений алгоритмом HyperLogLog. + * + * Теоретическая относительная погрешность ~1.04 / sqrt(2^precision) + * precision - длина префикса хэш-функции для индекса (число ячеек M = 2^precision) + * Рекомендуемые значения precision: 3..20 + * + * Источник: "HyperLogLog: The analysis of a near-optimal cardinality estimation algorithm" + * (P. Flajolet et al., AOFA '07: Proceedings of the 2007 International Conference on Analysis + * of Algorithms) + */ +template < + UInt8 precision, + typename Hash = IntHash32, + typename HashValueType = UInt32, + typename DenominatorType = double, + typename BiasEstimator = TrivialBiasEstimator, + HyperLogLogMode mode = HyperLogLogMode::FullFeatured, + bool stable_denominator_if_big = true> +class __attribute__ ((packed)) HyperLogLogCounter : private Hash +{ +private: + /// Число ячеек. + static constexpr size_t bucket_count = 1ULL << precision; + /// Размер счётчика ранга в битах. + static constexpr UInt8 rank_width = details::RankWidth::get(); + +private: + using Value_t = UInt64; + using RankStore = details::CompactArray; + +public: + void insert(Value_t value) + { + HashValueType hash = getHash(value); + + /// Разбиваем хэш-значение на два подзначения. Первое из них является номером ячейки + /// в хранилище для рангов (rank_storage), а со второго вычисляем ранг. + HashValueType bucket = extractBitSequence(hash, 0, precision); + HashValueType tail = extractBitSequence(hash, precision, sizeof(HashValueType) * 8); + UInt8 rank = calculateRank(tail); + + /// Обновляем максимальный ранг для текущей ячейки. + update(bucket, rank); + } + + UInt32 size() const + { + /// Нормализующий коэффициент, входящий в среднее гармоническое. + static constexpr double alpha_m = + bucket_count == 2 ? 0.351 : + bucket_count == 4 ? 0.532 : + bucket_count == 8 ? 0.626 : + bucket_count == 16 ? 0.673 : + bucket_count == 32 ? 0.697 : + bucket_count == 64 ? 0.709 : 0.7213 / (1 + 1.079 / bucket_count); + + /** Среднее гармоническое по всем корзинам из величин 2^rank равно: + * bucket_count / ∑ 2^-rank_i. + * Величина ∑ 2^-rank_i - это denominator. + */ + + double raw_estimate = alpha_m * bucket_count * bucket_count / denominator.get(); + + double final_estimate = fixRawEstimate(raw_estimate); + + return static_cast(final_estimate + 0.5); + } + + void merge(const HyperLogLogCounter & rhs) + { + const auto & rhs_rank_store = rhs.rank_store; + for (HashValueType bucket = 0; bucket < bucket_count; ++bucket) + update(bucket, rhs_rank_store[bucket]); + } + + void read(DB::ReadBuffer & in) + { + in.readStrict(reinterpret_cast(this), sizeof(*this)); + } + + void readAndMerge(DB::ReadBuffer & in) + { + RankStore other; + in.readStrict(reinterpret_cast(&other), sizeof(RankStore)); + for (HashValueType bucket = 0; bucket < bucket_count; ++bucket) + { + UInt8 rank = other[bucket]; + update(bucket, rank); + } + + in.ignore(sizeof(DenominatorCalculatorType) + sizeof(ZerosCounterType)); + } + + static void skip(DB::ReadBuffer & in) + { + in.ignore(sizeof(RankStore) + sizeof(DenominatorCalculatorType) + sizeof(ZerosCounterType)); + } + + void write(DB::WriteBuffer & out) const + { + out.write(reinterpret_cast(this), sizeof(*this)); + } + + /// Запись и чтение в текстовом виде неэффективно (зато совместимо с OLAPServer-ом и Metrage). + void readText(DB::ReadBuffer & in) + { + rank_store.readText(in); + + zeros = 0; + denominator.clear(); + for (HashValueType bucket = 0; bucket < bucket_count; ++bucket) + { + UInt8 rank = rank_store[bucket]; + if (rank == 0) + ++zeros; + denominator.update(rank); + } + } + + static void skipText(DB::ReadBuffer & in) + { + UInt8 dummy; + for (size_t i = 0; i < RankStore::size(); ++i) + { + if (i != 0) + DB::assertString(",", in); + DB::readIntText(dummy, in); + } + } + + void writeText(DB::WriteBuffer & out) const + { + rank_store.writeText(out); + } + +private: + /// Извлечь подмножество битов [begin, end[. + inline HashValueType extractBitSequence(HashValueType val, UInt8 begin, UInt8 end) const + { + return (val >> begin) & ((1ULL << (end - begin)) - 1); + } + + /// Ранг = число хвостовых (младших) нулей + 1 + inline UInt8 calculateRank(HashValueType val) const + { + if (unlikely(val == 0)) + return max_rank; + + auto zeros_plus_one = details::TrailingZerosCounter::apply(val) + 1; + + if (unlikely(zeros_plus_one) > max_rank) + return max_rank; + + return zeros_plus_one; + } + + inline HashValueType getHash(Value_t key) const + { + return Hash::operator()(key); + } + + /// Обновить максимальный ранг для заданной ячейки. + void update(HashValueType bucket, UInt8 rank) + { + typename RankStore::Locus content = rank_store[bucket]; + UInt8 cur_rank = static_cast(content); + + if (rank > cur_rank) + { + if (cur_rank == 0) + --zeros; + denominator.update(cur_rank, rank); + content = rank; + } + } + + template + inline double fixRawEstimate(double raw_estimate, + typename std::enable_if<(mode2 == HyperLogLogMode::Raw) + || ((mode2 == HyperLogLogMode::BiasCorrected) + && BiasEstimator::isTrivial())>::type * = nullptr) const + { + return raw_estimate; + } + + template + inline double fixRawEstimate(double raw_estimate, + typename std::enable_if<(mode2 == HyperLogLogMode::LinearCounting)>::type * = nullptr) const + { + return applyLinearCorrection(raw_estimate); + } + + template + inline double fixRawEstimate(double raw_estimate, + typename std::enable_if<(mode2 == HyperLogLogMode::BiasCorrected) + && !BiasEstimator::isTrivial()>::type * = nullptr) const + { + return applyBiasCorrection(raw_estimate); + } + + template + double fixRawEstimate(double raw_estimate, + typename std::enable_if<(mode2 == HyperLogLogMode::FullFeatured)>::type * = nullptr) const + { + static constexpr bool fix_big_cardinalities = std::is_same::value; + static constexpr double pow2_32 = 4294967296.0; + + double fixed_estimate; + + if (fix_big_cardinalities && (raw_estimate > (pow2_32 / 30.0))) + fixed_estimate = -pow2_32 * log(1.0 - raw_estimate / pow2_32); + else + fixed_estimate = applyCorrection(raw_estimate); + + return fixed_estimate; + } + + template + inline double applyCorrection(double raw_estimate, typename std::enable_if::type * = nullptr) const + { + double fixed_estimate; + + if (raw_estimate <= (2.5 * bucket_count)) + { + /// Поправка в случае маленкой оценки. + fixed_estimate = applyLinearCorrection(raw_estimate); + } + else + fixed_estimate = raw_estimate; + + return fixed_estimate; + } + + template + inline double applyCorrection(double raw_estimate, typename std::enable_if::type * = nullptr) const + { + double fixed_estimate = applyBiasCorrection(raw_estimate); + double linear_estimate = applyLinearCorrection(fixed_estimate); + + if (linear_estimate < BiasEstimator::getThreshold()) + fixed_estimate = linear_estimate; + + return fixed_estimate; + } + + /// Поправка из алгоритма HyperLogLog++. + /// Источник: "HyperLogLog in Practice: Algorithmic Engineering of a State of The Art + /// Cardinality Estimation Algorithm". + /// (S. Heule et al., Proceedings of the EDBT 2013 Conference). + inline double applyBiasCorrection(double raw_estimate) const + { + double fixed_estimate; + + if (raw_estimate <= (5 * bucket_count)) + fixed_estimate = raw_estimate - BiasEstimator::getBias(raw_estimate); + else + fixed_estimate = raw_estimate; + + return fixed_estimate; + } + + /// Подсчет уникальных значений по алгоритму LinearCounting. + /// Источник: "A Linear-time Probabilistic Counting Algorithm for Database Applications" + /// (Whang et al., ACM Trans. Database Syst., pp. 208-229, 1990) + inline double applyLinearCorrection(double raw_estimate) const + { + double fixed_estimate; + + if (zeros != 0) + fixed_estimate = bucket_count * (log_lut.getLog(bucket_count) - log_lut.getLog(zeros)); + else + fixed_estimate = raw_estimate; + + return fixed_estimate; + } + +private: + /// Максимальный ранг. + static constexpr int max_rank = sizeof(HashValueType) * 8 - precision + 1; + + /// Хранилище для рангов. + RankStore rank_store; + + /// Знаменатель формулы алгоритма HyperLogLog. + using DenominatorCalculatorType = details::Denominator; + DenominatorCalculatorType denominator{bucket_count}; + + /// Число нулей в хранилище для рангов. + using ZerosCounterType = typename details::MinCounterType::Type; + ZerosCounterType zeros = bucket_count; + + static details::LogLUT log_lut; + + /// Проверки. + static_assert(precision < (sizeof(HashValueType) * 8), "Invalid parameter value"); +}; + + +/// Определения статических переменных, нужные во время линковки. +template +< + UInt8 precision, + typename Hash, + typename HashValueType, + typename DenominatorType, + typename BiasEstimator, + HyperLogLogMode mode, + bool stable_denominator_if_big +> +details::LogLUT HyperLogLogCounter +< + precision, + Hash, + HashValueType, + DenominatorType, + BiasEstimator, + mode, + stable_denominator_if_big +>::log_lut; + + +/// Для Metrage, используется лёгкая реализация знаменателя формулы HyperLogLog, +/// чтобы формат сериализации не изменился. +typedef HyperLogLogCounter< + 12, + IntHash32, + UInt32, + double, + TrivialBiasEstimator, + HyperLogLogMode::FullFeatured, + false +> HLL12; diff --git a/dbms/include/DB/Common/HyperLogLogWithSmallSetOptimization.h b/dbms/include/DB/Common/HyperLogLogWithSmallSetOptimization.h index 6e8467f0033..3d9cb473c25 100644 --- a/dbms/include/DB/Common/HyperLogLogWithSmallSetOptimization.h +++ b/dbms/include/DB/Common/HyperLogLogWithSmallSetOptimization.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include namespace DB diff --git a/dbms/include/DB/Common/Increment.h b/dbms/include/DB/Common/Increment.h new file mode 100644 index 00000000000..4580b7d7034 --- /dev/null +++ b/dbms/include/DB/Common/Increment.h @@ -0,0 +1,87 @@ +#pragma once + +#include + + +/** Позволяет получать авто-инкрементное число, храня его в файле. + * Предназначен для редких вызовов (не рассчитан на производительность). + */ +class Increment +{ +public: + /// path - имя файла, включая путь + Increment(const std::string & path_) : counter(path_) {} + + /** Получить следующее число. + * Если параметр create_if_need не установлен в true, то + * в файле уже должно быть записано какое-нибудь число (если нет - создайте файл вручную с нулём). + * + * Для защиты от race condition-ов между разными процессами, используются файловые блокировки. + * (Но при первом создании файла race condition возможен, так что лучше создать файл заранее.) + * + * locked_callback вызывается при заблокированном файле со счетчиком. В него передается новое значение. + * locked_callback можно использовать, чтобы делать что-нибудь атомарно с увеличением счетчика (например, переименовывать файлы). + */ + template + UInt64 get(Callback && locked_callback, bool create_if_need = false) + { + return static_cast(counter.add(1, std::forward(locked_callback), create_if_need)); + } + + UInt64 get(bool create_if_need = false) + { + return getBunch(1, create_if_need); + } + + /// Посмотреть следующее значение. + UInt64 peek(bool create_if_need = false) + { + return getBunch(0, create_if_need); + } + + /** Получить следующее число и увеличить счетчик на count. + * Если параметр create_if_need не установлен в true, то + * в файле уже должно быть записано какое-нибудь число (если нет - создайте файл вручную с нулём). + * + * Для защиты от race condition-ов между разными процессами, используются файловые блокировки. + * (Но при первом создании файла race condition возможен, так что лучше создать файл заранее.) + */ + UInt64 getBunch(UInt64 count, bool create_if_need = false) + { + return static_cast(counter.add(static_cast(count), create_if_need) - count + 1); + } + + /// Изменить путь к файлу. + void setPath(std::string path_) + { + counter.setPath(path_); + } + + void fixIfBroken(UInt64 value) + { + counter.fixIfBroken(value); + } + +private: + CounterInFile counter; +}; + + +/** То же самое, но без хранения в файле. + */ +struct SimpleIncrement : private boost::noncopyable +{ + UInt64 value; + + SimpleIncrement(UInt64 start = 0) : value(start) {} + + void set(UInt64 new_value) + { + value = new_value; + } + + UInt64 get() + { + return __sync_add_and_fetch(&value, 1); + } +}; diff --git a/dbms/include/DB/Common/LRUCache.h b/dbms/include/DB/Common/LRUCache.h index 4945b04bac6..0430f00360a 100644 --- a/dbms/include/DB/Common/LRUCache.h +++ b/dbms/include/DB/Common/LRUCache.h @@ -7,7 +7,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/dbms/include/DB/Common/NetException.h b/dbms/include/DB/Common/NetException.h new file mode 100644 index 00000000000..b09845b3703 --- /dev/null +++ b/dbms/include/DB/Common/NetException.h @@ -0,0 +1,27 @@ +#pragma once + +#include + + +namespace DB +{ +class NetException : public DB::Exception +{ +public: + explicit NetException(int code = 0) : DB::Exception(code) {} + NetException(const std::string & msg, int code = 0) : DB::Exception(msg, code) {} + NetException(const std::string & msg, const std::string & arg, int code = 0) : DB::Exception(msg, arg, code) {} + NetException(const std::string & msg, const DB::Exception & exc, int code = 0) : DB::Exception(msg, exc, code) {} + + explicit NetException(const DB::Exception & exc) : DB::Exception(exc) {} + explicit NetException(const Poco::Exception & exc) : DB::Exception(exc.displayText()) {} + NetException(const DB::NetException & exc) = default; + + ~NetException() throw() override {} + + const char * name() const throw() override { return "DB::NetException"; } + const char * className() const throw() override { return "DB::NetException"; } + DB::NetException * clone() const override { return new DB::NetException(*this); } + void rethrow() const override { throw *this; } +}; +} \ No newline at end of file diff --git a/dbms/include/DB/Common/OptimizedRegularExpression.h b/dbms/include/DB/Common/OptimizedRegularExpression.h new file mode 100644 index 00000000000..5e51420f2a8 --- /dev/null +++ b/dbms/include/DB/Common/OptimizedRegularExpression.h @@ -0,0 +1,106 @@ +#pragma once + +#include +#include +#include + +#include + +#include +#include + + +/** Использует два способа оптимизации регулярного выражения: + * 1. Если регулярное выражение является тривиальным (сводится к поиску подстроки в строке), + * то заменяет поиск на strstr или strcasestr. + * 2. Если регулярное выражение содержит безальтернативную подстроку достаточной длины, + * то перед проверкой используется strstr или strcasestr достаточной длины; + * регулярное выражение проверяется полностью только если подстрока найдена. + * 3. В остальных случаях, используется движок re2. + * + * Это имеет смысл, так как strstr и strcasestr в libc под Linux хорошо оптимизированы. + * + * Подходит, если одновременно выполнены следующие условия: + * - если в большинстве вызовов, регулярное выражение не матчится; + * - если регулярное выражение совместимо с движком re2; + * - можете использовать на свой риск, так как, возможно, не все случаи учтены. + */ + +namespace OptimizedRegularExpressionDetails +{ + struct Match + { + std::string::size_type offset; + std::string::size_type length; + }; +} + +template +class OptimizedRegularExpressionImpl +{ +public: + enum Options + { + RE_CASELESS = 0x00000001, + RE_NO_CAPTURE = 0x00000010, + RE_DOT_NL = 0x00000100 + }; + + using Match = OptimizedRegularExpressionDetails::Match; + typedef std::vector MatchVec; + + using RegexType = typename std::conditional::type; + using StringPieceType = typename std::conditional::type; + + OptimizedRegularExpressionImpl(const std::string & regexp_, int options = 0); + + bool match(const std::string & subject) const + { + return match(subject.data(), subject.size()); + } + + bool match(const std::string & subject, Match & match_) const + { + return match(subject.data(), subject.size(), match_); + } + + unsigned match(const std::string & subject, MatchVec & matches) const + { + return match(subject.data(), subject.size(), matches); + } + + unsigned match(const char * subject, size_t subject_size, MatchVec & matches) const + { + return match(subject, subject_size, matches, number_of_subpatterns + 1); + } + + bool match(const char * subject, size_t subject_size) const; + bool match(const char * subject, size_t subject_size, Match & match) const; + unsigned match(const char * subject, size_t subject_size, MatchVec & matches, unsigned limit) const; + + unsigned getNumberOfSubpatterns() const { return number_of_subpatterns; } + + /// Получить регексп re2 или nullptr, если шаблон тривиален (для вывода в лог). + const std::unique_ptr& getRE2() const { return re2; } + + static void analyze(const std::string & regexp_, std::string & required_substring, bool & is_trivial, bool & required_substring_is_prefix); + + void getAnalyzeResult(std::string & out_required_substring, bool & out_is_trivial, bool & out_required_substring_is_prefix) const + { + out_required_substring = required_substring; + out_is_trivial = is_trivial; + out_required_substring_is_prefix = required_substring_is_prefix; + } + +private: + bool is_trivial; + bool required_substring_is_prefix; + bool is_case_insensitive; + std::string required_substring; + std::unique_ptr re2; + unsigned number_of_subpatterns; +}; + +using OptimizedRegularExpression = OptimizedRegularExpressionImpl; + +#include "OptimizedRegularExpression.inl" diff --git a/dbms/include/DB/Common/OptimizedRegularExpression.inl b/dbms/include/DB/Common/OptimizedRegularExpression.inl new file mode 100644 index 00000000000..46790dbb2ac --- /dev/null +++ b/dbms/include/DB/Common/OptimizedRegularExpression.inl @@ -0,0 +1,410 @@ +#include + +#include + +#include + + +#define MIN_LENGTH_FOR_STRSTR 3 +#define MAX_SUBPATTERNS 5 + +template +void OptimizedRegularExpressionImpl::analyze(const std::string & regexp, std::string & required_substring, bool & is_trivial, bool & required_substring_is_prefix) +{ + /** Выражение тривиально, если в нём все метасимволы эскейплены. + * Безальтернативная строка - это + * строка вне скобок, + * в которой все метасимволы эскейплены, + * а также если вне скобок нет '|', + * а также избегаются подстроки вида http:// или www. + */ + const char * begin = regexp.data(); + const char * pos = begin; + const char * end = regexp.data() + regexp.size(); + int depth = 0; + is_trivial = true; + required_substring_is_prefix = false; + required_substring.clear(); + bool has_alternative_on_depth_0 = false; + + /// Подстрока с позицией. + typedef std::pair Substring; + + typedef std::vector Substrings; + Substrings trivial_substrings(1); + Substring * last_substring = &trivial_substrings.back(); + + bool in_curly_braces = false; + bool in_square_braces = false; + + while (pos != end) + { + switch (*pos) + { + case '\0': + pos = end; + break; + case '\\': + { + ++pos; + if (pos == end) + break; + + switch (*pos) + { + case '|': case '(': case ')': case '^': case '$': case '.': case '[': case '?': case '*': case '+': case '{': + if (depth == 0 && !in_curly_braces && !in_square_braces) + { + if (last_substring->first.empty()) + last_substring->second = pos - begin; + last_substring->first.push_back(*pos); + } + break; + default: + /// все остальные escape-последовательности не поддерживаем + is_trivial = false; + if (!last_substring->first.empty()) + { + trivial_substrings.resize(trivial_substrings.size() + 1); + last_substring = &trivial_substrings.back(); + } + break; + } + + ++pos; + break; + } + case '|': + if (depth == 0) + has_alternative_on_depth_0 = true; + is_trivial = false; + if (!in_square_braces && !last_substring->first.empty()) + { + trivial_substrings.resize(trivial_substrings.size() + 1); + last_substring = &trivial_substrings.back(); + } + ++pos; + break; + case '(': + if (!in_square_braces) + { + ++depth; + is_trivial = false; + if (!last_substring->first.empty()) + { + trivial_substrings.resize(trivial_substrings.size() + 1); + last_substring = &trivial_substrings.back(); + } + } + ++pos; + break; + case '[': + in_square_braces = true; + ++depth; + is_trivial = false; + if (!last_substring->first.empty()) + { + trivial_substrings.resize(trivial_substrings.size() + 1); + last_substring = &trivial_substrings.back(); + } + ++pos; + break; + case ']': + in_square_braces = false; + --depth; + is_trivial = false; + if (!last_substring->first.empty()) + { + trivial_substrings.resize(trivial_substrings.size() + 1); + last_substring = &trivial_substrings.back(); + } + ++pos; + break; + case ')': + if (!in_square_braces) + { + --depth; + is_trivial = false; + if (!last_substring->first.empty()) + { + trivial_substrings.resize(trivial_substrings.size() + 1); + last_substring = &trivial_substrings.back(); + } + } + ++pos; + break; + case '^': case '$': case '.': case '+': + is_trivial = false; + if (!last_substring->first.empty() && !in_square_braces) + { + trivial_substrings.resize(trivial_substrings.size() + 1); + last_substring = &trivial_substrings.back(); + } + ++pos; + break; + /// Квантификаторы, допускающие нулевое количество. + case '{': + in_curly_braces = true; + case '?': case '*': + is_trivial = false; + if (!last_substring->first.empty() && !in_square_braces) + { + last_substring->first.resize(last_substring->first.size() - 1); + trivial_substrings.resize(trivial_substrings.size() + 1); + last_substring = &trivial_substrings.back(); + } + ++pos; + break; + case '}': + in_curly_braces = false; + ++pos; + break; + default: + if (depth == 0 && !in_curly_braces && !in_square_braces) + { + if (last_substring->first.empty()) + last_substring->second = pos - begin; + last_substring->first.push_back(*pos); + } + ++pos; + break; + } + } + + if (last_substring && last_substring->first.empty()) + trivial_substrings.pop_back(); + + if (!is_trivial) + { + if (!has_alternative_on_depth_0) + { + /** Выберем безальтернативную подстроку максимальной длины, среди префиксов, + * или безальтернативную подстроку максимальной длины. + */ + size_t max_length = 0; + Substrings::const_iterator candidate_it = trivial_substrings.begin(); + for (Substrings::const_iterator it = trivial_substrings.begin(); it != trivial_substrings.end(); ++it) + { + if (((it->second == 0 && candidate_it->second != 0) + || ((it->second == 0) == (candidate_it->second == 0) && it->first.size() > max_length)) + /// Тюнинг для предметной области + && (it->first.size() > strlen("://") || strncmp(it->first.data(), "://", strlen("://"))) + && (it->first.size() > strlen("http://") || strncmp(it->first.data(), "http", strlen("http"))) + && (it->first.size() > strlen("www.") || strncmp(it->first.data(), "www", strlen("www"))) + && (it->first.size() > strlen("Windows ") || strncmp(it->first.data(), "Windows ", strlen("Windows ")))) + { + max_length = it->first.size(); + candidate_it = it; + } + } + + if (max_length >= MIN_LENGTH_FOR_STRSTR) + { + required_substring = candidate_it->first; + required_substring_is_prefix = candidate_it->second == 0; + } + } + } + else + { + required_substring = trivial_substrings.front().first; + required_substring_is_prefix = trivial_substrings.front().second == 0; + } + +/* std::cerr + << "regexp: " << regexp + << ", is_trivial: " << is_trivial + << ", required_substring: " << required_substring + << ", required_substring_is_prefix: " << required_substring_is_prefix + << std::endl;*/ +} + + +template +OptimizedRegularExpressionImpl::OptimizedRegularExpressionImpl(const std::string & regexp_, int options) +{ + analyze(regexp_, required_substring, is_trivial, required_substring_is_prefix); + + /// Поддерживаются 3 опции + if (options & (~(RE_CASELESS | RE_NO_CAPTURE | RE_DOT_NL))) + throw Poco::Exception("OptimizedRegularExpression: Unsupported option."); + + is_case_insensitive = options & RE_CASELESS; + bool is_no_capture = options & RE_NO_CAPTURE; + bool is_dot_nl = options & RE_DOT_NL; + + number_of_subpatterns = 0; + if (!is_trivial) + { + /// Скомпилируем регулярное выражение re2. + typename RegexType::Options options; + + if (is_case_insensitive) + options.set_case_sensitive(false); + + if (is_dot_nl) + options.set_dot_nl(true); + + re2.reset(new RegexType(regexp_, options)); + if (!re2->ok()) + throw Poco::Exception("OptimizedRegularExpression: cannot compile re2: " + regexp_ + ", error: " + re2->error()); + + if (!is_no_capture) + { + number_of_subpatterns = re2->NumberOfCapturingGroups(); + if (number_of_subpatterns > MAX_SUBPATTERNS) + throw Poco::Exception("OptimizedRegularExpression: too many subpatterns in regexp: " + regexp_); + } + } +} + + +template +bool OptimizedRegularExpressionImpl::match(const char * subject, size_t subject_size) const +{ + if (is_trivial) + { + if (is_case_insensitive) + return nullptr != strcasestr(subject, required_substring.data()); + else + return nullptr != strstr(subject, required_substring.data()); + } + else + { + if (!required_substring.empty()) + { + const char * pos; + if (is_case_insensitive) + pos = strcasestr(subject, required_substring.data()); + else + pos = strstr(subject, required_substring.data()); + + if (nullptr == pos) + return 0; + } + + return re2->Match(StringPieceType(subject, subject_size), 0, subject_size, RegexType::UNANCHORED, nullptr, 0); + } +} + + +template +bool OptimizedRegularExpressionImpl::match(const char * subject, size_t subject_size, Match & match) const +{ + if (is_trivial) + { + const char * pos; + if (is_case_insensitive) + pos = strcasestr(subject, required_substring.data()); + else + pos = strstr(subject, required_substring.data()); + + if (pos == nullptr) + return 0; + else + { + match.offset = pos - subject; + match.length = required_substring.size(); + return 1; + } + } + else + { + if (!required_substring.empty()) + { + const char * pos; + if (is_case_insensitive) + pos = strcasestr(subject, required_substring.data()); + else + pos = strstr(subject, required_substring.data()); + + if (nullptr == pos) + return 0; + } + + StringPieceType piece; + + if (!RegexType::PartialMatch(StringPieceType(subject, subject_size), *re2, &piece)) + return 0; + else + { + match.offset = piece.data() - subject; + match.length = piece.length(); + return 1; + } + } +} + + +template +unsigned OptimizedRegularExpressionImpl::match(const char * subject, size_t subject_size, MatchVec & matches, unsigned limit) const +{ + matches.clear(); + + if (limit == 0) + return 0; + + if (limit > number_of_subpatterns + 1) + limit = number_of_subpatterns + 1; + + if (is_trivial) + { + const char * pos; + if (is_case_insensitive) + pos = strcasestr(subject, required_substring.data()); + else + pos = strstr(subject, required_substring.data()); + + if (pos == nullptr) + return 0; + else + { + Match match; + match.offset = pos - subject; + match.length = required_substring.size(); + matches.push_back(match); + return 1; + } + } + else + { + if (!required_substring.empty()) + { + const char * pos; + if (is_case_insensitive) + pos = strcasestr(subject, required_substring.data()); + else + pos = strstr(subject, required_substring.data()); + + if (nullptr == pos) + return 0; + } + + StringPieceType pieces[MAX_SUBPATTERNS]; + + if (!re2->Match(StringPieceType(subject, subject_size), 0, subject_size, RegexType::UNANCHORED, pieces, limit)) + return 0; + else + { + matches.resize(limit); + for (size_t i = 0; i < limit; ++i) + { + if (pieces[i] != nullptr) + { + matches[i].offset = pieces[i].data() - subject; + matches[i].length = pieces[i].length(); + } + else + { + matches[i].offset = std::string::npos; + matches[i].length = 0; + } + } + return limit; + } + } +} + +#undef MIN_LENGTH_FOR_STRSTR +#undef MAX_SUBPATTERNS + diff --git a/dbms/include/DB/Common/PODArray.h b/dbms/include/DB/Common/PODArray.h index 063f4025086..1bd81c7d9a1 100644 --- a/dbms/include/DB/Common/PODArray.h +++ b/dbms/include/DB/Common/PODArray.h @@ -12,7 +12,7 @@ #include #include -#include +#include #include diff --git a/dbms/include/DB/Common/PoolBase.h b/dbms/include/DB/Common/PoolBase.h new file mode 100644 index 00000000000..ec277e09c98 --- /dev/null +++ b/dbms/include/DB/Common/PoolBase.h @@ -0,0 +1,146 @@ +#pragma once + +#include +#include +#include +#include +#include + +#include + + +/** Класс, от которого можно унаследоваться и получить пул чего-нибудь. Используется для пулов соединений с БД. + * Наследник должен предоставить метод для создания нового объекта для помещения в пул. + */ + +template +class PoolBase : private boost::noncopyable +{ +public: + typedef TObject Object; + typedef Poco::SharedPtr ObjectPtr; + typedef Poco::SharedPtr > Ptr; + +private: + + /** Объект с флагом, используется ли он сейчас. */ + struct PooledObject + { + PooledObject(Poco::Condition & available_, ObjectPtr object_) + : object(object_), available(available_) + { + } + + ObjectPtr object; + bool in_use = false; + Poco::Condition & available; + }; + + typedef std::vector > Objects; + + /** Помощник, который устанавливает флаг использования объекта, а в деструкторе - снимает, + * а также уведомляет о событии с помощью condvar-а. + */ + struct PoolEntryHelper + { + PoolEntryHelper(PooledObject & data_) : data(data_) { data.in_use = true; } + ~PoolEntryHelper() { data.in_use = false; data.available.signal(); } + + PooledObject & data; + }; + +public: + /** То, что выдаётся пользователю. */ + class Entry + { + public: + friend class PoolBase; + + Entry() {} /// Для отложенной инициализации. + + /** Объект Entry защищает ресурс от использования другим потоком. + * Следующие методы запрещены для rvalue, чтобы нельзя было написать подобное + * + * sqxxl::Query q = pool.Get()->query("SELECT .."); // Упс, после этой строчки Entry уничтожился + * q.execute(); // Кто-то еще может использовать этот Connection + */ + Object * operator->() && = delete; + const Object * operator->() const && = delete; + Object & operator*() && = delete; + const Object & operator*() const && = delete; + + Object * operator->() & { return &*data->data.object; } + const Object * operator->() const & { return &*data->data.object; } + Object & operator*() & { return *data->data.object; } + const Object & operator*() const & { return *data->data.object; } + + bool isNull() const { return data.isNull(); } + + private: + Poco::SharedPtr data; + + Entry(PooledObject & object) : data(new PoolEntryHelper(object)) {} + }; + + virtual ~PoolBase() {} + + /** Выделяет объект для работы. При timeout < 0 таймаут бесконечный. */ + Entry get(Poco::Timespan::TimeDiff timeout) + { + Poco::ScopedLock lock(mutex); + + while (true) + { + for (typename Objects::iterator it = items.begin(); it != items.end(); it++) + if (!(*it)->in_use) + return Entry(**it); + + if (items.size() < max_items) + { + ObjectPtr object = allocObject(); + items.push_back(new PooledObject(available, object)); + return Entry(*items.back()); + } + + LOG_INFO(log, "No free connections in pool. Waiting."); + + if (timeout < 0) + available.wait(mutex); + else + available.wait(mutex, timeout); + } + } + + void reserve(size_t count) + { + Poco::ScopedLock lock(mutex); + + while (items.size() < count) + items.push_back(new PooledObject(available, allocObject())); + } + +private: + /** Максимальный размер пула. */ + unsigned max_items; + + /** Пул. */ + Objects items; + + /** Блокировка для доступа к пулу. */ + Poco::FastMutex mutex; + Poco::Condition available; + +protected: + + Logger * log; + + PoolBase(unsigned max_items_, Logger * log_) + : max_items(max_items_), log(log_) + { + items.reserve(max_items); + } + + /** Создает новый объект для помещения в пул. */ + virtual ObjectPtr allocObject() = 0; +}; + diff --git a/dbms/include/DB/Common/PoolWithFailoverBase.h b/dbms/include/DB/Common/PoolWithFailoverBase.h new file mode 100644 index 00000000000..90064b5b789 --- /dev/null +++ b/dbms/include/DB/Common/PoolWithFailoverBase.h @@ -0,0 +1,311 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + + +namespace +{ + /** Класс, который употребляется для того, чтобы оптимизировать выделение + * нескольких ресурсов в PoolWithFailoverBase. Проверки границ не проводятся, + * потому что мы предполагаем, что PoolWithFailoverBase делает все нужные + * проверки. + */ + class ResourceTracker + { + public: + ResourceTracker(size_t s) + : handles(s), unallocated_size(s) + { + size_t i = 0; + for (auto & index : handles) + { + index = i; + ++i; + } + } + + size_t getHandle(size_t i) const + { + return handles[i]; + } + + size_t getUnallocatedSize() const + { + return unallocated_size; + } + + void markAsAllocated(size_t i) + { + std::swap(handles[i], handles[unallocated_size - 1]); + --unallocated_size; + } + + private: + std::vector handles; + size_t unallocated_size; + }; +} + +/** Класс, от которого можно унаследоваться и получить пул с отказоустойчивостью. Используется для пулов соединений с реплицированной БД. + * Инициализируется несколькими другими PoolBase-ами. + * При получении соединения, пытается создать или выбрать живое соединение из какого-нибудь пула, + * перебирая их в некотором порядке, используя не более указанного количества попыток. + * Пулы перебираются в порядке лексикографического возрастания тройки (приоритет, число ошибок, случайное число). + * + * Замечание: если один из вложенных пулов заблокируется из-за переполнения, то этот пул тоже заблокируется. + * + * Наследник должен предоставить метод, достающий соединение из вложенного пула. + * Еще наследник можнет назначать приоритеты вложенным пулам. + */ + +template +class PoolWithFailoverBase : private boost::noncopyable +{ +public: + typedef TNestedPool NestedPool; + typedef Poco::SharedPtr NestedPoolPtr; + typedef typename NestedPool::Entry Entry; + typedef std::vector NestedPools; + + virtual ~PoolWithFailoverBase() {} + + PoolWithFailoverBase(NestedPools & nested_pools_, + size_t max_tries_, + time_t decrease_error_period_, + Logger * log_) + : nested_pools(nested_pools_.begin(), nested_pools_.end(), decrease_error_period_), max_tries(max_tries_), + log(log_) + { + } + + /** Выделяет соединение для работы. */ + Entry get(const DB::Settings * settings) + { + Entry entry; + std::stringstream fail_messages; + + bool skip_unavailable = settings ? UInt64(settings->skip_unavailable_shards) : false; + + if (getResource(entry, fail_messages, nullptr, settings)) + return entry; + else if (!skip_unavailable) + throw DB::NetException("All connection tries failed. Log: \n\n" + fail_messages.str() + "\n", DB::ErrorCodes::ALL_CONNECTION_TRIES_FAILED); + else + return {}; + } + + /** Выделяет до указанного количества соединений для работы + * Соединения предоставляют доступ к разным репликам одного шарда. + */ + std::vector getMany(const DB::Settings * settings) + { + ResourceTracker resource_tracker{nested_pools.size()}; + + UInt64 max_connections = settings ? UInt64(settings->max_parallel_replicas) : 1; + bool skip_unavailable = settings ? UInt64(settings->skip_unavailable_shards) : false; + + std::vector connections; + connections.reserve(max_connections); + + for (UInt64 i = 0; i < max_connections; ++i) + { + Entry entry; + std::stringstream fail_messages; + + if (getResource(entry, fail_messages, &resource_tracker, settings)) + connections.push_back(entry); + else if (i == 0 && !skip_unavailable) + throw DB::NetException("All connection tries failed. Log: \n\n" + fail_messages.str() + "\n", DB::ErrorCodes::ALL_CONNECTION_TRIES_FAILED); + else + break; + } + + return connections; + } + + +protected: + struct PoolWithErrorCount + { + public: + PoolWithErrorCount(const NestedPoolPtr & pool_) : pool(pool_) + { + struct timespec times; + if (clock_gettime(CLOCK_THREAD_CPUTIME_ID, ×)) + DB::throwFromErrno("Cannot clock_gettime.", DB::ErrorCodes::CANNOT_CLOCK_GETTIME); + + srand48_r(reinterpret_cast(this) ^ times.tv_nsec, &rand_state); + } + + void randomize() + { + long int rand_res; + lrand48_r(&rand_state, &rand_res); + state.random = rand_res; + } + + public: + struct State + { + static bool compare(const State & lhs, const State & rhs) + { + return std::tie(lhs.priority, lhs.error_count, lhs.random) + < std::tie(rhs.priority, rhs.error_count, rhs.random); + } + + Int64 priority = 0; + UInt64 error_count = 0; + UInt32 random = 0; + }; + + public: + NestedPoolPtr pool; + State state; + drand48_data rand_state; + }; + + using States = std::vector; + + class PoolsWithErrorCount : public std::vector + { + public: + PoolsWithErrorCount(typename NestedPools::iterator begin_, typename NestedPools::iterator end_, + time_t decrease_error_period_) + : std::vector(begin_, end_), + decrease_error_period(decrease_error_period_) + { + } + + /// Эта функция возвращает собственную копию состояния каждого пула, чтобы не возникло + /// состояния гонки при выделении соединений. + States update() + { + States states; + states.reserve(this->size()); + + { + Poco::ScopedLock lock(mutex); + + for (auto & pool : *this) + pool.randomize(); + + /// Каждые N секунд уменьшаем количество ошибок в 2 раза + time_t current_time = time(0); + + if (last_decrease_time) + { + time_t delta = current_time - last_decrease_time; + + if (delta >= 0) + { + /// Каждые decrease_error_period секунд, делим количество ошибок на два. + size_t shift_amount = delta / decrease_error_period; + /// обновляем время, не чаще раз в период + /// в противном случае при частых вызовах счетчик никогда не будет уменьшаться + if (shift_amount) + last_decrease_time = current_time; + + if (shift_amount >= sizeof(UInt64)) + { + for (auto & pool : *this) + pool.state.error_count = 0; + } + else if (shift_amount) + { + for (auto & pool : *this) + pool.state.error_count >>= shift_amount; + } + } + } + else + last_decrease_time = current_time; + + for (auto & pool : *this) + states.push_back(pool.state); + } + + return states; + } + + private: + /// Время, когда последний раз уменьшался счётчик ошибок. + time_t last_decrease_time = 0; + time_t decrease_error_period; + Poco::FastMutex mutex; + }; + + PoolsWithErrorCount nested_pools; + size_t max_tries; + Logger * log; + + virtual bool tryGet(NestedPoolPtr pool, const DB::Settings * settings, Entry & out_entry, std::stringstream & fail_message) = 0; + + +private: + /** Выделяет соединение из одной реплики для работы. */ + bool getResource(Entry & entry, std::stringstream & fail_messages, ResourceTracker * resource_tracker, const DB::Settings * settings) + { + /// Обновление случайных чисел, а также счётчиков ошибок. + States states = nested_pools.update(); + + struct IndexedPoolWithErrorCount + { + PoolWithErrorCount * pool; + typename PoolWithErrorCount::State * state; + size_t index; + }; + + using PoolPtrs = std::vector; + + size_t pools_size = resource_tracker ? resource_tracker->getUnallocatedSize() : nested_pools.size(); + PoolPtrs pool_ptrs(pools_size); + + for (size_t i = 0; i < pools_size; ++i) + { + auto & record = pool_ptrs[i]; + size_t pool_index = resource_tracker ? resource_tracker->getHandle(i) : i; + record.pool = &nested_pools[pool_index]; + record.state = &states[pool_index]; + record.index = i; + } + + std::sort(pool_ptrs.begin(), pool_ptrs.end(), + [](const IndexedPoolWithErrorCount & lhs, const IndexedPoolWithErrorCount & rhs) + { + return PoolWithErrorCount::State::compare(*(lhs.state), *(rhs.state)); + }); + + for (size_t try_no = 0; try_no < max_tries; ++try_no) + { + for (size_t i = 0; i < pools_size; ++i) + { + std::stringstream fail_message; + + if (tryGet(pool_ptrs[i].pool->pool, settings, entry, fail_message)) + { + if (resource_tracker) + resource_tracker->markAsAllocated(pool_ptrs[i].index); + return true; + } + + ProfileEvents::increment(ProfileEvents::DistributedConnectionFailTry); + + LOG_WARNING(log, "Connection failed at try №" + << (try_no + 1) << ", reason: " << fail_message.str()); + + fail_messages << fail_message.str() << std::endl; + + __sync_fetch_and_add(&pool_ptrs[i].pool->state.error_count, 1); + } + } + + ProfileEvents::increment(ProfileEvents::DistributedConnectionFailAtAll); + return false; + } +}; diff --git a/dbms/include/DB/Common/SimpleCache.h b/dbms/include/DB/Common/SimpleCache.h index 6a2bd544e1d..aadc6159a1f 100644 --- a/dbms/include/DB/Common/SimpleCache.h +++ b/dbms/include/DB/Common/SimpleCache.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include /** Простейший кэш для свободной функции. diff --git a/dbms/include/DB/Common/StackTrace.h b/dbms/include/DB/Common/StackTrace.h new file mode 100644 index 00000000000..02cfc4fdaf1 --- /dev/null +++ b/dbms/include/DB/Common/StackTrace.h @@ -0,0 +1,23 @@ +#pragma once + +#include +#include + +#define STACK_TRACE_MAX_DEPTH 32 + + +/// Позволяет получить стек-трейс +class StackTrace +{ +public: + /// Стектрейс снимается в момент создания объекта + StackTrace(); + + /// Вывести в строку + std::string toString() const; + +private: + typedef void* Frame; + Frame frames[STACK_TRACE_MAX_DEPTH]; + size_t frames_size; +}; diff --git a/dbms/include/DB/Common/Stopwatch.h b/dbms/include/DB/Common/Stopwatch.h new file mode 100644 index 00000000000..000cbd145e9 --- /dev/null +++ b/dbms/include/DB/Common/Stopwatch.h @@ -0,0 +1,75 @@ +#pragma once + +#include +#include +#include +#include + + +/** Отличается от Poco::Stopwatch только тем, что использует clock_gettime вместо gettimeofday, + * возвращает наносекунды вместо микросекунд, а также другими незначительными отличиями. + */ +class Stopwatch +{ +public: + /** CLOCK_MONOTONIC работает сравнительно эффективно (~15 млн. вызовов в сек.) и не приводит к системному вызову. + * Поставьте CLOCK_MONOTONIC_COARSE, если нужна больше производительность, но достаточно погрешности в несколько мс. + */ + Stopwatch(clockid_t clock_type_ = CLOCK_MONOTONIC) : clock_type(clock_type_) { restart(); } + + void start() { setStart(); is_running = true; } + void stop() { updateElapsed(); is_running = false; } + void restart() { elapsed_ns = 0; start(); } + UInt64 elapsed() const { updateElapsed(); return elapsed_ns; } + double elapsedSeconds() const { updateElapsed(); return static_cast(elapsed_ns) / 1000000000ULL; } + +private: + mutable UInt64 start_ns; + mutable UInt64 elapsed_ns; + clockid_t clock_type; + bool is_running; + + void setStart() + { + struct timespec ts; + clock_gettime(clock_type, &ts); + start_ns = ts.tv_sec * 1000000000ULL + ts.tv_nsec; + } + + void updateElapsed() const + { + if (is_running) + { + struct timespec ts; + clock_gettime(clock_type, &ts); + UInt64 current_ns = ts.tv_sec * 1000000000ULL + ts.tv_nsec; + elapsed_ns += current_ns - start_ns; + start_ns = current_ns; + } + } +}; + + +class StopwatchWithLock : public Stopwatch +{ +public: + /** Если прошло указанное количество секунд, то перезапускает таймер и возвращает true. + * Иначе возвращает false. + * thread-safe. + */ + bool lockTestAndRestart(double seconds) + { + Poco::ScopedLock lock(mutex); + + if (elapsedSeconds() >= seconds) + { + restart(); + return true; + } + else + return false; + } + +private: + Poco::FastMutex mutex; +}; diff --git a/dbms/include/DB/Common/Throttler.h b/dbms/include/DB/Common/Throttler.h index 8998c24f84d..63bb51280a3 100644 --- a/dbms/include/DB/Common/Throttler.h +++ b/dbms/include/DB/Common/Throttler.h @@ -2,8 +2,8 @@ #include #include -#include -#include +#include +#include #include diff --git a/dbms/include/DB/Core/Block.h b/dbms/include/DB/Core/Block.h index e72f903a9d3..d622c0efc64 100644 --- a/dbms/include/DB/Core/Block.h +++ b/dbms/include/DB/Core/Block.h @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include "ColumnsWithTypeAndName.h" diff --git a/dbms/include/DB/Core/BlockInfo.h b/dbms/include/DB/Core/BlockInfo.h index eaa4cabae9f..49d7be5c579 100644 --- a/dbms/include/DB/Core/BlockInfo.h +++ b/dbms/include/DB/Core/BlockInfo.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include #include diff --git a/dbms/include/DB/Core/ErrorCodes.h b/dbms/include/DB/Core/ErrorCodes.h index 037c2d6fa1e..8b5fb98e94f 100644 --- a/dbms/include/DB/Core/ErrorCodes.h +++ b/dbms/include/DB/Core/ErrorCodes.h @@ -252,7 +252,6 @@ namespace ErrorCodes CORRUPTED_DATA = 246, INCORRECT_MARK = 247, INVALID_PARTITION_NAME = 248, - NOT_LEADER = 249, NOT_ENOUGH_BLOCK_NUMBERS = 250, NO_SUCH_REPLICA = 251, TOO_MUCH_PARTS = 252, @@ -293,6 +292,8 @@ namespace ErrorCodes DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED = 288, REPLICA_IS_NOT_IN_QUORUM = 289, LIMIT_EXCEEDED = 290, + DATABASE_ACCESS_DENIED = 291, + LEADERSHIP_CHANGED = 292, KEEPER_EXCEPTION = 999, POCO_EXCEPTION = 1000, diff --git a/dbms/include/DB/Core/Exception.h b/dbms/include/DB/Core/Exception.h deleted file mode 100644 index e9d8ec3c737..00000000000 --- a/dbms/include/DB/Core/Exception.h +++ /dev/null @@ -1,40 +0,0 @@ -#pragma once - -#include -#include - -#include -#include - -namespace Poco { class Logger; } - -namespace DB -{ - -using Poco::SharedPtr; - -typedef SharedPtr ExceptionPtr; -typedef std::vector Exceptions; - - -void throwFromErrno(const std::string & s, int code = 0, int the_errno = errno); - - -/** Для использования в блоке catch (...). - * Преобразует Exception, Poco::Exception, std::exception или неизвестный exception в ExceptionPtr. - */ -ExceptionPtr cloneCurrentException(); - -/** Попробовать записать исключение в лог (и забыть про него). - * Можно использовать в деструкторах в блоке catch (...). - */ -void tryLogCurrentException(const char * log_name, const std::string & start_of_message = ""); -void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message = ""); - -std::string getCurrentExceptionMessage(bool with_stacktrace); - - -void rethrowFirstException(Exceptions & exceptions); - - -} diff --git a/dbms/include/DB/Core/Field.h b/dbms/include/DB/Core/Field.h index 33c0835f7fa..a5cab3221fe 100644 --- a/dbms/include/DB/Core/Field.h +++ b/dbms/include/DB/Core/Field.h @@ -11,7 +11,7 @@ #include #include -#include +#include #include #include #include diff --git a/dbms/include/DB/DataStreams/AsynchronousBlockInputStream.h b/dbms/include/DB/DataStreams/AsynchronousBlockInputStream.h index cd97d3aaa02..ac7951a607b 100644 --- a/dbms/include/DB/DataStreams/AsynchronousBlockInputStream.h +++ b/dbms/include/DB/DataStreams/AsynchronousBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -48,7 +48,7 @@ public: { pool.wait(); if (exception) - exception->rethrow(); + std::rethrow_exception(exception); children.back()->readSuffix(); started = false; } @@ -82,7 +82,7 @@ protected: bool started = false; Block block; - ExceptionPtr exception; + std::exception_ptr exception; Block readImpl() override @@ -97,7 +97,7 @@ protected: pool.wait(); if (exception) - exception->rethrow(); + std::rethrow_exception(exception); Block res = block; if (!res) @@ -130,7 +130,7 @@ protected: } catch (...) { - exception = cloneCurrentException(); + exception = std::current_exception(); } ready.set(); diff --git a/dbms/include/DB/DataStreams/BlockStreamProfileInfo.h b/dbms/include/DB/DataStreams/BlockStreamProfileInfo.h index 036aedc3a98..bfd29263918 100644 --- a/dbms/include/DB/DataStreams/BlockStreamProfileInfo.h +++ b/dbms/include/DB/DataStreams/BlockStreamProfileInfo.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include diff --git a/dbms/include/DB/DataStreams/MaterializingBlockOutputStream.h b/dbms/include/DB/DataStreams/MaterializingBlockOutputStream.h index b3f4a78ad30..02b9c7d8bab 100644 --- a/dbms/include/DB/DataStreams/MaterializingBlockOutputStream.h +++ b/dbms/include/DB/DataStreams/MaterializingBlockOutputStream.h @@ -2,7 +2,7 @@ #include #include -#include +#include namespace DB diff --git a/dbms/include/DB/DataStreams/ParallelAggregatingBlockInputStream.h b/dbms/include/DB/DataStreams/ParallelAggregatingBlockInputStream.h index 902e026c153..39cf19b7f33 100644 --- a/dbms/include/DB/DataStreams/ParallelAggregatingBlockInputStream.h +++ b/dbms/include/DB/DataStreams/ParallelAggregatingBlockInputStream.h @@ -163,7 +163,7 @@ private: { } - void onException(ExceptionPtr & exception, size_t thread_num) + void onException(std::exception_ptr & exception, size_t thread_num) { parent.exceptions[thread_num] = exception; parent.cancel(); diff --git a/dbms/include/DB/DataStreams/ParallelInputsProcessor.h b/dbms/include/DB/DataStreams/ParallelInputsProcessor.h index 8ce7bfe8f3d..06e6abd1e92 100644 --- a/dbms/include/DB/DataStreams/ParallelInputsProcessor.h +++ b/dbms/include/DB/DataStreams/ParallelInputsProcessor.h @@ -38,7 +38,7 @@ struct ParallelInputsHandler void onFinish() {} /// Обработка исключения. Разумно вызывать в этом методе метод ParallelInputsProcessor::cancel, а также передавать эксепшен в основной поток. - void onException(ExceptionPtr & exception, size_t thread_num) {} + void onException(std::exception_ptr & exception, size_t thread_num) {} }; @@ -140,7 +140,7 @@ private: void thread(MemoryTracker * memory_tracker, size_t thread_num) { current_memory_tracker = memory_tracker; - ExceptionPtr exception; + std::exception_ptr exception; setThreadName("ParalInputsProc"); @@ -150,7 +150,7 @@ private: } catch (...) { - exception = cloneCurrentException(); + exception = std::current_exception(); } if (exception) @@ -171,7 +171,7 @@ private: } catch (...) { - exception = cloneCurrentException(); + exception = std::current_exception(); } if (exception) diff --git a/dbms/include/DB/DataStreams/RemoteBlockOutputStream.h b/dbms/include/DB/DataStreams/RemoteBlockOutputStream.h index c438ba4da5e..0dd5fbd3777 100644 --- a/dbms/include/DB/DataStreams/RemoteBlockOutputStream.h +++ b/dbms/include/DB/DataStreams/RemoteBlockOutputStream.h @@ -5,7 +5,7 @@ #include #include -#include +#include namespace DB { diff --git a/dbms/include/DB/DataStreams/SummingSortedBlockInputStream.h b/dbms/include/DB/DataStreams/SummingSortedBlockInputStream.h index 468555b5aed..25377433496 100644 --- a/dbms/include/DB/DataStreams/SummingSortedBlockInputStream.h +++ b/dbms/include/DB/DataStreams/SummingSortedBlockInputStream.h @@ -6,8 +6,8 @@ #include #include #include -#include -#include +#include +#include namespace DB diff --git a/dbms/include/DB/DataStreams/UnionBlockInputStream.h b/dbms/include/DB/DataStreams/UnionBlockInputStream.h index 31d7e274bf6..8cebb08753a 100644 --- a/dbms/include/DB/DataStreams/UnionBlockInputStream.h +++ b/dbms/include/DB/DataStreams/UnionBlockInputStream.h @@ -97,7 +97,7 @@ protected: LOG_TRACE(log, "Waiting for threads to finish"); - ExceptionPtr exception; + std::exception_ptr exception; if (!all_read) { /** Прочитаем всё до конца, чтобы ParallelInputsProcessor не заблокировался при попытке вставить в очередь. @@ -113,8 +113,8 @@ protected: { if (!exception) exception = res.exception; - else if (DB::Exception * e = dynamic_cast(&*exception)) - e->addMessage("\n" + res.exception->displayText()); + else if (Exception * e = exception_cast(exception)) + e->addMessage("\n" + getExceptionMessage(res.exception, false)); } else if (!res.block) break; @@ -128,7 +128,7 @@ protected: LOG_TRACE(log, "Waited for threads to finish"); if (exception) - exception->rethrow(); + std::rethrow_exception(exception); } /** Возможны следующие варианты: @@ -157,7 +157,7 @@ protected: output_queue.pop(res); if (res.exception) - res.exception->rethrow(); + std::rethrow_exception(res.exception); if (!res.block) all_read = true; @@ -183,11 +183,11 @@ private: struct OutputData { Block block; - ExceptionPtr exception; + std::exception_ptr exception; OutputData() {} OutputData(Block & block_) : block(block_) {} - OutputData(ExceptionPtr & exception_) : exception(exception_) {} + OutputData(std::exception_ptr & exception_) : exception(exception_) {} }; /** Очередь готовых блоков. Также туда можно положить эксепшен вместо блока. @@ -216,7 +216,7 @@ private: parent.output_queue.push(OutputData()); } - void onException(ExceptionPtr & exception, size_t thread_num) + void onException(std::exception_ptr & exception, size_t thread_num) { //std::cerr << "pushing exception\n"; diff --git a/dbms/include/DB/DataTypes/DataTypeNested.h b/dbms/include/DB/DataTypes/DataTypeNested.h index eea1bb67137..44120433089 100644 --- a/dbms/include/DB/DataTypes/DataTypeNested.h +++ b/dbms/include/DB/DataTypes/DataTypeNested.h @@ -1,84 +1,39 @@ #pragma once -#include +#include #include + namespace DB { -using Poco::SharedPtr; - - -class DataTypeNested final : public IDataType +/** Хранит набор пар (имя, тип) для вложенной структуры данных. + * Используется только при создании таблицы. Во всех остальных случаях не используется, так как раскрывается в набор отдельных столбцов с типами. + */ +class DataTypeNested final : public IDataTypeDummy { private: /// Имена и типы вложенных массивов. NamesAndTypesListPtr nested; - /// Тип смещений. - DataTypePtr offsets; public: DataTypeNested(NamesAndTypesListPtr nested_); std::string getName() const override; + DataTypePtr clone() const override + { + return new DataTypeNested(nested); + } + + const NamesAndTypesListPtr & getNestedTypesList() const { return nested; } + static std::string concatenateNestedName(const std::string & nested_table_name, const std::string & nested_field_name); /// Возвращает префикс имени до первой точки '.'. Или имя без изменений, если точки нет. static std::string extractNestedTableName(const std::string & nested_name); /// Возвращает суффикс имени после первой точки справа '.'. Или имя без изменений, если точки нет. static std::string extractNestedColumnName(const std::string & nested_name); - DataTypePtr clone() const override - { - return new DataTypeNested(nested); - } - - void serializeBinary(const Field & field, WriteBuffer & ostr) const override; - void deserializeBinary(Field & field, ReadBuffer & istr) const override; - - void serializeText(const Field & field, WriteBuffer & ostr) const override; - void deserializeText(Field & field, ReadBuffer & istr) const override; - - void serializeTextEscaped(const Field & field, WriteBuffer & ostr) const override; - void deserializeTextEscaped(Field & field, ReadBuffer & istr) const override; - - void serializeTextQuoted(const Field & field, WriteBuffer & ostr) const override; - void deserializeTextQuoted(Field & field, ReadBuffer & istr) const override; - - void serializeTextJSON(const Field & field, WriteBuffer & ostr) const override; - - /** Потоковая сериализация массивов устроена по-особенному: - * - записываются/читаются элементы, уложенные подряд, без размеров массивов; - * - размеры записываются/читаются в отдельный столбец, - * и о записи/чтении размеров должна позаботиться вызывающая сторона. - * Это нужно, так как несколько массивов имеют общие размеры. - */ - - /** Записать только значения, без размеров. Вызывающая сторона также должна куда-нибудь записать смещения. */ - void serializeBinary(const IColumn & column, WriteBuffer & ostr, size_t offset = 0, size_t limit = 0) const override; - - /** Прочитать только значения, без размеров. - * При этом, в column уже заранее должны быть считаны все размеры. - */ - void deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; - - /** Записать размеры. */ - void serializeOffsets(const IColumn & column, WriteBuffer & ostr, size_t offset = 0, size_t limit = 0) const; - - /** Прочитать размеры. Вызывайте этот метод перед чтением значений. */ - void deserializeOffsets(IColumn & column, ReadBuffer & istr, size_t limit) const; - - ColumnPtr createColumn() const override; - ColumnPtr createConstColumn(size_t size, const Field & field) const override; - - Field getDefault() const override - { - throw Exception("Method getDefault is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); - } - - const NamesAndTypesListPtr & getNestedTypesList() const { return nested; } - const DataTypePtr & getOffsetsType() const { return offsets; } - /// Создает новый список в котором колонки типа Nested заменены на несколько вида имя_колонки.имя_вложенной_ячейки static NamesAndTypesListPtr expandNestedColumns(const NamesAndTypesList & names_and_types); }; diff --git a/dbms/include/DB/DataTypes/FieldToDataType.h b/dbms/include/DB/DataTypes/FieldToDataType.h index 846fda28ee2..d0ec05b7891 100644 --- a/dbms/include/DB/DataTypes/FieldToDataType.h +++ b/dbms/include/DB/DataTypes/FieldToDataType.h @@ -4,7 +4,7 @@ #include #include -#include +#include #include diff --git a/dbms/include/DB/Dictionaries/CacheDictionary.h b/dbms/include/DB/Dictionaries/CacheDictionary.h index 995f4945865..c6044ca6256 100644 --- a/dbms/include/DB/Dictionaries/CacheDictionary.h +++ b/dbms/include/DB/Dictionaries/CacheDictionary.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/include/DB/Dictionaries/ClickHouseDictionarySource.h b/dbms/include/DB/Dictionaries/ClickHouseDictionarySource.h index 5cf7394c9e3..d7ecab46e1d 100644 --- a/dbms/include/DB/Dictionaries/ClickHouseDictionarySource.h +++ b/dbms/include/DB/Dictionaries/ClickHouseDictionarySource.h @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include "writeParenthesisedString.h" diff --git a/dbms/include/DB/Dictionaries/FlatDictionary.h b/dbms/include/DB/Dictionaries/FlatDictionary.h index 19aced48322..0781d3ae406 100644 --- a/dbms/include/DB/Dictionaries/FlatDictionary.h +++ b/dbms/include/DB/Dictionaries/FlatDictionary.h @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/include/DB/Dictionaries/HashedDictionary.h b/dbms/include/DB/Dictionaries/HashedDictionary.h index 6a9fad13c14..7c01eb020b7 100644 --- a/dbms/include/DB/Dictionaries/HashedDictionary.h +++ b/dbms/include/DB/Dictionaries/HashedDictionary.h @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/include/DB/Dictionaries/MySQLBlockInputStream.h b/dbms/include/DB/Dictionaries/MySQLBlockInputStream.h index b8f4c4bb244..2cababde17b 100644 --- a/dbms/include/DB/Dictionaries/MySQLBlockInputStream.h +++ b/dbms/include/DB/Dictionaries/MySQLBlockInputStream.h @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/include/DB/Dictionaries/MySQLDictionarySource.h b/dbms/include/DB/Dictionaries/MySQLDictionarySource.h index 249c8488c98..b8f2e5e8d25 100644 --- a/dbms/include/DB/Dictionaries/MySQLDictionarySource.h +++ b/dbms/include/DB/Dictionaries/MySQLDictionarySource.h @@ -2,10 +2,9 @@ #include #include -#include +#include #include #include -#include #include "writeParenthesisedString.h" @@ -79,6 +78,28 @@ public: private: Logger * log = &Logger::get("MySQLDictionarySource"); + + static std::string quoteForLike(const std::string s) + { + std::string tmp; + tmp.reserve(s.size()); + + for (auto c : s) + { + if (c == '%' || c == '_' || c == '\\') + tmp.push_back('\\'); + tmp.push_back(c); + } + + std::string res; + { + WriteBufferFromString out(res); + writeQuoted(tmp, out); + } + return res; + } + + mysqlxx::DateTime getLastModification() const { mysqlxx::DateTime update_time{std::time(nullptr)}; @@ -89,7 +110,7 @@ private: try { auto connection = pool.Get(); - auto query = connection->query("SHOW TABLE STATUS LIKE '" + strconvert::escaped_for_like(table) + "'"); + auto query = connection->query("SHOW TABLE STATUS LIKE " + quoteForLike(table)); LOG_TRACE(log, query.str()); diff --git a/dbms/include/DB/Dictionaries/RangeHashedDictionary.h b/dbms/include/DB/Dictionaries/RangeHashedDictionary.h index d7c78270172..061b2f75773 100644 --- a/dbms/include/DB/Dictionaries/RangeHashedDictionary.h +++ b/dbms/include/DB/Dictionaries/RangeHashedDictionary.h @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/include/DB/Functions/FunctionsArray.h b/dbms/include/DB/Functions/FunctionsArray.h index 5b187e5259d..f3bc6268477 100644 --- a/dbms/include/DB/Functions/FunctionsArray.h +++ b/dbms/include/DB/Functions/FunctionsArray.h @@ -17,7 +17,7 @@ #include #include -#include +#include #include diff --git a/dbms/include/DB/Functions/FunctionsCoding.h b/dbms/include/DB/Functions/FunctionsCoding.h index 45f258dde62..4a45cc6444e 100644 --- a/dbms/include/DB/Functions/FunctionsCoding.h +++ b/dbms/include/DB/Functions/FunctionsCoding.h @@ -14,7 +14,7 @@ #include #include -#include +#include #include diff --git a/dbms/include/DB/Functions/FunctionsConversion.h b/dbms/include/DB/Functions/FunctionsConversion.h index 698e8ac24e2..6a6b3551686 100644 --- a/dbms/include/DB/Functions/FunctionsConversion.h +++ b/dbms/include/DB/Functions/FunctionsConversion.h @@ -11,7 +11,7 @@ #include #include #include -#include +#include namespace DB diff --git a/dbms/include/DB/Functions/FunctionsDictionaries.h b/dbms/include/DB/Functions/FunctionsDictionaries.h index 4cb6b449c34..1c60af47803 100644 --- a/dbms/include/DB/Functions/FunctionsDictionaries.h +++ b/dbms/include/DB/Functions/FunctionsDictionaries.h @@ -20,7 +20,7 @@ #include #include -#include +#include namespace DB diff --git a/dbms/include/DB/Functions/FunctionsHashing.h b/dbms/include/DB/Functions/FunctionsHashing.h index b356df55eb9..2d9e077fc0c 100644 --- a/dbms/include/DB/Functions/FunctionsHashing.h +++ b/dbms/include/DB/Functions/FunctionsHashing.h @@ -22,7 +22,7 @@ #include #include -#include +#include #include diff --git a/dbms/include/DB/Functions/FunctionsMiscellaneous.h b/dbms/include/DB/Functions/FunctionsMiscellaneous.h index 10c0cc262f5..db841629d5e 100644 --- a/dbms/include/DB/Functions/FunctionsMiscellaneous.h +++ b/dbms/include/DB/Functions/FunctionsMiscellaneous.h @@ -24,7 +24,7 @@ #include #include #include -#include +#include #include @@ -386,23 +386,16 @@ public: throw Exception("Second argument for function '" + getName() + "' must be Set; found " + column_set_ptr->getName(), ErrorCodes::ILLEGAL_COLUMN); - /// Столбцы, которые проверяются на принадлежность множеству. - ColumnNumbers left_arguments; + Block block_of_key_columns; /// Первый аргумент может быть tuple или одиночным столбцом. const ColumnTuple * tuple = typeid_cast(&*block.getByPosition(arguments[0]).column); if (tuple) - { - /// Находим в блоке столбцы из tuple. - const Block & tuple_elems = tuple->getData(); - size_t tuple_size = tuple_elems.columns(); - for (size_t i = 0; i < tuple_size; ++i) - left_arguments.push_back(block.getPositionByName(tuple_elems.getByPosition(i).name)); - } + block_of_key_columns = tuple->getData(); else - left_arguments.push_back(arguments[0]); + block_of_key_columns.insert(block.getByPosition(arguments[0])); - column_set->getData()->execute(block, left_arguments, result, negative); + block.getByPosition(result).column = column_set->getData()->execute(block_of_key_columns, negative); } }; diff --git a/dbms/include/DB/Functions/FunctionsString.h b/dbms/include/DB/Functions/FunctionsString.h index bc95afd8089..498221cfb8c 100644 --- a/dbms/include/DB/Functions/FunctionsString.h +++ b/dbms/include/DB/Functions/FunctionsString.h @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include #include diff --git a/dbms/include/DB/Functions/FunctionsStringSearch.h b/dbms/include/DB/Functions/FunctionsStringSearch.h index 603dae6b4b4..3f3cbe17d8d 100644 --- a/dbms/include/DB/Functions/FunctionsStringSearch.h +++ b/dbms/include/DB/Functions/FunctionsStringSearch.h @@ -2,7 +2,7 @@ #include -#include +#include #include #include @@ -19,7 +19,7 @@ #include #include -#include +#include #include diff --git a/dbms/include/DB/IO/AsynchronousWriteBuffer.h b/dbms/include/DB/IO/AsynchronousWriteBuffer.h index 7847fe9395e..764bcb56c43 100644 --- a/dbms/include/DB/IO/AsynchronousWriteBuffer.h +++ b/dbms/include/DB/IO/AsynchronousWriteBuffer.h @@ -7,7 +7,7 @@ #include #include -#include +#include #include @@ -46,7 +46,7 @@ private: started = true; if (exception) - exception->rethrow(); + std::rethrow_exception(exception); swapBuffers(); @@ -77,7 +77,7 @@ public: } } - ExceptionPtr exception; + std::exception_ptr exception; /// То, что выполняется в отдельном потоке void thread() @@ -88,7 +88,7 @@ public: } catch (...) { - exception = cloneCurrentException(); + exception = std::current_exception(); } } }; diff --git a/dbms/include/DB/IO/BufferWithOwnMemory.h b/dbms/include/DB/IO/BufferWithOwnMemory.h index 93f14173d30..a86650d1bfc 100644 --- a/dbms/include/DB/IO/BufferWithOwnMemory.h +++ b/dbms/include/DB/IO/BufferWithOwnMemory.h @@ -5,7 +5,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/include/DB/IO/CompressedReadBufferBase.h b/dbms/include/DB/IO/CompressedReadBufferBase.h index 3f38879f879..41b2315a681 100644 --- a/dbms/include/DB/IO/CompressedReadBufferBase.h +++ b/dbms/include/DB/IO/CompressedReadBufferBase.h @@ -9,7 +9,7 @@ #include #include -#include +#include #include #include #include diff --git a/dbms/include/DB/IO/ReadBuffer.h b/dbms/include/DB/IO/ReadBuffer.h index 97442d69717..c753d1c1b44 100644 --- a/dbms/include/DB/IO/ReadBuffer.h +++ b/dbms/include/DB/IO/ReadBuffer.h @@ -3,7 +3,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/include/DB/IO/ReadBufferAIO.h b/dbms/include/DB/IO/ReadBufferAIO.h index c7afdcf329a..a22ddff8bfc 100644 --- a/dbms/include/DB/IO/ReadBufferAIO.h +++ b/dbms/include/DB/IO/ReadBufferAIO.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include diff --git a/dbms/include/DB/IO/ReadBufferFromFileDescriptor.h b/dbms/include/DB/IO/ReadBufferFromFileDescriptor.h index 8f2e389cdd7..5aeaabf6774 100644 --- a/dbms/include/DB/IO/ReadBufferFromFileDescriptor.h +++ b/dbms/include/DB/IO/ReadBufferFromFileDescriptor.h @@ -5,7 +5,7 @@ #include -#include +#include #include #include diff --git a/dbms/include/DB/IO/ReadBufferFromIStream.h b/dbms/include/DB/IO/ReadBufferFromIStream.h index 92fce68749f..310c1137abe 100644 --- a/dbms/include/DB/IO/ReadBufferFromIStream.h +++ b/dbms/include/DB/IO/ReadBufferFromIStream.h @@ -2,7 +2,7 @@ #include -#include +#include #include #include diff --git a/dbms/include/DB/IO/ReadHelpers.h b/dbms/include/DB/IO/ReadHelpers.h index 13052d411d2..c774583b2ed 100644 --- a/dbms/include/DB/IO/ReadHelpers.h +++ b/dbms/include/DB/IO/ReadHelpers.h @@ -13,7 +13,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/include/DB/IO/WriteBuffer.h b/dbms/include/DB/IO/WriteBuffer.h index 54f38242efc..706874b4720 100644 --- a/dbms/include/DB/IO/WriteBuffer.h +++ b/dbms/include/DB/IO/WriteBuffer.h @@ -3,7 +3,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/include/DB/IO/WriteBufferAIO.h b/dbms/include/DB/IO/WriteBufferAIO.h index 26ce2178cde..7cbf3e4ac30 100644 --- a/dbms/include/DB/IO/WriteBufferAIO.h +++ b/dbms/include/DB/IO/WriteBufferAIO.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include diff --git a/dbms/include/DB/IO/WriteBufferFromFileDescriptor.h b/dbms/include/DB/IO/WriteBufferFromFileDescriptor.h index 9b3e5abc3b0..17fbb70acbc 100644 --- a/dbms/include/DB/IO/WriteBufferFromFileDescriptor.h +++ b/dbms/include/DB/IO/WriteBufferFromFileDescriptor.h @@ -3,7 +3,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/include/DB/IO/WriteBufferFromHTTPServerResponse.h b/dbms/include/DB/IO/WriteBufferFromHTTPServerResponse.h index 6996642d923..419a20ae100 100644 --- a/dbms/include/DB/IO/WriteBufferFromHTTPServerResponse.h +++ b/dbms/include/DB/IO/WriteBufferFromHTTPServerResponse.h @@ -2,12 +2,12 @@ #include -#include +#include #include #include #include -#include +#include namespace DB diff --git a/dbms/include/DB/IO/WriteBufferFromOStream.h b/dbms/include/DB/IO/WriteBufferFromOStream.h index b41a34c6ff5..4a87aa0a908 100644 --- a/dbms/include/DB/IO/WriteBufferFromOStream.h +++ b/dbms/include/DB/IO/WriteBufferFromOStream.h @@ -2,7 +2,7 @@ #include -#include +#include #include #include diff --git a/dbms/include/DB/IO/WriteHelpers.h b/dbms/include/DB/IO/WriteHelpers.h index 28531a0c723..bda31fc7e8b 100644 --- a/dbms/include/DB/IO/WriteHelpers.h +++ b/dbms/include/DB/IO/WriteHelpers.h @@ -12,7 +12,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/include/DB/Interpreters/AggregationCommon.h b/dbms/include/DB/Interpreters/AggregationCommon.h index c88a4897391..6a299e236a1 100644 --- a/dbms/include/DB/Interpreters/AggregationCommon.h +++ b/dbms/include/DB/Interpreters/AggregationCommon.h @@ -182,4 +182,19 @@ static inline StringRef ALWAYS_INLINE extractKeysAndPlaceInPoolContiguous( } +/** Сериализовать ключи в непрерывный кусок памяти. + */ +static inline StringRef ALWAYS_INLINE serializeKeysToPoolContiguous( + size_t i, size_t keys_size, const ConstColumnPlainPtrs & key_columns, StringRefs & keys, Arena & pool) +{ + const char * begin = nullptr; + + size_t sum_size = 0; + for (size_t j = 0; j < keys_size; ++j) + sum_size += key_columns[j]->serializeValueIntoArena(i, pool, begin).size; + + return {begin, sum_size}; +} + + } diff --git a/dbms/include/DB/Interpreters/Aggregator.h b/dbms/include/DB/Interpreters/Aggregator.h index 4c6bbd53c20..c3e6a2f05f7 100644 --- a/dbms/include/DB/Interpreters/Aggregator.h +++ b/dbms/include/DB/Interpreters/Aggregator.h @@ -5,7 +5,7 @@ #include #include -#include +#include #include #include @@ -319,7 +319,7 @@ struct AggregationMethodKeysFixed }; -/// Для остальных случаев. Агрегирует по конкатенации ключей. (При этом, строки, содержащие нули посередине, могут склеиться.) +/// Агрегирует по конкатенации ключей. (При этом, строки, содержащие нули посередине, могут склеиться.) template struct AggregationMethodConcat { @@ -392,6 +392,70 @@ struct AggregationMethodConcat }; +/** Агрегирует по конкатенации сериализованных значений ключей. + * Похож на AggregationMethodConcat, но подходит, например, для массивов строк или нескольких массивов. + * Сериализованное значение отличается тем, что позволяет однозначно его десериализовать, имея только позицию, с которой оно начинается. + * То есть, например, для строк, оно содержит сначала сериализованную длину строки, а потом байты. + * Поэтому, при агрегации по нескольким строкам, неоднозначностей не возникает. + */ +template +struct AggregationMethodSerialized +{ + typedef TData Data; + typedef typename Data::key_type Key; + typedef typename Data::mapped_type Mapped; + typedef typename Data::iterator iterator; + typedef typename Data::const_iterator const_iterator; + + Data data; + + AggregationMethodSerialized() {} + + template + AggregationMethodSerialized(const Other & other) : data(other.data) {} + + struct State + { + void init(ConstColumnPlainPtrs & key_columns) + { + } + + Key getKey( + const ConstColumnPlainPtrs & key_columns, + size_t keys_size, + size_t i, + const Sizes & key_sizes, + StringRefs & keys, + Arena & pool) const + { + return serializeKeysToPoolContiguous(i, keys_size, key_columns, keys, pool); + } + }; + + static AggregateDataPtr & getAggregateData(Mapped & value) { return value; } + static const AggregateDataPtr & getAggregateData(const Mapped & value) { return value; } + + static void onNewKey(typename Data::value_type & value, size_t keys_size, size_t i, StringRefs & keys, Arena & pool) + { + } + + static void onExistingKey(const Key & key, StringRefs & keys, Arena & pool) + { + pool.rollback(key.size); + } + + /// Если ключ уже был, то он удаляется из пула (затирается), и сравнить с ним следующий ключ уже нельзя. + static const bool no_consecutive_keys_optimization = true; + + static void insertKeyIntoColumns(const typename Data::value_type & value, ColumnPlainPtrs & key_columns, size_t keys_size, const Sizes & key_sizes) + { + auto pos = value.first.data; + for (size_t i = 0; i < keys_size; ++i) + pos = key_columns[i]->deserializeAndInsertFromArena(pos); + } +}; + + /// Для остальных случаев. Агрегирует по 128 битному хэшу от ключа. (При этом, строки, содержащие нули посередине, могут склеиться.) template struct AggregationMethodHashed @@ -492,6 +556,7 @@ struct AggregatedDataVariants : private boost::noncopyable std::unique_ptr> keys256; std::unique_ptr> hashed; std::unique_ptr> concat; + std::unique_ptr> serialized; std::unique_ptr> key32_two_level; std::unique_ptr> key64_two_level; @@ -501,6 +566,7 @@ struct AggregatedDataVariants : private boost::noncopyable std::unique_ptr> keys256_two_level; std::unique_ptr> hashed_two_level; std::unique_ptr> concat_two_level; + std::unique_ptr> serialized_two_level; /// В этом и подобных макросах, вариант without_key не учитывается. #define APPLY_FOR_AGGREGATED_VARIANTS(M) \ @@ -514,6 +580,7 @@ struct AggregatedDataVariants : private boost::noncopyable M(keys256, false) \ M(hashed, false) \ M(concat, false) \ + M(serialized, false) \ M(key32_two_level, true) \ M(key64_two_level, true) \ M(key_string_two_level, true) \ @@ -521,7 +588,8 @@ struct AggregatedDataVariants : private boost::noncopyable M(keys128_two_level, true) \ M(keys256_two_level, true) \ M(hashed_two_level, true) \ - M(concat_two_level, true) + M(concat_two_level, true) \ + M(serialized_two_level, true) \ enum class Type { @@ -636,7 +704,8 @@ struct AggregatedDataVariants : private boost::noncopyable M(keys128) \ M(keys256) \ M(hashed) \ - M(concat) + M(concat) \ + M(serialized) \ #define APPLY_FOR_VARIANTS_NOT_CONVERTIBLE_TO_TWO_LEVEL(M) \ M(key8) \ @@ -667,7 +736,8 @@ struct AggregatedDataVariants : private boost::noncopyable M(keys128_two_level) \ M(keys256_two_level) \ M(hashed_two_level) \ - M(concat_two_level) + M(concat_two_level) \ + M(serialized_two_level) }; typedef SharedPtr AggregatedDataVariantsPtr; diff --git a/dbms/include/DB/Interpreters/Compiler.h b/dbms/include/DB/Interpreters/Compiler.h index 13b74afa418..c17ea20efcc 100644 --- a/dbms/include/DB/Interpreters/Compiler.h +++ b/dbms/include/DB/Interpreters/Compiler.h @@ -9,10 +9,10 @@ #include #include -#include +#include #include -#include +#include #include diff --git a/dbms/include/DB/Interpreters/Context.h b/dbms/include/DB/Interpreters/Context.h index 01cd2a82cba..09731b0c37c 100644 --- a/dbms/include/DB/Interpreters/Context.h +++ b/dbms/include/DB/Interpreters/Context.h @@ -138,8 +138,14 @@ public: bool isTableExist(const String & database_name, const String & table_name) const; bool isDatabaseExist(const String & database_name) const; void assertTableExists(const String & database_name, const String & table_name) const; - void assertTableDoesntExist(const String & database_name, const String & table_name) const; - void assertDatabaseExists(const String & database_name) const; + + /** Параметр check_database_access_rights существует, чтобы не проверить повторно права доступа к БД, + * когда assertTableDoesnExist или assertDatabaseExists вызывается внутри другой функции, которая уже + * сделала эту проверку. + */ + void assertTableDoesntExist(const String & database_name, const String & table_name, bool check_database_acccess_rights = true) const; + void assertDatabaseExists(const String & database_name, bool check_database_acccess_rights = true) const; + void assertDatabaseDoesntExist(const String & database_name) const; Tables getExternalTables() const; @@ -266,6 +272,12 @@ public: void shutdown(); private: + /** Проверить, имеет ли текущий клиент доступ к заданной базе данных. + * Если доступ запрещён, кинуть исключение. + * NOTE: Этот метод надо всегда вызывать при захваченном мьютексе shared->mutex. + */ + void checkDatabaseAccessRights(const std::string & database_name) const; + const Dictionaries & getDictionariesImpl(bool throw_on_error) const; const ExternalDictionaries & getExternalDictionariesImpl(bool throw_on_error) const; diff --git a/dbms/include/DB/Interpreters/Dictionaries.h b/dbms/include/DB/Interpreters/Dictionaries.h index 5cef75de12c..c37f6a6dd70 100644 --- a/dbms/include/DB/Interpreters/Dictionaries.h +++ b/dbms/include/DB/Interpreters/Dictionaries.h @@ -146,7 +146,7 @@ public: Dictionaries(const bool throw_on_error) : Dictionaries(throw_on_error, - Application::instance().config() + Poco::Util::Application::instance().config() .getInt("builtin_dictionaries_reload_interval", 3600)) {} diff --git a/dbms/include/DB/Interpreters/ExternalDictionaries.h b/dbms/include/DB/Interpreters/ExternalDictionaries.h index 2cef0e615c0..917e9beaa8e 100644 --- a/dbms/include/DB/Interpreters/ExternalDictionaries.h +++ b/dbms/include/DB/Interpreters/ExternalDictionaries.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include #include diff --git a/dbms/include/DB/Interpreters/ProcessList.h b/dbms/include/DB/Interpreters/ProcessList.h index eee5bc859b3..41b0541377b 100644 --- a/dbms/include/DB/Interpreters/ProcessList.h +++ b/dbms/include/DB/Interpreters/ProcessList.h @@ -6,10 +6,10 @@ #include #include #include -#include +#include #include #include -#include +#include #include #include #include diff --git a/dbms/include/DB/Interpreters/Quota.h b/dbms/include/DB/Interpreters/Quota.h index dfdfedc7691..84a38b10585 100644 --- a/dbms/include/DB/Interpreters/Quota.h +++ b/dbms/include/DB/Interpreters/Quota.h @@ -14,7 +14,7 @@ #include #include -#include +#include #include #include @@ -187,7 +187,7 @@ class Quotas { private: /// Имя квоты -> квоты. - typedef std::unordered_map > Container; + typedef std::unordered_map> Container; Container cont; public: diff --git a/dbms/include/DB/Interpreters/Set.h b/dbms/include/DB/Interpreters/Set.h index bde12d53287..c8376892655 100644 --- a/dbms/include/DB/Interpreters/Set.h +++ b/dbms/include/DB/Interpreters/Set.h @@ -288,10 +288,10 @@ public: // Возвращает false, если превышено какое-нибудь ограничение, и больше не нужно вставлять. bool insertFromBlock(const Block & block, bool create_ordered_set = false); - /** Для указанных столбцов блока проверить принадлежность их значений множеству. + /** Для столбцов блока проверить принадлежность их значений множеству. * Записать результат в столбец в позиции result. */ - void execute(Block & block, const ColumnNumbers & arguments, size_t result, bool negative) const; + ColumnPtr execute(const Block & block, bool negative) const; std::string describe() const { diff --git a/dbms/include/DB/Interpreters/Settings.h b/dbms/include/DB/Interpreters/Settings.h index 231f6c741fe..bc49bc3978e 100644 --- a/dbms/include/DB/Interpreters/Settings.h +++ b/dbms/include/DB/Interpreters/Settings.h @@ -120,7 +120,7 @@ struct Settings M(SettingUInt64, merge_tree_max_rows_to_use_cache, (1024 * 1024)) \ \ /** Распределять чтение из MergeTree по потокам равномерно, обеспечивая стабильное среднее время исполнения каждого потока в пределах одного чтения. */ \ - M(SettingBool, merge_tree_uniform_read_distribution, false) \ + M(SettingBool, merge_tree_uniform_read_distribution, true) \ \ /** Минимальная длина выражения expr = x1 OR ... expr = xN для оптимизации */ \ M(SettingUInt64, optimize_min_equality_disjunction_chain_length, 3) \ diff --git a/dbms/include/DB/Interpreters/Users.h b/dbms/include/DB/Interpreters/Users.h index 04359accba6..3d7b6b1ca5f 100644 --- a/dbms/include/DB/Interpreters/Users.h +++ b/dbms/include/DB/Interpreters/Users.h @@ -11,7 +11,7 @@ #include #include -#include +#include #include #include #include @@ -23,6 +23,7 @@ #include +#include namespace DB { @@ -214,7 +215,7 @@ public: class AddressPatterns { private: - typedef std::vector > Container; + typedef std::vector> Container; Container patterns; public: @@ -252,19 +253,19 @@ public: for (Poco::Util::AbstractConfiguration::Keys::const_iterator it = config_keys.begin(); it != config_keys.end(); ++it) { - SharedPtr pattern; + Container::value_type pattern; String value = config.getString(config_elem + "." + *it); if (0 == it->compare(0, strlen("ip"), "ip")) - pattern = new IPAddressPattern(value); + pattern.reset(new IPAddressPattern(value)); else if (0 == it->compare(0, strlen("host_regexp"), "host_regexp")) - pattern = new HostRegexpPattern(value); + pattern.reset(new HostRegexpPattern(value)); else if (0 == it->compare(0, strlen("host"), "host")) - pattern = new HostExactPattern(value); + pattern.reset(new HostExactPattern(value)); else throw Exception("Unknown address pattern type: " + *it, ErrorCodes::UNKNOWN_ADDRESS_PATTERN_TYPE); - patterns.push_back(pattern); + patterns.emplace_back(std::move(pattern)); } } }; @@ -285,6 +286,10 @@ struct User AddressPatterns addresses; + /// Список разрешённых баз данных. + using DatabaseSet = std::unordered_set; + DatabaseSet databases; + User(const String & name_, const String & config_elem, Poco::Util::AbstractConfiguration & config) : name(name_) { @@ -312,6 +317,21 @@ struct User quota = config.getString(config_elem + ".quota"); addresses.addFromConfig(config_elem + ".networks", config); + + /// Заполнить список разрешённых баз данных. + const auto config_sub_elem = config_elem + ".allow_databases"; + if (config.has(config_sub_elem)) + { + Poco::Util::AbstractConfiguration::Keys config_keys; + config.keys(config_sub_elem, config_keys); + + databases.reserve(config_keys.size()); + for (const auto & key : config_keys) + { + const auto database_name = config.getString(config_sub_elem + "." + key); + databases.insert(database_name); + } + } } /// Для вставки в контейнер. @@ -384,6 +404,17 @@ public: return it->second; } + + /// Проверить, имеет ли заданный клиент доступ к заданной базе данных. + bool isAllowedDatabase(const std::string & user_name, const std::string & database_name) const + { + auto it = cont.find(user_name); + if (it == cont.end()) + throw Exception("Unknown user " + user_name, ErrorCodes::UNKNOWN_USER); + + const auto & user = it->second; + return user.databases.empty() || user.databases.count(database_name); + } }; diff --git a/dbms/include/DB/Parsers/ASTAlterQuery.h b/dbms/include/DB/Parsers/ASTAlterQuery.h index d0e88080071..d89e4c32bac 100644 --- a/dbms/include/DB/Parsers/ASTAlterQuery.h +++ b/dbms/include/DB/Parsers/ASTAlterQuery.h @@ -63,9 +63,9 @@ public: void clone(Parameters & p) const { p = *this; - p.col_decl = col_decl->clone(); - p.column = column->clone(); - p.partition = partition->clone(); + if (col_decl) p.col_decl = col_decl->clone(); + if (column) p.column = column->clone(); + if (partition) p.partition = partition->clone(); } }; typedef std::vector ParameterContainer; @@ -95,9 +95,7 @@ public: { ASTAlterQuery * res = new ASTAlterQuery(*this); for (ParameterContainer::size_type i = 0; i < parameters.size(); ++i) - { parameters[i].clone(res->parameters[i]); - } return res; } diff --git a/dbms/include/DB/Parsers/IAST.h b/dbms/include/DB/Parsers/IAST.h index 161b5fcd2cd..9e52c527b39 100644 --- a/dbms/include/DB/Parsers/IAST.h +++ b/dbms/include/DB/Parsers/IAST.h @@ -11,7 +11,7 @@ #include #include -#include +#include #include #include #include diff --git a/dbms/include/DB/Storages/CompressionMethodSelector.h b/dbms/include/DB/Storages/CompressionMethodSelector.h index 62d7708c2ca..bc7d85097fb 100644 --- a/dbms/include/DB/Storages/CompressionMethodSelector.h +++ b/dbms/include/DB/Storages/CompressionMethodSelector.h @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include diff --git a/dbms/include/DB/Storages/Distributed/DistributedBlockOutputStream.h b/dbms/include/DB/Storages/Distributed/DistributedBlockOutputStream.h index f09f824a685..318316242ce 100644 --- a/dbms/include/DB/Storages/Distributed/DistributedBlockOutputStream.h +++ b/dbms/include/DB/Storages/Distributed/DistributedBlockOutputStream.h @@ -10,7 +10,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/include/DB/Storages/IStorage.h b/dbms/include/DB/Storages/IStorage.h index 05fddedf760..2a75c147567 100644 --- a/dbms/include/DB/Storages/IStorage.h +++ b/dbms/include/DB/Storages/IStorage.h @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include @@ -208,14 +208,14 @@ public: /** Выполнить запрос (DROP|DETACH) PARTITION. */ - virtual void dropPartition(const Field & partition, bool detach, bool unreplicated, const Settings & settings) + virtual void dropPartition(ASTPtr query, const Field & partition, bool detach, bool unreplicated, const Settings & settings) { 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, const Settings & settings) + virtual void attachPartition(ASTPtr query, const Field & partition, bool unreplicated, bool part, const Settings & settings) { throw Exception("Method attachPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } diff --git a/dbms/include/DB/Storages/ITableDeclaration.h b/dbms/include/DB/Storages/ITableDeclaration.h index a20b92d1214..db03d72e7e7 100644 --- a/dbms/include/DB/Storages/ITableDeclaration.h +++ b/dbms/include/DB/Storages/ITableDeclaration.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/include/DB/Storages/MergeTree/AbandonableLockInZooKeeper.h b/dbms/include/DB/Storages/MergeTree/AbandonableLockInZooKeeper.h index c6463a7e60d..e38d768ea89 100644 --- a/dbms/include/DB/Storages/MergeTree/AbandonableLockInZooKeeper.h +++ b/dbms/include/DB/Storages/MergeTree/AbandonableLockInZooKeeper.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include diff --git a/dbms/include/DB/Storages/MergeTree/DiskSpaceMonitor.h b/dbms/include/DB/Storages/MergeTree/DiskSpaceMonitor.h index 1ebe0aab3e7..eb8df48faf4 100644 --- a/dbms/include/DB/Storages/MergeTree/DiskSpaceMonitor.h +++ b/dbms/include/DB/Storages/MergeTree/DiskSpaceMonitor.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/include/DB/Storages/MergeTree/MergeList.h b/dbms/include/DB/Storages/MergeTree/MergeList.h index 733ffd2f9ad..c49acc13a77 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeList.h +++ b/dbms/include/DB/Storages/MergeTree/MergeList.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h index eac78bf6b75..3ca8fc1e100 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h index bfd35850edd..beb7786b522 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h @@ -11,7 +11,6 @@ #include #include #include -#include #include diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeWhereOptimizer.h b/dbms/include/DB/Storages/MergeTree/MergeTreeWhereOptimizer.h index 0505bb8ffc9..165ea34b61b 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -10,8 +10,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include diff --git a/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeAddress.h b/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeAddress.h new file mode 100644 index 00000000000..931f87cf14c --- /dev/null +++ b/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeAddress.h @@ -0,0 +1,64 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +/// Позволяет узнать, куда отправлять запросы, чтобы попасть на реплику. + +struct ReplicatedMergeTreeAddress +{ + String host; + UInt16 replication_port; + UInt16 queries_port; + String database; + String table; + + ReplicatedMergeTreeAddress() {} + ReplicatedMergeTreeAddress(const String & str) + { + fromString(str); + } + + void writeText(WriteBuffer & out) const + { + out + << "host: " << escape << host << '\n' + << "port: " << replication_port << '\n' + << "tcp_port: " << queries_port << '\n' + << "database: " << escape << database << '\n' + << "table: " << escape << table << '\n'; + } + + void readText(ReadBuffer & in) + { + in + >> "host: " >> escape >> host >> "\n" + >> "port: " >> replication_port >> "\n" + >> "tcp_port: " >> queries_port >> "\n" + >> "database: " >> escape >> database >> "\n" + >> "table: " >> escape >> table >> "\n"; + } + + String toString() const + { + String res; + { + WriteBufferFromString out(res); + writeText(out); + } + return res; + } + + void fromString(const String & str) + { + ReadBufferFromString in(str); + readText(in); + } +}; + +} diff --git a/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index 6501e0dbebd..9bef6d7a98f 100644 --- a/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include @@ -86,7 +86,7 @@ struct ReplicatedMergeTreeLogEntryData /// Эти несколько полей имеют лишь информационный характер (для просмотра пользователем с помощью системных таблиц). /// Доступ под queue_mutex. size_t num_tries = 0; /// Количество попыток выполнить действие (с момента старта сервера; включая выполняющееся). - ExceptionPtr exception; /// Последний эксепшен, в случае безуспешной попытки выполнить действие. + std::exception_ptr exception; /// Последний эксепшен, в случае безуспешной попытки выполнить действие. time_t last_attempt_time = 0; /// Время начала последней попытки выполнить действие. size_t num_postponed = 0; /// Количество раз, когда действие было отложено. String postpone_reason; /// Причина, по которой действие было отложено, если оно отложено. diff --git a/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h b/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h index f416fbeda81..ac55f7a29e8 100644 --- a/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h +++ b/dbms/include/DB/Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h @@ -22,6 +22,12 @@ struct ReplicatedMergeTreeQuorumEntry size_t required_number_of_replicas; std::set replicas; + ReplicatedMergeTreeQuorumEntry() {} + ReplicatedMergeTreeQuorumEntry(const String & str) + { + fromString(str); + } + void writeText(WriteBuffer & out) const { out << "version: 1\n" diff --git a/dbms/include/DB/Storages/StorageChunkMerger.h b/dbms/include/DB/Storages/StorageChunkMerger.h index 2ca2a90f265..b6c9528ca20 100644 --- a/dbms/include/DB/Storages/StorageChunkMerger.h +++ b/dbms/include/DB/Storages/StorageChunkMerger.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include diff --git a/dbms/include/DB/Storages/StorageMerge.h b/dbms/include/DB/Storages/StorageMerge.h index e42e0efcd0b..a39952c95b0 100644 --- a/dbms/include/DB/Storages/StorageMerge.h +++ b/dbms/include/DB/Storages/StorageMerge.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include diff --git a/dbms/include/DB/Storages/StorageMergeTree.h b/dbms/include/DB/Storages/StorageMergeTree.h index da39ebf0a1e..344caff0af3 100644 --- a/dbms/include/DB/Storages/StorageMergeTree.h +++ b/dbms/include/DB/Storages/StorageMergeTree.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include namespace DB @@ -89,8 +89,8 @@ public: return merge(settings.min_bytes_to_use_direct_io, true); } - void dropPartition(const Field & partition, bool detach, bool unreplicated, const Settings & settings) override; - void attachPartition(const Field & partition, bool unreplicated, bool part, const Settings & settings) override; + void dropPartition(ASTPtr query, const Field & partition, bool detach, bool unreplicated, const Settings & settings) override; + void attachPartition(ASTPtr query, const Field & partition, bool unreplicated, bool part, const Settings & settings) override; void freezePartition(const Field & partition, const Settings & settings) override; void drop() override; diff --git a/dbms/include/DB/Storages/StorageReplicatedMergeTree.h b/dbms/include/DB/Storages/StorageReplicatedMergeTree.h index 60a21abd9f4..02e70818f58 100644 --- a/dbms/include/DB/Storages/StorageReplicatedMergeTree.h +++ b/dbms/include/DB/Storages/StorageReplicatedMergeTree.h @@ -127,8 +127,8 @@ public: void alter(const AlterCommands & params, const String & database_name, const String & table_name, Context & context) override; - void dropPartition(const Field & partition, bool detach, bool unreplicated, const Settings & settings) override; - void attachPartition(const Field & partition, bool unreplicated, bool part, const Settings & settings) override; + void dropPartition(ASTPtr query, const Field & partition, bool detach, bool unreplicated, const Settings & settings) override; + void attachPartition(ASTPtr query, const Field & partition, bool unreplicated, bool part, const Settings & settings) override; void fetchPartition(const Field & partition, const String & from, const Settings & settings) override; void freezePartition(const Field & partition, const Settings & settings) override; diff --git a/dbms/include/DB/TableFunctions/TableFunctionMerge.h b/dbms/include/DB/TableFunctions/TableFunctionMerge.h index 04d7dd0fe37..9dc6525e561 100644 --- a/dbms/include/DB/TableFunctions/TableFunctionMerge.h +++ b/dbms/include/DB/TableFunctions/TableFunctionMerge.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include diff --git a/dbms/src/AggregateFunctions/AggregateFunctionsUniq.cpp b/dbms/src/AggregateFunctions/AggregateFunctionsUniq.cpp index c3b23a8f36b..a6bf03abb4c 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionsUniq.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionsUniq.cpp @@ -39,16 +39,16 @@ AggregateFunctionPtr createAggregateFunctionUniqExact(const std::string & name, const IDataType & argument_type = *argument_types[0]; - AggregateFunctionPtr res = createWithNumericType(*argument_types[0]); + AggregateFunctionPtr res = createWithNumericType(*argument_types[0]); if (res) return res; else if (typeid_cast(&argument_type)) - return new AggregateFunctionUniq; + return new AggregateFunctionUniq >; else if (typeid_cast(&argument_type)) - return new AggregateFunctionUniq; + return new AggregateFunctionUniq >; else if (typeid_cast(&argument_type) || typeid_cast(&argument_type)) - return new AggregateFunctionUniq; + return new AggregateFunctionUniq >; else throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); diff --git a/dbms/src/Client/Benchmark.cpp b/dbms/src/Client/Benchmark.cpp index 4c88eebae41..ff872cd5b1e 100644 --- a/dbms/src/Client/Benchmark.cpp +++ b/dbms/src/Client/Benchmark.cpp @@ -11,15 +11,15 @@ #include #include -#include -#include +#include +#include #include #include #include -#include +#include #include #include diff --git a/dbms/src/Client/Client.cpp b/dbms/src/Client/Client.cpp index efd8e8012e1..af6b65607bb 100644 --- a/dbms/src/Client/Client.cpp +++ b/dbms/src/Client/Client.cpp @@ -21,9 +21,9 @@ #include -#include +#include -#include +#include #include #include @@ -60,7 +60,7 @@ #include #include -#include +#include /// http://en.wikipedia.org/wiki/ANSI_escape_code #define SAVE_CURSOR_POSITION "\033[s" @@ -135,7 +135,7 @@ private: ASTPtr parsed_query; /// Последнее полученное от сервера исключение. Для кода возврата в неинтерактивном режиме. - ExceptionPtr last_exception; + Poco::SharedPtr last_exception; /// Было ли в последнем запросе исключение. bool got_exception = false; diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 9ab713494a8..cd4b35ce9c3 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -5,7 +5,7 @@ #include #include -#include +#include #include #include @@ -21,7 +21,7 @@ #include -#include +#include namespace DB diff --git a/dbms/src/Client/InterruptListener.h b/dbms/src/Client/InterruptListener.h index f6c243fd4d7..a8e95eec669 100644 --- a/dbms/src/Client/InterruptListener.h +++ b/dbms/src/Client/InterruptListener.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include diff --git a/dbms/src/Common/ConfigProcessor.cpp b/dbms/src/Common/ConfigProcessor.cpp new file mode 100644 index 00000000000..f71bcb6d7ef --- /dev/null +++ b/dbms/src/Common/ConfigProcessor.cpp @@ -0,0 +1,385 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + +using namespace Poco::XML; + + +static bool endsWith(const std::string & s, const std::string & suffix) +{ + return s.size() >= suffix.size() && s.substr(s.size() - suffix.size()) == suffix; +} + +/// Извлекает из строки первое попавшееся число, состоящее из хотя бы двух цифр. +static std::string numberFromHost(const std::string & s) +{ + for (size_t i = 0; i < s.size(); ++i) + { + std::string res; + size_t j = i; + while (j < s.size() && isdigit(s[j])) + res += s[j++]; + if (res.size() >= 2) + { + while (res[0] == '0') + res.erase(res.begin()); + return res; + } + } + return ""; +} + +ConfigProcessor::ConfigProcessor(bool throw_on_bad_incl_, bool log_to_console, const Substitutions & substitutions_) + : throw_on_bad_incl(throw_on_bad_incl_), substitutions(substitutions_) +{ + if (log_to_console && Logger::has("ConfigProcessor") == nullptr) + { + Poco::Channel * channel = new Poco::ConsoleChannel; + channel_ptr = channel; + log = &Logger::create("ConfigProcessor", channel, Poco::Message::PRIO_TRACE); + } + else + { + log = &Logger::get("ConfigProcessor"); + } +} + +/// Вектор из имени элемента и отсортированного списка имен и значений атрибутов (кроме атрибутов replace и remove). +/// Взаимно однозначно задает имя элемента и список его атрибутов. Нужен, чтобы сравнивать элементы. +typedef std::vector ElementIdentifier; + +typedef Poco::AutoPtr NamedNodeMapPtr; +/// NOTE Можно избавиться от использования Node.childNodes() и итерации по полученному списку, потому что +/// доступ к i-му элементу этого списка работает за O(i). +typedef Poco::AutoPtr NodeListPtr; + +static ElementIdentifier getElementIdentifier(Node * element) +{ + NamedNodeMapPtr attrs = element->attributes(); + std::vector > attrs_kv; + for (size_t i = 0; i < attrs->length(); ++i) + { + Node * node = attrs->item(i); + std::string name = node->nodeName(); + if (name == "replace" || name == "remove" || name == "incl") + continue; + std::string value = node->nodeValue(); + attrs_kv.push_back(std::make_pair(name, value)); + } + std::sort(attrs_kv.begin(), attrs_kv.end()); + + ElementIdentifier res; + res.push_back(element->nodeName()); + for (const auto & attr : attrs_kv) + { + res.push_back(attr.first); + res.push_back(attr.second); + } + + return res; +} + +static Node * getRootNode(Document * document) +{ + NodeListPtr children = document->childNodes(); + for (size_t i = 0; i < children->length(); ++i) + { + Node * child = children->item(i); + /// Кроме корневого элемента на верхнем уровне могут быть комментарии. Пропустим их. + if (child->nodeType() == Node::ELEMENT_NODE) + return child; + } + + throw Poco::Exception("No root node in document"); +} + +static bool allWhitespace(const std::string & s) +{ + return s.find_first_not_of(" \t\n\r") == std::string::npos; +} + +void ConfigProcessor::mergeRecursive(DocumentPtr config, Node * config_root, Node * with_root) +{ + NodeListPtr with_nodes = with_root->childNodes(); + typedef std::multimap ElementsByIdentifier; + ElementsByIdentifier config_element_by_id; + for (Node * node = config_root->firstChild(); node;) + { + Node * next_node = node->nextSibling(); + /// Уберем исходный текст из объединяемой части. + if (node->nodeType() == Node::TEXT_NODE && !allWhitespace(node->getNodeValue())) + { + config_root->removeChild(node); + } + else if (node->nodeType() == Node::ELEMENT_NODE) + { + config_element_by_id.insert(ElementsByIdentifier::value_type(getElementIdentifier(node), node)); + } + node = next_node; + } + + for (size_t i = 0; i < with_nodes->length(); ++i) + { + Node * with_node = with_nodes->item(i); + + bool merged = false; + bool remove = false; + if (with_node->nodeType() == Node::ELEMENT_NODE) + { + Element * with_element = dynamic_cast(with_node); + remove = with_element->hasAttribute("remove"); + bool replace = with_element->hasAttribute("replace"); + + if (remove && replace) + throw Poco::Exception("remove and replace attributes on the same element"); + + ElementsByIdentifier::iterator it = config_element_by_id.find(getElementIdentifier(with_node)); + + if (it != config_element_by_id.end()) + { + Node * config_node = it->second; + config_element_by_id.erase(it); + + if (remove) + { + config_root->removeChild(config_node); + } + else if (replace) + { + with_element->removeAttribute("replace"); + NodePtr new_node = config->importNode(with_node, true); + config_root->replaceChild(new_node, config_node); + } + else + { + mergeRecursive(config, config_node, with_node); + } + merged = true; + } + } + if (!merged && !remove) + { + NodePtr new_node = config->importNode(with_node, true); + config_root->appendChild(new_node); + } + } +} + +void ConfigProcessor::merge(DocumentPtr config, DocumentPtr with) +{ + mergeRecursive(config, getRootNode(&*config), getRootNode(&*with)); +} + +std::string ConfigProcessor::layerFromHost() +{ + utsname buf; + if (uname(&buf)) + throw Poco::Exception(std::string("uname failed: ") + std::strerror(errno)); + + std::string layer = numberFromHost(buf.nodename); + if (layer.empty()) + throw Poco::Exception(std::string("no layer in host name: ") + buf.nodename); + + return layer; +} + +void ConfigProcessor::doIncludesRecursive(DocumentPtr config, DocumentPtr include_from, Node * node) +{ + if (node->nodeType() == Node::TEXT_NODE) + { + for (auto & substitution : substitutions) + { + std::string value = node->nodeValue(); + + bool replace_occured = false; + size_t pos; + while ((pos = value.find(substitution.first)) != std::string::npos) + { + value.replace(pos, substitution.first.length(), substitution.second); + replace_occured = true; + } + + if (replace_occured) + node->setNodeValue(value); + } + } + + if (node->nodeType() != Node::ELEMENT_NODE) + return; + + /// Будем заменять на число из имени хоста, только если во входном файле есть тег , и он пустой, и у него нет атрибутов + if ( node->nodeName() == "layer" && + !node->hasAttributes() && + !node->hasChildNodes() && + node->nodeValue().empty()) + { + NodePtr new_node = config->createTextNode(layerFromHost()); + node->appendChild(new_node); + return; + } + + NamedNodeMapPtr attributes = node->attributes(); + Node * incl_attribute = attributes->getNamedItem("incl"); + + /// Заменять имеющееся значение, а не добавлять к нему. + bool replace = attributes->getNamedItem("replace"); + + if (incl_attribute) + { + std::string name = incl_attribute->getNodeValue(); + Node * included_node = include_from ? include_from->getNodeByPath("yandex/" + name) : nullptr; + if (!included_node) + { + if (attributes->getNamedItem("optional")) + node->parentNode()->removeChild(node); + else if (throw_on_bad_incl) + throw Poco::Exception("Include not found: " + name); + else + LOG_WARNING(log, "Include not found: " << name); + } + else + { + if (replace) + while (Node * child = node->firstChild()) + node->removeChild(child); + + NodeListPtr children = included_node->childNodes(); + for (size_t i = 0; i < children->length(); ++i) + { + NodePtr new_node = config->importNode(children->item(i), true); + node->appendChild(new_node); + } + + Element * element = dynamic_cast(node); + element->removeAttribute("incl"); + + if (replace) + element->removeAttribute("replace"); + + NamedNodeMapPtr from_attrs = included_node->attributes(); + for (size_t i = 0; i < from_attrs->length(); ++i) + { + element->setAttributeNode(dynamic_cast(config->importNode(from_attrs->item(i), true))); + } + } + } + + NodeListPtr children = node->childNodes(); + for (size_t i = 0; i < children->length(); ++i) + { + doIncludesRecursive(config, include_from, children->item(i)); + } +} + +void ConfigProcessor::doIncludes(DocumentPtr config, DocumentPtr include_from) +{ + doIncludesRecursive(config, include_from, getRootNode(&*config)); +} + +XMLDocumentPtr ConfigProcessor::processConfig(const std::string & path_str) +{ + DocumentPtr config = DOMParser().parse(path_str); + + std::vector contributing_files; + contributing_files.push_back(path_str); + Poco::Path merge_dir_path(path_str); + merge_dir_path.setExtension("d"); + std::vector merge_dirs; + merge_dirs.push_back(merge_dir_path.toString()); + if (merge_dir_path.getBaseName() != "conf") + { + merge_dir_path.setBaseName("conf"); + merge_dirs.push_back(merge_dir_path.toString()); + } + for (const std::string & merge_dir_name : merge_dirs) + { + Poco::File merge_dir(merge_dir_name); + if (!merge_dir.exists() || !merge_dir.isDirectory()) + continue; + for (Poco::DirectoryIterator it(merge_dir_name); it != Poco::DirectoryIterator(); ++it) + { + Poco::File & file = *it; + try + { + if (file.isFile() && (endsWith(file.path(), ".xml") || endsWith(file.path(), ".conf"))) + { + contributing_files.push_back(file.path()); + DocumentPtr with = DOMParser().parse(file.path()); + merge(config, with); + } + } + catch (Poco::Exception & e) + { + throw Poco::Exception("Failed to merge config with " + file.path() + ": " + e.displayText()); + } + } + } + + try + { + Node * node = config->getNodeByPath("yandex/include_from"); + DocumentPtr include_from; + std::string include_from_path; + if (node) + { + include_from_path = node->innerText(); + } + else + { + std::string default_path = "/etc/metrika.xml"; + if (Poco::File(default_path).exists()) + include_from_path = default_path; + } + if (!include_from_path.empty()) + { + contributing_files.push_back(include_from_path); + include_from = DOMParser().parse(include_from_path); + } + + doIncludes(config, include_from); + } + catch (Poco::Exception & e) + { + throw Poco::Exception("Failed to preprocess config: " + e.displayText()); + } + + std::stringstream comment; + comment << " This file was generated automatically.\n"; + comment << " Do not edit it: it is likely to be discarded and generated again before it's read next time.\n"; + comment << " Files used to generate this file:"; + for (const std::string & path : contributing_files) + { + comment << "\n " << path; + } + comment<<" "; + NodePtr new_node = config->createTextNode("\n\n"); + config->insertBefore(new_node, config->firstChild()); + new_node = config->createComment(comment.str()); + config->insertBefore(new_node, config->firstChild()); + + return config; +} + +ConfigurationPtr ConfigProcessor::loadConfig(const std::string & path) +{ + DocumentPtr res = processConfig(path); + + Poco::Path preprocessed_path(path); + preprocessed_path.setBaseName(preprocessed_path.getBaseName() + "-preprocessed"); + try + { + DOMWriter().writeNode(preprocessed_path.toString(), res); + } + catch (Poco::Exception & e) + { + LOG_WARNING(log, "Couldn't save preprocessed config to " << preprocessed_path.toString() << ": " << e.displayText()); + } + + return new Poco::Util::XMLConfiguration(res); +} diff --git a/dbms/src/Core/Exception.cpp b/dbms/src/Common/Exception.cpp similarity index 78% rename from dbms/src/Core/Exception.cpp rename to dbms/src/Common/Exception.cpp index 84624039b69..ae8a58a67f5 100644 --- a/dbms/src/Core/Exception.cpp +++ b/dbms/src/Common/Exception.cpp @@ -6,7 +6,7 @@ #include -#include +#include namespace DB @@ -19,30 +19,6 @@ void throwFromErrno(const std::string & s, int code, int e) } -ExceptionPtr cloneCurrentException() -{ - try - { - throw; - } - catch (const Exception & e) - { - return e.clone(); - } - catch (const Poco::Exception & e) - { - return e.clone(); - } - catch (const std::exception & e) - { - return new Exception(e.what(), ErrorCodes::STD_EXCEPTION); - } - catch (...) - { - return new Exception("Unknown exception", ErrorCodes::UNKNOWN_EXCEPTION); - } -} - inline std::string demangle(const char * const mangled, int & status) { const auto demangled_str = abi::__cxa_demangle(mangled, 0, 0, &status); @@ -129,11 +105,73 @@ std::string getCurrentExceptionMessage(bool with_stacktrace) } +Poco::SharedPtr convertCurrentException() +{ + try + { + throw; + } + catch (const Exception & e) + { + return e.clone(); + } + catch (const Poco::Exception & e) + { + return e.clone(); + } + catch (const std::exception & e) + { + return new Exception(e.what(), ErrorCodes::STD_EXCEPTION); + } + catch (...) + { + return new Exception("Unknown exception", ErrorCodes::UNKNOWN_EXCEPTION); + } +} + + void rethrowFirstException(Exceptions & exceptions) { for (size_t i = 0, size = exceptions.size(); i < size; ++i) if (exceptions[i]) - exceptions[i]->rethrow(); + std::rethrow_exception(exceptions[i]); +} + + +void tryLogException(std::exception_ptr e, const char * log_name, const std::string & start_of_message) +{ + try + { + std::rethrow_exception(e); + } + catch (...) + { + tryLogCurrentException(log_name, start_of_message); + } +} + +void tryLogException(std::exception_ptr e, Poco::Logger * logger, const std::string & start_of_message) +{ + try + { + std::rethrow_exception(e); + } + catch (...) + { + tryLogCurrentException(logger, start_of_message); + } +} + +std::string getExceptionMessage(std::exception_ptr e, bool with_stacktrace) +{ + try + { + std::rethrow_exception(e); + } + catch (...) + { + return getCurrentExceptionMessage(with_stacktrace); + } } diff --git a/dbms/src/Common/Macros.cpp b/dbms/src/Common/Macros.cpp index 9428ed13fb0..9e09a60accf 100644 --- a/dbms/src/Common/Macros.cpp +++ b/dbms/src/Common/Macros.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include namespace DB diff --git a/dbms/src/Common/MemoryTracker.cpp b/dbms/src/Common/MemoryTracker.cpp index 39f17506b62..49a92bc4f64 100644 --- a/dbms/src/Common/MemoryTracker.cpp +++ b/dbms/src/Common/MemoryTracker.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/dbms/src/Common/StackTrace.cpp b/dbms/src/Common/StackTrace.cpp new file mode 100644 index 00000000000..da80cfda37e --- /dev/null +++ b/dbms/src/Common/StackTrace.cpp @@ -0,0 +1,74 @@ +#include +#include +#include +#include + +#include + +#include + + +StackTrace::StackTrace() +{ + frames_size = backtrace(frames, STACK_TRACE_MAX_DEPTH); +} + +std::string StackTrace::toString() const +{ + char ** symbols = backtrace_symbols(frames, frames_size); + std::stringstream res; + + if (!symbols) + return "Cannot get symbols for stack trace.\n"; + + try + { + for (size_t i = 0, size = frames_size; i < size; ++i) + { + /// Делаем demangling имён. Имя находится в скобках, до символа '+'. + + char * name_start = nullptr; + char * name_end = nullptr; + char * demangled_name = nullptr; + int status = 0; + + if (nullptr != (name_start = strchr(symbols[i], '(')) + && nullptr != (name_end = strchr(name_start, '+'))) + { + ++name_start; + *name_end = '\0'; + demangled_name = abi::__cxa_demangle(name_start, 0, 0, &status); + *name_end = '+'; + } + + try + { + res << i << ". "; + + if (nullptr != demangled_name && 0 == status) + { + res.write(symbols[i], name_start - symbols[i]); + res << demangled_name << name_end; + } + else + res << symbols[i]; + + res << std::endl; + } + catch (...) + { + free(demangled_name); + throw; + } + free(demangled_name); + } + } + catch (...) + { + free(symbols); + throw; + } + + free(symbols); + return res.str(); +} diff --git a/dbms/src/Common/localBackup.cpp b/dbms/src/Common/localBackup.cpp index 51e93736550..594ca18ec03 100644 --- a/dbms/src/Common/localBackup.cpp +++ b/dbms/src/Common/localBackup.cpp @@ -7,7 +7,7 @@ #include #include -#include +#include #include diff --git a/dbms/src/Common/setThreadName.cpp b/dbms/src/Common/setThreadName.cpp index 5f6b05254cb..fed59b3c1bd 100644 --- a/dbms/src/Common/setThreadName.cpp +++ b/dbms/src/Common/setThreadName.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include diff --git a/dbms/src/Common/tests/auto_array.cpp b/dbms/src/Common/tests/auto_array.cpp index 91385354044..ba6f057ba44 100644 --- a/dbms/src/Common/tests/auto_array.cpp +++ b/dbms/src/Common/tests/auto_array.cpp @@ -7,7 +7,7 @@ #include #include -#include +#include int main(int argc, char ** argv) diff --git a/dbms/src/Common/tests/hashes_test.cpp b/dbms/src/Common/tests/hashes_test.cpp index fbf25db689c..0d99c6a084c 100644 --- a/dbms/src/Common/tests/hashes_test.cpp +++ b/dbms/src/Common/tests/hashes_test.cpp @@ -4,7 +4,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/src/Common/tests/int_hashes_perf.cpp b/dbms/src/Common/tests/int_hashes_perf.cpp index ea17ac62f49..4a4259165b1 100644 --- a/dbms/src/Common/tests/int_hashes_perf.cpp +++ b/dbms/src/Common/tests/int_hashes_perf.cpp @@ -6,7 +6,7 @@ #include #include -#include +#include #include "AvalancheTest.h" /// Взято из SMHasher. diff --git a/dbms/src/Common/tests/parallel_aggregation.cpp b/dbms/src/Common/tests/parallel_aggregation.cpp index 56357a4fd4d..084b48a68e4 100644 --- a/dbms/src/Common/tests/parallel_aggregation.cpp +++ b/dbms/src/Common/tests/parallel_aggregation.cpp @@ -15,8 +15,8 @@ #include #include -#include -#include +#include +#include typedef UInt64 Key; diff --git a/dbms/src/Common/tests/parallel_aggregation2.cpp b/dbms/src/Common/tests/parallel_aggregation2.cpp index b3466c4bdd3..b341ce3d7c0 100644 --- a/dbms/src/Common/tests/parallel_aggregation2.cpp +++ b/dbms/src/Common/tests/parallel_aggregation2.cpp @@ -15,8 +15,8 @@ #include #include -#include -#include +#include +#include typedef UInt64 Key; diff --git a/dbms/src/Common/tests/sip_hash_perf.cpp b/dbms/src/Common/tests/sip_hash_perf.cpp index dcd397ad5b9..a41487e83d5 100644 --- a/dbms/src/Common/tests/sip_hash_perf.cpp +++ b/dbms/src/Common/tests/sip_hash_perf.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include /** Тестировать так: diff --git a/dbms/src/Core/Block.cpp b/dbms/src/Core/Block.cpp index 4d786c94a6e..1943e8a855a 100644 --- a/dbms/src/Core/Block.cpp +++ b/dbms/src/Core/Block.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include diff --git a/dbms/src/Core/tests/field.cpp b/dbms/src/Core/tests/field.cpp index 1dfee28ca0a..bf091ba6cef 100644 --- a/dbms/src/Core/tests/field.cpp +++ b/dbms/src/Core/tests/field.cpp @@ -4,7 +4,7 @@ #include -#include +#include #include #include #include diff --git a/dbms/src/Core/tests/string_pool.cpp b/dbms/src/Core/tests/string_pool.cpp index 1df818baf7f..6cf5496b3a7 100644 --- a/dbms/src/Core/tests/string_pool.cpp +++ b/dbms/src/Core/tests/string_pool.cpp @@ -4,7 +4,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/src/DataStreams/BlockInputStreamFromRowInputStream.cpp b/dbms/src/DataStreams/BlockInputStreamFromRowInputStream.cpp index af46a124cd0..9e9d28eb387 100644 --- a/dbms/src/DataStreams/BlockInputStreamFromRowInputStream.cpp +++ b/dbms/src/DataStreams/BlockInputStreamFromRowInputStream.cpp @@ -1,8 +1,7 @@ -#include +#include #include #include -#include #include @@ -53,9 +52,9 @@ Block BlockInputStreamFromRowInputStream::readImpl() e.addMessage("(at row " + toString(total_rows + 1) + ")"); throw; } - + res.optimizeNestedArraysOffsets(); - + return res; } diff --git a/dbms/src/DataStreams/tests/glue_streams.cpp b/dbms/src/DataStreams/tests/glue_streams.cpp index 70746b8cc54..a825315a00c 100644 --- a/dbms/src/DataStreams/tests/glue_streams.cpp +++ b/dbms/src/DataStreams/tests/glue_streams.cpp @@ -1,7 +1,7 @@ #include #include -#include +#include #include diff --git a/dbms/src/DataTypes/DataTypeNested.cpp b/dbms/src/DataTypes/DataTypeNested.cpp index e6d022f0141..cf2f70e1064 100644 --- a/dbms/src/DataTypes/DataTypeNested.cpp +++ b/dbms/src/DataTypes/DataTypeNested.cpp @@ -1,25 +1,18 @@ -#include -#include -#include - #include #include #include -#include - -#include - #include +#include namespace DB { -DataTypeNested::DataTypeNested(NamesAndTypesListPtr nested_) : nested(nested_) +DataTypeNested::DataTypeNested(NamesAndTypesListPtr nested_) + : nested(nested_) { - offsets = new DataTypeFromFieldType::Type; } @@ -71,174 +64,6 @@ std::string DataTypeNested::getName() const } -void DataTypeNested::serializeBinary(const Field & field, WriteBuffer & ostr) const -{ - throw Exception("Method serializeBinary(const Field &, WriteBuffer &) is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); -} - - -void DataTypeNested::deserializeBinary(Field & field, ReadBuffer & istr) const -{ - throw Exception("Method deserializeBinary(Field &, ReadBuffer &) is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); -} - - -void DataTypeNested::serializeBinary(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const -{ - const ColumnNested & column_nested = typeid_cast(column); - const ColumnNested::Offsets_t & offsets = column_nested.getOffsets(); - - if (offset > offsets.size()) - return; - - /** offset - с какого массива писать. - * limit - сколько массивов максимум записать, или 0, если писать всё, что есть. - * end - до какого массива заканчивается записываемый кусок. - * - * nested_offset - с какого элемента внутренностей писать. - * nested_limit - сколько элементов внутренностей писать, или 0, если писать всё, что есть. - */ - - size_t end = std::min(offset + limit, offsets.size()); - - size_t nested_offset = offset ? offsets[offset - 1] : 0; - size_t nested_limit = limit - ? offsets[end - 1] - nested_offset - : 0; - - if (limit == 0 || nested_limit) - { - NamesAndTypesList::const_iterator it = nested->begin(); - for (size_t i = 0; i < nested->size(); ++i, ++it) - it->type->serializeBinary(*column_nested.getData()[i], ostr, nested_offset, nested_limit); - } -} - - -void DataTypeNested::deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const -{ - ColumnNested & column_nested = typeid_cast(column); - ColumnNested::Offsets_t & offsets = column_nested.getOffsets(); - - /// Должно быть считано согласованное с offsets количество значений. - size_t last_offset = (offsets.empty() ? 0 : offsets.back()); - if (last_offset < column_nested.size()) - throw Exception("Nested column longer than last offset", ErrorCodes::LOGICAL_ERROR); - size_t nested_limit = (offsets.empty() ? 0 : offsets.back()) - column_nested.size(); - - NamesAndTypesList::const_iterator it = nested->begin(); - for (size_t i = 0; i < nested->size(); ++i, ++it) - { - it->type->deserializeBinary(*column_nested.getData()[i], istr, nested_limit, 0); - if (column_nested.getData()[i]->size() != last_offset) - throw Exception("Cannot read all nested column values", ErrorCodes::CANNOT_READ_ALL_DATA); - } -} - - -void DataTypeNested::serializeOffsets(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const -{ - const ColumnNested & column_nested = typeid_cast(column); - const ColumnNested::Offsets_t & offsets = column_nested.getOffsets(); - size_t size = offsets.size(); - - if (!size) - return; - - size_t end = limit && (offset + limit < size) - ? offset + limit - : size; - - if (offset == 0) - { - writeIntBinary(offsets[0], ostr); - ++offset; - } - - for (size_t i = offset; i < end; ++i) - writeIntBinary(offsets[i] - offsets[i - 1], ostr); -} - - -void DataTypeNested::deserializeOffsets(IColumn & column, ReadBuffer & istr, size_t limit) const -{ - ColumnNested & column_nested = typeid_cast(column); - ColumnNested::Offsets_t & offsets = column_nested.getOffsets(); - size_t initial_size = offsets.size(); - offsets.resize(initial_size + limit); - - size_t i = initial_size; - ColumnNested::Offset_t current_offset = initial_size ? offsets[initial_size - 1] : 0; - while (i < initial_size + limit && !istr.eof()) - { - ColumnNested::Offset_t current_size = 0; - readIntBinary(current_size, istr); - current_offset += current_size; - offsets[i] = current_offset; - ++i; - } - - offsets.resize(i); -} - - -void DataTypeNested::serializeText(const Field & field, WriteBuffer & ostr) const -{ - throw Exception("Method get is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); -} - - -void DataTypeNested::deserializeText(Field & field, ReadBuffer & istr) const -{ - throw Exception("Method get is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); -} - - -void DataTypeNested::serializeTextEscaped(const Field & field, WriteBuffer & ostr) const -{ - serializeText(field, ostr); -} - - -void DataTypeNested::deserializeTextEscaped(Field & field, ReadBuffer & istr) const -{ - deserializeText(field, istr); -} - - -void DataTypeNested::serializeTextQuoted(const Field & field, WriteBuffer & ostr) const -{ - serializeText(field, ostr); -} - - -void DataTypeNested::deserializeTextQuoted(Field & field, ReadBuffer & istr) const -{ - deserializeText(field, istr); -} - - -void DataTypeNested::serializeTextJSON(const Field & field, WriteBuffer & ostr) const -{ - throw Exception("Method get is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); -} - - -ColumnPtr DataTypeNested::createColumn() const -{ - Columns columns; - columns.reserve(nested->size()); - for (NamesAndTypesList::const_iterator it = nested->begin(); it != nested->end(); ++it) - columns.push_back(it->type->createColumn()); - return new ColumnNested(columns); -} - - -ColumnPtr DataTypeNested::createConstColumn(size_t size, const Field & field) const -{ - throw Exception("Method createConstColumn is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); -} - NamesAndTypesListPtr DataTypeNested::expandNestedColumns(const NamesAndTypesList & names_and_types) { NamesAndTypesListPtr columns = new NamesAndTypesList; @@ -258,4 +83,5 @@ NamesAndTypesListPtr DataTypeNested::expandNestedColumns(const NamesAndTypesList } return columns; } + } diff --git a/dbms/src/DataTypes/tests/data_type_nested.cpp b/dbms/src/DataTypes/tests/data_type_nested.cpp deleted file mode 100644 index 222af5b57d1..00000000000 --- a/dbms/src/DataTypes/tests/data_type_nested.cpp +++ /dev/null @@ -1,176 +0,0 @@ -#include - -#include -#include - -#include -#include - -#include -#include - -#include -#include -#include - -#include -#include -#include - -struct Nested { - DB::PODArray uint8; - DB::PODArray uint64; - std::vector string; -}; - -const size_t n = 4; -const size_t sizes[n] = {3, 1, 4, 2}; - -int main(int argc, char ** argv) -{ - try - { - Poco::Stopwatch stopwatch; - - /// Nested(uint8 UInt8, uint64 UInt64, string String) - Nested nested[n]; - for (size_t i = 0; i < n; ++i) - { - for (size_t j = 0; j < sizes[i]; ++j) - { - nested[i].uint8.push_back(i * 4 + j); - nested[i].uint64.push_back(1ULL << (63 - nested[i].uint8.back())); - nested[i].string.push_back(""); - { - DB::WriteBufferFromString wb(nested[i].string.back()); - DB::writeIntText(nested[i].uint8.back(), wb); - DB::writeString("SpAcE", wb); - DB::writeIntText(nested[i].uint64.back(), wb); - } - } - } - DB::NamesAndTypesListPtr types = new DB::NamesAndTypesList; - types->push_back(DB::NameAndTypePair("uint8", new DB::DataTypeUInt8)); - types->push_back(DB::NameAndTypePair("uint64", new DB::DataTypeUInt64)); - types->push_back(DB::NameAndTypePair("string", new DB::DataTypeString)); - - DB::DataTypeNested data_type(types); - - { - DB::ColumnPtr column_p = data_type.createColumn(); - DB::ColumnNested * column = typeid_cast(&*column_p); - DB::Columns & data = column->getData(); - DB::ColumnNested::Offsets_t & offsets = column->getOffsets(); - - data.resize(3); - data[0] = new DB::ColumnUInt8; - data[1] = new DB::ColumnUInt64; - data[2] = new DB::ColumnString; - - for (size_t i = 0; i < n; ++i) - { - for (size_t j = 0; j < sizes[i]; ++j) - { - data[0]->insert(DB::Field(UInt64(nested[i].uint8[j]))); - data[1]->insert(DB::Field(nested[i].uint64[j])); - data[2]->insert(DB::Field(nested[i].string[j].data(), nested[i].string[j].size())); - } - offsets.push_back((offsets.size() == 0 ? 0 : offsets.back()) + sizes[i]); - } - - stopwatch.restart(); - { - std::ofstream ostr("test.size"); - DB::WriteBufferFromOStream out_buf(ostr); - data_type.serializeOffsets(*column, out_buf); - } - { - std::ofstream ostr("test"); - DB::WriteBufferFromOStream out_buf(ostr); - data_type.serializeBinary(*column, out_buf); - } - stopwatch.stop(); - - std::cout << "Writing, elapsed: " << static_cast(stopwatch.elapsed()) / 1000000 << std::endl; - } - - { - DB::ColumnPtr column_p = data_type.createColumn(); - DB::ColumnNested * column = typeid_cast(&*column_p); - - std::ifstream istr("test"); - DB::ReadBufferFromIStream in_buf(istr); - - stopwatch.restart(); - { - std::ifstream istr("test.size"); - DB::ReadBufferFromIStream in_buf(istr); - data_type.deserializeOffsets(*column, in_buf, n); - } - { - std::ifstream istr("test"); - DB::ReadBufferFromIStream in_buf(istr); - data_type.deserializeBinary(*column, in_buf, n, 0); - } - stopwatch.stop(); - - std::cout << "Reading, elapsed: " << static_cast(stopwatch.elapsed()) / 1000000 << std::endl; - - std::cout << std::endl; - - DB::Columns & data = column->getData(); - DB::ColumnNested::Offsets_t & offsets = column->getOffsets(); - - Nested res; - res.uint8.assign(typeid_cast(*data[0]).getData()); - res.uint64.assign(typeid_cast(*data[1]).getData()); - DB::ColumnString & res_string = typeid_cast(*data[2]); - - std::cout << "offsets: ["; - for (size_t i = 0; i < offsets.size(); ++i) - { - if (i) std::cout << ", "; - std::cout << offsets[i]; - } - std::cout << "]\n" << std::endl; - - for (size_t i = 0; i < n; ++i) - { - size_t sh = i ? offsets[i - 1] : 0; - - std::cout << "["; - for (size_t j = 0; j < sizes[i]; ++j) - { - if (j) std::cout << ", "; - std::cout << int(res.uint8[sh + j]); - } - std::cout << "]\n"; - - std::cout << "["; - for (size_t j = 0; j < sizes[i]; ++j) - { - if (j) std::cout << ", "; - std::cout << res.uint64[sh + j]; - } - std::cout << "]\n"; - - std::cout << "["; - for (size_t j = 0; j < sizes[i]; ++j) - { - if (j) std::cout << ", "; - std::cout << '"' << res_string.getDataAt(sh + j).toString() << '"'; - } - std::cout << "]\n"; - - std::cout << std::endl; - } - } - } - catch (const DB::Exception & e) - { - std::cerr << e.what() << ", " << e.displayText() << std::endl; - return 1; - } - - return 0; -} diff --git a/dbms/src/Functions/tests/logical_functions_performance.cpp b/dbms/src/Functions/tests/logical_functions_performance.cpp index cfe5cbe614f..c3cf1e257ec 100644 --- a/dbms/src/Functions/tests/logical_functions_performance.cpp +++ b/dbms/src/Functions/tests/logical_functions_performance.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include diff --git a/dbms/src/IO/ReadBufferFromPocoSocket.cpp b/dbms/src/IO/ReadBufferFromPocoSocket.cpp index bdd219d61cb..580747ff4a8 100644 --- a/dbms/src/IO/ReadBufferFromPocoSocket.cpp +++ b/dbms/src/IO/ReadBufferFromPocoSocket.cpp @@ -1,10 +1,10 @@ #include -#include +#include #include #include -#include +#include namespace DB diff --git a/dbms/src/IO/WriteBufferFromPocoSocket.cpp b/dbms/src/IO/WriteBufferFromPocoSocket.cpp index d4b07ff2cf9..4725d0661a1 100644 --- a/dbms/src/IO/WriteBufferFromPocoSocket.cpp +++ b/dbms/src/IO/WriteBufferFromPocoSocket.cpp @@ -1,10 +1,10 @@ #include -#include +#include #include #include -#include +#include namespace DB diff --git a/dbms/src/IO/tests/cached_compressed_read_buffer.cpp b/dbms/src/IO/tests/cached_compressed_read_buffer.cpp index 144dd8fde97..7ca27b2b154 100644 --- a/dbms/src/IO/tests/cached_compressed_read_buffer.cpp +++ b/dbms/src/IO/tests/cached_compressed_read_buffer.cpp @@ -6,7 +6,7 @@ #include #include -#include +#include int main(int argc, char ** argv) diff --git a/dbms/src/IO/tests/mempbrk.cpp b/dbms/src/IO/tests/mempbrk.cpp index 4c1461d05aa..eacca96ed4e 100644 --- a/dbms/src/IO/tests/mempbrk.cpp +++ b/dbms/src/IO/tests/mempbrk.cpp @@ -4,7 +4,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/src/IO/tests/parse_int_perf.cpp b/dbms/src/IO/tests/parse_int_perf.cpp index 2fe16777710..b99cd2f6033 100644 --- a/dbms/src/IO/tests/parse_int_perf.cpp +++ b/dbms/src/IO/tests/parse_int_perf.cpp @@ -14,7 +14,7 @@ #include #include -#include +#include #if 0 diff --git a/dbms/src/IO/tests/parse_int_perf2.cpp b/dbms/src/IO/tests/parse_int_perf2.cpp index 20855ba3c1a..3b17b400e34 100644 --- a/dbms/src/IO/tests/parse_int_perf2.cpp +++ b/dbms/src/IO/tests/parse_int_perf2.cpp @@ -6,7 +6,7 @@ #include #include -#include +#include std::ostream & operator<< (std::ostream & ostr, const __m128i vec) diff --git a/dbms/src/IO/tests/valid_utf8_perf.cpp b/dbms/src/IO/tests/valid_utf8_perf.cpp index 045613476ab..835d17b55a3 100644 --- a/dbms/src/IO/tests/valid_utf8_perf.cpp +++ b/dbms/src/IO/tests/valid_utf8_perf.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 56ea0ea896e..4b8e2dc4885 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -4,10 +4,11 @@ #include -#include +#include #include #include +#include #include #include @@ -323,16 +324,33 @@ AggregatedDataVariants::Type Aggregator::chooseAggregationMethod(const ConstColu bool all_fixed = true; size_t keys_bytes = 0; + + size_t num_array_keys = 0; + bool has_arrays_of_non_fixed_elems = false; + bool all_non_array_keys_are_fixed = true; + key_sizes.resize(keys_size); for (size_t j = 0; j < keys_size; ++j) { - if (!key_columns[j]->isFixed()) + if (key_columns[j]->isFixed()) + { + key_sizes[j] = key_columns[j]->sizeOfField(); + keys_bytes += key_sizes[j]; + } + else { all_fixed = false; - break; + + if (const ColumnArray * arr = typeid_cast(key_columns[j])) + { + ++num_array_keys; + + if (!arr->getData().isFixed()) + has_arrays_of_non_fixed_elems = true; + } + else + all_non_array_keys_are_fixed = false; } - key_sizes[j] = key_columns[j]->sizeOfField(); - keys_bytes += key_sizes[j]; } /// Если ключей нет @@ -367,6 +385,13 @@ AggregatedDataVariants::Type Aggregator::chooseAggregationMethod(const ConstColu if (keys_size == 1 && typeid_cast(key_columns[0])) return AggregatedDataVariants::Type::key_fixed_string; + /** Если есть массивы. + * Если есть не более одного массива из элементов фиксированной длины, и остальные ключи фиксированной длины, + * то всё ещё можно использовать метод concat. Иначе - serialized. + */ + if (num_array_keys > 1 || has_arrays_of_non_fixed_elems || (num_array_keys == 1 && !all_non_array_keys_are_fixed)) + return AggregatedDataVariants::Type::serialized; + /// Иначе будем агрегировать по конкатенации ключей. return AggregatedDataVariants::Type::concat; @@ -1387,6 +1412,8 @@ AggregatedDataVariantsPtr Aggregator::merge(ManyAggregatedDataVariants & data_va mergeSingleLevelDataImplhashed)::element_type>(non_empty_data); else if (res->type == AggregatedDataVariants::Type::concat) mergeSingleLevelDataImplconcat)::element_type>(non_empty_data); + else if (res->type == AggregatedDataVariants::Type::serialized) + mergeSingleLevelDataImplserialized)::element_type>(non_empty_data); else if (res->type == AggregatedDataVariants::Type::key32_two_level) mergeTwoLevelDataImplkey32_two_level)::element_type>(non_empty_data, thread_pool.get()); else if (res->type == AggregatedDataVariants::Type::key64_two_level) @@ -1403,6 +1430,8 @@ AggregatedDataVariantsPtr Aggregator::merge(ManyAggregatedDataVariants & data_va mergeTwoLevelDataImplhashed_two_level)::element_type>(non_empty_data, thread_pool.get()); else if (res->type == AggregatedDataVariants::Type::concat_two_level) mergeTwoLevelDataImplconcat_two_level)::element_type>(non_empty_data, thread_pool.get()); + else if (res->type == AggregatedDataVariants::Type::serialized_two_level) + mergeTwoLevelDataImplserialized_two_level)::element_type>(non_empty_data, thread_pool.get()); else if (res->type != AggregatedDataVariants::Type::without_key) throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); diff --git a/dbms/src/Interpreters/Compiler.cpp b/dbms/src/Interpreters/Compiler.cpp index 0ba17271c71..42a19112e5e 100644 --- a/dbms/src/Interpreters/Compiler.cpp +++ b/dbms/src/Interpreters/Compiler.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index c91be012c8b..d2b6aaed7cf 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -41,7 +41,7 @@ #include #include -#include +#include #include @@ -238,15 +238,31 @@ QuotaForIntervals & Context::getQuota() return *quota; } +void Context::checkDatabaseAccessRights(const std::string & database_name) const +{ + if (user.empty() || (database_name == "system")) + { + /// Безымянный пользователь, т.е. сервер, имеет доступ ко всем БД. + /// Все пользователи имеют доступ к БД system. + return; + } + if (!shared->users.isAllowedDatabase(user, database_name)) + throw Exception("Access denied to database " + database_name, ErrorCodes::DATABASE_ACCESS_DENIED); +} + void Context::addDependency(const DatabaseAndTableName & from, const DatabaseAndTableName & where) { Poco::ScopedLock lock(shared->mutex); + checkDatabaseAccessRights(from.first); + checkDatabaseAccessRights(where.first); shared->view_dependencies[from].insert(where); } void Context::removeDependency(const DatabaseAndTableName & from, const DatabaseAndTableName & where) { Poco::ScopedLock lock(shared->mutex); + checkDatabaseAccessRights(from.first); + checkDatabaseAccessRights(where.first); shared->view_dependencies[from].erase(where); } @@ -255,6 +271,7 @@ Dependencies Context::getDependencies(const String & database_name, const String Poco::ScopedLock lock(shared->mutex); String db = database_name.empty() ? current_database : database_name; + checkDatabaseAccessRights(db); ViewDependencies::const_iterator iter = shared->view_dependencies.find(DatabaseAndTableName(db, table_name)); if (iter == shared->view_dependencies.end()) @@ -268,6 +285,7 @@ bool Context::isTableExist(const String & database_name, const String & table_na Poco::ScopedLock lock(shared->mutex); String db = database_name.empty() ? current_database : database_name; + checkDatabaseAccessRights(db); Databases::const_iterator it; return shared->databases.end() != (it = shared->databases.find(db)) @@ -279,6 +297,7 @@ bool Context::isDatabaseExist(const String & database_name) const { Poco::ScopedLock lock(shared->mutex); String db = database_name.empty() ? current_database : database_name; + checkDatabaseAccessRights(db); return shared->databases.end() != shared->databases.find(db); } @@ -288,6 +307,7 @@ void Context::assertTableExists(const String & database_name, const String & tab Poco::ScopedLock lock(shared->mutex); String db = database_name.empty() ? current_database : database_name; + checkDatabaseAccessRights(db); Databases::const_iterator it = shared->databases.find(db); if (shared->databases.end() == it) @@ -298,11 +318,13 @@ void Context::assertTableExists(const String & database_name, const String & tab } -void Context::assertTableDoesntExist(const String & database_name, const String & table_name) const +void Context::assertTableDoesntExist(const String & database_name, const String & table_name, bool check_database_access_rights) const { Poco::ScopedLock lock(shared->mutex); String db = database_name.empty() ? current_database : database_name; + if (check_database_access_rights) + checkDatabaseAccessRights(db); Databases::const_iterator it; if (shared->databases.end() != (it = shared->databases.find(db)) @@ -311,11 +333,13 @@ void Context::assertTableDoesntExist(const String & database_name, const String } -void Context::assertDatabaseExists(const String & database_name) const +void Context::assertDatabaseExists(const String & database_name, bool check_database_access_rights) const { Poco::ScopedLock lock(shared->mutex); String db = database_name.empty() ? current_database : database_name; + if (check_database_access_rights) + checkDatabaseAccessRights(db); if (shared->databases.end() == shared->databases.find(db)) throw Exception("Database " + db + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE); @@ -327,6 +351,7 @@ void Context::assertDatabaseDoesntExist(const String & database_name) const Poco::ScopedLock lock(shared->mutex); String db = database_name.empty() ? current_database : database_name; + checkDatabaseAccessRights(db); if (shared->databases.end() != shared->databases.find(db)) throw Exception("Database " + db + " already exists.", ErrorCodes::DATABASE_ALREADY_EXISTS); @@ -409,6 +434,7 @@ StoragePtr Context::getTableImpl(const String & database_name, const String & ta } String db = database_name.empty() ? current_database : database_name; + checkDatabaseAccessRights(db); Databases::const_iterator it = shared->databases.find(db); if (shared->databases.end() == it) @@ -453,9 +479,10 @@ void Context::addTable(const String & database_name, const String & table_name, Poco::ScopedLock lock(shared->mutex); String db = database_name.empty() ? current_database : database_name; + checkDatabaseAccessRights(db); - assertDatabaseExists(db); - assertTableDoesntExist(db, table_name); + assertDatabaseExists(db, false); + assertTableDoesntExist(db, table_name, false); shared->databases[db][table_name] = table; } diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index caa612434ce..201ed096b4a 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -376,7 +375,9 @@ std::string ExpressionAction::toString() const switch (type) { case ADD_COLUMN: - ss << "ADD " << result_name << " " << result_type->getName() << " " << added_column->getName(); + ss << "ADD " << result_name << " " + << (result_type ? result_type->getName() : "(no type)") << " " + << (added_column ? added_column->getName() : "(no column)"); break; case REMOVE_COLUMN: @@ -388,7 +389,9 @@ std::string ExpressionAction::toString() const break; case APPLY_FUNCTION: - ss << "FUNCTION " << result_name << " " << result_type->getName() << " = " << function->getName() << "("; + ss << "FUNCTION " << result_name << " " + << (result_type ? result_type->getName() : "(no type)") << " = " + << (function ? function->getName() : "(no function)") << "("; for (size_t i = 0; i < argument_names.size(); ++i) { if (i) @@ -457,9 +460,9 @@ void ExpressionActions::checkLimits(Block & block) const std::stringstream list_of_non_const_columns; for (size_t i = 0, size = block.columns(); i < size; ++i) if (!block.getByPosition(i).column->isConst()) - list_of_non_const_columns << (i == 0 ? "" : ", ") << block.getByPosition(i).name; + list_of_non_const_columns << "\n" << block.getByPosition(i).name; - throw Exception("Too many temporary non-const columns: " + list_of_non_const_columns.str() + throw Exception("Too many temporary non-const columns:" + list_of_non_const_columns.str() + ". Maximum: " + toString(limits.max_temporary_non_const_columns), ErrorCodes::TOO_MUCH_TEMPORARY_NON_CONST_COLUMNS); } @@ -629,8 +632,6 @@ std::string ExpressionActions::getSmallestColumn(const NamesAndTypesList & colum void ExpressionActions::finalize(const Names & output_columns) { -// std::cerr << "finalize\n"; - NameSet final_columns; for (size_t i = 0; i < output_columns.size(); ++i) { @@ -757,13 +758,66 @@ void ExpressionActions::finalize(const Names & output_columns) } } - for (int i = static_cast(sample_block.columns()) - 1; i >= 0; --i) +/* std::cerr << "\n"; + for (const auto & action : actions) + std::cerr << action.toString() << "\n"; + std::cerr << "\n";*/ + + /// Удаление ненужных временных столбцов. + + /// Если у столбца после выполнения функции refcount = 0, то его можно удалить. + std::map columns_refcount; + + for (const auto & name : final_columns) + ++columns_refcount[name]; + + for (const auto & action : actions) { - const std::string & name = sample_block.getByPosition(i).name; - if (!final_columns.count(name)) - add(ExpressionAction::removeColumn(name)); + if (!action.source_name.empty()) + ++columns_refcount[action.source_name]; + + for (const auto & name : action.argument_names) + ++columns_refcount[name]; + + for (const auto & name : action.prerequisite_names) + ++columns_refcount[name]; } + Actions new_actions; + new_actions.reserve(actions.size()); + + for (const auto & action : actions) + { + new_actions.push_back(action); + + auto process = [&] (const String & name) + { + auto refcount = --columns_refcount[name]; + if (refcount <= 0) + { + new_actions.push_back(ExpressionAction::removeColumn(name)); + if (sample_block.has(name)) + sample_block.erase(name); + } + }; + + if (!action.source_name.empty()) + process(action.source_name); + + for (const auto & name : action.argument_names) + process(name); + + for (const auto & name : action.prerequisite_names) + process(name); + } + + actions.swap(new_actions); + +/* std::cerr << "\n"; + for (const auto & action : actions) + std::cerr << action.toString() << "\n"; + std::cerr << "\n";*/ + optimize(); checkLimits(sample_block); } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 310f56c2aef..0ded822f3b2 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -44,7 +44,7 @@ #include -#include +#include namespace DB diff --git a/dbms/src/Interpreters/ExternalDictionaries.cpp b/dbms/src/Interpreters/ExternalDictionaries.cpp index 9a97899ba34..79813706f65 100644 --- a/dbms/src/Interpreters/ExternalDictionaries.cpp +++ b/dbms/src/Interpreters/ExternalDictionaries.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.cpp b/dbms/src/Interpreters/InterpreterAlterQuery.cpp index 27275435ac8..1f8cbcfcc36 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/src/Interpreters/InterpreterAlterQuery.cpp @@ -43,11 +43,11 @@ BlockIO InterpreterAlterQuery::execute() switch (command.type) { case PartitionCommand::DROP_PARTITION: - table->dropPartition(command.partition, command.detach, command.unreplicated, context.getSettingsRef()); + table->dropPartition(query_ptr, command.partition, command.detach, command.unreplicated, context.getSettingsRef()); break; case PartitionCommand::ATTACH_PARTITION: - table->attachPartition(command.partition, command.unreplicated, command.part, context.getSettingsRef()); + table->attachPartition(query_ptr, command.partition, command.unreplicated, command.part, context.getSettingsRef()); break; case PartitionCommand::FETCH_PARTITION: diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 82a063166af..01aea41fc95 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -244,9 +244,7 @@ BlockIO InterpreterCreateQuery::executeImpl(bool assume_metadata_exists) } if (create.is_temporary) - { context.getSessionContext().addExternalTable(table_name, res); - } else context.addTable(database_name, table_name, res); } @@ -261,8 +259,13 @@ BlockIO InterpreterCreateQuery::executeImpl(bool assume_metadata_exists) new ProhibitColumnsBlockOutputStream{ new AddingDefaultBlockOutputStream{ new MaterializingBlockOutputStream{ - new PushingToViewsBlockOutputStream{create.database, create.table, context, query_ptr} + new PushingToViewsBlockOutputStream{ + create.database, create.table, + create.is_temporary ? context.getSessionContext() : context, + query_ptr + } }, + /// @note shouldn't these two contexts be session contexts in case of temporary table? columns, column_defaults, context, context.getSettingsRef().strict_insert_defaults }, materialized_columns diff --git a/dbms/src/Interpreters/InterpreterShowTablesQuery.cpp b/dbms/src/Interpreters/InterpreterShowTablesQuery.cpp index c9652c73e7c..6131b9fd598 100644 --- a/dbms/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/dbms/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -32,7 +32,12 @@ String InterpreterShowTablesQuery::getRewrittenQuery() return "SELECT name FROM system.databases" + format_or_nothing; String database = query.from.empty() ? context.getCurrentDatabase() : query.from; - context.assertDatabaseExists(database); + + /** Параметр check_database_access_rights сбрасывается при обработке запроса SHOW TABLES для того, + * чтобы все клиенты могли видеть список всех БД и таблиц в них независимо от их прав доступа + * к этим БД. + */ + context.assertDatabaseExists(database, false); std::stringstream rewritten_query; rewritten_query << "SELECT name FROM system.tables WHERE database = " << mysqlxx::quote << database; diff --git a/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp b/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp index c777a5a0c7b..9da4c762e4f 100644 --- a/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp @@ -197,7 +197,8 @@ void LogicalExpressionsOptimizer::addInExpression(const DisjunctiveEqualityChain value_list->children.push_back(operands[1]); } - /// Отсортировать литералы. + /// Отсортировать литералы, чтобы они были указаны в одном и том же порядке в выражении IN. + /// Иначе они указывались бы в порядке адресов ASTLiteral, который недетерминирован. std::sort(value_list->children.begin(), value_list->children.end(), [](const DB::ASTPtr & lhs, const DB::ASTPtr & rhs) { const auto val_lhs = static_cast(&*lhs); diff --git a/dbms/src/Interpreters/QueryLog.cpp b/dbms/src/Interpreters/QueryLog.cpp index d53cd4d29ee..e491e5138ae 100644 --- a/dbms/src/Interpreters/QueryLog.cpp +++ b/dbms/src/Interpreters/QueryLog.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/dbms/src/Interpreters/Quota.cpp b/dbms/src/Interpreters/Quota.cpp index 8ede5a3b449..fe4dde914e0 100644 --- a/dbms/src/Interpreters/Quota.cpp +++ b/dbms/src/Interpreters/Quota.cpp @@ -293,7 +293,7 @@ void Quotas::loadFromConfig(Poco::Util::AbstractConfiguration & config) for (Poco::Util::AbstractConfiguration::Keys::const_iterator it = config_keys.begin(); it != config_keys.end(); ++it) { if (!cont[*it]) - cont[*it] = new Quota(); + cont[*it].reset(new Quota()); cont[*it]->loadFromConfig("quotas." + *it, *it, config, rng); } } diff --git a/dbms/src/Interpreters/Set.cpp b/dbms/src/Interpreters/Set.cpp index 44fdcc82a1c..8d74f222b45 100644 --- a/dbms/src/Interpreters/Set.cpp +++ b/dbms/src/Interpreters/Set.cpp @@ -478,12 +478,17 @@ void Set::createFromAST(DataTypes & types, ASTPtr node, const Context & context, } -void Set::execute(Block & block, const ColumnNumbers & arguments, size_t result, bool negative) const +ColumnPtr Set::execute(const Block & block, bool negative) const { - ColumnUInt8 * c_res = new ColumnUInt8; - block.getByPosition(result).column = c_res; - ColumnUInt8::Container_t & vec_res = c_res->getData(); - vec_res.resize(block.getByPosition(arguments[0]).column->size()); + size_t num_key_columns = block.columns(); + + if (0 == num_key_columns) + throw Exception("Logical error: no columns passed to Set::execute method.", ErrorCodes::LOGICAL_ERROR); + + ColumnUInt8 * p_res = new ColumnUInt8; + ColumnPtr res = p_res; + ColumnUInt8::Container_t & vec_res = p_res->getData(); + vec_res.resize(block.getByPosition(0).column->size()); Poco::ScopedReadRWLock lock(rwlock); @@ -494,19 +499,19 @@ void Set::execute(Block & block, const ColumnNumbers & arguments, size_t result, memset(&vec_res[0], 1, vec_res.size()); else memset(&vec_res[0], 0, vec_res.size()); - return; + return res; } - DataTypeArray * array_type = typeid_cast(&*block.getByPosition(arguments[0]).type); + const DataTypeArray * array_type = typeid_cast(&*block.getByPosition(0).type); if (array_type) { - if (data_types.size() != 1 || arguments.size() != 1) + if (data_types.size() != 1 || num_key_columns != 1) throw Exception("Number of columns in section IN doesn't match.", ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH); if (array_type->getNestedType()->getName() != data_types[0]->getName()) throw Exception(std::string() + "Types in section IN don't match: " + data_types[0]->getName() + " on the right, " + array_type->getNestedType()->getName() + " on the left.", ErrorCodes::TYPE_MISMATCH); - IColumn * in_column = &*block.getByPosition(arguments[0]).column; + const IColumn * in_column = &*block.getByPosition(0).column; /// Константный столбец слева от IN поддерживается не напрямую. Для этого, он сначала материализуется. ColumnPtr materialized_column; @@ -516,24 +521,26 @@ void Set::execute(Block & block, const ColumnNumbers & arguments, size_t result, in_column = materialized_column.get(); } - if (ColumnArray * col = typeid_cast(in_column)) + if (const ColumnArray * col = typeid_cast(in_column)) executeArray(col, vec_res, negative); else throw Exception("Unexpected array column type: " + in_column->getName(), ErrorCodes::ILLEGAL_COLUMN); } else { - if (data_types.size() != arguments.size()) + if (data_types.size() != num_key_columns) throw Exception("Number of columns in section IN doesn't match.", ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH); /// Запоминаем столбцы, с которыми будем работать. Также проверим, что типы данных правильные. - ConstColumnPlainPtrs key_columns(arguments.size()); - for (size_t i = 0; i < arguments.size(); ++i) + ConstColumnPlainPtrs key_columns(num_key_columns); + for (size_t i = 0; i < num_key_columns; ++i) { - key_columns[i] = block.getByPosition(arguments[i]).column; + key_columns[i] = block.getByPosition(i).column; - if (data_types[i]->getName() != block.getByPosition(arguments[i]).type->getName()) - throw Exception("Types of column " + toString(i + 1) + " in section IN don't match: " + data_types[i]->getName() + " on the right, " + block.getByPosition(arguments[i]).type->getName() + " on the left.", ErrorCodes::TYPE_MISMATCH); + if (data_types[i]->getName() != block.getByPosition(i).type->getName()) + throw Exception("Types of column " + toString(i + 1) + " in section IN don't match: " + + data_types[i]->getName() + " on the right, " + block.getByPosition(i).type->getName() + " on the left.", + ErrorCodes::TYPE_MISMATCH); } /// Константные столбцы слева от IN поддерживается не напрямую. Для этого, они сначала материализуется. @@ -549,6 +556,8 @@ void Set::execute(Block & block, const ColumnNumbers & arguments, size_t result, executeOrdinary(key_columns, vec_res, negative); } + + return res; } diff --git a/dbms/src/Interpreters/loadMetadata.cpp b/dbms/src/Interpreters/loadMetadata.cpp index 4ffaf637249..4311b917089 100644 --- a/dbms/src/Interpreters/loadMetadata.cpp +++ b/dbms/src/Interpreters/loadMetadata.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include @@ -19,7 +19,7 @@ #include #include -#include +#include namespace DB diff --git a/dbms/src/Interpreters/tests/hash_map.cpp b/dbms/src/Interpreters/tests/hash_map.cpp index 622913fd2b5..c5227762db7 100644 --- a/dbms/src/Interpreters/tests/hash_map.cpp +++ b/dbms/src/Interpreters/tests/hash_map.cpp @@ -7,7 +7,7 @@ #include #include -#include +#include /* #define DBMS_HASH_MAP_COUNT_COLLISIONS */ diff --git a/dbms/src/Interpreters/tests/hash_map2.cpp b/dbms/src/Interpreters/tests/hash_map2.cpp index 53b0d5ddc59..e6ec6717c8a 100644 --- a/dbms/src/Interpreters/tests/hash_map2.cpp +++ b/dbms/src/Interpreters/tests/hash_map2.cpp @@ -7,7 +7,7 @@ #include #include -#include +#include //#define DBMS_HASH_MAP_COUNT_COLLISIONS #define DBMS_HASH_MAP_DEBUG_RESIZES diff --git a/dbms/src/Interpreters/tests/hash_map3.cpp b/dbms/src/Interpreters/tests/hash_map3.cpp index 44fc9040cf1..202f2bc1174 100644 --- a/dbms/src/Interpreters/tests/hash_map3.cpp +++ b/dbms/src/Interpreters/tests/hash_map3.cpp @@ -11,7 +11,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/src/Interpreters/tests/hash_map_string.cpp b/dbms/src/Interpreters/tests/hash_map_string.cpp index 7d30e4f331e..f977909fdf9 100644 --- a/dbms/src/Interpreters/tests/hash_map_string.cpp +++ b/dbms/src/Interpreters/tests/hash_map_string.cpp @@ -7,7 +7,7 @@ #include #include -#include +#include //#define DBMS_HASH_MAP_COUNT_COLLISIONS #define DBMS_HASH_MAP_DEBUG_RESIZES diff --git a/dbms/src/Interpreters/tests/hash_map_string_2.cpp b/dbms/src/Interpreters/tests/hash_map_string_2.cpp index ec04789d948..ce195bc7259 100644 --- a/dbms/src/Interpreters/tests/hash_map_string_2.cpp +++ b/dbms/src/Interpreters/tests/hash_map_string_2.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include //#define DBMS_HASH_MAP_COUNT_COLLISIONS #define DBMS_HASH_MAP_DEBUG_RESIZES diff --git a/dbms/src/Interpreters/tests/hash_map_string_3.cpp b/dbms/src/Interpreters/tests/hash_map_string_3.cpp index 1e1a52509d1..ccc7b9f9b33 100644 --- a/dbms/src/Interpreters/tests/hash_map_string_3.cpp +++ b/dbms/src/Interpreters/tests/hash_map_string_3.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/src/Interpreters/tests/hash_map_string_small.cpp b/dbms/src/Interpreters/tests/hash_map_string_small.cpp index d7a70a26f35..d750430c570 100644 --- a/dbms/src/Interpreters/tests/hash_map_string_small.cpp +++ b/dbms/src/Interpreters/tests/hash_map_string_small.cpp @@ -7,7 +7,7 @@ #include #include -#include +#include //#define DBMS_HASH_MAP_COUNT_COLLISIONS #define DBMS_HASH_MAP_DEBUG_RESIZES diff --git a/dbms/src/Interpreters/tests/two_level_hash_map.cpp b/dbms/src/Interpreters/tests/two_level_hash_map.cpp index 26df5c372d7..b746794b35b 100644 --- a/dbms/src/Interpreters/tests/two_level_hash_map.cpp +++ b/dbms/src/Interpreters/tests/two_level_hash_map.cpp @@ -9,7 +9,7 @@ //#define DBMS_HASH_MAP_DEBUG_RESIZES -#include +#include #include #include diff --git a/dbms/src/Interpreters/tests/users.cpp b/dbms/src/Interpreters/tests/users.cpp new file mode 100644 index 00000000000..240f6b5b000 --- /dev/null +++ b/dbms/src/Interpreters/tests/users.cpp @@ -0,0 +1,287 @@ +#include +#include + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace +{ + +namespace fs = boost::filesystem; + +struct TestEntry +{ + std::string user_name; + std::string database_name; + bool is_allowed; +}; + +using TestEntries = std::vector; + +struct TestDescriptor +{ + const char * config_content; + TestEntries entries; +}; + +using TestSet = std::vector; + +/// Описание тестов. + +TestSet test_set = +{ + { + "" + " " + " " + " " + " defaultdefault" + " " + " default" + " test" + " " + " " + " " + " defaultdefault" + " " + " " + " " + "", + + { + { "default", "default", true }, + { "default", "test", true }, + { "default", "stats", false }, + { "web", "default", true }, + { "web", "test", true }, + { "web", "stats", true }, + { "analytics", "default", false }, + { "analytics", "test", false }, + { "analytics", "stats", false } + } + }, + + { + "" + " " + " " + " " + " defaultdefault" + " " + " default" + " " + " " + " " + " defaultdefault" + " " + " " + " " + "", + + { + { "default", "default", true }, + { "default", "test", false }, + { "default", "stats", false }, + { "web", "default", true }, + { "web", "test", true }, + { "web", "stats", true }, + { "analytics", "default", false }, + { "analytics", "test", false }, + { "analytics", "stats", false } + } + }, + + { + "" + " " + " " + " " + " defaultdefault" + " " + " " + " " + " " + " defaultdefault" + " " + " " + " " + "", + + { + { "default", "default", true }, + { "default", "test", true }, + { "default", "stats", true }, + { "web", "default", true }, + { "web", "test", true }, + { "web", "stats", true }, + { "analytics", "default", false }, + { "analytics", "test", false }, + { "analytics", "stats", false } + } + }, + + { + "" + " " + " " + " " + " defaultdefault" + " " + " default" + " " + " " + " " + " defaultdefault" + " " + " test" + " " + " " + " " + " " + "", + + { + { "default", "default", true }, + { "default", "test", false }, + { "default", "stats", false }, + { "web", "default", false }, + { "web", "test", true }, + { "web", "stats", false }, + { "analytics", "default", false }, + { "analytics", "test", false }, + { "analytics", "stats", false } + } + } +}; + +std::string createTmpPath(const std::string & filename); +void createFile(const std::string & filename, const char * data); +void runOneTest(size_t test_num, const TestDescriptor & test_descriptor); +auto runTestSet(const TestSet & test_set); + +std::string createTmpPath(const std::string & filename) +{ + char pattern[] = "/tmp/fileXXXXXX"; + char * dir = mkdtemp(pattern); + if (dir == nullptr) + throw std::runtime_error("Could not create directory"); + + return std::string(dir) + "/" + filename; +} + +void createFile(const std::string & filename, const char * data) +{ + std::ofstream ofs(filename.c_str()); + if (!ofs.is_open()) + throw std::runtime_error("Could not open file " + filename); + ofs << data; +} + +void runOneTest(size_t test_num, const TestDescriptor & test_descriptor) +{ + const auto path_name = createTmpPath("users.xml"); + createFile(path_name, test_descriptor.config_content); + + ConfigurationPtr config; + + try + { + config = ConfigProcessor{}.loadConfig(path_name); + } + catch (const Poco::Exception & ex) + { + std::ostringstream os; + os << "Error: " << ex.what() << ": " << ex.displayText(); + throw std::runtime_error(os.str()); + } + + DB::Users users; + + try + { + users.loadFromConfig(*config); + } + catch (const Poco::Exception & ex) + { + std::ostringstream os; + os << "Error: " << ex.what() << ": " << ex.displayText(); + throw std::runtime_error(os.str()); + } + + for (const auto & entry : test_descriptor.entries) + { + bool res; + + try + { + res = users.isAllowedDatabase(entry.user_name, entry.database_name); + } + catch (const Poco::Exception &) + { + res = false; + } + + if (res != entry.is_allowed) + { + auto to_string = [](bool access){ return (access ? "'granted'" : "'denied'"); }; + std::ostringstream os; + os << "(user=" << entry.user_name << ", database=" << entry.database_name << "): "; + os << "Expected " << to_string(entry.is_allowed) << " but got " << to_string(res); + throw std::runtime_error(os.str()); + } + } + + fs::remove_all(fs::path(path_name).parent_path().string()); +} + +auto runTestSet(const TestSet & test_set) +{ + size_t test_num = 1; + size_t failure_count = 0; + + for (const auto & test_descriptor : test_set) + { + try + { + runOneTest(test_num, test_descriptor); + std::cout << "Test " << test_num << " passed\n"; + } + catch (const std::runtime_error & ex) + { + std::cerr << "Test " << test_num << " failed with reason: " << ex.what() << "\n"; + ++failure_count; + } + catch (...) + { + std::cerr << "Test " << test_num << " failed with unknown reason\n"; + ++failure_count; + } + + ++test_num; + } + + return std::make_tuple(test_set.size(), failure_count); +} + +} + +int main() +{ + size_t test_count; + size_t failure_count; + + std::tie(test_count, failure_count) = runTestSet(test_set); + + std::cout << (test_count - failure_count) << " test(s) passed out of " << test_count << "\n"; + + return (failure_count == 0) ? 0 : EXIT_FAILURE; +} diff --git a/dbms/src/Server/HTTPHandler.cpp b/dbms/src/Server/HTTPHandler.cpp index 7649e3e01ea..c091651144a 100644 --- a/dbms/src/Server/HTTPHandler.cpp +++ b/dbms/src/Server/HTTPHandler.cpp @@ -2,7 +2,7 @@ #include -#include +#include #include diff --git a/dbms/src/Server/OLAPAttributesMetadata.h b/dbms/src/Server/OLAPAttributesMetadata.h index 17463589dfa..d5ac92a7a15 100644 --- a/dbms/src/Server/OLAPAttributesMetadata.h +++ b/dbms/src/Server/OLAPAttributesMetadata.h @@ -1,16 +1,17 @@ #pragma once #include // log2() +#include #include #include #include +#include #include #include -#include #include #include #include @@ -187,7 +188,18 @@ struct AttributeHashBase : public IAttributeMetadata { BinaryData parse(const std::string & s) const { - return strconvert::hash64(s); + union + { + unsigned char char_data[16]; + Poco::UInt64 uint64_data; + } buf; + + MD5_CTX ctx; + MD5_Init(&ctx); + MD5_Update(&ctx, reinterpret_cast(s.data()), s.size()); + MD5_Final(buf.char_data, &ctx); + + return Poco::ByteOrder::flipBytes(buf.uint64_data); } }; diff --git a/dbms/src/Server/OLAPHTTPHandler.cpp b/dbms/src/Server/OLAPHTTPHandler.cpp index 88fc7210e9c..ca212941208 100644 --- a/dbms/src/Server/OLAPHTTPHandler.cpp +++ b/dbms/src/Server/OLAPHTTPHandler.cpp @@ -12,7 +12,7 @@ #include "OLAPHTTPHandler.h" -#include +#include #include @@ -51,6 +51,7 @@ void OLAPHTTPHandler::processQuery(Poco::Net::HTTPServerRequest & request, Poco: context.setGlobalContext(*server.global_context); context.setUser(user, password, request.clientAddress().host(), quota_key); + context.setSetting("profile", profile); context.setInterface(Context::Interface::OLAP_HTTP); context.setHTTPMethod(Context::HTTPMethod::POST); diff --git a/dbms/src/Server/OLAPHTTPHandler.h b/dbms/src/Server/OLAPHTTPHandler.h index c9d61faa152..7124c67eb1c 100644 --- a/dbms/src/Server/OLAPHTTPHandler.h +++ b/dbms/src/Server/OLAPHTTPHandler.h @@ -11,8 +11,9 @@ class OLAPHTTPHandler : public Poco::Net::HTTPRequestHandler { public: OLAPHTTPHandler(Server & server_) - : server(server_) - , log(&Logger::get("OLAPHTTPHandler")) + : server(server_), + log(&Logger::get("OLAPHTTPHandler")), + profile(Poco::Util::Application::instance().config().getString("olap_compatibility.profile")) { } @@ -20,8 +21,8 @@ public: private: Server & server; - Logger * log; + const String profile; void processQuery(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response); }; diff --git a/dbms/src/Server/OLAPQueryConverter.cpp b/dbms/src/Server/OLAPQueryConverter.cpp index 9026f19199e..b2f7c4e1a95 100644 --- a/dbms/src/Server/OLAPQueryConverter.cpp +++ b/dbms/src/Server/OLAPQueryConverter.cpp @@ -10,8 +10,8 @@ namespace OLAP QueryConverter::QueryConverter(Poco::Util::AbstractConfiguration & config) { - table_for_single_counter = config.getString("olap_table_for_single_counter"); - table_for_all_counters = config.getString("olap_table_for_all_counters"); + table_for_single_counter = config.getString("olap_compatibility.table_for_single_counter"); + table_for_all_counters = config.getString("olap_compatibility.table_for_all_counters"); fillFormattedAttributeMap(); fillNumericAttributeMap(); diff --git a/dbms/src/Server/OLAPQueryParser.cpp b/dbms/src/Server/OLAPQueryParser.cpp index b98f7d92e9b..c15ddf622b5 100644 --- a/dbms/src/Server/OLAPQueryParser.cpp +++ b/dbms/src/Server/OLAPQueryParser.cpp @@ -4,7 +4,7 @@ #include "OLAPQueryParser.h" #include -#include +#include #include #include diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 4c04f24b030..b0fe342442c 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -9,8 +9,8 @@ #include #include -#include -#include +#include +#include #include #include @@ -613,7 +613,6 @@ int Server::main(const std::vector & args) const std::string listen_host = config().getString("listen_host", "::"); - bool use_olap_server = config().getBool("use_olap_http_server", false); Poco::Timespan keep_alive_timeout(config().getInt("keep_alive_timeout", 10), 0); Poco::ThreadPool server_pool(3, config().getInt("max_connections", 1024)); @@ -657,12 +656,13 @@ int Server::main(const std::vector & args) /// OLAP HTTP Poco::SharedPtr olap_http_server; + bool use_olap_server = config().has("olap_compatibility.port"); if (use_olap_server) { olap_parser.reset(new OLAP::QueryParser()); olap_converter.reset(new OLAP::QueryConverter(config())); - Poco::Net::ServerSocket olap_http_socket(Poco::Net::SocketAddress(listen_host, config().getInt("olap_http_port"))); + Poco::Net::ServerSocket olap_http_socket(Poco::Net::SocketAddress(listen_host, config().getInt("olap_compatibility.port"))); olap_http_socket.setReceiveTimeout(settings.receive_timeout); olap_http_socket.setSendTimeout(settings.send_timeout); olap_http_server = new Poco::Net::HTTPServer( diff --git a/dbms/src/Server/Server.h b/dbms/src/Server/Server.h index e3f017d726c..45e8b99a88e 100644 --- a/dbms/src/Server/Server.h +++ b/dbms/src/Server/Server.h @@ -19,7 +19,7 @@ #include #include -#include +#include #include #include "OLAPQueryParser.h" @@ -44,7 +44,7 @@ class Server : public Daemon public: /// Глобальные настройки севрера std::unique_ptr global_context; - + std::unique_ptr olap_parser; std::unique_ptr olap_converter; @@ -54,7 +54,7 @@ protected: Daemon::initialize(self); logger().information("starting up"); } - + void uninitialize() { logger().information("shutting down"); diff --git a/dbms/src/Server/TCPHandler.cpp b/dbms/src/Server/TCPHandler.cpp index 2fdf88d93da..aad26abee9d 100644 --- a/dbms/src/Server/TCPHandler.cpp +++ b/dbms/src/Server/TCPHandler.cpp @@ -4,7 +4,7 @@ #include -#include +#include #include #include @@ -28,7 +28,7 @@ #include "TCPHandler.h" -#include +#include namespace DB { diff --git a/dbms/src/Server/TCPHandler.h b/dbms/src/Server/TCPHandler.h index 1b45cf72191..4ae4fe51efa 100644 --- a/dbms/src/Server/TCPHandler.h +++ b/dbms/src/Server/TCPHandler.h @@ -8,7 +8,7 @@ #include -#include +#include #include "Server.h" diff --git a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp index 8a9ea98949d..f82a694b811 100644 --- a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp +++ b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 2b744c9ef0c..022588e2e8c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -17,7 +17,7 @@ #include #include #include -#include +#include #include #include diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 87e0952c6b9..081179c4ba4 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include #include diff --git a/dbms/src/Storages/MergeTree/MergeTreePartChecker.cpp b/dbms/src/Storages/MergeTree/MergeTreePartChecker.cpp index c8e8a84f3fe..c41d67898df 100644 --- a/dbms/src/Storages/MergeTree/MergeTreePartChecker.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreePartChecker.cpp @@ -307,7 +307,7 @@ void MergeTreePartChecker::checkDataPart( String any_column_name; size_t rows = Stream::UNKNOWN; - ExceptionPtr first_exception; + std::exception_ptr first_exception; for (const NameAndTypePair & column : columns) { @@ -348,15 +348,12 @@ void MergeTreePartChecker::checkDataPart( { if (!settings.verbose) throw; - ExceptionPtr e = cloneCurrentException(); + + std::exception_ptr e = std::current_exception(); if (!first_exception) first_exception = e; - std::cerr << " exception" << std::endl; - std::cerr << "Code: " << e->code() << ", e.displayText() = " << e->displayText() << ", e.what() = " << e->what() << std::endl; - if (auto dbe = dynamic_cast(&*e)) - std::cerr << "Stack trace:\n\n" << dbe->getStackTrace().toString() << std::endl; - std::cerr << std::endl; + std::cerr << getCurrentExceptionMessage(true) << std::endl; } if (settings.verbose && ok) @@ -382,7 +379,7 @@ void MergeTreePartChecker::checkDataPart( checksums_txt.checkEqual(checksums_data, true); if (first_exception) - first_exception->rethrow(); + std::rethrow_exception(first_exception); if (out_checksums) *out_checksums = checksums_data; diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 52f497014c0..183139a5864 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include @@ -224,14 +225,13 @@ void ReplicatedMergeTreeRestartingThread::activateReplica() auto host_port = storage.context.getInterserverIOAddress(); auto zookeeper = storage.getZooKeeper(); - std::string address; - { - WriteBufferFromString address_buf(address); - address_buf - << "host: " << host_port.first << '\n' - << "port: " << host_port.second << '\n' - << "tcp_port: " << storage.context.getTCPPort() << '\n'; - } + /// Как другие реплики могут обращаться к данной. + ReplicatedMergeTreeAddress address; + address.host = host_port.first; + address.replication_port = host_port.second; + address.queries_port = storage.context.getTCPPort(); + address.database = storage.database_name; + address.table = storage.table_name; String is_active_path = storage.replica_path + "/is_active"; @@ -258,7 +258,7 @@ void ReplicatedMergeTreeRestartingThread::activateReplica() zkutil::Ops ops; ops.push_back(new zkutil::Op::Create(is_active_path, active_node_identifier, zookeeper->getDefaultACL(), zkutil::CreateMode::Ephemeral)); - ops.push_back(new zkutil::Op::SetData(storage.replica_path + "/host", address, -1)); + ops.push_back(new zkutil::Op::SetData(storage.replica_path + "/host", address.toString(), -1)); try { diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 3801c8b5ef1..6081b50cf09 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -7,7 +7,7 @@ #include #include -#include +#include namespace DB diff --git a/dbms/src/Storages/StorageChunkMerger.cpp b/dbms/src/Storages/StorageChunkMerger.cpp index 78d88eb8fb9..2ed96263139 100644 --- a/dbms/src/Storages/StorageChunkMerger.cpp +++ b/dbms/src/Storages/StorageChunkMerger.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -200,7 +201,7 @@ BlockInputStreams StorageChunkMerger::read( processed_stage = tmp_processed_stage; } - return res; + return narrowBlockInputStreams(res, threads); } /// Построить блок состоящий только из возможных значений виртуальных столбцов diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index 4ef5aac3971..03561a46268 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -2,7 +2,7 @@ #include -#include +#include #include #include @@ -19,7 +19,6 @@ #include #include -#include #include diff --git a/dbms/src/Storages/StorageMemory.cpp b/dbms/src/Storages/StorageMemory.cpp index 5c602c9231c..0d22615f67f 100644 --- a/dbms/src/Storages/StorageMemory.cpp +++ b/dbms/src/Storages/StorageMemory.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 48f46d98335..3dd1526264c 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -179,7 +180,7 @@ BlockInputStreams StorageMerge::read( processed_stage = tmp_processed_stage; } - return res; + return narrowBlockInputStreams(res, threads); } /// Построить блок состоящий только из возможных значений виртуальных столбцов diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index c4cc5712f98..09aa2a03b16 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -280,7 +280,7 @@ bool StorageMergeTree::canMergeParts(const MergeTreeData::DataPartPtr & left, co } -void StorageMergeTree::dropPartition(const Field & partition, bool detach, bool unreplicated, const Settings & settings) +void StorageMergeTree::dropPartition(ASTPtr query, const Field & partition, bool detach, bool unreplicated, const Settings & settings) { if (unreplicated) throw Exception("UNREPLICATED option for DROP has meaning only for ReplicatedMergeTree", ErrorCodes::BAD_ARGUMENTS); @@ -314,7 +314,7 @@ void StorageMergeTree::dropPartition(const Field & partition, bool detach, bool } -void StorageMergeTree::attachPartition(const Field & field, bool unreplicated, bool part, const Settings & settings) +void StorageMergeTree::attachPartition(ASTPtr query, const Field & field, bool unreplicated, bool part, const Settings & settings) { if (unreplicated) throw Exception("UNREPLICATED option for ATTACH has meaning only for ReplicatedMergeTree", ErrorCodes::BAD_ARGUMENTS); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index a852cde7274..fcc51f7bd6c 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -15,6 +16,9 @@ #include #include #include +#include +#include +#include #include #include #include @@ -1454,7 +1458,7 @@ bool StorageReplicatedMergeTree::queueTask(BackgroundProcessingPool::Context & p bool was_exception = true; bool success = false; - ExceptionPtr saved_exception; + std::exception_ptr saved_exception; try { @@ -1474,7 +1478,7 @@ bool StorageReplicatedMergeTree::queueTask(BackgroundProcessingPool::Context & p } catch (...) { - saved_exception = cloneCurrentException(); + saved_exception = std::current_exception(); throw; } @@ -2415,15 +2419,9 @@ void StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin if (!to_detached) table_lock = lockStructure(true); - String host; - int port; + ReplicatedMergeTreeAddress address(zookeeper->get(replica_path + "/host")); - String host_port_str = zookeeper->get(replica_path + "/host"); - ReadBufferFromString buf(host_port_str); - buf >> "host: " >> host >> "\n" - >> "port: " >> port >> "\n"; - - MergeTreeData::MutableDataPartPtr part = fetcher.fetchPart(part_name, replica_path, host, port, to_detached); + MergeTreeData::MutableDataPartPtr part = fetcher.fetchPart(part_name, replica_path, address.host, address.replication_port, to_detached); if (!to_detached) { @@ -2809,21 +2807,52 @@ void StorageReplicatedMergeTree::dropUnreplicatedPartition(const Field & partiti } -void StorageReplicatedMergeTree::dropPartition(const Field & field, bool detach, bool unreplicated, const Settings & settings) +void StorageReplicatedMergeTree::dropPartition(ASTPtr query, const Field & field, bool detach, bool unreplicated, const Settings & settings) { if (unreplicated) { dropUnreplicatedPartition(field, detach, settings); - return; } auto zookeeper = getZooKeeper(); String month_name = MergeTreeData::getMonthName(field); - /// TODO: Делать запрос в лидера по TCP. if (!is_leader_node) - throw Exception(String(detach ? "DETACH" : "DROP") + " PARTITION can only be done on leader replica.", ErrorCodes::NOT_LEADER); + { + /// Проксируем запрос в лидера. + + auto live_replicas = zookeeper->getChildren(zookeeper_path + "/leader_election"); + if (live_replicas.empty()) + throw Exception("No active replicas", ErrorCodes::NO_ACTIVE_REPLICAS); + + const auto leader = zookeeper->get(zookeeper_path + "/leader_election/" + live_replicas.front()); + + if (leader == replica_name) + throw Exception("Leader was suddenly changed or logical error.", ErrorCodes::LEADERSHIP_CHANGED); + + ReplicatedMergeTreeAddress leader_address(zookeeper->get(zookeeper_path + "/replicas/" + leader + "/host")); + + auto new_query = query->clone(); + auto & alter = typeid_cast(*new_query); + + alter.database = leader_address.database; + alter.table = leader_address.table; + + /// NOTE Работает только если есть доступ от пользователя default без пароля. Можно исправить с помощью добавления параметра в конфиг сервера. + + Connection connection( + leader_address.host, + leader_address.queries_port, + leader_address.database, + "", "", "ClickHouse replica"); + + RemoteBlockInputStream stream(connection, formattedAST(new_query), &settings); + NullBlockOutputStream output; + + copyData(stream, output); + return; + } /** Пропустим один номер в block_numbers для удаляемого месяца, и будем удалять только куски до этого номера. * Это запретит мерджи удаляемых кусков с новыми вставляемыми данными. @@ -2876,7 +2905,7 @@ void StorageReplicatedMergeTree::dropPartition(const Field & field, bool detach, } -void StorageReplicatedMergeTree::attachPartition(const Field & field, bool unreplicated, bool attach_part, const Settings & settings) +void StorageReplicatedMergeTree::attachPartition(ASTPtr query, const Field & field, bool unreplicated, bool attach_part, const Settings & settings) { auto zookeeper = getZooKeeper(); String partition; diff --git a/dbms/src/Storages/StorageStripeLog.cpp b/dbms/src/Storages/StorageStripeLog.cpp index ecf966e9d01..074a5d611d6 100644 --- a/dbms/src/Storages/StorageStripeLog.cpp +++ b/dbms/src/Storages/StorageStripeLog.cpp @@ -5,7 +5,7 @@ #include -#include +#include #include #include @@ -24,7 +24,6 @@ #include #include -#include #include #include @@ -40,13 +39,13 @@ class StripeLogBlockInputStream : public IProfilingBlockInputStream { public: StripeLogBlockInputStream(const NameSet & column_names_, StorageStripeLog & storage_, size_t max_read_buffer_size_, - const Poco::SharedPtr & index_, + std::shared_ptr & index_, IndexForNativeFormat::Blocks::const_iterator index_begin_, IndexForNativeFormat::Blocks::const_iterator index_end_) : column_names(column_names_.begin(), column_names_.end()), storage(storage_), index(index_), index_begin(index_begin_), index_end(index_end_), - data_in(storage.full_path() + "data.bin", 0, 0, max_read_buffer_size_), - block_in(data_in, 0, true, index_begin, index_end) + data_in(std::make_unique(storage.full_path() + "data.bin", 0, 0, max_read_buffer_size_)), + block_in(std::make_unique(*data_in, 0, true, index_begin, index_end)) { } @@ -64,19 +63,37 @@ public: protected: Block readImpl() override { - return block_in.read(); + Block res; + + if (block_in) + { + res = block_in->read(); + + /// Освобождаем память раньше уничтожения объекта. + if (!res) + { + block_in.reset(); + data_in.reset(); + index.reset(); + } + } + + return res; } private: NameSet column_names; StorageStripeLog & storage; - const Poco::SharedPtr index; + std::shared_ptr index; IndexForNativeFormat::Blocks::const_iterator index_begin; IndexForNativeFormat::Blocks::const_iterator index_end; - CompressedReadBufferFromFile data_in; - NativeBlockInputStream block_in; + /** unique_ptr - чтобы удалять объекты (освобождать буферы) после исчерпания источника + * - для экономии оперативки при использовании большого количества источников. + */ + std::unique_ptr data_in; + std::unique_ptr block_in; }; @@ -216,7 +233,7 @@ BlockInputStreams StorageStripeLog::read( NameSet column_names_set(column_names.begin(), column_names.end()); CompressedReadBufferFromFile index_in(full_path() + "index.mrk", 0, 0, INDEX_BUFFER_SIZE); - Poco::SharedPtr index = new IndexForNativeFormat(index_in, column_names_set); + std::shared_ptr index{std::make_shared(index_in, column_names_set)}; BlockInputStreams res; diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index 3b6a9441d34..52a9689b09a 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -5,7 +5,7 @@ #include -#include +#include #include #include @@ -22,7 +22,6 @@ #include #include -#include #include #include diff --git a/dbms/src/Storages/System/StorageSystemDictionaries.cpp b/dbms/src/Storages/System/StorageSystemDictionaries.cpp index 260aef37344..634e40d0078 100644 --- a/dbms/src/Storages/System/StorageSystemDictionaries.cpp +++ b/dbms/src/Storages/System/StorageSystemDictionaries.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include namespace DB diff --git a/dbms/src/Storages/System/StorageSystemNumbers.cpp b/dbms/src/Storages/System/StorageSystemNumbers.cpp index 6f9029a5873..7a05fdfbbb8 100644 --- a/dbms/src/Storages/System/StorageSystemNumbers.cpp +++ b/dbms/src/Storages/System/StorageSystemNumbers.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include #include diff --git a/dbms/src/Storages/System/StorageSystemOne.cpp b/dbms/src/Storages/System/StorageSystemOne.cpp index 07653bdc6b4..5e92d99517e 100644 --- a/dbms/src/Storages/System/StorageSystemOne.cpp +++ b/dbms/src/Storages/System/StorageSystemOne.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include diff --git a/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp b/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp index 92e9b5f0083..e22d476075e 100644 --- a/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp +++ b/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp @@ -154,7 +154,7 @@ BlockInputStreams StorageSystemReplicationQueue::read( col_attach_source_part_name .column->insert(entry.source_part_name); col_is_currently_executing .column->insert(UInt64(entry.currently_executing)); col_num_tries .column->insert(UInt64(entry.num_tries)); - col_last_exception .column->insert(entry.exception ? entry.exception->displayText() : ""); + col_last_exception .column->insert(entry.exception ? getExceptionMessage(entry.exception, false) : ""); col_last_attempt_time .column->insert(UInt64(entry.last_attempt_time)); col_num_postponed .column->insert(UInt64(entry.num_postponed)); col_postpone_reason .column->insert(entry.postpone_reason); diff --git a/dbms/src/Storages/tests/seek_speed_test.cpp b/dbms/src/Storages/tests/seek_speed_test.cpp index 1d5e85628ac..21020501385 100644 --- a/dbms/src/Storages/tests/seek_speed_test.cpp +++ b/dbms/src/Storages/tests/seek_speed_test.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/dbms/tests/queries/0_stateless/00071_insert_fewer_columns.sql b/dbms/tests/queries/0_stateless/00071_insert_fewer_columns.sql old mode 100755 new mode 100644 diff --git a/dbms/tests/queries/0_stateless/00072_in_types.sql b/dbms/tests/queries/0_stateless/00072_in_types.sql old mode 100755 new mode 100644 diff --git a/dbms/tests/queries/0_stateless/00073_merge_sorting_empty_array_joined.sql b/dbms/tests/queries/0_stateless/00073_merge_sorting_empty_array_joined.sql old mode 100755 new mode 100644 diff --git a/dbms/tests/queries/0_stateless/00074_replicated_attach_race_condition_zookeeper.sql b/dbms/tests/queries/0_stateless/00074_replicated_attach_race_condition_zookeeper.sql old mode 100755 new mode 100644 diff --git a/dbms/tests/queries/0_stateless/00075_formatting_negate_of_negative_literal.sql b/dbms/tests/queries/0_stateless/00075_formatting_negate_of_negative_literal.sql old mode 100755 new mode 100644 diff --git a/dbms/tests/queries/0_stateless/00234_disjunctive_equality_chains_optimization.reference b/dbms/tests/queries/0_stateless/00234_disjunctive_equality_chains_optimization.reference new file mode 100644 index 00000000000..7a7a63dd50f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00234_disjunctive_equality_chains_optimization.reference @@ -0,0 +1 @@ +33 33 diff --git a/dbms/tests/queries/0_stateless/00234_disjunctive_equality_chains_optimization.sql b/dbms/tests/queries/0_stateless/00234_disjunctive_equality_chains_optimization.sql new file mode 100644 index 00000000000..10a25cfeea0 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00234_disjunctive_equality_chains_optimization.sql @@ -0,0 +1,4 @@ +CREATE TABLE IF NOT EXISTS test.foo(id UInt64) Engine=Memory; +INSERT INTO test.foo(id) VALUES (0),(4),(1),(1),(3),(1),(1),(2),(2),(2),(1),(2),(3),(2),(1),(1),(2),(1),(1),(1),(3),(1),(2),(2),(1),(1),(3),(1),(2),(1),(1),(3),(2),(1),(1),(4),(0); +SELECT sum(id = 3 OR id = 1 OR id = 2) AS x, sum(id = 3 OR id = 1 OR id = 2) AS x FROM test.foo; +DROP TABLE test.foo; diff --git a/dbms/tests/queries/0_stateless/00235_create_temporary_table_as.reference b/dbms/tests/queries/0_stateless/00235_create_temporary_table_as.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00235_create_temporary_table_as.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/0_stateless/00235_create_temporary_table_as.sql b/dbms/tests/queries/0_stateless/00235_create_temporary_table_as.sql new file mode 100644 index 00000000000..9057152e698 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00235_create_temporary_table_as.sql @@ -0,0 +1,2 @@ +create temporary table one as select 1; +select * from one; diff --git a/dbms/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper.reference b/dbms/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper.reference new file mode 100644 index 00000000000..0361ab7194b --- /dev/null +++ b/dbms/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper.reference @@ -0,0 +1,5 @@ +2014-01-01 +2014-02-01 +2014-03-01 +2014-01-01 +2014-03-01 diff --git a/dbms/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper.sql b/dbms/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper.sql new file mode 100644 index 00000000000..7d65e403b83 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00236_replicated_drop_on_non_leader_zookeeper.sql @@ -0,0 +1,18 @@ +SET replication_alter_partitions_sync = 2; + +DROP TABLE IF EXISTS test.attach_r1; +DROP TABLE IF EXISTS test.attach_r2; + +CREATE TABLE test.attach_r1 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/01/attach', 'r1', d, d, 8192); +CREATE TABLE test.attach_r2 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/01/attach', 'r2', d, d, 8192); + +INSERT INTO test.attach_r1 VALUES ('2014-01-01'), ('2014-02-01'), ('2014-03-01'); + +SELECT d FROM test.attach_r1 ORDER BY d; + +ALTER TABLE test.attach_r2 DROP PARTITION 201402; + +SELECT d FROM test.attach_r1 ORDER BY d; + +DROP TABLE test.attach_r1; +DROP TABLE test.attach_r2; diff --git a/dbms/tests/queries/0_stateless/00237_group_by_arrays.reference b/dbms/tests/queries/0_stateless/00237_group_by_arrays.reference new file mode 100644 index 00000000000..f68f88f88c6 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00237_group_by_arrays.reference @@ -0,0 +1,57 @@ +[] [1] 1 +[1] [] 1 +[] [0,1,2,3,4,5,6,7,8,9] 1 +[0] [0,1,2,3,4,5,6,7,8] 1 +[0,1] [0,1,2,3,4,5,6,7] 1 +[0,1,2] [0,1,2,3,4,5,6] 1 +[0,1,2,3] [0,1,2,3,4,5] 1 +[0,1,2,3,4] [0,1,2,3,4] 1 +[0,1,2,3,4,5] [0,1,2,3] 1 +[0,1,2,3,4,5,6] [0,1,2] 1 +[0,1,2,3,4,5,6,7] [0,1] 1 +[0,1,2,3,4,5,6,7,8] [0] 1 +[0,1,2,3,4,5,6,7,8,9] [] 1 +[] [0,1,2,3,4,5,6,7,8,9] 1 +['0'] [0,1,2,3,4,5,6,7,8] 1 +['0','1'] [0,1,2,3,4,5,6,7] 1 +['0','1','2'] [0,1,2,3,4,5,6] 1 +['0','1','2','3'] [0,1,2,3,4,5] 1 +['0','1','2','3','4'] [0,1,2,3,4] 1 +['0','1','2','3','4','5'] [0,1,2,3] 1 +['0','1','2','3','4','5','6'] [0,1,2] 1 +['0','1','2','3','4','5','6','7'] [0,1] 1 +['0','1','2','3','4','5','6','7','8'] [0] 1 +['0','1','2','3','4','5','6','7','8','9'] [] 1 +[] ['0','1','2','3','4','5','6','7','8','9'] 1 +['0'] ['0','1','2','3','4','5','6','7','8'] 1 +['0','1'] ['0','1','2','3','4','5','6','7'] 1 +['0','1','2'] ['0','1','2','3','4','5','6'] 1 +['0','1','2','3'] ['0','1','2','3','4','5'] 1 +['0','1','2','3','4'] ['0','1','2','3','4'] 1 +['0','1','2','3','4','5'] ['0','1','2','3'] 1 +['0','1','2','3','4','5','6'] ['0','1','2'] 1 +['0','1','2','3','4','5','6','7'] ['0','1'] 1 +['0','1','2','3','4','5','6','7','8'] ['0'] 1 +['0','1','2','3','4','5','6','7','8','9'] [] 1 +[] [] 1 +['0'] [[0,1,2,3,4,5,6,7,8]] 1 +['0','1'] [[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7]] 1 +['0','1','2'] [[0,1,2,3,4,5,6],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6]] 1 +['0','1','2','3'] [[0,1,2,3,4,5],[0,1,2,3,4,5],[0,1,2,3,4,5],[0,1,2,3,4,5]] 1 +['0','1','2','3','4'] [[0,1,2,3,4],[0,1,2,3,4],[0,1,2,3,4],[0,1,2,3,4],[0,1,2,3,4]] 1 +['0','1','2','3','4','5'] [[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3]] 1 +['0','1','2','3','4','5','6'] [[0,1,2],[0,1,2],[0,1,2],[0,1,2],[0,1,2],[0,1,2],[0,1,2]] 1 +['0','1','2','3','4','5','6','7'] [[0,1],[0,1],[0,1],[0,1],[0,1],[0,1],[0,1],[0,1]] 1 +['0','1','2','3','4','5','6','7','8'] [[0],[0],[0],[0],[0],[0],[0],[0],[0]] 1 +['0','1','2','3','4','5','6','7','8','9'] [[],[],[],[],[],[],[],[],[],[]] 1 +[] [] 3 +['0'] [[0,1,2,3,4,5,6,7,8]] 3 +['0','1'] [[0,1,2,3,4,5,6,7],[0,1,2,3,4,5,6,7]] 3 +['0','1','2'] [[0,1,2,3,4,5,6],[0,1,2,3,4,5,6],[0,1,2,3,4,5,6]] 3 +['0','1','2','3'] [[0,1,2,3,4,5],[0,1,2,3,4,5],[0,1,2,3,4,5],[0,1,2,3,4,5]] 3 +['0','1','2','3','4'] [[0,1,2,3,4],[0,1,2,3,4],[0,1,2,3,4],[0,1,2,3,4],[0,1,2,3,4]] 3 +['0','1','2','3','4','5'] [[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3],[0,1,2,3]] 3 +['0','1','2','3','4','5','6'] [[0,1,2],[0,1,2],[0,1,2],[0,1,2],[0,1,2],[0,1,2],[0,1,2]] 3 +['0','1','2','3','4','5','6','7'] [[0,1],[0,1],[0,1],[0,1],[0,1],[0,1],[0,1],[0,1]] 2 +['0','1','2','3','4','5','6','7','8'] [[0],[0],[0],[0],[0],[0],[0],[0],[0]] 2 +['0','1','2','3','4','5','6','7','8','9'] [[],[],[],[],[],[],[],[],[],[]] 2 diff --git a/dbms/tests/queries/0_stateless/00237_group_by_arrays.sql b/dbms/tests/queries/0_stateless/00237_group_by_arrays.sql new file mode 100644 index 00000000000..faf81ebc5e3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00237_group_by_arrays.sql @@ -0,0 +1,6 @@ +SELECT arr1, arr2, count() AS c FROM (SELECT emptyArrayUInt8() AS arr1, [1] AS arr2 UNION ALL SELECT [1], emptyArrayUInt8()) GROUP BY arr1, arr2 ORDER BY c DESC, arr1, arr2; +SELECT arr1, arr2, count() AS c FROM (SELECT range(number) AS arr1, range(toUInt64(10 - number)) AS arr2 FROM system.numbers LIMIT 11) GROUP BY arr1, arr2 ORDER BY c DESC, arr1, arr2; +SELECT arr1, arr2, count() AS c FROM (SELECT arrayMap(x -> toString(x), range(number)) AS arr1, range(toUInt64(10 - number)) AS arr2 FROM system.numbers LIMIT 11) GROUP BY arr1, arr2 ORDER BY c DESC, arr1, arr2; +SELECT arr1, arr2, count() AS c FROM (SELECT arrayMap(x -> toString(x), range(number)) AS arr1, arrayMap(x -> toString(x), range(toUInt64(10 - number))) AS arr2 FROM system.numbers LIMIT 11) GROUP BY arr1, arr2 ORDER BY c DESC, arr1, arr2; +SELECT arr1, arr2, count() AS c FROM (SELECT arrayMap(x -> toString(x), range(number)) AS arr1, replicate(range(toUInt64(10 - number)), arr1) AS arr2 FROM system.numbers LIMIT 11) GROUP BY arr1, arr2 ORDER BY c DESC, arr1, arr2; +SELECT arr1, arr2, count() AS c FROM (SELECT arrayMap(x -> toString(x), range(number)) AS arr1, replicate(range(toUInt64(10 - number)), arr1) AS arr2 FROM (SELECT number % 11 AS number FROM system.numbers LIMIT 30)) GROUP BY arr1, arr2 ORDER BY c DESC, arr1, arr2; diff --git a/dbms/tests/queries/0_stateless/00238_removal_of_temporary_columns.reference b/dbms/tests/queries/0_stateless/00238_removal_of_temporary_columns.reference new file mode 100644 index 00000000000..209e3ef4b62 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00238_removal_of_temporary_columns.reference @@ -0,0 +1 @@ +20 diff --git a/dbms/tests/queries/0_stateless/00238_removal_of_temporary_columns.sql b/dbms/tests/queries/0_stateless/00238_removal_of_temporary_columns.sql new file mode 100644 index 00000000000..a2a04a01a2b --- /dev/null +++ b/dbms/tests/queries/0_stateless/00238_removal_of_temporary_columns.sql @@ -0,0 +1,2 @@ +SET max_temporary_non_const_columns = 10; +SELECT number + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 AS x FROM system.numbers LIMIT 1; diff --git a/libs/libcommon/include/common/threadpool.hpp b/libs/libcommon/include/common/threadpool.hpp new file mode 100644 index 00000000000..3777adb3459 --- /dev/null +++ b/libs/libcommon/include/common/threadpool.hpp @@ -0,0 +1,28 @@ +/*! \file +* \brief Main include. +* +* This is the only file you have to include in order to use the +* complete threadpool library. +* +* Copyright (c) 2005-2007 Philipp Henkel +* +* Use, modification, and distribution are subject to the +* Boost Software License, Version 1.0. (See accompanying file +* LICENSE_1_0.txt or copy at http://www.boost.org/LICENSE_1_0.txt) +* +* http://threadpool.sourceforge.net +* +*/ + +#ifndef THREADPOOL_HPP_INCLUDED +#define THREADPOOL_HPP_INCLUDED + +#include "threadpool/future.hpp" +#include "threadpool/pool.hpp" + +#include "threadpool/pool_adaptors.hpp" +#include "threadpool/task_adaptors.hpp" + + +#endif // THREADPOOL_HPP_INCLUDED + diff --git a/libs/libcommon/include/common/threadpool/detail/future.hpp b/libs/libcommon/include/common/threadpool/detail/future.hpp new file mode 100644 index 00000000000..4e4b35bebe1 --- /dev/null +++ b/libs/libcommon/include/common/threadpool/detail/future.hpp @@ -0,0 +1,215 @@ +/*! \file +* \brief TODO. +* +* TODO. +* +* Copyright (c) 2005-2007 Philipp Henkel +* +* Use, modification, and distribution are subject to the +* Boost Software License, Version 1.0. (See accompanying file +* LICENSE_1_0.txt or copy at http://www.boost.org/LICENSE_1_0.txt) +* +* http://threadpool.sourceforge.net +* +*/ + + +#ifndef THREADPOOL_DETAIL_FUTURE_IMPL_HPP_INCLUDED +#define THREADPOOL_DETAIL_FUTURE_IMPL_HPP_INCLUDED + + +#include "locking_ptr.hpp" + +#include +#include +#include +#include +#include +#include +#include +#include + +namespace boost { namespace threadpool { namespace detail +{ + +template +class future_impl +{ +public: + typedef Result const & result_type; //!< Indicates the functor's result type. + + typedef Result future_result_type; //!< Indicates the future's result type. + typedef future_impl future_type; + +private: + volatile bool m_ready; + volatile future_result_type m_result; + + mutable mutex m_monitor; + mutable condition m_condition_ready; + + volatile bool m_is_cancelled; + volatile bool m_executing; + +public: + + +public: + + future_impl() + : m_ready(false) + , m_is_cancelled(false) + { + } + + bool ready() const volatile + { + return m_ready; + } + + void wait() const volatile + { + const future_type* self = const_cast(this); + mutex::scoped_lock lock(self->m_monitor); + + while(!m_ready) + { + self->m_condition_ready.wait(lock); + } + } + + + bool timed_wait(boost::xtime const & timestamp) const + { + const future_type* self = const_cast(this); + mutex::scoped_lock lock(self->m_monitor); + + while(!m_ready) + { + if(!self->m_condition_ready.timed_wait(lock, timestamp)) return false; + } + + return true; + } + + + result_type operator()() const volatile + { + wait(); +/* + if( throw_exception_ != 0 ) + { + throw_exception_( this ); + } +*/ + + return *(const_cast(&m_result)); + } + + + void set_value(future_result_type const & r) volatile + { + locking_ptr lockedThis(*this, m_monitor); + if(!m_ready && !m_is_cancelled) + { + lockedThis->m_result = r; + lockedThis->m_ready = true; + lockedThis->m_condition_ready.notify_all(); + } + } +/* + template void set_exception() // throw() + { + m_impl->template set_exception(); + } + + template void set_exception( char const * what ) // throw() + { + m_impl->template set_exception( what ); + } + */ + + + bool cancel() volatile + { + if(!m_ready || m_executing) + { + m_is_cancelled = true; + return true; + } + else + { + return false; + } + } + + + bool is_cancelled() const volatile + { + return m_is_cancelled; + } + + + void set_execution_status(bool executing) volatile + { + m_executing = executing; + } +}; + + +template< + template class Future, + typename Function +> +class future_impl_task_func +{ + +public: + typedef void result_type; //!< Indicates the functor's result type. + + typedef Function function_type; //!< Indicates the function's type. + typedef typename result_of::type future_result_type; //!< Indicates the future's result type. + typedef Future future_type; //!< Indicates the future's type. + + // The task is required to be a nullary function. + BOOST_STATIC_ASSERT(function_traits::arity == 0); + + // The task function's result type is required not to be void. + BOOST_STATIC_ASSERT(!is_void::value); + +private: + function_type m_function; + shared_ptr m_future; + +public: + future_impl_task_func(function_type const & function, shared_ptr const & future) + : m_function(function) + , m_future(future) + { + } + + void operator()() + { + if(m_function) + { + m_future->set_execution_status(true); + if(!m_future->is_cancelled()) + { + // TODO future exeception handling + m_future->set_value(m_function()); + } + m_future->set_execution_status(false); // TODO consider exceptions + } + } + +}; + + + + + +} } } // namespace boost::threadpool::detail + +#endif // THREADPOOL_DETAIL_FUTURE_IMPL_HPP_INCLUDED + + diff --git a/libs/libcommon/include/common/threadpool/detail/locking_ptr.hpp b/libs/libcommon/include/common/threadpool/detail/locking_ptr.hpp new file mode 100644 index 00000000000..2d44db70fe9 --- /dev/null +++ b/libs/libcommon/include/common/threadpool/detail/locking_ptr.hpp @@ -0,0 +1,102 @@ +/*! \file +* \brief The locking_ptr is smart pointer with a scoped locking mechanism. +* +* The class is a wrapper for a volatile pointer. It enables synchronized access to the +* internal pointer by locking the passed mutex. +* locking_ptr is based on Andrei Alexandrescu's LockingPtr. For more information +* see article "volatile - Multithreaded Programmer's Best Friend" by A. Alexandrescu. +* +* +* Copyright (c) 2005-2007 Philipp Henkel +* +* Use, modification, and distribution are subject to the +* Boost Software License, Version 1.0. (See accompanying file +* LICENSE_1_0.txt or copy at http://www.boost.org/LICENSE_1_0.txt) +* +* http://threadpool.sourceforge.net +* +*/ + + +#ifndef THREADPOOL_DETAIL_LOCKING_PTR_HPP_INCLUDED +#define THREADPOOL_DETAIL_LOCKING_PTR_HPP_INCLUDED + +#include + +// Support for old boost::thread +//********************************************** +#include +#ifndef BOOST_THREAD_MUTEX_HPP +#include +#endif +//********************************************** + +namespace boost { namespace threadpool { namespace detail +{ + +/*! \brief Smart pointer with a scoped locking mechanism. + * + * This class is a wrapper for a volatile pointer. It enables synchronized access to the + * internal pointer by locking the passed mutex. + */ + template + class locking_ptr + : private noncopyable + { + T* m_obj; //!< The instance pointer. + Mutex & m_mutex; //!< Mutex is used for scoped locking. + + public: + /// Constructor. + locking_ptr(volatile T& obj, const volatile Mutex& mtx) + : m_obj(const_cast(&obj)) + , m_mutex(*const_cast(&mtx)) + { + // Lock mutex + +#ifndef BOOST_THREAD_MUTEX_HPP + // Support for old boost::thread + boost::detail::thread::lock_ops::lock(m_mutex); +#else + m_mutex.lock(); +#endif + } + + + /// Destructor. + ~locking_ptr() + { + // Unlock mutex +#ifndef BOOST_THREAD_MUTEX_HPP + // Support for old boost::thread + boost::detail::thread::lock_ops::unlock(m_mutex); +#else + m_mutex.unlock(); +#endif + } + + + /*! Returns a reference to the stored instance. + * \return The instance's reference. + */ + T& operator*() const + { + return *m_obj; + } + + + /*! Returns a pointer to the stored instance. + * \return The instance's pointer. + */ + T* operator->() const + { + return m_obj; + } + }; + + +} } } // namespace boost::threadpool::detail + + +#endif // THREADPOOL_DETAIL_LOCKING_PTR_HPP_INCLUDED + diff --git a/libs/libcommon/include/common/threadpool/detail/pool_core.hpp b/libs/libcommon/include/common/threadpool/detail/pool_core.hpp new file mode 100644 index 00000000000..87ec1873cf8 --- /dev/null +++ b/libs/libcommon/include/common/threadpool/detail/pool_core.hpp @@ -0,0 +1,453 @@ +/*! \file +* \brief Thread pool core. +* +* This file contains the threadpool's core class: pool. +* +* Thread pools are a mechanism for asynchronous and parallel processing +* within the same process. The pool class provides a convenient way +* for dispatching asynchronous tasks as functions objects. The scheduling +* of these tasks can be easily controlled by using customized schedulers. +* +* Copyright (c) 2005-2007 Philipp Henkel +* +* Use, modification, and distribution are subject to the +* Boost Software License, Version 1.0. (See accompanying file +* LICENSE_1_0.txt or copy at http://www.boost.org/LICENSE_1_0.txt) +* +* http://threadpool.sourceforge.net +* +*/ + + +#ifndef THREADPOOL_POOL_CORE_HPP_INCLUDED +#define THREADPOOL_POOL_CORE_HPP_INCLUDED + + + + +#include "locking_ptr.hpp" +#include "worker_thread.hpp" + +#include "../task_adaptors.hpp" + +#include +#include +#include +#include +#include +#include +#include +#include + +#include + + +/// The namespace threadpool contains a thread pool and related utility classes. +namespace boost { namespace threadpool { namespace detail +{ + + /*! \brief Thread pool. + * + * Thread pools are a mechanism for asynchronous and parallel processing + * within the same process. The pool class provides a convenient way + * for dispatching asynchronous tasks as functions objects. The scheduling + * of these tasks can be easily controlled by using customized schedulers. + * A task must not throw an exception. + * + * A pool_impl is DefaultConstructible and NonCopyable. + * + * \param Task A function object which implements the operator 'void operator() (void) const'. The operator () is called by the pool to execute the task. Exceptions are ignored. + * \param Scheduler A task container which determines how tasks are scheduled. It is guaranteed that this container is accessed only by one thread at a time. The scheduler shall not throw exceptions. + * + * \remarks The pool class is thread-safe. + * + * \see Tasks: task_func, prio_task_func + * \see Scheduling policies: fifo_scheduler, lifo_scheduler, prio_scheduler + */ + template < + typename Task, + + template class SchedulingPolicy, + template class SizePolicy, + template class SizePolicyController, + template class ShutdownPolicy + > + class pool_core + : public enable_shared_from_this< pool_core > + , private noncopyable + { + + public: // Type definitions + typedef Task task_type; //!< Indicates the task's type. + typedef SchedulingPolicy scheduler_type; //!< Indicates the scheduler's type. + typedef pool_core pool_type; //!< Indicates the thread pool's type. + typedef SizePolicy size_policy_type; //!< Indicates the sizer's type. + //typedef typename size_policy_type::size_controller size_controller_type; + + typedef SizePolicyController size_controller_type; + +// typedef SizePolicy::size_controller size_controller_type; + typedef ShutdownPolicy shutdown_policy_type;//!< Indicates the shutdown policy's type. + + typedef worker_thread worker_type; + + // The task is required to be a nullary function. + BOOST_STATIC_ASSERT(function_traits::arity == 0); + + // The task function's result type is required to be void. + BOOST_STATIC_ASSERT(is_void::type >::value); + + + private: // Friends + friend class worker_thread; + +#if defined(__SUNPRO_CC) && (__SUNPRO_CC <= 0x580) // Tested with CC: Sun C++ 5.8 Patch 121018-08 2006/12/06 + friend class SizePolicy; + friend class ShutdownPolicy; +#else + friend class SizePolicy; + friend class ShutdownPolicy; +#endif + + private: // The following members may be accessed by _multiple_ threads at the same time: + volatile size_t m_worker_count; + volatile size_t m_target_worker_count; + volatile size_t m_active_worker_count; + + + + private: // The following members are accessed only by _one_ thread at the same time: + scheduler_type m_scheduler; + scoped_ptr m_size_policy; // is never null + + bool m_terminate_all_workers; // Indicates if termination of all workers was triggered. + std::vector > m_terminated_workers; // List of workers which are terminated but not fully destructed. + + private: // The following members are implemented thread-safe: + mutable recursive_mutex m_monitor; + mutable condition m_worker_idle_or_terminated_event; // A worker is idle or was terminated. + mutable condition m_task_or_terminate_workers_event; // Task is available OR total worker count should be reduced. + + public: + /// Constructor. + pool_core() + : m_worker_count(0) + , m_target_worker_count(0) + , m_active_worker_count(0) + , m_terminate_all_workers(false) + { + pool_type volatile & self_ref = *this; + m_size_policy.reset(new size_policy_type(self_ref)); + + m_scheduler.clear(); + } + + + /// Destructor. + ~pool_core() + { + } + + /*! Gets the size controller which manages the number of threads in the pool. + * \return The size controller. + * \see SizePolicy + */ + size_controller_type size_controller() + { + return size_controller_type(*m_size_policy, this->shared_from_this()); + } + + /*! Gets the number of threads in the pool. + * \return The number of threads. + */ + size_t size() const volatile + { + return m_worker_count; + } + +// TODO is only called once + void shutdown() + { + ShutdownPolicy::shutdown(*this); + } + + /*! Schedules a task for asynchronous execution. The task will be executed once only. + * \param task The task function object. It should not throw execeptions. + * \return true, if the task could be scheduled and false otherwise. + */ + bool schedule(task_type const & task) volatile + { + locking_ptr lockedThis(*this, m_monitor); + + if(lockedThis->m_scheduler.push(task)) + { + lockedThis->m_task_or_terminate_workers_event.notify_one(); + return true; + } + else + { + return false; + } + } + + + /*! Returns the number of tasks which are currently executed. + * \return The number of active tasks. + */ + size_t active() const volatile + { + return m_active_worker_count; + } + + + /*! Returns the number of tasks which are ready for execution. + * \return The number of pending tasks. + */ + size_t pending() const volatile + { + locking_ptr lockedThis(*this, m_monitor); + return lockedThis->m_scheduler.size(); + } + + + /*! Removes all pending tasks from the pool's scheduler. + */ + void clear() volatile + { + locking_ptr lockedThis(*this, m_monitor); + lockedThis->m_scheduler.clear(); + } + + + /*! Indicates that there are no tasks pending. + * \return true if there are no tasks ready for execution. + * \remarks This function is more efficient that the check 'pending() == 0'. + */ + bool empty() const volatile + { + locking_ptr lockedThis(*this, m_monitor); + return lockedThis->m_scheduler.empty(); + } + + + /*! The current thread of execution is blocked until the sum of all active + * and pending tasks is equal or less than a given threshold. + * \param task_threshold The maximum number of tasks in pool and scheduler. + */ + void wait(size_t const task_threshold = 0) const volatile + { + const pool_type* self = const_cast(this); + recursive_mutex::scoped_lock lock(self->m_monitor); + + if(0 == task_threshold) + { + while(0 != self->m_active_worker_count || !self->m_scheduler.empty()) + { + self->m_worker_idle_or_terminated_event.wait(lock); + } + } + else + { + while(task_threshold < self->m_active_worker_count + self->m_scheduler.size()) + { + self->m_worker_idle_or_terminated_event.wait(lock); + } + } + } + + /*! The current thread of execution is blocked until the timestamp is met + * or the sum of all active and pending tasks is equal or less + * than a given threshold. + * \param timestamp The time when function returns at the latest. + * \param task_threshold The maximum number of tasks in pool and scheduler. + * \return true if the task sum is equal or less than the threshold, false otherwise. + */ + bool wait(xtime const & timestamp, size_t const task_threshold = 0) const volatile + { + const pool_type* self = const_cast(this); + recursive_mutex::scoped_lock lock(self->m_monitor); + + if(0 == task_threshold) + { + while(0 != self->m_active_worker_count || !self->m_scheduler.empty()) + { + if(!self->m_worker_idle_or_terminated_event.timed_wait(lock, timestamp)) return false; + } + } + else + { + while(task_threshold < self->m_active_worker_count + self->m_scheduler.size()) + { + if(!self->m_worker_idle_or_terminated_event.timed_wait(lock, timestamp)) return false; + } + } + + return true; + } + + + private: + + + void terminate_all_workers(bool const wait) volatile + { + pool_type* self = const_cast(this); + recursive_mutex::scoped_lock lock(self->m_monitor); + + self->m_terminate_all_workers = true; + + m_target_worker_count = 0; + self->m_task_or_terminate_workers_event.notify_all(); + + if(wait) + { + while(m_worker_count > 0) + { + self->m_worker_idle_or_terminated_event.wait(lock); + } + + for(typename std::vector >::iterator it = self->m_terminated_workers.begin(); + it != self->m_terminated_workers.end(); + ++it) + { + (*it)->join(); + } + self->m_terminated_workers.clear(); + } + } + + + /*! Changes the number of worker threads in the pool. The resizing + * is handled by the SizePolicy. + * \param threads The new number of worker threads. + * \return true, if pool will be resized and false if not. + */ + bool resize(size_t const worker_count) volatile + { + locking_ptr lockedThis(*this, m_monitor); + + if(!m_terminate_all_workers) + { + m_target_worker_count = worker_count; + } + else + { + return false; + } + + + if(m_worker_count <= m_target_worker_count) + { // increase worker count + while(m_worker_count < m_target_worker_count) + { + try + { + worker_thread::create_and_attach(lockedThis->shared_from_this()); + m_worker_count++; + m_active_worker_count++; + } + catch(thread_resource_error) + { + return false; + } + } + } + else + { // decrease worker count + lockedThis->m_task_or_terminate_workers_event.notify_all(); // TODO: Optimize number of notified workers + } + + return true; + } + + + // worker died with unhandled exception + void worker_died_unexpectedly(shared_ptr worker) volatile + { + locking_ptr lockedThis(*this, m_monitor); + + m_worker_count--; + m_active_worker_count--; + lockedThis->m_worker_idle_or_terminated_event.notify_all(); + + if(m_terminate_all_workers) + { + lockedThis->m_terminated_workers.push_back(worker); + } + else + { + lockedThis->m_size_policy->worker_died_unexpectedly(m_worker_count); + } + } + + void worker_destructed(shared_ptr worker) volatile + { + locking_ptr lockedThis(*this, m_monitor); + m_worker_count--; + m_active_worker_count--; + lockedThis->m_worker_idle_or_terminated_event.notify_all(); + + if(m_terminate_all_workers) + { + lockedThis->m_terminated_workers.push_back(worker); + } + } + + + bool execute_task() volatile + { + function0 task; + + { // fetch task + pool_type* lockedThis = const_cast(this); + recursive_mutex::scoped_lock lock(lockedThis->m_monitor); + + // decrease number of threads if necessary + if(m_worker_count > m_target_worker_count) + { + return false; // terminate worker + } + + + // wait for tasks + while(lockedThis->m_scheduler.empty()) + { + // decrease number of workers if necessary + if(m_worker_count > m_target_worker_count) + { + return false; // terminate worker + } + else + { + m_active_worker_count--; + lockedThis->m_worker_idle_or_terminated_event.notify_all(); + lockedThis->m_task_or_terminate_workers_event.wait(lock); + m_active_worker_count++; + } + } + + task = lockedThis->m_scheduler.top(); + lockedThis->m_scheduler.pop(); + } + + // call task function + if(task) + { + task(); + } + + //guard->disable(); + return true; + } + }; + + + + +} } } // namespace boost::threadpool::detail + +#endif // THREADPOOL_POOL_CORE_HPP_INCLUDED diff --git a/libs/libcommon/include/common/threadpool/detail/scope_guard.hpp b/libs/libcommon/include/common/threadpool/detail/scope_guard.hpp new file mode 100644 index 00000000000..68634654ba5 --- /dev/null +++ b/libs/libcommon/include/common/threadpool/detail/scope_guard.hpp @@ -0,0 +1,65 @@ +/*! \file +* \brief TODO. +* +* TODO. +* +* Copyright (c) 2005-2007 Philipp Henkel +* +* Use, modification, and distribution are subject to the +* Boost Software License, Version 1.0. (See accompanying file +* LICENSE_1_0.txt or copy at http://www.boost.org/LICENSE_1_0.txt) +* +* http://threadpool.sourceforge.net +* +*/ + + +#ifndef THREADPOOL_DETAIL_SCOPE_GUARD_HPP_INCLUDED +#define THREADPOOL_DETAIL_SCOPE_GUARD_HPP_INCLUDED + + + +#include + + +namespace boost { namespace threadpool { namespace detail +{ + +// TODO documentation +class scope_guard +: private boost::noncopyable +{ + function0 const m_function; + bool m_is_active; + +public: + scope_guard(function0 const & call_on_exit) + : m_function(call_on_exit) + , m_is_active(true) + { + } + + ~scope_guard() + { + if(m_is_active && m_function) + { + m_function(); + } + } + + void disable() + { + m_is_active = false; + } +}; + + + + + + +} } } // namespace boost::threadpool::detail + +#endif // THREADPOOL_DETAIL_SCOPE_GUARD_HPP_INCLUDED + + diff --git a/libs/libcommon/include/common/threadpool/detail/worker_thread.hpp b/libs/libcommon/include/common/threadpool/detail/worker_thread.hpp new file mode 100644 index 00000000000..d33467a5279 --- /dev/null +++ b/libs/libcommon/include/common/threadpool/detail/worker_thread.hpp @@ -0,0 +1,115 @@ +/*! \file +* \brief Thread pool worker. +* +* The worker thread instance is attached to a pool +* and executes tasks of this pool. +* +* Copyright (c) 2005-2007 Philipp Henkel +* +* Use, modification, and distribution are subject to the +* Boost Software License, Version 1.0. (See accompanying file +* LICENSE_1_0.txt or copy at http://www.boost.org/LICENSE_1_0.txt) +* +* http://threadpool.sourceforge.net +* +*/ + +#ifndef THREADPOOL_DETAIL_WORKER_THREAD_HPP_INCLUDED +#define THREADPOOL_DETAIL_WORKER_THREAD_HPP_INCLUDED + + +#include "scope_guard.hpp" + +#include +#include +#include +#include +#include + + +namespace boost { namespace threadpool { namespace detail +{ + + /*! \brief Thread pool worker. + * + * A worker_thread represents a thread of execution. The worker is attached to a + * thread pool and processes tasks of that pool. The lifetime of the worker and its + * internal boost::thread is managed automatically. + * + * This class is a helper class and cannot be constructed or accessed directly. + * + * \see pool_core + */ + template + class worker_thread + : public enable_shared_from_this< worker_thread > + , private noncopyable + { + public: + typedef Pool pool_type; //!< Indicates the pool's type. + + private: + shared_ptr m_pool; //!< Pointer to the pool which created the worker. + shared_ptr m_thread; //!< Pointer to the thread which executes the run loop. + + + /*! Constructs a new worker. + * \param pool Pointer to it's parent pool. + * \see function create_and_attach + */ + worker_thread(shared_ptr const & pool) + : m_pool(pool) + { + assert(pool); + } + + + /*! Notifies that an exception occurred in the run loop. + */ + void died_unexpectedly() + { + m_pool->worker_died_unexpectedly(this->shared_from_this()); + } + + + public: + /*! Executes pool's tasks sequentially. + */ + void run() + { + scope_guard notify_exception(bind(&worker_thread::died_unexpectedly, this)); + + while(m_pool->execute_task()) {} + + notify_exception.disable(); + m_pool->worker_destructed(this->shared_from_this()); + } + + + /*! Joins the worker's thread. + */ + void join() + { + m_thread->join(); + } + + + /*! Constructs a new worker thread and attaches it to the pool. + * \param pool Pointer to the pool. + */ + static void create_and_attach(shared_ptr const & pool) + { + shared_ptr worker(new worker_thread(pool)); + if(worker) + { + worker->m_thread.reset(new boost::thread(bind(&worker_thread::run, worker))); + } + } + + }; + + +} } } // namespace boost::threadpool::detail + +#endif // THREADPOOL_DETAIL_WORKER_THREAD_HPP_INCLUDED + diff --git a/libs/libcommon/include/common/threadpool/future.hpp b/libs/libcommon/include/common/threadpool/future.hpp new file mode 100644 index 00000000000..57568edefd9 --- /dev/null +++ b/libs/libcommon/include/common/threadpool/future.hpp @@ -0,0 +1,144 @@ +/*! \file +* \brief TODO. +* +* TODO. +* +* Copyright (c) 2005-2007 Philipp Henkel +* +* Use, modification, and distribution are subject to the +* Boost Software License, Version 1.0. (See accompanying file +* LICENSE_1_0.txt or copy at http://www.boost.org/LICENSE_1_0.txt) +* +* http://threadpool.sourceforge.net +* +*/ + +#ifndef THREADPOOL_FUTURE_HPP_INCLUDED +#define THREADPOOL_FUTURE_HPP_INCLUDED + + + +#include "detail/future.hpp" +#include + +//#include "pool.hpp" +//#include + +//#include + + +namespace boost { namespace threadpool +{ + + /*! \brief Experimental. Do not use in production code. TODO. + * + * TODO Future + * + * \see TODO + * + */ + + +template +class future +{ +private: + shared_ptr > m_impl; + +public: + typedef Result const & result_type; //!< Indicates the functor's result type. + typedef Result future_result_type; //!< Indicates the future's result type. + + +public: + + future() + : m_impl(new detail::future_impl()) // TODO remove this + { + } + + // only for internal usage + future(shared_ptr > const & impl) + : m_impl(impl) + { + } + + bool ready() const + { + return m_impl->ready(); + } + + void wait() const + { + m_impl->wait(); + } + + bool timed_wait(boost::xtime const & timestamp) const + { + return m_impl->timed_wait(timestamp); + } + + result_type operator()() // throw( thread::cancelation_exception, ... ) + { + return (*m_impl)(); + } + + result_type get() // throw( thread::cancelation_exception, ... ) + { + return (*m_impl)(); + } + + bool cancel() + { + return m_impl->cancel(); + } + + bool is_cancelled() const + { + return m_impl->is_cancelled(); + } +}; + + + + + +template +typename disable_if < + is_void< typename result_of< Function() >::type >, + future< typename result_of< Function() >::type > +>::type +schedule(Pool& pool, const Function& task) +{ + typedef typename result_of< Function() >::type future_result_type; + + // create future impl and future + shared_ptr > impl(new detail::future_impl); + future res(impl); + + // schedule future impl + pool.schedule(detail::future_impl_task_func(task, impl)); + + // return future + return res; + +/* + TODO + if(pool->schedule(bind(&Future::run, future))) + { + return future; + } + else + { + // construct empty future + return error_future; + } + */ +} + + + +} } // namespace boost::threadpool + +#endif // THREADPOOL_FUTURE_HPP_INCLUDED + diff --git a/libs/libcommon/include/common/threadpool/pool.hpp b/libs/libcommon/include/common/threadpool/pool.hpp new file mode 100644 index 00000000000..87ec33021b0 --- /dev/null +++ b/libs/libcommon/include/common/threadpool/pool.hpp @@ -0,0 +1,232 @@ +/*! \file +* \brief Thread pool core. +* +* This file contains the threadpool's core class: pool. +* +* Thread pools are a mechanism for asynchronous and parallel processing +* within the same process. The pool class provides a convenient way +* for dispatching asynchronous tasks as functions objects. The scheduling +* of these tasks can be easily controlled by using customized schedulers. +* +* Copyright (c) 2005-2007 Philipp Henkel +* +* Use, modification, and distribution are subject to the +* Boost Software License, Version 1.0. (See accompanying file +* LICENSE_1_0.txt or copy at http://www.boost.org/LICENSE_1_0.txt) +* +* http://threadpool.sourceforge.net +* +*/ + + +#ifndef THREADPOOL_POOL_HPP_INCLUDED +#define THREADPOOL_POOL_HPP_INCLUDED + +#include + +#include "detail/pool_core.hpp" + +#include "task_adaptors.hpp" + +#include "detail/locking_ptr.hpp" + +#include "scheduling_policies.hpp" +#include "size_policies.hpp" +#include "shutdown_policies.hpp" + + + +/// The namespace threadpool contains a thread pool and related utility classes. +namespace boost { namespace threadpool +{ + + + + /*! \brief Thread pool. + * + * Thread pools are a mechanism for asynchronous and parallel processing + * within the same process. The pool class provides a convenient way + * for dispatching asynchronous tasks as functions objects. The scheduling + * of these tasks can be easily controlled by using customized schedulers. + * A task must not throw an exception. + * + * A pool is DefaultConstructible, CopyConstructible and Assignable. + * It has reference semantics; all copies of the same pool are equivalent and interchangeable. + * All operations on a pool except assignment are strongly thread safe or sequentially consistent; + * that is, the behavior of concurrent calls is as if the calls have been issued sequentially in an unspecified order. + * + * \param Task A function object which implements the operator 'void operator() (void) const'. The operator () is called by the pool to execute the task. Exceptions are ignored. + * \param SchedulingPolicy A task container which determines how tasks are scheduled. It is guaranteed that this container is accessed only by one thread at a time. The scheduler shall not throw exceptions. + * + * \remarks The pool class is thread-safe. + * + * \see Tasks: task_func, prio_task_func + * \see Scheduling policies: fifo_scheduler, lifo_scheduler, prio_scheduler + */ + template < + typename Task = task_func, + template class SchedulingPolicy = fifo_scheduler, + template class SizePolicy = static_size, + template class SizePolicyController = resize_controller, + template class ShutdownPolicy = wait_for_all_tasks + > + class thread_pool + { + typedef detail::pool_core pool_core_type; + shared_ptr m_core; // pimpl idiom + shared_ptr m_shutdown_controller; // If the last pool holding a pointer to the core is deleted the controller shuts the pool down. + + public: // Type definitions + typedef Task task_type; //!< Indicates the task's type. + typedef SchedulingPolicy scheduler_type; //!< Indicates the scheduler's type. + /* typedef thread_pool pool_type; //!< Indicates the thread pool's type. + */ + typedef SizePolicy size_policy_type; + typedef SizePolicyController size_controller_type; + + + public: + /*! Constructor. + * \param initial_threads The pool is immediately resized to set the specified number of threads. The pool's actual number threads depends on the SizePolicy. + */ + thread_pool(size_t initial_threads = 0) + : m_core(new pool_core_type) + , m_shutdown_controller(static_cast(0), bind(&pool_core_type::shutdown, m_core)) + { + size_policy_type::init(*m_core, initial_threads); + } + + + /*! Gets the size controller which manages the number of threads in the pool. + * \return The size controller. + * \see SizePolicy + */ + size_controller_type size_controller() + { + return m_core->size_controller(); + } + + + /*! Gets the number of threads in the pool. + * \return The number of threads. + */ + size_t size() const + { + return m_core->size(); + } + + + /*! Schedules a task for asynchronous execution. The task will be executed once only. + * \param task The task function object. It should not throw execeptions. + * \return true, if the task could be scheduled and false otherwise. + */ + bool schedule(task_type const & task) + { + return m_core->schedule(task); + } + + + /*! Returns the number of tasks which are currently executed. + * \return The number of active tasks. + */ + size_t active() const + { + return m_core->active(); + } + + + /*! Returns the number of tasks which are ready for execution. + * \return The number of pending tasks. + */ + size_t pending() const + { + return m_core->pending(); + } + + + /*! Removes all pending tasks from the pool's scheduler. + */ + void clear() + { + m_core->clear(); + } + + + /*! Indicates that there are no tasks pending. + * \return true if there are no tasks ready for execution. + * \remarks This function is more efficient that the check 'pending() == 0'. + */ + bool empty() const + { + return m_core->empty(); + } + + + /*! The current thread of execution is blocked until the sum of all active + * and pending tasks is equal or less than a given threshold. + * \param task_threshold The maximum number of tasks in pool and scheduler. + */ + void wait(size_t task_threshold = 0) const + { + m_core->wait(task_threshold); + } + + + /*! The current thread of execution is blocked until the timestamp is met + * or the sum of all active and pending tasks is equal or less + * than a given threshold. + * \param timestamp The time when function returns at the latest. + * \param task_threshold The maximum number of tasks in pool and scheduler. + * \return true if the task sum is equal or less than the threshold, false otherwise. + */ + bool wait(xtime const & timestamp, size_t task_threshold = 0) const + { + return m_core->wait(timestamp, task_threshold); + } + }; + + + + /*! \brief Fifo pool. + * + * The pool's tasks are fifo scheduled task_func functors. + * + */ + typedef thread_pool fifo_pool; + + + /*! \brief Lifo pool. + * + * The pool's tasks are lifo scheduled task_func functors. + * + */ + typedef thread_pool lifo_pool; + + + /*! \brief Pool for prioritized task. + * + * The pool's tasks are prioritized prio_task_func functors. + * + */ + typedef thread_pool prio_pool; + + + /*! \brief A standard pool. + * + * The pool's tasks are fifo scheduled task_func functors. + * + */ + typedef fifo_pool pool; + + + +} } // namespace boost::threadpool + +#endif // THREADPOOL_POOL_HPP_INCLUDED diff --git a/libs/libcommon/include/common/threadpool/pool_adaptors.hpp b/libs/libcommon/include/common/threadpool/pool_adaptors.hpp new file mode 100644 index 00000000000..6cde152d2ae --- /dev/null +++ b/libs/libcommon/include/common/threadpool/pool_adaptors.hpp @@ -0,0 +1,70 @@ +/*! \file +* \brief Pool adaptors. +* +* This file contains an easy-to-use adaptor similar to a smart +* pointer for the pool class. +* +* Copyright (c) 2005-2007 Philipp Henkel +* +* Use, modification, and distribution are subject to the +* Boost Software License, Version 1.0. (See accompanying file +* LICENSE_1_0.txt or copy at http://www.boost.org/LICENSE_1_0.txt) +* +* http://threadpool.sourceforge.net +* +*/ + + +#ifndef THREADPOOL_POOL_ADAPTORS_HPP_INCLUDED +#define THREADPOOL_POOL_ADAPTORS_HPP_INCLUDED + +#include + + +namespace boost { namespace threadpool +{ + + +// TODO convenience scheduling function + /*! Schedules a Runnable for asynchronous execution. A Runnable is an arbitrary class with a run() + * member function. This a convenience shorthand for pool->schedule(bind(&Runnable::run, task_object)). + * \param + * \param obj The Runnable object. The member function run() will be exectued and should not throw execeptions. + * \return true, if the task could be scheduled and false otherwise. + */ + template + bool schedule(Pool& pool, shared_ptr const & obj) + { + return pool->schedule(bind(&Runnable::run, obj)); + } + + /*! Schedules a task for asynchronous execution. The task will be executed once only. + * \param task The task function object. + */ + template + typename enable_if < + is_void< typename result_of< typename Pool::task_type() >::type >, + bool + >::type + schedule(Pool& pool, typename Pool::task_type const & task) + { + return pool.schedule(task); + } + + + template + typename enable_if < + is_void< typename result_of< typename Pool::task_type() >::type >, + bool + >::type + schedule(shared_ptr const pool, typename Pool::task_type const & task) + { + return pool->schedule(task); + } + + +} } // namespace boost::threadpool + +#endif // THREADPOOL_POOL_ADAPTORS_HPP_INCLUDED + + diff --git a/libs/libcommon/include/common/threadpool/scheduling_policies.hpp b/libs/libcommon/include/common/threadpool/scheduling_policies.hpp new file mode 100644 index 00000000000..2d6c7c1baed --- /dev/null +++ b/libs/libcommon/include/common/threadpool/scheduling_policies.hpp @@ -0,0 +1,262 @@ +/*! \file +* \brief Task scheduling policies. +* +* This file contains some fundamental scheduling policies for the pool class. +* A scheduling policy is realized by a task container which controls the access to +* the tasks. Fundamentally the container determines the order the tasks are processed +* by the thread pool. +* The task containers need not to be thread-safe because they are used by the pool +* in thread-safe way. +* +* Copyright (c) 2005-2007 Philipp Henkel +* +* Use, modification, and distribution are subject to the +* Boost Software License, Version 1.0. (See accompanying file +* LICENSE_1_0.txt or copy at http://www.boost.org/LICENSE_1_0.txt) +* +* http://threadpool.sourceforge.net +* +*/ + + +#ifndef THREADPOOL_SCHEDULING_POLICIES_HPP_INCLUDED +#define THREADPOOL_SCHEDULING_POLICIES_HPP_INCLUDED + + +#include +#include + +#include "task_adaptors.hpp" + +namespace boost { namespace threadpool +{ + + /*! \brief SchedulingPolicy which implements FIFO ordering. + * + * This container implements a FIFO scheduling policy. + * The first task to be added to the scheduler will be the first to be removed. + * The processing proceeds sequentially in the same order. + * FIFO stands for "first in, first out". + * + * \param Task A function object which implements the operator()(void). + * + */ + template + class fifo_scheduler + { + public: + typedef Task task_type; //!< Indicates the scheduler's task type. + + protected: + std::deque m_container; //!< Internal task container. + + + public: + /*! Adds a new task to the scheduler. + * \param task The task object. + * \return true, if the task could be scheduled and false otherwise. + */ + bool push(task_type const & task) + { + m_container.push_back(task); + return true; + } + + /*! Removes the task which should be executed next. + */ + void pop() + { + m_container.pop_front(); + } + + /*! Gets the task which should be executed next. + * \return The task object to be executed. + */ + task_type const & top() const + { + return m_container.front(); + } + + /*! Gets the current number of tasks in the scheduler. + * \return The number of tasks. + * \remarks Prefer empty() to size() == 0 to check if the scheduler is empty. + */ + size_t size() const + { + return m_container.size(); + } + + /*! Checks if the scheduler is empty. + * \return true if the scheduler contains no tasks, false otherwise. + * \remarks Is more efficient than size() == 0. + */ + bool empty() const + { + return m_container.empty(); + } + + /*! Removes all tasks from the scheduler. + */ + void clear() + { + m_container.clear(); + } + }; + + + + /*! \brief SchedulingPolicy which implements LIFO ordering. + * + * This container implements a LIFO scheduling policy. + * The last task to be added to the scheduler will be the first to be removed. + * LIFO stands for "last in, first out". + * + * \param Task A function object which implements the operator()(void). + * + */ + template + class lifo_scheduler + { + public: + typedef Task task_type; //!< Indicates the scheduler's task type. + + protected: + std::deque m_container; //!< Internal task container. + + public: + /*! Adds a new task to the scheduler. + * \param task The task object. + * \return true, if the task could be scheduled and false otherwise. + */ + bool push(task_type const & task) + { + m_container.push_front(task); + return true; + } + + /*! Removes the task which should be executed next. + */ + void pop() + { + m_container.pop_front(); + } + + /*! Gets the task which should be executed next. + * \return The task object to be executed. + */ + task_type const & top() const + { + return m_container.front(); + } + + /*! Gets the current number of tasks in the scheduler. + * \return The number of tasks. + * \remarks Prefer empty() to size() == 0 to check if the scheduler is empty. + */ + size_t size() const + { + return m_container.size(); + } + + /*! Checks if the scheduler is empty. + * \return true if the scheduler contains no tasks, false otherwise. + * \remarks Is more efficient than size() == 0. + */ + bool empty() const + { + return m_container.empty(); + } + + /*! Removes all tasks from the scheduler. + */ + void clear() + { + m_container.clear(); + } + + }; + + + + /*! \brief SchedulingPolicy which implements prioritized ordering. + * + * This container implements a scheduling policy based on task priorities. + * The task with highest priority will be the first to be removed. + * It must be possible to compare two tasks using operator<. + * + * \param Task A function object which implements the operator() and operator<. operator< must be a partial ordering. + * + * \see prio_thread_func + * + */ + template + class prio_scheduler + { + public: + typedef Task task_type; //!< Indicates the scheduler's task type. + + protected: + std::priority_queue m_container; //!< Internal task container. + + + public: + /*! Adds a new task to the scheduler. + * \param task The task object. + * \return true, if the task could be scheduled and false otherwise. + */ + bool push(task_type const & task) + { + m_container.push(task); + return true; + } + + /*! Removes the task which should be executed next. + */ + void pop() + { + m_container.pop(); + } + + /*! Gets the task which should be executed next. + * \return The task object to be executed. + */ + task_type const & top() const + { + return m_container.top(); + } + + /*! Gets the current number of tasks in the scheduler. + * \return The number of tasks. + * \remarks Prefer empty() to size() == 0 to check if the scheduler is empty. + */ + size_t size() const + { + return m_container.size(); + } + + /*! Checks if the scheduler is empty. + * \return true if the scheduler contains no tasks, false otherwise. + * \remarks Is more efficient than size() == 0. + */ + bool empty() const + { + return m_container.empty(); + } + + /*! Removes all tasks from the scheduler. + */ + void clear() + { + while(!m_container.empty()) + { + m_container.pop(); + } + } + }; + + +} } // namespace boost::threadpool + + +#endif // THREADPOOL_SCHEDULING_POLICIES_HPP_INCLUDED + diff --git a/libs/libcommon/include/common/threadpool/shutdown_policies.hpp b/libs/libcommon/include/common/threadpool/shutdown_policies.hpp new file mode 100644 index 00000000000..047a6eb516d --- /dev/null +++ b/libs/libcommon/include/common/threadpool/shutdown_policies.hpp @@ -0,0 +1,83 @@ +/*! \file +* \brief Shutdown policies. +* +* This file contains shutdown policies for thread_pool. +* A shutdown policy controls the pool's behavior from the time +* when the pool is not referenced any longer. +* +* Copyright (c) 2005-2007 Philipp Henkel +* +* Use, modification, and distribution are subject to the +* Boost Software License, Version 1.0. (See accompanying file +* LICENSE_1_0.txt or copy at http://www.boost.org/LICENSE_1_0.txt) +* +* http://threadpool.sourceforge.net +* +*/ + + +#ifndef THREADPOOL_SHUTDOWN_POLICIES_HPP_INCLUDED +#define THREADPOOL_SHUTDOWN_POLICIES_HPP_INCLUDED + + + +/// The namespace threadpool contains a thread pool and related utility classes. +namespace boost { namespace threadpool +{ + + +/*! \brief ShutdownPolicy which waits for the completion of all tasks + * and the worker termination afterwards. + * + * \param Pool The pool's core type. + */ + template + class wait_for_all_tasks + { + public: + static void shutdown(Pool& pool) + { + pool.wait(); + pool.terminate_all_workers(true); + } + }; + + + /*! \brief ShutdownPolicy which waits for the completion of all active tasks + * and the worker termination afterwards. + * + * \param Pool The pool's core type. + */ + template + class wait_for_active_tasks + { + public: + static void shutdown(Pool& pool) + { + pool.clear(); + pool.wait(); + pool.terminate_all_workers(true); + } + }; + + + /*! \brief ShutdownPolicy which does not wait for any tasks or worker termination. + * + * This policy does not wait for any tasks. Nevertheless all active tasks will be processed completely. + * + * \param Pool The pool's core type. + */ + template + class immediately + { + public: + static void shutdown(Pool& pool) + { + pool.clear(); + pool.terminate_all_workers(false); + } + }; + +} } // namespace boost::threadpool + +#endif // THREADPOOL_SHUTDOWN_POLICIES_HPP_INCLUDED diff --git a/libs/libcommon/include/common/threadpool/size_policies.hpp b/libs/libcommon/include/common/threadpool/size_policies.hpp new file mode 100644 index 00000000000..e3c08038db5 --- /dev/null +++ b/libs/libcommon/include/common/threadpool/size_policies.hpp @@ -0,0 +1,99 @@ +/*! \file +* \brief Size policies. +* +* This file contains size policies for thread_pool. A size +* policy controls the number of worker threads in the pool. +* +* Copyright (c) 2005-2007 Philipp Henkel +* +* Use, modification, and distribution are subject to the +* Boost Software License, Version 1.0. (See accompanying file +* LICENSE_1_0.txt or copy at http://www.boost.org/LICENSE_1_0.txt) +* +* http://threadpool.sourceforge.net +* +*/ + + +#ifndef THREADPOOL_SIZE_POLICIES_HPP_INCLUDED +#define THREADPOOL_SIZE_POLICIES_HPP_INCLUDED + + + +/// The namespace threadpool contains a thread pool and related utility classes. +namespace boost { namespace threadpool +{ + + /*! \brief SizePolicyController which provides no functionality. + * + * \param Pool The pool's core type. + */ + template + struct empty_controller + { + empty_controller(typename Pool::size_policy_type&, shared_ptr) {} + }; + + + /*! \brief SizePolicyController which allows resizing. + * + * \param Pool The pool's core type. + */ + template< typename Pool > + class resize_controller + { + typedef typename Pool::size_policy_type size_policy_type; + reference_wrapper m_policy; + shared_ptr m_pool; //!< to make sure that the pool is alive (the policy pointer is valid) as long as the controller exists + + public: + resize_controller(size_policy_type& policy, shared_ptr pool) + : m_policy(policy) + , m_pool(pool) + { + } + + bool resize(size_t worker_count) + { + return m_policy.get().resize(worker_count); + } + }; + + + /*! \brief SizePolicy which preserves the thread count. + * + * \param Pool The pool's core type. + */ + template + class static_size + { + reference_wrapper m_pool; + + public: + static void init(Pool& pool, size_t const worker_count) + { + pool.resize(worker_count); + } + + static_size(Pool volatile & pool) + : m_pool(pool) + {} + + bool resize(size_t const worker_count) + { + return m_pool.get().resize(worker_count); + } + + void worker_died_unexpectedly(size_t const new_worker_count) + { + m_pool.get().resize(new_worker_count + 1); + } + + // TODO this functions are not called yet + void task_scheduled() {} + void task_finished() {} + }; + +} } // namespace boost::threadpool + +#endif // THREADPOOL_SIZE_POLICIES_HPP_INCLUDED diff --git a/libs/libcommon/include/common/threadpool/task_adaptors.hpp b/libs/libcommon/include/common/threadpool/task_adaptors.hpp new file mode 100644 index 00000000000..7e2dcca51ff --- /dev/null +++ b/libs/libcommon/include/common/threadpool/task_adaptors.hpp @@ -0,0 +1,176 @@ +/*! \file +* \brief Task adaptors. +* +* This file contains adaptors for task function objects. +* +* Copyright (c) 2005-2007 Philipp Henkel +* +* Use, modification, and distribution are subject to the +* Boost Software License, Version 1.0. (See accompanying file +* LICENSE_1_0.txt or copy at http://www.boost.org/LICENSE_1_0.txt) +* +* http://threadpool.sourceforge.net +* +*/ + + +#ifndef THREADPOOL_TASK_ADAPTERS_HPP_INCLUDED +#define THREADPOOL_TASK_ADAPTERS_HPP_INCLUDED + +#include + +#if BOOST_VERSION >= 105000 + #ifndef TIME_UTC + #define TIME_UTC TIME_UTC_ + #endif +#endif + + +#include +#include +#include + + +namespace boost { namespace threadpool +{ + + /*! \brief Standard task function object. + * + * This function object wraps a nullary function which returns void. + * The wrapped function is invoked by calling the operator (). + * + * \see boost function library + * + */ + typedef function0 task_func; + + + + + /*! \brief Prioritized task function object. + * + * This function object wraps a task_func object and binds a priority to it. + * prio_task_funcs can be compared using the operator < which realises a partial ordering. + * The wrapped task function is invoked by calling the operator (). + * + * \see prio_scheduler + * + */ + class prio_task_func + { + private: + unsigned int m_priority; //!< The priority of the task's function. + task_func m_function; //!< The task's function. + + public: + typedef void result_type; //!< Indicates the functor's result type. + + public: + /*! Constructor. + * \param priority The priority of the task. + * \param function The task's function object. + */ + prio_task_func(unsigned int const priority, task_func const & function) + : m_priority(priority) + , m_function(function) + { + } + + /*! Executes the task function. + */ + void operator() (void) const + { + if(m_function) + { + m_function(); + } + } + + /*! Comparison operator which realises a partial ordering based on priorities. + * \param rhs The object to compare with. + * \return true if the priority of *this is less than right hand side's priority, false otherwise. + */ + bool operator< (const prio_task_func& rhs) const + { + return m_priority < rhs.m_priority; + } + + }; // prio_task_func + + + + + + + + + /*! \brief Looped task function object. + * + * This function object wraps a boolean thread function object. + * The wrapped task function is invoked by calling the operator () and it is executed in regular + * time intervals until false is returned. The interval length may be zero. + * Please note that a pool's thread is engaged as long as the task is looped. + * + */ + class looped_task_func + { + private: + function0 m_function; //!< The task's function. + unsigned int m_break_s; //!< Duration of breaks in seconds. + unsigned int m_break_ns; //!< Duration of breaks in nano seconds. + + public: + typedef void result_type; //!< Indicates the functor's result type. + + public: + /*! Constructor. + * \param function The task's function object which is looped until false is returned. + * \param interval The minimum break time in milli seconds before the first execution of the task function and between the following ones. + */ + looped_task_func(function0 const & function, unsigned int const interval = 0) + : m_function(function) + { + m_break_s = interval / 1000; + m_break_ns = (interval - m_break_s * 1000) * 1000 * 1000; + } + + /*! Executes the task function. + */ + void operator() (void) const + { + if(m_function) + { + if(m_break_s > 0 || m_break_ns > 0) + { // Sleep some time before first execution + xtime xt; + xtime_get(&xt, TIME_UTC); + xt.nsec += m_break_ns; + xt.sec += m_break_s; + thread::sleep(xt); + } + + while(m_function()) + { + if(m_break_s > 0 || m_break_ns > 0) + { + xtime xt; + xtime_get(&xt, TIME_UTC); + xt.nsec += m_break_ns; + xt.sec += m_break_s; + thread::sleep(xt); + } + else + { + thread::yield(); // Be fair to other threads + } + } + } + } + + }; // looped_task_func + + +} } // namespace boost::threadpool + +#endif // THREADPOOL_TASK_ADAPTERS_HPP_INCLUDED + diff --git a/libs/libcommon/include/ext/function_traits.hpp b/libs/libcommon/include/ext/function_traits.hpp new file mode 100644 index 00000000000..31555a7f69f --- /dev/null +++ b/libs/libcommon/include/ext/function_traits.hpp @@ -0,0 +1,16 @@ +#pragma once + +#include +#include + + +template +struct function_traits; + +template +struct function_traits +{ + using result = ReturnType; + using arguments = std::tuple; + using arguments_decay = std::tuple::type...>; +}; diff --git a/libs/libcommon/include/ext/map.hpp b/libs/libcommon/include/ext/map.hpp new file mode 100644 index 00000000000..6a7941614ec --- /dev/null +++ b/libs/libcommon/include/ext/map.hpp @@ -0,0 +1,86 @@ +#pragma once + +#include +#include + +namespace ext +{ + /// \brief Strip type off top level reference and cv-qualifiers thus allowing storage in containers + template + using unqualified_t = std::remove_cv_t>; + + template + using apply_t = typename std::result_of::type; + + template + struct map_iterator : std::iterator< + typename It::iterator_category, + std::remove_reference_t>, + std::ptrdiff_t, + std::add_pointer_t>>, + apply_t> + { + using base_iterator = std::iterator< + typename It::iterator_category, + std::remove_reference_t>, + std::ptrdiff_t, + std::add_pointer>>, + apply_t>; + + It current; + Mapper mapper; + + map_iterator(const It it, const Mapper mapper) : current{it}, mapper{mapper} {} + + typename base_iterator::reference operator*() { return mapper(*current); } + + map_iterator & operator++() { return ++current, *this; } + map_iterator & operator--() { return --current, *this; } + + bool operator==(const map_iterator & other) { return current == other.current; } + bool operator!=(const map_iterator & other) { return current != other.current; } + + typename base_iterator::difference_type operator-(const map_iterator & other) { return current - other.current; } + }; + + template + auto make_map_iterator(const It it, const Mapper mapper) -> ext::map_iterator + { + return { it, mapper }; + } + + /** \brief Returns collection of the same container-type as the input collection, + * with each element transformed by the application of `mapper`. */ + template