Distributed DDL PR rebase. [#CLICKHOUSE-5]

This commit is contained in:
Vitaliy Lyudvichenko 2017-04-13 16:42:29 +03:00
parent 5df9848326
commit aa7504a7f5
11 changed files with 324 additions and 7 deletions

View File

@ -124,6 +124,7 @@ struct ContextShared
std::unique_ptr<Compiler> compiler; /// Used for dynamic compilation of queries' parts if it necessary.
std::unique_ptr<QueryLog> query_log; /// Used to log queries.
std::shared_ptr<PartLog> part_log; /// Used to log operations with parts
std::shared_ptr<DDLWorker> ddl_worker; /// Process ddl commands from zk.
/// Rules for selecting the compression method, depending on the size of the part.
mutable std::unique_ptr<CompressionMethodSelector> compression_method_selector;
std::unique_ptr<MergeTreeSettings> merge_tree_settings; /// Settings of MergeTree* engines.
@ -963,6 +964,22 @@ ReshardingWorker & Context::getReshardingWorker()
return *shared->resharding_worker;
}
void Context::setDDLWorker(std::shared_ptr<DDLWorker> ddl_worker)
{
auto lock = getLock();
if (shared->ddl_worker)
throw Exception("DDL background thread has already been initialized.", ErrorCodes::LOGICAL_ERROR);
shared->ddl_worker = ddl_worker;
}
DDLWorker & Context::getDDLWorker()
{
auto lock = getLock();
if (!shared->ddl_worker)
throw Exception("DDL background thread not initialized.", ErrorCodes::LOGICAL_ERROR);
return *shared->ddl_worker;
}
void Context::resetCaches() const
{
auto lock = getLock();

View File

@ -51,6 +51,7 @@ class PartLog;
struct MergeTreeSettings;
class IDatabase;
class DDLGuard;
class DDLWorker;
class IStorage;
using StoragePtr = std::shared_ptr<IStorage>;
using Tables = std::map<String, StoragePtr>;
@ -267,6 +268,9 @@ public:
void setReshardingWorker(std::shared_ptr<ReshardingWorker> resharding_worker);
ReshardingWorker & getReshardingWorker();
void setDDLWorker(std::shared_ptr<DDLWorker> ddl_worker);
DDLWorker & getDDLWorker();
/** Очистить кэши разжатых блоков и засечек.
* Обычно это делается при переименовании таблиц, изменении типа столбцов, удалении таблицы.
* - так как кэши привязаны к именам файлов, и становятся некорректными.

View File

@ -0,0 +1,138 @@
#include <Common/getFQDNOrHostName.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/executeQuery.h>
#include <zkutil/ZooKeeper.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
extern const int INVALID_CONFIG_PARAMETER;
}
namespace {
/// Helper class which extracts from the ClickHouse configuration file
/// the parameters we need for operating the resharding thread.
class Arguments final
{
public:
Arguments(const Poco::Util::AbstractConfiguration & config, const std::string & config_name)
{
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_name, keys);
for (const auto & key : keys)
{
if (key == "task_queue_path")
task_queue_path = config.getString(config_name + "." + key);
else
throw Exception{"Unknown parameter in resharding configuration", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG};
}
if (task_queue_path.empty())
throw Exception{"Resharding: missing parameter task_queue_path", ErrorCodes::INVALID_CONFIG_PARAMETER};
}
Arguments(const Arguments &) = delete;
Arguments & operator=(const Arguments &) = delete;
std::string getTaskQueuePath() const
{
return task_queue_path;
}
private:
std::string task_queue_path;
};
}
DDLWorker::DDLWorker(const Poco::Util::AbstractConfiguration & config,
const std::string & config_name, Context & context_)
: context(context_)
, stop_flag(false)
{
Arguments arguments(config, config_name);
auto zookeeper = context.getZooKeeper();
std::string root = arguments.getTaskQueuePath();
if (root.back() != '/')
root += "/";
auto current_host = getFQDNOrHostName();
host_task_queue_path = "/clickhouse/task_queue/ddl/" + current_host;
thread = std::thread(&DDLWorker::run, this);
}
DDLWorker::~DDLWorker()
{
stop_flag = true;
cond_var.notify_one();
thread.join();
}
void DDLWorker::processTasks()
{
processCreate(host_task_queue_path + "/create");
}
void DDLWorker::processCreate(const std::string & path)
{
auto zookeeper = context.getZooKeeper();
if (!zookeeper->exists(path))
return;
const Strings & children = zookeeper->getChildren(path);
for (const auto & name : children)
{
const std::string & query_path = path + "/" + name;
try
{
std::string value;
if (zookeeper->tryGet(query_path, value))
{
if (!value.empty())
executeQuery(value, context);
zookeeper->remove(query_path);
}
}
catch (const std::exception & ex)
{
LOG_ERROR(log, ex.what() + std::string(" on ") + query_path);
}
}
}
void DDLWorker::run()
{
using namespace std::chrono_literals;
while (!stop_flag)
{
try
{
processTasks();
}
catch (const std::exception & ex)
{
LOG_ERROR(log, ex.what());
}
std::unique_lock<std::mutex> g(lock);
cond_var.wait_for(g, 10s);
}
}
}

View File

@ -0,0 +1,40 @@
#pragma once
#include <Interpreters/Context.h>
#include <common/logger_useful.h>
#include <atomic>
#include <chrono>
#include <condition_variable>
#include <mutex>
#include <thread>
namespace DB
{
class DDLWorker
{
public:
DDLWorker(const Poco::Util::AbstractConfiguration & config,
const std::string & config_name, Context & context_);
~DDLWorker();
private:
void processTasks();
void processCreate(const std::string & path);
void run();
private:
Context & context;
Logger * log = &Logger::get("DDLWorker");
std::string host_task_queue_path;
std::atomic<bool> stop_flag;
std::condition_variable cond_var;
std::mutex lock;
std::thread thread;
};
}

View File

@ -39,6 +39,7 @@
#include <Databases/DatabaseFactory.h>
#include <Databases/IDatabase.h>
#include <zkutil/ZooKeeper.h>
namespace DB
{
@ -56,6 +57,21 @@ namespace ErrorCodes
extern const int DUPLICATE_COLUMN;
}
static void ExecuteQuery(const Cluster::Address & addr, const std::string & query)
{
Connection conn(addr.host_name, addr.port, "", addr.user, addr.password);
conn.sendQuery(query);
while (true)
{
Connection::Packet packet = conn.receivePacket();
if (packet.type == Protocol::Server::Exception)
throw Exception(*packet.exception.get());
else if (packet.type == Protocol::Server::EndOfStream)
break;
}
}
InterpreterCreateQuery::InterpreterCreateQuery(const ASTPtr & query_ptr_, Context & context_)
: query_ptr(query_ptr_), context(context_)
@ -562,6 +578,62 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
return {};
}
ASTPtr InterpreterCreateQuery::createQueryWithoutCluster(ASTCreateQuery & create) const
{
ASTPtr cloned = create.clone();
ASTCreateQuery & tmp = typeid_cast<ASTCreateQuery &>(*cloned);
tmp.cluster.clear();
if (tmp.database.empty())
tmp.database = context.getCurrentDatabase();
return cloned;
}
void InterpreterCreateQuery::writeToZookeeper(ASTPtr query, const std::vector<Cluster::Address> & addrs)
{
auto zookeeper = context.getZooKeeper();
ASTCreateQuery & create = typeid_cast<ASTCreateQuery &>(*query);
std::string table_name = create.database + "." + create.table;
for (const auto & addr : addrs)
{
const std::string & path =
"/clickhouse/task_queue/ddl/" +
addr.host_name +
"/create/" + table_name;
// TODO catch exceptions
zookeeper->createAncestors(path);
zookeeper->create(path, formatASTToString(*query), 0);
}
}
BlockIO InterpreterCreateQuery::createTableOnCluster(ASTCreateQuery & create)
{
ClusterPtr cluster = context.getCluster(create.cluster);
Cluster::AddressesWithFailover shards = cluster->getShardsWithFailoverAddresses();
ASTPtr query_ptr = createQueryWithoutCluster(create);
std::string query = formatASTToString(*query_ptr);
std::vector<Cluster::Address> failed;
for (const auto & shard : shards)
{
for (const auto & addr : shard)
{
try
{
ExecuteQuery(addr, query);
}
catch (...)
{
failed.push_back(addr);
}
}
}
writeToZookeeper(query_ptr, failed);
return {};
}
BlockIO InterpreterCreateQuery::execute()
{
@ -573,6 +645,8 @@ BlockIO InterpreterCreateQuery::execute()
createDatabase(create);
return {};
}
else if (!create.cluster.empty())
return createTableOnCluster(create);
else
return createTable(create);
}

View File

@ -57,6 +57,10 @@ public:
private:
void createDatabase(ASTCreateQuery & create);
BlockIO createTable(ASTCreateQuery & create);
BlockIO createTableOnCluster(ASTCreateQuery & create);
ASTPtr createQueryWithoutCluster(ASTCreateQuery & create) const;
void writeToZookeeper(ASTPtr query, const std::vector<Cluster::Address>& addrs);
/// Calculate list of columns of table and return it.
ColumnsInfo setColumns(ASTCreateQuery & create, const Block & as_select_sample, const StoragePtr & as_storage) const;

View File

@ -21,6 +21,7 @@ public:
bool is_temporary{false};
String database;
String table;
String cluster;
ASTPtr columns;
ASTPtr storage;
ASTPtr inner_storage; /// Internal engine for the CREATE MATERIALIZED VIEW query

View File

@ -151,6 +151,8 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
ParserString s_attach("ATTACH", true, true);
ParserString s_table("TABLE", true, true);
ParserString s_database("DATABASE", true, true);
ParserString s_on("ON", true, true);
ParserString s_cluster("CLUSTER", true, true);
ParserString s_dot(".");
ParserString s_lparen("(");
ParserString s_rparen(")");
@ -173,6 +175,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
ASTPtr inner_storage;
ASTPtr as_database;
ASTPtr as_table;
ASTPtr cluster;
ASTPtr select;
bool attach = false;
bool if_not_exists = false;
@ -244,7 +247,22 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
ws.ignore(pos, end);
}
/// Columns list
if (s_on.ignore(pos, end, max_parsed_pos, expected))
{
ws.ignore(pos, end);
if (!s_cluster.ignore(pos, end, max_parsed_pos, expected))
return false;
ws.ignore(pos, end);
if (!name_p.parse(pos, end, cluster, max_parsed_pos, expected))
return false;
ws.ignore(pos, end);
}
/// List of columns.
if (s_lparen.ignore(pos, end, max_parsed_pos, expected))
{
ws.ignore(pos, end);
@ -403,6 +421,8 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
query->database = typeid_cast<ASTIdentifier &>(*database).name;
if (table)
query->table = typeid_cast<ASTIdentifier &>(*table).name;
if (cluster)
query->cluster = typeid_cast<ASTIdentifier &>(*cluster).name;
if (inner_storage)
query->inner_storage = inner_storage;

View File

@ -1,9 +1,23 @@
#include <Parsers/formatAST.h>
#include <sstream>
namespace DB
{
void formatAST(const IAST & ast, std::ostream & s, size_t indent, bool hilite, bool one_line)
{
IAST::FormatSettings settings(s, hilite, one_line);
ast.format(settings);
}
std::string formatASTToString(const IAST & ast)
{
std::stringstream s;
formatAST(ast, s, 0, false, true);
return s.str();
}
String formatColumnsForCreateQuery(NamesAndTypesList & columns)
{
std::string res;

View File

@ -12,12 +12,9 @@ namespace DB
/** Takes a syntax tree and turns it back into text.
* In case of INSERT query, the data will be missing.
*/
inline void formatAST(const IAST & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false)
{
IAST::FormatSettings settings(s, hilite, one_line);
ast.format(settings);
}
void formatAST(const IAST & ast, std::ostream & s, size_t indent = 0, bool hilite = true, bool one_line = false);
std::string formatASTToString(const IAST & ast);
String formatColumnsForCreateQuery(NamesAndTypesList & columns);

View File

@ -21,6 +21,8 @@
#include <Interpreters/AsynchronousMetrics.h>
#include <Interpreters/ProcessList.h>
#include <Interpreters/loadMetadata.h>
#include <Interpreters/DDLWorker.h>
#include <Storages/MergeTree/ReshardingWorker.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/System/attachSystemTables.h>
@ -412,7 +414,13 @@ int Server::main(const std::vector<std::string> & args)
has_resharding_worker = true;
}
SCOPE_EXIT({
if (has_zookeeper && config().has("distributed_ddl"))
{
auto ddl_worker = std::make_shared<DDLWorker>(config(), "distributed_ddl", *global_context);
global_context->setDDLWorker(ddl_worker);
}
SCOPE_EXIT(
/** Ask to cancel background jobs all table engines,
* and also query_log.
* It is important to do early, not in destructor of Context, because