提交 2d5192c4 编写于 作者: A Alexey Milovidov

dbms: StorageBuffer: development [#METR-13297].

上级 08e93dbb
......@@ -41,10 +41,10 @@ public:
/// (от storage ожидают получить только столбцы таблицы).
remove_prewhere_column = !pre_name_set.count(prewhere_column);
Names post_column_names;
for (size_t i = 0; i < column_names.size(); ++i)
for (const auto & name : column_names)
{
if (!pre_name_set.count(column_names[i]))
post_column_names.push_back(column_names[i]);
if (!pre_name_set.count(name))
post_column_names.push_back(name);
}
column_names = post_column_names;
}
......
#pragma once
#include <mutex>
#include <thread>
#include <DB/Core/NamesAndTypes.h>
#include <DB/Storages/IStorage.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/DataStreams/IBlockOutputStream.h>
#include <DB/Interpreters/Context.h>
namespace DB
{
/** При вставке, буферизует данные в оперативке, пока не превышены некоторые пороги.
* Когда пороги превышены - сбрасывает данные в другую таблицу.
* При чтении, читает как из своих буферов, так и из подчинённой таблицы.
*
* Буфер представляет собой набор из num_shards блоков.
* При записи, выбирается номер блока по остатку от деления ThreadNumber на num_buckets (или один из других),
* и в соответствующий блок добавляются строчки.
* При использовании блока, он блокируется некоторым mutex-ом. Если при записи, соответствующий блок уже занят
* - пробуем заблокировать следующий по кругу блок, и так не более num_buckets раз (далее блокируемся).
* Пороги проверяются при вставке, а также, периодически, в фоновом потоке (чтобы реализовать пороги по времени).
* Пороги действуют независимо для каждого shard-а. Каждый shard может быть сброшен независимо от других.
* Если в таблицу вставляется блок, который сам по себе превышает max-пороги, то он записывается сразу в подчинённую таблицу без буферизации.
* Пороги могут быть превышены. Например, если max_rows = 1 000 000, в буфере уже было 500 000 строк,
* и добавляется кусок из 800 000 строк, то в буфере окажется 1 300 000 строк, и затем такой блок будет записан в подчинённую таблицу
*
* При уничтожении таблицы типа Buffer и при завершении работы, все данные сбрасываются.
* Данные в буфере не реплицируются, не логгируются на диск, не индексируются. При грубом перезапуске сервера, данные пропадают.
*/
class StorageBuffer : public IStorage
{
friend class BufferBlockInputStream;
friend class BufferBlockOutputStream;
public:
/// Пороги.
struct Thresholds
{
time_t time; /// Количество секунд от момента вставки первой строчки в блок.
size_t rows; /// Количество строк в блоке.
size_t bytes; /// Количество (несжатых) байт в блоке.
};
/** num_shards - уровень внутреннего параллелизма (количество независимых буферов)
* Буфер сбрасывается, если превышены все минимальные пороги или хотя бы один из максимальных.
*/
static StoragePtr create(const std::string & name_, NamesAndTypesListPtr columns_, Context & context_,
size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_,
const String & destination_database_, const String & destination_table_);
std::string getName() const override { return "Buffer"; }
std::string getTableName() const override { return name; }
const NamesAndTypesList & getColumnsList() const override { return *columns; }
BlockInputStreams read(
const Names & column_names,
ASTPtr query,
const Settings & settings,
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size = DEFAULT_BLOCK_SIZE,
unsigned threads = 1) override;
BlockOutputStreamPtr write(ASTPtr query) override;
/// Сбрасывает все буферы в подчинённую таблицу.
void shutdown() override;
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override { name = new_table_name; }
bool supportsSampling() const override { return true; }
bool supportsFinal() const override { return true; }
bool supportsPrewhere() const override { return true; }
/// в подтаблицах добавлять и удалять столбы нужно вручную
/// структура подтаблиц не проверяется
void alter(const AlterCommands & params, const String & database_name, const String & table_name, Context & context) override;
private:
String name;
NamesAndTypesListPtr columns;
Context & context;
struct Buffer
{
time_t first_write_time = 0;
Block data;
std::mutex mutex;
};
/// Имеется num_shards независимых буферов.
const size_t num_shards;
std::vector<Buffer> buffers;
const Thresholds min_thresholds;
const Thresholds max_thresholds;
const String destination_database;
const String destination_table;
bool no_destination; /// Если задано - не записывать данные из буфера, а просто опустошать буфер.
Logger * log;
/// Выполняет сброс данных по таймауту.
std::thread flush_thread;
StorageBuffer(const std::string & name_, NamesAndTypesListPtr columns_, Context & context_,
size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_,
const String & destination_database_, const String & destination_table_);
/// Сбросить буфер. Если выставлено check_thresholds - сбрасывает только если превышены пороги.
void flushBuffer(Buffer & buffer, bool check_thresholds);
bool checkThresholds(Buffer & buffer, time_t current_time, size_t additional_rows = 0, size_t additional_bytes = 0);
Poco::Event shutdown_event;
void flushThread();
};
}
......@@ -24,13 +24,13 @@ public:
std::stringstream res;
res << "Memory(" << &*begin << ", " << &*end;
for (size_t i = 0; i < column_names.size(); ++i)
res << ", " << column_names[i];
for (const auto & name : column_names)
res << ", " << name;
res << ")";
return res.str();
}
protected:
Block readImpl();
private:
......@@ -92,7 +92,7 @@ private:
BlocksList data;
Poco::FastMutex mutex;
StorageMemory(const std::string & name_, NamesAndTypesListPtr columns_);
};
......
......@@ -77,12 +77,9 @@ StoragePtr InterpreterCreateQuery::execute(bool assume_metadata_exists)
SharedPtr<InterpreterSelectQuery> interpreter_select;
Block select_sample;
/// Для таблиц типа вью, чтобы получить столбцы, может понадобиться sample block.
/// Для таблиц типа view, чтобы получить столбцы, может понадобиться sample_block.
if (create.select && (!create.attach || (!create.columns && (create.is_view || create.is_materialized_view))))
{
interpreter_select = new InterpreterSelectQuery(create.select, context);
select_sample = interpreter_select->getSampleBlock();
}
select_sample = InterpreterSelectQuery(create.select, context).getSampleBlock();
StoragePtr res;
String storage_name;
......
#include <DB/Interpreters/InterpreterSelectQuery.h>
#include <DB/Interpreters/InterpreterAlterQuery.h>
#include <DB/Storages/StorageBuffer.h>
#include <Poco/Ext/ThreadNumber.h>
namespace DB
{
StoragePtr StorageBuffer::create(const std::string & name_, NamesAndTypesListPtr columns_, Context & context_,
size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_,
const String & destination_database_, const String & destination_table_)
{
return (new StorageBuffer{
name_, columns_, context_, num_shards_, min_thresholds_, max_thresholds_, destination_database_, destination_table_})->thisPtr();
}
StorageBuffer::StorageBuffer(const std::string & name_, NamesAndTypesListPtr columns_, Context & context_,
size_t num_shards_, const Thresholds & min_thresholds_, const Thresholds & max_thresholds_,
const String & destination_database_, const String & destination_table_)
: name(name_), columns(columns_), context(context_),
num_shards(num_shards_), buffers(num_shards_),
min_thresholds(min_thresholds_), max_thresholds(max_thresholds_),
destination_database(destination_database_), destination_table(destination_table_),
no_destination(destination_database.empty() && destination_table.empty()),
log(&Logger::get("StorageBuffer (" + name + ")")),
flush_thread([this] { flushThread(); })
{
}
/// Читает из одного буфера (из одного блока) под его mutex-ом.
class BufferBlockInputStream : public IProfilingBlockInputStream
{
public:
BufferBlockInputStream(const Names & column_names_, StorageBuffer::Buffer & buffer_)
: column_names(column_names_.begin(), column_names_.end()), buffer(buffer_) {}
String getName() const { return "BufferBlockInputStream"; }
String getID() const
{
std::stringstream res;
res << "Buffer(" << &buffer;
for (const auto & name : column_names)
res << ", " << name;
res << ")";
return res.str();
}
protected:
Block readImpl()
{
std::lock_guard<std::mutex> lock(buffer.mutex);
Block res;
if (!buffer.data)
return res;
for (size_t i = 0, size = buffer.data.columns(); i < size; ++i)
{
auto & col = buffer.data.unsafeGetByPosition(i);
if (column_names.count(col.name))
res.insert(col);
}
return res;
}
private:
NameSet column_names;
StorageBuffer::Buffer & buffer;
};
BlockInputStreams StorageBuffer::read(
const Names & column_names,
ASTPtr query,
const Settings & settings,
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size,
unsigned threads)
{
processed_stage = QueryProcessingStage::FetchColumns;
BlockInputStreams streams_from_dst;
if (!no_destination)
streams_from_dst = context.getTable(destination_database, destination_table)->read(
column_names, query, settings, processed_stage, max_block_size, threads);
BlockInputStreams streams_from_buffers;
streams_from_buffers.reserve(num_shards);
for (auto & buf : buffers)
streams_from_buffers.push_back(new BufferBlockInputStream(column_names, buf));
/** Если источники из таблицы были обработаны до какой-то не начальной стадии выполнения запроса,
* то тогда источники из буферов надо тоже обернуть в конвейер обработки до той же стадии.
*/
if (processed_stage > QueryProcessingStage::FetchColumns)
for (auto & stream : streams_from_buffers)
stream = InterpreterSelectQuery(query, context, processed_stage, 0, stream).execute();
streams_from_dst.insert(streams_from_dst.end(), streams_from_buffers.begin(), streams_from_buffers.end());
return streams_from_dst;
}
class BufferBlockOutputStream : public IBlockOutputStream
{
public:
BufferBlockOutputStream(StorageBuffer & storage_) : storage(storage_) {}
void write(const Block & block)
{
if (!block)
return;
size_t rows = block.rowsInFirstColumn();
if (!rows)
return;
size_t bytes = block.bytes();
/// Если блок уже превышает максимальные ограничения, то пишем минуя буфер.
if (rows > storage.max_thresholds.rows || bytes > storage.max_thresholds.bytes)
{
LOG_TRACE(storage.log, "Writing block with " << rows << " rows, " << bytes << " bytes directly.");
writeDirect(block);
return;
}
/// Распределяем нагрузку по шардам по номеру потока.
const auto start_shard_num = Poco::ThreadNumber::get() % storage.num_shards;
/// Перебираем буферы по кругу, пытаясь заблокировать mutex. Не более одного круга.
auto shard_num = start_shard_num;
size_t try_no = 0;
for (; try_no != storage.num_shards; ++try_no)
{
std::unique_lock<std::mutex> lock(storage.buffers[shard_num].mutex, std::try_to_lock_t());
if (lock.owns_lock())
{
insertIntoBuffer(block, storage.buffers[shard_num], std::move(lock));
break;
}
++shard_num;
if (shard_num == storage.num_shards)
shard_num = 0;
}
/// Если так и не удалось ничего сразу заблокировать, то будем ждать на mutex-е.
if (try_no == storage.num_shards)
insertIntoBuffer(block, storage.buffers[start_shard_num], std::unique_lock<std::mutex>(storage.buffers[start_shard_num].mutex));
}
private:
StorageBuffer & storage;
void insertIntoBuffer(const Block & block, StorageBuffer::Buffer & buffer, std::unique_lock<std::mutex> && lock)
{
if (!buffer.data)
{
buffer.first_write_time = time(0);
buffer.data = block.cloneEmpty();
}
/// Если после вставки в буфер, ограничения будут превышены, то будем сбрасывать буфер.
if (storage.checkThresholds(buffer, time(0), block.rowsInFirstColumn(), block.bytes()))
{
/// Вытащим из буфера блок, заменим буфер на пустой. После этого можно разблокировать mutex.
Block block_to_write;
buffer.data.swap(block_to_write);
buffer.first_write_time = 0;
lock.unlock();
appendBlock(block, block_to_write);
writeDirect(block_to_write);
}
else
appendBlock(block, buffer.data);
}
void appendBlock(const Block & from, Block & to)
{
size_t rows = from.rows();
for (size_t column_no = 0, columns = to.columns(); column_no < columns; ++column_no)
{
const IColumn & col_from = *from.getByPosition(column_no).column.get();
IColumn & col_to = *to.unsafeGetByPosition(column_no).column.get();
for (size_t row_no = 0; row_no < rows; ++row_no)
col_to.insertFrom(col_from, row_no);
}
}
void writeDirect(const Block & block)
{
auto table = storage.context.getTable(storage.destination_database, storage.destination_table);
auto dst = table->write(nullptr);
dst->writePrefix();
dst->write(block);
dst->writeSuffix();
}
};
BlockOutputStreamPtr StorageBuffer::write(ASTPtr query)
{
return new BufferBlockOutputStream(*this);
}
void StorageBuffer::shutdown()
{
shutdown_event.set();
if (flush_thread.joinable())
flush_thread.join();
for (auto & buf : buffers)
flushBuffer(buf, false);
}
bool StorageBuffer::checkThresholds(Buffer & buffer, time_t current_time, size_t additional_rows, size_t additional_bytes)
{
time_t time_passed = 0;
if (buffer.first_write_time)
time_passed = current_time - buffer.first_write_time;
size_t rows = buffer.data.rowsInFirstColumn() + additional_rows;
size_t bytes = buffer.data.bytes() + additional_bytes;
bool res =
(time_passed > min_thresholds.time && rows > min_thresholds.rows && bytes > min_thresholds.bytes)
|| (time_passed > max_thresholds.time || rows > max_thresholds.rows || bytes > max_thresholds.bytes);
if (res)
LOG_TRACE(log, "Flushing buffer with " << rows << " rows, " << bytes << " bytes, age " << time_passed << " seconds.");
return res;
}
void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds)
{
StoragePtr table;
Block block_to_write;
if (!no_destination)
table = context.tryGetTable(destination_database, destination_table);
time_t current_time = check_thresholds ? time(0) : 0;
{
std::lock_guard<std::mutex> lock(buffer.mutex);
if (check_thresholds && !checkThresholds(buffer, current_time))
return;
buffer.data.swap(block_to_write);
buffer.first_write_time = 0;
}
if (!table)
{
if (!no_destination)
LOG_ERROR(log, "Destination table " << destination_database << "." << destination_table << " doesn't exists.");
return;
}
if (block_to_write)
{
auto dst = table->write(nullptr);
dst->writePrefix();
dst->write(block_to_write);
dst->writeSuffix();
}
}
void StorageBuffer::flushThread()
{
do
{
try
{
for (auto & buf : buffers)
flushBuffer(buf, true);
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
} while (!shutdown_event.tryWait(1000));
}
void StorageBuffer::alter(const AlterCommands & params, const String & database_name, const String & table_name, Context & context)
{
auto lock = lockStructureForAlter();
params.apply(*columns);
InterpreterAlterQuery::updateMetadata(database_name, table_name, *columns, context);
}
}
......@@ -11,6 +11,7 @@
#include <DB/Storages/StorageLog.h>
#include <DB/Storages/StorageTinyLog.h>
#include <DB/Storages/StorageMemory.h>
#include <DB/Storages/StorageBuffer.h>
#include <DB/Storages/StorageNull.h>
#include <DB/Storages/StorageMerge.h>
#include <DB/Storages/StorageMergeTree.h>
......@@ -190,6 +191,46 @@ StoragePtr StorageFactory::get(
return StorageDistributed::create(
table_name, columns, remote_database, remote_table, cluster_name, context, sharding_key, data_path);
}
else if (name == "Buffer")
{
/** Buffer(db, table, num_buckets, min_time, max_time, min_rows, max_rows, min_bytes, max_bytes)
*
* db, table - в какую таблицу сбрасывать данные из буфера.
* num_buckets - уровень параллелизма.
* min_time, max_time, min_rows, max_rows, min_bytes, max_bytes - условия вытеснения из буфера.
*/
ASTs & args_func = typeid_cast<ASTFunction &>(*typeid_cast<ASTCreateQuery &>(*query).storage).children;
if (args_func.size() != 1)
throw Exception("Storage Buffer requires 9 parameters: "
" destination database, destination table, num_buckets, min_time, max_time, min_rows, max_rows, min_bytes, max_bytes.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
ASTs & args = typeid_cast<ASTExpressionList &>(*args_func.at(0)).children;
if (args.size() != 9)
throw Exception("Storage Buffer requires 9 parameters: "
" destination_database, destination_table, num_buckets, min_time, max_time, min_rows, max_rows, min_bytes, max_bytes.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
String destination_database = reinterpretAsIdentifier(args[0], local_context).name;
String destination_table = typeid_cast<ASTIdentifier &>(*args[1]).name;
size_t num_buckets = apply_visitor(FieldVisitorConvertToNumber<size_t>(), typeid_cast<ASTLiteral &>(*args[2]).value);
time_t min_time = apply_visitor(FieldVisitorConvertToNumber<size_t>(), typeid_cast<ASTLiteral &>(*args[3]).value);
time_t max_time = apply_visitor(FieldVisitorConvertToNumber<size_t>(), typeid_cast<ASTLiteral &>(*args[4]).value);
size_t min_rows = apply_visitor(FieldVisitorConvertToNumber<size_t>(), typeid_cast<ASTLiteral &>(*args[5]).value);
size_t max_rows = apply_visitor(FieldVisitorConvertToNumber<size_t>(), typeid_cast<ASTLiteral &>(*args[6]).value);
size_t min_bytes = apply_visitor(FieldVisitorConvertToNumber<size_t>(), typeid_cast<ASTLiteral &>(*args[7]).value);
size_t max_bytes = apply_visitor(FieldVisitorConvertToNumber<size_t>(), typeid_cast<ASTLiteral &>(*args[8]).value);
return StorageBuffer::create(
table_name, columns, context,
num_buckets, {min_time, min_rows, min_bytes}, {max_time, max_rows, max_bytes},
destination_database, destination_table);
}
else if (endsWith(name, "MergeTree"))
{
/** Движки [Replicated][Summing|Collapsing|Aggregating|]MergeTree (8 комбинаций)
......
......@@ -43,8 +43,8 @@ String LogBlockInputStream::getID() const
std::stringstream res;
res << "Log(" << storage.getTableName() << ", " << &storage << ", " << mark_number << ", " << rows_limit;
for (size_t i = 0; i < column_names.size(); ++i)
res << ", " << column_names[i];
for (const auto & name : column_names)
res << ", " << name;
res << ")";
return res.str();
......
......@@ -38,8 +38,8 @@ String TinyLogBlockInputStream::getID() const
std::stringstream res;
res << "TinyLog(" << storage.getTableName() << ", " << &storage;
for (size_t i = 0; i < column_names.size(); ++i)
res << ", " << column_names[i];
for (const auto & name : column_names)
res << ", " << name;
res << ")";
return res.str();
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册