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

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

#include <Storages/ColumnsDescription.h>
#include <Storages/StorageReplicatedMergeTree.h>
8
#include <Storages/MergeTree/MergeTreeDataPart.h>
9 10 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

M
Merge  
Michael Kolupaev 已提交
62 63 64
namespace DB
{

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

M
Merge  
Michael Kolupaev 已提交
101

102 103
static const auto QUEUE_UPDATE_ERROR_SLEEP_MS     = 1 * 1000;
static const auto MERGE_SELECTING_SLEEP_MS        = 5 * 1000;
M
Merge  
Michael Kolupaev 已提交
104 105


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


143
/** For randomized selection of replicas. */
P
proller 已提交
144 145 146 147 148
/// avoid error: non-local variable 'DB::rng' declared '__thread' needs dynamic initialization
#ifndef __APPLE__
thread_local
#endif
    pcg64 rng{randomSeed()};
149 150


A
Merge  
Alexey Milovidov 已提交
151 152
void StorageReplicatedMergeTree::setZooKeeper(zkutil::ZooKeeperPtr zookeeper)
{
153 154
    std::lock_guard<std::mutex> lock(current_zookeeper_mutex);
    current_zookeeper = zookeeper;
A
Merge  
Alexey Milovidov 已提交
155 156 157 158
}

zkutil::ZooKeeperPtr StorageReplicatedMergeTree::tryGetZooKeeper()
{
159 160
    std::lock_guard<std::mutex> lock(current_zookeeper_mutex);
    return current_zookeeper;
A
Merge  
Alexey Milovidov 已提交
161 162 163 164
}

zkutil::ZooKeeperPtr StorageReplicatedMergeTree::getZooKeeper()
{
165 166 167 168
    auto res = tryGetZooKeeper();
    if (!res)
        throw Exception("Cannot get ZooKeeper", ErrorCodes::NO_ZOOKEEPER);
    return res;
A
Merge  
Alexey Milovidov 已提交
169 170 171
}


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

206 207
    if (!zookeeper_path.empty() && zookeeper_path.back() == '/')
        zookeeper_path.resize(zookeeper_path.size() - 1);
K
KochetovNicolai 已提交
208 209 210
    /// If zookeeper chroot prefix is used, path should starts with '/', because chroot concatenates without it.
    if (!zookeeper_path.empty() && zookeeper_path.front() != '/')
        zookeeper_path = "/" + zookeeper_path;
211 212 213 214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230 231 232 233 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 263 264 265 266 267 268 269 270
    replica_path = zookeeper_path + "/replicas/" + replica_name;

    bool skip_sanity_checks = false;

    try
    {
        if (current_zookeeper && current_zookeeper->exists(replica_path + "/flags/force_restore_data"))
        {
            skip_sanity_checks = true;
            current_zookeeper->remove(replica_path + "/flags/force_restore_data");

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

            LOG_WARNING(log, "Skipping the limits on severity of changes to data parts and columns (flag force_restore_data).");
        }
    }
    catch (const zkutil::KeeperException & e)
    {
        /// Failed to connect to ZK (this became known when trying to perform the first operation).
        if (e.code == ZCONNECTIONLOSS)
        {
            tryLogCurrentException(__PRETTY_FUNCTION__);
            current_zookeeper = nullptr;
        }
        else
            throw;
    }

    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);
271 272 273 274

        /// 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);
275 276 277
    }

    createNewZooKeeperNodes();
M
Merge  
Michael Kolupaev 已提交
278 279
}

280

281 282
void StorageReplicatedMergeTree::createNewZooKeeperNodes()
{
283
    auto zookeeper = getZooKeeper();
284

285 286 287 288
    /// Working with quorum.
    zookeeper->createIfNotExists(zookeeper_path + "/quorum", "");
    zookeeper->createIfNotExists(zookeeper_path + "/quorum/last_part", "");
    zookeeper->createIfNotExists(zookeeper_path + "/quorum/failed_parts", "");
289

290 291 292
    /// Tracking lag of replicas.
    zookeeper->createIfNotExists(replica_path + "/min_unprocessed_insert_time", "");
    zookeeper->createIfNotExists(replica_path + "/max_processed_insert_time", "");
293 294 295
}


M
Merge  
Michael Kolupaev 已提交
296 297
static String formattedAST(const ASTPtr & ast)
{
298 299 300
    if (!ast)
        return "";
    std::stringstream ss;
A
Alexey Milovidov 已提交
301
    formatAST(*ast, ss, false, true);
302
    return ss.str();
M
Merge  
Michael Kolupaev 已提交
303
}
M
Merge  
Michael Kolupaev 已提交
304

A
Merge  
Alexey Milovidov 已提交
305

306 307
namespace
{
308 309 310 311 312 313 314
    /** The basic parameters of table engine for saving in ZooKeeper.
      * Lets you verify that they match local ones.
      */
    struct TableMetadata
    {
        const MergeTreeData & data;

315
        explicit TableMetadata(const MergeTreeData & data_)
316 317 318 319 320
            : data(data_) {}

        void write(WriteBuffer & out) const
        {
            out << "metadata format version: 1" << "\n"
321 322 323 324 325 326 327 328
                << "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"
329 330 331 332
                << "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";
333 334 335 336 337 338

            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";
            }
339 340 341 342
        }

        String toString() const
        {
343
            WriteBufferFromOwnString out;
344
            write(out);
345
            return out.str();
346 347 348 349 350 351 352 353 354
        }

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

            in >> "metadata format version: 1";

            in >> "\ndate column: ";
355 356 357 358 359 360 361 362 363 364 365 366 367 368 369
            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.",
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 415 416 417 418 419 420 421 422
                    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";
423 424 425 426 427 428 429 430 431 432 433 434 435 436 437 438 439 440 441 442 443 444 445 446 447 448 449 450 451 452 453
            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";
            }

454 455 456 457 458 459 460 461 462
            assertEOF(in);
        }

        void check(const String & s) const
        {
            ReadBufferFromString in(s);
            check(in);
        }
    };
463 464 465
}


M
Merge  
Michael Kolupaev 已提交
466
void StorageReplicatedMergeTree::createTableIfNotExists()
M
Merge  
Michael Kolupaev 已提交
467
{
468 469 470 471 472 473 474 475 476 477 478 479 480 481 482 483 484 485 486
    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();

    auto acl = zookeeper->getDefaultACL();

    zkutil::Ops ops;
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(zookeeper_path, "",
        acl, zkutil::CreateMode::Persistent));
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(zookeeper_path + "/metadata", metadata,
        acl, zkutil::CreateMode::Persistent));
487
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(zookeeper_path + "/columns", getColumns().toString(),
488 489 490 491 492 493 494 495 496 497 498 499 500 501 502 503 504 505 506
        acl, zkutil::CreateMode::Persistent));
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(zookeeper_path + "/log", "",
        acl, zkutil::CreateMode::Persistent));
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(zookeeper_path + "/blocks", "",
        acl, zkutil::CreateMode::Persistent));
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(zookeeper_path + "/block_numbers", "",
        acl, zkutil::CreateMode::Persistent));
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(zookeeper_path + "/nonincrement_block_numbers", "",
        acl, zkutil::CreateMode::Persistent));
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(zookeeper_path + "/leader_election", "",
        acl, zkutil::CreateMode::Persistent));
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(zookeeper_path + "/temp", "",
        acl, zkutil::CreateMode::Persistent));
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(zookeeper_path + "/replicas", "",
        acl, zkutil::CreateMode::Persistent));

    auto code = zookeeper->tryMulti(ops);
    if (code != ZOK && code != ZNODEEXISTS)
        throw zkutil::KeeperException(code);
M
Merge  
Michael Kolupaev 已提交
507
}
M
Merge  
Michael Kolupaev 已提交
508

A
Merge  
Alexey Milovidov 已提交
509

F
f1yegor 已提交
510
/** Verify that list of columns and table settings match those specified in ZK (/ metadata).
511 512
    * If not, throw an exception.
    */
M
Merge  
Michael Kolupaev 已提交
513
void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bool allow_alter)
M
Merge  
Michael Kolupaev 已提交
514
{
515 516 517 518 519 520
    auto zookeeper = getZooKeeper();

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

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

524 525
    const ColumnsDescription & old_columns = getColumns();
    if (columns_from_zk != old_columns)
526 527 528
    {
        if (allow_alter &&
            (skip_sanity_checks ||
529 530
             old_columns.ordinary.sizeOfDifference(columns_from_zk.ordinary) +
             old_columns.materialized.sizeOfDifference(columns_from_zk.materialized) <= 2))
531 532 533 534
        {
            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.
535 536
            context.getDatabase(database_name)->alterTable(context, table_name, columns_from_zk, {});

537
            setColumns(std::move(columns_from_zk));
538 539 540
        }
        else
        {
541
            throw Exception("Table structure in ZooKeeper is too different from local table structure",
542 543 544
                            ErrorCodes::INCOMPATIBLE_COLUMNS);
        }
    }
M
Merge  
Michael Kolupaev 已提交
545
}
M
Merge  
Michael Kolupaev 已提交
546

A
Merge  
Alexey Milovidov 已提交
547

F
f1yegor 已提交
548 549 550 551
/** 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).
552 553 554
  */
static time_t tryGetPartCreateTime(zkutil::ZooKeeperPtr & zookeeper, const String & replica_path, const String & part_name)
{
555
    time_t res = 0;
556

557 558 559 560 561
    /// 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;
562

563
    return res;
564 565 566
}


M
Merge  
Michael Kolupaev 已提交
567 568
void StorageReplicatedMergeTree::createReplica()
{
569 570 571 572 573 574 575 576 577 578 579 580 581 582 583 584 585 586 587 588 589 590 591 592 593 594 595 596 597 598 599 600 601 602 603 604 605 606 607 608 609 610
    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.
    auto acl = zookeeper->getDefaultACL();
    zkutil::Ops ops;
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(replica_path, "", acl, zkutil::CreateMode::Persistent));
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(replica_path + "/host", "", acl, zkutil::CreateMode::Persistent));
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(replica_path + "/log_pointer", "", acl, zkutil::CreateMode::Persistent));
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(replica_path + "/queue", "", acl, zkutil::CreateMode::Persistent));
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(replica_path + "/parts", "", acl, zkutil::CreateMode::Persistent));
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(replica_path + "/flags", "", acl, zkutil::CreateMode::Persistent));

    try
    {
        zookeeper->multi(ops);
    }
    catch (const zkutil::KeeperException & e)
    {
        if (e.code == ZNODEEXISTS)
            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;

    Stat stat;
    zookeeper->exists(replica_path, &stat);
    auto my_create_time = stat.czxid;

611
    std::shuffle(replicas.begin(), replicas.end(), rng);
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 663 664 665 666 667 668 669 670 671 672
    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");
673
        ActiveDataPartSet active_parts_set(data.format_version, parts);
674 675 676 677 678 679 680 681 682 683 684 685 686 687 688 689 690 691 692 693 694 695 696 697 698

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

699
    zookeeper->create(replica_path + "/columns", getColumns().toString(), zkutil::CreateMode::Persistent);
M
Merge  
Michael Kolupaev 已提交
700
}
M
Merge  
Michael Kolupaev 已提交
701 702


M
Merge  
Michael Kolupaev 已提交
703
void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
M
Merge  
Michael Kolupaev 已提交
704
{
705 706 707 708 709 710 711
    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());

712 713
    /// There are no PreCommitted parts at startup.
    auto parts = data.getDataParts({MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated});
714 715 716 717 718 719 720 721 722 723 724 725 726 727

    /// 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;
728
    UInt64 parts_to_add_rows = 0;
729 730 731 732 733 734 735 736 737 738 739 740 741 742 743

    /// 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);
744
                parts_to_add_rows += containing->rows_count;
745 746 747 748 749 750 751 752 753 754 755 756
            }
        }
        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);

757

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

796

797 798 799 800 801 802 803
    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).";
804

805 806 807
    /** 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%).
808
      *
809 810
      * 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).
811 812 813
      * In this case, the protection mechanism does not allow the server to start.
      */

814 815 816 817 818 819
    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;
820

821
    bool insane = total_suspicious_rows > total_rows_on_filesystem * data.settings.replicated_max_ratio_of_wrong_parts;
822 823

    if (insane && !skip_sanity_checks)
824 825 826 827 828 829 830 831 832
    {
        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);
    }
833

834 835
    if (total_suspicious_rows_no_new > 0)
        LOG_WARNING(log, sanity_report.str());
836 837 838 839 840 841 842 843 844 845 846 847 848

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

        zkutil::Ops ops;
        checkPartAndAddToZooKeeper(part, ops);
        zookeeper->multi(ops);
    }

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

852
        removePartsFromZooKeeper(zookeeper, Strings(expected_parts.begin(), expected_parts.end()));
853 854 855 856 857 858 859 860 861 862 863 864 865 866 867
    }

    /// 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).
        zkutil::Ops ops;
868
        removePartFromZooKeeper(name, ops);
869 870 871 872 873 874 875 876 877 878 879
        ops.emplace_back(std::make_unique<zkutil::Op::Create>(
            replica_path + "/queue/queue-", log_entry.toString(), zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
        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 已提交
880
}
M
Merge  
Michael Kolupaev 已提交
881

A
Merge  
Alexey Milovidov 已提交
882

883
void StorageReplicatedMergeTree::checkPartAndAddToZooKeeper(
884
    const MergeTreeData::DataPartPtr & part, zkutil::Ops & ops, String part_name)
M
Merge  
Michael Kolupaev 已提交
885
{
886 887 888 889 890 891 892 893 894
    auto zookeeper = getZooKeeper();

    if (part_name.empty())
        part_name = part->name;

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

    Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
895
    std::shuffle(replicas.begin(), replicas.end(), rng);
896 897 898 899 900 901 902 903 904 905 906 907 908 909 910 911 912 913 914 915 916 917 918 919 920 921 922 923 924 925 926 927 928 929 930 931 932 933 934 935 936 937 938 939 940 941 942 943 944 945 946 947 948 949 950 951
    String expected_columns_str = part->columns.toString();

    for (const String & replica : replicas)
    {
        zkutil::Stat stat_before, stat_after;
        String columns_str;
        if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name + "/columns", columns_str, &stat_before))
            continue;
        if (columns_str != expected_columns_str)
        {
            LOG_INFO(log, "Not checking checksums of part " << part_name << " with replica " << replica
                << " because columns are different");
            continue;
        }
        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.
        if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name + "/checksums", checksums_str) ||
            !zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name + "/columns", &stat_after) ||
            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;
        }

        auto checksums = MergeTreeData::DataPart::Checksums::parse(checksums_str);
        checksums.checkEqual(part->checksums, true);
    }

    if (zookeeper->exists(replica_path + "/parts/" + part_name))
    {
        LOG_ERROR(log, "checkPartAndAddToZooKeeper: node " << replica_path + "/parts/" + part_name << " already exists");
        return;
    }

    auto acl = zookeeper->getDefaultACL();

    ops.emplace_back(std::make_unique<zkutil::Op::Check>(
        zookeeper_path + "/columns",
        expected_columns_version));
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(
        replica_path + "/parts/" + part_name,
        "",
        acl,
        zkutil::CreateMode::Persistent));
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(
        replica_path + "/parts/" + part_name + "/columns",
        part->columns.toString(),
        acl,
        zkutil::CreateMode::Persistent));
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(
        replica_path + "/parts/" + part_name + "/checksums",
        part->checksums.toString(),
        acl,
        zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
952 953
}

A
Merge  
Alexey Milovidov 已提交
954

955
void StorageReplicatedMergeTree::pullLogsToQueue(zkutil::EventPtr next_update_event)
M
Merge  
Michael Kolupaev 已提交
956
{
957 958 959 960 961
    if (queue.pullLogsToQueue(getZooKeeper(), next_update_event))
    {
        if (queue_task_handle)
            queue_task_handle->wake();
    }
M
Merge  
Michael Kolupaev 已提交
962 963
}

A
Merge  
Alexey Milovidov 已提交
964

965
bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
966
{
967 968 969 970 971 972
    if (entry.type == LogEntry::ATTACH_PART)
    {
        LOG_ERROR(log, "Log entries of type ATTACH_PART are obsolete. Skipping.");
        return true;
    }

973 974 975 976 977 978
    if (entry.type == LogEntry::DROP_RANGE)
    {
        executeDropRange(entry);
        return true;
    }

979
    if (entry.type == LogEntry::CLEAR_COLUMN)
980
    {
981
        executeClearColumnInPartition(entry);
982 983 984
        return true;
    }

985
    if (entry.type == LogEntry::GET_PART ||
986
        entry.type == LogEntry::MERGE_PARTS)
987 988
    {
        /// If we already have this part or a part covering it, we do not need to do anything.
A
Alexey Zatelepin 已提交
989 990
        /// 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.
991 992 993
        MergeTreeData::DataPartPtr existing_part = data.getPartIfExists(entry.new_part_name, {MergeTreeDataPartState::PreCommitted});
        if (!existing_part)
            existing_part = data.getActiveContainingPart(entry.new_part_name);
994 995

        /// Even if the part is locally, it (in exceptional cases) may not be in ZooKeeper. Let's check that it is there.
996
        if (existing_part && getZooKeeper()->exists(replica_path + "/parts/" + existing_part->name))
997 998
        {
            if (!(entry.type == LogEntry::GET_PART && entry.source_replica == replica_name))
999 1000 1001
            {
                LOG_DEBUG(log, "Skipping action for part " << entry.new_part_name << " because part " + existing_part->name + " already exists.");
            }
1002 1003 1004 1005 1006 1007 1008 1009 1010 1011 1012 1013 1014 1015 1016 1017 1018 1019 1020 1021
            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)
1022 1023 1024 1025 1026 1027 1028 1029 1030 1031 1032 1033 1034 1035 1036 1037 1038 1039 1040 1041 1042 1043
    {
        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
1044 1045 1046 1047 1048 1049 1050 1051
    {
        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());
1052
    }
1053

1054 1055 1056 1057 1058 1059
    MergeTreeData::DataPartsVector parts;
    bool have_all_parts = true;
    for (const String & name : entry.parts_to_merge)
    {
        MergeTreeData::DataPartPtr part = data.getActiveContainingPart(name);
        if (!part)
1060
        {
1061 1062
            have_all_parts = false;
            break;
1063
        }
1064
        if (part->name != name)
1065
        {
1066 1067 1068 1069
            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;
1070
        }
1071 1072 1073 1074 1075 1076 1077 1078 1079 1080 1081 1082 1083
        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.
1084

1085 1086 1087
        size_t sum_parts_size_in_bytes = 0;
        for (const auto & part : parts)
            sum_parts_size_in_bytes += part->size_in_bytes;
1088

1089 1090 1091 1092
        if (sum_parts_size_in_bytes >= data.settings.prefer_fetch_merged_part_size_threshold)
        {
            String replica = findReplicaHavingPart(entry.new_part_name, true);    /// NOTE excessive ZK requests for same data later, may remove.
            if (!replica.empty())
1093
            {
1094 1095
                do_fetch = true;
                LOG_DEBUG(log, "Prefer to fetch " << entry.new_part_name << " from replica " << replica);
1096 1097
            }
        }
1098
    }
1099

1100 1101
    if (do_fetch)
        return;
1102

1103
    /// Start to make the main work
1104

1105
    size_t estimated_space_for_merge = MergeTreeDataMerger::estimateDiskSpaceForMerge(parts);
1106

1107 1108
    /// Can throw an exception.
    DiskSpaceMonitor::ReservationPtr reserved_space = DiskSpaceMonitor::reserve(full_path, estimated_space_for_merge);
1109

1110
    auto table_lock = lockStructure(false, __PRETTY_FUNCTION__);
1111

1112 1113 1114
    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;
1115

1116 1117 1118 1119 1120 1121
    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);
    }
1122

1123 1124 1125 1126
    /// Logging
    Stopwatch stopwatch;
    ExecutionStatus execution_status;
    MergeTreeData::MutableDataPartPtr part;
1127

1128 1129 1130 1131
    auto write_part_log = [&] (const ExecutionStatus & execution_status)
    {
        try
        {
1132
            auto part_log = context.getPartLog(database_name);
1133 1134
            if (!part_log)
                return;
1135

1136
            PartLogElement part_log_elem;
1137

1138 1139 1140 1141
            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;
1142

1143 1144 1145
            part_log_elem.database_name = database_name;
            part_log_elem.table_name = table_name;
            part_log_elem.part_name = entry.new_part_name;
1146

1147 1148
            if (part)
                part_log_elem.bytes_compressed_on_disk = part->size_in_bytes;
1149

1150 1151 1152
            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);
1153

1154 1155
            part_log_elem.rows_read = (*merge_entry)->bytes_read_uncompressed;
            part_log_elem.bytes_read_uncompressed = (*merge_entry)->bytes_read_uncompressed;
1156

1157 1158
            part_log_elem.rows = (*merge_entry)->rows_written;
            part_log_elem.bytes_uncompressed = (*merge_entry)->bytes_written_uncompressed;
1159

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

1163 1164 1165 1166 1167 1168 1169
            part_log->add(part_log_elem);
        }
        catch (...)
        {
            tryLogCurrentException(log, __PRETTY_FUNCTION__);
        }
    };
1170

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

1176
        zkutil::Ops ops;
1177

1178 1179 1180 1181 1182 1183 1184 1185 1186 1187 1188 1189 1190 1191 1192 1193 1194 1195 1196 1197 1198 1199 1200 1201 1202 1203 1204 1205 1206 1207 1208 1209
        try
        {
            /// Checksums are checked here and `ops` is filled. In fact, the part is added to ZK just below, when executing `multi`.
            checkPartAndAddToZooKeeper(part, ops, entry.new_part_name);
        }
        catch (const Exception & e)
        {
            if (e.code() == ErrorCodes::CHECKSUM_DOESNT_MATCH
                || e.code() == ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART
                || e.code() == ErrorCodes::NO_FILE_IN_DATA_PART
                || e.code() == ErrorCodes::UNEXPECTED_FILE_IN_DATA_PART)
            {
                do_fetch = true;
                part->remove();

                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;
1210
            }
1211 1212 1213 1214 1215 1216 1217 1218 1219 1220 1221

            throw;
        }

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

        /// Do not commit if the part is obsolete
        if (!transaction.isEmpty())
        {
            getZooKeeper()->multi(ops); /// After long merge, get fresh ZK handle, because previous session may be expired.
            transaction.commit();
1222
        }
1223 1224 1225 1226 1227 1228 1229 1230 1231 1232 1233

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

        /** With `ZCONNECTIONLOSS` or `ZOPERATIONTIMEOUT`, we can inadvertently roll back local changes to the parts.
          * 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({});
1234
    }
1235
    catch (...)
1236
    {
1237 1238
        write_part_log(ExecutionStatus::fromCurrentException());
        throw;
1239
    }
1240
}
1241 1242


1243 1244 1245
bool StorageReplicatedMergeTree::executeFetch(const StorageReplicatedMergeTree::LogEntry & entry)
{
    String replica = findReplicaHavingCoveringPart(entry, true);
1246

1247 1248 1249 1250
    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 已提交
1251
            ErrorCodes::TOO_MANY_FETCHES);
1252
    }
1253

1254 1255
    ++total_fetches;
    SCOPE_EXIT({--total_fetches;});
1256

1257 1258 1259
    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 已提交
1260
            ErrorCodes::TOO_MANY_FETCHES);
1261
    }
1262

1263 1264 1265 1266 1267 1268
    ++current_table_fetches;
    SCOPE_EXIT({--current_table_fetches;});

    try
    {
        if (replica.empty())
1269
        {
1270 1271 1272 1273 1274 1275
            /** 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)
1276
            {
1277 1278 1279 1280 1281 1282 1283 1284 1285 1286 1287 1288 1289 1290 1291
                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.
1292 1293
                  */

1294 1295 1296 1297
                /** 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.
                  */
1298

1299
                auto zookeeper = getZooKeeper();
1300

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

1303
                zkutil::Ops ops;
1304

1305 1306 1307 1308 1309 1310 1311
                for (size_t i = 0, size = replicas.size(); i < size; ++i)
                {
                    Stat stat;
                    String path = zookeeper_path + "/replicas/" + replicas[i] + "/host";
                    zookeeper->get(path, &stat);
                    ops.emplace_back(std::make_unique<zkutil::Op::Check>(path, stat.version));
                }
1312

1313 1314
                /// 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);
1315

1316 1317 1318 1319 1320 1321 1322 1323 1324 1325 1326 1327
                /// 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())
                {
                    Stat quorum_stat;
                    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)
1328
                    {
1329
                        ops.emplace_back(std::make_unique<zkutil::Op::Remove>(quorum_path, quorum_stat.version));
1330

1331
                        auto part_info = MergeTreePartInfo::fromPartName(entry.new_part_name, data.format_version);
1332

1333 1334 1335
                        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);
1336

1337 1338 1339
                        zookeeper->createIfNotExists(zookeeper_path + "/nonincrement_block_numbers/" + part_info.partition_id, "");

                        auto acl = zookeeper->getDefaultACL();
1340

1341 1342 1343 1344 1345 1346 1347 1348 1349 1350 1351 1352 1353 1354 1355 1356 1357 1358 1359
                        ops.emplace_back(std::make_unique<zkutil::Op::Create>(
                            zookeeper_path + "/nonincrement_block_numbers/" + part_info.partition_id + "/block-" + padIndex(part_info.min_block),
                            "",
                            acl,
                            zkutil::CreateMode::Persistent));

                        ops.emplace_back(std::make_unique<zkutil::Op::Create>(
                            zookeeper_path + "/quorum/failed_parts/" + entry.new_part_name,
                            "",
                            acl,
                            zkutil::CreateMode::Persistent));

                        /// Deleting from `blocks`.
                        if (!entry.block_id.empty() && zookeeper->exists(zookeeper_path + "/blocks/" + entry.block_id))
                            ops.emplace_back(std::make_unique<zkutil::Op::Remove>(zookeeper_path + "/blocks/" + entry.block_id, -1));

                        auto code = zookeeper->tryMulti(ops);

                        if (code == ZOK)
1360
                        {
1361 1362
                            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.
1363
                        }
1364
                        else if (code == ZBADVERSION || code == ZNONODE || code == ZNODEEXISTS)
1365
                        {
1366 1367
                            LOG_DEBUG(log, "State was changed or isn't expected when trying to mark quorum for part "
                                << entry.new_part_name << " as failed. Code: " << zerror(code));
1368
                        }
1369 1370 1371 1372 1373 1374 1375 1376 1377 1378
                        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.");
1379 1380 1381 1382
                    }
                }
            }

1383
            if (replica.empty())
1384
            {
1385 1386
                ProfileEvents::increment(ProfileEvents::ReplicatedPartFailedFetches);
                throw Exception("No active replica has part " + entry.new_part_name + " or covering part", ErrorCodes::NO_REPLICA_HAS_PART);
1387
            }
1388
        }
1389

1390 1391 1392 1393
        try
        {
            if (!fetchPart(entry.actual_new_part_name, zookeeper_path + "/replicas/" + replica, false, entry.quorum))
                return false;
1394
        }
1395
        catch (Exception & e)
1396
        {
1397 1398 1399 1400 1401
            /// No stacktrace, just log message
            if (e.code() == ErrorCodes::RECEIVED_ERROR_TOO_MANY_REQUESTS)
                e.addMessage("Too busy replica. Will try later.");
            throw;
        }
1402

1403 1404 1405 1406 1407 1408 1409 1410 1411 1412 1413 1414
        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);
1415

1416
            if (!parts_for_merge.empty() && replica.empty())
1417
            {
1418 1419
                LOG_INFO(log, "No active replica has part " << entry.new_part_name << ". Will fetch merged part instead.");
                return false;
1420 1421
            }

1422 1423 1424 1425 1426
            /** 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);
1427
        }
1428 1429 1430 1431 1432 1433
        catch (...)
        {
            tryLogCurrentException(__PRETTY_FUNCTION__);
        }

        throw;
1434 1435 1436
    }

    return true;
M
Merge  
Michael Kolupaev 已提交
1437 1438
}

A
Merge  
Alexey Milovidov 已提交
1439

M
Merge  
Michael Kolupaev 已提交
1440
void StorageReplicatedMergeTree::executeDropRange(const StorageReplicatedMergeTree::LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
1441
{
1442 1443 1444 1445 1446 1447 1448
    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;

1449
    auto entry_part_info = MergeTreePartInfo::fromPartName(entry.new_part_name, data.format_version);
1450

1451 1452 1453 1454
    /// 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.
1455 1456 1457
    /// Therefore, we use all data parts.
    auto parts = data.getDataParts({MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated});

1458 1459
    for (const auto & part : parts)
    {
1460
        if (!entry_part_info.contains(part->info))
1461 1462 1463 1464 1465 1466 1467 1468 1469 1470 1471 1472 1473 1474 1475 1476 1477 1478 1479 1480
            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);

        zkutil::Ops ops;
        removePartFromZooKeeper(part->name, ops);
        auto code = getZooKeeper()->tryMulti(ops);

        /// If the part is already removed (for example, because it was never added to ZK due to crash,
        /// see ReplicatedMergeTreeBlockOutputStream), then Ok.
        if (code != ZOK && code != ZNONODE)
            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)
1481
            data.removePartsFromWorkingSet({part}, true);
1482 1483 1484
    }

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

A
Merge  
Alexey Milovidov 已提交
1487

1488
void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry & entry)
1489
{
1490
    LOG_INFO(log, "Clear column " << entry.column_name << " in parts inside " << entry.new_part_name << " range");
1491 1492 1493 1494 1495

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

1496
    auto entry_part_info = MergeTreePartInfo::fromPartName(entry.new_part_name, data.format_version);
1497

1498 1499 1500
    /// 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)
1501
    auto lock_read_structure = lockStructure(false, __PRETTY_FUNCTION__);
1502 1503 1504 1505 1506 1507 1508

    auto zookeeper = getZooKeeper();

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

1509
    auto new_columns = getColumns();
1510
    alter_command.apply(new_columns);
1511 1512

    size_t modified_parts = 0;
1513
    auto parts = data.getDataParts();
1514
    auto columns_for_parts = new_columns.getPhysical();
1515 1516
    for (const auto & part : parts)
    {
1517
        if (!entry_part_info.contains(part->info))
1518 1519
            continue;

1520
        LOG_DEBUG(log, "Clearing column " << entry.column_name << " in part " << part->name);
1521 1522 1523 1524 1525 1526 1527 1528 1529 1530 1531 1532 1533 1534 1535 1536 1537 1538

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

        /// Update part metadata in ZooKeeper.
        zkutil::Ops ops;
        ops.emplace_back(std::make_unique<zkutil::Op::SetData>(
            replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1));
        ops.emplace_back(std::make_unique<zkutil::Op::SetData>(
            replica_path + "/parts/" + part->name + "/checksums", transaction->getNewChecksums().toString(), -1));

        zookeeper->multi(ops);

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

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

1541
    /// Recalculate columns size (not only for the modified column)
1542 1543 1544 1545
    data.recalculateColumnSizes();
}


M
Merge  
Michael Kolupaev 已提交
1546 1547
void StorageReplicatedMergeTree::queueUpdatingThread()
{
1548
    setThreadName("ReplMTQueueUpd");
1549

1550 1551
    bool update_in_progress = false;
    while (!shutdown_called)
S
Silviu Caragea 已提交
1552
    {
1553 1554 1555 1556 1557 1558 1559 1560 1561 1562 1563 1564 1565 1566 1567 1568
        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 (const zkutil::KeeperException & e)
        {
            if (e.code == ZINVALIDSTATE)
                restarting_thread->wakeup();
1569

1570 1571 1572 1573 1574 1575 1576 1577
            tryLogCurrentException(__PRETTY_FUNCTION__);
            queue_updating_event->tryWait(QUEUE_UPDATE_ERROR_SLEEP_MS);
        }
        catch (...)
        {
            tryLogCurrentException(__PRETTY_FUNCTION__);
            queue_updating_event->tryWait(QUEUE_UPDATE_ERROR_SLEEP_MS);
        }
S
Silviu Caragea 已提交
1578
    }
1579 1580

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

A
Merge  
Alexey Milovidov 已提交
1583

1584
bool StorageReplicatedMergeTree::queueTask()
M
Merge  
Michael Kolupaev 已提交
1585
{
1586 1587 1588 1589 1590 1591 1592 1593 1594 1595 1596 1597 1598 1599 1600 1601 1602 1603 1604 1605 1606 1607 1608 1609 1610 1611 1612 1613 1614 1615 1616 1617 1618 1619 1620 1621 1622
    /// This object will mark the element of the queue as running.
    ReplicatedMergeTreeQueue::SelectedEntry selected;

    try
    {
        selected = queue.selectEntryToProcess(merger, data);
    }
    catch (...)
    {
        tryLogCurrentException(__PRETTY_FUNCTION__);
    }

    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());
            }
1623 1624 1625 1626 1627
            else if (e.code() == ErrorCodes::PART_IS_TEMPORARILY_LOCKED)
            {
                /// Part cannot be added temporarily
                LOG_INFO(log, e.displayText());
            }
1628 1629 1630 1631 1632 1633 1634 1635 1636 1637 1638 1639 1640 1641 1642 1643 1644 1645 1646 1647 1648
            else
                tryLogCurrentException(__PRETTY_FUNCTION__);

            /** 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 (...)
        {
            tryLogCurrentException(__PRETTY_FUNCTION__);
            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 已提交
1649 1650
}

A
Merge  
Alexey Milovidov 已提交
1651

1652
namespace
M
Merge  
Michael Kolupaev 已提交
1653
{
1654 1655 1656
    bool canMergePartsAccordingToZooKeeperInfo(
        const MergeTreeData::DataPartPtr & left,
        const MergeTreeData::DataPartPtr & right,
1657
        zkutil::ZooKeeperPtr && zookeeper, const String & zookeeper_path, const MergeTreeData & data, String * out_reason = nullptr)
1658
    {
1659
        const String & partition_id = left->info.partition_id;
1660

1661 1662 1663 1664 1665 1666 1667
        /// 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);
1668

1669
            auto part_info = MergeTreePartInfo::fromPartName(quorum_entry.part_name, data.format_version);
1670

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

1674
            if (left->info.max_block <= part_info.min_block && right->info.min_block >= part_info.max_block)
1675
            {
1676 1677
                if (out_reason)
                    *out_reason = "Quorum status condition is unsatisfied";
1678
                return false;
1679
            }
1680
        }
1681

1682 1683 1684 1685 1686
        /// 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)
        {
1687
            auto part_info = MergeTreePartInfo::fromPartName(quorum_last_part, data.format_version);
1688

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

1692
            if (left->info.max_block <= part_info.min_block && right->info.min_block >= part_info.max_block)
1693
            {
1694
                if (out_reason)
1695
                    *out_reason = "Quorum 'last part' condition is unsatisfied";
1696
                return false;
1697
            }
1698
        }
1699 1700

        /// You can merge the parts, if all the numbers between them are abandoned - do not correspond to any blocks.
1701
        for (Int64 number = left->info.max_block + 1; number <= right->info.min_block - 1; ++number)
1702
        {
1703 1704
            String path1 = zookeeper_path +              "/block_numbers/" + partition_id + "/block-" + padIndex(number);
            String path2 = zookeeper_path + "/nonincrement_block_numbers/" + partition_id + "/block-" + padIndex(number);
1705 1706 1707

            if (AbandonableLockInZooKeeper::check(path1, *zookeeper) != AbandonableLockInZooKeeper::ABANDONED &&
                AbandonableLockInZooKeeper::check(path2, *zookeeper) != AbandonableLockInZooKeeper::ABANDONED)
1708 1709 1710 1711
            {
                if (out_reason)
                    *out_reason = "Block " + toString(number) + " in gap between merging parts " + left->name + " and "
                                  + right->name + " is not abandoned";
1712
                return false;
1713
            }
1714
        }
1715 1716

        return true;
1717
    }
1718

1719

1720
    /// If any of the parts is already going to be merged into a larger one, do not agree to merge it.
1721 1722 1723 1724 1725 1726 1727 1728 1729 1730 1731 1732 1733 1734 1735 1736 1737 1738 1739 1740
    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;
    }


1741 1742 1743 1744 1745 1746 1747 1748 1749 1750 1751
    /** 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.
    */
1752

1753 1754 1755 1756 1757 1758 1759 1760 1761
    /** 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;
1762

1763 1764 1765 1766 1767
        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;
1768

1769 1770
            clock::time_point expire_time;
            clock::duration delay = clock::duration::zero();
1771

1772 1773 1774 1775 1776 1777 1778 1779 1780 1781
            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;
                }
1782

1783 1784 1785 1786
                expire_time = now + delay;
            }

            bool expired(clock::time_point now) const
1787
            {
1788
                return now > expire_time;
1789
            }
1790
        };
1791

1792 1793
        std::set<Key> true_keys;
        std::map<Key, Expiration> false_keys;
1794

1795 1796
        template <typename Function, typename ArgsToKey, typename... Args>
        bool get(clock::time_point now, Function && function, ArgsToKey && args_to_key, Args &&... args)
1797
        {
1798
            Key key{args_to_key(std::forward<Args>(args)...)};
1799

1800 1801
            if (true_keys.count(key))
                return true;
1802

1803 1804 1805
            auto it = false_keys.find(key);
            if (false_keys.end() != it && !it->second.expired(now))
                return false;
1806

1807
            bool value = function(std::forward<Args>(args)...);
1808

1809 1810 1811 1812
            if (value)
                true_keys.insert(key);
            else
                false_keys[key].next(now);
1813

1814 1815 1816
            return value;
        }
    };
1817

1818 1819 1820
    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;
1821
}
1822

1823

1824
void StorageReplicatedMergeTree::mergeSelectingThread()
1825
{
1826 1827 1828 1829
    setThreadName("ReplMTMergeSel");
    LOG_DEBUG(log, "Merge selecting thread started");

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

1831
    auto uncached_merging_predicate = [this](const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right)
1832
    {
1833
        return canMergePartsAccordingToZooKeeperInfo(left, right, getZooKeeper(), zookeeper_path, data);
1834
    };
1835

1836
    auto merging_predicate_args_to_key = [](const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right)
1837
    {
1838 1839 1840
        return std::make_pair(left->name, right->name);
    };

1841
    CachedMergingPredicate<std::pair<std::string, std::string>> cached_merging_predicate;
1842 1843

    /// Will be updated below.
1844
    std::chrono::steady_clock::time_point now;
1845

1846
    auto can_merge = [&] (const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, String *)
1847
    {
1848 1849
        return partsWillNotBeMergedOrDisabled(left, right, queue)
               && cached_merging_predicate.get(now, uncached_merging_predicate, merging_predicate_args_to_key, left, right);
1850
    };
S
Silviu Caragea 已提交
1851

1852
    while (!shutdown_called && is_leader_node)
S
Silviu Caragea 已提交
1853
    {
1854 1855 1856
        bool success = false;

        try
1857
        {
1858
            std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
1859

1860 1861 1862
            /// 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.
1863
            if (merge_selecting_logs_pulling_is_required)
1864 1865
            {
                pullLogsToQueue();
1866
                merge_selecting_logs_pulling_is_required = false;
1867
            }
1868

1869 1870 1871
            /// 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.
1872
            size_t merges_queued = queue.countMerges();
1873

1874 1875 1876 1877 1878 1879 1880 1881 1882
            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;
1883

1884
                size_t max_parts_size_for_merge = merger.getMaxPartsSizeForMerge(data.settings.max_replicated_merges_in_queue, merges_queued);
1885

1886
                now = std::chrono::steady_clock::now();
1887

1888
                if (max_parts_size_for_merge > 0
1889
                    && merger.selectPartsToMerge(future_merged_part, false, max_parts_size_for_merge, can_merge))
1890
                {
1891 1892
                    merge_selecting_logs_pulling_is_required = true;
                    success = createLogEntryToMergeParts(future_merged_part.parts, future_merged_part.name, deduplicate);
1893
                }
1894 1895
            }
        }
1896 1897 1898 1899
        catch (...)
        {
            tryLogCurrentException(__PRETTY_FUNCTION__);
        }
1900

1901 1902
        if (shutdown_called || !is_leader_node)
            break;
S
Silviu Caragea 已提交
1903

1904 1905 1906 1907 1908
        if (!success)
            merge_selecting_event.tryWait(MERGE_SELECTING_SLEEP_MS);
    }

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

M
Merge  
Michael Kolupaev 已提交
1911

1912
bool StorageReplicatedMergeTree::createLogEntryToMergeParts(
Y
Yuri Dyachenko 已提交
1913
    const MergeTreeData::DataPartsVector & parts, const String & merged_name, bool deduplicate, ReplicatedMergeTreeLogEntryData * out_log_entry)
1914
{
1915 1916 1917 1918 1919 1920 1921 1922 1923 1924
    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;

1925
            if (part->modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER < time(nullptr))
1926 1927
            {
                LOG_WARNING(log, "Part " << part->name << " (that was selected for merge)"
1928
                    << " with age " << (time(nullptr) - part->modification_time)
1929 1930 1931 1932 1933 1934 1935 1936 1937
                    << " 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;

1938
    ReplicatedMergeTreeLogEntryData entry;
1939 1940 1941
    entry.type = LogEntry::MERGE_PARTS;
    entry.source_replica = replica_name;
    entry.new_part_name = merged_name;
Y
Yuri Dyachenko 已提交
1942
    entry.deduplicate = deduplicate;
1943
    entry.create_time = time(nullptr);
1944 1945 1946 1947 1948 1949 1950

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

1951
    const String & partition_id = parts[0]->info.partition_id;
1952 1953 1954
    for (size_t i = 0; i + 1 < parts.size(); ++i)
    {
        /// Remove the unnecessary entries about non-existent blocks.
1955
        for (Int64 number = parts[i]->info.max_block + 1; number <= parts[i + 1]->info.min_block - 1; ++number)
1956
        {
1957
            zookeeper->tryRemove(zookeeper_path + "/block_numbers/" + partition_id + "/block-" + padIndex(number));
1958
            zookeeper->tryRemove(zookeeper_path + "/nonincrement_block_numbers/" + partition_id + "/block-" + padIndex(number));
1959 1960 1961 1962 1963 1964 1965
        }
    }

    if (out_log_entry)
        *out_log_entry = entry;

    return true;
1966 1967 1968
}


1969 1970
void StorageReplicatedMergeTree::removePartFromZooKeeper(const String & part_name, zkutil::Ops & ops)
{
1971
    String part_path = replica_path + "/parts/" + part_name;
1972

1973 1974 1975
    ops.emplace_back(std::make_unique<zkutil::Op::Remove>(part_path + "/checksums", -1));
    ops.emplace_back(std::make_unique<zkutil::Op::Remove>(part_path + "/columns", -1));
    ops.emplace_back(std::make_unique<zkutil::Op::Remove>(part_path, -1));
1976 1977 1978
}


M
Merge  
Michael Kolupaev 已提交
1979 1980
void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_name)
{
1981
    auto zookeeper = getZooKeeper();
A
Merge  
Alexey Milovidov 已提交
1982

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

1985 1986 1987 1988 1989
    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 已提交
1990

1991 1992 1993 1994
    zkutil::Ops ops;
    ops.emplace_back(std::make_unique<zkutil::Op::Create>(
        replica_path + "/queue/queue-", log_entry->toString(), zookeeper->getDefaultACL(),
        zkutil::CreateMode::PersistentSequential));
1995

1996
    removePartFromZooKeeper(part_name, ops);
1997

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

2000 2001 2002
    String path_created = dynamic_cast<zkutil::Op::Create &>(*ops[0]).getPathCreated();
    log_entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
    queue.insert(zookeeper, log_entry);
M
Merge  
Michael Kolupaev 已提交
2003 2004
}

A
Merge  
Alexey Milovidov 已提交
2005

M
Merge  
Michael Kolupaev 已提交
2006 2007
void StorageReplicatedMergeTree::becomeLeader()
{
2008
    std::lock_guard<std::mutex> lock(leader_node_mutex);
2009

2010 2011
    if (shutdown_called)
        return;
2012

2013 2014 2015 2016 2017 2018 2019
    if (merge_selecting_thread.joinable())
    {
        LOG_INFO(log, "Deleting old leader");
        is_leader_node = false; /// exit trigger inside thread
        merge_selecting_thread.join();
    }

2020 2021
    LOG_INFO(log, "Became leader");
    is_leader_node = true;
2022
    merge_selecting_thread = std::thread(&StorageReplicatedMergeTree::mergeSelectingThread, this);
M
Merge  
Michael Kolupaev 已提交
2023 2024
}

A
Merge  
Alexey Milovidov 已提交
2025

M
Merge  
Michael Kolupaev 已提交
2026
String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_name, bool active)
M
Merge  
Michael Kolupaev 已提交
2027
{
2028 2029
    auto zookeeper = getZooKeeper();
    Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
M
Merge  
Michael Kolupaev 已提交
2030

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

2034 2035 2036 2037 2038
    for (const String & replica : replicas)
    {
        /// We don't interested in ourself.
        if (replica == replica_name)
            continue;
A
Alexey Milovidov 已提交
2039

2040 2041 2042
        if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name) &&
            (!active || zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")))
            return replica;
2043

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

2047
    return {};
2048 2049 2050
}


2051
String StorageReplicatedMergeTree::findReplicaHavingCoveringPart(const LogEntry & entry, bool active)
2052
{
2053 2054 2055 2056
    auto zookeeper = getZooKeeper();
    Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");

    /// Select replicas in uniformly random order.
2057
    std::shuffle(replicas.begin(), replicas.end(), rng);
2058 2059 2060 2061 2062 2063 2064 2065 2066 2067 2068 2069 2070

    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)
        {
2071 2072
            if (part_on_replica == entry.new_part_name
                || MergeTreePartInfo::contains(part_on_replica, entry.new_part_name, data.format_version))
2073
            {
2074 2075
                if (largest_part_found.empty()
                    || MergeTreePartInfo::contains(part_on_replica, largest_part_found, data.format_version))
2076 2077 2078 2079 2080 2081 2082 2083
                {
                    largest_part_found = part_on_replica;
                }
            }
        }

        if (!largest_part_found.empty())
        {
2084 2085 2086 2087 2088 2089 2090 2091 2092 2093 2094 2095 2096 2097 2098 2099 2100
            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;
            }

2101 2102 2103 2104 2105
            return replica;
        }
    }

    return {};
M
Merge  
Michael Kolupaev 已提交
2106 2107
}

A
Merge  
Alexey Milovidov 已提交
2108

F
f1yegor 已提交
2109
/** If a quorum is tracked for a part, update information about it in ZK.
2110
  */
2111
void StorageReplicatedMergeTree::updateQuorum(const String & part_name)
2112
{
2113 2114 2115 2116 2117 2118 2119 2120 2121 2122 2123 2124 2125 2126 2127 2128 2129 2130 2131 2132 2133 2134 2135 2136 2137 2138 2139 2140 2141 2142 2143 2144 2145 2146 2147 2148 2149 2150 2151 2152 2153 2154 2155 2156 2157 2158 2159 2160 2161 2162 2163 2164 2165 2166 2167 2168 2169 2170 2171 2172 2173 2174 2175 2176 2177 2178 2179 2180 2181 2182 2183 2184 2185
    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.

            zkutil::Ops ops;
            ops.emplace_back(std::make_unique<zkutil::Op::Remove>(quorum_status_path, stat.version));
            ops.emplace_back(std::make_unique<zkutil::Op::SetData>(quorum_last_part_path, part_name, -1));
            auto code = zookeeper->tryMulti(ops);

            if (code == ZOK)
            {
                break;
            }
            else if (code == ZNONODE)
            {
                /// The quorum has already been achieved.
                break;
            }
            else if (code == ZBADVERSION)
            {
                /// 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);

            if (code == ZOK)
            {
                break;
            }
            else if (code == ZNONODE)
            {
                /// The quorum has already been achieved.
                break;
            }
            else if (code == ZBADVERSION)
            {
                /// Node was updated meanwhile. We must re-read it and repeat all the actions.
                continue;
            }
            else
                throw zkutil::KeeperException(code, quorum_status_path);
        }
    }
2186 2187 2188
}


A
Alexey Milovidov 已提交
2189
bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const String & replica_path, bool to_detached, size_t quorum)
M
Merge  
Michael Kolupaev 已提交
2190
{
2191 2192 2193
    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");
2194
        /// Force immediate parts cleanup to delete the part that was left from the previous fetch attempt.
2195
        cleanup_thread_event.set();
2196 2197 2198
        return false;
    }

2199 2200 2201 2202 2203 2204 2205 2206 2207 2208 2209 2210 2211 2212 2213 2214 2215 2216 2217
    {
        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)
2218
        table_lock = lockStructure(true, __PRETTY_FUNCTION__);
2219 2220

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

2223
    /// Logging
2224
    Stopwatch stopwatch;
2225 2226
    MergeTreeData::MutableDataPartPtr part;
    MergeTreeData::DataPartsVector replaced_parts;
2227

2228
    auto write_part_log = [&] (const ExecutionStatus & execution_status)
2229
    {
2230 2231
        try
        {
2232
            auto part_log = context.getPartLog(database_name);
2233 2234
            if (!part_log)
                return;
2235

2236
            PartLogElement part_log_elem;
2237

2238 2239 2240 2241
            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;
2242

2243 2244 2245 2246 2247 2248 2249 2250 2251 2252 2253 2254 2255 2256 2257 2258 2259 2260 2261 2262
            part_log_elem.database_name = database_name;
            part_log_elem.table_name = table_name;
            part_log_elem.part_name = part_name;

            if (part)
            {
                part_log_elem.bytes_compressed_on_disk = part->size_in_bytes;
                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 (...)
2263
        {
2264
            tryLogCurrentException(log, __PRETTY_FUNCTION__);
2265
        }
2266 2267 2268 2269 2270
    };

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

2272
        if (!to_detached)
2273
        {
2274
            zkutil::Ops ops;
2275

2276 2277 2278 2279 2280 2281
            /** 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.
              */
            checkPartAndAddToZooKeeper(part, ops, part_name);
2282

2283
            MergeTreeData::Transaction transaction;
2284
            data.renameTempPartAndReplace(part, nullptr, &transaction);
2285

2286 2287
            /// Do not commit if the part is obsolete
            if (!transaction.isEmpty())
2288
            {
2289
                getZooKeeper()->multi(ops);
2290
                replaced_parts = transaction.commit();
2291 2292
            }

2293 2294 2295 2296 2297
            /** 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);
2298

2299 2300 2301 2302 2303 2304 2305
            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);
            }
2306

2307 2308 2309
            write_part_log({});
        }
        else
2310
        {
2311
            part->renameTo("detached/" + part_name);
2312 2313
        }
    }
2314
    catch (...)
2315
    {
2316 2317 2318 2319
        if (!to_detached)
            write_part_log(ExecutionStatus::fromCurrentException());

        throw;
2320 2321 2322 2323 2324 2325
    }

    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 已提交
2326
}
M
Merge  
Michael Kolupaev 已提交
2327

A
Merge  
Alexey Milovidov 已提交
2328

2329 2330 2331 2332 2333 2334 2335 2336 2337 2338 2339
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);
2340 2341 2342
    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.
2343

A
Alexey Zatelepin 已提交
2344 2345 2346 2347 2348
    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());

2349 2350 2351 2352 2353
    /// In this thread replica will be activated.
    restarting_thread = std::make_unique<ReplicatedMergeTreeRestartingThread>(*this);
}


M
Merge  
Michael Kolupaev 已提交
2354 2355
void StorageReplicatedMergeTree::shutdown()
{
2356 2357 2358 2359
    /** This must be done before waiting for restarting_thread.
      * Because restarting_thread will wait for finishing of tasks in background pool,
      *  and parts are fetched in that tasks.
      */
2360
    fetcher.blocker.cancelForever();
2361 2362 2363 2364 2365 2366 2367

    if (restarting_thread)
    {
        restarting_thread->stop();
        restarting_thread.reset();
    }

2368
    if (data_parts_exchange_endpoint_holder)
2369
    {
2370 2371
        data_parts_exchange_endpoint_holder->cancelForever();
        data_parts_exchange_endpoint_holder = nullptr;
2372
    }
M
Merge  
Michael Kolupaev 已提交
2373 2374 2375
}


M
Merge  
Michael Kolupaev 已提交
2376 2377
StorageReplicatedMergeTree::~StorageReplicatedMergeTree()
{
2378 2379 2380 2381 2382 2383 2384 2385
    try
    {
        shutdown();
    }
    catch(...)
    {
        tryLogCurrentException(__PRETTY_FUNCTION__);
    }
M
Merge  
Michael Kolupaev 已提交
2386 2387
}

A
Merge  
Alexey Milovidov 已提交
2388

M
Merge  
Michael Kolupaev 已提交
2389
BlockInputStreams StorageReplicatedMergeTree::read(
2390
    const Names & column_names,
2391
    const SelectQueryInfo & query_info,
2392 2393 2394
    const Context & context,
    QueryProcessingStage::Enum & processed_stage,
    const size_t max_block_size,
2395
    const unsigned num_streams)
M
Merge  
Michael Kolupaev 已提交
2396
{
2397
    const Settings & settings = context.getSettingsRef();
2398

2399 2400 2401 2402 2403 2404 2405
    /** 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)
2406
    {
2407
        auto zookeeper = getZooKeeper();
2408

2409 2410
        String last_part;
        zookeeper->tryGet(zookeeper_path + "/quorum/last_part", last_part);
2411

2412
        if (!last_part.empty() && !data.getActiveContainingPart(last_part))    /// TODO Disable replica for distributed queries.
2413 2414
            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);
2415

2416 2417 2418 2419
        if (last_part.empty())  /// If no part has been written with quorum.
        {
            String quorum_str;
            if (zookeeper->tryGet(zookeeper_path + "/quorum/status", quorum_str))
2420
            {
2421 2422
                ReplicatedMergeTreeQuorumEntry quorum_entry;
                quorum_entry.fromString(quorum_str);
2423
                auto part_info = MergeTreePartInfo::fromPartName(quorum_entry.part_name, data.format_version);
2424
                max_block_number_to_read = part_info.min_block - 1;
2425 2426
            }
        }
2427
        else
2428
        {
2429
            auto part_info = MergeTreePartInfo::fromPartName(last_part, data.format_version);
2430
            max_block_number_to_read = part_info.max_block;
2431 2432 2433
        }
    }

2434
    return reader.read(
A
Alexey Zatelepin 已提交
2435
        column_names, query_info, context, processed_stage, max_block_size, num_streams, max_block_number_to_read);
M
Merge  
Michael Kolupaev 已提交
2436 2437
}

A
Merge  
Alexey Milovidov 已提交
2438

A
Merge  
Alexey Milovidov 已提交
2439
void StorageReplicatedMergeTree::assertNotReadonly() const
M
Merge  
Michael Kolupaev 已提交
2440
{
2441 2442
    if (is_readonly)
        throw Exception("Table is in readonly mode", ErrorCodes::TABLE_IS_READ_ONLY);
A
Merge  
Alexey Milovidov 已提交
2443 2444 2445
}


A
Alexey Milovidov 已提交
2446
BlockOutputStreamPtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const Settings & settings)
A
Merge  
Alexey Milovidov 已提交
2447
{
2448
    assertNotReadonly();
M
Merge  
Michael Kolupaev 已提交
2449

2450 2451
    bool deduplicate = data.settings.replicated_deduplication_window != 0 && settings.insert_deduplicate;

2452
    return std::make_shared<ReplicatedMergeTreeBlockOutputStream>(*this,
2453
        settings.insert_quorum, settings.insert_quorum_timeout.totalMilliseconds(), deduplicate);
M
Merge  
Michael Kolupaev 已提交
2454
}
M
Merge  
Michael Kolupaev 已提交
2455

A
Merge  
Alexey Milovidov 已提交
2456

2457
bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context)
M
Merge  
Michael Kolupaev 已提交
2458
{
2459
    assertNotReadonly();
2460

2461
    if (!is_leader_node)
2462
    {
2463
        sendRequestToLeaderReplica(query, context.getSettingsRef());
2464 2465
        return true;
    }
2466

2467
    auto can_merge = [this] (const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, String * out_reason)
2468
    {
2469 2470
        return partsWillNotBeMergedOrDisabled(left, right, queue, out_reason)
               && canMergePartsAccordingToZooKeeperInfo(left, right, getZooKeeper(), zookeeper_path, data, out_reason);
2471
    };
2472

2473 2474 2475
    ReplicatedMergeTreeLogEntryData merge_entry;
    {
        std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
2476

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

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

A
Alexey Zatelepin 已提交
2482
        MergeTreeDataMerger::FuturePart future_merged_part;
2483
        String disable_reason;
2484
        bool selected = false;
2485

2486
        if (!partition)
2487
        {
A
Alexey Zatelepin 已提交
2488
            selected = merger.selectPartsToMerge(
2489
                future_merged_part, true, data.settings.max_bytes_to_merge_at_max_space_in_pool, can_merge, &disable_reason);
2490 2491 2492
        }
        else
        {
2493
            String partition_id = data.getPartitionIDFromQuery(partition, context);
2494
            selected = merger.selectAllPartsToMergeWithinPartition(future_merged_part, disk_space, can_merge, partition_id, final, &disable_reason);
2495
        }
2496

2497
        auto handle_noop = [&] (const String & message)
2498
        {
2499 2500
            if (context.getSettingsRef().optimize_throw_if_noop)
                throw Exception(message, ErrorCodes::CANNOT_ASSIGN_OPTIMIZE);
2501
            return false;
2502 2503 2504 2505 2506 2507
        };

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

2510 2511 2512
        /// 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 已提交
2513
        if (!createLogEntryToMergeParts(future_merged_part.parts, future_merged_part.name, deduplicate, &merge_entry))
2514
            return handle_noop("Can't create merge queue node in ZooKeeper");
2515
    }
2516

2517 2518
    waitForAllReplicasToProcessLogEntry(merge_entry);
    return true;
M
Merge  
Michael Kolupaev 已提交
2519 2520
}

A
Merge  
Alexey Milovidov 已提交
2521

M
Merge  
Michael Kolupaev 已提交
2522
void StorageReplicatedMergeTree::alter(const AlterCommands & params,
A
Alexey Milovidov 已提交
2523
    const String & /*database_name*/, const String & /*table_name*/, const Context & context)
M
Merge  
Michael Kolupaev 已提交
2524
{
2525
    assertNotReadonly();
A
Merge  
Alexey Milovidov 已提交
2526

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

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

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

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

2540
        data.checkAlter(params);
M
Merge  
Michael Kolupaev 已提交
2541

2542 2543 2544 2545
        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);

2546
        ColumnsDescription new_columns = data.getColumns();
2547
        params.apply(new_columns);
2548

2549
        new_columns_str = new_columns.toString();
2550 2551 2552 2553 2554 2555 2556 2557 2558 2559 2560 2561 2562 2563 2564 2565 2566 2567 2568 2569 2570 2571 2572 2573 2574 2575 2576 2577 2578 2579 2580 2581 2582 2583 2584 2585 2586 2587 2588 2589

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

2591 2592 2593
                inactive_replicas.emplace(replica);
                break;
            }
2594

2595
            String replica_columns_str;
M
Merge  
Michael Kolupaev 已提交
2596

2597 2598 2599 2600 2601 2602
            /// 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 已提交
2603

2604
            int replica_columns_version = stat.version;
M
Merge  
Michael Kolupaev 已提交
2605

2606 2607 2608
            /// The ALTER has been successfully applied.
            if (replica_columns_str == new_columns_str)
                break;
M
Merge  
Michael Kolupaev 已提交
2609

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

2613 2614 2615 2616 2617 2618
            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 已提交
2619

2620 2621 2622 2623 2624 2625 2626 2627 2628 2629 2630 2631 2632 2633 2634 2635 2636 2637 2638 2639 2640 2641 2642 2643 2644 2645 2646 2647 2648 2649 2650 2651 2652 2653 2654 2655 2656 2657 2658 2659 2660 2661 2662 2663 2664 2665 2666 2667 2668 2669 2670 2671 2672 2673 2674 2675 2676 2677 2678 2679 2680 2681 2682 2683 2684
            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 已提交
2685 2686
}

M
Merge  
Michael Kolupaev 已提交
2687

2688
/// 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.
2689 2690
static String getFakePartNameCoveringPartRange(
        MergeTreeDataFormatVersion format_version, const String & partition_id, UInt64 left, UInt64 right)
M
Merge  
Michael Kolupaev 已提交
2691
{
2692
    /// Artificial high level is choosen, to make this part "covering" all parts inside.
2693
    MergeTreePartInfo part_info(partition_id, left, right, 999999999);
2694
    if (format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING)
2695 2696 2697 2698 2699 2700 2701 2702 2703 2704
    {
        /// 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 已提交
2705 2706
}

A
Merge  
Alexey Milovidov 已提交
2707

2708 2709
String StorageReplicatedMergeTree::getFakePartNameCoveringAllPartsInPartition(
    const String & partition_id, Int64 * out_min_block, Int64 * out_max_block)
A
Merge  
Andrey Mironov 已提交
2710
{
2711 2712
    /// 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.
2713
    Int64 left = 0;
2714

2715
    /** Let's skip one number in `block_numbers` for the partition being deleted, and we will only delete parts until this number.
2716 2717 2718 2719 2720 2721 2722 2723
      * 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;

    {
2724
        auto zookeeper = getZooKeeper();
2725
        AbandonableLockInZooKeeper block_number_lock = allocateBlockNumber(partition_id, zookeeper);
2726 2727 2728 2729
        right = block_number_lock.getNumber();
        block_number_lock.unlock();
    }

2730
    /// Empty partition.
2731
    if (right == 0)
2732
        return {};
2733

2734
    --right;
2735 2736 2737 2738 2739

    if (out_min_block)
        *out_min_block = left;
    if (out_max_block)
        *out_max_block = right;
2740
    return getFakePartNameCoveringPartRange(data.format_version, partition_id, left, right);
2741 2742 2743
}


2744
void StorageReplicatedMergeTree::clearColumnInPartition(
2745
    const ASTPtr & partition, const Field & column_name, const Context & context)
2746 2747 2748 2749 2750
{
    assertNotReadonly();

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

2751
    String partition_id = data.getPartitionIDFromQuery(partition, context);
2752
    String fake_part_name = getFakePartNameCoveringAllPartsInPartition(partition_id);
2753

2754 2755
    if (fake_part_name.empty())
    {
2756
        LOG_INFO(log, "Will not clear partition " << partition_id << ", it is empty.");
2757 2758 2759
        return;
    }

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

2762
    LogEntry entry;
2763
    entry.type = LogEntry::CLEAR_COLUMN;
2764 2765
    entry.new_part_name = fake_part_name;
    entry.column_name = column_name.safeGet<String>();
2766
    entry.create_time = time(nullptr);
2767 2768 2769 2770 2771

    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.
2772
    if (context.getSettingsRef().replication_alter_partitions_sync != 0)
2773
    {
2774
        if (context.getSettingsRef().replication_alter_partitions_sync == 1)
2775 2776 2777 2778 2779 2780
            waitForReplicaToProcessLogEntry(replica_name, entry);
        else
            waitForAllReplicasToProcessLogEntry(entry);
    }
}

2781
void StorageReplicatedMergeTree::dropPartition(const ASTPtr & query, const ASTPtr & partition, bool detach, const Context & context)
2782 2783 2784
{
    assertNotReadonly();

2785 2786
    zkutil::ZooKeeperPtr zookeeper = getZooKeeper();

2787 2788
    if (!is_leader_node)
    {
2789
        sendRequestToLeaderReplica(query, context.getSettingsRef());
2790 2791 2792
        return;
    }

2793
    String partition_id = data.getPartitionIDFromQuery(partition, context);
2794 2795 2796 2797

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

2799 2800
    if (fake_part_name.empty())
    {
2801
        LOG_INFO(log, "Will not drop partition " << partition_id << ", it is empty.");
2802 2803 2804
        return;
    }

2805 2806
    clearBlocksInPartition(*zookeeper, partition_id, min_block, max_block);

2807
    /** Forbid to choose the parts to be deleted for merging.
F
f1yegor 已提交
2808
      * Invariant: after the `DROP_RANGE` entry appears in the log, merge of deleted parts will not appear in the log.
2809 2810 2811 2812 2813 2814
      */
    {
        std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
        queue.disableMergesInRange(fake_part_name);
    }

2815
    LOG_DEBUG(log, "Disabled merges covered by range " << fake_part_name);
2816 2817 2818 2819 2820 2821 2822

    /// 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;
2823
    entry.create_time = time(nullptr);
2824

2825
    String log_znode_path = zookeeper->create(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential);
2826 2827 2828
    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.
2829
    if (context.getSettingsRef().replication_alter_partitions_sync != 0)
2830
    {
2831
        if (context.getSettingsRef().replication_alter_partitions_sync == 1)
2832 2833 2834 2835
            waitForReplicaToProcessLogEntry(replica_name, entry);
        else
            waitForAllReplicasToProcessLogEntry(entry);
    }
A
Merge  
Alexey Milovidov 已提交
2836
}
A
Merge  
Alexey Milovidov 已提交
2837

A
Merge  
Alexey Arno 已提交
2838

2839
void StorageReplicatedMergeTree::attachPartition(const ASTPtr & partition, bool attach_part, const Context & context)
M
Merge  
Michael Kolupaev 已提交
2840
{
2841 2842
    assertNotReadonly();

2843
    String partition_id;
2844 2845

    if (attach_part)
2846
        partition_id = typeid_cast<const ASTLiteral &>(*partition).value.safeGet<String>();
2847
    else
2848
        partition_id = data.getPartitionIDFromQuery(partition, context);
2849

2850
    String source_dir = "detached/";
2851 2852 2853 2854 2855

    /// Let's compose a list of parts that should be added.
    Strings parts;
    if (attach_part)
    {
2856
        parts.push_back(partition_id);
2857 2858 2859
    }
    else
    {
2860
        LOG_DEBUG(log, "Looking for parts for partition " << partition_id << " in " << source_dir);
2861
        ActiveDataPartSet active_parts(data.format_version);
2862 2863 2864 2865 2866

        std::set<String> part_names;
        for (Poco::DirectoryIterator it = Poco::DirectoryIterator(full_path + source_dir); it != Poco::DirectoryIterator(); ++it)
        {
            String name = it.name();
2867
            MergeTreePartInfo part_info;
2868
            if (!MergeTreePartInfo::tryParsePartName(name, &part_info, data.format_version))
2869
                continue;
2870
            if (part_info.partition_id != partition_id)
2871 2872 2873 2874 2875 2876 2877 2878 2879 2880 2881 2882 2883 2884 2885 2886 2887
                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);
        }
    }

2888
    /// Synchronously check that added parts exist and are not broken. We will write checksums.txt if it does not exist.
2889
    LOG_DEBUG(log, "Checking parts");
2890
    std::vector<MergeTreeData::MutableDataPartPtr> loaded_parts;
2891 2892 2893
    for (const String & part : parts)
    {
        LOG_DEBUG(log, "Checking part " << part);
2894
        loaded_parts.push_back(data.loadPartAndFixMetadata(source_dir + part));
2895 2896
    }

2897
    ReplicatedMergeTreeBlockOutputStream output(*this, 0, 0, false);   /// TODO Allow to use quorum here.
2898
    for (auto & part : loaded_parts)
2899
    {
2900 2901 2902
        String old_name = part->name;
        output.writeExistingPart(part);
        LOG_DEBUG(log, "Attached part " << old_name << " as " << part->name);
2903
    }
M
Merge  
Michael Kolupaev 已提交
2904 2905
}

2906

2907 2908
bool StorageReplicatedMergeTree::checkTableCanBeDropped() const
{
2909
    /// Consider only synchronized data
2910
    const_cast<MergeTreeData &>(getData()).recalculateColumnSizes();
2911 2912
    context.checkTableCanBeDropped(database_name, table_name, getData().getTotalCompressedSize());
    return true;
2913
}
A
Merge  
Alexey Milovidov 已提交
2914

2915

M
Merge  
Michael Kolupaev 已提交
2916 2917
void StorageReplicatedMergeTree::drop()
{
2918 2919
    {
        auto zookeeper = tryGetZooKeeper();
A
Merge  
Alexey Milovidov 已提交
2920

2921 2922
        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 已提交
2923

2924
        // checkTableCanBeDropped(); // uncomment to feel yourself safe
2925

2926
        shutdown();
M
Merge  
Michael Kolupaev 已提交
2927

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

2931 2932 2933
        LOG_INFO(log, "Removing replica " << replica_path);
        replica_is_active_node = nullptr;
        zookeeper->tryRemoveRecursive(replica_path);
M
Merge  
Michael Kolupaev 已提交
2934

2935 2936 2937 2938 2939 2940 2941 2942
        /// Check that `zookeeper_path` exists: it could have been deleted by another replica after execution of previous line.
        Strings replicas;
        if (zookeeper->tryGetChildren(zookeeper_path + "/replicas", replicas) == ZOK && replicas.empty())
        {
            LOG_INFO(log, "Removing table " << zookeeper_path << " (this might take several minutes)");
            zookeeper->tryRemoveRecursive(zookeeper_path);
        }
    }
M
Merge  
Michael Kolupaev 已提交
2943

2944
    data.dropAllData();
M
Merge  
Michael Kolupaev 已提交
2945 2946
}

A
Merge  
Alexey Milovidov 已提交
2947

M
Merge  
Michael Kolupaev 已提交
2948 2949
void StorageReplicatedMergeTree::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name)
{
2950
    std::string new_full_path = new_path_to_db + escapeForFileName(new_table_name) + '/';
M
Merge  
Michael Kolupaev 已提交
2951

2952
    data.setPath(new_full_path);
M
Merge  
Michael Kolupaev 已提交
2953

2954 2955 2956
    database_name = new_database_name;
    table_name = new_table_name;
    full_path = new_full_path;
M
Merge  
Michael Kolupaev 已提交
2957

2958
    /// TODO: You can update names of loggers.
M
Merge  
Michael Kolupaev 已提交
2959 2960
}

A
Merge  
Alexey Milovidov 已提交
2961

2962 2963
bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path)
{
2964 2965 2966 2967 2968
    {
        std::lock_guard<std::mutex> lock(existing_nodes_cache_mutex);
        if (existing_nodes_cache.count(path))
            return true;
    }
2969

2970
    bool res = getZooKeeper()->exists(path);
2971

2972 2973 2974 2975 2976
    if (res)
    {
        std::lock_guard<std::mutex> lock(existing_nodes_cache_mutex);
        existing_nodes_cache.insert(path);
    }
2977

2978
    return res;
2979 2980 2981
}


2982 2983
AbandonableLockInZooKeeper StorageReplicatedMergeTree::allocateBlockNumber(const String & partition_id, zkutil::ZooKeeperPtr & zookeeper,
                                                                           zkutil::Ops * precheck_ops)
M
Merge  
Michael Kolupaev 已提交
2984
{
2985 2986
    String partition_path = zookeeper_path + "/block_numbers/" + partition_id;
    if (!existsNodeCached(partition_path))
2987
    {
2988
        int code = zookeeper->tryCreate(partition_path, "", zkutil::CreateMode::Persistent);
2989
        if (code != ZOK && code != ZNODEEXISTS)
2990
            throw zkutil::KeeperException(code, partition_path);
2991
    }
2992 2993

    return AbandonableLockInZooKeeper(
2994
        partition_path + "/block-",
2995
        zookeeper_path + "/temp", *zookeeper, precheck_ops);
M
Merge  
Michael Kolupaev 已提交
2996 2997
}

A
Merge  
Alexey Milovidov 已提交
2998

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

3003 3004 3005
    Strings replicas = getZooKeeper()->getChildren(zookeeper_path + "/replicas");
    for (const String & replica : replicas)
        waitForReplicaToProcessLogEntry(replica, entry);
A
Merge  
Alexey Milovidov 已提交
3006

3007
    LOG_DEBUG(log, "Finished waiting for all replicas to process " << entry.znode_name);
A
Merge  
Alexey Milovidov 已提交
3008 3009 3010
}


3011
void StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry(const String & replica, const ReplicatedMergeTreeLogEntryData & entry)
A
Merge  
Alexey Milovidov 已提交
3012
{
3013 3014 3015 3016 3017 3018 3019 3020 3021 3022 3023 3024 3025 3026 3027 3028 3029 3030 3031 3032 3033 3034 3035 3036 3037 3038 3039 3040 3041 3042 3043 3044 3045 3046 3047 3048 3049 3050 3051 3052 3053 3054 3055 3056 3057 3058 3059 3060 3061 3062 3063 3064 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
    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 已提交
3138 3139 3140
}


3141
void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields)
3142
{
3143 3144 3145 3146 3147 3148 3149
    auto zookeeper = tryGetZooKeeper();

    res.is_leader = is_leader_node;
    res.is_readonly = is_readonly;
    res.is_session_expired = !zookeeper || zookeeper->expired();

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

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

3193

3194 3195 3196 3197 3198 3199 3200 3201 3202 3203 3204 3205 3206 3207 3208
/// 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"));

3209
    /// TODO: add setters and getters interface for database and table fields of AST
3210
    auto new_query = query->clone();
3211 3212 3213 3214 3215 3216 3217 3218 3219 3220 3221 3222
    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);
3223 3224 3225

    /// NOTE Works only if there is access from the default user without a password. You can fix it by adding a parameter to the server config.

3226
    auto timeouts = ConnectionTimeouts::getTCPTimeouts(context.getSettingsRef());
3227 3228 3229 3230
    Connection connection(
        leader_address.host,
        leader_address.queries_port,
        leader_address.database,
3231
        "", "", timeouts, "ClickHouse replica");
3232

3233
    RemoteBlockInputStream stream(connection, formattedAST(new_query), {}, context, &settings);
3234
    NullBlockOutputStream output({});
3235 3236 3237 3238 3239 3240

    copyData(stream, output);
    return;
}


3241 3242
void StorageReplicatedMergeTree::getQueue(LogEntriesData & res, String & replica_name_)
{
3243 3244
    replica_name_ = replica_name;
    queue.getEntries(res);
3245 3246
}

3247 3248 3249 3250 3251 3252
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);

3253 3254 3255 3256
    /// 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();
3257 3258 3259

    time_t current_time = time(nullptr);

3260
    if (!queue_update_finish_time)
3261
    {
3262
        /// We have not updated queue even once yet (perhaps replica is readonly).
3263 3264 3265 3266 3267 3268 3269 3270
        /// 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;
    }
3271
    else if (queue_update_start_time > queue_update_finish_time)
3272 3273 3274 3275
    {
        /// 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.
3276
        return (current_time > queue_update_start_time) ? (current_time - queue_update_start_time) : 0;
3277 3278 3279 3280 3281 3282 3283
    }
    else
    {
        /// Everything is up-to-date.
        return 0;
    }
}
3284

3285
void StorageReplicatedMergeTree::getReplicaDelays(time_t & out_absolute_delay, time_t & out_relative_delay)
3286
{
3287
    assertNotReadonly();
3288

3289
    time_t current_time = time(nullptr);
3290

3291
    out_absolute_delay = getAbsoluteDelay();
3292
    out_relative_delay = 0;
3293

3294 3295 3296 3297
    /** 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.
      */
3298

3299 3300
    if (out_absolute_delay < static_cast<time_t>(data.settings.min_relative_delay_to_yield_leadership))
        return;
3301

3302
    auto zookeeper = getZooKeeper();
3303

3304 3305
    time_t max_replicas_unprocessed_insert_time = 0;
    bool have_replica_with_nothing_unprocessed = false;
3306

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

3309 3310 3311 3312
    for (const auto & replica : replicas)
    {
        if (replica == replica_name)
            continue;
3313

3314 3315 3316
        /// Skip dead replicas.
        if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))
            continue;
3317

3318 3319 3320
        String value;
        if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/min_unprocessed_insert_time", value))
            continue;
3321

3322
        time_t replica_time = value.empty() ? 0 : parse<time_t>(value);
3323

3324 3325 3326 3327 3328 3329 3330 3331 3332
        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.
              */
3333

3334 3335 3336
            have_replica_with_nothing_unprocessed = true;
            break;
        }
3337

3338 3339 3340
        if (replica_time > max_replicas_unprocessed_insert_time)
            max_replicas_unprocessed_insert_time = replica_time;
    }
3341

3342 3343
    if (have_replica_with_nothing_unprocessed)
        out_relative_delay = out_absolute_delay;
3344 3345 3346 3347 3348 3349 3350
    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;
    }
3351 3352 3353
}


3354
void StorageReplicatedMergeTree::fetchPartition(const ASTPtr & partition, const String & from_, const Context & context)
3355
{
3356
    String partition_id = data.getPartitionIDFromQuery(partition, context);
3357 3358 3359 3360 3361

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

3362
    LOG_INFO(log, "Will fetch partition " << partition_id << " from shard " << from_);
3363 3364 3365 3366 3367 3368

    /** 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)
3369 3370
    {
        MergeTreePartInfo part_info;
3371 3372
        if (MergeTreePartInfo::tryParsePartName(dir_it.name(), &part_info, data.format_version)
              && part_info.partition_id == partition_id)
3373 3374
            throw Exception("Detached partition " + partition_id + " already exists.", ErrorCodes::PARTITION_ALREADY_EXISTS);
    }
3375 3376 3377 3378 3379 3380 3381 3382 3383 3384 3385 3386 3387 3388 3389 3390 3391 3392 3393 3394 3395 3396 3397 3398 3399 3400 3401 3402 3403 3404 3405 3406 3407 3408 3409 3410 3411 3412 3413 3414 3415 3416 3417 3418 3419 3420 3421 3422 3423 3424 3425 3426 3427 3428 3429 3430 3431 3432 3433 3434 3435 3436 3437 3438 3439 3440 3441 3442 3443 3444 3445 3446 3447 3448

    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 已提交
3449
            throw Exception("Too many retries to fetch parts from " + best_replica_path, ErrorCodes::TOO_MANY_RETRIES_TO_FETCH_PARTS);
3450 3451

        Strings parts = getZooKeeper()->getChildren(best_replica_path + "/parts");
3452
        ActiveDataPartSet active_parts_set(data.format_version, parts);
3453 3454 3455 3456 3457 3458 3459 3460 3461
        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)
3462
            {
3463
                if (MergeTreePartInfo::fromPartName(part, data.format_version).partition_id == partition_id)
3464
                    parts_to_fetch_partition.push_back(part);
3465
            }
3466 3467 3468 3469

            parts_to_fetch = std::move(parts_to_fetch_partition);

            if (parts_to_fetch.empty())
3470
                throw Exception("Partition " + partition_id + " on " + best_replica_path + " doesn't exist", ErrorCodes::PARTITION_DOESNT_EXIST);
3471 3472 3473 3474 3475 3476 3477 3478 3479 3480 3481 3482 3483 3484 3485 3486 3487 3488 3489 3490 3491 3492 3493 3494
        }
        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)
            {
3495
                if (e.code() != ErrorCodes::RECEIVED_ERROR_FROM_REMOTE_IO_SERVER && e.code() != ErrorCodes::RECEIVED_ERROR_TOO_MANY_REQUESTS)
3496 3497 3498 3499 3500 3501 3502 3503 3504
                    throw;

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

        ++try_no;
    } while (!missing_parts.empty());
3505 3506 3507
}


3508
void StorageReplicatedMergeTree::freezePartition(const ASTPtr & partition, const String & with_name, const Context & context)
3509
{
3510
    data.freezePartition(partition, with_name, context);
3511 3512
}

3513

3514
void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK()
3515 3516 3517
{
    /// Critical section is not required (since grabOldParts() returns unique part set on each call)

3518
    auto table_lock = lockStructure(false, __PRETTY_FUNCTION__);
3519 3520 3521
    auto zookeeper = getZooKeeper();

    MergeTreeData::DataPartsVector parts = data.grabOldParts();
3522
    if (parts.empty())
3523 3524
        return;

3525 3526 3527 3528 3529 3530
    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)
3531
    {
3532 3533 3534 3535 3536 3537
        if (!part->is_duplicate)
            parts_to_delete_completely.emplace_back(part);
        else
            parts_to_delete_only_from_filesystem.emplace_back(part);
    }
    parts.clear();
3538

3539 3540 3541
    auto remove_parts_from_filesystem = [log=log] (const MergeTreeData::DataPartsVector & parts_to_remove)
    {
        for (auto & part : parts_to_remove)
3542
        {
3543 3544 3545 3546 3547 3548 3549 3550
            try
            {
                part->remove();
            }
            catch (...)
            {
                tryLogCurrentException(log, "There is a problem with deleting part " + part->name + " from filesystem");
            }
3551 3552 3553
        }
    };

3554 3555 3556 3557 3558 3559 3560 3561 3562 3563
    /// 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
3564
    NameSet part_names_to_retry_deletion;
3565 3566
    try
    {
3567 3568 3569
        Strings part_names_to_delete_completely;
        for (const auto & part : parts_to_delete_completely)
            part_names_to_delete_completely.emplace_back(part->name);
3570

3571 3572
        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);
3573 3574 3575
    }
    catch (...)
    {
3576 3577
        LOG_ERROR(log, "There is a problem with deleting parts from ZooKeeper: " << getCurrentExceptionMessage(false));
    }
3578

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

3583 3584
    /// Delete normal parts on two sets
    for (auto & part : parts_to_delete_completely)
3585 3586 3587 3588 3589
    {
        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);
3590 3591
    }

3592 3593
    /// Will retry deletion
    if (!parts_to_retry_deletion.empty())
3594
    {
3595
        data.rollbackDeletingParts(parts_to_retry_deletion);
3596 3597
        LOG_DEBUG(log, "Will retry deletion of " << parts_to_retry_deletion.size() << " parts in the next time");
    }
3598

3599
    /// Remove parts from filesystem and finally from data_parts
3600
    if (!parts_to_remove_from_filesystem.empty())
3601
    {
3602 3603
        remove_parts_from_filesystem(parts_to_remove_from_filesystem);
        data.removePartsFinally(parts_to_remove_from_filesystem);
3604

3605 3606
        LOG_DEBUG(log, "Removed " << parts_to_remove_from_filesystem.size() << " old parts");
    }
3607 3608 3609
}


3610 3611 3612 3613 3614 3615 3616 3617 3618 3619 3620 3621 3622 3623 3624 3625
static int32_t tryMultiWithRetries(zkutil::ZooKeeperPtr & zookeeper, zkutil::Ops & ops) noexcept
{
    int32_t code;
    try
    {
        code = zookeeper->tryMultiWithRetries(ops);
    }
    catch (const zkutil::KeeperException & e)
    {
        code = e.code;
    }

    return code;
}


3626 3627
void StorageReplicatedMergeTree::removePartsFromZooKeeper(zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names,
                                                          NameSet * parts_should_be_retied)
3628 3629
{
    zkutil::Ops ops;
3630
    auto it_first_node_in_batch = part_names.cbegin();
3631 3632 3633 3634 3635

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

3636 3637
        auto it_next = std::next(it);
        if (ops.size() >= zkutil::MULTI_BATCH_SIZE || it_next == part_names.cend())
3638
        {
3639
            /// It is Ok to use multi with retries to delete nodes, because new nodes with the same names cannot appear here
3640
            auto code = tryMultiWithRetries(zookeeper, ops);
3641
            ops.clear();
3642 3643 3644 3645 3646 3647 3648 3649 3650 3651 3652 3653 3654

            if (code == ZNONODE)
            {
                /// 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)
                {
                    zkutil::Ops cur_ops;
                    removePartFromZooKeeper(*it_in_batch, cur_ops);
                    auto cur_code = tryMultiWithRetries(zookeeper, cur_ops);

                    if (cur_code == ZNONODE)
3655
                    {
3656
                        LOG_DEBUG(log, "There is no part " << *it_in_batch << " in ZooKeeper, it was only in filesystem");
3657 3658 3659 3660 3661
                    }
                    else if (parts_should_be_retied && zkutil::isHardwareErrorCode(cur_code))
                    {
                        parts_should_be_retied->emplace(*it_in_batch);
                    }
3662
                    else if (cur_code != ZOK)
3663
                    {
3664
                        LOG_WARNING(log, "Cannot remove part " << *it_in_batch << " from ZooKeeper: " << ::zerror(cur_code));
3665
                    }
3666 3667
                }
            }
3668 3669 3670 3671 3672
            else if (parts_should_be_retied && zkutil::isHardwareErrorCode(code))
            {
                for (auto it_in_batch = it_first_node_in_batch; it_in_batch != it_next; ++it_in_batch)
                    parts_should_be_retied->emplace(*it_in_batch);
            }
3673 3674 3675 3676 3677 3678 3679
            else if (code != ZOK)
            {
                LOG_WARNING(log, "There was a problem with deleting " << (it_next - it_first_node_in_batch)
                    << " nodes from ZooKeeper: " << ::zerror(code));
            }

            it_first_node_in_batch = it_next;
3680 3681 3682 3683 3684
        }
    }
}


3685 3686 3687 3688 3689 3690 3691 3692 3693 3694 3695 3696 3697 3698 3699 3700 3701 3702 3703 3704 3705 3706 3707 3708 3709 3710 3711 3712 3713 3714 3715 3716 3717 3718 3719 3720 3721 3722 3723 3724 3725 3726 3727 3728 3729 3730 3731 3732 3733 3734 3735 3736
void StorageReplicatedMergeTree::clearBlocksInPartition(
    zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num)
{
    Strings blocks;
    if (ZOK != zookeeper.tryGetChildren(zookeeper_path + "/blocks", blocks))
        throw Exception(zookeeper_path + "/blocks doesn't exist", ErrorCodes::NOT_FOUND_NODE);

    String partition_prefix = partition_id + "_";
    std::vector<std::pair<String, zkutil::ZooKeeper::TryGetFuture>> get_futures;
    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));
        }
    }

    std::vector<std::pair<String, zkutil::ZooKeeper::TryRemoveFuture>> to_delete_futures;
    for (auto & pair : get_futures)
    {
        const String & path = pair.first;
        zkutil::ZooKeeper::ValueAndStatAndExists result = pair.second.get();

        if (!result.exists)
            continue;

        ReadBufferFromString buf(result.value);
        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;
        int32_t rc = pair.second.get();
        if (rc == ZNOTEMPTY)
        {
             /// Can happen if there are leftover block nodes with children created by previous server versions.
            zookeeper.removeRecursive(path);
        }
        else if (rc != ZOK)
            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);
}


M
Merge  
Michael Kolupaev 已提交
3737
}