2012-05-21 20:38:34 +00:00
|
|
|
|
#include <DB/DataStreams/RemoteBlockInputStream.h>
|
2015-10-12 14:53:16 +00:00
|
|
|
|
#include <DB/DataStreams/BlockExtraInfoInputStream.h>
|
2016-01-28 01:00:27 +00:00
|
|
|
|
#include <DB/DataStreams/UnionBlockInputStream.h>
|
2012-05-21 20:38:34 +00:00
|
|
|
|
|
2016-05-13 21:08:19 +00:00
|
|
|
|
#include <DB/Databases/IDatabase.h>
|
|
|
|
|
|
2012-05-21 20:38:34 +00:00
|
|
|
|
#include <DB/Storages/StorageDistributed.h>
|
2014-07-31 09:11:49 +00:00
|
|
|
|
#include <DB/Storages/VirtualColumnFactory.h>
|
2014-08-12 13:46:46 +00:00
|
|
|
|
#include <DB/Storages/Distributed/DistributedBlockOutputStream.h>
|
2014-08-15 09:50:05 +00:00
|
|
|
|
#include <DB/Storages/Distributed/DirectoryMonitor.h>
|
2016-03-01 17:47:53 +00:00
|
|
|
|
#include <DB/Storages/MergeTree/ReshardingWorker.h>
|
2016-05-13 21:08:19 +00:00
|
|
|
|
|
2014-08-13 09:20:15 +00:00
|
|
|
|
#include <DB/Common/escapeForFileName.h>
|
2016-05-13 21:08:19 +00:00
|
|
|
|
|
2015-03-02 01:10:58 +00:00
|
|
|
|
#include <DB/Parsers/ASTInsertQuery.h>
|
2015-05-06 23:35:37 +00:00
|
|
|
|
#include <DB/Parsers/ASTSelectQuery.h>
|
2015-11-08 00:28:12 +00:00
|
|
|
|
#include <DB/Parsers/ASTIdentifier.h>
|
2015-10-12 14:53:16 +00:00
|
|
|
|
#include <DB/Parsers/TablePropertiesQueriesASTs.h>
|
2016-01-28 01:00:27 +00:00
|
|
|
|
#include <DB/Parsers/ParserAlterQuery.h>
|
|
|
|
|
#include <DB/Parsers/parseQuery.h>
|
|
|
|
|
#include <DB/Parsers/ASTWeightedZooKeeperPath.h>
|
|
|
|
|
#include <DB/Parsers/ASTLiteral.h>
|
2014-08-13 09:20:15 +00:00
|
|
|
|
|
2013-11-28 10:31:17 +00:00
|
|
|
|
#include <DB/Interpreters/InterpreterSelectQuery.h>
|
2014-07-11 08:12:03 +00:00
|
|
|
|
#include <DB/Interpreters/InterpreterAlterQuery.h>
|
2015-10-12 14:53:16 +00:00
|
|
|
|
#include <DB/Interpreters/InterpreterDescribeQuery.h>
|
2015-05-06 23:35:37 +00:00
|
|
|
|
#include <DB/Interpreters/ExpressionAnalyzer.h>
|
2016-01-28 01:00:27 +00:00
|
|
|
|
#include <DB/Interpreters/ClusterProxy/Query.h>
|
|
|
|
|
#include <DB/Interpreters/ClusterProxy/SelectQueryConstructor.h>
|
|
|
|
|
#include <DB/Interpreters/ClusterProxy/DescribeQueryConstructor.h>
|
|
|
|
|
#include <DB/Interpreters/ClusterProxy/AlterQueryConstructor.h>
|
2014-08-13 15:07:53 +00:00
|
|
|
|
|
2014-02-11 17:10:48 +00:00
|
|
|
|
#include <DB/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
|
|
|
|
|
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
|
|
|
|
|
{
|
|
|
|
|
extern const int STORAGE_REQUIRES_PARAMETER;
|
2016-03-01 17:47:53 +00:00
|
|
|
|
extern const int RESHARDING_NO_WORKER;
|
|
|
|
|
extern const int RESHARDING_INVALID_PARAMETERS;
|
|
|
|
|
extern const int RESHARDING_INITIATOR_CHECK_FAILED;
|
2016-01-11 21:46:36 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-08-21 12:07:29 +00:00
|
|
|
|
namespace
|
|
|
|
|
{
|
2014-08-13 13:43:54 +00:00
|
|
|
|
/// select query has database and table names as AST pointers
|
2014-08-22 14:05:34 +00:00
|
|
|
|
/// Создает копию запроса, меняет имена базы данных и таблицы.
|
2014-12-12 18:11:13 +00:00
|
|
|
|
inline ASTPtr rewriteSelectQuery(const ASTPtr & query, const std::string & database, const std::string & table)
|
2014-08-13 12:52:30 +00:00
|
|
|
|
{
|
2014-12-12 18:11:13 +00:00
|
|
|
|
auto modified_query_ast = query->clone();
|
2016-07-22 20:39:28 +00:00
|
|
|
|
typeid_cast<ASTSelectQuery &>(*modified_query_ast).replaceDatabaseAndTable(database, table);
|
2014-08-22 14:05:34 +00:00
|
|
|
|
return modified_query_ast;
|
2014-08-13 12:52:30 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-08-22 14:05:34 +00:00
|
|
|
|
/// insert query has database and table names as bare strings
|
2014-08-13 12:52:30 +00:00
|
|
|
|
/// Создает копию запроса, меняет имена базы данных и таблицы.
|
2014-08-22 14:05:34 +00:00
|
|
|
|
inline ASTPtr rewriteInsertQuery(const ASTPtr & query, const std::string & database, const std::string & table)
|
2014-08-19 08:04:13 +00:00
|
|
|
|
{
|
2014-08-13 12:52:30 +00:00
|
|
|
|
auto modified_query_ast = query->clone();
|
|
|
|
|
|
2014-08-22 14:05:34 +00:00
|
|
|
|
auto & actual_query = typeid_cast<ASTInsertQuery &>(*modified_query_ast);
|
|
|
|
|
actual_query.database = database;
|
|
|
|
|
actual_query.table = table;
|
|
|
|
|
/// make sure query is not INSERT SELECT
|
|
|
|
|
actual_query.select = nullptr;
|
2014-08-13 12:52:30 +00:00
|
|
|
|
|
2014-08-21 12:07:29 +00:00
|
|
|
|
return modified_query_ast;
|
2014-08-13 12:52:30 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2012-11-06 17:04:38 +00:00
|
|
|
|
StorageDistributed::StorageDistributed(
|
|
|
|
|
const std::string & name_,
|
|
|
|
|
NamesAndTypesListPtr columns_,
|
|
|
|
|
const String & remote_database_,
|
|
|
|
|
const String & remote_table_,
|
2016-10-10 08:44:52 +00:00
|
|
|
|
const String & cluster_name_,
|
2014-08-15 09:50:05 +00:00
|
|
|
|
Context & context_,
|
2014-08-13 09:20:15 +00:00
|
|
|
|
const ASTPtr & sharding_key_,
|
|
|
|
|
const String & data_path_)
|
2012-11-06 17:04:38 +00:00
|
|
|
|
: name(name_), columns(columns_),
|
|
|
|
|
remote_database(remote_database_), remote_table(remote_table_),
|
2016-10-10 08:44:52 +00:00
|
|
|
|
context(context_), cluster_name(cluster_name_), has_sharding_key(sharding_key_),
|
2015-07-15 01:26:35 +00:00
|
|
|
|
sharding_key_expr(sharding_key_ ? ExpressionAnalyzer(sharding_key_, context, nullptr, *columns).getActions(false) : nullptr),
|
2014-08-13 09:20:15 +00:00
|
|
|
|
sharding_key_column_name(sharding_key_ ? sharding_key_->getColumnName() : String{}),
|
2014-12-30 03:53:41 +00:00
|
|
|
|
path(data_path_.empty() ? "" : (data_path_ + escapeForFileName(name) + '/'))
|
2013-11-28 10:31:17 +00:00
|
|
|
|
{
|
2014-08-13 11:26:13 +00:00
|
|
|
|
createDirectoryMonitors();
|
2013-11-28 10:31:17 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-09-30 03:08:47 +00:00
|
|
|
|
StorageDistributed::StorageDistributed(
|
|
|
|
|
const std::string & name_,
|
|
|
|
|
NamesAndTypesListPtr columns_,
|
2014-10-03 15:30:10 +00:00
|
|
|
|
const NamesAndTypesList & materialized_columns_,
|
2014-09-30 03:08:47 +00:00
|
|
|
|
const NamesAndTypesList & alias_columns_,
|
|
|
|
|
const ColumnDefaults & column_defaults_,
|
|
|
|
|
const String & remote_database_,
|
|
|
|
|
const String & remote_table_,
|
2016-10-10 08:44:52 +00:00
|
|
|
|
const String & cluster_name_,
|
2014-09-30 03:08:47 +00:00
|
|
|
|
Context & context_,
|
|
|
|
|
const ASTPtr & sharding_key_,
|
|
|
|
|
const String & data_path_)
|
2014-10-03 15:30:10 +00:00
|
|
|
|
: IStorage{materialized_columns_, alias_columns_, column_defaults_},
|
2014-09-30 03:08:47 +00:00
|
|
|
|
name(name_), columns(columns_),
|
|
|
|
|
remote_database(remote_database_), remote_table(remote_table_),
|
2016-10-10 08:44:52 +00:00
|
|
|
|
context(context_), cluster_name(cluster_name_), has_sharding_key(sharding_key_),
|
2015-07-15 01:26:35 +00:00
|
|
|
|
sharding_key_expr(sharding_key_ ? ExpressionAnalyzer(sharding_key_, context, nullptr, *columns).getActions(false) : nullptr),
|
2014-09-30 03:08:47 +00:00
|
|
|
|
sharding_key_column_name(sharding_key_ ? sharding_key_->getColumnName() : String{}),
|
2014-12-30 03:53:41 +00:00
|
|
|
|
path(data_path_.empty() ? "" : (data_path_ + escapeForFileName(name) + '/'))
|
2014-09-30 03:08:47 +00:00
|
|
|
|
{
|
|
|
|
|
createDirectoryMonitors();
|
|
|
|
|
}
|
|
|
|
|
|
2013-02-06 11:26:35 +00:00
|
|
|
|
StoragePtr StorageDistributed::create(
|
|
|
|
|
const std::string & name_,
|
|
|
|
|
NamesAndTypesListPtr columns_,
|
2014-10-03 15:30:10 +00:00
|
|
|
|
const NamesAndTypesList & materialized_columns_,
|
2014-09-30 03:08:47 +00:00
|
|
|
|
const NamesAndTypesList & alias_columns_,
|
|
|
|
|
const ColumnDefaults & column_defaults_,
|
2013-02-06 11:26:35 +00:00
|
|
|
|
const String & remote_database_,
|
|
|
|
|
const String & remote_table_,
|
2016-10-10 08:44:52 +00:00
|
|
|
|
const String & cluster_name_,
|
2014-08-12 13:46:46 +00:00
|
|
|
|
Context & context_,
|
2014-08-13 09:20:15 +00:00
|
|
|
|
const ASTPtr & sharding_key_,
|
|
|
|
|
const String & data_path_)
|
2013-02-06 11:26:35 +00:00
|
|
|
|
{
|
2016-08-26 21:25:05 +00:00
|
|
|
|
return make_shared(
|
2014-10-03 15:30:10 +00:00
|
|
|
|
name_, columns_,
|
|
|
|
|
materialized_columns_, alias_columns_, column_defaults_,
|
2014-09-30 03:08:47 +00:00
|
|
|
|
remote_database_, remote_table_,
|
2016-10-10 08:44:52 +00:00
|
|
|
|
cluster_name_, context_,
|
2014-08-13 09:20:15 +00:00
|
|
|
|
sharding_key_, data_path_
|
2016-08-26 21:25:05 +00:00
|
|
|
|
);
|
2013-02-06 11:26:35 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-02-07 15:11:57 +00:00
|
|
|
|
|
|
|
|
|
StoragePtr StorageDistributed::create(
|
|
|
|
|
const std::string & name_,
|
|
|
|
|
NamesAndTypesListPtr columns_,
|
|
|
|
|
const String & remote_database_,
|
|
|
|
|
const String & remote_table_,
|
2016-10-10 08:44:52 +00:00
|
|
|
|
ClusterPtr & owned_cluster_,
|
2014-08-13 09:20:15 +00:00
|
|
|
|
Context & context_)
|
2014-02-07 15:11:57 +00:00
|
|
|
|
{
|
2016-08-26 21:25:05 +00:00
|
|
|
|
auto res = make_shared(
|
2014-08-13 09:20:15 +00:00
|
|
|
|
name_, columns_, remote_database_,
|
2016-10-10 08:44:52 +00:00
|
|
|
|
remote_table_, String{}, context_
|
2016-08-26 21:25:05 +00:00
|
|
|
|
);
|
2014-02-23 02:27:09 +00:00
|
|
|
|
|
|
|
|
|
res->owned_cluster = owned_cluster_;
|
|
|
|
|
|
2016-08-26 21:25:05 +00:00
|
|
|
|
return res;
|
2014-02-07 15:11:57 +00:00
|
|
|
|
}
|
|
|
|
|
|
2012-05-21 20:38:34 +00:00
|
|
|
|
BlockInputStreams StorageDistributed::read(
|
|
|
|
|
const Names & column_names,
|
|
|
|
|
ASTPtr query,
|
2014-12-17 11:53:17 +00:00
|
|
|
|
const Context & context,
|
2013-02-01 19:02:04 +00:00
|
|
|
|
const Settings & settings,
|
2012-05-22 18:32:45 +00:00
|
|
|
|
QueryProcessingStage::Enum & processed_stage,
|
2014-12-17 11:53:17 +00:00
|
|
|
|
const size_t max_block_size,
|
|
|
|
|
const unsigned threads)
|
2012-05-21 20:38:34 +00:00
|
|
|
|
{
|
2016-10-10 08:44:52 +00:00
|
|
|
|
auto cluster = getCluster();
|
|
|
|
|
|
|
|
|
|
size_t result_size = (cluster->getRemoteShardCount() * settings.max_parallel_replicas) + cluster->getLocalShardCount();
|
2014-02-11 17:10:48 +00:00
|
|
|
|
|
2015-06-28 06:48:42 +00:00
|
|
|
|
processed_stage = result_size == 1 || settings.distributed_group_by_no_merge
|
2014-02-11 17:10:48 +00:00
|
|
|
|
? QueryProcessingStage::Complete
|
|
|
|
|
: QueryProcessingStage::WithMergeableState;
|
|
|
|
|
|
2014-08-22 14:05:34 +00:00
|
|
|
|
const auto & modified_query_ast = rewriteSelectQuery(
|
2014-08-20 02:20:07 +00:00
|
|
|
|
query, remote_database, remote_table);
|
2015-02-10 20:48:17 +00:00
|
|
|
|
|
2015-09-05 01:22:09 +00:00
|
|
|
|
Tables external_tables;
|
|
|
|
|
|
|
|
|
|
if (settings.global_subqueries_method == GlobalSubqueriesMethod::PUSH)
|
|
|
|
|
external_tables = context.getExternalTables();
|
2015-03-15 08:05:28 +00:00
|
|
|
|
|
2015-11-06 17:44:01 +00:00
|
|
|
|
/// Отключаем мультиплексирование шардов, если есть ORDER BY без GROUP BY.
|
2015-11-13 17:21:16 +00:00
|
|
|
|
//const ASTSelectQuery & ast = *(static_cast<const ASTSelectQuery *>(modified_query_ast.get()));
|
2015-11-12 01:17:44 +00:00
|
|
|
|
|
|
|
|
|
/** Функциональность shard_multiplexing не доделана - выключаем её.
|
|
|
|
|
* (Потому что установка соединений с разными шардами в рамках одного потока выполняется не параллельно.)
|
|
|
|
|
* Подробнее смотрите в https://███████████.yandex-team.ru/METR-18300
|
|
|
|
|
*/
|
|
|
|
|
//bool enable_shard_multiplexing = !(ast.order_expression_list && !ast.group_expression_list);
|
|
|
|
|
bool enable_shard_multiplexing = false;
|
2015-11-06 17:44:01 +00:00
|
|
|
|
|
2016-03-25 11:48:45 +00:00
|
|
|
|
ClusterProxy::SelectQueryConstructor select_query_constructor{processed_stage, external_tables};
|
2015-11-06 17:44:01 +00:00
|
|
|
|
|
2016-03-25 11:48:45 +00:00
|
|
|
|
return ClusterProxy::Query{select_query_constructor, cluster, modified_query_ast,
|
|
|
|
|
context, settings, enable_shard_multiplexing}.execute();
|
2012-05-21 20:38:34 +00:00
|
|
|
|
}
|
|
|
|
|
|
2015-09-10 20:43:42 +00:00
|
|
|
|
BlockOutputStreamPtr StorageDistributed::write(ASTPtr query, const Settings & settings)
|
2014-08-12 13:46:46 +00:00
|
|
|
|
{
|
2016-10-10 08:44:52 +00:00
|
|
|
|
auto cluster = context.getCluster(cluster_name);
|
|
|
|
|
|
|
|
|
|
/// TODO: !path.empty() can be replaced by !owned_cluster or !cluster_name.empty() ?
|
|
|
|
|
bool write_enabled = !path.empty() && (((cluster->getLocalShardCount() + cluster->getRemoteShardCount()) < 2) || has_sharding_key);
|
|
|
|
|
|
2014-08-12 13:46:46 +00:00
|
|
|
|
if (!write_enabled)
|
|
|
|
|
throw Exception{
|
2014-08-21 12:07:29 +00:00
|
|
|
|
"Method write is not supported by storage " + getName() +
|
|
|
|
|
" with more than one shard and no sharding key provided",
|
|
|
|
|
ErrorCodes::STORAGE_REQUIRES_PARAMETER
|
|
|
|
|
};
|
|
|
|
|
|
2016-10-10 08:44:52 +00:00
|
|
|
|
/// DistributedBlockOutputStream will not own cluster, but will own ConnectionPools of the cluster
|
2016-05-28 12:22:22 +00:00
|
|
|
|
return std::make_shared<DistributedBlockOutputStream>(
|
2016-10-10 08:44:52 +00:00
|
|
|
|
*this, rewriteInsertQuery(query, remote_database, remote_table), cluster);
|
2014-08-12 13:46:46 +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
|
|
|
|
{
|
2016-05-05 18:28:46 +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);
|
|
|
|
|
|
2014-07-11 08:12:03 +00:00
|
|
|
|
auto lock = lockStructureForAlter();
|
2014-10-16 13:37:01 +00:00
|
|
|
|
params.apply(*columns, materialized_columns, alias_columns, column_defaults);
|
2016-05-13 21:08:19 +00:00
|
|
|
|
|
|
|
|
|
context.getDatabase(database_name)->alterTable(
|
|
|
|
|
context, table_name,
|
|
|
|
|
*columns, materialized_columns, alias_columns, column_defaults, {});
|
2013-09-23 12:01:19 +00:00
|
|
|
|
}
|
2014-02-04 15:44:15 +00:00
|
|
|
|
|
2014-08-13 11:26:13 +00:00
|
|
|
|
void StorageDistributed::shutdown()
|
|
|
|
|
{
|
2014-08-19 08:04:13 +00:00
|
|
|
|
directory_monitors.clear();
|
2014-08-13 11:26:13 +00:00
|
|
|
|
}
|
|
|
|
|
|
2016-03-01 17:47:53 +00:00
|
|
|
|
void StorageDistributed::reshardPartitions(ASTPtr query, const String & database_name,
|
|
|
|
|
const Field & first_partition, const Field & last_partition,
|
|
|
|
|
const WeightedZooKeeperPaths & weighted_zookeeper_paths,
|
2016-03-25 11:48:45 +00:00
|
|
|
|
const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator,
|
2016-03-01 17:47:53 +00:00
|
|
|
|
const Settings & settings)
|
2015-10-12 14:53:16 +00:00
|
|
|
|
{
|
2016-03-01 17:47:53 +00:00
|
|
|
|
auto & resharding_worker = context.getReshardingWorker();
|
|
|
|
|
if (!resharding_worker.isStarted())
|
2016-03-25 11:48:45 +00:00
|
|
|
|
throw Exception{"Resharding background thread is not running", ErrorCodes::RESHARDING_NO_WORKER};
|
2015-10-12 14:53:16 +00:00
|
|
|
|
|
2016-05-28 15:50:16 +00:00
|
|
|
|
if (!coordinator.isNull())
|
2016-03-25 11:48:45 +00:00
|
|
|
|
throw Exception{"Use of COORDINATE WITH is forbidden in ALTER TABLE ... RESHARD"
|
2016-03-01 17:47:53 +00:00
|
|
|
|
" queries for distributed tables",
|
2016-03-25 11:48:45 +00:00
|
|
|
|
ErrorCodes::RESHARDING_INVALID_PARAMETERS};
|
2015-10-12 14:53:16 +00:00
|
|
|
|
|
2016-10-10 08:44:52 +00:00
|
|
|
|
auto cluster = getCluster();
|
|
|
|
|
|
2016-10-14 15:06:46 +00:00
|
|
|
|
/// resharding_worker doesn't need to own cluster, here only meta-information of cluster is used
|
2016-10-10 08:44:52 +00:00
|
|
|
|
std::string coordinator_id = resharding_worker.createCoordinator(*cluster);
|
2015-10-12 14:53:16 +00:00
|
|
|
|
|
2016-03-05 10:33:01 +00:00
|
|
|
|
std::atomic<bool> has_notified_error{false};
|
|
|
|
|
|
2016-03-25 11:48:45 +00:00
|
|
|
|
std::string dumped_coordinator_state;
|
|
|
|
|
|
|
|
|
|
auto handle_exception = [&](const std::string & msg = "")
|
|
|
|
|
{
|
|
|
|
|
try
|
|
|
|
|
{
|
|
|
|
|
if (!has_notified_error)
|
|
|
|
|
resharding_worker.setStatus(coordinator_id, ReshardingWorker::STATUS_ERROR, msg);
|
|
|
|
|
dumped_coordinator_state = resharding_worker.dumpCoordinatorState(coordinator_id);
|
|
|
|
|
resharding_worker.deleteCoordinator(coordinator_id);
|
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
2016-03-01 17:47:53 +00:00
|
|
|
|
try
|
|
|
|
|
{
|
2016-03-25 11:48:45 +00:00
|
|
|
|
/// Создать запрос ALTER TABLE ... RESHARD [COPY] PARTITION ... COORDINATE WITH ...
|
2015-10-12 14:53:16 +00:00
|
|
|
|
|
2016-05-28 15:42:22 +00:00
|
|
|
|
ASTPtr alter_query_ptr = std::make_shared<ASTAlterQuery>();
|
2016-03-01 17:47:53 +00:00
|
|
|
|
auto & alter_query = static_cast<ASTAlterQuery &>(*alter_query_ptr);
|
2015-10-12 14:53:16 +00:00
|
|
|
|
|
2016-03-01 17:47:53 +00:00
|
|
|
|
alter_query.database = remote_database;
|
|
|
|
|
alter_query.table = remote_table;
|
2015-11-06 17:44:01 +00:00
|
|
|
|
|
2016-03-01 17:47:53 +00:00
|
|
|
|
alter_query.parameters.emplace_back();
|
|
|
|
|
ASTAlterQuery::Parameters & parameters = alter_query.parameters.back();
|
2015-11-06 17:44:01 +00:00
|
|
|
|
|
2016-03-01 17:47:53 +00:00
|
|
|
|
parameters.type = ASTAlterQuery::RESHARD_PARTITION;
|
2016-05-28 15:50:16 +00:00
|
|
|
|
if (!first_partition.isNull())
|
2016-05-28 15:54:40 +00:00
|
|
|
|
parameters.partition = std::make_shared<ASTLiteral>(StringRange(), first_partition);
|
2016-05-28 15:50:16 +00:00
|
|
|
|
if (!last_partition.isNull())
|
2016-05-28 15:54:40 +00:00
|
|
|
|
parameters.last_partition = std::make_shared<ASTLiteral>(StringRange(), last_partition);
|
2015-11-12 01:17:44 +00:00
|
|
|
|
|
2016-05-28 15:42:22 +00:00
|
|
|
|
ASTPtr expr_list = std::make_shared<ASTExpressionList>();
|
2016-03-01 17:47:53 +00:00
|
|
|
|
for (const auto & entry : weighted_zookeeper_paths)
|
|
|
|
|
{
|
2016-05-28 15:42:22 +00:00
|
|
|
|
ASTPtr weighted_path_ptr = std::make_shared<ASTWeightedZooKeeperPath>();
|
2016-03-01 17:47:53 +00:00
|
|
|
|
auto & weighted_path = static_cast<ASTWeightedZooKeeperPath &>(*weighted_path_ptr);
|
|
|
|
|
weighted_path.path = entry.first;
|
|
|
|
|
weighted_path.weight = entry.second;
|
|
|
|
|
expr_list->children.push_back(weighted_path_ptr);
|
|
|
|
|
}
|
2015-11-06 17:44:01 +00:00
|
|
|
|
|
2016-03-01 17:47:53 +00:00
|
|
|
|
parameters.weighted_zookeeper_paths = expr_list;
|
|
|
|
|
parameters.sharding_key_expr = sharding_key_expr;
|
2016-03-25 11:48:45 +00:00
|
|
|
|
parameters.do_copy = do_copy;
|
2016-05-28 15:54:40 +00:00
|
|
|
|
parameters.coordinator = std::make_shared<ASTLiteral>(StringRange(), Field(coordinator_id));
|
2016-03-01 17:47:53 +00:00
|
|
|
|
|
|
|
|
|
resharding_worker.registerQuery(coordinator_id, queryToString(alter_query_ptr));
|
|
|
|
|
|
|
|
|
|
/** Функциональность shard_multiplexing не доделана - выключаем её.
|
|
|
|
|
* (Потому что установка соединений с разными шардами в рамках одного потока выполняется не параллельно.)
|
|
|
|
|
* Подробнее смотрите в https://███████████.yandex-team.ru/METR-18300
|
|
|
|
|
*/
|
|
|
|
|
bool enable_shard_multiplexing = false;
|
2016-01-28 01:00:27 +00:00
|
|
|
|
|
2016-03-10 15:42:36 +00:00
|
|
|
|
ClusterProxy::AlterQueryConstructor alter_query_constructor;
|
|
|
|
|
|
2016-03-25 11:48:45 +00:00
|
|
|
|
BlockInputStreams streams = ClusterProxy::Query{alter_query_constructor, cluster, alter_query_ptr,
|
|
|
|
|
context, settings, enable_shard_multiplexing}.execute();
|
2016-03-01 17:47:53 +00:00
|
|
|
|
|
2016-03-08 15:38:06 +00:00
|
|
|
|
/// This callback is called if an exception has occurred while attempting to read
|
|
|
|
|
/// a block from a shard. This is to avoid a potential deadlock if other shards are
|
|
|
|
|
/// waiting inside a barrier. Actually, even without this solution, we would avoid
|
|
|
|
|
/// such a deadlock because we would eventually time out while trying to get remote
|
|
|
|
|
/// blocks. Nevertheless this is not the ideal way of sorting out this issue since
|
|
|
|
|
/// we would then not get to know the actual cause of the failure.
|
2016-03-05 10:33:01 +00:00
|
|
|
|
auto exception_callback = [&resharding_worker, coordinator_id, &has_notified_error]()
|
2016-03-04 16:33:31 +00:00
|
|
|
|
{
|
|
|
|
|
try
|
|
|
|
|
{
|
|
|
|
|
resharding_worker.setStatus(coordinator_id, ReshardingWorker::STATUS_ERROR);
|
2016-03-05 10:33:01 +00:00
|
|
|
|
has_notified_error = true;
|
2016-03-04 16:33:31 +00:00
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
2016-05-28 12:22:22 +00:00
|
|
|
|
streams[0] = std::make_shared<UnionBlockInputStream<>>(
|
|
|
|
|
streams, nullptr, settings.max_distributed_connections, exception_callback);
|
2016-03-01 17:47:53 +00:00
|
|
|
|
streams.resize(1);
|
|
|
|
|
|
|
|
|
|
auto stream_ptr = dynamic_cast<IProfilingBlockInputStream *>(&*streams[0]);
|
|
|
|
|
if (stream_ptr == nullptr)
|
2016-03-25 11:48:45 +00:00
|
|
|
|
throw Exception{"StorageDistributed: Internal error", ErrorCodes::LOGICAL_ERROR};
|
2016-03-01 17:47:53 +00:00
|
|
|
|
auto & stream = *stream_ptr;
|
|
|
|
|
|
2016-03-03 14:37:15 +00:00
|
|
|
|
stream.readPrefix();
|
|
|
|
|
|
2016-03-01 17:47:53 +00:00
|
|
|
|
while (!stream.isCancelled() && stream.read())
|
|
|
|
|
;
|
2016-03-03 14:37:15 +00:00
|
|
|
|
|
2016-03-05 10:33:01 +00:00
|
|
|
|
if (!stream.isCancelled())
|
|
|
|
|
stream.readSuffix();
|
2016-03-01 17:47:53 +00:00
|
|
|
|
}
|
2016-03-25 11:48:45 +00:00
|
|
|
|
catch (const Exception & ex)
|
|
|
|
|
{
|
|
|
|
|
handle_exception(ex.message());
|
|
|
|
|
LOG_ERROR(log, dumped_coordinator_state);
|
|
|
|
|
throw;
|
|
|
|
|
}
|
|
|
|
|
catch (const std::exception & ex)
|
|
|
|
|
{
|
|
|
|
|
handle_exception(ex.what());
|
|
|
|
|
LOG_ERROR(log, dumped_coordinator_state);
|
|
|
|
|
throw;
|
|
|
|
|
}
|
2016-03-01 17:47:53 +00:00
|
|
|
|
catch (...)
|
|
|
|
|
{
|
2016-03-25 11:48:45 +00:00
|
|
|
|
handle_exception();
|
|
|
|
|
LOG_ERROR(log, dumped_coordinator_state);
|
2016-03-01 17:47:53 +00:00
|
|
|
|
throw;
|
|
|
|
|
}
|
2016-01-28 01:00:27 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
BlockInputStreams StorageDistributed::describe(const Context & context, const Settings & settings)
|
|
|
|
|
{
|
|
|
|
|
/// Создать запрос DESCRIBE TABLE.
|
2016-10-10 08:44:52 +00:00
|
|
|
|
auto cluster = getCluster();
|
2016-01-28 01:00:27 +00:00
|
|
|
|
|
2016-05-28 15:42:22 +00:00
|
|
|
|
ASTPtr describe_query_ptr = std::make_shared<ASTDescribeQuery>();
|
2016-01-28 01:00:27 +00:00
|
|
|
|
auto & describe_query = static_cast<ASTDescribeQuery &>(*describe_query_ptr);
|
|
|
|
|
|
|
|
|
|
describe_query.database = remote_database;
|
|
|
|
|
describe_query.table = remote_table;
|
|
|
|
|
|
|
|
|
|
/** Функциональность shard_multiplexing не доделана - выключаем её.
|
|
|
|
|
* (Потому что установка соединений с разными шардами в рамках одного потока выполняется не параллельно.)
|
|
|
|
|
* Подробнее смотрите в https://███████████.yandex-team.ru/METR-18300
|
|
|
|
|
*/
|
|
|
|
|
bool enable_shard_multiplexing = false;
|
|
|
|
|
|
|
|
|
|
ClusterProxy::DescribeQueryConstructor describe_query_constructor;
|
2015-10-12 14:53:16 +00:00
|
|
|
|
|
2016-03-25 11:48:45 +00:00
|
|
|
|
return ClusterProxy::Query{describe_query_constructor, cluster, describe_query_ptr,
|
|
|
|
|
context, settings, enable_shard_multiplexing}.execute();
|
2015-10-12 14:53:16 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-07-31 09:11:49 +00:00
|
|
|
|
NameAndTypePair StorageDistributed::getColumn(const String & column_name) const
|
|
|
|
|
{
|
2014-08-15 09:50:05 +00:00
|
|
|
|
if (const auto & type = VirtualColumnFactory::tryGetType(column_name))
|
|
|
|
|
return { column_name, type };
|
2014-07-31 09:11:49 +00:00
|
|
|
|
|
|
|
|
|
return getRealColumn(column_name);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
bool StorageDistributed::hasColumn(const String & column_name) const
|
|
|
|
|
{
|
2014-10-03 15:30:10 +00:00
|
|
|
|
return VirtualColumnFactory::hasColumn(column_name) || IStorage::hasColumn(column_name);
|
2014-07-31 09:11:49 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-08-13 11:26:13 +00:00
|
|
|
|
void StorageDistributed::createDirectoryMonitor(const std::string & name)
|
|
|
|
|
{
|
2016-12-12 03:33:34 +00:00
|
|
|
|
directory_monitors.emplace(name, std::make_unique<StorageDistributedDirectoryMonitor>(*this, name));
|
2014-08-13 11:26:13 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-08-15 09:50:05 +00:00
|
|
|
|
void StorageDistributed::createDirectoryMonitors()
|
2014-08-13 11:26:13 +00:00
|
|
|
|
{
|
2014-12-30 03:53:41 +00:00
|
|
|
|
if (path.empty())
|
|
|
|
|
return;
|
|
|
|
|
|
2014-08-15 09:50:05 +00:00
|
|
|
|
Poco::File{path}.createDirectory();
|
2014-08-14 11:50:36 +00:00
|
|
|
|
|
2014-08-15 09:50:05 +00:00
|
|
|
|
Poco::DirectoryIterator end;
|
|
|
|
|
for (Poco::DirectoryIterator it{path}; it != end; ++it)
|
|
|
|
|
if (it->isDirectory())
|
|
|
|
|
createDirectoryMonitor(it.name());
|
2014-08-13 11:26:13 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-08-19 08:04:13 +00:00
|
|
|
|
void StorageDistributed::requireDirectoryMonitor(const std::string & name)
|
|
|
|
|
{
|
|
|
|
|
if (!directory_monitors.count(name))
|
|
|
|
|
createDirectoryMonitor(name);
|
|
|
|
|
}
|
|
|
|
|
|
2015-09-18 13:36:10 +00:00
|
|
|
|
size_t StorageDistributed::getShardCount() const
|
|
|
|
|
{
|
2016-10-10 08:44:52 +00:00
|
|
|
|
return getCluster()->getRemoteShardCount();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
ClusterPtr StorageDistributed::getCluster() const
|
|
|
|
|
{
|
|
|
|
|
return (owned_cluster) ? owned_cluster : context.getCluster(cluster_name);
|
2015-09-18 13:36:10 +00:00
|
|
|
|
}
|
|
|
|
|
|
2012-05-21 20:38:34 +00:00
|
|
|
|
}
|