DDLWorker.cpp 15.1 KB
Newer Older
1
#include <Interpreters/DDLWorker.h>
2 3

#include <Parsers/ASTAlterQuery.h>
4
#include <Parsers/ASTQueryWithOnCluster.h>
5

6 7 8 9 10
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <IO/Operators.h>
#include <IO/ReadBufferFromString.h>

11
#include <Interpreters/executeQuery.h>
12 13 14 15 16 17 18 19 20 21 22
#include <Interpreters/Cluster.h>
#include <DataStreams/OneBlockInputStream.h>
#include <Common/getFQDNOrHostName.h>

#include <Core/Block.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeArray.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnArray.h>
23 24

#include <zkutil/ZooKeeper.h>
25 26
#include <Poco/Timestamp.h>

27 28 29 30 31 32 33 34

namespace DB
{

namespace ErrorCodes
{
    extern const int UNKNOWN_ELEMENT_IN_CONFIG;
    extern const int INVALID_CONFIG_PARAMETER;
35
    extern const int UNKNOWN_FORMAT_VERSION;
36
    extern const int UNFINISHED;
37 38 39
}


40 41 42 43 44 45 46 47 48 49 50 51 52 53 54 55
struct DDLLogEntry
{
    String query;
    Strings hosts;
    String initiator;

    static constexpr char CURRENT_VERSION = '1';

    String toString()
    {
        String res;
        {
            WriteBufferFromString wb(res);

            writeChar(CURRENT_VERSION, wb);
            wb << "\n";
56 57 58
            wb << "query: " << query << "\n";
            wb << "hosts: " << hosts << "\n";
            wb << "initiator: " << initiator << "\n";
59 60 61 62 63 64 65 66 67 68 69 70 71 72 73
        }

        return res;
    }

    void parse(const String & data)
    {
        ReadBufferFromString rb(data);

        char version;
        readChar(version, rb);
        if (version != CURRENT_VERSION)
            throw Exception("Unknown DDLLogEntry format version: " + version, ErrorCodes::UNKNOWN_FORMAT_VERSION);

        rb >> "\n";
74 75 76
        rb >> "query: " >> query >> "\n";
        rb >> "hosts: " >> hosts >> "\n";
        rb >> "initiator: " >> initiator >> "\n";
77 78 79 80

        assertEOF(rb);
    }
};
81 82


83 84 85 86 87 88
DDLWorker::DDLWorker(const std::string & zk_root_dir, Context & context_)
    : context(context_), stop_flag(false)
{
    root_dir = zk_root_dir;
    if (root_dir.back() == '/')
        root_dir.resize(root_dir.size() - 1);
89

90
    hostname = getFQDNOrHostName() + ':' + DB::toString(context.getTCPPort());
91

92 93 94
    zookeeper = context.getZooKeeper();
    event_queue_updated = std::make_shared<Poco::Event>();

95 96 97
    thread = std::thread(&DDLWorker::run, this);
}

98

99 100 101
DDLWorker::~DDLWorker()
{
    stop_flag = true;
102 103
    //cond_var.notify_one();
    event_queue_updated->set();
104 105 106
    thread.join();
}

107

108 109
void DDLWorker::processTasks()
{
110
    LOG_DEBUG(log, "processTasks");
111

112 113
    Strings queue_nodes = zookeeper->getChildren(root_dir, nullptr, event_queue_updated);
    if (queue_nodes.empty())
114 115
        return;

116 117
    LOG_DEBUG(log, "fetched " << queue_nodes.size() << " nodes");

118
    bool server_startup = last_processed_node_name.empty();
119

120 121 122 123
    std::sort(queue_nodes.begin(), queue_nodes.end());
    auto begin_node = server_startup
        ? queue_nodes.begin()
        : std::upper_bound(queue_nodes.begin(), queue_nodes.end(), last_processed_node_name);
124

125
    for (auto it = begin_node; it != queue_nodes.end(); ++it)
126
    {
127
        String node_data, node_name = *it, node_path = root_dir + "/" + node_name;
128 129 130 131 132 133
        LOG_DEBUG(log, "Fetching node " << node_path);
        if (!zookeeper->tryGet(node_path, node_data))
        {
            /// It is Ok that node could be deleted just now. It means that there are no current host in node's host list.
            continue;
        }
134

135
        LOG_DEBUG(log, "Fetched data for node " << node_name);
136 137 138 139 140 141

        DDLLogEntry node;
        node.parse(node_data);

        bool host_in_hostlist = std::find(node.hosts.cbegin(), node.hosts.cend(), hostname) != node.hosts.cend();

142 143 144 145
        bool already_processed = zookeeper->exists(node_path + "/failed/" + hostname)
                                 || zookeeper->exists(node_path + "/sucess/" + hostname);

        LOG_DEBUG(log, "node " << node_name << ", " << node.query << " status: " << host_in_hostlist << " " << already_processed);
146 147 148 149 150 151 152 153 154 155 156 157 158 159 160 161 162 163 164 165 166 167 168 169 170 171

        if (!server_startup && already_processed)
        {
            throw Exception(
                "Server expects that DDL node " + node_name + " should be processed, but it was already processed according to ZK",
                ErrorCodes::LOGICAL_ERROR);
        }

        if (host_in_hostlist && !already_processed)
        {
            try
            {
                processTask(node, node_name);
            }
            catch (...)
            {
                /// It could be network error, but we mark node as processed anyway.
                last_processed_node_name = node_name;

                tryLogCurrentException(log,
                    "An unexpected error occurred during processing DLL query " + node.query + " (" + node_name + ")");
                throw;
            }
        }

        last_processed_node_name = node_name;
172
    }
173 174
}

175 176 177 178 179 180 181
void DDLWorker::cleanupQueue(const Strings * node_names_to_check)
{
    /// Both ZK and Poco use Unix epoch
    size_t current_time_seconds = Poco::Timestamp().epochTime();
    constexpr size_t zookeeper_time_resolution = 1000;

    // Too early to check
182
    if (last_cleanup_time_seconds && current_time_seconds < last_cleanup_time_seconds + cleanup_after_seconds)
183 184 185 186 187 188 189 190 191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230
        return;

    last_cleanup_time_seconds = current_time_seconds;

    String data;
    zkutil::Stat stat;
    DDLLogEntry node;
    Strings failed_hosts, sucess_hosts;

    Strings node_names_fetched = node_names_to_check ? Strings{} : zookeeper->getChildren(root_dir);
    const Strings & node_names = (node_names_to_check) ? *node_names_to_check : node_names_fetched;

    for (const String & node_name : node_names)
    {
        /// TODO: Add /root/locks/node_name lock to avoid rare race counditions.
        try
        {
            String node_path = root_dir + "/" + node_name;
            if (!zookeeper->tryGet(node_path, data, &stat))
                continue;

            node.parse(data);

            size_t zookeeper_time_seconds = stat.mtime / zookeeper_time_resolution;
            if (zookeeper_time_seconds + node_max_lifetime_seconds < current_time_seconds)
            {
                size_t lifetime_seconds = current_time_seconds - zookeeper_time_seconds;
                LOG_INFO(log, "Lifetime of node " << node_name << " (" << lifetime_seconds << " sec.) is expired, deleting it");
                zookeeper->removeRecursive(node_path);
                continue;
            }

            Strings sucess_nodes = zookeeper->getChildren(node_path + "/sucess");
            Strings failed_nodes = zookeeper->getChildren(node_path + "/failed");

            if (sucess_nodes.size() + failed_nodes.size() >= node.hosts.size())
            {
                LOG_INFO(log, "Node " << node_name << " had been executed by each host, deleting it");
                zookeeper->removeRecursive(node_path);
            }
        }
        catch (...)
        {
            tryLogCurrentException(log, "An error occured while checking and cleaning node " + node_name + " from queue");
        }
    }
}

231

232 233
/// Try to create unexisting "status" dirs for a node
void DDLWorker::createStatusDirs(const std::string & node_path)
234
{
235
    auto acl = zookeeper->getDefaultACL();
236

237 238 239 240
    zkutil::Ops ops;
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(node_path + "/active", "", acl, zkutil::CreateMode::Persistent));
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(node_path + "/sucess", "", acl, zkutil::CreateMode::Persistent));
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(node_path + "/failed", "", acl, zkutil::CreateMode::Persistent));
241

242 243 244 245 246
    int code = zookeeper->tryMulti(ops);

    if (code != ZOK && code != ZNODEEXISTS)
        throw zkutil::KeeperException(code);
}
247 248


249 250
bool DDLWorker::processTask(const DDLLogEntry & node, const std::string & node_name)
{
251 252
    LOG_DEBUG(log, "Process " << node_name << " node, query " << node.query);

253 254 255
    String node_path = root_dir + "/" + node_name;
    createStatusDirs(node_path);

256 257
    LOG_DEBUG(log, "Created status dirs");

258 259
    String active_flag_path = node_path + "/active/" + hostname;
    zookeeper->create(active_flag_path, "", zkutil::CreateMode::Ephemeral);
260

261 262
    LOG_DEBUG(log, "Created active flag");

263
    /// At the end we will delete active flag and ...
264
    zkutil::Ops ops;
265 266
    auto acl = zookeeper->getDefaultACL();
    ops.emplace_back(std::make_unique<zkutil::Op::Remove>(active_flag_path, -1));
267

268 269
    LOG_DEBUG(log, "Executing query: " << node.query);

270 271
    try
    {
272
        executeQuery(node.query, context);
273 274 275 276
    }
    catch (...)
    {
        /// ... and create fail flag
277
        String fail_flag_path = node_path + "/failed/" + hostname;
278
        String exception_msg = getCurrentExceptionMessage(false, true);
279 280

        ops.emplace_back(std::make_unique<zkutil::Op::Create>(fail_flag_path, exception_msg, acl, zkutil::CreateMode::Persistent));
281 282
        zookeeper->multi(ops);

283
        tryLogCurrentException(log, "Query " + node.query + " wasn't finished successfully");
284 285 286
        return false;
    }

287 288
    LOG_DEBUG(log, "Executed query: " << node.query);

289 290 291
    /// ... and create sucess flag
    String fail_flag_path = node_path + "/sucess/" + hostname;
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(fail_flag_path, "", acl, zkutil::CreateMode::Persistent));
292 293
    zookeeper->multi(ops);

294 295
    LOG_DEBUG(log, "Removed flags");

296 297 298 299
    return true;
}


300
String DDLWorker::enqueueQuery(DDLLogEntry & entry)
301
{
302
    if (entry.hosts.empty())
303
        return {};
304

305
    String query_path_prefix = root_dir + "/query-";
306
    zookeeper->createAncestors(query_path_prefix);
307

308 309
    String node_path = zookeeper->create(query_path_prefix, entry.toString(), zkutil::CreateMode::PersistentSequential);
    createStatusDirs(node_path);
310 311

    return node_path;
312 313
}

314

315 316 317 318
void DDLWorker::run()
{
    using namespace std::chrono_literals;

319
    zookeeper->createAncestors(root_dir + "/");
320 321
    LOG_DEBUG(log, "Started DDLWorker thread");

322 323
    while (!stop_flag)
    {
324 325
        LOG_DEBUG(log, "Begin tasks processing");

326 327
        try
        {
328
            cleanupQueue();
329
        }
330
        catch (...)
331
        {
332
            tryLogCurrentException(log);
333 334
        }

335 336
        try
        {
337
            processTasks();
338 339 340 341 342 343
        }
        catch (...)
        {
            tryLogCurrentException(log);
        }

344 345 346
        //std::unique_lock<std::mutex> g(lock);
        //cond_var.wait_for(g, 10s);

347
        LOG_DEBUG(log, "Waiting watch");
348
        event_queue_updated->wait();
349 350 351
    }
}

352

353
class DDLQueryStatusInputSream : public IProfilingBlockInputStream
354
{
355
public:
356

357 358 359 360 361 362 363 364 365 366 367 368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395
    DDLQueryStatusInputSream(const String & zk_node_path, Context & context, size_t num_hosts)
    : node_path(zk_node_path), context(context)
    {
        sample = Block{
            {std::make_shared<DataTypeString>(),    "host"},
            {std::make_shared<DataTypeUInt64>(),    "status"},
            {std::make_shared<DataTypeString>(),    "error"},
            {std::make_shared<DataTypeUInt64>(),    "num_hosts_remaining"},
            {std::make_shared<DataTypeUInt64>(),    "num_hosts_active"},
        };

        setTotalRowsApprox(num_hosts);
    }

    String getName() const override
    {
        return "DDLQueryStatusInputSream";
    }

    String getID() const override
    {
        return "DDLQueryStatusInputSream(" + node_path + ")";
    }

    Block readImpl() override
    {
        Block res;
        if (num_hosts_finished >= total_rows_approx)
            return res;

        auto zookeeper = context.getZooKeeper();
        size_t try_number = 0;

        while(res.rows() == 0)
        {
            if (is_cancelled)
                return res;

            if (num_hosts_finished != 0 || try_number != 0)
396
                std::this_thread::sleep_for(std::chrono::milliseconds(50 * std::min(20LU, try_number + 1)));
397

398
            /// TODO: add /lock node somewhere
399
            if (!zookeeper->exists(node_path))
400 401 402 403 404
            {
                throw Exception("Cannot provide query execution status. The query's node " + node_path
                                + " had been deleted by cleaner since it was finished (or its lifetime is expired)",
                                ErrorCodes::UNFINISHED);
            }
405

406 407
            Strings new_sucess_hosts = getNewAndUpdate(sucess_hosts_set, getChildrenAllowNoNode(zookeeper, node_path + "/sucess"));
            Strings new_failed_hosts = getNewAndUpdate(failed_hosts_set, getChildrenAllowNoNode(zookeeper, node_path + "/failed"));
408 409 410 411 412 413 414 415

            Strings new_hosts = new_sucess_hosts;
            new_hosts.insert(new_hosts.end(), new_failed_hosts.cbegin(), new_failed_hosts.cend());
            ++try_number;

            if (new_hosts.empty())
                continue;

416 417
            Strings cur_active_hosts = getChildrenAllowNoNode(zookeeper, node_path + "/active");

418 419 420 421 422 423 424 425
            res = sample.cloneEmpty();
            for (size_t i = 0; i < new_hosts.size(); ++i)
            {
                bool fail = i >= new_sucess_hosts.size();
                res.getByName("host").column->insert(new_hosts[i]);
                res.getByName("status").column->insert(static_cast<UInt64>(fail));
                res.getByName("error").column->insert(fail ? zookeeper->get(node_path + "/failed/" + new_hosts[i]) : String{});
                res.getByName("num_hosts_remaining").column->insert(total_rows_approx - (++num_hosts_finished));
426
                res.getByName("num_hosts_active").column->insert(cur_active_hosts.size());
427 428
            }
        }
429

430 431 432
        return res;
    }

433 434 435 436 437 438 439 440 441 442
    static Strings getChildrenAllowNoNode(const std::shared_ptr<zkutil::ZooKeeper> & zookeeper, const String & node_path)
    {
        Strings res;
        int code = zookeeper->tryGetChildren(node_path, res);
        if (code != ZOK && code != ZNONODE)
            throw zkutil::KeeperException(code, node_path);
        return res;
    }

    static Strings getNewAndUpdate(NameSet & prev, const Strings & cur_list)
443 444 445 446 447
    {
        Strings diff;
        for (const String & elem : cur_list)
        {
            if (!prev.count(elem))
448
            {
449
                diff.emplace_back(elem);
450 451
                prev.emplace(elem);
            }
452 453 454 455 456 457 458 459 460 461 462 463 464 465 466 467
        }

        return diff;
    }

    ~DDLQueryStatusInputSream() override = default;

    Block sample;

private:
    String node_path;
    Context & context;

    NameSet sucess_hosts_set;
    NameSet failed_hosts_set;
    size_t num_hosts_finished = 0;
468
};
469 470


471
BlockIO executeDDLQueryOnCluster(const ASTQueryWithOnCluster & query, Context & context)
472
{
473
    ClusterPtr cluster = context.getCluster(query.cluster);
474 475
    DDLWorker & ddl_worker = context.getDDLWorker();

476 477 478
    /// Do we really should use that database for each server?
    String query_str = query.getRewrittenQueryWithoutOnCluster(context.getCurrentDatabase());

479
    DDLLogEntry entry;
480
    entry.query = query_str;
481 482
    entry.initiator = ddl_worker.getHostName();

483
    Cluster::AddressesWithFailover shards = cluster->getShardsWithFailoverAddresses();
484 485
    for (const auto & shard : shards)
        for (const auto & addr : shard)
486
            entry.hosts.emplace_back(addr.toString());
487

488
    String node_path = ddl_worker.enqueueQuery(entry);
489 490

    BlockIO io;
491 492 493 494 495 496
    if (node_path.empty())
        return io;

    auto stream = std::make_shared<DDLQueryStatusInputSream>(node_path, context, entry.hosts.size());
    io.in_sample = stream->sample.cloneEmpty();
    io.in = std::move(stream);
497 498 499 500
    return io;
}


501
}