2015-09-24 03:50:09 +00:00
|
|
|
#include <DB/Storages/System/StorageSystemClusters.h>
|
2015-04-30 12:43:16 +00:00
|
|
|
#include <DB/Interpreters/Cluster.h>
|
|
|
|
#include <DB/Columns/ColumnString.h>
|
|
|
|
#include <DB/Columns/ColumnsNumber.h>
|
|
|
|
#include <DB/DataTypes/DataTypeString.h>
|
|
|
|
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
|
|
|
#include <DB/DataStreams/OneBlockInputStream.h>
|
|
|
|
#include <DB/Interpreters/Context.h>
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
|
2015-04-30 12:43:16 +00:00
|
|
|
StorageSystemClusters::StorageSystemClusters(const std::string & name_, Context & context_)
|
|
|
|
: name(name_)
|
|
|
|
, columns{
|
2016-05-28 07:48:40 +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>() },
|
2016-08-22 20:34:21 +00:00
|
|
|
{ "user", std::make_shared<DataTypeString>() },
|
|
|
|
{ "default_database", std::make_shared<DataTypeString>() }
|
2015-04-30 12:43:16 +00:00
|
|
|
}
|
|
|
|
, context(context_)
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
StoragePtr StorageSystemClusters::create(const std::string & name_, Context & context_)
|
|
|
|
{
|
2016-08-26 21:25:05 +00:00
|
|
|
return make_shared(name_, context_);
|
2015-04-30 12:43:16 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
BlockInputStreams StorageSystemClusters::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;
|
|
|
|
|
2016-05-28 05:31:36 +00:00
|
|
|
ColumnPtr cluster_column = std::make_shared<ColumnString>();
|
|
|
|
ColumnPtr shard_num_column = std::make_shared<ColumnUInt32>();
|
|
|
|
ColumnPtr shard_weight_column = std::make_shared<ColumnUInt32>();
|
|
|
|
ColumnPtr replica_num_column = std::make_shared<ColumnUInt32>();
|
|
|
|
ColumnPtr host_name_column = std::make_shared<ColumnString>();
|
|
|
|
ColumnPtr host_address_column = std::make_shared<ColumnString>();
|
|
|
|
ColumnPtr port_column = std::make_shared<ColumnUInt16>();
|
|
|
|
ColumnPtr is_local_column = std::make_shared<ColumnUInt8>();
|
|
|
|
ColumnPtr user_column = std::make_shared<ColumnString>();
|
2016-08-22 20:34:21 +00:00
|
|
|
ColumnPtr default_database_column = std::make_shared<ColumnString>();
|
2015-04-30 12:43:16 +00:00
|
|
|
|
|
|
|
auto updateColumns = [&](const std::string & cluster_name, const Cluster::ShardInfo & shard_info,
|
|
|
|
const Cluster::Address & address)
|
|
|
|
{
|
|
|
|
cluster_column->insert(cluster_name);
|
|
|
|
shard_num_column->insert(static_cast<UInt64>(shard_info.shard_num));
|
|
|
|
shard_weight_column->insert(static_cast<UInt64>(shard_info.weight));
|
|
|
|
replica_num_column->insert(static_cast<UInt64>(address.replica_num));
|
|
|
|
|
2015-05-05 10:13:45 +00:00
|
|
|
host_name_column->insert(address.host_name);
|
2015-05-28 21:41:28 +00:00
|
|
|
host_address_column->insert(address.resolved_address.host().toString());
|
|
|
|
port_column->insert(static_cast<UInt64>(address.port));
|
2016-03-29 16:58:52 +00:00
|
|
|
is_local_column->insert(static_cast<UInt64>(shard_info.isLocal()));
|
2015-04-30 12:43:16 +00:00
|
|
|
user_column->insert(address.user);
|
2016-08-22 20:34:21 +00:00
|
|
|
default_database_column->insert(address.default_database);
|
2015-04-30 12:43:16 +00:00
|
|
|
};
|
|
|
|
|
2016-10-14 15:06:46 +00:00
|
|
|
auto clusters = context.getClusters().getContainer();
|
2016-10-10 08:44:52 +00:00
|
|
|
for (const auto & entry : clusters)
|
2015-04-30 12:43:16 +00:00
|
|
|
{
|
|
|
|
const std::string cluster_name = entry.first;
|
2016-10-10 08:44:52 +00:00
|
|
|
const ClusterPtr cluster = entry.second;
|
|
|
|
const auto & addresses = cluster->getShardsAddresses();
|
|
|
|
const auto & addresses_with_failover = cluster->getShardsWithFailoverAddresses();
|
|
|
|
const auto & shards_info = cluster->getShardsInfo();
|
2015-04-30 12:43:16 +00:00
|
|
|
|
|
|
|
if (!addresses.empty())
|
|
|
|
{
|
|
|
|
auto it1 = addresses.cbegin();
|
|
|
|
auto it2 = shards_info.cbegin();
|
|
|
|
|
|
|
|
while (it1 != addresses.cend())
|
|
|
|
{
|
|
|
|
const auto & address = *it1;
|
|
|
|
const auto & shard_info = *it2;
|
|
|
|
|
|
|
|
updateColumns(cluster_name, shard_info, address);
|
|
|
|
|
|
|
|
++it1;
|
|
|
|
++it2;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else if (!addresses_with_failover.empty())
|
|
|
|
{
|
|
|
|
auto it1 = addresses_with_failover.cbegin();
|
|
|
|
auto it2 = shards_info.cbegin();
|
|
|
|
|
|
|
|
while (it1 != addresses_with_failover.cend())
|
|
|
|
{
|
|
|
|
const auto & addresses = *it1;
|
|
|
|
const auto & shard_info = *it2;
|
|
|
|
|
|
|
|
for (const auto & address : addresses)
|
|
|
|
updateColumns(cluster_name, shard_info, address);
|
|
|
|
|
|
|
|
++it1;
|
|
|
|
++it2;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-08-22 20:34:21 +00:00
|
|
|
Block block
|
|
|
|
{
|
|
|
|
{cluster_column, std::make_shared<DataTypeString>(), "cluster"},
|
|
|
|
{shard_num_column, std::make_shared<DataTypeUInt32>(), "shard_num"},
|
|
|
|
{shard_weight_column, std::make_shared<DataTypeUInt32>(), "shard_weight"},
|
|
|
|
{replica_num_column, std::make_shared<DataTypeUInt32>(), "replica_num"},
|
|
|
|
{host_name_column, std::make_shared<DataTypeString>(), "host_name"},
|
|
|
|
{host_address_column, std::make_shared<DataTypeString>(), "host_address"},
|
|
|
|
{port_column, std::make_shared<DataTypeUInt16>(), "port"},
|
|
|
|
{is_local_column, std::make_shared<DataTypeUInt8>(), "is_local"},
|
|
|
|
{user_column, std::make_shared<DataTypeString>(), "user"},
|
|
|
|
{default_database_column, std::make_shared<DataTypeString>(), "default_database"}
|
|
|
|
};
|
2015-04-30 12:43:16 +00:00
|
|
|
|
2016-05-28 12:22:22 +00:00
|
|
|
return BlockInputStreams{ 1, std::make_shared<OneBlockInputStream>(block) };
|
2015-04-30 12:43:16 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|