StorageReplicatedMergeTree.cpp 185.7 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/ASTOptimizeQuery.h>
19
#include <Parsers/ASTLiteral.h>
20 21 22

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

#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 已提交
33
#include <Storages/VirtualColumnUtils.h>
34 35 36
#include <Common/formatReadable.h>
#include <Common/setThreadName.h>
#include <Common/escapeForFileName.h>
37
#include <Common/StringUtils/StringUtils.h>
38
#include <Common/typeid_cast.h>
39

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

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

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

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


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

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


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

74 75
namespace ErrorCodes
{
76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92
    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 已提交
93
    extern const int TOO_MANY_RETRIES_TO_FETCH_PARTS;
94 95 96 97
    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;
98 99
    extern const int UNEXPECTED_FILE_IN_DATA_PART;
    extern const int NO_FILE_IN_DATA_PART;
100 101
    extern const int UNFINISHED;
    extern const int METADATA_MISMATCH;
102
    extern const int RECEIVED_ERROR_TOO_MANY_REQUESTS;
A
Alexey Milovidov 已提交
103
    extern const int TOO_MANY_FETCHES;
104
    extern const int BAD_DATA_PART_NAME;
105
    extern const int PART_IS_TEMPORARILY_LOCKED;
106
    extern const int INCORRECT_FILE_NAME;
107
    extern const int CANNOT_ASSIGN_OPTIMIZE;
108
    extern const int KEEPER_EXCEPTION;
V
VadimPE 已提交
109
    extern const int ALL_REPLICAS_LOST;
V
VadimPE 已提交
110
    extern const int REPLICA_STATUS_CHANGED;
111 112 113 114 115 116 117 118
}

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

M
Merge  
Michael Kolupaev 已提交
121

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

F
f1yegor 已提交
126 127
/** There are three places for each part, where it should be
  * 1. In the RAM, MergeTreeData::data_parts, all_data_parts.
128
  * 2. In the filesystem (FS), the directory with the data of the table.
F
f1yegor 已提交
129
  * 3. in ZooKeeper (ZK).
130
  *
F
f1yegor 已提交
131 132
  * When adding a part, it must be added immediately to these three places.
  * This is done like this
133 134
  * - [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 已提交
135 136 137 138
  * - [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);
139
  * - [FS, ZK] by the way, removing the covered (old) parts from filesystem, from ZooKeeper and from `all_data_parts`
F
f1yegor 已提交
140
  *   is delayed, after a few minutes.
141
  *
F
f1yegor 已提交
142 143 144
  * 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.
145
  *
F
f1yegor 已提交
146 147 148 149 150
  * 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.
151
  *
F
f1yegor 已提交
152 153 154 155
  * 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.
156
  *
F
f1yegor 已提交
157 158
  * 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.
159
  */
160
extern const int MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER = 5 * 60;
161 162


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


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

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

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


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

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

233 234 235 236 237 238 239 240
    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();

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

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

246 247
    bool skip_sanity_checks = false;

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

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

        LOG_WARNING(log, "Skipping the limits on severity of changes to data parts and columns (flag force_restore_data).");
261 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
    }

    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);
290 291 292 293

        /// 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);
294 295 296
    }

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

299

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

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

309
    /// Tracking lag of replicas.
310 311 312 313 314
    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());
315
    zookeeper->createIfNotExists(replica_path + "/mutation_pointer", String());
316 317 318
}


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

A
Merge  
Alexey Milovidov 已提交
328

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

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

        void write(WriteBuffer & out) const
        {
            out << "metadata format version: 1" << "\n"
344 345 346 347 348 349 350 351
                << "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"
352 353 354 355
                << "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";
356 357 358 359 360 361

            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";
            }
362 363 364 365
        }

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

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

            in >> "metadata format version: 1";

            in >> "\ndate column: ";
378 379 380 381 382 383 384 385 386 387 388 389 390 391 392
            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.",
393 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
                    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";
446 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
            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";
            }

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

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


M
Merge  
Michael Kolupaev 已提交
489
void StorageReplicatedMergeTree::createTableIfNotExists()
M
Merge  
Michael Kolupaev 已提交
490
{
491 492 493 494 495 496 497 498 499 500 501 502
    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();

503
    Coordination::Requests ops;
504 505 506 507 508 509 510 511 512 513 514 515 516
    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", "",
517
        zkutil::CreateMode::Persistent)); /// /nonincrement_block_numbers dir is unused, but is created nonetheless for backwards compatibility.
518 519 520 521 522 523
    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));
524

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

A
Merge  
Alexey Milovidov 已提交
531

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

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

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

546 547
    const ColumnsDescription & old_columns = getColumns();
    if (columns_from_zk != old_columns)
548 549 550
    {
        if (allow_alter &&
            (skip_sanity_checks ||
551 552
             old_columns.ordinary.sizeOfDifference(columns_from_zk.ordinary) +
             old_columns.materialized.sizeOfDifference(columns_from_zk.materialized) <= 2))
553 554 555 556
        {
            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.
557 558
            context.getDatabase(database_name)->alterTable(context, table_name, columns_from_zk, {});

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

A
Merge  
Alexey Milovidov 已提交
569

F
f1yegor 已提交
570 571 572 573
/** 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).
574 575 576
  */
static time_t tryGetPartCreateTime(zkutil::ZooKeeperPtr & zookeeper, const String & replica_path, const String & part_name)
{
577
    time_t res = 0;
578

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

585
    return res;
586 587 588
}


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

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

595
    int32_t code;
A
Alexey Milovidov 已提交
596

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

A
Alexey Milovidov 已提交
602 603
        /// 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 已提交
604

V
VadimPE 已提交
605 606
        Coordination::Requests ops;
        Coordination::Responses resps;
607 608 609 610 611 612 613 614
        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 已提交
615
        /// Check version of /replicas to see if there are any replicas created at the same moment of time.
V
VadimPE 已提交
616
        ops.emplace_back(zkutil::makeSetRequest(zookeeper_path + "/replicas", "last added replica: " + replica_name, replicas_stat.version));
617 618

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


M
Merge  
Michael Kolupaev 已提交
629
void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
M
Merge  
Michael Kolupaev 已提交
630
{
631 632 633 634 635 636 637
    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());

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

    /// 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;
654
    UInt64 parts_to_add_rows = 0;
655 656 657 658 659 660 661 662 663 664 665 666 667 668 669

    /// 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);
670
                parts_to_add_rows += containing->rows_count;
671 672 673 674 675 676 677 678 679 680 681 682
            }
        }
        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);

683

684 685 686 687 688
    /** 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;
689 690
    UInt64 unexpected_parts_nonnew_rows = 0;
    UInt64 unexpected_parts_rows = 0;
691
    for (const auto & part : unexpected_parts)
692
    {
693
        if (part->info.level > 0)
694
        {
695
            ++unexpected_parts_nonnew;
696 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
            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).";
728

729 730 731
    /** 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%).
732
      *
733 734
      * 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).
735 736 737
      * In this case, the protection mechanism does not allow the server to start.
      */

738 739 740 741 742 743
    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;
744

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

    if (insane && !skip_sanity_checks)
748 749 750 751 752 753 754 755 756
    {
        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);
    }
757

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

    /// 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);

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

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

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

    /// 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).
791
        Coordination::Requests ops;
792
        removePartFromZooKeeper(name, ops);
793 794
        ops.emplace_back(zkutil::makeCreateRequest(
            replica_path + "/queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential));
795 796 797 798 799 800 801
        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);
802
        data.forgetPartAndMoveToDetached(part, "ignored_", true);
803
    }
M
Merge  
Michael Kolupaev 已提交
804
}
M
Merge  
Michael Kolupaev 已提交
805

A
Merge  
Alexey Milovidov 已提交
806

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

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

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

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

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

832
            continue;
833 834
        }

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

842 843 844
        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.
845 846
        if (!zookeeper->tryGet(current_part_path + "/checksums", checksums_str) ||
            !zookeeper->exists(current_part_path + "/columns", &stat_after) ||
847 848 849 850 851 852 853
            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;
        }

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

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

        /// 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;
        }
867 868
    }

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

873
        ops.emplace_back(zkutil::makeCheckRequest(
874
            zookeeper_path + "/columns", expected_columns_version));
875 876 877 878 879 880
        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));
881 882 883 884 885
    }
    else
    {
        LOG_WARNING(log, "checkPartAndAddToZooKeeper: node " << replica_path + "/parts/" + part_name << " already exists."
            << " Will not commit any nodes.");
886
    }
887
}
888

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

894 895
    while (true)
    {
896
        Coordination::Requests ops;
897 898 899 900 901 902 903 904 905 906 907 908
        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())
        {
909
            Coordination::Requests new_ops;
910 911
            for (const String & part_path : absent_part_paths_on_replicas)
            {
912 913
                new_ops.emplace_back(zkutil::makeCreateRequest(part_path, "", zkutil::CreateMode::Persistent));
                new_ops.emplace_back(zkutil::makeRemoveRequest(part_path, -1));
914 915 916 917 918 919 920 921 922 923 924 925
            }

            /// 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();
        }
926
        catch (const zkutil::KeeperMultiException & e)
927 928
        {
            size_t num_check_ops = 2 * absent_part_paths_on_replicas.size();
929
            size_t failed_op_index = e.failed_op_index;
930

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

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

A
Merge  
Alexey Milovidov 已提交
947

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

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

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

968
    if (entry.type == LogEntry::GET_PART ||
969 970
        entry.type == LogEntry::MERGE_PARTS ||
        entry.type == LogEntry::MUTATE_PART)
971 972
    {
        /// If we already have this part or a part covering it, we do not need to do anything.
A
Alexey Zatelepin 已提交
973 974
        /// 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.
975 976 977
        MergeTreeData::DataPartPtr existing_part = data.getPartIfExists(entry.new_part_name, {MergeTreeDataPartState::PreCommitted});
        if (!existing_part)
            existing_part = data.getActiveContainingPart(entry.new_part_name);
978 979

        /// Even if the part is locally, it (in exceptional cases) may not be in ZooKeeper. Let's check that it is there.
980
        if (existing_part && getZooKeeper()->exists(replica_path + "/parts/" + existing_part->name))
981 982
        {
            if (!(entry.type == LogEntry::GET_PART && entry.source_replica == replica_name))
983 984 985
            {
                LOG_DEBUG(log, "Skipping action for part " << entry.new_part_name << " because part " + existing_part->name + " already exists.");
            }
986 987 988 989 990 991 992 993 994 995 996 997 998 999 1000 1001 1002 1003 1004 1005
            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)
1006
    {
1007 1008 1009 1010 1011
        do_fetch = !tryExecuteMerge(entry);
    }
    else if (entry.type == LogEntry::MUTATE_PART)
    {
        do_fetch = !tryExecutePartMutation(entry);
1012 1013 1014 1015 1016 1017 1018 1019 1020 1021 1022 1023 1024
    }
    else
    {
        throw Exception("Unexpected log entry type: " + toString(static_cast<int>(entry.type)));
    }

    if (do_fetch)
        return executeFetch(entry);

    return true;
}


1025 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
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__);
    }
}


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

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

1094 1095
    MergeTreeData::DataPartsVector parts;
    bool have_all_parts = true;
1096
    for (const String & name : entry.source_parts)
1097 1098 1099
    {
        MergeTreeData::DataPartPtr part = data.getActiveContainingPart(name);
        if (!part)
1100
        {
1101 1102
            have_all_parts = false;
            break;
1103
        }
1104
        if (part->name != name)
1105
        {
1106 1107 1108 1109
            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;
1110
        }
1111 1112 1113 1114 1115 1116 1117
        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");
1118
        return false;
1119 1120 1121 1122 1123
    }
    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.
1124

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

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

1140
    /// Start to make the main work
1141

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

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

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

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

1151
    MergeTreeDataMergerMutator::FuturePart future_merged_part(parts);
1152 1153 1154 1155 1156
    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);
    }
1157

1158
    MergeTreeData::Transaction transaction;
1159
    MergeTreeData::MutableDataPartPtr part;
1160

1161
    Stopwatch stopwatch;
1162

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

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

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

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

                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());
1203 1204

                return false;
1205
            }
1206 1207 1208 1209

            throw;
        }

1210
        /** Removing old parts from ZK and from the disk is delayed - see ReplicatedMergeTreeCleanupThread, clearOldParts.
1211 1212
          */

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

        write_part_log({});
1220 1221 1222 1223 1224 1225 1226 1227 1228 1229 1230 1231 1232 1233 1234 1235 1236 1237 1238

        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)
    {
1239
        LOG_DEBUG(log, "Source part " + source_part_name + " for " << entry.new_part_name << " is not ready; will try to fetch it instead");
1240 1241 1242 1243 1244 1245 1246 1247 1248 1249 1250
        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?
1251
    size_t estimated_space_for_result = MergeTreeDataMergerMutator::estimateNeededDiskSpace({source_part});
1252 1253 1254 1255 1256 1257 1258 1259 1260 1261 1262 1263 1264 1265 1266 1267 1268 1269 1270 1271 1272 1273 1274 1275 1276 1277

    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;
    MergeTreeData::Transaction transaction;

1278 1279 1280 1281 1282
    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;

1283
    Stopwatch stopwatch;
1284 1285

    auto write_part_log = [&] (const ExecutionStatus & execution_status)
1286
    {
1287 1288 1289
        writePartLog(
            PartLogElement::MUTATE_PART, execution_status, stopwatch.elapsed(),
            entry.new_part_name, new_part, future_mutated_part.parts, nullptr);
1290 1291 1292 1293
    };

    try
    {
1294
        new_part = merger_mutator.mutatePartToTemporaryPart(future_mutated_part, commands, context);
1295
        data.renameTempPartAndReplace(new_part, nullptr, &transaction);
1296 1297 1298 1299 1300 1301 1302 1303 1304 1305 1306

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

1307
                ProfileEvents::increment(ProfileEvents::DataAfterMutationDiffersFromReplica);
1308 1309 1310 1311 1312 1313 1314 1315 1316 1317 1318 1319 1320 1321 1322 1323

                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());

                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.
          */
1324
        merge_selecting_task->schedule();
1325
        ProfileEvents::increment(ProfileEvents::ReplicatedPartMutations);
1326 1327 1328
        write_part_log({});

        return true;
1329
    }
1330
    catch (...)
1331
    {
1332 1333
        write_part_log(ExecutionStatus::fromCurrentException());
        throw;
1334
    }
1335
}
1336 1337


1338
bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry)
1339 1340
{
    String replica = findReplicaHavingCoveringPart(entry, true);
1341

1342 1343 1344 1345
    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 已提交
1346
            ErrorCodes::TOO_MANY_FETCHES);
1347
    }
1348

1349 1350
    ++total_fetches;
    SCOPE_EXIT({--total_fetches;});
1351

1352 1353 1354
    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 已提交
1355
            ErrorCodes::TOO_MANY_FETCHES);
1356
    }
1357

1358 1359 1360 1361 1362 1363
    ++current_table_fetches;
    SCOPE_EXIT({--current_table_fetches;});

    try
    {
        if (replica.empty())
1364
        {
1365 1366 1367 1368 1369 1370
            /** 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)
1371
            {
1372 1373 1374 1375 1376 1377 1378 1379 1380 1381 1382 1383 1384 1385
                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.
1386 1387
                  */

1388 1389 1390 1391
                /** 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.
                  */
1392

1393
                auto zookeeper = getZooKeeper();
1394

1395
                Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
1396

1397
                Coordination::Requests ops;
1398

1399 1400
                for (size_t i = 0, size = replicas.size(); i < size; ++i)
                {
1401
                    Coordination::Stat stat;
1402 1403
                    String path = zookeeper_path + "/replicas/" + replicas[i] + "/host";
                    zookeeper->get(path, &stat);
1404
                    ops.emplace_back(zkutil::makeCheckRequest(path, stat.version));
1405
                }
1406

1407 1408
                /// 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);
1409

1410 1411 1412 1413 1414
                /// 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())
                {
1415
                    Coordination::Stat quorum_stat;
1416 1417 1418 1419 1420 1421
                    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)
1422
                    {
1423
                        ops.emplace_back(zkutil::makeRemoveRequest(quorum_path, quorum_stat.version));
1424

1425
                        auto part_info = MergeTreePartInfo::fromPartName(entry.new_part_name, data.format_version);
1426

1427 1428 1429
                        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);
1430

1431
                        ops.emplace_back(zkutil::makeCreateRequest(
1432 1433 1434 1435 1436 1437
                            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))
1438
                            ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/blocks/" + entry.block_id, -1));
1439

1440
                        Coordination::Responses responses;
1441
                        auto code = zookeeper->tryMulti(ops, responses);
1442

1443
                        if (code == Coordination::ZOK)
1444
                        {
1445
                            LOG_DEBUG(log, "Marked quorum for part " << entry.new_part_name << " as failed.");
1446 1447
                            queue.removeFromVirtualParts(part_info);
                            return true;
1448
                        }
1449
                        else if (code == Coordination::ZBADVERSION || code == Coordination::ZNONODE || code == Coordination::ZNODEEXISTS)
1450
                        {
1451
                            LOG_DEBUG(log, "State was changed or isn't expected when trying to mark quorum for part "
1452
                                << entry.new_part_name << " as failed. Code: " << zkutil::ZooKeeper::error2string(code));
1453
                        }
1454
                        else
1455
                            throw Coordination::Exception(code);
1456 1457 1458 1459 1460 1461 1462 1463
                    }
                    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.");
1464 1465 1466 1467
                    }
                }
            }

1468
            if (replica.empty())
1469
            {
1470 1471
                ProfileEvents::increment(ProfileEvents::ReplicatedPartFailedFetches);
                throw Exception("No active replica has part " + entry.new_part_name + " or covering part", ErrorCodes::NO_REPLICA_HAS_PART);
1472
            }
1473
        }
1474

1475 1476 1477 1478
        try
        {
            if (!fetchPart(entry.actual_new_part_name, zookeeper_path + "/replicas/" + replica, false, entry.quorum))
                return false;
1479
        }
1480
        catch (Exception & e)
1481
        {
1482 1483 1484 1485 1486
            /// No stacktrace, just log message
            if (e.code() == ErrorCodes::RECEIVED_ERROR_TOO_MANY_REQUESTS)
                e.addMessage("Too busy replica. Will try later.");
            throw;
        }
1487

1488 1489 1490 1491 1492 1493 1494 1495 1496 1497 1498 1499
        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);
1500

1501
            if (!parts_for_merge.empty() && replica.empty())
1502
            {
1503 1504
                LOG_INFO(log, "No active replica has part " << entry.new_part_name << ". Will fetch merged part instead.");
                return false;
1505 1506
            }

1507 1508 1509 1510 1511
            /** 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);
1512
        }
1513 1514
        catch (...)
        {
1515
            tryLogCurrentException(log, __PRETTY_FUNCTION__);
1516 1517 1518
        }

        throw;
1519 1520 1521
    }

    return true;
M
Merge  
Michael Kolupaev 已提交
1522 1523
}

A
Merge  
Alexey Milovidov 已提交
1524

1525
void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
1526
{
1527
    auto drop_range_info = MergeTreePartInfo::fromPartName(entry.new_part_name, data.format_version);
1528
    queue.removePartProducingOpsInRange(getZooKeeper(), drop_range_info, entry);
1529 1530

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

1532 1533 1534 1535
    /// 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.
1536 1537
    /// Therefore, we use all data parts.

1538
    MergeTreeData::DataPartsVector parts_to_remove;
1539
    {
1540 1541 1542
        auto data_parts_lock = data.lockParts();
        parts_to_remove = data.removePartsInRangeFromWorkingSet(drop_range_info, true, true, data_parts_lock);
    }
1543

1544 1545 1546 1547 1548 1549 1550 1551 1552
    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("");
        }
    }
1553

1554 1555
    /// Forcibly remove parts from ZooKeeper
    tryRemovePartsFromZooKeeperWithRetries(parts_to_remove);
1556

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

1559 1560 1561
    /// 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();
1562
    cleanup_thread.wakeup();
M
Merge  
Michael Kolupaev 已提交
1563 1564
}

A
Merge  
Alexey Milovidov 已提交
1565

1566
void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry & entry)
1567
{
1568
    LOG_INFO(log, "Clear column " << entry.column_name << " in parts inside " << entry.new_part_name << " range");
1569

1570 1571
    auto entry_part_info = MergeTreePartInfo::fromPartName(entry.new_part_name, data.format_version);

1572 1573 1574
    /// 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)
1575
    auto lock_read_structure = lockStructure(false, __PRETTY_FUNCTION__);
1576 1577 1578 1579 1580 1581 1582

    auto zookeeper = getZooKeeper();

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

1583
    auto new_columns = getColumns();
1584
    alter_command.apply(new_columns);
1585 1586

    size_t modified_parts = 0;
1587
    auto parts = data.getDataParts();
A
Alexey Zatelepin 已提交
1588
    auto columns_for_parts = new_columns.getAllPhysical();
1589 1590 1591 1592 1593 1594 1595 1596

    /// 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);

1597 1598
    for (const auto & part : parts)
    {
1599
        if (!entry_part_info.contains(part->info))
1600 1601
            continue;

1602
        LOG_DEBUG(log, "Clearing column " << entry.column_name << " in part " << part->name);
1603 1604 1605 1606 1607 1608

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

        /// Update part metadata in ZooKeeper.
1609
        Coordination::Requests ops;
1610
        ops.emplace_back(zkutil::makeSetRequest(
1611
            replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1));
1612
        ops.emplace_back(zkutil::makeSetRequest(
1613
            replica_path + "/parts/" + part->name + "/checksums", getChecksumsForZooKeeper(transaction->getNewChecksums()), -1));
1614 1615 1616 1617 1618 1619 1620

        zookeeper->multi(ops);

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

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

1623
    /// Recalculate columns size (not only for the modified column)
1624 1625 1626 1627
    data.recalculateColumnSizes();
}


1628
bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
1629
{
1630 1631
    Stopwatch watch;
    auto & entry_replace = *entry.replace_range_entry;
1632

1633 1634 1635 1636
    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;

1637
    queue.removePartProducingOpsInRange(getZooKeeper(), drop_range, entry);
1638 1639

    struct PartDescription
S
Silviu Caragea 已提交
1640
    {
1641 1642 1643 1644 1645 1646 1647 1648 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
        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)
1692
        {
1693 1694
            if (!data.getActiveContainingPart(part_desc->new_part_info, MergeTreeDataPartState::Committed, data_parts_lock))
                parts_to_add.emplace_back(part_desc);
1695
        }
1696 1697 1698 1699 1700 1701 1702 1703 1704 1705 1706 1707

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

1708
    if (parts_to_add.size() < all_parts.size())
1709 1710 1711 1712 1713 1714 1715 1716 1717 1718 1719 1720 1721 1722 1723 1724 1725 1726
    {
        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;
1727 1728
        try
        {
1729
            src_data = data.checkStructureAndGetMergeTreeData(source_table);
1730
        }
1731
        catch (Exception &)
1732
        {
1733 1734 1735 1736
            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;
        }
1737

1738 1739 1740 1741 1742 1743 1744 1745 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
        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;
1771
        }
1772 1773 1774 1775 1776 1777 1778 1779 1780 1781 1782 1783 1784

        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)
1785
        {
1786
            /// It is clonable part
1787
            adding_parts_active_set.add(part_desc->new_part_name);
1788 1789
            part_name_to_desc.emplace(part_desc->new_part_name, part_desc);
            continue;
1790
        }
1791 1792 1793 1794 1795 1796 1797 1798 1799 1800 1801 1802 1803 1804 1805 1806 1807 1808 1809 1810 1811 1812 1813 1814 1815 1816 1817 1818 1819

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

1820
        adding_parts_active_set.add(part_desc->found_new_part_name);
1821
        part_name_to_desc.emplace(part_desc->found_new_part_name, part_desc);
S
Silviu Caragea 已提交
1822
    }
1823

1824 1825 1826
    /// Check that we could cover whole range
    for (PartDescriptionPtr & part_desc : parts_to_add)
    {
1827
        if (adding_parts_active_set.getContainingPart(part_desc->new_part_info).empty())
1828 1829 1830 1831 1832
        {
            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 已提交
1833

1834 1835 1836
    /// Filter covered parts
    PartDescriptions final_parts;
    {
1837
        Strings final_part_names = adding_parts_active_set.getParts();
A
Merge  
Alexey Milovidov 已提交
1838

1839 1840 1841 1842 1843 1844 1845
        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);
1846

1847 1848 1849 1850 1851 1852 1853 1854 1855 1856 1857 1858 1859 1860 1861 1862 1863
            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)
1864
    {
1865
        if (part_desc->src_table_part)
1866
        {
1867 1868 1869 1870 1871 1872 1873
            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);
1874
        }
1875
        else if (!part_desc->replica.empty())
1876
        {
1877 1878 1879
            String replica_path = zookeeper_path + "/replicas/" + part_desc->replica;
            ReplicatedMergeTreeAddress address(getZooKeeper()->get(replica_path + "/host"));
            auto timeouts = ConnectionTimeouts::getHTTPTimeouts(context.getSettingsRef());
1880
            auto [user, password] = context.getInterserverCredentials();
1881
            String interserver_scheme = context.getInterserverScheme();
1882

1883 1884
            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);
1885 1886

            part_desc->res_part = fetcher.fetchPart(part_desc->found_new_part_name, replica_path,
1887
                                                    address.host, address.replication_port, timeouts, user, password, interserver_scheme, false, TMP_PREFIX + "fetch_");
1888 1889 1890 1891 1892 1893 1894 1895 1896 1897 1898 1899 1900 1901 1902 1903 1904 1905 1906 1907 1908 1909 1910 1911

            /// 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();
        MergeTreeData::Transaction transaction;

1912
        Coordination::Requests ops;
1913 1914 1915 1916 1917 1918 1919 1920 1921 1922
        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();
            }
1923
        }
1924 1925 1926 1927 1928 1929 1930 1931 1932 1933 1934 1935 1936 1937 1938 1939 1940 1941 1942 1943 1944 1945 1946

        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();
1947
    cleanup_thread.wakeup();
1948 1949 1950 1951 1952

    return true;
}


V
VadimPE 已提交
1953
void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coordination::Stat source_is_lost_stat, zkutil::ZooKeeperPtr & zookeeper)
A
Alexey Milovidov 已提交
1954
{
1955 1956 1957
    LOG_INFO(log, "Will mimic " << source_replica);

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

V
Vadim 已提交
1959
    /** TODO: it will be deleted! (It is only to support old version of CH server).
A
Alexey Milovidov 已提交
1960 1961 1962
      * In current code, the replica is created in single transaction.
      * If the reference/master replica is not yet fully created, let's wait.
      */
1963 1964 1965 1966 1967 1968 1969 1970 1971 1972 1973 1974 1975 1976 1977
    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 已提交
1978

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

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

V
VadimPE 已提交
1984
    /// For support old versions CH.
V
VadimPE 已提交
1985
    if (source_is_lost_stat.version == -1)
V
VadimPE 已提交
1986
    {
A
Alexey Milovidov 已提交
1987 1988 1989
        /// 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.
        ops.push_back(zkutil::makeCreateRequest(replica_path + "/is_lost", "0", zkutil::CreateMode::Persistent));
1990
        ops.push_back(zkutil::makeRemoveRequest(replica_path + "/is_lost", -1));
V
VadimPE 已提交
1991
    }
A
Alexey Milovidov 已提交
1992
    else    /// The replica we clone should not suddenly become lost.
V
VadimPE 已提交
1993
        ops.push_back(zkutil::makeCheckRequest(source_path + "/is_lost", source_is_lost_stat.version));
A
Alexey Milovidov 已提交
1994

V
VadimPE 已提交
1995
    Coordination::Responses resp;
A
Alexey Milovidov 已提交
1996

1997
    auto error = zookeeper->tryMulti(ops, resp);
V
VadimPE 已提交
1998
    if (error == Coordination::Error::ZBADVERSION)
V
VadimPE 已提交
1999
        throw Exception("Can not clone replica, because the " + source_replica + " became lost", ErrorCodes::REPLICA_STATUS_CHANGED);
V
VadimPE 已提交
2000
    else if (error == Coordination::Error::ZNODEEXISTS)
A
Alexey Milovidov 已提交
2001
        throw Exception("Can not clone replica, because the " + source_replica + " updated to new ClickHouse version", ErrorCodes::REPLICA_STATUS_CHANGED);
V
VadimPE 已提交
2002 2003
    else
        zkutil::KeeperMultiException::check(error, ops, resp);
A
Alexey Milovidov 已提交
2004

2005 2006 2007 2008 2009 2010 2011 2012 2013 2014 2015 2016 2017 2018 2019 2020 2021 2022 2023 2024 2025 2026 2027 2028 2029
    /// 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);

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

2033 2034 2035 2036 2037 2038 2039 2040 2041
    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");
2042 2043 2044
}


2045
void StorageReplicatedMergeTree::cloneReplicaIfNeeded(zkutil::ZooKeeperPtr zookeeper)
2046
{
2047 2048 2049 2050 2051 2052 2053 2054
    String res;
    if (zookeeper->tryGet(replica_path + "/is_lost", res))
    {
        if (res == "0")
            return;
    }
    else
    {
V
VadimPE 已提交
2055
        /// Replica was created by old version of CH, so me must create "/is_lost".
A
Alexey Milovidov 已提交
2056
        /// Note that in old version of CH there was no "lost" replicas possible.
2057
        zookeeper->create(replica_path + "/is_lost", "0", zkutil::CreateMode::Persistent);
2058
        return;
2059
    }
2060

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

2063
    String source_replica;
V
VadimPE 已提交
2064
    Coordination::Stat source_is_lost_stat;
V
VadimPE 已提交
2065
    source_is_lost_stat.version = -1;
2066

V
VadimPE 已提交
2067
    for (const String & replica_name : zookeeper->getChildren(zookeeper_path + "/replicas"))
2068
    {
V
VadimPE 已提交
2069
        String source_replica_path = zookeeper_path + "/replicas/" + replica_name;
A
Alexey Milovidov 已提交
2070

A
Alexey Milovidov 已提交
2071
        /// Do not clone from myself.
2072
        if (source_replica_path != replica_path)
2073
        {
A
Alexey Milovidov 已提交
2074 2075 2076 2077
            /// 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 已提交
2078
            {
2079
                source_replica = replica_name;
V
VadimPE 已提交
2080
                break;
V
Vadim 已提交
2081
            }
2082
        }
V
VadimPE 已提交
2083
    }
A
Alexey Milovidov 已提交
2084

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

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

A
Alexey Milovidov 已提交
2092
    /// If replica is repaired successfully, we remove is_lost flag.
2093
    zookeeper->set(replica_path + "/is_lost", "0");
2094 2095 2096
}


2097
void StorageReplicatedMergeTree::queueUpdatingTask()
M
Merge  
Michael Kolupaev 已提交
2098
{
2099
    if (!queue_update_in_progress)
S
Silviu Caragea 已提交
2100
    {
2101 2102 2103 2104 2105
        last_queue_update_start_time.store(time(nullptr));
        queue_update_in_progress = true;
    }
    try
    {
2106
        queue.pullLogsToQueue(getZooKeeper(), queue_updating_task->getWatchCallback());
2107 2108
        last_queue_update_finish_time.store(time(nullptr));
        queue_update_in_progress = false;
2109
    }
2110
    catch (const Coordination::Exception & e)
2111 2112
    {
        tryLogCurrentException(log, __PRETTY_FUNCTION__);
2113

2114
        if (e.code == Coordination::ZSESSIONEXPIRED)
2115
        {
2116
            restarting_thread.wakeup();
2117
            return;
2118
        }
2119

2120
        queue_updating_task->scheduleAfter(QUEUE_UPDATE_ERROR_SLEEP_MS);
2121
    }
2122 2123
    catch (...)
    {
2124
        tryLogCurrentException(log, __PRETTY_FUNCTION__);
2125
        queue_updating_task->scheduleAfter(QUEUE_UPDATE_ERROR_SLEEP_MS);
S
Silviu Caragea 已提交
2126
    }
M
Merge  
Michael Kolupaev 已提交
2127
}
M
Merge  
Michael Kolupaev 已提交
2128

A
Merge  
Alexey Milovidov 已提交
2129

2130
void StorageReplicatedMergeTree::mutationsUpdatingTask()
2131 2132 2133
{
    try
    {
2134
        queue.updateMutations(getZooKeeper(), mutations_updating_task->getWatchCallback());
2135
    }
2136
    catch (const Coordination::Exception & e)
2137 2138 2139
    {
        tryLogCurrentException(log, __PRETTY_FUNCTION__);

2140
        if (e.code == Coordination::ZSESSIONEXPIRED)
2141 2142
            return;

2143
        mutations_updating_task->scheduleAfter(QUEUE_UPDATE_ERROR_SLEEP_MS);
2144 2145 2146 2147
    }
    catch (...)
    {
        tryLogCurrentException(log, __PRETTY_FUNCTION__);
2148
        mutations_updating_task->scheduleAfter(QUEUE_UPDATE_ERROR_SLEEP_MS);
2149
    }
2150 2151 2152
}


2153
bool StorageReplicatedMergeTree::queueTask()
M
Merge  
Michael Kolupaev 已提交
2154
{
2155
    /// If replication queue is stopped exit immediately as we successfully executed the task
2156
    if (queue.actions_blocker.isCancelled())
2157 2158 2159 2160 2161
    {
        std::this_thread::sleep_for(std::chrono::milliseconds(5));
        return true;
    }

2162 2163 2164 2165 2166
    /// This object will mark the element of the queue as running.
    ReplicatedMergeTreeQueue::SelectedEntry selected;

    try
    {
2167
        selected = queue.selectEntryToProcess(merger_mutator, data);
2168 2169 2170
    }
    catch (...)
    {
2171
        tryLogCurrentException(log, __PRETTY_FUNCTION__);
2172 2173 2174 2175 2176 2177 2178 2179 2180 2181 2182 2183 2184 2185 2186 2187 2188 2189 2190 2191 2192 2193 2194 2195 2196 2197 2198
    }

    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());
            }
2199 2200 2201 2202 2203
            else if (e.code() == ErrorCodes::PART_IS_TEMPORARILY_LOCKED)
            {
                /// Part cannot be added temporarily
                LOG_INFO(log, e.displayText());
            }
2204
            else
2205
                tryLogCurrentException(log, __PRETTY_FUNCTION__);
2206 2207 2208 2209 2210 2211 2212 2213 2214

            /** 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 (...)
        {
2215
            tryLogCurrentException(log, __PRETTY_FUNCTION__);
2216 2217 2218 2219 2220 2221 2222 2223 2224
            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 已提交
2225 2226
}

A
Merge  
Alexey Milovidov 已提交
2227

2228
void StorageReplicatedMergeTree::mergeSelectingTask()
2229
{
2230
    if (!is_leader)
2231
        return;
2232

2233
    const bool deduplicate = false; /// TODO: read deduplicate option from table config
2234

2235
    bool success = false;
2236

2237
    try
S
Silviu Caragea 已提交
2238
    {
2239 2240
        /// We must select parts for merge under merge_selecting_mutex because other threads
        /// (OPTIMIZE queries) can assign new merges.
2241
        std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
2242

2243
        auto zookeeper = getZooKeeper();
2244

2245
        ReplicatedMergeTreeMergePredicate merge_pred = queue.getMergePredicate(zookeeper);
2246

2247 2248 2249
        /// 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.
2250
        size_t merges_and_mutations_queued = queue.countMergesAndPartMutations();
2251
        if (merges_and_mutations_queued >= data.settings.max_replicated_merges_in_queue)
2252
        {
2253
            LOG_TRACE(log, "Number of queued merges and part mutations (" << merges_and_mutations_queued
2254
                << ") is greater than max_replicated_merges_in_queue ("
2255
                << data.settings.max_replicated_merges_in_queue << "), so won't select new parts to merge or mutate.");
2256 2257
        }
        else
2258
        {
2259 2260
            size_t max_source_parts_size = merger_mutator.getMaxSourcePartsSize(
                data.settings.max_replicated_merges_in_queue, merges_and_mutations_queued);
2261

2262
            if (max_source_parts_size > 0)
2263
            {
2264 2265
                MergeTreeDataMergerMutator::FuturePart future_merged_part;
                if (merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size, merge_pred))
2266
                {
2267 2268
                    success = createLogEntryToMergeParts(zookeeper, future_merged_part.parts, future_merged_part.name, deduplicate);
                }
2269
                else if (queue.countMutations() > 0)
2270 2271
                {
                    /// Choose a part to mutate.
2272

2273 2274
                    MergeTreeData::DataPartsVector data_parts = data.getDataPartsVector();
                    for (const auto & part : data_parts)
2275
                    {
2276 2277
                        if (part->bytes_on_disk > max_source_parts_size)
                            continue;
2278

2279 2280 2281
                        std::optional<Int64> desired_mutation_version = merge_pred.getDesiredMutationVersion(part);
                        if (!desired_mutation_version)
                            continue;
2282

2283
                        if (createLogEntryToMutatePart(*part, *desired_mutation_version))
2284
                        {
2285 2286
                            success = true;
                            break;
2287 2288
                        }
                    }
2289
                }
2290 2291
            }
        }
2292 2293 2294
    }
    catch (...)
    {
2295
        tryLogCurrentException(log, __PRETTY_FUNCTION__);
2296
    }
2297

2298
    if (!is_leader)
2299
        return;
S
Silviu Caragea 已提交
2300

2301
    if (!success)
2302
        merge_selecting_task->scheduleAfter(MERGE_SELECTING_SLEEP_MS);
2303
    else
2304
        merge_selecting_task->schedule();
2305

M
Merge  
Michael Kolupaev 已提交
2306 2307
}

M
Merge  
Michael Kolupaev 已提交
2308

2309 2310 2311 2312 2313 2314 2315 2316 2317 2318 2319 2320 2321 2322 2323 2324 2325 2326 2327
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);
}


2328
bool StorageReplicatedMergeTree::createLogEntryToMergeParts(
2329 2330 2331 2332 2333
    zkutil::ZooKeeperPtr & zookeeper,
    const MergeTreeData::DataPartsVector & parts,
    const String & merged_name,
    bool deduplicate,
    ReplicatedMergeTreeLogEntryData * out_log_entry)
2334
{
2335
    std::vector<std::future<Coordination::ExistsResponse>> exists_futures;
2336
    exists_futures.reserve(parts.size());
2337
    for (const auto & part : parts)
2338 2339 2340 2341
        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)
2342 2343
    {
        /// If there is no information about part in ZK, we will not merge it.
2344
        if (exists_futures[i].get().error == Coordination::ZNONODE)
2345 2346 2347
        {
            all_in_zk = false;

2348
            const auto & part = parts[i];
2349
            if (part->modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER < time(nullptr))
2350 2351
            {
                LOG_WARNING(log, "Part " << part->name << " (that was selected for merge)"
2352
                    << " with age " << (time(nullptr) - part->modification_time)
2353 2354 2355 2356 2357 2358
                    << " seconds exists locally but not in ZooKeeper."
                    << " Won't do merge with that part and will check it.");
                enqueuePartForCheck(part->name);
            }
        }
    }
2359

2360 2361 2362
    if (!all_in_zk)
        return false;

2363
    ReplicatedMergeTreeLogEntryData entry;
2364 2365 2366
    entry.type = LogEntry::MERGE_PARTS;
    entry.source_replica = replica_name;
    entry.new_part_name = merged_name;
Y
Yuri Dyachenko 已提交
2367
    entry.deduplicate = deduplicate;
2368
    entry.create_time = time(nullptr);
2369 2370

    for (const auto & part : parts)
2371
        entry.source_parts.push_back(part->name);
2372 2373 2374 2375 2376 2377 2378 2379

    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;
2380 2381 2382
}


2383 2384 2385 2386 2387 2388 2389 2390 2391 2392 2393 2394 2395 2396 2397 2398 2399 2400 2401 2402 2403 2404
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;

2405
    String new_part_name = part.getNewName(new_part_info);
2406 2407 2408 2409 2410 2411 2412 2413 2414 2415 2416 2417 2418

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


2419
void StorageReplicatedMergeTree::removePartFromZooKeeper(const String & part_name, Coordination::Requests & ops)
2420
{
2421
    String part_path = replica_path + "/parts/" + part_name;
2422

2423 2424 2425
    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));
2426 2427 2428
}


M
Merge  
Michael Kolupaev 已提交
2429 2430
void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_name)
{
2431
    auto zookeeper = getZooKeeper();
A
Merge  
Alexey Milovidov 已提交
2432

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

2435 2436 2437 2438 2439
    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 已提交
2440

2441
    Coordination::Requests ops;
2442 2443
    ops.emplace_back(zkutil::makeCreateRequest(
        replica_path + "/queue/queue-", log_entry->toString(),
2444
        zkutil::CreateMode::PersistentSequential));
2445

2446
    removePartFromZooKeeper(part_name, ops);
2447

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

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

A
Merge  
Alexey Milovidov 已提交
2455

2456
void StorageReplicatedMergeTree::enterLeaderElection()
M
Merge  
Michael Kolupaev 已提交
2457
{
2458 2459 2460 2461 2462 2463
    auto callback = [this]()
    {
        CurrentMetrics::add(CurrentMetrics::LeaderReplica);
        LOG_INFO(log, "Became leader");

        is_leader = true;
2464
        merge_selecting_task->activateAndSchedule();
2465
    };
2466

2467 2468 2469
    try
    {
        leader_election = std::make_shared<zkutil::LeaderElection>(
2470
            context.getSchedulePool(),
2471 2472 2473 2474 2475 2476 2477 2478 2479 2480 2481 2482 2483 2484 2485 2486
            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)
2487
        return;
2488

2489 2490 2491 2492 2493
    /// 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)
2494
    {
2495 2496 2497 2498
        CurrentMetrics::sub(CurrentMetrics::LeaderReplica);
        LOG_INFO(log, "Stopped being leader");

        is_leader = false;
2499
        merge_selecting_task->deactivate();
2500 2501
    }

2502 2503 2504
    /// 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 已提交
2505 2506
}

A
Merge  
Alexey Milovidov 已提交
2507

M
Merge  
Michael Kolupaev 已提交
2508
String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_name, bool active)
M
Merge  
Michael Kolupaev 已提交
2509
{
2510 2511
    auto zookeeper = getZooKeeper();
    Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
M
Merge  
Michael Kolupaev 已提交
2512

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

2516 2517 2518 2519 2520
    for (const String & replica : replicas)
    {
        /// We don't interested in ourself.
        if (replica == replica_name)
            continue;
A
Alexey Milovidov 已提交
2521

2522 2523 2524
        if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name) &&
            (!active || zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")))
            return replica;
2525

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

2529
    return {};
2530 2531 2532
}


2533
String StorageReplicatedMergeTree::findReplicaHavingCoveringPart(LogEntry & entry, bool active)
2534
{
2535 2536 2537 2538
    auto zookeeper = getZooKeeper();
    Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");

    /// Select replicas in uniformly random order.
2539
    std::shuffle(replicas.begin(), replicas.end(), rng);
2540 2541 2542 2543 2544 2545 2546 2547 2548 2549 2550 2551 2552

    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)
        {
2553 2554
            if (part_on_replica == entry.new_part_name
                || MergeTreePartInfo::contains(part_on_replica, entry.new_part_name, data.format_version))
2555
            {
2556 2557
                if (largest_part_found.empty()
                    || MergeTreePartInfo::contains(part_on_replica, largest_part_found, data.format_version))
2558 2559 2560 2561 2562 2563 2564 2565
                {
                    largest_part_found = part_on_replica;
                }
            }
        }

        if (!largest_part_found.empty())
        {
2566 2567 2568 2569 2570 2571 2572 2573 2574 2575 2576 2577 2578 2579 2580 2581 2582
            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;
            }

2583 2584 2585 2586 2587
            return replica;
        }
    }

    return {};
M
Merge  
Michael Kolupaev 已提交
2588 2589
}

A
Merge  
Alexey Milovidov 已提交
2590

2591 2592 2593 2594 2595 2596 2597 2598 2599 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
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 已提交
2633
/** If a quorum is tracked for a part, update information about it in ZK.
2634
  */
2635
void StorageReplicatedMergeTree::updateQuorum(const String & part_name)
2636
{
2637 2638 2639 2640 2641 2642 2643 2644
    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;
2645
    Coordination::Stat stat;
2646 2647 2648 2649 2650 2651 2652 2653 2654 2655 2656 2657 2658 2659 2660 2661 2662 2663 2664

    /// 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.

2665 2666
            Coordination::Requests ops;
            Coordination::Responses responses;
2667 2668
            ops.emplace_back(zkutil::makeRemoveRequest(quorum_status_path, stat.version));
            ops.emplace_back(zkutil::makeSetRequest(quorum_last_part_path, part_name, -1));
2669
            auto code = zookeeper->tryMulti(ops, responses);
2670

2671
            if (code == Coordination::ZOK)
2672 2673 2674
            {
                break;
            }
2675
            else if (code == Coordination::ZNONODE)
2676 2677 2678 2679
            {
                /// The quorum has already been achieved.
                break;
            }
2680
            else if (code == Coordination::ZBADVERSION)
2681 2682 2683 2684 2685
            {
                /// Node was updated meanwhile. We must re-read it and repeat all the actions.
                continue;
            }
            else
2686
                throw Coordination::Exception(code, quorum_status_path);
2687 2688 2689 2690 2691 2692
        }
        else
        {
            /// We update the node, registering there one more replica.
            auto code = zookeeper->trySet(quorum_status_path, quorum_entry.toString(), stat.version);

2693
            if (code == Coordination::ZOK)
2694 2695 2696
            {
                break;
            }
2697
            else if (code == Coordination::ZNONODE)
2698 2699 2700 2701
            {
                /// The quorum has already been achieved.
                break;
            }
2702
            else if (code == Coordination::ZBADVERSION)
2703 2704 2705 2706 2707
            {
                /// Node was updated meanwhile. We must re-read it and repeat all the actions.
                continue;
            }
            else
2708
                throw Coordination::Exception(code, quorum_status_path);
2709 2710
        }
    }
2711 2712 2713
}


A
Alexey Milovidov 已提交
2714
bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const String & replica_path, bool to_detached, size_t quorum)
M
Merge  
Michael Kolupaev 已提交
2715
{
2716 2717 2718
    const auto part_info = MergeTreePartInfo::fromPartName(part_name, data.format_version);

    if (auto part = data.getPartIfExists(part_info, {MergeTreeDataPart::State::Outdated, MergeTreeDataPart::State::Deleting}))
2719 2720
    {
        LOG_DEBUG(log, "Part " << part->getNameWithState() << " should be deleted after previous attempt before fetch");
2721
        /// Force immediate parts cleanup to delete the part that was left from the previous fetch attempt.
2722
        cleanup_thread.wakeup();
2723 2724 2725
        return false;
    }

2726 2727 2728 2729 2730 2731 2732 2733 2734 2735 2736 2737 2738 2739 2740 2741 2742 2743 2744
    {
        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)
2745
        table_lock = lockStructure(true, __PRETTY_FUNCTION__);
2746

2747
    /// Logging
2748
    Stopwatch stopwatch;
2749 2750
    MergeTreeData::MutableDataPartPtr part;
    MergeTreeData::DataPartsVector replaced_parts;
2751

2752
    auto write_part_log = [&] (const ExecutionStatus & execution_status)
2753
    {
2754 2755 2756
        writePartLog(
            PartLogElement::DOWNLOAD_PART, execution_status, stopwatch.elapsed(),
            part_name, part, replaced_parts, nullptr);
2757 2758
    };

2759 2760 2761 2762 2763 2764 2765 2766 2767
    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);
2768

2769 2770 2771 2772 2773 2774 2775 2776 2777 2778 2779 2780 2781 2782 2783 2784 2785 2786
        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)
2787
    {
2788 2789 2790 2791 2792 2793 2794 2795 2796 2797 2798 2799 2800 2801 2802 2803 2804 2805
        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());
        auto [user, password] = context.getInterserverCredentials();
        String interserver_scheme = context.getInterserverScheme();

        get_part = [&, address, timeouts, user, password, interserver_scheme]()
        {
            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);
2806

2807 2808 2809 2810 2811 2812 2813 2814 2815 2816
            return fetcher.fetchPart(
                part_name, replica_path,
                address.host, address.replication_port,
                timeouts, user, password, interserver_scheme, to_detached);
        };
    }

    try
    {
        part = get_part();
2817

2818
        if (!to_detached)
2819
        {
2820 2821
            MergeTreeData::Transaction transaction;
            data.renameTempPartAndReplace(part, nullptr, &transaction);
2822

2823 2824 2825 2826 2827
            /** 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.
              */
2828
            replaced_parts = checkPartChecksumsAndCommit(transaction, part);
2829

2830 2831 2832 2833 2834
            /** 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);
2835

2836
            merge_selecting_task->schedule();
2837 2838 2839 2840 2841 2842

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

2844 2845 2846
            write_part_log({});
        }
        else
2847
        {
2848
            part->renameTo("detached/" + part_name);
2849 2850
        }
    }
2851
    catch (...)
2852
    {
2853 2854 2855 2856
        if (!to_detached)
            write_part_log(ExecutionStatus::fromCurrentException());

        throw;
2857 2858 2859 2860
    }

    ProfileEvents::increment(ProfileEvents::ReplicatedPartFetches);

2861 2862 2863 2864 2865
    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)" : ""));

2866
    return true;
M
Merge  
Michael Kolupaev 已提交
2867
}
M
Merge  
Michael Kolupaev 已提交
2868

A
Merge  
Alexey Milovidov 已提交
2869

2870 2871 2872 2873 2874 2875 2876 2877
void StorageReplicatedMergeTree::startup()
{
    if (is_readonly)
        return;

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

A
Alexey Zatelepin 已提交
2880 2881 2882 2883 2884
    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());

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

2887
    /// In this thread replica will be activated.
2888
    restarting_thread.start();
2889 2890 2891

    /// Wait while restarting_thread initializes LeaderElection (and so on) or makes first attmept to do it
    startup_event.wait();
2892 2893 2894
}


M
Merge  
Michael Kolupaev 已提交
2895 2896
void StorageReplicatedMergeTree::shutdown()
{
2897 2898 2899 2900 2901 2902 2903 2904 2905
    /// 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();
2906

2907
    if (data_parts_exchange_endpoint_holder)
2908
    {
2909
        data_parts_exchange_endpoint_holder->getBlocker().cancelForever();
2910
        data_parts_exchange_endpoint_holder = nullptr;
2911
    }
M
Merge  
Michael Kolupaev 已提交
2912 2913 2914
}


M
Merge  
Michael Kolupaev 已提交
2915 2916
StorageReplicatedMergeTree::~StorageReplicatedMergeTree()
{
2917 2918 2919 2920 2921 2922 2923 2924
    try
    {
        shutdown();
    }
    catch(...)
    {
        tryLogCurrentException(__PRETTY_FUNCTION__);
    }
M
Merge  
Michael Kolupaev 已提交
2925 2926
}

A
Merge  
Alexey Milovidov 已提交
2927

M
Merge  
Michael Kolupaev 已提交
2928
BlockInputStreams StorageReplicatedMergeTree::read(
2929
    const Names & column_names,
2930
    const SelectQueryInfo & query_info,
2931
    const Context & context,
2932
    QueryProcessingStage::Enum processed_stage,
2933
    const size_t max_block_size,
2934
    const unsigned num_streams)
M
Merge  
Michael Kolupaev 已提交
2935
{
2936
    checkQueryProcessingStage(processed_stage, context);
2937
    const Settings & settings = context.getSettingsRef();
2938

2939 2940 2941 2942 2943 2944 2945
    /** 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)
2946
    {
2947
        auto zookeeper = getZooKeeper();
2948

2949 2950
        String last_part;
        zookeeper->tryGet(zookeeper_path + "/quorum/last_part", last_part);
2951

2952
        if (!last_part.empty() && !data.getActiveContainingPart(last_part))    /// TODO Disable replica for distributed queries.
2953 2954
            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);
2955

2956 2957 2958 2959
        if (last_part.empty())  /// If no part has been written with quorum.
        {
            String quorum_str;
            if (zookeeper->tryGet(zookeeper_path + "/quorum/status", quorum_str))
2960
            {
2961 2962
                ReplicatedMergeTreeQuorumEntry quorum_entry;
                quorum_entry.fromString(quorum_str);
2963
                auto part_info = MergeTreePartInfo::fromPartName(quorum_entry.part_name, data.format_version);
2964
                max_block_number_to_read = part_info.min_block - 1;
2965 2966
            }
        }
2967
        else
2968
        {
2969
            auto part_info = MergeTreePartInfo::fromPartName(last_part, data.format_version);
2970
            max_block_number_to_read = part_info.max_block;
2971 2972 2973
        }
    }

2974
    return reader.read(column_names, query_info, context, max_block_size, num_streams, max_block_number_to_read);
M
Merge  
Michael Kolupaev 已提交
2975 2976
}

A
Merge  
Alexey Milovidov 已提交
2977

A
Merge  
Alexey Milovidov 已提交
2978
void StorageReplicatedMergeTree::assertNotReadonly() const
M
Merge  
Michael Kolupaev 已提交
2979
{
2980 2981
    if (is_readonly)
        throw Exception("Table is in readonly mode", ErrorCodes::TABLE_IS_READ_ONLY);
A
Merge  
Alexey Milovidov 已提交
2982 2983 2984
}


A
Alexey Milovidov 已提交
2985
BlockOutputStreamPtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const Settings & settings)
A
Merge  
Alexey Milovidov 已提交
2986
{
2987
    assertNotReadonly();
M
Merge  
Michael Kolupaev 已提交
2988

2989 2990
    bool deduplicate = data.settings.replicated_deduplication_window != 0 && settings.insert_deduplicate;

2991
    return std::make_shared<ReplicatedMergeTreeBlockOutputStream>(*this,
2992
        settings.insert_quorum, settings.insert_quorum_timeout.totalMilliseconds(), deduplicate);
M
Merge  
Michael Kolupaev 已提交
2993
}
M
Merge  
Michael Kolupaev 已提交
2994

A
Merge  
Alexey Milovidov 已提交
2995

2996
bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context)
M
Merge  
Michael Kolupaev 已提交
2997
{
2998
    assertNotReadonly();
2999

3000
    if (!is_leader)
3001
    {
3002
        sendRequestToLeaderReplica(query, context.getSettingsRef());
3003 3004
        return true;
    }
3005

3006 3007
    ReplicatedMergeTreeLogEntryData merge_entry;
    {
3008 3009
        /// We must select parts for merge under merge_selecting_mutex because other threads
        /// (merge_selecting_thread or OPTIMIZE queries) could assign new merges.
3010
        std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
3011

3012
        size_t disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
3013 3014 3015
        auto zookeeper = getZooKeeper();
        ReplicatedMergeTreeMergePredicate can_merge = queue.getMergePredicate(zookeeper);

3016
        auto handle_noop = [&] (const String & message)
3017
        {
3018 3019
            if (context.getSettingsRef().optimize_throw_if_noop)
                throw Exception(message, ErrorCodes::CANNOT_ASSIGN_OPTIMIZE);
3020
            return false;
3021 3022
        };

3023
        if (!partition && final)
3024
        {
3025 3026 3027 3028 3029 3030 3031 3032 3033 3034 3035 3036 3037 3038 3039
            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");
            }
3040
        }
3041 3042 3043 3044 3045 3046 3047 3048 3049 3050 3051 3052 3053 3054 3055 3056 3057 3058 3059 3060 3061 3062
        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);
            }
3063

3064 3065 3066
            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");
        }
3067
    }
3068

3069 3070 3071 3072
    /// TODO: Bad setting name for such purpose
    if (context.getSettingsRef().replication_alter_partitions_sync != 0)
        waitForAllReplicasToProcessLogEntry(merge_entry);

3073
    return true;
M
Merge  
Michael Kolupaev 已提交
3074 3075
}

A
Merge  
Alexey Milovidov 已提交
3076

M
Merge  
Michael Kolupaev 已提交
3077
void StorageReplicatedMergeTree::alter(const AlterCommands & params,
A
Alexey Milovidov 已提交
3078
    const String & /*database_name*/, const String & /*table_name*/, const Context & context)
M
Merge  
Michael Kolupaev 已提交
3079
{
3080
    assertNotReadonly();
A
Merge  
Alexey Milovidov 已提交
3081

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

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

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

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

3095
        data.checkAlter(params);
M
Merge  
Michael Kolupaev 已提交
3096

3097 3098 3099 3100
        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);

3101
        ColumnsDescription new_columns = data.getColumns();
3102
        params.apply(new_columns);
3103

3104
        new_columns_str = new_columns.toString();
3105 3106 3107 3108 3109 3110 3111 3112 3113 3114 3115 3116 3117 3118 3119 3120 3121 3122 3123 3124 3125 3126 3127 3128 3129 3130 3131 3132 3133 3134 3135 3136 3137

        /// 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");

3138
        while (!partial_shutdown_called)
3139 3140 3141 3142 3143 3144
        {
            /// 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.");
3145

3146 3147 3148
                inactive_replicas.emplace(replica);
                break;
            }
3149

3150
            String replica_columns_str;
M
Merge  
Michael Kolupaev 已提交
3151

3152 3153 3154 3155 3156 3157
            /// 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 已提交
3158

3159
            int replica_columns_version = stat.version;
M
Merge  
Michael Kolupaev 已提交
3160

3161 3162 3163
            /// The ALTER has been successfully applied.
            if (replica_columns_str == new_columns_str)
                break;
M
Merge  
Michael Kolupaev 已提交
3164

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

3168 3169 3170 3171 3172 3173
            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 已提交
3174

3175 3176 3177 3178 3179 3180 3181 3182 3183 3184 3185 3186 3187 3188 3189 3190 3191 3192 3193 3194 3195 3196 3197 3198 3199 3200 3201 3202
            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;
            }
        }

3203
        if (partial_shutdown_called)
3204 3205 3206 3207 3208 3209 3210 3211 3212 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
            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 已提交
3240 3241
}

M
Merge  
Michael Kolupaev 已提交
3242

3243 3244
/// 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 已提交
3245
{
3246
    if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
3247 3248 3249
    {
        /// The date range is all month long.
        const auto & lut = DateLUT::instance();
3250
        time_t start_time = lut.YYYYMMDDToDate(parse<UInt32>(part_info.partition_id + "01"));
3251 3252
        DayNum left_date = lut.toDayNum(start_time);
        DayNum right_date = DayNum(static_cast<size_t>(left_date) + lut.daysInMonth(start_time) - 1);
3253 3254
        return part_info.getPartNameV0(left_date, right_date);
    }
M
Merge  
Michael Kolupaev 已提交
3255

3256 3257
    return part_info.getPartName();
}
A
Merge  
Alexey Milovidov 已提交
3258

3259
bool StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(const String & partition_id, MergeTreePartInfo & part_info)
A
Merge  
Andrey Mironov 已提交
3260
{
3261 3262
    /// 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.
3263
    Int64 left = 0;
3264

3265
    /** Let's skip one number in `block_numbers` for the partition being deleted, and we will only delete parts until this number.
3266 3267 3268 3269 3270 3271
      * 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;
3272
    Int64 mutation_version;
3273 3274

    {
3275
        auto zookeeper = getZooKeeper();
3276
        auto block_number_lock = allocateBlockNumber(partition_id, zookeeper);
3277
        right = block_number_lock->getNumber();
3278
        block_number_lock->unlock();
3279
        mutation_version = queue.getCurrentMutationVersion(partition_id, right);
3280 3281
    }

3282
    /// Empty partition.
3283
    if (right == 0)
3284
        return false;
3285

3286
    --right;
3287

3288
    /// Artificial high level is choosen, to make this part "covering" all parts inside.
3289
    part_info = MergeTreePartInfo(partition_id, left, right, MergeTreePartInfo::MAX_LEVEL, mutation_version);
3290
    return true;
3291 3292 3293
}


3294
void StorageReplicatedMergeTree::clearColumnInPartition(
3295
    const ASTPtr & partition, const Field & column_name, const Context & context)
3296 3297 3298 3299 3300
{
    assertNotReadonly();

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

3301
    String partition_id = data.getPartitionIDFromQuery(partition, context);
3302
    MergeTreePartInfo drop_range_info;
3303

3304
    if (!getFakePartCoveringAllPartsInPartition(partition_id, drop_range_info))
3305
    {
3306
        LOG_INFO(log, "Will not clear partition " << partition_id << ", it is empty.");
3307 3308 3309
        return;
    }

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

3312
    LogEntry entry;
3313
    entry.type = LogEntry::CLEAR_COLUMN;
3314
    entry.new_part_name = getPartNamePossiblyFake(data.format_version, drop_range_info);
3315
    entry.column_name = column_name.safeGet<String>();
3316
    entry.create_time = time(nullptr);
3317 3318 3319 3320 3321

    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.
3322
    if (context.getSettingsRef().replication_alter_partitions_sync != 0)
3323
    {
3324
        if (context.getSettingsRef().replication_alter_partitions_sync == 1)
3325 3326 3327 3328 3329 3330
            waitForReplicaToProcessLogEntry(replica_name, entry);
        else
            waitForAllReplicasToProcessLogEntry(entry);
    }
}

3331

3332
void StorageReplicatedMergeTree::dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & context)
3333 3334 3335
{
    assertNotReadonly();

3336 3337
    zkutil::ZooKeeperPtr zookeeper = getZooKeeper();

3338
    if (!is_leader)
3339
    {
3340
        sendRequestToLeaderReplica(query, context.getSettingsRef());
3341 3342 3343
        return;
    }

3344
    String partition_id = data.getPartitionIDFromQuery(partition, context);
3345

Z
zhang2014 已提交
3346
    LogEntry entry;
3347
    if (dropPartsInPartition(*zookeeper, partition_id, entry, detach))
3348
    {
Z
zhang2014 已提交
3349 3350 3351 3352 3353 3354 3355 3356
        /// 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);
        }
3357
    }
Z
zhang2014 已提交
3358
}
3359

3360

Z
zhang2014 已提交
3361 3362 3363
void StorageReplicatedMergeTree::truncate(const ASTPtr & query)
{
    assertNotReadonly();
3364

Z
zhang2014 已提交
3365
    zkutil::ZooKeeperPtr zookeeper = getZooKeeper();
3366

Z
zhang2014 已提交
3367
    if (!is_leader)
3368
    {
Z
zhang2014 已提交
3369 3370
        sendRequestToLeaderReplica(query, context.getSettingsRef());
        return;
3371
    }
3372

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

Z
zhang2014 已提交
3375
    for (String & partition_id : partitions)
3376
    {
Z
zhang2014 已提交
3377 3378
        LogEntry entry;

3379
        if (dropPartsInPartition(*zookeeper, partition_id, entry, false))
3380 3381
            waitForAllReplicasToProcessLogEntry(entry);
    }
A
Merge  
Alexey Milovidov 已提交
3382
}
A
Merge  
Alexey Milovidov 已提交
3383

A
Merge  
Alexey Arno 已提交
3384

3385
void StorageReplicatedMergeTree::attachPartition(const ASTPtr & partition, bool attach_part, const Context & context)
M
Merge  
Michael Kolupaev 已提交
3386
{
3387 3388
    assertNotReadonly();

3389
    String partition_id;
3390 3391

    if (attach_part)
3392
        partition_id = typeid_cast<const ASTLiteral &>(*partition).value.safeGet<String>();
3393
    else
3394
        partition_id = data.getPartitionIDFromQuery(partition, context);
3395

3396
    String source_dir = "detached/";
3397 3398 3399 3400 3401

    /// Let's compose a list of parts that should be added.
    Strings parts;
    if (attach_part)
    {
3402
        parts.push_back(partition_id);
3403 3404 3405
    }
    else
    {
3406
        LOG_DEBUG(log, "Looking for parts for partition " << partition_id << " in " << source_dir);
3407
        ActiveDataPartSet active_parts(data.format_version);
3408 3409 3410 3411 3412

        std::set<String> part_names;
        for (Poco::DirectoryIterator it = Poco::DirectoryIterator(full_path + source_dir); it != Poco::DirectoryIterator(); ++it)
        {
            String name = it.name();
3413
            MergeTreePartInfo part_info;
3414
            if (!MergeTreePartInfo::tryParsePartName(name, &part_info, data.format_version))
3415
                continue;
3416
            if (part_info.partition_id != partition_id)
3417 3418 3419 3420 3421 3422 3423 3424 3425 3426 3427 3428 3429 3430 3431 3432 3433
                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);
        }
    }

3434
    /// Synchronously check that added parts exist and are not broken. We will write checksums.txt if it does not exist.
3435
    LOG_DEBUG(log, "Checking parts");
3436
    std::vector<MergeTreeData::MutableDataPartPtr> loaded_parts;
3437 3438 3439
    for (const String & part : parts)
    {
        LOG_DEBUG(log, "Checking part " << part);
3440
        loaded_parts.push_back(data.loadPartAndFixMetadata(source_dir + part));
3441 3442
    }

3443
    ReplicatedMergeTreeBlockOutputStream output(*this, 0, 0, false);   /// TODO Allow to use quorum here.
3444
    for (auto & part : loaded_parts)
3445
    {
3446 3447 3448
        String old_name = part->name;
        output.writeExistingPart(part);
        LOG_DEBUG(log, "Attached part " << old_name << " as " << part->name);
3449
    }
M
Merge  
Michael Kolupaev 已提交
3450 3451
}

3452

3453
void StorageReplicatedMergeTree::checkTableCanBeDropped() const
3454
{
3455
    /// Consider only synchronized data
3456
    const_cast<MergeTreeData &>(getData()).recalculateColumnSizes();
3457
    context.checkTableCanBeDropped(database_name, table_name, getData().getTotalActiveSizeInBytes());
3458
}
A
Merge  
Alexey Milovidov 已提交
3459

3460

3461
void StorageReplicatedMergeTree::checkPartitionCanBeDropped(const ASTPtr & partition)
3462 3463
{
    const_cast<MergeTreeData &>(getData()).recalculateColumnSizes();
3464

3465 3466
    const String partition_id = data.getPartitionIDFromQuery(partition, context);
    auto parts_to_remove = data.getDataPartsVectorInPartition(MergeTreeDataPartState::Committed, partition_id);
3467 3468

    UInt64 partition_size = 0;
3469

3470 3471
    for (const auto & part : parts_to_remove)
    {
3472
        partition_size += part->bytes_on_disk;
3473
    }
3474
    context.checkPartitionCanBeDropped(database_name, table_name, partition_size);
3475
}
A
Merge  
Alexey Milovidov 已提交
3476

3477

M
Merge  
Michael Kolupaev 已提交
3478 3479
void StorageReplicatedMergeTree::drop()
{
3480 3481
    {
        auto zookeeper = tryGetZooKeeper();
A
Merge  
Alexey Milovidov 已提交
3482

3483 3484
        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 已提交
3485

3486
        shutdown();
M
Merge  
Michael Kolupaev 已提交
3487

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

3491 3492 3493
        LOG_INFO(log, "Removing replica " << replica_path);
        replica_is_active_node = nullptr;
        zookeeper->tryRemoveRecursive(replica_path);
M
Merge  
Michael Kolupaev 已提交
3494

3495 3496
        /// Check that `zookeeper_path` exists: it could have been deleted by another replica after execution of previous line.
        Strings replicas;
3497
        if (zookeeper->tryGetChildren(zookeeper_path + "/replicas", replicas) == Coordination::ZOK && replicas.empty())
3498 3499 3500 3501 3502
        {
            LOG_INFO(log, "Removing table " << zookeeper_path << " (this might take several minutes)");
            zookeeper->tryRemoveRecursive(zookeeper_path);
        }
    }
M
Merge  
Michael Kolupaev 已提交
3503

3504
    data.dropAllData();
M
Merge  
Michael Kolupaev 已提交
3505 3506
}

A
Merge  
Alexey Milovidov 已提交
3507

M
Merge  
Michael Kolupaev 已提交
3508 3509
void StorageReplicatedMergeTree::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name)
{
3510
    std::string new_full_path = new_path_to_db + escapeForFileName(new_table_name) + '/';
M
Merge  
Michael Kolupaev 已提交
3511

3512
    data.setPath(new_full_path);
M
Merge  
Michael Kolupaev 已提交
3513

3514 3515 3516
    database_name = new_database_name;
    table_name = new_table_name;
    full_path = new_full_path;
M
Merge  
Michael Kolupaev 已提交
3517

3518 3519 3520 3521
    /// Update table name in zookeeper
    auto zookeeper = getZooKeeper();
    zookeeper->set(replica_path + "/host", getReplicatedMergeTreeAddress().toString());

3522
    /// TODO: You can update names of loggers.
M
Merge  
Michael Kolupaev 已提交
3523 3524
}

A
Merge  
Alexey Milovidov 已提交
3525

3526 3527
bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path)
{
3528 3529 3530 3531 3532
    {
        std::lock_guard<std::mutex> lock(existing_nodes_cache_mutex);
        if (existing_nodes_cache.count(path))
            return true;
    }
3533

3534
    bool res = getZooKeeper()->exists(path);
3535

3536 3537 3538 3539 3540
    if (res)
    {
        std::lock_guard<std::mutex> lock(existing_nodes_cache_mutex);
        existing_nodes_cache.insert(path);
    }
3541

3542
    return res;
3543 3544 3545
}


3546
std::optional<EphemeralLockInZooKeeper>
3547 3548
StorageReplicatedMergeTree::allocateBlockNumber(
    const String & partition_id, zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_block_id_path)
M
Merge  
Michael Kolupaev 已提交
3549
{
3550
    /// Lets check for duplicates in advance, to avoid superflous block numbers allocation
3551
    Coordination::Requests deduplication_check_ops;
3552
    if (!zookeeper_block_id_path.empty())
3553
    {
3554 3555
        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));
3556
    }
3557

3558 3559
    String block_numbers_path = zookeeper_path + "/block_numbers";
    String partition_path = block_numbers_path + "/" + partition_id;
3560

3561
    if (!existsNodeCached(partition_path))
3562
    {
3563
        Coordination::Requests ops;
3564 3565 3566 3567 3568 3569
        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));

3570
        Coordination::Responses responses;
3571
        int code = zookeeper->tryMulti(ops, responses);
3572
        if (code && code != Coordination::ZNODEEXISTS)
3573
            zkutil::KeeperMultiException::check(code, ops, responses);
3574
    }
3575

3576
    EphemeralLockInZooKeeper lock;
3577 3578 3579
    /// 2 RTT
    try
    {
3580
        lock = EphemeralLockInZooKeeper(
3581
            partition_path + "/block-", zookeeper_path + "/temp", *zookeeper, &deduplication_check_ops);
3582 3583 3584
    }
    catch (const zkutil::KeeperMultiException & e)
    {
3585
        if (e.code == Coordination::ZNODEEXISTS && e.getPathForFirstFailedOp() == zookeeper_block_id_path)
3586 3587 3588 3589
            return {};

        throw Exception("Cannot allocate block number in ZooKeeper: " + e.displayText(), ErrorCodes::KEEPER_EXCEPTION);
    }
3590
    catch (const Coordination::Exception & e)
3591 3592 3593 3594 3595
    {
        throw Exception("Cannot allocate block number in ZooKeeper: " + e.displayText(), ErrorCodes::KEEPER_EXCEPTION);
    }

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

A
Merge  
Alexey Milovidov 已提交
3598

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

3603 3604 3605
    Strings replicas = getZooKeeper()->getChildren(zookeeper_path + "/replicas");
    for (const String & replica : replicas)
        waitForReplicaToProcessLogEntry(replica, entry);
A
Merge  
Alexey Milovidov 已提交
3606

3607
    LOG_DEBUG(log, "Finished waiting for all replicas to process " << entry.znode_name);
A
Merge  
Alexey Milovidov 已提交
3608 3609 3610
}


3611
void StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry(const String & replica, const ReplicatedMergeTreeLogEntryData & entry)
A
Merge  
Alexey Milovidov 已提交
3612
{
3613 3614 3615 3616 3617 3618 3619 3620 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
    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 已提交
3738 3739 3740
}


3741
void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields)
3742
{
3743 3744
    auto zookeeper = tryGetZooKeeper();

3745
    res.is_leader = is_leader;
3746 3747 3748 3749
    res.is_readonly = is_readonly;
    res.is_session_expired = !zookeeper || zookeeper->expired();

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

3752 3753 3754 3755 3756 3757 3758 3759 3760 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
    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;
    }
3791 3792
}

3793

3794 3795 3796 3797 3798 3799 3800 3801 3802 3803 3804 3805 3806 3807 3808
/// 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"));

3809
    /// TODO: add setters and getters interface for database and table fields of AST
3810
    auto new_query = query->clone();
3811 3812 3813 3814 3815 3816 3817 3818 3819 3820 3821 3822
    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;
    }
    else
        throw Exception("Can't proxy this query. Unsupported query type", ErrorCodes::NOT_IMPLEMENTED);
3823

P
proller 已提交
3824
    /// Query send with current user credentials
3825

3826
    auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithoutFailover(context.getSettingsRef());
3827 3828 3829 3830
    Connection connection(
        leader_address.host,
        leader_address.queries_port,
        leader_address.database,
3831
        context.getClientInfo().current_user, context.getClientInfo().current_password, timeouts, "ClickHouse replica");
3832

3833
    RemoteBlockInputStream stream(connection, formattedAST(new_query), {}, context, &settings);
3834
    NullBlockOutputStream output({});
3835 3836 3837 3838 3839 3840

    copyData(stream, output);
    return;
}


3841 3842
void StorageReplicatedMergeTree::getQueue(LogEntriesData & res, String & replica_name_)
{
3843 3844
    replica_name_ = replica_name;
    queue.getEntries(res);
3845 3846
}

3847 3848 3849 3850 3851 3852
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);

3853 3854 3855 3856
    /// 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();
3857 3858 3859

    time_t current_time = time(nullptr);

3860
    if (!queue_update_finish_time)
3861
    {
3862
        /// We have not updated queue even once yet (perhaps replica is readonly).
3863 3864 3865 3866 3867 3868 3869 3870
        /// 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;
    }
3871
    else if (queue_update_start_time > queue_update_finish_time)
3872 3873 3874 3875
    {
        /// 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.
3876
        return (current_time > queue_update_start_time) ? (current_time - queue_update_start_time) : 0;
3877 3878 3879 3880 3881 3882 3883
    }
    else
    {
        /// Everything is up-to-date.
        return 0;
    }
}
3884

3885
void StorageReplicatedMergeTree::getReplicaDelays(time_t & out_absolute_delay, time_t & out_relative_delay)
3886
{
3887
    assertNotReadonly();
3888

3889
    time_t current_time = time(nullptr);
3890

3891
    out_absolute_delay = getAbsoluteDelay();
3892
    out_relative_delay = 0;
3893

3894 3895 3896 3897
    /** 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.
      */
3898

3899 3900
    if (out_absolute_delay < static_cast<time_t>(data.settings.min_relative_delay_to_yield_leadership))
        return;
3901

3902
    auto zookeeper = getZooKeeper();
3903

3904 3905
    time_t max_replicas_unprocessed_insert_time = 0;
    bool have_replica_with_nothing_unprocessed = false;
3906

3907
    Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
3908

3909 3910 3911 3912
    for (const auto & replica : replicas)
    {
        if (replica == replica_name)
            continue;
3913

3914 3915 3916
        /// Skip dead replicas.
        if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))
            continue;
3917

3918 3919 3920
        String value;
        if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/min_unprocessed_insert_time", value))
            continue;
3921

3922
        time_t replica_time = value.empty() ? 0 : parse<time_t>(value);
3923

3924 3925 3926 3927 3928 3929
        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.
3930
              * If the replica for some reason has stalled `queueUpdatingTask`,
3931 3932
              *  then `min_unprocessed_insert_time` will be incorrect.
              */
3933

3934 3935 3936
            have_replica_with_nothing_unprocessed = true;
            break;
        }
3937

3938 3939 3940
        if (replica_time > max_replicas_unprocessed_insert_time)
            max_replicas_unprocessed_insert_time = replica_time;
    }
3941

3942 3943
    if (have_replica_with_nothing_unprocessed)
        out_relative_delay = out_absolute_delay;
3944 3945 3946 3947 3948 3949 3950
    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;
    }
3951 3952 3953
}


3954
void StorageReplicatedMergeTree::fetchPartition(const ASTPtr & partition, const String & from_, const Context & context)
3955
{
3956
    String partition_id = data.getPartitionIDFromQuery(partition, context);
3957 3958 3959 3960 3961

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

3962
    LOG_INFO(log, "Will fetch partition " << partition_id << " from shard " << from_);
3963 3964 3965 3966 3967 3968

    /** 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)
3969 3970
    {
        MergeTreePartInfo part_info;
3971 3972
        if (MergeTreePartInfo::tryParsePartName(dir_it.name(), &part_info, data.format_version)
              && part_info.partition_id == partition_id)
3973 3974
            throw Exception("Detached partition " + partition_id + " already exists.", ErrorCodes::PARTITION_ALREADY_EXISTS);
    }
3975 3976 3977 3978 3979 3980 3981 3982 3983 3984 3985 3986 3987 3988 3989 3990 3991 3992 3993 3994 3995 3996 3997 3998 3999 4000 4001 4002 4003 4004 4005 4006 4007 4008 4009 4010 4011

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

4012
            Coordination::Stat stat;
4013 4014 4015 4016 4017 4018 4019 4020 4021 4022 4023 4024 4025 4026 4027 4028 4029 4030 4031 4032 4033 4034 4035 4036 4037 4038 4039 4040 4041 4042 4043 4044 4045 4046 4047
            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 已提交
4048
        if (try_no >= context.getSettings().max_fetch_partition_retries_count)
A
Alexey Milovidov 已提交
4049
            throw Exception("Too many retries to fetch parts from " + best_replica_path, ErrorCodes::TOO_MANY_RETRIES_TO_FETCH_PARTS);
4050 4051

        Strings parts = getZooKeeper()->getChildren(best_replica_path + "/parts");
4052
        ActiveDataPartSet active_parts_set(data.format_version, parts);
4053 4054 4055 4056 4057 4058 4059 4060 4061
        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)
4062
            {
4063
                if (MergeTreePartInfo::fromPartName(part, data.format_version).partition_id == partition_id)
4064
                    parts_to_fetch_partition.push_back(part);
4065
            }
4066 4067 4068 4069

            parts_to_fetch = std::move(parts_to_fetch_partition);

            if (parts_to_fetch.empty())
4070
                throw Exception("Partition " + partition_id + " on " + best_replica_path + " doesn't exist", ErrorCodes::PARTITION_DOESNT_EXIST);
4071 4072 4073 4074 4075 4076 4077 4078 4079 4080 4081 4082 4083 4084 4085 4086 4087 4088 4089 4090 4091 4092 4093 4094
        }
        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 已提交
4095 4096
                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)
4097 4098 4099 4100 4101 4102 4103 4104 4105
                    throw;

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

        ++try_no;
    } while (!missing_parts.empty());
4106 4107 4108
}


4109
void StorageReplicatedMergeTree::freezePartition(const ASTPtr & partition, const String & with_name, const Context & context)
4110
{
4111
    data.freezePartition(partition, with_name, context);
4112 4113
}

4114

4115 4116
void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, const Context &)
{
4117 4118 4119 4120 4121 4122 4123 4124 4125 4126 4127 4128 4129 4130 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
    /// 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.
    /// TODO: add a way to track the progress of mutations and a process to clean old mutations.

4171 4172 4173 4174 4175 4176 4177 4178 4179 4180 4181 4182 4183
    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();

4184
        Coordination::Stat mutations_stat;
4185 4186 4187 4188 4189 4190 4191 4192 4193 4194
        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);

4195
        Coordination::Requests requests;
4196 4197 4198 4199
        requests.emplace_back(zkutil::makeSetRequest(mutations_path, String(), mutations_stat.version));
        requests.emplace_back(zkutil::makeCreateRequest(
            mutations_path + "/", entry.toString(), zkutil::CreateMode::PersistentSequential));

4200
        Coordination::Responses responses;
4201 4202
        int32_t rc = zookeeper->tryMulti(requests, responses);

4203
        if (rc == Coordination::ZOK)
4204 4205
        {
            const String & path_created =
4206
                dynamic_cast<const Coordination::CreateResponse *>(responses[1].get())->path_created;
4207
            entry.znode_name = path_created.substr(path_created.find_last_of('/') + 1);
4208
            LOG_TRACE(log, "Created mutation with ID " << entry.znode_name);
4209
            break;
4210
        }
4211
        else if (rc == Coordination::ZBADVERSION)
4212 4213 4214 4215 4216
        {
            LOG_TRACE(log, "Version conflict when trying to create a mutation node, retrying...");
            continue;
        }
        else
4217
            throw Coordination::Exception("Unable to create a mutation znode", rc);
4218 4219 4220
    }
}

4221 4222 4223 4224 4225
std::vector<MergeTreeMutationStatus> StorageReplicatedMergeTree::getMutationsStatus() const
{
    return queue.getMutationsStatus();
}

4226

4227
void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK()
4228 4229 4230
{
    /// Critical section is not required (since grabOldParts() returns unique part set on each call)

4231
    auto table_lock = lockStructure(false, __PRETTY_FUNCTION__);
4232 4233 4234
    auto zookeeper = getZooKeeper();

    MergeTreeData::DataPartsVector parts = data.grabOldParts();
4235
    if (parts.empty())
4236 4237
        return;

4238 4239 4240 4241 4242 4243
    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)
4244
    {
4245 4246 4247 4248 4249 4250
        if (!part->is_duplicate)
            parts_to_delete_completely.emplace_back(part);
        else
            parts_to_delete_only_from_filesystem.emplace_back(part);
    }
    parts.clear();
4251

4252 4253 4254
    auto remove_parts_from_filesystem = [log=log] (const MergeTreeData::DataPartsVector & parts_to_remove)
    {
        for (auto & part : parts_to_remove)
4255
        {
4256 4257 4258 4259 4260 4261 4262 4263
            try
            {
                part->remove();
            }
            catch (...)
            {
                tryLogCurrentException(log, "There is a problem with deleting part " + part->name + " from filesystem");
            }
4264 4265 4266
        }
    };

4267 4268 4269 4270 4271 4272 4273 4274 4275 4276
    /// 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
4277
    NameSet part_names_to_retry_deletion;
4278 4279
    try
    {
4280 4281 4282
        Strings part_names_to_delete_completely;
        for (const auto & part : parts_to_delete_completely)
            part_names_to_delete_completely.emplace_back(part->name);
4283

4284 4285
        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);
4286 4287 4288
    }
    catch (...)
    {
4289
        LOG_ERROR(log, "There is a problem with deleting parts from ZooKeeper: " << getCurrentExceptionMessage(true));
4290
    }
4291

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

4296 4297
    /// Delete normal parts on two sets
    for (auto & part : parts_to_delete_completely)
4298 4299 4300 4301 4302
    {
        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);
4303 4304
    }

4305 4306
    /// Will retry deletion
    if (!parts_to_retry_deletion.empty())
4307
    {
4308
        data.rollbackDeletingParts(parts_to_retry_deletion);
4309 4310
        LOG_DEBUG(log, "Will retry deletion of " << parts_to_retry_deletion.size() << " parts in the next time");
    }
4311

4312
    /// Remove parts from filesystem and finally from data_parts
4313
    if (!parts_to_remove_from_filesystem.empty())
4314
    {
4315 4316
        remove_parts_from_filesystem(parts_to_remove_from_filesystem);
        data.removePartsFinally(parts_to_remove_from_filesystem);
4317

4318 4319
        LOG_DEBUG(log, "Removed " << parts_to_remove_from_filesystem.size() << " old parts");
    }
4320 4321 4322
}


4323 4324 4325 4326 4327 4328 4329 4330 4331 4332 4333
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)
{
4334
    using MultiFuture = std::future<Coordination::MultiResponse>;
4335 4336 4337 4338 4339 4340 4341 4342 4343 4344 4345 4346 4347 4348 4349 4350 4351 4352

    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)
            {
4353
                Coordination::Requests ops;
4354 4355 4356 4357 4358 4359 4360 4361 4362
                removePartFromZooKeeper(part_name, ops);

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

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

4363
                if (response.error == 0 || response.error == Coordination::ZNONODE)
4364 4365
                    continue;

4366
                if (Coordination::isHardwareError(response.error))
4367 4368 4369 4370 4371
                {
                    sucess = false;
                    continue;
                }

4372
                throw Coordination::Exception(response.error);
4373 4374
            }
        }
4375
        catch (Coordination::Exception & e)
4376 4377 4378
        {
            sucess = false;

4379
            if (Coordination::isHardwareError(e.code))
4380 4381 4382 4383 4384 4385 4386 4387 4388 4389 4390 4391 4392
                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
4393
void StorageReplicatedMergeTree::removePartsFromZooKeeper(zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names,
A
Alexey Milovidov 已提交
4394
                                                          NameSet * parts_should_be_retried)
4395
{
4396
    Coordination::Requests ops;
4397
    auto it_first_node_in_batch = part_names.cbegin();
4398 4399 4400 4401 4402

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

4403 4404
        auto it_next = std::next(it);
        if (ops.size() >= zkutil::MULTI_BATCH_SIZE || it_next == part_names.cend())
4405
        {
4406
            Coordination::Responses unused_responses;
4407
            auto code = zookeeper->tryMultiNoThrow(ops, unused_responses);
4408
            ops.clear();
4409

4410
            if (code == Coordination::ZNONODE)
4411 4412 4413 4414 4415 4416
            {
                /// Fallback
                LOG_DEBUG(log, "There are no some part nodes in ZooKeeper, will remove part nodes sequentially");

                for (auto it_in_batch = it_first_node_in_batch; it_in_batch != it_next; ++it_in_batch)
                {
4417
                    Coordination::Requests cur_ops;
4418
                    removePartFromZooKeeper(*it_in_batch, cur_ops);
4419
                    auto cur_code = zookeeper->tryMultiNoThrow(cur_ops, unused_responses);
4420

4421
                    if (cur_code == Coordination::ZNONODE)
4422
                    {
4423
                        LOG_DEBUG(log, "There is no part " << *it_in_batch << " in ZooKeeper, it was only in filesystem");
4424
                    }
4425
                    else if (parts_should_be_retried && Coordination::isHardwareError(cur_code))
4426
                    {
A
Alexey Milovidov 已提交
4427
                        parts_should_be_retried->emplace(*it_in_batch);
4428
                    }
4429
                    else if (cur_code)
4430
                    {
4431
                        LOG_WARNING(log, "Cannot remove part " << *it_in_batch << " from ZooKeeper: " << zkutil::ZooKeeper::error2string(cur_code));
4432
                    }
4433 4434
                }
            }
4435
            else if (parts_should_be_retried && Coordination::isHardwareError(code))
4436 4437
            {
                for (auto it_in_batch = it_first_node_in_batch; it_in_batch != it_next; ++it_in_batch)
A
Alexey Milovidov 已提交
4438
                    parts_should_be_retried->emplace(*it_in_batch);
4439
            }
4440
            else if (code)
4441 4442
            {
                LOG_WARNING(log, "There was a problem with deleting " << (it_next - it_first_node_in_batch)
4443
                    << " nodes from ZooKeeper: " << ::zkutil::ZooKeeper::error2string(code));
4444 4445 4446
            }

            it_first_node_in_batch = it_next;
4447 4448 4449 4450 4451
        }
    }
}


4452 4453 4454 4455
void StorageReplicatedMergeTree::clearBlocksInPartition(
    zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num)
{
    Strings blocks;
4456
    if (zookeeper.tryGetChildren(zookeeper_path + "/blocks", blocks))
4457 4458 4459
        throw Exception(zookeeper_path + "/blocks doesn't exist", ErrorCodes::NOT_FOUND_NODE);

    String partition_prefix = partition_id + "_";
4460
    zkutil::AsyncResponses<Coordination::GetResponse> get_futures;
4461 4462 4463 4464 4465 4466 4467 4468 4469
    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));
        }
    }

4470
    zkutil::AsyncResponses<Coordination::RemoveResponse> to_delete_futures;
4471 4472 4473
    for (auto & pair : get_futures)
    {
        const String & path = pair.first;
4474
        auto result = pair.second.get();
4475

4476
        if (result.error == Coordination::ZNONODE)
4477 4478
            continue;

4479
        ReadBufferFromString buf(result.data);
4480 4481 4482 4483 4484 4485 4486 4487 4488
        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;
4489
        int32_t rc = pair.second.get().error;
4490
        if (rc == Coordination::ZNOTEMPTY)
4491 4492 4493 4494
        {
             /// Can happen if there are leftover block nodes with children created by previous server versions.
            zookeeper.removeRecursive(path);
        }
4495
        else if (rc)
4496 4497 4498 4499 4500 4501 4502
            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);
}

4503 4504 4505 4506 4507 4508 4509 4510 4511 4512 4513 4514 4515 4516 4517
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;
4518
    std::vector<EphemeralLockInZooKeeper> ephemeral_locks;
4519 4520 4521 4522 4523 4524 4525 4526 4527 4528 4529 4530 4531 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

    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);
4574
        ephemeral_locks.emplace_back(std::move(*lock));
4575 4576 4577 4578 4579 4580 4581 4582 4583 4584 4585 4586 4587 4588 4589 4590 4591 4592 4593 4594 4595 4596 4597 4598 4599
        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
4600
    queue.removePartProducingOpsInRange(zookeeper, drop_range, entry);
4601 4602 4603 4604 4605 4606

    /// 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;
4607
    Coordination::Responses op_results;
4608 4609 4610

    try
    {
4611
        Coordination::Requests ops;
4612 4613 4614
        for (size_t i = 0; i < dst_parts.size(); ++i)
        {
            getCommitPartOps(ops, dst_parts[i], block_id_paths[i]);
4615
            ephemeral_locks[i].getUnlockOps(ops);
4616 4617 4618 4619 4620 4621 4622 4623 4624 4625 4626 4627 4628 4629 4630 4631 4632 4633 4634 4635 4636 4637 4638 4639 4640 4641 4642 4643 4644 4645 4646 4647 4648 4649 4650 4651 4652

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

        MergeTreeData::Transaction transaction;
        {
            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;
    }

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

4656
    for (auto & lock : ephemeral_locks)
4657 4658 4659 4660 4661 4662 4663
        lock.assumeUnlocked();

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

    /// Speedup removing of replaced parts from filesystem
    parts_to_remove.clear();
4664
    cleanup_thread.wakeup();
4665 4666 4667 4668 4669 4670 4671

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

void StorageReplicatedMergeTree::getCommitPartOps(
4672
    Coordination::Requests & ops,
4673 4674 4675 4676 4677 4678 4679 4680 4681 4682 4683 4684 4685 4686 4687 4688 4689 4690 4691 4692 4693 4694 4695 4696 4697 4698 4699 4700 4701 4702 4703 4704 4705 4706
    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));
}

4707 4708 4709 4710 4711 4712 4713 4714 4715 4716
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;
4717
    res.scheme = context.getInterserverScheme();
4718 4719 4720
    return res;
}

4721
ActionLock StorageReplicatedMergeTree::getActionLock(StorageActionBlockType action_type)
4722 4723
{
    if (action_type == ActionLocks::PartsMerge)
4724
        return merger_mutator.actions_blocker.cancel();
4725 4726 4727 4728 4729 4730 4731 4732

    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)
4733
        return queue.actions_blocker.cancel();
4734 4735 4736 4737

    return {};
}

4738

4739 4740 4741
bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UInt64 max_wait_milliseconds)
{
    /// Let's fetch new log entries firstly
4742
    queue.pullLogsToQueue(getZooKeeper());
4743 4744 4745 4746 4747 4748 4749 4750 4751 4752 4753 4754 4755 4756 4757 4758 4759 4760 4761 4762 4763 4764 4765 4766 4767

    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;

4768
        if (partial_shutdown_called)
4769 4770 4771
            throw Exception("Shutdown is called for table", ErrorCodes::ABORTED);
    }

4772
    return cond_reached.load(std::memory_order_relaxed);
4773
}
4774

4775

4776
bool StorageReplicatedMergeTree::dropPartsInPartition(
Z
zhang2014 已提交
4777 4778
    zkutil::ZooKeeper & zookeeper, String & partition_id, StorageReplicatedMergeTree::LogEntry & entry, bool detach)
{
4779 4780
    MergeTreePartInfo drop_range_info;
    if (!getFakePartCoveringAllPartsInPartition(partition_id, drop_range_info))
Z
zhang2014 已提交
4781 4782 4783 4784 4785
    {
        LOG_INFO(log, "Will not drop partition " << partition_id << ", it is empty.");
        return false;
    }

4786
    clearBlocksInPartition(zookeeper, partition_id, drop_range_info.min_block, drop_range_info.max_block);
Z
zhang2014 已提交
4787 4788 4789 4790

    /** 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.
      */
4791
    String drop_range_fake_part_name = getPartNamePossiblyFake(data.format_version, drop_range_info);
Z
zhang2014 已提交
4792 4793
    {
        std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
4794
        queue.disableMergesInRange(drop_range_fake_part_name);
Z
zhang2014 已提交
4795 4796
    }

4797
    LOG_DEBUG(log, "Disabled merges covered by range " << drop_range_fake_part_name);
Z
zhang2014 已提交
4798 4799 4800 4801

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

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

    return true;
4810 4811
}

M
Merge  
Michael Kolupaev 已提交
4812
}