dbms: Added more comments. Made some code more understandable. [#METR-18510]

This commit is contained in:
Alexey Arno 2016-03-28 16:00:00 +03:00
parent 639413da5e
commit 09985be3ba
12 changed files with 74 additions and 31 deletions

View File

@ -8,10 +8,15 @@
#include <common/logger_useful.h>
/// This type specifies the possible behaviors of an object pool allocator.
enum class PoolMode
{
/// Get exactly one object from a given pool.
GET_ONE = 0,
/// Get a number of objects from a given pool, this number being
/// read from a configuration parameter.
GET_MANY,
/// Get all the objects from a given pool.
GET_ALL
};

View File

@ -42,6 +42,7 @@ public:
~RemoteBlockInputStream() override;
/// Specify how we allocate connections on a shard.
void setPoolMode(PoolMode pool_mode_);
/// Кроме блоков, получить информацию о блоках.

View File

@ -18,7 +18,7 @@ public:
const Settings & settings, ThrottlerPtr throttler, const Context & context) override;
BlockInputStreamPtr createRemote(ConnectionPoolsPtr & pools, const std::string & query,
const Settings & settings, ThrottlerPtr throttler, const Context & context) override;
bool localAndRemote() const override;
PoolMode getPoolMode() const override;
};
}

View File

@ -18,7 +18,7 @@ public:
const Settings & settings, ThrottlerPtr throttler, const Context & context) override;
BlockInputStreamPtr createRemote(ConnectionPoolsPtr & pools, const std::string & query,
const Settings & settings, ThrottlerPtr throttler, const Context & context) override;
bool localAndRemote() const override;
PoolMode getPoolMode() const override;
};
}

View File

@ -16,17 +16,23 @@ class Throttler;
namespace ClusterProxy
{
/// Base class for the implementation of the details of distributed query
/// execution that are specific to the query type.
class IQueryConstructor
{
public:
virtual ~IQueryConstructor() {}
/// Create an input stream for local query execution.
virtual BlockInputStreamPtr createLocal(ASTPtr query_ast, const Context & context, const Cluster::Address & address) = 0;
/// Create an input stream for remote query execution on one shard.
virtual BlockInputStreamPtr createRemote(IConnectionPool * pool, const std::string & query,
const Settings & settings, ThrottlerPtr throttler, const Context & context) = 0;
/// Create an input stream for remote query execution on one or more shards.
virtual BlockInputStreamPtr createRemote(ConnectionPoolsPtr & pools, const std::string & query,
const Settings & new_settings, ThrottlerPtr throttler, const Context & context) = 0;
virtual bool localAndRemote() const = 0;
/// Specify how we allocate connections on a shard.
virtual PoolMode getPoolMode() const = 0;
};
}

View File

@ -16,11 +16,20 @@ namespace ClusterProxy
class IQueryConstructor;
/// This class is designed for distributed queries execution. It hides from
/// the caller the details about the actual locations at which a distributed
/// query is performed. Depending on the type of query to be performed,
/// (currently SELECT, DESCRIBE, or ALTER (for resharding)), a so-called
/// query constructor is specified. Such an object states, among other things,
/// how connections must be allocated for remote execution.
class Query
{
public:
Query(IQueryConstructor & query_constructor_, const Cluster & cluster_,
ASTPtr query_ast_, const Context & context_, const Settings & settings_, bool enable_shard_multiplexing_);
/// For each location at which we perform the query, create an input stream
/// from which we can fetch the result.
BlockInputStreams execute();
private:

View File

@ -20,7 +20,7 @@ public:
const Settings & settings, ThrottlerPtr throttler, const Context & context) override;
BlockInputStreamPtr createRemote(ConnectionPoolsPtr & pools, const std::string & query,
const Settings & settings, ThrottlerPtr throttler, const Context & context) override;
bool localAndRemote() const override;
PoolMode getPoolMode() const override;
private:
const QueryProcessingStage::Enum & processed_stage;

View File

@ -5,6 +5,13 @@
namespace DB
{
namespace
{
constexpr PoolMode pool_mode = PoolMode::GET_ONE;
}
namespace ClusterProxy
{
@ -18,7 +25,7 @@ BlockInputStreamPtr AlterQueryConstructor::createRemote(IConnectionPool * pool,
const Settings & settings, ThrottlerPtr throttler, const Context & context)
{
auto stream = new RemoteBlockInputStream{pool, query, &settings, throttler};
stream->setPoolMode(PoolMode::GET_ONE);
stream->setPoolMode(pool_mode);
return stream;
}
@ -26,13 +33,13 @@ BlockInputStreamPtr AlterQueryConstructor::createRemote(ConnectionPoolsPtr & poo
const Settings & settings, ThrottlerPtr throttler, const Context & context)
{
auto stream = new RemoteBlockInputStream{pools, query, &settings, throttler};
stream->setPoolMode(PoolMode::GET_ONE);
stream->setPoolMode(pool_mode);
return stream;
}
bool AlterQueryConstructor::localAndRemote() const
PoolMode AlterQueryConstructor::getPoolMode() const
{
return false;
return pool_mode;
}
}

View File

@ -9,16 +9,20 @@ namespace DB
namespace
{
BlockExtraInfo toBlockExtraInfo(const Cluster::Address & address)
{
BlockExtraInfo block_extra_info;
block_extra_info.host = address.host_name;
block_extra_info.resolved_address = address.resolved_address.toString();
block_extra_info.port = address.port;
block_extra_info.user = address.user;
block_extra_info.is_valid = true;
return block_extra_info;
}
constexpr PoolMode pool_mode = PoolMode::GET_ALL;
BlockExtraInfo toBlockExtraInfo(const Cluster::Address & address)
{
BlockExtraInfo block_extra_info;
block_extra_info.host = address.host_name;
block_extra_info.resolved_address = address.resolved_address.toString();
block_extra_info.port = address.port;
block_extra_info.user = address.user;
block_extra_info.is_valid = true;
return block_extra_info;
}
}
namespace ClusterProxy
@ -42,7 +46,7 @@ BlockInputStreamPtr DescribeQueryConstructor::createRemote(IConnectionPool * poo
const Settings & settings, ThrottlerPtr throttler, const Context & context)
{
auto stream = new RemoteBlockInputStream{pool, query, &settings, throttler};
stream->setPoolMode(PoolMode::GET_ALL);
stream->setPoolMode(pool_mode);
stream->appendExtraInfo();
return stream;
}
@ -51,16 +55,17 @@ BlockInputStreamPtr DescribeQueryConstructor::createRemote(ConnectionPoolsPtr &
const Settings & settings, ThrottlerPtr throttler, const Context & context)
{
auto stream = new RemoteBlockInputStream{pools, query, &settings, throttler};
stream->setPoolMode(PoolMode::GET_ALL);
stream->setPoolMode(pool_mode);
stream->appendExtraInfo();
return stream;
}
bool DescribeQueryConstructor::localAndRemote() const
PoolMode DescribeQueryConstructor::getPoolMode() const
{
return true;
return pool_mode;
}
}
}

View File

@ -43,7 +43,7 @@ BlockInputStreams Query::execute()
size_t remote_count = 0;
if (query_constructor.localAndRemote())
if (query_constructor.getPoolMode() == PoolMode::GET_ALL)
{
for (const auto & shard_info : cluster.getShardsInfo())
{
@ -76,7 +76,12 @@ BlockInputStreams Query::execute()
for (const auto & shard_info : cluster.getShardsInfo())
{
bool create_local_queries = shard_info.isLocal();
bool create_remote_queries = query_constructor.localAndRemote() ? shard_info.hasRemoteConnections() : !create_local_queries;
bool create_remote_queries;
if (query_constructor.getPoolMode() == PoolMode::GET_ALL)
create_remote_queries = shard_info.hasRemoteConnections();
else
create_remote_queries = !create_local_queries;
if (create_local_queries)
{

View File

@ -6,6 +6,13 @@
namespace DB
{
namespace
{
constexpr PoolMode pool_mode = PoolMode::GET_MANY;
}
namespace ClusterProxy
{
@ -31,7 +38,7 @@ BlockInputStreamPtr SelectQueryConstructor::createRemote(IConnectionPool * pool,
const Settings & settings, ThrottlerPtr throttler, const Context & context)
{
auto stream = new RemoteBlockInputStream{pool, query, &settings, throttler, external_tables, processed_stage, context};
stream->setPoolMode(PoolMode::GET_MANY);
stream->setPoolMode(pool_mode);
return stream;
}
@ -39,13 +46,13 @@ BlockInputStreamPtr SelectQueryConstructor::createRemote(ConnectionPoolsPtr & po
const Settings & settings, ThrottlerPtr throttler, const Context & context)
{
auto stream = new RemoteBlockInputStream{pools, query, &settings, throttler, external_tables, processed_stage, context};
stream->setPoolMode(PoolMode::GET_MANY);
stream->setPoolMode(pool_mode);
return stream;
}
bool SelectQueryConstructor::localAndRemote() const
PoolMode SelectQueryConstructor::getPoolMode() const
{
return false;
return pool_mode;
}
}

View File

@ -659,9 +659,7 @@ void ReshardingWorker::perform(const std::string & job_descriptor, const std::st
hardCleanup();
}
else if (ex.code() == ErrorCodes::RESHARDING_COORDINATOR_DELETED)
{
/// nothing here
}
unfreezeSourcePartition();
else if (ex.code() == ErrorCodes::RESHARDING_DISTRIBUTED_JOB_ON_HOLD)
{
/// The current distributed job is on hold and one or more required nodes