ClusterCopier.cpp 78.5 KB
Newer Older
1
#include "ClusterCopier.h"
2

N
Nikita Mikhaylov 已提交
3 4
#include "Internals.h"

5
#include <Common/ZooKeeper/ZooKeeper.h>
A
Alexey Milovidov 已提交
6
#include <Common/ZooKeeper/KeeperException.h>
7 8

namespace DB
A
Alexey Milovidov 已提交
9
{
10

11 12 13 14 15 16 17
namespace ErrorCodes
{
    extern const int NOT_IMPLEMENTED;
    extern const int LOGICAL_ERROR;
    extern const int UNFINISHED;
    extern const int BAD_ARGUMENTS;
}
A
Alexey Milovidov 已提交
18

19

N
Nikita Mikhaylov 已提交
20
void ClusterCopier::init()
21
{
N
Nikita Mikhaylov 已提交
22
    auto zookeeper = context.getZooKeeper();
23

N
Nikita Mikhaylov 已提交
24 25 26 27
    task_description_watch_callback = [this] (const Coordination::WatchResponse & response)
    {
        if (response.error != Coordination::ZOK)
            return;
N
Nikita Mikhaylov 已提交
28
        UInt64 version = ++task_description_version;
A
Alexey Milovidov 已提交
29
        LOG_DEBUG(log, "Task description should be updated, local version {}", version);
N
Nikita Mikhaylov 已提交
30
    };
31

N
Nikita Mikhaylov 已提交
32 33
    task_description_path = task_zookeeper_path + "/description";
    task_cluster = std::make_unique<TaskCluster>(task_zookeeper_path, working_database_name);
34

N
Nikita Mikhaylov 已提交
35 36
    reloadTaskDescription();
    task_cluster_initial_config = task_cluster_current_config;
37

N
Nikita Mikhaylov 已提交
38 39 40 41 42 43
    task_cluster->loadTasks(*task_cluster_initial_config);
    context.setClustersConfig(task_cluster_initial_config, task_cluster->clusters_prefix);

    /// Set up shards and their priority
    task_cluster->random_engine.seed(task_cluster->random_device());
    for (auto & task_table : task_cluster->table_tasks)
44
    {
N
Nikita Mikhaylov 已提交
45 46 47
        task_table.cluster_pull = context.getCluster(task_table.cluster_pull_name);
        task_table.cluster_push = context.getCluster(task_table.cluster_push_name);
        task_table.initShards(task_cluster->random_engine);
48 49
    }

A
Alexey Milovidov 已提交
50
    LOG_DEBUG(log, "Will process {} table tasks", task_cluster->table_tasks.size());
N
Nikita Mikhaylov 已提交
51 52 53 54 55

    /// Do not initialize tables, will make deferred initialization in process()

    zookeeper->createAncestors(getWorkersPathVersion() + "/");
    zookeeper->createAncestors(getWorkersPath() + "/");
56 57
}

N
Nikita Mikhaylov 已提交
58 59
template <typename T>
decltype(auto) ClusterCopier::retry(T && func, UInt64 max_tries)
60
{
N
Nikita Mikhaylov 已提交
61 62 63 64 65 66 67 68 69 70 71 72 73 74 75 76 77 78 79 80
    std::exception_ptr exception;

    for (UInt64 try_number = 1; try_number <= max_tries; ++try_number)
    {
        try
        {
            return func();
        }
        catch (...)
        {
            exception = std::current_exception();
            if (try_number < max_tries)
            {
                tryLogCurrentException(log, "Will retry");
                std::this_thread::sleep_for(default_sleep_time);
            }
        }
    }

    std::rethrow_exception(exception);
81 82 83
}


N
Nikita Mikhaylov 已提交
84
void ClusterCopier::discoverShardPartitions(const ConnectionTimeouts & timeouts, const TaskShardPtr & task_shard)
85
{
N
Nikita Mikhaylov 已提交
86
    TaskTable & task_table = task_shard->task_table;
87

A
Alexey Milovidov 已提交
88
    LOG_INFO(log, "Discover partitions of shard {}", task_shard->getDescription());
89

N
Nikita Mikhaylov 已提交
90 91 92 93
    auto get_partitions = [&] () { return getShardPartitions(timeouts, *task_shard); };
    auto existing_partitions_names = retry(get_partitions, 60);
    Strings filtered_partitions_names;
    Strings missing_partitions;
94

N
Nikita Mikhaylov 已提交
95 96
    /// Check that user specified correct partition names
    auto check_partition_format = [] (const DataTypePtr & type, const String & partition_text_quoted)
97
    {
N
Nikita Mikhaylov 已提交
98 99
        MutableColumnPtr column_dummy = type->createColumn();
        ReadBufferFromString rb(partition_text_quoted);
100

N
Nikita Mikhaylov 已提交
101 102 103 104 105 106 107 108 109
        try
        {
            type->deserializeAsTextQuoted(*column_dummy, rb, FormatSettings());
        }
        catch (Exception & e)
        {
            throw Exception("Partition " + partition_text_quoted + " has incorrect format. " + e.displayText(), ErrorCodes::BAD_ARGUMENTS);
        }
    };
110

N
Nikita Mikhaylov 已提交
111
    if (task_table.has_enabled_partitions)
112
    {
N
Nikita Mikhaylov 已提交
113 114 115 116 117
        /// Process partition in order specified by <enabled_partitions/>
        for (const String & partition_name : task_table.enabled_partitions)
        {
            /// Check that user specified correct partition names
            check_partition_format(task_shard->partition_key_column.type, partition_name);
118

N
Nikita Mikhaylov 已提交
119
            auto it = existing_partitions_names.find(partition_name);
120

N
Nikita Mikhaylov 已提交
121 122 123 124 125 126 127 128 129
            /// Do not process partition if it is not in enabled_partitions list
            if (it == existing_partitions_names.end())
            {
                missing_partitions.emplace_back(partition_name);
                continue;
            }

            filtered_partitions_names.emplace_back(*it);
        }
130

N
Nikita Mikhaylov 已提交
131 132 133 134
        for (const String & partition_name : existing_partitions_names)
        {
            if (!task_table.enabled_partitions_set.count(partition_name))
            {
A
Alexey Milovidov 已提交
135
                LOG_DEBUG(log, "Partition {} will not be processed, since it is not in enabled_partitions of {}", partition_name, task_table.table_id);
N
Nikita Mikhaylov 已提交
136 137 138 139 140 141 142
            }
        }
    }
    else
    {
        for (const String & partition_name : existing_partitions_names)
            filtered_partitions_names.emplace_back(partition_name);
143 144
    }

N
Nikita Mikhaylov 已提交
145 146
    for (const String & partition_name : filtered_partitions_names)
    {
N
Nikita Mikhaylov 已提交
147 148
        const size_t number_of_splits = task_table.number_of_splits;
        task_shard->partition_tasks.emplace(partition_name, ShardPartition(*task_shard, partition_name, number_of_splits));
N
Nikita Mikhaylov 已提交
149
        task_shard->checked_partitions.emplace(partition_name, true);
N
Nikita Mikhaylov 已提交
150 151 152 153 154 155 156 157 158

        auto shard_partition_it = task_shard->partition_tasks.find(partition_name);
        PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces;

        for (size_t piece_number = 0; piece_number < number_of_splits; ++piece_number)
        {
            bool res = checkPresentPartitionPiecesOnCurrentShard(timeouts, *task_shard, partition_name, piece_number);
            shard_partition_pieces.emplace_back(shard_partition_it->second, piece_number, res);
        }
N
Nikita Mikhaylov 已提交
159
    }
160

N
Nikita Mikhaylov 已提交
161 162 163 164 165
    if (!missing_partitions.empty())
    {
        std::stringstream ss;
        for (const String & missing_partition : missing_partitions)
            ss << " " << missing_partition;
166

A
Alexey Milovidov 已提交
167
        LOG_WARNING(log, "There are no {} partitions from enabled_partitions in shard {} :{}", missing_partitions.size(), task_shard->getDescription(), ss.str());
N
Nikita Mikhaylov 已提交
168
    }
169

A
Alexey Milovidov 已提交
170
    LOG_DEBUG(log, "Will copy {} partitions from shard {}", task_shard->partition_tasks.size(), task_shard->getDescription());
N
Nikita Mikhaylov 已提交
171
}
172

N
Nikita Mikhaylov 已提交
173
void ClusterCopier::discoverTablePartitions(const ConnectionTimeouts & timeouts, TaskTable & task_table, UInt64 num_threads)
174
{
N
Nikita Mikhaylov 已提交
175 176 177
    /// Fetch partitions list from a shard
    {
        ThreadPool thread_pool(num_threads ? num_threads : 2 * getNumberOfPhysicalCPUCores());
178

N
Nikita Mikhaylov 已提交
179 180
        for (const TaskShardPtr & task_shard : task_table.all_shards)
            thread_pool.scheduleOrThrowOnError([this, timeouts, task_shard]() { discoverShardPartitions(timeouts, task_shard); });
181

A
Alexey Milovidov 已提交
182
        LOG_DEBUG(log, "Waiting for {} setup jobs", thread_pool.active());
N
Nikita Mikhaylov 已提交
183 184 185
        thread_pool.wait();
    }
}
186

N
Nikita Mikhaylov 已提交
187
void ClusterCopier::uploadTaskDescription(const std::string & task_path, const std::string & task_file, const bool force)
188
{
N
Nikita Mikhaylov 已提交
189
    auto local_task_description_path = task_path + "/description";
190

N
Nikita Mikhaylov 已提交
191
    String task_config_str;
192
    {
N
Nikita Mikhaylov 已提交
193 194
        ReadBufferFromFile in(task_file);
        readStringUntilEOF(task_config_str, in);
195
    }
N
Nikita Mikhaylov 已提交
196 197
    if (task_config_str.empty())
        return;
198

N
Nikita Mikhaylov 已提交
199
    auto zookeeper = context.getZooKeeper();
200

N
Nikita Mikhaylov 已提交
201 202 203 204
    zookeeper->createAncestors(local_task_description_path);
    auto code = zookeeper->tryCreate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent);
    if (code && force)
        zookeeper->createOrUpdate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent);
205

A
Alexey Milovidov 已提交
206
    LOG_DEBUG(log, "Task description {} uploaded to {} with result {} ({})",
A
Alexey Milovidov 已提交
207
        ((code && !force) ? "not " : ""), local_task_description_path, code, zookeeper->error2string(code));
N
Nikita Mikhaylov 已提交
208
}
209

N
Nikita Mikhaylov 已提交
210 211 212 213
void ClusterCopier::reloadTaskDescription()
{
    auto zookeeper = context.getZooKeeper();
    task_description_watch_zookeeper = zookeeper;
214

N
Nikita Mikhaylov 已提交
215
    String task_config_str;
N
Nikita Mikhaylov 已提交
216
    Coordination::Stat stat{};
N
Nikita Mikhaylov 已提交
217
    int code;
218

N
Nikita Mikhaylov 已提交
219 220 221
    zookeeper->tryGetWatch(task_description_path, task_config_str, &stat, task_description_watch_callback, &code);
    if (code)
        throw Exception("Can't get description node " + task_description_path, ErrorCodes::BAD_ARGUMENTS);
222

A
Alexey Milovidov 已提交
223
    LOG_DEBUG(log, "Loading description, zxid={}", task_description_current_stat.czxid);
N
Nikita Mikhaylov 已提交
224
    auto config = getConfigurationFromXMLString(task_config_str);
225

N
Nikita Mikhaylov 已提交
226 227
    /// Setup settings
    task_cluster->reloadSettings(*config);
228
    context.setSettings(task_cluster->settings_common);
229

N
Nikita Mikhaylov 已提交
230
    task_cluster_current_config = config;
N
Nikita Mikhaylov 已提交
231
    task_description_current_stat = stat;
N
Nikita Mikhaylov 已提交
232
}
233

N
Nikita Mikhaylov 已提交
234
void ClusterCopier::updateConfigIfNeeded()
235
{
N
Nikita Mikhaylov 已提交
236 237 238
    UInt64 version_to_update = task_description_version;
    bool is_outdated_version = task_description_current_version != version_to_update;
    bool is_expired_session  = !task_description_watch_zookeeper || task_description_watch_zookeeper->expired();
239

N
Nikita Mikhaylov 已提交
240 241
    if (!is_outdated_version && !is_expired_session)
        return;
242

A
Alexey Milovidov 已提交
243
    LOG_DEBUG(log, "Updating task description");
N
Nikita Mikhaylov 已提交
244
    reloadTaskDescription();
245

N
Nikita Mikhaylov 已提交
246
    task_description_current_version = version_to_update;
N
Nikita Mikhaylov 已提交
247
}
248

N
Nikita Mikhaylov 已提交
249 250 251 252
void ClusterCopier::process(const ConnectionTimeouts & timeouts)
{
    for (TaskTable & task_table : task_cluster->table_tasks)
    {
A
Alexey Milovidov 已提交
253
        LOG_INFO(log, "Process table task {} with {} shards, {} of them are local ones", task_table.table_id, task_table.all_shards.size(), task_table.local_shards.size());
254

N
Nikita Mikhaylov 已提交
255 256
        if (task_table.all_shards.empty())
            continue;
257

N
Nikita Mikhaylov 已提交
258 259 260 261 262
        /// Discover partitions of each shard and total set of partitions
        if (!task_table.has_enabled_partitions)
        {
            /// If there are no specified enabled_partitions, we must discover them manually
            discoverTablePartitions(timeouts, task_table);
263

N
Nikita Mikhaylov 已提交
264 265 266 267 268 269 270 271 272
            /// After partitions of each shard are initialized, initialize cluster partitions
            for (const TaskShardPtr & task_shard : task_table.all_shards)
            {
                for (const auto & partition_elem : task_shard->partition_tasks)
                {
                    const String & partition_name = partition_elem.first;
                    task_table.cluster_partitions.emplace(partition_name, ClusterPartition{});
                }
            }
273

N
Nikita Mikhaylov 已提交
274 275 276
            for (auto & partition_elem : task_table.cluster_partitions)
            {
                const String & partition_name = partition_elem.first;
277

N
Nikita Mikhaylov 已提交
278 279
                for (const TaskShardPtr & task_shard : task_table.all_shards)
                    task_shard->checked_partitions.emplace(partition_name);
280

N
Nikita Mikhaylov 已提交
281 282 283 284 285 286 287
                task_table.ordered_partition_names.emplace_back(partition_name);
            }
        }
        else
        {
            /// If enabled_partitions are specified, assume that each shard has all partitions
            /// We will refine partition set of each shard in future
288

N
Nikita Mikhaylov 已提交
289 290 291 292 293 294
            for (const String & partition_name : task_table.enabled_partitions)
            {
                task_table.cluster_partitions.emplace(partition_name, ClusterPartition{});
                task_table.ordered_partition_names.emplace_back(partition_name);
            }
        }
295

N
Nikita Mikhaylov 已提交
296
        task_table.watch.restart();
297

N
Nikita Mikhaylov 已提交
298 299 300 301 302 303 304 305 306 307
        /// Retry table processing
        bool table_is_done = false;
        for (UInt64 num_table_tries = 0; num_table_tries < max_table_tries; ++num_table_tries)
        {
            if (tryProcessTable(timeouts, task_table))
            {
                table_is_done = true;
                break;
            }
        }
308

N
Nikita Mikhaylov 已提交
309 310 311
        /// Delete helping tables in both cases (whole table is done or not)
        dropHelpingTables(task_table);

N
Nikita Mikhaylov 已提交
312 313 314 315 316
        if (!table_is_done)
        {
            throw Exception("Too many tries to process table " + task_table.table_id + ". Abort remaining execution",
                            ErrorCodes::UNFINISHED);
        }
317
    }
N
Nikita Mikhaylov 已提交
318
}
319

N
Nikita Mikhaylov 已提交
320
/// Protected section
321

N
Nikita Mikhaylov 已提交
322 323 324 325 326 327 328

/*
 * Creates task worker node and checks maximum number of workers not to exceed the limit.
 * To achive this we have to check version of workers_version_path node and create current_worker_path
 * node atomically.
 * */

N
Nikita Mikhaylov 已提交
329 330 331 332
zkutil::EphemeralNodeHolder::Ptr ClusterCopier::createTaskWorkerNodeAndWaitIfNeed(
    const zkutil::ZooKeeperPtr & zookeeper,
    const String & description,
    bool unprioritized)
333
{
N
Nikita Mikhaylov 已提交
334 335
    std::chrono::milliseconds current_sleep_time = default_sleep_time;
    static constexpr std::chrono::milliseconds max_sleep_time(30000); // 30 sec
336

N
Nikita Mikhaylov 已提交
337 338
    if (unprioritized)
        std::this_thread::sleep_for(current_sleep_time);
339

N
Nikita Mikhaylov 已提交
340
    String workers_version_path = getWorkersPathVersion();
N
Nikita Mikhaylov 已提交
341 342
    String workers_path         = getWorkersPath();
    String current_worker_path  = getCurrentWorkerNodePath();
343

N
Nikita Mikhaylov 已提交
344
    UInt64 num_bad_version_errors = 0;
345

N
Nikita Mikhaylov 已提交
346 347 348
    while (true)
    {
        updateConfigIfNeeded();
349

N
Nikita Mikhaylov 已提交
350 351 352 353
        Coordination::Stat stat;
        zookeeper->get(workers_version_path, &stat);
        auto version = stat.version;
        zookeeper->get(workers_path, &stat);
354

N
Nikita Mikhaylov 已提交
355 356
        if (static_cast<UInt64>(stat.numChildren) >= task_cluster->max_workers)
        {
A
Alexey Milovidov 已提交
357
            LOG_DEBUG(log, "Too many workers ({}, maximum {}). Postpone processing {}", stat.numChildren, task_cluster->max_workers, description);
358

N
Nikita Mikhaylov 已提交
359 360
            if (unprioritized)
                current_sleep_time = std::min(max_sleep_time, current_sleep_time + default_sleep_time);
361

N
Nikita Mikhaylov 已提交
362 363 364 365 366 367 368 369 370 371
            std::this_thread::sleep_for(current_sleep_time);
            num_bad_version_errors = 0;
        }
        else
        {
            Coordination::Requests ops;
            ops.emplace_back(zkutil::makeSetRequest(workers_version_path, description, version));
            ops.emplace_back(zkutil::makeCreateRequest(current_worker_path, description, zkutil::CreateMode::Ephemeral));
            Coordination::Responses responses;
            auto code = zookeeper->tryMulti(ops, responses);
372

N
Nikita Mikhaylov 已提交
373 374
            if (code == Coordination::ZOK || code == Coordination::ZNODEEXISTS)
                return std::make_shared<zkutil::EphemeralNodeHolder>(current_worker_path, *zookeeper, false, false, description);
375

N
Nikita Mikhaylov 已提交
376 377 378
            if (code == Coordination::ZBADVERSION)
            {
                ++num_bad_version_errors;
379

N
Nikita Mikhaylov 已提交
380 381 382
                /// Try to make fast retries
                if (num_bad_version_errors > 3)
                {
A
Alexey Milovidov 已提交
383
                    LOG_DEBUG(log, "A concurrent worker has just been added, will check free worker slots again");
N
Nikita Mikhaylov 已提交
384 385 386 387 388 389 390 391 392
                    std::chrono::milliseconds random_sleep_time(std::uniform_int_distribution<int>(1, 1000)(task_cluster->random_engine));
                    std::this_thread::sleep_for(random_sleep_time);
                    num_bad_version_errors = 0;
                }
            }
            else
                throw Coordination::Exception(code);
        }
    }
393 394
}

N
Nikita Mikhaylov 已提交
395

N
Nikita Mikhaylov 已提交
396 397 398
bool ClusterCopier::checkPartitionPieceIsClean(
        const zkutil::ZooKeeperPtr & zookeeper,
        const CleanStateClock & clean_state_clock,
N
Nikita Mikhaylov 已提交
399
        const String & task_status_path)
400
{
N
Nikita Mikhaylov 已提交
401
    LogicalClock task_start_clock;
N
Nikita Mikhaylov 已提交
402

N
Nikita Mikhaylov 已提交
403 404 405
    Coordination::Stat stat{};
    if (zookeeper->exists(task_status_path, &stat))
        task_start_clock = LogicalClock(stat.mzxid);
N
Nikita Mikhaylov 已提交
406

N
Nikita Mikhaylov 已提交
407
    return clean_state_clock.is_clean() && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock);
N
Nikita Mikhaylov 已提交
408 409
}

N
Nikita Mikhaylov 已提交
410 411

bool ClusterCopier::checkAllPiecesInPartitionAreDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition)
N
Nikita Mikhaylov 已提交
412 413
{
    bool answer = true;
N
Nikita Mikhaylov 已提交
414 415 416 417
    for (size_t piece_number = 0; piece_number < task_table.number_of_splits; ++piece_number)
    {
        bool piece_is_done = checkPartitionPieceIsDone(task_table, partition_name, piece_number, shards_with_partition);
        if (!piece_is_done)
A
Alexey Milovidov 已提交
418
            LOG_DEBUG(log, "Partition {} piece {} is not already done.", partition_name, piece_number);
N
Nikita Mikhaylov 已提交
419 420 421
        answer &= piece_is_done;
    }

N
Nikita Mikhaylov 已提交
422 423 424 425 426 427 428 429 430 431 432 433
    return answer;
}


/* The same as function above
 * Assume that we don't know on which shards do we have partition certain piece.
 * We'll check them all (I mean shards that contain the whole partition)
 * And shards that don't have certain piece MUST mark that piece is_done true.
 * */
bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, const String & partition_name,
                               size_t piece_number, const TasksShard & shards_with_partition)
{
A
Alexey Milovidov 已提交
434
    LOG_DEBUG(log, "Check that all shards processed partition {} piece {} successfully", partition_name, piece_number);
435

N
Nikita Mikhaylov 已提交
436
    auto zookeeper = context.getZooKeeper();
437

N
Nikita Mikhaylov 已提交
438 439
    /// Collect all shards that contain partition piece number piece_number.
    Strings piece_status_paths;
A
Alexey Milovidov 已提交
440
    for (const auto & shard : shards_with_partition)
N
Nikita Mikhaylov 已提交
441 442
    {
        ShardPartition & task_shard_partition = shard->partition_tasks.find(partition_name)->second;
N
Nikita Mikhaylov 已提交
443 444
        ShardPartitionPiece & shard_partition_piece = task_shard_partition.pieces[piece_number];
        piece_status_paths.emplace_back(shard_partition_piece.getShardStatusPath());
N
Nikita Mikhaylov 已提交
445
    }
446

N
Nikita Mikhaylov 已提交
447
    std::vector<int64_t> zxid1, zxid2;
448

N
Nikita Mikhaylov 已提交
449 450 451
    try
    {
        std::vector<zkutil::ZooKeeper::FutureGet> get_futures;
N
Nikita Mikhaylov 已提交
452
        for (const String & path : piece_status_paths)
N
Nikita Mikhaylov 已提交
453
            get_futures.emplace_back(zookeeper->asyncGet(path));
454

N
Nikita Mikhaylov 已提交
455 456 457 458
        // Check that state is Finished and remember zxid
        for (auto & future : get_futures)
        {
            auto res = future.get();
459

N
Nikita Mikhaylov 已提交
460 461 462
            TaskStateWithOwner status = TaskStateWithOwner::fromString(res.data);
            if (status.state != TaskState::Finished)
            {
A
Alexey Milovidov 已提交
463
                LOG_INFO(log, "The task {} is being rewritten by {}. Partition piece will be rechecked", res.data, status.owner);
N
Nikita Mikhaylov 已提交
464 465
                return false;
            }
466

N
Nikita Mikhaylov 已提交
467 468
            zxid1.push_back(res.stat.pzxid);
        }
469

N
Nikita Mikhaylov 已提交
470 471 472 473 474 475 476 477
        const String piece_is_dirty_flag_path = task_table.getCertainPartitionPieceIsDirtyPath(partition_name, piece_number);
        const String piece_is_dirty_cleaned_path = task_table.getCertainPartitionPieceIsCleanedPath(partition_name, piece_number);
        const String piece_task_status_path = task_table.getCertainPartitionPieceTaskStatusPath(partition_name, piece_number);

        CleanStateClock clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path);

        const bool is_clean = checkPartitionPieceIsClean(zookeeper, clean_state_clock, piece_task_status_path);

N
Nikita Mikhaylov 已提交
478

N
Nikita Mikhaylov 已提交
479
        if (!is_clean)
N
Nikita Mikhaylov 已提交
480
        {
A
Alexey Milovidov 已提交
481
            LOG_INFO(log, "Partition {} become dirty", partition_name);
N
Nikita Mikhaylov 已提交
482
            return false;
N
Nikita Mikhaylov 已提交
483
        }
484

N
Nikita Mikhaylov 已提交
485
        get_futures.clear();
N
Nikita Mikhaylov 已提交
486
        for (const String & path : piece_status_paths)
N
Nikita Mikhaylov 已提交
487
            get_futures.emplace_back(zookeeper->asyncGet(path));
488

N
Nikita Mikhaylov 已提交
489 490 491 492 493 494 495 496 497
        // Remember zxid of states again
        for (auto & future : get_futures)
        {
            auto res = future.get();
            zxid2.push_back(res.stat.pzxid);
        }
    }
    catch (const Coordination::Exception & e)
    {
A
Alexey Milovidov 已提交
498
        LOG_INFO(log, "A ZooKeeper error occurred while checking partition {} piece number {}. Will recheck the partition. Error: {}", partition_name, toString(piece_number), e.displayText());
N
Nikita Mikhaylov 已提交
499 500
        return false;
    }
501

N
Nikita Mikhaylov 已提交
502
    // If all task is finished and zxid is not changed then partition could not become dirty again
N
Nikita Mikhaylov 已提交
503
    for (UInt64 shard_num = 0; shard_num < piece_status_paths.size(); ++shard_num)
N
Nikita Mikhaylov 已提交
504 505 506
    {
        if (zxid1[shard_num] != zxid2[shard_num])
        {
A
Alexey Milovidov 已提交
507
            LOG_INFO(log, "The task {} is being modified now. Partition piece will be rechecked", piece_status_paths[shard_num]);
N
Nikita Mikhaylov 已提交
508 509 510
            return false;
        }
    }
511

A
Alexey Milovidov 已提交
512
    LOG_INFO(log, "Partition {} piece number {} is copied successfully", partition_name, toString(piece_number));
N
Nikita Mikhaylov 已提交
513
    return true;
514 515
}

N
Nikita Mikhaylov 已提交
516

N
Nikita Mikhaylov 已提交
517
TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & task_table, const String & partition_name)
N
Nikita Mikhaylov 已提交
518
{
N
Nikita Mikhaylov 已提交
519 520 521 522 523 524 525
    bool inject_fault = false;
    if (move_fault_probability > 0)
    {
        double value = std::uniform_real_distribution<>(0, 1)(task_table.task_cluster.random_engine);
        inject_fault = value < move_fault_probability;
    }

A
Alexey Milovidov 已提交
526
    LOG_DEBUG(log, "Try to move  {} to destionation table", partition_name);
N
Nikita Mikhaylov 已提交
527 528 529 530 531 532 533 534 535 536 537 538 539 540 541 542 543

    auto zookeeper = context.getZooKeeper();

    const auto current_partition_attach_is_active = task_table.getPartitionAttachIsActivePath(partition_name);
    const auto current_partition_attach_is_done   = task_table.getPartitionAttachIsDonePath(partition_name);

    /// Create ephemeral node to mark that we are active and process the partition
    zookeeper->createAncestors(current_partition_attach_is_active);
    zkutil::EphemeralNodeHolderPtr partition_attach_node_holder;
    try
    {
        partition_attach_node_holder = zkutil::EphemeralNodeHolder::create(current_partition_attach_is_active, *zookeeper, host_id);
    }
    catch (const Coordination::Exception & e)
    {
        if (e.code == Coordination::ZNODEEXISTS)
        {
A
Alexey Milovidov 已提交
544
            LOG_DEBUG(log, "Someone is already moving pieces {}", current_partition_attach_is_active);
N
Nikita Mikhaylov 已提交
545
            return TaskStatus::Active;
N
Nikita Mikhaylov 已提交
546 547 548 549 550 551 552 553 554 555 556 557 558 559 560
        }

        throw;
    }


    /// Exit if task has been already processed;
    /// create blocking node to signal cleaning up if it is abandoned
    {
        String status_data;
        if (zookeeper->tryGet(current_partition_attach_is_done, status_data))
        {
            TaskStateWithOwner status = TaskStateWithOwner::fromString(status_data);
            if (status.state == TaskState::Finished)
            {
A
Alexey Milovidov 已提交
561
                LOG_DEBUG(log, "All pieces for partition from this task {} has been successfully moved to destination table by {}", current_partition_attach_is_active, status.owner);
N
Nikita Mikhaylov 已提交
562
                return TaskStatus::Finished;
N
Nikita Mikhaylov 已提交
563 564 565 566
            }

            /// Task is abandoned, because previously we created ephemeral node, possibly in other copier's process.
            /// Initialize DROP PARTITION
A
Alexey Milovidov 已提交
567
            LOG_DEBUG(log, "Moving piece for partition {} has not been successfully finished by {}. Will try to move by myself.", current_partition_attach_is_active, status.owner);
N
better  
Nikita Mikhaylov 已提交
568 569 570

            /// Remove is_done marker.
            zookeeper->remove(current_partition_attach_is_done);
N
Nikita Mikhaylov 已提交
571 572 573 574 575 576 577 578 579 580
        }
    }


    /// Try start processing, create node about it
    {
        String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id);
        zookeeper->create(current_partition_attach_is_done, start_state, zkutil::CreateMode::Persistent);
    }

N
Nikita Mikhaylov 已提交
581
    /// Move partition to original destination table.
N
Nikita Mikhaylov 已提交
582 583
    for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number)
    {
A
Alexey Milovidov 已提交
584
        LOG_DEBUG(log, "Trying to move partition {} piece {} to original table", partition_name, toString(current_piece_number));
N
Nikita Mikhaylov 已提交
585

N
Nikita Mikhaylov 已提交
586 587
        ASTPtr query_alter_ast;
        String query_alter_ast_string;
N
Nikita Mikhaylov 已提交
588

N
Nikita Mikhaylov 已提交
589 590 591 592
        DatabaseAndTableName original_table = task_table.table_push;
        DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first,
                                                                  original_table.second + "_piece_" +
                                                                  toString(current_piece_number));
N
Nikita Mikhaylov 已提交
593

N
nikitamikhaylov 已提交
594 595 596 597 598 599 600
        Settings settings_push = task_cluster->settings_push;

        /// It is important, ALTER ATTACH PARTITION must be done synchronously
        /// And we will execute this ALTER query on each replica of a shard.
        /// It is correct, because this query is idempotent.
        settings_push.replication_alter_partitions_sync = 2;

N
Nikita Mikhaylov 已提交
601 602
        query_alter_ast_string += " ALTER TABLE " + getQuotedTable(original_table) +
                                  " ATTACH PARTITION " + partition_name +
N
nikitamikhaylov 已提交
603
                                  " FROM " + getQuotedTable(helping_table);
N
Nikita Mikhaylov 已提交
604

A
Alexey Milovidov 已提交
605
        LOG_DEBUG(log, "Executing ALTER query: {}", query_alter_ast_string);
N
Nikita Mikhaylov 已提交
606

N
Nikita Mikhaylov 已提交
607 608
        try
        {
N
Nikita Mikhaylov 已提交
609
            size_t num_nodes = executeQueryOnCluster(
N
Nikita Mikhaylov 已提交
610 611 612
                    task_table.cluster_push,
                    query_alter_ast_string,
                    nullptr,
N
nikitamikhaylov 已提交
613
                    &settings_push,
N
Nikita Mikhaylov 已提交
614 615
                    PoolMode::GET_MANY,
                    ClusterExecutionMode::ON_EACH_NODE);
N
Nikita Mikhaylov 已提交
616

A
Alexey Milovidov 已提交
617
            LOG_INFO(log, "Number of nodes that executed ALTER query successfully : {}", toString(num_nodes));
N
Nikita Mikhaylov 已提交
618 619 620
        }
        catch (...)
        {
A
Alexey Milovidov 已提交
621
            LOG_DEBUG(log, "Error while moving partition {} piece {} to original table", partition_name, toString(current_piece_number));
N
Nikita Mikhaylov 已提交
622 623
            throw;
        }
N
Nikita Mikhaylov 已提交
624

N
Nikita Mikhaylov 已提交
625 626
        if (inject_fault)
            throw Exception("Copy fault injection is activated", ErrorCodes::UNFINISHED);
N
Nikita Mikhaylov 已提交
627

N
Nikita Mikhaylov 已提交
628 629 630
        try
        {
            String query_deduplicate_ast_string;
N
Nikita Mikhaylov 已提交
631 632
            if (!task_table.isReplicatedTable())
            {
N
Nikita Mikhaylov 已提交
633 634 635
                query_deduplicate_ast_string += " OPTIMIZE TABLE " + getQuotedTable(original_table) +
                                                " PARTITION " + partition_name + " DEDUPLICATE;";

A
Alexey Milovidov 已提交
636
                LOG_DEBUG(log, "Executing OPTIMIZE DEDUPLICATE query: {}", query_alter_ast_string);
N
Nikita Mikhaylov 已提交
637 638 639 640 641 642 643 644

                UInt64 num_nodes = executeQueryOnCluster(
                        task_table.cluster_push,
                        query_deduplicate_ast_string,
                        nullptr,
                        &task_cluster->settings_push,
                        PoolMode::GET_MANY);

A
Alexey Milovidov 已提交
645
                LOG_INFO(log, "Number of shard that executed OPTIMIZE DEDUPLICATE query successfully : {}", toString(num_nodes));
N
Nikita Mikhaylov 已提交
646 647
            }
        }
N
Nikita Mikhaylov 已提交
648 649
        catch (...)
        {
A
Alexey Milovidov 已提交
650
            LOG_DEBUG(log, "Error while executing OPTIMIZE DEDUPLICATE partition {}in the original table", partition_name);
N
Nikita Mikhaylov 已提交
651 652
            throw;
        }
N
Nikita Mikhaylov 已提交
653 654 655 656 657 658 659 660
    }

    /// Create node to signal that we finished moving
    {
        String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id);
        zookeeper->set(current_partition_attach_is_done, state_finished, 0);
    }

N
Nikita Mikhaylov 已提交
661
    return TaskStatus::Finished;
N
Nikita Mikhaylov 已提交
662 663
}

N
Nikita Mikhaylov 已提交
664
/// Removes MATERIALIZED and ALIAS columns from create table query
N
Nikita Mikhaylov 已提交
665
ASTPtr ClusterCopier::removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast)
666
{
N
Nikita Mikhaylov 已提交
667 668
    const ASTs & column_asts = query_ast->as<ASTCreateQuery &>().columns_list->columns->children;
    auto new_columns = std::make_shared<ASTExpressionList>();
669

N
Nikita Mikhaylov 已提交
670 671 672
    for (const ASTPtr & column_ast : column_asts)
    {
        const auto & column = column_ast->as<ASTColumnDeclaration &>();
673

N
Nikita Mikhaylov 已提交
674 675 676 677 678 679
        if (!column.default_specifier.empty())
        {
            ColumnDefaultKind kind = columnDefaultKindFromString(column.default_specifier);
            if (kind == ColumnDefaultKind::Materialized || kind == ColumnDefaultKind::Alias)
                continue;
        }
680

N
Nikita Mikhaylov 已提交
681 682
        new_columns->children.emplace_back(column_ast->clone());
    }
683

N
Nikita Mikhaylov 已提交
684 685
    ASTPtr new_query_ast = query_ast->clone();
    auto & new_query = new_query_ast->as<ASTCreateQuery &>();
686

N
Nikita Mikhaylov 已提交
687 688
    auto new_columns_list = std::make_shared<ASTColumns>();
    new_columns_list->set(new_columns_list->columns, new_columns);
A
Alexey Milovidov 已提交
689
    if (const auto * indices = query_ast->as<ASTCreateQuery>()->columns_list->indices)
N
Nikita Mikhaylov 已提交
690
        new_columns_list->set(new_columns_list->indices, indices->clone());
691

N
Nikita Mikhaylov 已提交
692 693 694
    new_query.replace(new_query.columns_list, new_columns_list);

    return new_query_ast;
695 696
}

A
Alexey Milovidov 已提交
697
/// Replaces ENGINE and table name in a create query
698 699 700
std::shared_ptr<ASTCreateQuery> rewriteCreateQueryStorage(const ASTPtr & create_query_ast,
                                                          const DatabaseAndTableName & new_table,
                                                          const ASTPtr & new_storage_ast)
701
{
N
Nikita Mikhaylov 已提交
702 703
    const auto & create = create_query_ast->as<ASTCreateQuery &>();
    auto res = std::make_shared<ASTCreateQuery>(create);
704

N
Nikita Mikhaylov 已提交
705 706
    if (create.storage == nullptr || new_storage_ast == nullptr)
        throw Exception("Storage is not specified", ErrorCodes::LOGICAL_ERROR);
707

N
Nikita Mikhaylov 已提交
708 709
    res->database = new_table.first;
    res->table = new_table.second;
710

N
Nikita Mikhaylov 已提交
711 712 713
    res->children.clear();
    res->set(res->columns_list, create.columns_list->clone());
    res->set(res->storage, new_storage_ast->clone());
714

N
Nikita Mikhaylov 已提交
715
    return res;
716 717
}

718

N
Nikita Mikhaylov 已提交
719 720 721 722 723
bool ClusterCopier::tryDropPartitionPiece(
        ShardPartition & task_partition,
        const size_t current_piece_number,
        const zkutil::ZooKeeperPtr & zookeeper,
        const CleanStateClock & clean_state_clock)
724
{
N
Nikita Mikhaylov 已提交
725 726
    if (is_safe_mode)
        throw Exception("DROP PARTITION is prohibited in safe mode", ErrorCodes::NOT_IMPLEMENTED);
727

N
Nikita Mikhaylov 已提交
728
    TaskTable & task_table = task_partition.task_shard.task_table;
N
Nikita Mikhaylov 已提交
729
    ShardPartitionPiece & partition_piece = task_partition.pieces[current_piece_number];
730

N
Nikita Mikhaylov 已提交
731 732 733 734 735
    const String current_shards_path                  = partition_piece.getPartitionPieceShardsPath();
    const String current_partition_active_workers_dir = partition_piece.getPartitionPieceActiveWorkersPath();
    const String is_dirty_flag_path                   = partition_piece.getPartitionPieceIsDirtyPath();
    const String dirty_cleaner_path                   = partition_piece.getPartitionPieceCleanerPath();
    const String is_dirty_cleaned_path                = partition_piece.getPartitionPieceIsCleanedPath();
736

N
Nikita Mikhaylov 已提交
737 738
    zkutil::EphemeralNodeHolder::Ptr cleaner_holder;
    try
739
    {
N
Nikita Mikhaylov 已提交
740
        cleaner_holder = zkutil::EphemeralNodeHolder::create(dirty_cleaner_path, *zookeeper, host_id);
741
    }
N
Nikita Mikhaylov 已提交
742
    catch (const Coordination::Exception & e)
743
    {
N
Nikita Mikhaylov 已提交
744 745
        if (e.code == Coordination::ZNODEEXISTS)
        {
A
Alexey Milovidov 已提交
746
            LOG_DEBUG(log, "Partition {} piece {} is cleaning now by somebody, sleep", task_partition.name, toString(current_piece_number));
N
Nikita Mikhaylov 已提交
747 748 749
            std::this_thread::sleep_for(default_sleep_time);
            return false;
        }
750

N
Nikita Mikhaylov 已提交
751
        throw;
752
    }
753

N
Nikita Mikhaylov 已提交
754
    Coordination::Stat stat{};
N
Nikita Mikhaylov 已提交
755
    if (zookeeper->exists(current_partition_active_workers_dir, &stat))
756
    {
N
Nikita Mikhaylov 已提交
757
        if (stat.numChildren != 0)
758
        {
A
Alexey Milovidov 已提交
759
            LOG_DEBUG(log, "Partition {} contains {} active workers while trying to drop it. Going to sleep.", task_partition.name, stat.numChildren);
N
Nikita Mikhaylov 已提交
760 761
            std::this_thread::sleep_for(default_sleep_time);
            return false;
762 763 764
        }
        else
        {
N
Nikita Mikhaylov 已提交
765
            zookeeper->remove(current_partition_active_workers_dir);
766
        }
767
    }
768

N
Nikita Mikhaylov 已提交
769 770 771 772 773 774 775 776 777 778
    {
        zkutil::EphemeralNodeHolder::Ptr active_workers_lock;
        try
        {
            active_workers_lock = zkutil::EphemeralNodeHolder::create(current_partition_active_workers_dir, *zookeeper, host_id);
        }
        catch (const Coordination::Exception & e)
        {
            if (e.code == Coordination::ZNODEEXISTS)
            {
A
Alexey Milovidov 已提交
779
                LOG_DEBUG(log, "Partition {} is being filled now by somebody, sleep", task_partition.name);
N
Nikita Mikhaylov 已提交
780 781
                return false;
            }
782

N
Nikita Mikhaylov 已提交
783 784
            throw;
        }
785

N
Nikita Mikhaylov 已提交
786 787
        // Lock the dirty flag
        zookeeper->set(is_dirty_flag_path, host_id, clean_state_clock.discovery_version.value());
N
Nikita Mikhaylov 已提交
788
        zookeeper->tryRemove(partition_piece.getPartitionPieceCleanStartPath());
N
Nikita Mikhaylov 已提交
789
        CleanStateClock my_clock(zookeeper, is_dirty_flag_path, is_dirty_cleaned_path);
790

N
Nikita Mikhaylov 已提交
791
        /// Remove all status nodes
792
        {
N
Nikita Mikhaylov 已提交
793 794 795 796 797 798
            Strings children;
            if (zookeeper->tryGetChildren(current_shards_path, children) == Coordination::ZOK)
                for (const auto & child : children)
                {
                    zookeeper->removeRecursive(current_shards_path + "/" + child);
                }
799 800
        }

N
Nikita Mikhaylov 已提交
801 802 803 804 805

        DatabaseAndTableName original_table = task_table.table_push;
        DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number));

        String query = "ALTER TABLE " + getQuotedTable(helping_table);
N
Nikita Mikhaylov 已提交
806
        query += " DROP PARTITION " + task_partition.name + "";
807

N
Nikita Mikhaylov 已提交
808 809
        /// TODO: use this statement after servers will be updated up to 1.1.54310
        // query += " DROP PARTITION ID '" + task_partition.name + "'";
810

N
Nikita Mikhaylov 已提交
811 812
        ClusterPtr & cluster_push = task_table.cluster_push;
        Settings settings_push = task_cluster->settings_push;
813

N
Nikita Mikhaylov 已提交
814 815
        /// It is important, DROP PARTITION must be done synchronously
        settings_push.replication_alter_partitions_sync = 2;
816

A
Alexey Milovidov 已提交
817
        LOG_DEBUG(log, "Execute distributed DROP PARTITION: {}", query);
N
better  
Nikita Mikhaylov 已提交
818
        /// We have to drop partition_piece on each replica
N
Nikita Mikhaylov 已提交
819
        size_t num_shards = executeQueryOnCluster(
N
Nikita Mikhaylov 已提交
820 821 822
                cluster_push, query,
                nullptr,
                &settings_push,
N
better  
Nikita Mikhaylov 已提交
823 824
                PoolMode::GET_MANY,
                ClusterExecutionMode::ON_EACH_NODE);
825

A
Alexey Milovidov 已提交
826
        LOG_INFO(log, "DROP PARTITION was successfully executed on {} nodes of a cluster.", num_shards);
827

N
Nikita Mikhaylov 已提交
828 829 830 831 832
        /// Update the locking node
        if (!my_clock.is_stale())
        {
            zookeeper->set(is_dirty_flag_path, host_id, my_clock.discovery_version.value());
            if (my_clock.clean_state_version)
N
Nikita Mikhaylov 已提交
833
                zookeeper->set(is_dirty_cleaned_path, host_id, my_clock.clean_state_version.value());
N
Nikita Mikhaylov 已提交
834
            else
N
Nikita Mikhaylov 已提交
835
                zookeeper->create(is_dirty_cleaned_path, host_id, zkutil::CreateMode::Persistent);
N
Nikita Mikhaylov 已提交
836 837
        }
        else
838
        {
A
Alexey Milovidov 已提交
839
            LOG_DEBUG(log, "Clean state is altered when dropping the partition, cowardly bailing");
N
Nikita Mikhaylov 已提交
840 841 842
            /// clean state is stale
            return false;
        }
843

A
Alexey Milovidov 已提交
844
        LOG_INFO(log, "Partition {} piece {} was dropped on cluster {}", task_partition.name, toString(current_piece_number), task_table.cluster_push_name);
N
Nikita Mikhaylov 已提交
845 846 847
        if (zookeeper->tryCreate(current_shards_path, host_id, zkutil::CreateMode::Persistent) == Coordination::ZNODEEXISTS)
            zookeeper->set(current_shards_path, host_id);
    }
848

A
Alexey Milovidov 已提交
849
    LOG_INFO(log, "Partition {} piece {} is safe for work now.", task_partition.name, toString(current_piece_number));
N
Nikita Mikhaylov 已提交
850 851
    return true;
}
852

N
Nikita Mikhaylov 已提交
853
bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table)
854
{
N
Nikita Mikhaylov 已提交
855 856
    /// An heuristic: if previous shard is already done, then check next one without sleeps due to max_workers constraint
    bool previous_shard_is_instantly_finished = false;
857

N
Nikita Mikhaylov 已提交
858 859 860 861 862
    /// Process each partition that is present in cluster
    for (const String & partition_name : task_table.ordered_partition_names)
    {
        if (!task_table.cluster_partitions.count(partition_name))
            throw Exception("There are no expected partition " + partition_name + ". It is a bug", ErrorCodes::LOGICAL_ERROR);
863

N
Nikita Mikhaylov 已提交
864
        ClusterPartition & cluster_partition = task_table.cluster_partitions[partition_name];
865

N
Nikita Mikhaylov 已提交
866
        Stopwatch watch;
N
Nikita Mikhaylov 已提交
867
        /// We will check all the shards of the table and check if they contain current partition.
N
Nikita Mikhaylov 已提交
868 869
        TasksShard expected_shards;
        UInt64 num_failed_shards = 0;
870

N
Nikita Mikhaylov 已提交
871
        ++cluster_partition.total_tries;
872

A
Alexey Milovidov 已提交
873
        LOG_DEBUG(log, "Processing partition {} for the whole cluster", partition_name);
874

N
Nikita Mikhaylov 已提交
875 876 877 878
        /// Process each source shard having current partition and copy current partition
        /// NOTE: shards are sorted by "distance" to current host
        bool has_shard_to_process = false;
        for (const TaskShardPtr & shard : task_table.all_shards)
879
        {
N
Nikita Mikhaylov 已提交
880 881
            /// Does shard have a node with current partition?
            if (shard->partition_tasks.count(partition_name) == 0)
882
            {
N
Nikita Mikhaylov 已提交
883 884
                /// If not, did we check existence of that partition previously?
                if (shard->checked_partitions.count(partition_name) == 0)
885
                {
N
Nikita Mikhaylov 已提交
886 887
                    auto check_shard_has_partition = [&] () { return checkShardHasPartition(timeouts, *shard, partition_name); };
                    bool has_partition = retry(check_shard_has_partition);
888

N
Nikita Mikhaylov 已提交
889
                    shard->checked_partitions.emplace(partition_name);
890

N
Nikita Mikhaylov 已提交
891 892
                    if (has_partition)
                    {
N
Nikita Mikhaylov 已提交
893 894
                        const size_t number_of_splits = task_table.number_of_splits;
                        shard->partition_tasks.emplace(partition_name, ShardPartition(*shard, partition_name, number_of_splits));
A
Alexey Milovidov 已提交
895
                        LOG_DEBUG(log, "Discovered partition {} in shard {}", partition_name, shard->getDescription());
N
Nikita Mikhaylov 已提交
896 897 898 899 900 901 902 903 904
                        /// To save references in the future.
                        auto shard_partition_it = shard->partition_tasks.find(partition_name);
                        PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces;

                        for (size_t piece_number = 0; piece_number < number_of_splits; ++piece_number)
                        {
                            auto res = checkPresentPartitionPiecesOnCurrentShard(timeouts, *shard, partition_name, piece_number);
                            shard_partition_pieces.emplace_back(shard_partition_it->second, piece_number, res);
                        }
N
Nikita Mikhaylov 已提交
905 906 907
                    }
                    else
                    {
A
Alexey Milovidov 已提交
908
                        LOG_DEBUG(log, "Found that shard {} does not contain current partition {}", shard->getDescription(), partition_name);
N
Nikita Mikhaylov 已提交
909 910 911 912
                        continue;
                    }
                }
                else
913
                {
N
Nikita Mikhaylov 已提交
914 915 916
                    /// We have already checked that partition, but did not discover it
                    previous_shard_is_instantly_finished = true;
                    continue;
917
                }
N
Nikita Mikhaylov 已提交
918
            }
919

N
Nikita Mikhaylov 已提交
920
            auto it_shard_partition = shard->partition_tasks.find(partition_name);
N
Nikita Mikhaylov 已提交
921 922
            /// Previously when we discovered that shard does not contain current partition, we skipped it.
            /// At this moment partition have to be present.
N
Nikita Mikhaylov 已提交
923
            if (it_shard_partition == shard->partition_tasks.end())
N
Nikita Mikhaylov 已提交
924
                throw Exception("There are no such partition in a shard. This is a bug.", ErrorCodes::LOGICAL_ERROR);
N
Nikita Mikhaylov 已提交
925
            auto & partition = it_shard_partition->second;
926

N
Nikita Mikhaylov 已提交
927
            expected_shards.emplace_back(shard);
928

N
Nikita Mikhaylov 已提交
929 930
            /// Do not sleep if there is a sequence of already processed shards to increase startup
            bool is_unprioritized_task = !previous_shard_is_instantly_finished && shard->priority.is_remote;
N
Nikita Mikhaylov 已提交
931
            TaskStatus task_status = TaskStatus::Error;
N
Nikita Mikhaylov 已提交
932 933 934
            bool was_error = false;
            has_shard_to_process = true;
            for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num)
935
            {
N
Nikita Mikhaylov 已提交
936
                task_status = tryProcessPartitionTask(timeouts, partition, is_unprioritized_task);
937

N
Nikita Mikhaylov 已提交
938
                /// Exit if success
N
Nikita Mikhaylov 已提交
939
                if (task_status == TaskStatus::Finished)
N
Nikita Mikhaylov 已提交
940
                    break;
941

N
Nikita Mikhaylov 已提交
942
                was_error = true;
943

N
Nikita Mikhaylov 已提交
944
                /// Skip if the task is being processed by someone
N
Nikita Mikhaylov 已提交
945
                if (task_status == TaskStatus::Active)
N
Nikita Mikhaylov 已提交
946
                    break;
947

N
Nikita Mikhaylov 已提交
948 949
                /// Repeat on errors
                std::this_thread::sleep_for(default_sleep_time);
950
            }
N
Nikita Mikhaylov 已提交
951

N
Nikita Mikhaylov 已提交
952
            if (task_status == TaskStatus::Error)
N
Nikita Mikhaylov 已提交
953 954 955
                ++num_failed_shards;

            previous_shard_is_instantly_finished = !was_error;
956 957
        }

N
Nikita Mikhaylov 已提交
958 959 960 961
        cluster_partition.elapsed_time_seconds += watch.elapsedSeconds();

        /// Check that whole cluster partition is done
        /// Firstly check the number of failed partition tasks, then look into ZooKeeper and ensure that each partition is done
N
Nikita Mikhaylov 已提交
962
        bool partition_copying_is_done = num_failed_shards == 0;
N
Nikita Mikhaylov 已提交
963
        try
964
        {
N
Nikita Mikhaylov 已提交
965
            partition_copying_is_done =
N
Nikita Mikhaylov 已提交
966
                    !has_shard_to_process
N
Nikita Mikhaylov 已提交
967
                    || (partition_copying_is_done && checkAllPiecesInPartitionAreDone(task_table, partition_name, expected_shards));
968
        }
N
Nikita Mikhaylov 已提交
969
        catch (...)
970
        {
N
Nikita Mikhaylov 已提交
971
            tryLogCurrentException(log);
N
Nikita Mikhaylov 已提交
972 973 974 975 976 977 978 979
            partition_copying_is_done = false;
        }


        bool partition_moving_is_done = false;
        /// Try to move only if all pieces were copied.
        if (partition_copying_is_done)
        {
N
Nikita Mikhaylov 已提交
980
            for (UInt64 try_num = 0; try_num < max_shard_partition_piece_tries_for_alter; ++try_num)
N
Nikita Mikhaylov 已提交
981
            {
N
Nikita Mikhaylov 已提交
982 983 984
                try
                {
                    auto res = tryMoveAllPiecesToDestinationTable(task_table, partition_name);
N
Nikita Mikhaylov 已提交
985 986
                    /// Exit and mark current task is done.
                    if (res == TaskStatus::Finished)
N
Nikita Mikhaylov 已提交
987 988 989 990 991
                    {
                        partition_moving_is_done = true;
                        break;
                    }

N
Nikita Mikhaylov 已提交
992 993
                    /// Exit if this task is active.
                    if (res == TaskStatus::Active)
N
better  
Nikita Mikhaylov 已提交
994
                        break;
N
Nikita Mikhaylov 已提交
995

N
Nikita Mikhaylov 已提交
996
                    /// Repeat on errors.
N
better  
Nikita Mikhaylov 已提交
997
                    std::this_thread::sleep_for(default_sleep_time);
N
Nikita Mikhaylov 已提交
998
                }
N
Nikita Mikhaylov 已提交
999 1000 1001
                catch (...)
                {
                    tryLogCurrentException(log, "Some error occured while moving pieces to destination table for partition " + partition_name);
N
Nikita Mikhaylov 已提交
1002
                }
N
Nikita Mikhaylov 已提交
1003
            }
1004 1005
        }

N
Nikita Mikhaylov 已提交
1006
        if (partition_copying_is_done && partition_moving_is_done)
1007
        {
N
Nikita Mikhaylov 已提交
1008
            task_table.finished_cluster_partitions.emplace(partition_name);
1009

N
Nikita Mikhaylov 已提交
1010 1011 1012
            task_table.bytes_copied += cluster_partition.bytes_copied;
            task_table.rows_copied += cluster_partition.rows_copied;
            double elapsed = cluster_partition.elapsed_time_seconds;
1013

A
Alexey Milovidov 已提交
1014
            LOG_INFO(log, "It took {} seconds to copy partition {}: {} uncompressed bytes, {} rows and {} source blocks are copied",
A
Alexey Milovidov 已提交
1015 1016 1017 1018
                elapsed, partition_name,
                formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied),
                formatReadableQuantity(cluster_partition.rows_copied),
                cluster_partition.blocks_copied);
1019

N
Nikita Mikhaylov 已提交
1020
            if (cluster_partition.rows_copied)
1021
            {
A
Alexey Milovidov 已提交
1022
                LOG_INFO(log, "Average partition speed: {} per second.", formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied / elapsed));
1023
            }
1024

N
Nikita Mikhaylov 已提交
1025
            if (task_table.rows_copied)
1026
            {
A
Alexey Milovidov 已提交
1027
                LOG_INFO(log, "Average table {} speed: {} per second.", task_table.table_id, formatReadableSizeWithDecimalSuffix(task_table.bytes_copied / elapsed));
N
Nikita Mikhaylov 已提交
1028 1029 1030
            }
        }
    }
1031

N
Nikita Mikhaylov 已提交
1032 1033 1034
    UInt64 required_partitions = task_table.cluster_partitions.size();
    UInt64 finished_partitions = task_table.finished_cluster_partitions.size();
    bool table_is_done = finished_partitions >= required_partitions;
1035

N
Nikita Mikhaylov 已提交
1036 1037
    if (!table_is_done)
    {
A
Alexey Milovidov 已提交
1038
        LOG_INFO(log, "Table {} is not processed yet.Copied {} of {}, will retry", task_table.table_id, finished_partitions, required_partitions);
N
Nikita Mikhaylov 已提交
1039
    }
1040

N
Nikita Mikhaylov 已提交
1041 1042
    return table_is_done;
}
1043

N
Nikita Mikhaylov 已提交
1044
/// Job for copying partition from particular shard.
N
Nikita Mikhaylov 已提交
1045
TaskStatus ClusterCopier::tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task)
N
Nikita Mikhaylov 已提交
1046
{
N
Nikita Mikhaylov 已提交
1047
    TaskStatus res;
1048

N
Nikita Mikhaylov 已提交
1049 1050
    try
    {
N
Nikita Mikhaylov 已提交
1051
        res = iterateThroughAllPiecesInPartition(timeouts, task_partition, is_unprioritized_task);
N
Nikita Mikhaylov 已提交
1052 1053 1054 1055
    }
    catch (...)
    {
        tryLogCurrentException(log, "An error occurred while processing partition " + task_partition.name);
N
Nikita Mikhaylov 已提交
1056
        res = TaskStatus::Error;
N
Nikita Mikhaylov 已提交
1057
    }
1058

N
Nikita Mikhaylov 已提交
1059 1060 1061 1062 1063 1064 1065 1066 1067
    /// At the end of each task check if the config is updated
    try
    {
        updateConfigIfNeeded();
    }
    catch (...)
    {
        tryLogCurrentException(log, "An error occurred while updating the config");
    }
1068

N
Nikita Mikhaylov 已提交
1069 1070
    return res;
}
1071

N
Nikita Mikhaylov 已提交
1072
TaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const ConnectionTimeouts & timeouts, ShardPartition & task_partition,
N
Nikita Mikhaylov 已提交
1073 1074 1075 1076
                                                       bool is_unprioritized_task)
{
    const size_t total_number_of_pieces = task_partition.task_shard.task_table.number_of_splits;

N
Nikita Mikhaylov 已提交
1077
    TaskStatus res{TaskStatus::Finished};
N
Nikita Mikhaylov 已提交
1078 1079 1080

    bool was_failed_pieces = false;
    bool was_active_pieces = false;
N
Nikita Mikhaylov 已提交
1081

N
Nikita Mikhaylov 已提交
1082 1083
    for (size_t piece_number = 0; piece_number < total_number_of_pieces; piece_number++)
    {
N
Nikita Mikhaylov 已提交
1084 1085
        for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num)
        {
A
Alexey Milovidov 已提交
1086
            LOG_INFO(log, "Attempt number {} to process partition {} piece number {} on shard number {} with index {}.",
A
Alexey Milovidov 已提交
1087 1088 1089 1090
                try_num, task_partition.name, piece_number,
                task_partition.task_shard.numberInCluster(),
                task_partition.task_shard.indexInCluster());

N
Nikita Mikhaylov 已提交
1091 1092 1093
            res = processPartitionPieceTaskImpl(timeouts, task_partition, piece_number, is_unprioritized_task);

            /// Exit if success
N
Nikita Mikhaylov 已提交
1094
            if (res == TaskStatus::Finished)
N
Nikita Mikhaylov 已提交
1095 1096 1097
                break;

            /// Skip if the task is being processed by someone
N
Nikita Mikhaylov 已提交
1098
            if (res == TaskStatus::Active)
N
Nikita Mikhaylov 已提交
1099 1100 1101 1102 1103 1104
                break;

            /// Repeat on errors
            std::this_thread::sleep_for(default_sleep_time);
        }

N
Nikita Mikhaylov 已提交
1105 1106
        was_active_pieces = (res == TaskStatus::Active);
        was_failed_pieces = (res == TaskStatus::Error);
N
Nikita Mikhaylov 已提交
1107
    }
N
Nikita Mikhaylov 已提交
1108

N
Nikita Mikhaylov 已提交
1109
    if (was_failed_pieces)
N
Nikita Mikhaylov 已提交
1110
        return TaskStatus::Error;
N
Nikita Mikhaylov 已提交
1111 1112

    if (was_active_pieces)
N
Nikita Mikhaylov 已提交
1113
        return TaskStatus::Active;
N
Nikita Mikhaylov 已提交
1114

N
Nikita Mikhaylov 已提交
1115
    return TaskStatus::Finished;
N
Nikita Mikhaylov 已提交
1116 1117
}

N
Nikita Mikhaylov 已提交
1118

N
Nikita Mikhaylov 已提交
1119
TaskStatus ClusterCopier::processPartitionPieceTaskImpl(
N
Nikita Mikhaylov 已提交
1120 1121
        const ConnectionTimeouts & timeouts, ShardPartition & task_partition,
        const size_t current_piece_number, bool is_unprioritized_task)
N
Nikita Mikhaylov 已提交
1122 1123 1124
{
    TaskShard & task_shard = task_partition.task_shard;
    TaskTable & task_table = task_shard.task_table;
N
Nikita Mikhaylov 已提交
1125
    ClusterPartition & cluster_partition  = task_table.getClusterPartition(task_partition.name);
N
Nikita Mikhaylov 已提交
1126 1127 1128 1129
    ShardPartitionPiece & partition_piece = task_partition.pieces[current_piece_number];

    const size_t number_of_splits = task_table.number_of_splits;
    const String primary_key_comma_separated = task_table.primary_key_comma_separated;
1130

N
Nikita Mikhaylov 已提交
1131
    /// We need to update table definitions for each partition, it could be changed after ALTER
N
Nikita Mikhaylov 已提交
1132 1133 1134
    createShardInternalTables(timeouts, task_shard, true);

    auto split_table_for_current_piece = task_shard.list_of_split_tables_on_shard[current_piece_number];
1135

N
Nikita Mikhaylov 已提交
1136
    auto zookeeper = context.getZooKeeper();
1137

N
Nikita Mikhaylov 已提交
1138 1139
    const String piece_is_dirty_flag_path          = partition_piece.getPartitionPieceIsDirtyPath();
    const String piece_is_dirty_cleaned_path       = partition_piece.getPartitionPieceIsCleanedPath();
N
Nikita Mikhaylov 已提交
1140
    const String current_task_piece_is_active_path = partition_piece.getActiveWorkerPath();
N
Nikita Mikhaylov 已提交
1141
    const String current_task_piece_status_path    = partition_piece.getShardStatusPath();
1142

N
Nikita Mikhaylov 已提交
1143
    /// Auxiliary functions:
1144

N
Nikita Mikhaylov 已提交
1145
    /// Creates is_dirty node to initialize DROP PARTITION
N
Nikita Mikhaylov 已提交
1146
    auto create_is_dirty_node = [&] (const CleanStateClock & clock)
N
Nikita Mikhaylov 已提交
1147 1148
    {
        if (clock.is_stale())
A
Alexey Milovidov 已提交
1149
            LOG_DEBUG(log, "Clean state clock is stale while setting dirty flag, cowardly bailing");
N
Nikita Mikhaylov 已提交
1150
        else if (!clock.is_clean())
A
Alexey Milovidov 已提交
1151
            LOG_DEBUG(log, "Thank you, Captain Obvious");
N
Nikita Mikhaylov 已提交
1152 1153
        else if (clock.discovery_version)
        {
A
Alexey Milovidov 已提交
1154
            LOG_DEBUG(log, "Updating clean state clock");
N
Nikita Mikhaylov 已提交
1155
            zookeeper->set(piece_is_dirty_flag_path, host_id, clock.discovery_version.value());
N
Nikita Mikhaylov 已提交
1156 1157
        }
        else
1158
        {
A
Alexey Milovidov 已提交
1159
            LOG_DEBUG(log, "Creating clean state clock");
N
Nikita Mikhaylov 已提交
1160
            zookeeper->create(piece_is_dirty_flag_path, host_id, zkutil::CreateMode::Persistent);
1161
        }
N
Nikita Mikhaylov 已提交
1162
    };
1163

N
Nikita Mikhaylov 已提交
1164
    /// Returns SELECT query filtering current partition and applying user filter
N
Nikita Mikhaylov 已提交
1165
    auto get_select_query = [&] (const DatabaseAndTableName & from_table, const String & fields, bool enable_splitting, String limit = "")
1166
    {
N
Nikita Mikhaylov 已提交
1167 1168
        String query;
        query += "SELECT " + fields + " FROM " + getQuotedTable(from_table);
N
Nikita Mikhaylov 已提交
1169

A
alexey-milovidov 已提交
1170 1171
        if (enable_splitting && experimental_use_sample_offset)
            query += " SAMPLE 1/" + toString(number_of_splits) + " OFFSET " + toString(current_piece_number) + "/" + toString(number_of_splits);
N
Nikita Mikhaylov 已提交
1172

N
Nikita Mikhaylov 已提交
1173 1174
        /// TODO: Bad, it is better to rewrite with ASTLiteral(partition_key_field)
        query += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))";
N
Nikita Mikhaylov 已提交
1175

A
alexey-milovidov 已提交
1176 1177
        if (enable_splitting && !experimental_use_sample_offset)
            query += " AND ( cityHash64(" + primary_key_comma_separated + ") %" + toString(number_of_splits) + " = " + toString(current_piece_number) + " )";
N
Nikita Mikhaylov 已提交
1178

N
Nikita Mikhaylov 已提交
1179 1180
        if (!task_table.where_condition_str.empty())
            query += " AND (" + task_table.where_condition_str + ")";
N
Nikita Mikhaylov 已提交
1181

N
Nikita Mikhaylov 已提交
1182 1183
        if (!limit.empty())
            query += " LIMIT " + limit;
1184

N
Nikita Mikhaylov 已提交
1185
        ParserQuery p_query(query.data() + query.size());
1186 1187 1188

        const auto & settings = context.getSettingsRef();
        return parseQuery(p_query, query, settings.max_query_size, settings.max_parser_depth);
N
Nikita Mikhaylov 已提交
1189
    };
1190

N
Nikita Mikhaylov 已提交
1191
    /// Load balancing
N
Nikita Mikhaylov 已提交
1192
    auto worker_node_holder = createTaskWorkerNodeAndWaitIfNeed(zookeeper, current_task_piece_status_path, is_unprioritized_task);
1193

A
Alexey Milovidov 已提交
1194
    LOG_DEBUG(log, "Processing {}", current_task_piece_status_path);
1195

N
Nikita Mikhaylov 已提交
1196
    const String piece_status_path = partition_piece.getPartitionPieceShardsPath();
1197

N
Nikita Mikhaylov 已提交
1198 1199 1200
    CleanStateClock clean_state_clock(zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path);

    const bool is_clean = checkPartitionPieceIsClean(zookeeper, clean_state_clock, piece_status_path);
1201

N
Nikita Mikhaylov 已提交
1202
    /// Do not start if partition piece is dirty, try to clean it
N
Nikita Mikhaylov 已提交
1203
    if (is_clean)
1204
    {
A
Alexey Milovidov 已提交
1205
        LOG_DEBUG(log, "Partition {} piece {} appears to be clean", task_partition.name, current_piece_number);
N
Nikita Mikhaylov 已提交
1206
        zookeeper->createAncestors(current_task_piece_status_path);
1207
    }
N
Nikita Mikhaylov 已提交
1208
    else
1209
    {
A
Alexey Milovidov 已提交
1210
        LOG_DEBUG(log, "Partition {} piece {} is dirty, try to drop it", task_partition.name, current_piece_number);
1211

N
Nikita Mikhaylov 已提交
1212 1213
        try
        {
N
Nikita Mikhaylov 已提交
1214
            tryDropPartitionPiece(task_partition, current_piece_number, zookeeper, clean_state_clock);
N
Nikita Mikhaylov 已提交
1215 1216 1217 1218 1219
        }
        catch (...)
        {
            tryLogCurrentException(log, "An error occurred when clean partition");
        }
1220

N
Nikita Mikhaylov 已提交
1221
        return TaskStatus::Error;
1222 1223
    }

N
Nikita Mikhaylov 已提交
1224
    /// Create ephemeral node to mark that we are active and process the partition
N
Nikita Mikhaylov 已提交
1225
    zookeeper->createAncestors(current_task_piece_is_active_path);
N
Nikita Mikhaylov 已提交
1226 1227
    zkutil::EphemeralNodeHolderPtr partition_task_node_holder;
    try
1228
    {
N
Nikita Mikhaylov 已提交
1229
        partition_task_node_holder = zkutil::EphemeralNodeHolder::create(current_task_piece_is_active_path, *zookeeper, host_id);
N
Nikita Mikhaylov 已提交
1230 1231 1232 1233
    }
    catch (const Coordination::Exception & e)
    {
        if (e.code == Coordination::ZNODEEXISTS)
1234
        {
A
Alexey Milovidov 已提交
1235
            LOG_DEBUG(log, "Someone is already processing {}", current_task_piece_is_active_path);
N
Nikita Mikhaylov 已提交
1236
            return TaskStatus::Active;
1237 1238
        }

N
Nikita Mikhaylov 已提交
1239 1240
        throw;
    }
1241

N
Nikita Mikhaylov 已提交
1242 1243 1244 1245
    /// Exit if task has been already processed;
    /// create blocking node to signal cleaning up if it is abandoned
    {
        String status_data;
N
Nikita Mikhaylov 已提交
1246
        if (zookeeper->tryGet(current_task_piece_status_path, status_data))
1247
        {
N
Nikita Mikhaylov 已提交
1248 1249
            TaskStateWithOwner status = TaskStateWithOwner::fromString(status_data);
            if (status.state == TaskState::Finished)
1250
            {
A
Alexey Milovidov 已提交
1251
                LOG_DEBUG(log, "Task {} has been successfully executed by {}", current_task_piece_status_path, status.owner);
N
Nikita Mikhaylov 已提交
1252
                return TaskStatus::Finished;
1253 1254
            }

N
Nikita Mikhaylov 已提交
1255 1256
            /// Task is abandoned, because previously we created ephemeral node, possibly in other copier's process.
            /// Initialize DROP PARTITION
A
Alexey Milovidov 已提交
1257
            LOG_DEBUG(log, "Task {} has not been successfully finished by {}. Partition will be dropped and refilled.", current_task_piece_status_path, status.owner);
1258

N
Nikita Mikhaylov 已提交
1259
            create_is_dirty_node(clean_state_clock);
N
Nikita Mikhaylov 已提交
1260
            return TaskStatus::Error;
N
Nikita Mikhaylov 已提交
1261
        }
1262 1263
    }

N
Nikita Mikhaylov 已提交
1264 1265 1266 1267 1268 1269 1270 1271

    /// Exit if current piece is absent on this shard. Also mark it as finished, because we will check
    /// whether each shard have processed each partitition (and its pieces).
    if (partition_piece.is_absent_piece)
    {
        String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id);
        auto res = zookeeper->tryCreate(current_task_piece_status_path, state_finished, zkutil::CreateMode::Persistent);
        if (res == Coordination::ZNODEEXISTS)
A
Alexey Milovidov 已提交
1272
            LOG_DEBUG(log, "Partition {} piece {} is absent on current replica of a shard. But other replicas have already marked it as done.", task_partition.name, current_piece_number);
N
Nikita Mikhaylov 已提交
1273
        if (res == Coordination::ZOK)
A
Alexey Milovidov 已提交
1274
            LOG_DEBUG(log, "Partition {} piece {} is absent on current replica of a shard. Will mark it as done. Other replicas will do the same.", task_partition.name, current_piece_number);
N
Nikita Mikhaylov 已提交
1275
        return TaskStatus::Finished;
N
Nikita Mikhaylov 已提交
1276 1277
    }

N
Nikita Mikhaylov 已提交
1278 1279 1280
    /// Check that destination partition is empty if we are first worker
    /// NOTE: this check is incorrect if pull and push tables have different partition key!
    String clean_start_status;
N
Nikita Mikhaylov 已提交
1281
    if (!zookeeper->tryGet(partition_piece.getPartitionPieceCleanStartPath(), clean_start_status) || clean_start_status != "ok")
1282
    {
N
Nikita Mikhaylov 已提交
1283 1284
        zookeeper->createIfNotExists(partition_piece.getPartitionPieceCleanStartPath(), "");
        auto checker = zkutil::EphemeralNodeHolder::create(partition_piece.getPartitionPieceCleanStartPath() + "/checker",
N
Nikita Mikhaylov 已提交
1285
                                                           *zookeeper, host_id);
N
Nikita Mikhaylov 已提交
1286
        // Maybe we are the first worker
N
Nikita Mikhaylov 已提交
1287 1288

        ASTPtr query_select_ast = get_select_query(split_table_for_current_piece, "count()", /*enable_splitting*/ true);
N
Nikita Mikhaylov 已提交
1289 1290 1291 1292
        UInt64 count;
        {
            Context local_context = context;
            // Use pull (i.e. readonly) settings, but fetch data from destination servers
1293 1294
            local_context.setSettings(task_cluster->settings_pull);
            local_context.setSetting("skip_unavailable_shards", true);
N
Nikita Mikhaylov 已提交
1295 1296 1297 1298

            Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_select_ast, local_context)->execute().in);
            count = (block) ? block.safeGetByPosition(0).column->getUInt(0) : 0;
        }
1299

N
Nikita Mikhaylov 已提交
1300 1301
        if (count != 0)
        {
A
Alexey Milovidov 已提交
1302
            LOG_INFO(log, "Partition {} piece {}is not empty. In contains {} rows.", task_partition.name, current_piece_number, count);
N
Nikita Mikhaylov 已提交
1303
            Coordination::Stat stat_shards{};
N
Nikita Mikhaylov 已提交
1304
            zookeeper->get(partition_piece.getPartitionPieceShardsPath(), &stat_shards);
1305

N
Nikita Mikhaylov 已提交
1306 1307 1308
            /// NOTE: partition is still fresh if dirt discovery happens before cleaning
            if (stat_shards.numChildren == 0)
            {
A
Alexey Milovidov 已提交
1309
                LOG_WARNING(log, "There are no workers for partition {} piece {}, but destination table contains {} rows. Partition will be dropped and refilled.", task_partition.name, toString(current_piece_number), count);
1310

N
Nikita Mikhaylov 已提交
1311
                create_is_dirty_node(clean_state_clock);
N
Nikita Mikhaylov 已提交
1312
                return TaskStatus::Error;
N
Nikita Mikhaylov 已提交
1313 1314
            }
        }
N
Nikita Mikhaylov 已提交
1315
        zookeeper->set(partition_piece.getPartitionPieceCleanStartPath(), "ok");
N
Nikita Mikhaylov 已提交
1316 1317 1318
    }
    /// At this point, we need to sync that the destination table is clean
    /// before any actual work
1319

N
Nikita Mikhaylov 已提交
1320 1321 1322
    /// Try start processing, create node about it
    {
        String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id);
N
Nikita Mikhaylov 已提交
1323
        CleanStateClock new_clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path);
N
Nikita Mikhaylov 已提交
1324 1325
        if (clean_state_clock != new_clean_state_clock)
        {
A
Alexey Milovidov 已提交
1326
            LOG_INFO(log, "Partition {} piece {} clean state changed, cowardly bailing", task_partition.name, toString(current_piece_number));
N
Nikita Mikhaylov 已提交
1327
            return TaskStatus::Error;
N
Nikita Mikhaylov 已提交
1328 1329 1330
        }
        else if (!new_clean_state_clock.is_clean())
        {
A
Alexey Milovidov 已提交
1331
            LOG_INFO(log, "Partition {} piece {} is dirty and will be dropped and refilled", task_partition.name, toString(current_piece_number));
N
Nikita Mikhaylov 已提交
1332
            create_is_dirty_node(new_clean_state_clock);
N
Nikita Mikhaylov 已提交
1333
            return TaskStatus::Error;
N
Nikita Mikhaylov 已提交
1334
        }
N
Nikita Mikhaylov 已提交
1335
        zookeeper->create(current_task_piece_status_path, start_state, zkutil::CreateMode::Persistent);
1336 1337
    }

N
Nikita Mikhaylov 已提交
1338
    /// Try create table (if not exists) on each shard
1339
    {
N
Nikita Mikhaylov 已提交
1340 1341 1342
        /// Define push table for current partition piece
        auto database_and_table_for_current_piece= std::pair<String, String>(
                task_table.table_push.first,
N
Nikita Mikhaylov 已提交
1343
                task_table.table_push.second + "_piece_" + toString(current_piece_number));
N
Nikita Mikhaylov 已提交
1344

N
Nikita Mikhaylov 已提交
1345 1346 1347
        auto new_engine_push_ast = task_table.engine_push_ast;
        if (task_table.isReplicatedTable())
        {
1348
            new_engine_push_ast = task_table.rewriteReplicatedCreateQueryToPlain();
N
Nikita Mikhaylov 已提交
1349 1350 1351 1352 1353 1354
        }

        auto create_query_push_ast = rewriteCreateQueryStorage(
                task_shard.current_pull_table_create_query,
                database_and_table_for_current_piece, new_engine_push_ast);

N
Nikita Mikhaylov 已提交
1355 1356 1357
        create_query_push_ast->as<ASTCreateQuery &>().if_not_exists = true;
        String query = queryToString(create_query_push_ast);

A
Alexey Milovidov 已提交
1358
        LOG_DEBUG(log, "Create destination tables. Query: {}", query);
N
Nikita Mikhaylov 已提交
1359 1360 1361
        UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query,
                                              create_query_push_ast, &task_cluster->settings_push,
                                              PoolMode::GET_MANY);
A
Alexey Milovidov 已提交
1362
        LOG_DEBUG(log, "Destination tables {} have been created on {} shards of {}", getQuotedTable(task_table.table_push), shards, task_table.cluster_push->getShardCount());
N
Nikita Mikhaylov 已提交
1363
    }
1364

N
Nikita Mikhaylov 已提交
1365 1366 1367 1368
    /// Do the copying
    {
        bool inject_fault = false;
        if (copy_fault_probability > 0)
1369
        {
N
Nikita Mikhaylov 已提交
1370 1371
            double value = std::uniform_real_distribution<>(0, 1)(task_table.task_cluster.random_engine);
            inject_fault = value < copy_fault_probability;
1372 1373
        }

N
Nikita Mikhaylov 已提交
1374
        // Select all fields
N
Nikita Mikhaylov 已提交
1375
        ASTPtr query_select_ast = get_select_query(task_shard.table_read_shard, "*", /*enable_splitting*/ true, inject_fault ? "1" : "");
1376

A
Alexey Milovidov 已提交
1377
        LOG_DEBUG(log, "Executing SELECT query and pull from {} : {}", task_shard.getDescription(), queryToString(query_select_ast));
N
Nikita Mikhaylov 已提交
1378 1379

        ASTPtr query_insert_ast;
1380
        {
N
Nikita Mikhaylov 已提交
1381
            String query;
N
Nikita Mikhaylov 已提交
1382
            query += "INSERT INTO " + getQuotedTable(split_table_for_current_piece) + " VALUES ";
1383

N
Nikita Mikhaylov 已提交
1384
            ParserQuery p_query(query.data() + query.size());
1385 1386
            const auto & settings = context.getSettingsRef();
            query_insert_ast = parseQuery(p_query, query, settings.max_query_size, settings.max_parser_depth);
N
Nikita Mikhaylov 已提交
1387

A
Alexey Milovidov 已提交
1388
            LOG_DEBUG(log, "Executing INSERT query: {}", query);
N
Nikita Mikhaylov 已提交
1389 1390 1391 1392 1393 1394
        }

        try
        {
            /// Custom INSERT SELECT implementation
            Context context_select = context;
1395
            context_select.setSettings(task_cluster->settings_pull);
1396

N
Nikita Mikhaylov 已提交
1397
            Context context_insert = context;
1398
            context_insert.setSettings(task_cluster->settings_push);
1399

N
Nikita Mikhaylov 已提交
1400 1401
            BlockInputStreamPtr input;
            BlockOutputStreamPtr output;
1402
            {
N
Nikita Mikhaylov 已提交
1403 1404
                BlockIO io_select = InterpreterFactory::get(query_select_ast, context_select)->execute();
                BlockIO io_insert = InterpreterFactory::get(query_insert_ast, context_insert)->execute();
1405

N
Nikolai Kochetov 已提交
1406
                input = io_select.getInputStream();
N
Nikita Mikhaylov 已提交
1407
                output = io_insert.out;
1408 1409
            }

N
Nikita Mikhaylov 已提交
1410 1411 1412 1413 1414 1415 1416 1417
            /// Fail-fast optimization to abort copying when the current clean state expires
            std::future<Coordination::ExistsResponse> future_is_dirty_checker;

            Stopwatch watch(CLOCK_MONOTONIC_COARSE);
            constexpr UInt64 check_period_milliseconds = 500;

            /// Will asynchronously check that ZooKeeper connection and is_dirty flag appearing while copying data
            auto cancel_check = [&] ()
1418
            {
N
Nikita Mikhaylov 已提交
1419 1420
                if (zookeeper->expired())
                    throw Exception("ZooKeeper session is expired, cancel INSERT SELECT", ErrorCodes::UNFINISHED);
1421

N
Nikita Mikhaylov 已提交
1422
                if (!future_is_dirty_checker.valid())
N
Nikita Mikhaylov 已提交
1423
                    future_is_dirty_checker = zookeeper->asyncExists(piece_is_dirty_flag_path);
1424

N
Nikita Mikhaylov 已提交
1425 1426 1427
                /// check_period_milliseconds should less than average insert time of single block
                /// Otherwise, the insertion will slow a little bit
                if (watch.elapsedMilliseconds() >= check_period_milliseconds)
1428
                {
N
Nikita Mikhaylov 已提交
1429
                    Coordination::ExistsResponse status = future_is_dirty_checker.get();
1430

N
Nikita Mikhaylov 已提交
1431
                    if (status.error != Coordination::ZNONODE)
1432
                    {
N
Nikita Mikhaylov 已提交
1433 1434 1435 1436
                        LogicalClock dirt_discovery_epoch (status.stat.mzxid);
                        if (dirt_discovery_epoch == clean_state_clock.discovery_zxid)
                            return false;
                        throw Exception("Partition is dirty, cancel INSERT SELECT", ErrorCodes::UNFINISHED);
1437 1438 1439
                    }
                }

N
Nikita Mikhaylov 已提交
1440 1441
                return false;
            };
1442

N
Nikita Mikhaylov 已提交
1443 1444 1445 1446 1447 1448 1449 1450
            /// Update statistics
            /// It is quite rough: bytes_copied don't take into account DROP PARTITION.
            auto update_stats = [&cluster_partition] (const Block & block)
            {
                cluster_partition.bytes_copied += block.bytes();
                cluster_partition.rows_copied += block.rows();
                cluster_partition.blocks_copied += 1;
            };
1451

N
Nikita Mikhaylov 已提交
1452 1453
            /// Main work is here
            copyData(*input, *output, cancel_check, update_stats);
1454

N
Nikita Mikhaylov 已提交
1455 1456 1457
            // Just in case
            if (future_is_dirty_checker.valid())
                future_is_dirty_checker.get();
1458

N
Nikita Mikhaylov 已提交
1459 1460 1461 1462 1463 1464
            if (inject_fault)
                throw Exception("Copy fault injection is activated", ErrorCodes::UNFINISHED);
        }
        catch (...)
        {
            tryLogCurrentException(log, "An error occurred during copying, partition will be marked as dirty");
N
better  
Nikita Mikhaylov 已提交
1465
            create_is_dirty_node(clean_state_clock);
N
Nikita Mikhaylov 已提交
1466
            return TaskStatus::Error;
N
Nikita Mikhaylov 已提交
1467
        }
1468 1469
    }

A
Alexey Milovidov 已提交
1470
    LOG_INFO(log, "Partition {} piece {} copied. But not moved to original destination table.", task_partition.name, toString(current_piece_number));
N
Nikita Mikhaylov 已提交
1471 1472 1473


    /// Try create original table (if not exists) on each shard
N
Nikita Mikhaylov 已提交
1474
    try
N
Nikita Mikhaylov 已提交
1475 1476 1477 1478 1479 1480
    {
        auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query,
                                                               task_table.table_push, task_table.engine_push_ast);
        create_query_push_ast->as<ASTCreateQuery &>().if_not_exists = true;
        String query = queryToString(create_query_push_ast);

A
Alexey Milovidov 已提交
1481
        LOG_DEBUG(log, "Create destination tables. Query: {}", query);
N
Nikita Mikhaylov 已提交
1482 1483 1484
        UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query,
                                              create_query_push_ast, &task_cluster->settings_push,
                                              PoolMode::GET_MANY);
A
Alexey Milovidov 已提交
1485
        LOG_DEBUG(log, "Destination tables {} have been created on {} shards of {}", getQuotedTable(task_table.table_push), shards, task_table.cluster_push->getShardCount());
N
Nikita Mikhaylov 已提交
1486
    }
N
Nikita Mikhaylov 已提交
1487 1488 1489 1490
    catch (...)
    {
        tryLogCurrentException(log, "Error while creating original table. Maybe we are not first.");
    }
N
Nikita Mikhaylov 已提交
1491

N
Nikita Mikhaylov 已提交
1492 1493 1494
    /// Finalize the processing, change state of current partition task (and also check is_dirty flag)
    {
        String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id);
N
Nikita Mikhaylov 已提交
1495
        CleanStateClock new_clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path);
N
Nikita Mikhaylov 已提交
1496 1497
        if (clean_state_clock != new_clean_state_clock)
        {
A
Alexey Milovidov 已提交
1498
            LOG_INFO(log, "Partition {} piece {} clean state changed, cowardly bailing", task_partition.name, toString(current_piece_number));
N
Nikita Mikhaylov 已提交
1499
            return TaskStatus::Error;
N
Nikita Mikhaylov 已提交
1500 1501 1502
        }
        else if (!new_clean_state_clock.is_clean())
        {
A
Alexey Milovidov 已提交
1503
            LOG_INFO(log, "Partition {} piece {} became dirty and will be dropped and refilled", task_partition.name, toString(current_piece_number));
N
Nikita Mikhaylov 已提交
1504
            create_is_dirty_node(new_clean_state_clock);
N
Nikita Mikhaylov 已提交
1505
            return TaskStatus::Error;
N
Nikita Mikhaylov 已提交
1506
        }
N
Nikita Mikhaylov 已提交
1507
        zookeeper->set(current_task_piece_status_path, state_finished, 0);
N
Nikita Mikhaylov 已提交
1508
    }
1509

N
Nikita Mikhaylov 已提交
1510
    return TaskStatus::Finished;
N
Nikita Mikhaylov 已提交
1511
}
1512

N
Nikita Mikhaylov 已提交
1513 1514 1515 1516
void ClusterCopier::dropAndCreateLocalTable(const ASTPtr & create_ast)
{
    const auto & create = create_ast->as<ASTCreateQuery &>();
    dropLocalTableIfExists({create.database, create.table});
1517

N
Nikita Mikhaylov 已提交
1518 1519 1520
    InterpreterCreateQuery interpreter(create_ast, context);
    interpreter.execute();
}
1521

N
Nikita Mikhaylov 已提交
1522 1523 1524 1525 1526 1527
void ClusterCopier::dropLocalTableIfExists(const DatabaseAndTableName & table_name) const
{
    auto drop_ast = std::make_shared<ASTDropQuery>();
    drop_ast->if_exists = true;
    drop_ast->database = table_name.first;
    drop_ast->table = table_name.second;
1528

N
Nikita Mikhaylov 已提交
1529 1530 1531
    InterpreterDropQuery interpreter(drop_ast, context);
    interpreter.execute();
}
1532

N
Nikita Mikhaylov 已提交
1533 1534 1535

void ClusterCopier::dropHelpingTables(const TaskTable & task_table)
{
A
Alexey Milovidov 已提交
1536
    LOG_DEBUG(log, "Removing helping tables");
N
Nikita Mikhaylov 已提交
1537 1538 1539 1540 1541 1542 1543 1544 1545 1546
    for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number)
    {
        DatabaseAndTableName original_table = task_table.table_push;
        DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number));

        String query = "DROP TABLE IF EXISTS " + getQuotedTable(helping_table);

        const ClusterPtr & cluster_push = task_table.cluster_push;
        Settings settings_push = task_cluster->settings_push;

A
Alexey Milovidov 已提交
1547
        LOG_DEBUG(log, "Execute distributed DROP TABLE: {}", query);
N
Nikita Mikhaylov 已提交
1548 1549 1550 1551 1552 1553 1554 1555
        /// We have to drop partition_piece on each replica
        UInt64 num_nodes = executeQueryOnCluster(
                cluster_push, query,
                nullptr,
                &settings_push,
                PoolMode::GET_MANY,
                ClusterExecutionMode::ON_EACH_NODE);

A
Alexey Milovidov 已提交
1556
        LOG_DEBUG(log, "DROP TABLE query was successfully executed on {} nodes.", toString(num_nodes));
N
Nikita Mikhaylov 已提交
1557 1558 1559
    }
}

N
Nikita Mikhaylov 已提交
1560 1561 1562

void ClusterCopier::dropParticularPartitionPieceFromAllHelpingTables(const TaskTable & task_table, const String & partition_name)
{
A
Alexey Milovidov 已提交
1563
    LOG_DEBUG(log, "Try drop partition partition from all helping tables.");
N
Nikita Mikhaylov 已提交
1564 1565 1566 1567 1568 1569 1570 1571 1572 1573
    for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number)
    {
        DatabaseAndTableName original_table = task_table.table_push;
        DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number));

        String query = "ALTER TABLE " + getQuotedTable(helping_table) + " DROP PARTITION " + partition_name;

        const ClusterPtr & cluster_push = task_table.cluster_push;
        Settings settings_push = task_cluster->settings_push;

A
Alexey Milovidov 已提交
1574
        LOG_DEBUG(log, "Execute distributed DROP PARTITION: {}", query);
N
Nikita Mikhaylov 已提交
1575 1576 1577 1578 1579 1580 1581 1582
        /// We have to drop partition_piece on each replica
        UInt64 num_nodes = executeQueryOnCluster(
                cluster_push, query,
                nullptr,
                &settings_push,
                PoolMode::GET_MANY,
                ClusterExecutionMode::ON_EACH_NODE);

A
Alexey Milovidov 已提交
1583
        LOG_DEBUG(log, "DROP PARTITION query was successfully executed on {} nodes.", toString(num_nodes));
N
Nikita Mikhaylov 已提交
1584
    }
A
Alexey Milovidov 已提交
1585
    LOG_DEBUG(log, "All helping tables dropped partition {}", partition_name);
N
Nikita Mikhaylov 已提交
1586 1587
}

N
Nikita Mikhaylov 已提交
1588 1589 1590 1591
String ClusterCopier::getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings)
{
    String query = "SHOW CREATE TABLE " + getQuotedTable(table);
    Block block = getBlockWithAllStreamData(std::make_shared<RemoteBlockInputStream>(
N
Nikita Mikhaylov 已提交
1592
            connection, query, InterpreterShowCreateQuery::getSampleBlock(), context, settings));
1593

N
Nikita Mikhaylov 已提交
1594 1595
    return typeid_cast<const ColumnString &>(*block.safeGetByPosition(0).column).getDataAt(0).toString();
}
1596

N
Nikita Mikhaylov 已提交
1597 1598 1599
ASTPtr ClusterCopier::getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard)
{
    /// Fetch and parse (possibly) new definition
1600
    auto connection_entry = task_shard.info.pool->get(timeouts, &task_cluster->settings_pull, true);
N
Nikita Mikhaylov 已提交
1601
    String create_query_pull_str = getRemoteCreateTable(
N
Nikita Mikhaylov 已提交
1602 1603 1604
            task_shard.task_table.table_pull,
            *connection_entry,
            &task_cluster->settings_pull);
N
Nikita Mikhaylov 已提交
1605 1606

    ParserCreateQuery parser_create_query;
1607 1608
    const auto & settings = context.getSettingsRef();
    return parseQuery(parser_create_query, create_query_pull_str, settings.max_query_size, settings.max_parser_depth);
N
Nikita Mikhaylov 已提交
1609
}
1610

N
Nikita Mikhaylov 已提交
1611
/// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it.
N
Nikita Mikhaylov 已提交
1612
void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeouts,
N
Nikita Mikhaylov 已提交
1613
        TaskShard & task_shard, bool create_split)
1614
{
N
Nikita Mikhaylov 已提交
1615
    TaskTable & task_table = task_shard.task_table;
1616

N
Nikita Mikhaylov 已提交
1617 1618
    /// We need to update table definitions for each part, it could be changed after ALTER
    task_shard.current_pull_table_create_query = getCreateTableForPullShard(timeouts, task_shard);
1619

N
Nikita Mikhaylov 已提交
1620 1621 1622 1623 1624
    /// Create local Distributed tables:
    ///  a table fetching data from current shard and a table inserting data to the whole destination cluster
    String read_shard_prefix = ".read_shard_" + toString(task_shard.indexInCluster()) + ".";
    String split_shard_prefix = ".split.";
    task_shard.table_read_shard = DatabaseAndTableName(working_database_name, read_shard_prefix + task_table.table_id);
N
Nikita Mikhaylov 已提交
1625 1626 1627 1628 1629 1630 1631
    task_shard.main_table_split_shard = DatabaseAndTableName(working_database_name, split_shard_prefix + task_table.table_id);

    for (const auto & piece_number : ext::range(0, task_table.number_of_splits))
    {
        task_shard.list_of_split_tables_on_shard[piece_number] =
                DatabaseAndTableName(working_database_name, split_shard_prefix + task_table.table_id + "_piece_" + toString(piece_number));
    }
1632

N
Nikita Mikhaylov 已提交
1633 1634 1635 1636
    /// Create special cluster with single shard
    String shard_read_cluster_name = read_shard_prefix + task_table.cluster_pull_name;
    ClusterPtr cluster_pull_current_shard = task_table.cluster_pull->getClusterWithSingleShard(task_shard.indexInCluster());
    context.setCluster(shard_read_cluster_name, cluster_pull_current_shard);
1637

N
Nikita Mikhaylov 已提交
1638
    auto storage_shard_ast = createASTStorageDistributed(shard_read_cluster_name, task_table.table_pull.first, task_table.table_pull.second);
1639

N
Nikita Mikhaylov 已提交
1640
    auto create_query_ast = removeAliasColumnsFromCreateQuery(task_shard.current_pull_table_create_query);
1641

N
Nikita Mikhaylov 已提交
1642
    auto create_table_pull_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_read_shard, storage_shard_ast);
N
Nikita Mikhaylov 已提交
1643
    dropAndCreateLocalTable(create_table_pull_ast);
1644

N
Nikita Mikhaylov 已提交
1645
    if (create_split)
N
Nikita Mikhaylov 已提交
1646 1647 1648 1649 1650 1651
    {
        auto create_table_split_piece_ast = rewriteCreateQueryStorage(
                create_query_ast,
                task_shard.main_table_split_shard,
                task_table.main_engine_split_ast);

N
Nikita Mikhaylov 已提交
1652
        dropAndCreateLocalTable(create_table_split_piece_ast);
N
Nikita Mikhaylov 已提交
1653 1654 1655 1656 1657 1658 1659 1660 1661 1662 1663 1664 1665 1666 1667

        /// Create auxilary split tables for each piece
        for (const auto & piece_number : ext::range(0, task_table.number_of_splits))
        {
            const auto & storage_piece_split_ast = task_table.auxiliary_engine_split_asts[piece_number];

            create_table_split_piece_ast = rewriteCreateQueryStorage(
                    create_query_ast,
                    task_shard.list_of_split_tables_on_shard[piece_number],
                    storage_piece_split_ast);

            dropAndCreateLocalTable(create_table_split_piece_ast);
        }
    }

1668
}
1669 1670


N
Nikita Mikhaylov 已提交
1671
std::set<String> ClusterCopier::getShardPartitions(const ConnectionTimeouts & timeouts, TaskShard & task_shard)
1672
{
N
Nikita Mikhaylov 已提交
1673
    createShardInternalTables(timeouts, task_shard, false);
1674

N
Nikita Mikhaylov 已提交
1675
    TaskTable & task_table = task_shard.task_table;
1676

N
Nikita Mikhaylov 已提交
1677 1678 1679 1680 1681 1682 1683
    String query;
    {
        WriteBufferFromOwnString wb;
        wb << "SELECT DISTINCT " << queryToString(task_table.engine_push_partition_key_ast) << " AS partition FROM"
           << " " << getQuotedTable(task_shard.table_read_shard) << " ORDER BY partition DESC";
        query = wb.str();
    }
1684

N
Nikita Mikhaylov 已提交
1685
    ParserQuery parser_query(query.data() + query.size());
1686 1687
    const auto & settings = context.getSettingsRef();
    ASTPtr query_ast = parseQuery(parser_query, query, settings.max_query_size, settings.max_parser_depth);
N
Nikita Mikhaylov 已提交
1688

A
Alexey Milovidov 已提交
1689
    LOG_DEBUG(log, "Computing destination partition set, executing query: {}", query);
N
Nikita Mikhaylov 已提交
1690 1691 1692

    Context local_context = context;
    local_context.setSettings(task_cluster->settings_pull);
N
Nikolai Kochetov 已提交
1693
    Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_ast, local_context)->execute().getInputStream());
N
Nikita Mikhaylov 已提交
1694 1695 1696 1697 1698 1699 1700 1701 1702 1703 1704 1705 1706 1707 1708

    std::set<String> res;
    if (block)
    {
        ColumnWithTypeAndName & column = block.getByPosition(0);
        task_shard.partition_key_column = column;

        for (size_t i = 0; i < column.column->size(); ++i)
        {
            WriteBufferFromOwnString wb;
            column.type->serializeAsTextQuoted(*column.column, i, wb, FormatSettings());
            res.emplace(wb.str());
        }
    }

A
Alexey Milovidov 已提交
1709
    LOG_DEBUG(log, "There are {} destination partitions in shard {}", res.size(), task_shard.getDescription());
1710

N
Nikita Mikhaylov 已提交
1711 1712
    return res;
}
1713

N
Nikita Mikhaylov 已提交
1714 1715
bool ClusterCopier::checkShardHasPartition(const ConnectionTimeouts & timeouts,
        TaskShard & task_shard, const String & partition_quoted_name)
1716
{
N
Nikita Mikhaylov 已提交
1717 1718 1719 1720 1721
    createShardInternalTables(timeouts, task_shard, false);

    TaskTable & task_table = task_shard.task_table;

    std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard)
N
Nikita Mikhaylov 已提交
1722 1723
                        + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) +
                        " = (" + partition_quoted_name + " AS partition_key))";
N
Nikita Mikhaylov 已提交
1724 1725 1726 1727 1728 1729

    if (!task_table.where_condition_str.empty())
        query += " AND (" + task_table.where_condition_str + ")";

    query += " LIMIT 1";

A
Alexey Milovidov 已提交
1730
    LOG_DEBUG(log, "Checking shard {} for partition {} existence, executing query: {}", task_shard.getDescription(), partition_quoted_name, query);
N
Nikita Mikhaylov 已提交
1731 1732

    ParserQuery parser_query(query.data() + query.size());
1733 1734
const auto & settings = context.getSettingsRef();
    ASTPtr query_ast = parseQuery(parser_query, query, settings.max_query_size, settings.max_parser_depth);
N
Nikita Mikhaylov 已提交
1735 1736 1737 1738

    Context local_context = context;
    local_context.setSettings(task_cluster->settings_pull);
    return InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows() != 0;
1739
}
1740

N
Nikita Mikhaylov 已提交
1741
bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTimeouts & timeouts,
N
Nikita Mikhaylov 已提交
1742
                           TaskShard & task_shard, const String & partition_quoted_name, size_t current_piece_number)
N
Nikita Mikhaylov 已提交
1743 1744 1745 1746 1747 1748 1749
{
    createShardInternalTables(timeouts, task_shard, false);

    TaskTable & task_table = task_shard.task_table;
    const size_t number_of_splits = task_table.number_of_splits;
    const String & primary_key_comma_separated = task_table.primary_key_comma_separated;

N
Nikita Mikhaylov 已提交
1750 1751 1752 1753 1754 1755 1756 1757 1758 1759 1760 1761 1762
    UNUSED(primary_key_comma_separated);

    std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard);

    if (experimental_use_sample_offset)
        query += " SAMPLE 1/" + toString(number_of_splits) + " OFFSET " + toString(current_piece_number) + "/" + toString(number_of_splits);

    query += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast)
                        + " = (" + partition_quoted_name + " AS partition_key))";

    if (!experimental_use_sample_offset)
        query += " AND (cityHash64(" + primary_key_comma_separated + ") % "
                 + std::to_string(number_of_splits) + " = " + std::to_string(current_piece_number) + " )";
N
Nikita Mikhaylov 已提交
1763 1764 1765 1766 1767 1768

    if (!task_table.where_condition_str.empty())
        query += " AND (" + task_table.where_condition_str + ")";

    query += " LIMIT 1";

A
Alexey Milovidov 已提交
1769
    LOG_DEBUG(log, "Checking shard {} for partition {} piece {} existence, executing query: {}", task_shard.getDescription(), partition_quoted_name, std::to_string(current_piece_number), query);
N
Nikita Mikhaylov 已提交
1770 1771

    ParserQuery parser_query(query.data() + query.size());
1772 1773
    const auto & settings = context.getSettingsRef();
    ASTPtr query_ast = parseQuery(parser_query, query, settings.max_query_size, settings.max_parser_depth);
N
Nikita Mikhaylov 已提交
1774 1775 1776

    Context local_context = context;
    local_context.setSettings(task_cluster->settings_pull);
N
Nikolai Kochetov 已提交
1777
    auto result = InterpreterFactory::get(query_ast, local_context)->execute().getInputStream()->read().rows();
N
Nikita Mikhaylov 已提交
1778
    if (result != 0)
A
Alexey Milovidov 已提交
1779
        LOG_DEBUG(log, "Partition {} piece number {} is PRESENT on shard {}", partition_quoted_name, std::to_string(current_piece_number), task_shard.getDescription());
N
Nikita Mikhaylov 已提交
1780
    else
A
Alexey Milovidov 已提交
1781
        LOG_DEBUG(log, "Partition {} piece number {} is ABSENT on shard {}", partition_quoted_name, std::to_string(current_piece_number), task_shard.getDescription());
N
Nikita Mikhaylov 已提交
1782 1783
    return result != 0;
}
1784

N
Nikita Mikhaylov 已提交
1785 1786 1787
/** Executes simple query (without output streams, for example DDL queries) on each shard of the cluster
  * Returns number of shards for which at least one replica executed query successfully
  */
N
Nikita Mikhaylov 已提交
1788
UInt64 ClusterCopier::executeQueryOnCluster(
N
Nikita Mikhaylov 已提交
1789 1790
        const ClusterPtr & cluster,
        const String & query,
N
Nikita Mikhaylov 已提交
1791 1792 1793
        const ASTPtr & query_ast_,
        const Settings * settings,
        PoolMode pool_mode,
N
Nikita Mikhaylov 已提交
1794
        ClusterExecutionMode execution_mode,
N
Nikita Mikhaylov 已提交
1795
        UInt64 max_successful_executions_per_shard) const
1796
{
1797 1798
    Settings current_settings = settings ? *settings : task_cluster->settings_common;

N
Nikita Mikhaylov 已提交
1799 1800
    auto num_shards = cluster->getShardsInfo().size();
    std::vector<UInt64> per_shard_num_successful_replicas(num_shards, 0);
1801

N
Nikita Mikhaylov 已提交
1802 1803
    ASTPtr query_ast;
    if (query_ast_ == nullptr)
1804
    {
N
Nikita Mikhaylov 已提交
1805
        ParserQuery p_query(query.data() + query.size());
1806
        query_ast = parseQuery(p_query, query, current_settings.max_query_size, current_settings.max_parser_depth);
1807
    }
N
Nikita Mikhaylov 已提交
1808 1809 1810
    else
        query_ast = query_ast_;

N
Nikita Mikhaylov 已提交
1811 1812 1813 1814 1815
    /// We will have to execute query on each replica of a shard.
    if (execution_mode == ClusterExecutionMode::ON_EACH_NODE)
        max_successful_executions_per_shard = 0;

    std::atomic<size_t> origin_replicas_number;
N
Nikita Mikhaylov 已提交
1816 1817 1818

    /// We need to execute query on one replica at least
    auto do_for_shard = [&] (UInt64 shard_index)
1819
    {
N
Nikita Mikhaylov 已提交
1820 1821 1822
        const Cluster::ShardInfo & shard = cluster->getShardsInfo().at(shard_index);
        UInt64 & num_successful_executions = per_shard_num_successful_replicas.at(shard_index);
        num_successful_executions = 0;
1823

N
Nikita Mikhaylov 已提交
1824
        auto increment_and_check_exit = [&] () -> bool
N
Nikita Mikhaylov 已提交
1825 1826 1827 1828
        {
            ++num_successful_executions;
            return max_successful_executions_per_shard && num_successful_executions >= max_successful_executions_per_shard;
        };
1829

N
Nikita Mikhaylov 已提交
1830
        UInt64 num_replicas = cluster->getShardsAddresses().at(shard_index).size();
N
Nikita Mikhaylov 已提交
1831 1832

        origin_replicas_number += num_replicas;
N
Nikita Mikhaylov 已提交
1833 1834
        UInt64 num_local_replicas = shard.getLocalNodeCount();
        UInt64 num_remote_replicas = num_replicas - num_local_replicas;
1835

N
Nikita Mikhaylov 已提交
1836 1837 1838 1839 1840
        /// In that case we don't have local replicas, but do it just in case
        for (UInt64 i = 0; i < num_local_replicas; ++i)
        {
            auto interpreter = InterpreterFactory::get(query_ast, context);
            interpreter->execute();
1841

N
Nikita Mikhaylov 已提交
1842 1843 1844
            if (increment_and_check_exit())
                return;
        }
1845

N
Nikita Mikhaylov 已提交
1846 1847 1848 1849 1850 1851 1852 1853 1854 1855 1856 1857 1858 1859 1860 1861 1862 1863 1864 1865 1866 1867 1868 1869 1870
        /// Will try to make as many as possible queries
        if (shard.hasRemoteConnections())
        {
            current_settings.max_parallel_replicas = num_remote_replicas ? num_remote_replicas : 1;

            auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings).getSaturated(current_settings.max_execution_time);
            auto connections = shard.pool->getMany(timeouts, &current_settings, pool_mode);

            for (auto & connection : connections)
            {
                if (connection.isNull())
                    continue;

                try
                {
                    /// CREATE TABLE and DROP PARTITION queries return empty block
                    RemoteBlockInputStream stream{*connection, query, Block{}, context, &current_settings};
                    NullBlockOutputStream output{Block{}};
                    copyData(stream, output);

                    if (increment_and_check_exit())
                        return;
                }
                catch (const Exception &)
                {
A
Alexey Milovidov 已提交
1871
                    LOG_INFO(log, getCurrentExceptionMessage(false, true));
N
Nikita Mikhaylov 已提交
1872 1873 1874 1875
                }
            }
        }
    };
1876 1877

    {
N
Nikita Mikhaylov 已提交
1878 1879 1880 1881
        ThreadPool thread_pool(std::min<UInt64>(num_shards, getNumberOfPhysicalCPUCores()));

        for (UInt64 shard_index = 0; shard_index < num_shards; ++shard_index)
            thread_pool.scheduleOrThrowOnError([=] { do_for_shard(shard_index); });
1882

N
Nikita Mikhaylov 已提交
1883
        thread_pool.wait();
1884
    }
N
Nikita Mikhaylov 已提交
1885

N
Nikita Mikhaylov 已提交
1886
    UInt64 successful_nodes = 0;
N
Nikita Mikhaylov 已提交
1887
    for (UInt64 num_replicas : per_shard_num_successful_replicas)
N
Nikita Mikhaylov 已提交
1888 1889 1890 1891 1892 1893 1894
    {
        if (execution_mode == ClusterExecutionMode::ON_EACH_NODE)
            successful_nodes += num_replicas;
        else
            /// Count only successful shards
            successful_nodes += (num_replicas > 0);
    }
N
Nikita Mikhaylov 已提交
1895

N
Nikita Mikhaylov 已提交
1896
    if (execution_mode == ClusterExecutionMode::ON_EACH_NODE && successful_nodes != origin_replicas_number)
N
Nikita Mikhaylov 已提交
1897
    {
A
Alexey Milovidov 已提交
1898
        LOG_INFO(log, "There was an error while executing ALTER on each node. Query was executed on {} nodes. But had to be executed on {}", toString(successful_nodes), toString(origin_replicas_number.load()));
N
Nikita Mikhaylov 已提交
1899
    }
N
Nikita Mikhaylov 已提交
1900

N
Nikita Mikhaylov 已提交
1901

N
Nikita Mikhaylov 已提交
1902
    return successful_nodes;
N
Nikita Mikhaylov 已提交
1903
}
1904
}