2017-04-01 09:19:00 +00:00
|
|
|
#include <DataStreams/RemoteBlockInputStream.h>
|
|
|
|
#include <DataStreams/BlockExtraInfoInputStream.h>
|
|
|
|
#include <DataStreams/UnionBlockInputStream.h>
|
|
|
|
|
|
|
|
#include <Databases/IDatabase.h>
|
|
|
|
|
|
|
|
#include <Storages/StorageDistributed.h>
|
|
|
|
#include <Storages/VirtualColumnFactory.h>
|
|
|
|
#include <Storages/Distributed/DistributedBlockOutputStream.h>
|
|
|
|
#include <Storages/Distributed/DirectoryMonitor.h>
|
2017-12-30 00:36:06 +00:00
|
|
|
#include <Storages/StorageFactory.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
|
|
|
|
#include <Common/escapeForFileName.h>
|
2017-07-13 20:58:19 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
|
|
|
|
#include <Parsers/ASTInsertQuery.h>
|
|
|
|
#include <Parsers/ASTSelectQuery.h>
|
|
|
|
#include <Parsers/ASTIdentifier.h>
|
|
|
|
#include <Parsers/TablePropertiesQueriesASTs.h>
|
|
|
|
#include <Parsers/ParserAlterQuery.h>
|
|
|
|
#include <Parsers/parseQuery.h>
|
|
|
|
#include <Parsers/ASTWeightedZooKeeperPath.h>
|
|
|
|
#include <Parsers/ASTLiteral.h>
|
|
|
|
#include <Parsers/ASTExpressionList.h>
|
|
|
|
|
|
|
|
#include <Interpreters/InterpreterSelectQuery.h>
|
|
|
|
#include <Interpreters/InterpreterAlterQuery.h>
|
|
|
|
#include <Interpreters/InterpreterDescribeQuery.h>
|
|
|
|
#include <Interpreters/ExpressionAnalyzer.h>
|
2017-12-30 00:36:06 +00:00
|
|
|
#include <Interpreters/evaluateConstantExpression.h>
|
2017-07-28 12:58:24 +00:00
|
|
|
#include <Interpreters/ClusterProxy/executeQuery.h>
|
|
|
|
#include <Interpreters/ClusterProxy/SelectStreamFactory.h>
|
|
|
|
#include <Interpreters/ClusterProxy/DescribeStreamFactory.h>
|
|
|
|
#include <Interpreters/ClusterProxy/AlterStreamFactory.h>
|
2017-12-30 00:36:06 +00:00
|
|
|
#include <Interpreters/getClusterName.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
|
|
|
|
#include <Core/Field.h>
|
2012-05-21 20:38:34 +00:00
|
|
|
|
2016-12-12 03:33:34 +00:00
|
|
|
#include <Poco/DirectoryIterator.h>
|
|
|
|
|
2015-02-10 21:10:58 +00:00
|
|
|
#include <memory>
|
2014-08-19 08:04:13 +00:00
|
|
|
|
2017-05-10 06:39:37 +00:00
|
|
|
#include <boost/filesystem.hpp>
|
2017-11-01 14:34:05 +00:00
|
|
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
2017-05-10 06:39:37 +00:00
|
|
|
|
2016-12-12 03:33:34 +00:00
|
|
|
|
2012-05-21 20:38:34 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int STORAGE_REQUIRES_PARAMETER;
|
2017-12-30 00:36:06 +00:00
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
|
|
|
extern const int INCORRECT_NUMBER_OF_COLUMNS;
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2014-08-21 12:07:29 +00:00
|
|
|
namespace
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-05-10 06:39:37 +00:00
|
|
|
/// select query has database and table names as AST pointers
|
|
|
|
/// Creates a copy of query, changes database and table names.
|
|
|
|
ASTPtr rewriteSelectQuery(const ASTPtr & query, const std::string & database, const std::string & table)
|
|
|
|
{
|
2017-06-15 14:07:31 +00:00
|
|
|
auto modified_query_ast = typeid_cast<const ASTSelectQuery &>(*query).cloneFirstSelect();
|
|
|
|
modified_query_ast->replaceDatabaseAndTable(database, table);
|
2017-05-10 06:39:37 +00:00
|
|
|
return modified_query_ast;
|
|
|
|
}
|
|
|
|
|
|
|
|
/// insert query has database and table names as bare strings
|
|
|
|
/// Creates a copy of query, changes the database and table names.
|
|
|
|
ASTPtr rewriteInsertQuery(const ASTPtr & query, const std::string & database, const std::string & table)
|
|
|
|
{
|
|
|
|
auto modified_query_ast = query->clone();
|
|
|
|
|
|
|
|
auto & actual_query = typeid_cast<ASTInsertQuery &>(*modified_query_ast);
|
|
|
|
actual_query.database = database;
|
|
|
|
actual_query.table = table;
|
2017-11-02 14:01:11 +00:00
|
|
|
actual_query.table_function = nullptr;
|
2017-05-10 06:39:37 +00:00
|
|
|
/// make sure query is not INSERT SELECT
|
|
|
|
actual_query.select = nullptr;
|
|
|
|
|
|
|
|
return modified_query_ast;
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Calculate maximum number in file names in directory and all subdirectories.
|
|
|
|
/// To ensure global order of data blocks yet to be sent across server restarts.
|
|
|
|
UInt64 getMaximumFileNumber(const std::string & path)
|
|
|
|
{
|
|
|
|
UInt64 res = 0;
|
|
|
|
|
|
|
|
boost::filesystem::recursive_directory_iterator begin(path);
|
|
|
|
boost::filesystem::recursive_directory_iterator end;
|
|
|
|
for (auto it = begin; it != end; ++it)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-05-10 06:39:37 +00:00
|
|
|
const auto & path = it->path();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-05-10 06:39:37 +00:00
|
|
|
if (it->status().type() != boost::filesystem::regular_file || !endsWith(path.filename().string(), ".bin"))
|
|
|
|
continue;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-05-10 06:39:37 +00:00
|
|
|
UInt64 num = 0;
|
|
|
|
try
|
|
|
|
{
|
|
|
|
num = parse<UInt64>(path.filename().stem().string());
|
|
|
|
}
|
|
|
|
catch (Exception & e)
|
|
|
|
{
|
|
|
|
e.addMessage("Unexpected file name " + path.filename().string() + " found at " + path.parent_path().string() + ", should have numeric base name.");
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (num > res)
|
|
|
|
res = num;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2017-05-10 06:39:37 +00:00
|
|
|
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
|
|
|
void initializeFileNamesIncrement(const std::string & path, SimpleIncrement & increment)
|
|
|
|
{
|
|
|
|
if (!path.empty())
|
|
|
|
increment.set(getMaximumFileNumber(path));
|
|
|
|
}
|
|
|
|
|
2014-08-13 12:52:30 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-06-06 18:48:38 +00:00
|
|
|
/// For destruction of std::unique_ptr of type that is incomplete in class definition.
|
|
|
|
StorageDistributed::~StorageDistributed() = default;
|
|
|
|
|
|
|
|
|
2012-11-06 17:04:38 +00:00
|
|
|
StorageDistributed::StorageDistributed(
|
2017-04-01 07:20:54 +00:00
|
|
|
const std::string & name_,
|
2017-12-25 21:57:29 +00:00
|
|
|
const NamesAndTypesList & columns_,
|
2017-04-01 07:20:54 +00:00
|
|
|
const String & remote_database_,
|
|
|
|
const String & remote_table_,
|
|
|
|
const String & cluster_name_,
|
2017-05-23 18:37:14 +00:00
|
|
|
const Context & context_,
|
2017-04-01 07:20:54 +00:00
|
|
|
const ASTPtr & sharding_key_,
|
|
|
|
const String & data_path_)
|
|
|
|
: name(name_), columns(columns_),
|
|
|
|
remote_database(remote_database_), remote_table(remote_table_),
|
|
|
|
context(context_), cluster_name(cluster_name_), has_sharding_key(sharding_key_),
|
2017-12-25 21:10:46 +00:00
|
|
|
sharding_key_expr(sharding_key_ ? ExpressionAnalyzer(sharding_key_, context, nullptr, columns).getActions(false) : nullptr),
|
2017-04-01 07:20:54 +00:00
|
|
|
sharding_key_column_name(sharding_key_ ? sharding_key_->getColumnName() : String{}),
|
|
|
|
path(data_path_.empty() ? "" : (data_path_ + escapeForFileName(name) + '/'))
|
2013-11-28 10:31:17 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2017-03-09 03:34:09 +00:00
|
|
|
|
2014-09-30 03:08:47 +00:00
|
|
|
StorageDistributed::StorageDistributed(
|
2017-04-01 07:20:54 +00:00
|
|
|
const std::string & name_,
|
2017-12-25 21:57:29 +00:00
|
|
|
const NamesAndTypesList & columns_,
|
|
|
|
const NamesAndTypesList & materialized_columns_,
|
|
|
|
const NamesAndTypesList & alias_columns_,
|
2017-04-01 07:20:54 +00:00
|
|
|
const ColumnDefaults & column_defaults_,
|
|
|
|
const String & remote_database_,
|
|
|
|
const String & remote_table_,
|
|
|
|
const String & cluster_name_,
|
2017-05-23 18:37:14 +00:00
|
|
|
const Context & context_,
|
2017-04-01 07:20:54 +00:00
|
|
|
const ASTPtr & sharding_key_,
|
|
|
|
const String & data_path_)
|
|
|
|
: IStorage{materialized_columns_, alias_columns_, column_defaults_},
|
|
|
|
name(name_), columns(columns_),
|
|
|
|
remote_database(remote_database_), remote_table(remote_table_),
|
|
|
|
context(context_), cluster_name(cluster_name_), has_sharding_key(sharding_key_),
|
2017-12-25 21:10:46 +00:00
|
|
|
sharding_key_expr(sharding_key_ ? ExpressionAnalyzer(sharding_key_, context, nullptr, columns).getActions(false) : nullptr),
|
2017-04-01 07:20:54 +00:00
|
|
|
sharding_key_column_name(sharding_key_ ? sharding_key_->getColumnName() : String{}),
|
|
|
|
path(data_path_.empty() ? "" : (data_path_ + escapeForFileName(name) + '/'))
|
2014-09-30 03:08:47 +00:00
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2017-03-09 03:34:09 +00:00
|
|
|
|
2017-12-11 18:09:20 +00:00
|
|
|
StoragePtr StorageDistributed::createWithOwnCluster(
|
2017-04-01 07:20:54 +00:00
|
|
|
const std::string & name_,
|
2017-12-25 21:57:29 +00:00
|
|
|
const NamesAndTypesList & columns_,
|
2017-04-01 07:20:54 +00:00
|
|
|
const String & remote_database_,
|
|
|
|
const String & remote_table_,
|
2017-12-11 18:09:20 +00:00
|
|
|
ClusterPtr & owned_cluster_,
|
2017-05-23 18:37:14 +00:00
|
|
|
const Context & context_)
|
2014-02-07 15:11:57 +00:00
|
|
|
{
|
2017-11-04 03:20:18 +00:00
|
|
|
auto res = ext::shared_ptr_helper<StorageDistributed>::create(
|
2017-04-01 07:20:54 +00:00
|
|
|
name_, columns_, remote_database_,
|
2017-06-06 17:06:14 +00:00
|
|
|
remote_table_, String{}, context_);
|
2014-02-23 02:27:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
res->owned_cluster = owned_cluster_;
|
2014-02-23 02:27:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return res;
|
2014-02-07 15:11:57 +00:00
|
|
|
}
|
|
|
|
|
2017-03-09 03:34:09 +00:00
|
|
|
|
2012-05-21 20:38:34 +00:00
|
|
|
BlockInputStreams StorageDistributed::read(
|
2017-12-01 21:13:25 +00:00
|
|
|
const Names & /*column_names*/,
|
2017-07-15 03:48:36 +00:00
|
|
|
const SelectQueryInfo & query_info,
|
2017-04-01 07:20:54 +00:00
|
|
|
const Context & context,
|
|
|
|
QueryProcessingStage::Enum & processed_stage,
|
2017-12-01 21:13:25 +00:00
|
|
|
const size_t /*max_block_size*/,
|
|
|
|
const unsigned /*num_streams*/)
|
2012-05-21 20:38:34 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
auto cluster = getCluster();
|
2016-10-10 08:44:52 +00:00
|
|
|
|
2017-05-24 21:06:29 +00:00
|
|
|
const Settings & settings = context.getSettingsRef();
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t result_size = (cluster->getRemoteShardCount() * settings.max_parallel_replicas) + cluster->getLocalShardCount();
|
2014-02-11 17:10:48 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
processed_stage = result_size == 1 || settings.distributed_group_by_no_merge
|
|
|
|
? QueryProcessingStage::Complete
|
|
|
|
: QueryProcessingStage::WithMergeableState;
|
2014-02-11 17:10:48 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
const auto & modified_query_ast = rewriteSelectQuery(
|
2017-07-15 03:48:36 +00:00
|
|
|
query_info.query, remote_database, remote_table);
|
2015-02-10 20:48:17 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Tables external_tables;
|
2015-09-05 01:22:09 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (settings.global_subqueries_method == GlobalSubqueriesMethod::PUSH)
|
|
|
|
external_tables = context.getExternalTables();
|
2015-03-15 08:05:28 +00:00
|
|
|
|
2017-07-28 12:58:24 +00:00
|
|
|
ClusterProxy::SelectStreamFactory select_stream_factory(
|
2017-07-03 21:04:10 +00:00
|
|
|
processed_stage, QualifiedTableName{remote_database, remote_table}, external_tables);
|
2015-11-06 17:44:01 +00:00
|
|
|
|
2017-07-28 12:58:24 +00:00
|
|
|
return ClusterProxy::executeQuery(
|
2017-07-31 14:07:40 +00:00
|
|
|
select_stream_factory, cluster, modified_query_ast, context, settings);
|
2012-05-21 20:38:34 +00:00
|
|
|
}
|
|
|
|
|
2017-03-09 03:34:09 +00:00
|
|
|
|
2017-05-21 22:25:25 +00:00
|
|
|
BlockOutputStreamPtr StorageDistributed::write(const ASTPtr & query, const Settings & settings)
|
2014-08-12 13:46:46 +00:00
|
|
|
{
|
2017-12-11 18:09:20 +00:00
|
|
|
auto cluster = owned_cluster ? owned_cluster : context.getCluster(cluster_name);
|
2016-10-10 08:44:52 +00:00
|
|
|
|
2017-12-11 18:09:20 +00:00
|
|
|
/// TODO: !path.empty() can be replaced by !owned_cluster or !cluster_name.empty() ?
|
|
|
|
/// owned_cluster for remote table function use sync insertion => doesn't need a path.
|
|
|
|
bool write_enabled = (!path.empty() || owned_cluster)
|
2017-11-02 14:01:11 +00:00
|
|
|
&& (((cluster->getLocalShardCount() + cluster->getRemoteShardCount()) < 2) || has_sharding_key);
|
2016-10-10 08:44:52 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!write_enabled)
|
|
|
|
throw Exception{
|
|
|
|
"Method write is not supported by storage " + getName() +
|
|
|
|
" with more than one shard and no sharding key provided",
|
|
|
|
ErrorCodes::STORAGE_REQUIRES_PARAMETER};
|
2014-08-21 12:07:29 +00:00
|
|
|
|
2017-12-11 18:09:20 +00:00
|
|
|
bool insert_sync = settings.insert_distributed_sync || owned_cluster;
|
2017-11-02 14:01:11 +00:00
|
|
|
auto timeout = settings.insert_distributed_timeout;
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// DistributedBlockOutputStream will not own cluster, but will own ConnectionPools of the cluster
|
|
|
|
return std::make_shared<DistributedBlockOutputStream>(
|
2017-11-02 14:01:11 +00:00
|
|
|
*this, rewriteInsertQuery(query, remote_database, remote_table), cluster, insert_sync, timeout);
|
2014-08-12 13:46:46 +00:00
|
|
|
}
|
|
|
|
|
2017-03-09 03:34:09 +00:00
|
|
|
|
2016-01-28 01:00:27 +00:00
|
|
|
void StorageDistributed::alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context)
|
2013-09-23 12:01:19 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
for (const auto & param : params)
|
|
|
|
if (param.type == AlterCommand::MODIFY_PRIMARY_KEY)
|
|
|
|
throw Exception("Storage engine " + getName() + " doesn't support primary key.", ErrorCodes::NOT_IMPLEMENTED);
|
2016-05-05 18:28:46 +00:00
|
|
|
|
2017-09-01 15:05:23 +00:00
|
|
|
auto lock = lockStructureForAlter(__PRETTY_FUNCTION__);
|
2017-12-25 21:10:46 +00:00
|
|
|
params.apply(columns, materialized_columns, alias_columns, column_defaults);
|
2016-05-13 21:08:19 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
context.getDatabase(database_name)->alterTable(
|
|
|
|
context, table_name,
|
2017-12-25 21:10:46 +00:00
|
|
|
columns, materialized_columns, alias_columns, column_defaults, {});
|
2013-09-23 12:01:19 +00:00
|
|
|
}
|
2014-02-04 15:44:15 +00:00
|
|
|
|
2017-03-09 03:34:09 +00:00
|
|
|
|
2017-06-06 17:06:14 +00:00
|
|
|
void StorageDistributed::startup()
|
|
|
|
{
|
|
|
|
createDirectoryMonitors();
|
|
|
|
initializeFileNamesIncrement(path, file_names_increment);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2014-08-13 11:26:13 +00:00
|
|
|
void StorageDistributed::shutdown()
|
|
|
|
{
|
2017-07-27 15:24:39 +00:00
|
|
|
cluster_nodes_data.clear();
|
2014-08-13 11:26:13 +00:00
|
|
|
}
|
|
|
|
|
2017-03-09 03:34:09 +00:00
|
|
|
|
2016-01-28 01:00:27 +00:00
|
|
|
BlockInputStreams StorageDistributed::describe(const Context & context, const Settings & settings)
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Create DESCRIBE TABLE query.
|
|
|
|
auto cluster = getCluster();
|
2016-01-28 01:00:27 +00:00
|
|
|
|
2017-11-01 14:34:05 +00:00
|
|
|
auto describe_query = std::make_shared<ASTDescribeQuery>();
|
2016-01-28 01:00:27 +00:00
|
|
|
|
2017-11-01 14:34:05 +00:00
|
|
|
std::string name = remote_database + '.' + remote_table;
|
|
|
|
|
|
|
|
auto id = std::make_shared<ASTIdentifier>();
|
|
|
|
id->name = name;
|
|
|
|
|
|
|
|
auto desc_database = std::make_shared<ASTIdentifier>();
|
|
|
|
auto desc_table = std::make_shared<ASTIdentifier>();
|
|
|
|
desc_database->name = remote_database;
|
|
|
|
desc_table->name = remote_table;
|
|
|
|
|
|
|
|
id->children.push_back(desc_database);
|
|
|
|
id->children.push_back(desc_table);
|
|
|
|
|
|
|
|
auto table_expression = std::make_shared<ASTTableExpression>();
|
|
|
|
table_expression->database_and_table_name = id;
|
|
|
|
|
|
|
|
describe_query->table_expression = table_expression;
|
2016-01-28 01:00:27 +00:00
|
|
|
|
2017-07-28 12:58:24 +00:00
|
|
|
ClusterProxy::DescribeStreamFactory describe_stream_factory;
|
2015-10-12 14:53:16 +00:00
|
|
|
|
2017-07-28 12:58:24 +00:00
|
|
|
return ClusterProxy::executeQuery(
|
2017-11-01 14:34:05 +00:00
|
|
|
describe_stream_factory, cluster, describe_query, context, settings);
|
2015-10-12 14:53:16 +00:00
|
|
|
}
|
|
|
|
|
2017-03-09 03:34:09 +00:00
|
|
|
|
2017-12-25 21:57:29 +00:00
|
|
|
NameAndTypePair StorageDistributed::getColumn(const String & column_name) const
|
2014-07-31 09:11:49 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (const auto & type = VirtualColumnFactory::tryGetType(column_name))
|
|
|
|
return { column_name, type };
|
2014-07-31 09:11:49 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return getRealColumn(column_name);
|
2014-07-31 09:11:49 +00:00
|
|
|
}
|
|
|
|
|
2017-03-09 03:34:09 +00:00
|
|
|
|
2014-07-31 09:11:49 +00:00
|
|
|
bool StorageDistributed::hasColumn(const String & column_name) const
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
return VirtualColumnFactory::hasColumn(column_name) || IStorage::hasColumn(column_name);
|
2014-07-31 09:11:49 +00:00
|
|
|
}
|
|
|
|
|
2014-08-15 09:50:05 +00:00
|
|
|
void StorageDistributed::createDirectoryMonitors()
|
2014-08-13 11:26:13 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (path.empty())
|
|
|
|
return;
|
2014-12-30 03:53:41 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Poco::File{path}.createDirectory();
|
2014-08-14 11:50:36 +00:00
|
|
|
|
2017-05-10 06:39:37 +00:00
|
|
|
boost::filesystem::directory_iterator begin(path);
|
|
|
|
boost::filesystem::directory_iterator end;
|
|
|
|
for (auto it = begin; it != end; ++it)
|
|
|
|
if (it->status().type() == boost::filesystem::directory_file)
|
2017-07-27 15:24:39 +00:00
|
|
|
requireDirectoryMonitor(it->path().filename().string());
|
2014-08-13 11:26:13 +00:00
|
|
|
}
|
|
|
|
|
2017-03-09 03:34:09 +00:00
|
|
|
|
2017-07-27 15:24:39 +00:00
|
|
|
void StorageDistributed::requireDirectoryMonitor(const std::string & name)
|
|
|
|
{
|
|
|
|
cluster_nodes_data[name].requireDirectoryMonitor(name, *this);
|
|
|
|
}
|
|
|
|
|
|
|
|
ConnectionPoolPtr StorageDistributed::requireConnectionPool(const std::string & name)
|
2014-08-19 08:04:13 +00:00
|
|
|
{
|
2017-07-27 15:24:39 +00:00
|
|
|
auto & node_data = cluster_nodes_data[name];
|
|
|
|
node_data.requireConnectionPool(name, *this);
|
|
|
|
return node_data.conneciton_pool;
|
2014-08-19 08:04:13 +00:00
|
|
|
}
|
|
|
|
|
2015-09-18 13:36:10 +00:00
|
|
|
size_t StorageDistributed::getShardCount() const
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
return getCluster()->getRemoteShardCount();
|
2016-10-10 08:44:52 +00:00
|
|
|
}
|
|
|
|
|
2017-03-09 03:34:09 +00:00
|
|
|
|
2016-10-10 08:44:52 +00:00
|
|
|
ClusterPtr StorageDistributed::getCluster() const
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
return (owned_cluster) ? owned_cluster : context.getCluster(cluster_name);
|
2015-09-18 13:36:10 +00:00
|
|
|
}
|
|
|
|
|
2017-07-27 15:24:39 +00:00
|
|
|
void StorageDistributed::ClusterNodeData::requireConnectionPool(const std::string & name, const StorageDistributed & storage)
|
|
|
|
{
|
|
|
|
if (!conneciton_pool)
|
|
|
|
conneciton_pool = StorageDistributedDirectoryMonitor::createPool(name, storage);
|
|
|
|
}
|
|
|
|
|
|
|
|
void StorageDistributed::ClusterNodeData::requireDirectoryMonitor(const std::string & name, StorageDistributed & storage)
|
|
|
|
{
|
|
|
|
requireConnectionPool(name, storage);
|
|
|
|
if (!directory_monitor)
|
|
|
|
directory_monitor = std::make_unique<StorageDistributedDirectoryMonitor>(storage, name, conneciton_pool);
|
|
|
|
}
|
|
|
|
|
2017-12-30 00:36:06 +00:00
|
|
|
|
|
|
|
void registerStorageDistributed(StorageFactory & factory)
|
|
|
|
{
|
|
|
|
factory.registerStorage("Distributed", [](const StorageFactory::Arguments & args)
|
|
|
|
{
|
|
|
|
/** Arguments of engine is following:
|
|
|
|
* - name of cluster in configuration;
|
|
|
|
* - name of remote database;
|
|
|
|
* - name of remote table;
|
|
|
|
*
|
|
|
|
* Remote database may be specified in following form:
|
|
|
|
* - identifier;
|
|
|
|
* - constant expression with string result, like currentDatabase();
|
|
|
|
* -- string literal as specific case;
|
|
|
|
* - empty string means 'use default database from cluster'.
|
|
|
|
*/
|
|
|
|
|
|
|
|
ASTs & engine_args = args.engine_args;
|
|
|
|
|
|
|
|
if (!(engine_args.size() == 3 || engine_args.size() == 4))
|
|
|
|
throw Exception("Storage Distributed requires 3 or 4 parameters"
|
|
|
|
" - name of configuration section with list of remote servers, name of remote database, name of remote table,"
|
|
|
|
" sharding key expression (optional).", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
String cluster_name = getClusterName(*engine_args[0]);
|
|
|
|
|
|
|
|
engine_args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[1], args.local_context);
|
|
|
|
engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.local_context);
|
|
|
|
|
|
|
|
String remote_database = static_cast<const ASTLiteral &>(*engine_args[1]).value.safeGet<String>();
|
|
|
|
String remote_table = static_cast<const ASTLiteral &>(*engine_args[2]).value.safeGet<String>();
|
|
|
|
|
|
|
|
const auto & sharding_key = engine_args.size() == 4 ? engine_args[3] : nullptr;
|
|
|
|
|
|
|
|
/// Check that sharding_key exists in the table and has numeric type.
|
|
|
|
if (sharding_key)
|
|
|
|
{
|
|
|
|
auto sharding_expr = ExpressionAnalyzer(sharding_key, args.context, nullptr, args.columns).getActions(true);
|
|
|
|
const Block & block = sharding_expr->getSampleBlock();
|
|
|
|
|
|
|
|
if (block.columns() != 1)
|
|
|
|
throw Exception("Sharding expression must return exactly one column", ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS);
|
|
|
|
|
|
|
|
auto type = block.getByPosition(0).type;
|
|
|
|
|
|
|
|
if (!type->isValueRepresentedByInteger())
|
|
|
|
throw Exception("Sharding expression has type " + type->getName() +
|
|
|
|
", but should be one of integer type", ErrorCodes::TYPE_MISMATCH);
|
|
|
|
}
|
|
|
|
|
|
|
|
return StorageDistributed::create(
|
|
|
|
args.table_name, args.columns,
|
|
|
|
args.materialized_columns, args.alias_columns, args.column_defaults,
|
|
|
|
remote_database, remote_table, cluster_name,
|
|
|
|
args.context, sharding_key, args.data_path);
|
|
|
|
});
|
|
|
|
}
|
|
|
|
|
2012-05-21 20:38:34 +00:00
|
|
|
}
|