naive split implementation for DistributeBlockOutputStream (IColumn::insertFrom). [#METR-12221]

This commit is contained in:
Andrey Mironov 2014-08-12 17:46:46 +04:00
parent 3c6a79f02b
commit e73bfe7b8a
6 changed files with 167 additions and 15 deletions

View File

@ -29,6 +29,14 @@ public:
/// Соединения с удалёнными серверами.
ConnectionPools pools;
struct ShardInfo
{
std::string dir_name;
int weight;
};
std::vector<ShardInfo> shard_info_vec;
std::vector<size_t> slot_to_shard;
/// используеться для выставления ограничения на размер таймаута
static Poco::Timespan saturate(const Poco::Timespan & v, const Poco::Timespan & limit);
@ -56,6 +64,7 @@ private:
String password;
Address(const String & config_prefix);
Address(const String & host, int port, const String & config_prefix);
Address(const String & host_port_, const String & user_, const String & password_);
};

View File

@ -0,0 +1,69 @@
#pragma once
#include <DB/Storages/StorageDistributed.h>
#include <iostream>
namespace DB
{
class DistributedBlockOutputStream : public IBlockOutputStream
{
public:
DistributedBlockOutputStream(
StorageDistributed & storage, Cluster & cluster,
const ExpressionActionsPtr & sharding_key_expr,
const std::string & key_column_name
)
: storage(storage), cluster(cluster)
, sharding_key_expr(sharding_key_expr)
, key_column_name(key_column_name)
{
}
virtual void write(const Block & block) override
{
if (sharding_key_expr && cluster.shard_info_vec.size() > 1)
splitWrite(block, block);
else
writeImpl(block);
}
private:
void splitWrite(const Block & block, Block block_with_key)
{
sharding_key_expr->execute(block_with_key);
const auto & key_column = block_with_key.getByName(key_column_name).column;
const auto total_weight = cluster.slot_to_shard.size();
/// seems like cloned blocks have the same underlying container
Blocks target_blocks(cluster.shard_info_vec.size(), block.cloneEmpty());
for (size_t row = 0; row < block.rows(); ++row)
{
const auto target_block_idx = key_column->get64(row) % total_weight;
auto & target_block = target_blocks[target_block_idx];
for (size_t col = 0; col < block.columns(); ++col)
{
target_block.getByPosition(col).column->insertFrom(
*block.getByPosition(col).column, row
);
}
}
for (size_t i = 0; i < target_blocks.size(); ++i)
writeImpl(target_blocks[i], i);
}
void writeImpl(const Block & block, const size_t shard_num = 0)
{
std::cout << "dummy write block of " << block.bytes() << " bytes to shard " << shard_num << std::endl;
}
StorageDistributed & storage;
Cluster & cluster;
ExpressionActionsPtr sharding_key_expr;
std::string key_column_name;
};
}

View File

@ -25,7 +25,8 @@ public:
const String & remote_database_, /// БД на удалённых серверах.
const String & remote_table_, /// Имя таблицы на удалённых серверах.
const String & cluster_name,
Context & context_);
Context & context_,
const ASTPtr & sharding_key_ = nullptr);
static StoragePtr create(
const std::string & name_, /// Имя таблицы.
@ -33,7 +34,8 @@ public:
const String & remote_database_, /// БД на удалённых серверах.
const String & remote_table_, /// Имя таблицы на удалённых серверах.
SharedPtr<Cluster> & owned_cluster_,
Context & context_);
Context & context_,
const ASTPtr & sharding_key_ = nullptr);
std::string getName() const { return "Distributed"; }
std::string getTableName() const { return name; }
@ -57,12 +59,16 @@ public:
size_t max_block_size = DEFAULT_BLOCK_SIZE,
unsigned threads = 1);
virtual BlockOutputStreamPtr write(ASTPtr query) override;
void drop() override {}
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) { name = new_table_name; }
/// в подтаблицах добавлять и удалять столбы нужно вручную
/// структура подтаблиц не проверяется
void alter(const AlterCommands & params, const String & database_name, const String & table_name, Context & context);
const ASTPtr & getShardingKey() const { return sharding_key; }
private:
StorageDistributed(
const std::string & name_,
@ -70,7 +76,8 @@ private:
const String & remote_database_,
const String & remote_table_,
Cluster & cluster_,
const Context & context_);
const Context & context_,
const ASTPtr & sharding_key_ = nullptr);
/// Создает копию запроса, меняет имена базы данных и таблицы.
ASTPtr rewriteQuery(ASTPtr query);
@ -91,6 +98,9 @@ private:
/// Соединения с удалёнными серверами.
Cluster & cluster;
ASTPtr sharding_key;
bool write_enabled;
};
}

View File

@ -7,6 +7,7 @@
namespace DB
{
Cluster::Address::Address(const String & config_prefix)
{
Poco::Util::AbstractConfiguration & config = Poco::Util::Application::instance().config();
@ -17,6 +18,14 @@ Cluster::Address::Address(const String & config_prefix)
password = config.getString(config_prefix + ".password", "");
}
Cluster::Address::Address(const String & host, const int port, const String & config_prefix)
: host_port(host, port)
{
Poco::Util::AbstractConfiguration & config = Poco::Util::Application::instance().config();
user = config.getString(config_prefix + ".user", "default");
password = config.getString(config_prefix + ".password", "");
}
Cluster::Address::Address(const String & host_port_, const String & user_, const String & password_)
: user(user_), password(password_)
@ -53,11 +62,18 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
String config_prefix = cluster_name + ".";
for (Poco::Util::AbstractConfiguration::Keys::const_iterator it = config_keys.begin(); it != config_keys.end(); ++it)
for (auto it = config_keys.begin(); it != config_keys.end(); ++it)
{
if (0 == strncmp(it->c_str(), "node", strlen("node")))
{
addresses.push_back(Address(config_prefix + *it));
const auto & prefix = config_prefix + *it;
const auto & host = config.getString(prefix + ".host");
const auto port = config.getInt(prefix + ".port");
const auto weight = config.getInt(prefix + ".weight", 1);
addresses.emplace_back(host, port, prefix);
slot_to_shard.insert(std::end(slot_to_shard), weight, shard_info_vec.size());
shard_info_vec.push_back({host + ':' + std::to_string(port), weight});
}
else if (0 == strncmp(it->c_str(), "shard", strlen("shard")))
{
@ -67,13 +83,33 @@ Cluster::Cluster(const Settings & settings, const DataTypeFactory & data_type_fa
addresses_with_failover.push_back(Addresses());
Addresses & replica_addresses = addresses_with_failover.back();
for (Poco::Util::AbstractConfiguration::Keys::const_iterator jt = replica_keys.begin(); jt != replica_keys.end(); ++jt)
const auto & partial_prefix = config_prefix + *it + ".";
const auto weight = config.getInt(partial_prefix + ".weight", 1);
std::string dir_name{};
auto first = false;
for (auto jt = replica_keys.begin(); jt != replica_keys.end(); ++jt)
{
if (0 == strncmp(jt->data(), "weight", strlen("weight")))
continue;
if (!first) first = true;
if (0 == strncmp(jt->c_str(), "replica", strlen("replica")))
replica_addresses.push_back(Address(config_prefix + *it + "." + *jt));
{
const auto & prefix = partial_prefix + *jt;
const auto & host = config.getString(prefix + ".host");
const auto port = config.getInt(prefix + ".port");
replica_addresses.emplace_back(host, port, prefix);
dir_name += (first ? "" : ",") + host + ':' + std::to_string(port);
}
else
throw Exception("Unknown element in config: " + *jt, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
}
slot_to_shard.insert(std::end(slot_to_shard), weight, shard_info_vec.size());
shard_info_vec.push_back({dir_name, weight});
}
else
throw Exception("Unknown element in config: " + *it, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);

View File

@ -5,6 +5,7 @@
#include <DB/Storages/StorageDistributed.h>
#include <DB/Storages/VirtualColumnFactory.h>
#include <DB/Storages/Distributed/DistributedBlockOutputStream.h>
#include <Poco/Net/NetworkInterface.h>
#include <DB/Client/ConnectionPool.h>
@ -23,12 +24,18 @@ StorageDistributed::StorageDistributed(
const String & remote_database_,
const String & remote_table_,
Cluster & cluster_,
const Context & context_)
const Context & context_,
const ASTPtr & sharding_key_)
: name(name_), columns(columns_),
remote_database(remote_database_), remote_table(remote_table_),
context(context_),
cluster(cluster_)
cluster(cluster_),
sharding_key(sharding_key_),
write_enabled(cluster.getLocalNodesNum() + cluster.pools.size() < 2 || sharding_key_)
{
for (auto & shard_info : cluster.shard_info_vec) {
std::cout << "shard '" << shard_info.dir_name << "' with weight " << shard_info.weight << std::endl;
}
}
StoragePtr StorageDistributed::create(
@ -37,10 +44,11 @@ StoragePtr StorageDistributed::create(
const String & remote_database_,
const String & remote_table_,
const String & cluster_name,
Context & context_)
Context & context_,
const ASTPtr & sharding_key_)
{
context_.initClusters();
return (new StorageDistributed(name_, columns_, remote_database_, remote_table_, context_.getCluster(cluster_name), context_))->thisPtr();
return (new StorageDistributed(name_, columns_, remote_database_, remote_table_, context_.getCluster(cluster_name), context_, sharding_key_))->thisPtr();
}
@ -50,9 +58,10 @@ StoragePtr StorageDistributed::create(
const String & remote_database_,
const String & remote_table_,
SharedPtr<Cluster> & owned_cluster_,
Context & context_)
Context & context_,
const ASTPtr & sharding_key_)
{
auto res = new StorageDistributed(name_, columns_, remote_database_, remote_table_, *owned_cluster_, context_);
auto res = new StorageDistributed(name_, columns_, remote_database_, remote_table_, *owned_cluster_, context_, sharding_key_);
/// Захватываем владение объектом-кластером.
res->owned_cluster = owned_cluster_;
@ -134,6 +143,21 @@ BlockInputStreams StorageDistributed::read(
return res;
}
BlockOutputStreamPtr StorageDistributed::write(ASTPtr query)
{
if (!write_enabled)
throw Exception{
"Method write is not supported by storage " + getName() + " with no sharding key provided",
ErrorCodes::NOT_IMPLEMENTED
};
return new DistributedBlockOutputStream{
*this, this->cluster,
sharding_key ? ExpressionAnalyzer(sharding_key, context, *columns).getActions(false) : nullptr,
sharding_key ? sharding_key->getColumnName() : std::string{}
};
}
void StorageDistributed::alter(const AlterCommands & params, const String & database_name, const String & table_name, Context & context)
{
auto lock = lockStructureForAlter();

View File

@ -176,7 +176,7 @@ StoragePtr StorageFactory::get(
ASTs & args = typeid_cast<ASTExpressionList &>(*args_func.at(0)).children;
if (args.size() != 3)
if (args.size() != 3 && args.size() != 4)
throw Exception("Storage Distributed requires 3 parameters"
" - name of configuration section with list of remote servers, name of remote database, name of remote table.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
@ -185,7 +185,11 @@ StoragePtr StorageFactory::get(
String remote_database = typeid_cast<ASTIdentifier &>(*args[1]).name;
String remote_table = typeid_cast<ASTIdentifier &>(*args[2]).name;
return StorageDistributed::create(table_name, columns, remote_database, remote_table, cluster_name, context);
const auto & sharding_key = args.size() == 4 ? args[3] : nullptr;
return StorageDistributed::create(
table_name, columns, remote_database, remote_table, cluster_name, context, sharding_key
);
}
else if (endsWith(name, "MergeTree"))
{