2014-09-17 23:57:25 +00:00
|
|
|
|
#include <statdaemons/ext/range.hpp>
|
2015-04-11 03:10:23 +00:00
|
|
|
|
#include <DB/Storages/ColumnsDescription.h>
|
2014-03-21 13:42:14 +00:00
|
|
|
|
#include <DB/Storages/StorageReplicatedMergeTree.h>
|
2014-04-02 10:10:37 +00:00
|
|
|
|
#include <DB/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h>
|
2014-04-02 13:45:39 +00:00
|
|
|
|
#include <DB/Storages/MergeTree/ReplicatedMergeTreePartsExchange.h>
|
2015-09-11 02:13:59 +00:00
|
|
|
|
#include <DB/Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h>
|
2014-07-22 13:49:52 +00:00
|
|
|
|
#include <DB/Storages/MergeTree/MergeTreePartChecker.h>
|
2015-04-16 06:12:35 +00:00
|
|
|
|
#include <DB/Storages/MergeTree/MergeList.h>
|
2015-05-20 11:58:21 +00:00
|
|
|
|
#include <DB/Storages/MergeTree/MergeTreeWhereOptimizer.h>
|
2014-03-22 14:44:44 +00:00
|
|
|
|
#include <DB/Parsers/formatAST.h>
|
|
|
|
|
#include <DB/IO/WriteBufferFromOStream.h>
|
|
|
|
|
#include <DB/IO/ReadBufferFromString.h>
|
2015-09-24 04:50:53 +00:00
|
|
|
|
#include <DB/IO/Operators.h>
|
2014-07-10 10:16:50 +00:00
|
|
|
|
#include <DB/Interpreters/InterpreterAlterQuery.h>
|
2014-07-28 09:53:57 +00:00
|
|
|
|
#include <DB/Common/VirtualColumnUtils.h>
|
2015-03-02 01:10:58 +00:00
|
|
|
|
#include <DB/Parsers/ASTInsertQuery.h>
|
2014-12-30 18:04:53 +00:00
|
|
|
|
#include <DB/DataStreams/AddingConstColumnBlockInputStream.h>
|
2015-04-16 06:12:35 +00:00
|
|
|
|
#include <DB/Common/Macros.h>
|
2015-06-11 00:35:36 +00:00
|
|
|
|
#include <DB/Common/formatReadable.h>
|
2015-04-16 06:12:35 +00:00
|
|
|
|
#include <Poco/DirectoryIterator.h>
|
2014-05-13 10:10:26 +00:00
|
|
|
|
#include <time.h>
|
2014-03-21 13:42:14 +00:00
|
|
|
|
|
2014-12-30 18:04:53 +00:00
|
|
|
|
|
2014-03-21 13:42:14 +00:00
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
|
2014-04-03 11:48:28 +00:00
|
|
|
|
|
2014-07-15 14:37:49 +00:00
|
|
|
|
const auto ERROR_SLEEP_MS = 1000;
|
2014-07-02 10:16:49 +00:00
|
|
|
|
const auto MERGE_SELECTING_SLEEP_MS = 5 * 1000;
|
2014-04-03 11:48:28 +00:00
|
|
|
|
|
2015-08-17 21:09:36 +00:00
|
|
|
|
const Int64 RESERVED_BLOCK_NUMBERS = 200;
|
2014-08-08 08:28:13 +00:00
|
|
|
|
|
2014-04-03 11:48:28 +00:00
|
|
|
|
|
2015-09-19 05:56:40 +00:00
|
|
|
|
/** Для каждого куска есть сразу три места, где он должен быть:
|
|
|
|
|
* 1. В оперативке (RAM), MergeTreeData::data_parts, all_data_parts.
|
|
|
|
|
* 2. В файловой системе (FS), директория с данными таблицы.
|
|
|
|
|
* 3. В ZooKeeper (ZK).
|
|
|
|
|
*
|
|
|
|
|
* При добавлении куска, его надо добавить сразу в эти три места.
|
|
|
|
|
* Это делается так:
|
|
|
|
|
* - [FS] сначала записываем кусок во временную директорию на файловой системе;
|
|
|
|
|
* - [FS] переименовываем временный кусок в результирующий на файловой системе;
|
|
|
|
|
* - [RAM] сразу же после этого добавляем его в data_parts, и удаляем из data_parts покрываемые им куски;
|
|
|
|
|
* - [RAM] также устанавливаем объект Transaction, который в случае исключения (в следующем пункте),
|
|
|
|
|
* откатит изменения в data_parts (из предыдущего пункта) назад;
|
|
|
|
|
* - [ZK] затем отправляем транзакцию (multi) на добавление куска в ZooKeeper (и ещё некоторых действий);
|
|
|
|
|
* - [FS, ZK] кстати, удаление покрываемых (старых) кусков из файловой системы, из ZooKeeper и из all_data_parts
|
|
|
|
|
* делается отложенно, через несколько минут.
|
|
|
|
|
*
|
|
|
|
|
* Здесь нет никакой атомарности.
|
|
|
|
|
* Можно было бы добиться атомарности с помощью undo/redo логов и флага в DataPart, когда он полностью готов.
|
|
|
|
|
* Но это было бы неудобно - пришлось бы писать undo/redo логи для каждого Part-а в ZK, а это увеличило бы и без того большое количество взаимодействий.
|
|
|
|
|
*
|
|
|
|
|
* Вместо этого, мы вынуждены работать в ситуации, когда в любой момент времени
|
|
|
|
|
* (из другого потока, или после рестарта сервера) может наблюдаться недоделанная до конца транзакция.
|
|
|
|
|
* (заметим - для этого кусок должен быть в RAM)
|
|
|
|
|
* Из этих случаев наиболее частый - когда кусок уже есть в data_parts, но его ещё нет в ZooKeeper.
|
|
|
|
|
* Этот случай надо отличить от случая, когда такая ситуация достигается вследствие какого-то повреждения состояния.
|
|
|
|
|
*
|
|
|
|
|
* Делаем это с помощью порога на время.
|
|
|
|
|
* Если кусок достаточно молодой, то его отсутствие в ZooKeeper будем воспринимать оптимистично - как будто он просто не успел ещё туда добавиться
|
|
|
|
|
* - как будто транзакция ещё не выполнена, но скоро выполнится.
|
|
|
|
|
* А если кусок старый, то его отсутствие в ZooKeeper будем воспринимать как недоделанную транзакцию, которую нужно откатить.
|
|
|
|
|
*
|
|
|
|
|
* PS. Возможно, было бы лучше добавить в DataPart флаг о том, что кусок вставлен в ZK.
|
|
|
|
|
* Но здесь уже слишком легко запутаться с консистентностью этого флага.
|
|
|
|
|
*/
|
|
|
|
|
const auto MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER = 5 * 60;
|
|
|
|
|
|
|
|
|
|
|
2014-03-21 13:42:14 +00:00
|
|
|
|
StorageReplicatedMergeTree::StorageReplicatedMergeTree(
|
|
|
|
|
const String & zookeeper_path_,
|
|
|
|
|
const String & replica_name_,
|
2014-03-21 19:17:59 +00:00
|
|
|
|
bool attach,
|
2014-05-08 07:12:01 +00:00
|
|
|
|
const String & path_, const String & database_name_, const String & name_,
|
|
|
|
|
NamesAndTypesListPtr columns_,
|
2014-10-03 15:30:10 +00:00
|
|
|
|
const NamesAndTypesList & materialized_columns_,
|
2014-09-30 03:08:47 +00:00
|
|
|
|
const NamesAndTypesList & alias_columns_,
|
|
|
|
|
const ColumnDefaults & column_defaults_,
|
2014-03-22 14:44:44 +00:00
|
|
|
|
Context & context_,
|
2014-03-21 13:42:14 +00:00
|
|
|
|
ASTPtr & primary_expr_ast_,
|
|
|
|
|
const String & date_column_name_,
|
|
|
|
|
const ASTPtr & sampling_expression_,
|
|
|
|
|
size_t index_granularity_,
|
|
|
|
|
MergeTreeData::Mode mode_,
|
|
|
|
|
const String & sign_column_,
|
2014-11-22 02:22:30 +00:00
|
|
|
|
const Names & columns_to_sum_,
|
2014-03-21 13:42:14 +00:00
|
|
|
|
const MergeTreeSettings & settings_)
|
2014-10-03 15:30:10 +00:00
|
|
|
|
: IStorage{materialized_columns_, alias_columns_, column_defaults_}, context(context_),
|
2014-12-12 20:50:32 +00:00
|
|
|
|
current_zookeeper(context.getZooKeeper()), database_name(database_name_),
|
2014-08-11 15:59:01 +00:00
|
|
|
|
table_name(name_), full_path(path_ + escapeForFileName(table_name) + '/'),
|
|
|
|
|
zookeeper_path(context.getMacros().expand(zookeeper_path_)),
|
|
|
|
|
replica_name(context.getMacros().expand(replica_name_)),
|
2014-10-03 15:30:10 +00:00
|
|
|
|
data(full_path, columns_,
|
|
|
|
|
materialized_columns_, alias_columns_, column_defaults_,
|
|
|
|
|
context_, primary_expr_ast_, date_column_name_,
|
2014-11-22 02:22:30 +00:00
|
|
|
|
sampling_expression_, index_granularity_, mode_, sign_column_, columns_to_sum_,
|
2014-10-03 15:30:10 +00:00
|
|
|
|
settings_, database_name_ + "." + table_name, true,
|
|
|
|
|
std::bind(&StorageReplicatedMergeTree::enqueuePartForCheck, this, std::placeholders::_1)),
|
2014-12-15 04:06:39 +00:00
|
|
|
|
reader(data), writer(data), merger(data), fetcher(data), shutdown_event(false),
|
|
|
|
|
log(&Logger::get(database_name + "." + table_name + " (StorageReplicatedMergeTree)"))
|
2014-03-21 13:42:14 +00:00
|
|
|
|
{
|
2014-10-09 23:14:06 +00:00
|
|
|
|
if (!zookeeper_path.empty() && zookeeper_path.back() == '/')
|
|
|
|
|
zookeeper_path.resize(zookeeper_path.size() - 1);
|
2014-08-13 08:07:52 +00:00
|
|
|
|
replica_path = zookeeper_path + "/replicas/" + replica_name;
|
|
|
|
|
|
|
|
|
|
bool skip_sanity_checks = false;
|
|
|
|
|
|
2014-11-07 01:12:55 +00:00
|
|
|
|
try
|
2014-08-13 08:07:52 +00:00
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
if (current_zookeeper && current_zookeeper->exists(replica_path + "/flags/force_restore_data"))
|
2014-11-07 01:12:55 +00:00
|
|
|
|
{
|
|
|
|
|
skip_sanity_checks = true;
|
2014-12-12 20:50:32 +00:00
|
|
|
|
current_zookeeper->remove(replica_path + "/flags/force_restore_data");
|
2014-08-13 08:07:52 +00:00
|
|
|
|
|
2014-11-07 01:12:55 +00:00
|
|
|
|
LOG_WARNING(log, "Skipping the limits on severity of changes to data parts and columns (flag "
|
|
|
|
|
<< replica_path << "/flags/force_restore_data).");
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
catch (const zkutil::KeeperException & e)
|
|
|
|
|
{
|
|
|
|
|
/// Не удалось соединиться с ZK (об этом стало известно при попытке выполнить первую операцию).
|
|
|
|
|
if (e.code == ZCONNECTIONLOSS)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
2014-12-12 20:50:32 +00:00
|
|
|
|
current_zookeeper = nullptr;
|
2014-11-07 01:12:55 +00:00
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
throw;
|
2014-08-13 08:07:52 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
data.loadDataParts(skip_sanity_checks);
|
|
|
|
|
|
2014-12-12 20:50:32 +00:00
|
|
|
|
if (!current_zookeeper)
|
2014-05-13 11:24:04 +00:00
|
|
|
|
{
|
2014-07-23 13:58:38 +00:00
|
|
|
|
if (!attach)
|
|
|
|
|
throw Exception("Can't create replicated table without ZooKeeper", ErrorCodes::NO_ZOOKEEPER);
|
|
|
|
|
|
2014-12-11 01:56:42 +00:00
|
|
|
|
/// Не активируем реплику. Она будет в режиме readonly.
|
2014-05-13 11:24:04 +00:00
|
|
|
|
return;
|
|
|
|
|
}
|
|
|
|
|
|
2014-03-21 19:17:59 +00:00
|
|
|
|
if (!attach)
|
|
|
|
|
{
|
2014-08-11 14:00:24 +00:00
|
|
|
|
createTableIfNotExists();
|
2014-03-22 14:44:44 +00:00
|
|
|
|
|
2014-08-12 12:41:39 +00:00
|
|
|
|
checkTableStructure(false, false);
|
2014-07-10 10:16:50 +00:00
|
|
|
|
createReplica();
|
2014-03-21 19:17:59 +00:00
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
2014-08-12 12:41:39 +00:00
|
|
|
|
checkTableStructure(skip_sanity_checks, true);
|
2014-07-10 08:40:59 +00:00
|
|
|
|
checkParts(skip_sanity_checks);
|
2014-03-21 19:17:59 +00:00
|
|
|
|
}
|
2014-03-22 14:44:44 +00:00
|
|
|
|
|
2014-05-26 11:40:22 +00:00
|
|
|
|
initVirtualParts();
|
2014-04-24 10:20:02 +00:00
|
|
|
|
|
|
|
|
|
String unreplicated_path = full_path + "unreplicated/";
|
|
|
|
|
if (Poco::File(unreplicated_path).exists())
|
|
|
|
|
{
|
|
|
|
|
LOG_INFO(log, "Have unreplicated data");
|
2014-08-15 20:10:44 +00:00
|
|
|
|
|
2014-10-03 15:30:10 +00:00
|
|
|
|
unreplicated_data.reset(new MergeTreeData(unreplicated_path, columns_,
|
|
|
|
|
materialized_columns_, alias_columns_, column_defaults_,
|
2014-09-30 03:08:47 +00:00
|
|
|
|
context_, primary_expr_ast_,
|
2014-11-22 02:22:30 +00:00
|
|
|
|
date_column_name_, sampling_expression_, index_granularity_, mode_, sign_column_, columns_to_sum_, settings_,
|
2014-07-09 13:39:19 +00:00
|
|
|
|
database_name_ + "." + table_name + "[unreplicated]", false));
|
2014-08-15 20:10:44 +00:00
|
|
|
|
|
|
|
|
|
unreplicated_data->loadDataParts(skip_sanity_checks);
|
|
|
|
|
|
2014-04-24 10:20:02 +00:00
|
|
|
|
unreplicated_reader.reset(new MergeTreeDataSelectExecutor(*unreplicated_data));
|
2014-05-08 08:03:03 +00:00
|
|
|
|
unreplicated_merger.reset(new MergeTreeDataMerger(*unreplicated_data));
|
2014-04-24 10:20:02 +00:00
|
|
|
|
}
|
2014-04-25 13:55:15 +00:00
|
|
|
|
|
2014-10-17 01:05:51 +00:00
|
|
|
|
loadQueue();
|
2014-05-13 10:10:26 +00:00
|
|
|
|
|
2014-09-11 03:13:27 +00:00
|
|
|
|
/// В этом потоке реплика будет активирована.
|
2014-10-17 01:05:51 +00:00
|
|
|
|
restarting_thread.reset(new ReplicatedMergeTreeRestartingThread(*this));
|
2014-03-21 13:42:14 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-10-17 01:05:51 +00:00
|
|
|
|
|
2014-03-21 13:42:14 +00:00
|
|
|
|
StoragePtr StorageReplicatedMergeTree::create(
|
|
|
|
|
const String & zookeeper_path_,
|
|
|
|
|
const String & replica_name_,
|
2014-03-21 19:17:59 +00:00
|
|
|
|
bool attach,
|
2014-05-08 07:12:01 +00:00
|
|
|
|
const String & path_, const String & database_name_, const String & name_,
|
|
|
|
|
NamesAndTypesListPtr columns_,
|
2014-10-03 15:30:10 +00:00
|
|
|
|
const NamesAndTypesList & materialized_columns_,
|
2014-09-30 03:08:47 +00:00
|
|
|
|
const NamesAndTypesList & alias_columns_,
|
|
|
|
|
const ColumnDefaults & column_defaults_,
|
2014-03-22 14:44:44 +00:00
|
|
|
|
Context & context_,
|
2014-03-21 13:42:14 +00:00
|
|
|
|
ASTPtr & primary_expr_ast_,
|
|
|
|
|
const String & date_column_name_,
|
|
|
|
|
const ASTPtr & sampling_expression_,
|
|
|
|
|
size_t index_granularity_,
|
|
|
|
|
MergeTreeData::Mode mode_,
|
|
|
|
|
const String & sign_column_,
|
2015-07-16 21:06:45 +00:00
|
|
|
|
const Names & columns_to_sum_,
|
2014-03-21 13:42:14 +00:00
|
|
|
|
const MergeTreeSettings & settings_)
|
|
|
|
|
{
|
2014-09-30 03:08:47 +00:00
|
|
|
|
auto res = new StorageReplicatedMergeTree{
|
|
|
|
|
zookeeper_path_, replica_name_, attach,
|
|
|
|
|
path_, database_name_, name_,
|
2014-10-03 15:30:10 +00:00
|
|
|
|
columns_, materialized_columns_, alias_columns_, column_defaults_,
|
2014-09-30 03:08:47 +00:00
|
|
|
|
context_, primary_expr_ast_, date_column_name_,
|
|
|
|
|
sampling_expression_, index_granularity_, mode_,
|
2014-12-12 20:50:32 +00:00
|
|
|
|
sign_column_, columns_to_sum_, settings_};
|
|
|
|
|
|
2014-04-02 07:59:43 +00:00
|
|
|
|
StoragePtr res_ptr = res->thisPtr();
|
2014-10-17 01:05:51 +00:00
|
|
|
|
|
2014-12-12 20:50:32 +00:00
|
|
|
|
if (res->getZooKeeper())
|
2014-05-13 11:24:04 +00:00
|
|
|
|
{
|
|
|
|
|
String endpoint_name = "ReplicatedMergeTree:" + res->replica_path;
|
2014-07-22 13:49:52 +00:00
|
|
|
|
InterserverIOEndpointPtr endpoint = new ReplicatedMergeTreePartsServer(res->data, *res);
|
2014-05-13 11:24:04 +00:00
|
|
|
|
res->endpoint_holder = new InterserverIOEndpointHolder(endpoint_name, endpoint, res->context.getInterserverIOHandler());
|
|
|
|
|
}
|
2014-10-17 01:05:51 +00:00
|
|
|
|
|
2014-04-02 07:59:43 +00:00
|
|
|
|
return res_ptr;
|
2014-03-21 13:42:14 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-03-22 14:44:44 +00:00
|
|
|
|
static String formattedAST(const ASTPtr & ast)
|
|
|
|
|
{
|
|
|
|
|
if (!ast)
|
|
|
|
|
return "";
|
|
|
|
|
std::stringstream ss;
|
|
|
|
|
formatAST(*ast, ss, 0, false, true);
|
|
|
|
|
return ss.str();
|
|
|
|
|
}
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-08-11 14:00:24 +00:00
|
|
|
|
void StorageReplicatedMergeTree::createTableIfNotExists()
|
2014-03-21 19:17:59 +00:00
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
|
2014-08-11 14:00:24 +00:00
|
|
|
|
if (zookeeper->exists(zookeeper_path))
|
|
|
|
|
return;
|
2014-05-26 18:27:25 +00:00
|
|
|
|
|
2014-08-11 14:00:24 +00:00
|
|
|
|
LOG_DEBUG(log, "Creating table " << zookeeper_path);
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-08-11 14:05:38 +00:00
|
|
|
|
zookeeper->createAncestors(zookeeper_path);
|
|
|
|
|
|
2014-07-10 10:16:50 +00:00
|
|
|
|
/// Запишем метаданные таблицы, чтобы реплики могли сверять с ними параметры таблицы.
|
2014-03-22 14:44:44 +00:00
|
|
|
|
std::stringstream metadata;
|
|
|
|
|
metadata << "metadata format version: 1" << std::endl;
|
|
|
|
|
metadata << "date column: " << data.date_column_name << std::endl;
|
|
|
|
|
metadata << "sampling expression: " << formattedAST(data.sampling_expression) << std::endl;
|
|
|
|
|
metadata << "index granularity: " << data.index_granularity << std::endl;
|
|
|
|
|
metadata << "mode: " << static_cast<int>(data.mode) << std::endl;
|
|
|
|
|
metadata << "sign column: " << data.sign_column << std::endl;
|
|
|
|
|
metadata << "primary key: " << formattedAST(data.primary_expr_ast) << std::endl;
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2015-09-24 04:52:02 +00:00
|
|
|
|
auto acl = zookeeper->getDefaultACL();
|
|
|
|
|
|
2014-08-11 14:00:24 +00:00
|
|
|
|
zkutil::Ops ops;
|
|
|
|
|
ops.push_back(new zkutil::Op::Create(zookeeper_path, "",
|
2015-09-24 04:52:02 +00:00
|
|
|
|
acl, zkutil::CreateMode::Persistent));
|
2014-08-11 14:00:24 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/metadata", metadata.str(),
|
2015-09-24 04:52:02 +00:00
|
|
|
|
acl, zkutil::CreateMode::Persistent));
|
2014-11-10 16:16:43 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/columns", ColumnsDescription<false>{
|
|
|
|
|
data.getColumnsListNonMaterialized(), data.materialized_columns,
|
|
|
|
|
data.alias_columns, data.column_defaults}.toString(),
|
2015-09-24 04:52:02 +00:00
|
|
|
|
acl, zkutil::CreateMode::Persistent));
|
2014-08-11 14:00:24 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/log", "",
|
2015-09-24 04:52:02 +00:00
|
|
|
|
acl, zkutil::CreateMode::Persistent));
|
2014-08-11 14:00:24 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/blocks", "",
|
2015-09-24 04:52:02 +00:00
|
|
|
|
acl, zkutil::CreateMode::Persistent));
|
2014-08-11 14:00:24 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/block_numbers", "",
|
2015-09-24 04:52:02 +00:00
|
|
|
|
acl, zkutil::CreateMode::Persistent));
|
2014-08-11 14:00:24 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/nonincrement_block_numbers", "",
|
2015-09-24 04:52:02 +00:00
|
|
|
|
acl, zkutil::CreateMode::Persistent));
|
2014-08-11 14:00:24 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/leader_election", "",
|
2015-09-24 04:52:02 +00:00
|
|
|
|
acl, zkutil::CreateMode::Persistent));
|
2014-08-11 14:00:24 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/temp", "",
|
2015-09-24 04:52:02 +00:00
|
|
|
|
acl, zkutil::CreateMode::Persistent));
|
2014-08-11 14:00:24 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::Create(zookeeper_path + "/replicas", "",
|
2015-09-24 04:52:02 +00:00
|
|
|
|
acl, zkutil::CreateMode::Persistent));
|
2014-08-11 14:00:24 +00:00
|
|
|
|
|
|
|
|
|
auto code = zookeeper->tryMulti(ops);
|
|
|
|
|
if (code != ZOK && code != ZNODEEXISTS)
|
|
|
|
|
throw zkutil::KeeperException(code);
|
2014-03-22 14:44:44 +00:00
|
|
|
|
}
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-03-22 14:44:44 +00:00
|
|
|
|
/** Проверить, что список столбцов и настройки таблицы совпадают с указанными в ZK (/metadata).
|
|
|
|
|
* Если нет - бросить исключение.
|
2014-03-21 19:17:59 +00:00
|
|
|
|
*/
|
2014-08-12 12:41:39 +00:00
|
|
|
|
void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bool allow_alter)
|
2014-03-22 14:44:44 +00:00
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
String metadata_str = zookeeper->get(zookeeper_path + "/metadata");
|
2014-03-22 14:44:44 +00:00
|
|
|
|
ReadBufferFromString buf(metadata_str);
|
|
|
|
|
assertString("metadata format version: 1", buf);
|
|
|
|
|
assertString("\ndate column: ", buf);
|
|
|
|
|
assertString(data.date_column_name, buf);
|
|
|
|
|
assertString("\nsampling expression: ", buf);
|
|
|
|
|
assertString(formattedAST(data.sampling_expression), buf);
|
|
|
|
|
assertString("\nindex granularity: ", buf);
|
|
|
|
|
assertString(toString(data.index_granularity), buf);
|
|
|
|
|
assertString("\nmode: ", buf);
|
|
|
|
|
assertString(toString(static_cast<int>(data.mode)), buf);
|
|
|
|
|
assertString("\nsign column: ", buf);
|
|
|
|
|
assertString(data.sign_column, buf);
|
|
|
|
|
assertString("\nprimary key: ", buf);
|
2014-07-10 10:16:50 +00:00
|
|
|
|
/// NOTE: Можно сделать менее строгую проверку совпадения выражений, чтобы таблицы не ломались от небольших изменений
|
|
|
|
|
/// в коде formatAST.
|
2014-03-22 14:44:44 +00:00
|
|
|
|
assertString(formattedAST(data.primary_expr_ast), buf);
|
2014-07-15 11:42:06 +00:00
|
|
|
|
assertString("\n", buf);
|
2014-07-10 08:40:59 +00:00
|
|
|
|
assertEOF(buf);
|
2014-07-10 10:16:50 +00:00
|
|
|
|
|
|
|
|
|
zkutil::Stat stat;
|
2015-05-28 03:49:28 +00:00
|
|
|
|
auto columns_desc = ColumnsDescription<true>::parse(zookeeper->get(zookeeper_path + "/columns", &stat));
|
2014-11-10 16:16:43 +00:00
|
|
|
|
|
|
|
|
|
auto & columns = columns_desc.columns;
|
|
|
|
|
auto & materialized_columns = columns_desc.materialized;
|
|
|
|
|
auto & alias_columns = columns_desc.alias;
|
|
|
|
|
auto & column_defaults = columns_desc.defaults;
|
2014-07-10 10:16:50 +00:00
|
|
|
|
columns_version = stat.version;
|
2014-11-10 16:16:43 +00:00
|
|
|
|
|
2014-10-21 12:11:20 +00:00
|
|
|
|
if (columns != data.getColumnsListNonMaterialized() ||
|
|
|
|
|
materialized_columns != data.materialized_columns ||
|
|
|
|
|
alias_columns != data.alias_columns ||
|
|
|
|
|
column_defaults != data.column_defaults)
|
2014-03-22 14:44:44 +00:00
|
|
|
|
{
|
2014-10-21 12:11:20 +00:00
|
|
|
|
if (allow_alter &&
|
|
|
|
|
(skip_sanity_checks ||
|
|
|
|
|
data.getColumnsListNonMaterialized().sizeOfDifference(columns) +
|
|
|
|
|
data.materialized_columns.sizeOfDifference(materialized_columns) <= 2))
|
2014-07-10 10:16:50 +00:00
|
|
|
|
{
|
2014-07-10 08:40:59 +00:00
|
|
|
|
LOG_WARNING(log, "Table structure in ZooKeeper is a little different from local table structure. Assuming ALTER.");
|
2014-07-14 15:49:03 +00:00
|
|
|
|
|
|
|
|
|
/// Без всяких блокировок, потому что таблица еще не создана.
|
2014-10-16 13:37:01 +00:00
|
|
|
|
InterpreterAlterQuery::updateMetadata(database_name, table_name, columns,
|
|
|
|
|
materialized_columns, alias_columns, column_defaults, context);
|
2014-07-14 15:49:03 +00:00
|
|
|
|
data.setColumnsList(columns);
|
2014-10-16 13:37:01 +00:00
|
|
|
|
data.materialized_columns = std::move(materialized_columns);
|
|
|
|
|
data.alias_columns = std::move(alias_columns);
|
|
|
|
|
data.column_defaults = std::move(column_defaults);
|
2014-07-10 10:16:50 +00:00
|
|
|
|
}
|
2014-07-10 08:40:59 +00:00
|
|
|
|
else
|
2014-07-10 10:16:50 +00:00
|
|
|
|
{
|
2014-08-12 12:41:39 +00:00
|
|
|
|
throw Exception("Table structure in ZooKeeper is too different from local table structure.",
|
2014-07-10 08:40:59 +00:00
|
|
|
|
ErrorCodes::INCOMPATIBLE_COLUMNS);
|
2014-07-10 10:16:50 +00:00
|
|
|
|
}
|
2014-03-22 14:44:44 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2015-09-20 05:50:15 +00:00
|
|
|
|
/** При необходимости восстановить кусок, реплика сама добавляет в свою очередь запись на его получение.
|
|
|
|
|
* Какое поставить время для этой записи в очереди? Время учитывается при расчёте отставания реплики.
|
|
|
|
|
* Для этих целей имеет смысл использовать время создания недостающего куска
|
|
|
|
|
* (то есть, при расчёте отставания будет учитано, насколько старый кусок нам нужно восстановить).
|
|
|
|
|
*/
|
|
|
|
|
static time_t tryGetPartCreateTime(zkutil::ZooKeeperPtr & zookeeper, const String & replica_path, const String & part_name)
|
|
|
|
|
{
|
|
|
|
|
time_t res = 0;
|
|
|
|
|
|
|
|
|
|
/// Узнаем время создания part-а, если он ещё существует (не был, например, смерджен).
|
|
|
|
|
zkutil::Stat stat;
|
|
|
|
|
String unused;
|
|
|
|
|
if (zookeeper->tryGet(replica_path + "/parts/" + part_name, unused, &stat))
|
|
|
|
|
res = stat.ctime / 1000;
|
|
|
|
|
|
|
|
|
|
return res;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-03-22 14:44:44 +00:00
|
|
|
|
void StorageReplicatedMergeTree::createReplica()
|
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
|
2014-05-26 18:29:29 +00:00
|
|
|
|
LOG_DEBUG(log, "Creating replica " << replica_path);
|
2014-05-26 18:27:25 +00:00
|
|
|
|
|
2014-08-11 14:00:24 +00:00
|
|
|
|
/// Создадим пустую реплику. Ноду columns создадим в конце - будем использовать ее в качестве признака, что создание реплики завершено.
|
2015-09-24 04:52:02 +00:00
|
|
|
|
auto acl = zookeeper->getDefaultACL();
|
2014-08-11 14:00:24 +00:00
|
|
|
|
zkutil::Ops ops;
|
2015-09-24 04:52:02 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::Create(replica_path, "", acl, zkutil::CreateMode::Persistent));
|
|
|
|
|
ops.push_back(new zkutil::Op::Create(replica_path + "/host", "", acl, zkutil::CreateMode::Persistent));
|
|
|
|
|
ops.push_back(new zkutil::Op::Create(replica_path + "/log_pointer", "", acl, zkutil::CreateMode::Persistent));
|
|
|
|
|
ops.push_back(new zkutil::Op::Create(replica_path + "/queue", "", acl, zkutil::CreateMode::Persistent));
|
|
|
|
|
ops.push_back(new zkutil::Op::Create(replica_path + "/parts", "", acl, zkutil::CreateMode::Persistent));
|
|
|
|
|
ops.push_back(new zkutil::Op::Create(replica_path + "/flags", "", acl, zkutil::CreateMode::Persistent));
|
2014-10-02 23:31:18 +00:00
|
|
|
|
|
|
|
|
|
try
|
|
|
|
|
{
|
|
|
|
|
zookeeper->multi(ops);
|
|
|
|
|
}
|
|
|
|
|
catch (const zkutil::KeeperException & e)
|
|
|
|
|
{
|
|
|
|
|
if (e.code == ZNODEEXISTS)
|
|
|
|
|
throw Exception("Replica " + replica_path + " is already exist.", ErrorCodes::REPLICA_IS_ALREADY_EXIST);
|
|
|
|
|
|
|
|
|
|
throw;
|
|
|
|
|
}
|
2014-05-26 18:14:52 +00:00
|
|
|
|
|
2014-07-15 14:37:49 +00:00
|
|
|
|
/** Нужно изменить данные ноды /replicas на что угодно, чтобы поток, удаляющий старые записи в логе,
|
|
|
|
|
* споткнулся об это изменение и не удалил записи, которые мы еще не прочитали.
|
|
|
|
|
*/
|
|
|
|
|
zookeeper->set(zookeeper_path + "/replicas", "last added replica: " + replica_name);
|
|
|
|
|
|
2014-08-11 14:00:24 +00:00
|
|
|
|
Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
|
2014-05-26 18:14:52 +00:00
|
|
|
|
|
2014-08-11 14:00:24 +00:00
|
|
|
|
/** "Эталонная" реплика, у которой мы возьмем информацию о множестве кусков, очередь и указатель на лог.
|
|
|
|
|
* Возьмем случайную из реплик, созданных раньше этой.
|
|
|
|
|
*/
|
|
|
|
|
String source_replica;
|
2014-05-26 18:14:52 +00:00
|
|
|
|
|
2014-08-11 14:00:24 +00:00
|
|
|
|
Stat stat;
|
|
|
|
|
zookeeper->exists(replica_path, &stat);
|
|
|
|
|
auto my_create_time = stat.czxid;
|
2014-05-26 18:14:52 +00:00
|
|
|
|
|
2014-08-11 14:00:24 +00:00
|
|
|
|
std::random_shuffle(replicas.begin(), replicas.end());
|
|
|
|
|
for (const String & replica : replicas)
|
2014-05-26 18:14:52 +00:00
|
|
|
|
{
|
2014-08-11 14:00:24 +00:00
|
|
|
|
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;
|
|
|
|
|
}
|
2014-05-26 18:14:52 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-08-11 14:00:24 +00:00
|
|
|
|
if (source_replica.empty())
|
2014-05-26 18:14:52 +00:00
|
|
|
|
{
|
2014-08-11 14:00:24 +00:00
|
|
|
|
LOG_INFO(log, "This is the first replica");
|
2014-05-26 18:14:52 +00:00
|
|
|
|
}
|
2014-08-11 14:00:24 +00:00
|
|
|
|
else
|
2014-05-26 18:14:52 +00:00
|
|
|
|
{
|
2014-08-11 14:00:24 +00:00
|
|
|
|
LOG_INFO(log, "Will mimic " << source_replica);
|
2014-05-26 18:14:52 +00:00
|
|
|
|
|
2014-08-11 14:00:24 +00:00
|
|
|
|
String source_path = zookeeper_path + "/replicas/" + source_replica;
|
2014-05-26 18:14:52 +00:00
|
|
|
|
|
2014-08-11 14:00:24 +00:00
|
|
|
|
/** Если эталонная реплика еще не до конца создана, подождем.
|
|
|
|
|
* NOTE: Если при ее создании что-то пошло не так, можем провисеть тут вечно.
|
|
|
|
|
* Можно создавать на время создания эфемерную ноду, чтобы быть уверенным, что реплика создается, а не заброшена.
|
|
|
|
|
* То же можно делать и для таблицы. Можно автоматически удалять ноду реплики/таблицы,
|
|
|
|
|
* если видно, что она создана не до конца, а создающий ее умер.
|
|
|
|
|
*/
|
|
|
|
|
while (!zookeeper->exists(source_path + "/columns"))
|
|
|
|
|
{
|
|
|
|
|
LOG_INFO(log, "Waiting for replica " << source_path << " to be fully created");
|
|
|
|
|
|
|
|
|
|
zkutil::EventPtr event = new Poco::Event;
|
|
|
|
|
if (zookeeper->exists(source_path + "/columns", nullptr, event))
|
|
|
|
|
{
|
|
|
|
|
LOG_WARNING(log, "Oops, a watch has leaked");
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
event->wait();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Порядок следующих трех действий важен. Записи в логе могут продублироваться, но не могут потеряться.
|
|
|
|
|
|
|
|
|
|
/// Скопируем у эталонной реплики ссылку на лог.
|
|
|
|
|
zookeeper->set(replica_path + "/log_pointer", zookeeper->get(source_path + "/log_pointer"));
|
|
|
|
|
|
|
|
|
|
/// Запомним очередь эталонной реплики.
|
|
|
|
|
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);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Добавим в очередь задания на получение всех активных кусков, которые есть у эталонной реплики.
|
|
|
|
|
Strings parts = zookeeper->getChildren(source_path + "/parts");
|
2014-10-10 00:46:37 +00:00
|
|
|
|
ActiveDataPartSet active_parts_set(parts);
|
|
|
|
|
|
2014-08-11 14:00:24 +00:00
|
|
|
|
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;
|
2015-09-20 05:50:15 +00:00
|
|
|
|
log_entry.create_time = tryGetPartCreateTime(zookeeper, source_path, name);
|
2015-09-17 21:30:43 +00:00
|
|
|
|
|
2014-08-11 14:00:24 +00:00
|
|
|
|
zookeeper->create(replica_path + "/queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential);
|
|
|
|
|
}
|
|
|
|
|
LOG_DEBUG(log, "Queued " << active_parts.size() << " parts to be fetched");
|
|
|
|
|
|
|
|
|
|
/// Добавим в очередь содержимое очереди эталонной реплики.
|
|
|
|
|
for (const String & entry : source_queue)
|
|
|
|
|
{
|
|
|
|
|
zookeeper->create(replica_path + "/queue/queue-", entry, zkutil::CreateMode::PersistentSequential);
|
|
|
|
|
}
|
2015-09-20 05:50:15 +00:00
|
|
|
|
|
|
|
|
|
/// Далее оно будет загружено в переменную queue в методе loadQueue.
|
|
|
|
|
|
2014-08-11 14:00:24 +00:00
|
|
|
|
LOG_DEBUG(log, "Copied " << source_queue.size() << " queue entries");
|
2014-05-26 18:14:52 +00:00
|
|
|
|
}
|
2014-08-11 14:00:24 +00:00
|
|
|
|
|
2014-11-10 16:16:43 +00:00
|
|
|
|
zookeeper->create(replica_path + "/columns", ColumnsDescription<false>{
|
|
|
|
|
data.getColumnsListNonMaterialized(),
|
|
|
|
|
data.materialized_columns,
|
|
|
|
|
data.alias_columns,
|
|
|
|
|
data.column_defaults
|
|
|
|
|
}.toString(), zkutil::CreateMode::Persistent);
|
2014-03-22 14:44:44 +00:00
|
|
|
|
}
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
|
|
|
|
|
2014-07-10 08:40:59 +00:00
|
|
|
|
void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
|
2014-04-02 07:59:43 +00:00
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
Strings expected_parts_vec = zookeeper->getChildren(replica_path + "/parts");
|
2014-05-22 10:37:17 +00:00
|
|
|
|
|
|
|
|
|
/// Куски в ZK.
|
2014-04-02 07:59:43 +00:00
|
|
|
|
NameSet expected_parts(expected_parts_vec.begin(), expected_parts_vec.end());
|
|
|
|
|
|
2014-04-09 16:32:32 +00:00
|
|
|
|
MergeTreeData::DataParts parts = data.getAllDataParts();
|
2014-04-02 07:59:43 +00:00
|
|
|
|
|
2014-05-22 10:37:17 +00:00
|
|
|
|
/// Локальные куски, которых нет в ZK.
|
2014-04-08 17:45:21 +00:00
|
|
|
|
MergeTreeData::DataParts unexpected_parts;
|
2014-05-22 10:37:17 +00:00
|
|
|
|
|
2014-04-02 07:59:43 +00:00
|
|
|
|
for (const auto & part : parts)
|
|
|
|
|
{
|
|
|
|
|
if (expected_parts.count(part->name))
|
|
|
|
|
expected_parts.erase(part->name);
|
|
|
|
|
else
|
2014-04-08 17:45:21 +00:00
|
|
|
|
unexpected_parts.insert(part);
|
2014-04-02 07:59:43 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-05-22 10:37:17 +00:00
|
|
|
|
/// Какие локальные куски добавить в ZK.
|
2014-04-08 17:45:21 +00:00
|
|
|
|
MergeTreeData::DataPartsVector parts_to_add;
|
2014-05-22 10:37:17 +00:00
|
|
|
|
|
|
|
|
|
/// Какие куски нужно забрать с других реплик.
|
|
|
|
|
Strings parts_to_fetch;
|
|
|
|
|
|
2014-04-08 17:45:21 +00:00
|
|
|
|
for (const String & missing_name : expected_parts)
|
|
|
|
|
{
|
2014-05-22 10:37:17 +00:00
|
|
|
|
/// Если локально не хватает какого-то куска, но есть покрывающий его кусок, можно заменить в ZK недостающий покрывающим.
|
2014-07-25 11:38:46 +00:00
|
|
|
|
auto containing = data.getActiveContainingPart(missing_name);
|
2014-05-22 10:37:17 +00:00
|
|
|
|
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);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
else
|
2014-04-08 17:45:21 +00:00
|
|
|
|
{
|
2014-08-13 08:07:52 +00:00
|
|
|
|
LOG_ERROR(log, "Fetching missing part " << missing_name);
|
2014-05-22 10:37:17 +00:00
|
|
|
|
parts_to_fetch.push_back(missing_name);
|
2014-04-08 17:45:21 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
2014-04-02 07:59:43 +00:00
|
|
|
|
|
2014-05-22 10:37:17 +00:00
|
|
|
|
for (const String & name : parts_to_fetch)
|
|
|
|
|
expected_parts.erase(name);
|
|
|
|
|
|
2015-09-24 01:17:10 +00:00
|
|
|
|
/** Для проверки адекватности, для кусков, которые есть в ФС, но нет в ZK, будем учитывать только не самые новые куски.
|
|
|
|
|
* Потому что неожиданные новые куски обычно возникают лишь оттого, что они не успели записаться в ZK при грубом перезапуске сервера.
|
|
|
|
|
* Также это возникает от дедуплицированных кусков, которые не успели удалиться.
|
|
|
|
|
*/
|
|
|
|
|
size_t unexpected_parts_nonnew = 0;
|
|
|
|
|
for (const auto & part : unexpected_parts)
|
|
|
|
|
if (part->level > 0 || part->right < RESERVED_BLOCK_NUMBERS)
|
|
|
|
|
++unexpected_parts_nonnew;
|
|
|
|
|
|
|
|
|
|
String sanity_report = "There are "
|
|
|
|
|
+ toString(unexpected_parts.size()) + " unexpected parts ("
|
|
|
|
|
+ toString(unexpected_parts_nonnew) + " of them is not just-written), "
|
|
|
|
|
+ toString(parts_to_add.size()) + " unexpectedly merged parts, "
|
|
|
|
|
+ toString(expected_parts.size()) + " missing obsolete parts, "
|
|
|
|
|
+ toString(parts_to_fetch.size()) + " missing parts";
|
2014-05-22 10:37:17 +00:00
|
|
|
|
|
2014-10-15 19:42:32 +00:00
|
|
|
|
/** Можно автоматически синхронизировать данные,
|
2015-09-24 01:17:10 +00:00
|
|
|
|
* если количество ошибок каждого из четырёх типов не больше соответствующих порогов,
|
|
|
|
|
* или если отношение общего количества ошибок к общему количеству кусков (минимальному - в локальной файловой системе или в ZK)
|
2014-10-15 19:42:32 +00:00
|
|
|
|
* не больше некоторого отношения (например 5%).
|
2015-09-24 01:17:10 +00:00
|
|
|
|
*
|
|
|
|
|
* Большое количество несовпадений в данных на файловой системе и ожидаемых данных
|
|
|
|
|
* может свидетельствовать об ошибке конфигурации (сервер случайно подключили как реплику не от того шарда).
|
|
|
|
|
* В этом случае, защитный механизм не даёт стартовать серверу.
|
2014-10-15 19:42:32 +00:00
|
|
|
|
*/
|
|
|
|
|
|
|
|
|
|
size_t min_parts_local_or_expected = std::min(expected_parts_vec.size(), parts.size());
|
2015-09-24 01:17:10 +00:00
|
|
|
|
size_t total_difference = parts_to_add.size() + unexpected_parts_nonnew + expected_parts.size() + parts_to_fetch.size();
|
2014-10-15 19:42:32 +00:00
|
|
|
|
|
|
|
|
|
bool insane =
|
|
|
|
|
(parts_to_add.size() > data.settings.replicated_max_unexpectedly_merged_parts
|
2015-09-24 01:17:10 +00:00
|
|
|
|
|| unexpected_parts_nonnew > data.settings.replicated_max_unexpected_parts
|
2014-10-15 19:42:32 +00:00
|
|
|
|
|| expected_parts.size() > data.settings.replicated_max_missing_obsolete_parts
|
|
|
|
|
|| parts_to_fetch.size() > data.settings.replicated_max_missing_active_parts)
|
2015-09-24 01:17:10 +00:00
|
|
|
|
&& (total_difference > min_parts_local_or_expected * data.settings.replicated_max_ratio_of_wrong_parts);
|
2014-04-08 17:45:21 +00:00
|
|
|
|
|
2015-09-24 01:17:10 +00:00
|
|
|
|
if (insane && !skip_sanity_checks)
|
|
|
|
|
throw Exception("The local set of parts of table " + getTableName() + " doesn't look like the set of parts in ZooKeeper. "
|
|
|
|
|
+ sanity_report, ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS);
|
|
|
|
|
|
|
|
|
|
if (total_difference > 0)
|
|
|
|
|
LOG_WARNING(log, sanity_report);
|
2014-07-10 10:16:50 +00:00
|
|
|
|
|
2014-04-08 17:45:21 +00:00
|
|
|
|
/// Добавим в ZK информацию о кусках, покрывающих недостающие куски.
|
2014-09-29 20:26:46 +00:00
|
|
|
|
for (const MergeTreeData::DataPartPtr & part : parts_to_add)
|
2014-04-08 17:45:21 +00:00
|
|
|
|
{
|
2014-04-09 16:32:32 +00:00
|
|
|
|
LOG_ERROR(log, "Adding unexpected local part to ZooKeeper: " << part->name);
|
2014-05-22 10:37:17 +00:00
|
|
|
|
|
2014-04-08 17:45:21 +00:00
|
|
|
|
zkutil::Ops ops;
|
2014-07-15 09:56:17 +00:00
|
|
|
|
checkPartAndAddToZooKeeper(part, ops);
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper->multi(ops);
|
2014-04-08 17:45:21 +00:00
|
|
|
|
}
|
2014-04-02 07:59:43 +00:00
|
|
|
|
|
2014-04-08 17:45:21 +00:00
|
|
|
|
/// Удалим из ZK информацию о кусках, покрытых только что добавленными.
|
|
|
|
|
for (const String & name : expected_parts)
|
|
|
|
|
{
|
2014-05-22 10:37:17 +00:00
|
|
|
|
LOG_ERROR(log, "Removing unexpectedly merged local part from ZooKeeper: " << name);
|
|
|
|
|
|
|
|
|
|
zkutil::Ops ops;
|
2015-09-20 11:02:59 +00:00
|
|
|
|
removePartFromZooKeeper(name, ops);
|
2014-05-22 10:37:17 +00:00
|
|
|
|
zookeeper->multi(ops);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Добавим в очередь задание забрать недостающие куски с других реплик и уберем из ZK информацию, что они у нас есть.
|
|
|
|
|
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;
|
2014-05-26 17:12:46 +00:00
|
|
|
|
log_entry.source_replica = "";
|
2014-05-22 10:37:17 +00:00
|
|
|
|
log_entry.new_part_name = name;
|
2015-09-20 05:50:15 +00:00
|
|
|
|
log_entry.create_time = tryGetPartCreateTime(zookeeper, replica_path, name);
|
2014-05-22 10:37:17 +00:00
|
|
|
|
|
|
|
|
|
/// Полагаемся, что это происходит до загрузки очереди (loadQueue).
|
2014-04-08 17:45:21 +00:00
|
|
|
|
zkutil::Ops ops;
|
2015-09-20 11:02:59 +00:00
|
|
|
|
removePartFromZooKeeper(name, ops);
|
2014-05-22 10:37:17 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::Create(
|
|
|
|
|
replica_path + "/queue/queue-", log_entry.toString(), zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper->multi(ops);
|
2014-04-08 17:45:21 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Удалим лишние локальные куски.
|
2014-09-29 20:26:46 +00:00
|
|
|
|
for (const MergeTreeData::DataPartPtr & part : unexpected_parts)
|
2014-04-02 07:59:43 +00:00
|
|
|
|
{
|
2014-05-22 10:37:17 +00:00
|
|
|
|
LOG_ERROR(log, "Renaming unexpected part " << part->name << " to ignored_" + part->name);
|
2014-07-28 09:46:28 +00:00
|
|
|
|
data.renameAndDetachPart(part, "ignored_", true);
|
2014-04-02 07:59:43 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-05-26 11:40:22 +00:00
|
|
|
|
void StorageReplicatedMergeTree::initVirtualParts()
|
|
|
|
|
{
|
|
|
|
|
auto parts = data.getDataParts();
|
|
|
|
|
for (const auto & part : parts)
|
|
|
|
|
virtual_parts.add(part->name);
|
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-09-29 20:26:46 +00:00
|
|
|
|
void StorageReplicatedMergeTree::checkPartAndAddToZooKeeper(const MergeTreeData::DataPartPtr & part, zkutil::Ops & ops, String part_name)
|
2014-04-08 17:45:21 +00:00
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
|
2014-08-08 08:28:13 +00:00
|
|
|
|
if (part_name.empty())
|
|
|
|
|
part_name = part->name;
|
|
|
|
|
|
2014-07-14 15:49:03 +00:00
|
|
|
|
check(part->columns);
|
2014-07-15 09:56:17 +00:00
|
|
|
|
int expected_columns_version = columns_version;
|
2014-07-14 15:49:03 +00:00
|
|
|
|
|
2014-07-10 10:16:50 +00:00
|
|
|
|
Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
|
|
|
|
|
std::random_shuffle(replicas.begin(), replicas.end());
|
|
|
|
|
String expected_columns_str = part->columns.toString();
|
|
|
|
|
|
|
|
|
|
for (const String & replica : replicas)
|
2014-04-08 17:45:21 +00:00
|
|
|
|
{
|
2014-07-10 10:16:50 +00:00
|
|
|
|
zkutil::Stat stat_before, stat_after;
|
|
|
|
|
String columns_str;
|
2014-08-08 08:28:13 +00:00
|
|
|
|
if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name + "/columns", columns_str, &stat_before))
|
2014-07-10 10:16:50 +00:00
|
|
|
|
continue;
|
|
|
|
|
if (columns_str != expected_columns_str)
|
|
|
|
|
{
|
2014-08-08 08:28:13 +00:00
|
|
|
|
LOG_INFO(log, "Not checking checksums of part " << part_name << " with replica " << replica
|
2014-07-10 10:16:50 +00:00
|
|
|
|
<< " because columns are different");
|
|
|
|
|
continue;
|
|
|
|
|
}
|
2014-04-14 13:08:26 +00:00
|
|
|
|
String checksums_str;
|
2014-07-10 10:16:50 +00:00
|
|
|
|
/// Проверим, что версия ноды со столбцами не изменилась, пока мы читали checksums.
|
|
|
|
|
/// Это гарантирует, что столбцы и чексуммы относятся к одним и тем же данным.
|
2014-08-08 08:28:13 +00:00
|
|
|
|
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) ||
|
2014-07-10 10:16:50 +00:00
|
|
|
|
stat_before.version != stat_after.version)
|
2014-04-14 13:08:26 +00:00
|
|
|
|
{
|
2014-08-08 08:28:13 +00:00
|
|
|
|
LOG_INFO(log, "Not checking checksums of part " << part_name << " with replica " << replica
|
2014-07-10 10:16:50 +00:00
|
|
|
|
<< " because part changed while we were reading its checksums");
|
|
|
|
|
continue;
|
2014-04-14 13:08:26 +00:00
|
|
|
|
}
|
2014-07-10 10:16:50 +00:00
|
|
|
|
|
|
|
|
|
auto checksums = MergeTreeData::DataPart::Checksums::parse(checksums_str);
|
|
|
|
|
checksums.checkEqual(part->checksums, true);
|
2014-04-08 17:45:21 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-08-08 08:28:13 +00:00
|
|
|
|
if (zookeeper->exists(replica_path + "/parts/" + part_name))
|
2014-07-28 14:31:07 +00:00
|
|
|
|
{
|
2014-08-08 08:28:13 +00:00
|
|
|
|
LOG_ERROR(log, "checkPartAndAddToZooKeeper: node " << replica_path + "/parts/" + part_name << " already exists");
|
2014-07-28 14:31:07 +00:00
|
|
|
|
return;
|
|
|
|
|
}
|
|
|
|
|
|
2015-09-24 04:52:02 +00:00
|
|
|
|
auto acl = zookeeper->getDefaultACL();
|
|
|
|
|
|
2014-07-14 15:49:03 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::Check(
|
|
|
|
|
zookeeper_path + "/columns",
|
|
|
|
|
expected_columns_version));
|
2014-04-08 17:45:21 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::Create(
|
2014-08-08 08:28:13 +00:00
|
|
|
|
replica_path + "/parts/" + part_name,
|
2014-04-08 17:45:21 +00:00
|
|
|
|
"",
|
2015-09-24 04:52:02 +00:00
|
|
|
|
acl,
|
2014-04-08 17:45:21 +00:00
|
|
|
|
zkutil::CreateMode::Persistent));
|
2014-07-10 10:16:50 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::Create(
|
2014-08-08 08:28:13 +00:00
|
|
|
|
replica_path + "/parts/" + part_name + "/columns",
|
2014-07-10 10:16:50 +00:00
|
|
|
|
part->columns.toString(),
|
2015-09-24 04:52:02 +00:00
|
|
|
|
acl,
|
2014-07-10 10:16:50 +00:00
|
|
|
|
zkutil::CreateMode::Persistent));
|
2014-04-08 17:45:21 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::Create(
|
2014-08-08 08:28:13 +00:00
|
|
|
|
replica_path + "/parts/" + part_name + "/checksums",
|
2014-04-08 17:45:21 +00:00
|
|
|
|
part->checksums.toString(),
|
2015-09-24 04:52:02 +00:00
|
|
|
|
acl,
|
2014-04-08 17:45:21 +00:00
|
|
|
|
zkutil::CreateMode::Persistent));
|
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-04-03 08:47:59 +00:00
|
|
|
|
void StorageReplicatedMergeTree::loadQueue()
|
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
|
2014-12-02 19:08:18 +00:00
|
|
|
|
std::lock_guard<std::mutex> lock(queue_mutex);
|
2014-04-03 08:47:59 +00:00
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
Strings children = zookeeper->getChildren(replica_path + "/queue");
|
2014-04-03 08:47:59 +00:00
|
|
|
|
std::sort(children.begin(), children.end());
|
|
|
|
|
for (const String & child : children)
|
|
|
|
|
{
|
2014-11-08 23:55:24 +00:00
|
|
|
|
zkutil::Stat stat;
|
|
|
|
|
String s = zookeeper->get(replica_path + "/queue/" + child, &stat);
|
2015-09-17 21:30:43 +00:00
|
|
|
|
LogEntryPtr entry = LogEntry::parse(s, stat);
|
2014-08-05 13:49:44 +00:00
|
|
|
|
entry->znode_name = child;
|
|
|
|
|
entry->addResultToVirtualParts(*this);
|
2014-04-03 11:48:28 +00:00
|
|
|
|
queue.push_back(entry);
|
2014-04-03 08:47:59 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-07-15 14:37:49 +00:00
|
|
|
|
void StorageReplicatedMergeTree::pullLogsToQueue(zkutil::EventPtr next_update_event)
|
2014-04-03 08:47:59 +00:00
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
|
2014-12-02 19:08:18 +00:00
|
|
|
|
std::lock_guard<std::mutex> lock(queue_mutex);
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-07-15 14:37:49 +00:00
|
|
|
|
String index_str = zookeeper->get(replica_path + "/log_pointer");
|
|
|
|
|
UInt64 index;
|
2014-04-03 12:49:01 +00:00
|
|
|
|
|
2014-07-15 14:37:49 +00:00
|
|
|
|
if (index_str.empty())
|
2014-04-03 12:49:01 +00:00
|
|
|
|
{
|
2014-07-15 14:37:49 +00:00
|
|
|
|
/// Если у нас еще нет указателя на лог, поставим указатель на первую запись в нем.
|
|
|
|
|
Strings entries = zookeeper->getChildren(zookeeper_path + "/log");
|
|
|
|
|
index = entries.empty() ? 0 : parse<UInt64>(std::min_element(entries.begin(), entries.end())->substr(strlen("log-")));
|
2014-04-03 12:49:01 +00:00
|
|
|
|
|
2014-07-23 09:50:29 +00:00
|
|
|
|
zookeeper->set(replica_path + "/log_pointer", toString(index));
|
2014-07-15 14:37:49 +00:00
|
|
|
|
}
|
|
|
|
|
else
|
2014-04-03 08:47:59 +00:00
|
|
|
|
{
|
2014-07-15 14:37:49 +00:00
|
|
|
|
index = parse<UInt64>(index_str);
|
2014-04-03 12:49:01 +00:00
|
|
|
|
}
|
2014-04-03 08:47:59 +00:00
|
|
|
|
|
2014-07-25 13:28:49 +00:00
|
|
|
|
UInt64 first_index = index;
|
|
|
|
|
|
2014-04-04 12:47:57 +00:00
|
|
|
|
size_t count = 0;
|
2014-07-15 14:37:49 +00:00
|
|
|
|
String entry_str;
|
2014-11-08 23:55:24 +00:00
|
|
|
|
zkutil::Stat stat;
|
|
|
|
|
while (zookeeper->tryGet(zookeeper_path + "/log/log-" + padIndex(index), entry_str, &stat))
|
2014-04-03 12:49:01 +00:00
|
|
|
|
{
|
2014-04-04 12:47:57 +00:00
|
|
|
|
++count;
|
2014-07-15 14:37:49 +00:00
|
|
|
|
++index;
|
2014-04-03 11:48:28 +00:00
|
|
|
|
|
2015-09-17 21:30:43 +00:00
|
|
|
|
LogEntryPtr entry = LogEntry::parse(entry_str, stat);
|
2014-04-03 08:47:59 +00:00
|
|
|
|
|
2014-04-03 12:49:01 +00:00
|
|
|
|
/// Одновременно добавим запись в очередь и продвинем указатель на лог.
|
|
|
|
|
zkutil::Ops ops;
|
|
|
|
|
ops.push_back(new zkutil::Op::Create(
|
2014-07-15 14:37:49 +00:00
|
|
|
|
replica_path + "/queue/queue-", entry_str, zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
|
2014-04-03 12:49:01 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::SetData(
|
2014-07-15 14:37:49 +00:00
|
|
|
|
replica_path + "/log_pointer", toString(index), -1));
|
2014-05-13 10:10:26 +00:00
|
|
|
|
auto results = zookeeper->multi(ops);
|
2014-04-03 12:49:01 +00:00
|
|
|
|
|
2014-06-04 13:49:22 +00:00
|
|
|
|
String path_created = dynamic_cast<zkutil::Op::Create &>(ops[0]).getPathCreated();
|
2014-08-05 13:49:44 +00:00
|
|
|
|
entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
|
|
|
|
|
entry->addResultToVirtualParts(*this);
|
2014-04-03 12:49:01 +00:00
|
|
|
|
queue.push_back(entry);
|
2014-07-15 14:37:49 +00:00
|
|
|
|
}
|
2014-04-03 12:49:01 +00:00
|
|
|
|
|
2015-09-20 06:40:17 +00:00
|
|
|
|
last_queue_update = time(0);
|
|
|
|
|
|
2014-07-15 14:37:49 +00:00
|
|
|
|
if (next_update_event)
|
|
|
|
|
{
|
|
|
|
|
if (zookeeper->exists(zookeeper_path + "/log/log-" + padIndex(index), nullptr, next_update_event))
|
|
|
|
|
next_update_event->set();
|
2014-04-03 08:47:59 +00:00
|
|
|
|
}
|
2014-04-04 12:47:57 +00:00
|
|
|
|
|
2014-07-15 15:51:27 +00:00
|
|
|
|
if (!count)
|
|
|
|
|
return;
|
|
|
|
|
|
2014-07-08 14:41:15 +00:00
|
|
|
|
if (queue_task_handle)
|
|
|
|
|
queue_task_handle->wake();
|
2014-07-15 14:37:49 +00:00
|
|
|
|
|
2014-07-25 13:28:49 +00:00
|
|
|
|
LOG_DEBUG(log, "Pulled " << count << " entries to queue: log-" << padIndex(first_index) << " - log-" << padIndex(index - 1));
|
2014-04-03 08:47:59 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2015-09-24 00:21:02 +00:00
|
|
|
|
bool StorageReplicatedMergeTree::shouldExecuteLogEntry(const LogEntry & entry, String & out_postpone_reason)
|
2014-04-03 08:47:59 +00:00
|
|
|
|
{
|
2015-09-20 06:31:19 +00:00
|
|
|
|
/// queue_mutex уже захвачен. Функция вызывается только из queueTask.
|
|
|
|
|
|
|
|
|
|
if (entry.type == LogEntry::MERGE_PARTS || entry.type == LogEntry::GET_PART || entry.type == LogEntry::ATTACH_PART)
|
2014-07-04 11:18:04 +00:00
|
|
|
|
{
|
2015-09-20 06:31:19 +00:00
|
|
|
|
/// Проверим, не создаётся ли сейчас этот же кусок другим действием.
|
|
|
|
|
if (future_parts.count(entry.new_part_name))
|
|
|
|
|
{
|
2015-09-24 00:21:02 +00:00
|
|
|
|
String reason = "Not executing log entry for part " + entry.new_part_name
|
|
|
|
|
+ " because another log entry for the same part is being processed. This shouldn't happen often.";
|
|
|
|
|
LOG_DEBUG(log, reason);
|
|
|
|
|
out_postpone_reason = reason;
|
2015-09-20 06:31:19 +00:00
|
|
|
|
return false;
|
|
|
|
|
|
|
|
|
|
/** Когда соответствующее действие завершится, то shouldExecuteLogEntry, в следующий раз, пройдёт успешно,
|
|
|
|
|
* и элемент очереди будет обработан. Сразу же в функции executeLogEntry будет выяснено, что кусок у нас уже есть,
|
|
|
|
|
* и элемент очереди будет сразу считаться обработанным.
|
|
|
|
|
*/
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Более сложная проверка - не создаётся ли сейчас другим действием кусок, который покроет этот кусок.
|
|
|
|
|
/// NOTE То, что выше - избыточно, но оставлено ради более удобного сообщения в логе.
|
|
|
|
|
ActiveDataPartSet::Part result_part;
|
|
|
|
|
ActiveDataPartSet::parsePartName(entry.new_part_name, result_part);
|
|
|
|
|
|
|
|
|
|
/// Оно может тормозить при большом размере future_parts. Но он не может быть большим, так как ограничен BackgroundProcessingPool.
|
|
|
|
|
for (const auto & future_part_name : future_parts)
|
|
|
|
|
{
|
|
|
|
|
ActiveDataPartSet::Part future_part;
|
|
|
|
|
ActiveDataPartSet::parsePartName(future_part_name, future_part);
|
|
|
|
|
|
|
|
|
|
if (future_part.contains(result_part))
|
|
|
|
|
{
|
2015-09-24 00:21:02 +00:00
|
|
|
|
String reason = "Not executing log entry for part " + entry.new_part_name
|
|
|
|
|
+ " because another log entry for covering part " + future_part_name + " is being processed.";
|
|
|
|
|
LOG_DEBUG(log, reason);
|
|
|
|
|
out_postpone_reason = reason;
|
2015-09-20 06:31:19 +00:00
|
|
|
|
return false;
|
|
|
|
|
}
|
|
|
|
|
}
|
2014-07-04 11:18:04 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-04-07 15:45:46 +00:00
|
|
|
|
if (entry.type == LogEntry::MERGE_PARTS)
|
|
|
|
|
{
|
|
|
|
|
/** Если какая-то из нужных частей сейчас передается или мерджится, подождем окончания этой операции.
|
|
|
|
|
* Иначе, даже если всех нужных частей для мерджа нет, нужно попытаться сделать мердж.
|
|
|
|
|
* Если каких-то частей не хватает, вместо мерджа будет попытка скачать кусок.
|
|
|
|
|
* Такая ситуация возможна, если получение какого-то куска пофейлилось, и его переместили в конец очереди.
|
|
|
|
|
*/
|
|
|
|
|
for (const auto & name : entry.parts_to_merge)
|
|
|
|
|
{
|
|
|
|
|
if (future_parts.count(name))
|
|
|
|
|
{
|
2015-09-24 00:21:02 +00:00
|
|
|
|
String reason = "Not merging into part " + entry.new_part_name
|
|
|
|
|
+ " because part " + name + " is not ready yet (log entry for that part is being processed).";
|
|
|
|
|
LOG_TRACE(log, reason);
|
|
|
|
|
out_postpone_reason = reason;
|
2014-04-07 15:45:46 +00:00
|
|
|
|
return false;
|
|
|
|
|
}
|
|
|
|
|
}
|
2015-09-18 06:39:39 +00:00
|
|
|
|
|
|
|
|
|
if (merger.isCancelled())
|
|
|
|
|
{
|
2015-09-24 00:21:02 +00:00
|
|
|
|
String reason = "Not executing log entry for part " + entry.new_part_name + " because merges are cancelled now.";
|
|
|
|
|
LOG_DEBUG(log, reason);
|
|
|
|
|
out_postpone_reason = reason;
|
2015-09-18 06:39:39 +00:00
|
|
|
|
return false;
|
|
|
|
|
}
|
2014-04-07 15:45:46 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
return true;
|
2014-04-03 11:48:28 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-07-18 15:41:04 +00:00
|
|
|
|
bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry, BackgroundProcessingPool::Context & pool_context)
|
2014-04-03 11:48:28 +00:00
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
|
2014-08-05 13:49:44 +00:00
|
|
|
|
if (entry.type == LogEntry::DROP_RANGE)
|
2014-08-08 08:28:13 +00:00
|
|
|
|
{
|
|
|
|
|
executeDropRange(entry);
|
|
|
|
|
return true;
|
|
|
|
|
}
|
2014-08-05 13:49:44 +00:00
|
|
|
|
|
2014-04-03 11:48:28 +00:00
|
|
|
|
if (entry.type == LogEntry::GET_PART ||
|
2014-08-08 08:28:13 +00:00
|
|
|
|
entry.type == LogEntry::MERGE_PARTS ||
|
|
|
|
|
entry.type == LogEntry::ATTACH_PART)
|
2014-04-03 11:48:28 +00:00
|
|
|
|
{
|
|
|
|
|
/// Если у нас уже есть этот кусок или покрывающий его кусок, ничего делать не нужно.
|
2014-07-25 11:38:46 +00:00
|
|
|
|
MergeTreeData::DataPartPtr containing_part = data.getActiveContainingPart(entry.new_part_name);
|
2014-04-03 11:48:28 +00:00
|
|
|
|
|
2015-09-14 22:45:19 +00:00
|
|
|
|
/// Даже если кусок есть локально, его (в исключительных случаях) может не быть в zookeeper. Проверим, что он там есть.
|
2014-05-13 10:10:26 +00:00
|
|
|
|
if (containing_part && zookeeper->exists(replica_path + "/parts/" + containing_part->name))
|
2014-04-04 12:47:57 +00:00
|
|
|
|
{
|
2014-04-07 15:45:46 +00:00
|
|
|
|
if (!(entry.type == LogEntry::GET_PART && entry.source_replica == replica_name))
|
2015-09-20 11:02:59 +00:00
|
|
|
|
LOG_DEBUG(log, "Skipping action for part " << entry.new_part_name << " - part already exists.");
|
2014-07-18 15:41:04 +00:00
|
|
|
|
return true;
|
2014-04-04 12:47:57 +00:00
|
|
|
|
}
|
2014-04-03 11:48:28 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-04-07 15:45:46 +00:00
|
|
|
|
if (entry.type == LogEntry::GET_PART && entry.source_replica == replica_name)
|
2014-05-27 10:03:13 +00:00
|
|
|
|
LOG_WARNING(log, "Part " << entry.new_part_name << " from own log doesn't exist.");
|
2014-04-07 15:45:46 +00:00
|
|
|
|
|
2015-09-20 11:02:59 +00:00
|
|
|
|
/// Возможно, этот кусок нам не нужен, так как при записи с кворумом, кворум пофейлился (см. ниже про /quorum/failed_parts).
|
2015-09-21 18:47:46 +00:00
|
|
|
|
if (entry.quorum && zookeeper->exists(zookeeper_path + "/quorum/failed_parts/" + entry.new_part_name))
|
2015-09-20 11:02:59 +00:00
|
|
|
|
{
|
|
|
|
|
LOG_DEBUG(log, "Skipping action for part " << entry.new_part_name << " because quorum for that part was failed.");
|
|
|
|
|
return true; /// NOTE Удаление из virtual_parts не делается, но оно нужно только для мерджей.
|
|
|
|
|
}
|
|
|
|
|
|
2014-04-07 15:45:46 +00:00
|
|
|
|
bool do_fetch = false;
|
|
|
|
|
|
2014-04-04 12:47:57 +00:00
|
|
|
|
if (entry.type == LogEntry::GET_PART)
|
|
|
|
|
{
|
2014-04-07 15:45:46 +00:00
|
|
|
|
do_fetch = true;
|
2014-04-04 12:47:57 +00:00
|
|
|
|
}
|
2014-08-08 08:28:13 +00:00
|
|
|
|
else if (entry.type == LogEntry::ATTACH_PART)
|
|
|
|
|
{
|
|
|
|
|
do_fetch = !executeAttachPart(entry);
|
|
|
|
|
}
|
2014-04-04 12:47:57 +00:00
|
|
|
|
else if (entry.type == LogEntry::MERGE_PARTS)
|
|
|
|
|
{
|
2014-09-17 23:57:25 +00:00
|
|
|
|
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());
|
|
|
|
|
|
2014-04-04 13:27:47 +00:00
|
|
|
|
MergeTreeData::DataPartsVector parts;
|
2014-07-14 15:49:03 +00:00
|
|
|
|
bool have_all_parts = true;
|
2014-04-04 13:27:47 +00:00
|
|
|
|
for (const String & name : entry.parts_to_merge)
|
|
|
|
|
{
|
2014-07-25 11:38:46 +00:00
|
|
|
|
MergeTreeData::DataPartPtr part = data.getActiveContainingPart(name);
|
2014-04-07 15:45:46 +00:00
|
|
|
|
if (!part)
|
|
|
|
|
{
|
|
|
|
|
have_all_parts = false;
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
if (part->name != name)
|
|
|
|
|
{
|
2014-08-08 10:46:38 +00:00
|
|
|
|
LOG_WARNING(log, "Part " << name << " is covered by " << part->name
|
|
|
|
|
<< " but should be merged into " << entry.new_part_name << ". This shouldn't happen often.");
|
2014-04-07 15:45:46 +00:00
|
|
|
|
have_all_parts = false;
|
|
|
|
|
break;
|
|
|
|
|
}
|
2014-04-04 13:27:47 +00:00
|
|
|
|
parts.push_back(part);
|
|
|
|
|
}
|
2014-04-04 17:20:45 +00:00
|
|
|
|
|
2014-04-07 15:45:46 +00:00
|
|
|
|
if (!have_all_parts)
|
2014-04-04 13:27:47 +00:00
|
|
|
|
{
|
2014-04-07 15:45:46 +00:00
|
|
|
|
/// Если нет всех нужных кусков, попробуем взять у кого-нибудь уже помердженный кусок.
|
|
|
|
|
do_fetch = true;
|
|
|
|
|
LOG_DEBUG(log, "Don't have all parts for merge " << entry.new_part_name << "; will try to fetch it instead");
|
2014-04-04 13:27:47 +00:00
|
|
|
|
}
|
2014-04-07 15:45:46 +00:00
|
|
|
|
else
|
|
|
|
|
{
|
2014-07-02 12:30:38 +00:00
|
|
|
|
/// Если собираемся сливать большие куски, увеличим счетчик потоков, сливающих большие куски.
|
|
|
|
|
for (const auto & part : parts)
|
|
|
|
|
{
|
2014-07-23 15:24:45 +00:00
|
|
|
|
if (part->size_in_bytes > data.settings.max_bytes_to_merge_parts_small)
|
2014-07-02 12:30:38 +00:00
|
|
|
|
{
|
|
|
|
|
pool_context.incrementCounter("big merges");
|
|
|
|
|
pool_context.incrementCounter("replicated big merges");
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2015-06-11 00:35:36 +00:00
|
|
|
|
size_t sum_parts_size_in_bytes = MergeTreeDataMerger::estimateDiskSpaceForMerge(parts);
|
2015-09-20 05:21:43 +00:00
|
|
|
|
|
|
|
|
|
/// Может бросить исключение.
|
|
|
|
|
DiskSpaceMonitor::ReservationPtr reserved_space = DiskSpaceMonitor::reserve(full_path, sum_parts_size_in_bytes);
|
2015-06-11 00:35:36 +00:00
|
|
|
|
|
2014-07-28 14:31:07 +00:00
|
|
|
|
auto table_lock = lockStructure(false);
|
2014-07-14 15:49:03 +00:00
|
|
|
|
|
2014-09-10 11:34:26 +00:00
|
|
|
|
const auto & merge_entry = context.getMergeList().insert(database_name, table_name, entry.new_part_name);
|
2014-07-01 15:58:25 +00:00
|
|
|
|
MergeTreeData::Transaction transaction;
|
2015-04-10 17:51:39 +00:00
|
|
|
|
size_t aio_threshold = context.getSettings().min_bytes_to_use_direct_io;
|
2015-09-20 05:21:43 +00:00
|
|
|
|
|
|
|
|
|
MergeTreeData::DataPartPtr part = merger.mergeParts(
|
|
|
|
|
parts, entry.new_part_name, *merge_entry, aio_threshold, &transaction, reserved_space);
|
2014-04-07 15:45:46 +00:00
|
|
|
|
|
|
|
|
|
zkutil::Ops ops;
|
2014-07-15 09:56:17 +00:00
|
|
|
|
checkPartAndAddToZooKeeper(part, ops);
|
2014-04-07 15:45:46 +00:00
|
|
|
|
|
2014-08-08 10:41:53 +00:00
|
|
|
|
/** TODO: Переименование нового куска лучше делать здесь, а не пятью строчками выше,
|
|
|
|
|
* чтобы оно было как можно ближе к zookeeper->multi.
|
|
|
|
|
*/
|
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper->multi(ops);
|
2014-07-28 14:31:07 +00:00
|
|
|
|
|
2015-09-14 22:45:19 +00:00
|
|
|
|
/** Удаление старых кусков из ZK и с диска делается отложенно - см. ReplicatedMergeTreeCleanupThread, clearOldParts.
|
|
|
|
|
*/
|
|
|
|
|
|
2014-07-28 14:31:07 +00:00
|
|
|
|
/** При ZCONNECTIONLOSS или ZOPERATIONTIMEOUT можем зря откатить локальные изменения кусков.
|
|
|
|
|
* Это не проблема, потому что в таком случае слияние останется в очереди, и мы попробуем снова.
|
|
|
|
|
*/
|
2014-07-01 15:58:25 +00:00
|
|
|
|
transaction.commit();
|
2014-07-02 10:16:49 +00:00
|
|
|
|
merge_selecting_event.set();
|
2014-04-07 15:45:46 +00:00
|
|
|
|
|
|
|
|
|
ProfileEvents::increment(ProfileEvents::ReplicatedPartMerges);
|
|
|
|
|
}
|
2014-04-04 12:47:57 +00:00
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
throw Exception("Unexpected log entry type: " + toString(static_cast<int>(entry.type)));
|
|
|
|
|
}
|
2014-04-07 15:45:46 +00:00
|
|
|
|
|
|
|
|
|
if (do_fetch)
|
|
|
|
|
{
|
2014-07-18 15:41:04 +00:00
|
|
|
|
String replica;
|
|
|
|
|
|
2014-04-07 15:45:46 +00:00
|
|
|
|
try
|
|
|
|
|
{
|
2014-07-18 15:41:04 +00:00
|
|
|
|
replica = findReplicaHavingPart(entry.new_part_name, true);
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
|
|
|
|
if (replica.empty() && entry.type == LogEntry::ATTACH_PART)
|
|
|
|
|
{
|
|
|
|
|
/** Если ATTACH - куска может не быть, потому что реплика, на которой кусок есть, ещё сама не успела его прицепить.
|
|
|
|
|
* В таком случае, надо подождать этого.
|
|
|
|
|
*/
|
|
|
|
|
|
|
|
|
|
/// Кусок должен быть на реплике-инициаторе.
|
|
|
|
|
if (entry.source_replica.empty() || entry.source_replica == replica_name)
|
|
|
|
|
throw Exception("Logical error: no source replica specified for ATTACH_PART log entry;"
|
|
|
|
|
" or trying to fetch part on source replica", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
|
|
/// Подождём, пока реплика-инициатор подцепит кусок.
|
|
|
|
|
waitForReplicaToProcessLogEntry(entry.source_replica, entry);
|
|
|
|
|
replica = findReplicaHavingPart(entry.new_part_name, true);
|
|
|
|
|
}
|
|
|
|
|
|
2014-04-08 17:45:21 +00:00
|
|
|
|
if (replica.empty())
|
|
|
|
|
{
|
2015-09-20 11:02:59 +00:00
|
|
|
|
/** Если кусок должен быть записан с кворумом, и кворум ещё недостигнут,
|
|
|
|
|
* то (из-за того, что кусок невозможно прямо сейчас скачать),
|
|
|
|
|
* кворумную запись следует считать безуспешной.
|
|
|
|
|
* TODO Сложный код, вынести отдельно.
|
|
|
|
|
*/
|
|
|
|
|
if (entry.quorum)
|
|
|
|
|
{
|
|
|
|
|
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.");
|
|
|
|
|
|
|
|
|
|
/** Атомарно:
|
|
|
|
|
* - если реплики не стали активными;
|
|
|
|
|
* - если существует узел quorum с этим куском;
|
|
|
|
|
* - удалим узел quorum;
|
|
|
|
|
* - установим nonincrement_block_numbers, чтобы разрешить мерджи через номер потерянного куска;
|
|
|
|
|
* - добавим кусок в список quorum/failed_parts.
|
|
|
|
|
*
|
2015-09-21 22:43:38 +00:00
|
|
|
|
* TODO Удаление из blocks.
|
|
|
|
|
*
|
2015-09-20 11:02:59 +00:00
|
|
|
|
* Если что-то изменится, то ничего не сделаем - попадём сюда снова в следующий раз.
|
|
|
|
|
*/
|
|
|
|
|
|
|
|
|
|
/** Соберём версии узлов host у реплик.
|
|
|
|
|
* Когда реплика становится активной, она в той же транзакции (с созданием is_active), меняет значение host.
|
|
|
|
|
* Это позволит проследить, что реплики не стали активными.
|
|
|
|
|
*/
|
|
|
|
|
|
|
|
|
|
Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
|
|
|
|
|
|
|
|
|
|
zkutil::Ops ops;
|
|
|
|
|
|
|
|
|
|
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.push_back(new zkutil::Op::Check(path, stat.version));
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Проверяем, что пока мы собирали версии, не ожила реплика с нужным куском.
|
|
|
|
|
replica = findReplicaHavingPart(entry.new_part_name, true);
|
|
|
|
|
|
|
|
|
|
/// Также за это время могла быть создана совсем новая реплика. Но если на старых не появится куска, то на новой его тоже не может быть.
|
|
|
|
|
|
|
|
|
|
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)
|
|
|
|
|
{
|
|
|
|
|
ops.push_back(new zkutil::Op::Remove(quorum_path, quorum_stat.version));
|
|
|
|
|
|
|
|
|
|
const auto partition_str = entry.new_part_name.substr(0, 6);
|
|
|
|
|
ActiveDataPartSet::Part part_info;
|
|
|
|
|
ActiveDataPartSet::parsePartName(entry.new_part_name, part_info);
|
|
|
|
|
|
|
|
|
|
if (part_info.left != part_info.right)
|
|
|
|
|
throw Exception("Logical error: log entry with quorum for part covering more than one block number",
|
|
|
|
|
ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
2015-09-24 04:52:02 +00:00
|
|
|
|
auto acl = zookeeper->getDefaultACL();
|
|
|
|
|
|
2015-09-20 11:02:59 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::Create(
|
|
|
|
|
zookeeper_path + "/nonincrement_block_numbers/" + partition_str + "/block-" + padIndex(part_info.left),
|
|
|
|
|
"",
|
2015-09-24 04:52:02 +00:00
|
|
|
|
acl,
|
2015-09-20 11:02:59 +00:00
|
|
|
|
zkutil::CreateMode::Persistent));
|
|
|
|
|
|
|
|
|
|
ops.push_back(new zkutil::Op::Create(
|
|
|
|
|
zookeeper_path + "/quorum/failed_parts/" + entry.new_part_name,
|
|
|
|
|
"",
|
2015-09-24 04:52:02 +00:00
|
|
|
|
acl,
|
2015-09-20 11:02:59 +00:00
|
|
|
|
zkutil::CreateMode::Persistent));
|
|
|
|
|
|
|
|
|
|
auto code = zookeeper->tryMulti(ops);
|
|
|
|
|
|
|
|
|
|
if (code == ZOK)
|
|
|
|
|
{
|
|
|
|
|
LOG_DEBUG(log, "Marked quorum for part " << entry.new_part_name << " as failed.");
|
|
|
|
|
return true; /// NOTE Удаление из virtual_parts не делается, но оно нужно только для мерджей.
|
|
|
|
|
}
|
|
|
|
|
else if (code == ZBADVERSION || code == ZNONODE || code == ZNODEEXISTS)
|
|
|
|
|
{
|
|
|
|
|
LOG_DEBUG(log, "State was changed or isn't expected when trying to mark quorum for part "
|
|
|
|
|
<< entry.new_part_name << " as failed.");
|
|
|
|
|
}
|
|
|
|
|
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.");
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (replica.empty())
|
|
|
|
|
{
|
|
|
|
|
ProfileEvents::increment(ProfileEvents::ReplicatedPartFailedFetches);
|
|
|
|
|
throw Exception("No active replica has part " + entry.new_part_name, ErrorCodes::NO_REPLICA_HAS_PART);
|
|
|
|
|
}
|
2014-04-08 17:45:21 +00:00
|
|
|
|
}
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2015-09-11 02:13:59 +00:00
|
|
|
|
fetchPart(entry.new_part_name, zookeeper_path + "/replicas/" + replica, false, entry.quorum);
|
2014-04-07 15:45:46 +00:00
|
|
|
|
|
|
|
|
|
if (entry.type == LogEntry::MERGE_PARTS)
|
|
|
|
|
ProfileEvents::increment(ProfileEvents::ReplicatedPartFetchesOfMerged);
|
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
/** Если не получилось скачать кусок, нужный для какого-то мерджа, лучше не пытаться получить другие куски для этого мерджа,
|
|
|
|
|
* а попытаться сразу получить помердженный кусок. Чтобы так получилось, переместим действия для получения остальных кусков
|
|
|
|
|
* для этого мерджа в конец очереди.
|
|
|
|
|
*/
|
|
|
|
|
try
|
|
|
|
|
{
|
2014-12-02 19:08:18 +00:00
|
|
|
|
std::lock_guard<std::mutex> lock(queue_mutex);
|
2014-04-07 15:45:46 +00:00
|
|
|
|
|
|
|
|
|
/// Найдем действие по объединению этого куска с другими. Запомним других.
|
|
|
|
|
StringSet parts_for_merge;
|
|
|
|
|
LogEntries::iterator merge_entry;
|
|
|
|
|
for (LogEntries::iterator it = queue.begin(); it != queue.end(); ++it)
|
|
|
|
|
{
|
2014-08-05 13:49:44 +00:00
|
|
|
|
if ((*it)->type == LogEntry::MERGE_PARTS)
|
2014-04-07 15:45:46 +00:00
|
|
|
|
{
|
2014-08-05 13:49:44 +00:00
|
|
|
|
if (std::find((*it)->parts_to_merge.begin(), (*it)->parts_to_merge.end(), entry.new_part_name)
|
|
|
|
|
!= (*it)->parts_to_merge.end())
|
2014-04-07 15:45:46 +00:00
|
|
|
|
{
|
2014-08-05 13:49:44 +00:00
|
|
|
|
parts_for_merge = StringSet((*it)->parts_to_merge.begin(), (*it)->parts_to_merge.end());
|
2014-04-07 15:45:46 +00:00
|
|
|
|
merge_entry = it;
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (!parts_for_merge.empty())
|
|
|
|
|
{
|
|
|
|
|
/// Переместим в конец очереди действия, получающие parts_for_merge.
|
|
|
|
|
for (LogEntries::iterator it = queue.begin(); it != queue.end();)
|
|
|
|
|
{
|
|
|
|
|
auto it0 = it;
|
|
|
|
|
++it;
|
|
|
|
|
|
|
|
|
|
if (it0 == merge_entry)
|
|
|
|
|
break;
|
|
|
|
|
|
2014-08-05 13:49:44 +00:00
|
|
|
|
if (((*it0)->type == LogEntry::MERGE_PARTS || (*it0)->type == LogEntry::GET_PART)
|
|
|
|
|
&& parts_for_merge.count((*it0)->new_part_name))
|
2014-04-07 15:45:46 +00:00
|
|
|
|
{
|
|
|
|
|
queue.splice(queue.end(), queue, it0, it);
|
|
|
|
|
}
|
|
|
|
|
}
|
2014-07-18 15:41:04 +00:00
|
|
|
|
|
|
|
|
|
/** Если этого куска ни у кого нет, но в очереди упоминается мердж с его участием, то наверно этот кусок такой старый,
|
|
|
|
|
* что его все померджили и удалили. Не будем бросать исключение, чтобы queueTask лишний раз не спала.
|
|
|
|
|
*/
|
|
|
|
|
if (replica.empty())
|
|
|
|
|
{
|
2015-09-14 22:45:19 +00:00
|
|
|
|
LOG_INFO(log, "No active replica has part " << entry.new_part_name << ". Will fetch merged part instead.");
|
2014-07-18 15:41:04 +00:00
|
|
|
|
return false;
|
|
|
|
|
}
|
2014-04-07 15:45:46 +00:00
|
|
|
|
}
|
2014-07-23 09:15:41 +00:00
|
|
|
|
|
2015-09-14 22:45:19 +00:00
|
|
|
|
/** Если ни у какой активной реплики нет куска, и в очереди нет слияний с его участием,
|
|
|
|
|
* проверим, есть ли у любой (активной или неактивной) реплики такой кусок или покрывающий его.
|
|
|
|
|
*/
|
2014-07-23 09:15:41 +00:00
|
|
|
|
if (replica.empty())
|
|
|
|
|
enqueuePartForCheck(entry.new_part_name);
|
2014-04-07 15:45:46 +00:00
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
throw;
|
|
|
|
|
}
|
|
|
|
|
}
|
2014-07-18 15:41:04 +00:00
|
|
|
|
|
|
|
|
|
return true;
|
2014-04-03 11:48:28 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-08-08 08:28:13 +00:00
|
|
|
|
void StorageReplicatedMergeTree::executeDropRange(const StorageReplicatedMergeTree::LogEntry & entry)
|
2014-08-05 13:49:44 +00:00
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
|
2014-08-07 09:23:55 +00:00
|
|
|
|
LOG_INFO(log, (entry.detach ? "Detaching" : "Removing") << " parts inside " << entry.new_part_name << ".");
|
2014-08-05 13:49:44 +00:00
|
|
|
|
|
|
|
|
|
{
|
|
|
|
|
LogEntries to_wait;
|
|
|
|
|
size_t removed_entries = 0;
|
|
|
|
|
|
|
|
|
|
/// Удалим из очереди операции с кусками, содержащимися в удаляемом диапазоне.
|
|
|
|
|
std::unique_lock<std::mutex> lock(queue_mutex);
|
|
|
|
|
for (LogEntries::iterator it = queue.begin(); it != queue.end();)
|
|
|
|
|
{
|
|
|
|
|
if (((*it)->type == LogEntry::GET_PART || (*it)->type == LogEntry::MERGE_PARTS) &&
|
|
|
|
|
ActiveDataPartSet::contains(entry.new_part_name, (*it)->new_part_name))
|
|
|
|
|
{
|
|
|
|
|
if ((*it)->currently_executing)
|
|
|
|
|
to_wait.push_back(*it);
|
|
|
|
|
auto code = zookeeper->tryRemove(replica_path + "/queue/" + (*it)->znode_name);
|
|
|
|
|
if (code != ZOK)
|
|
|
|
|
LOG_INFO(log, "Couldn't remove " << replica_path + "/queue/" + (*it)->znode_name << ": "
|
|
|
|
|
<< zkutil::ZooKeeper::error2string(code));
|
|
|
|
|
queue.erase(it++);
|
|
|
|
|
++removed_entries;
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
++it;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
LOG_DEBUG(log, "Removed " << removed_entries << " entries from queue. "
|
|
|
|
|
"Waiting for " << to_wait.size() << " entries that are currently executing.");
|
|
|
|
|
|
|
|
|
|
/// Дождемся завершения операций с кусками, содержащимися в удаляемом диапазоне.
|
2014-08-07 11:46:01 +00:00
|
|
|
|
for (LogEntryPtr & entry : to_wait)
|
|
|
|
|
entry->execution_complete.wait(lock, [&entry] { return !entry->currently_executing; });
|
2014-08-05 13:49:44 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-08-07 09:23:55 +00:00
|
|
|
|
LOG_DEBUG(log, (entry.detach ? "Detaching" : "Removing") << " parts.");
|
2014-08-05 13:49:44 +00:00
|
|
|
|
size_t removed_parts = 0;
|
|
|
|
|
|
|
|
|
|
/// Удалим куски, содержащиеся в удаляемом диапазоне.
|
|
|
|
|
auto parts = data.getDataParts();
|
|
|
|
|
for (const auto & part : parts)
|
|
|
|
|
{
|
|
|
|
|
if (!ActiveDataPartSet::contains(entry.new_part_name, part->name))
|
|
|
|
|
continue;
|
|
|
|
|
LOG_DEBUG(log, "Removing part " << part->name);
|
|
|
|
|
++removed_parts;
|
|
|
|
|
|
2014-08-07 09:23:55 +00:00
|
|
|
|
/// Если кусок удалять не нужно, надежнее переместить директорию до изменений в ZooKeeper.
|
|
|
|
|
if (entry.detach)
|
|
|
|
|
data.renameAndDetachPart(part);
|
|
|
|
|
|
2014-08-05 13:49:44 +00:00
|
|
|
|
zkutil::Ops ops;
|
2015-09-20 11:02:59 +00:00
|
|
|
|
removePartFromZooKeeper(part->name, ops);
|
2014-08-05 13:49:44 +00:00
|
|
|
|
zookeeper->multi(ops);
|
|
|
|
|
|
2014-08-07 11:46:01 +00:00
|
|
|
|
/// Если кусок нужно удалить, надежнее удалить директорию после изменений в ZooKeeper.
|
2014-08-07 09:23:55 +00:00
|
|
|
|
if (!entry.detach)
|
2014-08-08 08:28:13 +00:00
|
|
|
|
data.replaceParts({part}, {}, true);
|
2014-08-05 13:49:44 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-08-07 11:46:01 +00:00
|
|
|
|
LOG_INFO(log, (entry.detach ? "Detached " : "Removed ") << removed_parts << " parts inside " << entry.new_part_name << ".");
|
2014-08-08 08:28:13 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-08-08 08:28:13 +00:00
|
|
|
|
bool StorageReplicatedMergeTree::executeAttachPart(const StorageReplicatedMergeTree::LogEntry & entry)
|
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
|
2014-08-08 08:28:13 +00:00
|
|
|
|
String source_path = (entry.attach_unreplicated ? "unreplicated/" : "detached/") + entry.source_part_name;
|
|
|
|
|
|
|
|
|
|
LOG_INFO(log, "Attaching part " << entry.source_part_name << " from " << source_path << " as " << entry.new_part_name);
|
|
|
|
|
|
2014-08-08 10:28:46 +00:00
|
|
|
|
if (!Poco::File(data.getFullPath() + source_path).exists())
|
2014-08-08 08:28:13 +00:00
|
|
|
|
{
|
2014-08-08 10:28:46 +00:00
|
|
|
|
LOG_INFO(log, "No part at " << source_path << ". Will fetch it instead");
|
2014-08-08 08:28:13 +00:00
|
|
|
|
return false;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
LOG_DEBUG(log, "Checking data");
|
|
|
|
|
MergeTreeData::MutableDataPartPtr part = data.loadPartAndFixMetadata(source_path);
|
|
|
|
|
|
|
|
|
|
zkutil::Ops ops;
|
|
|
|
|
checkPartAndAddToZooKeeper(part, ops, entry.new_part_name);
|
|
|
|
|
|
|
|
|
|
if (entry.attach_unreplicated && unreplicated_data)
|
|
|
|
|
{
|
|
|
|
|
MergeTreeData::DataPartPtr unreplicated_part = unreplicated_data->getPartIfExists(entry.source_part_name);
|
|
|
|
|
if (unreplicated_part)
|
|
|
|
|
unreplicated_data->detachPartInPlace(unreplicated_part);
|
|
|
|
|
else
|
|
|
|
|
LOG_WARNING(log, "Unreplicated part " << entry.source_part_name << " is already detached");
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
zookeeper->multi(ops);
|
|
|
|
|
|
|
|
|
|
/// NOTE: Не можем использовать renameTempPartAndAdd, потому что кусок не временный - если что-то пойдет не так, его не нужно удалять.
|
|
|
|
|
part->renameTo(entry.new_part_name);
|
|
|
|
|
part->name = entry.new_part_name;
|
|
|
|
|
ActiveDataPartSet::parsePartName(part->name, *part);
|
|
|
|
|
|
|
|
|
|
data.attachPart(part);
|
|
|
|
|
|
|
|
|
|
LOG_INFO(log, "Finished attaching part " << entry.new_part_name);
|
2014-08-05 13:49:44 +00:00
|
|
|
|
|
2014-08-08 09:58:56 +00:00
|
|
|
|
/// На месте удаленных кусков могут появиться новые, с другими данными.
|
|
|
|
|
context.resetCaches();
|
|
|
|
|
|
2014-08-05 13:49:44 +00:00
|
|
|
|
return true;
|
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-04-03 11:48:28 +00:00
|
|
|
|
void StorageReplicatedMergeTree::queueUpdatingThread()
|
|
|
|
|
{
|
|
|
|
|
while (!shutdown_called)
|
|
|
|
|
{
|
2014-04-04 10:37:33 +00:00
|
|
|
|
try
|
|
|
|
|
{
|
2014-07-15 14:37:49 +00:00
|
|
|
|
pullLogsToQueue(queue_updating_event);
|
|
|
|
|
queue_updating_event->wait();
|
2014-04-04 10:37:33 +00:00
|
|
|
|
}
|
2014-10-02 23:31:18 +00:00
|
|
|
|
catch (const zkutil::KeeperException & e)
|
2014-08-08 12:53:55 +00:00
|
|
|
|
{
|
|
|
|
|
if (e.code == ZINVALIDSTATE)
|
2014-10-17 01:05:51 +00:00
|
|
|
|
restarting_thread->wakeup();
|
2014-08-08 12:53:55 +00:00
|
|
|
|
|
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
|
|
|
|
|
|
queue_updating_event->tryWait(ERROR_SLEEP_MS);
|
|
|
|
|
}
|
2014-04-04 10:37:33 +00:00
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
|
|
2014-07-15 14:37:49 +00:00
|
|
|
|
queue_updating_event->tryWait(ERROR_SLEEP_MS);
|
|
|
|
|
}
|
2014-04-03 11:48:28 +00:00
|
|
|
|
}
|
2014-07-15 14:37:49 +00:00
|
|
|
|
|
2014-10-06 05:18:17 +00:00
|
|
|
|
LOG_DEBUG(log, "Queue updating thread finished");
|
2014-04-03 08:47:59 +00:00
|
|
|
|
}
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-07-02 12:30:38 +00:00
|
|
|
|
bool StorageReplicatedMergeTree::queueTask(BackgroundProcessingPool::Context & pool_context)
|
2014-04-03 11:48:28 +00:00
|
|
|
|
{
|
2014-08-05 13:49:44 +00:00
|
|
|
|
LogEntryPtr entry;
|
2014-04-03 11:48:28 +00:00
|
|
|
|
|
2014-07-02 12:30:38 +00:00
|
|
|
|
try
|
|
|
|
|
{
|
2014-12-02 19:08:18 +00:00
|
|
|
|
std::lock_guard<std::mutex> lock(queue_mutex);
|
2014-07-02 12:30:38 +00:00
|
|
|
|
bool empty = queue.empty();
|
|
|
|
|
if (!empty)
|
2014-04-03 11:48:28 +00:00
|
|
|
|
{
|
2014-07-02 12:30:38 +00:00
|
|
|
|
for (LogEntries::iterator it = queue.begin(); it != queue.end(); ++it)
|
2014-04-03 11:48:28 +00:00
|
|
|
|
{
|
2015-09-24 00:21:02 +00:00
|
|
|
|
if ((*it)->currently_executing)
|
|
|
|
|
continue;
|
|
|
|
|
|
|
|
|
|
if (shouldExecuteLogEntry(**it, (*it)->postpone_reason))
|
2014-04-07 15:45:46 +00:00
|
|
|
|
{
|
2014-07-02 12:30:38 +00:00
|
|
|
|
entry = *it;
|
2014-08-05 13:49:44 +00:00
|
|
|
|
entry->tagPartAsFuture(*this);
|
2014-08-04 15:48:03 +00:00
|
|
|
|
queue.splice(queue.end(), queue, it);
|
2014-08-05 13:49:44 +00:00
|
|
|
|
entry->currently_executing = true;
|
2015-09-21 22:43:38 +00:00
|
|
|
|
++entry->num_tries;
|
|
|
|
|
entry->last_attempt_time = time(0);
|
2014-07-02 12:30:38 +00:00
|
|
|
|
break;
|
2014-04-07 15:45:46 +00:00
|
|
|
|
}
|
2015-09-24 00:21:02 +00:00
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
++entry->num_postponed;
|
|
|
|
|
entry->last_postpone_time = time(0);
|
|
|
|
|
}
|
2014-04-03 11:48:28 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
2014-07-02 12:30:38 +00:00
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
|
}
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-08-05 13:49:44 +00:00
|
|
|
|
if (!entry)
|
2014-07-02 12:30:38 +00:00
|
|
|
|
return false;
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2015-09-21 22:43:38 +00:00
|
|
|
|
bool was_exception = true;
|
2014-07-02 12:30:38 +00:00
|
|
|
|
bool success = false;
|
2015-09-21 22:43:38 +00:00
|
|
|
|
ExceptionPtr saved_exception;
|
2014-04-02 07:59:43 +00:00
|
|
|
|
|
2014-07-02 12:30:38 +00:00
|
|
|
|
try
|
|
|
|
|
{
|
2015-09-21 22:43:38 +00:00
|
|
|
|
try
|
2014-07-18 16:05:57 +00:00
|
|
|
|
{
|
2015-09-21 22:43:38 +00:00
|
|
|
|
if (executeLogEntry(*entry, pool_context))
|
|
|
|
|
{
|
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
auto code = zookeeper->tryRemove(replica_path + "/queue/" + entry->znode_name);
|
2014-08-04 15:48:03 +00:00
|
|
|
|
|
2015-09-21 22:43:38 +00:00
|
|
|
|
if (code != ZOK)
|
|
|
|
|
LOG_ERROR(log, "Couldn't remove " << replica_path + "/queue/" + entry->znode_name << ": "
|
|
|
|
|
<< zkutil::ZooKeeper::error2string(code) + ". This shouldn't happen often.");
|
2014-08-04 15:48:03 +00:00
|
|
|
|
|
2015-09-21 22:43:38 +00:00
|
|
|
|
success = true;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
saved_exception = cloneCurrentException();
|
|
|
|
|
throw;
|
2014-07-18 16:05:57 +00:00
|
|
|
|
}
|
2014-04-03 11:48:28 +00:00
|
|
|
|
|
2015-09-21 22:43:38 +00:00
|
|
|
|
was_exception = false;
|
2014-07-02 12:30:38 +00:00
|
|
|
|
}
|
2015-09-18 06:39:39 +00:00
|
|
|
|
catch (const Exception & e)
|
2014-07-02 12:30:38 +00:00
|
|
|
|
{
|
|
|
|
|
if (e.code() == ErrorCodes::NO_REPLICA_HAS_PART)
|
2015-09-18 06:39:39 +00:00
|
|
|
|
{
|
2014-07-18 15:41:04 +00:00
|
|
|
|
/// Если ни у кого нет нужного куска, наверно, просто не все реплики работают; не будем писать в лог с уровнем Error.
|
2014-07-02 12:30:38 +00:00
|
|
|
|
LOG_INFO(log, e.displayText());
|
2015-09-18 06:39:39 +00:00
|
|
|
|
}
|
|
|
|
|
else if (e.code() == ErrorCodes::ABORTED)
|
|
|
|
|
{
|
|
|
|
|
/// Прерванный мердж - не ошибка.
|
|
|
|
|
LOG_INFO(log, "Merge cancelled");
|
|
|
|
|
}
|
2014-07-02 12:30:38 +00:00
|
|
|
|
else
|
2014-04-04 10:37:33 +00:00
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
2014-07-02 12:30:38 +00:00
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
|
}
|
2014-04-03 11:48:28 +00:00
|
|
|
|
|
2014-08-05 13:49:44 +00:00
|
|
|
|
entry->future_part_tagger = nullptr;
|
|
|
|
|
|
2014-12-02 19:08:18 +00:00
|
|
|
|
std::lock_guard<std::mutex> lock(queue_mutex);
|
2014-08-05 13:49:44 +00:00
|
|
|
|
|
|
|
|
|
entry->currently_executing = false;
|
2015-09-21 22:43:38 +00:00
|
|
|
|
entry->exception = saved_exception;
|
2014-08-05 13:49:44 +00:00
|
|
|
|
entry->execution_complete.notify_all();
|
|
|
|
|
|
|
|
|
|
if (success)
|
2014-07-02 12:30:38 +00:00
|
|
|
|
{
|
2014-08-05 13:49:44 +00:00
|
|
|
|
/// Удалим задание из очереди.
|
|
|
|
|
/// Нельзя просто обратиться по заранее сохраненному итератору, потому что задание мог успеть удалить кто-то другой.
|
|
|
|
|
for (LogEntries::iterator it = queue.end(); it != queue.begin();)
|
2014-08-04 15:48:03 +00:00
|
|
|
|
{
|
2014-08-05 13:49:44 +00:00
|
|
|
|
--it;
|
|
|
|
|
if (*it == entry)
|
|
|
|
|
{
|
2014-08-04 15:48:03 +00:00
|
|
|
|
queue.erase(it);
|
2014-08-05 13:49:44 +00:00
|
|
|
|
break;
|
|
|
|
|
}
|
2014-08-04 15:48:03 +00:00
|
|
|
|
}
|
2014-04-03 11:48:28 +00:00
|
|
|
|
}
|
2014-07-02 12:30:38 +00:00
|
|
|
|
|
2014-07-18 15:41:04 +00:00
|
|
|
|
/// Если не было исключения, не нужно спать.
|
2015-09-21 22:43:38 +00:00
|
|
|
|
return !was_exception;
|
2014-04-03 11:48:28 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-04-04 10:37:33 +00:00
|
|
|
|
void StorageReplicatedMergeTree::mergeSelectingThread()
|
|
|
|
|
{
|
2014-07-25 14:31:50 +00:00
|
|
|
|
bool need_pull = true;
|
2014-04-04 10:37:33 +00:00
|
|
|
|
|
2014-09-12 00:32:27 +00:00
|
|
|
|
/** Может много времени тратиться на определение, можно ли мерджить два рядом стоящих куска.
|
|
|
|
|
* Два рядом стоящих куска можно мерджить, если все номера блоков между их номерами не используются ("заброшены", abandoned).
|
|
|
|
|
* Это значит, что между этими кусками не может быть вставлен другой кусок.
|
|
|
|
|
*
|
|
|
|
|
* Но если номера соседних блоков отличаются достаточно сильно (обычно, если между ними много "заброшенных" блоков),
|
|
|
|
|
* то делается слишком много чтений из ZooKeeper, чтобы узнать, можно ли их мерджить.
|
|
|
|
|
*
|
2014-09-12 01:06:38 +00:00
|
|
|
|
* Воспользуемся утверждением, что если пару кусков было можно мерджить, и их мердж ещё не запланирован,
|
|
|
|
|
* то и сейчас их можно мерджить, и будем запоминать это состояние, чтобы не делать много раз одинаковые запросы в ZooKeeper.
|
2014-09-12 00:32:27 +00:00
|
|
|
|
*
|
|
|
|
|
* TODO Интересно, как это сочетается с DROP PARTITION и затем ATTACH PARTITION.
|
|
|
|
|
*/
|
|
|
|
|
std::set<std::pair<std::string, std::string>> memoized_parts_that_could_be_merged;
|
|
|
|
|
|
|
|
|
|
auto can_merge = [&memoized_parts_that_could_be_merged, this]
|
|
|
|
|
(const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right) -> bool
|
|
|
|
|
{
|
2014-09-12 01:06:38 +00:00
|
|
|
|
/// Если какой-то из кусков уже собираются слить в больший, не соглашаемся его сливать.
|
|
|
|
|
if (virtual_parts.getContainingPart(left->name) != left->name ||
|
|
|
|
|
virtual_parts.getContainingPart(right->name) != right->name)
|
|
|
|
|
return false;
|
|
|
|
|
|
2014-09-12 00:32:27 +00:00
|
|
|
|
auto key = std::make_pair(left->name, right->name);
|
|
|
|
|
if (memoized_parts_that_could_be_merged.count(key))
|
|
|
|
|
return true;
|
|
|
|
|
|
2014-09-12 01:06:38 +00:00
|
|
|
|
String month_name = left->name.substr(0, 6);
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
2014-09-12 01:06:38 +00:00
|
|
|
|
|
2015-09-11 02:13:59 +00:00
|
|
|
|
/// Нельзя сливать куски, среди которых находится кусок, для которого неудовлетворён кворум.
|
|
|
|
|
/// Замечание: теоретически, это можно было бы разрешить. Но это сделает логику более сложной.
|
|
|
|
|
String quorum_node_value;
|
|
|
|
|
if (zookeeper->tryGet(zookeeper_path + "/quorum/status", quorum_node_value))
|
|
|
|
|
{
|
|
|
|
|
ReplicatedMergeTreeQuorumEntry quorum_entry;
|
|
|
|
|
quorum_entry.fromString(quorum_node_value);
|
|
|
|
|
|
|
|
|
|
ActiveDataPartSet::Part part_info;
|
|
|
|
|
ActiveDataPartSet::parsePartName(quorum_entry.part_name, part_info);
|
|
|
|
|
|
|
|
|
|
if (part_info.left != part_info.right)
|
|
|
|
|
throw Exception("Logical error: part written with quorum covers more than one block numbers", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
|
|
|
|
|
if (left->right <= part_info.left && right->left >= part_info.right)
|
|
|
|
|
return false;
|
|
|
|
|
}
|
|
|
|
|
|
2014-09-12 01:06:38 +00:00
|
|
|
|
/// Можно слить куски, если все номера между ними заброшены - не соответствуют никаким блокам.
|
2015-08-17 21:09:36 +00:00
|
|
|
|
/// Номера до RESERVED_BLOCK_NUMBERS всегда не соответствуют никаким блокам.
|
|
|
|
|
for (Int64 number = std::max(RESERVED_BLOCK_NUMBERS, left->right + 1); number <= right->left - 1; ++number)
|
2014-09-12 01:06:38 +00:00
|
|
|
|
{
|
|
|
|
|
String path1 = zookeeper_path + "/block_numbers/" + month_name + "/block-" + padIndex(number);
|
|
|
|
|
String path2 = zookeeper_path + "/nonincrement_block_numbers/" + month_name + "/block-" + padIndex(number);
|
2014-09-12 00:32:27 +00:00
|
|
|
|
|
2014-09-12 01:06:38 +00:00
|
|
|
|
if (AbandonableLockInZooKeeper::check(path1, *zookeeper) != AbandonableLockInZooKeeper::ABANDONED &&
|
|
|
|
|
AbandonableLockInZooKeeper::check(path2, *zookeeper) != AbandonableLockInZooKeeper::ABANDONED)
|
|
|
|
|
return false;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
memoized_parts_that_could_be_merged.insert(key);
|
|
|
|
|
return true;
|
2014-09-12 00:32:27 +00:00
|
|
|
|
};
|
|
|
|
|
|
2014-04-04 10:37:33 +00:00
|
|
|
|
while (!shutdown_called && is_leader_node)
|
|
|
|
|
{
|
2014-04-07 18:14:39 +00:00
|
|
|
|
bool success = false;
|
2014-04-04 10:37:33 +00:00
|
|
|
|
|
2014-04-07 18:14:39 +00:00
|
|
|
|
try
|
2014-04-04 10:37:33 +00:00
|
|
|
|
{
|
2014-12-02 19:08:18 +00:00
|
|
|
|
std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
|
2014-08-07 09:23:55 +00:00
|
|
|
|
|
2014-07-25 14:31:50 +00:00
|
|
|
|
if (need_pull)
|
|
|
|
|
{
|
|
|
|
|
/// Нужно загрузить новую запись в очередь перед тем, как выбирать куски для слияния.
|
|
|
|
|
/// (чтобы кусок добавился в virtual_parts).
|
|
|
|
|
pullLogsToQueue();
|
|
|
|
|
need_pull = false;
|
|
|
|
|
}
|
|
|
|
|
|
2014-09-17 23:57:25 +00:00
|
|
|
|
/** Сколько в очереди или в фоновом потоке мерджей крупных кусков.
|
|
|
|
|
* Если их больше половины от размера пула потоков для мерджа, то можно мерджить только мелкие куски.
|
|
|
|
|
*/
|
|
|
|
|
auto & background_pool = context.getBackgroundPool();
|
|
|
|
|
|
|
|
|
|
size_t big_merges_current = background_pool.getCounter("replicated big merges");
|
|
|
|
|
size_t max_number_of_big_merges = background_pool.getNumberOfThreads() / 2;
|
2014-04-07 18:14:39 +00:00
|
|
|
|
size_t merges_queued = 0;
|
2014-09-17 23:57:25 +00:00
|
|
|
|
size_t big_merges_queued = 0;
|
2014-04-04 10:37:33 +00:00
|
|
|
|
|
2014-09-17 23:57:25 +00:00
|
|
|
|
if (big_merges_current < max_number_of_big_merges)
|
2014-04-07 18:14:39 +00:00
|
|
|
|
{
|
2014-12-02 19:08:18 +00:00
|
|
|
|
std::lock_guard<std::mutex> lock(queue_mutex);
|
2014-04-04 10:37:33 +00:00
|
|
|
|
|
2014-04-07 18:14:39 +00:00
|
|
|
|
for (const auto & entry : queue)
|
2014-05-26 11:40:22 +00:00
|
|
|
|
{
|
2014-08-05 13:49:44 +00:00
|
|
|
|
if (entry->type == LogEntry::MERGE_PARTS)
|
2014-05-26 11:40:22 +00:00
|
|
|
|
{
|
2014-04-07 18:14:39 +00:00
|
|
|
|
++merges_queued;
|
2014-05-26 11:40:22 +00:00
|
|
|
|
|
2014-09-17 23:57:25 +00:00
|
|
|
|
if (big_merges_current + big_merges_queued < max_number_of_big_merges)
|
2014-05-26 11:40:22 +00:00
|
|
|
|
{
|
2014-08-05 13:49:44 +00:00
|
|
|
|
for (const String & name : entry->parts_to_merge)
|
2014-05-26 11:40:22 +00:00
|
|
|
|
{
|
2014-07-25 11:38:46 +00:00
|
|
|
|
MergeTreeData::DataPartPtr part = data.getActiveContainingPart(name);
|
2014-05-26 11:40:22 +00:00
|
|
|
|
if (!part || part->name != name)
|
|
|
|
|
continue;
|
2014-09-17 23:57:25 +00:00
|
|
|
|
|
2014-07-23 15:24:45 +00:00
|
|
|
|
if (part->size_in_bytes > data.settings.max_bytes_to_merge_parts_small)
|
2014-05-26 11:40:22 +00:00
|
|
|
|
{
|
2014-09-17 23:57:25 +00:00
|
|
|
|
++big_merges_queued;
|
2014-05-26 11:40:22 +00:00
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
2014-04-07 18:14:39 +00:00
|
|
|
|
}
|
2014-04-04 10:37:33 +00:00
|
|
|
|
|
2014-09-17 23:57:25 +00:00
|
|
|
|
bool only_small = big_merges_current + big_merges_queued >= max_number_of_big_merges;
|
|
|
|
|
|
2014-12-21 01:27:56 +00:00
|
|
|
|
if (big_merges_current || merges_queued)
|
|
|
|
|
LOG_TRACE(log, "Currently executing big merges: " << big_merges_current
|
|
|
|
|
<< ". Queued big merges: " << big_merges_queued
|
|
|
|
|
<< ". All merges in queue: " << merges_queued
|
|
|
|
|
<< ". Max number of big merges: " << max_number_of_big_merges
|
|
|
|
|
<< (only_small ? ". So, will select only small parts to merge." : "."));
|
2014-09-17 23:57:25 +00:00
|
|
|
|
|
2014-07-02 10:16:49 +00:00
|
|
|
|
do
|
2014-04-07 18:14:39 +00:00
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
|
2014-07-02 12:30:38 +00:00
|
|
|
|
if (merges_queued >= data.settings.max_replicated_merges_in_queue)
|
2014-09-17 23:57:25 +00:00
|
|
|
|
{
|
2015-09-19 05:56:40 +00:00
|
|
|
|
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.");
|
2014-07-02 10:16:49 +00:00
|
|
|
|
break;
|
2014-09-17 23:57:25 +00:00
|
|
|
|
}
|
2014-04-04 10:37:33 +00:00
|
|
|
|
|
2014-07-02 10:16:49 +00:00
|
|
|
|
MergeTreeData::DataPartsVector parts;
|
2014-04-04 12:47:57 +00:00
|
|
|
|
|
2014-04-04 17:20:45 +00:00
|
|
|
|
String merged_name;
|
|
|
|
|
|
2015-06-11 00:35:36 +00:00
|
|
|
|
size_t disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);
|
|
|
|
|
|
|
|
|
|
if ( !merger.selectPartsToMerge(parts, merged_name, disk_space, false, false, only_small, can_merge)
|
|
|
|
|
&& !merger.selectPartsToMerge(parts, merged_name, disk_space, true, false, only_small, can_merge))
|
2014-09-17 23:57:25 +00:00
|
|
|
|
{
|
2014-07-02 10:16:49 +00:00
|
|
|
|
break;
|
2014-09-17 23:57:25 +00:00
|
|
|
|
}
|
2014-04-04 17:20:45 +00:00
|
|
|
|
|
2014-08-01 09:52:55 +00:00
|
|
|
|
bool all_in_zk = true;
|
|
|
|
|
for (const auto & part : parts)
|
|
|
|
|
{
|
|
|
|
|
/// Если о каком-то из кусков нет информации в ZK, не будем сливать.
|
|
|
|
|
if (!zookeeper->exists(replica_path + "/parts/" + part->name))
|
|
|
|
|
{
|
|
|
|
|
all_in_zk = false;
|
2015-09-19 05:56:40 +00:00
|
|
|
|
|
|
|
|
|
if (part->modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER < time(0))
|
|
|
|
|
{
|
|
|
|
|
LOG_WARNING(log, "Part " << part->name << " (that was selected for merge)"
|
|
|
|
|
<< " with age " << (time(0) - part->modification_time)
|
|
|
|
|
<< " seconds exists locally but not in ZooKeeper."
|
|
|
|
|
<< " Won't do merge with that part and will check it.");
|
|
|
|
|
enqueuePartForCheck(part->name);
|
|
|
|
|
}
|
2014-08-01 09:52:55 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
if (!all_in_zk)
|
|
|
|
|
break;
|
|
|
|
|
|
2014-07-02 10:16:49 +00:00
|
|
|
|
LogEntry entry;
|
|
|
|
|
entry.type = LogEntry::MERGE_PARTS;
|
|
|
|
|
entry.source_replica = replica_name;
|
|
|
|
|
entry.new_part_name = merged_name;
|
2014-04-04 17:20:45 +00:00
|
|
|
|
|
2014-07-02 10:16:49 +00:00
|
|
|
|
for (const auto & part : parts)
|
|
|
|
|
entry.parts_to_merge.push_back(part->name);
|
2014-04-04 10:37:33 +00:00
|
|
|
|
|
2014-07-25 14:31:50 +00:00
|
|
|
|
need_pull = true;
|
2014-07-02 10:16:49 +00:00
|
|
|
|
|
2014-07-25 14:31:50 +00:00
|
|
|
|
zookeeper->create(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential);
|
2014-05-07 13:58:20 +00:00
|
|
|
|
|
|
|
|
|
String month_name = parts[0]->name.substr(0, 6);
|
|
|
|
|
for (size_t i = 0; i + 1 < parts.size(); ++i)
|
2014-04-04 17:20:45 +00:00
|
|
|
|
{
|
2014-05-07 13:58:20 +00:00
|
|
|
|
/// Уберем больше не нужные отметки о несуществующих блоках.
|
2015-08-17 21:09:36 +00:00
|
|
|
|
for (Int64 number = std::max(RESERVED_BLOCK_NUMBERS, parts[i]->right + 1); number <= parts[i + 1]->left - 1; ++number)
|
2014-05-07 13:58:20 +00:00
|
|
|
|
{
|
2014-07-25 16:32:02 +00:00
|
|
|
|
zookeeper->tryRemove(zookeeper_path + "/block_numbers/" + month_name + "/block-" + padIndex(number));
|
|
|
|
|
zookeeper->tryRemove(zookeeper_path + "/nonincrement_block_numbers/" + month_name + "/block-" + padIndex(number));
|
2014-05-07 13:58:20 +00:00
|
|
|
|
}
|
2014-04-04 17:20:45 +00:00
|
|
|
|
}
|
2014-07-02 10:16:49 +00:00
|
|
|
|
|
|
|
|
|
success = true;
|
2014-04-04 10:37:33 +00:00
|
|
|
|
}
|
2014-09-12 01:06:38 +00:00
|
|
|
|
while (false);
|
2014-04-04 10:37:33 +00:00
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
|
}
|
|
|
|
|
|
2014-04-14 10:56:06 +00:00
|
|
|
|
if (shutdown_called || !is_leader_node)
|
2014-04-04 10:37:33 +00:00
|
|
|
|
break;
|
|
|
|
|
|
2014-04-14 10:56:06 +00:00
|
|
|
|
if (!success)
|
2014-07-02 10:16:49 +00:00
|
|
|
|
merge_selecting_event.tryWait(MERGE_SELECTING_SLEEP_MS);
|
2014-04-14 10:56:06 +00:00
|
|
|
|
}
|
2014-07-15 14:37:49 +00:00
|
|
|
|
|
2014-09-12 02:30:01 +00:00
|
|
|
|
LOG_DEBUG(log, "Merge selecting thread finished");
|
2014-04-14 10:56:06 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-04-04 10:37:33 +00:00
|
|
|
|
|
2014-07-15 15:51:27 +00:00
|
|
|
|
void StorageReplicatedMergeTree::alterThread()
|
|
|
|
|
{
|
|
|
|
|
bool force_recheck_parts = true;
|
|
|
|
|
|
|
|
|
|
while (!shutdown_called)
|
|
|
|
|
{
|
|
|
|
|
try
|
|
|
|
|
{
|
2014-09-26 00:49:50 +00:00
|
|
|
|
/** Имеем описание столбцов в ZooKeeper, общее для всех реплик (Пример: /clickhouse/tables/02-06/visits/columns),
|
|
|
|
|
* а также описание столбцов в локальном файле с метаданными (data.getColumnsList()).
|
|
|
|
|
*
|
|
|
|
|
* Если эти описания отличаются - нужно сделать ALTER.
|
|
|
|
|
*
|
|
|
|
|
* Если запомненная версия ноды (columns_version) отличается от версии в ZK,
|
|
|
|
|
* то описание столбцов в ZK не обязательно отличается от локального
|
|
|
|
|
* - такое может быть при цикле из ALTER-ов, который в целом, ничего не меняет.
|
|
|
|
|
* В этом случае, надо обновить запомненный номер версии,
|
|
|
|
|
* а также всё-равно проверить структуру кусков, и, при необходимости, сделать ALTER.
|
|
|
|
|
*
|
|
|
|
|
* Запомненный номер версии нужно обновить после обновления метаданных, под блокировкой.
|
|
|
|
|
* Этот номер версии проверяется на соответствие актуальному при INSERT-е.
|
|
|
|
|
* То есть, так добиваемся, чтобы вставлялись блоки с правильной структурой.
|
|
|
|
|
*
|
|
|
|
|
* При старте сервера, мог быть не завершён предыдущий ALTER.
|
|
|
|
|
* Поэтому, в первый раз, независимо от изменений, проверяем структуру всех part-ов,
|
|
|
|
|
* (Пример: /clickhouse/tables/02-06/visits/replicas/example02-06-1.yandex.ru/parts/20140806_20140831_131664_134988_3296/columns)
|
|
|
|
|
* и делаем ALTER, если необходимо.
|
|
|
|
|
*
|
|
|
|
|
* TODO: Слишком сложно, всё переделать.
|
|
|
|
|
*/
|
|
|
|
|
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
|
2014-07-15 15:51:27 +00:00
|
|
|
|
zkutil::Stat stat;
|
2014-10-21 12:11:20 +00:00
|
|
|
|
const String columns_str = zookeeper->get(zookeeper_path + "/columns", &stat, alter_thread_event);
|
2015-05-28 03:49:28 +00:00
|
|
|
|
auto columns_desc = ColumnsDescription<true>::parse(columns_str);
|
2014-11-10 16:16:43 +00:00
|
|
|
|
|
|
|
|
|
auto & columns = columns_desc.columns;
|
|
|
|
|
auto & materialized_columns = columns_desc.materialized;
|
|
|
|
|
auto & alias_columns = columns_desc.alias;
|
|
|
|
|
auto & column_defaults = columns_desc.defaults;
|
2014-07-15 15:51:27 +00:00
|
|
|
|
|
2014-09-26 00:49:50 +00:00
|
|
|
|
bool changed_version = (stat.version != columns_version);
|
2014-07-15 15:51:27 +00:00
|
|
|
|
|
2014-07-28 14:31:07 +00:00
|
|
|
|
MergeTreeData::DataParts parts;
|
|
|
|
|
|
2014-07-15 15:51:27 +00:00
|
|
|
|
/// Если описание столбцов изменилось, обновим структуру таблицы локально.
|
2014-09-26 00:49:50 +00:00
|
|
|
|
if (changed_version)
|
2014-07-15 15:51:27 +00:00
|
|
|
|
{
|
2015-03-26 23:48:50 +00:00
|
|
|
|
LOG_INFO(log, "Changed version of 'columns' node in ZooKeeper. Waiting for structure write lock.");
|
|
|
|
|
|
2014-07-15 15:51:27 +00:00
|
|
|
|
auto table_lock = lockStructureForAlter();
|
2014-09-26 00:49:50 +00:00
|
|
|
|
|
2014-10-21 12:11:20 +00:00
|
|
|
|
const auto columns_changed = columns != data.getColumnsListNonMaterialized();
|
|
|
|
|
const auto materialized_columns_changed = materialized_columns != data.materialized_columns;
|
|
|
|
|
const auto alias_columns_changed = alias_columns != data.alias_columns;
|
2014-10-23 12:16:24 +00:00
|
|
|
|
const auto column_defaults_changed = column_defaults != data.column_defaults;
|
2014-10-21 12:11:20 +00:00
|
|
|
|
|
|
|
|
|
if (columns_changed || materialized_columns_changed || alias_columns_changed ||
|
|
|
|
|
column_defaults_changed)
|
2014-07-15 15:51:27 +00:00
|
|
|
|
{
|
|
|
|
|
LOG_INFO(log, "Columns list changed in ZooKeeper. Applying changes locally.");
|
2014-09-26 00:49:50 +00:00
|
|
|
|
|
2014-10-16 13:37:01 +00:00
|
|
|
|
InterpreterAlterQuery::updateMetadata(database_name, table_name, columns,
|
|
|
|
|
materialized_columns, alias_columns, column_defaults, context);
|
|
|
|
|
|
2014-10-21 12:11:20 +00:00
|
|
|
|
if (columns_changed)
|
|
|
|
|
{
|
|
|
|
|
data.setColumnsList(columns);
|
|
|
|
|
|
|
|
|
|
if (unreplicated_data)
|
|
|
|
|
unreplicated_data->setColumnsList(columns);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (materialized_columns_changed)
|
|
|
|
|
{
|
|
|
|
|
this->materialized_columns = materialized_columns;
|
|
|
|
|
data.materialized_columns = std::move(materialized_columns);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (alias_columns_changed)
|
|
|
|
|
{
|
|
|
|
|
this->alias_columns = alias_columns;
|
|
|
|
|
data.alias_columns = std::move(alias_columns);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (column_defaults_changed)
|
|
|
|
|
{
|
|
|
|
|
this->column_defaults = column_defaults;
|
|
|
|
|
data.column_defaults = std::move(column_defaults);
|
|
|
|
|
}
|
|
|
|
|
|
2014-07-15 15:51:27 +00:00
|
|
|
|
LOG_INFO(log, "Applied changes to table.");
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
2014-09-26 00:49:50 +00:00
|
|
|
|
LOG_INFO(log, "Columns version changed in ZooKeeper, but data wasn't changed. It's like cyclic ALTERs.");
|
2014-07-15 15:51:27 +00:00
|
|
|
|
}
|
2014-09-26 00:49:50 +00:00
|
|
|
|
|
|
|
|
|
/// Нужно получить список кусков под блокировкой таблицы, чтобы избежать race condition с мерджем.
|
|
|
|
|
parts = data.getDataParts();
|
|
|
|
|
|
|
|
|
|
columns_version = stat.version;
|
2014-07-15 15:51:27 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Обновим куски.
|
2014-09-26 00:49:50 +00:00
|
|
|
|
if (changed_version || force_recheck_parts)
|
2014-07-15 15:51:27 +00:00
|
|
|
|
{
|
2014-09-26 00:49:50 +00:00
|
|
|
|
auto table_lock = lockStructure(false);
|
|
|
|
|
|
|
|
|
|
if (changed_version)
|
2014-07-15 15:51:27 +00:00
|
|
|
|
LOG_INFO(log, "ALTER-ing parts");
|
|
|
|
|
|
|
|
|
|
int changed_parts = 0;
|
|
|
|
|
|
2014-09-26 00:49:50 +00:00
|
|
|
|
if (!changed_version)
|
2014-07-28 14:31:07 +00:00
|
|
|
|
parts = data.getDataParts();
|
|
|
|
|
|
2014-10-21 12:11:20 +00:00
|
|
|
|
const auto columns_plus_materialized = data.getColumnsList();
|
|
|
|
|
|
2014-07-15 15:51:27 +00:00
|
|
|
|
for (const MergeTreeData::DataPartPtr & part : parts)
|
|
|
|
|
{
|
|
|
|
|
/// Обновим кусок и запишем результат во временные файлы.
|
2014-07-23 12:44:23 +00:00
|
|
|
|
/// TODO: Можно пропускать проверку на слишком большие изменения, если в ZooKeeper есть, например,
|
|
|
|
|
/// нода /flags/force_alter.
|
2014-10-21 12:11:20 +00:00
|
|
|
|
auto transaction = data.alterDataPart(part, columns_plus_materialized);
|
2014-07-15 15:51:27 +00:00
|
|
|
|
|
|
|
|
|
if (!transaction)
|
|
|
|
|
continue;
|
|
|
|
|
|
|
|
|
|
++changed_parts;
|
|
|
|
|
|
|
|
|
|
/// Обновим метаданные куска в ZooKeeper.
|
|
|
|
|
zkutil::Ops ops;
|
2015-02-01 03:49:36 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::SetData(
|
|
|
|
|
replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1));
|
|
|
|
|
ops.push_back(new zkutil::Op::SetData(
|
|
|
|
|
replica_path + "/parts/" + part->name + "/checksums", transaction->getNewChecksums().toString(), -1));
|
2014-07-15 15:51:27 +00:00
|
|
|
|
zookeeper->multi(ops);
|
|
|
|
|
|
|
|
|
|
/// Применим изменения файлов.
|
|
|
|
|
transaction->commit();
|
|
|
|
|
}
|
|
|
|
|
|
2014-07-16 09:32:15 +00:00
|
|
|
|
/// То же самое для нереплицируемых данных.
|
|
|
|
|
if (unreplicated_data)
|
|
|
|
|
{
|
|
|
|
|
parts = unreplicated_data->getDataParts();
|
|
|
|
|
|
|
|
|
|
for (const MergeTreeData::DataPartPtr & part : parts)
|
|
|
|
|
{
|
2014-10-21 12:11:20 +00:00
|
|
|
|
auto transaction = unreplicated_data->alterDataPart(part, columns_plus_materialized);
|
2014-07-16 09:32:15 +00:00
|
|
|
|
|
|
|
|
|
if (!transaction)
|
|
|
|
|
continue;
|
|
|
|
|
|
|
|
|
|
++changed_parts;
|
|
|
|
|
|
|
|
|
|
transaction->commit();
|
|
|
|
|
}
|
|
|
|
|
}
|
2014-07-16 08:58:59 +00:00
|
|
|
|
|
2014-09-26 00:49:50 +00:00
|
|
|
|
/// Список столбцов для конкретной реплики.
|
2014-11-10 16:16:43 +00:00
|
|
|
|
zookeeper->set(replica_path + "/columns", columns_str);
|
2014-07-16 08:58:59 +00:00
|
|
|
|
|
2014-09-26 00:49:50 +00:00
|
|
|
|
if (changed_version)
|
|
|
|
|
{
|
|
|
|
|
if (changed_parts != 0)
|
|
|
|
|
LOG_INFO(log, "ALTER-ed " << changed_parts << " parts");
|
|
|
|
|
else
|
|
|
|
|
LOG_INFO(log, "No parts ALTER-ed");
|
|
|
|
|
}
|
|
|
|
|
|
2014-07-15 15:51:27 +00:00
|
|
|
|
force_recheck_parts = false;
|
|
|
|
|
}
|
|
|
|
|
|
2014-10-01 00:11:32 +00:00
|
|
|
|
parts.clear();
|
2014-07-15 15:51:27 +00:00
|
|
|
|
alter_thread_event->wait();
|
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
|
|
|
|
|
|
force_recheck_parts = true;
|
|
|
|
|
|
|
|
|
|
alter_thread_event->tryWait(ERROR_SLEEP_MS);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2014-09-26 00:49:50 +00:00
|
|
|
|
LOG_DEBUG(log, "Alter thread finished");
|
2014-07-15 15:51:27 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2015-09-20 11:02:59 +00:00
|
|
|
|
void StorageReplicatedMergeTree::removePartFromZooKeeper(const String & part_name, zkutil::Ops & ops)
|
|
|
|
|
{
|
|
|
|
|
String part_path = replica_path + "/parts/" + part_name;
|
|
|
|
|
|
|
|
|
|
ops.push_back(new zkutil::Op::Remove(part_path + "/checksums", -1));
|
|
|
|
|
ops.push_back(new zkutil::Op::Remove(part_path + "/columns", -1));
|
|
|
|
|
ops.push_back(new zkutil::Op::Remove(part_path, -1));
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-07-22 13:49:52 +00:00
|
|
|
|
void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_name)
|
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
|
2014-07-22 13:49:52 +00:00
|
|
|
|
String part_path = replica_path + "/parts/" + part_name;
|
|
|
|
|
|
2014-08-05 13:49:44 +00:00
|
|
|
|
LogEntryPtr log_entry = new LogEntry;
|
|
|
|
|
log_entry->type = LogEntry::GET_PART;
|
2015-09-20 05:50:15 +00:00
|
|
|
|
log_entry->create_time = tryGetPartCreateTime(zookeeper, replica_path, part_name);
|
2014-08-05 13:49:44 +00:00
|
|
|
|
log_entry->source_replica = "";
|
|
|
|
|
log_entry->new_part_name = part_name;
|
2014-07-22 13:49:52 +00:00
|
|
|
|
|
|
|
|
|
zkutil::Ops ops;
|
|
|
|
|
ops.push_back(new zkutil::Op::Create(
|
2014-08-05 13:49:44 +00:00
|
|
|
|
replica_path + "/queue/queue-", log_entry->toString(), zookeeper->getDefaultACL(),
|
2014-07-22 13:49:52 +00:00
|
|
|
|
zkutil::CreateMode::PersistentSequential));
|
2015-09-20 11:02:59 +00:00
|
|
|
|
|
|
|
|
|
removePartFromZooKeeper(part_name, ops);
|
|
|
|
|
|
2014-07-22 13:49:52 +00:00
|
|
|
|
auto results = zookeeper->multi(ops);
|
|
|
|
|
|
|
|
|
|
{
|
2014-12-02 19:08:18 +00:00
|
|
|
|
std::lock_guard<std::mutex> lock(queue_mutex);
|
2014-07-22 13:49:52 +00:00
|
|
|
|
|
|
|
|
|
String path_created = dynamic_cast<zkutil::Op::Create &>(ops[0]).getPathCreated();
|
2014-08-05 13:49:44 +00:00
|
|
|
|
log_entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
|
|
|
|
|
log_entry->addResultToVirtualParts(*this);
|
2014-07-22 13:49:52 +00:00
|
|
|
|
queue.push_back(log_entry);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-07-22 13:49:52 +00:00
|
|
|
|
void StorageReplicatedMergeTree::enqueuePartForCheck(const String & name)
|
|
|
|
|
{
|
2014-10-06 05:18:17 +00:00
|
|
|
|
std::lock_guard<std::mutex> lock(parts_to_check_mutex);
|
2014-07-22 13:49:52 +00:00
|
|
|
|
|
|
|
|
|
if (parts_to_check_set.count(name))
|
|
|
|
|
return;
|
|
|
|
|
parts_to_check_queue.push_back(name);
|
|
|
|
|
parts_to_check_set.insert(name);
|
|
|
|
|
parts_to_check_event.set();
|
|
|
|
|
}
|
|
|
|
|
|
2015-09-15 01:20:40 +00:00
|
|
|
|
|
|
|
|
|
void StorageReplicatedMergeTree::searchForMissingPart(const String & part_name)
|
2014-07-21 10:05:56 +00:00
|
|
|
|
{
|
2015-09-15 01:20:40 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
String part_path = replica_path + "/parts/" + part_name;
|
|
|
|
|
|
|
|
|
|
/// Если кусок есть в ZooKeeper, удалим его оттуда и добавим в очередь задание скачать его.
|
|
|
|
|
if (zookeeper->exists(part_path))
|
2014-07-21 10:05:56 +00:00
|
|
|
|
{
|
2015-09-15 01:20:40 +00:00
|
|
|
|
LOG_WARNING(log, "Checker: Part " << part_name << " exists in ZooKeeper but not locally. "
|
|
|
|
|
"Removing from ZooKeeper and queueing a fetch.");
|
|
|
|
|
ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
|
2014-12-12 20:50:32 +00:00
|
|
|
|
|
2015-09-15 01:20:40 +00:00
|
|
|
|
removePartAndEnqueueFetch(part_name);
|
|
|
|
|
return;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Если куска нет в ZooKeeper, проверим есть ли он хоть у кого-то.
|
|
|
|
|
ActiveDataPartSet::Part part_info;
|
|
|
|
|
ActiveDataPartSet::parsePartName(part_name, part_info);
|
|
|
|
|
|
|
|
|
|
/** Логика такая:
|
|
|
|
|
* - если у какой-то живой или неактивной реплики есть такой кусок, или покрывающий его кусок
|
|
|
|
|
* - всё Ок, ничего делать не нужно, он скачается затем при обработке очереди, когда реплика оживёт;
|
|
|
|
|
* - или, если реплика никогда не оживёт, то администратор удалит или создаст новую реплику с тем же адресом и см. всё сначала;
|
|
|
|
|
* - если ни у кого нет такого или покрывающего его куска, то
|
|
|
|
|
* - если у кого-то есть все составляющие куски, то ничего делать не будем - это просто значит, что другие реплики ещё недоделали мердж
|
|
|
|
|
* - если ни у кого нет всех составляющих кусков, то признаем кусок навечно потерянным,
|
|
|
|
|
* и удалим запись из очереди репликации.
|
|
|
|
|
*/
|
|
|
|
|
|
|
|
|
|
LOG_WARNING(log, "Checker: Checking if anyone has part covering " << part_name << ".");
|
|
|
|
|
|
|
|
|
|
bool found = false;
|
|
|
|
|
|
|
|
|
|
size_t part_length_in_blocks = part_info.right + 1 - part_info.left;
|
|
|
|
|
std::vector<char> found_blocks(part_length_in_blocks);
|
|
|
|
|
|
|
|
|
|
Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
|
|
|
|
|
for (const String & replica : replicas)
|
|
|
|
|
{
|
|
|
|
|
Strings parts = zookeeper->getChildren(zookeeper_path + "/replicas/" + replica + "/parts");
|
|
|
|
|
for (const String & part_on_replica : parts)
|
|
|
|
|
{
|
|
|
|
|
if (part_on_replica == part_name || ActiveDataPartSet::contains(part_on_replica, part_name))
|
2014-07-21 10:05:56 +00:00
|
|
|
|
{
|
2015-09-15 01:20:40 +00:00
|
|
|
|
found = true;
|
|
|
|
|
LOG_WARNING(log, "Checker: Found part " << part_on_replica << " on " << replica);
|
|
|
|
|
break;
|
2014-07-21 10:05:56 +00:00
|
|
|
|
}
|
2015-09-15 01:20:40 +00:00
|
|
|
|
|
|
|
|
|
if (ActiveDataPartSet::contains(part_name, part_on_replica))
|
2014-07-21 10:05:56 +00:00
|
|
|
|
{
|
2015-09-15 01:20:40 +00:00
|
|
|
|
ActiveDataPartSet::Part part_on_replica_info;
|
|
|
|
|
ActiveDataPartSet::parsePartName(part_on_replica, part_on_replica_info);
|
|
|
|
|
|
2015-09-16 20:38:50 +00:00
|
|
|
|
for (auto block_num = part_on_replica_info.left; block_num <= part_on_replica_info.right; ++block_num)
|
2015-09-16 20:40:01 +00:00
|
|
|
|
found_blocks.at(block_num - part_info.left) = 1;
|
2014-07-21 10:05:56 +00:00
|
|
|
|
}
|
2015-09-15 01:20:40 +00:00
|
|
|
|
}
|
|
|
|
|
if (found)
|
|
|
|
|
break;
|
|
|
|
|
}
|
2014-07-21 10:05:56 +00:00
|
|
|
|
|
2015-09-15 01:20:40 +00:00
|
|
|
|
if (found)
|
|
|
|
|
{
|
|
|
|
|
/// На какой-то живой или мёртвой реплике есть нужный кусок или покрывающий его.
|
|
|
|
|
return;
|
|
|
|
|
}
|
2014-07-21 10:05:56 +00:00
|
|
|
|
|
2015-09-15 01:20:40 +00:00
|
|
|
|
size_t num_found_blocks = 0;
|
|
|
|
|
for (auto found_block : found_blocks)
|
|
|
|
|
num_found_blocks += (found_block == 1);
|
2014-07-21 10:05:56 +00:00
|
|
|
|
|
2015-09-15 01:20:40 +00:00
|
|
|
|
if (num_found_blocks == part_length_in_blocks)
|
|
|
|
|
{
|
|
|
|
|
/// На совокупности живых или мёртвых реплик есть все куски, из которых можно составить нужный кусок. Ничего делать не будем.
|
|
|
|
|
LOG_WARNING(log, "Checker: Found all blocks for missing part. Will wait for them to be merged.");
|
|
|
|
|
return;
|
|
|
|
|
}
|
2015-09-14 22:45:19 +00:00
|
|
|
|
|
2015-09-15 01:20:40 +00:00
|
|
|
|
/// Ни у кого нет такого куска.
|
|
|
|
|
LOG_ERROR(log, "Checker: No replica has part covering " << part_name);
|
2014-07-21 10:05:56 +00:00
|
|
|
|
|
2015-09-15 01:20:40 +00:00
|
|
|
|
if (num_found_blocks != 0)
|
|
|
|
|
LOG_WARNING(log, "When looking for smaller parts, that is covered by " << part_name
|
|
|
|
|
<< ", we found just " << num_found_blocks << " of " << part_length_in_blocks << " blocks.");
|
2014-07-23 11:02:22 +00:00
|
|
|
|
|
2015-09-15 01:20:40 +00:00
|
|
|
|
ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
|
2015-09-14 22:45:19 +00:00
|
|
|
|
|
2015-09-15 01:20:40 +00:00
|
|
|
|
/// Есть ли он в очереди репликации? Если есть - удалим, так как задачу невозможно обработать.
|
|
|
|
|
bool was_in_queue = false;
|
|
|
|
|
{
|
|
|
|
|
std::lock_guard<std::mutex> lock(queue_mutex);
|
2015-09-14 22:45:19 +00:00
|
|
|
|
|
2015-09-15 01:20:40 +00:00
|
|
|
|
for (LogEntries::iterator it = queue.begin(); it != queue.end();)
|
|
|
|
|
{
|
|
|
|
|
if ((*it)->new_part_name == part_name)
|
|
|
|
|
{
|
|
|
|
|
zookeeper->tryRemove(replica_path + "/queue/" + (*it)->znode_name);
|
|
|
|
|
queue.erase(it++);
|
|
|
|
|
was_in_queue = true;
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
++it;
|
|
|
|
|
}
|
|
|
|
|
}
|
2015-09-14 22:45:19 +00:00
|
|
|
|
|
2015-09-15 01:20:40 +00:00
|
|
|
|
if (!was_in_queue)
|
|
|
|
|
{
|
|
|
|
|
/// Куска не было в нашей очереди. С чего бы это?
|
|
|
|
|
LOG_ERROR(log, "Checker: Missing part " << part_name << " is not in our queue.");
|
|
|
|
|
return;
|
|
|
|
|
}
|
2014-07-22 13:49:52 +00:00
|
|
|
|
|
2015-09-15 01:20:40 +00:00
|
|
|
|
/** Такая ситуация возможна, если на всех репликах, где был кусок, он испортился.
|
|
|
|
|
* Например, у реплики, которая только что его записала, отключили питание, и данные не записались из кеша на диск.
|
|
|
|
|
*/
|
|
|
|
|
LOG_ERROR(log, "Checker: Part " << part_name << " is lost forever.");
|
|
|
|
|
ProfileEvents::increment(ProfileEvents::ReplicatedDataLoss);
|
|
|
|
|
|
|
|
|
|
/** Нужно добавить отсутствующий кусок в block_numbers, чтобы он не мешал слияниям.
|
|
|
|
|
* Вот только в сам block_numbers мы его добавить не можем - если так сделать,
|
|
|
|
|
* ZooKeeper зачем-то пропустит один номер для автоинкремента,
|
|
|
|
|
* и в номерах блоков все равно останется дырка.
|
|
|
|
|
* Специально из-за этого приходится отдельно иметь nonincrement_block_numbers.
|
|
|
|
|
*
|
|
|
|
|
* Кстати, если мы здесь сдохнем, то слияния не будут делаться сквозь эти отсутствующие куски.
|
2015-09-23 02:39:07 +00:00
|
|
|
|
*
|
|
|
|
|
* А ещё, не будем добавлять, если:
|
|
|
|
|
* - потребовалось бы создать слишком много (больше 1000) узлов;
|
|
|
|
|
* - кусок является первым в партиции или был при-ATTACH-ен.
|
|
|
|
|
* NOTE Возможно, добавить также условие, если запись в очереди очень старая.
|
2015-09-15 01:20:40 +00:00
|
|
|
|
*/
|
2015-09-23 02:39:07 +00:00
|
|
|
|
|
|
|
|
|
if (part_length_in_blocks > 1000)
|
|
|
|
|
{
|
|
|
|
|
LOG_ERROR(log, "Won't add nonincrement_block_numbers because part spans too much blocks (" << part_length_in_blocks << ")");
|
|
|
|
|
return;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (part_info.left <= RESERVED_BLOCK_NUMBERS)
|
|
|
|
|
{
|
|
|
|
|
LOG_ERROR(log, "Won't add nonincrement_block_numbers because part is one of first in partition");
|
|
|
|
|
return;
|
|
|
|
|
}
|
|
|
|
|
|
2015-09-15 01:20:40 +00:00
|
|
|
|
const auto partition_str = part_name.substr(0, 6);
|
|
|
|
|
for (auto i = part_info.left; i <= part_info.right; ++i)
|
|
|
|
|
{
|
|
|
|
|
zookeeper->createIfNotExists(zookeeper_path + "/nonincrement_block_numbers", "");
|
|
|
|
|
zookeeper->createIfNotExists(zookeeper_path + "/nonincrement_block_numbers/" + partition_str, "");
|
|
|
|
|
AbandonableLockInZooKeeper::createAbandonedIfNotExists(
|
|
|
|
|
zookeeper_path + "/nonincrement_block_numbers/" + partition_str + "/block-" + padIndex(i),
|
|
|
|
|
*zookeeper);
|
|
|
|
|
}
|
|
|
|
|
}
|
2014-07-23 11:02:22 +00:00
|
|
|
|
|
|
|
|
|
|
2015-09-15 01:20:40 +00:00
|
|
|
|
void StorageReplicatedMergeTree::checkPart(const String & part_name)
|
|
|
|
|
{
|
|
|
|
|
LOG_WARNING(log, "Checker: Checking part " << part_name);
|
|
|
|
|
ProfileEvents::increment(ProfileEvents::ReplicatedPartChecks);
|
2014-07-25 16:09:58 +00:00
|
|
|
|
|
2015-09-15 01:20:40 +00:00
|
|
|
|
auto part = data.getActiveContainingPart(part_name);
|
2014-07-23 11:02:22 +00:00
|
|
|
|
|
2015-09-15 01:20:40 +00:00
|
|
|
|
/// Этого или покрывающего куска у нас нет.
|
|
|
|
|
if (!part)
|
|
|
|
|
{
|
|
|
|
|
searchForMissingPart(part_name);
|
|
|
|
|
}
|
|
|
|
|
/// У нас есть этот кусок, и он активен. Будем проверять, нужен ли нам этот кусок и правильные ли у него данные.
|
|
|
|
|
else if (part->name == part_name)
|
|
|
|
|
{
|
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
auto table_lock = lockStructure(false);
|
2014-07-25 16:09:58 +00:00
|
|
|
|
|
2015-09-15 01:20:40 +00:00
|
|
|
|
/// Если кусок есть в ZooKeeper, сверим его данные с его чексуммами, а их с ZooKeeper.
|
|
|
|
|
if (zookeeper->exists(replica_path + "/parts/" + part_name))
|
|
|
|
|
{
|
|
|
|
|
LOG_WARNING(log, "Checker: Checking data of part " << part_name << ".");
|
|
|
|
|
|
|
|
|
|
try
|
|
|
|
|
{
|
|
|
|
|
auto zk_checksums = MergeTreeData::DataPart::Checksums::parse(
|
|
|
|
|
zookeeper->get(replica_path + "/parts/" + part_name + "/checksums"));
|
|
|
|
|
zk_checksums.checkEqual(part->checksums, true);
|
|
|
|
|
|
|
|
|
|
auto zk_columns = NamesAndTypesList::parse(
|
|
|
|
|
zookeeper->get(replica_path + "/parts/" + part_name + "/columns"));
|
|
|
|
|
if (part->columns != zk_columns)
|
|
|
|
|
throw Exception("Columns of local part " + part_name + " are different from ZooKeeper");
|
|
|
|
|
|
|
|
|
|
MergeTreePartChecker::Settings settings;
|
|
|
|
|
settings.setIndexGranularity(data.index_granularity);
|
|
|
|
|
settings.setRequireChecksums(true);
|
|
|
|
|
settings.setRequireColumnFiles(true);
|
|
|
|
|
MergeTreePartChecker::checkDataPart(
|
|
|
|
|
data.getFullPath() + part_name, settings, data.primary_key_sample);
|
|
|
|
|
|
|
|
|
|
LOG_INFO(log, "Checker: Part " << part_name << " looks good.");
|
2014-07-21 10:05:56 +00:00
|
|
|
|
}
|
2015-09-15 01:20:40 +00:00
|
|
|
|
catch (...)
|
2014-07-21 10:05:56 +00:00
|
|
|
|
{
|
2015-09-15 01:20:40 +00:00
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
2014-07-28 14:31:07 +00:00
|
|
|
|
|
2015-09-15 01:20:40 +00:00
|
|
|
|
LOG_ERROR(log, "Checker: Part " << part_name << " looks broken. Removing it and queueing a fetch.");
|
|
|
|
|
ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
|
2014-07-22 13:49:52 +00:00
|
|
|
|
|
2015-09-15 01:20:40 +00:00
|
|
|
|
removePartAndEnqueueFetch(part_name);
|
2014-07-22 13:49:52 +00:00
|
|
|
|
|
2015-09-15 01:20:40 +00:00
|
|
|
|
/// Удалим кусок локально.
|
|
|
|
|
data.renameAndDetachPart(part, "broken_");
|
|
|
|
|
}
|
|
|
|
|
}
|
2015-09-19 05:56:40 +00:00
|
|
|
|
else if (part->modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER < time(0))
|
2015-09-15 01:20:40 +00:00
|
|
|
|
{
|
|
|
|
|
/// Если куска нет в ZooKeeper, удалим его локально.
|
|
|
|
|
/// Возможно, кусок кто-то только что записал, и еще не успел добавить в ZK.
|
|
|
|
|
/// Поэтому удаляем только если кусок старый (не очень надежно).
|
|
|
|
|
ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
|
|
|
|
|
|
2015-09-19 05:56:40 +00:00
|
|
|
|
LOG_ERROR(log, "Checker: Unexpected part " << part_name << " in filesystem. Removing.");
|
2015-09-15 01:20:40 +00:00
|
|
|
|
data.renameAndDetachPart(part, "unexpected_");
|
|
|
|
|
}
|
2015-09-19 05:56:40 +00:00
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
LOG_TRACE(log, "Checker: Young part " << part_name
|
|
|
|
|
<< " with age " << (time(0) - part->modification_time)
|
|
|
|
|
<< " seconds hasn't been added to ZooKeeper yet. It's ok.");
|
|
|
|
|
}
|
2015-09-15 01:20:40 +00:00
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
/// Если у нас есть покрывающий кусок, игнорируем все проблемы с этим куском.
|
|
|
|
|
/// В худшем случае в лог еще old_parts_lifetime секунд будут валиться ошибки, пока кусок не удалится как старый.
|
|
|
|
|
LOG_WARNING(log, "Checker: We have part " << part->name << " covering part " << part_name);
|
|
|
|
|
}
|
|
|
|
|
}
|
2014-07-22 13:49:52 +00:00
|
|
|
|
|
|
|
|
|
|
2015-09-15 01:20:40 +00:00
|
|
|
|
void StorageReplicatedMergeTree::partCheckThread()
|
|
|
|
|
{
|
|
|
|
|
while (!shutdown_called)
|
|
|
|
|
{
|
|
|
|
|
try
|
|
|
|
|
{
|
|
|
|
|
/// Достанем из очереди кусок для проверки.
|
|
|
|
|
String part_name;
|
|
|
|
|
{
|
|
|
|
|
std::lock_guard<std::mutex> lock(parts_to_check_mutex);
|
|
|
|
|
if (parts_to_check_queue.empty())
|
|
|
|
|
{
|
|
|
|
|
if (!parts_to_check_set.empty())
|
|
|
|
|
{
|
|
|
|
|
LOG_ERROR(log, "Checker: Non-empty parts_to_check_set with empty parts_to_check_queue. This is a bug.");
|
|
|
|
|
parts_to_check_set.clear();
|
2014-07-22 13:49:52 +00:00
|
|
|
|
}
|
2014-07-21 10:05:56 +00:00
|
|
|
|
}
|
2015-09-15 01:20:40 +00:00
|
|
|
|
else
|
2014-07-21 10:05:56 +00:00
|
|
|
|
{
|
2015-09-15 01:20:40 +00:00
|
|
|
|
part_name = parts_to_check_queue.front();
|
2014-07-21 10:05:56 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
2015-09-15 01:20:40 +00:00
|
|
|
|
|
|
|
|
|
if (part_name.empty())
|
2014-07-21 10:05:56 +00:00
|
|
|
|
{
|
2015-09-15 01:20:40 +00:00
|
|
|
|
parts_to_check_event.wait();
|
|
|
|
|
continue;
|
2014-07-21 10:05:56 +00:00
|
|
|
|
}
|
|
|
|
|
|
2015-09-15 01:20:40 +00:00
|
|
|
|
checkPart(part_name);
|
|
|
|
|
|
|
|
|
|
/// Удалим кусок из очереди проверок.
|
2014-07-21 10:05:56 +00:00
|
|
|
|
{
|
2014-10-06 05:18:17 +00:00
|
|
|
|
std::lock_guard<std::mutex> lock(parts_to_check_mutex);
|
2014-07-21 10:05:56 +00:00
|
|
|
|
if (parts_to_check_queue.empty() || parts_to_check_queue.front() != part_name)
|
|
|
|
|
{
|
2015-09-15 01:20:40 +00:00
|
|
|
|
LOG_ERROR(log, "Checker: Someone changed parts_to_check_queue.front(). This is a bug.");
|
2014-07-21 10:05:56 +00:00
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
parts_to_check_queue.pop_front();
|
|
|
|
|
parts_to_check_set.erase(part_name);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
|
parts_to_check_event.tryWait(ERROR_SLEEP_MS);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-04-04 10:37:33 +00:00
|
|
|
|
void StorageReplicatedMergeTree::becomeLeader()
|
|
|
|
|
{
|
2014-04-04 12:47:57 +00:00
|
|
|
|
LOG_INFO(log, "Became leader");
|
2014-04-04 10:37:33 +00:00
|
|
|
|
is_leader_node = true;
|
|
|
|
|
merge_selecting_thread = std::thread(&StorageReplicatedMergeTree::mergeSelectingThread, this);
|
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-04-08 17:45:21 +00:00
|
|
|
|
String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_name, bool active)
|
2014-04-03 11:48:28 +00:00
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
2014-05-13 10:10:26 +00:00
|
|
|
|
Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
|
2014-04-03 11:48:28 +00:00
|
|
|
|
|
|
|
|
|
/// Из реплик, у которых есть кусок, выберем одну равновероятно.
|
|
|
|
|
std::random_shuffle(replicas.begin(), replicas.end());
|
|
|
|
|
|
|
|
|
|
for (const String & replica : replicas)
|
|
|
|
|
{
|
2014-05-13 10:10:26 +00:00
|
|
|
|
if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name) &&
|
|
|
|
|
(!active || zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")))
|
2014-04-03 11:48:28 +00:00
|
|
|
|
return replica;
|
2015-09-15 01:20:40 +00:00
|
|
|
|
|
|
|
|
|
/// Конечно, реплика может перестать быть активной или даже перестать существовать после возврата из этой функции.
|
2014-04-03 11:48:28 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-04-08 17:45:21 +00:00
|
|
|
|
return "";
|
2014-04-03 11:48:28 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2015-09-11 02:13:59 +00:00
|
|
|
|
/** Если для куска отслеживается кворум, то обновить информацию о нём в ZK.
|
|
|
|
|
*/
|
2015-09-20 11:02:59 +00:00
|
|
|
|
void StorageReplicatedMergeTree::updateQuorum(const String & part_name)
|
2015-09-11 02:13:59 +00:00
|
|
|
|
{
|
2015-09-20 11:02:59 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
2015-09-11 02:13:59 +00:00
|
|
|
|
|
2015-09-20 11:02:59 +00:00
|
|
|
|
/// Информация, на какие реплики был добавлен кусок, если кворум ещё не достигнут.
|
2015-09-11 02:13:59 +00:00
|
|
|
|
const String quorum_status_path = zookeeper_path + "/quorum/status";
|
2015-09-20 11:02:59 +00:00
|
|
|
|
/// Имя предыдущего куска, для которого был достигнут кворум.
|
|
|
|
|
const String quorum_last_part_path = zookeeper_path + "/quorum/last_part";
|
|
|
|
|
|
2015-09-11 02:13:59 +00:00
|
|
|
|
String value;
|
|
|
|
|
zkutil::Stat stat;
|
|
|
|
|
|
|
|
|
|
/// Если узла нет, значит по всем кворумным INSERT-ам уже был достигнут кворум, и ничего делать не нужно.
|
|
|
|
|
while (zookeeper->tryGet(quorum_status_path, value, &stat))
|
|
|
|
|
{
|
|
|
|
|
ReplicatedMergeTreeQuorumEntry quorum_entry;
|
|
|
|
|
quorum_entry.fromString(value);
|
|
|
|
|
|
|
|
|
|
if (quorum_entry.part_name != part_name)
|
|
|
|
|
{
|
|
|
|
|
/// Кворум уже был достигнут. Более того, уже начался другой INSERT с кворумом.
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
quorum_entry.replicas.insert(replica_name);
|
|
|
|
|
|
|
|
|
|
if (quorum_entry.replicas.size() >= quorum_entry.required_number_of_replicas)
|
|
|
|
|
{
|
2015-09-20 11:02:59 +00:00
|
|
|
|
/// Кворум достигнут. Удаляем узел, а также обновляем информацию о последнем куске, который был успешно записан с кворумом.
|
|
|
|
|
|
|
|
|
|
zkutil::Ops ops;
|
|
|
|
|
ops.push_back(new zkutil::Op::Remove(quorum_status_path, stat.version));
|
|
|
|
|
ops.push_back(new zkutil::Op::SetData(quorum_last_part_path, part_name, -1));
|
|
|
|
|
auto code = zookeeper->tryMulti(ops);
|
2015-09-11 02:13:59 +00:00
|
|
|
|
|
2015-09-11 02:51:35 +00:00
|
|
|
|
if (code == ZOK)
|
|
|
|
|
{
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
else if (code == ZNONODE)
|
2015-09-11 02:13:59 +00:00
|
|
|
|
{
|
|
|
|
|
/// Кворум уже был достигнут.
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
else if (code == ZBADVERSION)
|
|
|
|
|
{
|
|
|
|
|
/// Узел успели обновить. Надо заново его прочитать и повторить все действия.
|
|
|
|
|
continue;
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
throw zkutil::KeeperException(code, quorum_status_path);
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
/// Обновляем узел, прописывая туда на одну реплику больше.
|
|
|
|
|
auto code = zookeeper->trySet(quorum_status_path, quorum_entry.toString(), stat.version);
|
|
|
|
|
|
2015-09-11 02:51:35 +00:00
|
|
|
|
if (code == ZOK)
|
|
|
|
|
{
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
else if (code == ZNONODE)
|
2015-09-11 02:13:59 +00:00
|
|
|
|
{
|
|
|
|
|
/// Кворум уже был достигнут.
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
else if (code == ZBADVERSION)
|
|
|
|
|
{
|
|
|
|
|
/// Узел успели обновить. Надо заново его прочитать и повторить все действия.
|
|
|
|
|
continue;
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
throw zkutil::KeeperException(code, quorum_status_path);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
void StorageReplicatedMergeTree::fetchPart(const String & part_name, const String & replica_path, bool to_detached, size_t quorum)
|
2014-04-03 11:48:28 +00:00
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
|
2014-10-13 17:28:59 +00:00
|
|
|
|
LOG_DEBUG(log, "Fetching part " << part_name << " from " << replica_path);
|
2014-04-04 12:47:57 +00:00
|
|
|
|
|
2014-10-13 17:28:59 +00:00
|
|
|
|
TableStructureReadLockPtr table_lock;
|
|
|
|
|
if (!to_detached)
|
|
|
|
|
table_lock = lockStructure(true);
|
2014-04-03 11:48:28 +00:00
|
|
|
|
|
|
|
|
|
String host;
|
|
|
|
|
int port;
|
|
|
|
|
|
2014-10-13 17:28:59 +00:00
|
|
|
|
String host_port_str = zookeeper->get(replica_path + "/host");
|
2014-04-03 11:48:28 +00:00
|
|
|
|
ReadBufferFromString buf(host_port_str);
|
2015-09-24 04:50:53 +00:00
|
|
|
|
buf >> "host: " >> host >> "\n"
|
2015-09-24 06:20:50 +00:00
|
|
|
|
>> "port: " >> port >> "\n";
|
2014-04-03 11:48:28 +00:00
|
|
|
|
|
2014-10-13 17:28:59 +00:00
|
|
|
|
MergeTreeData::MutableDataPartPtr part = fetcher.fetchPart(part_name, replica_path, host, port, to_detached);
|
2014-07-01 15:58:25 +00:00
|
|
|
|
|
2014-10-13 17:28:59 +00:00
|
|
|
|
if (!to_detached)
|
|
|
|
|
{
|
|
|
|
|
zkutil::Ops ops;
|
|
|
|
|
checkPartAndAddToZooKeeper(part, ops, part_name);
|
2014-08-08 10:41:53 +00:00
|
|
|
|
|
2014-10-13 17:28:59 +00:00
|
|
|
|
MergeTreeData::Transaction transaction;
|
|
|
|
|
auto removed_parts = data.renameTempPartAndReplace(part, nullptr, &transaction);
|
2014-04-07 15:45:46 +00:00
|
|
|
|
|
2014-10-13 17:28:59 +00:00
|
|
|
|
zookeeper->multi(ops);
|
|
|
|
|
transaction.commit();
|
2015-09-11 02:13:59 +00:00
|
|
|
|
|
|
|
|
|
/** Если для этого куска отслеживается кворум, то надо его обновить.
|
2015-09-20 11:02:59 +00:00
|
|
|
|
* Если не успеем, в случае потери сессии, при перезапуске сервера - см. метод ReplicatedMergeTreeRestartingThread::updateQuorumIfWeHavePart.
|
2015-09-11 02:13:59 +00:00
|
|
|
|
*/
|
2015-09-20 11:02:59 +00:00
|
|
|
|
if (quorum)
|
|
|
|
|
updateQuorum(part_name);
|
2015-09-11 02:13:59 +00:00
|
|
|
|
|
2014-10-13 17:28:59 +00:00
|
|
|
|
merge_selecting_event.set();
|
2014-04-09 15:52:47 +00:00
|
|
|
|
|
2014-10-13 17:28:59 +00:00
|
|
|
|
for (const auto & removed_part : removed_parts)
|
|
|
|
|
{
|
|
|
|
|
LOG_DEBUG(log, "Part " << removed_part->name << " is rendered obsolete by fetching part " << part_name);
|
|
|
|
|
ProfileEvents::increment(ProfileEvents::ObsoleteReplicatedParts);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
else
|
2014-04-07 18:14:39 +00:00
|
|
|
|
{
|
2014-10-13 17:28:59 +00:00
|
|
|
|
Poco::File(data.getFullPath() + "detached/tmp_" + part_name).renameTo(data.getFullPath() + "detached/" + part_name);
|
2014-04-07 18:14:39 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-04-07 15:45:46 +00:00
|
|
|
|
ProfileEvents::increment(ProfileEvents::ReplicatedPartFetches);
|
|
|
|
|
|
2015-09-19 08:19:25 +00:00
|
|
|
|
LOG_DEBUG(log, "Fetched part " << part_name << " from " << replica_path << (to_detached ? " (to 'detached' directory)" : ""));
|
2014-04-03 11:48:28 +00:00
|
|
|
|
}
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-03-22 14:44:44 +00:00
|
|
|
|
void StorageReplicatedMergeTree::shutdown()
|
|
|
|
|
{
|
2014-10-17 01:05:51 +00:00
|
|
|
|
if (restarting_thread)
|
2014-05-26 11:40:22 +00:00
|
|
|
|
{
|
2014-10-17 01:05:51 +00:00
|
|
|
|
restarting_thread->stop();
|
|
|
|
|
restarting_thread.reset();
|
2014-05-26 11:40:22 +00:00
|
|
|
|
}
|
2014-07-15 14:37:49 +00:00
|
|
|
|
|
|
|
|
|
endpoint_holder = nullptr;
|
2014-04-25 13:55:15 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-03-22 14:44:44 +00:00
|
|
|
|
StorageReplicatedMergeTree::~StorageReplicatedMergeTree()
|
|
|
|
|
{
|
|
|
|
|
try
|
|
|
|
|
{
|
|
|
|
|
shutdown();
|
|
|
|
|
}
|
|
|
|
|
catch(...)
|
|
|
|
|
{
|
2014-10-17 01:05:51 +00:00
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
2014-03-22 14:44:44 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-03-22 14:44:44 +00:00
|
|
|
|
BlockInputStreams StorageReplicatedMergeTree::read(
|
2014-12-17 11:53:17 +00:00
|
|
|
|
const Names & column_names,
|
|
|
|
|
ASTPtr query,
|
|
|
|
|
const Context & context,
|
|
|
|
|
const Settings & settings,
|
|
|
|
|
QueryProcessingStage::Enum & processed_stage,
|
|
|
|
|
const size_t max_block_size,
|
|
|
|
|
const unsigned threads)
|
2014-03-22 14:44:44 +00:00
|
|
|
|
{
|
2015-09-21 22:33:20 +00:00
|
|
|
|
/** У таблицы может быть два вида данных:
|
|
|
|
|
* - реплицируемые данные;
|
|
|
|
|
* - старые, нереплицируемые данные - они лежат отдельно и их целостность никак не контролируется.
|
|
|
|
|
* А ещё движок таблицы предоставляет возможность использовать "виртуальные столбцы".
|
|
|
|
|
* Один из них - _replicated позволяет определить, из какой части прочитаны данные,
|
|
|
|
|
* или, при использовании в WHERE - выбрать данные только из одной части.
|
|
|
|
|
*/
|
|
|
|
|
|
2014-07-28 09:53:57 +00:00
|
|
|
|
Names virt_column_names;
|
|
|
|
|
Names real_column_names;
|
|
|
|
|
for (const auto & it : column_names)
|
|
|
|
|
if (it == "_replicated")
|
|
|
|
|
virt_column_names.push_back(it);
|
|
|
|
|
else
|
|
|
|
|
real_column_names.push_back(it);
|
|
|
|
|
|
2015-09-21 12:13:05 +00:00
|
|
|
|
auto & select = typeid_cast<const ASTSelectQuery &>(*query);
|
2015-05-20 11:58:21 +00:00
|
|
|
|
|
|
|
|
|
/// Try transferring some condition from WHERE to PREWHERE if enabled and viable
|
|
|
|
|
if (settings.optimize_move_to_prewhere)
|
|
|
|
|
if (select.where_expression && !select.prewhere_expression)
|
2015-09-21 12:13:05 +00:00
|
|
|
|
MergeTreeWhereOptimizer{query, context, data, real_column_names, log};
|
2015-05-20 11:58:21 +00:00
|
|
|
|
|
2014-07-28 09:53:57 +00:00
|
|
|
|
Block virtual_columns_block;
|
|
|
|
|
ColumnUInt8 * column = new ColumnUInt8(2);
|
|
|
|
|
ColumnPtr column_ptr = column;
|
|
|
|
|
column->getData()[0] = 0;
|
|
|
|
|
column->getData()[1] = 1;
|
2015-07-17 01:27:35 +00:00
|
|
|
|
virtual_columns_block.insert(ColumnWithTypeAndName(column_ptr, new DataTypeUInt8, "_replicated"));
|
2014-07-28 09:53:57 +00:00
|
|
|
|
|
2015-09-21 22:33:20 +00:00
|
|
|
|
/// Если запрошен столбец _replicated, пробуем индексировать.
|
2014-07-28 09:53:57 +00:00
|
|
|
|
if (!virt_column_names.empty())
|
2014-10-07 18:42:35 +00:00
|
|
|
|
VirtualColumnUtils::filterBlockWithQuery(query, virtual_columns_block, context);
|
2014-07-28 09:53:57 +00:00
|
|
|
|
|
2014-07-29 14:05:15 +00:00
|
|
|
|
std::multiset<UInt8> values = VirtualColumnUtils::extractSingleValueFromBlock<UInt8>(virtual_columns_block, "_replicated");
|
2014-04-24 10:20:02 +00:00
|
|
|
|
|
2014-07-28 09:53:57 +00:00
|
|
|
|
BlockInputStreams res;
|
|
|
|
|
|
2014-07-30 12:10:34 +00:00
|
|
|
|
size_t part_index = 0;
|
|
|
|
|
|
2015-09-21 22:33:20 +00:00
|
|
|
|
/** Настройки parallel_replica_offset и parallel_replicas_count позволяют читать с одной реплики одну часть данных, а с другой - другую.
|
|
|
|
|
* Для реплицируемых, данные разбиваются таким же механизмом, как работает секция SAMPLE.
|
|
|
|
|
* А для нереплицируемых данных, так как их целостность между репликами не контролируется,
|
|
|
|
|
* с первой (settings.parallel_replica_offset == 0) реплики выбираются все данные, а с остальных - никакие.
|
|
|
|
|
*/
|
|
|
|
|
|
2015-02-08 22:37:55 +00:00
|
|
|
|
if ((settings.parallel_replica_offset == 0) && unreplicated_reader && values.count(0))
|
2014-04-24 10:20:02 +00:00
|
|
|
|
{
|
2014-12-17 11:53:17 +00:00
|
|
|
|
res = unreplicated_reader->read(real_column_names, query,
|
|
|
|
|
context, settings, processed_stage,
|
2015-09-20 11:54:58 +00:00
|
|
|
|
max_block_size, threads, &part_index, 0);
|
2014-07-28 09:53:57 +00:00
|
|
|
|
|
|
|
|
|
for (auto & virtual_column : virt_column_names)
|
|
|
|
|
{
|
|
|
|
|
if (virtual_column == "_replicated")
|
|
|
|
|
{
|
|
|
|
|
for (auto & stream : res)
|
2014-07-30 12:10:34 +00:00
|
|
|
|
stream = new AddingConstColumnBlockInputStream<UInt8>(stream, new DataTypeUInt8, 0, "_replicated");
|
2014-07-28 09:53:57 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2014-07-30 12:10:34 +00:00
|
|
|
|
if (values.count(1))
|
2014-07-28 09:53:57 +00:00
|
|
|
|
{
|
2015-09-20 11:54:58 +00:00
|
|
|
|
/** Настройка select_sequential_consistency имеет два смысла:
|
|
|
|
|
* 1. Кидать исключение, если на реплике есть не все куски, которые были записаны на кворум остальных реплик.
|
|
|
|
|
* 2. Не читать куски, которые ещё не были записаны на кворум реплик.
|
|
|
|
|
* Для этого приходится синхронно сходить в ZooKeeper.
|
|
|
|
|
*/
|
|
|
|
|
Int64 max_block_number_to_read = 0;
|
|
|
|
|
if (settings.select_sequential_consistency)
|
|
|
|
|
{
|
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
String last_part;
|
|
|
|
|
zookeeper->tryGet(zookeeper_path + "/quorum/last_part", last_part);
|
|
|
|
|
|
|
|
|
|
if (!last_part.empty() && !data.getPartIfExists(last_part)) /// TODO Отключение реплики при распределённых запросах.
|
|
|
|
|
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);
|
|
|
|
|
|
|
|
|
|
if (last_part.empty()) /// Если ещё ни один кусок не был записан с кворумом.
|
|
|
|
|
{
|
|
|
|
|
String quorum_str;
|
|
|
|
|
if (zookeeper->tryGet(zookeeper_path + "/quorum/status", quorum_str))
|
|
|
|
|
{
|
|
|
|
|
ReplicatedMergeTreeQuorumEntry quorum_entry;
|
|
|
|
|
quorum_entry.fromString(quorum_str);
|
|
|
|
|
ActiveDataPartSet::Part part_info;
|
|
|
|
|
ActiveDataPartSet::parsePartName(quorum_entry.part_name, part_info);
|
|
|
|
|
max_block_number_to_read = part_info.left - 1;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
ActiveDataPartSet::Part part_info;
|
|
|
|
|
ActiveDataPartSet::parsePartName(last_part, part_info);
|
|
|
|
|
max_block_number_to_read = part_info.right;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
auto res2 = reader.read(
|
|
|
|
|
real_column_names, query, context, settings, processed_stage, max_block_size, threads, &part_index, max_block_number_to_read);
|
2014-07-28 09:53:57 +00:00
|
|
|
|
|
|
|
|
|
for (auto & virtual_column : virt_column_names)
|
|
|
|
|
{
|
|
|
|
|
if (virtual_column == "_replicated")
|
|
|
|
|
{
|
|
|
|
|
for (auto & stream : res2)
|
2014-07-30 12:10:34 +00:00
|
|
|
|
stream = new AddingConstColumnBlockInputStream<UInt8>(stream, new DataTypeUInt8, 1, "_replicated");
|
2014-07-28 09:53:57 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2014-07-30 12:10:34 +00:00
|
|
|
|
res.insert(res.end(), res2.begin(), res2.end());
|
2014-04-24 10:20:02 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
return res;
|
2014-03-22 14:44:44 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2015-09-10 20:43:42 +00:00
|
|
|
|
BlockOutputStreamPtr StorageReplicatedMergeTree::write(ASTPtr query, const Settings & settings)
|
2014-04-02 10:10:37 +00:00
|
|
|
|
{
|
2014-12-11 02:04:13 +00:00
|
|
|
|
if (is_readonly)
|
|
|
|
|
throw Exception("Table is in readonly mode", ErrorCodes::TABLE_IS_READ_ONLY);
|
2014-05-13 11:24:04 +00:00
|
|
|
|
|
2014-04-02 10:10:37 +00:00
|
|
|
|
String insert_id;
|
2014-08-05 08:57:17 +00:00
|
|
|
|
if (query)
|
|
|
|
|
if (ASTInsertQuery * insert = typeid_cast<ASTInsertQuery *>(&*query))
|
|
|
|
|
insert_id = insert->insert_id;
|
2014-04-02 10:10:37 +00:00
|
|
|
|
|
2015-09-10 20:43:42 +00:00
|
|
|
|
return new ReplicatedMergeTreeBlockOutputStream(*this, insert_id, settings.insert_quorum);
|
2014-04-02 10:10:37 +00:00
|
|
|
|
}
|
2014-03-22 14:44:44 +00:00
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2015-04-14 14:58:59 +00:00
|
|
|
|
bool StorageReplicatedMergeTree::optimize(const Settings & settings)
|
2014-05-08 08:03:03 +00:00
|
|
|
|
{
|
2014-07-14 15:49:03 +00:00
|
|
|
|
/// Померджим какие-нибудь куски из директории unreplicated.
|
|
|
|
|
/// TODO: Мерджить реплицируемые куски тоже.
|
2014-05-08 08:03:03 +00:00
|
|
|
|
|
|
|
|
|
if (!unreplicated_data)
|
|
|
|
|
return false;
|
|
|
|
|
|
2014-10-06 05:18:17 +00:00
|
|
|
|
std::lock_guard<std::mutex> lock(unreplicated_mutex);
|
2014-08-04 14:23:47 +00:00
|
|
|
|
|
2014-05-08 08:03:03 +00:00
|
|
|
|
unreplicated_data->clearOldParts();
|
|
|
|
|
|
|
|
|
|
MergeTreeData::DataPartsVector parts;
|
|
|
|
|
String merged_name;
|
2015-04-14 13:44:38 +00:00
|
|
|
|
auto always_can_merge = [](const MergeTreeData::DataPartPtr & a, const MergeTreeData::DataPartPtr & b) { return true; };
|
2014-05-08 08:03:03 +00:00
|
|
|
|
if (!unreplicated_merger->selectPartsToMerge(parts, merged_name, 0, true, true, false, always_can_merge))
|
|
|
|
|
return false;
|
|
|
|
|
|
2014-09-10 11:34:26 +00:00
|
|
|
|
const auto & merge_entry = context.getMergeList().insert(database_name, table_name, merged_name);
|
2015-04-14 14:58:59 +00:00
|
|
|
|
unreplicated_merger->mergeParts(parts, merged_name, *merge_entry, settings.min_bytes_to_use_direct_io);
|
2015-04-14 13:44:38 +00:00
|
|
|
|
|
2014-05-08 08:03:03 +00:00
|
|
|
|
return true;
|
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-08-07 09:23:55 +00:00
|
|
|
|
void StorageReplicatedMergeTree::alter(const AlterCommands & params,
|
|
|
|
|
const String & database_name, const String & table_name, Context & context)
|
2014-07-16 08:58:59 +00:00
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
2014-12-17 18:37:23 +00:00
|
|
|
|
const MergeTreeMergeBlocker merge_blocker{merger};
|
|
|
|
|
const auto unreplicated_merge_blocker = unreplicated_merger ?
|
2015-02-10 21:10:58 +00:00
|
|
|
|
std::make_unique<MergeTreeMergeBlocker>(*unreplicated_merger) : nullptr;
|
2014-12-12 20:50:32 +00:00
|
|
|
|
|
2014-07-16 08:58:59 +00:00
|
|
|
|
LOG_DEBUG(log, "Doing ALTER");
|
|
|
|
|
|
|
|
|
|
NamesAndTypesList new_columns;
|
2014-10-16 13:37:01 +00:00
|
|
|
|
NamesAndTypesList new_materialized_columns;
|
|
|
|
|
NamesAndTypesList new_alias_columns;
|
|
|
|
|
ColumnDefaults new_column_defaults;
|
2014-07-16 08:58:59 +00:00
|
|
|
|
String new_columns_str;
|
|
|
|
|
int new_columns_version;
|
|
|
|
|
zkutil::Stat stat;
|
|
|
|
|
|
|
|
|
|
{
|
|
|
|
|
auto table_lock = lockStructureForAlter();
|
|
|
|
|
|
2014-12-11 02:04:13 +00:00
|
|
|
|
if (is_readonly)
|
|
|
|
|
throw Exception("Can't ALTER readonly table", ErrorCodes::TABLE_IS_READ_ONLY);
|
2014-08-12 09:27:00 +00:00
|
|
|
|
|
2014-07-16 08:58:59 +00:00
|
|
|
|
data.checkAlter(params);
|
|
|
|
|
|
2014-10-16 13:37:01 +00:00
|
|
|
|
new_columns = data.getColumnsListNonMaterialized();
|
|
|
|
|
new_materialized_columns = data.materialized_columns;
|
|
|
|
|
new_alias_columns = data.alias_columns;
|
|
|
|
|
new_column_defaults = data.column_defaults;
|
|
|
|
|
params.apply(new_columns, new_materialized_columns, new_alias_columns, new_column_defaults);
|
2014-07-16 08:58:59 +00:00
|
|
|
|
|
2014-11-10 16:16:43 +00:00
|
|
|
|
new_columns_str = ColumnsDescription<false>{
|
|
|
|
|
new_columns, new_materialized_columns,
|
|
|
|
|
new_alias_columns, new_column_defaults
|
|
|
|
|
}.toString();
|
2014-07-16 08:58:59 +00:00
|
|
|
|
|
|
|
|
|
/// Делаем ALTER.
|
|
|
|
|
zookeeper->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.");
|
|
|
|
|
|
|
|
|
|
/// Ждем, пока все реплики обновят данные.
|
|
|
|
|
|
|
|
|
|
/// Подпишемся на изменения столбцов, чтобы перестать ждать, если кто-то еще сделает ALTER.
|
|
|
|
|
if (!zookeeper->exists(zookeeper_path + "/columns", &stat, alter_query_event))
|
|
|
|
|
throw Exception(zookeeper_path + "/columns doesn't exist", ErrorCodes::NOT_FOUND_NODE);
|
2014-09-26 00:49:50 +00:00
|
|
|
|
|
2014-07-16 08:58:59 +00:00
|
|
|
|
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 = zookeeper->getChildren(zookeeper_path + "/replicas");
|
|
|
|
|
for (const String & replica : replicas)
|
|
|
|
|
{
|
|
|
|
|
LOG_DEBUG(log, "Waiting for " << replica << " to apply changes");
|
|
|
|
|
|
|
|
|
|
while (!shutdown_called)
|
|
|
|
|
{
|
|
|
|
|
String replica_columns_str;
|
|
|
|
|
|
|
|
|
|
/// Реплику могли успеть удалить.
|
|
|
|
|
if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/columns", replica_columns_str, &stat))
|
|
|
|
|
{
|
|
|
|
|
LOG_WARNING(log, replica << " was removed");
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
int replica_columns_version = stat.version;
|
|
|
|
|
|
|
|
|
|
if (replica_columns_str == new_columns_str)
|
|
|
|
|
break;
|
|
|
|
|
|
|
|
|
|
if (!zookeeper->exists(zookeeper_path + "/columns", &stat))
|
|
|
|
|
throw Exception(zookeeper_path + "/columns doesn't exist", ErrorCodes::NOT_FOUND_NODE);
|
2014-09-26 00:49:50 +00:00
|
|
|
|
|
2014-07-16 08:58:59 +00:00
|
|
|
|
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;
|
|
|
|
|
}
|
|
|
|
|
|
2014-07-17 10:44:17 +00:00
|
|
|
|
if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/columns", &stat, alter_query_event))
|
2014-07-16 08:58:59 +00:00
|
|
|
|
{
|
|
|
|
|
LOG_WARNING(log, replica << " was removed");
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (stat.version != replica_columns_version)
|
|
|
|
|
continue;
|
|
|
|
|
|
|
|
|
|
alter_query_event->wait();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (shutdown_called)
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
LOG_DEBUG(log, "ALTER finished");
|
|
|
|
|
}
|
|
|
|
|
|
2014-08-07 09:23:55 +00:00
|
|
|
|
|
|
|
|
|
/// Название воображаемого куска, покрывающего все возможные куски в указанном месяце с номерами в указанном диапазоне.
|
|
|
|
|
static String getFakePartNameForDrop(const String & month_name, UInt64 left, UInt64 right)
|
|
|
|
|
{
|
|
|
|
|
/// Диапазон дат - весь месяц.
|
2015-07-07 23:11:30 +00:00
|
|
|
|
const auto & lut = DateLUT::instance();
|
2015-06-26 15:11:31 +00:00
|
|
|
|
time_t start_time = lut.YYYYMMDDToDate(parse<UInt32>(month_name + "01"));
|
2014-08-07 09:23:55 +00:00
|
|
|
|
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);
|
|
|
|
|
|
|
|
|
|
/// Уровень - right-left+1: кусок не мог образоваться в результате такого или большего количества слияний.
|
|
|
|
|
return ActiveDataPartSet::getPartName(left_date, right_date, left, right, right - left + 1);
|
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2015-06-22 18:23:33 +00:00
|
|
|
|
void StorageReplicatedMergeTree::dropUnreplicatedPartition(const Field & partition, const bool detach, const Settings & settings)
|
2014-08-07 09:23:55 +00:00
|
|
|
|
{
|
2015-04-21 13:10:08 +00:00
|
|
|
|
if (!unreplicated_data)
|
|
|
|
|
return;
|
|
|
|
|
|
|
|
|
|
/// Просит завершить мерджи и не позволяет им начаться.
|
|
|
|
|
/// Это защищает от "оживания" данных за удалённую партицию после завершения мерджа.
|
|
|
|
|
const MergeTreeMergeBlocker merge_blocker{*unreplicated_merger};
|
|
|
|
|
auto structure_lock = lockStructure(true);
|
|
|
|
|
|
|
|
|
|
const DayNum_t month = MergeTreeData::getMonthDayNum(partition);
|
|
|
|
|
|
|
|
|
|
size_t removed_parts = 0;
|
|
|
|
|
MergeTreeData::DataParts parts = unreplicated_data->getDataParts();
|
|
|
|
|
|
|
|
|
|
for (const auto & part : parts)
|
|
|
|
|
{
|
2015-08-17 21:09:36 +00:00
|
|
|
|
if (part->month != month)
|
2015-04-21 13:10:08 +00:00
|
|
|
|
continue;
|
|
|
|
|
|
|
|
|
|
LOG_DEBUG(log, "Removing unreplicated part " << part->name);
|
|
|
|
|
++removed_parts;
|
|
|
|
|
|
2015-06-22 18:23:33 +00:00
|
|
|
|
if (detach)
|
|
|
|
|
unreplicated_data->renameAndDetachPart(part, "");
|
|
|
|
|
else
|
|
|
|
|
unreplicated_data->replaceParts({part}, {}, false);
|
2015-04-21 13:10:08 +00:00
|
|
|
|
}
|
|
|
|
|
|
2015-06-22 18:23:33 +00:00
|
|
|
|
LOG_INFO(log, (detach ? "Detached " : "Removed ") << removed_parts << " unreplicated parts inside " << apply_visitor(FieldVisitorToString(), partition) << ".");
|
2015-04-21 13:10:08 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
void StorageReplicatedMergeTree::dropPartition(const Field & field, bool detach, bool unreplicated, const Settings & settings)
|
|
|
|
|
{
|
|
|
|
|
if (unreplicated)
|
|
|
|
|
{
|
2015-06-22 18:23:33 +00:00
|
|
|
|
dropUnreplicatedPartition(field, detach, settings);
|
2015-04-21 13:10:08 +00:00
|
|
|
|
|
|
|
|
|
return;
|
|
|
|
|
}
|
|
|
|
|
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
2014-10-03 17:57:01 +00:00
|
|
|
|
String month_name = MergeTreeData::getMonthName(field);
|
2014-08-07 09:23:55 +00:00
|
|
|
|
|
|
|
|
|
/// TODO: Делать запрос в лидера по TCP.
|
|
|
|
|
if (!is_leader_node)
|
2015-07-17 18:11:10 +00:00
|
|
|
|
throw Exception(String(detach ? "DETACH" : "DROP") + " PARTITION can only be done on leader replica.", ErrorCodes::NOT_LEADER);
|
2014-08-07 09:23:55 +00:00
|
|
|
|
|
|
|
|
|
/** Пропустим один номер в block_numbers для удаляемого месяца, и будем удалять только куски до этого номера.
|
|
|
|
|
* Это запретит мерджи удаляемых кусков с новыми вставляемыми данными.
|
|
|
|
|
* Инвариант: в логе не появятся слияния удаляемых кусков с другими кусками.
|
|
|
|
|
* NOTE: Если понадобится аналогично поддержать запрос DROP PART, для него придется придумать какой-нибудь новый механизм,
|
|
|
|
|
* чтобы гарантировать этот инвариант.
|
|
|
|
|
*/
|
2015-08-17 21:09:36 +00:00
|
|
|
|
Int64 right;
|
2014-08-07 09:23:55 +00:00
|
|
|
|
|
|
|
|
|
{
|
|
|
|
|
AbandonableLockInZooKeeper block_number_lock = allocateBlockNumber(month_name);
|
|
|
|
|
right = block_number_lock.getNumber();
|
|
|
|
|
block_number_lock.unlock();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Такого никогда не должно происходить.
|
|
|
|
|
if (right == 0)
|
2015-09-24 05:47:17 +00:00
|
|
|
|
throw Exception("Logical error: just allocated block number is zero", ErrorCodes::LOGICAL_ERROR);
|
2014-08-07 09:23:55 +00:00
|
|
|
|
--right;
|
|
|
|
|
|
|
|
|
|
String fake_part_name = getFakePartNameForDrop(month_name, 0, right);
|
|
|
|
|
|
|
|
|
|
/** Запретим выбирать для слияния удаляемые куски - сделаем вид, что их всех уже собираются слить в fake_part_name.
|
|
|
|
|
* Инвариант: после появления в логе записи DROP_RANGE, в логе не появятся слияния удаляемых кусков.
|
|
|
|
|
*/
|
|
|
|
|
{
|
2014-12-02 19:08:18 +00:00
|
|
|
|
std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
|
2014-08-07 09:23:55 +00:00
|
|
|
|
|
|
|
|
|
virtual_parts.add(fake_part_name);
|
|
|
|
|
}
|
|
|
|
|
|
2014-08-07 11:46:01 +00:00
|
|
|
|
/// Наконец, добившись нужных инвариантов, можно положить запись в лог.
|
2014-08-07 09:23:55 +00:00
|
|
|
|
LogEntry entry;
|
|
|
|
|
entry.type = LogEntry::DROP_RANGE;
|
|
|
|
|
entry.source_replica = replica_name;
|
|
|
|
|
entry.new_part_name = fake_part_name;
|
|
|
|
|
entry.detach = detach;
|
|
|
|
|
String log_znode_path = zookeeper->create(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential);
|
2014-08-07 11:46:01 +00:00
|
|
|
|
entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);
|
2014-08-07 09:23:55 +00:00
|
|
|
|
|
2014-10-18 19:14:09 +00:00
|
|
|
|
/// Если надо - дожидаемся выполнения операции на себе или на всех репликах.
|
|
|
|
|
if (settings.replication_alter_partitions_sync != 0)
|
|
|
|
|
{
|
|
|
|
|
if (settings.replication_alter_partitions_sync == 1)
|
|
|
|
|
waitForReplicaToProcessLogEntry(replica_name, entry);
|
|
|
|
|
else
|
|
|
|
|
waitForAllReplicasToProcessLogEntry(entry);
|
|
|
|
|
}
|
2014-08-07 09:23:55 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-10-18 19:14:09 +00:00
|
|
|
|
void StorageReplicatedMergeTree::attachPartition(const Field & field, bool unreplicated, bool attach_part, const Settings & settings)
|
2014-08-07 11:46:01 +00:00
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
2014-10-03 17:57:01 +00:00
|
|
|
|
String partition;
|
2014-08-08 08:28:13 +00:00
|
|
|
|
|
2014-10-03 17:57:01 +00:00
|
|
|
|
if (attach_part)
|
2015-08-17 21:09:36 +00:00
|
|
|
|
partition = field.safeGet<String>();
|
2014-10-03 17:57:01 +00:00
|
|
|
|
else
|
|
|
|
|
partition = MergeTreeData::getMonthName(field);
|
2014-08-08 08:28:13 +00:00
|
|
|
|
|
|
|
|
|
String source_dir = (unreplicated ? "unreplicated/" : "detached/");
|
|
|
|
|
|
|
|
|
|
/// Составим список кусков, которые нужно добавить.
|
|
|
|
|
Strings parts;
|
|
|
|
|
if (attach_part)
|
|
|
|
|
{
|
|
|
|
|
parts.push_back(partition);
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
LOG_DEBUG(log, "Looking for parts for partition " << partition << " in " << source_dir);
|
|
|
|
|
ActiveDataPartSet active_parts;
|
|
|
|
|
for (Poco::DirectoryIterator it = Poco::DirectoryIterator(full_path + source_dir); it != Poco::DirectoryIterator(); ++it)
|
|
|
|
|
{
|
|
|
|
|
String name = it.name();
|
|
|
|
|
if (!ActiveDataPartSet::isPartDirectory(name))
|
|
|
|
|
continue;
|
2014-10-15 19:59:12 +00:00
|
|
|
|
if (0 != name.compare(0, partition.size(), partition))
|
2014-08-08 08:28:13 +00:00
|
|
|
|
continue;
|
|
|
|
|
LOG_DEBUG(log, "Found part " << name);
|
|
|
|
|
active_parts.add(name);
|
|
|
|
|
}
|
|
|
|
|
LOG_DEBUG(log, active_parts.size() << " of them are active");
|
|
|
|
|
parts = active_parts.getParts();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Синхронно проверим, что добавляемые куски существуют и не испорчены хотя бы на этой реплике. Запишем checksums.txt, если его нет.
|
|
|
|
|
LOG_DEBUG(log, "Checking parts");
|
|
|
|
|
for (const String & part : parts)
|
|
|
|
|
{
|
|
|
|
|
LOG_DEBUG(log, "Checking part " << part);
|
|
|
|
|
data.loadPartAndFixMetadata(source_dir + part);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Выделим добавляемым кускам максимальные свободные номера, меньшие RESERVED_BLOCK_NUMBERS.
|
|
|
|
|
/// NOTE: Проверка свободности номеров никак не синхронизируется. Выполнять несколько запросов ATTACH/DETACH/DROP одновременно нельзя.
|
2015-08-17 21:09:36 +00:00
|
|
|
|
Int64 min_used_number = RESERVED_BLOCK_NUMBERS;
|
|
|
|
|
DayNum_t month = DateLUT::instance().makeDayNum(parse<UInt16>(partition.substr(0, 4)), parse<UInt8>(partition.substr(4, 2)), 0);
|
2014-08-08 08:28:13 +00:00
|
|
|
|
|
|
|
|
|
{
|
|
|
|
|
auto existing_parts = data.getDataParts();
|
|
|
|
|
for (const auto & part : existing_parts)
|
2015-08-17 21:09:36 +00:00
|
|
|
|
if (part->month == month)
|
|
|
|
|
min_used_number = std::min(min_used_number, part->left);
|
2014-08-08 08:28:13 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Добавим записи в лог.
|
|
|
|
|
std::reverse(parts.begin(), parts.end());
|
|
|
|
|
std::list<LogEntry> entries;
|
|
|
|
|
zkutil::Ops ops;
|
|
|
|
|
for (const String & part_name : parts)
|
|
|
|
|
{
|
|
|
|
|
ActiveDataPartSet::Part part;
|
|
|
|
|
ActiveDataPartSet::parsePartName(part_name, part);
|
|
|
|
|
part.left = part.right = --min_used_number;
|
|
|
|
|
String new_part_name = ActiveDataPartSet::getPartName(part.left_date, part.right_date, part.left, part.right, part.level);
|
|
|
|
|
|
|
|
|
|
LOG_INFO(log, "Will attach " << part_name << " as " << new_part_name);
|
|
|
|
|
|
|
|
|
|
entries.emplace_back();
|
|
|
|
|
LogEntry & entry = entries.back();
|
|
|
|
|
entry.type = LogEntry::ATTACH_PART;
|
|
|
|
|
entry.source_replica = replica_name;
|
|
|
|
|
entry.source_part_name = part_name;
|
|
|
|
|
entry.new_part_name = new_part_name;
|
|
|
|
|
entry.attach_unreplicated = unreplicated;
|
|
|
|
|
ops.push_back(new zkutil::Op::Create(
|
|
|
|
|
zookeeper_path + "/log/log-", entry.toString(), zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
LOG_DEBUG(log, "Adding attaches to log");
|
|
|
|
|
zookeeper->multi(ops);
|
2014-10-15 20:28:43 +00:00
|
|
|
|
|
2014-10-18 19:14:09 +00:00
|
|
|
|
/// Если надо - дожидаемся выполнения операции на себе или на всех репликах.
|
|
|
|
|
if (settings.replication_alter_partitions_sync != 0)
|
2014-08-08 08:28:13 +00:00
|
|
|
|
{
|
2014-10-18 19:14:09 +00:00
|
|
|
|
size_t i = 0;
|
|
|
|
|
for (LogEntry & entry : entries)
|
|
|
|
|
{
|
|
|
|
|
String log_znode_path = dynamic_cast<zkutil::Op::Create &>(ops[i]).getPathCreated();
|
|
|
|
|
entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);
|
|
|
|
|
|
|
|
|
|
if (settings.replication_alter_partitions_sync == 1)
|
|
|
|
|
waitForReplicaToProcessLogEntry(replica_name, entry);
|
|
|
|
|
else
|
|
|
|
|
waitForAllReplicasToProcessLogEntry(entry);
|
2014-08-08 08:28:13 +00:00
|
|
|
|
|
2014-10-18 19:14:09 +00:00
|
|
|
|
++i;
|
|
|
|
|
}
|
2014-08-08 08:28:13 +00:00
|
|
|
|
}
|
2014-08-07 11:46:01 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-03-22 14:44:44 +00:00
|
|
|
|
void StorageReplicatedMergeTree::drop()
|
|
|
|
|
{
|
2014-12-11 02:04:13 +00:00
|
|
|
|
if (is_readonly)
|
|
|
|
|
throw Exception("Can't drop readonly replicated table (need to drop data in ZooKeeper as well)", ErrorCodes::TABLE_IS_READ_ONLY);
|
2014-07-23 13:58:38 +00:00
|
|
|
|
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
|
2014-04-03 11:48:28 +00:00
|
|
|
|
shutdown();
|
|
|
|
|
|
2014-05-27 12:08:40 +00:00
|
|
|
|
LOG_INFO(log, "Removing replica " << replica_path);
|
2014-03-22 14:44:44 +00:00
|
|
|
|
replica_is_active_node = nullptr;
|
2014-07-07 09:51:42 +00:00
|
|
|
|
zookeeper->tryRemoveRecursive(replica_path);
|
|
|
|
|
|
2014-07-07 11:39:06 +00:00
|
|
|
|
/// Проверяем, что zookeeper_path существует: его могла удалить другая реплика после выполнения предыдущей строки.
|
2014-07-07 09:51:42 +00:00
|
|
|
|
Strings replicas;
|
|
|
|
|
if (zookeeper->tryGetChildren(zookeeper_path + "/replicas", replicas) == ZOK && replicas.empty())
|
2014-05-27 12:08:40 +00:00
|
|
|
|
{
|
|
|
|
|
LOG_INFO(log, "Removing table " << zookeeper_path << " (this might take several minutes)");
|
2014-07-07 09:51:42 +00:00
|
|
|
|
zookeeper->tryRemoveRecursive(zookeeper_path);
|
2014-05-27 12:08:40 +00:00
|
|
|
|
}
|
2014-07-02 12:30:38 +00:00
|
|
|
|
|
|
|
|
|
data.dropAllData();
|
2014-03-22 14:44:44 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-07-28 14:33:30 +00:00
|
|
|
|
void StorageReplicatedMergeTree::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name)
|
|
|
|
|
{
|
|
|
|
|
std::string new_full_path = new_path_to_db + escapeForFileName(new_table_name) + '/';
|
|
|
|
|
|
|
|
|
|
data.setPath(new_full_path, true);
|
|
|
|
|
if (unreplicated_data)
|
|
|
|
|
unreplicated_data->setPath(new_full_path + "unreplicated/", false);
|
|
|
|
|
|
|
|
|
|
database_name = new_database_name;
|
|
|
|
|
table_name = new_table_name;
|
|
|
|
|
full_path = new_full_path;
|
|
|
|
|
|
|
|
|
|
/// TODO: Можно обновить названия логгеров.
|
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-08-07 09:23:55 +00:00
|
|
|
|
AbandonableLockInZooKeeper StorageReplicatedMergeTree::allocateBlockNumber(const String & month_name)
|
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
|
2014-08-07 09:23:55 +00:00
|
|
|
|
String month_path = zookeeper_path + "/block_numbers/" + month_name;
|
|
|
|
|
if (!zookeeper->exists(month_path))
|
|
|
|
|
{
|
|
|
|
|
/// Создадим в block_numbers ноду для месяца и пропустим в ней 200 значений инкремента.
|
|
|
|
|
/// Нужно, чтобы в будущем при необходимости можно было добавить данные в начало.
|
|
|
|
|
zkutil::Ops ops;
|
|
|
|
|
auto acl = zookeeper->getDefaultACL();
|
|
|
|
|
ops.push_back(new zkutil::Op::Create(month_path, "", acl, zkutil::CreateMode::Persistent));
|
2014-08-08 08:28:13 +00:00
|
|
|
|
for (size_t i = 0; i < RESERVED_BLOCK_NUMBERS; ++i)
|
2014-08-07 09:23:55 +00:00
|
|
|
|
{
|
|
|
|
|
ops.push_back(new zkutil::Op::Create(month_path + "/skip_increment", "", acl, zkutil::CreateMode::Persistent));
|
|
|
|
|
ops.push_back(new zkutil::Op::Remove(month_path + "/skip_increment", -1));
|
|
|
|
|
}
|
|
|
|
|
/// Игнорируем ошибки - не получиться могло только если кто-то еще выполнил эту строчку раньше нас.
|
|
|
|
|
zookeeper->tryMulti(ops);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
return AbandonableLockInZooKeeper(
|
|
|
|
|
zookeeper_path + "/block_numbers/" + month_name + "/block-",
|
|
|
|
|
zookeeper_path + "/temp", *zookeeper);
|
|
|
|
|
}
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
|
2014-08-08 08:28:13 +00:00
|
|
|
|
void StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry(const LogEntry & entry)
|
2014-08-07 09:23:55 +00:00
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
2014-08-07 11:46:01 +00:00
|
|
|
|
LOG_DEBUG(log, "Waiting for all replicas to process " << entry.znode_name);
|
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
|
|
|
|
|
for (const String & replica : replicas)
|
|
|
|
|
waitForReplicaToProcessLogEntry(replica, entry);
|
|
|
|
|
|
|
|
|
|
LOG_DEBUG(log, "Finished waiting for all replicas to process " << entry.znode_name);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
void StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry(const String & replica, const LogEntry & entry)
|
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
|
2015-06-04 02:07:30 +00:00
|
|
|
|
String entry_str = entry.toString();
|
|
|
|
|
String log_node_name;
|
2014-08-07 09:23:55 +00:00
|
|
|
|
|
2015-06-04 02:07:30 +00:00
|
|
|
|
/** В эту функцию могут передать entry двух видов:
|
|
|
|
|
* 1. (более часто) Из директории log - общего лога, откуда реплики копируют записи в свою queue.
|
|
|
|
|
* 2. Из директории queue одной из реплик.
|
|
|
|
|
*
|
|
|
|
|
* Проблема в том, что номера (sequential нод) элементов очереди в log и в queue не совпадают.
|
|
|
|
|
* (И в queue не совпадают номера у одного и того же элемента лога для разных реплик.)
|
|
|
|
|
*
|
|
|
|
|
* Поэтому следует рассматривать эти случаи по-отдельности.
|
|
|
|
|
*/
|
2014-08-07 11:46:01 +00:00
|
|
|
|
|
2015-06-04 02:07:30 +00:00
|
|
|
|
/** Первое - нужно дождаться, пока реплика возьмёт к себе в queue элемент очереди из log,
|
|
|
|
|
* если она ещё этого не сделала (см. функцию pullLogsToQueue).
|
|
|
|
|
*
|
|
|
|
|
* Для этого проверяем её узел log_pointer - максимальный номер взятого элемента из log плюс единица.
|
|
|
|
|
*/
|
|
|
|
|
|
|
|
|
|
if (0 == entry.znode_name.compare(0, strlen("log-"), "log-"))
|
2014-10-18 17:37:55 +00:00
|
|
|
|
{
|
2015-06-04 02:07:30 +00:00
|
|
|
|
/** В этом случае просто берём номер из имени ноды log-xxxxxxxxxx.
|
|
|
|
|
*/
|
2014-08-07 09:23:55 +00:00
|
|
|
|
|
2015-06-04 02:07:30 +00:00
|
|
|
|
UInt64 log_index = parse<UInt64>(entry.znode_name.substr(entry.znode_name.size() - 10));
|
|
|
|
|
log_node_name = entry.znode_name;
|
2014-08-07 09:23:55 +00:00
|
|
|
|
|
2015-06-04 02:07:30 +00:00
|
|
|
|
LOG_DEBUG(log, "Waiting for " << replica << " to pull " << log_node_name << " to queue");
|
|
|
|
|
|
|
|
|
|
/// Дождемся, пока запись попадет в очередь реплики.
|
|
|
|
|
while (true)
|
|
|
|
|
{
|
|
|
|
|
zkutil::EventPtr event = new Poco::Event;
|
|
|
|
|
|
|
|
|
|
String log_pointer = zookeeper->get(zookeeper_path + "/replicas/" + replica + "/log_pointer", nullptr, event);
|
|
|
|
|
if (!log_pointer.empty() && parse<UInt64>(log_pointer) > log_index)
|
|
|
|
|
break;
|
|
|
|
|
|
|
|
|
|
event->wait();
|
|
|
|
|
}
|
2014-10-18 17:37:55 +00:00
|
|
|
|
}
|
2015-06-04 02:07:30 +00:00
|
|
|
|
else if (0 == entry.znode_name.compare(0, strlen("queue-"), "queue-"))
|
|
|
|
|
{
|
|
|
|
|
/** В этом случае номер log-ноды неизвестен. Нужно просмотреть все от log_pointer до конца,
|
|
|
|
|
* ища ноду с таким же содержимым. И если мы её не найдём - значит реплика уже взяла эту запись в свою queue.
|
|
|
|
|
*/
|
|
|
|
|
|
|
|
|
|
String log_pointer = zookeeper->get(zookeeper_path + "/replicas/" + replica + "/log_pointer");
|
2014-08-07 09:23:55 +00:00
|
|
|
|
|
2015-06-04 02:07:30 +00:00
|
|
|
|
Strings log_entries = zookeeper->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 = zookeeper->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");
|
|
|
|
|
|
|
|
|
|
/// Дождемся, пока запись попадет в очередь реплики.
|
|
|
|
|
while (true)
|
|
|
|
|
{
|
|
|
|
|
zkutil::EventPtr event = new Poco::Event;
|
|
|
|
|
|
|
|
|
|
String log_pointer = zookeeper->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");
|
|
|
|
|
|
|
|
|
|
/** Второе - найдем соответствующую запись в очереди указанной реплики (replica).
|
|
|
|
|
* Её номер может не совпадать ни с log-узлом, ни с queue-узлом у текущей реплики (у нас).
|
|
|
|
|
* Поэтому, ищем путём сравнения содержимого.
|
|
|
|
|
*/
|
2014-08-07 11:46:01 +00:00
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
Strings queue_entries = zookeeper->getChildren(zookeeper_path + "/replicas/" + replica + "/queue");
|
2015-06-04 02:07:30 +00:00
|
|
|
|
String queue_entry_to_wait_for;
|
2014-08-07 09:23:55 +00:00
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
for (const String & entry_name : queue_entries)
|
|
|
|
|
{
|
|
|
|
|
String queue_entry_str;
|
|
|
|
|
bool exists = zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/queue/" + entry_name, queue_entry_str);
|
2015-06-04 02:07:30 +00:00
|
|
|
|
if (exists && queue_entry_str == entry_str)
|
2014-08-07 09:23:55 +00:00
|
|
|
|
{
|
2015-06-04 02:07:30 +00:00
|
|
|
|
queue_entry_to_wait_for = entry_name;
|
2014-10-18 17:37:55 +00:00
|
|
|
|
break;
|
2014-08-07 09:23:55 +00:00
|
|
|
|
}
|
2014-10-18 17:37:55 +00:00
|
|
|
|
}
|
2014-08-07 09:23:55 +00:00
|
|
|
|
|
2014-10-18 17:37:55 +00:00
|
|
|
|
/// Пока искали запись, ее уже выполнили и удалили.
|
2015-06-04 02:07:30 +00:00
|
|
|
|
if (queue_entry_to_wait_for.empty())
|
|
|
|
|
{
|
|
|
|
|
LOG_DEBUG(log, "No corresponding node found. Assuming it has been already processed.");
|
2014-10-18 17:37:55 +00:00
|
|
|
|
return;
|
2015-06-04 02:07:30 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
LOG_DEBUG(log, "Waiting for " << queue_entry_to_wait_for << " to disappear from " << replica << " queue");
|
2014-08-07 09:23:55 +00:00
|
|
|
|
|
2015-06-04 02:07:30 +00:00
|
|
|
|
/// Третье - дождемся, пока запись исчезнет из очереди реплики.
|
2015-09-11 02:13:59 +00:00
|
|
|
|
zookeeper->waitForDisappear(zookeeper_path + "/replicas/" + replica + "/queue/" + queue_entry_to_wait_for);
|
2014-08-07 09:23:55 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-10-07 18:44:03 +00:00
|
|
|
|
void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields)
|
2014-10-06 05:18:17 +00:00
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
|
2014-10-06 05:18:17 +00:00
|
|
|
|
res.is_leader = is_leader_node;
|
2014-12-11 02:04:13 +00:00
|
|
|
|
res.is_readonly = is_readonly;
|
2014-11-07 01:12:55 +00:00
|
|
|
|
res.is_session_expired = !zookeeper || zookeeper->expired();
|
2014-10-06 05:18:17 +00:00
|
|
|
|
|
|
|
|
|
{
|
|
|
|
|
std::lock_guard<std::mutex> lock(queue_mutex);
|
|
|
|
|
res.future_parts = future_parts.size();
|
|
|
|
|
res.queue_size = queue.size();
|
2015-09-20 05:21:43 +00:00
|
|
|
|
res.last_queue_update = last_queue_update;
|
2014-10-06 05:18:17 +00:00
|
|
|
|
|
|
|
|
|
res.inserts_in_queue = 0;
|
|
|
|
|
res.merges_in_queue = 0;
|
2014-11-08 23:55:24 +00:00
|
|
|
|
res.queue_oldest_time = 0;
|
2015-07-16 19:11:59 +00:00
|
|
|
|
res.inserts_oldest_time = 0;
|
|
|
|
|
res.merges_oldest_time = 0;
|
2014-10-06 05:18:17 +00:00
|
|
|
|
|
|
|
|
|
for (const LogEntryPtr & entry : queue)
|
|
|
|
|
{
|
2015-07-16 19:11:59 +00:00
|
|
|
|
if (entry->create_time && (!res.queue_oldest_time || entry->create_time < res.queue_oldest_time))
|
|
|
|
|
res.queue_oldest_time = entry->create_time;
|
|
|
|
|
|
2014-10-06 05:18:17 +00:00
|
|
|
|
if (entry->type == LogEntry::GET_PART)
|
2015-07-16 19:11:59 +00:00
|
|
|
|
{
|
2014-10-06 05:18:17 +00:00
|
|
|
|
++res.inserts_in_queue;
|
2015-07-16 19:11:59 +00:00
|
|
|
|
|
|
|
|
|
if (entry->create_time && (!res.inserts_oldest_time || entry->create_time < res.inserts_oldest_time))
|
2015-09-19 05:56:40 +00:00
|
|
|
|
{
|
2015-07-16 19:11:59 +00:00
|
|
|
|
res.inserts_oldest_time = entry->create_time;
|
2015-09-19 05:56:40 +00:00
|
|
|
|
res.oldest_part_to_get = entry->new_part_name;
|
|
|
|
|
}
|
2015-07-16 19:11:59 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-10-06 05:18:17 +00:00
|
|
|
|
if (entry->type == LogEntry::MERGE_PARTS)
|
2015-07-16 19:11:59 +00:00
|
|
|
|
{
|
2014-10-06 05:18:17 +00:00
|
|
|
|
++res.merges_in_queue;
|
2014-11-08 23:55:24 +00:00
|
|
|
|
|
2015-07-16 19:11:59 +00:00
|
|
|
|
if (entry->create_time && (!res.merges_oldest_time || entry->create_time < res.merges_oldest_time))
|
2015-09-19 05:56:40 +00:00
|
|
|
|
{
|
2015-07-16 19:11:59 +00:00
|
|
|
|
res.merges_oldest_time = entry->create_time;
|
2015-09-19 05:56:40 +00:00
|
|
|
|
res.oldest_part_to_merge_to = entry->new_part_name;
|
|
|
|
|
}
|
2015-07-16 19:11:59 +00:00
|
|
|
|
}
|
2014-10-06 05:18:17 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
{
|
|
|
|
|
std::lock_guard<std::mutex> lock(parts_to_check_mutex);
|
|
|
|
|
res.parts_to_check = parts_to_check_set.size();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
res.zookeeper_path = zookeeper_path;
|
|
|
|
|
res.replica_name = replica_name;
|
|
|
|
|
res.replica_path = replica_path;
|
|
|
|
|
res.columns_version = columns_version;
|
|
|
|
|
|
2014-10-07 18:44:03 +00:00
|
|
|
|
if (res.is_session_expired || !with_zk_fields)
|
2014-10-06 05:18:17 +00:00
|
|
|
|
{
|
|
|
|
|
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");
|
|
|
|
|
|
2014-10-07 18:44:03 +00:00
|
|
|
|
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);
|
2014-10-06 05:18:17 +00:00
|
|
|
|
|
|
|
|
|
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;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2014-10-09 20:28:33 +00:00
|
|
|
|
|
2015-09-24 00:21:02 +00:00
|
|
|
|
void StorageReplicatedMergeTree::getQueue(LogEntriesData & res, String & replica_name_)
|
|
|
|
|
{
|
|
|
|
|
res.clear();
|
|
|
|
|
replica_name_ = replica_name;
|
|
|
|
|
|
|
|
|
|
std::lock_guard<std::mutex> lock(queue_mutex);
|
|
|
|
|
res.reserve(queue.size());
|
|
|
|
|
for (const auto & entry : queue)
|
|
|
|
|
res.emplace_back(*entry);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-10-18 19:14:09 +00:00
|
|
|
|
void StorageReplicatedMergeTree::fetchPartition(const Field & partition, const String & from_, const Settings & settings)
|
2014-10-09 20:28:33 +00:00
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = getZooKeeper();
|
|
|
|
|
|
2014-10-10 00:46:37 +00:00
|
|
|
|
String partition_str = MergeTreeData::getMonthName(partition);
|
|
|
|
|
|
2014-10-09 20:28:33 +00:00
|
|
|
|
String from = from_;
|
2014-10-09 23:14:06 +00:00
|
|
|
|
if (from.back() == '/')
|
|
|
|
|
from.resize(from.size() - 1);
|
2014-10-09 20:28:33 +00:00
|
|
|
|
|
2014-10-10 00:46:37 +00:00
|
|
|
|
LOG_INFO(log, "Will fetch partition " << partition_str << " from shard " << from_);
|
|
|
|
|
|
2014-10-13 17:28:59 +00:00
|
|
|
|
/** Проверим, что в директории detached (куда мы будем записывать скаченные куски) ещё нет такой партиции.
|
|
|
|
|
* Ненадёжно (есть race condition) - такая партиция может появиться чуть позже.
|
|
|
|
|
*/
|
|
|
|
|
Poco::DirectoryIterator dir_end;
|
|
|
|
|
for (Poco::DirectoryIterator dir_it{data.getFullPath() + "detached/"}; dir_it != dir_end; ++dir_it)
|
|
|
|
|
if (0 == dir_it.name().compare(0, partition_str.size(), partition_str))
|
|
|
|
|
throw Exception("Detached partition " + partition_str + " is already exists.", ErrorCodes::PARTITION_ALREADY_EXISTS);
|
|
|
|
|
|
2014-10-09 20:28:33 +00:00
|
|
|
|
/// Список реплик шарда-источника.
|
|
|
|
|
zkutil::Strings replicas = zookeeper->getChildren(from + "/replicas");
|
|
|
|
|
|
|
|
|
|
/// Оставим только активные реплики.
|
|
|
|
|
zkutil::Strings 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())
|
2014-10-13 17:28:59 +00:00
|
|
|
|
throw Exception("No active replicas for shard " + from, ErrorCodes::NO_ACTIVE_REPLICAS);
|
2014-10-09 20:28:33 +00:00
|
|
|
|
|
|
|
|
|
/** Надо выбрать лучшую (наиболее актуальную) реплику.
|
2014-10-10 00:46:37 +00:00
|
|
|
|
* Это реплика с максимальным log_pointer, затем с минимальным размером queue.
|
2014-10-09 20:28:33 +00:00
|
|
|
|
* NOTE Это не совсем лучший критерий. Для скачивания старых партиций это не имеет смысла,
|
|
|
|
|
* и было бы неплохо уметь выбирать реплику, ближайшую по сети.
|
2014-10-10 00:46:37 +00:00
|
|
|
|
* NOTE Разумеется, здесь есть data race-ы. Можно решить ретраями.
|
2014-10-09 20:28:33 +00:00
|
|
|
|
*/
|
2014-10-10 00:46:37 +00:00
|
|
|
|
Int64 max_log_pointer = -1;
|
|
|
|
|
UInt64 min_queue_size = std::numeric_limits<UInt64>::max();
|
|
|
|
|
String best_replica;
|
|
|
|
|
|
|
|
|
|
for (const String & replica : active_replicas)
|
|
|
|
|
{
|
2014-10-13 17:28:59 +00:00
|
|
|
|
String current_replica_path = from + "/replicas/" + replica;
|
2014-10-10 00:46:37 +00:00
|
|
|
|
|
2014-10-13 17:28:59 +00:00
|
|
|
|
String log_pointer_str = zookeeper->get(current_replica_path + "/log_pointer");
|
2014-10-10 00:46:37 +00:00
|
|
|
|
Int64 log_pointer = log_pointer_str.empty() ? 0 : parse<UInt64>(log_pointer_str);
|
|
|
|
|
|
|
|
|
|
zkutil::Stat stat;
|
2014-10-13 17:28:59 +00:00
|
|
|
|
zookeeper->get(current_replica_path + "/queue", &stat);
|
2014-10-10 00:46:37 +00:00
|
|
|
|
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.");
|
|
|
|
|
|
2014-10-13 17:28:59 +00:00
|
|
|
|
String best_replica_path = from + "/replicas/" + best_replica;
|
|
|
|
|
|
2014-10-10 00:46:37 +00:00
|
|
|
|
/// Выясним, какие куски есть на лучшей реплике.
|
2014-10-09 20:28:33 +00:00
|
|
|
|
|
2014-10-13 17:28:59 +00:00
|
|
|
|
/** Пытаемся скачать эти куски.
|
|
|
|
|
* Часть из них могла удалиться из-за мерджа.
|
|
|
|
|
* В этом случае, обновляем информацию о доступных кусках и пробуем снова.
|
|
|
|
|
*/
|
|
|
|
|
|
|
|
|
|
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)
|
|
|
|
|
throw Exception("Too much retries to fetch parts from " + best_replica_path, ErrorCodes::TOO_MUCH_RETRIES_TO_FETCH_PARTS);
|
|
|
|
|
|
|
|
|
|
Strings parts = zookeeper->getChildren(best_replica_path + "/parts");
|
|
|
|
|
ActiveDataPartSet active_parts_set(parts);
|
|
|
|
|
Strings parts_to_fetch;
|
|
|
|
|
|
|
|
|
|
if (missing_parts.empty())
|
|
|
|
|
{
|
|
|
|
|
parts_to_fetch = active_parts_set.getParts();
|
|
|
|
|
|
|
|
|
|
/// Оставляем только куски нужной партиции.
|
|
|
|
|
Strings parts_to_fetch_partition;
|
|
|
|
|
for (const String & part : parts_to_fetch)
|
|
|
|
|
if (0 == part.compare(0, partition_str.size(), partition_str))
|
|
|
|
|
parts_to_fetch_partition.push_back(part);
|
|
|
|
|
|
|
|
|
|
parts_to_fetch = std::move(parts_to_fetch_partition);
|
2014-10-13 18:01:58 +00:00
|
|
|
|
|
|
|
|
|
if (parts_to_fetch.empty())
|
|
|
|
|
throw Exception("Partition " + partition_str + " on " + best_replica_path + " doesn't exist", ErrorCodes::PARTITION_DOESNT_EXIST);
|
2014-10-13 17:28:59 +00:00
|
|
|
|
}
|
|
|
|
|
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
|
|
|
|
|
{
|
2015-09-11 02:13:59 +00:00
|
|
|
|
fetchPart(part, best_replica_path, true, 0);
|
2014-10-13 17:28:59 +00:00
|
|
|
|
}
|
|
|
|
|
catch (const DB::Exception & e)
|
|
|
|
|
{
|
|
|
|
|
if (e.code() != ErrorCodes::RECEIVED_ERROR_FROM_REMOTE_IO_SERVER)
|
|
|
|
|
throw;
|
|
|
|
|
|
|
|
|
|
LOG_INFO(log, e.displayText());
|
|
|
|
|
missing_parts.push_back(part);
|
|
|
|
|
}
|
|
|
|
|
}
|
2014-10-09 20:28:33 +00:00
|
|
|
|
|
2014-10-13 17:28:59 +00:00
|
|
|
|
++try_no;
|
|
|
|
|
} while (!missing_parts.empty());
|
2014-10-09 20:28:33 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-11-11 04:11:07 +00:00
|
|
|
|
void StorageReplicatedMergeTree::freezePartition(const Field & partition, const Settings & settings)
|
|
|
|
|
{
|
|
|
|
|
/// Префикс может быть произвольным. Не обязательно месяц - можно указать лишь год.
|
|
|
|
|
String prefix = partition.getType() == Field::Types::UInt64
|
|
|
|
|
? toString(partition.get<UInt64>())
|
|
|
|
|
: partition.safeGet<String>();
|
|
|
|
|
|
|
|
|
|
data.freezePartition(prefix);
|
|
|
|
|
if (unreplicated_data)
|
|
|
|
|
unreplicated_data->freezePartition(prefix);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-03-21 13:42:14 +00:00
|
|
|
|
}
|