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

#include <Parsers/ASTAlterQuery.h>
4
#include <Parsers/ASTQueryWithOnCluster.h>
5 6 7
#include <Parsers/ParserQuery.h>
#include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
8

9 10 11 12 13
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <IO/Operators.h>
#include <IO/ReadBufferFromString.h>

14
#include <Storages/IStorage.h>
15
#include <DataStreams/OneBlockInputStream.h>
16

17
#include <Interpreters/executeQuery.h>
18
#include <Interpreters/Cluster.h>
19

20
#include <Common/getFQDNOrHostName.h>
21 22
#include <Common/setThreadName.h>
#include <Common/Stopwatch.h>
23 24 25 26 27 28 29

#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeArray.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnArray.h>
30 31

#include <zkutil/ZooKeeper.h>
32
#include <zkutil/Lock.h>
33 34
#include <Poco/Timestamp.h>

35 36
#include <experimental/optional>

37 38 39 40 41 42 43 44

namespace DB
{

namespace ErrorCodes
{
    extern const int UNKNOWN_ELEMENT_IN_CONFIG;
    extern const int INVALID_CONFIG_PARAMETER;
45
    extern const int UNKNOWN_FORMAT_VERSION;
46 47
    extern const int INCONSISTENT_TABLE_ACCROSS_SHARDS;
    extern const int INCONSISTENT_CLUSTER_DEFINITION;
48
    extern const int TIMEOUT_EXCEEDED;
49
    extern const int UNFINISHED;
50 51 52
}


53 54 55 56
const size_t DDLWorker::node_max_lifetime_seconds = 7 * 24 * 60 * 60; // week
const size_t DDLWorker::cleanup_min_period_seconds = 60; // minute


57 58 59 60
struct DDLLogEntry
{
    String query;
    Strings hosts;
61
    String initiator; // optional
62

63
    static constexpr int CURRENT_VERSION = '1';
64 65 66 67 68 69 70

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

71
            wb << "version: " << CURRENT_VERSION << "\n";
72
            wb << "query: " << escape << query << "\n";
73 74
            wb << "hosts: " << hosts << "\n";
            wb << "initiator: " << initiator << "\n";
75 76 77 78 79 80 81 82 83
        }

        return res;
    }

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

84 85 86
        int version;
        rb >> "version: " >> version >> "\n";

87 88 89
        if (version != CURRENT_VERSION)
            throw Exception("Unknown DDLLogEntry format version: " + version, ErrorCodes::UNKNOWN_FORMAT_VERSION);

90
        rb >> "query: " >> escape >> query >> "\n";
91
        rb >> "hosts: " >> hosts >> "\n";
92 93 94 95 96

        if (!rb.eof())
            rb >> "initiator: " >> initiator >> "\n";
        else
            initiator.clear();
97 98 99 100

        assertEOF(rb);
    }
};
101 102


103 104
using ShardAndHostNum = std::experimental::optional<std::pair<size_t, size_t>>;
static ShardAndHostNum tryGetShardAndHostNum(const Cluster::AddressesWithFailover & cluster, const String & host_name, UInt16 port)
105 106 107 108 109 110 111
{
    for (size_t shard_num = 0; shard_num < cluster.size(); ++shard_num)
    {
        for (size_t host_num = 0; host_num < cluster[shard_num].size(); ++host_num)
        {
            const Cluster::Address & address = cluster[shard_num][host_num];
            if (address.host_name == host_name && address.port == port)
112
                return std::make_pair(shard_num, host_num);
113 114 115
        }
    }

116
    return {};
117 118 119 120 121 122 123 124 125 126 127 128 129 130 131 132 133
}


static bool isSupportedAlterType(int type)
{
    static const std::unordered_set<int> supported_alter_types{
        ASTAlterQuery::ADD_COLUMN,
        ASTAlterQuery::DROP_COLUMN,
        ASTAlterQuery::MODIFY_COLUMN,
        ASTAlterQuery::MODIFY_PRIMARY_KEY,
        ASTAlterQuery::DROP_PARTITION
    };

    return supported_alter_types.count(type);
}


134
DDLWorker::DDLWorker(const std::string & zk_root_dir, Context & context_)
135
    : context(context_)
136
{
137 138 139
    queue_dir = zk_root_dir;
    if (queue_dir.back() == '/')
        queue_dir.resize(queue_dir.size() - 1);
140

141 142
    host_name = getFQDNOrHostName();
    port = context.getTCPPort();
143
    host_id = Cluster::Address::toString(host_name, port);
144

145 146
    event_queue_updated = std::make_shared<Poco::Event>();

147 148 149
    thread = std::thread(&DDLWorker::run, this);
}

150

151 152 153
DDLWorker::~DDLWorker()
{
    stop_flag = true;
154
    event_queue_updated->set();
155 156 157
    thread.join();
}

158

159 160
void DDLWorker::processTasks()
{
161
    LOG_DEBUG(log, "Processing tasks");
162

163
    Strings queue_nodes = zookeeper->getChildren(queue_dir, nullptr, event_queue_updated);
164
    if (queue_nodes.empty())
165 166
        return;

167
    bool server_startup = last_processed_node_name.empty();
168

169 170 171 172
    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);
173

174
    for (auto it = begin_node; it != queue_nodes.end(); ++it)
175
    {
176 177 178
        const String & node_name = *it;
        String node_path = queue_dir + "/" + node_name;
        String node_data;
179

180 181 182 183 184
        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;
        }
185 186 187 188

        DDLLogEntry node;
        node.parse(node_data);

189
        bool host_in_hostlist = std::find(node.hosts.cbegin(), node.hosts.cend(), host_id) != node.hosts.cend();
190
        bool already_processed = zookeeper->exists(node_path + "/finished/" + host_id);
191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206

        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 (...)
            {
207
                tryLogCurrentException(log, "An error occurred while processing node " + node_name + " (" + node.query + ")");
208 209 210
                throw;
            }
        }
211 212 213 214
        else
        {
            LOG_DEBUG(log, "Node " << node_name << " (" << node.query << ") will not be processed");
        }
215 216

        last_processed_node_name = node_name;
217
    }
218 219
}

220

221
static bool tryExecuteQuery(const String & query, Context & context, ExecutionStatus & status, Logger * log = nullptr)
222 223 224 225 226 227 228
{
    try
    {
        executeQuery(query, context);
    }
    catch (...)
    {
229 230
        status = ExecutionStatus::fromCurrentException();

231 232 233 234 235 236
        if (log)
            tryLogCurrentException(log, "Query " + query + " wasn't finished successfully");

        return false;
    }

237
    status = ExecutionStatus(0);
238 239 240 241 242 243 244
    if (log)
        LOG_DEBUG(log, "Executed query: " << query);

    return true;
}


245
void DDLWorker::processTask(const DDLLogEntry & node, const std::string & node_name)
246
{
247
    LOG_DEBUG(log, "Processing node " << node_name << " (" << node.query << ")");
248

249
    String node_path = queue_dir + "/" + node_name;
250 251
    createStatusDirs(node_path);

252
    bool should_not_execute = current_node == node_name && current_node_was_executed;
253

254 255 256 257
    if (!should_not_execute)
    {
        current_node = node_name;
        current_node_was_executed = false;
258

259
        zookeeper->create(node_path + "/active/" + host_id, "", zkutil::CreateMode::Ephemeral);
260

261 262 263 264 265 266 267 268 269
        try
        {
            ASTPtr query_ast;
            {
                ParserQuery parser_query;
                String description;
                IParser::Pos begin = &node.query.front();
                query_ast = parseQuery(parser_query, begin, begin + node.query.size(), description);
            }
270

271 272 273 274 275 276 277
            const ASTQueryWithOnCluster * query;
            if (!query_ast || !(query = dynamic_cast<const ASTQueryWithOnCluster *>(query_ast.get())))
                throw Exception("Recieved unsupported DDL query", ErrorCodes::NOT_IMPLEMENTED);

            String cluster_name = query->cluster;
            auto cluster = context.getCluster(cluster_name);

278 279
            auto shard_host_num = tryGetShardAndHostNum(cluster->getShardsWithFailoverAddresses(), host_name, port);
            if (!shard_host_num)
280 281 282 283
            {
                throw Exception("Cannot find own address (" + host_id + ") in cluster " + cluster_name + " configuration",
                                ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION);
            }
284

285 286 287
            size_t shard_num = shard_host_num->first;
            size_t host_num = shard_host_num->second;

288 289 290
            const auto & host_address = cluster->getShardsWithFailoverAddresses().at(shard_num).at(host_num);
            ASTPtr rewritten_ast = query->getRewrittenASTWithoutOnCluster(host_address.default_database);
            String rewritten_query = queryToString(rewritten_ast);
291

292
            LOG_DEBUG(log, "Executing query: " << rewritten_query);
293

294 295 296 297 298 299 300 301 302 303 304 305 306 307 308 309 310 311 312 313
            if (auto query_alter = dynamic_cast<const ASTAlterQuery *>(rewritten_ast.get()))
            {
                processTaskAlter(query_alter, rewritten_query, cluster, shard_num, node_path);
            }
            else
            {
                tryExecuteQuery(rewritten_query, context, current_node_execution_status, log);
            }
        }
        catch (const zkutil::KeeperException & e)
        {
            throw;
        }
        catch (...)
        {
            current_node_execution_status = ExecutionStatus::fromCurrentException("An error occured during query preparation");
        }

        /// We need to distinguish ZK errors occured before and after query executing
        current_node_was_executed = true;
314 315
    }

316 317 318 319 320
    /// Delete active flag and create finish flag
    zkutil::Ops ops;
    ops.emplace_back(std::make_unique<zkutil::Op::Remove>(node_path + "/active/" + host_id, -1));
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(node_path + "/finished/" + host_id,
        current_node_execution_status.serializeText(), zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
321

322 323 324 325
    int code = zookeeper->tryMultiWithRetries(ops);
    if (code != ZOK && code != ZNONODE)
        throw zkutil::KeeperException("Cannot commit executed node " + node_name, code);
}
326 327


328 329 330 331 332 333 334 335 336
void DDLWorker::processTaskAlter(
    const ASTAlterQuery * query_alter,
    const String & rewritten_query,
    const std::shared_ptr<Cluster> & cluster,
    ssize_t shard_num,
    const String & node_path)
{
    String database = query_alter->database.empty() ? context.getCurrentDatabase() : query_alter->database;
    StoragePtr storage = context.getTable(database, query_alter->table);
337

338 339
    bool execute_once_on_replica = storage->supportsReplication();
    bool execute_on_leader_replica = false;
340

341
    for (const auto & param : query_alter->parameters)
342
    {
343 344
        if (!isSupportedAlterType(param.type))
            throw Exception("Unsupported type of ALTER query", ErrorCodes::NOT_IMPLEMENTED);
345

346 347 348
        if (execute_once_on_replica)
            execute_on_leader_replica |= param.type == ASTAlterQuery::DROP_PARTITION;
    }
349

350 351
    const auto & shard_info = cluster->getShardsInfo().at(shard_num);
    bool config_is_replicated_shard = shard_info.hasInternalReplication();
352

353 354 355 356 357 358 359 360 361 362
    if (execute_once_on_replica && !config_is_replicated_shard)
    {
        throw Exception("Table " + query_alter->table + " is replicated, but shard #" + toString(shard_num + 1) +
            " isn't replicated according to its cluster definition", ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION);
    }
    else if (!execute_once_on_replica && config_is_replicated_shard)
    {
        throw Exception("Table " + query_alter->table + " isn't replicated, but shard #" + toString(shard_num + 1) +
            " replicated according to its cluster definition", ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION);
    }
363

364 365
    if (execute_once_on_replica)
    {
366 367 368 369
        /// The following code may perform ALTER twice if
        ///  current secver aquires lock, executes replicated alter,
        ///  losts zookeeper connection and doesn't have time to create /executed node, second server executes replicated alter again
        /// To avoid this problem alter() method of replicated tables should be changed and takes into account ddl query id tag.
370 371
        if (!context.getSettingsRef().distributed_ddl_allow_replicated_alter)
            throw Exception("Distributed DDL alters don't work properly yet", ErrorCodes::NOT_IMPLEMENTED);
372

373 374 375 376
        Strings replica_names;
        for (const auto & address : cluster->getShardsWithFailoverAddresses().at(shard_num))
            replica_names.emplace_back(address.toString());
        std::sort(replica_names.begin(), replica_names.end());
377

378 379 380
        String shard_dir_name;
        for (auto it = replica_names.begin(); it != replica_names.end(); ++it)
            shard_dir_name += *it + (std::next(it) != replica_names.end() ? "," : "");
381

382 383 384
        String shard_path = node_path + "/shards/" + shard_dir_name;
        String is_executed_path = shard_path + "/executed";
        zookeeper->createAncestors(shard_path + "/");
385

386 387 388 389
        bool alter_executed_by_replica = false;
        {
            auto zookeeper_holder = std::make_shared<zkutil::ZooKeeperHolder>();
            zookeeper_holder->initFromInstance(zookeeper);
390

391 392 393 394
            zkutil::Lock lock(zookeeper_holder, shard_path, "lock", host_id);
            std::mt19937 rng(std::hash<String>{}(host_id) + reinterpret_cast<intptr_t>(&rng));

            for (int num_tries = 0; num_tries < 10; ++num_tries)
395
            {
396 397 398 399 400
                if (zookeeper->exists(is_executed_path))
                {
                    alter_executed_by_replica = true;
                    break;
                }
401

402
                if (lock.tryLock())
403
                {
404
                    tryExecuteQuery(rewritten_query, context, current_node_execution_status, log);
405

406
                    if (execute_on_leader_replica && current_node_execution_status.code == ErrorCodes::NOT_IMPLEMENTED)
407
                    {
408
                        /// TODO: it is ok to recieve exception "host is not leader"
409 410
                    }

411 412 413 414
                    zookeeper->create(is_executed_path, host_id, zkutil::CreateMode::Persistent);
                    lock.unlock();
                    alter_executed_by_replica = true;
                    break;
415 416
                }

417
                std::this_thread::sleep_for(std::chrono::duration<double>(std::uniform_real_distribution<double>(0, 1)(rng)));
418 419
            }
        }
420 421 422

        if (!alter_executed_by_replica)
            current_node_execution_status = ExecutionStatus(ErrorCodes::NOT_IMPLEMENTED, "Cannot enqueue replicated DDL query");
423 424 425
    }
    else
    {
426
        tryExecuteQuery(rewritten_query, context, current_node_execution_status, log);
427 428 429 430
    }
}


431 432 433 434 435 436 437
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
438
    if (last_cleanup_time_seconds && current_time_seconds < last_cleanup_time_seconds + cleanup_min_period_seconds)
439 440 441 442
        return;

    last_cleanup_time_seconds = current_time_seconds;

443 444
    LOG_DEBUG(log, "Cleaning queue");

445 446 447 448
    String data;
    zkutil::Stat stat;
    DDLLogEntry node;

449
    Strings node_names_fetched = node_names_to_check ? Strings{} : zookeeper->getChildren(queue_dir);
450 451 452 453 454 455
    const Strings & node_names = (node_names_to_check) ? *node_names_to_check : node_names_fetched;

    for (const String & node_name : node_names)
    {
        try
        {
456
            String node_path = queue_dir + "/" + node_name;
457 458 459
            if (!zookeeper->tryGet(node_path, data, &stat))
                continue;

460
            /// TODO: Add shared lock to avoid rare race counditions.
461 462 463 464 465 466 467 468 469 470

            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;
            }

471
            Strings finished_nodes = zookeeper->getChildren(node_path + "/finished");
472 473
            node.parse(data);

474
            if (finished_nodes.size() >= node.hosts.size())
475 476 477 478 479 480 481 482 483 484 485 486
            {
                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");
        }
    }
}

487

488 489
/// Try to create unexisting "status" dirs for a node
void DDLWorker::createStatusDirs(const std::string & node_path)
490
{
491
    zkutil::Ops ops;
492
    auto acl = zookeeper->getDefaultACL();
493
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(node_path + "/active", "", acl, zkutil::CreateMode::Persistent));
494
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(node_path + "/finished", "", acl, zkutil::CreateMode::Persistent));
495

496 497 498 499
    int code = zookeeper->tryMulti(ops);
    if (code != ZOK && code != ZNODEEXISTS)
        throw zkutil::KeeperException(code);
}
500 501


502
String DDLWorker::enqueueQuery(DDLLogEntry & entry)
503
{
504
    if (entry.hosts.empty())
505
        return {};
506

507
    String query_path_prefix = queue_dir + "/query-";
508
    zookeeper->createAncestors(query_path_prefix);
509

510 511
    String node_path = zookeeper->create(query_path_prefix, entry.toString(), zkutil::CreateMode::PersistentSequential);
    createStatusDirs(node_path);
512 513

    return node_path;
514 515
}

516

517 518
void DDLWorker::run()
{
519
    setThreadName("DDLWorker");
520 521
    LOG_DEBUG(log, "Started DDLWorker thread");

522 523 524
    zookeeper = context.getZooKeeper();
    zookeeper->createAncestors(queue_dir + "/");

525 526 527 528
    while (!stop_flag)
    {
        try
        {
529
            processTasks();
530

531 532 533 534 535
            LOG_DEBUG(log, "Waiting watch");
            event_queue_updated->wait();

            if (stop_flag)
                break;
536 537

            cleanupQueue();
538
        }
539 540 541 542 543
        catch (zkutil::KeeperException &)
        {
            LOG_DEBUG(log, "Recovering ZooKeeper session");
            zookeeper = context.getZooKeeper();
        }
544 545 546
        catch (...)
        {
            tryLogCurrentException(log);
547
            throw;
548
        }
549 550 551
    }
}

552

553
class DDLQueryStatusInputSream : public IProfilingBlockInputStream
554
{
555
public:
556

557
    DDLQueryStatusInputSream(const String & zk_node_path, Context & context, size_t num_hosts)
558
    : node_path(zk_node_path), context(context), watch(CLOCK_MONOTONIC_COARSE)
559 560 561 562 563 564 565 566 567 568 569 570 571 572 573 574 575 576 577 578 579 580
    {
        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 + ")";
    }

581 582
    static constexpr size_t timeout_seconds = 120;

583 584 585 586 587 588 589 590 591 592 593 594 595 596
    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;

597 598
            auto elapsed_seconds = watch.elapsedSeconds();
            if (elapsed_seconds > timeout_seconds)
599
                throw Exception("Watching query is executing too long (" + toString(std::round(elapsed_seconds)) + " sec.)", ErrorCodes::TIMEOUT_EXCEEDED);
600

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

604
            /// TODO: add shared lock
605
            if (!zookeeper->exists(node_path))
606 607 608 609 610
            {
                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);
            }
611

612
            Strings new_hosts = getNewAndUpdate(finished_hosts_set, getChildrenAllowNoNode(zookeeper, node_path + "/finished"));
613 614 615 616
            ++try_number;
            if (new_hosts.empty())
                continue;

617 618
            Strings cur_active_hosts = getChildrenAllowNoNode(zookeeper, node_path + "/active");

619
            res = sample.cloneEmpty();
620
            for (const String & host : new_hosts)
621
            {
622 623 624 625 626 627 628 629 630 631
                ExecutionStatus status(1, "Cannot obtain error message");
                {
                    String status_data;
                    if (zookeeper->tryGet(node_path + "/finished/" + host, status_data))
                        status.deserializeText(status_data);
                }

                res.getByName("host").column->insert(host);
                res.getByName("status").column->insert(static_cast<UInt64>(status.code));
                res.getByName("error").column->insert(status.message);
632
                res.getByName("num_hosts_remaining").column->insert(total_rows_approx - (++num_hosts_finished));
633
                res.getByName("num_hosts_active").column->insert(cur_active_hosts.size());
634 635
            }
        }
636

637 638 639
        return res;
    }

640 641 642 643 644 645 646 647 648 649
    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)
650 651 652 653 654
    {
        Strings diff;
        for (const String & elem : cur_list)
        {
            if (!prev.count(elem))
655
            {
656
                diff.emplace_back(elem);
657 658
                prev.emplace(elem);
            }
659 660 661 662 663 664 665 666 667 668 669 670 671
        }

        return diff;
    }

    ~DDLQueryStatusInputSream() override = default;

    Block sample;

private:
    String node_path;
    Context & context;

672 673 674
    Stopwatch watch;

    NameSet finished_hosts_set;
675
    size_t num_hosts_finished = 0;
676
};
677 678


679
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, Context & context)
680
{
681 682 683 684 685 686
    const auto query = dynamic_cast<const ASTQueryWithOnCluster *>(query_ptr.get());
    if (!query)
    {
        throw Exception("Distributed execution is not supported for such DDL queries",
                        ErrorCodes::NOT_IMPLEMENTED);
    }
687

688 689 690 691 692 693 694 695 696 697 698 699
    auto query_alter = dynamic_cast<const ASTAlterQuery *>(query_ptr.get());
    if (query_alter)
    {
        for (const auto & param : query_alter->parameters)
        {
            if (!isSupportedAlterType(param.type))
                throw Exception("Unsupported type of ALTER query", ErrorCodes::NOT_IMPLEMENTED);
        }
    }

    ClusterPtr cluster = context.getCluster(query->cluster);
    DDLWorker & ddl_worker = context.getDDLWorker();
700

701
    DDLLogEntry entry;
702
    entry.query = queryToString(query_ptr);
703 704
    entry.initiator = ddl_worker.getHostName();

705
    Cluster::AddressesWithFailover shards = cluster->getShardsWithFailoverAddresses();
706
    for (const auto & shard : shards)
707
    {
708
        for (const auto & addr : shard)
709
            entry.hosts.emplace_back(addr.toString());
710
    }
711

712
    String node_path = ddl_worker.enqueueQuery(entry);
713 714

    BlockIO io;
715 716 717 718 719 720
    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);
721 722 723 724
    return io;
}


725
}