ClickHouse/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp

194 lines
11 KiB
C++
Raw Normal View History

#include <DB/Columns/ColumnString.h>
#include <DB/Columns/ColumnArray.h>
#include <DB/Columns/ColumnsNumber.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypesNumber.h>
#include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataStreams/OneBlockInputStream.h>
2015-09-24 03:50:09 +00:00
#include <DB/Storages/System/StorageSystemReplicationQueue.h>
#include <DB/Storages/StorageReplicatedMergeTree.h>
#include <DB/Common/VirtualColumnUtils.h>
Squashed commit of the following: commit f9b478181cd49224154cc350fb57df7121842f1c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 04:06:36 2016 +0300 Database engines: development [#METR-19997]. commit f7a10a67761ccfd05f3dac32d6444920cd8d4d60 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 03:44:37 2016 +0300 Database engines: development [#METR-19997]. commit bd98a8558e98bad2bed278e5762c4e0fc66e6f38 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:33:59 2016 +0300 Database engines: development [#METR-19997]. commit 19712fd884c22a4e2c2b67474086dea8f44e7c7b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:03:11 2016 +0300 Database engines: development [#METR-19997]. commit 50274d6df7e91fcc34aab8a8c72347daa2c6512f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 23:24:57 2016 +0300 Database engines: development [#METR-19997]. commit 4a0b99b19b34e90ef8b7be2d199f6232e36ef3f7 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 22:50:36 2016 +0300 Database engines: development [#METR-19997]. commit 44ff3ebba7a3e460a27a89f31ddf199dbea1d182 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 15:09:17 2016 +0300 Database engines: development [#METR-19997]. commit 137c31f3004cfd282473b6acb01cbe1b4ca2aadd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:26:34 2016 +0300 Database engines: development [#METR-19997]. commit aa4c0496d4afe4a691164254be2bd5600542b38a Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:22:59 2016 +0300 Database engines: development [#METR-19997]. commit 5a94d1f0607450a2dac28a4d7df8b1393a864c23 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 01:02:40 2016 +0300 Database engines: development [#METR-19997]. commit 50fd5b52ea1141955a5dfba0dcb191f3289ac25b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 23:23:40 2016 +0300 Database engines: development [#METR-19997]. commit a333d91b058e4f56dd83a6d2878c3c2bd8efc002 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 20:29:07 2016 +0300 Database engines: development [#METR-19997]. commit f81d366e7ac8348436f2698d040f8e341743a024 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 01:30:23 2016 +0300 Database engines: development [#METR-19997]. commit d0696860c9060827896214c08d147c759ea79376 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 21:55:31 2016 +0300 Database engines: development [#METR-19997]. commit 46a168c2ada140a0e95cd8d4b9d8ba9bac855d11 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 08:00:58 2016 +0300 Database engines: development [#METR-19997]. commit 20a2bad161454225fc1b5f9b919b842fbebc3231 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:51:10 2016 +0300 Database engines: development [#METR-19997]. commit ca0a77fcc2a8d0b276eb3743c53551ad3fe16314 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:02:20 2016 +0300 Reverted erroneous modification [#METR-19997]. commit 1370bdcc4594182f6ef2b146f9afabfe1c295080 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 00:41:34 2016 +0300 Database engines: development [#METR-19997]. commit 16e72c67041cae6471509d3f0f3d4a9aa7b7dc0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Tue Mar 15 00:41:48 2016 +0300 Database engines: development [#METR-19997].
2016-03-19 01:18:49 +00:00
#include <DB/Databases/IDatabase.h>
namespace DB
{
StorageSystemReplicationQueue::StorageSystemReplicationQueue(const std::string & name_)
: name(name_)
, columns{
/// Table properties.
{ "database", std::make_shared<DataTypeString>() },
{ "table", std::make_shared<DataTypeString>() },
{ "replica_name", std::make_shared<DataTypeString>() },
/// Constant element properties.
{ "position", std::make_shared<DataTypeUInt32>() },
{ "node_name", std::make_shared<DataTypeString>() },
{ "type", std::make_shared<DataTypeString>() },
{ "create_time", std::make_shared<DataTypeDateTime>()},
{ "required_quorum", std::make_shared<DataTypeUInt32>() },
{ "source_replica", std::make_shared<DataTypeString>() },
{ "new_part_name", std::make_shared<DataTypeString>() },
{ "parts_to_merge", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()) },
{ "is_detach", std::make_shared<DataTypeUInt8>() },
{ "is_attach_unreplicated", std::make_shared<DataTypeUInt8>() },
{ "attach_source_part_name",std::make_shared<DataTypeString>() },
/// Processing status of item.
{ "is_currently_executing", std::make_shared<DataTypeUInt8>() },
{ "num_tries", std::make_shared<DataTypeUInt32>() },
{ "last_exception", std::make_shared<DataTypeString>() },
{ "last_attempt_time", std::make_shared<DataTypeDateTime>()},
{ "num_postponed", std::make_shared<DataTypeUInt32>() },
{ "postpone_reason", std::make_shared<DataTypeString>() },
{ "last_postpone_time", std::make_shared<DataTypeDateTime>()},
}
{
}
StoragePtr StorageSystemReplicationQueue::create(const std::string & name_)
{
return make_shared(name_);
}
BlockInputStreams StorageSystemReplicationQueue::read(
const Names & column_names,
ASTPtr query,
const Context & context,
const Settings & settings,
QueryProcessingStage::Enum & processed_stage,
const size_t max_block_size,
const unsigned threads)
{
check(column_names);
processed_stage = QueryProcessingStage::FetchColumns;
std::map<String, std::map<String, StoragePtr>> replicated_tables;
for (const auto & db : context.getDatabases())
for (auto iterator = db.second->getIterator(); iterator->isValid(); iterator->next())
if (typeid_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
replicated_tables[db.first][iterator->name()] = iterator->table();
ColumnWithTypeAndName col_database_to_filter { std::make_shared<ColumnString>(), std::make_shared<DataTypeString>(), "database" };
ColumnWithTypeAndName col_table_to_filter { std::make_shared<ColumnString>(), std::make_shared<DataTypeString>(), "table" };
for (auto & db : replicated_tables)
{
for (auto & table : db.second)
{
col_database_to_filter.column->insert(db.first);
col_table_to_filter.column->insert(table.first);
}
}
/// Determine what tables are needed by the conditions in the query.
{
Block filtered_block { col_database_to_filter, col_table_to_filter };
VirtualColumnUtils::filterBlockWithQuery(query, filtered_block, context);
if (!filtered_block.rows())
return BlockInputStreams();
col_database_to_filter = filtered_block.getByName("database");
col_table_to_filter = filtered_block.getByName("table");
}
ColumnWithTypeAndName col_database { std::make_shared<ColumnString>(), std::make_shared<DataTypeString>(), "database" };
ColumnWithTypeAndName col_table { std::make_shared<ColumnString>(), std::make_shared<DataTypeString>(), "table" };
ColumnWithTypeAndName col_replica_name { std::make_shared<ColumnString>(), std::make_shared<DataTypeString>(), "replica_name" };
ColumnWithTypeAndName col_position { std::make_shared<ColumnUInt32>(), std::make_shared<DataTypeUInt32>(), "position" };
ColumnWithTypeAndName col_node_name { std::make_shared<ColumnString>(), std::make_shared<DataTypeString>(), "node_name" };
ColumnWithTypeAndName col_type { std::make_shared<ColumnString>(), std::make_shared<DataTypeString>(), "type" };
ColumnWithTypeAndName col_create_time { std::make_shared<ColumnUInt32>(), std::make_shared<DataTypeDateTime>(), "create_time" };
ColumnWithTypeAndName col_required_quorum { std::make_shared<ColumnUInt32>(), std::make_shared<DataTypeUInt32>(), "required_quorum" };
ColumnWithTypeAndName col_source_replica { std::make_shared<ColumnString>(), std::make_shared<DataTypeString>(), "source_replica" };
ColumnWithTypeAndName col_new_part_name { std::make_shared<ColumnString>(), std::make_shared<DataTypeString>(), "new_part_name" };
ColumnWithTypeAndName col_parts_to_merge { std::make_shared<ColumnArray>(std::make_shared<ColumnString>()),
std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "parts_to_merge" };
ColumnWithTypeAndName col_is_detach { std::make_shared<ColumnUInt8>(), std::make_shared<DataTypeUInt8>(), "is_detach" };
ColumnWithTypeAndName col_is_attach_unreplicated { std::make_shared<ColumnUInt8>(), std::make_shared<DataTypeUInt8>(), "is_attach_unreplicated" };
ColumnWithTypeAndName col_attach_source_part_name { std::make_shared<ColumnString>(), std::make_shared<DataTypeString>(), "attach_source_part_name" };
ColumnWithTypeAndName col_is_currently_executing { std::make_shared<ColumnUInt8>(), std::make_shared<DataTypeUInt8>(), "is_currently_executing" };
ColumnWithTypeAndName col_num_tries { std::make_shared<ColumnUInt32>(), std::make_shared<DataTypeUInt32>(), "num_tries" };
ColumnWithTypeAndName col_last_exception { std::make_shared<ColumnString>(), std::make_shared<DataTypeString>(), "last_exception" };
ColumnWithTypeAndName col_last_attempt_time { std::make_shared<ColumnUInt32>(), std::make_shared<DataTypeDateTime>(), "last_attempt_time" };
ColumnWithTypeAndName col_num_postponed { std::make_shared<ColumnUInt32>(), std::make_shared<DataTypeUInt32>(), "num_postponed" };
ColumnWithTypeAndName col_postpone_reason { std::make_shared<ColumnString>(), std::make_shared<DataTypeString>(), "postpone_reason" };
ColumnWithTypeAndName col_last_postpone_time { std::make_shared<ColumnUInt32>(), std::make_shared<DataTypeDateTime>(), "last_postpone_time" };
StorageReplicatedMergeTree::LogEntriesData queue;
String replica_name;
for (size_t i = 0, tables_size = col_database_to_filter.column->size(); i < tables_size; ++i)
{
String database = (*col_database_to_filter.column)[i].safeGet<const String &>();
String table = (*col_table_to_filter.column)[i].safeGet<const String &>();
typeid_cast<StorageReplicatedMergeTree &>(*replicated_tables[database][table]).getQueue(queue, replica_name);
for (size_t j = 0, queue_size = queue.size(); j < queue_size; ++j)
{
const auto & entry = queue[j];
Array parts_to_merge;
parts_to_merge.reserve(entry.parts_to_merge.size());
for (const auto & name : entry.parts_to_merge)
parts_to_merge.push_back(name);
col_database .column->insert(database);
col_table .column->insert(table);
col_replica_name .column->insert(replica_name);
col_position .column->insert(UInt64(j));
col_node_name .column->insert(entry.znode_name);
col_type .column->insert(entry.typeToString());
col_create_time .column->insert(UInt64(entry.create_time));
col_required_quorum .column->insert(UInt64(entry.quorum));
col_source_replica .column->insert(entry.source_replica);
col_new_part_name .column->insert(entry.new_part_name);
col_parts_to_merge .column->insert(parts_to_merge);
col_is_detach .column->insert(UInt64(entry.detach));
col_is_attach_unreplicated .column->insert(UInt64(entry.attach_unreplicated));
col_attach_source_part_name .column->insert(entry.source_part_name);
col_is_currently_executing .column->insert(UInt64(entry.currently_executing));
col_num_tries .column->insert(UInt64(entry.num_tries));
col_last_exception .column->insert(entry.exception ? getExceptionMessage(entry.exception, false) : "");
col_last_attempt_time .column->insert(UInt64(entry.last_attempt_time));
col_num_postponed .column->insert(UInt64(entry.num_postponed));
col_postpone_reason .column->insert(entry.postpone_reason);
col_last_postpone_time .column->insert(UInt64(entry.last_postpone_time));
}
}
Block block{
col_database,
col_table,
col_replica_name,
col_position,
col_node_name,
col_type,
col_create_time,
col_required_quorum,
col_source_replica,
col_new_part_name,
col_parts_to_merge,
col_is_detach,
col_is_attach_unreplicated,
col_attach_source_part_name,
col_is_currently_executing,
col_num_tries,
col_last_exception,
col_last_attempt_time,
col_num_postponed,
col_postpone_reason,
col_last_postpone_time,
};
return BlockInputStreams(1, std::make_shared<OneBlockInputStream>(block));
}
}