2012-05-21 20:38:34 +00:00
|
|
|
#include <DB/Parsers/formatAST.h>
|
|
|
|
|
|
|
|
#include <DB/DataStreams/RemoteBlockInputStream.h>
|
2014-02-11 17:10:48 +00:00
|
|
|
#include <DB/DataStreams/RemoveColumnsBlockInputStream.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>
|
2012-05-21 20:38:34 +00:00
|
|
|
|
2013-11-28 10:31:17 +00:00
|
|
|
#include <Poco/Net/NetworkInterface.h>
|
2014-01-22 14:24:05 +00:00
|
|
|
#include <DB/Client/ConnectionPool.h>
|
2013-11-28 10:31:17 +00:00
|
|
|
|
2014-08-13 09:20:15 +00:00
|
|
|
#include <DB/Common/escapeForFileName.h>
|
|
|
|
|
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>
|
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
|
|
|
|
2014-08-13 15:07:53 +00:00
|
|
|
#include <boost/algorithm/string/find_iterator.hpp>
|
|
|
|
#include <boost/algorithm/string/finder.hpp>
|
|
|
|
|
2012-05-21 20:38:34 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2014-08-13 12:52:30 +00:00
|
|
|
namespace {
|
|
|
|
template <typename ASTType>
|
|
|
|
inline std::string queryToString(const ASTPtr & query)
|
|
|
|
{
|
2014-08-13 13:43:54 +00:00
|
|
|
const auto & query_ast = typeid_cast<const ASTType &>(*query);
|
2014-08-13 12:52:30 +00:00
|
|
|
|
|
|
|
std::ostringstream s;
|
2014-08-13 13:43:54 +00:00
|
|
|
formatAST(query_ast, s, 0, false, true);
|
2014-08-13 12:52:30 +00:00
|
|
|
|
|
|
|
return s.str();
|
|
|
|
}
|
|
|
|
|
2014-08-13 13:43:54 +00:00
|
|
|
/// select and insert query have different types for database and table, hence two specializations
|
|
|
|
template <typename ASTType> struct rewrite_traits;
|
|
|
|
template <> struct rewrite_traits<ASTSelectQuery> { using type = ASTPtr; };
|
|
|
|
template <> struct rewrite_traits<ASTInsertQuery> { using type = const std::string &; };
|
|
|
|
|
|
|
|
template <typename ASTType>
|
|
|
|
typename rewrite_traits<ASTType>::type rewrite(const std::string & name, const ASTIdentifier::Kind kind) = delete;
|
|
|
|
|
|
|
|
/// select query has database and table names as AST pointers
|
|
|
|
template <>
|
|
|
|
inline ASTPtr rewrite<ASTSelectQuery>(const std::string & name, const ASTIdentifier::Kind kind)
|
2014-08-13 12:52:30 +00:00
|
|
|
{
|
|
|
|
return new ASTIdentifier{{}, name, kind};
|
|
|
|
}
|
|
|
|
|
2014-08-13 13:43:54 +00:00
|
|
|
/// insert query has database and table names as bare strings
|
|
|
|
template <>
|
|
|
|
inline const std::string & rewrite<ASTInsertQuery>(const std::string & name, ASTIdentifier::Kind)
|
2014-08-13 12:52:30 +00:00
|
|
|
{
|
|
|
|
return name;
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Создает копию запроса, меняет имена базы данных и таблицы.
|
|
|
|
template <typename ASTType>
|
|
|
|
inline ASTPtr rewriteQuery(const ASTPtr & query, const std::string & database, const std::string & table)
|
|
|
|
{
|
|
|
|
/// Создаем копию запроса.
|
|
|
|
auto modified_query_ast = query->clone();
|
|
|
|
|
|
|
|
/// Меняем имена таблицы и базы данных
|
|
|
|
auto & modified_query = typeid_cast<ASTType &>(*modified_query_ast);
|
2014-08-13 13:43:54 +00:00
|
|
|
modified_query.database = rewrite<ASTType>(database, ASTIdentifier::Database);
|
|
|
|
modified_query.table = rewrite<ASTType>(table, ASTIdentifier::Table);
|
2014-08-13 12:52:30 +00:00
|
|
|
|
|
|
|
/// copy elision and RVO will work as intended, but let's be more explicit
|
|
|
|
return std::move(modified_query_ast);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2012-11-06 17:04:38 +00:00
|
|
|
StorageDistributed::StorageDistributed(
|
|
|
|
const std::string & name_,
|
|
|
|
NamesAndTypesListPtr columns_,
|
|
|
|
const String & remote_database_,
|
|
|
|
const String & remote_table_,
|
2013-12-07 16:51:29 +00:00
|
|
|
Cluster & cluster_,
|
2014-08-12 13:46:46 +00:00
|
|
|
const 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_),
|
2014-08-13 09:20:15 +00:00
|
|
|
context(context_), cluster(cluster_),
|
|
|
|
sharding_key_expr(sharding_key_ ? ExpressionAnalyzer(sharding_key_, context, *columns).getActions(false) : nullptr),
|
|
|
|
sharding_key_column_name(sharding_key_ ? sharding_key_->getColumnName() : String{}),
|
|
|
|
write_enabled(cluster.getLocalNodesNum() + cluster.pools.size() < 2 || sharding_key_),
|
|
|
|
path(data_path_ + escapeForFileName(name) + '/')
|
2013-11-28 10:31:17 +00:00
|
|
|
{
|
2014-08-13 09:20:15 +00:00
|
|
|
std::cout << "table `" << name << "` in " << path << std::endl;
|
2014-08-13 11:26:13 +00:00
|
|
|
|
|
|
|
createDirectoryMonitors();
|
2013-11-28 10:31:17 +00:00
|
|
|
}
|
|
|
|
|
2013-02-06 11:26:35 +00:00
|
|
|
StoragePtr StorageDistributed::create(
|
|
|
|
const std::string & name_,
|
|
|
|
NamesAndTypesListPtr columns_,
|
|
|
|
const String & remote_database_,
|
|
|
|
const String & remote_table_,
|
2013-12-10 17:06:57 +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
|
|
|
{
|
2013-12-10 17:06:57 +00:00
|
|
|
context_.initClusters();
|
2014-08-13 09:20:15 +00:00
|
|
|
|
|
|
|
return (new StorageDistributed{
|
|
|
|
name_, columns_, remote_database_, remote_table_,
|
|
|
|
context_.getCluster(cluster_name), context_,
|
|
|
|
sharding_key_, data_path_
|
|
|
|
})->thisPtr();
|
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_,
|
2014-02-23 02:27:09 +00:00
|
|
|
SharedPtr<Cluster> & owned_cluster_,
|
2014-08-13 09:20:15 +00:00
|
|
|
Context & context_)
|
2014-02-07 15:11:57 +00:00
|
|
|
{
|
2014-08-13 09:20:15 +00:00
|
|
|
auto res = new StorageDistributed{
|
|
|
|
name_, columns_, remote_database_,
|
|
|
|
remote_table_, *owned_cluster_, context_
|
|
|
|
};
|
2014-02-23 02:27:09 +00:00
|
|
|
|
|
|
|
/// Захватываем владение объектом-кластером.
|
|
|
|
res->owned_cluster = owned_cluster_;
|
|
|
|
|
|
|
|
return res->thisPtr();
|
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,
|
2013-02-01 19:02:04 +00:00
|
|
|
const Settings & settings,
|
2012-05-22 18:32:45 +00:00
|
|
|
QueryProcessingStage::Enum & processed_stage,
|
2012-05-21 20:38:34 +00:00
|
|
|
size_t max_block_size,
|
2012-05-30 04:45:49 +00:00
|
|
|
unsigned threads)
|
2012-05-21 20:38:34 +00:00
|
|
|
{
|
2013-05-08 10:30:29 +00:00
|
|
|
Settings new_settings = settings;
|
2014-02-22 21:50:27 +00:00
|
|
|
new_settings.queue_max_wait_ms = Cluster::saturate(new_settings.queue_max_wait_ms, settings.limits.max_execution_time);
|
2012-05-21 20:38:34 +00:00
|
|
|
|
2014-04-06 21:59:27 +00:00
|
|
|
size_t result_size = cluster.pools.size() + cluster.getLocalNodesNum();
|
2014-02-11 17:10:48 +00:00
|
|
|
|
|
|
|
processed_stage = result_size == 1
|
|
|
|
? QueryProcessingStage::Complete
|
|
|
|
: QueryProcessingStage::WithMergeableState;
|
|
|
|
|
2012-05-21 20:38:34 +00:00
|
|
|
BlockInputStreams res;
|
2014-08-13 12:52:30 +00:00
|
|
|
const auto & modified_query_ast = rewriteQuery<ASTSelectQuery>(
|
|
|
|
query, remote_database, remote_table
|
|
|
|
);
|
|
|
|
const auto & modified_query = queryToString<ASTSelectQuery>(modified_query_ast);
|
2012-05-21 20:38:34 +00:00
|
|
|
|
2014-04-07 00:00:23 +00:00
|
|
|
/// Цикл по шардам.
|
2014-02-23 02:27:09 +00:00
|
|
|
for (auto & conn_pool : cluster.pools)
|
2014-08-13 12:52:30 +00:00
|
|
|
res.emplace_back(new RemoteBlockInputStream{
|
|
|
|
conn_pool, modified_query, &new_settings,
|
|
|
|
external_tables, processed_stage
|
|
|
|
});
|
2014-01-22 14:24:05 +00:00
|
|
|
|
2014-04-07 00:00:23 +00:00
|
|
|
/// Добавляем запросы к локальному ClickHouse.
|
2014-04-06 21:59:27 +00:00
|
|
|
if (cluster.getLocalNodesNum() > 0)
|
2013-11-28 10:31:17 +00:00
|
|
|
{
|
|
|
|
DB::Context new_context = context;
|
|
|
|
new_context.setSettings(new_settings);
|
2014-03-13 15:00:06 +00:00
|
|
|
for (auto & it : external_tables)
|
2014-03-19 15:07:29 +00:00
|
|
|
if (!new_context.tryGetExternalTable(it.first))
|
|
|
|
new_context.addExternalTable(it.first, it.second);
|
2014-02-20 20:05:06 +00:00
|
|
|
|
|
|
|
for(size_t i = 0; i < cluster.getLocalNodesNum(); ++i)
|
2013-11-28 10:31:17 +00:00
|
|
|
{
|
2014-02-20 20:05:06 +00:00
|
|
|
InterpreterSelectQuery interpreter(modified_query_ast, new_context, processed_stage);
|
|
|
|
res.push_back(interpreter.execute());
|
2013-11-28 10:31:17 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2014-04-06 21:59:27 +00:00
|
|
|
external_tables.clear();
|
2012-05-21 20:38:34 +00:00
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
2014-08-12 13:46:46 +00:00
|
|
|
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
|
|
|
|
};
|
|
|
|
|
2014-08-13 12:52:30 +00:00
|
|
|
return new DistributedBlockOutputStream{
|
|
|
|
*this, this->cluster,
|
|
|
|
queryToString<ASTInsertQuery>(rewriteQuery<ASTInsertQuery>(
|
|
|
|
query, remote_database, remote_table
|
|
|
|
))
|
|
|
|
};
|
2014-08-12 13:46:46 +00:00
|
|
|
}
|
|
|
|
|
2014-07-11 08:12:03 +00:00
|
|
|
void StorageDistributed::alter(const AlterCommands & params, const String & database_name, const String & table_name, Context & context)
|
2013-09-23 12:01:19 +00:00
|
|
|
{
|
2014-07-11 08:12:03 +00:00
|
|
|
auto lock = lockStructureForAlter();
|
|
|
|
params.apply(*columns);
|
|
|
|
InterpreterAlterQuery::updateMetadata(database_name, table_name, *columns, context);
|
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()
|
|
|
|
{
|
|
|
|
quit.store(true, std::memory_order_relaxed);
|
|
|
|
|
|
|
|
for (auto & name_thread_pair : directory_monitor_threads)
|
|
|
|
name_thread_pair.second.join();
|
|
|
|
}
|
|
|
|
|
2014-07-31 09:11:49 +00:00
|
|
|
NameAndTypePair StorageDistributed::getColumn(const String & column_name) const
|
|
|
|
{
|
|
|
|
auto type = VirtualColumnFactory::tryGetType(column_name);
|
|
|
|
if (type)
|
|
|
|
return NameAndTypePair(column_name, type);
|
|
|
|
|
|
|
|
return getRealColumn(column_name);
|
|
|
|
}
|
|
|
|
|
|
|
|
bool StorageDistributed::hasColumn(const String & column_name) const
|
|
|
|
{
|
|
|
|
return VirtualColumnFactory::hasColumn(column_name) || hasRealColumn(column_name);
|
|
|
|
}
|
|
|
|
|
2014-08-13 11:26:13 +00:00
|
|
|
void StorageDistributed::createDirectoryMonitors()
|
|
|
|
{
|
|
|
|
Poco::File(path).createDirectory();
|
|
|
|
|
|
|
|
Poco::DirectoryIterator end;
|
|
|
|
for (Poco::DirectoryIterator it(path); it != end; ++it)
|
|
|
|
if (it->isDirectory())
|
|
|
|
createDirectoryMonitor(it.name());
|
|
|
|
}
|
|
|
|
|
|
|
|
void StorageDistributed::createDirectoryMonitor(const std::string & name)
|
|
|
|
{
|
|
|
|
if (directory_monitor_threads.count(name))
|
|
|
|
return;
|
|
|
|
|
|
|
|
directory_monitor_threads.emplace(
|
|
|
|
name,
|
|
|
|
std::thread{
|
2014-08-13 15:07:53 +00:00
|
|
|
&StorageDistributed::directoryMonitorFunc, this, name
|
2014-08-13 11:26:13 +00:00
|
|
|
}
|
|
|
|
);
|
|
|
|
}
|
|
|
|
|
2014-08-13 15:07:53 +00:00
|
|
|
void StorageDistributed::directoryMonitorFunc(const std::string & name)
|
2014-08-13 11:26:13 +00:00
|
|
|
{
|
2014-08-14 10:38:06 +00:00
|
|
|
const auto & path = this->path + name + '/';
|
2014-08-13 11:26:13 +00:00
|
|
|
std::cout << "created monitor for directory " << path << std::endl;
|
|
|
|
|
2014-08-14 10:38:06 +00:00
|
|
|
// ConnectionPools pools;
|
2014-08-13 15:07:53 +00:00
|
|
|
for (auto it = boost::make_split_iterator(name, boost::first_finder(",")); it != decltype(it){}; ++it)
|
|
|
|
{
|
|
|
|
const auto & address = boost::copy_range<std::string>(*it);
|
|
|
|
|
2014-08-14 10:38:06 +00:00
|
|
|
const auto user_pw_end = strchr(address.data(), '@');
|
|
|
|
const auto colon = strchr(address.data(), ':');
|
|
|
|
if (!user_pw_end || !colon)
|
|
|
|
throw Exception{"Shard address '" + address + "' does not match to 'user[:password]@host:port' pattern"};
|
|
|
|
|
|
|
|
const auto has_pw = colon < user_pw_end;
|
|
|
|
const auto host_end = has_pw ? colon : strchr(user_pw_end + 1, ':');
|
2014-08-13 15:07:53 +00:00
|
|
|
if (!host_end)
|
|
|
|
throw Exception{"Shard address '" + address + "' does not contain port"};
|
|
|
|
|
2014-08-14 10:38:06 +00:00
|
|
|
const std::string user{address.data(), has_pw ? colon : user_pw_end};
|
|
|
|
const auto password = has_pw ? std::string{colon + 1, user_pw_end} : std::string{};
|
|
|
|
const std::string host{user_pw_end + 1, host_end};
|
2014-08-13 15:07:53 +00:00
|
|
|
const auto port = DB::parse<UInt16>(host_end + 1);
|
|
|
|
|
2014-08-14 10:38:06 +00:00
|
|
|
// pools.emplace_back(new ConnectionPool(1, host, port, remote_database, "default", "", getName() + '_' + name));
|
|
|
|
std::cout
|
|
|
|
<< "\taddress " << host
|
|
|
|
<< " port " << port
|
|
|
|
<< " user " << user
|
|
|
|
<< " password " << password
|
|
|
|
<< std::endl;
|
2014-08-13 15:07:53 +00:00
|
|
|
}
|
|
|
|
|
2014-08-14 10:38:06 +00:00
|
|
|
// auto pool = pools.size() == 1 ? pools[0] : new ConnectionPoolWithFailover(pools, DB::LoadBalancing::RANDOM);
|
2014-08-13 15:07:53 +00:00
|
|
|
|
2014-08-13 11:26:13 +00:00
|
|
|
while (!quit.load(std::memory_order_relaxed))
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
std::cout << "exiting monitor for directory " << path << std::endl;
|
|
|
|
}
|
|
|
|
|
2012-05-21 20:38:34 +00:00
|
|
|
}
|