This commit is contained in:
Pavel Kartavyy 2015-10-08 16:21:23 +03:00
commit 2163af290a
243 changed files with 6584 additions and 1199 deletions

View File

@ -15,7 +15,7 @@
#include <DB/Columns/ColumnArray.h> #include <DB/Columns/ColumnArray.h>
#include <stats/IntHash.h> #include <stats/IntHash.h>
#include <statdaemons/ext/range.hpp> #include <ext/range.hpp>
namespace DB namespace DB

View File

@ -4,7 +4,7 @@
#include <DB/DataTypes/DataTypeDateTime.h> #include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/DataTypes/DataTypesNumberFixed.h> #include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/Parsers/CommonParsers.h> #include <DB/Parsers/CommonParsers.h>
#include <statdaemons/ext/range.hpp> #include <ext/range.hpp>
#include <boost/range/iterator_range_core.hpp> #include <boost/range/iterator_range_core.hpp>
#include <DB/Parsers/ExpressionElementParsers.h> #include <DB/Parsers/ExpressionElementParsers.h>
#include <DB/Parsers/ASTLiteral.h> #include <DB/Parsers/ASTLiteral.h>

View File

@ -1,6 +1,6 @@
#pragma once #pragma once
#include <statdaemons/PoolBase.h> #include <DB/Common/PoolBase.h>
#include <DB/Client/Connection.h> #include <DB/Client/Connection.h>

View File

@ -3,7 +3,7 @@
#include <Poco/Net/NetException.h> #include <Poco/Net/NetException.h>
#include <Poco/Net/DNS.h> #include <Poco/Net/DNS.h>
#include <statdaemons/PoolWithFailoverBase.h> #include <DB/Common/PoolWithFailoverBase.h>
#include <DB/Common/getFQDNOrHostName.h> #include <DB/Common/getFQDNOrHostName.h>
#include <DB/Client/ConnectionPool.h> #include <DB/Client/ConnectionPool.h>

View File

@ -182,6 +182,16 @@ public:
throw Exception("Method insertDefault is not supported for ColumnAggregateFunction.", ErrorCodes::NOT_IMPLEMENTED); 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 size_t byteSize() const override
{ {
return getData().size() * sizeof(getData()[0]); return getData().size() * sizeof(getData()[0]);

View File

@ -4,8 +4,9 @@
#include <Poco/SharedPtr.h> #include <Poco/SharedPtr.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/Common/Arena.h>
#include <DB/Columns/IColumn.h> #include <DB/Columns/IColumn.h>
#include <DB/Columns/ColumnsNumber.h> #include <DB/Columns/ColumnsNumber.h>
@ -119,6 +120,33 @@ public:
getOffsets().push_back((getOffsets().size() == 0 ? 0 : getOffsets().back()) + elems); 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<const size_t *>(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; ColumnPtr cut(size_t start, size_t length) const override;
void insert(const Field & x) override void insert(const Field & x) override

View File

@ -3,7 +3,7 @@
#include <Poco/SharedPtr.h> #include <Poco/SharedPtr.h>
#include <DB/Core/Field.h> #include <DB/Core/Field.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/Columns/ColumnVector.h> #include <DB/Columns/ColumnVector.h>
#include <DB/Columns/IColumn.h> #include <DB/Columns/IColumn.h>
@ -89,6 +89,16 @@ public:
void insertDefault() override { ++s; } 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 ColumnPtr filter(const Filter & filt) const override
{ {
if (s != filt.size()) if (s != filt.size())

View File

@ -3,6 +3,7 @@
#include <string.h> // memcpy #include <string.h> // memcpy
#include <DB/Common/PODArray.h> #include <DB/Common/PODArray.h>
#include <DB/Common/Arena.h>
#include <DB/Columns/IColumn.h> #include <DB/Columns/IColumn.h>
@ -111,6 +112,21 @@ public:
chars.resize_fill(chars.size() + n); 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 int compareAt(size_t p1, size_t p2, const IColumn & rhs_, int nan_direction_hint) const override
{ {
const ColumnFixedString & rhs = static_cast<const ColumnFixedString &>(rhs_); const ColumnFixedString & rhs = static_cast<const ColumnFixedString &>(rhs_);

View File

@ -1,314 +0,0 @@
#pragma once
#include <string.h> // memcpy
#include <Poco/SharedPtr.h>
#include <DB/Core/Defines.h>
#include <DB/Core/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/Columns/IColumn.h>
#include <DB/Columns/ColumnsNumber.h>
namespace DB
{
using Poco::SharedPtr;
/** Cтолбeц значений типа вложенная таблица.
* В памяти это выглядит, как столбцы вложенных типов одинковой длины, равной сумме размеров всех массивов с общим именем,
* и как общий для всех столбцов массив смещений, который позволяет достать каждый элемент.
*
* Не предназначен для возвращения результа в запросах SELECT. Предполагается, что для SELECT'а будут отдаваться
* столбцы вида ColumnArray, ссылающиеся на один массив Offset'ов и соответствующий массив с данными.
*
* Используется для сериализации вложенной таблицы.
*/
class ColumnNested final : public IColumn
{
public:
/** По индексу i находится смещение до начала i + 1 -го элемента. */
typedef ColumnVector<Offset_t> 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<ColumnOffsets_t *>(&*offsets_column))
throw Exception("offsets_column must be a ColumnVector<UInt64>", 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<const ColumnNested &>(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<ColumnOffsets_t &>(*offsets.get()).getData();
}
const Offsets_t & ALWAYS_INLINE getOffsets() const
{
return static_cast<const ColumnOffsets_t &>(*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]); }
};
}

View File

@ -7,6 +7,7 @@
#include <DB/Columns/IColumn.h> #include <DB/Columns/IColumn.h>
#include <DB/Common/Collator.h> #include <DB/Common/Collator.h>
#include <DB/Common/PODArray.h> #include <DB/Common/PODArray.h>
#include <DB/Common/Arena.h>
namespace DB namespace DB
@ -115,6 +116,34 @@ public:
offsets.push_back((offsets.size() == 0 ? 0 : offsets.back()) + length); 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<const size_t *>(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 ColumnPtr cut(size_t start, size_t length) const override
{ {
if (length == 0) if (length == 0)

View File

@ -42,8 +42,8 @@ public:
{ {
Array res; Array res;
for (Columns::const_iterator it = columns.begin(); it != columns.end(); ++it) for (const auto & column : columns)
res.push_back((**it)[n]); res.push_back((*column)[n]);
return res; return res;
} }
@ -93,8 +93,26 @@ public:
void insertDefault() override void insertDefault() override
{ {
for (Columns::iterator it = columns.begin(); it != columns.end(); ++it) for (auto & column : columns)
(*it)->insertDefault(); 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) Less(const Columns & columns)
{ {
for (Columns::const_iterator it = columns.begin(); it != columns.end(); ++it) for (const auto & column : columns)
plain_columns.push_back(&**it); plain_columns.push_back(column.get());
} }
bool operator() (size_t a, size_t b) const bool operator() (size_t a, size_t b) const
@ -201,15 +219,15 @@ public:
void reserve(size_t n) override void reserve(size_t n) override
{ {
for (Columns::iterator it = columns.begin(); it != columns.end(); ++it) for (auto & column : columns)
(*it)->reserve(n); column->reserve(n);
} }
size_t byteSize() const override size_t byteSize() const override
{ {
size_t res = 0; size_t res = 0;
for (Columns::const_iterator it = columns.begin(); it != columns.end(); ++it) for (const auto & column : columns)
res += (*it)->byteSize(); res += column->byteSize();
return res; return res;
} }

View File

@ -2,8 +2,9 @@
#include <string.h> #include <string.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/Common/Arena.h>
#include <DB/Columns/IColumn.h> #include <DB/Columns/IColumn.h>
@ -150,6 +151,19 @@ public:
data.push_back(T()); 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<const T *>(pos));
return pos + sizeof(T);
}
size_t byteSize() const override size_t byteSize() const override
{ {
return data.size() * sizeof(data[0]); return data.size() * sizeof(data[0]);

View File

@ -6,7 +6,7 @@
#include <DB/Common/typeid_cast.h> #include <DB/Common/typeid_cast.h>
#include <DB/Core/Field.h> #include <DB/Core/Field.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/Core/StringRef.h> #include <DB/Core/StringRef.h>
@ -22,6 +22,8 @@ typedef std::vector<ColumnPtr> Columns;
typedef std::vector<IColumn *> ColumnPlainPtrs; typedef std::vector<IColumn *> ColumnPlainPtrs;
typedef std::vector<const IColumn *> ConstColumnPlainPtrs; typedef std::vector<const IColumn *> ConstColumnPlainPtrs;
class Arena;
/** Интерфейс для хранения столбцов значений в оперативке. /** Интерфейс для хранения столбцов значений в оперативке.
*/ */
@ -134,6 +136,20 @@ public:
*/ */
virtual void insertDefault() = 0; 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;
/** Соединить столбец с одним или несколькими другими. /** Соединить столбец с одним или несколькими другими.
* Используется при склейке маленьких блоков. * Используется при склейке маленьких блоков.
*/ */

View File

@ -29,6 +29,16 @@ public:
StringRef getDataAt(size_t n) const override { throw Exception("Method getDataAt 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); } 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 void getExtremes(Field & min, Field & max) const override
{ {
throw Exception("Method getExtremes is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); throw Exception("Method getExtremes is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);

View File

@ -0,0 +1,52 @@
#pragma once
#include <linux/aio_abi.h>
#include <unistd.h>
#include <sys/syscall.h>
#include <boost/noncopyable.hpp>
#include <DB/Common/Exception.h>
/** Небольшие обёртки для асинхронного ввода-вывода.
*/
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);
}
};

View File

@ -5,7 +5,7 @@
#include <sys/mman.h> #include <sys/mman.h>
#include <DB/Common/MemoryTracker.h> #include <DB/Common/MemoryTracker.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>

View File

@ -5,6 +5,7 @@
#include <vector> #include <vector>
#include <Poco/SharedPtr.h> #include <Poco/SharedPtr.h>
#include <common/likely.h> #include <common/likely.h>
#include <DB/Core/Defines.h>
#include <DB/Common/ProfileEvents.h> #include <DB/Common/ProfileEvents.h>
#include <DB/Common/Allocator.h> #include <DB/Common/Allocator.h>
@ -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); head = new Chunk(nextSize(min_size), head);
size_in_bytes += head->size(); size_in_bytes += head->size();
@ -127,16 +128,15 @@ public:
*/ */
char * allocContinue(size_t size, char const *& begin) 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; char * prev_end = head->pos;
addChunk(size); addChunk(size);
if (begin) if (begin)
{
begin = insert(begin, prev_end - begin); begin = insert(begin, prev_end - begin);
return allocContinue(size, begin); else
} break;
} }
char * res = head->pos; char * res = head->pos;

View File

@ -3,7 +3,7 @@
#include <unicode/ucol.h> #include <unicode/ucol.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/IO/WriteHelpers.h> #include <DB/IO/WriteHelpers.h>
#include <common/Common.h> #include <common/Common.h>

View File

@ -2,7 +2,7 @@
#include <DB/Common/HashTable/SmallTable.h> #include <DB/Common/HashTable/SmallTable.h>
#include <DB/Common/HashTable/HashSet.h> #include <DB/Common/HashTable/HashSet.h>
#include <statdaemons/HyperLogLogCounter.h> #include <DB/Common/HyperLogLogCounter.h>
#include <DB/Core/Defines.h> #include <DB/Core/Defines.h>

View File

@ -0,0 +1,65 @@
#pragma once
#include <string>
#include <Poco/DOM/Document.h>
#include <Poco/DOM/DOMParser.h>
#include <Poco/DOM/DOMWriter.h>
#include <Poco/DOM/NodeList.h>
#include <Poco/DOM/NamedNodeMap.h>
#include <Poco/AutoPtr.h>
#include <Poco/File.h>
#include <Poco/Path.h>
#include <Poco/DirectoryIterator.h>
#include <Poco/ConsoleChannel.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <common/logger_useful.h>
typedef Poco::AutoPtr<Poco::Util::AbstractConfiguration> ConfigurationPtr;
typedef Poco::AutoPtr<Poco::XML::Document> XMLDocumentPtr;
class ConfigProcessor
{
public:
using Substitutions = std::vector<std::pair<std::string, std::string> >;
/// 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) Берем из конфига путь к файлу, из которого будем делать подстановки: <include_from>/path2/metrika.xml</include_from>.
* Если путь не указан, используем /etc/metrika.xml
* 3) Заменяем элементы вида "<foo incl="bar"/>" на "<foo>содержимое элемента yandex.bar из metrika.xml</foo>"
* 4) Заменяет "<layer/>" на "<layer>номер слоя из имени хоста</layer>"
*/
XMLDocumentPtr processConfig(const std::string & path);
/** Делает processConfig и создает из результата Poco::Util::XMLConfiguration.
* Еще сохраняет результат в файл по пути, полученному из path приписыванием строки "-preprocessed" к имени файла.
*/
ConfigurationPtr loadConfig(const std::string & path);
private:
Logger * log;
Poco::AutoPtr<Poco::Channel> channel_ptr;
bool throw_on_bad_incl;
Substitutions substitutions;
typedef XMLDocumentPtr DocumentPtr;
typedef Poco::AutoPtr<Poco::XML::Node> 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);
};

View File

@ -0,0 +1,188 @@
#pragma once
#include <fcntl.h>
#include <sys/file.h>
#include <string>
#include <iostream>
#include <Poco/File.h>
#include <Poco/Exception.h>
#include <Poco/Mutex.h>
#include <Poco/ScopedLock.h>
#include <boost/function.hpp>
#include <DB/Common/Exception.h>
#include <DB/IO/ReadBufferFromFileDescriptor.h>
#include <DB/IO/WriteBufferFromFileDescriptor.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/IO/WriteHelpers.h>
#include <common/Common.h>
#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 <typename Callback>
Int64 add(Int64 delta, Callback && locked_callback, bool create_if_need = false)
{
Poco::ScopedLock<Poco::FastMutex> 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

View File

@ -0,0 +1,113 @@
#pragma once
#include <cerrno>
#include <vector>
#include <Poco/Exception.h>
#include <Poco/SharedPtr.h>
#include <DB/Common/StackTrace.h>
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<std::exception_ptr> 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<Poco::Exception> convertCurrentException();
template <typename T>
typename std::enable_if<std::is_pointer<T>::value, T>::type exception_cast(std::exception_ptr e)
{
try
{
std::rethrow_exception(e);
}
catch (typename std::remove_pointer<T>::type & concrete)
{
return &concrete;
}
catch (...)
{
return nullptr;
}
}
}

View File

@ -14,7 +14,7 @@
#include <Poco/Net/HTMLForm.h> #include <Poco/Net/HTMLForm.h>
#include <Poco/Net/PartHandler.h> #include <Poco/Net/PartHandler.h>
#include <Poco/Net/MessageHeader.h> #include <Poco/Net/MessageHeader.h>
#include <statdaemons/HTMLForm.h> #include <DB/Common/HTMLForm.h>
namespace DB namespace DB

View File

@ -0,0 +1,26 @@
#pragma once
#include <sstream>
#include <Poco/Net/HTMLForm.h>
#include <Poco/Net/HTTPRequest.h>
#include <Poco/URI.h>
/** Почему-то при методе 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);
}
};

View File

@ -15,7 +15,7 @@
#include <DB/Core/Defines.h> #include <DB/Core/Defines.h>
#include <DB/Core/Types.h> #include <DB/Core/Types.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/IO/WriteBuffer.h> #include <DB/IO/WriteBuffer.h>
@ -29,7 +29,7 @@
#ifdef DBMS_HASH_MAP_DEBUG_RESIZES #ifdef DBMS_HASH_MAP_DEBUG_RESIZES
#include <iostream> #include <iostream>
#include <iomanip> #include <iomanip>
#include <statdaemons/Stopwatch.h> #include <DB/Common/Stopwatch.h>
#endif #endif

View File

@ -5,7 +5,7 @@
#include <sys/mman.h> #include <sys/mman.h>
#include <DB/Common/MemoryTracker.h> #include <DB/Common/MemoryTracker.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
/** При использовании HashTableAllocatorWithStackMemory, размещённом на стеке, /** При использовании HashTableAllocatorWithStackMemory, размещённом на стеке,

View File

@ -0,0 +1,109 @@
#pragma once
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <algorithm>
#include <limits>
#include <tuple>
#include <type_traits>
/** Этот класс предоставляет способ, чтобы оценить погрешность результата применения алгоритма 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 <typename BiasData>
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<TRawEstimatesRef>::type;
using TBiasDataRef = decltype(BiasData::getBiases());
using TBiasData = typename std::remove_reference<TBiasDataRef>::type;
static_assert(std::is_same<TRawEstimates, TBiasData>::value, "Bias estimator data have inconsistent types");
static_assert(std::tuple_size<TRawEstimates>::value > 0, "Bias estimator has no raw estimate data");
static_assert(std::tuple_size<TBiasData>::value > 0, "Bias estimator has no bias data");
static_assert(std::tuple_size<TRawEstimates>::value == std::tuple_size<TBiasData>::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;
}
};

View File

@ -0,0 +1,727 @@
#pragma once
#include <common/Common.h>
#include <stats/IntHash.h>
#include <DB/Common/HyperLogLogBiasEstimator.h>
#include <DB/IO/ReadBuffer.h>
#include <DB/IO/WriteBuffer.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/IO/WriteHelpers.h>
#include <DB/Core/Defines.h>
#include <cmath>
#include <cstring>
namespace details
{
/// Look-up table логарифмов от целых чисел для использования в HyperLogLogCounter.
template<UInt8 K>
struct LogLUT
{
LogLUT()
{
log_table[0] = 0.0;
for (size_t i = 1; i <= M; ++i)
log_table[i] = log(static_cast<double>(i));
}
double getLog(size_t x) const
{
if (x <= M)
return log_table[x];
else
return log(static_cast<double>(x));
}
private:
static constexpr size_t M = 1 << ((static_cast<unsigned int>(K) <= 12) ? K : 12);
double log_table[M + 1];
};
template<UInt8 K> 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<UInt64 MaxValue> 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<typename BucketIndex, UInt8 content_width, size_t bucket_count>
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<size_t>(bucket_count) * content_width + 7) / 8;
UInt8 bitset[BITSET_SIZE] = { 0 };
};
/** Структура Locus содержит необходимую информацию, чтобы найти для каждой компактной ячейки
* соответствующие физическую ячейку и смещение, в битах, от начала ячейки. Поскольку в общем
* случае размер одной физической ячейки не делится на размер одной компактной ячейки, возможны
* случаи, когда одна компактная ячейка перекрывает две физические ячейки. Поэтому структура
* Locus содержит две пары (индекс, смещение).
*/
template<typename BucketIndex, UInt8 content_width, size_t bucket_count>
class CompactArray<BucketIndex, content_width, bucket_count>::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<size_t>(bucket_index) * content_width;
index_l = l >> 3;
offset_l = l & 7;
size_t r = static_cast<size_t>(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<size_t>::max() / content_width), "Invalid parameter value");
};
/** Знаменатель формулы алгоритма HyperLogLog
*/
template<UInt8 precision, int max_rank, typename HashValueType, typename DenominatorType,
bool stable_denominator_if_big, typename Enable = void>
class __attribute__ ((packed)) Denominator;
namespace
{
/// Возвращает true, если хранилище для рангов большое.
constexpr bool isBigRankStore(UInt8 precision)
{
return precision >= 12;
}
}
/** Тип употребляемый для вычисления знаменателя.
*/
template <typename HashValueType>
struct IntermediateDenominator;
template <>
struct IntermediateDenominator<UInt32>
{
using Type = double;
};
template <>
struct IntermediateDenominator<UInt64>
{
using Type = long double;
};
/** "Лёгкая" реализация знаменателя формулы HyperLogLog.
* Занимает минимальный объём памяти, зато вычисления могут быть неустойчивы.
* Подходит, когда хранилище для рангов небольшое.
*/
template<UInt8 precision, int max_rank, typename HashValueType, typename DenominatorType,
bool stable_denominator_if_big>
class __attribute__ ((packed)) Denominator<precision, max_rank, HashValueType, DenominatorType,
stable_denominator_if_big,
typename std::enable_if<!details::isBigRankStore(precision) || !stable_denominator_if_big>::type>
{
private:
using T = typename IntermediateDenominator<HashValueType>::Type;
public:
Denominator(DenominatorType initial_value)
: denominator(initial_value)
{
}
public:
inline void update(UInt8 cur_rank, UInt8 new_rank)
{
denominator -= static_cast<T>(1.0) / (1ULL << cur_rank);
denominator += static_cast<T>(1.0) / (1ULL << new_rank);
}
inline void update(UInt8 rank)
{
denominator += static_cast<T>(1.0) / (1ULL << rank);
}
void clear()
{
denominator = 0;
}
DenominatorType get() const
{
return denominator;
}
private:
T denominator;
};
/** "Тяжёлая" версия знаменателя формулы HyperLogLog.
* Занимает больший объём памяти, чем лёгкая версия, зато вычисления всегда устойчивы.
* Подходит, когда хранилище для рангов довольно большое.
*/
template<UInt8 precision, int max_rank, typename HashValueType, typename DenominatorType,
bool stable_denominator_if_big>
class __attribute__ ((packed)) Denominator<precision, max_rank, HashValueType, DenominatorType,
stable_denominator_if_big,
typename std::enable_if<details::isBigRankStore(precision) && stable_denominator_if_big>::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 <typename T>
struct TrailingZerosCounter;
template <>
struct TrailingZerosCounter<UInt32>
{
static int apply(UInt32 val)
{
return __builtin_ctz(val);
}
};
template <>
struct TrailingZerosCounter<UInt64>
{
static int apply(UInt64 val)
{
return __builtin_ctzll(val);
}
};
/** Размер счётчика ранга в битах.
*/
template <typename T>
struct RankWidth;
template <>
struct RankWidth<UInt32>
{
static constexpr UInt8 get()
{
return 5;
}
};
template <>
struct RankWidth<UInt64>
{
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<UInt64>,
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<HashValueType>::get();
private:
using Value_t = UInt64;
using RankStore = details::CompactArray<HashValueType, rank_width, bucket_count>;
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<UInt32>(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<char *>(this), sizeof(*this));
}
void readAndMerge(DB::ReadBuffer & in)
{
RankStore other;
in.readStrict(reinterpret_cast<char *>(&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<const char *>(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<HashValueType>::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<UInt8>(content);
if (rank > cur_rank)
{
if (cur_rank == 0)
--zeros;
denominator.update(cur_rank, rank);
content = rank;
}
}
template <HyperLogLogMode mode2 = mode>
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 <HyperLogLogMode mode2 = mode>
inline double fixRawEstimate(double raw_estimate,
typename std::enable_if<(mode2 == HyperLogLogMode::LinearCounting)>::type * = nullptr) const
{
return applyLinearCorrection(raw_estimate);
}
template <HyperLogLogMode mode2 = mode>
inline double fixRawEstimate(double raw_estimate,
typename std::enable_if<(mode2 == HyperLogLogMode::BiasCorrected)
&& !BiasEstimator::isTrivial()>::type * = nullptr) const
{
return applyBiasCorrection(raw_estimate);
}
template <HyperLogLogMode mode2 = mode>
double fixRawEstimate(double raw_estimate,
typename std::enable_if<(mode2 == HyperLogLogMode::FullFeatured)>::type * = nullptr) const
{
static constexpr bool fix_big_cardinalities = std::is_same<HashValueType, UInt32>::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 <bool is_trivial = BiasEstimator::isTrivial()>
inline double applyCorrection(double raw_estimate, typename std::enable_if<is_trivial>::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 <bool is_trivial = BiasEstimator::isTrivial()>
inline double applyCorrection(double raw_estimate, typename std::enable_if<!is_trivial>::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<precision, max_rank, HashValueType, DenominatorType, stable_denominator_if_big>;
DenominatorCalculatorType denominator{bucket_count};
/// Число нулей в хранилище для рангов.
using ZerosCounterType = typename details::MinCounterType<bucket_count>::Type;
ZerosCounterType zeros = bucket_count;
static details::LogLUT<precision> 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<precision> HyperLogLogCounter
<
precision,
Hash,
HashValueType,
DenominatorType,
BiasEstimator,
mode,
stable_denominator_if_big
>::log_lut;
/// Для Metrage, используется лёгкая реализация знаменателя формулы HyperLogLog,
/// чтобы формат сериализации не изменился.
typedef HyperLogLogCounter<
12,
IntHash32<UInt64>,
UInt32,
double,
TrivialBiasEstimator,
HyperLogLogMode::FullFeatured,
false
> HLL12;

View File

@ -1,6 +1,6 @@
#pragma once #pragma once
#include <statdaemons/HyperLogLogCounter.h> #include <DB/Common/HyperLogLogCounter.h>
#include <DB/Common/HashTable/SmallTable.h> #include <DB/Common/HashTable/SmallTable.h>
namespace DB namespace DB

View File

@ -0,0 +1,87 @@
#pragma once
#include <DB/Common/CounterInFile.h>
/** Позволяет получать авто-инкрементное число, храня его в файле.
* Предназначен для редких вызовов (не рассчитан на производительность).
*/
class Increment
{
public:
/// path - имя файла, включая путь
Increment(const std::string & path_) : counter(path_) {}
/** Получить следующее число.
* Если параметр create_if_need не установлен в true, то
* в файле уже должно быть записано какое-нибудь число (если нет - создайте файл вручную с нулём).
*
* Для защиты от race condition-ов между разными процессами, используются файловые блокировки.
* (Но при первом создании файла race condition возможен, так что лучше создать файл заранее.)
*
* locked_callback вызывается при заблокированном файле со счетчиком. В него передается новое значение.
* locked_callback можно использовать, чтобы делать что-нибудь атомарно с увеличением счетчика (например, переименовывать файлы).
*/
template <typename Callback>
UInt64 get(Callback && locked_callback, bool create_if_need = false)
{
return static_cast<UInt64>(counter.add(1, std::forward<Callback>(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<UInt64>(counter.add(static_cast<Int64>(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);
}
};

View File

@ -7,7 +7,7 @@
#include <Poco/ScopedLock.h> #include <Poco/ScopedLock.h>
#include <Poco/Mutex.h> #include <Poco/Mutex.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
namespace DB namespace DB
{ {

View File

@ -0,0 +1,27 @@
#pragma once
#include <DB/Common/Exception.h>
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; }
};
}

View File

@ -0,0 +1,106 @@
#pragma once
#include <string>
#include <vector>
#include <memory>
#include <Poco/SharedPtr.h>
#include <re2/re2.h>
#include <re2_st/re2.h>
/** Использует два способа оптимизации регулярного выражения:
* 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 <bool thread_safe>
class OptimizedRegularExpressionImpl
{
public:
enum Options
{
RE_CASELESS = 0x00000001,
RE_NO_CAPTURE = 0x00000010,
RE_DOT_NL = 0x00000100
};
using Match = OptimizedRegularExpressionDetails::Match;
typedef std::vector<Match> MatchVec;
using RegexType = typename std::conditional<thread_safe, re2::RE2, re2_st::RE2>::type;
using StringPieceType = typename std::conditional<thread_safe, re2::StringPiece, re2_st::StringPiece>::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<RegexType>& 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<RegexType> re2;
unsigned number_of_subpatterns;
};
using OptimizedRegularExpression = OptimizedRegularExpressionImpl<true>;
#include "OptimizedRegularExpression.inl"

View File

@ -0,0 +1,410 @@
#include <iostream>
#include <Poco/Exception.h>
#include <DB/Common/OptimizedRegularExpression.h>
#define MIN_LENGTH_FOR_STRSTR 3
#define MAX_SUBPATTERNS 5
template <bool b>
void OptimizedRegularExpressionImpl<b>::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<std::string, size_t> Substring;
typedef std::vector<Substring> 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 <bool b>
OptimizedRegularExpressionImpl<b>::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 b>
bool OptimizedRegularExpressionImpl<b>::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 b>
bool OptimizedRegularExpressionImpl<b>::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 <bool b>
unsigned OptimizedRegularExpressionImpl<b>::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

View File

@ -12,7 +12,7 @@
#include <common/strong_typedef.h> #include <common/strong_typedef.h>
#include <DB/Common/Allocator.h> #include <DB/Common/Allocator.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>

View File

@ -0,0 +1,146 @@
#pragma once
#include <Poco/SharedPtr.h>
#include <Poco/Mutex.h>
#include <Poco/Condition.h>
#include <Poco/Timespan.h>
#include <boost/noncopyable.hpp>
#include <common/logger_useful.h>
/** Класс, от которого можно унаследоваться и получить пул чего-нибудь. Используется для пулов соединений с БД.
* Наследник должен предоставить метод для создания нового объекта для помещения в пул.
*/
template <typename TObject>
class PoolBase : private boost::noncopyable
{
public:
typedef TObject Object;
typedef Poco::SharedPtr<Object> ObjectPtr;
typedef Poco::SharedPtr<PoolBase<TObject> > 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<Poco::SharedPtr<PooledObject> > 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<Object>;
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<PoolEntryHelper> data;
Entry(PooledObject & object) : data(new PoolEntryHelper(object)) {}
};
virtual ~PoolBase() {}
/** Выделяет объект для работы. При timeout < 0 таймаут бесконечный. */
Entry get(Poco::Timespan::TimeDiff timeout)
{
Poco::ScopedLock<Poco::FastMutex> 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<Poco::FastMutex> 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;
};

View File

@ -0,0 +1,311 @@
#pragma once
#include <time.h>
#include <DB/Common/PoolBase.h>
#include <DB/Common/ProfileEvents.h>
#include <DB/Common/NetException.h>
#include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/Interpreters/Settings.h>
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<size_t> handles;
size_t unallocated_size;
};
}
/** Класс, от которого можно унаследоваться и получить пул с отказоустойчивостью. Используется для пулов соединений с реплицированной БД.
* Инициализируется несколькими другими PoolBase-ами.
* При получении соединения, пытается создать или выбрать живое соединение из какого-нибудь пула,
* перебирая их в некотором порядке, используя не более указанного количества попыток.
* Пулы перебираются в порядке лексикографического возрастания тройки (приоритет, число ошибок, случайное число).
*
* Замечание: если один из вложенных пулов заблокируется из-за переполнения, то этот пул тоже заблокируется.
*
* Наследник должен предоставить метод, достающий соединение из вложенного пула.
* Еще наследник можнет назначать приоритеты вложенным пулам.
*/
template <typename TNestedPool>
class PoolWithFailoverBase : private boost::noncopyable
{
public:
typedef TNestedPool NestedPool;
typedef Poco::SharedPtr<NestedPool> NestedPoolPtr;
typedef typename NestedPool::Entry Entry;
typedef std::vector<NestedPoolPtr> 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<Entry> 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<Entry> 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, &times))
DB::throwFromErrno("Cannot clock_gettime.", DB::ErrorCodes::CANNOT_CLOCK_GETTIME);
srand48_r(reinterpret_cast<intptr_t>(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<typename PoolWithErrorCount::State>;
class PoolsWithErrorCount : public std::vector<PoolWithErrorCount>
{
public:
PoolsWithErrorCount(typename NestedPools::iterator begin_, typename NestedPools::iterator end_,
time_t decrease_error_period_)
: std::vector<PoolWithErrorCount>(begin_, end_),
decrease_error_period(decrease_error_period_)
{
}
/// Эта функция возвращает собственную копию состояния каждого пула, чтобы не возникло
/// состояния гонки при выделении соединений.
States update()
{
States states;
states.reserve(this->size());
{
Poco::ScopedLock<Poco::FastMutex> 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<IndexedPoolWithErrorCount>;
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;
}
};

View File

@ -3,7 +3,7 @@
#include <map> #include <map>
#include <tuple> #include <tuple>
#include <mutex> #include <mutex>
#include <statdaemons/ext/function_traits.hpp> #include <ext/function_traits.hpp>
/** Простейший кэш для свободной функции. /** Простейший кэш для свободной функции.

View File

@ -0,0 +1,23 @@
#pragma once
#include <string>
#include <vector>
#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;
};

View File

@ -0,0 +1,75 @@
#pragma once
#include <time.h>
#include <Poco/Mutex.h>
#include <Poco/ScopedLock.h>
#include <common/Common.h>
/** Отличается от 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<double>(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<Poco::FastMutex> lock(mutex);
if (elapsedSeconds() >= seconds)
{
restart();
return true;
}
else
return false;
}
private:
Poco::FastMutex mutex;
};

View File

@ -2,8 +2,8 @@
#include <mutex> #include <mutex>
#include <memory> #include <memory>
#include <statdaemons/Stopwatch.h> #include <DB/Common/Stopwatch.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/IO/WriteHelpers.h> #include <DB/IO/WriteHelpers.h>

View File

@ -8,7 +8,7 @@
#include <DB/Core/BlockInfo.h> #include <DB/Core/BlockInfo.h>
#include <DB/Core/ColumnWithTypeAndName.h> #include <DB/Core/ColumnWithTypeAndName.h>
#include <DB/Core/NamesAndTypes.h> #include <DB/Core/NamesAndTypes.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include "ColumnsWithTypeAndName.h" #include "ColumnsWithTypeAndName.h"

View File

@ -1,7 +1,7 @@
#pragma once #pragma once
#include <DB/Core/Types.h> #include <DB/Core/Types.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/IO/ReadBuffer.h> #include <DB/IO/ReadBuffer.h>
#include <DB/IO/WriteBuffer.h> #include <DB/IO/WriteBuffer.h>

View File

@ -252,7 +252,6 @@ namespace ErrorCodes
CORRUPTED_DATA = 246, CORRUPTED_DATA = 246,
INCORRECT_MARK = 247, INCORRECT_MARK = 247,
INVALID_PARTITION_NAME = 248, INVALID_PARTITION_NAME = 248,
NOT_LEADER = 249,
NOT_ENOUGH_BLOCK_NUMBERS = 250, NOT_ENOUGH_BLOCK_NUMBERS = 250,
NO_SUCH_REPLICA = 251, NO_SUCH_REPLICA = 251,
TOO_MUCH_PARTS = 252, TOO_MUCH_PARTS = 252,
@ -293,6 +292,8 @@ namespace ErrorCodes
DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED = 288, DISTRIBUTED_IN_JOIN_SUBQUERY_DENIED = 288,
REPLICA_IS_NOT_IN_QUORUM = 289, REPLICA_IS_NOT_IN_QUORUM = 289,
LIMIT_EXCEEDED = 290, LIMIT_EXCEEDED = 290,
DATABASE_ACCESS_DENIED = 291,
LEADERSHIP_CHANGED = 292,
KEEPER_EXCEPTION = 999, KEEPER_EXCEPTION = 999,
POCO_EXCEPTION = 1000, POCO_EXCEPTION = 1000,

View File

@ -1,40 +0,0 @@
#pragma once
#include <cerrno>
#include <vector>
#include <statdaemons/Exception.h>
#include <Poco/SharedPtr.h>
namespace Poco { class Logger; }
namespace DB
{
using Poco::SharedPtr;
typedef SharedPtr<Poco::Exception> ExceptionPtr;
typedef std::vector<ExceptionPtr> 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);
}

View File

@ -11,7 +11,7 @@
#include <mysqlxx/Manip.h> #include <mysqlxx/Manip.h>
#include <DB/Core/Types.h> #include <DB/Core/Types.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/IO/ReadHelpers.h> #include <DB/IO/ReadHelpers.h>
#include <DB/IO/WriteHelpers.h> #include <DB/IO/WriteHelpers.h>

View File

@ -1,6 +1,6 @@
#pragma once #pragma once
#include <statdaemons/threadpool.hpp> #include <common/threadpool.hpp>
#include <Poco/Event.h> #include <Poco/Event.h>
@ -48,7 +48,7 @@ public:
{ {
pool.wait(); pool.wait();
if (exception) if (exception)
exception->rethrow(); std::rethrow_exception(exception);
children.back()->readSuffix(); children.back()->readSuffix();
started = false; started = false;
} }
@ -82,7 +82,7 @@ protected:
bool started = false; bool started = false;
Block block; Block block;
ExceptionPtr exception; std::exception_ptr exception;
Block readImpl() override Block readImpl() override
@ -97,7 +97,7 @@ protected:
pool.wait(); pool.wait();
if (exception) if (exception)
exception->rethrow(); std::rethrow_exception(exception);
Block res = block; Block res = block;
if (!res) if (!res)
@ -130,7 +130,7 @@ protected:
} }
catch (...) catch (...)
{ {
exception = cloneCurrentException(); exception = std::current_exception();
} }
ready.set(); ready.set();

View File

@ -1,7 +1,7 @@
#pragma once #pragma once
#include <vector> #include <vector>
#include <statdaemons/Stopwatch.h> #include <DB/Common/Stopwatch.h>
#include <DB/Core/Types.h> #include <DB/Core/Types.h>

View File

@ -2,7 +2,7 @@
#include <DB/Columns/ColumnConst.h> #include <DB/Columns/ColumnConst.h>
#include <DB/DataStreams/IBlockOutputStream.h> #include <DB/DataStreams/IBlockOutputStream.h>
#include <statdaemons/ext/range.hpp> #include <ext/range.hpp>
namespace DB namespace DB

View File

@ -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.exceptions[thread_num] = exception;
parent.cancel(); parent.cancel();

View File

@ -38,7 +38,7 @@ struct ParallelInputsHandler
void onFinish() {} void onFinish() {}
/// Обработка исключения. Разумно вызывать в этом методе метод ParallelInputsProcessor::cancel, а также передавать эксепшен в основной поток. /// Обработка исключения. Разумно вызывать в этом методе метод 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) void thread(MemoryTracker * memory_tracker, size_t thread_num)
{ {
current_memory_tracker = memory_tracker; current_memory_tracker = memory_tracker;
ExceptionPtr exception; std::exception_ptr exception;
setThreadName("ParalInputsProc"); setThreadName("ParalInputsProc");
@ -150,7 +150,7 @@ private:
} }
catch (...) catch (...)
{ {
exception = cloneCurrentException(); exception = std::current_exception();
} }
if (exception) if (exception)
@ -171,7 +171,7 @@ private:
} }
catch (...) catch (...)
{ {
exception = cloneCurrentException(); exception = std::current_exception();
} }
if (exception) if (exception)

View File

@ -5,7 +5,7 @@
#include <DB/Client/Connection.h> #include <DB/Client/Connection.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <statdaemons/NetException.h> #include <DB/Common/NetException.h>
namespace DB namespace DB
{ {

View File

@ -6,8 +6,8 @@
#include <DB/Core/ColumnNumbers.h> #include <DB/Core/ColumnNumbers.h>
#include <DB/DataStreams/MergingSortedBlockInputStream.h> #include <DB/DataStreams/MergingSortedBlockInputStream.h>
#include <DB/Storages/MergeTree/PKCondition.h> #include <DB/Storages/MergeTree/PKCondition.h>
#include <statdaemons/ext/range.hpp> #include <ext/range.hpp>
#include <statdaemons/ext/map.hpp> #include <ext/map.hpp>
namespace DB namespace DB

View File

@ -97,7 +97,7 @@ protected:
LOG_TRACE(log, "Waiting for threads to finish"); LOG_TRACE(log, "Waiting for threads to finish");
ExceptionPtr exception; std::exception_ptr exception;
if (!all_read) if (!all_read)
{ {
/** Прочитаем всё до конца, чтобы ParallelInputsProcessor не заблокировался при попытке вставить в очередь. /** Прочитаем всё до конца, чтобы ParallelInputsProcessor не заблокировался при попытке вставить в очередь.
@ -113,8 +113,8 @@ protected:
{ {
if (!exception) if (!exception)
exception = res.exception; exception = res.exception;
else if (DB::Exception * e = dynamic_cast<DB::Exception *>(&*exception)) else if (Exception * e = exception_cast<Exception *>(exception))
e->addMessage("\n" + res.exception->displayText()); e->addMessage("\n" + getExceptionMessage(res.exception, false));
} }
else if (!res.block) else if (!res.block)
break; break;
@ -128,7 +128,7 @@ protected:
LOG_TRACE(log, "Waited for threads to finish"); LOG_TRACE(log, "Waited for threads to finish");
if (exception) if (exception)
exception->rethrow(); std::rethrow_exception(exception);
} }
/** Возможны следующие варианты: /** Возможны следующие варианты:
@ -157,7 +157,7 @@ protected:
output_queue.pop(res); output_queue.pop(res);
if (res.exception) if (res.exception)
res.exception->rethrow(); std::rethrow_exception(res.exception);
if (!res.block) if (!res.block)
all_read = true; all_read = true;
@ -183,11 +183,11 @@ private:
struct OutputData struct OutputData
{ {
Block block; Block block;
ExceptionPtr exception; std::exception_ptr exception;
OutputData() {} OutputData() {}
OutputData(Block & block_) : block(block_) {} 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()); 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"; //std::cerr << "pushing exception\n";

View File

@ -1,84 +1,39 @@
#pragma once #pragma once
#include <DB/DataTypes/IDataType.h> #include <DB/DataTypes/IDataTypeDummy.h>
#include <DB/Core/NamesAndTypes.h> #include <DB/Core/NamesAndTypes.h>
namespace DB namespace DB
{ {
using Poco::SharedPtr; /** Хранит набор пар (имя, тип) для вложенной структуры данных.
* Используется только при создании таблицы. Во всех остальных случаях не используется, так как раскрывается в набор отдельных столбцов с типами.
*/
class DataTypeNested final : public IDataType class DataTypeNested final : public IDataTypeDummy
{ {
private: private:
/// Имена и типы вложенных массивов. /// Имена и типы вложенных массивов.
NamesAndTypesListPtr nested; NamesAndTypesListPtr nested;
/// Тип смещений.
DataTypePtr offsets;
public: public:
DataTypeNested(NamesAndTypesListPtr nested_); DataTypeNested(NamesAndTypesListPtr nested_);
std::string getName() const override; 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 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 extractNestedTableName(const std::string & nested_name);
/// Возвращает суффикс имени после первой точки справа '.'. Или имя без изменений, если точки нет. /// Возвращает суффикс имени после первой точки справа '.'. Или имя без изменений, если точки нет.
static std::string extractNestedColumnName(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 заменены на несколько вида имя_колонки.имя_вложенной_ячейки /// Создает новый список в котором колонки типа Nested заменены на несколько вида имя_колонки.имя_вложенной_ячейки
static NamesAndTypesListPtr expandNestedColumns(const NamesAndTypesList & names_and_types); static NamesAndTypesListPtr expandNestedColumns(const NamesAndTypesList & names_and_types);
}; };

View File

@ -4,7 +4,7 @@
#include <DB/DataTypes/DataTypeString.h> #include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeArray.h> #include <DB/DataTypes/DataTypeArray.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>

View File

@ -6,7 +6,7 @@
#include <DB/Common/HashTable/HashMap.h> #include <DB/Common/HashTable/HashMap.h>
#include <DB/Columns/ColumnString.h> #include <DB/Columns/ColumnString.h>
#include <DB/Common/HashTable/HashMap.h> #include <DB/Common/HashTable/HashMap.h>
#include <statdaemons/ext/scope_guard.hpp> #include <ext/scope_guard.hpp>
#include <Poco/RWLock.h> #include <Poco/RWLock.h>
#include <cmath> #include <cmath>
#include <atomic> #include <atomic>

View File

@ -5,7 +5,7 @@
#include <DB/DataStreams/RemoteBlockInputStream.h> #include <DB/DataStreams/RemoteBlockInputStream.h>
#include <DB/Interpreters/executeQuery.h> #include <DB/Interpreters/executeQuery.h>
#include <DB/Common/isLocalAddress.h> #include <DB/Common/isLocalAddress.h>
#include <statdaemons/ext/range.hpp> #include <ext/range.hpp>
#include <Poco/Util/AbstractConfiguration.h> #include <Poco/Util/AbstractConfiguration.h>
#include "writeParenthesisedString.h" #include "writeParenthesisedString.h"

View File

@ -5,7 +5,7 @@
#include <DB/Dictionaries/DictionaryStructure.h> #include <DB/Dictionaries/DictionaryStructure.h>
#include <DB/Columns/ColumnString.h> #include <DB/Columns/ColumnString.h>
#include <DB/Common/Arena.h> #include <DB/Common/Arena.h>
#include <statdaemons/ext/range.hpp> #include <ext/range.hpp>
#include <atomic> #include <atomic>
#include <vector> #include <vector>
#include <tuple> #include <tuple>

View File

@ -5,7 +5,7 @@
#include <DB/Dictionaries/DictionaryStructure.h> #include <DB/Dictionaries/DictionaryStructure.h>
#include <DB/Common/HashTable/HashMap.h> #include <DB/Common/HashTable/HashMap.h>
#include <DB/Columns/ColumnString.h> #include <DB/Columns/ColumnString.h>
#include <statdaemons/ext/range.hpp> #include <ext/range.hpp>
#include <atomic> #include <atomic>
#include <memory> #include <memory>
#include <tuple> #include <tuple>

View File

@ -7,7 +7,7 @@
#include <DB/DataTypes/DataTypeDate.h> #include <DB/DataTypes/DataTypeDate.h>
#include <DB/DataTypes/DataTypeDateTime.h> #include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/Columns/ColumnString.h> #include <DB/Columns/ColumnString.h>
#include <statdaemons/ext/range.hpp> #include <ext/range.hpp>
#include <mysqlxx/Query.h> #include <mysqlxx/Query.h>
#include <mysqlxx/PoolWithFailover.h> #include <mysqlxx/PoolWithFailover.h>
#include <vector> #include <vector>

View File

@ -2,10 +2,9 @@
#include <DB/Dictionaries/IDictionarySource.h> #include <DB/Dictionaries/IDictionarySource.h>
#include <DB/Dictionaries/MySQLBlockInputStream.h> #include <DB/Dictionaries/MySQLBlockInputStream.h>
#include <statdaemons/ext/range.hpp> #include <ext/range.hpp>
#include <mysqlxx/Pool.h> #include <mysqlxx/Pool.h>
#include <Poco/Util/AbstractConfiguration.h> #include <Poco/Util/AbstractConfiguration.h>
#include <strconvert/escape.h>
#include "writeParenthesisedString.h" #include "writeParenthesisedString.h"
@ -79,6 +78,28 @@ public:
private: private:
Logger * log = &Logger::get("MySQLDictionarySource"); 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 getLastModification() const
{ {
mysqlxx::DateTime update_time{std::time(nullptr)}; mysqlxx::DateTime update_time{std::time(nullptr)};
@ -89,7 +110,7 @@ private:
try try
{ {
auto connection = pool.Get(); 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()); LOG_TRACE(log, query.str());

View File

@ -5,7 +5,7 @@
#include <DB/Dictionaries/DictionaryStructure.h> #include <DB/Dictionaries/DictionaryStructure.h>
#include <DB/Common/HashTable/HashMap.h> #include <DB/Common/HashTable/HashMap.h>
#include <DB/Columns/ColumnString.h> #include <DB/Columns/ColumnString.h>
#include <statdaemons/ext/range.hpp> #include <ext/range.hpp>
#include <atomic> #include <atomic>
#include <memory> #include <memory>
#include <tuple> #include <tuple>

View File

@ -17,7 +17,7 @@
#include <DB/Functions/NumberTraits.h> #include <DB/Functions/NumberTraits.h>
#include <DB/Functions/FunctionsConditional.h> #include <DB/Functions/FunctionsConditional.h>
#include <statdaemons/ext/range.hpp> #include <ext/range.hpp>
#include <unordered_map> #include <unordered_map>

View File

@ -14,7 +14,7 @@
#include <DB/Functions/IFunction.h> #include <DB/Functions/IFunction.h>
#include <arpa/inet.h> #include <arpa/inet.h>
#include <statdaemons/ext/range.hpp> #include <ext/range.hpp>
#include <array> #include <array>

View File

@ -11,7 +11,7 @@
#include <DB/Columns/ColumnFixedString.h> #include <DB/Columns/ColumnFixedString.h>
#include <DB/Columns/ColumnConst.h> #include <DB/Columns/ColumnConst.h>
#include <DB/Functions/IFunction.h> #include <DB/Functions/IFunction.h>
#include <statdaemons/ext/range.hpp> #include <ext/range.hpp>
namespace DB namespace DB

View File

@ -20,7 +20,7 @@
#include <DB/Dictionaries/CacheDictionary.h> #include <DB/Dictionaries/CacheDictionary.h>
#include <DB/Dictionaries/RangeHashedDictionary.h> #include <DB/Dictionaries/RangeHashedDictionary.h>
#include <statdaemons/ext/range.hpp> #include <ext/range.hpp>
namespace DB namespace DB

View File

@ -22,7 +22,7 @@
#include <DB/Common/HashTable/Hash.h> #include <DB/Common/HashTable/Hash.h>
#include <DB/Functions/IFunction.h> #include <DB/Functions/IFunction.h>
#include <statdaemons/ext/range.hpp> #include <ext/range.hpp>
#include <stats/IntHash.h> #include <stats/IntHash.h>

View File

@ -24,7 +24,7 @@
#include <DB/Common/UnicodeBar.h> #include <DB/Common/UnicodeBar.h>
#include <DB/Functions/IFunction.h> #include <DB/Functions/IFunction.h>
#include <DB/Interpreters/ExpressionActions.h> #include <DB/Interpreters/ExpressionActions.h>
#include <statdaemons/ext/range.hpp> #include <ext/range.hpp>
#include <cmath> #include <cmath>
@ -386,23 +386,16 @@ public:
throw Exception("Second argument for function '" + getName() + "' must be Set; found " + column_set_ptr->getName(), throw Exception("Second argument for function '" + getName() + "' must be Set; found " + column_set_ptr->getName(),
ErrorCodes::ILLEGAL_COLUMN); ErrorCodes::ILLEGAL_COLUMN);
/// Столбцы, которые проверяются на принадлежность множеству. Block block_of_key_columns;
ColumnNumbers left_arguments;
/// Первый аргумент может быть tuple или одиночным столбцом. /// Первый аргумент может быть tuple или одиночным столбцом.
const ColumnTuple * tuple = typeid_cast<const ColumnTuple *>(&*block.getByPosition(arguments[0]).column); const ColumnTuple * tuple = typeid_cast<const ColumnTuple *>(&*block.getByPosition(arguments[0]).column);
if (tuple) if (tuple)
{ block_of_key_columns = tuple->getData();
/// Находим в блоке столбцы из 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));
}
else 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);
} }
}; };

View File

@ -12,7 +12,7 @@
#include <DB/Columns/ColumnFixedString.h> #include <DB/Columns/ColumnFixedString.h>
#include <DB/Columns/ColumnConst.h> #include <DB/Columns/ColumnConst.h>
#include <DB/Functions/IFunction.h> #include <DB/Functions/IFunction.h>
#include <statdaemons/ext/range.hpp> #include <ext/range.hpp>
#include <emmintrin.h> #include <emmintrin.h>
#include <nmmintrin.h> #include <nmmintrin.h>

View File

@ -2,7 +2,7 @@
#include <Poco/Mutex.h> #include <Poco/Mutex.h>
#include <statdaemons/OptimizedRegularExpression.h> #include <DB/Common/OptimizedRegularExpression.h>
#include <memory> #include <memory>
#include <DB/DataTypes/DataTypesNumberFixed.h> #include <DB/DataTypes/DataTypesNumberFixed.h>
@ -19,7 +19,7 @@
#include <mutex> #include <mutex>
#include <stack> #include <stack>
#include <statdaemons/ext/range.hpp> #include <ext/range.hpp>
#include <Poco/Unicode.h> #include <Poco/Unicode.h>

View File

@ -7,7 +7,7 @@
#include <Poco/SharedPtr.h> #include <Poco/SharedPtr.h>
#include <Poco/Net/NetException.h> #include <Poco/Net/NetException.h>
#include <statdaemons/threadpool.hpp> #include <common/threadpool.hpp>
#include <DB/IO/WriteBuffer.h> #include <DB/IO/WriteBuffer.h>
@ -46,7 +46,7 @@ private:
started = true; started = true;
if (exception) if (exception)
exception->rethrow(); std::rethrow_exception(exception);
swapBuffers(); swapBuffers();
@ -77,7 +77,7 @@ public:
} }
} }
ExceptionPtr exception; std::exception_ptr exception;
/// То, что выполняется в отдельном потоке /// То, что выполняется в отдельном потоке
void thread() void thread()
@ -88,7 +88,7 @@ public:
} }
catch (...) catch (...)
{ {
exception = cloneCurrentException(); exception = std::current_exception();
} }
} }
}; };

View File

@ -5,7 +5,7 @@
#include <DB/Common/ProfileEvents.h> #include <DB/Common/ProfileEvents.h>
#include <DB/Common/Allocator.h> #include <DB/Common/Allocator.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/Core/Defines.h> #include <DB/Core/Defines.h>

View File

@ -9,7 +9,7 @@
#include <DB/Common/PODArray.h> #include <DB/Common/PODArray.h>
#include <DB/Common/ProfileEvents.h> #include <DB/Common/ProfileEvents.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/IO/ReadBuffer.h> #include <DB/IO/ReadBuffer.h>
#include <DB/IO/BufferWithOwnMemory.h> #include <DB/IO/BufferWithOwnMemory.h>

View File

@ -3,7 +3,7 @@
#include <cstring> #include <cstring>
#include <algorithm> #include <algorithm>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/IO/BufferBase.h> #include <DB/IO/BufferBase.h>

View File

@ -4,7 +4,7 @@
#include <DB/IO/ReadBuffer.h> #include <DB/IO/ReadBuffer.h>
#include <DB/IO/BufferWithOwnMemory.h> #include <DB/IO/BufferWithOwnMemory.h>
#include <DB/Core/Defines.h> #include <DB/Core/Defines.h>
#include <statdaemons/AIO.h> #include <DB/Common/AIO.h>
#include <string> #include <string>
#include <limits> #include <limits>

View File

@ -5,7 +5,7 @@
#include <DB/Common/ProfileEvents.h> #include <DB/Common/ProfileEvents.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/IO/ReadBufferFromFileBase.h> #include <DB/IO/ReadBufferFromFileBase.h>

View File

@ -2,7 +2,7 @@
#include <iostream> #include <iostream>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/IO/ReadBuffer.h> #include <DB/IO/ReadBuffer.h>

View File

@ -13,7 +13,7 @@
#include <mysqlxx/DateTime.h> #include <mysqlxx/DateTime.h>
#include <DB/Core/Types.h> #include <DB/Core/Types.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/IO/ReadBuffer.h> #include <DB/IO/ReadBuffer.h>

View File

@ -3,7 +3,7 @@
#include <algorithm> #include <algorithm>
#include <cstring> #include <cstring>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/IO/BufferBase.h> #include <DB/IO/BufferBase.h>

View File

@ -4,7 +4,7 @@
#include <DB/IO/WriteBuffer.h> #include <DB/IO/WriteBuffer.h>
#include <DB/IO/BufferWithOwnMemory.h> #include <DB/IO/BufferWithOwnMemory.h>
#include <DB/Core/Defines.h> #include <DB/Core/Defines.h>
#include <statdaemons/AIO.h> #include <DB/Common/AIO.h>
#include <string> #include <string>
#include <unistd.h> #include <unistd.h>

View File

@ -3,7 +3,7 @@
#include <unistd.h> #include <unistd.h>
#include <errno.h> #include <errno.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/Common/ProfileEvents.h> #include <DB/Common/ProfileEvents.h>

View File

@ -2,12 +2,12 @@
#include <Poco/Net/HTTPServerResponse.h> #include <Poco/Net/HTTPServerResponse.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/IO/WriteBuffer.h> #include <DB/IO/WriteBuffer.h>
#include <DB/IO/BufferWithOwnMemory.h> #include <DB/IO/BufferWithOwnMemory.h>
#include <statdaemons/NetException.h> #include <DB/Common/NetException.h>
namespace DB namespace DB

View File

@ -2,7 +2,7 @@
#include <iostream> #include <iostream>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/IO/WriteBuffer.h> #include <DB/IO/WriteBuffer.h>

View File

@ -12,7 +12,7 @@
#include <mysqlxx/Null.h> #include <mysqlxx/Null.h>
#include <DB/Core/Types.h> #include <DB/Core/Types.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/IO/WriteBuffer.h> #include <DB/IO/WriteBuffer.h>

View File

@ -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};
}
} }

View File

@ -5,7 +5,7 @@
#include <functional> #include <functional>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <statdaemons/threadpool.hpp> #include <common/threadpool.hpp>
#include <DB/Core/StringRef.h> #include <DB/Core/StringRef.h>
#include <DB/Common/Arena.h> #include <DB/Common/Arena.h>
@ -319,7 +319,7 @@ struct AggregationMethodKeysFixed
}; };
/// Для остальных случаев. Агрегирует по конкатенации ключей. (При этом, строки, содержащие нули посередине, могут склеиться.) /// Агрегирует по конкатенации ключей. (При этом, строки, содержащие нули посередине, могут склеиться.)
template <typename TData> template <typename TData>
struct AggregationMethodConcat struct AggregationMethodConcat
{ {
@ -392,6 +392,70 @@ struct AggregationMethodConcat
}; };
/** Агрегирует по конкатенации сериализованных значений ключей.
* Похож на AggregationMethodConcat, но подходит, например, для массивов строк или нескольких массивов.
* Сериализованное значение отличается тем, что позволяет однозначно его десериализовать, имея только позицию, с которой оно начинается.
* То есть, например, для строк, оно содержит сначала сериализованную длину строки, а потом байты.
* Поэтому, при агрегации по нескольким строкам, неоднозначностей не возникает.
*/
template <typename TData>
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 <typename Other>
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 битному хэшу от ключа. (При этом, строки, содержащие нули посередине, могут склеиться.) /// Для остальных случаев. Агрегирует по 128 битному хэшу от ключа. (При этом, строки, содержащие нули посередине, могут склеиться.)
template <typename TData> template <typename TData>
struct AggregationMethodHashed struct AggregationMethodHashed
@ -492,6 +556,7 @@ struct AggregatedDataVariants : private boost::noncopyable
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys256>> keys256; std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys256>> keys256;
std::unique_ptr<AggregationMethodHashed<AggregatedDataHashed>> hashed; std::unique_ptr<AggregationMethodHashed<AggregatedDataHashed>> hashed;
std::unique_ptr<AggregationMethodConcat<AggregatedDataWithStringKey>> concat; std::unique_ptr<AggregationMethodConcat<AggregatedDataWithStringKey>> concat;
std::unique_ptr<AggregationMethodSerialized<AggregatedDataWithStringKey>> serialized;
std::unique_ptr<AggregationMethodOneNumber<UInt32, AggregatedDataWithUInt64KeyTwoLevel>> key32_two_level; std::unique_ptr<AggregationMethodOneNumber<UInt32, AggregatedDataWithUInt64KeyTwoLevel>> key32_two_level;
std::unique_ptr<AggregationMethodOneNumber<UInt64, AggregatedDataWithUInt64KeyTwoLevel>> key64_two_level; std::unique_ptr<AggregationMethodOneNumber<UInt64, AggregatedDataWithUInt64KeyTwoLevel>> key64_two_level;
@ -501,6 +566,7 @@ struct AggregatedDataVariants : private boost::noncopyable
std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys256TwoLevel>> keys256_two_level; std::unique_ptr<AggregationMethodKeysFixed<AggregatedDataWithKeys256TwoLevel>> keys256_two_level;
std::unique_ptr<AggregationMethodHashed<AggregatedDataHashedTwoLevel>> hashed_two_level; std::unique_ptr<AggregationMethodHashed<AggregatedDataHashedTwoLevel>> hashed_two_level;
std::unique_ptr<AggregationMethodConcat<AggregatedDataWithStringKeyTwoLevel>> concat_two_level; std::unique_ptr<AggregationMethodConcat<AggregatedDataWithStringKeyTwoLevel>> concat_two_level;
std::unique_ptr<AggregationMethodSerialized<AggregatedDataWithStringKeyTwoLevel>> serialized_two_level;
/// В этом и подобных макросах, вариант without_key не учитывается. /// В этом и подобных макросах, вариант without_key не учитывается.
#define APPLY_FOR_AGGREGATED_VARIANTS(M) \ #define APPLY_FOR_AGGREGATED_VARIANTS(M) \
@ -514,6 +580,7 @@ struct AggregatedDataVariants : private boost::noncopyable
M(keys256, false) \ M(keys256, false) \
M(hashed, false) \ M(hashed, false) \
M(concat, false) \ M(concat, false) \
M(serialized, false) \
M(key32_two_level, true) \ M(key32_two_level, true) \
M(key64_two_level, true) \ M(key64_two_level, true) \
M(key_string_two_level, true) \ M(key_string_two_level, true) \
@ -521,7 +588,8 @@ struct AggregatedDataVariants : private boost::noncopyable
M(keys128_two_level, true) \ M(keys128_two_level, true) \
M(keys256_two_level, true) \ M(keys256_two_level, true) \
M(hashed_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 enum class Type
{ {
@ -636,7 +704,8 @@ struct AggregatedDataVariants : private boost::noncopyable
M(keys128) \ M(keys128) \
M(keys256) \ M(keys256) \
M(hashed) \ M(hashed) \
M(concat) M(concat) \
M(serialized) \
#define APPLY_FOR_VARIANTS_NOT_CONVERTIBLE_TO_TWO_LEVEL(M) \ #define APPLY_FOR_VARIANTS_NOT_CONVERTIBLE_TO_TWO_LEVEL(M) \
M(key8) \ M(key8) \
@ -667,7 +736,8 @@ struct AggregatedDataVariants : private boost::noncopyable
M(keys128_two_level) \ M(keys128_two_level) \
M(keys256_two_level) \ M(keys256_two_level) \
M(hashed_two_level) \ M(hashed_two_level) \
M(concat_two_level) M(concat_two_level) \
M(serialized_two_level)
}; };
typedef SharedPtr<AggregatedDataVariants> AggregatedDataVariantsPtr; typedef SharedPtr<AggregatedDataVariants> AggregatedDataVariantsPtr;

View File

@ -9,10 +9,10 @@
#include <unordered_map> #include <unordered_map>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <statdaemons/threadpool.hpp> #include <common/threadpool.hpp>
#include <DB/Core/Types.h> #include <DB/Core/Types.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Common/UInt128.h> #include <DB/Common/UInt128.h>

View File

@ -138,8 +138,14 @@ public:
bool isTableExist(const String & database_name, const String & table_name) const; bool isTableExist(const String & database_name, const String & table_name) const;
bool isDatabaseExist(const String & database_name) const; bool isDatabaseExist(const String & database_name) const;
void assertTableExists(const String & database_name, const String & table_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; void assertDatabaseDoesntExist(const String & database_name) const;
Tables getExternalTables() const; Tables getExternalTables() const;
@ -266,6 +272,12 @@ public:
void shutdown(); void shutdown();
private: private:
/** Проверить, имеет ли текущий клиент доступ к заданной базе данных.
* Если доступ запрещён, кинуть исключение.
* NOTE: Этот метод надо всегда вызывать при захваченном мьютексе shared->mutex.
*/
void checkDatabaseAccessRights(const std::string & database_name) const;
const Dictionaries & getDictionariesImpl(bool throw_on_error) const; const Dictionaries & getDictionariesImpl(bool throw_on_error) const;
const ExternalDictionaries & getExternalDictionariesImpl(bool throw_on_error) const; const ExternalDictionaries & getExternalDictionariesImpl(bool throw_on_error) const;

View File

@ -146,7 +146,7 @@ public:
Dictionaries(const bool throw_on_error) Dictionaries(const bool throw_on_error)
: Dictionaries(throw_on_error, : Dictionaries(throw_on_error,
Application::instance().config() Poco::Util::Application::instance().config()
.getInt("builtin_dictionaries_reload_interval", 3600)) .getInt("builtin_dictionaries_reload_interval", 3600))
{} {}

View File

@ -1,7 +1,7 @@
#pragma once #pragma once
#include <DB/Dictionaries/IDictionary.h> #include <DB/Dictionaries/IDictionary.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/Common/setThreadName.h> #include <DB/Common/setThreadName.h>
#include <common/MultiVersion.h> #include <common/MultiVersion.h>

View File

@ -6,10 +6,10 @@
#include <Poco/Mutex.h> #include <Poco/Mutex.h>
#include <Poco/Condition.h> #include <Poco/Condition.h>
#include <Poco/Net/IPAddress.h> #include <Poco/Net/IPAddress.h>
#include <statdaemons/Stopwatch.h> #include <DB/Common/Stopwatch.h>
#include <DB/Core/Defines.h> #include <DB/Core/Defines.h>
#include <DB/Core/Progress.h> #include <DB/Core/Progress.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/Common/MemoryTracker.h> #include <DB/Common/MemoryTracker.h>
#include <DB/IO/WriteHelpers.h> #include <DB/IO/WriteHelpers.h>

View File

@ -14,7 +14,7 @@
#include <common/Common.h> #include <common/Common.h>
#include <DB/Core/Types.h> #include <DB/Core/Types.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/IO/WriteHelpers.h> #include <DB/IO/WriteHelpers.h>
@ -187,7 +187,7 @@ class Quotas
{ {
private: private:
/// Имя квоты -> квоты. /// Имя квоты -> квоты.
typedef std::unordered_map<String, SharedPtr<Quota> > Container; typedef std::unordered_map<String, std::unique_ptr<Quota>> Container;
Container cont; Container cont;
public: public:

View File

@ -288,10 +288,10 @@ public:
// Возвращает false, если превышено какое-нибудь ограничение, и больше не нужно вставлять. // Возвращает false, если превышено какое-нибудь ограничение, и больше не нужно вставлять.
bool insertFromBlock(const Block & block, bool create_ordered_set = false); bool insertFromBlock(const Block & block, bool create_ordered_set = false);
/** Для указанных столбцов блока проверить принадлежность их значений множеству. /** Для столбцов блока проверить принадлежность их значений множеству.
* Записать результат в столбец в позиции result. * Записать результат в столбец в позиции 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 std::string describe() const
{ {

View File

@ -120,7 +120,7 @@ struct Settings
M(SettingUInt64, merge_tree_max_rows_to_use_cache, (1024 * 1024)) \ M(SettingUInt64, merge_tree_max_rows_to_use_cache, (1024 * 1024)) \
\ \
/** Распределять чтение из MergeTree по потокам равномерно, обеспечивая стабильное среднее время исполнения каждого потока в пределах одного чтения. */ \ /** Распределять чтение из MergeTree по потокам равномерно, обеспечивая стабильное среднее время исполнения каждого потока в пределах одного чтения. */ \
M(SettingBool, merge_tree_uniform_read_distribution, false) \ M(SettingBool, merge_tree_uniform_read_distribution, true) \
\ \
/** Минимальная длина выражения expr = x1 OR ... expr = xN для оптимизации */ \ /** Минимальная длина выражения expr = x1 OR ... expr = xN для оптимизации */ \
M(SettingUInt64, optimize_min_equality_disjunction_chain_length, 3) \ M(SettingUInt64, optimize_min_equality_disjunction_chain_length, 3) \

View File

@ -11,7 +11,7 @@
#include <Poco/String.h> #include <Poco/String.h>
#include <DB/Core/Types.h> #include <DB/Core/Types.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/IO/ReadHelpers.h> #include <DB/IO/ReadHelpers.h>
#include <DB/IO/HexWriteBuffer.h> #include <DB/IO/HexWriteBuffer.h>
@ -23,6 +23,7 @@
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <unordered_set>
namespace DB namespace DB
{ {
@ -214,7 +215,7 @@ public:
class AddressPatterns class AddressPatterns
{ {
private: private:
typedef std::vector<SharedPtr<IAddressPattern> > Container; typedef std::vector<std::unique_ptr<IAddressPattern>> Container;
Container patterns; Container patterns;
public: public:
@ -252,19 +253,19 @@ public:
for (Poco::Util::AbstractConfiguration::Keys::const_iterator it = config_keys.begin(); it != config_keys.end(); ++it) for (Poco::Util::AbstractConfiguration::Keys::const_iterator it = config_keys.begin(); it != config_keys.end(); ++it)
{ {
SharedPtr<IAddressPattern> pattern; Container::value_type pattern;
String value = config.getString(config_elem + "." + *it); String value = config.getString(config_elem + "." + *it);
if (0 == it->compare(0, strlen("ip"), "ip")) 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")) 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")) else if (0 == it->compare(0, strlen("host"), "host"))
pattern = new HostExactPattern(value); pattern.reset(new HostExactPattern(value));
else else
throw Exception("Unknown address pattern type: " + *it, ErrorCodes::UNKNOWN_ADDRESS_PATTERN_TYPE); 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; AddressPatterns addresses;
/// Список разрешённых баз данных.
using DatabaseSet = std::unordered_set<std::string>;
DatabaseSet databases;
User(const String & name_, const String & config_elem, Poco::Util::AbstractConfiguration & config) User(const String & name_, const String & config_elem, Poco::Util::AbstractConfiguration & config)
: name(name_) : name(name_)
{ {
@ -312,6 +317,21 @@ struct User
quota = config.getString(config_elem + ".quota"); quota = config.getString(config_elem + ".quota");
addresses.addFromConfig(config_elem + ".networks", config); 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; 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);
}
}; };

View File

@ -63,9 +63,9 @@ public:
void clone(Parameters & p) const void clone(Parameters & p) const
{ {
p = *this; p = *this;
p.col_decl = col_decl->clone(); if (col_decl) p.col_decl = col_decl->clone();
p.column = column->clone(); if (column) p.column = column->clone();
p.partition = partition->clone(); if (partition) p.partition = partition->clone();
} }
}; };
typedef std::vector<Parameters> ParameterContainer; typedef std::vector<Parameters> ParameterContainer;
@ -95,9 +95,7 @@ public:
{ {
ASTAlterQuery * res = new ASTAlterQuery(*this); ASTAlterQuery * res = new ASTAlterQuery(*this);
for (ParameterContainer::size_type i = 0; i < parameters.size(); ++i) for (ParameterContainer::size_type i = 0; i < parameters.size(); ++i)
{
parameters[i].clone(res->parameters[i]); parameters[i].clone(res->parameters[i]);
}
return res; return res;
} }

View File

@ -11,7 +11,7 @@
#include <common/Common.h> #include <common/Common.h>
#include <DB/Core/Types.h> #include <DB/Core/Types.h>
#include <DB/Core/Exception.h> #include <DB/Common/Exception.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <DB/IO/WriteHelpers.h> #include <DB/IO/WriteHelpers.h>
#include <DB/Parsers/StringRange.h> #include <DB/Parsers/StringRange.h>

View File

@ -1,7 +1,7 @@
#include <DB/IO/CompressedStream.h> #include <DB/IO/CompressedStream.h>
#include <DB/IO/ReadHelpers.h> #include <DB/IO/ReadHelpers.h>
#include <DB/Core/ErrorCodes.h> #include <DB/Core/ErrorCodes.h>
#include <statdaemons/Exception.h> #include <DB/Common/Exception.h>
#include <Poco/Util/AbstractConfiguration.h> #include <Poco/Util/AbstractConfiguration.h>

View File

@ -10,7 +10,7 @@
#include <DB/Interpreters/InterpreterInsertQuery.h> #include <DB/Interpreters/InterpreterInsertQuery.h>
#include <DB/Interpreters/Cluster.h> #include <DB/Interpreters/Cluster.h>
#include <statdaemons/Increment.h> #include <DB/Common/Increment.h>
#include <memory> #include <memory>
#include <common/Revision.h> #include <common/Revision.h>

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