2018-07-24 14:42:52 +00:00
|
|
|
#include <DataTypes/DataTypeString.h>
|
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/Cluster.h>
|
|
|
|
#include <Interpreters/Context.h>
|
2018-07-24 14:28:56 +00:00
|
|
|
#include <Storages/System/StorageSystemClusters.h>
|
2021-02-08 09:14:17 +00:00
|
|
|
#include <Databases/DatabaseReplicated.h>
|
2015-04-30 12:43:16 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
2018-07-24 18:46:23 +00:00
|
|
|
|
2018-07-24 14:28:56 +00:00
|
|
|
NamesAndTypesList StorageSystemClusters::getNamesAndTypes()
|
2018-01-25 14:42:39 +00:00
|
|
|
{
|
2019-07-08 01:43:41 +00:00
|
|
|
return
|
|
|
|
{
|
2018-07-24 14:28:56 +00:00
|
|
|
{"cluster", std::make_shared<DataTypeString>()},
|
|
|
|
{"shard_num", std::make_shared<DataTypeUInt32>()},
|
|
|
|
{"shard_weight", std::make_shared<DataTypeUInt32>()},
|
|
|
|
{"replica_num", std::make_shared<DataTypeUInt32>()},
|
|
|
|
{"host_name", std::make_shared<DataTypeString>()},
|
|
|
|
{"host_address", std::make_shared<DataTypeString>()},
|
|
|
|
{"port", std::make_shared<DataTypeUInt16>()},
|
|
|
|
{"is_local", std::make_shared<DataTypeUInt8>()},
|
|
|
|
{"user", std::make_shared<DataTypeString>()},
|
|
|
|
{"default_database", std::make_shared<DataTypeString>()},
|
2019-08-06 04:36:38 +00:00
|
|
|
{"errors_count", std::make_shared<DataTypeUInt32>()},
|
2021-03-05 23:45:17 +00:00
|
|
|
{"slowdowns_count", std::make_shared<DataTypeUInt32>()},
|
2019-08-06 04:36:38 +00:00
|
|
|
{"estimated_recovery_time", std::make_shared<DataTypeUInt32>()}
|
2018-07-24 14:28:56 +00:00
|
|
|
};
|
2015-04-30 12:43:16 +00:00
|
|
|
}
|
|
|
|
|
2021-02-08 09:14:17 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
|
2015-04-30 12:43:16 +00:00
|
|
|
{
|
2021-06-26 14:15:57 +00:00
|
|
|
for (const auto & name_and_cluster : context->getClusters()->getContainer())
|
2021-02-08 09:14:17 +00:00
|
|
|
writeCluster(res_columns, name_and_cluster);
|
|
|
|
|
|
|
|
const auto databases = DatabaseCatalog::instance().getDatabases();
|
|
|
|
for (const auto & name_and_database : databases)
|
2018-07-24 18:46:23 +00:00
|
|
|
{
|
2021-04-01 13:08:18 +00:00
|
|
|
if (const auto * replicated = typeid_cast<const DatabaseReplicated *>(name_and_database.second.get()))
|
2021-04-01 12:21:29 +00:00
|
|
|
{
|
2022-08-18 14:13:27 +00:00
|
|
|
if (auto database_cluster = replicated->tryGetCluster())
|
|
|
|
writeCluster(res_columns, {name_and_database.first, database_cluster});
|
2021-04-01 12:21:29 +00:00
|
|
|
}
|
2021-02-08 09:14:17 +00:00
|
|
|
}
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-02-08 19:36:17 +00:00
|
|
|
void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const NameAndCluster & name_and_cluster)
|
2021-02-08 09:14:17 +00:00
|
|
|
{
|
|
|
|
const String & cluster_name = name_and_cluster.first;
|
|
|
|
const ClusterPtr & cluster = name_and_cluster.second;
|
|
|
|
const auto & shards_info = cluster->getShardsInfo();
|
|
|
|
const auto & addresses_with_failover = cluster->getShardsAddresses();
|
|
|
|
|
|
|
|
for (size_t shard_index = 0; shard_index < shards_info.size(); ++shard_index)
|
|
|
|
{
|
|
|
|
const auto & shard_info = shards_info[shard_index];
|
|
|
|
const auto & shard_addresses = addresses_with_failover[shard_index];
|
|
|
|
const auto pool_status = shard_info.pool->getStatus();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-02-08 09:14:17 +00:00
|
|
|
for (size_t replica_index = 0; replica_index < shard_addresses.size(); ++replica_index)
|
|
|
|
{
|
|
|
|
size_t i = 0;
|
|
|
|
const auto & address = shard_addresses[replica_index];
|
2018-12-02 02:17:08 +00:00
|
|
|
|
2021-02-08 09:14:17 +00:00
|
|
|
res_columns[i++]->insert(cluster_name);
|
|
|
|
res_columns[i++]->insert(shard_info.shard_num);
|
|
|
|
res_columns[i++]->insert(shard_info.weight);
|
|
|
|
res_columns[i++]->insert(replica_index + 1);
|
|
|
|
res_columns[i++]->insert(address.host_name);
|
|
|
|
auto resolved = address.getResolvedAddress();
|
|
|
|
res_columns[i++]->insert(resolved ? resolved->host().toString() : String());
|
|
|
|
res_columns[i++]->insert(address.port);
|
|
|
|
res_columns[i++]->insert(address.is_local);
|
|
|
|
res_columns[i++]->insert(address.user);
|
|
|
|
res_columns[i++]->insert(address.default_database);
|
|
|
|
res_columns[i++]->insert(pool_status[replica_index].error_count);
|
2021-03-05 23:45:17 +00:00
|
|
|
res_columns[i++]->insert(pool_status[replica_index].slowdown_count);
|
2021-02-08 09:14:17 +00:00
|
|
|
res_columns[i++]->insert(pool_status[replica_index].estimated_recovery_time.count());
|
2015-04-30 12:43:16 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|