StorageReplicatedMergeTree.cpp 145.9 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 11 12 13 14 15 16
#include <Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h>
#include <Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h>
#include <Storages/MergeTree/MergeList.h>
#include <Storages/MergeTree/ReplicatedMergeTreeAddress.h>

#include <Databases/IDatabase.h>

#include <Parsers/formatAST.h>
17
#include <Parsers/ASTOptimizeQuery.h>
18
#include <Parsers/ASTLiteral.h>
19 20 21

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

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

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

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

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

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


52 53
namespace ProfileEvents
{
54 55 56 57 58
    extern const Event ReplicatedPartMerges;
    extern const Event ReplicatedPartFailedFetches;
    extern const Event ReplicatedPartFetchesOfMerged;
    extern const Event ObsoleteReplicatedParts;
    extern const Event ReplicatedPartFetches;
59
    extern const Event DataAfterMergeDiffersFromReplica;
60
}
61

62 63 64 65 66 67
namespace CurrentMetrics
{
    extern const Metric LeaderReplica;
}


M
Merge  
Michael Kolupaev 已提交
68 69 70
namespace DB
{

71 72
namespace ErrorCodes
{
73 74 75 76 77 78 79 80 81 82 83 84 85 86 87 88 89
    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 已提交
90
    extern const int TOO_MANY_RETRIES_TO_FETCH_PARTS;
91 92 93 94
    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;
95 96
    extern const int UNEXPECTED_FILE_IN_DATA_PART;
    extern const int NO_FILE_IN_DATA_PART;
97 98
    extern const int UNFINISHED;
    extern const int METADATA_MISMATCH;
99
    extern const int RECEIVED_ERROR_TOO_MANY_REQUESTS;
A
Alexey Milovidov 已提交
100
    extern const int TOO_MANY_FETCHES;
101
    extern const int BAD_DATA_PART_NAME;
102
    extern const int PART_IS_TEMPORARILY_LOCKED;
103
    extern const int INCORRECT_FILE_NAME;
104
    extern const int CANNOT_ASSIGN_OPTIMIZE;
105 106
}

M
Merge  
Michael Kolupaev 已提交
107

108 109
static const auto QUEUE_UPDATE_ERROR_SLEEP_MS     = 1 * 1000;
static const auto MERGE_SELECTING_SLEEP_MS        = 5 * 1000;
M
Merge  
Michael Kolupaev 已提交
110 111


F
f1yegor 已提交
112 113
/** There are three places for each part, where it should be
  * 1. In the RAM, MergeTreeData::data_parts, all_data_parts.
114
  * 2. In the filesystem (FS), the directory with the data of the table.
F
f1yegor 已提交
115
  * 3. in ZooKeeper (ZK).
116
  *
F
f1yegor 已提交
117 118
  * When adding a part, it must be added immediately to these three places.
  * This is done like this
119 120
  * - [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 已提交
121 122 123 124
  * - [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);
125
  * - [FS, ZK] by the way, removing the covered (old) parts from filesystem, from ZooKeeper and from `all_data_parts`
F
f1yegor 已提交
126
  *   is delayed, after a few minutes.
127
  *
F
f1yegor 已提交
128 129 130
  * 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.
131
  *
F
f1yegor 已提交
132 133 134 135 136
  * 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.
137
  *
F
f1yegor 已提交
138 139 140 141
  * 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.
142
  *
F
f1yegor 已提交
143 144
  * 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.
145
  */
146
extern const int MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER = 5 * 60;
147 148


149
/** For randomized selection of replicas. */
P
proller 已提交
150 151 152 153 154
/// avoid error: non-local variable 'DB::rng' declared '__thread' needs dynamic initialization
#ifndef __APPLE__
thread_local
#endif
    pcg64 rng{randomSeed()};
155 156


A
Merge  
Alexey Milovidov 已提交
157 158
void StorageReplicatedMergeTree::setZooKeeper(zkutil::ZooKeeperPtr zookeeper)
{
159 160
    std::lock_guard<std::mutex> lock(current_zookeeper_mutex);
    current_zookeeper = zookeeper;
A
Merge  
Alexey Milovidov 已提交
161 162 163 164
}

zkutil::ZooKeeperPtr StorageReplicatedMergeTree::tryGetZooKeeper()
{
165 166
    std::lock_guard<std::mutex> lock(current_zookeeper_mutex);
    return current_zookeeper;
A
Merge  
Alexey Milovidov 已提交
167 168 169 170
}

zkutil::ZooKeeperPtr StorageReplicatedMergeTree::getZooKeeper()
{
171 172 173 174
    auto res = tryGetZooKeeper();
    if (!res)
        throw Exception("Cannot get ZooKeeper", ErrorCodes::NO_ZOOKEEPER);
    return res;
A
Merge  
Alexey Milovidov 已提交
175 176 177
}


M
Merge  
Michael Kolupaev 已提交
178
StorageReplicatedMergeTree::StorageReplicatedMergeTree(
179 180 181 182
    const String & zookeeper_path_,
    const String & replica_name_,
    bool attach,
    const String & path_, const String & database_name_, const String & name_,
183
    const ColumnsDescription & columns_,
184
    Context & context_,
185
    const ASTPtr & primary_expr_ast_,
186
    const ASTPtr & secondary_sorting_expr_list_,
187 188
    const String & date_column_name,
    const ASTPtr & partition_expr_ast_,
189 190
    const ASTPtr & sampling_expression_,
    const MergeTreeData::MergingParams & merging_params_,
191 192
    const MergeTreeSettings & settings_,
    bool has_force_restore_data_flag)
193
    : context(context_),
194 195
    current_zookeeper(context.getZooKeeper()), database_name(database_name_),
    table_name(name_), full_path(path_ + escapeForFileName(table_name) + '/'),
196 197
    zookeeper_path(context.getMacros()->expand(zookeeper_path_)),
    replica_name(context.getMacros()->expand(replica_name_)),
198 199
    data(database_name, table_name,
        full_path, columns_,
200
        context_, primary_expr_ast_, secondary_sorting_expr_list_, date_column_name, partition_expr_ast_,
201
        sampling_expression_, merging_params_,
A
Alexey Zatelepin 已提交
202
        settings_, true, attach,
A
Alexey Milovidov 已提交
203
        [this] (const std::string & name) { enqueuePartForCheck(name); }),
204
    reader(data), writer(data), merger(data, context.getBackgroundPool()), queue(data.format_version),
205
    fetcher(data),
206 207
    shutdown_event(false), part_check_thread(*this),
    log(&Logger::get(database_name + "." + table_name + " (StorageReplicatedMergeTree)"))
M
Merge  
Michael Kolupaev 已提交
208
{
209 210 211
    if (path_.empty())
        throw Exception("ReplicatedMergeTree storages require data path", ErrorCodes::INCORRECT_FILE_NAME);

212 213
    if (!zookeeper_path.empty() && zookeeper_path.back() == '/')
        zookeeper_path.resize(zookeeper_path.size() - 1);
214
    /// If zookeeper chroot prefix is used, path should start with '/', because chroot concatenates without it.
K
KochetovNicolai 已提交
215 216
    if (!zookeeper_path.empty() && zookeeper_path.front() != '/')
        zookeeper_path = "/" + zookeeper_path;
217 218 219 220
    replica_path = zookeeper_path + "/replicas/" + replica_name;

    bool skip_sanity_checks = false;

221
    if (current_zookeeper && current_zookeeper->exists(replica_path + "/flags/force_restore_data"))
222
    {
223 224
        skip_sanity_checks = true;
        current_zookeeper->remove(replica_path + "/flags/force_restore_data");
225

226 227
        LOG_WARNING(log, "Skipping the limits on severity of changes to data parts and columns (flag "
            << replica_path << "/flags/force_restore_data).");
228
    }
229
    else if (has_force_restore_data_flag)
230
    {
231 232 233
        skip_sanity_checks = true;

        LOG_WARNING(log, "Skipping the limits on severity of changes to data parts and columns (flag force_restore_data).");
234 235 236 237 238 239 240 241 242 243 244 245 246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262
    }

    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);
263 264 265 266

        /// 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);
267 268 269
    }

    createNewZooKeeperNodes();
M
Merge  
Michael Kolupaev 已提交
270 271
}

272

273 274
void StorageReplicatedMergeTree::createNewZooKeeperNodes()
{
275
    auto zookeeper = getZooKeeper();
276

277 278 279 280
    /// Working with quorum.
    zookeeper->createIfNotExists(zookeeper_path + "/quorum", "");
    zookeeper->createIfNotExists(zookeeper_path + "/quorum/last_part", "");
    zookeeper->createIfNotExists(zookeeper_path + "/quorum/failed_parts", "");
281

282 283 284
    /// Tracking lag of replicas.
    zookeeper->createIfNotExists(replica_path + "/min_unprocessed_insert_time", "");
    zookeeper->createIfNotExists(replica_path + "/max_processed_insert_time", "");
285 286 287
}


M
Merge  
Michael Kolupaev 已提交
288 289
static String formattedAST(const ASTPtr & ast)
{
290 291 292
    if (!ast)
        return "";
    std::stringstream ss;
A
Alexey Milovidov 已提交
293
    formatAST(*ast, ss, false, true);
294
    return ss.str();
M
Merge  
Michael Kolupaev 已提交
295
}
M
Merge  
Michael Kolupaev 已提交
296

A
Merge  
Alexey Milovidov 已提交
297

298 299
namespace
{
300 301 302 303 304 305 306
    /** The basic parameters of table engine for saving in ZooKeeper.
      * Lets you verify that they match local ones.
      */
    struct TableMetadata
    {
        const MergeTreeData & data;

307
        explicit TableMetadata(const MergeTreeData & data_)
308 309 310 311 312
            : data(data_) {}

        void write(WriteBuffer & out) const
        {
            out << "metadata format version: 1" << "\n"
313 314 315 316 317 318 319 320
                << "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"
321 322 323 324
                << "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";
325 326 327 328 329 330

            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";
            }
331 332 333 334
        }

        String toString() const
        {
335
            WriteBufferFromOwnString out;
336
            write(out);
337
            return out.str();
338 339 340 341 342 343 344 345 346
        }

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

            in >> "metadata format version: 1";

            in >> "\ndate column: ";
347 348 349 350 351 352 353 354 355 356 357 358 359 360 361
            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.",
362 363 364 365 366 367 368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402 403 404 405 406 407 408 409 410 411 412 413 414
                    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";
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
            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";
            }

446 447 448 449 450 451 452 453 454
            assertEOF(in);
        }

        void check(const String & s) const
        {
            ReadBufferFromString in(s);
            check(in);
        }
    };
455 456 457
}


M
Merge  
Michael Kolupaev 已提交
458
void StorageReplicatedMergeTree::createTableIfNotExists()
M
Merge  
Michael Kolupaev 已提交
459
{
460 461 462 463 464 465 466 467 468 469 470 471
    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();

472 473 474 475 476 477 478 479 480 481 482 483 484 485 486 487 488 489 490 491 492
    zkutil::Requests ops;
    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", "",
        zkutil::CreateMode::Persistent));
    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));
493

494 495
    zkutil::Responses responses;
    auto code = zookeeper->tryMulti(ops, responses);
496
    if (code && code != ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
497
        throw zkutil::KeeperException(code);
M
Merge  
Michael Kolupaev 已提交
498
}
M
Merge  
Michael Kolupaev 已提交
499

A
Merge  
Alexey Milovidov 已提交
500

F
f1yegor 已提交
501
/** Verify that list of columns and table settings match those specified in ZK (/ metadata).
502 503
    * If not, throw an exception.
    */
M
Merge  
Michael Kolupaev 已提交
504
void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bool allow_alter)
M
Merge  
Michael Kolupaev 已提交
505
{
506 507 508 509 510 511
    auto zookeeper = getZooKeeper();

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

    zkutil::Stat stat;
512
    auto columns_from_zk = ColumnsDescription::parse(zookeeper->get(zookeeper_path + "/columns", &stat));
513 514
    columns_version = stat.version;

515 516
    const ColumnsDescription & old_columns = getColumns();
    if (columns_from_zk != old_columns)
517 518 519
    {
        if (allow_alter &&
            (skip_sanity_checks ||
520 521
             old_columns.ordinary.sizeOfDifference(columns_from_zk.ordinary) +
             old_columns.materialized.sizeOfDifference(columns_from_zk.materialized) <= 2))
522 523 524 525
        {
            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.
526 527
            context.getDatabase(database_name)->alterTable(context, table_name, columns_from_zk, {});

528
            setColumns(std::move(columns_from_zk));
529 530 531
        }
        else
        {
532
            throw Exception("Table structure in ZooKeeper is too different from local table structure",
533 534 535
                            ErrorCodes::INCOMPATIBLE_COLUMNS);
        }
    }
M
Merge  
Michael Kolupaev 已提交
536
}
M
Merge  
Michael Kolupaev 已提交
537

A
Merge  
Alexey Milovidov 已提交
538

F
f1yegor 已提交
539 540 541 542
/** 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).
543 544 545
  */
static time_t tryGetPartCreateTime(zkutil::ZooKeeperPtr & zookeeper, const String & replica_path, const String & part_name)
{
546
    time_t res = 0;
547

548 549 550 551 552
    /// We get creation time of part, if it still exists (was not merged, for example).
    zkutil::Stat stat;
    String unused;
    if (zookeeper->tryGet(replica_path + "/parts/" + part_name, unused, &stat))
        res = stat.ctime / 1000;
553

554
    return res;
555 556 557
}


M
Merge  
Michael Kolupaev 已提交
558 559
void StorageReplicatedMergeTree::createReplica()
{
560 561 562 563 564
    auto zookeeper = getZooKeeper();

    LOG_DEBUG(log, "Creating replica " << replica_path);

    /// Create an empty replica. We'll create `columns` node at the end - we'll use it as a sign that replica creation is complete.
565 566 567 568 569 570 571
    zkutil::Requests ops;
    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));
572 573 574 575 576 577 578

    try
    {
        zookeeper->multi(ops);
    }
    catch (const zkutil::KeeperException & e)
    {
579
        if (e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
580 581 582 583 584 585 586 587 588 589 590 591 592 593 594 595 596
            throw Exception("Replica " + replica_path + " already exists.", ErrorCodes::REPLICA_IS_ALREADY_EXIST);

        throw;
    }

    /** You need to change the data of nodes/replicas to anything, so that the thread that removes old entries in the log,
      *  stumbled over this change and does not delete the entries we have not yet read.
      */
    zookeeper->set(zookeeper_path + "/replicas", "last added replica: " + replica_name);

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

    /** "Reference" replica, from which we take information about the set of parts, queue and pointer to the log.
      * Take random replica created earlier than this.
      */
    String source_replica;

597
    zkutil::Stat stat;
598 599 600
    zookeeper->exists(replica_path, &stat);
    auto my_create_time = stat.czxid;

601
    std::shuffle(replicas.begin(), replicas.end(), rng);
602 603 604 605 606 607 608 609 610 611 612 613 614 615 616 617 618 619 620 621 622 623 624 625 626 627 628 629 630 631 632 633 634 635 636 637 638 639 640 641 642 643 644 645 646 647 648 649 650 651 652 653 654 655 656 657 658 659 660 661 662
    for (const String & replica : replicas)
    {
        if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica, &stat))
            throw Exception("Replica " + zookeeper_path + "/replicas/" + replica + " was removed from right under our feet.",
                            ErrorCodes::NO_SUCH_REPLICA);
        if (stat.czxid < my_create_time)
        {
            source_replica = replica;
            break;
        }
    }

    if (source_replica.empty())
    {
        LOG_INFO(log, "This is the first replica");
    }
    else
    {
        LOG_INFO(log, "Will mimic " << source_replica);

        String source_path = zookeeper_path + "/replicas/" + source_replica;

        /** If the reference/master replica is not yet fully created, let's wait.
          * NOTE: If something went wrong while creating it, we can hang around forever.
          *    You can create an ephemeral node at the time of creation to make sure that the replica is created, and not abandoned.
          *    The same can be done for the table. You can automatically delete a replica/table node,
          *     if you see that it was not created up to the end, and the one who created it died.
          */
        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.

        /// Copy reference to the log from `reference/master` replica.
        zookeeper->set(replica_path + "/log_pointer", zookeeper->get(source_path + "/log_pointer"));

        /// 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");
663
        ActiveDataPartSet active_parts_set(data.format_version, parts);
664 665 666 667 668 669 670 671 672 673 674 675 676 677 678 679 680 681 682 683 684 685 686 687 688

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

            zookeeper->create(replica_path + "/queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential);
        }
        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);
        }

        /// It will then be loaded into the queue variable in `queue.initialize` method.

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

689
    zookeeper->create(replica_path + "/columns", getColumns().toString(), zkutil::CreateMode::Persistent);
M
Merge  
Michael Kolupaev 已提交
690
}
M
Merge  
Michael Kolupaev 已提交
691 692


M
Merge  
Michael Kolupaev 已提交
693
void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
M
Merge  
Michael Kolupaev 已提交
694
{
695 696 697 698 699 700 701
    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());

702 703
    /// There are no PreCommitted parts at startup.
    auto parts = data.getDataParts({MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated});
704 705 706 707 708 709 710 711 712 713 714 715 716 717

    /// 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;
718
    UInt64 parts_to_add_rows = 0;
719 720 721 722 723 724 725 726 727 728 729 730 731 732 733

    /// 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);
734
                parts_to_add_rows += containing->rows_count;
735 736 737 738 739 740 741 742 743 744 745 746
            }
        }
        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);

747

748 749 750 751 752
    /** 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;
753 754
    UInt64 unexpected_parts_nonnew_rows = 0;
    UInt64 unexpected_parts_rows = 0;
755
    for (const auto & part : unexpected_parts)
756
    {
757
        if (part->info.level > 0)
758
        {
759
            ++unexpected_parts_nonnew;
760 761 762 763 764 765 766 767 768 769 770 771 772 773 774 775 776 777 778 779 780 781 782 783 784 785 786 787 788 789 790 791
            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).";
792

793 794 795
    /** 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%).
796
      *
797 798
      * 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).
799 800 801
      * In this case, the protection mechanism does not allow the server to start.
      */

802 803 804 805 806 807
    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;
808

809
    bool insane = total_suspicious_rows > total_rows_on_filesystem * data.settings.replicated_max_ratio_of_wrong_parts;
810 811

    if (insane && !skip_sanity_checks)
812 813 814 815 816 817 818 819 820
    {
        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);
    }
821

822 823
    if (total_suspicious_rows_no_new > 0)
        LOG_WARNING(log, sanity_report.str());
824 825 826 827 828 829

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

830
        zkutil::Requests ops;
831
        checkPartChecksumsAndAddCommitOps(zookeeper, part, ops);
832 833 834 835 836
        zookeeper->multi(ops);
    }

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

840
        removePartsFromZooKeeper(zookeeper, Strings(expected_parts.begin(), expected_parts.end()));
841 842 843 844 845 846 847 848 849 850 851 852 853 854
    }

    /// 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).
855
        zkutil::Requests ops;
856
        removePartFromZooKeeper(name, ops);
857 858
        ops.emplace_back(zkutil::makeCreateRequest(
            replica_path + "/queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential));
859 860 861 862 863 864 865 866 867
        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);
        data.renameAndDetachPart(part, "ignored_", true);
    }
M
Merge  
Michael Kolupaev 已提交
868
}
M
Merge  
Michael Kolupaev 已提交
869

A
Merge  
Alexey Milovidov 已提交
870

871
void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil::ZooKeeperPtr & zookeeper,
872
    const MergeTreeData::DataPartPtr & part, zkutil::Requests & ops, String part_name, NameSet * absent_replicas_paths)
M
Merge  
Michael Kolupaev 已提交
873
{
874 875 876 877 878 879 880
    if (part_name.empty())
        part_name = part->name;

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

    Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
881
    std::shuffle(replicas.begin(), replicas.end(), rng);
882
    String expected_columns_str = part->columns.toString();
883
    bool has_been_alredy_added = false;
884 885 886 887

    for (const String & replica : replicas)
    {
        zkutil::Stat stat_before, stat_after;
888 889
        String current_part_path = zookeeper_path + "/replicas/" + replica + "/parts/" + part_name;

890
        String columns_str;
891 892 893 894 895
        if (!zookeeper->tryGet(current_part_path + "/columns", columns_str, &stat_before))
        {
            if (absent_replicas_paths)
                absent_replicas_paths->emplace(current_part_path);

896
            continue;
897 898
        }

899 900 901 902 903 904
        if (columns_str != expected_columns_str)
        {
            LOG_INFO(log, "Not checking checksums of part " << part_name << " with replica " << replica
                << " because columns are different");
            continue;
        }
905

906 907 908
        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.
909 910
        if (!zookeeper->tryGet(current_part_path + "/checksums", checksums_str) ||
            !zookeeper->exists(current_part_path + "/columns", &stat_after) ||
911 912 913 914 915 916 917
            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;
        }

918 919 920 921 922
        auto zk_checksums = MinimalisticDataPartChecksums::deserializeFrom(checksums_str);
        zk_checksums.checkEqual(part->checksums, true);

        if (replica == replica_name)
            has_been_alredy_added = true;
923 924
    }

925
    if (!has_been_alredy_added)
926
    {
927 928
        String part_path = replica_path + "/parts/" + part_name;

929
        ops.emplace_back(zkutil::makeCheckRequest(
930
            zookeeper_path + "/columns", expected_columns_version));
931 932 933 934 935 936
        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));
937 938 939 940 941
    }
    else
    {
        LOG_WARNING(log, "checkPartAndAddToZooKeeper: node " << replica_path + "/parts/" + part_name << " already exists."
            << " Will not commit any nodes.");
942
    }
943
}
944

945 946 947 948
MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAndCommit(MergeTreeData::Transaction & transaction,
    const MergeTreeData::DataPartPtr & part)
{
    auto zookeeper = getZooKeeper();
949

950 951
    while (true)
    {
952
        zkutil::Requests ops;
953 954 955 956 957 958 959 960 961 962 963 964
        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())
        {
965
            zkutil::Requests new_ops;
966 967
            for (const String & part_path : absent_part_paths_on_replicas)
            {
968 969
                new_ops.emplace_back(zkutil::makeCreateRequest(part_path, "", zkutil::CreateMode::Persistent));
                new_ops.emplace_back(zkutil::makeRemoveRequest(part_path, -1));
970 971 972 973 974 975 976 977 978 979 980 981
            }

            /// 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();
        }
982
        catch (const zkutil::KeeperMultiException & e)
983 984
        {
            size_t num_check_ops = 2 * absent_part_paths_on_replicas.size();
985
            size_t failed_op_index = e.failed_op_index;
986

987
            if (failed_op_index < num_check_ops && e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
988
            {
989
                LOG_INFO(log, "The part " << e.getPathForFirstFailedOp() << " on a replica suddenly appeared, will recheck checksums");
990 991 992 993 994 995 996 997 998 999 1000
            }
            else
                throw;
        }
    }
}

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

A
Merge  
Alexey Milovidov 已提交
1003

1004
void StorageReplicatedMergeTree::pullLogsToQueue(zkutil::EventPtr next_update_event)
M
Merge  
Michael Kolupaev 已提交
1005
{
1006 1007 1008 1009 1010
    if (queue.pullLogsToQueue(getZooKeeper(), next_update_event))
    {
        if (queue_task_handle)
            queue_task_handle->wake();
    }
M
Merge  
Michael Kolupaev 已提交
1011 1012
}

A
Merge  
Alexey Milovidov 已提交
1013

1014
bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
1015
{
1016 1017 1018 1019 1020 1021
    if (entry.type == LogEntry::ATTACH_PART)
    {
        LOG_ERROR(log, "Log entries of type ATTACH_PART are obsolete. Skipping.");
        return true;
    }

1022 1023 1024 1025 1026 1027
    if (entry.type == LogEntry::DROP_RANGE)
    {
        executeDropRange(entry);
        return true;
    }

1028
    if (entry.type == LogEntry::CLEAR_COLUMN)
1029
    {
1030
        executeClearColumnInPartition(entry);
1031 1032 1033
        return true;
    }

1034
    if (entry.type == LogEntry::GET_PART ||
1035
        entry.type == LogEntry::MERGE_PARTS)
1036 1037
    {
        /// If we already have this part or a part covering it, we do not need to do anything.
A
Alexey Zatelepin 已提交
1038 1039
        /// 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.
1040 1041 1042
        MergeTreeData::DataPartPtr existing_part = data.getPartIfExists(entry.new_part_name, {MergeTreeDataPartState::PreCommitted});
        if (!existing_part)
            existing_part = data.getActiveContainingPart(entry.new_part_name);
1043 1044

        /// Even if the part is locally, it (in exceptional cases) may not be in ZooKeeper. Let's check that it is there.
1045
        if (existing_part && getZooKeeper()->exists(replica_path + "/parts/" + existing_part->name))
1046 1047
        {
            if (!(entry.type == LogEntry::GET_PART && entry.source_replica == replica_name))
1048 1049 1050
            {
                LOG_DEBUG(log, "Skipping action for part " << entry.new_part_name << " because part " + existing_part->name + " already exists.");
            }
1051 1052 1053 1054 1055 1056 1057 1058 1059 1060 1061 1062 1063 1064 1065 1066 1067 1068 1069 1070
            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)
1071 1072 1073 1074 1075 1076 1077 1078 1079 1080 1081 1082 1083 1084 1085 1086 1087 1088 1089 1090 1091 1092
    {
        tryExecuteMerge(entry, do_fetch);
    }
    else
    {
        throw Exception("Unexpected log entry type: " + toString(static_cast<int>(entry.type)));
    }

    if (do_fetch)
        return executeFetch(entry);

    return true;
}


void StorageReplicatedMergeTree::tryExecuteMerge(const StorageReplicatedMergeTree::LogEntry & entry, bool & do_fetch)
{
    /// The caller has already decided to make the fetch
    if (do_fetch)
        return;

    // Log source part names just in case
1093 1094 1095 1096 1097 1098 1099 1100
    {
        std::stringstream log_message;
        log_message << "Executing log entry to merge parts ";
        for (auto i : ext::range(0, entry.parts_to_merge.size()))
            log_message << (i != 0 ? ", " : "") << entry.parts_to_merge[i];
        log_message << " to " << entry.new_part_name;

        LOG_TRACE(log, log_message.rdbuf());
1101
    }
1102

1103 1104 1105 1106 1107 1108
    MergeTreeData::DataPartsVector parts;
    bool have_all_parts = true;
    for (const String & name : entry.parts_to_merge)
    {
        MergeTreeData::DataPartPtr part = data.getActiveContainingPart(name);
        if (!part)
1109
        {
1110 1111
            have_all_parts = false;
            break;
1112
        }
1113
        if (part->name != name)
1114
        {
1115 1116 1117 1118
            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;
1119
        }
1120 1121 1122 1123 1124 1125 1126 1127 1128 1129 1130 1131 1132
        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.
        do_fetch = true;
        LOG_DEBUG(log, "Don't have all parts for merge " << entry.new_part_name << "; will try to fetch it instead");
    }
    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.
1133

1134
        size_t sum_parts_bytes_on_disk = 0;
1135
        for (const auto & part : parts)
1136
            sum_parts_bytes_on_disk += part->bytes_on_disk;
1137

1138
        if (sum_parts_bytes_on_disk >= data.settings.prefer_fetch_merged_part_size_threshold)
1139 1140 1141
        {
            String replica = findReplicaHavingPart(entry.new_part_name, true);    /// NOTE excessive ZK requests for same data later, may remove.
            if (!replica.empty())
1142
            {
1143 1144
                do_fetch = true;
                LOG_DEBUG(log, "Prefer to fetch " << entry.new_part_name << " from replica " << replica);
1145 1146
            }
        }
1147
    }
1148

1149 1150
    if (do_fetch)
        return;
1151

1152
    /// Start to make the main work
1153

1154
    size_t estimated_space_for_merge = MergeTreeDataMerger::estimateDiskSpaceForMerge(parts);
1155

1156 1157
    /// Can throw an exception.
    DiskSpaceMonitor::ReservationPtr reserved_space = DiskSpaceMonitor::reserve(full_path, estimated_space_for_merge);
1158

1159
    auto table_lock = lockStructure(false, __PRETTY_FUNCTION__);
1160

1161 1162 1163
    MergeList::EntryPtr merge_entry = context.getMergeList().insert(database_name, table_name, entry.new_part_name, parts);
    MergeTreeData::Transaction transaction;
    size_t aio_threshold = context.getSettings().min_bytes_to_use_direct_io;
1164

1165 1166 1167 1168 1169 1170
    MergeTreeDataMerger::FuturePart future_merged_part(parts);
    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);
    }
1171

1172 1173 1174 1175
    /// Logging
    Stopwatch stopwatch;
    ExecutionStatus execution_status;
    MergeTreeData::MutableDataPartPtr part;
1176

1177 1178 1179 1180
    auto write_part_log = [&] (const ExecutionStatus & execution_status)
    {
        try
        {
1181
            auto part_log = context.getPartLog(database_name);
1182 1183
            if (!part_log)
                return;
1184

1185
            PartLogElement part_log_elem;
1186

1187 1188 1189 1190
            part_log_elem.event_type = PartLogElement::MERGE_PARTS;
            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 = stopwatch.elapsed() / 1000000;
1191

1192 1193 1194
            part_log_elem.database_name = database_name;
            part_log_elem.table_name = table_name;
            part_log_elem.part_name = entry.new_part_name;
1195

1196
            if (part)
1197
                part_log_elem.bytes_compressed_on_disk = part->bytes_on_disk;
1198

1199 1200 1201
            part_log_elem.source_part_names.reserve(parts.size());
            for (const auto & source_part : parts)
                part_log_elem.source_part_names.push_back(source_part->name);
1202

1203 1204
            part_log_elem.rows_read = (*merge_entry)->bytes_read_uncompressed;
            part_log_elem.bytes_read_uncompressed = (*merge_entry)->bytes_read_uncompressed;
1205

1206 1207
            part_log_elem.rows = (*merge_entry)->rows_written;
            part_log_elem.bytes_uncompressed = (*merge_entry)->bytes_written_uncompressed;
1208

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

1212 1213 1214 1215 1216 1217 1218
            part_log->add(part_log_elem);
        }
        catch (...)
        {
            tryLogCurrentException(log, __PRETTY_FUNCTION__);
        }
    };
1219

1220 1221 1222 1223
    try
    {
        part = merger.mergePartsToTemporaryPart(
            future_merged_part, *merge_entry, aio_threshold, entry.create_time, reserved_space.get(), entry.deduplicate);
1224

1225
        merger.renameMergedTemporaryPart(part, parts, &transaction);
1226

1227 1228
        try
        {
1229
            checkPartChecksumsAndCommit(transaction, part);
1230 1231 1232
        }
        catch (const Exception & e)
        {
1233
            if (MergeTreeDataPartChecksums::isBadChecksumsErrorCode(e.code()))
1234 1235
            {
                do_fetch = true;
1236
                transaction.rollback();
1237 1238 1239 1240 1241 1242 1243 1244 1245 1246 1247 1248 1249 1250 1251 1252 1253 1254

                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());
                return;
1255
            }
1256 1257 1258 1259 1260 1261 1262

            throw;
        }

        /** Removing old chunks from ZK and from the disk is delayed - see ReplicatedMergeTreeCleanupThread, clearOldParts.
          */

1263
        /** With `ZSESSIONEXPIRED` or `ZOPERATIONTIMEOUT`, we can inadvertently roll back local changes to the parts.
1264 1265 1266 1267 1268 1269
          * This is not a problem, because in this case the merge will remain in the queue, and we will try again.
          */
        merge_selecting_event.set();
        ProfileEvents::increment(ProfileEvents::ReplicatedPartMerges);

        write_part_log({});
1270
    }
1271
    catch (...)
1272
    {
1273 1274
        write_part_log(ExecutionStatus::fromCurrentException());
        throw;
1275
    }
1276
}
1277 1278


1279 1280 1281
bool StorageReplicatedMergeTree::executeFetch(const StorageReplicatedMergeTree::LogEntry & entry)
{
    String replica = findReplicaHavingCoveringPart(entry, true);
1282

1283 1284 1285 1286
    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 已提交
1287
            ErrorCodes::TOO_MANY_FETCHES);
1288
    }
1289

1290 1291
    ++total_fetches;
    SCOPE_EXIT({--total_fetches;});
1292

1293 1294 1295
    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 已提交
1296
            ErrorCodes::TOO_MANY_FETCHES);
1297
    }
1298

1299 1300 1301 1302 1303 1304
    ++current_table_fetches;
    SCOPE_EXIT({--current_table_fetches;});

    try
    {
        if (replica.empty())
1305
        {
1306 1307 1308 1309 1310 1311
            /** 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)
1312
            {
1313 1314 1315 1316 1317 1318 1319 1320 1321 1322 1323 1324 1325 1326 1327
                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;
                  * - set `nonincrement_block_numbers` to resolve merges through the number of the lost part;
                  * - 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.
1328 1329
                  */

1330 1331 1332 1333
                /** 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.
                  */
1334

1335
                auto zookeeper = getZooKeeper();
1336

1337
                Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
1338

1339
                zkutil::Requests ops;
1340

1341 1342
                for (size_t i = 0, size = replicas.size(); i < size; ++i)
                {
1343
                    zkutil::Stat stat;
1344 1345
                    String path = zookeeper_path + "/replicas/" + replicas[i] + "/host";
                    zookeeper->get(path, &stat);
1346
                    ops.emplace_back(zkutil::makeCheckRequest(path, stat.version));
1347
                }
1348

1349 1350
                /// 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);
1351

1352 1353 1354 1355 1356
                /// 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())
                {
1357
                    zkutil::Stat quorum_stat;
1358 1359 1360 1361 1362 1363
                    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)
1364
                    {
1365
                        ops.emplace_back(zkutil::makeRemoveRequest(quorum_path, quorum_stat.version));
1366

1367
                        auto part_info = MergeTreePartInfo::fromPartName(entry.new_part_name, data.format_version);
1368

1369 1370 1371
                        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);
1372

1373 1374
                        zookeeper->createIfNotExists(zookeeper_path + "/nonincrement_block_numbers/" + part_info.partition_id, "");

1375
                        ops.emplace_back(zkutil::makeCreateRequest(
1376 1377 1378 1379
                            zookeeper_path + "/nonincrement_block_numbers/" + part_info.partition_id + "/block-" + padIndex(part_info.min_block),
                            "",
                            zkutil::CreateMode::Persistent));

1380
                        ops.emplace_back(zkutil::makeCreateRequest(
1381 1382 1383 1384 1385 1386
                            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))
1387
                            ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/blocks/" + entry.block_id, -1));
1388

1389 1390
                        zkutil::Responses responses;
                        auto code = zookeeper->tryMulti(ops, responses);
1391

1392
                        if (code == ZooKeeperImpl::ZooKeeper::ZOK)
1393
                        {
1394 1395
                            LOG_DEBUG(log, "Marked quorum for part " << entry.new_part_name << " as failed.");
                            return true;    /// NOTE Deletion from `virtual_parts` is not done, but it is only necessary for merges.
1396
                        }
1397
                        else if (code == ZooKeeperImpl::ZooKeeper::ZBADVERSION || code == ZooKeeperImpl::ZooKeeper::ZNONODE || code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
1398
                        {
1399
                            LOG_DEBUG(log, "State was changed or isn't expected when trying to mark quorum for part "
1400
                                << entry.new_part_name << " as failed. Code: " << zkutil::ZooKeeper::error2string(code));
1401
                        }
1402 1403 1404 1405 1406 1407 1408 1409 1410 1411
                        else
                            throw zkutil::KeeperException(code);
                    }
                    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.");
1412 1413 1414 1415
                    }
                }
            }

1416
            if (replica.empty())
1417
            {
1418 1419
                ProfileEvents::increment(ProfileEvents::ReplicatedPartFailedFetches);
                throw Exception("No active replica has part " + entry.new_part_name + " or covering part", ErrorCodes::NO_REPLICA_HAS_PART);
1420
            }
1421
        }
1422

1423 1424 1425 1426
        try
        {
            if (!fetchPart(entry.actual_new_part_name, zookeeper_path + "/replicas/" + replica, false, entry.quorum))
                return false;
1427
        }
1428
        catch (Exception & e)
1429
        {
1430 1431 1432 1433 1434
            /// No stacktrace, just log message
            if (e.code() == ErrorCodes::RECEIVED_ERROR_TOO_MANY_REQUESTS)
                e.addMessage("Too busy replica. Will try later.");
            throw;
        }
1435

1436 1437 1438 1439 1440 1441 1442 1443 1444 1445 1446 1447
        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);
1448

1449
            if (!parts_for_merge.empty() && replica.empty())
1450
            {
1451 1452
                LOG_INFO(log, "No active replica has part " << entry.new_part_name << ". Will fetch merged part instead.");
                return false;
1453 1454
            }

1455 1456 1457 1458 1459
            /** 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);
1460
        }
1461 1462
        catch (...)
        {
1463
            tryLogCurrentException(log, __PRETTY_FUNCTION__);
1464 1465 1466
        }

        throw;
1467 1468 1469
    }

    return true;
M
Merge  
Michael Kolupaev 已提交
1470 1471
}

A
Merge  
Alexey Milovidov 已提交
1472

M
Merge  
Michael Kolupaev 已提交
1473
void StorageReplicatedMergeTree::executeDropRange(const StorageReplicatedMergeTree::LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
1474
{
1475 1476 1477 1478 1479 1480 1481
    LOG_INFO(log, (entry.detach ? "Detaching" : "Removing") << " parts inside " << entry.new_part_name << ".");

    queue.removeGetsAndMergesInRange(getZooKeeper(), entry.new_part_name);

    LOG_DEBUG(log, (entry.detach ? "Detaching" : "Removing") << " parts.");
    size_t removed_parts = 0;

1482
    auto entry_part_info = MergeTreePartInfo::fromPartName(entry.new_part_name, data.format_version);
1483

1484 1485 1486 1487
    /// 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.
1488 1489 1490
    /// Therefore, we use all data parts.
    auto parts = data.getDataParts({MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated});

1491 1492
    for (const auto & part : parts)
    {
1493
        if (!entry_part_info.contains(part->info))
1494 1495 1496 1497 1498 1499 1500 1501 1502
            continue;

        LOG_DEBUG(log, "Removing part " << part->name);
        ++removed_parts;

        /// If you do not need to delete a part, it's more reliable to move the directory before making changes to ZooKeeper.
        if (entry.detach)
            data.renameAndDetachPart(part);

1503
        zkutil::Requests ops;
1504
        zkutil::Responses responses;
1505
        removePartFromZooKeeper(part->name, ops);
1506
        auto code = getZooKeeper()->tryMulti(ops, responses);
1507 1508 1509

        /// If the part is already removed (for example, because it was never added to ZK due to crash,
        /// see ReplicatedMergeTreeBlockOutputStream), then Ok.
1510
        if (code && code != ZooKeeperImpl::ZooKeeper::ZNONODE)
1511 1512 1513 1514
            throw zkutil::KeeperException(code);

        /// If the part needs to be removed, it is more reliable to delete the directory after the changes in ZooKeeper.
        if (!entry.detach)
1515
            data.removePartsFromWorkingSet({part}, true);
1516 1517 1518
    }

    LOG_INFO(log, (entry.detach ? "Detached " : "Removed ") << removed_parts << " parts inside " << entry.new_part_name << ".");
M
Merge  
Michael Kolupaev 已提交
1519 1520
}

A
Merge  
Alexey Milovidov 已提交
1521

1522
void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry & entry)
1523
{
1524
    LOG_INFO(log, "Clear column " << entry.column_name << " in parts inside " << entry.new_part_name << " range");
1525 1526 1527 1528 1529

    /// Assume optimistic scenario, i.e. conflicts are very rare
    /// So, if conflicts are found, throw an exception and will retry execution later
    queue.disableMergesAndFetchesInRange(entry);

1530
    auto entry_part_info = MergeTreePartInfo::fromPartName(entry.new_part_name, data.format_version);
1531

1532 1533 1534
    /// 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)
1535
    auto lock_read_structure = lockStructure(false, __PRETTY_FUNCTION__);
1536 1537 1538 1539 1540 1541 1542

    auto zookeeper = getZooKeeper();

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

1543
    auto new_columns = getColumns();
1544
    alter_command.apply(new_columns);
1545 1546

    size_t modified_parts = 0;
1547
    auto parts = data.getDataParts();
A
Alexey Zatelepin 已提交
1548
    auto columns_for_parts = new_columns.getAllPhysical();
1549 1550
    for (const auto & part : parts)
    {
1551
        if (!entry_part_info.contains(part->info))
1552 1553
            continue;

1554
        LOG_DEBUG(log, "Clearing column " << entry.column_name << " in part " << part->name);
1555 1556 1557 1558 1559 1560

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

        /// Update part metadata in ZooKeeper.
1561 1562
        zkutil::Requests ops;
        ops.emplace_back(zkutil::makeSetRequest(
1563
            replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1));
1564
        ops.emplace_back(zkutil::makeSetRequest(
1565
            replica_path + "/parts/" + part->name + "/checksums", getChecksumsForZooKeeper(transaction->getNewChecksums()), -1));
1566 1567 1568 1569 1570 1571 1572

        zookeeper->multi(ops);

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

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

1575
    /// Recalculate columns size (not only for the modified column)
1576 1577 1578 1579
    data.recalculateColumnSizes();
}


M
Merge  
Michael Kolupaev 已提交
1580 1581
void StorageReplicatedMergeTree::queueUpdatingThread()
{
1582
    setThreadName("ReplMTQueueUpd");
1583

1584 1585
    bool update_in_progress = false;
    while (!shutdown_called)
S
Silviu Caragea 已提交
1586
    {
1587 1588 1589 1590 1591 1592 1593 1594 1595 1596 1597 1598 1599 1600
        if (!update_in_progress)
        {
            last_queue_update_start_time.store(time(nullptr));
            update_in_progress = true;
        }
        try
        {
            pullLogsToQueue(queue_updating_event);
            last_queue_update_finish_time.store(time(nullptr));
            update_in_progress = false;
            queue_updating_event->wait();
        }
        catch (...)
        {
1601
            tryLogCurrentException(log, __PRETTY_FUNCTION__);
1602 1603
            queue_updating_event->tryWait(QUEUE_UPDATE_ERROR_SLEEP_MS);
        }
S
Silviu Caragea 已提交
1604
    }
1605 1606

    LOG_DEBUG(log, "Queue updating thread finished");
M
Merge  
Michael Kolupaev 已提交
1607
}
M
Merge  
Michael Kolupaev 已提交
1608

A
Merge  
Alexey Milovidov 已提交
1609

1610
bool StorageReplicatedMergeTree::queueTask()
M
Merge  
Michael Kolupaev 已提交
1611
{
1612 1613 1614 1615 1616 1617 1618 1619 1620
    /// This object will mark the element of the queue as running.
    ReplicatedMergeTreeQueue::SelectedEntry selected;

    try
    {
        selected = queue.selectEntryToProcess(merger, data);
    }
    catch (...)
    {
1621
        tryLogCurrentException(log, __PRETTY_FUNCTION__);
1622 1623 1624 1625 1626 1627 1628 1629 1630 1631 1632 1633 1634 1635 1636 1637 1638 1639 1640 1641 1642 1643 1644 1645 1646 1647 1648
    }

    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());
            }
1649 1650 1651 1652 1653
            else if (e.code() == ErrorCodes::PART_IS_TEMPORARILY_LOCKED)
            {
                /// Part cannot be added temporarily
                LOG_INFO(log, e.displayText());
            }
1654
            else
1655
                tryLogCurrentException(log, __PRETTY_FUNCTION__);
1656 1657 1658 1659 1660 1661 1662 1663 1664

            /** 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 (...)
        {
1665
            tryLogCurrentException(log, __PRETTY_FUNCTION__);
1666 1667 1668 1669 1670 1671 1672 1673 1674
            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 已提交
1675 1676
}

A
Merge  
Alexey Milovidov 已提交
1677

1678
namespace
M
Merge  
Michael Kolupaev 已提交
1679
{
1680 1681 1682
    bool canMergePartsAccordingToZooKeeperInfo(
        const MergeTreeData::DataPartPtr & left,
        const MergeTreeData::DataPartPtr & right,
1683
        zkutil::ZooKeeperPtr && zookeeper, const String & zookeeper_path, const MergeTreeData & data, String * out_reason = nullptr)
1684
    {
1685
        const String & partition_id = left->info.partition_id;
1686

1687 1688 1689 1690 1691 1692 1693
        /// You can not merge parts, among which is a part for which the quorum is unsatisfied.
        /// Note: theoretically, this could be resolved. But this will make logic more complex.
        String quorum_node_value;
        if (zookeeper->tryGet(zookeeper_path + "/quorum/status", quorum_node_value))
        {
            ReplicatedMergeTreeQuorumEntry quorum_entry;
            quorum_entry.fromString(quorum_node_value);
1694

1695
            auto part_info = MergeTreePartInfo::fromPartName(quorum_entry.part_name, data.format_version);
1696

1697
            if (part_info.min_block != part_info.max_block)
1698
                throw Exception("Logical error: part written with quorum covers more than one block numbers", ErrorCodes::LOGICAL_ERROR);
1699

1700
            if (left->info.max_block <= part_info.min_block && right->info.min_block >= part_info.max_block)
1701
            {
1702 1703
                if (out_reason)
                    *out_reason = "Quorum status condition is unsatisfied";
1704
                return false;
1705
            }
1706
        }
1707

1708 1709 1710 1711 1712
        /// Won't merge last_part even if quorum is satisfied, because we gonna check if replica has this part
        /// on SELECT execution.
        String quorum_last_part;
        if (zookeeper->tryGet(zookeeper_path + "/quorum/last_part", quorum_last_part) && quorum_last_part.empty() == false)
        {
1713
            auto part_info = MergeTreePartInfo::fromPartName(quorum_last_part, data.format_version);
1714

1715
            if (part_info.min_block != part_info.max_block)
1716
                throw Exception("Logical error: part written with quorum covers more than one block numbers", ErrorCodes::LOGICAL_ERROR);
1717

1718
            if (left->info.max_block <= part_info.min_block && right->info.min_block >= part_info.max_block)
1719
            {
1720
                if (out_reason)
1721
                    *out_reason = "Quorum 'last part' condition is unsatisfied";
1722
                return false;
1723
            }
1724
        }
1725 1726

        /// You can merge the parts, if all the numbers between them are abandoned - do not correspond to any blocks.
1727
        for (Int64 number = left->info.max_block + 1; number <= right->info.min_block - 1; ++number)
1728
        {
1729 1730
            String path1 = zookeeper_path +              "/block_numbers/" + partition_id + "/block-" + padIndex(number);
            String path2 = zookeeper_path + "/nonincrement_block_numbers/" + partition_id + "/block-" + padIndex(number);
1731 1732 1733

            if (AbandonableLockInZooKeeper::check(path1, *zookeeper) != AbandonableLockInZooKeeper::ABANDONED &&
                AbandonableLockInZooKeeper::check(path2, *zookeeper) != AbandonableLockInZooKeeper::ABANDONED)
1734 1735 1736 1737
            {
                if (out_reason)
                    *out_reason = "Block " + toString(number) + " in gap between merging parts " + left->name + " and "
                                  + right->name + " is not abandoned";
1738
                return false;
1739
            }
1740
        }
1741 1742

        return true;
1743
    }
1744

1745

1746
    /// If any of the parts is already going to be merged into a larger one, do not agree to merge it.
1747 1748 1749 1750 1751 1752 1753 1754 1755 1756 1757 1758 1759 1760 1761 1762 1763 1764 1765 1766
    bool partsWillNotBeMergedOrDisabled(const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right,
                                        ReplicatedMergeTreeQueue & queue, String * out_reason = nullptr)
    {
        auto set_reason = [&out_reason] (const String & part_name)
        {
            if (out_reason)
                *out_reason = "Part " + part_name + " cannot be merged yet, a merge has already assigned for it or it is temporarily disabled";
            return false;
        };

        if (queue.partWillBeMergedOrMergesDisabled(left->name))
            return set_reason(left->name);

        if (left.get() != right.get() && queue.partWillBeMergedOrMergesDisabled(right->name))
            return set_reason(right->name);

        return true;
    }


1767 1768 1769 1770 1771 1772 1773 1774 1775 1776 1777
    /** It can take a long time to determine whether it is possible to merge two adjacent parts.
    * Two adjacent parts can be merged if all block numbers between their numbers are not used (abandoned).
    * This means that another part can not be inserted between these parts.
    *
    * But if the numbers of adjacent blocks differ much (usually if there are many "abandoned" blocks between them),
    *  then too many read requests are made to ZooKeeper to find out if it's possible to merge them.
    *
    * Let's use a statement that if a couple of parts were possible to merge, and their merge is not yet planned,
    *  then now they can be merged, and we will remember this state,
    *  not to send multiple identical requests to ZooKeeper.
    */
1778

1779 1780 1781 1782 1783 1784 1785 1786 1787
    /** Cache for function, that returns bool.
    * If function returned true, cache it forever.
    * If function returned false, cache it for exponentially growing time.
    * Not thread safe.
    */
    template <typename Key>
    struct CachedMergingPredicate
    {
        using clock = std::chrono::steady_clock;
1788

1789 1790 1791 1792 1793
        struct Expiration
        {
            static constexpr clock::duration min_delay = std::chrono::seconds(1);
            static constexpr clock::duration max_delay = std::chrono::seconds(600);
            static constexpr double exponent_base = 2;
1794

1795 1796
            clock::time_point expire_time;
            clock::duration delay = clock::duration::zero();
1797

1798 1799 1800 1801 1802 1803 1804 1805 1806 1807
            void next(clock::time_point now)
            {
                if (delay == clock::duration::zero())
                    delay = min_delay;
                else
                {
                    delay *= exponent_base;
                    if (delay > max_delay)
                        delay = max_delay;
                }
1808

1809 1810 1811 1812
                expire_time = now + delay;
            }

            bool expired(clock::time_point now) const
1813
            {
1814
                return now > expire_time;
1815
            }
1816
        };
1817

1818 1819
        std::set<Key> true_keys;
        std::map<Key, Expiration> false_keys;
1820

1821 1822
        template <typename Function, typename ArgsToKey, typename... Args>
        bool get(clock::time_point now, Function && function, ArgsToKey && args_to_key, Args &&... args)
1823
        {
1824
            Key key{args_to_key(std::forward<Args>(args)...)};
1825

1826 1827
            if (true_keys.count(key))
                return true;
1828

1829 1830 1831
            auto it = false_keys.find(key);
            if (false_keys.end() != it && !it->second.expired(now))
                return false;
1832

1833
            bool value = function(std::forward<Args>(args)...);
1834

1835 1836 1837 1838
            if (value)
                true_keys.insert(key);
            else
                false_keys[key].next(now);
1839

1840 1841 1842
            return value;
        }
    };
1843

1844 1845 1846
    template <typename Key> constexpr CachedMergingPredicate<Key>::clock::duration CachedMergingPredicate<Key>::Expiration::min_delay;
    template <typename Key> constexpr CachedMergingPredicate<Key>::clock::duration CachedMergingPredicate<Key>::Expiration::max_delay;
    template <typename Key> constexpr double CachedMergingPredicate<Key>::Expiration::exponent_base;
1847
}
1848

1849

1850
void StorageReplicatedMergeTree::mergeSelectingThread()
1851
{
1852 1853 1854 1855
    setThreadName("ReplMTMergeSel");
    LOG_DEBUG(log, "Merge selecting thread started");

    bool deduplicate = false; /// TODO: read deduplicate option from table config
1856

1857
    auto uncached_merging_predicate = [this](const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right)
1858
    {
1859
        return canMergePartsAccordingToZooKeeperInfo(left, right, getZooKeeper(), zookeeper_path, data);
1860
    };
1861

1862
    auto merging_predicate_args_to_key = [](const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right)
1863
    {
1864 1865 1866
        return std::make_pair(left->name, right->name);
    };

1867
    CachedMergingPredicate<std::pair<std::string, std::string>> cached_merging_predicate;
1868 1869

    /// Will be updated below.
1870
    std::chrono::steady_clock::time_point now;
1871

1872
    auto can_merge = [&] (const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, String *)
1873
    {
1874 1875
        return partsWillNotBeMergedOrDisabled(left, right, queue)
               && cached_merging_predicate.get(now, uncached_merging_predicate, merging_predicate_args_to_key, left, right);
1876
    };
S
Silviu Caragea 已提交
1877

1878
    while (is_leader)
S
Silviu Caragea 已提交
1879
    {
1880 1881 1882
        bool success = false;

        try
1883
        {
1884
            std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
1885

1886 1887 1888
            /// You need to load new entries into the queue before you select parts to merge.
            ///  (so we know which parts are already going to be merged).
            /// We must select parts for merge under the mutex because other threads (OPTIMIZE queries) could push new merges.
1889
            if (merge_selecting_logs_pulling_is_required)
1890 1891
            {
                pullLogsToQueue();
1892
                merge_selecting_logs_pulling_is_required = false;
1893
            }
1894

1895 1896 1897
            /// 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.
1898
            size_t merges_queued = queue.countMerges();
1899

1900 1901 1902 1903 1904 1905 1906 1907 1908
            if (merges_queued >= data.settings.max_replicated_merges_in_queue)
            {
                LOG_TRACE(log, "Number of queued merges (" << merges_queued
                    << ") is greater than max_replicated_merges_in_queue ("
                    << data.settings.max_replicated_merges_in_queue << "), so won't select new parts to merge.");
            }
            else
            {
                MergeTreeDataMerger::FuturePart future_merged_part;
1909

1910
                size_t max_parts_size_for_merge = merger.getMaxPartsSizeForMerge(data.settings.max_replicated_merges_in_queue, merges_queued);
1911

1912
                now = std::chrono::steady_clock::now();
1913

1914
                if (max_parts_size_for_merge > 0
1915
                    && merger.selectPartsToMerge(future_merged_part, false, max_parts_size_for_merge, can_merge))
1916
                {
1917 1918
                    merge_selecting_logs_pulling_is_required = true;
                    success = createLogEntryToMergeParts(future_merged_part.parts, future_merged_part.name, deduplicate);
1919
                }
1920 1921
            }
        }
1922 1923
        catch (...)
        {
1924
            tryLogCurrentException(log, __PRETTY_FUNCTION__);
1925
        }
1926

1927
        if (!is_leader)
1928
            break;
S
Silviu Caragea 已提交
1929

1930 1931 1932 1933 1934
        if (!success)
            merge_selecting_event.tryWait(MERGE_SELECTING_SLEEP_MS);
    }

    LOG_DEBUG(log, "Merge selecting thread finished");
M
Merge  
Michael Kolupaev 已提交
1935 1936
}

M
Merge  
Michael Kolupaev 已提交
1937

1938
bool StorageReplicatedMergeTree::createLogEntryToMergeParts(
Y
Yuri Dyachenko 已提交
1939
    const MergeTreeData::DataPartsVector & parts, const String & merged_name, bool deduplicate, ReplicatedMergeTreeLogEntryData * out_log_entry)
1940
{
1941 1942 1943 1944 1945 1946 1947 1948 1949 1950
    auto zookeeper = getZooKeeper();

    bool all_in_zk = true;
    for (const auto & part : parts)
    {
        /// If there is no information about part in ZK, we will not merge it.
        if (!zookeeper->exists(replica_path + "/parts/" + part->name))
        {
            all_in_zk = false;

1951
            if (part->modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER < time(nullptr))
1952 1953
            {
                LOG_WARNING(log, "Part " << part->name << " (that was selected for merge)"
1954
                    << " with age " << (time(nullptr) - part->modification_time)
1955 1956 1957 1958 1959 1960 1961 1962 1963
                    << " seconds exists locally but not in ZooKeeper."
                    << " Won't do merge with that part and will check it.");
                enqueuePartForCheck(part->name);
            }
        }
    }
    if (!all_in_zk)
        return false;

1964
    ReplicatedMergeTreeLogEntryData entry;
1965 1966 1967
    entry.type = LogEntry::MERGE_PARTS;
    entry.source_replica = replica_name;
    entry.new_part_name = merged_name;
Y
Yuri Dyachenko 已提交
1968
    entry.deduplicate = deduplicate;
1969
    entry.create_time = time(nullptr);
1970 1971 1972 1973 1974 1975 1976

    for (const auto & part : parts)
        entry.parts_to_merge.push_back(part->name);

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

1977
    const String & partition_id = parts[0]->info.partition_id;
1978 1979 1980
    for (size_t i = 0; i + 1 < parts.size(); ++i)
    {
        /// Remove the unnecessary entries about non-existent blocks.
1981
        for (Int64 number = parts[i]->info.max_block + 1; number <= parts[i + 1]->info.min_block - 1; ++number)
1982
        {
1983
            zookeeper->tryRemove(zookeeper_path + "/block_numbers/" + partition_id + "/block-" + padIndex(number));
1984
            zookeeper->tryRemove(zookeeper_path + "/nonincrement_block_numbers/" + partition_id + "/block-" + padIndex(number));
1985 1986 1987 1988 1989 1990 1991
        }
    }

    if (out_log_entry)
        *out_log_entry = entry;

    return true;
1992 1993 1994
}


1995
void StorageReplicatedMergeTree::removePartFromZooKeeper(const String & part_name, zkutil::Requests & ops)
1996
{
1997
    String part_path = replica_path + "/parts/" + part_name;
1998

1999 2000 2001
    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));
2002 2003 2004
}


M
Merge  
Michael Kolupaev 已提交
2005 2006
void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_name)
{
2007
    auto zookeeper = getZooKeeper();
A
Merge  
Alexey Milovidov 已提交
2008

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

2011 2012 2013 2014 2015
    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 已提交
2016

2017 2018 2019
    zkutil::Requests ops;
    ops.emplace_back(zkutil::makeCreateRequest(
        replica_path + "/queue/queue-", log_entry->toString(),
2020
        zkutil::CreateMode::PersistentSequential));
2021

2022
    removePartFromZooKeeper(part_name, ops);
2023

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

2026
    String path_created = dynamic_cast<const zkutil::CreateResponse &>(*results[0]).path_created;
2027 2028
    log_entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
    queue.insert(zookeeper, log_entry);
M
Merge  
Michael Kolupaev 已提交
2029 2030
}

A
Merge  
Alexey Milovidov 已提交
2031

2032
void StorageReplicatedMergeTree::enterLeaderElection()
M
Merge  
Michael Kolupaev 已提交
2033
{
2034 2035 2036 2037 2038 2039 2040 2041
    auto callback = [this]()
    {
        CurrentMetrics::add(CurrentMetrics::LeaderReplica);
        LOG_INFO(log, "Became leader");

        is_leader = true;
        merge_selecting_thread = std::thread(&StorageReplicatedMergeTree::mergeSelectingThread, this);
    };
2042

2043 2044 2045 2046 2047 2048 2049 2050 2051 2052 2053 2054 2055 2056 2057 2058 2059 2060 2061
    try
    {
        leader_election = std::make_shared<zkutil::LeaderElection>(
            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)
2062
        return;
2063

2064 2065 2066 2067 2068
    /// 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)
2069
    {
2070 2071 2072 2073 2074
        CurrentMetrics::sub(CurrentMetrics::LeaderReplica);
        LOG_INFO(log, "Stopped being leader");

        is_leader = false;
        merge_selecting_event.set();
2075 2076 2077
        merge_selecting_thread.join();
    }

2078 2079 2080
    /// 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 已提交
2081 2082
}

A
Merge  
Alexey Milovidov 已提交
2083

M
Merge  
Michael Kolupaev 已提交
2084
String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_name, bool active)
M
Merge  
Michael Kolupaev 已提交
2085
{
2086 2087
    auto zookeeper = getZooKeeper();
    Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
M
Merge  
Michael Kolupaev 已提交
2088

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

2092 2093 2094 2095 2096
    for (const String & replica : replicas)
    {
        /// We don't interested in ourself.
        if (replica == replica_name)
            continue;
A
Alexey Milovidov 已提交
2097

2098 2099 2100
        if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name) &&
            (!active || zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")))
            return replica;
2101

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

2105
    return {};
2106 2107 2108
}


2109
String StorageReplicatedMergeTree::findReplicaHavingCoveringPart(const LogEntry & entry, bool active)
2110
{
2111 2112 2113 2114
    auto zookeeper = getZooKeeper();
    Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");

    /// Select replicas in uniformly random order.
2115
    std::shuffle(replicas.begin(), replicas.end(), rng);
2116 2117 2118 2119 2120 2121 2122 2123 2124 2125 2126 2127 2128

    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)
        {
2129 2130
            if (part_on_replica == entry.new_part_name
                || MergeTreePartInfo::contains(part_on_replica, entry.new_part_name, data.format_version))
2131
            {
2132 2133
                if (largest_part_found.empty()
                    || MergeTreePartInfo::contains(part_on_replica, largest_part_found, data.format_version))
2134 2135 2136 2137 2138 2139 2140 2141
                {
                    largest_part_found = part_on_replica;
                }
            }
        }

        if (!largest_part_found.empty())
        {
2142 2143 2144 2145 2146 2147 2148 2149 2150 2151 2152 2153 2154 2155 2156 2157 2158
            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;
            }

2159 2160 2161 2162 2163
            return replica;
        }
    }

    return {};
M
Merge  
Michael Kolupaev 已提交
2164 2165
}

A
Merge  
Alexey Milovidov 已提交
2166

F
f1yegor 已提交
2167
/** If a quorum is tracked for a part, update information about it in ZK.
2168
  */
2169
void StorageReplicatedMergeTree::updateQuorum(const String & part_name)
2170
{
2171 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
    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;
    zkutil::Stat stat;

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

2199
            zkutil::Requests ops;
2200
            zkutil::Responses responses;
2201 2202
            ops.emplace_back(zkutil::makeRemoveRequest(quorum_status_path, stat.version));
            ops.emplace_back(zkutil::makeSetRequest(quorum_last_part_path, part_name, -1));
2203
            auto code = zookeeper->tryMulti(ops, responses);
2204

2205
            if (code == ZooKeeperImpl::ZooKeeper::ZOK)
2206 2207 2208
            {
                break;
            }
2209
            else if (code == ZooKeeperImpl::ZooKeeper::ZNONODE)
2210 2211 2212 2213
            {
                /// The quorum has already been achieved.
                break;
            }
2214
            else if (code == ZooKeeperImpl::ZooKeeper::ZBADVERSION)
2215 2216 2217 2218 2219 2220 2221 2222 2223 2224 2225 2226
            {
                /// Node was updated meanwhile. We must re-read it and repeat all the actions.
                continue;
            }
            else
                throw zkutil::KeeperException(code, quorum_status_path);
        }
        else
        {
            /// We update the node, registering there one more replica.
            auto code = zookeeper->trySet(quorum_status_path, quorum_entry.toString(), stat.version);

2227
            if (code == ZooKeeperImpl::ZooKeeper::ZOK)
2228 2229 2230
            {
                break;
            }
2231
            else if (code == ZooKeeperImpl::ZooKeeper::ZNONODE)
2232 2233 2234 2235
            {
                /// The quorum has already been achieved.
                break;
            }
2236
            else if (code == ZooKeeperImpl::ZooKeeper::ZBADVERSION)
2237 2238 2239 2240 2241 2242 2243 2244
            {
                /// Node was updated meanwhile. We must re-read it and repeat all the actions.
                continue;
            }
            else
                throw zkutil::KeeperException(code, quorum_status_path);
        }
    }
2245 2246 2247
}


A
Alexey Milovidov 已提交
2248
bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const String & replica_path, bool to_detached, size_t quorum)
M
Merge  
Michael Kolupaev 已提交
2249
{
2250 2251 2252
    if (auto part = data.getPartIfExists(part_name, {MergeTreeDataPart::State::Outdated, MergeTreeDataPart::State::Deleting}))
    {
        LOG_DEBUG(log, "Part " << part->getNameWithState() << " should be deleted after previous attempt before fetch");
2253
        /// Force immediate parts cleanup to delete the part that was left from the previous fetch attempt.
2254
        cleanup_thread_event.set();
2255 2256 2257
        return false;
    }

2258 2259 2260 2261 2262 2263 2264 2265 2266 2267 2268 2269 2270 2271 2272 2273 2274 2275 2276
    {
        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)
2277
        table_lock = lockStructure(true, __PRETTY_FUNCTION__);
2278 2279

    ReplicatedMergeTreeAddress address(getZooKeeper()->get(replica_path + "/host"));
2280
    auto timeouts = ConnectionTimeouts::getHTTPTimeouts(context.getSettingsRef());
2281

2282
    /// Logging
2283
    Stopwatch stopwatch;
2284 2285
    MergeTreeData::MutableDataPartPtr part;
    MergeTreeData::DataPartsVector replaced_parts;
2286

2287
    auto write_part_log = [&] (const ExecutionStatus & execution_status)
2288
    {
2289 2290
        try
        {
2291
            auto part_log = context.getPartLog(database_name);
2292 2293
            if (!part_log)
                return;
2294

2295
            PartLogElement part_log_elem;
2296

2297 2298 2299 2300
            part_log_elem.event_time = time(nullptr);
            part_log_elem.event_type = PartLogElement::DOWNLOAD_PART;
            /// TODO: Stop stopwatch in outer code to exclude ZK timings and so on
            part_log_elem.duration_ms = stopwatch.elapsed() / 10000000;
2301

2302 2303 2304 2305 2306 2307
            part_log_elem.database_name = database_name;
            part_log_elem.table_name = table_name;
            part_log_elem.part_name = part_name;

            if (part)
            {
2308
                part_log_elem.bytes_compressed_on_disk = part->bytes_on_disk;
2309 2310 2311 2312 2313 2314 2315 2316 2317 2318 2319 2320 2321
                part_log_elem.rows = part->rows_count; /// Could be approximate (?)
            }

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

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

            part_log->add(part_log_elem);
        }
        catch (...)
2322
        {
2323
            tryLogCurrentException(log, __PRETTY_FUNCTION__);
2324
        }
2325 2326 2327 2328 2329
    };

    try
    {
        part = fetcher.fetchPart(part_name, replica_path, address.host, address.replication_port, timeouts, to_detached);
2330

2331
        if (!to_detached)
2332
        {
2333 2334
            MergeTreeData::Transaction transaction;
            data.renameTempPartAndReplace(part, nullptr, &transaction);
2335

2336 2337 2338 2339 2340
            /** 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.
              */
2341
            replaced_parts = checkPartChecksumsAndCommit(transaction, part);
2342

2343 2344 2345 2346 2347
            /** 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);
2348

2349 2350 2351 2352 2353 2354 2355
            merge_selecting_event.set();

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

2357 2358 2359
            write_part_log({});
        }
        else
2360
        {
2361
            part->renameTo("detached/" + part_name);
2362 2363
        }
    }
2364
    catch (...)
2365
    {
2366 2367 2368 2369
        if (!to_detached)
            write_part_log(ExecutionStatus::fromCurrentException());

        throw;
2370 2371 2372 2373 2374 2375
    }

    ProfileEvents::increment(ProfileEvents::ReplicatedPartFetches);

    LOG_DEBUG(log, "Fetched part " << part_name << " from " << replica_path << (to_detached ? " (to 'detached' directory)" : ""));
    return true;
M
Merge  
Michael Kolupaev 已提交
2376
}
M
Merge  
Michael Kolupaev 已提交
2377

A
Merge  
Alexey Milovidov 已提交
2378

2379 2380 2381 2382 2383 2384 2385 2386 2387 2388 2389
void StorageReplicatedMergeTree::startup()
{
    if (is_readonly)
        return;

    queue.initialize(
        zookeeper_path, replica_path,
        database_name + "." + table_name + " (ReplicatedMergeTreeQueue)",
        data.getDataParts(), current_zookeeper);

    queue.pullLogsToQueue(current_zookeeper, nullptr);
2390 2391 2392
    last_queue_update_finish_time.store(time(nullptr));
    /// NOTE: not updating last_queue_update_start_time because it must contain the time when
    /// the notification of queue change was received. In the beginning it is effectively infinite.
2393

A
Alexey Zatelepin 已提交
2394 2395 2396 2397 2398
    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());

2399 2400
    /// In this thread replica will be activated.
    restarting_thread = std::make_unique<ReplicatedMergeTreeRestartingThread>(*this);
2401 2402 2403

    /// Wait while restarting_thread initializes LeaderElection (and so on) or makes first attmept to do it
    startup_event.wait();
2404 2405 2406
}


M
Merge  
Michael Kolupaev 已提交
2407 2408
void StorageReplicatedMergeTree::shutdown()
{
2409 2410 2411 2412 2413 2414
    if (restarting_thread)
    {
        restarting_thread->stop();
        restarting_thread.reset();
    }

2415
    if (data_parts_exchange_endpoint_holder)
2416
    {
2417 2418
        data_parts_exchange_endpoint_holder->cancelForever();
        data_parts_exchange_endpoint_holder = nullptr;
2419
    }
2420 2421

    fetcher.blocker.cancelForever();
M
Merge  
Michael Kolupaev 已提交
2422 2423 2424
}


M
Merge  
Michael Kolupaev 已提交
2425 2426
StorageReplicatedMergeTree::~StorageReplicatedMergeTree()
{
2427 2428 2429 2430 2431 2432 2433 2434
    try
    {
        shutdown();
    }
    catch(...)
    {
        tryLogCurrentException(__PRETTY_FUNCTION__);
    }
M
Merge  
Michael Kolupaev 已提交
2435 2436
}

A
Merge  
Alexey Milovidov 已提交
2437

M
Merge  
Michael Kolupaev 已提交
2438
BlockInputStreams StorageReplicatedMergeTree::read(
2439
    const Names & column_names,
2440
    const SelectQueryInfo & query_info,
2441 2442 2443
    const Context & context,
    QueryProcessingStage::Enum & processed_stage,
    const size_t max_block_size,
2444
    const unsigned num_streams)
M
Merge  
Michael Kolupaev 已提交
2445
{
2446
    const Settings & settings = context.getSettingsRef();
2447

2448 2449 2450 2451 2452 2453 2454
    /** 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)
2455
    {
2456
        auto zookeeper = getZooKeeper();
2457

2458 2459
        String last_part;
        zookeeper->tryGet(zookeeper_path + "/quorum/last_part", last_part);
2460

2461
        if (!last_part.empty() && !data.getActiveContainingPart(last_part))    /// TODO Disable replica for distributed queries.
2462 2463
            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);
2464

2465 2466 2467 2468
        if (last_part.empty())  /// If no part has been written with quorum.
        {
            String quorum_str;
            if (zookeeper->tryGet(zookeeper_path + "/quorum/status", quorum_str))
2469
            {
2470 2471
                ReplicatedMergeTreeQuorumEntry quorum_entry;
                quorum_entry.fromString(quorum_str);
2472
                auto part_info = MergeTreePartInfo::fromPartName(quorum_entry.part_name, data.format_version);
2473
                max_block_number_to_read = part_info.min_block - 1;
2474 2475
            }
        }
2476
        else
2477
        {
2478
            auto part_info = MergeTreePartInfo::fromPartName(last_part, data.format_version);
2479
            max_block_number_to_read = part_info.max_block;
2480 2481 2482
        }
    }

2483
    return reader.read(
A
Alexey Zatelepin 已提交
2484
        column_names, query_info, context, processed_stage, max_block_size, num_streams, max_block_number_to_read);
M
Merge  
Michael Kolupaev 已提交
2485 2486
}

A
Merge  
Alexey Milovidov 已提交
2487

A
Merge  
Alexey Milovidov 已提交
2488
void StorageReplicatedMergeTree::assertNotReadonly() const
M
Merge  
Michael Kolupaev 已提交
2489
{
2490 2491
    if (is_readonly)
        throw Exception("Table is in readonly mode", ErrorCodes::TABLE_IS_READ_ONLY);
A
Merge  
Alexey Milovidov 已提交
2492 2493 2494
}


A
Alexey Milovidov 已提交
2495
BlockOutputStreamPtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const Settings & settings)
A
Merge  
Alexey Milovidov 已提交
2496
{
2497
    assertNotReadonly();
M
Merge  
Michael Kolupaev 已提交
2498

2499 2500
    bool deduplicate = data.settings.replicated_deduplication_window != 0 && settings.insert_deduplicate;

2501
    return std::make_shared<ReplicatedMergeTreeBlockOutputStream>(*this,
2502
        settings.insert_quorum, settings.insert_quorum_timeout.totalMilliseconds(), deduplicate);
M
Merge  
Michael Kolupaev 已提交
2503
}
M
Merge  
Michael Kolupaev 已提交
2504

A
Merge  
Alexey Milovidov 已提交
2505

2506
bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context)
M
Merge  
Michael Kolupaev 已提交
2507
{
2508
    assertNotReadonly();
2509

2510
    if (!is_leader)
2511
    {
2512
        sendRequestToLeaderReplica(query, context.getSettingsRef());
2513 2514
        return true;
    }
2515

2516
    auto can_merge = [this] (const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, String * out_reason)
2517
    {
2518 2519
        return partsWillNotBeMergedOrDisabled(left, right, queue, out_reason)
               && canMergePartsAccordingToZooKeeperInfo(left, right, getZooKeeper(), zookeeper_path, data, out_reason);
2520
    };
2521

2522 2523 2524
    ReplicatedMergeTreeLogEntryData merge_entry;
    {
        std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
2525

2526 2527 2528
        /// We must select parts for merge under the mutex because other threads (OPTIMIZE queries) could push new merges.
        pullLogsToQueue();

2529
        size_t disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
M
Merge  
Michael Kolupaev 已提交
2530

A
Alexey Zatelepin 已提交
2531
        MergeTreeDataMerger::FuturePart future_merged_part;
2532
        String disable_reason;
2533
        bool selected = false;
2534

2535
        if (!partition)
2536
        {
A
Alexey Zatelepin 已提交
2537
            selected = merger.selectPartsToMerge(
2538
                future_merged_part, true, data.settings.max_bytes_to_merge_at_max_space_in_pool, can_merge, &disable_reason);
2539 2540 2541
        }
        else
        {
2542
            String partition_id = data.getPartitionIDFromQuery(partition, context);
2543
            selected = merger.selectAllPartsToMergeWithinPartition(future_merged_part, disk_space, can_merge, partition_id, final, &disable_reason);
2544
        }
2545

2546
        auto handle_noop = [&] (const String & message)
2547
        {
2548 2549
            if (context.getSettingsRef().optimize_throw_if_noop)
                throw Exception(message, ErrorCodes::CANNOT_ASSIGN_OPTIMIZE);
2550
            return false;
2551 2552 2553 2554 2555 2556
        };

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

2559 2560 2561
        /// It is important to pull new logs (even if creation of the entry fails due to network error)
        merge_selecting_logs_pulling_is_required = true;

A
Alexey Zatelepin 已提交
2562
        if (!createLogEntryToMergeParts(future_merged_part.parts, future_merged_part.name, deduplicate, &merge_entry))
2563
            return handle_noop("Can't create merge queue node in ZooKeeper");
2564
    }
2565

2566 2567
    waitForAllReplicasToProcessLogEntry(merge_entry);
    return true;
M
Merge  
Michael Kolupaev 已提交
2568 2569
}

A
Merge  
Alexey Milovidov 已提交
2570

M
Merge  
Michael Kolupaev 已提交
2571
void StorageReplicatedMergeTree::alter(const AlterCommands & params,
A
Alexey Milovidov 已提交
2572
    const String & /*database_name*/, const String & /*table_name*/, const Context & context)
M
Merge  
Michael Kolupaev 已提交
2573
{
2574
    assertNotReadonly();
A
Merge  
Alexey Milovidov 已提交
2575

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

2578
    int new_columns_version = -1;   /// Initialization is to suppress (useless) false positive warning found by cppcheck.
2579 2580
    String new_columns_str;
    zkutil::Stat stat;
M
Merge  
Michael Kolupaev 已提交
2581

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

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

2589
        data.checkAlter(params);
M
Merge  
Michael Kolupaev 已提交
2590

2591 2592 2593 2594
        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);

2595
        ColumnsDescription new_columns = data.getColumns();
2596
        params.apply(new_columns);
2597

2598
        new_columns_str = new_columns.toString();
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 2633 2634 2635 2636 2637 2638

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

        while (!shutdown_called)
        {
            /// 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.");
2639

2640 2641 2642
                inactive_replicas.emplace(replica);
                break;
            }
2643

2644
            String replica_columns_str;
M
Merge  
Michael Kolupaev 已提交
2645

2646 2647 2648 2649 2650 2651
            /// 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 已提交
2652

2653
            int replica_columns_version = stat.version;
M
Merge  
Michael Kolupaev 已提交
2654

2655 2656 2657
            /// The ALTER has been successfully applied.
            if (replica_columns_str == new_columns_str)
                break;
M
Merge  
Michael Kolupaev 已提交
2658

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

2662 2663 2664 2665 2666 2667
            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 已提交
2668

2669 2670 2671 2672 2673 2674 2675 2676 2677 2678 2679 2680 2681 2682 2683 2684 2685 2686 2687 2688 2689 2690 2691 2692 2693 2694 2695 2696 2697 2698 2699 2700 2701 2702 2703 2704 2705 2706 2707 2708 2709 2710 2711 2712 2713 2714 2715 2716 2717 2718 2719 2720 2721 2722 2723 2724 2725 2726 2727 2728 2729 2730 2731 2732 2733
            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;
            }
        }

        if (shutdown_called)
            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 已提交
2734 2735
}

M
Merge  
Michael Kolupaev 已提交
2736

2737
/// The name of an imaginary part covering all possible parts in the specified partition with numbers in the range from zero to specified right bound.
2738 2739
static String getFakePartNameCoveringPartRange(
        MergeTreeDataFormatVersion format_version, const String & partition_id, UInt64 left, UInt64 right)
M
Merge  
Michael Kolupaev 已提交
2740
{
2741
    /// Artificial high level is choosen, to make this part "covering" all parts inside.
2742
    MergeTreePartInfo part_info(partition_id, left, right, 999999999);
2743
    if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
2744 2745 2746 2747 2748 2749 2750 2751 2752 2753
    {
        /// The date range is all month long.
        const auto & lut = DateLUT::instance();
        time_t start_time = lut.YYYYMMDDToDate(parse<UInt32>(partition_id + "01"));
        DayNum_t left_date = lut.toDayNum(start_time);
        DayNum_t right_date = DayNum_t(static_cast<size_t>(left_date) + lut.daysInMonth(start_time) - 1);
        return part_info.getPartNameV0(left_date, right_date);
    }
    else
        return part_info.getPartName();
M
Merge  
Michael Kolupaev 已提交
2754 2755
}

A
Merge  
Alexey Milovidov 已提交
2756

2757 2758
String StorageReplicatedMergeTree::getFakePartNameCoveringAllPartsInPartition(
    const String & partition_id, Int64 * out_min_block, Int64 * out_max_block)
A
Merge  
Andrey Mironov 已提交
2759
{
2760 2761
    /// 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.
2762
    Int64 left = 0;
2763

2764
    /** Let's skip one number in `block_numbers` for the partition being deleted, and we will only delete parts until this number.
2765 2766 2767 2768 2769 2770 2771 2772
      * 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;

    {
2773
        auto zookeeper = getZooKeeper();
2774
        AbandonableLockInZooKeeper block_number_lock = allocateBlockNumber(partition_id, zookeeper);
2775 2776 2777 2778
        right = block_number_lock.getNumber();
        block_number_lock.unlock();
    }

2779
    /// Empty partition.
2780
    if (right == 0)
2781
        return {};
2782

2783
    --right;
2784 2785 2786 2787 2788

    if (out_min_block)
        *out_min_block = left;
    if (out_max_block)
        *out_max_block = right;
2789
    return getFakePartNameCoveringPartRange(data.format_version, partition_id, left, right);
2790 2791 2792
}


2793
void StorageReplicatedMergeTree::clearColumnInPartition(
2794
    const ASTPtr & partition, const Field & column_name, const Context & context)
2795 2796 2797 2798 2799
{
    assertNotReadonly();

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

2800
    String partition_id = data.getPartitionIDFromQuery(partition, context);
2801
    String fake_part_name = getFakePartNameCoveringAllPartsInPartition(partition_id);
2802

2803 2804
    if (fake_part_name.empty())
    {
2805
        LOG_INFO(log, "Will not clear partition " << partition_id << ", it is empty.");
2806 2807 2808
        return;
    }

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

2811
    LogEntry entry;
2812
    entry.type = LogEntry::CLEAR_COLUMN;
2813 2814
    entry.new_part_name = fake_part_name;
    entry.column_name = column_name.safeGet<String>();
2815
    entry.create_time = time(nullptr);
2816 2817 2818 2819 2820

    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.
2821
    if (context.getSettingsRef().replication_alter_partitions_sync != 0)
2822
    {
2823
        if (context.getSettingsRef().replication_alter_partitions_sync == 1)
2824 2825 2826 2827 2828 2829
            waitForReplicaToProcessLogEntry(replica_name, entry);
        else
            waitForAllReplicasToProcessLogEntry(entry);
    }
}

2830
void StorageReplicatedMergeTree::dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & context)
2831 2832 2833
{
    assertNotReadonly();

2834 2835
    zkutil::ZooKeeperPtr zookeeper = getZooKeeper();

2836
    if (!is_leader)
2837
    {
2838
        sendRequestToLeaderReplica(query, context.getSettingsRef());
2839 2840 2841
        return;
    }

2842
    String partition_id = data.getPartitionIDFromQuery(partition, context);
2843 2844 2845 2846

    Int64 min_block = 0;
    Int64 max_block = 0;
    String fake_part_name = getFakePartNameCoveringAllPartsInPartition(partition_id, &min_block, &max_block);
2847

2848 2849
    if (fake_part_name.empty())
    {
2850
        LOG_INFO(log, "Will not drop partition " << partition_id << ", it is empty.");
2851 2852 2853
        return;
    }

2854 2855
    clearBlocksInPartition(*zookeeper, partition_id, min_block, max_block);

2856
    /** Forbid to choose the parts to be deleted for merging.
F
f1yegor 已提交
2857
      * Invariant: after the `DROP_RANGE` entry appears in the log, merge of deleted parts will not appear in the log.
2858 2859 2860 2861 2862 2863
      */
    {
        std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
        queue.disableMergesInRange(fake_part_name);
    }

2864
    LOG_DEBUG(log, "Disabled merges covered by range " << fake_part_name);
2865 2866 2867 2868 2869 2870 2871

    /// Finally, having achieved the necessary invariants, you can put an entry in the log.
    LogEntry entry;
    entry.type = LogEntry::DROP_RANGE;
    entry.source_replica = replica_name;
    entry.new_part_name = fake_part_name;
    entry.detach = detach;
2872
    entry.create_time = time(nullptr);
2873

2874
    String log_znode_path = zookeeper->create(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential);
2875 2876 2877
    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.
2878
    if (context.getSettingsRef().replication_alter_partitions_sync != 0)
2879
    {
2880
        if (context.getSettingsRef().replication_alter_partitions_sync == 1)
2881 2882 2883 2884
            waitForReplicaToProcessLogEntry(replica_name, entry);
        else
            waitForAllReplicasToProcessLogEntry(entry);
    }
A
Merge  
Alexey Milovidov 已提交
2885
}
A
Merge  
Alexey Milovidov 已提交
2886

A
Merge  
Alexey Arno 已提交
2887

2888
void StorageReplicatedMergeTree::attachPartition(const ASTPtr & partition, bool attach_part, const Context & context)
M
Merge  
Michael Kolupaev 已提交
2889
{
2890 2891
    assertNotReadonly();

2892
    String partition_id;
2893 2894

    if (attach_part)
2895
        partition_id = typeid_cast<const ASTLiteral &>(*partition).value.safeGet<String>();
2896
    else
2897
        partition_id = data.getPartitionIDFromQuery(partition, context);
2898

2899
    String source_dir = "detached/";
2900 2901 2902 2903 2904

    /// Let's compose a list of parts that should be added.
    Strings parts;
    if (attach_part)
    {
2905
        parts.push_back(partition_id);
2906 2907 2908
    }
    else
    {
2909
        LOG_DEBUG(log, "Looking for parts for partition " << partition_id << " in " << source_dir);
2910
        ActiveDataPartSet active_parts(data.format_version);
2911 2912 2913 2914 2915

        std::set<String> part_names;
        for (Poco::DirectoryIterator it = Poco::DirectoryIterator(full_path + source_dir); it != Poco::DirectoryIterator(); ++it)
        {
            String name = it.name();
2916
            MergeTreePartInfo part_info;
2917
            if (!MergeTreePartInfo::tryParsePartName(name, &part_info, data.format_version))
2918
                continue;
2919
            if (part_info.partition_id != partition_id)
2920 2921 2922 2923 2924 2925 2926 2927 2928 2929 2930 2931 2932 2933 2934 2935 2936
                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);
        }
    }

2937
    /// Synchronously check that added parts exist and are not broken. We will write checksums.txt if it does not exist.
2938
    LOG_DEBUG(log, "Checking parts");
2939
    std::vector<MergeTreeData::MutableDataPartPtr> loaded_parts;
2940 2941 2942
    for (const String & part : parts)
    {
        LOG_DEBUG(log, "Checking part " << part);
2943
        loaded_parts.push_back(data.loadPartAndFixMetadata(source_dir + part));
2944 2945
    }

2946
    ReplicatedMergeTreeBlockOutputStream output(*this, 0, 0, false);   /// TODO Allow to use quorum here.
2947
    for (auto & part : loaded_parts)
2948
    {
2949 2950 2951
        String old_name = part->name;
        output.writeExistingPart(part);
        LOG_DEBUG(log, "Attached part " << old_name << " as " << part->name);
2952
    }
M
Merge  
Michael Kolupaev 已提交
2953 2954
}

2955

2956 2957
bool StorageReplicatedMergeTree::checkTableCanBeDropped() const
{
2958
    /// Consider only synchronized data
2959
    const_cast<MergeTreeData &>(getData()).recalculateColumnSizes();
2960
    context.checkTableCanBeDropped(database_name, table_name, getData().getTotalActiveSizeInBytes());
2961
    return true;
2962
}
A
Merge  
Alexey Milovidov 已提交
2963

2964

M
Merge  
Michael Kolupaev 已提交
2965 2966
void StorageReplicatedMergeTree::drop()
{
2967 2968
    {
        auto zookeeper = tryGetZooKeeper();
A
Merge  
Alexey Milovidov 已提交
2969

2970 2971
        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 已提交
2972

2973
        // checkTableCanBeDropped(); // uncomment to feel yourself safe
2974

2975
        shutdown();
M
Merge  
Michael Kolupaev 已提交
2976

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

2980 2981 2982
        LOG_INFO(log, "Removing replica " << replica_path);
        replica_is_active_node = nullptr;
        zookeeper->tryRemoveRecursive(replica_path);
M
Merge  
Michael Kolupaev 已提交
2983

2984 2985
        /// Check that `zookeeper_path` exists: it could have been deleted by another replica after execution of previous line.
        Strings replicas;
2986
        if (zookeeper->tryGetChildren(zookeeper_path + "/replicas", replicas) == ZooKeeperImpl::ZooKeeper::ZOK && replicas.empty())
2987 2988 2989 2990 2991
        {
            LOG_INFO(log, "Removing table " << zookeeper_path << " (this might take several minutes)");
            zookeeper->tryRemoveRecursive(zookeeper_path);
        }
    }
M
Merge  
Michael Kolupaev 已提交
2992

2993
    data.dropAllData();
M
Merge  
Michael Kolupaev 已提交
2994 2995
}

A
Merge  
Alexey Milovidov 已提交
2996

M
Merge  
Michael Kolupaev 已提交
2997 2998
void StorageReplicatedMergeTree::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name)
{
2999
    std::string new_full_path = new_path_to_db + escapeForFileName(new_table_name) + '/';
M
Merge  
Michael Kolupaev 已提交
3000

3001
    data.setPath(new_full_path);
M
Merge  
Michael Kolupaev 已提交
3002

3003 3004 3005
    database_name = new_database_name;
    table_name = new_table_name;
    full_path = new_full_path;
M
Merge  
Michael Kolupaev 已提交
3006

3007 3008 3009 3010
    /// Update table name in zookeeper
    auto zookeeper = getZooKeeper();
    zookeeper->set(replica_path + "/host", getReplicatedMergeTreeAddress().toString());

3011
    /// TODO: You can update names of loggers.
M
Merge  
Michael Kolupaev 已提交
3012 3013
}

A
Merge  
Alexey Milovidov 已提交
3014

3015 3016
bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path)
{
3017 3018 3019 3020 3021
    {
        std::lock_guard<std::mutex> lock(existing_nodes_cache_mutex);
        if (existing_nodes_cache.count(path))
            return true;
    }
3022

3023
    bool res = getZooKeeper()->exists(path);
3024

3025 3026 3027 3028 3029
    if (res)
    {
        std::lock_guard<std::mutex> lock(existing_nodes_cache_mutex);
        existing_nodes_cache.insert(path);
    }
3030

3031
    return res;
3032 3033 3034
}


3035
AbandonableLockInZooKeeper StorageReplicatedMergeTree::allocateBlockNumber(const String & partition_id, zkutil::ZooKeeperPtr & zookeeper,
3036
                                                                           zkutil::Requests * precheck_ops)
M
Merge  
Michael Kolupaev 已提交
3037
{
3038 3039
    String partition_path = zookeeper_path + "/block_numbers/" + partition_id;
    if (!existsNodeCached(partition_path))
3040
    {
3041
        int code = zookeeper->tryCreate(partition_path, "", zkutil::CreateMode::Persistent);
3042
        if (code && code != ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
3043
            throw zkutil::KeeperException(code, partition_path);
3044
    }
3045 3046

    return AbandonableLockInZooKeeper(
3047
        partition_path + "/block-",
3048
        zookeeper_path + "/temp", *zookeeper, precheck_ops);
M
Merge  
Michael Kolupaev 已提交
3049 3050
}

A
Merge  
Alexey Milovidov 已提交
3051

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

3056 3057 3058
    Strings replicas = getZooKeeper()->getChildren(zookeeper_path + "/replicas");
    for (const String & replica : replicas)
        waitForReplicaToProcessLogEntry(replica, entry);
A
Merge  
Alexey Milovidov 已提交
3059

3060
    LOG_DEBUG(log, "Finished waiting for all replicas to process " << entry.znode_name);
A
Merge  
Alexey Milovidov 已提交
3061 3062 3063
}


3064
void StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry(const String & replica, const ReplicatedMergeTreeLogEntryData & entry)
A
Merge  
Alexey Milovidov 已提交
3065
{
3066 3067 3068 3069 3070 3071 3072 3073 3074 3075 3076 3077 3078 3079 3080 3081 3082 3083 3084 3085 3086 3087 3088 3089 3090 3091 3092 3093 3094 3095 3096 3097 3098 3099 3100 3101 3102 3103 3104 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 3138 3139 3140 3141 3142 3143 3144 3145 3146 3147 3148 3149 3150 3151 3152 3153 3154 3155 3156 3157 3158 3159 3160 3161 3162 3163 3164 3165 3166 3167 3168 3169 3170 3171 3172 3173 3174 3175 3176 3177 3178 3179 3180 3181 3182 3183 3184 3185 3186 3187 3188 3189 3190
    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 已提交
3191 3192 3193
}


3194
void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields)
3195
{
3196 3197
    auto zookeeper = tryGetZooKeeper();

3198
    res.is_leader = is_leader;
3199 3200 3201 3202
    res.is_readonly = is_readonly;
    res.is_session_expired = !zookeeper || zookeeper->expired();

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

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 3240 3241 3242 3243
    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;
    }
3244 3245
}

3246

3247 3248 3249 3250 3251 3252 3253 3254 3255 3256 3257 3258 3259 3260 3261
/// 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"));

3262
    /// TODO: add setters and getters interface for database and table fields of AST
3263
    auto new_query = query->clone();
3264 3265 3266 3267 3268 3269 3270 3271 3272 3273 3274 3275
    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);
3276

P
proller 已提交
3277
    /// Query send with current user credentials
3278

3279
    auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithoutFailover(context.getSettingsRef());
3280 3281 3282 3283
    Connection connection(
        leader_address.host,
        leader_address.queries_port,
        leader_address.database,
3284
        context.getClientInfo().current_user, context.getClientInfo().current_password, timeouts, "ClickHouse replica");
3285

3286
    RemoteBlockInputStream stream(connection, formattedAST(new_query), {}, context, &settings);
3287
    NullBlockOutputStream output({});
3288 3289 3290 3291 3292 3293

    copyData(stream, output);
    return;
}


3294 3295
void StorageReplicatedMergeTree::getQueue(LogEntriesData & res, String & replica_name_)
{
3296 3297
    replica_name_ = replica_name;
    queue.getEntries(res);
3298 3299
}

3300 3301 3302 3303 3304 3305
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);

3306 3307 3308 3309
    /// 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();
3310 3311 3312

    time_t current_time = time(nullptr);

3313
    if (!queue_update_finish_time)
3314
    {
3315
        /// We have not updated queue even once yet (perhaps replica is readonly).
3316 3317 3318 3319 3320 3321 3322 3323
        /// 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;
    }
3324
    else if (queue_update_start_time > queue_update_finish_time)
3325 3326 3327 3328
    {
        /// 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.
3329
        return (current_time > queue_update_start_time) ? (current_time - queue_update_start_time) : 0;
3330 3331 3332 3333 3334 3335 3336
    }
    else
    {
        /// Everything is up-to-date.
        return 0;
    }
}
3337

3338
void StorageReplicatedMergeTree::getReplicaDelays(time_t & out_absolute_delay, time_t & out_relative_delay)
3339
{
3340
    assertNotReadonly();
3341

3342
    time_t current_time = time(nullptr);
3343

3344
    out_absolute_delay = getAbsoluteDelay();
3345
    out_relative_delay = 0;
3346

3347 3348 3349 3350
    /** 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.
      */
3351

3352 3353
    if (out_absolute_delay < static_cast<time_t>(data.settings.min_relative_delay_to_yield_leadership))
        return;
3354

3355
    auto zookeeper = getZooKeeper();
3356

3357 3358
    time_t max_replicas_unprocessed_insert_time = 0;
    bool have_replica_with_nothing_unprocessed = false;
3359

3360
    Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
3361

3362 3363 3364 3365
    for (const auto & replica : replicas)
    {
        if (replica == replica_name)
            continue;
3366

3367 3368 3369
        /// Skip dead replicas.
        if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))
            continue;
3370

3371 3372 3373
        String value;
        if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/min_unprocessed_insert_time", value))
            continue;
3374

3375
        time_t replica_time = value.empty() ? 0 : parse<time_t>(value);
3376

3377 3378 3379 3380 3381 3382 3383 3384 3385
        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.
              * If the replica for some reason has stalled `queueUpdatingThread`,
              *  then `min_unprocessed_insert_time` will be incorrect.
              */
3386

3387 3388 3389
            have_replica_with_nothing_unprocessed = true;
            break;
        }
3390

3391 3392 3393
        if (replica_time > max_replicas_unprocessed_insert_time)
            max_replicas_unprocessed_insert_time = replica_time;
    }
3394

3395 3396
    if (have_replica_with_nothing_unprocessed)
        out_relative_delay = out_absolute_delay;
3397 3398 3399 3400 3401 3402 3403
    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;
    }
3404 3405 3406
}


3407
void StorageReplicatedMergeTree::fetchPartition(const ASTPtr & partition, const String & from_, const Context & context)
3408
{
3409
    String partition_id = data.getPartitionIDFromQuery(partition, context);
3410 3411 3412 3413 3414

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

3415
    LOG_INFO(log, "Will fetch partition " << partition_id << " from shard " << from_);
3416 3417 3418 3419 3420 3421

    /** 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)
3422 3423
    {
        MergeTreePartInfo part_info;
3424 3425
        if (MergeTreePartInfo::tryParsePartName(dir_it.name(), &part_info, data.format_version)
              && part_info.partition_id == partition_id)
3426 3427
            throw Exception("Detached partition " + partition_id + " already exists.", ErrorCodes::PARTITION_ALREADY_EXISTS);
    }
3428 3429 3430 3431 3432 3433 3434 3435 3436 3437 3438 3439 3440 3441 3442 3443 3444 3445 3446 3447 3448 3449 3450 3451 3452 3453 3454 3455 3456 3457 3458 3459 3460 3461 3462 3463 3464 3465 3466 3467 3468 3469 3470 3471 3472 3473 3474 3475 3476 3477 3478 3479 3480 3481 3482 3483 3484 3485 3486 3487 3488 3489 3490 3491 3492 3493 3494 3495 3496 3497 3498 3499 3500 3501

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

            zkutil::Stat stat;
            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.");

        if (try_no >= 5)
A
Alexey Milovidov 已提交
3502
            throw Exception("Too many retries to fetch parts from " + best_replica_path, ErrorCodes::TOO_MANY_RETRIES_TO_FETCH_PARTS);
3503 3504

        Strings parts = getZooKeeper()->getChildren(best_replica_path + "/parts");
3505
        ActiveDataPartSet active_parts_set(data.format_version, parts);
3506 3507 3508 3509 3510 3511 3512 3513 3514
        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)
3515
            {
3516
                if (MergeTreePartInfo::fromPartName(part, data.format_version).partition_id == partition_id)
3517
                    parts_to_fetch_partition.push_back(part);
3518
            }
3519 3520 3521 3522

            parts_to_fetch = std::move(parts_to_fetch_partition);

            if (parts_to_fetch.empty())
3523
                throw Exception("Partition " + partition_id + " on " + best_replica_path + " doesn't exist", ErrorCodes::PARTITION_DOESNT_EXIST);
3524 3525 3526 3527 3528 3529 3530 3531 3532 3533 3534 3535 3536 3537 3538 3539 3540 3541 3542 3543 3544 3545 3546 3547
        }
        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)
            {
3548
                if (e.code() != ErrorCodes::RECEIVED_ERROR_FROM_REMOTE_IO_SERVER && e.code() != ErrorCodes::RECEIVED_ERROR_TOO_MANY_REQUESTS)
3549 3550 3551 3552 3553 3554 3555 3556 3557
                    throw;

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

        ++try_no;
    } while (!missing_parts.empty());
3558 3559 3560
}


3561
void StorageReplicatedMergeTree::freezePartition(const ASTPtr & partition, const String & with_name, const Context & context)
3562
{
3563
    data.freezePartition(partition, with_name, context);
3564 3565
}

3566

3567
void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK()
3568 3569 3570
{
    /// Critical section is not required (since grabOldParts() returns unique part set on each call)

3571
    auto table_lock = lockStructure(false, __PRETTY_FUNCTION__);
3572 3573 3574
    auto zookeeper = getZooKeeper();

    MergeTreeData::DataPartsVector parts = data.grabOldParts();
3575
    if (parts.empty())
3576 3577
        return;

3578 3579 3580 3581 3582 3583
    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)
3584
    {
3585 3586 3587 3588 3589 3590
        if (!part->is_duplicate)
            parts_to_delete_completely.emplace_back(part);
        else
            parts_to_delete_only_from_filesystem.emplace_back(part);
    }
    parts.clear();
3591

3592 3593 3594
    auto remove_parts_from_filesystem = [log=log] (const MergeTreeData::DataPartsVector & parts_to_remove)
    {
        for (auto & part : parts_to_remove)
3595
        {
3596 3597 3598 3599 3600 3601 3602 3603
            try
            {
                part->remove();
            }
            catch (...)
            {
                tryLogCurrentException(log, "There is a problem with deleting part " + part->name + " from filesystem");
            }
3604 3605 3606
        }
    };

3607 3608 3609 3610 3611 3612 3613 3614 3615 3616
    /// 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
3617
    NameSet part_names_to_retry_deletion;
3618 3619
    try
    {
3620 3621 3622
        Strings part_names_to_delete_completely;
        for (const auto & part : parts_to_delete_completely)
            part_names_to_delete_completely.emplace_back(part->name);
3623

3624 3625
        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);
3626 3627 3628
    }
    catch (...)
    {
3629 3630
        LOG_ERROR(log, "There is a problem with deleting parts from ZooKeeper: " << getCurrentExceptionMessage(false));
    }
3631

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

3636 3637
    /// Delete normal parts on two sets
    for (auto & part : parts_to_delete_completely)
3638 3639 3640 3641 3642
    {
        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);
3643 3644
    }

3645 3646
    /// Will retry deletion
    if (!parts_to_retry_deletion.empty())
3647
    {
3648
        data.rollbackDeletingParts(parts_to_retry_deletion);
3649 3650
        LOG_DEBUG(log, "Will retry deletion of " << parts_to_retry_deletion.size() << " parts in the next time");
    }
3651

3652
    /// Remove parts from filesystem and finally from data_parts
3653
    if (!parts_to_remove_from_filesystem.empty())
3654
    {
3655 3656
        remove_parts_from_filesystem(parts_to_remove_from_filesystem);
        data.removePartsFinally(parts_to_remove_from_filesystem);
3657

3658 3659
        LOG_DEBUG(log, "Removed " << parts_to_remove_from_filesystem.size() << " old parts");
    }
3660 3661 3662
}


3663
void StorageReplicatedMergeTree::removePartsFromZooKeeper(zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names,
A
Alexey Milovidov 已提交
3664
                                                          NameSet * parts_should_be_retried)
3665
{
3666
    zkutil::Requests ops;
3667
    auto it_first_node_in_batch = part_names.cbegin();
3668 3669 3670 3671 3672

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

3673 3674
        auto it_next = std::next(it);
        if (ops.size() >= zkutil::MULTI_BATCH_SIZE || it_next == part_names.cend())
3675
        {
3676 3677
            zkutil::Responses unused_responses;
            auto code = zookeeper->tryMultiNoThrow(ops, unused_responses);
3678
            ops.clear();
3679

3680
            if (code == ZooKeeperImpl::ZooKeeper::ZNONODE)
3681 3682 3683 3684 3685 3686
            {
                /// 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)
                {
3687
                    zkutil::Requests cur_ops;
3688
                    removePartFromZooKeeper(*it_in_batch, cur_ops);
3689
                    auto cur_code = zookeeper->tryMultiNoThrow(cur_ops, unused_responses);
3690

3691
                    if (cur_code == ZooKeeperImpl::ZooKeeper::ZNONODE)
3692
                    {
3693
                        LOG_DEBUG(log, "There is no part " << *it_in_batch << " in ZooKeeper, it was only in filesystem");
3694
                    }
A
Alexey Milovidov 已提交
3695
                    else if (parts_should_be_retried && zkutil::isHardwareError(cur_code))
3696
                    {
A
Alexey Milovidov 已提交
3697
                        parts_should_be_retried->emplace(*it_in_batch);
3698
                    }
3699
                    else if (cur_code)
3700
                    {
3701
                        LOG_WARNING(log, "Cannot remove part " << *it_in_batch << " from ZooKeeper: " << zkutil::ZooKeeper::error2string(cur_code));
3702
                    }
3703 3704
                }
            }
A
Alexey Milovidov 已提交
3705
            else if (parts_should_be_retried && zkutil::isHardwareError(code))
3706 3707
            {
                for (auto it_in_batch = it_first_node_in_batch; it_in_batch != it_next; ++it_in_batch)
A
Alexey Milovidov 已提交
3708
                    parts_should_be_retried->emplace(*it_in_batch);
3709
            }
3710
            else if (code)
3711 3712
            {
                LOG_WARNING(log, "There was a problem with deleting " << (it_next - it_first_node_in_batch)
3713
                    << " nodes from ZooKeeper: " << ::zkutil::ZooKeeper::error2string(code));
3714 3715 3716
            }

            it_first_node_in_batch = it_next;
3717 3718 3719 3720 3721
        }
    }
}


3722 3723 3724 3725
void StorageReplicatedMergeTree::clearBlocksInPartition(
    zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num)
{
    Strings blocks;
3726
    if (zookeeper.tryGetChildren(zookeeper_path + "/blocks", blocks))
3727 3728 3729
        throw Exception(zookeeper_path + "/blocks doesn't exist", ErrorCodes::NOT_FOUND_NODE);

    String partition_prefix = partition_id + "_";
3730
    std::vector<std::pair<String, std::future<zkutil::GetResponse>>> get_futures;
3731 3732 3733 3734 3735 3736 3737 3738 3739
    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));
        }
    }

3740
    std::vector<std::pair<String, std::future<zkutil::RemoveResponse>>> to_delete_futures;
3741 3742 3743
    for (auto & pair : get_futures)
    {
        const String & path = pair.first;
3744
        auto result = pair.second.get();
3745

3746
        if (result.error == ZooKeeperImpl::ZooKeeper::ZNONODE)
3747 3748
            continue;

3749
        ReadBufferFromString buf(result.data);
3750 3751 3752 3753 3754 3755 3756 3757 3758
        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;
3759
        int32_t rc = pair.second.get().error;
3760
        if (rc == ZooKeeperImpl::ZooKeeper::ZNOTEMPTY)
3761 3762 3763 3764
        {
             /// Can happen if there are leftover block nodes with children created by previous server versions.
            zookeeper.removeRecursive(path);
        }
3765
        else if (rc)
3766 3767 3768 3769 3770 3771 3772
            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);
}

3773 3774 3775 3776 3777 3778 3779 3780 3781 3782 3783 3784 3785
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;
    return res;
}

M
Merge  
Michael Kolupaev 已提交
3786
}