2017-04-01 09:19:00 +00:00
|
|
|
#include <Columns/ColumnString.h>
|
|
|
|
#include <Columns/ColumnArray.h>
|
|
|
|
#include <Columns/ColumnsNumber.h>
|
|
|
|
#include <DataTypes/DataTypeString.h>
|
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
|
|
|
#include <DataTypes/DataTypeDateTime.h>
|
|
|
|
#include <DataTypes/DataTypeArray.h>
|
|
|
|
#include <Storages/System/StorageSystemReplicationQueue.h>
|
|
|
|
#include <Storages/StorageReplicatedMergeTree.h>
|
2017-11-20 05:22:54 +00:00
|
|
|
#include <Storages/VirtualColumnUtils.h>
|
2020-03-07 17:37:38 +00:00
|
|
|
#include <Access/ContextAccess.h>
|
2017-07-13 20:58:19 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Databases/IDatabase.h>
|
2015-09-24 00:21:02 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
|
2018-07-24 14:28:56 +00:00
|
|
|
NamesAndTypesList StorageSystemReplicationQueue::getNamesAndTypes()
|
2018-01-25 14:42:39 +00:00
|
|
|
{
|
2018-07-24 14:28:56 +00:00
|
|
|
return {
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Table properties.
|
2017-12-02 02:47:12 +00:00
|
|
|
{ "database", std::make_shared<DataTypeString>() },
|
|
|
|
{ "table", std::make_shared<DataTypeString>() },
|
|
|
|
{ "replica_name", std::make_shared<DataTypeString>() },
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Constant element properties.
|
2017-12-02 02:47:12 +00:00
|
|
|
{ "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>() },
|
2017-05-24 21:38:56 +00:00
|
|
|
{ "parts_to_merge", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()) },
|
2017-12-02 02:47:12 +00:00
|
|
|
{ "is_detach", std::make_shared<DataTypeUInt8>() },
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Processing status of item.
|
2017-12-02 02:47:12 +00:00
|
|
|
{ "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>() },
|
2018-07-24 14:28:56 +00:00
|
|
|
};
|
2015-09-24 00:21:02 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-07-24 14:28:56 +00:00
|
|
|
void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const
|
2015-09-24 00:21:02 +00:00
|
|
|
{
|
2020-03-07 17:37:38 +00:00
|
|
|
const auto access = context.getAccess();
|
|
|
|
const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES);
|
2020-01-24 16:20:36 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
std::map<String, std::map<String, StoragePtr>> replicated_tables;
|
2020-02-10 13:10:17 +00:00
|
|
|
for (const auto & db : DatabaseCatalog::instance().getDatabases())
|
2018-08-13 09:11:58 +00:00
|
|
|
{
|
2019-10-01 12:44:17 +00:00
|
|
|
/// Lazy database can not contain replicated tables
|
|
|
|
if (db.second->getEngineName() == "Lazy")
|
|
|
|
continue;
|
|
|
|
|
2020-03-07 17:37:38 +00:00
|
|
|
const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first);
|
2020-01-24 16:20:36 +00:00
|
|
|
|
|
|
|
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
2018-08-13 09:11:58 +00:00
|
|
|
{
|
2020-01-24 16:20:36 +00:00
|
|
|
if (!dynamic_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
|
|
|
|
continue;
|
2020-03-07 17:37:38 +00:00
|
|
|
if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name()))
|
2020-01-24 16:20:36 +00:00
|
|
|
continue;
|
|
|
|
replicated_tables[db.first][iterator->name()] = iterator->table();
|
2018-08-13 09:11:58 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-16 00:49:03 +00:00
|
|
|
MutableColumnPtr col_database_mut = ColumnString::create();
|
|
|
|
MutableColumnPtr col_table_mut = ColumnString::create();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
for (auto & db : replicated_tables)
|
|
|
|
{
|
|
|
|
for (auto & table : db.second)
|
|
|
|
{
|
2017-12-16 00:49:03 +00:00
|
|
|
col_database_mut->insert(db.first);
|
|
|
|
col_table_mut->insert(table.first);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-12-16 00:49:03 +00:00
|
|
|
ColumnPtr col_database_to_filter = std::move(col_database_mut);
|
|
|
|
ColumnPtr col_table_to_filter = std::move(col_table_mut);
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Determine what tables are needed by the conditions in the query.
|
|
|
|
{
|
2017-12-16 00:49:03 +00:00
|
|
|
Block filtered_block
|
|
|
|
{
|
|
|
|
{ col_database_to_filter, std::make_shared<DataTypeString>(), "database" },
|
|
|
|
{ col_table_to_filter, std::make_shared<DataTypeString>(), "table" },
|
|
|
|
};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-07-15 03:48:36 +00:00
|
|
|
VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (!filtered_block.rows())
|
2018-07-24 14:28:56 +00:00
|
|
|
return;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-16 00:49:03 +00:00
|
|
|
col_database_to_filter = filtered_block.getByName("database").column;
|
|
|
|
col_table_to_filter = filtered_block.getByName("table").column;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
StorageReplicatedMergeTree::LogEntriesData queue;
|
|
|
|
String replica_name;
|
|
|
|
|
2017-12-16 00:49:03 +00:00
|
|
|
for (size_t i = 0, tables_size = col_database_to_filter->size(); i < tables_size; ++i)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-12-16 00:49:03 +00:00
|
|
|
String database = (*col_database_to_filter)[i].safeGet<const String &>();
|
|
|
|
String table = (*col_table_to_filter)[i].safeGet<const String &>();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-11-04 16:46:14 +00:00
|
|
|
dynamic_cast<StorageReplicatedMergeTree &>(*replicated_tables[database][table]).getQueue(queue, replica_name);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
for (size_t j = 0, queue_size = queue.size(); j < queue_size; ++j)
|
|
|
|
{
|
|
|
|
const auto & entry = queue[j];
|
|
|
|
|
|
|
|
Array parts_to_merge;
|
2018-04-20 16:18:16 +00:00
|
|
|
parts_to_merge.reserve(entry.source_parts.size());
|
2019-08-03 11:02:40 +00:00
|
|
|
for (const auto & part_name : entry.source_parts)
|
|
|
|
parts_to_merge.push_back(part_name);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-16 00:49:03 +00:00
|
|
|
size_t col_num = 0;
|
|
|
|
res_columns[col_num++]->insert(database);
|
|
|
|
res_columns[col_num++]->insert(table);
|
|
|
|
res_columns[col_num++]->insert(replica_name);
|
2018-10-22 08:54:54 +00:00
|
|
|
res_columns[col_num++]->insert(j);
|
2017-12-16 00:49:03 +00:00
|
|
|
res_columns[col_num++]->insert(entry.znode_name);
|
|
|
|
res_columns[col_num++]->insert(entry.typeToString());
|
2018-10-22 08:54:54 +00:00
|
|
|
res_columns[col_num++]->insert(entry.create_time);
|
|
|
|
res_columns[col_num++]->insert(entry.quorum);
|
2017-12-16 00:49:03 +00:00
|
|
|
res_columns[col_num++]->insert(entry.source_replica);
|
|
|
|
res_columns[col_num++]->insert(entry.new_part_name);
|
|
|
|
res_columns[col_num++]->insert(parts_to_merge);
|
2018-10-22 08:54:54 +00:00
|
|
|
res_columns[col_num++]->insert(entry.detach);
|
|
|
|
res_columns[col_num++]->insert(entry.currently_executing);
|
|
|
|
res_columns[col_num++]->insert(entry.num_tries);
|
2017-12-16 00:49:03 +00:00
|
|
|
res_columns[col_num++]->insert(entry.exception ? getExceptionMessage(entry.exception, false) : "");
|
|
|
|
res_columns[col_num++]->insert(UInt64(entry.last_attempt_time));
|
2018-10-22 08:54:54 +00:00
|
|
|
res_columns[col_num++]->insert(entry.num_postponed);
|
2017-12-16 00:49:03 +00:00
|
|
|
res_columns[col_num++]->insert(entry.postpone_reason);
|
|
|
|
res_columns[col_num++]->insert(UInt64(entry.last_postpone_time));
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
2015-09-24 00:21:02 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|