StorageReplicatedMergeTree.cpp 186.4 KB
Newer Older
1 2
#include <Common/ZooKeeper/Types.h>
#include <Common/ZooKeeper/KeeperException.h>
3

4
#include <Common/FieldVisitors.h>
5 6 7

#include <Storages/ColumnsDescription.h>
#include <Storages/StorageReplicatedMergeTree.h>
8
#include <Storages/MergeTree/MergeTreeDataPart.h>
9 10
#include <Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h>
#include <Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h>
11
#include <Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h>
12 13 14 15 16 17
#include <Storages/MergeTree/MergeList.h>
#include <Storages/MergeTree/ReplicatedMergeTreeAddress.h>

#include <Databases/IDatabase.h>

#include <Parsers/formatAST.h>
18
#include <Parsers/ASTDropQuery.h>
19
#include <Parsers/ASTOptimizeQuery.h>
20
#include <Parsers/ASTLiteral.h>
21 22 23

#include <IO/ReadBufferFromString.h>
#include <IO/Operators.h>
24
#include <IO/ConnectionTimeouts.h>
25 26 27 28 29 30 31 32 33

#include <Interpreters/InterpreterAlterQuery.h>
#include <Interpreters/PartLog.h>

#include <DataStreams/RemoteBlockInputStream.h>
#include <DataStreams/NullBlockOutputStream.h>
#include <DataStreams/copyData.h>

#include <Common/Macros.h>
A
Alexey Milovidov 已提交
34
#include <Storages/VirtualColumnUtils.h>
35 36 37
#include <Common/formatReadable.h>
#include <Common/setThreadName.h>
#include <Common/escapeForFileName.h>
38
#include <Common/StringUtils/StringUtils.h>
39
#include <Common/typeid_cast.h>
40

41
#include <Poco/DirectoryIterator.h>
M
Merge  
Michael Kolupaev 已提交
42

P
proller 已提交
43
#include <common/ThreadPool.h>
A
Merge  
Alexey Milovidov 已提交
44

45 46
#include <ext/range.h>
#include <ext/scope_guard.h>
A
Alexey Milovidov 已提交
47

A
Merge  
Alexey Milovidov 已提交
48 49 50 51 52 53
#include <cfenv>
#include <ctime>
#include <thread>
#include <future>


54 55
namespace ProfileEvents
{
56
    extern const Event ReplicatedPartMerges;
57
    extern const Event ReplicatedPartMutations;
58 59 60 61
    extern const Event ReplicatedPartFailedFetches;
    extern const Event ReplicatedPartFetchesOfMerged;
    extern const Event ObsoleteReplicatedParts;
    extern const Event ReplicatedPartFetches;
62
    extern const Event DataAfterMergeDiffersFromReplica;
63
    extern const Event DataAfterMutationDiffersFromReplica;
64
}
65

66 67 68 69 70 71
namespace CurrentMetrics
{
    extern const Metric LeaderReplica;
}


M
Merge  
Michael Kolupaev 已提交
72 73 74
namespace DB
{

75 76
namespace ErrorCodes
{
77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93
    extern const int NO_ZOOKEEPER;
    extern const int INCORRECT_DATA;
    extern const int INCOMPATIBLE_COLUMNS;
    extern const int REPLICA_IS_ALREADY_EXIST;
    extern const int NO_SUCH_REPLICA;
    extern const int NO_REPLICA_HAS_PART;
    extern const int LOGICAL_ERROR;
    extern const int TOO_MANY_UNEXPECTED_DATA_PARTS;
    extern const int ABORTED;
    extern const int REPLICA_IS_NOT_IN_QUORUM;
    extern const int TABLE_IS_READ_ONLY;
    extern const int NOT_FOUND_NODE;
    extern const int NO_ACTIVE_REPLICAS;
    extern const int LEADERSHIP_CHANGED;
    extern const int TABLE_IS_READ_ONLY;
    extern const int TABLE_WAS_NOT_DROPPED;
    extern const int PARTITION_ALREADY_EXISTS;
A
Alexey Milovidov 已提交
94
    extern const int TOO_MANY_RETRIES_TO_FETCH_PARTS;
95 96 97 98
    extern const int RECEIVED_ERROR_FROM_REMOTE_IO_SERVER;
    extern const int PARTITION_DOESNT_EXIST;
    extern const int CHECKSUM_DOESNT_MATCH;
    extern const int BAD_SIZE_OF_FILE_IN_DATA_PART;
99 100
    extern const int UNEXPECTED_FILE_IN_DATA_PART;
    extern const int NO_FILE_IN_DATA_PART;
101 102
    extern const int UNFINISHED;
    extern const int METADATA_MISMATCH;
103
    extern const int RECEIVED_ERROR_TOO_MANY_REQUESTS;
A
Alexey Milovidov 已提交
104
    extern const int TOO_MANY_FETCHES;
105
    extern const int BAD_DATA_PART_NAME;
106
    extern const int PART_IS_TEMPORARILY_LOCKED;
107
    extern const int INCORRECT_FILE_NAME;
108
    extern const int CANNOT_ASSIGN_OPTIMIZE;
109
    extern const int KEEPER_EXCEPTION;
V
VadimPE 已提交
110
    extern const int ALL_REPLICAS_LOST;
V
VadimPE 已提交
111
    extern const int REPLICA_STATUS_CHANGED;
112 113 114 115 116 117 118 119
}

namespace ActionLocks
{
    extern const StorageActionBlockType PartsMerge;
    extern const StorageActionBlockType PartsFetch;
    extern const StorageActionBlockType PartsSend;
    extern const StorageActionBlockType ReplicationQueue;
120 121
}

M
Merge  
Michael Kolupaev 已提交
122

123 124
static const auto QUEUE_UPDATE_ERROR_SLEEP_MS     = 1 * 1000;
static const auto MERGE_SELECTING_SLEEP_MS        = 5 * 1000;
125
static const auto MUTATIONS_FINALIZING_SLEEP_MS   = 1 * 1000;
M
Merge  
Michael Kolupaev 已提交
126

F
f1yegor 已提交
127 128
/** There are three places for each part, where it should be
  * 1. In the RAM, MergeTreeData::data_parts, all_data_parts.
129
  * 2. In the filesystem (FS), the directory with the data of the table.
F
f1yegor 已提交
130
  * 3. in ZooKeeper (ZK).
131
  *
F
f1yegor 已提交
132 133
  * When adding a part, it must be added immediately to these three places.
  * This is done like this
134 135
  * - [FS] first write the part into a temporary directory on the filesystem;
  * - [FS] rename the temporary part to the result on the filesystem;
F
f1yegor 已提交
136 137 138 139
  * - [RAM] immediately afterwards add it to the `data_parts`, and remove from `data_parts` any parts covered by this one;
  * - [RAM] also set the `Transaction` object, which in case of an exception (in next point),
  *   rolls back the changes in `data_parts` (from the previous point) back;
  * - [ZK] then send a transaction (multi) to add a part to ZooKeeper (and some more actions);
140
  * - [FS, ZK] by the way, removing the covered (old) parts from filesystem, from ZooKeeper and from `all_data_parts`
F
f1yegor 已提交
141
  *   is delayed, after a few minutes.
142
  *
F
f1yegor 已提交
143 144 145
  * There is no atomicity here.
  * It could be possible to achieve atomicity using undo/redo logs and a flag in `DataPart` when it is completely ready.
  * But it would be inconvenient - I would have to write undo/redo logs for each `Part` in ZK, and this would increase already large number of interactions.
146
  *
F
f1yegor 已提交
147 148 149 150 151
  * Instead, we are forced to work in a situation where at any time
  *  (from another thread, or after server restart), there may be an unfinished transaction.
  *  (note - for this the part should be in RAM)
  * From these cases the most frequent one is when the part is already in the data_parts, but it's not yet in ZooKeeper.
  * This case must be distinguished from the case where such a situation is achieved due to some kind of damage to the state.
152
  *
F
f1yegor 已提交
153 154 155 156
  * Do this with the threshold for the time.
  * If the part is young enough, its lack in ZooKeeper will be perceived optimistically - as if it just did not have time to be added there
  *  - as if the transaction has not yet been executed, but will soon be executed.
  * And if the part is old, its absence in ZooKeeper will be perceived as an unfinished transaction that needs to be rolled back.
157
  *
F
f1yegor 已提交
158 159
  * PS. Perhaps it would be better to add a flag to the DataPart that a part is inserted into ZK.
  * But here it's too easy to get confused with the consistency of this flag.
160
  */
161
extern const int MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER = 5 * 60;
162 163


164
/** For randomized selection of replicas. */
P
proller 已提交
165 166 167 168 169
/// avoid error: non-local variable 'DB::rng' declared '__thread' needs dynamic initialization
#ifndef __APPLE__
thread_local
#endif
    pcg64 rng{randomSeed()};
170 171


A
Merge  
Alexey Milovidov 已提交
172 173
void StorageReplicatedMergeTree::setZooKeeper(zkutil::ZooKeeperPtr zookeeper)
{
174 175
    std::lock_guard<std::mutex> lock(current_zookeeper_mutex);
    current_zookeeper = zookeeper;
A
Merge  
Alexey Milovidov 已提交
176 177 178 179
}

zkutil::ZooKeeperPtr StorageReplicatedMergeTree::tryGetZooKeeper()
{
180 181
    std::lock_guard<std::mutex> lock(current_zookeeper_mutex);
    return current_zookeeper;
A
Merge  
Alexey Milovidov 已提交
182 183 184 185
}

zkutil::ZooKeeperPtr StorageReplicatedMergeTree::getZooKeeper()
{
186 187 188 189
    auto res = tryGetZooKeeper();
    if (!res)
        throw Exception("Cannot get ZooKeeper", ErrorCodes::NO_ZOOKEEPER);
    return res;
A
Merge  
Alexey Milovidov 已提交
190 191 192
}


M
Merge  
Michael Kolupaev 已提交
193
StorageReplicatedMergeTree::StorageReplicatedMergeTree(
194 195 196 197
    const String & zookeeper_path_,
    const String & replica_name_,
    bool attach,
    const String & path_, const String & database_name_, const String & name_,
198
    const ColumnsDescription & columns_,
199
    Context & context_,
200
    const ASTPtr & primary_expr_ast_,
201
    const ASTPtr & secondary_sorting_expr_list_,
202 203
    const String & date_column_name,
    const ASTPtr & partition_expr_ast_,
204 205
    const ASTPtr & sampling_expression_,
    const MergeTreeData::MergingParams & merging_params_,
206 207
    const MergeTreeSettings & settings_,
    bool has_force_restore_data_flag)
208
    : context(context_),
209
    database_name(database_name_),
210
    table_name(name_), full_path(path_ + escapeForFileName(table_name) + '/'),
C
CurtizJ 已提交
211 212
    zookeeper_path(context.getMacros()->expand(zookeeper_path_, database_name, table_name)),
    replica_name(context.getMacros()->expand(replica_name_, database_name, table_name)),
213 214
    data(database_name, table_name,
        full_path, columns_,
215
        context_, primary_expr_ast_, secondary_sorting_expr_list_, date_column_name, partition_expr_ast_,
216
        sampling_expression_, merging_params_,
A
Alexey Zatelepin 已提交
217
        settings_, true, attach,
A
Alexey Milovidov 已提交
218
        [this] (const std::string & name) { enqueuePartForCheck(name); }),
219
    reader(data), writer(data), merger_mutator(data, context.getBackgroundPool()), queue(*this),
220
    fetcher(data),
221
    cleanup_thread(*this), alter_thread(*this), part_check_thread(*this), restarting_thread(*this),
222
    log(&Logger::get(database_name + "." + table_name + " (StorageReplicatedMergeTree)"))
M
Merge  
Michael Kolupaev 已提交
223
{
224 225 226
    if (path_.empty())
        throw Exception("ReplicatedMergeTree storages require data path", ErrorCodes::INCORRECT_FILE_NAME);

227 228
    if (!zookeeper_path.empty() && zookeeper_path.back() == '/')
        zookeeper_path.resize(zookeeper_path.size() - 1);
229
    /// If zookeeper chroot prefix is used, path should start with '/', because chroot concatenates without it.
K
KochetovNicolai 已提交
230 231
    if (!zookeeper_path.empty() && zookeeper_path.front() != '/')
        zookeeper_path = "/" + zookeeper_path;
232 233
    replica_path = zookeeper_path + "/replicas/" + replica_name;

234 235 236 237 238 239 240 241
    queue_updating_task = context.getSchedulePool().createTask(database_name + "." + table_name + " (StorageReplicatedMergeTree::queueUpdatingTask)", [this]{ queueUpdatingTask(); });

    mutations_updating_task = context.getSchedulePool().createTask(database_name + "." + table_name + " (StorageReplicatedMergeTree::mutationsUpdatingTask)", [this]{ mutationsUpdatingTask(); });

    merge_selecting_task = context.getSchedulePool().createTask(database_name + "." + table_name + " (StorageReplicatedMergeTree::mergeSelectingTask)", [this] { mergeSelectingTask(); });
    /// Will be activated if we win leader election.
    merge_selecting_task->deactivate();

242 243
    mutations_finalizing_task = context.getSchedulePool().createTask(database_name + "." + table_name + " (StorageReplicatedMergeTree::mutationsFinalizingTask)", [this] { mutationsFinalizingTask(); });

244 245 246
    if (context.hasZooKeeper())
        current_zookeeper = context.getZooKeeper();

247 248
    bool skip_sanity_checks = false;

249
    if (current_zookeeper && current_zookeeper->exists(replica_path + "/flags/force_restore_data"))
250
    {
251 252
        skip_sanity_checks = true;
        current_zookeeper->remove(replica_path + "/flags/force_restore_data");
253

254 255
        LOG_WARNING(log, "Skipping the limits on severity of changes to data parts and columns (flag "
            << replica_path << "/flags/force_restore_data).");
256
    }
257
    else if (has_force_restore_data_flag)
258
    {
259 260 261
        skip_sanity_checks = true;

        LOG_WARNING(log, "Skipping the limits on severity of changes to data parts and columns (flag force_restore_data).");
262 263 264 265 266 267 268 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284 285 286 287 288 289 290
    }

    data.loadDataParts(skip_sanity_checks);

    if (!current_zookeeper)
    {
        if (!attach)
            throw Exception("Can't create replicated table without ZooKeeper", ErrorCodes::NO_ZOOKEEPER);

        /// Do not activate the replica. It will be readonly.
        LOG_ERROR(log, "No ZooKeeper: table will be in readonly mode.");
        is_readonly = true;
        return;
    }

    if (!attach)
    {
        if (!data.getDataParts().empty())
            throw Exception("Data directory for table already containing data parts - probably it was unclean DROP table or manual intervention. You must either clear directory by hand or use ATTACH TABLE instead of CREATE TABLE if you need to use that parts.", ErrorCodes::INCORRECT_DATA);

        createTableIfNotExists();

        checkTableStructure(false, false);
        createReplica();
    }
    else
    {
        checkTableStructure(skip_sanity_checks, true);
        checkParts(skip_sanity_checks);
291 292 293 294

        /// Temporary directories contain unfinalized results of Merges or Fetches (after forced restart)
        ///  and don't allow to reinitialize them, so delete each of them immediately
        data.clearOldTemporaryDirectories(0);
295 296 297
    }

    createNewZooKeeperNodes();
M
Merge  
Michael Kolupaev 已提交
298 299
}

300

301 302
void StorageReplicatedMergeTree::createNewZooKeeperNodes()
{
303
    auto zookeeper = getZooKeeper();
304

305
    /// Working with quorum.
306 307 308
    zookeeper->createIfNotExists(zookeeper_path + "/quorum", String());
    zookeeper->createIfNotExists(zookeeper_path + "/quorum/last_part", String());
    zookeeper->createIfNotExists(zookeeper_path + "/quorum/failed_parts", String());
309

310
    /// Tracking lag of replicas.
311 312 313 314 315
    zookeeper->createIfNotExists(replica_path + "/min_unprocessed_insert_time", String());
    zookeeper->createIfNotExists(replica_path + "/max_processed_insert_time", String());

    /// Mutations
    zookeeper->createIfNotExists(zookeeper_path + "/mutations", String());
316
    zookeeper->createIfNotExists(replica_path + "/mutation_pointer", String());
317 318 319
}


M
Merge  
Michael Kolupaev 已提交
320 321
static String formattedAST(const ASTPtr & ast)
{
322 323 324
    if (!ast)
        return "";
    std::stringstream ss;
A
Alexey Milovidov 已提交
325
    formatAST(*ast, ss, false, true);
326
    return ss.str();
M
Merge  
Michael Kolupaev 已提交
327
}
M
Merge  
Michael Kolupaev 已提交
328

A
Merge  
Alexey Milovidov 已提交
329

330 331
namespace
{
332 333 334 335 336 337 338
    /** The basic parameters of table engine for saving in ZooKeeper.
      * Lets you verify that they match local ones.
      */
    struct TableMetadata
    {
        const MergeTreeData & data;

339
        explicit TableMetadata(const MergeTreeData & data_)
340 341 342 343 344
            : data(data_) {}

        void write(WriteBuffer & out) const
        {
            out << "metadata format version: 1" << "\n"
345 346 347 348 349 350 351 352
                << "date column: ";

            if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
                out << data.minmax_idx_columns[data.minmax_idx_date_column_pos] << "\n";
            else
                out << "\n";

            out << "sampling expression: " << formattedAST(data.sampling_expression) << "\n"
353 354 355 356
                << "index granularity: " << data.index_granularity << "\n"
                << "mode: " << static_cast<int>(data.merging_params.mode) << "\n"
                << "sign column: " << data.merging_params.sign_column << "\n"
                << "primary key: " << formattedAST(data.primary_expr_ast) << "\n";
357 358 359 360 361 362

            if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
            {
                out << "data format version: " << data.format_version.toUnderType() << "\n";
                out << "partition key: " << formattedAST(data.partition_expr_ast) << "\n";
            }
363 364 365 366
        }

        String toString() const
        {
367
            WriteBufferFromOwnString out;
368
            write(out);
369
            return out.str();
370 371 372 373 374 375 376 377 378
        }

        void check(ReadBuffer & in) const
        {
            /// TODO Can be made less cumbersome.

            in >> "metadata format version: 1";

            in >> "\ndate column: ";
379 380 381 382 383 384 385 386 387 388 389 390 391 392 393
            String read_date_column;
            in >> read_date_column;

            if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
            {
                const String & local_date_column = data.minmax_idx_columns[data.minmax_idx_date_column_pos];
                if (local_date_column != read_date_column)
                    throw Exception("Existing table metadata in ZooKeeper differs in date index column."
                        " Stored in ZooKeeper: " + read_date_column + ", local: " + local_date_column,
                        ErrorCodes::METADATA_MISMATCH);
            }
            else if (!read_date_column.empty())
                throw Exception(
                    "Existing table metadata in ZooKeeper differs in date index column."
                    " Stored in ZooKeeper: " + read_date_column + ", local is custom-partitioned.",
394 395 396 397 398 399 400 401 402 403 404 405 406 407 408 409 410 411 412 413 414 415 416 417 418 419 420 421 422 423 424 425 426 427 428 429 430 431 432 433 434 435 436 437 438 439 440 441 442 443 444 445 446
                    ErrorCodes::METADATA_MISMATCH);

            in >> "\nsampling expression: ";
            String read_sample_expression;
            String local_sample_expression = formattedAST(data.sampling_expression);
            in >> read_sample_expression;

            if (read_sample_expression != local_sample_expression)
                throw Exception("Existing table metadata in ZooKeeper differs in sample expression."
                    " Stored in ZooKeeper: " + read_sample_expression + ", local: " + local_sample_expression,
                    ErrorCodes::METADATA_MISMATCH);

            in >> "\nindex granularity: ";
            size_t read_index_granularity = 0;
            in >> read_index_granularity;

            if (read_index_granularity != data.index_granularity)
                throw Exception("Existing table metadata in ZooKeeper differs in index granularity."
                    " Stored in ZooKeeper: " + DB::toString(read_index_granularity) + ", local: " + DB::toString(data.index_granularity),
                    ErrorCodes::METADATA_MISMATCH);

            in >> "\nmode: ";
            int read_mode = 0;
            in >> read_mode;

            if (read_mode != static_cast<int>(data.merging_params.mode))
                throw Exception("Existing table metadata in ZooKeeper differs in mode of merge operation."
                    " Stored in ZooKeeper: " + DB::toString(read_mode) + ", local: "
                    + DB::toString(static_cast<int>(data.merging_params.mode)),
                    ErrorCodes::METADATA_MISMATCH);

            in >> "\nsign column: ";
            String read_sign_column;
            in >> read_sign_column;

            if (read_sign_column != data.merging_params.sign_column)
                throw Exception("Existing table metadata in ZooKeeper differs in sign column."
                    " Stored in ZooKeeper: " + read_sign_column + ", local: " + data.merging_params.sign_column,
                    ErrorCodes::METADATA_MISMATCH);

            in >> "\nprimary key: ";
            String read_primary_key;
            String local_primary_key = formattedAST(data.primary_expr_ast);
            in >> read_primary_key;

            /// NOTE: You can make a less strict check of match expressions so that tables do not break from small changes
            ///    in formatAST code.
            if (read_primary_key != local_primary_key)
                throw Exception("Existing table metadata in ZooKeeper differs in primary key."
                    " Stored in ZooKeeper: " + read_primary_key + ", local: " + local_primary_key,
                    ErrorCodes::METADATA_MISMATCH);

            in >> "\n";
447 448 449 450 451 452 453 454 455 456 457 458 459 460 461 462 463 464 465 466 467 468 469 470 471 472 473 474 475 476 477
            MergeTreeDataFormatVersion read_data_format_version;
            if (in.eof())
                read_data_format_version = 0;
            else
            {
                in >> "data format version: ";
                in >> read_data_format_version.toUnderType();
            }

            if (read_data_format_version != data.format_version)
                throw Exception("Existing table metadata in ZooKeeper differs in data format version."
                    " Stored in ZooKeeper: " + DB::toString(read_data_format_version.toUnderType()) +
                    ", local: " + DB::toString(data.format_version.toUnderType()),
                    ErrorCodes::METADATA_MISMATCH);

            if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
            {
                in >> "\npartition key: ";
                String read_partition_key;
                String local_partition_key = formattedAST(data.partition_expr_ast);
                in >> read_partition_key;

                if (read_partition_key != local_partition_key)
                    throw Exception(
                        "Existing table metadata in ZooKeeper differs in partition key expression."
                        " Stored in ZooKeeper: " + read_partition_key + ", local: " + local_partition_key,
                        ErrorCodes::METADATA_MISMATCH);

                in >> "\n";
            }

478 479 480 481 482 483 484 485 486
            assertEOF(in);
        }

        void check(const String & s) const
        {
            ReadBufferFromString in(s);
            check(in);
        }
    };
487 488 489
}


M
Merge  
Michael Kolupaev 已提交
490
void StorageReplicatedMergeTree::createTableIfNotExists()
M
Merge  
Michael Kolupaev 已提交
491
{
492 493 494 495 496 497 498 499 500 501 502 503
    auto zookeeper = getZooKeeper();

    if (zookeeper->exists(zookeeper_path))
        return;

    LOG_DEBUG(log, "Creating table " << zookeeper_path);

    zookeeper->createAncestors(zookeeper_path);

    /// We write metadata of table so that the replicas can check table parameters with them.
    String metadata = TableMetadata(data).toString();

504
    Coordination::Requests ops;
505 506 507 508 509 510 511 512 513 514 515 516 517
    ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "",
        zkutil::CreateMode::Persistent));
    ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/metadata", metadata,
        zkutil::CreateMode::Persistent));
    ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/columns", getColumns().toString(),
        zkutil::CreateMode::Persistent));
    ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/log", "",
        zkutil::CreateMode::Persistent));
    ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/blocks", "",
        zkutil::CreateMode::Persistent));
    ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/block_numbers", "",
        zkutil::CreateMode::Persistent));
    ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/nonincrement_block_numbers", "",
518
        zkutil::CreateMode::Persistent)); /// /nonincrement_block_numbers dir is unused, but is created nonetheless for backwards compatibility.
519 520 521 522 523 524
    ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/leader_election", "",
        zkutil::CreateMode::Persistent));
    ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/temp", "",
        zkutil::CreateMode::Persistent));
    ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/replicas", "",
        zkutil::CreateMode::Persistent));
525

526
    Coordination::Responses responses;
527
    auto code = zookeeper->tryMulti(ops, responses);
528 529
    if (code && code != Coordination::ZNODEEXISTS)
        throw Coordination::Exception(code);
M
Merge  
Michael Kolupaev 已提交
530
}
M
Merge  
Michael Kolupaev 已提交
531

A
Merge  
Alexey Milovidov 已提交
532

F
f1yegor 已提交
533
/** Verify that list of columns and table settings match those specified in ZK (/ metadata).
534 535
    * If not, throw an exception.
    */
M
Merge  
Michael Kolupaev 已提交
536
void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bool allow_alter)
M
Merge  
Michael Kolupaev 已提交
537
{
538 539 540 541 542
    auto zookeeper = getZooKeeper();

    String metadata_str = zookeeper->get(zookeeper_path + "/metadata");
    TableMetadata(data).check(metadata_str);

543
    Coordination::Stat stat;
544
    auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(zookeeper_path + "/columns", &stat));
545 546
    columns_version = stat.version;

547 548
    const ColumnsDescription & old_columns = getColumns();
    if (columns_from_zk != old_columns)
549 550 551
    {
        if (allow_alter &&
            (skip_sanity_checks ||
552 553
             old_columns.ordinary.sizeOfDifference(columns_from_zk.ordinary) +
             old_columns.materialized.sizeOfDifference(columns_from_zk.materialized) <= 2))
554 555 556 557
        {
            LOG_WARNING(log, "Table structure in ZooKeeper is a little different from local table structure. Assuming ALTER.");

            /// Without any locks, because table has not been created yet.
558 559
            context.getDatabase(database_name)->alterTable(context, table_name, columns_from_zk, {});

560
            setColumns(std::move(columns_from_zk));
561 562 563
        }
        else
        {
564
            throw Exception("Table structure in ZooKeeper is too different from local table structure",
565 566 567
                            ErrorCodes::INCOMPATIBLE_COLUMNS);
        }
    }
M
Merge  
Michael Kolupaev 已提交
568
}
M
Merge  
Michael Kolupaev 已提交
569

A
Merge  
Alexey Milovidov 已提交
570

F
f1yegor 已提交
571 572 573 574
/** If necessary, restore a part, replica itself adds a record for its receipt.
  * What time should I put for this entry in the queue? Time is taken into account when calculating lag of replica.
  * For these purposes, it makes sense to use creation time of missing part
  *  (that is, in calculating lag, it will be taken into account how old is the part we need to recover).
575 576 577
  */
static time_t tryGetPartCreateTime(zkutil::ZooKeeperPtr & zookeeper, const String & replica_path, const String & part_name)
{
578
    time_t res = 0;
579

580
    /// We get creation time of part, if it still exists (was not merged, for example).
581
    Coordination::Stat stat;
582 583 584
    String unused;
    if (zookeeper->tryGet(replica_path + "/parts/" + part_name, unused, &stat))
        res = stat.ctime / 1000;
585

586
    return res;
587 588 589
}


M
Merge  
Michael Kolupaev 已提交
590 591
void StorageReplicatedMergeTree::createReplica()
{
592 593 594
    auto zookeeper = getZooKeeper();

    LOG_DEBUG(log, "Creating replica " << replica_path);
A
Alexey Milovidov 已提交
595

596
    int32_t code;
A
Alexey Milovidov 已提交
597

598
    do
599
    {
V
VadimPE 已提交
600
        Coordination::Stat replicas_stat;
601
        String last_added_replica = zookeeper->get(zookeeper_path + "/replicas", &replicas_stat);
A
Alexey Milovidov 已提交
602

A
Alexey Milovidov 已提交
603 604
        /// If it is not the first replica, we will mark it as "lost", to immediately repair (clone) from existing replica.
        String is_lost_value = last_added_replica.empty() ? "0" : "1";
A
Alexey Milovidov 已提交
605

V
VadimPE 已提交
606 607
        Coordination::Requests ops;
        Coordination::Responses resps;
608 609 610 611 612 613 614 615
        ops.emplace_back(zkutil::makeCreateRequest(replica_path, "", zkutil::CreateMode::Persistent));
        ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/host", "", zkutil::CreateMode::Persistent));
        ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/log_pointer", "", zkutil::CreateMode::Persistent));
        ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/queue", "", zkutil::CreateMode::Persistent));
        ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/parts", "", zkutil::CreateMode::Persistent));
        ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/flags", "", zkutil::CreateMode::Persistent));
        ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/is_lost", is_lost_value, zkutil::CreateMode::Persistent));
        ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/columns", getColumns().toString(), zkutil::CreateMode::Persistent));
A
Alexey Milovidov 已提交
616
        /// Check version of /replicas to see if there are any replicas created at the same moment of time.
V
VadimPE 已提交
617
        ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/replicas", "last added replica: " + replica_name, replicas_stat.version));
618 619

        code = zookeeper->tryMulti(ops, resps);
V
VadimPE 已提交
620
        if (code == Coordination::Error::ZNODEEXISTS)
621
            throw Exception("Replica " + replica_path + " already exists.", ErrorCodes::REPLICA_IS_ALREADY_EXIST);
V
VadimPE 已提交
622
        else if (code == Coordination::Error::ZBADVERSION)
A
Alexey Milovidov 已提交
623
            LOG_ERROR(log, "Retrying createReplica(), because some other replicas were created at the same time");
V
VadimPE 已提交
624
        else
V
Vadim 已提交
625
            zkutil::KeeperMultiException::check(code, ops, resps);
V
VadimPE 已提交
626
    } while (code == Coordination::Error::ZBADVERSION);
M
Merge  
Michael Kolupaev 已提交
627
}
M
Merge  
Michael Kolupaev 已提交
628 629


M
Merge  
Michael Kolupaev 已提交
630
void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
M
Merge  
Michael Kolupaev 已提交
631
{
632 633 634 635 636 637 638
    auto zookeeper = getZooKeeper();

    Strings expected_parts_vec = zookeeper->getChildren(replica_path + "/parts");

    /// Parts in ZK.
    NameSet expected_parts(expected_parts_vec.begin(), expected_parts_vec.end());

639 640
    /// There are no PreCommitted parts at startup.
    auto parts = data.getDataParts({MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated});
641 642 643 644 645 646 647 648 649 650 651 652 653 654

    /// Local parts that are not in ZK.
    MergeTreeData::DataParts unexpected_parts;

    for (const auto & part : parts)
    {
        if (expected_parts.count(part->name))
            expected_parts.erase(part->name);
        else
            unexpected_parts.insert(part);
    }

    /// Which local parts to added into ZK.
    MergeTreeData::DataPartsVector parts_to_add;
655
    UInt64 parts_to_add_rows = 0;
656 657 658 659 660 661 662 663 664 665 666 667 668 669 670

    /// Which parts should be taken from other replicas.
    Strings parts_to_fetch;

    for (const String & missing_name : expected_parts)
    {
        /// If locally some part is missing, but there is a part covering it, you can replace it in ZK with the covering one.
        auto containing = data.getActiveContainingPart(missing_name);
        if (containing)
        {
            LOG_ERROR(log, "Ignoring missing local part " << missing_name << " because part " << containing->name << " exists");
            if (unexpected_parts.count(containing))
            {
                parts_to_add.push_back(containing);
                unexpected_parts.erase(containing);
671
                parts_to_add_rows += containing->rows_count;
672 673 674 675 676 677 678 679 680 681 682 683
            }
        }
        else
        {
            LOG_ERROR(log, "Fetching missing part " << missing_name);
            parts_to_fetch.push_back(missing_name);
        }
    }

    for (const String & name : parts_to_fetch)
        expected_parts.erase(name);

684

685 686 687 688 689
    /** To check the adequacy, for the parts that are in the FS, but not in ZK, we will only consider not the most recent parts.
      * Because unexpected new parts usually arise only because they did not have time to enroll in ZK with a rough restart of the server.
      * It also occurs from deduplicated parts that did not have time to retire.
      */
    size_t unexpected_parts_nonnew = 0;
690 691
    UInt64 unexpected_parts_nonnew_rows = 0;
    UInt64 unexpected_parts_rows = 0;
692
    for (const auto & part : unexpected_parts)
693
    {
694
        if (part->info.level > 0)
695
        {
696
            ++unexpected_parts_nonnew;
697 698 699 700 701 702 703 704 705 706 707 708 709 710 711 712 713 714 715 716 717 718 719 720 721 722 723 724 725 726 727 728
            unexpected_parts_nonnew_rows += part->rows_count;
        }

        unexpected_parts_rows += part->rows_count;
    }

    /// Additional helpful statistics
    auto get_blocks_count_in_data_part = [&] (const String & part_name) -> UInt64
    {
        MergeTreePartInfo part_info;
        if (MergeTreePartInfo::tryParsePartName(part_name, &part_info, data.format_version))
            return part_info.getBlocksCount();

        LOG_ERROR(log, "Unexpected part name: " << part_name);
        return 0;
    };

    UInt64 parts_to_fetch_blocks = 0;
    for (const String & name : parts_to_fetch)
        parts_to_fetch_blocks += get_blocks_count_in_data_part(name);

    UInt64 expected_parts_blocks = 0;
    for (const String & name : expected_parts)
        expected_parts_blocks += get_blocks_count_in_data_part(name);

    std::stringstream sanity_report;
    sanity_report << "There are "
        << unexpected_parts.size() << " unexpected parts with " << unexpected_parts_rows << " rows ("
        << unexpected_parts_nonnew << " of them is not just-written with " << unexpected_parts_rows << " rows), "
        << parts_to_add.size() << " unexpectedly merged parts with " << parts_to_add_rows << " rows, "
        << expected_parts.size() << " missing obsolete parts (with " << expected_parts_blocks << " blocks), "
        << parts_to_fetch.size() << " missing parts (with " << parts_to_fetch_blocks << " blocks).";
729

730 731 732
    /** We can automatically synchronize data,
      *  if the ratio of the total number of errors to the total number of parts (minimum - on the local filesystem or in ZK)
      *  is no more than some threshold (for example 50%).
733
      *
734 735
      * A large ratio of mismatches in the data on the filesystem and the expected data
      *  may indicate a configuration error (the server accidentally connected as a replica not from right shard).
736 737 738
      * In this case, the protection mechanism does not allow the server to start.
      */

739 740 741 742 743 744
    UInt64 total_rows_on_filesystem = 0;
    for (const auto & part : parts)
        total_rows_on_filesystem += part->rows_count;

    UInt64 total_suspicious_rows = parts_to_add_rows + unexpected_parts_rows;
    UInt64 total_suspicious_rows_no_new = parts_to_add_rows + unexpected_parts_nonnew_rows;
745

746
    bool insane = total_suspicious_rows > total_rows_on_filesystem * data.settings.replicated_max_ratio_of_wrong_parts;
747 748

    if (insane && !skip_sanity_checks)
749 750 751 752 753 754 755 756 757
    {
        std::stringstream why;
        why << "The local set of parts of table " << database_name  << "." << table_name << " doesn't look like the set of parts "
            << "in ZooKeeper: "
            << formatReadableQuantity(total_suspicious_rows) << " rows of " << formatReadableQuantity(total_rows_on_filesystem)
            << " total rows in filesystem are suspicious.";

        throw Exception(why.str() + " " + sanity_report.str(), ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS);
    }
758

759 760
    if (total_suspicious_rows_no_new > 0)
        LOG_WARNING(log, sanity_report.str());
761 762 763 764 765 766

    /// Add information to the ZK about the parts that cover the missing parts.
    for (const MergeTreeData::DataPartPtr & part : parts_to_add)
    {
        LOG_ERROR(log, "Adding unexpected local part to ZooKeeper: " << part->name);

767
        Coordination::Requests ops;
768
        checkPartChecksumsAndAddCommitOps(zookeeper, part, ops);
769 770 771 772 773
        zookeeper->multi(ops);
    }

    /// Remove from ZK information about the parts covered by the newly added ones.
    {
774 775
        for (const String & name : expected_parts)
            LOG_ERROR(log, "Removing unexpectedly merged local part from ZooKeeper: " << name);
776

777
        removePartsFromZooKeeper(zookeeper, Strings(expected_parts.begin(), expected_parts.end()));
778 779 780 781 782 783 784 785 786 787 788 789 790 791
    }

    /// Add to the queue job to pick up the missing parts from other replicas and remove from ZK the information that we have them.
    for (const String & name : parts_to_fetch)
    {
        LOG_ERROR(log, "Removing missing part from ZooKeeper and queueing a fetch: " << name);

        LogEntry log_entry;
        log_entry.type = LogEntry::GET_PART;
        log_entry.source_replica = "";
        log_entry.new_part_name = name;
        log_entry.create_time = tryGetPartCreateTime(zookeeper, replica_path, name);

        /// We assume that this occurs before the queue is loaded (queue.initialize).
792
        Coordination::Requests ops;
793
        removePartFromZooKeeper(name, ops);
794 795
        ops.emplace_back(zkutil::makeCreateRequest(
            replica_path + "/queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential));
796 797 798 799 800 801 802
        zookeeper->multi(ops);
    }

    /// Remove extra local parts.
    for (const MergeTreeData::DataPartPtr & part : unexpected_parts)
    {
        LOG_ERROR(log, "Renaming unexpected part " << part->name << " to ignored_" + part->name);
803
        data.forgetPartAndMoveToDetached(part, "ignored_", true);
804
    }
M
Merge  
Michael Kolupaev 已提交
805
}
M
Merge  
Michael Kolupaev 已提交
806

A
Merge  
Alexey Milovidov 已提交
807

808
void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil::ZooKeeperPtr & zookeeper,
809
    const MergeTreeData::DataPartPtr & part, Coordination::Requests & ops, String part_name, NameSet * absent_replicas_paths)
M
Merge  
Michael Kolupaev 已提交
810
{
811 812 813 814 815 816 817
    if (part_name.empty())
        part_name = part->name;

    check(part->columns);
    int expected_columns_version = columns_version;

    Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
818
    std::shuffle(replicas.begin(), replicas.end(), rng);
819
    String expected_columns_str = part->columns.toString();
820
    bool has_been_alredy_added = false;
821 822 823

    for (const String & replica : replicas)
    {
824
        Coordination::Stat stat_before, stat_after;
825 826
        String current_part_path = zookeeper_path + "/replicas/" + replica + "/parts/" + part_name;

827
        String columns_str;
828 829 830 831 832
        if (!zookeeper->tryGet(current_part_path + "/columns", columns_str, &stat_before))
        {
            if (absent_replicas_paths)
                absent_replicas_paths->emplace(current_part_path);

833
            continue;
834 835
        }

836 837 838 839 840 841
        if (columns_str != expected_columns_str)
        {
            LOG_INFO(log, "Not checking checksums of part " << part_name << " with replica " << replica
                << " because columns are different");
            continue;
        }
842

843 844 845
        String checksums_str;
        /// Let's check that the node's version with the columns did not change while we were reading the checksums.
        /// This ensures that the columns and the checksum refer to the same data.
846 847
        if (!zookeeper->tryGet(current_part_path + "/checksums", checksums_str) ||
            !zookeeper->exists(current_part_path + "/columns", &stat_after) ||
848 849 850 851 852 853 854
            stat_before.version != stat_after.version)
        {
            LOG_INFO(log, "Not checking checksums of part " << part_name << " with replica " << replica
                << " because part changed while we were reading its checksums");
            continue;
        }

855 856 857 858 859
        auto zk_checksums = MinimalisticDataPartChecksums::deserializeFrom(checksums_str);
        zk_checksums.checkEqual(part->checksums, true);

        if (replica == replica_name)
            has_been_alredy_added = true;
860 861 862 863 864 865 866 867

        /// If we verify checksums in "sequential manner" (i.e. recheck absence of checksums on other replicas when commit)
        /// then it is enough to verify checksums on at least one replica since checksums on other replicas must be the same.
        if (absent_replicas_paths)
        {
            absent_replicas_paths->clear();
            break;
        }
868 869
    }

870
    if (!has_been_alredy_added)
871
    {
872 873
        String part_path = replica_path + "/parts/" + part_name;

874
        ops.emplace_back(zkutil::makeCheckRequest(
875
            zookeeper_path + "/columns", expected_columns_version));
876 877 878 879 880 881
        ops.emplace_back(zkutil::makeCreateRequest(
            part_path, "", zkutil::CreateMode::Persistent));
        ops.emplace_back(zkutil::makeCreateRequest(
            part_path + "/columns", part->columns.toString(), zkutil::CreateMode::Persistent));
        ops.emplace_back(zkutil::makeCreateRequest(
            part_path + "/checksums", getChecksumsForZooKeeper(part->checksums), zkutil::CreateMode::Persistent));
882 883 884 885 886
    }
    else
    {
        LOG_WARNING(log, "checkPartAndAddToZooKeeper: node " << replica_path + "/parts/" + part_name << " already exists."
            << " Will not commit any nodes.");
887
    }
888
}
889

890 891 892 893
MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAndCommit(MergeTreeData::Transaction & transaction,
    const MergeTreeData::DataPartPtr & part)
{
    auto zookeeper = getZooKeeper();
894

895 896
    while (true)
    {
897
        Coordination::Requests ops;
898 899 900 901 902 903 904 905 906 907 908 909
        NameSet absent_part_paths_on_replicas;

        /// Checksums are checked here and `ops` is filled. In fact, the part is added to ZK just below, when executing `multi`.
        checkPartChecksumsAndAddCommitOps(zookeeper, part, ops, part->name, &absent_part_paths_on_replicas);

        /// Do not commit if the part is obsolete, we have just briefly checked its checksums
        if (transaction.isEmpty())
            return {};

        /// Will check that the part did not suddenly appear on skipped replicas
        if (!absent_part_paths_on_replicas.empty())
        {
910
            Coordination::Requests new_ops;
911 912
            for (const String & part_path : absent_part_paths_on_replicas)
            {
913 914
                new_ops.emplace_back(zkutil::makeCreateRequest(part_path, "", zkutil::CreateMode::Persistent));
                new_ops.emplace_back(zkutil::makeRemoveRequest(part_path, -1));
915 916 917 918 919 920 921 922 923 924 925 926
            }

            /// Add check ops at the beginning
            new_ops.insert(new_ops.end(), ops.begin(), ops.end());
            ops = std::move(new_ops);
        }

        try
        {
            zookeeper->multi(ops);
            return transaction.commit();
        }
927
        catch (const zkutil::KeeperMultiException & e)
928 929
        {
            size_t num_check_ops = 2 * absent_part_paths_on_replicas.size();
930
            size_t failed_op_index = e.failed_op_index;
931

932
            if (failed_op_index < num_check_ops && e.code == Coordination::ZNODEEXISTS)
933
            {
934
                LOG_INFO(log, "The part " << e.getPathForFirstFailedOp() << " on a replica suddenly appeared, will recheck checksums");
935 936 937 938 939 940 941
            }
            else
                throw;
        }
    }
}

942
String StorageReplicatedMergeTree::getChecksumsForZooKeeper(const MergeTreeDataPartChecksums & checksums) const
943 944 945
{
    return MinimalisticDataPartChecksums::getSerializedString(checksums,
                                                              static_cast<bool>(data.settings.use_minimalistic_checksums_in_zookeeper));
M
Merge  
Michael Kolupaev 已提交
946 947
}

A
Merge  
Alexey Milovidov 已提交
948

949
bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
950
{
951 952 953 954 955 956
    if (entry.type == LogEntry::DROP_RANGE)
    {
        executeDropRange(entry);
        return true;
    }

957
    if (entry.type == LogEntry::CLEAR_COLUMN)
958
    {
959
        executeClearColumnInPartition(entry);
960 961 962
        return true;
    }

963 964 965 966 967 968
    if (entry.type == LogEntry::REPLACE_RANGE)
    {
        executeReplaceRange(entry);
        return true;
    }

969
    if (entry.type == LogEntry::GET_PART ||
970 971
        entry.type == LogEntry::MERGE_PARTS ||
        entry.type == LogEntry::MUTATE_PART)
972 973
    {
        /// If we already have this part or a part covering it, we do not need to do anything.
A
Alexey Zatelepin 已提交
974 975
        /// The part may be still in the PreCommitted -> Committed transition so we first search
        /// among PreCommitted parts to definitely find the desired part if it exists.
976 977 978
        MergeTreeData::DataPartPtr existing_part = data.getPartIfExists(entry.new_part_name, {MergeTreeDataPartState::PreCommitted});
        if (!existing_part)
            existing_part = data.getActiveContainingPart(entry.new_part_name);
979 980

        /// Even if the part is locally, it (in exceptional cases) may not be in ZooKeeper. Let's check that it is there.
981
        if (existing_part && getZooKeeper()->exists(replica_path + "/parts/" + existing_part->name))
982 983
        {
            if (!(entry.type == LogEntry::GET_PART && entry.source_replica == replica_name))
984 985 986
            {
                LOG_DEBUG(log, "Skipping action for part " << entry.new_part_name << " because part " + existing_part->name + " already exists.");
            }
987 988 989 990 991 992 993 994 995 996 997 998 999 1000 1001 1002 1003 1004 1005 1006
            return true;
        }
    }

    if (entry.type == LogEntry::GET_PART && entry.source_replica == replica_name)
        LOG_WARNING(log, "Part " << entry.new_part_name << " from own log doesn't exist.");

    /// Perhaps we don't need this part, because during write with quorum, the quorum has failed (see below about `/quorum/failed_parts`).
    if (entry.quorum && getZooKeeper()->exists(zookeeper_path + "/quorum/failed_parts/" + entry.new_part_name))
    {
        LOG_DEBUG(log, "Skipping action for part " << entry.new_part_name << " because quorum for that part was failed.");
        return true;    /// NOTE Deletion from `virtual_parts` is not done, but it is only necessary for merge.
    }

    bool do_fetch = false;
    if (entry.type == LogEntry::GET_PART)
    {
        do_fetch = true;
    }
    else if (entry.type == LogEntry::MERGE_PARTS)
1007
    {
1008 1009 1010 1011 1012
        do_fetch = !tryExecuteMerge(entry);
    }
    else if (entry.type == LogEntry::MUTATE_PART)
    {
        do_fetch = !tryExecutePartMutation(entry);
1013 1014 1015 1016 1017 1018 1019 1020 1021 1022 1023 1024 1025
    }
    else
    {
        throw Exception("Unexpected log entry type: " + toString(static_cast<int>(entry.type)));
    }

    if (do_fetch)
        return executeFetch(entry);

    return true;
}


1026 1027 1028 1029 1030 1031 1032 1033 1034 1035 1036 1037 1038 1039 1040 1041 1042 1043 1044 1045 1046 1047 1048 1049 1050 1051 1052 1053 1054 1055 1056 1057 1058 1059 1060 1061 1062 1063 1064 1065 1066 1067 1068 1069 1070 1071 1072 1073 1074 1075 1076 1077 1078 1079 1080 1081
void StorageReplicatedMergeTree::writePartLog(
    PartLogElement::Type type, const ExecutionStatus & execution_status, UInt64 elapsed_ns,
    const String & new_part_name,
    const MergeTreeData::DataPartPtr & result_part,
    const MergeTreeData::DataPartsVector & source_parts,
    const MergeListEntry * merge_entry) const
{
    try
    {
        auto part_log = context.getPartLog(database_name);
        if (!part_log)
            return;

        PartLogElement part_log_elem;

        part_log_elem.event_type = type;

        part_log_elem.error = static_cast<UInt16>(execution_status.code);
        part_log_elem.exception = execution_status.message;

        part_log_elem.event_time = time(nullptr);
        /// TODO: Stop stopwatch in outer code to exclude ZK timings and so on
        part_log_elem.duration_ms = elapsed_ns / 10000000;

        part_log_elem.database_name = database_name;
        part_log_elem.table_name = table_name;
        part_log_elem.part_name = new_part_name;

        if (result_part)
        {
            part_log_elem.bytes_compressed_on_disk = result_part->bytes_on_disk;
            part_log_elem.rows = result_part->rows_count;
        }

        part_log_elem.source_part_names.reserve(source_parts.size());
        for (const auto & source_part : source_parts)
            part_log_elem.source_part_names.push_back(source_part->name);

        if (merge_entry)
        {
            part_log_elem.rows_read = (*merge_entry)->bytes_read_uncompressed;
            part_log_elem.bytes_read_uncompressed = (*merge_entry)->bytes_read_uncompressed;

            part_log_elem.rows = (*merge_entry)->rows_written;
            part_log_elem.bytes_uncompressed = (*merge_entry)->bytes_written_uncompressed;
        }

        part_log->add(part_log_elem);
    }
    catch (...)
    {
        tryLogCurrentException(log, __PRETTY_FUNCTION__);
    }
}


1082
bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry)
1083 1084
{
    // Log source part names just in case
1085 1086 1087
    {
        std::stringstream log_message;
        log_message << "Executing log entry to merge parts ";
1088 1089
        for (auto i : ext::range(0, entry.source_parts.size()))
            log_message << (i != 0 ? ", " : "") << entry.source_parts[i];
1090 1091 1092
        log_message << " to " << entry.new_part_name;

        LOG_TRACE(log, log_message.rdbuf());
1093
    }
1094

1095 1096
    MergeTreeData::DataPartsVector parts;
    bool have_all_parts = true;
1097
    for (const String & name : entry.source_parts)
1098 1099 1100
    {
        MergeTreeData::DataPartPtr part = data.getActiveContainingPart(name);
        if (!part)
1101
        {
1102 1103
            have_all_parts = false;
            break;
1104
        }
1105
        if (part->name != name)
1106
        {
1107 1108 1109 1110
            LOG_WARNING(log, "Part " << name << " is covered by " << part->name
                << " but should be merged into " << entry.new_part_name << ". This shouldn't happen often.");
            have_all_parts = false;
            break;
1111
        }
1112 1113 1114 1115 1116 1117 1118
        parts.push_back(part);
    }

    if (!have_all_parts)
    {
        /// If you do not have all the necessary parts, try to take some already merged part from someone.
        LOG_DEBUG(log, "Don't have all parts for merge " << entry.new_part_name << "; will try to fetch it instead");
1119
        return false;
1120 1121 1122 1123 1124
    }
    else if (entry.create_time + data.settings.prefer_fetch_merged_part_time_threshold.totalSeconds() <= time(nullptr))
    {
        /// If entry is old enough, and have enough size, and part are exists in any replica,
        ///  then prefer fetching of merged part from replica.
1125

1126
        size_t sum_parts_bytes_on_disk = 0;
1127
        for (const auto & part : parts)
1128
            sum_parts_bytes_on_disk += part->bytes_on_disk;
1129

1130
        if (sum_parts_bytes_on_disk >= data.settings.prefer_fetch_merged_part_size_threshold)
1131 1132 1133
        {
            String replica = findReplicaHavingPart(entry.new_part_name, true);    /// NOTE excessive ZK requests for same data later, may remove.
            if (!replica.empty())
1134
            {
1135
                LOG_DEBUG(log, "Prefer to fetch " << entry.new_part_name << " from replica " << replica);
1136
                return false;
1137 1138
            }
        }
1139
    }
1140

1141
    /// Start to make the main work
1142

1143
    size_t estimated_space_for_merge = MergeTreeDataMergerMutator::estimateNeededDiskSpace(parts);
1144

1145 1146
    /// Can throw an exception.
    DiskSpaceMonitor::ReservationPtr reserved_space = DiskSpaceMonitor::reserve(full_path, estimated_space_for_merge);
1147

1148
    auto table_lock = lockStructure(false, __PRETTY_FUNCTION__);
1149

1150
    MergeList::EntryPtr merge_entry = context.getMergeList().insert(database_name, table_name, entry.new_part_name, parts);
1151

1152
    MergeTreeDataMergerMutator::FuturePart future_merged_part(parts);
1153 1154 1155 1156 1157
    if (future_merged_part.name != entry.new_part_name)
    {
        throw Exception("Future merged part name `" + future_merged_part.name +  "` differs from part name in log entry: `"
                        + entry.new_part_name + "`", ErrorCodes::BAD_DATA_PART_NAME);
    }
1158

1159
    MergeTreeData::Transaction transaction(data);
1160
    MergeTreeData::MutableDataPartPtr part;
1161

1162
    Stopwatch stopwatch;
1163

1164 1165
    auto write_part_log = [&] (const ExecutionStatus & execution_status)
    {
1166 1167 1168
        writePartLog(
            PartLogElement::MERGE_PARTS, execution_status, stopwatch.elapsed(),
            entry.new_part_name, part, parts, merge_entry.get());
1169
    };
1170

1171 1172
    try
    {
1173
        part = merger_mutator.mergePartsToTemporaryPart(
1174
            future_merged_part, *merge_entry, entry.create_time, reserved_space.get(), entry.deduplicate);
1175

1176
        merger_mutator.renameMergedTemporaryPart(part, parts, &transaction);
1177

1178 1179
        try
        {
1180
            checkPartChecksumsAndCommit(transaction, part);
1181 1182 1183
        }
        catch (const Exception & e)
        {
1184
            if (MergeTreeDataPartChecksums::isBadChecksumsErrorCode(e.code()))
1185
            {
1186
                transaction.rollback();
1187 1188 1189 1190 1191 1192 1193 1194 1195 1196 1197 1198 1199 1200 1201 1202 1203

                ProfileEvents::increment(ProfileEvents::DataAfterMergeDiffersFromReplica);

                LOG_ERROR(log, getCurrentExceptionMessage(false) << ". "
                    "Data after merge is not byte-identical to data on another replicas. "
                    "There could be several reasons: "
                    "1. Using newer version of compression library after server update. "
                    "2. Using another compression method. "
                    "3. Non-deterministic compression algorithm (highly unlikely). "
                    "4. Non-deterministic merge algorithm due to logical error in code. "
                    "5. Data corruption in memory due to bug in code. "
                    "6. Data corruption in memory due to hardware issue. "
                    "7. Manual modification of source data after server startup. "
                    "8. Manual modification of checksums stored in ZooKeeper. "
                    "We will download merged part from replica to force byte-identical result.");

                write_part_log(ExecutionStatus::fromCurrentException());
1204

1205 1206 1207 1208
                data.tryRemovePartImmediately(std::move(part));
                /// No need to delete the part from ZK because we can be sure that the commit transaction
                /// didn't go through.

1209
                return false;
1210
            }
1211 1212 1213 1214

            throw;
        }

1215
        /** Removing old parts from ZK and from the disk is delayed - see ReplicatedMergeTreeCleanupThread, clearOldParts.
1216 1217
          */

1218
        /** With `ZSESSIONEXPIRED` or `ZOPERATIONTIMEOUT`, we can inadvertently roll back local changes to the parts.
1219 1220
          * This is not a problem, because in this case the merge will remain in the queue, and we will try again.
          */
1221
        merge_selecting_task->schedule();
1222 1223 1224
        ProfileEvents::increment(ProfileEvents::ReplicatedPartMerges);

        write_part_log({});
1225 1226 1227 1228 1229 1230 1231 1232 1233 1234 1235 1236 1237 1238 1239 1240 1241 1242 1243

        return true;
    }
    catch (...)
    {
        write_part_log(ExecutionStatus::fromCurrentException());
        throw;
    }
}


bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedMergeTree::LogEntry & entry)
{
    const String & source_part_name = entry.source_parts.at(0);
    LOG_TRACE(log, "Executing log entry to mutate part " << source_part_name << " to " << entry.new_part_name);

    MergeTreeData::DataPartPtr source_part = data.getActiveContainingPart(source_part_name);
    if (!source_part)
    {
1244
        LOG_DEBUG(log, "Source part " + source_part_name + " for " << entry.new_part_name << " is not ready; will try to fetch it instead");
1245 1246 1247 1248 1249 1250 1251 1252 1253 1254 1255
        return false;
    }

    if (source_part->name != source_part_name)
    {
        throw Exception("Part " + source_part_name + " is covered by " + source_part->name
            + " but should be mutated to " + entry.new_part_name + ". This is a bug.",
            ErrorCodes::LOGICAL_ERROR);
    }

    /// TODO - some better heuristic?
1256
    size_t estimated_space_for_result = MergeTreeDataMergerMutator::estimateNeededDiskSpace({source_part});
1257 1258 1259 1260 1261 1262 1263 1264 1265 1266 1267 1268 1269 1270 1271 1272 1273 1274 1275 1276 1277 1278 1279 1280

    if (entry.create_time + data.settings.prefer_fetch_merged_part_time_threshold.totalSeconds() <= time(nullptr)
        && estimated_space_for_result >= data.settings.prefer_fetch_merged_part_size_threshold)
    {
        /// If entry is old enough, and have enough size, and some replica has the desired part,
        /// then prefer fetching from replica.
        String replica = findReplicaHavingPart(entry.new_part_name, true);    /// NOTE excessive ZK requests for same data later, may remove.
        if (!replica.empty())
        {
            LOG_DEBUG(log, "Prefer to fetch " << entry.new_part_name << " from replica " << replica);
            return false;
        }
    }

    MergeTreePartInfo new_part_info = MergeTreePartInfo::fromPartName(
        entry.new_part_name, data.format_version);
    MutationCommands commands = queue.getMutationCommands(source_part, new_part_info.mutation);

    /// Can throw an exception.
    DiskSpaceMonitor::ReservationPtr reserved_space = DiskSpaceMonitor::reserve(full_path, estimated_space_for_result);

    auto table_lock = lockStructure(false, __PRETTY_FUNCTION__);

    MergeTreeData::MutableDataPartPtr new_part;
1281
    MergeTreeData::Transaction transaction(data);
1282

1283 1284 1285 1286 1287
    MergeTreeDataMergerMutator::FuturePart future_mutated_part;
    future_mutated_part.parts.push_back(source_part);
    future_mutated_part.part_info = new_part_info;
    future_mutated_part.name = entry.new_part_name;

1288
    Stopwatch stopwatch;
1289 1290

    auto write_part_log = [&] (const ExecutionStatus & execution_status)
1291
    {
1292 1293 1294
        writePartLog(
            PartLogElement::MUTATE_PART, execution_status, stopwatch.elapsed(),
            entry.new_part_name, new_part, future_mutated_part.parts, nullptr);
1295 1296 1297 1298
    };

    try
    {
1299
        new_part = merger_mutator.mutatePartToTemporaryPart(future_mutated_part, commands, context);
1300
        data.renameTempPartAndReplace(new_part, nullptr, &transaction);
1301 1302 1303 1304 1305 1306 1307 1308 1309 1310 1311

        try
        {
            checkPartChecksumsAndCommit(transaction, new_part);
        }
        catch (const Exception & e)
        {
            if (MergeTreeDataPartChecksums::isBadChecksumsErrorCode(e.code()))
            {
                transaction.rollback();

1312
                ProfileEvents::increment(ProfileEvents::DataAfterMutationDiffersFromReplica);
1313 1314 1315 1316 1317 1318 1319

                LOG_ERROR(log, getCurrentExceptionMessage(false) << ". "
                    "Data after mutation is not byte-identical to data on another replicas. "
                    "We will download merged part from replica to force byte-identical result.");

                write_part_log(ExecutionStatus::fromCurrentException());

1320 1321 1322 1323
                data.tryRemovePartImmediately(std::move(new_part));
                /// No need to delete the part from ZK because we can be sure that the commit transaction
                /// didn't go through.

1324 1325 1326 1327 1328 1329 1330 1331 1332
                return false;
            }

            throw;
        }

        /** With `ZSESSIONEXPIRED` or `ZOPERATIONTIMEOUT`, we can inadvertently roll back local changes to the parts.
          * This is not a problem, because in this case the entry will remain in the queue, and we will try again.
          */
1333
        merge_selecting_task->schedule();
1334
        ProfileEvents::increment(ProfileEvents::ReplicatedPartMutations);
1335 1336 1337
        write_part_log({});

        return true;
1338
    }
1339
    catch (...)
1340
    {
1341 1342
        write_part_log(ExecutionStatus::fromCurrentException());
        throw;
1343
    }
1344
}
1345 1346


1347
bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry)
1348 1349
{
    String replica = findReplicaHavingCoveringPart(entry, true);
1350

1351 1352 1353 1354
    static std::atomic_uint total_fetches {0};
    if (data.settings.replicated_max_parallel_fetches && total_fetches >= data.settings.replicated_max_parallel_fetches)
    {
        throw Exception("Too many total fetches from replicas, maximum: " + data.settings.replicated_max_parallel_fetches.toString(),
A
Alexey Milovidov 已提交
1355
            ErrorCodes::TOO_MANY_FETCHES);
1356
    }
1357

1358 1359
    ++total_fetches;
    SCOPE_EXIT({--total_fetches;});
1360

1361 1362 1363
    if (data.settings.replicated_max_parallel_fetches_for_table && current_table_fetches >= data.settings.replicated_max_parallel_fetches_for_table)
    {
        throw Exception("Too many fetches from replicas for table, maximum: " + data.settings.replicated_max_parallel_fetches_for_table.toString(),
A
Alexey Milovidov 已提交
1364
            ErrorCodes::TOO_MANY_FETCHES);
1365
    }
1366

1367 1368 1369 1370 1371 1372
    ++current_table_fetches;
    SCOPE_EXIT({--current_table_fetches;});

    try
    {
        if (replica.empty())
1373
        {
1374 1375 1376 1377 1378 1379
            /** If a part is to be written with a quorum and the quorum is not reached yet,
              *  then (due to the fact that a part is impossible to download right now),
              *  the quorum entry should be considered unsuccessful.
              * TODO Complex code, extract separately.
              */
            if (entry.quorum)
1380
            {
1381 1382 1383 1384 1385 1386 1387 1388 1389 1390 1391 1392 1393 1394
                if (entry.type != LogEntry::GET_PART)
                    throw Exception("Logical error: log entry with quorum but type is not GET_PART", ErrorCodes::LOGICAL_ERROR);

                LOG_DEBUG(log, "No active replica has part " << entry.new_part_name << " which needs to be written with quorum."
                    " Will try to mark that quorum as failed.");

                /** Atomically:
                  * - if replicas do not become active;
                  * - if there is a `quorum` node with this part;
                  * - delete `quorum` node;
                  * - add a part to the list `quorum/failed_parts`;
                  * - if the part is not already removed from the list for deduplication `blocks/block_num`, then delete it;
                  *
                  * If something changes, then we will nothing - we'll get here again next time.
1395 1396
                  */

1397 1398 1399 1400
                /** We collect the `host` node versions from the replicas.
                  * When the replica becomes active, it changes the value of host in the same transaction (with the creation of `is_active`).
                  * This will ensure that the replicas do not become active.
                  */
1401

1402
                auto zookeeper = getZooKeeper();
1403

1404
                Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
1405

1406
                Coordination::Requests ops;
1407

1408 1409
                for (size_t i = 0, size = replicas.size(); i < size; ++i)
                {
1410
                    Coordination::Stat stat;
1411 1412
                    String path = zookeeper_path + "/replicas/" + replicas[i] + "/host";
                    zookeeper->get(path, &stat);
1413
                    ops.emplace_back(zkutil::makeCheckRequest(path, stat.version));
1414
                }
1415

1416 1417
                /// We verify that while we were collecting versions, the replica with the necessary part did not come alive.
                replica = findReplicaHavingPart(entry.new_part_name, true);
1418

1419 1420 1421 1422 1423
                /// Also during this time a completely new replica could be created.
                /// But if a part does not appear on the old, then it can not be on the new one either.

                if (replica.empty())
                {
1424
                    Coordination::Stat quorum_stat;
1425 1426 1427 1428 1429 1430
                    String quorum_path = zookeeper_path + "/quorum/status";
                    String quorum_str = zookeeper->get(quorum_path, &quorum_stat);
                    ReplicatedMergeTreeQuorumEntry quorum_entry;
                    quorum_entry.fromString(quorum_str);

                    if (quorum_entry.part_name == entry.new_part_name)
1431
                    {
1432
                        ops.emplace_back(zkutil::makeRemoveRequest(quorum_path, quorum_stat.version));
1433

1434
                        auto part_info = MergeTreePartInfo::fromPartName(entry.new_part_name, data.format_version);
1435

1436 1437 1438
                        if (part_info.min_block != part_info.max_block)
                            throw Exception("Logical error: log entry with quorum for part covering more than one block number",
                                ErrorCodes::LOGICAL_ERROR);
1439

1440
                        ops.emplace_back(zkutil::makeCreateRequest(
1441 1442 1443 1444 1445 1446
                            zookeeper_path + "/quorum/failed_parts/" + entry.new_part_name,
                            "",
                            zkutil::CreateMode::Persistent));

                        /// Deleting from `blocks`.
                        if (!entry.block_id.empty() && zookeeper->exists(zookeeper_path + "/blocks/" + entry.block_id))
1447
                            ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/blocks/" + entry.block_id, -1));
1448

1449
                        Coordination::Responses responses;
1450
                        auto code = zookeeper->tryMulti(ops, responses);
1451

1452
                        if (code == Coordination::ZOK)
1453
                        {
1454
                            LOG_DEBUG(log, "Marked quorum for part " << entry.new_part_name << " as failed.");
1455 1456
                            queue.removeFromVirtualParts(part_info);
                            return true;
1457
                        }
1458
                        else if (code == Coordination::ZBADVERSION || code == Coordination::ZNONODE || code == Coordination::ZNODEEXISTS)
1459
                        {
1460
                            LOG_DEBUG(log, "State was changed or isn't expected when trying to mark quorum for part "
1461
                                << entry.new_part_name << " as failed. Code: " << zkutil::ZooKeeper::error2string(code));
1462
                        }
1463
                        else
1464
                            throw Coordination::Exception(code);
1465 1466 1467 1468 1469 1470 1471 1472
                    }
                    else
                    {
                        LOG_WARNING(log, "No active replica has part " << entry.new_part_name
                            << ", but that part needs quorum and /quorum/status contains entry about another part " << quorum_entry.part_name
                            << ". It means that part was successfully written to " << entry.quorum
                            << " replicas, but then all of them goes offline."
                            << " Or it is a bug.");
1473 1474 1475 1476
                    }
                }
            }

1477
            if (replica.empty())
1478
            {
1479 1480
                ProfileEvents::increment(ProfileEvents::ReplicatedPartFailedFetches);
                throw Exception("No active replica has part " + entry.new_part_name + " or covering part", ErrorCodes::NO_REPLICA_HAS_PART);
1481
            }
1482
        }
1483

1484 1485 1486 1487
        try
        {
            if (!fetchPart(entry.actual_new_part_name, zookeeper_path + "/replicas/" + replica, false, entry.quorum))
                return false;
1488
        }
1489
        catch (Exception & e)
1490
        {
1491 1492 1493 1494 1495
            /// No stacktrace, just log message
            if (e.code() == ErrorCodes::RECEIVED_ERROR_TOO_MANY_REQUESTS)
                e.addMessage("Too busy replica. Will try later.");
            throw;
        }
1496

1497 1498 1499 1500 1501 1502 1503 1504 1505 1506 1507 1508
        if (entry.type == LogEntry::MERGE_PARTS)
            ProfileEvents::increment(ProfileEvents::ReplicatedPartFetchesOfMerged);
    }
    catch (...)
    {
        /** If you can not download the part you need for some merge, it's better not to try to get other parts for this merge,
          * but try to get already merged part. To do this, move the action to get the remaining parts
          * for this merge at the end of the queue.
          */
        try
        {
            auto parts_for_merge = queue.moveSiblingPartsForMergeToEndOfQueue(entry.new_part_name);
1509

1510
            if (!parts_for_merge.empty() && replica.empty())
1511
            {
1512 1513
                LOG_INFO(log, "No active replica has part " << entry.new_part_name << ". Will fetch merged part instead.");
                return false;
1514 1515
            }

1516 1517 1518 1519 1520
            /** If no active replica has a part, and there is no merge in the queue with its participation,
              * check to see if any (active or inactive) replica has such a part or covering it.
              */
            if (replica.empty())
                enqueuePartForCheck(entry.new_part_name);
1521
        }
1522 1523
        catch (...)
        {
1524
            tryLogCurrentException(log, __PRETTY_FUNCTION__);
1525 1526 1527
        }

        throw;
1528 1529 1530
    }

    return true;
M
Merge  
Michael Kolupaev 已提交
1531 1532
}

A
Merge  
Alexey Milovidov 已提交
1533

1534
void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
1535
{
1536
    auto drop_range_info = MergeTreePartInfo::fromPartName(entry.new_part_name, data.format_version);
1537
    queue.removePartProducingOpsInRange(getZooKeeper(), drop_range_info, entry);
1538 1539

    LOG_DEBUG(log, (entry.detach ? "Detaching" : "Removing") << " parts.");
1540

1541 1542 1543 1544
    /// Delete the parts contained in the range to be deleted.
    /// It's important that no old parts remain (after the merge), because otherwise,
    ///  after adding a new replica, this new replica downloads them, but does not delete them.
    /// And, if you do not, the parts will come to life after the server is restarted.
1545 1546
    /// Therefore, we use all data parts.

1547
    MergeTreeData::DataPartsVector parts_to_remove;
1548
    {
1549 1550 1551
        auto data_parts_lock = data.lockParts();
        parts_to_remove = data.removePartsInRangeFromWorkingSet(drop_range_info, true, true, data_parts_lock);
    }
1552

1553 1554 1555 1556 1557 1558 1559 1560 1561
    if (entry.detach)
    {
        /// If DETACH clone parts to detached/ directory
        for (const auto & part : parts_to_remove)
        {
            LOG_INFO(log, "Detaching " << part->relative_path);
            part->makeCloneInDetached("");
        }
    }
1562

1563 1564
    /// Forcibly remove parts from ZooKeeper
    tryRemovePartsFromZooKeeperWithRetries(parts_to_remove);
1565

1566
    LOG_INFO(log, (entry.detach ? "Detached " : "Removed ") << parts_to_remove.size() << " parts inside " << entry.new_part_name << ".");
1567

1568 1569 1570
    /// We want to remove dropped parts from disk as soon as possible
    /// To be removed a partition should have zero refcount, therefore call the cleanup thread at exit
    parts_to_remove.clear();
1571
    cleanup_thread.wakeup();
M
Merge  
Michael Kolupaev 已提交
1572 1573
}

A
Merge  
Alexey Milovidov 已提交
1574

1575
void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry & entry)
1576
{
1577
    LOG_INFO(log, "Clear column " << entry.column_name << " in parts inside " << entry.new_part_name << " range");
1578

1579 1580
    auto entry_part_info = MergeTreePartInfo::fromPartName(entry.new_part_name, data.format_version);

1581 1582 1583
    /// We don't change table structure, only data in some parts
    /// To disable reading from these parts, we will sequentially acquire write lock for each part inside alterDataPart()
    /// If we will lock the whole table here, a deadlock can occur. For example, if use use Buffer table (CLICKHOUSE-3238)
1584
    auto lock_read_structure = lockStructure(false, __PRETTY_FUNCTION__);
1585 1586 1587 1588 1589 1590 1591

    auto zookeeper = getZooKeeper();

    AlterCommand alter_command;
    alter_command.type = AlterCommand::DROP_COLUMN;
    alter_command.column_name = entry.column_name;

1592
    auto new_columns = getColumns();
1593
    alter_command.apply(new_columns);
1594 1595

    size_t modified_parts = 0;
1596
    auto parts = data.getDataParts();
A
Alexey Zatelepin 已提交
1597
    auto columns_for_parts = new_columns.getAllPhysical();
1598 1599 1600 1601 1602 1603 1604 1605

    /// Check there are no merges in range again
    /// TODO: Currently, there are no guarantees that a merge covering entry_part_info will happen during the execution.
    /// To solve this problem we could add read/write flags for each part in future_parts
    ///  and make more sophisticated checks for merges in shouldExecuteLogEntry().
    /// But this feature will be useless when the mutation feature is implemented.
    queue.checkThereAreNoConflictsInRange(entry_part_info, entry);

1606 1607
    for (const auto & part : parts)
    {
1608
        if (!entry_part_info.contains(part->info))
1609 1610
            continue;

1611
        LOG_DEBUG(log, "Clearing column " << entry.column_name << " in part " << part->name);
1612 1613 1614 1615 1616 1617

        auto transaction = data.alterDataPart(part, columns_for_parts, data.primary_expr_ast, false);
        if (!transaction)
            continue;

        /// Update part metadata in ZooKeeper.
1618
        Coordination::Requests ops;
1619
        ops.emplace_back(zkutil::makeSetRequest(
1620
            replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1));
1621
        ops.emplace_back(zkutil::makeSetRequest(
1622
            replica_path + "/parts/" + part->name + "/checksums", getChecksumsForZooKeeper(transaction->getNewChecksums()), -1));
1623 1624 1625 1626 1627 1628 1629

        zookeeper->multi(ops);

        transaction->commit();
        ++modified_parts;
    }

1630
    LOG_DEBUG(log, "Cleared column " << entry.column_name << " in " << modified_parts << " parts");
1631

1632
    /// Recalculate columns size (not only for the modified column)
1633 1634 1635 1636
    data.recalculateColumnSizes();
}


1637
bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
1638
{
1639 1640
    Stopwatch watch;
    auto & entry_replace = *entry.replace_range_entry;
1641

1642 1643 1644 1645
    MergeTreePartInfo drop_range = MergeTreePartInfo::fromPartName(entry_replace.drop_range_part_name, data.format_version);
    /// Range with only one block has special meaning ATTACH PARTITION
    bool replace = drop_range.getBlocksCount() > 1;

1646
    queue.removePartProducingOpsInRange(getZooKeeper(), drop_range, entry);
1647 1648

    struct PartDescription
S
Silviu Caragea 已提交
1649
    {
1650 1651 1652 1653 1654 1655 1656 1657 1658 1659 1660 1661 1662 1663 1664 1665 1666 1667 1668 1669 1670 1671 1672 1673 1674 1675 1676 1677 1678 1679 1680 1681 1682 1683 1684 1685 1686 1687 1688 1689 1690 1691 1692 1693 1694 1695 1696 1697 1698 1699 1700
        PartDescription(size_t index_, const String & src_part_name_, const String & new_part_name_, const String & checksum_hex_,
                        MergeTreeDataFormatVersion format_version)
            : index(index_),
            src_part_name(src_part_name_), src_part_info(MergeTreePartInfo::fromPartName(src_part_name_, format_version)),
            new_part_name(new_part_name_), new_part_info(MergeTreePartInfo::fromPartName(new_part_name_, format_version)),
            checksum_hex(checksum_hex_) {}

        size_t index; // in log entry arrays
        String src_part_name;
        MergeTreePartInfo src_part_info;
        String new_part_name;
        MergeTreePartInfo new_part_info;
        String checksum_hex;

        /// Part which will be comitted
        MergeTreeData::MutableDataPartPtr res_part;

        /// We could find a covering part
        MergeTreePartInfo found_new_part_info;
        String found_new_part_name;

        /// Hold pointer to part in source table if will clone it from local table
        MergeTreeData::DataPartPtr src_table_part;

        /// A replica that will be used to fetch part
        String replica;
    };

    using PartDescriptionPtr = std::shared_ptr<PartDescription>;
    using PartDescriptions = std::vector<PartDescriptionPtr>;

    PartDescriptions all_parts;
    PartDescriptions parts_to_add;
    MergeTreeData::DataPartsVector parts_to_remove;

    auto structure_lock_dst_table = lockStructure(false, __PRETTY_FUNCTION__);

    for (size_t i = 0; i < entry_replace.new_part_names.size(); ++i)
    {
        all_parts.emplace_back(std::make_shared<PartDescription>(i,
            entry_replace.src_part_names.at(i),
            entry_replace.new_part_names.at(i),
            entry_replace.part_names_checksums.at(i),
            data.format_version));
    }

    /// What parts we should add? Or we have already added all required parts (we an replica-intializer)
    {
        auto data_parts_lock = data.lockParts();

        for (const PartDescriptionPtr & part_desc : all_parts)
1701
        {
1702 1703
            if (!data.getActiveContainingPart(part_desc->new_part_info, MergeTreeDataPartState::Committed, data_parts_lock))
                parts_to_add.emplace_back(part_desc);
1704
        }
1705 1706 1707 1708 1709 1710 1711 1712 1713 1714 1715 1716

        if (parts_to_add.empty() && replace)
            parts_to_remove = data.removePartsInRangeFromWorkingSet(drop_range, true, false, data_parts_lock);
    }

    if (parts_to_add.empty())
    {
        LOG_INFO(log, "All parts from REPLACE PARTITION command have been already attached");
        tryRemovePartsFromZooKeeperWithRetries(parts_to_remove);
        return true;
    }

1717
    if (parts_to_add.size() < all_parts.size())
1718 1719 1720 1721 1722 1723 1724 1725 1726 1727 1728 1729 1730 1731 1732 1733 1734 1735
    {
        LOG_WARNING(log, "Some (but not all) parts from REPLACE PARTITION command already exist. REPLACE PARTITION will not be atomic.");
    }

    StoragePtr source_table;
    TableStructureReadLockPtr structure_lock_src_table;
    String source_table_name = entry_replace.from_database + "." + entry_replace.from_table;

    auto clone_data_parts_from_source_table = [&] () -> size_t
    {
        source_table = context.tryGetTable(entry_replace.from_database, entry_replace.from_table);
        if (!source_table)
        {
            LOG_DEBUG(log, "Can't use " << source_table_name << " as source table for REPLACE PARTITION command. It does not exist.");
            return 0;
        }

        MergeTreeData * src_data = nullptr;
1736 1737
        try
        {
1738
            src_data = data.checkStructureAndGetMergeTreeData(source_table);
1739
        }
1740
        catch (Exception &)
1741
        {
1742 1743 1744 1745
            LOG_INFO(log, "Can't use " << source_table_name << " as source table for REPLACE PARTITION command. Will fetch all parts."
                           << " Reason: " << getCurrentExceptionMessage(false));
            return 0;
        }
1746

1747 1748 1749 1750 1751 1752 1753 1754 1755 1756 1757 1758 1759 1760 1761 1762 1763 1764 1765 1766 1767 1768 1769 1770 1771 1772 1773 1774 1775 1776 1777 1778 1779
        structure_lock_src_table = source_table->lockStructure(false, __PRETTY_FUNCTION__);

        MergeTreeData::DataPartStates valid_states{MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed,
                                                   MergeTreeDataPartState::Outdated};

        size_t num_clonable_parts = 0;
        for (PartDescriptionPtr & part_desc : parts_to_add)
        {
            auto src_part = src_data->getPartIfExists(part_desc->src_part_info, valid_states);
            if (!src_part)
            {
                LOG_DEBUG(log, "There is no part " << part_desc->src_part_name << " in " << source_table_name);
                continue;
            }

            String checksum_hex;
            {
                std::shared_lock<std::shared_mutex> part_lock(src_part->columns_lock);
                checksum_hex = src_part->checksums.getTotalChecksumHex();
            }

            if (checksum_hex != part_desc->checksum_hex)
            {
                LOG_DEBUG(log, "Part " << part_desc->src_part_name << " of " << source_table_name << " has inappropriate checksum");
                /// TODO: check version
                continue;
            }

            part_desc->found_new_part_name = part_desc->new_part_name;
            part_desc->found_new_part_info = part_desc->new_part_info;
            part_desc->src_table_part = src_part;

            ++num_clonable_parts;
1780
        }
1781 1782 1783 1784 1785 1786 1787 1788 1789 1790 1791 1792 1793

        return num_clonable_parts;
    };

    size_t num_clonable_parts = clone_data_parts_from_source_table();
    LOG_DEBUG(log, "Found " << num_clonable_parts << " parts that could be cloned (of " << parts_to_add.size() << " required parts)");

    ActiveDataPartSet adding_parts_active_set(data.format_version);
    std::unordered_map<String, PartDescriptionPtr> part_name_to_desc;

    for (PartDescriptionPtr & part_desc : parts_to_add)
    {
        if (part_desc->src_table_part)
1794
        {
1795
            /// It is clonable part
1796
            adding_parts_active_set.add(part_desc->new_part_name);
1797 1798
            part_name_to_desc.emplace(part_desc->new_part_name, part_desc);
            continue;
1799
        }
1800 1801 1802 1803 1804 1805 1806 1807 1808 1809 1810 1811 1812 1813 1814 1815 1816 1817 1818 1819 1820 1821 1822 1823 1824 1825 1826 1827 1828

        /// Firstly, try find exact part to produce more accurate part set
        String replica = findReplicaHavingPart(part_desc->new_part_name, true);
        String found_part_name;
        /// TODO: check version

        if (replica.empty())
        {
            LOG_DEBUG(log, "Part " <<  part_desc->new_part_name << " is not found on remote replicas");

            /// Fallback to covering part
            replica = findReplicaHavingCoveringPart(part_desc->new_part_name, true, found_part_name);

            if (replica.empty())
            {
                /// It is not fail, since adjacent parts could cover current part
                LOG_DEBUG(log, "Parts covering " <<  part_desc->new_part_name << " are not found on remote replicas");
                continue;
            }
        }
        else
        {
            found_part_name = part_desc->new_part_name;
        }

        part_desc->found_new_part_name = found_part_name;
        part_desc->found_new_part_info = MergeTreePartInfo::fromPartName(found_part_name, data.format_version);
        part_desc->replica = replica;

1829
        adding_parts_active_set.add(part_desc->found_new_part_name);
1830
        part_name_to_desc.emplace(part_desc->found_new_part_name, part_desc);
S
Silviu Caragea 已提交
1831
    }
1832

1833 1834 1835
    /// Check that we could cover whole range
    for (PartDescriptionPtr & part_desc : parts_to_add)
    {
1836
        if (adding_parts_active_set.getContainingPart(part_desc->new_part_info).empty())
1837 1838 1839 1840 1841
        {
            throw Exception("Not found part " + part_desc->new_part_name +
                            " (or part covering it) neither source table neither remote replicas" , ErrorCodes::NO_REPLICA_HAS_PART);
        }
    }
M
Merge  
Michael Kolupaev 已提交
1842

1843 1844 1845
    /// Filter covered parts
    PartDescriptions final_parts;
    {
1846
        Strings final_part_names = adding_parts_active_set.getParts();
A
Merge  
Alexey Milovidov 已提交
1847

1848 1849 1850 1851 1852 1853 1854
        for (const String & final_part_name : final_part_names)
        {
            auto part_desc = part_name_to_desc[final_part_name];
            if (!part_desc)
                throw Exception("There is no final part " + final_part_name + ". This is a bug", ErrorCodes::LOGICAL_ERROR);

            final_parts.emplace_back(part_desc);
1855

1856 1857 1858 1859 1860 1861 1862 1863 1864 1865 1866 1867 1868 1869 1870 1871 1872
            if (final_parts.size() > 1)
            {
                auto & prev = *final_parts[final_parts.size() - 2];
                auto & curr = *final_parts[final_parts.size() - 1];

                if (!prev.found_new_part_info.isDisjoint(curr.found_new_part_info))
                {
                    throw Exception("Intersected final parts detected: " + prev.found_new_part_name
                                    + " and " + curr.found_new_part_name + ". It should be investigated.");
                }
            }
        }
    }

    static const String TMP_PREFIX = "tmp_replace_from_";

    auto obtain_part = [&] (PartDescriptionPtr & part_desc)
1873
    {
1874
        if (part_desc->src_table_part)
1875
        {
1876 1877 1878 1879 1880 1881 1882
            std::shared_lock<std::shared_mutex> part_lock(part_desc->src_table_part->columns_lock);

            if (part_desc->checksum_hex != part_desc->src_table_part->checksums.getTotalChecksumHex())
                throw Exception("Checksums of " + part_desc->src_table_part->name + " is suddenly changed", ErrorCodes::UNFINISHED);

            part_desc->res_part = data.cloneAndLoadDataPart(
                part_desc->src_table_part, TMP_PREFIX + "clone_", part_desc->new_part_info);
1883
        }
1884
        else if (!part_desc->replica.empty())
1885
        {
1886 1887 1888
            String replica_path = zookeeper_path + "/replicas/" + part_desc->replica;
            ReplicatedMergeTreeAddress address(getZooKeeper()->get(replica_path + "/host"));
            auto timeouts = ConnectionTimeouts::getHTTPTimeouts(context.getSettingsRef());
1889
            auto [user, password] = context.getInterserverCredentials();
1890
            String interserver_scheme = context.getInterserverScheme();
1891

1892 1893
            if (interserver_scheme != address.scheme)
                throw Exception("Interserver schemes are different '" + interserver_scheme + "' != '" + address.scheme + "', can't fetch part from " + address.host, ErrorCodes::LOGICAL_ERROR);
1894 1895

            part_desc->res_part = fetcher.fetchPart(part_desc->found_new_part_name, replica_path,
1896
                                                    address.host, address.replication_port, timeouts, user, password, interserver_scheme, false, TMP_PREFIX + "fetch_");
1897 1898 1899 1900 1901 1902 1903 1904 1905 1906 1907 1908 1909 1910 1911 1912 1913 1914 1915 1916 1917 1918

            /// TODO: check columns_version of fetched part

            ProfileEvents::increment(ProfileEvents::ReplicatedPartFetches);
        }
        else
            throw Exception("There is no receipt to produce part " + part_desc->new_part_name + ". This is bug", ErrorCodes::LOGICAL_ERROR);
    };

    /// Download or clone parts
    /// TODO: make it in parallel
    for (PartDescriptionPtr & part_desc : final_parts)
        obtain_part(part_desc);

    MergeTreeData::MutableDataPartsVector res_parts;
    for (PartDescriptionPtr & part_desc : final_parts)
        res_parts.emplace_back(part_desc->res_part);

    try
    {
        /// Commit parts
        auto zookeeper = getZooKeeper();
1919
        MergeTreeData::Transaction transaction(data);
1920

1921
        Coordination::Requests ops;
1922 1923 1924 1925 1926 1927 1928 1929 1930 1931
        for (PartDescriptionPtr & part_desc : final_parts)
        {
            data.renameTempPartAndReplace(part_desc->res_part, nullptr, &transaction);
            getCommitPartOps(ops, part_desc->res_part);

            if (ops.size() > zkutil::MULTI_BATCH_SIZE)
            {
                zookeeper->multi(ops);
                ops.clear();
            }
1932
        }
1933 1934 1935 1936 1937 1938 1939 1940 1941 1942 1943 1944 1945 1946 1947 1948 1949 1950 1951 1952 1953 1954 1955

        if (!ops.empty())
            zookeeper->multi(ops);

        {
            auto data_parts_lock = data.lockParts();

            transaction.commit(&data_parts_lock);
            if (replace)
                parts_to_remove = data.removePartsInRangeFromWorkingSet(drop_range, true, false, data_parts_lock);
        }

        PartLog::addNewParts(this->context, res_parts, watch.elapsed());
    }
    catch (...)
    {
        PartLog::addNewParts(this->context, res_parts, watch.elapsed(), ExecutionStatus::fromCurrentException());
        throw;
    }

    tryRemovePartsFromZooKeeperWithRetries(parts_to_remove);
    res_parts.clear();
    parts_to_remove.clear();
1956
    cleanup_thread.wakeup();
1957 1958 1959 1960 1961

    return true;
}


V
VadimPE 已提交
1962
void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coordination::Stat source_is_lost_stat, zkutil::ZooKeeperPtr & zookeeper)
A
Alexey Milovidov 已提交
1963
{
1964 1965 1966
    LOG_INFO(log, "Will mimic " << source_replica);

    String source_path = zookeeper_path + "/replicas/" + source_replica;
A
Alexey Milovidov 已提交
1967

V
Vadim 已提交
1968
    /** TODO: it will be deleted! (It is only to support old version of CH server).
A
Alexey Milovidov 已提交
1969 1970 1971
      * In current code, the replica is created in single transaction.
      * If the reference/master replica is not yet fully created, let's wait.
      */
1972 1973 1974 1975 1976 1977 1978 1979 1980 1981 1982 1983 1984 1985 1986
    while (!zookeeper->exists(source_path + "/columns"))
    {
        LOG_INFO(log, "Waiting for replica " << source_path << " to be fully created");

        zkutil::EventPtr event = std::make_shared<Poco::Event>();
        if (zookeeper->exists(source_path + "/columns", nullptr, event))
        {
            LOG_WARNING(log, "Oops, a watch has leaked");
            break;
        }

        event->wait();
    }

    /// The order of the following three actions is important. Entries in the log can be duplicated, but they can not be lost.
A
Alexey Milovidov 已提交
1987

V
VadimPE 已提交
1988
    String raw_log_pointer = zookeeper->get(source_path + "/log_pointer");
1989

V
VadimPE 已提交
1990
    Coordination::Requests ops;
1991
    ops.push_back(zkutil::makeSetRequest(replica_path + "/log_pointer", raw_log_pointer, -1));
A
Alexey Milovidov 已提交
1992

V
VadimPE 已提交
1993
    /// For support old versions CH.
V
VadimPE 已提交
1994
    if (source_is_lost_stat.version == -1)
V
VadimPE 已提交
1995
    {
A
Alexey Milovidov 已提交
1996 1997
        /// We check that it was not suddenly upgraded to new version.
        /// Otherwise it can be upgraded and instantly become lost, but we cannot notice that.
V
VadimPE 已提交
1998 1999
        ops.push_back(zkutil::makeCreateRequest(source_path + "/is_lost", "0", zkutil::CreateMode::Persistent));
        ops.push_back(zkutil::makeRemoveRequest(source_path + "/is_lost", -1));
V
VadimPE 已提交
2000
    }
A
Alexey Milovidov 已提交
2001
    else    /// The replica we clone should not suddenly become lost.
V
VadimPE 已提交
2002
        ops.push_back(zkutil::makeCheckRequest(source_path + "/is_lost", source_is_lost_stat.version));
A
Alexey Milovidov 已提交
2003

V
VadimPE 已提交
2004
    Coordination::Responses resp;
A
Alexey Milovidov 已提交
2005

2006
    auto error = zookeeper->tryMulti(ops, resp);
V
VadimPE 已提交
2007
    if (error == Coordination::Error::ZBADVERSION)
V
VadimPE 已提交
2008
        throw Exception("Can not clone replica, because the " + source_replica + " became lost", ErrorCodes::REPLICA_STATUS_CHANGED);
V
VadimPE 已提交
2009
    else if (error == Coordination::Error::ZNODEEXISTS)
A
Alexey Milovidov 已提交
2010
        throw Exception("Can not clone replica, because the " + source_replica + " updated to new ClickHouse version", ErrorCodes::REPLICA_STATUS_CHANGED);
V
VadimPE 已提交
2011 2012
    else
        zkutil::KeeperMultiException::check(error, ops, resp);
A
Alexey Milovidov 已提交
2013

2014 2015 2016 2017 2018 2019 2020 2021 2022 2023 2024 2025 2026 2027 2028 2029 2030 2031 2032 2033 2034 2035 2036 2037 2038
    /// Let's remember the queue of the reference/master replica.
    Strings source_queue_names = zookeeper->getChildren(source_path + "/queue");
    std::sort(source_queue_names.begin(), source_queue_names.end());
    Strings source_queue;
    for (const String & entry_name : source_queue_names)
    {
        String entry;
        if (!zookeeper->tryGet(source_path + "/queue/" + entry_name, entry))
            continue;
        source_queue.push_back(entry);
    }

    /// Add to the queue jobs to receive all the active parts that the reference/master replica has.
    Strings parts = zookeeper->getChildren(source_path + "/parts");
    ActiveDataPartSet active_parts_set(data.format_version, parts);

    Strings active_parts = active_parts_set.getParts();
    for (const String & name : active_parts)
    {
        LogEntry log_entry;
        log_entry.type = LogEntry::GET_PART;
        log_entry.source_replica =  "";
        log_entry.new_part_name = name;
        log_entry.create_time = tryGetPartCreateTime(zookeeper, source_path, name);

2039
        zookeeper->create(replica_path + "/queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential);
2040
    }
A
Alexey Milovidov 已提交
2041

2042 2043 2044 2045 2046 2047 2048 2049 2050
    LOG_DEBUG(log, "Queued " << active_parts.size() << " parts to be fetched");

    /// Add content of the reference/master replica queue to the queue.
    for (const String & entry : source_queue)
    {
        zookeeper->create(replica_path + "/queue/queue-", entry, zkutil::CreateMode::PersistentSequential);
    }

    LOG_DEBUG(log, "Copied " << source_queue.size() << " queue entries");
2051 2052 2053
}


2054
void StorageReplicatedMergeTree::cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zookeeper)
2055
{
2056 2057 2058 2059 2060 2061 2062 2063
    String res;
    if (zookeeper->tryGet(replica_path + "/is_lost", res))
    {
        if (res == "0")
            return;
    }
    else
    {
V
VadimPE 已提交
2064
        /// Replica was created by old version of CH, so me must create "/is_lost".
A
Alexey Milovidov 已提交
2065
        /// Note that in old version of CH there was no "lost" replicas possible.
2066
        zookeeper->create(replica_path + "/is_lost", "0", zkutil::CreateMode::Persistent);
2067
        return;
2068
    }
2069

A
Alexey Milovidov 已提交
2070 2071
    /// is_lost is "1": it means that we are in repair mode.

2072
    String source_replica;
V
VadimPE 已提交
2073
    Coordination::Stat source_is_lost_stat;
V
VadimPE 已提交
2074
    source_is_lost_stat.version = -1;
2075

V
VadimPE 已提交
2076
    for (const String & replica_name : zookeeper->getChildren(zookeeper_path + "/replicas"))
2077
    {
V
VadimPE 已提交
2078
        String source_replica_path = zookeeper_path + "/replicas/" + replica_name;
A
Alexey Milovidov 已提交
2079

A
Alexey Milovidov 已提交
2080
        /// Do not clone from myself.
2081
        if (source_replica_path != replica_path)
2082
        {
A
Alexey Milovidov 已提交
2083 2084 2085 2086
            /// Do not clone from lost replicas.
            String source_replica_is_lost_value;
            if (!zookeeper->tryGet(source_replica_path + "/is_lost", source_replica_is_lost_value, &source_is_lost_stat)
                || source_replica_is_lost_value == "0")
V
Vadim 已提交
2087
            {
2088
                source_replica = replica_name;
V
VadimPE 已提交
2089
                break;
V
Vadim 已提交
2090
            }
2091
        }
V
VadimPE 已提交
2092
    }
A
Alexey Milovidov 已提交
2093

A
Alexey Milovidov 已提交
2094
    if (source_replica.empty())
V
VadimPE 已提交
2095
        throw Exception("All replicas are lost", ErrorCodes::ALL_REPLICAS_LOST);
A
Alexey Milovidov 已提交
2096

A
Alexey Milovidov 已提交
2097
    /// Will do repair from the selected replica.
V
VadimPE 已提交
2098
    cloneReplica(source_replica, source_is_lost_stat, zookeeper);
A
Alexey Milovidov 已提交
2099
    /// If repair fails to whatever reason, the exception is thrown, is_lost will remain "1" and the replica will be repaired later.
2100

A
Alexey Milovidov 已提交
2101
    /// If replica is repaired successfully, we remove is_lost flag.
2102
    zookeeper->set(replica_path + "/is_lost", "0");
2103 2104 2105
}


2106
void StorageReplicatedMergeTree::queueUpdatingTask()
M
Merge  
Michael Kolupaev 已提交
2107
{
2108
    if (!queue_update_in_progress)
S
Silviu Caragea 已提交
2109
    {
2110 2111 2112 2113 2114
        last_queue_update_start_time.store(time(nullptr));
        queue_update_in_progress = true;
    }
    try
    {
2115
        queue.pullLogsToQueue(getZooKeeper(), queue_updating_task->getWatchCallback());
2116 2117
        last_queue_update_finish_time.store(time(nullptr));
        queue_update_in_progress = false;
2118
    }
2119
    catch (const Coordination::Exception & e)
2120 2121
    {
        tryLogCurrentException(log, __PRETTY_FUNCTION__);
2122

2123
        if (e.code == Coordination::ZSESSIONEXPIRED)
2124
        {
2125
            restarting_thread.wakeup();
2126
            return;
2127
        }
2128

2129
        queue_updating_task->scheduleAfter(QUEUE_UPDATE_ERROR_SLEEP_MS);
2130
    }
2131 2132
    catch (...)
    {
2133
        tryLogCurrentException(log, __PRETTY_FUNCTION__);
2134
        queue_updating_task->scheduleAfter(QUEUE_UPDATE_ERROR_SLEEP_MS);
S
Silviu Caragea 已提交
2135
    }
M
Merge  
Michael Kolupaev 已提交
2136
}
M
Merge  
Michael Kolupaev 已提交
2137

A
Merge  
Alexey Milovidov 已提交
2138

2139
void StorageReplicatedMergeTree::mutationsUpdatingTask()
2140 2141 2142
{
    try
    {
2143
        queue.updateMutations(getZooKeeper(), mutations_updating_task->getWatchCallback());
2144
    }
2145
    catch (const Coordination::Exception & e)
2146 2147 2148
    {
        tryLogCurrentException(log, __PRETTY_FUNCTION__);

2149
        if (e.code == Coordination::ZSESSIONEXPIRED)
2150 2151
            return;

2152
        mutations_updating_task->scheduleAfter(QUEUE_UPDATE_ERROR_SLEEP_MS);
2153 2154 2155 2156
    }
    catch (...)
    {
        tryLogCurrentException(log, __PRETTY_FUNCTION__);
2157
        mutations_updating_task->scheduleAfter(QUEUE_UPDATE_ERROR_SLEEP_MS);
2158
    }
2159 2160 2161
}


2162
bool StorageReplicatedMergeTree::queueTask()
M
Merge  
Michael Kolupaev 已提交
2163
{
2164
    /// If replication queue is stopped exit immediately as we successfully executed the task
2165
    if (queue.actions_blocker.isCancelled())
2166 2167 2168 2169 2170
    {
        std::this_thread::sleep_for(std::chrono::milliseconds(5));
        return true;
    }

2171 2172 2173 2174 2175
    /// This object will mark the element of the queue as running.
    ReplicatedMergeTreeQueue::SelectedEntry selected;

    try
    {
2176
        selected = queue.selectEntryToProcess(merger_mutator, data);
2177 2178 2179
    }
    catch (...)
    {
2180
        tryLogCurrentException(log, __PRETTY_FUNCTION__);
2181 2182 2183 2184 2185 2186 2187 2188 2189 2190 2191 2192 2193 2194 2195 2196 2197 2198 2199 2200 2201 2202 2203 2204 2205 2206 2207
    }

    LogEntryPtr & entry = selected.first;

    if (!entry)
        return false;

    time_t prev_attempt_time = entry->last_attempt_time;

    bool res = queue.processEntry([this]{ return getZooKeeper(); }, entry, [&](LogEntryPtr & entry)
    {
        try
        {
            return executeLogEntry(*entry);
        }
        catch (const Exception & e)
        {
            if (e.code() == ErrorCodes::NO_REPLICA_HAS_PART)
            {
                /// If no one has the right part, probably not all replicas work; We will not write to log with Error level.
                LOG_INFO(log, e.displayText());
            }
            else if (e.code() == ErrorCodes::ABORTED)
            {
                /// Interrupted merge or downloading a part is not an error.
                LOG_INFO(log, e.message());
            }
2208 2209 2210 2211
            else if (e.code() == ErrorCodes::PART_IS_TEMPORARILY_LOCKED)
            {
                /// Part cannot be added temporarily
                LOG_INFO(log, e.displayText());
2212
                cleanup_thread.wakeup();
2213
            }
2214
            else
2215
                tryLogCurrentException(log, __PRETTY_FUNCTION__);
2216 2217 2218 2219 2220 2221 2222 2223 2224

            /** This exception will be written to the queue element, and it can be looked up using `system.replication_queue` table.
              * The thread that performs this action will sleep a few seconds after the exception.
              * See `queue.processEntry` function.
              */
            throw;
        }
        catch (...)
        {
2225
            tryLogCurrentException(log, __PRETTY_FUNCTION__);
2226 2227 2228 2229 2230 2231 2232 2233 2234
            throw;
        }
    });

    /// We will go to sleep if the processing fails and if we have already processed this record recently.
    bool need_sleep = !res && (entry->last_attempt_time - prev_attempt_time < 10);

    /// If there was no exception, you do not need to sleep.
    return !need_sleep;
M
Merge  
Michael Kolupaev 已提交
2235 2236
}

A
Merge  
Alexey Milovidov 已提交
2237

2238
void StorageReplicatedMergeTree::mergeSelectingTask()
2239
{
2240
    if (!is_leader)
2241
        return;
2242

2243
    const bool deduplicate = false; /// TODO: read deduplicate option from table config
2244

2245
    bool success = false;
2246

2247
    try
S
Silviu Caragea 已提交
2248
    {
2249 2250
        /// We must select parts for merge under merge_selecting_mutex because other threads
        /// (OPTIMIZE queries) can assign new merges.
2251
        std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
2252

2253
        auto zookeeper = getZooKeeper();
2254

2255
        ReplicatedMergeTreeMergePredicate merge_pred = queue.getMergePredicate(zookeeper);
2256

2257 2258 2259
        /// If many merges is already queued, then will queue only small enough merges.
        /// Otherwise merge queue could be filled with only large merges,
        /// and in the same time, many small parts could be created and won't be merged.
2260
        size_t merges_and_mutations_queued = queue.countMergesAndPartMutations();
2261
        if (merges_and_mutations_queued >= data.settings.max_replicated_merges_in_queue)
2262
        {
2263
            LOG_TRACE(log, "Number of queued merges and part mutations (" << merges_and_mutations_queued
2264
                << ") is greater than max_replicated_merges_in_queue ("
2265
                << data.settings.max_replicated_merges_in_queue << "), so won't select new parts to merge or mutate.");
2266 2267
        }
        else
2268
        {
2269 2270
            size_t max_source_parts_size = merger_mutator.getMaxSourcePartsSize(
                data.settings.max_replicated_merges_in_queue, merges_and_mutations_queued);
2271

2272
            if (max_source_parts_size > 0)
2273
            {
2274 2275
                MergeTreeDataMergerMutator::FuturePart future_merged_part;
                if (merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size, merge_pred))
2276
                {
2277 2278
                    success = createLogEntryToMergeParts(zookeeper, future_merged_part.parts, future_merged_part.name, deduplicate);
                }
2279
                else if (queue.countMutations() > 0)
2280 2281
                {
                    /// Choose a part to mutate.
2282

2283 2284
                    MergeTreeData::DataPartsVector data_parts = data.getDataPartsVector();
                    for (const auto & part : data_parts)
2285
                    {
2286 2287
                        if (part->bytes_on_disk > max_source_parts_size)
                            continue;
2288

2289 2290 2291
                        std::optional<Int64> desired_mutation_version = merge_pred.getDesiredMutationVersion(part);
                        if (!desired_mutation_version)
                            continue;
2292

2293
                        if (createLogEntryToMutatePart(*part, *desired_mutation_version))
2294
                        {
2295 2296
                            success = true;
                            break;
2297 2298
                        }
                    }
2299
                }
2300 2301
            }
        }
2302 2303 2304
    }
    catch (...)
    {
2305
        tryLogCurrentException(log, __PRETTY_FUNCTION__);
2306
    }
2307

2308
    if (!is_leader)
2309
        return;
S
Silviu Caragea 已提交
2310

2311
    if (!success)
2312
        merge_selecting_task->scheduleAfter(MERGE_SELECTING_SLEEP_MS);
2313
    else
2314
        merge_selecting_task->schedule();
2315

M
Merge  
Michael Kolupaev 已提交
2316 2317
}

M
Merge  
Michael Kolupaev 已提交
2318

2319 2320 2321 2322 2323 2324 2325 2326 2327 2328 2329 2330 2331 2332 2333 2334 2335 2336 2337
void StorageReplicatedMergeTree::mutationsFinalizingTask()
{
    bool needs_reschedule = false;

    try
    {
        needs_reschedule = queue.tryFinalizeMutations(getZooKeeper());
    }
    catch (...)
    {
        tryLogCurrentException(log, __PRETTY_FUNCTION__);
        needs_reschedule = true;
    }

    if (needs_reschedule)
        mutations_finalizing_task->scheduleAfter(MUTATIONS_FINALIZING_SLEEP_MS);
}


2338
bool StorageReplicatedMergeTree::createLogEntryToMergeParts(
2339 2340 2341 2342 2343
    zkutil::ZooKeeperPtr & zookeeper,
    const MergeTreeData::DataPartsVector & parts,
    const String & merged_name,
    bool deduplicate,
    ReplicatedMergeTreeLogEntryData * out_log_entry)
2344
{
2345
    std::vector<std::future<Coordination::ExistsResponse>> exists_futures;
2346
    exists_futures.reserve(parts.size());
2347
    for (const auto & part : parts)
2348 2349 2350 2351
        exists_futures.emplace_back(zookeeper->asyncExists(replica_path + "/parts/" + part->name));

    bool all_in_zk = true;
    for (size_t i = 0; i < parts.size(); ++i)
2352 2353
    {
        /// If there is no information about part in ZK, we will not merge it.
2354
        if (exists_futures[i].get().error == Coordination::ZNONODE)
2355 2356 2357
        {
            all_in_zk = false;

2358
            const auto & part = parts[i];
2359
            if (part->modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER < time(nullptr))
2360 2361
            {
                LOG_WARNING(log, "Part " << part->name << " (that was selected for merge)"
2362
                    << " with age " << (time(nullptr) - part->modification_time)
2363 2364 2365 2366 2367 2368
                    << " seconds exists locally but not in ZooKeeper."
                    << " Won't do merge with that part and will check it.");
                enqueuePartForCheck(part->name);
            }
        }
    }
2369

2370 2371 2372
    if (!all_in_zk)
        return false;

2373
    ReplicatedMergeTreeLogEntryData entry;
2374 2375 2376
    entry.type = LogEntry::MERGE_PARTS;
    entry.source_replica = replica_name;
    entry.new_part_name = merged_name;
Y
Yuri Dyachenko 已提交
2377
    entry.deduplicate = deduplicate;
2378
    entry.create_time = time(nullptr);
2379 2380

    for (const auto & part : parts)
2381
        entry.source_parts.push_back(part->name);
2382 2383 2384 2385 2386 2387 2388 2389

    String path_created = zookeeper->create(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential);
    entry.znode_name = path_created.substr(path_created.find_last_of('/') + 1);

    if (out_log_entry)
        *out_log_entry = entry;

    return true;
2390 2391 2392
}


2393 2394 2395 2396 2397 2398 2399 2400 2401 2402 2403 2404 2405 2406 2407 2408 2409 2410 2411 2412 2413 2414
bool StorageReplicatedMergeTree::createLogEntryToMutatePart(const MergeTreeDataPart & part, Int64 mutation_version)
{
    auto zookeeper = getZooKeeper();

    /// If there is no information about part in ZK, we will not mutate it.
    if (!zookeeper->exists(replica_path + "/parts/" + part.name))
    {
        if (part.modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER < time(nullptr))
        {
            LOG_WARNING(log, "Part " << part.name << " (that was selected for mutation)"
                << " with age " << (time(nullptr) - part.modification_time)
                << " seconds exists locally but not in ZooKeeper."
                << " Won't mutate that part and will check it.");
            enqueuePartForCheck(part.name);
        }

        return false;
    }

    MergeTreePartInfo new_part_info = part.info;
    new_part_info.mutation = mutation_version;

2415
    String new_part_name = part.getNewName(new_part_info);
2416 2417 2418 2419 2420 2421 2422 2423 2424 2425 2426 2427 2428

    ReplicatedMergeTreeLogEntryData entry;
    entry.type = LogEntry::MUTATE_PART;
    entry.source_replica = replica_name;
    entry.source_parts.push_back(part.name);
    entry.new_part_name = new_part_name;
    entry.create_time = time(nullptr);

    zookeeper->create(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential);
    return true;
}


2429
void StorageReplicatedMergeTree::removePartFromZooKeeper(const String & part_name, Coordination::Requests & ops)
2430
{
2431
    String part_path = replica_path + "/parts/" + part_name;
2432

2433 2434 2435
    ops.emplace_back(zkutil::makeRemoveRequest(part_path + "/checksums", -1));
    ops.emplace_back(zkutil::makeRemoveRequest(part_path + "/columns", -1));
    ops.emplace_back(zkutil::makeRemoveRequest(part_path, -1));
2436 2437 2438
}


M
Merge  
Michael Kolupaev 已提交
2439 2440
void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_name)
{
2441
    auto zookeeper = getZooKeeper();
A
Merge  
Alexey Milovidov 已提交
2442

2443
    String part_path = replica_path + "/parts/" + part_name;
M
Merge  
Michael Kolupaev 已提交
2444

2445 2446 2447 2448 2449
    LogEntryPtr log_entry = std::make_shared<LogEntry>();
    log_entry->type = LogEntry::GET_PART;
    log_entry->create_time = tryGetPartCreateTime(zookeeper, replica_path, part_name);
    log_entry->source_replica = "";
    log_entry->new_part_name = part_name;
M
Merge  
Michael Kolupaev 已提交
2450

2451
    Coordination::Requests ops;
2452 2453
    ops.emplace_back(zkutil::makeCreateRequest(
        replica_path + "/queue/queue-", log_entry->toString(),
2454
        zkutil::CreateMode::PersistentSequential));
2455

2456
    removePartFromZooKeeper(part_name, ops);
2457

2458
    auto results = zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
2459

2460
    String path_created = dynamic_cast<const Coordination::CreateResponse &>(*results[0]).path_created;
2461 2462
    log_entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
    queue.insert(zookeeper, log_entry);
M
Merge  
Michael Kolupaev 已提交
2463 2464
}

A
Merge  
Alexey Milovidov 已提交
2465

2466
void StorageReplicatedMergeTree::enterLeaderElection()
M
Merge  
Michael Kolupaev 已提交
2467
{
2468 2469 2470 2471 2472 2473
    auto callback = [this]()
    {
        CurrentMetrics::add(CurrentMetrics::LeaderReplica);
        LOG_INFO(log, "Became leader");

        is_leader = true;
2474
        merge_selecting_task->activateAndSchedule();
2475
    };
2476

2477 2478 2479
    try
    {
        leader_election = std::make_shared<zkutil::LeaderElection>(
2480
            context.getSchedulePool(),
2481 2482 2483 2484 2485 2486 2487 2488 2489 2490 2491 2492 2493 2494 2495 2496
            zookeeper_path + "/leader_election",
            *current_zookeeper,    /// current_zookeeper lives for the lifetime of leader_election,
                                   ///  since before changing `current_zookeeper`, `leader_election` object is destroyed in `partialShutdown` method.
            callback,
            replica_name);
    }
    catch (...)
    {
        leader_election = nullptr;
        throw;
    }
}

void StorageReplicatedMergeTree::exitLeaderElection()
{
    if (!leader_election)
2497
        return;
2498

2499 2500 2501 2502 2503
    /// Shut down the leader election thread to avoid suddenly becoming the leader again after
    /// we have stopped the merge_selecting_thread, but before we have deleted the leader_election object.
    leader_election->shutdown();

    if (is_leader)
2504
    {
2505 2506 2507 2508
        CurrentMetrics::sub(CurrentMetrics::LeaderReplica);
        LOG_INFO(log, "Stopped being leader");

        is_leader = false;
2509
        merge_selecting_task->deactivate();
2510 2511
    }

2512 2513 2514
    /// Delete the node in ZK only after we have stopped the merge_selecting_thread - so that only one
    /// replica assigns merges at any given time.
    leader_election = nullptr;
M
Merge  
Michael Kolupaev 已提交
2515 2516
}

A
Merge  
Alexey Milovidov 已提交
2517

M
Merge  
Michael Kolupaev 已提交
2518
String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_name, bool active)
M
Merge  
Michael Kolupaev 已提交
2519
{
2520 2521
    auto zookeeper = getZooKeeper();
    Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
M
Merge  
Michael Kolupaev 已提交
2522

2523
    /// Select replicas in uniformly random order.
2524
    std::shuffle(replicas.begin(), replicas.end(), rng);
M
Merge  
Michael Kolupaev 已提交
2525

2526 2527 2528 2529 2530
    for (const String & replica : replicas)
    {
        /// We don't interested in ourself.
        if (replica == replica_name)
            continue;
A
Alexey Milovidov 已提交
2531

2532 2533 2534
        if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name) &&
            (!active || zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")))
            return replica;
2535

2536 2537
        /// Obviously, replica could become inactive or even vanish after return from this method.
    }
M
Merge  
Michael Kolupaev 已提交
2538

2539
    return {};
2540 2541 2542
}


2543
String StorageReplicatedMergeTree::findReplicaHavingCoveringPart(LogEntry & entry, bool active)
2544
{
2545 2546 2547 2548
    auto zookeeper = getZooKeeper();
    Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");

    /// Select replicas in uniformly random order.
2549
    std::shuffle(replicas.begin(), replicas.end(), rng);
2550 2551 2552 2553 2554 2555 2556 2557 2558 2559 2560 2561 2562

    for (const String & replica : replicas)
    {
        if (replica == replica_name)
            continue;

        if (active && !zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))
            continue;

        String largest_part_found;
        Strings parts = zookeeper->getChildren(zookeeper_path + "/replicas/" + replica + "/parts");
        for (const String & part_on_replica : parts)
        {
2563 2564
            if (part_on_replica == entry.new_part_name
                || MergeTreePartInfo::contains(part_on_replica, entry.new_part_name, data.format_version))
2565
            {
2566 2567
                if (largest_part_found.empty()
                    || MergeTreePartInfo::contains(part_on_replica, largest_part_found, data.format_version))
2568 2569 2570 2571 2572 2573 2574 2575
                {
                    largest_part_found = part_on_replica;
                }
            }
        }

        if (!largest_part_found.empty())
        {
2576 2577 2578 2579 2580 2581 2582 2583 2584 2585 2586 2587 2588 2589 2590 2591 2592
            bool the_same_part = largest_part_found == entry.new_part_name;

            /// Make a check in case if selected part differs from source part
            if (!the_same_part)
            {
                String reject_reason;
                if (!queue.addFuturePartIfNotCoveredByThem(largest_part_found, entry, reject_reason))
                {
                    LOG_INFO(log, "Will not fetch part " << largest_part_found << " covering " << entry.new_part_name << ". " << reject_reason);
                    return {};
                }
            }
            else
            {
                entry.actual_new_part_name = entry.new_part_name;
            }

2593 2594 2595 2596 2597
            return replica;
        }
    }

    return {};
M
Merge  
Michael Kolupaev 已提交
2598 2599
}

A
Merge  
Alexey Milovidov 已提交
2600

2601 2602 2603 2604 2605 2606 2607 2608 2609 2610 2611 2612 2613 2614 2615 2616 2617 2618 2619 2620 2621 2622 2623 2624 2625 2626 2627 2628 2629 2630 2631 2632 2633 2634 2635 2636 2637 2638 2639 2640 2641 2642
String StorageReplicatedMergeTree::findReplicaHavingCoveringPart(
    const String & part_name, bool active, String & found_part_name)
{
    auto zookeeper = getZooKeeper();
    Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");

    /// Select replicas in uniformly random order.
    std::shuffle(replicas.begin(), replicas.end(), rng);

    String largest_part_found;
    String largest_replica_found;

    for (const String & replica : replicas)
    {
        if (replica == replica_name)
            continue;

        if (active && !zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))
            continue;

        Strings parts = zookeeper->getChildren(zookeeper_path + "/replicas/" + replica + "/parts");
        for (const String & part_on_replica : parts)
        {
            if (part_on_replica == part_name
                || MergeTreePartInfo::contains(part_on_replica, part_name, data.format_version))
            {
                if (largest_part_found.empty()
                    || MergeTreePartInfo::contains(part_on_replica, largest_part_found, data.format_version))
                {
                    largest_part_found = part_on_replica;
                    largest_replica_found = replica;
                }
            }
        }
    }

    found_part_name = largest_part_found;
    return largest_replica_found;
}



F
f1yegor 已提交
2643
/** If a quorum is tracked for a part, update information about it in ZK.
2644
  */
2645
void StorageReplicatedMergeTree::updateQuorum(const String & part_name)
2646
{
2647 2648 2649 2650 2651 2652 2653 2654
    auto zookeeper = getZooKeeper();

    /// Information on which replicas a part has been added, if the quorum has not yet been reached.
    const String quorum_status_path = zookeeper_path + "/quorum/status";
    /// The name of the previous part for which the quorum was reached.
    const String quorum_last_part_path = zookeeper_path + "/quorum/last_part";

    String value;
2655
    Coordination::Stat stat;
2656 2657 2658 2659 2660 2661 2662 2663 2664 2665 2666 2667 2668 2669 2670 2671 2672 2673 2674

    /// If there is no node, then all quorum INSERTs have already reached the quorum, and nothing is needed.
    while (zookeeper->tryGet(quorum_status_path, value, &stat))
    {
        ReplicatedMergeTreeQuorumEntry quorum_entry;
        quorum_entry.fromString(value);

        if (quorum_entry.part_name != part_name)
        {
            /// The quorum has already been achieved. Moreover, another INSERT with a quorum has already started.
            break;
        }

        quorum_entry.replicas.insert(replica_name);

        if (quorum_entry.replicas.size() >= quorum_entry.required_number_of_replicas)
        {
            /// The quorum is reached. Delete the node, and update information about the last part that was successfully written with quorum.

2675 2676
            Coordination::Requests ops;
            Coordination::Responses responses;
2677 2678
            ops.emplace_back(zkutil::makeRemoveRequest(quorum_status_path, stat.version));
            ops.emplace_back(zkutil::makeSetRequest(quorum_last_part_path, part_name, -1));
2679
            auto code = zookeeper->tryMulti(ops, responses);
2680

2681
            if (code == Coordination::ZOK)
2682 2683 2684
            {
                break;
            }
2685
            else if (code == Coordination::ZNONODE)
2686 2687 2688 2689
            {
                /// The quorum has already been achieved.
                break;
            }
2690
            else if (code == Coordination::ZBADVERSION)
2691 2692 2693 2694 2695
            {
                /// Node was updated meanwhile. We must re-read it and repeat all the actions.
                continue;
            }
            else
2696
                throw Coordination::Exception(code, quorum_status_path);
2697 2698 2699 2700 2701 2702
        }
        else
        {
            /// We update the node, registering there one more replica.
            auto code = zookeeper->trySet(quorum_status_path, quorum_entry.toString(), stat.version);

2703
            if (code == Coordination::ZOK)
2704 2705 2706
            {
                break;
            }
2707
            else if (code == Coordination::ZNONODE)
2708 2709 2710 2711
            {
                /// The quorum has already been achieved.
                break;
            }
2712
            else if (code == Coordination::ZBADVERSION)
2713 2714 2715 2716 2717
            {
                /// Node was updated meanwhile. We must re-read it and repeat all the actions.
                continue;
            }
            else
2718
                throw Coordination::Exception(code, quorum_status_path);
2719 2720
        }
    }
2721 2722 2723
}


A
Alexey Milovidov 已提交
2724
bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const String & replica_path, bool to_detached, size_t quorum)
M
Merge  
Michael Kolupaev 已提交
2725
{
2726 2727 2728
    const auto part_info = MergeTreePartInfo::fromPartName(part_name, data.format_version);

    if (auto part = data.getPartIfExists(part_info, {MergeTreeDataPart::State::Outdated, MergeTreeDataPart::State::Deleting}))
2729 2730
    {
        LOG_DEBUG(log, "Part " << part->getNameWithState() << " should be deleted after previous attempt before fetch");
2731
        /// Force immediate parts cleanup to delete the part that was left from the previous fetch attempt.
2732
        cleanup_thread.wakeup();
2733 2734 2735
        return false;
    }

2736 2737 2738 2739 2740 2741 2742 2743 2744 2745 2746 2747 2748 2749 2750 2751 2752 2753 2754
    {
        std::lock_guard<std::mutex> lock(currently_fetching_parts_mutex);
        if (!currently_fetching_parts.insert(part_name).second)
        {
            LOG_DEBUG(log, "Part " << part_name << " is already fetching right now");
            return false;
        }
    }

    SCOPE_EXIT
    ({
        std::lock_guard<std::mutex> lock(currently_fetching_parts_mutex);
        currently_fetching_parts.erase(part_name);
    });

    LOG_DEBUG(log, "Fetching part " << part_name << " from " << replica_path);

    TableStructureReadLockPtr table_lock;
    if (!to_detached)
2755
        table_lock = lockStructure(true, __PRETTY_FUNCTION__);
2756

2757
    /// Logging
2758
    Stopwatch stopwatch;
2759 2760
    MergeTreeData::MutableDataPartPtr part;
    MergeTreeData::DataPartsVector replaced_parts;
2761

2762
    auto write_part_log = [&] (const ExecutionStatus & execution_status)
2763
    {
2764 2765 2766
        writePartLog(
            PartLogElement::DOWNLOAD_PART, execution_status, stopwatch.elapsed(),
            part_name, part, replaced_parts, nullptr);
2767 2768
    };

2769 2770 2771 2772 2773 2774 2775 2776 2777
    MergeTreeData::DataPartPtr part_to_clone;
    {
        /// If the desired part is a result of a part mutation, try to find the source part and compare
        /// its checksums to the checksums of the desired part. If they match, we can just clone the local part.

        /// If we have the source part, its part_info will contain covered_part_info.
        auto covered_part_info = part_info;
        covered_part_info.mutation = 0;
        auto source_part = data.getActiveContainingPart(covered_part_info);
2778

2779 2780 2781 2782 2783 2784 2785 2786 2787 2788 2789 2790 2791 2792 2793 2794 2795 2796
        if (source_part)
        {
            MinimalisticDataPartChecksums source_part_checksums;
            source_part_checksums.computeTotalChecksums(source_part->checksums);

            String desired_checksums_str = getZooKeeper()->get(replica_path + "/parts/" + part_name + "/checksums");
            auto desired_checksums = MinimalisticDataPartChecksums::deserializeFrom(desired_checksums_str);
            if (source_part_checksums == desired_checksums)
            {
                LOG_TRACE(log, "Found local part " << source_part->name << " with the same checksums as " << part_name);
                part_to_clone = source_part;
            }
        }

    }

    std::function<MergeTreeData::MutableDataPartPtr()> get_part;
    if (part_to_clone)
2797
    {
2798 2799 2800 2801 2802 2803 2804 2805 2806
        get_part = [&, part_to_clone]()
        {
            return data.cloneAndLoadDataPart(part_to_clone, "tmp_clone_", part_info);
        };
    }
    else
    {
        ReplicatedMergeTreeAddress address(getZooKeeper()->get(replica_path + "/host"));
        auto timeouts = ConnectionTimeouts::getHTTPTimeouts(context.getSettingsRef());
2807
        auto user_password = context.getInterserverCredentials();
2808 2809
        String interserver_scheme = context.getInterserverScheme();

2810
        get_part = [&, address, timeouts, user_password, interserver_scheme]()
2811 2812 2813 2814 2815
        {
            if (interserver_scheme != address.scheme)
                throw Exception("Interserver schemes are different: '" + interserver_scheme
                    + "' != '" + address.scheme + "', can't fetch part from " + address.host,
                    ErrorCodes::LOGICAL_ERROR);
2816

2817 2818 2819
            return fetcher.fetchPart(
                part_name, replica_path,
                address.host, address.replication_port,
2820
                timeouts, user_password.first, user_password.second, interserver_scheme, to_detached);
2821 2822 2823 2824 2825 2826
        };
    }

    try
    {
        part = get_part();
2827

2828
        if (!to_detached)
2829
        {
2830
            MergeTreeData::Transaction transaction(data);
2831
            data.renameTempPartAndReplace(part, nullptr, &transaction);
2832

2833 2834 2835 2836 2837
            /** NOTE
              * Here, an error occurs if ALTER occurred with a change in the column type or column deletion,
              *  and the part on remote server has not yet been modified.
              * After a while, one of the following attempts to make `fetchPart` succeed.
              */
2838
            replaced_parts = checkPartChecksumsAndCommit(transaction, part);
2839

2840 2841 2842 2843 2844
            /** If a quorum is tracked for this part, you must update it.
              * If you do not have time, in case of losing the session, when you restart the server - see the `ReplicatedMergeTreeRestartingThread::updateQuorumIfWeHavePart` method.
              */
            if (quorum)
                updateQuorum(part_name);
2845

2846
            merge_selecting_task->schedule();
2847 2848 2849 2850 2851 2852

            for (const auto & replaced_part : replaced_parts)
            {
                LOG_DEBUG(log, "Part " << replaced_part->name << " is rendered obsolete by fetching part " << part_name);
                ProfileEvents::increment(ProfileEvents::ObsoleteReplicatedParts);
            }
2853

2854 2855 2856
            write_part_log({});
        }
        else
2857
        {
2858
            part->renameTo("detached/" + part_name);
2859 2860
        }
    }
2861
    catch (...)
2862
    {
2863 2864 2865 2866
        if (!to_detached)
            write_part_log(ExecutionStatus::fromCurrentException());

        throw;
2867 2868 2869 2870
    }

    ProfileEvents::increment(ProfileEvents::ReplicatedPartFetches);

2871 2872 2873 2874 2875
    if (part_to_clone)
        LOG_DEBUG(log, "Cloned part " << part_name << " from " << part_to_clone->name << (to_detached ? " (to 'detached' directory)" : ""));
    else
        LOG_DEBUG(log, "Fetched part " << part_name << " from " << replica_path << (to_detached ? " (to 'detached' directory)" : ""));

2876
    return true;
M
Merge  
Michael Kolupaev 已提交
2877
}
M
Merge  
Michael Kolupaev 已提交
2878

A
Merge  
Alexey Milovidov 已提交
2879

2880 2881 2882 2883 2884 2885 2886 2887
void StorageReplicatedMergeTree::startup()
{
    if (is_readonly)
        return;

    queue.initialize(
        zookeeper_path, replica_path,
        database_name + "." + table_name + " (ReplicatedMergeTreeQueue)",
V
Vadim 已提交
2888
        data.getDataParts());
2889

A
Alexey Zatelepin 已提交
2890 2891 2892 2893 2894
    StoragePtr ptr = shared_from_this();
    InterserverIOEndpointPtr data_parts_exchange_endpoint = std::make_shared<DataPartsExchange::Service>(data, ptr);
    data_parts_exchange_endpoint_holder = std::make_shared<InterserverIOEndpointHolder>(
        data_parts_exchange_endpoint->getId(replica_path), data_parts_exchange_endpoint, context.getInterserverIOHandler());

2895 2896
    queue_task_handle = context.getBackgroundPool().addTask([this] { return queueTask(); });

2897
    /// In this thread replica will be activated.
2898
    restarting_thread.start();
2899 2900 2901

    /// Wait while restarting_thread initializes LeaderElection (and so on) or makes first attmept to do it
    startup_event.wait();
2902 2903 2904
}


M
Merge  
Michael Kolupaev 已提交
2905 2906
void StorageReplicatedMergeTree::shutdown()
{
2907 2908 2909 2910 2911 2912 2913 2914 2915
    /// Cancel fetches, merges and mutations to force the queue_task to finish ASAP.
    fetcher.blocker.cancelForever();
    merger_mutator.actions_blocker.cancelForever();

    restarting_thread.shutdown();

    if (queue_task_handle)
        context.getBackgroundPool().removeTask(queue_task_handle);
    queue_task_handle.reset();
2916

2917
    if (data_parts_exchange_endpoint_holder)
2918
    {
2919
        data_parts_exchange_endpoint_holder->getBlocker().cancelForever();
2920
        data_parts_exchange_endpoint_holder = nullptr;
2921
    }
M
Merge  
Michael Kolupaev 已提交
2922 2923 2924
}


M
Merge  
Michael Kolupaev 已提交
2925 2926
StorageReplicatedMergeTree::~StorageReplicatedMergeTree()
{
2927 2928 2929 2930 2931 2932 2933 2934
    try
    {
        shutdown();
    }
    catch(...)
    {
        tryLogCurrentException(__PRETTY_FUNCTION__);
    }
M
Merge  
Michael Kolupaev 已提交
2935 2936
}

A
Merge  
Alexey Milovidov 已提交
2937

M
Merge  
Michael Kolupaev 已提交
2938
BlockInputStreams StorageReplicatedMergeTree::read(
2939
    const Names & column_names,
2940
    const SelectQueryInfo & query_info,
2941
    const Context & context,
2942
    QueryProcessingStage::Enum /*processed_stage*/,
2943
    const size_t max_block_size,
2944
    const unsigned num_streams)
M
Merge  
Michael Kolupaev 已提交
2945
{
2946
    const Settings & settings = context.getSettingsRef();
2947

2948 2949 2950 2951 2952 2953 2954
    /** The `select_sequential_consistency` setting has two meanings:
    * 1. To throw an exception if on a replica there are not all parts which have been written down on quorum of remaining replicas.
    * 2. Do not read parts that have not yet been written to the quorum of the replicas.
    * For this you have to synchronously go to ZooKeeper.
    */
    Int64 max_block_number_to_read = 0;
    if (settings.select_sequential_consistency)
2955
    {
2956
        auto zookeeper = getZooKeeper();
2957

2958 2959
        String last_part;
        zookeeper->tryGet(zookeeper_path + "/quorum/last_part", last_part);
2960

2961
        if (!last_part.empty() && !data.getActiveContainingPart(last_part))    /// TODO Disable replica for distributed queries.
2962 2963
            throw Exception("Replica doesn't have part " + last_part + " which was successfully written to quorum of other replicas."
                " Send query to another replica or disable 'select_sequential_consistency' setting.", ErrorCodes::REPLICA_IS_NOT_IN_QUORUM);
2964

2965 2966 2967 2968
        if (last_part.empty())  /// If no part has been written with quorum.
        {
            String quorum_str;
            if (zookeeper->tryGet(zookeeper_path + "/quorum/status", quorum_str))
2969
            {
2970 2971
                ReplicatedMergeTreeQuorumEntry quorum_entry;
                quorum_entry.fromString(quorum_str);
2972
                auto part_info = MergeTreePartInfo::fromPartName(quorum_entry.part_name, data.format_version);
2973
                max_block_number_to_read = part_info.min_block - 1;
2974 2975
            }
        }
2976
        else
2977
        {
2978
            auto part_info = MergeTreePartInfo::fromPartName(last_part, data.format_version);
2979
            max_block_number_to_read = part_info.max_block;
2980 2981 2982
        }
    }

2983
    return reader.read(column_names, query_info, context, max_block_size, num_streams, max_block_number_to_read);
M
Merge  
Michael Kolupaev 已提交
2984 2985
}

A
Merge  
Alexey Milovidov 已提交
2986

A
Merge  
Alexey Milovidov 已提交
2987
void StorageReplicatedMergeTree::assertNotReadonly() const
M
Merge  
Michael Kolupaev 已提交
2988
{
2989 2990
    if (is_readonly)
        throw Exception("Table is in readonly mode", ErrorCodes::TABLE_IS_READ_ONLY);
A
Merge  
Alexey Milovidov 已提交
2991 2992 2993
}


A
Alexey Milovidov 已提交
2994
BlockOutputStreamPtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const Settings & settings)
A
Merge  
Alexey Milovidov 已提交
2995
{
2996
    assertNotReadonly();
M
Merge  
Michael Kolupaev 已提交
2997

2998 2999
    bool deduplicate = data.settings.replicated_deduplication_window != 0 && settings.insert_deduplicate;

3000
    return std::make_shared<ReplicatedMergeTreeBlockOutputStream>(*this,
3001
        settings.insert_quorum, settings.insert_quorum_timeout.totalMilliseconds(), deduplicate);
M
Merge  
Michael Kolupaev 已提交
3002
}
M
Merge  
Michael Kolupaev 已提交
3003

A
Merge  
Alexey Milovidov 已提交
3004

3005
bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context)
M
Merge  
Michael Kolupaev 已提交
3006
{
3007
    assertNotReadonly();
3008

3009
    if (!is_leader)
3010
    {
3011
        sendRequestToLeaderReplica(query, context.getSettingsRef());
3012 3013
        return true;
    }
3014

3015 3016
    ReplicatedMergeTreeLogEntryData merge_entry;
    {
3017 3018
        /// We must select parts for merge under merge_selecting_mutex because other threads
        /// (merge_selecting_thread or OPTIMIZE queries) could assign new merges.
3019
        std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
3020

3021
        size_t disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
3022 3023 3024
        auto zookeeper = getZooKeeper();
        ReplicatedMergeTreeMergePredicate can_merge = queue.getMergePredicate(zookeeper);

3025
        auto handle_noop = [&] (const String & message)
3026
        {
3027 3028
            if (context.getSettingsRef().optimize_throw_if_noop)
                throw Exception(message, ErrorCodes::CANNOT_ASSIGN_OPTIMIZE);
3029
            return false;
3030 3031
        };

3032
        if (!partition && final)
3033
        {
3034 3035 3036 3037 3038 3039 3040 3041 3042 3043 3044 3045 3046 3047 3048
            MergeTreeData::DataPartsVector data_parts = data.getDataPartsVector();
            std::unordered_set<String> partition_ids;

            for (const MergeTreeData::DataPartPtr & part : data_parts)
                partition_ids.emplace(part->info.partition_id);

            for (const String & partition_id : partition_ids)
            {
                MergeTreeDataMergerMutator::FuturePart future_merged_part;
                bool selected = merger_mutator.selectAllPartsToMergeWithinPartition(
                    future_merged_part, disk_space, can_merge, partition_id, true, nullptr);
                if (selected &&
                    !createLogEntryToMergeParts(zookeeper, future_merged_part.parts, future_merged_part.name, deduplicate, &merge_entry))
                    return handle_noop("Can't create merge queue node in ZooKeeper");
            }
3049
        }
3050 3051 3052 3053 3054 3055 3056 3057 3058 3059 3060 3061 3062 3063 3064 3065 3066 3067 3068 3069 3070 3071
        else
        {
            MergeTreeDataMergerMutator::FuturePart future_merged_part;
            String disable_reason;
            bool selected = false;
            if (!partition)
            {
                selected = merger_mutator.selectPartsToMerge(
                    future_merged_part, true, data.settings.max_bytes_to_merge_at_max_space_in_pool, can_merge, &disable_reason);
            }
            else
            {
                String partition_id = data.getPartitionIDFromQuery(partition, context);
                selected = merger_mutator.selectAllPartsToMergeWithinPartition(
                    future_merged_part, disk_space, can_merge, partition_id, final, &disable_reason);
            }

            if (!selected)
            {
                LOG_INFO(log, "Cannot select parts for optimization" + (disable_reason.empty() ? "" : ": " + disable_reason));
                return handle_noop(disable_reason);
            }
3072

3073 3074 3075
            if (!createLogEntryToMergeParts(zookeeper, future_merged_part.parts, future_merged_part.name, deduplicate, &merge_entry))
                return handle_noop("Can't create merge queue node in ZooKeeper");
        }
3076
    }
3077

3078 3079 3080 3081
    /// TODO: Bad setting name for such purpose
    if (context.getSettingsRef().replication_alter_partitions_sync != 0)
        waitForAllReplicasToProcessLogEntry(merge_entry);

3082
    return true;
M
Merge  
Michael Kolupaev 已提交
3083 3084
}

A
Merge  
Alexey Milovidov 已提交
3085

M
Merge  
Michael Kolupaev 已提交
3086
void StorageReplicatedMergeTree::alter(const AlterCommands & params,
A
Alexey Milovidov 已提交
3087
    const String & /*database_name*/, const String & /*table_name*/, const Context & context)
M
Merge  
Michael Kolupaev 已提交
3088
{
3089
    assertNotReadonly();
A
Merge  
Alexey Milovidov 已提交
3090

3091
    LOG_DEBUG(log, "Doing ALTER");
M
Merge  
Michael Kolupaev 已提交
3092

3093
    int new_columns_version = -1;   /// Initialization is to suppress (useless) false positive warning found by cppcheck.
3094
    String new_columns_str;
3095
    Coordination::Stat stat;
M
Merge  
Michael Kolupaev 已提交
3096

3097 3098
    {
        /// Just to read current structure. Alter will be done in separate thread.
3099
        auto table_lock = lockStructure(false, __PRETTY_FUNCTION__);
M
Merge  
Michael Kolupaev 已提交
3100

3101 3102
        if (is_readonly)
            throw Exception("Can't ALTER readonly table", ErrorCodes::TABLE_IS_READ_ONLY);
M
Merge  
Michael Kolupaev 已提交
3103

3104
        data.checkAlter(params);
M
Merge  
Michael Kolupaev 已提交
3105

3106 3107 3108 3109
        for (const AlterCommand & param : params)
            if (param.type == AlterCommand::MODIFY_PRIMARY_KEY)
                throw Exception("Modification of primary key is not supported for replicated tables", ErrorCodes::NOT_IMPLEMENTED);

3110
        ColumnsDescription new_columns = data.getColumns();
3111
        params.apply(new_columns);
3112

3113
        new_columns_str = new_columns.toString();
3114 3115 3116 3117 3118 3119 3120 3121 3122 3123 3124 3125 3126 3127 3128 3129 3130 3131 3132 3133 3134 3135 3136 3137 3138 3139 3140 3141 3142 3143 3144 3145 3146

        /// Do ALTER.
        getZooKeeper()->set(zookeeper_path + "/columns", new_columns_str, -1, &stat);

        new_columns_version = stat.version;
    }

    LOG_DEBUG(log, "Updated columns in ZooKeeper. Waiting for replicas to apply changes.");

    /// Wait until all replicas will apply ALTER.

    /// Subscribe to change of columns, to finish waiting if someone will do another ALTER.
    if (!getZooKeeper()->exists(zookeeper_path + "/columns", &stat, alter_query_event))
        throw Exception(zookeeper_path + "/columns doesn't exist", ErrorCodes::NOT_FOUND_NODE);

    if (stat.version != new_columns_version)
    {
        LOG_WARNING(log, zookeeper_path + "/columns changed before this ALTER finished; "
            "overlapping ALTER-s are fine but use caution with nontransitive changes");
        return;
    }

    Strings replicas = getZooKeeper()->getChildren(zookeeper_path + "/replicas");

    std::set<String> inactive_replicas;
    std::set<String> timed_out_replicas;

    time_t replication_alter_columns_timeout = context.getSettingsRef().replication_alter_columns_timeout;

    for (const String & replica : replicas)
    {
        LOG_DEBUG(log, "Waiting for " << replica << " to apply changes");

3147
        while (!partial_shutdown_called)
3148 3149 3150 3151 3152 3153
        {
            /// Replica could be inactive.
            if (!getZooKeeper()->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))
            {
                LOG_WARNING(log, "Replica " << replica << " is not active during ALTER query."
                    " ALTER will be done asynchronously when replica becomes active.");
3154

3155 3156 3157
                inactive_replicas.emplace(replica);
                break;
            }
3158

3159
            String replica_columns_str;
M
Merge  
Michael Kolupaev 已提交
3160

3161 3162 3163 3164 3165 3166
            /// Replica could has been removed.
            if (!getZooKeeper()->tryGet(zookeeper_path + "/replicas/" + replica + "/columns", replica_columns_str, &stat))
            {
                LOG_WARNING(log, replica << " was removed");
                break;
            }
M
Merge  
Michael Kolupaev 已提交
3167

3168
            int replica_columns_version = stat.version;
M
Merge  
Michael Kolupaev 已提交
3169

3170 3171 3172
            /// The ALTER has been successfully applied.
            if (replica_columns_str == new_columns_str)
                break;
M
Merge  
Michael Kolupaev 已提交
3173

3174 3175
            if (!getZooKeeper()->exists(zookeeper_path + "/columns", &stat))
                throw Exception(zookeeper_path + "/columns doesn't exist", ErrorCodes::NOT_FOUND_NODE);
3176

3177 3178 3179 3180 3181 3182
            if (stat.version != new_columns_version)
            {
                LOG_WARNING(log, zookeeper_path + "/columns changed before ALTER finished; "
                    "overlapping ALTER-s are fine but use caution with nontransitive changes");
                return;
            }
M
Merge  
Michael Kolupaev 已提交
3183

3184 3185 3186 3187 3188 3189 3190 3191 3192 3193 3194 3195 3196 3197 3198 3199 3200 3201 3202 3203 3204 3205 3206 3207 3208 3209 3210 3211
            if (!getZooKeeper()->exists(zookeeper_path + "/replicas/" + replica + "/columns", &stat, alter_query_event))
            {
                LOG_WARNING(log, replica << " was removed");
                break;
            }

            if (stat.version != replica_columns_version)
                continue;

            if (!replication_alter_columns_timeout)
            {
                alter_query_event->wait();
                /// Everything is fine.
            }
            else if (alter_query_event->tryWait(replication_alter_columns_timeout * 1000))
            {
                /// Everything is fine.
            }
            else
            {
                LOG_WARNING(log, "Timeout when waiting for replica " << replica << " to apply ALTER."
                    " ALTER will be done asynchronously.");

                timed_out_replicas.emplace(replica);
                break;
            }
        }

3212
        if (partial_shutdown_called)
3213 3214 3215 3216 3217 3218 3219 3220 3221 3222 3223 3224 3225 3226 3227 3228 3229 3230 3231 3232 3233 3234 3235 3236 3237 3238 3239 3240 3241 3242 3243 3244 3245 3246 3247 3248
            throw Exception("Alter is not finished because table shutdown was called. Alter will be done after table restart.",
                ErrorCodes::UNFINISHED);

        if (!inactive_replicas.empty() || !timed_out_replicas.empty())
        {
            std::stringstream exception_message;
            exception_message << "Alter is not finished because";

            if (!inactive_replicas.empty())
            {
                exception_message << " some replicas are inactive right now";

                for (auto it = inactive_replicas.begin(); it != inactive_replicas.end(); ++it)
                    exception_message << (it == inactive_replicas.begin() ? ": " : ", ") << *it;
            }

            if (!timed_out_replicas.empty() && !inactive_replicas.empty())
                exception_message << " and";

            if (!timed_out_replicas.empty())
            {
                exception_message << " timeout when waiting for some replicas";

                for (auto it = timed_out_replicas.begin(); it != timed_out_replicas.end(); ++it)
                    exception_message << (it == timed_out_replicas.begin() ? ": " : ", ") << *it;

                exception_message << " (replication_alter_columns_timeout = " << replication_alter_columns_timeout << ")";
            }

            exception_message << ". Alter will be done asynchronously.";

            throw Exception(exception_message.str(), ErrorCodes::UNFINISHED);
        }
    }

    LOG_DEBUG(log, "ALTER finished");
M
Merge  
Michael Kolupaev 已提交
3249 3250
}

M
Merge  
Michael Kolupaev 已提交
3251

3252 3253
/// If new version returns ordinary name, else returns part name containing the first and last month of the month
static String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const MergeTreePartInfo & part_info)
M
Merge  
Michael Kolupaev 已提交
3254
{
3255
    if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
3256 3257 3258
    {
        /// The date range is all month long.
        const auto & lut = DateLUT::instance();
3259
        time_t start_time = lut.YYYYMMDDToDate(parse<UInt32>(part_info.partition_id + "01"));
3260 3261
        DayNum left_date = lut.toDayNum(start_time);
        DayNum right_date = DayNum(static_cast<size_t>(left_date) + lut.daysInMonth(start_time) - 1);
3262 3263
        return part_info.getPartNameV0(left_date, right_date);
    }
M
Merge  
Michael Kolupaev 已提交
3264

3265 3266
    return part_info.getPartName();
}
A
Merge  
Alexey Milovidov 已提交
3267

3268
bool StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(const String & partition_id, MergeTreePartInfo & part_info)
A
Merge  
Andrey Mironov 已提交
3269
{
3270 3271
    /// Even if there is no data in the partition, you still need to mark the range for deletion.
    /// - Because before executing DETACH, tasks for downloading parts to this partition can be executed.
3272
    Int64 left = 0;
3273

3274
    /** Let's skip one number in `block_numbers` for the partition being deleted, and we will only delete parts until this number.
3275 3276 3277 3278 3279 3280
      * This prohibits merges of deleted parts with the new inserted data.
      * Invariant: merges of deleted parts with other parts do not appear in the log.
      * NOTE: If you need to similarly support a `DROP PART` request, you will have to think of some new mechanism for it,
      *     to guarantee this invariant.
      */
    Int64 right;
3281
    Int64 mutation_version;
3282 3283

    {
3284
        auto zookeeper = getZooKeeper();
3285
        auto block_number_lock = allocateBlockNumber(partition_id, zookeeper);
3286
        right = block_number_lock->getNumber();
3287
        block_number_lock->unlock();
3288
        mutation_version = queue.getCurrentMutationVersion(partition_id, right);
3289 3290
    }

3291
    /// Empty partition.
3292
    if (right == 0)
3293
        return false;
3294

3295
    --right;
3296

3297
    /// Artificial high level is choosen, to make this part "covering" all parts inside.
3298
    part_info = MergeTreePartInfo(partition_id, left, right, MergeTreePartInfo::MAX_LEVEL, mutation_version);
3299
    return true;
3300 3301 3302
}


3303
void StorageReplicatedMergeTree::clearColumnInPartition(
3304
    const ASTPtr & partition, const Field & column_name, const Context & context)
3305 3306 3307 3308 3309
{
    assertNotReadonly();

    /// We don't block merges, so anyone can manage this task (not only leader)

3310
    String partition_id = data.getPartitionIDFromQuery(partition, context);
3311
    MergeTreePartInfo drop_range_info;
3312

3313
    if (!getFakePartCoveringAllPartsInPartition(partition_id, drop_range_info))
3314
    {
3315
        LOG_INFO(log, "Will not clear partition " << partition_id << ", it is empty.");
3316 3317 3318
        return;
    }

3319 3320
    /// We allocated new block number for this part, so new merges can't merge clearing parts with new ones

3321
    LogEntry entry;
3322
    entry.type = LogEntry::CLEAR_COLUMN;
3323
    entry.new_part_name = getPartNamePossiblyFake(data.format_version, drop_range_info);
3324
    entry.column_name = column_name.safeGet<String>();
3325
    entry.create_time = time(nullptr);
3326 3327 3328 3329 3330

    String log_znode_path = getZooKeeper()->create(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential);
    entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);

    /// If necessary, wait until the operation is performed on itself or on all replicas.
3331
    if (context.getSettingsRef().replication_alter_partitions_sync != 0)
3332
    {
3333
        if (context.getSettingsRef().replication_alter_partitions_sync == 1)
3334 3335 3336 3337 3338 3339
            waitForReplicaToProcessLogEntry(replica_name, entry);
        else
            waitForAllReplicasToProcessLogEntry(entry);
    }
}

3340

3341
void StorageReplicatedMergeTree::dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & context)
3342 3343 3344
{
    assertNotReadonly();

3345 3346
    zkutil::ZooKeeperPtr zookeeper = getZooKeeper();

3347
    if (!is_leader)
3348
    {
3349
        sendRequestToLeaderReplica(query, context.getSettingsRef());
3350 3351 3352
        return;
    }

3353
    String partition_id = data.getPartitionIDFromQuery(partition, context);
3354

Z
zhang2014 已提交
3355
    LogEntry entry;
3356
    if (dropPartsInPartition(*zookeeper, partition_id, entry, detach))
3357
    {
Z
zhang2014 已提交
3358 3359 3360 3361 3362 3363 3364 3365
        /// If necessary, wait until the operation is performed on itself or on all replicas.
        if (context.getSettingsRef().replication_alter_partitions_sync != 0)
        {
            if (context.getSettingsRef().replication_alter_partitions_sync == 1)
                waitForReplicaToProcessLogEntry(replica_name, entry);
            else
                waitForAllReplicasToProcessLogEntry(entry);
        }
3366
    }
Z
zhang2014 已提交
3367
}
3368

3369

Z
zhang2014 已提交
3370 3371 3372
void StorageReplicatedMergeTree::truncate(const ASTPtr & query)
{
    assertNotReadonly();
3373

Z
zhang2014 已提交
3374
    zkutil::ZooKeeperPtr zookeeper = getZooKeeper();
3375

Z
zhang2014 已提交
3376
    if (!is_leader)
3377
    {
Z
zhang2014 已提交
3378 3379
        sendRequestToLeaderReplica(query, context.getSettingsRef());
        return;
3380
    }
3381

Z
zhang2014 已提交
3382
    Strings partitions = zookeeper->getChildren(zookeeper_path + "/block_numbers");
3383

Z
zhang2014 已提交
3384
    for (String & partition_id : partitions)
3385
    {
Z
zhang2014 已提交
3386 3387
        LogEntry entry;

3388
        if (dropPartsInPartition(*zookeeper, partition_id, entry, false))
3389 3390
            waitForAllReplicasToProcessLogEntry(entry);
    }
A
Merge  
Alexey Milovidov 已提交
3391
}
A
Merge  
Alexey Milovidov 已提交
3392

A
Merge  
Alexey Arno 已提交
3393

3394
void StorageReplicatedMergeTree::attachPartition(const ASTPtr & partition, bool attach_part, const Context & context)
M
Merge  
Michael Kolupaev 已提交
3395
{
3396 3397
    assertNotReadonly();

3398
    String partition_id;
3399 3400

    if (attach_part)
3401
        partition_id = typeid_cast<const ASTLiteral &>(*partition).value.safeGet<String>();
3402
    else
3403
        partition_id = data.getPartitionIDFromQuery(partition, context);
3404

3405
    String source_dir = "detached/";
3406 3407 3408 3409 3410

    /// Let's compose a list of parts that should be added.
    Strings parts;
    if (attach_part)
    {
3411
        parts.push_back(partition_id);
3412 3413 3414
    }
    else
    {
3415
        LOG_DEBUG(log, "Looking for parts for partition " << partition_id << " in " << source_dir);
3416
        ActiveDataPartSet active_parts(data.format_version);
3417 3418 3419 3420 3421

        std::set<String> part_names;
        for (Poco::DirectoryIterator it = Poco::DirectoryIterator(full_path + source_dir); it != Poco::DirectoryIterator(); ++it)
        {
            String name = it.name();
3422
            MergeTreePartInfo part_info;
3423
            if (!MergeTreePartInfo::tryParsePartName(name, &part_info, data.format_version))
3424
                continue;
3425
            if (part_info.partition_id != partition_id)
3426 3427 3428 3429 3430 3431 3432 3433 3434 3435 3436 3437 3438 3439 3440 3441 3442
                continue;
            LOG_DEBUG(log, "Found part " << name);
            active_parts.add(name);
            part_names.insert(name);
        }
        LOG_DEBUG(log, active_parts.size() << " of them are active");
        parts = active_parts.getParts();

        /// Inactive parts rename so they can not be attached in case of repeated ATTACH.
        for (const auto & name : part_names)
        {
            String containing_part = active_parts.getContainingPart(name);
            if (!containing_part.empty() && containing_part != name)
                Poco::File(full_path + source_dir + name).renameTo(full_path + source_dir + "inactive_" + name);
        }
    }

3443
    /// Synchronously check that added parts exist and are not broken. We will write checksums.txt if it does not exist.
3444
    LOG_DEBUG(log, "Checking parts");
3445
    std::vector<MergeTreeData::MutableDataPartPtr> loaded_parts;
3446 3447 3448
    for (const String & part : parts)
    {
        LOG_DEBUG(log, "Checking part " << part);
3449
        loaded_parts.push_back(data.loadPartAndFixMetadata(source_dir + part));
3450 3451
    }

3452
    ReplicatedMergeTreeBlockOutputStream output(*this, 0, 0, false);   /// TODO Allow to use quorum here.
3453
    for (auto & part : loaded_parts)
3454
    {
3455 3456 3457
        String old_name = part->name;
        output.writeExistingPart(part);
        LOG_DEBUG(log, "Attached part " << old_name << " as " << part->name);
3458
    }
M
Merge  
Michael Kolupaev 已提交
3459 3460
}

3461

3462
void StorageReplicatedMergeTree::checkTableCanBeDropped() const
3463
{
3464
    /// Consider only synchronized data
3465
    const_cast<MergeTreeData &>(getData()).recalculateColumnSizes();
3466
    context.checkTableCanBeDropped(database_name, table_name, getData().getTotalActiveSizeInBytes());
3467
}
A
Merge  
Alexey Milovidov 已提交
3468

3469

3470
void StorageReplicatedMergeTree::checkPartitionCanBeDropped(const ASTPtr & partition)
3471 3472
{
    const_cast<MergeTreeData &>(getData()).recalculateColumnSizes();
3473

3474 3475
    const String partition_id = data.getPartitionIDFromQuery(partition, context);
    auto parts_to_remove = data.getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id);
3476 3477

    UInt64 partition_size = 0;
3478

3479 3480
    for (const auto & part : parts_to_remove)
    {
3481
        partition_size += part->bytes_on_disk;
3482
    }
3483
    context.checkPartitionCanBeDropped(database_name, table_name, partition_size);
3484
}
A
Merge  
Alexey Milovidov 已提交
3485

3486

M
Merge  
Michael Kolupaev 已提交
3487 3488
void StorageReplicatedMergeTree::drop()
{
3489 3490
    {
        auto zookeeper = tryGetZooKeeper();
A
Merge  
Alexey Milovidov 已提交
3491

3492 3493
        if (is_readonly || !zookeeper)
            throw Exception("Can't drop readonly replicated table (need to drop data in ZooKeeper as well)", ErrorCodes::TABLE_IS_READ_ONLY);
M
Merge  
Michael Kolupaev 已提交
3494

3495
        shutdown();
M
Merge  
Michael Kolupaev 已提交
3496

3497 3498
        if (zookeeper->expired())
            throw Exception("Table was not dropped because ZooKeeper session has expired.", ErrorCodes::TABLE_WAS_NOT_DROPPED);
3499

3500 3501 3502
        LOG_INFO(log, "Removing replica " << replica_path);
        replica_is_active_node = nullptr;
        zookeeper->tryRemoveRecursive(replica_path);
M
Merge  
Michael Kolupaev 已提交
3503

3504 3505
        /// Check that `zookeeper_path` exists: it could have been deleted by another replica after execution of previous line.
        Strings replicas;
3506
        if (zookeeper->tryGetChildren(zookeeper_path + "/replicas", replicas) == Coordination::ZOK && replicas.empty())
3507 3508 3509 3510 3511
        {
            LOG_INFO(log, "Removing table " << zookeeper_path << " (this might take several minutes)");
            zookeeper->tryRemoveRecursive(zookeeper_path);
        }
    }
M
Merge  
Michael Kolupaev 已提交
3512

3513
    data.dropAllData();
M
Merge  
Michael Kolupaev 已提交
3514 3515
}

A
Merge  
Alexey Milovidov 已提交
3516

M
Merge  
Michael Kolupaev 已提交
3517 3518
void StorageReplicatedMergeTree::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name)
{
3519
    std::string new_full_path = new_path_to_db + escapeForFileName(new_table_name) + '/';
M
Merge  
Michael Kolupaev 已提交
3520

3521
    data.setPath(new_full_path);
M
Merge  
Michael Kolupaev 已提交
3522

3523 3524 3525
    database_name = new_database_name;
    table_name = new_table_name;
    full_path = new_full_path;
M
Merge  
Michael Kolupaev 已提交
3526

3527 3528 3529 3530
    /// Update table name in zookeeper
    auto zookeeper = getZooKeeper();
    zookeeper->set(replica_path + "/host", getReplicatedMergeTreeAddress().toString());

3531
    /// TODO: You can update names of loggers.
M
Merge  
Michael Kolupaev 已提交
3532 3533
}

A
Merge  
Alexey Milovidov 已提交
3534

3535 3536
bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path)
{
3537 3538 3539 3540 3541
    {
        std::lock_guard<std::mutex> lock(existing_nodes_cache_mutex);
        if (existing_nodes_cache.count(path))
            return true;
    }
3542

3543
    bool res = getZooKeeper()->exists(path);
3544

3545 3546 3547 3548 3549
    if (res)
    {
        std::lock_guard<std::mutex> lock(existing_nodes_cache_mutex);
        existing_nodes_cache.insert(path);
    }
3550

3551
    return res;
3552 3553 3554
}


3555
std::optional<EphemeralLockInZooKeeper>
3556 3557
StorageReplicatedMergeTree::allocateBlockNumber(
    const String & partition_id, zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_block_id_path)
M
Merge  
Michael Kolupaev 已提交
3558
{
3559
    /// Lets check for duplicates in advance, to avoid superflous block numbers allocation
3560
    Coordination::Requests deduplication_check_ops;
3561
    if (!zookeeper_block_id_path.empty())
3562
    {
3563 3564
        deduplication_check_ops.emplace_back(zkutil::makeCreateRequest(zookeeper_block_id_path, "", zkutil::CreateMode::Persistent));
        deduplication_check_ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_block_id_path, -1));
3565
    }
3566

3567 3568
    String block_numbers_path = zookeeper_path + "/block_numbers";
    String partition_path = block_numbers_path + "/" + partition_id;
3569

3570
    if (!existsNodeCached(partition_path))
3571
    {
3572
        Coordination::Requests ops;
3573 3574 3575 3576 3577 3578
        ops.push_back(zkutil::makeCreateRequest(partition_path, "", zkutil::CreateMode::Persistent));
        /// We increment data version of the block_numbers node so that it becomes possible
        /// to check in a ZK transaction that the set of partitions didn't change
        /// (unfortunately there is no CheckChildren op).
        ops.push_back(zkutil::makeSetRequest(block_numbers_path, "", -1));

3579
        Coordination::Responses responses;
3580
        int code = zookeeper->tryMulti(ops, responses);
3581
        if (code && code != Coordination::ZNODEEXISTS)
3582
            zkutil::KeeperMultiException::check(code, ops, responses);
3583
    }
3584

3585
    EphemeralLockInZooKeeper lock;
3586 3587 3588
    /// 2 RTT
    try
    {
3589
        lock = EphemeralLockInZooKeeper(
3590
            partition_path + "/block-", zookeeper_path + "/temp", *zookeeper, &deduplication_check_ops);
3591 3592 3593
    }
    catch (const zkutil::KeeperMultiException & e)
    {
3594
        if (e.code == Coordination::ZNODEEXISTS && e.getPathForFirstFailedOp() == zookeeper_block_id_path)
3595 3596 3597 3598
            return {};

        throw Exception("Cannot allocate block number in ZooKeeper: " + e.displayText(), ErrorCodes::KEEPER_EXCEPTION);
    }
3599
    catch (const Coordination::Exception & e)
3600 3601 3602 3603 3604
    {
        throw Exception("Cannot allocate block number in ZooKeeper: " + e.displayText(), ErrorCodes::KEEPER_EXCEPTION);
    }

    return {std::move(lock)};
M
Merge  
Michael Kolupaev 已提交
3605 3606
}

A
Merge  
Alexey Milovidov 已提交
3607

3608
void StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry(const ReplicatedMergeTreeLogEntryData & entry)
M
Merge  
Michael Kolupaev 已提交
3609
{
3610
    LOG_DEBUG(log, "Waiting for all replicas to process " << entry.znode_name);
M
Merge  
Michael Kolupaev 已提交
3611

3612 3613 3614
    Strings replicas = getZooKeeper()->getChildren(zookeeper_path + "/replicas");
    for (const String & replica : replicas)
        waitForReplicaToProcessLogEntry(replica, entry);
A
Merge  
Alexey Milovidov 已提交
3615

3616
    LOG_DEBUG(log, "Finished waiting for all replicas to process " << entry.znode_name);
A
Merge  
Alexey Milovidov 已提交
3617 3618 3619
}


3620
void StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry(const String & replica, const ReplicatedMergeTreeLogEntryData & entry)
A
Merge  
Alexey Milovidov 已提交
3621
{
3622 3623 3624 3625 3626 3627 3628 3629 3630 3631 3632 3633 3634 3635 3636 3637 3638 3639 3640 3641 3642 3643 3644 3645 3646 3647 3648 3649 3650 3651 3652 3653 3654 3655 3656 3657 3658 3659 3660 3661 3662 3663 3664 3665 3666 3667 3668 3669 3670 3671 3672 3673 3674 3675 3676 3677 3678 3679 3680 3681 3682 3683 3684 3685 3686 3687 3688 3689 3690 3691 3692 3693 3694 3695 3696 3697 3698 3699 3700 3701 3702 3703 3704 3705 3706 3707 3708 3709 3710 3711 3712 3713 3714 3715 3716 3717 3718 3719 3720 3721 3722 3723 3724 3725 3726 3727 3728 3729 3730 3731 3732 3733 3734 3735 3736 3737 3738 3739 3740 3741 3742 3743 3744 3745 3746
    String entry_str = entry.toString();
    String log_node_name;

    /** Two types of entries can be passed to this function
      * 1. (more often) From `log` directory - a common log, from where replicas copy entries to their queue.
      * 2. From the `queue` directory of one of the replicas.
      *
      * The problem is that the numbers (`sequential` node) of the queue elements in `log` and in `queue` do not match.
      * (And the numbers of the same log element for different replicas do not match in the `queue`.)
      *
      * Therefore, you should consider these cases separately.
      */

    /** First, you need to wait until replica takes `queue` element from the `log` to its queue,
      *  if it has not been done already (see the `pullLogsToQueue` function).
      *
      * To do this, check its node `log_pointer` - the maximum number of the element taken from `log` + 1.
      */

    if (startsWith(entry.znode_name, "log-"))
    {
        /** In this case, just take the number from the node name `log-xxxxxxxxxx`.
          */

        UInt64 log_index = parse<UInt64>(entry.znode_name.substr(entry.znode_name.size() - 10));
        log_node_name = entry.znode_name;

        LOG_DEBUG(log, "Waiting for " << replica << " to pull " << log_node_name << " to queue");

        /// Let's wait until entry gets into the replica queue.
        while (true)
        {
            zkutil::EventPtr event = std::make_shared<Poco::Event>();

            String log_pointer = getZooKeeper()->get(zookeeper_path + "/replicas/" + replica + "/log_pointer", nullptr, event);
            if (!log_pointer.empty() && parse<UInt64>(log_pointer) > log_index)
                break;

            event->wait();
        }
    }
    else if (startsWith(entry.znode_name, "queue-"))
    {
        /** In this case, the number of `log` node is unknown. You need look through everything from `log_pointer` to the end,
          *  looking for a node with the same content. And if we do not find it - then the replica has already taken this entry in its queue.
          */

        String log_pointer = getZooKeeper()->get(zookeeper_path + "/replicas/" + replica + "/log_pointer");

        Strings log_entries = getZooKeeper()->getChildren(zookeeper_path + "/log");
        UInt64 log_index = 0;
        bool found = false;

        for (const String & log_entry_name : log_entries)
        {
            log_index = parse<UInt64>(log_entry_name.substr(log_entry_name.size() - 10));

            if (!log_pointer.empty() && log_index < parse<UInt64>(log_pointer))
                continue;

            String log_entry_str;
            bool exists = getZooKeeper()->tryGet(zookeeper_path + "/log/" + log_entry_name, log_entry_str);
            if (exists && entry_str == log_entry_str)
            {
                found = true;
                log_node_name = log_entry_name;
                break;
            }
        }

        if (found)
        {
            LOG_DEBUG(log, "Waiting for " << replica << " to pull " << log_node_name << " to queue");

            /// Let's wait until the entry gets into the replica queue.
            while (true)
            {
                zkutil::EventPtr event = std::make_shared<Poco::Event>();

                String log_pointer = getZooKeeper()->get(zookeeper_path + "/replicas/" + replica + "/log_pointer", nullptr, event);
                if (!log_pointer.empty() && parse<UInt64>(log_pointer) > log_index)
                    break;

                event->wait();
            }
        }
    }
    else
        throw Exception("Logical error: unexpected name of log node: " + entry.znode_name, ErrorCodes::LOGICAL_ERROR);

    if (!log_node_name.empty())
        LOG_DEBUG(log, "Looking for node corresponding to " << log_node_name << " in " << replica << " queue");
    else
        LOG_DEBUG(log, "Looking for corresponding node in " << replica << " queue");

    /** Second - find the corresponding entry in the queue of the specified replica.
      * Its number may match neither the `log` node nor the `queue` node of the current replica (for us).
      * Therefore, we search by comparing the content.
      */

    Strings queue_entries = getZooKeeper()->getChildren(zookeeper_path + "/replicas/" + replica + "/queue");
    String queue_entry_to_wait_for;

    for (const String & entry_name : queue_entries)
    {
        String queue_entry_str;
        bool exists = getZooKeeper()->tryGet(zookeeper_path + "/replicas/" + replica + "/queue/" + entry_name, queue_entry_str);
        if (exists && queue_entry_str == entry_str)
        {
            queue_entry_to_wait_for = entry_name;
            break;
        }
    }

    /// While looking for the record, it has already been executed and deleted.
    if (queue_entry_to_wait_for.empty())
    {
        LOG_DEBUG(log, "No corresponding node found. Assuming it has been already processed." " Found " << queue_entries.size() << " nodes.");
        return;
    }

    LOG_DEBUG(log, "Waiting for " << queue_entry_to_wait_for << " to disappear from " << replica << " queue");

    /// Third - wait until the entry disappears from the replica queue.
    getZooKeeper()->waitForDisappear(zookeeper_path + "/replicas/" + replica + "/queue/" + queue_entry_to_wait_for);
M
Merge  
Michael Kolupaev 已提交
3747 3748 3749
}


3750
void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields)
3751
{
3752 3753
    auto zookeeper = tryGetZooKeeper();

3754
    res.is_leader = is_leader;
3755 3756 3757 3758
    res.is_readonly = is_readonly;
    res.is_session_expired = !zookeeper || zookeeper->expired();

    res.queue = queue.getStatus();
3759 3760
    res.absolute_delay = getAbsoluteDelay(); /// NOTE: may be slightly inconsistent with queue status.

3761 3762 3763 3764 3765 3766 3767 3768 3769 3770 3771 3772 3773 3774 3775 3776 3777 3778 3779 3780 3781 3782 3783 3784 3785 3786 3787 3788 3789 3790 3791 3792 3793 3794 3795 3796 3797 3798 3799
    res.parts_to_check = part_check_thread.size();

    res.zookeeper_path = zookeeper_path;
    res.replica_name = replica_name;
    res.replica_path = replica_path;
    res.columns_version = columns_version;

    if (res.is_session_expired || !with_zk_fields)
    {
        res.log_max_index = 0;
        res.log_pointer = 0;
        res.total_replicas = 0;
        res.active_replicas = 0;
    }
    else
    {
        auto log_entries = zookeeper->getChildren(zookeeper_path + "/log");

        if (log_entries.empty())
        {
            res.log_max_index = 0;
        }
        else
        {
            const String & last_log_entry = *std::max_element(log_entries.begin(), log_entries.end());
            res.log_max_index = parse<UInt64>(last_log_entry.substr(strlen("log-")));
        }

        String log_pointer_str = zookeeper->get(replica_path + "/log_pointer");
        res.log_pointer = log_pointer_str.empty() ? 0 : parse<UInt64>(log_pointer_str);

        auto all_replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
        res.total_replicas = all_replicas.size();

        res.active_replicas = 0;
        for (const String & replica : all_replicas)
            if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))
                ++res.active_replicas;
    }
3800 3801
}

3802

3803 3804 3805 3806 3807 3808 3809 3810 3811 3812 3813 3814 3815 3816 3817
/// TODO: Probably it is better to have queue in ZK with tasks for leader (like DDL)
void StorageReplicatedMergeTree::sendRequestToLeaderReplica(const ASTPtr & query, const Settings & settings)
{
    auto live_replicas = getZooKeeper()->getChildren(zookeeper_path + "/leader_election");
    if (live_replicas.empty())
        throw Exception("No active replicas", ErrorCodes::NO_ACTIVE_REPLICAS);

    std::sort(live_replicas.begin(), live_replicas.end());
    const auto leader = getZooKeeper()->get(zookeeper_path + "/leader_election/" + live_replicas.front());

    if (leader == replica_name)
        throw Exception("Leader was suddenly changed or logical error.", ErrorCodes::LEADERSHIP_CHANGED);

    ReplicatedMergeTreeAddress leader_address(getZooKeeper()->get(zookeeper_path + "/replicas/" + leader + "/host"));

3818
    /// TODO: add setters and getters interface for database and table fields of AST
3819
    auto new_query = query->clone();
3820 3821 3822 3823 3824 3825 3826 3827 3828 3829
    if (auto * alter = typeid_cast<ASTAlterQuery *>(new_query.get()))
    {
        alter->database = leader_address.database;
        alter->table = leader_address.table;
    }
    else if (auto * optimize = typeid_cast<ASTOptimizeQuery *>(new_query.get()))
    {
        optimize->database = leader_address.database;
        optimize->table = leader_address.table;
    }
K
Kirill Shvakov 已提交
3830
    else if (auto * drop = typeid_cast<ASTDropQuery *>(new_query.get()); drop->kind == ASTDropQuery::Kind::Truncate)
3831 3832 3833 3834
    {
        drop->database = leader_address.database;
        drop->table    = leader_address.table;
    }
3835 3836
    else
        throw Exception("Can't proxy this query. Unsupported query type", ErrorCodes::NOT_IMPLEMENTED);
3837

P
proller 已提交
3838
    /// Query send with current user credentials
3839

3840
    auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithoutFailover(context.getSettingsRef());
3841 3842 3843 3844
    Connection connection(
        leader_address.host,
        leader_address.queries_port,
        leader_address.database,
3845
        context.getClientInfo().current_user, context.getClientInfo().current_password, timeouts, "ClickHouse replica");
3846

3847
    RemoteBlockInputStream stream(connection, formattedAST(new_query), {}, context, &settings);
3848
    NullBlockOutputStream output({});
3849 3850 3851 3852 3853 3854

    copyData(stream, output);
    return;
}


3855 3856
void StorageReplicatedMergeTree::getQueue(LogEntriesData & res, String & replica_name_)
{
3857 3858
    replica_name_ = replica_name;
    queue.getEntries(res);
3859 3860
}

3861 3862 3863 3864 3865 3866
time_t StorageReplicatedMergeTree::getAbsoluteDelay() const
{
    time_t min_unprocessed_insert_time = 0;
    time_t max_processed_insert_time = 0;
    queue.getInsertTimes(min_unprocessed_insert_time, max_processed_insert_time);

3867 3868 3869 3870
    /// Load start time, then finish time to avoid reporting false delay when start time is updated
    /// between loading of two variables.
    time_t queue_update_start_time = last_queue_update_start_time.load();
    time_t queue_update_finish_time = last_queue_update_finish_time.load();
3871 3872 3873

    time_t current_time = time(nullptr);

3874
    if (!queue_update_finish_time)
3875
    {
3876
        /// We have not updated queue even once yet (perhaps replica is readonly).
3877 3878 3879 3880 3881 3882 3883 3884
        /// As we have no info about the current state of replication log, return effectively infinite delay.
        return current_time;
    }
    else if (min_unprocessed_insert_time)
    {
        /// There are some unprocessed insert entries in queue.
        return (current_time > min_unprocessed_insert_time) ? (current_time - min_unprocessed_insert_time) : 0;
    }
3885
    else if (queue_update_start_time > queue_update_finish_time)
3886 3887 3888 3889
    {
        /// Queue is empty, but there are some in-flight or failed queue update attempts
        /// (likely because of problems with connecting to ZooKeeper).
        /// Return the time passed since last attempt.
3890
        return (current_time > queue_update_start_time) ? (current_time - queue_update_start_time) : 0;
3891 3892 3893 3894 3895 3896 3897
    }
    else
    {
        /// Everything is up-to-date.
        return 0;
    }
}
3898

3899
void StorageReplicatedMergeTree::getReplicaDelays(time_t & out_absolute_delay, time_t & out_relative_delay)
3900
{
3901
    assertNotReadonly();
3902

3903
    time_t current_time = time(nullptr);
3904

3905
    out_absolute_delay = getAbsoluteDelay();
3906
    out_relative_delay = 0;
3907

3908 3909 3910 3911
    /** Relative delay is the maximum difference of absolute delay from any other replica,
      *  (if this replica lags behind any other live replica, or zero, otherwise).
      * Calculated only if the absolute delay is large enough.
      */
3912

3913 3914
    if (out_absolute_delay < static_cast<time_t>(data.settings.min_relative_delay_to_yield_leadership))
        return;
3915

3916
    auto zookeeper = getZooKeeper();
3917

3918 3919
    time_t max_replicas_unprocessed_insert_time = 0;
    bool have_replica_with_nothing_unprocessed = false;
3920

3921
    Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
3922

3923 3924 3925 3926
    for (const auto & replica : replicas)
    {
        if (replica == replica_name)
            continue;
3927

3928 3929 3930
        /// Skip dead replicas.
        if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))
            continue;
3931

3932 3933 3934
        String value;
        if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/min_unprocessed_insert_time", value))
            continue;
3935

3936
        time_t replica_time = value.empty() ? 0 : parse<time_t>(value);
3937

3938 3939 3940 3941 3942 3943
        if (replica_time == 0)
        {
            /** Note
              * The conclusion that the replica does not lag may be incorrect,
              *  because the information about `min_unprocessed_insert_time` is taken
              *  only from that part of the log that has been moved to the queue.
3944
              * If the replica for some reason has stalled `queueUpdatingTask`,
3945 3946
              *  then `min_unprocessed_insert_time` will be incorrect.
              */
3947

3948 3949 3950
            have_replica_with_nothing_unprocessed = true;
            break;
        }
3951

3952 3953 3954
        if (replica_time > max_replicas_unprocessed_insert_time)
            max_replicas_unprocessed_insert_time = replica_time;
    }
3955

3956 3957
    if (have_replica_with_nothing_unprocessed)
        out_relative_delay = out_absolute_delay;
3958 3959 3960 3961 3962 3963 3964
    else
    {
        max_replicas_unprocessed_insert_time = std::min(current_time, max_replicas_unprocessed_insert_time);
        time_t min_replicas_delay = current_time - max_replicas_unprocessed_insert_time;
        if (out_absolute_delay > min_replicas_delay)
            out_relative_delay = out_absolute_delay - min_replicas_delay;
    }
3965 3966 3967
}


3968
void StorageReplicatedMergeTree::fetchPartition(const ASTPtr & partition, const String & from_, const Context & context)
3969
{
3970
    String partition_id = data.getPartitionIDFromQuery(partition, context);
3971 3972 3973 3974 3975

    String from = from_;
    if (from.back() == '/')
        from.resize(from.size() - 1);

3976
    LOG_INFO(log, "Will fetch partition " << partition_id << " from shard " << from_);
3977 3978 3979 3980 3981 3982

    /** Let's check that there is no such partition in the `detached` directory (where we will write the downloaded parts).
      * Unreliable (there is a race condition) - such a partition may appear a little later.
      */
    Poco::DirectoryIterator dir_end;
    for (Poco::DirectoryIterator dir_it{data.getFullPath() + "detached/"}; dir_it != dir_end; ++dir_it)
3983 3984
    {
        MergeTreePartInfo part_info;
3985 3986
        if (MergeTreePartInfo::tryParsePartName(dir_it.name(), &part_info, data.format_version)
              && part_info.partition_id == partition_id)
3987 3988
            throw Exception("Detached partition " + partition_id + " already exists.", ErrorCodes::PARTITION_ALREADY_EXISTS);
    }
3989 3990 3991 3992 3993 3994 3995 3996 3997 3998 3999 4000 4001 4002 4003 4004 4005 4006 4007 4008 4009 4010 4011 4012 4013 4014 4015 4016 4017 4018 4019 4020 4021 4022 4023 4024 4025

    zkutil::Strings replicas;
    zkutil::Strings active_replicas;
    String best_replica;

    {
        auto zookeeper = getZooKeeper();

        /// List of replicas of source shard.
        replicas = zookeeper->getChildren(from + "/replicas");

        /// Leave only active replicas.
        active_replicas.reserve(replicas.size());

        for (const String & replica : replicas)
            if (zookeeper->exists(from + "/replicas/" + replica + "/is_active"))
                active_replicas.push_back(replica);

        if (active_replicas.empty())
            throw Exception("No active replicas for shard " + from, ErrorCodes::NO_ACTIVE_REPLICAS);

        /** You must select the best (most relevant) replica.
        * This is a replica with the maximum `log_pointer`, then with the minimum `queue` size.
        * NOTE This is not exactly the best criteria. It does not make sense to download old partitions,
        *  and it would be nice to be able to choose the replica closest by network.
        * NOTE Of course, there are data races here. You can solve it by retrying.
        */
        Int64 max_log_pointer = -1;
        UInt64 min_queue_size = std::numeric_limits<UInt64>::max();

        for (const String & replica : active_replicas)
        {
            String current_replica_path = from + "/replicas/" + replica;

            String log_pointer_str = zookeeper->get(current_replica_path + "/log_pointer");
            Int64 log_pointer = log_pointer_str.empty() ? 0 : parse<UInt64>(log_pointer_str);

4026
            Coordination::Stat stat;
4027 4028 4029 4030 4031 4032 4033 4034 4035 4036 4037 4038 4039 4040 4041 4042 4043 4044 4045 4046 4047 4048 4049 4050 4051 4052 4053 4054 4055 4056 4057 4058 4059 4060 4061
            zookeeper->get(current_replica_path + "/queue", &stat);
            size_t queue_size = stat.numChildren;

            if (log_pointer > max_log_pointer
                || (log_pointer == max_log_pointer && queue_size < min_queue_size))
            {
                max_log_pointer = log_pointer;
                min_queue_size = queue_size;
                best_replica = replica;
            }
        }
    }

    if (best_replica.empty())
        throw Exception("Logical error: cannot choose best replica.", ErrorCodes::LOGICAL_ERROR);

    LOG_INFO(log, "Found " << replicas.size() << " replicas, " << active_replicas.size() << " of them are active."
        << " Selected " << best_replica << " to fetch from.");

    String best_replica_path = from + "/replicas/" + best_replica;

    /// Let's find out which parts are on the best replica.

    /** Trying to download these parts.
      * Some of them could be deleted due to the merge.
      * In this case, update the information about the available parts and try again.
      */

    unsigned try_no = 0;
    Strings missing_parts;
    do
    {
        if (try_no)
            LOG_INFO(log, "Some of parts (" << missing_parts.size() << ") are missing. Will try to fetch covering parts.");

A
alesapin 已提交
4062
        if (try_no >= context.getSettings().max_fetch_partition_retries_count)
A
Alexey Milovidov 已提交
4063
            throw Exception("Too many retries to fetch parts from " + best_replica_path, ErrorCodes::TOO_MANY_RETRIES_TO_FETCH_PARTS);
4064 4065

        Strings parts = getZooKeeper()->getChildren(best_replica_path + "/parts");
4066
        ActiveDataPartSet active_parts_set(data.format_version, parts);
4067 4068 4069 4070 4071 4072 4073 4074 4075
        Strings parts_to_fetch;

        if (missing_parts.empty())
        {
            parts_to_fetch = active_parts_set.getParts();

            /// Leaving only the parts of the desired partition.
            Strings parts_to_fetch_partition;
            for (const String & part : parts_to_fetch)
4076
            {
4077
                if (MergeTreePartInfo::fromPartName(part, data.format_version).partition_id == partition_id)
4078
                    parts_to_fetch_partition.push_back(part);
4079
            }
4080 4081 4082 4083

            parts_to_fetch = std::move(parts_to_fetch_partition);

            if (parts_to_fetch.empty())
4084
                throw Exception("Partition " + partition_id + " on " + best_replica_path + " doesn't exist", ErrorCodes::PARTITION_DOESNT_EXIST);
4085 4086 4087 4088 4089 4090 4091 4092 4093 4094 4095 4096 4097 4098 4099 4100 4101 4102 4103 4104 4105 4106 4107 4108
        }
        else
        {
            for (const String & missing_part : missing_parts)
            {
                String containing_part = active_parts_set.getContainingPart(missing_part);
                if (!containing_part.empty())
                    parts_to_fetch.push_back(containing_part);
                else
                    LOG_WARNING(log, "Part " << missing_part << " on replica " << best_replica_path << " has been vanished.");
            }
        }

        LOG_INFO(log, "Parts to fetch: " << parts_to_fetch.size());

        missing_parts.clear();
        for (const String & part : parts_to_fetch)
        {
            try
            {
                fetchPart(part, best_replica_path, true, 0);
            }
            catch (const DB::Exception & e)
            {
A
alesapin 已提交
4109 4110
                if (e.code() != ErrorCodes::RECEIVED_ERROR_FROM_REMOTE_IO_SERVER && e.code() != ErrorCodes::RECEIVED_ERROR_TOO_MANY_REQUESTS
                    && e.code() != ErrorCodes::CANNOT_READ_ALL_DATA)
4111 4112 4113 4114 4115 4116 4117 4118 4119
                    throw;

                LOG_INFO(log, e.displayText());
                missing_parts.push_back(part);
            }
        }

        ++try_no;
    } while (!missing_parts.empty());
4120 4121 4122
}


4123
void StorageReplicatedMergeTree::freezePartition(const ASTPtr & partition, const String & with_name, const Context & context)
4124
{
4125
    data.freezePartition(partition, with_name, context);
4126 4127
}

4128

4129 4130
void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, const Context &)
{
4131 4132 4133 4134 4135 4136 4137 4138 4139 4140 4141 4142 4143 4144 4145 4146 4147 4148 4149 4150 4151 4152 4153 4154 4155 4156 4157 4158 4159 4160 4161 4162 4163 4164 4165 4166 4167 4168 4169 4170 4171 4172 4173 4174 4175 4176 4177 4178 4179 4180 4181 4182 4183
    /// Overview of the mutation algorithm.
    ///
    /// When the client executes a mutation, this method is called. It acquires block numbers in all
    /// partitions, saves them in the mutation entry and writes the mutation entry to a new ZK node in
    /// the /mutations folder. This block numbers are needed to determine which parts should be mutated and
    /// which shouldn't (parts inserted after the mutation will have the block number higher than the
    /// block number acquired by the mutation in that partition and so will not be mutatied).
    /// This block number is called "mutation version" in that partition.
    ///
    /// Mutation versions are acquired atomically in all partitions, so the case when an insert in some
    /// partition has the block number higher than the mutation version but the following insert into another
    /// partition acquires the block number lower than the mutation version in that partition is impossible.
    /// Another important invariant: mutation entries appear in /mutations in the order of their mutation
    /// versions (in any partition). This means that mutations form a sequence and we can execute them in
    /// the order of their mutation versions and not worry that some mutation with the smaller version
    /// will suddenly appear.
    ///
    /// During mutations individual parts are immutable - when we want to change the contents of a part
    /// we prepare the new part and add it to MergeTreeData (the original part gets replaced). The fact that
    /// we have mutated the part is recorded in the part->info.mutation field of MergeTreePartInfo.
    /// The relation with the original part is preserved because the new part covers the same block range
    /// as the original one.
    ///
    /// We then can for each part determine its "mutation version": the version of the last mutation in
    /// the mutation sequence that we regard as already applied to that part. All mutations with the greater
    /// version number will still need to be applied to that part.
    ///
    /// Execution of mutations is done asynchronously. All replicas watch the /mutations directory and
    /// load new mutation entries as they appear (see mutationsUpdatingTask()). Next we need to determine
    /// how to mutate individual parts consistently with part merges. This is done by the leader replica
    /// (see mergeSelectingTask() and class ReplicatedMergeTreeMergePredicate for details). Important
    /// invariants here are that a) all source parts for a single merge must have the same mutation version
    /// and b) any part can be mutated only once or merged only once (e.g. once we have decided to mutate
    /// a part then we need to execute that mutation and can assign merges only to the new part and not to the
    /// original part). Multiple consecutive mutations can be executed at once (without writing the
    /// intermediate result to a part).
    ///
    /// Leader replica records its decisions to the replication log (/log directory in ZK) in the form of
    /// MUTATE_PART entries and all replicas then execute them in the background pool
    /// (see tryExecutePartMutation() function). When a replica encounters a MUTATE_PART command, it is
    /// guaranteed that the corresponding mutation entry is already loaded (when we pull entries from
    /// replication log into the replica queue, we also load mutation entries). Note that just as with merges
    /// the replica can decide not to do the mutation locally and fetch the mutated part from another replica
    /// instead.
    ///
    /// Mutations of individual parts are in fact pretty similar to merges, e.g. their assignment and execution
    /// is governed by the same settings. TODO: support a single "merge-mutation" operation when the data
    /// read from the the source parts is first mutated on the fly to some uniform mutation version and then
    /// merged to a resulting part.
    ///
    /// After all needed parts are mutated (i.e. all active parts have the mutation version greater than
    /// the version of this mutation), the mutation is considered done and can be deleted.

4184 4185 4186 4187 4188 4189 4190 4191 4192 4193 4194 4195 4196
    ReplicatedMergeTreeMutationEntry entry;
    entry.source_replica = replica_name;
    entry.commands = commands;

    String mutations_path = zookeeper_path + "/mutations";

    /// Update the mutations_path node when creating the mutation and check its version to ensure that
    /// nodes for mutations are created in the same order as the corresponding block numbers.
    /// Should work well if the number of concurrent mutation requests is small.
    while (true)
    {
        auto zookeeper = getZooKeeper();

4197
        Coordination::Stat mutations_stat;
4198 4199 4200 4201 4202 4203 4204 4205 4206 4207
        zookeeper->get(mutations_path, &mutations_stat);

        EphemeralLocksInAllPartitions block_number_locks(
            zookeeper_path + "/block_numbers", "block-", zookeeper_path + "/temp", *zookeeper);

        for (const auto & lock : block_number_locks.getLocks())
            entry.block_numbers[lock.partition_id] = lock.number;

        entry.create_time = time(nullptr);

4208
        Coordination::Requests requests;
4209 4210 4211 4212
        requests.emplace_back(zkutil::makeSetRequest(mutations_path, String(), mutations_stat.version));
        requests.emplace_back(zkutil::makeCreateRequest(
            mutations_path + "/", entry.toString(), zkutil::CreateMode::PersistentSequential));

4213
        Coordination::Responses responses;
4214 4215
        int32_t rc = zookeeper->tryMulti(requests, responses);

4216
        if (rc == Coordination::ZOK)
4217 4218
        {
            const String & path_created =
4219
                dynamic_cast<const Coordination::CreateResponse *>(responses[1].get())->path_created;
4220
            entry.znode_name = path_created.substr(path_created.find_last_of('/') + 1);
4221
            LOG_TRACE(log, "Created mutation with ID " << entry.znode_name);
4222
            break;
4223
        }
4224
        else if (rc == Coordination::ZBADVERSION)
4225 4226 4227 4228 4229
        {
            LOG_TRACE(log, "Version conflict when trying to create a mutation node, retrying...");
            continue;
        }
        else
4230
            throw Coordination::Exception("Unable to create a mutation znode", rc);
4231 4232 4233
    }
}

4234 4235 4236 4237 4238
std::vector<MergeTreeMutationStatus> StorageReplicatedMergeTree::getMutationsStatus() const
{
    return queue.getMutationsStatus();
}

4239

4240
void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK()
4241 4242 4243
{
    /// Critical section is not required (since grabOldParts() returns unique part set on each call)

4244
    auto table_lock = lockStructure(false, __PRETTY_FUNCTION__);
4245 4246 4247
    auto zookeeper = getZooKeeper();

    MergeTreeData::DataPartsVector parts = data.grabOldParts();
4248
    if (parts.empty())
4249 4250
        return;

4251 4252 4253 4254 4255 4256
    MergeTreeData::DataPartsVector parts_to_delete_only_from_filesystem;    // Only duplicates
    MergeTreeData::DataPartsVector parts_to_delete_completely;              // All parts except duplicates
    MergeTreeData::DataPartsVector parts_to_retry_deletion;                 // Parts that should be retried due to network problems
    MergeTreeData::DataPartsVector parts_to_remove_from_filesystem;         // Parts removed from ZK

    for (const auto & part : parts)
4257
    {
4258 4259 4260 4261 4262 4263
        if (!part->is_duplicate)
            parts_to_delete_completely.emplace_back(part);
        else
            parts_to_delete_only_from_filesystem.emplace_back(part);
    }
    parts.clear();
4264

4265 4266 4267
    auto remove_parts_from_filesystem = [log=log] (const MergeTreeData::DataPartsVector & parts_to_remove)
    {
        for (auto & part : parts_to_remove)
4268
        {
4269 4270 4271 4272 4273 4274 4275 4276
            try
            {
                part->remove();
            }
            catch (...)
            {
                tryLogCurrentException(log, "There is a problem with deleting part " + part->name + " from filesystem");
            }
4277 4278 4279
        }
    };

4280 4281 4282 4283 4284 4285 4286 4287 4288 4289
    /// Delete duplicate parts from filesystem
    if (!parts_to_delete_only_from_filesystem.empty())
    {
        remove_parts_from_filesystem(parts_to_delete_only_from_filesystem);
        data.removePartsFinally(parts_to_delete_only_from_filesystem);

        LOG_DEBUG(log, "Removed " << parts_to_delete_only_from_filesystem.size() << " old duplicate parts");
    }

    /// Delete normal parts from ZooKeeper
4290
    NameSet part_names_to_retry_deletion;
4291 4292
    try
    {
4293 4294 4295
        Strings part_names_to_delete_completely;
        for (const auto & part : parts_to_delete_completely)
            part_names_to_delete_completely.emplace_back(part->name);
4296

4297 4298
        LOG_DEBUG(log, "Removing " << parts_to_delete_completely.size() << " old parts from ZooKeeper");
        removePartsFromZooKeeper(zookeeper, part_names_to_delete_completely, &part_names_to_retry_deletion);
4299 4300 4301
    }
    catch (...)
    {
4302
        LOG_ERROR(log, "There is a problem with deleting parts from ZooKeeper: " << getCurrentExceptionMessage(true));
4303
    }
4304

4305
    /// Part names that were reliably deleted from ZooKeeper should be deleted from filesystem
4306
    auto num_reliably_deleted_parts = parts_to_delete_completely.size() - part_names_to_retry_deletion.size();
4307
    LOG_DEBUG(log, "Removed " << num_reliably_deleted_parts << " old parts from ZooKeeper. Removing them from filesystem.");
4308

4309 4310
    /// Delete normal parts on two sets
    for (auto & part : parts_to_delete_completely)
4311 4312 4313 4314 4315
    {
        if (part_names_to_retry_deletion.count(part->name) == 0)
            parts_to_remove_from_filesystem.emplace_back(part);
        else
            parts_to_retry_deletion.emplace_back(part);
4316 4317
    }

4318 4319
    /// Will retry deletion
    if (!parts_to_retry_deletion.empty())
4320
    {
4321
        data.rollbackDeletingParts(parts_to_retry_deletion);
4322 4323
        LOG_DEBUG(log, "Will retry deletion of " << parts_to_retry_deletion.size() << " parts in the next time");
    }
4324

4325
    /// Remove parts from filesystem and finally from data_parts
4326
    if (!parts_to_remove_from_filesystem.empty())
4327
    {
4328 4329
        remove_parts_from_filesystem(parts_to_remove_from_filesystem);
        data.removePartsFinally(parts_to_remove_from_filesystem);
4330

4331 4332
        LOG_DEBUG(log, "Removed " << parts_to_remove_from_filesystem.size() << " old parts");
    }
4333 4334 4335
}


4336 4337 4338 4339 4340 4341 4342 4343 4344 4345 4346
bool StorageReplicatedMergeTree::tryRemovePartsFromZooKeeperWithRetries(MergeTreeData::DataPartsVector & parts, size_t max_retries)
{
    Strings part_names_to_remove;
    for (const auto & part : parts)
        part_names_to_remove.emplace_back(part->name);

    return tryRemovePartsFromZooKeeperWithRetries(part_names_to_remove, max_retries);
}

bool StorageReplicatedMergeTree::tryRemovePartsFromZooKeeperWithRetries(const Strings & part_names, size_t max_retries)
{
4347
    using MultiFuture = std::future<Coordination::MultiResponse>;
4348 4349 4350 4351 4352 4353 4354 4355 4356 4357 4358 4359 4360 4361 4362 4363 4364 4365

    size_t num_tries = 0;
    bool sucess = false;

    while (!sucess && (max_retries == 0 || num_tries < max_retries))
    {
        std::vector<MultiFuture> futures;
        futures.reserve(part_names.size());

        ++num_tries;
        sucess = true;

        try
        {
            auto zookeeper = getZooKeeper();

            for (const String & part_name : part_names)
            {
4366
                Coordination::Requests ops;
4367 4368 4369 4370 4371 4372 4373 4374 4375
                removePartFromZooKeeper(part_name, ops);

                futures.emplace_back(zookeeper->tryAsyncMulti(ops));
            }

            for (auto & future : futures)
            {
                auto response = future.get();

4376
                if (response.error == 0 || response.error == Coordination::ZNONODE)
4377 4378
                    continue;

4379
                if (Coordination::isHardwareError(response.error))
4380 4381 4382 4383 4384
                {
                    sucess = false;
                    continue;
                }

4385
                throw Coordination::Exception(response.error);
4386 4387
            }
        }
4388
        catch (Coordination::Exception & e)
4389 4390 4391
        {
            sucess = false;

4392
            if (Coordination::isHardwareError(e.code))
4393 4394 4395 4396 4397 4398 4399 4400 4401 4402 4403 4404 4405
                tryLogCurrentException(log, __PRETTY_FUNCTION__);
            else
                throw;
        }

        if (!sucess && num_tries < max_retries)
            std::this_thread::sleep_for(std::chrono::milliseconds(1000));
    }

    return sucess;
}

/// TODO: rewrite this code using async Multi ops after final ZooKeeper library update
4406
void StorageReplicatedMergeTree::removePartsFromZooKeeper(zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names,
A
Alexey Milovidov 已提交
4407
                                                          NameSet * parts_should_be_retried)
4408
{
4409
    Coordination::Requests ops;
4410
    auto it_first_node_in_batch = part_names.cbegin();
4411 4412 4413 4414 4415

    for (auto it = part_names.cbegin(); it != part_names.cend(); ++it)
    {
        removePartFromZooKeeper(*it, ops);

4416 4417
        auto it_next = std::next(it);
        if (ops.size() >= zkutil::MULTI_BATCH_SIZE || it_next == part_names.cend())
4418
        {
4419
            Coordination::Responses unused_responses;
4420
            auto code = zookeeper->tryMultiNoThrow(ops, unused_responses);
4421
            ops.clear();
4422

4423
            if (code == Coordination::ZNONODE)
4424 4425
            {
                /// Fallback
4426
                LOG_DEBUG(log, "ZooKeeper nodes for some parts in the batch are missing, will remove part nodes one by one");
4427 4428 4429

                for (auto it_in_batch = it_first_node_in_batch; it_in_batch != it_next; ++it_in_batch)
                {
4430
                    Coordination::Requests cur_ops;
4431
                    removePartFromZooKeeper(*it_in_batch, cur_ops);
4432
                    auto cur_code = zookeeper->tryMultiNoThrow(cur_ops, unused_responses);
4433

4434
                    if (cur_code == Coordination::ZNONODE)
4435
                    {
4436
                        LOG_DEBUG(log, "There is no part " << *it_in_batch << " in ZooKeeper, it was only in filesystem");
4437
                    }
4438
                    else if (parts_should_be_retried && Coordination::isHardwareError(cur_code))
4439
                    {
A
Alexey Milovidov 已提交
4440
                        parts_should_be_retried->emplace(*it_in_batch);
4441
                    }
4442
                    else if (cur_code)
4443
                    {
4444
                        LOG_WARNING(log, "Cannot remove part " << *it_in_batch << " from ZooKeeper: " << zkutil::ZooKeeper::error2string(cur_code));
4445
                    }
4446 4447
                }
            }
4448
            else if (parts_should_be_retried && Coordination::isHardwareError(code))
4449 4450
            {
                for (auto it_in_batch = it_first_node_in_batch; it_in_batch != it_next; ++it_in_batch)
A
Alexey Milovidov 已提交
4451
                    parts_should_be_retried->emplace(*it_in_batch);
4452
            }
4453
            else if (code)
4454 4455
            {
                LOG_WARNING(log, "There was a problem with deleting " << (it_next - it_first_node_in_batch)
4456
                    << " nodes from ZooKeeper: " << ::zkutil::ZooKeeper::error2string(code));
4457 4458 4459
            }

            it_first_node_in_batch = it_next;
4460 4461 4462 4463 4464
        }
    }
}


4465 4466 4467 4468
void StorageReplicatedMergeTree::clearBlocksInPartition(
    zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num)
{
    Strings blocks;
4469
    if (zookeeper.tryGetChildren(zookeeper_path + "/blocks", blocks))
4470 4471 4472
        throw Exception(zookeeper_path + "/blocks doesn't exist", ErrorCodes::NOT_FOUND_NODE);

    String partition_prefix = partition_id + "_";
4473
    zkutil::AsyncResponses<Coordination::GetResponse> get_futures;
4474 4475 4476 4477 4478 4479 4480 4481 4482
    for (const String & block_id : blocks)
    {
        if (startsWith(block_id, partition_prefix))
        {
            String path = zookeeper_path + "/blocks/" + block_id;
            get_futures.emplace_back(path, zookeeper.asyncTryGet(path));
        }
    }

4483
    zkutil::AsyncResponses<Coordination::RemoveResponse> to_delete_futures;
4484 4485 4486
    for (auto & pair : get_futures)
    {
        const String & path = pair.first;
4487
        auto result = pair.second.get();
4488

4489
        if (result.error == Coordination::ZNONODE)
4490 4491
            continue;

4492
        ReadBufferFromString buf(result.data);
4493 4494 4495 4496 4497 4498 4499 4500 4501
        Int64 block_num = 0;
        bool parsed = tryReadIntText(block_num, buf) && buf.eof();
        if (!parsed || (min_block_num <= block_num && block_num <= max_block_num))
            to_delete_futures.emplace_back(path, zookeeper.asyncTryRemove(path));
    }

    for (auto & pair : to_delete_futures)
    {
        const String & path = pair.first;
4502
        int32_t rc = pair.second.get().error;
4503
        if (rc == Coordination::ZNOTEMPTY)
4504 4505 4506 4507
        {
             /// Can happen if there are leftover block nodes with children created by previous server versions.
            zookeeper.removeRecursive(path);
        }
4508
        else if (rc)
4509 4510 4511 4512 4513 4514 4515
            LOG_WARNING(log,
                "Error while deleting ZooKeeper path `" << path << "`: " + zkutil::ZooKeeper::error2string(rc) << ", ignoring.");
    }

    LOG_TRACE(log, "Deleted " << to_delete_futures.size() << " deduplication block IDs in partition ID " << partition_id);
}

4516 4517 4518 4519 4520 4521 4522 4523 4524 4525 4526 4527 4528 4529 4530
void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace,
                                                      const Context & context)
{
    auto lock1 = lockStructure(false, __PRETTY_FUNCTION__);
    auto lock2 = source_table->lockStructure(false, __PRETTY_FUNCTION__);

    Stopwatch watch;
    MergeTreeData * src_data = data.checkStructureAndGetMergeTreeData(source_table);
    String partition_id = data.getPartitionIDFromQuery(partition, context);

    MergeTreeData::DataPartsVector src_all_parts = src_data->getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id);
    MergeTreeData::DataPartsVector src_parts;
    MergeTreeData::MutableDataPartsVector dst_parts;
    Strings block_id_paths;
    Strings part_checksums;
4531
    std::vector<EphemeralLockInZooKeeper> ephemeral_locks;
4532 4533 4534 4535 4536 4537 4538 4539 4540 4541 4542 4543 4544 4545 4546 4547 4548 4549 4550 4551 4552 4553 4554 4555 4556 4557 4558 4559 4560 4561 4562 4563 4564 4565 4566 4567 4568 4569 4570 4571 4572 4573 4574 4575 4576 4577 4578 4579 4580 4581 4582 4583 4584 4585 4586

    LOG_DEBUG(log, "Cloning " << src_all_parts.size() << " parts");

    static const String TMP_PREFIX = "tmp_replace_from_";
    auto zookeeper = getZooKeeper();

    /// Firstly, generate last block number and compute drop_range
    /// NOTE: Even if we make ATTACH PARTITION instead of REPLACE PARTITION drop_range will not be empty, it will contain a block.
    /// So, such case has special meaning, if drop_range contains only one block it means that nothing to drop.
    MergeTreePartInfo drop_range;
    drop_range.partition_id = partition_id;
    drop_range.max_block = allocateBlockNumber(partition_id, zookeeper)->getNumber();
    drop_range.min_block = replace ? 0 : drop_range.max_block;
    drop_range.level = std::numeric_limits<decltype(drop_range.level)>::max();

    String drop_range_fake_part_name = getPartNamePossiblyFake(data.format_version, drop_range);

    if (drop_range.getBlocksCount() > 1)
    {
        /// We have to prohibit merges in drop_range, since new merge log entry appeared after this REPLACE FROM entry
        ///  could produce new merged part instead in place of just deleted parts.
        /// It is better to prohibit them on leader replica (like DROP PARTITION makes),
        ///  but it is inconvenient for a user since he could actually use source table from this replica.
        /// Therefore prohibit merges on the initializer server now and on the remaining servers when log entry will be executed.
        /// It does not provides strong guarantees, but is suitable for intended use case (assume merges are quite rare).

        {
            std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
            queue.disableMergesInRange(drop_range_fake_part_name);
        }
    }

    for (size_t i = 0; i < src_all_parts.size(); ++i)
    {
        /// We also make some kind of deduplication to avoid duplicated parts in case of ATTACH PARTITION
        /// Assume that merges in the partiton are quite rare
        /// Save deduplication block ids with special prefix replace_partition

        auto & src_part = src_all_parts[i];
        String hash_hex = src_part->checksums.getTotalChecksumHex();
        String block_id_path = replace ? "" : (zookeeper_path + "/blocks/" + partition_id + "_replace_from_" + hash_hex);

        auto lock = allocateBlockNumber(partition_id, zookeeper, block_id_path);
        if (!lock)
        {
            LOG_INFO(log, "Part " << src_part->name << " (hash " << hash_hex << ") has been already attached");
            continue;
        }

        UInt64 index = lock->getNumber();
        MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level);
        auto dst_part = data.cloneAndLoadDataPart(src_part, TMP_PREFIX, dst_part_info);

        src_parts.emplace_back(src_part);
        dst_parts.emplace_back(dst_part);
4587
        ephemeral_locks.emplace_back(std::move(*lock));
4588 4589 4590 4591 4592 4593 4594 4595 4596 4597 4598 4599 4600 4601 4602 4603 4604 4605 4606 4607 4608 4609 4610 4611 4612
        block_id_paths.emplace_back(block_id_path);
        part_checksums.emplace_back(hash_hex);
    }

    ReplicatedMergeTreeLogEntryData entry;
    {
        entry.type = ReplicatedMergeTreeLogEntryData::REPLACE_RANGE;
        entry.source_replica = replica_name;
        entry.create_time = time(nullptr);
        entry.replace_range_entry = std::make_shared<ReplicatedMergeTreeLogEntryData::ReplaceRangeEntry>();

        auto & entry_replace = *entry.replace_range_entry;
        entry_replace.drop_range_part_name = drop_range_fake_part_name;
        entry_replace.from_database = src_data->database_name;
        entry_replace.from_table = src_data->table_name;
        for (const auto & part : src_parts)
            entry_replace.src_part_names.emplace_back(part->name);
        for (const auto & part : dst_parts)
            entry_replace.new_part_names.emplace_back(part->name);
        for (const String & checksum : part_checksums)
            entry_replace.part_names_checksums.emplace_back(checksum);
        entry_replace.columns_version = columns_version;
    }

    /// We are almost ready to commit changes, remove fetches and merges from drop range
4613
    queue.removePartProducingOpsInRange(zookeeper, drop_range, entry);
4614 4615 4616 4617 4618 4619

    /// Remove deduplication block_ids of replacing parts
    if (replace)
        clearBlocksInPartition(*zookeeper, drop_range.partition_id, drop_range.max_block, drop_range.max_block);

    MergeTreeData::DataPartsVector parts_to_remove;
4620
    Coordination::Responses op_results;
4621 4622 4623

    try
    {
4624
        Coordination::Requests ops;
4625 4626 4627
        for (size_t i = 0; i < dst_parts.size(); ++i)
        {
            getCommitPartOps(ops, dst_parts[i], block_id_paths[i]);
4628
            ephemeral_locks[i].getUnlockOps(ops);
4629 4630 4631 4632 4633 4634 4635 4636 4637 4638 4639

            if (ops.size() > zkutil::MULTI_BATCH_SIZE)
            {
                /// It is unnecessary to add parts to working set until we commit log entry
                zookeeper->multi(ops);
                ops.clear();
            }
        }

        ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential));

4640
        MergeTreeData::Transaction transaction(data);
4641 4642 4643 4644 4645 4646 4647 4648 4649 4650 4651 4652 4653 4654 4655 4656 4657 4658 4659 4660 4661 4662 4663 4664 4665
        {
            auto data_parts_lock = data.lockParts();

            for (MergeTreeData::MutableDataPartPtr & part : dst_parts)
                data.renameTempPartAndReplace(part, nullptr, &transaction, data_parts_lock);
        }

        op_results = zookeeper->multi(ops);

        {
            auto data_parts_lock = data.lockParts();

            transaction.commit(&data_parts_lock);
            if (replace)
                parts_to_remove = data.removePartsInRangeFromWorkingSet(drop_range, true, false, data_parts_lock);
        }

        PartLog::addNewParts(this->context, dst_parts, watch.elapsed());
    }
    catch (...)
    {
        PartLog::addNewParts(this->context, dst_parts, watch.elapsed(), ExecutionStatus::fromCurrentException());
        throw;
    }

4666
    String log_znode_path = dynamic_cast<const Coordination::CreateResponse &>(*op_results.back()).path_created;
4667 4668
    entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);

4669
    for (auto & lock : ephemeral_locks)
4670 4671 4672 4673 4674 4675 4676
        lock.assumeUnlocked();

    /// Forcibly remove replaced parts from ZooKeeper
    tryRemovePartsFromZooKeeperWithRetries(parts_to_remove);

    /// Speedup removing of replaced parts from filesystem
    parts_to_remove.clear();
4677
    cleanup_thread.wakeup();
4678 4679 4680 4681 4682 4683 4684

    /// If necessary, wait until the operation is performed on all replicas.
    if (context.getSettingsRef().replication_alter_partitions_sync > 1)
        waitForAllReplicasToProcessLogEntry(entry);
}

void StorageReplicatedMergeTree::getCommitPartOps(
4685
    Coordination::Requests & ops,
4686 4687 4688 4689 4690 4691 4692 4693 4694 4695 4696 4697 4698 4699 4700 4701 4702 4703 4704 4705 4706 4707 4708 4709 4710 4711 4712 4713 4714 4715 4716 4717 4718 4719
    MergeTreeData::MutableDataPartPtr & part,
    const String & block_id_path) const
{
    const String & part_name = part->name;

    if (!block_id_path.empty())
    {
        /// Make final duplicate check and commit block_id
        ops.emplace_back(
            zkutil::makeCreateRequest(
                block_id_path,
                part_name,  /// We will be able to know original part number for duplicate blocks, if we want.
                zkutil::CreateMode::Persistent));
    }

    /// Information about the part, in the replica data.

    ops.emplace_back(zkutil::makeCheckRequest(
        zookeeper_path + "/columns",
        columns_version));
    ops.emplace_back(zkutil::makeCreateRequest(
        replica_path + "/parts/" + part->name,
        "",
        zkutil::CreateMode::Persistent));
    ops.emplace_back(zkutil::makeCreateRequest(
        replica_path + "/parts/" + part->name + "/columns",
        part->columns.toString(),
        zkutil::CreateMode::Persistent));
    ops.emplace_back(zkutil::makeCreateRequest(
        replica_path + "/parts/" + part->name + "/checksums",
        getChecksumsForZooKeeper(part->checksums),
        zkutil::CreateMode::Persistent));
}

4720 4721 4722 4723 4724 4725 4726 4727 4728 4729
ReplicatedMergeTreeAddress StorageReplicatedMergeTree::getReplicatedMergeTreeAddress() const
{
    auto host_port = context.getInterserverIOAddress();

    ReplicatedMergeTreeAddress res;
    res.host = host_port.first;
    res.replication_port = host_port.second;
    res.queries_port = context.getTCPPort();
    res.database = database_name;
    res.table = table_name;
4730
    res.scheme = context.getInterserverScheme();
4731 4732 4733
    return res;
}

4734
ActionLock StorageReplicatedMergeTree::getActionLock(StorageActionBlockType action_type)
4735 4736
{
    if (action_type == ActionLocks::PartsMerge)
4737
        return merger_mutator.actions_blocker.cancel();
4738 4739 4740 4741 4742 4743 4744 4745

    if (action_type == ActionLocks::PartsFetch)
        return fetcher.blocker.cancel();

    if (action_type == ActionLocks::PartsSend)
        return data_parts_exchange_endpoint_holder ? data_parts_exchange_endpoint_holder->getBlocker().cancel() : ActionLock();

    if (action_type == ActionLocks::ReplicationQueue)
4746
        return queue.actions_blocker.cancel();
4747 4748 4749 4750

    return {};
}

4751

4752 4753 4754
bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UInt64 max_wait_milliseconds)
{
    /// Let's fetch new log entries firstly
4755
    queue.pullLogsToQueue(getZooKeeper());
4756 4757 4758 4759 4760 4761 4762 4763 4764 4765 4766 4767 4768 4769 4770 4771 4772 4773 4774 4775 4776 4777 4778 4779 4780

    Stopwatch watch;
    Poco::Event event;
    std::atomic<bool> cond_reached{false};

    auto callback = [&event, &cond_reached, queue_size] (size_t new_queue_size)
    {
        if (new_queue_size <= queue_size)
            cond_reached.store(true, std::memory_order_relaxed);

        event.set();
    };

    auto handler = queue.addSubscriber(std::move(callback));

    while (true)
    {
        event.tryWait(50);

        if (max_wait_milliseconds && watch.elapsedMilliseconds() > max_wait_milliseconds)
            break;

        if (cond_reached)
            break;

4781
        if (partial_shutdown_called)
4782 4783 4784
            throw Exception("Shutdown is called for table", ErrorCodes::ABORTED);
    }

4785
    return cond_reached.load(std::memory_order_relaxed);
4786
}
4787

4788

4789
bool StorageReplicatedMergeTree::dropPartsInPartition(
Z
zhang2014 已提交
4790 4791
    zkutil::ZooKeeper & zookeeper, String & partition_id, StorageReplicatedMergeTree::LogEntry & entry, bool detach)
{
4792 4793
    MergeTreePartInfo drop_range_info;
    if (!getFakePartCoveringAllPartsInPartition(partition_id, drop_range_info))
Z
zhang2014 已提交
4794 4795 4796 4797 4798
    {
        LOG_INFO(log, "Will not drop partition " << partition_id << ", it is empty.");
        return false;
    }

4799
    clearBlocksInPartition(zookeeper, partition_id, drop_range_info.min_block, drop_range_info.max_block);
Z
zhang2014 已提交
4800 4801 4802 4803

    /** Forbid to choose the parts to be deleted for merging.
      * Invariant: after the `DROP_RANGE` entry appears in the log, merge of deleted parts will not appear in the log.
      */
4804
    String drop_range_fake_part_name = getPartNamePossiblyFake(data.format_version, drop_range_info);
Z
zhang2014 已提交
4805 4806
    {
        std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
4807
        queue.disableMergesInRange(drop_range_fake_part_name);
Z
zhang2014 已提交
4808 4809
    }

4810
    LOG_DEBUG(log, "Disabled merges covered by range " << drop_range_fake_part_name);
Z
zhang2014 已提交
4811 4812 4813 4814

    /// Finally, having achieved the necessary invariants, you can put an entry in the log.
    entry.type = LogEntry::DROP_RANGE;
    entry.source_replica = replica_name;
4815
    entry.new_part_name = drop_range_fake_part_name;
Z
zhang2014 已提交
4816 4817 4818
    entry.detach = detach;
    entry.create_time = time(nullptr);

4819
    String log_znode_path = zookeeper.create(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential);
Z
zhang2014 已提交
4820 4821 4822
    entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);

    return true;
4823 4824
}

M
Merge  
Michael Kolupaev 已提交
4825
}