StorageReplicatedMergeTree.cpp 130.2 KB
Newer Older
A
Merge  
Alexey Milovidov 已提交
1 2 3
#include <time.h>
#include <ext/range.hpp>

4 5 6
#include <zkutil/Types.h>
#include <zkutil/KeeperException.h>

7
#include <DB/Core/FieldVisitors.h>
8

9
#include <DB/Storages/ColumnsDescription.h>
M
Merge  
Michael Kolupaev 已提交
10
#include <DB/Storages/StorageReplicatedMergeTree.h>
M
Merge  
Michael Kolupaev 已提交
11
#include <DB/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h>
A
Merge  
Alexey Milovidov 已提交
12
#include <DB/Storages/MergeTree/ReplicatedMergeTreePartsExchange.h>
13
#include <DB/Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h>
M
Merge  
Michael Kolupaev 已提交
14
#include <DB/Storages/MergeTree/MergeTreePartChecker.h>
15
#include <DB/Storages/MergeTree/MergeList.h>
A
Merge  
Andrey Mironov 已提交
16
#include <DB/Storages/MergeTree/MergeTreeWhereOptimizer.h>
17
#include <DB/Storages/MergeTree/ReplicatedMergeTreeAddress.h>
18

M
Merge  
Michael Kolupaev 已提交
19
#include <DB/Parsers/formatAST.h>
20 21
#include <DB/Parsers/ASTInsertQuery.h>

M
Merge  
Michael Kolupaev 已提交
22 23
#include <DB/IO/WriteBufferFromOStream.h>
#include <DB/IO/ReadBufferFromString.h>
24
#include <DB/IO/Operators.h>
25

M
Merge  
Michael Kolupaev 已提交
26
#include <DB/Interpreters/InterpreterAlterQuery.h>
27

28
#include <DB/DataStreams/AddingConstColumnBlockInputStream.h>
29 30 31
#include <DB/DataStreams/RemoteBlockInputStream.h>
#include <DB/DataStreams/NullBlockOutputStream.h>
#include <DB/DataStreams/copyData.h>
32

33
#include <DB/Common/Macros.h>
34
#include <DB/Common/VirtualColumnUtils.h>
A
Merge  
Alexey Milovidov 已提交
35
#include <DB/Common/formatReadable.h>
36
#include <DB/Common/setThreadName.h>
37

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

40

M
Merge  
Michael Kolupaev 已提交
41 42 43
namespace DB
{

44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67
namespace ErrorCodes
{
	extern const int NO_ZOOKEEPER;
	extern const int INCORRECT_DATA;
	extern const int INCOMPATIBLE_COLUMNS;
	extern const int REPLICA_IS_ALREADY_EXIST;
	extern const int NO_SUCH_REPLICA;
	extern const int NO_REPLICA_HAS_PART;
	extern const int LOGICAL_ERROR;
	extern const int TOO_MANY_UNEXPECTED_DATA_PARTS;
	extern const int ABORTED;
	extern const int REPLICA_IS_NOT_IN_QUORUM;
	extern const int TABLE_IS_READ_ONLY;
	extern const int NOT_FOUND_NODE;
	extern const int NO_ACTIVE_REPLICAS;
	extern const int LEADERSHIP_CHANGED;
	extern const int TABLE_IS_READ_ONLY;
	extern const int TABLE_WAS_NOT_DROPPED;
	extern const int PARTITION_ALREADY_EXISTS;
	extern const int TOO_MUCH_RETRIES_TO_FETCH_PARTS;
	extern const int RECEIVED_ERROR_FROM_REMOTE_IO_SERVER;
	extern const int PARTITION_DOESNT_EXIST;
}

M
Merge  
Michael Kolupaev 已提交
68

M
Merge  
Michael Kolupaev 已提交
69
const auto ERROR_SLEEP_MS = 1000;
70 71 72 73 74 75

/// Если ждём какого-то события с помощью watch-а, то просыпаться на всякий случай вхолостую раз в указанное время.
const auto WAIT_FOR_NEW_LOGS_SLEEP_MS = 60 * 1000;
const auto WAIT_FOR_ALTER_SLEEP_MS = 300 * 1000;
const auto WAIT_FOR_REPLICA_QUEUE_MS = 10 * 1000;

M
Merge  
Michael Kolupaev 已提交
76
const auto MERGE_SELECTING_SLEEP_MS = 5 * 1000;
M
Merge  
Michael Kolupaev 已提交
77

A
Merge  
Alexey Milovidov 已提交
78
const Int64 RESERVED_BLOCK_NUMBERS = 200;
M
Merge  
Michael Kolupaev 已提交
79

M
Merge  
Michael Kolupaev 已提交
80

81 82 83 84 85 86 87 88 89 90 91 92 93 94 95 96 97 98 99 100 101 102 103 104 105 106 107 108 109 110 111 112 113 114 115 116 117
/** Для каждого куска есть сразу три места, где он должен быть:
  * 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;


A
Merge  
Alexey Milovidov 已提交
118 119 120 121 122 123 124 125 126 127 128 129 130 131 132 133 134 135 136 137 138
void StorageReplicatedMergeTree::setZooKeeper(zkutil::ZooKeeperPtr zookeeper)
{
	std::lock_guard<std::mutex> lock(current_zookeeper_mutex);
	current_zookeeper = zookeeper;
}

zkutil::ZooKeeperPtr StorageReplicatedMergeTree::tryGetZooKeeper()
{
	std::lock_guard<std::mutex> lock(current_zookeeper_mutex);
	return current_zookeeper;
}

zkutil::ZooKeeperPtr StorageReplicatedMergeTree::getZooKeeper()
{
	auto res = tryGetZooKeeper();
	if (!res)
		throw Exception("Cannot get ZooKeeper", ErrorCodes::NO_ZOOKEEPER);
	return res;
}


M
Merge  
Michael Kolupaev 已提交
139 140 141
StorageReplicatedMergeTree::StorageReplicatedMergeTree(
	const String & zookeeper_path_,
	const String & replica_name_,
M
Merge  
Michael Kolupaev 已提交
142
	bool attach,
M
Merge  
Michael Kolupaev 已提交
143 144
	const String & path_, const String & database_name_, const String & name_,
	NamesAndTypesListPtr columns_,
145
	const NamesAndTypesList & materialized_columns_,
146 147
	const NamesAndTypesList & alias_columns_,
	const ColumnDefaults & column_defaults_,
M
Merge  
Michael Kolupaev 已提交
148
	Context & context_,
M
Merge  
Michael Kolupaev 已提交
149 150 151 152 153 154
	ASTPtr & primary_expr_ast_,
	const String & date_column_name_,
	const ASTPtr & sampling_expression_,
	size_t index_granularity_,
	MergeTreeData::Mode mode_,
	const String & sign_column_,
A
Merge  
Alexey Milovidov 已提交
155
	const Names & columns_to_sum_,
M
Merge  
Michael Kolupaev 已提交
156
	const MergeTreeSettings & settings_)
157
    : IStorage{materialized_columns_, alias_columns_, column_defaults_}, context(context_),
A
Merge  
Alexey Milovidov 已提交
158
	current_zookeeper(context.getZooKeeper()), database_name(database_name_),
M
Merge  
Michael Kolupaev 已提交
159 160 161
	table_name(name_), full_path(path_ + escapeForFileName(table_name) + '/'),
	zookeeper_path(context.getMacros().expand(zookeeper_path_)),
	replica_name(context.getMacros().expand(replica_name_)),
162 163 164
	data(full_path, columns_,
		 materialized_columns_, alias_columns_, column_defaults_,
		 context_, primary_expr_ast_, date_column_name_,
A
Merge  
Alexey Milovidov 已提交
165
		 sampling_expression_, index_granularity_, mode_, sign_column_, columns_to_sum_,
166 167
		 settings_, database_name_ + "." + table_name, true,
		 std::bind(&StorageReplicatedMergeTree::enqueuePartForCheck, this, std::placeholders::_1)),
168 169
	reader(data), writer(data), merger(data), fetcher(data), shutdown_event(false),
	log(&Logger::get(database_name + "." + table_name + " (StorageReplicatedMergeTree)"))
M
Merge  
Michael Kolupaev 已提交
170
{
171 172
	if (!zookeeper_path.empty() && zookeeper_path.back() == '/')
		zookeeper_path.resize(zookeeper_path.size() - 1);
M
Merge  
Michael Kolupaev 已提交
173 174 175 176
	replica_path = zookeeper_path + "/replicas/" + replica_name;

	bool skip_sanity_checks = false;

177
	try
M
Merge  
Michael Kolupaev 已提交
178
	{
A
Merge  
Alexey Milovidov 已提交
179
		if (current_zookeeper && current_zookeeper->exists(replica_path + "/flags/force_restore_data"))
180 181
		{
			skip_sanity_checks = true;
A
Merge  
Alexey Milovidov 已提交
182
			current_zookeeper->remove(replica_path + "/flags/force_restore_data");
M
Merge  
Michael Kolupaev 已提交
183

184 185 186 187 188 189 190 191 192 193
			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__);
A
Merge  
Alexey Milovidov 已提交
194
			current_zookeeper = nullptr;
195 196 197
		}
		else
			throw;
M
Merge  
Michael Kolupaev 已提交
198 199 200 201
	}

	data.loadDataParts(skip_sanity_checks);

A
Merge  
Alexey Milovidov 已提交
202
	if (!current_zookeeper)
M
Merge  
Michael Kolupaev 已提交
203
	{
M
Merge  
Michael Kolupaev 已提交
204 205 206
		if (!attach)
			throw Exception("Can't create replicated table without ZooKeeper", ErrorCodes::NO_ZOOKEEPER);

207
		/// Не активируем реплику. Она будет в режиме readonly.
A
Merge  
Alexey Milovidov 已提交
208 209
		LOG_ERROR(log, "No ZooKeeper: table will be in readonly mode.");
		is_readonly = true;
M
Merge  
Michael Kolupaev 已提交
210 211 212
		return;
	}

M
Merge  
Michael Kolupaev 已提交
213 214
	if (!attach)
	{
A
Merge  
Alexey Milovidov 已提交
215 216 217
		if (!data.getDataParts().empty())
			throw Exception("Data directory for table already containing data parts - probably it was unclean DROP table or manual intervention. You must either clear directory by hand or use ATTACH TABLE instead of CREATE TABLE if you need to use that parts.", ErrorCodes::INCORRECT_DATA);

M
Merge  
Michael Kolupaev 已提交
218
		createTableIfNotExists();
M
Merge  
Michael Kolupaev 已提交
219

M
Merge  
Michael Kolupaev 已提交
220
		checkTableStructure(false, false);
M
Merge  
Michael Kolupaev 已提交
221
		createReplica();
M
Merge  
Michael Kolupaev 已提交
222 223 224
	}
	else
	{
M
Merge  
Michael Kolupaev 已提交
225
		checkTableStructure(skip_sanity_checks, true);
M
Merge  
Michael Kolupaev 已提交
226
		checkParts(skip_sanity_checks);
M
Merge  
Michael Kolupaev 已提交
227
	}
M
Merge  
Michael Kolupaev 已提交
228

229 230
	createNewZooKeeperNodes();

M
Merge  
Michael Kolupaev 已提交
231 232 233
	String unreplicated_path = full_path + "unreplicated/";
	if (Poco::File(unreplicated_path).exists())
	{
234 235
		unreplicated_data.reset(new MergeTreeData(unreplicated_path, columns_,
			materialized_columns_, alias_columns_, column_defaults_,
236
			context_, primary_expr_ast_,
A
Merge  
Alexey Milovidov 已提交
237
			date_column_name_, sampling_expression_, index_granularity_, mode_, sign_column_, columns_to_sum_, settings_,
M
Merge  
Michael Kolupaev 已提交
238
			database_name_ + "." + table_name + "[unreplicated]", false));
239 240 241

		unreplicated_data->loadDataParts(skip_sanity_checks);

242 243 244 245 246 247 248 249 250 251
		if (unreplicated_data->getDataPartsVector().empty())
		{
			unreplicated_data.reset();
		}
		else
		{
			LOG_INFO(log, "Have unreplicated data");
			unreplicated_reader.reset(new MergeTreeDataSelectExecutor(*unreplicated_data));
			unreplicated_merger.reset(new MergeTreeDataMerger(*unreplicated_data));
		}
M
Merge  
Michael Kolupaev 已提交
252
	}
M
Merge  
Michael Kolupaev 已提交
253

A
Alexey Milovidov 已提交
254 255 256 257
	queue.initialize(
		zookeeper_path, replica_path,
		database_name + "." + table_name + " (ReplicatedMergeTreeQueue)",
		data.getDataParts(), current_zookeeper);
M
Merge  
Michael Kolupaev 已提交
258

259 260
	queue.pullLogsToQueue(current_zookeeper, nullptr);

261
	/// В этом потоке реплика будет активирована.
262
	restarting_thread.reset(new ReplicatedMergeTreeRestartingThread(*this));
M
Merge  
Michael Kolupaev 已提交
263 264
}

265

266 267 268 269 270 271 272 273 274 275 276
void StorageReplicatedMergeTree::createNewZooKeeperNodes()
{
	auto zookeeper = getZooKeeper();

	/// Работа с кворумом.
	zookeeper->createIfNotExists(zookeeper_path + "/quorum", "");
	zookeeper->createIfNotExists(zookeeper_path + "/quorum/last_part", "");
	zookeeper->createIfNotExists(zookeeper_path + "/quorum/failed_parts", "");

	/// Отслеживание отставания реплик.
	zookeeper->createIfNotExists(replica_path + "/min_unprocessed_insert_time", "");
277
	zookeeper->createIfNotExists(replica_path + "/max_processed_insert_time", "");
278 279 280
}


M
Merge  
Michael Kolupaev 已提交
281 282 283
StoragePtr StorageReplicatedMergeTree::create(
	const String & zookeeper_path_,
	const String & replica_name_,
M
Merge  
Michael Kolupaev 已提交
284
	bool attach,
M
Merge  
Michael Kolupaev 已提交
285 286
	const String & path_, const String & database_name_, const String & name_,
	NamesAndTypesListPtr columns_,
287
	const NamesAndTypesList & materialized_columns_,
288 289
	const NamesAndTypesList & alias_columns_,
	const ColumnDefaults & column_defaults_,
M
Merge  
Michael Kolupaev 已提交
290
	Context & context_,
M
Merge  
Michael Kolupaev 已提交
291 292 293 294 295 296
	ASTPtr & primary_expr_ast_,
	const String & date_column_name_,
	const ASTPtr & sampling_expression_,
	size_t index_granularity_,
	MergeTreeData::Mode mode_,
	const String & sign_column_,
297
	const Names & columns_to_sum_,
M
Merge  
Michael Kolupaev 已提交
298 299
	const MergeTreeSettings & settings_)
{
300 301 302
	auto res = new StorageReplicatedMergeTree{
		zookeeper_path_, replica_name_, attach,
		path_, database_name_, name_,
303
		columns_, materialized_columns_, alias_columns_, column_defaults_,
304 305
		context_, primary_expr_ast_, date_column_name_,
		sampling_expression_, index_granularity_, mode_,
A
Merge  
Alexey Milovidov 已提交
306 307
		sign_column_, columns_to_sum_, settings_};

M
Merge  
Michael Kolupaev 已提交
308
	StoragePtr res_ptr = res->thisPtr();
309

310
	if (res->tryGetZooKeeper())
M
Merge  
Michael Kolupaev 已提交
311
	{
A
Merge  
Alexey Milovidov 已提交
312 313 314
		String endpoint_name = "ReplicatedMergeTree:" + res->replica_path;
		InterserverIOEndpointPtr endpoint = new ReplicatedMergeTreePartsServer(res->data, *res);
		res->endpoint_holder = new InterserverIOEndpointHolder(endpoint_name, endpoint, res->context.getInterserverIOHandler());
M
Merge  
Michael Kolupaev 已提交
315
	}
316

M
Merge  
Michael Kolupaev 已提交
317
	return res_ptr;
M
Merge  
Michael Kolupaev 已提交
318 319
}

A
Merge  
Alexey Milovidov 已提交
320

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

A
Merge  
Alexey Milovidov 已提交
330

M
Merge  
Michael Kolupaev 已提交
331
void StorageReplicatedMergeTree::createTableIfNotExists()
M
Merge  
Michael Kolupaev 已提交
332
{
A
Merge  
Alexey Milovidov 已提交
333 334
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
335 336
	if (zookeeper->exists(zookeeper_path))
		return;
M
Merge  
Michael Kolupaev 已提交
337

M
Merge  
Michael Kolupaev 已提交
338
	LOG_DEBUG(log, "Creating table " << zookeeper_path);
M
Merge  
Michael Kolupaev 已提交
339

M
Merge  
Michael Kolupaev 已提交
340 341
	zookeeper->createAncestors(zookeeper_path);

M
Merge  
Michael Kolupaev 已提交
342
	/// Запишем метаданные таблицы, чтобы реплики могли сверять с ними параметры таблицы.
A
Merge  
Alexey Milovidov 已提交
343 344 345 346 347 348 349 350 351 352 353
	std::string metadata;
	{
		WriteBufferFromString out(metadata);
		out << "metadata format version: 1" << "\n"
			<< "date column: " << data.date_column_name << "\n"
			<< "sampling expression: " << formattedAST(data.sampling_expression) << "\n"
			<< "index granularity: " << data.index_granularity << "\n"
			<< "mode: " << static_cast<int>(data.mode) << "\n"
			<< "sign column: " << data.sign_column << "\n"
			<< "primary key: " << formattedAST(data.primary_expr_ast) << "\n";
	}
M
Merge  
Michael Kolupaev 已提交
354

A
Alexey Milovidov 已提交
355 356
	auto acl = zookeeper->getDefaultACL();

M
Merge  
Michael Kolupaev 已提交
357 358
	zkutil::Ops ops;
	ops.push_back(new zkutil::Op::Create(zookeeper_path, "",
A
Alexey Milovidov 已提交
359
										 acl, zkutil::CreateMode::Persistent));
A
Merge  
Alexey Milovidov 已提交
360
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/metadata", metadata,
A
Alexey Milovidov 已提交
361
										 acl, zkutil::CreateMode::Persistent));
362 363 364
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/columns", ColumnsDescription<false>{
				data.getColumnsListNonMaterialized(), data.materialized_columns,
				data.alias_columns, data.column_defaults}.toString(),
A
Alexey Milovidov 已提交
365
										 acl, zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
366
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/log", "",
A
Alexey Milovidov 已提交
367
										 acl, zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
368
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/blocks", "",
A
Alexey Milovidov 已提交
369
										 acl, zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
370
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/block_numbers", "",
A
Alexey Milovidov 已提交
371
										 acl, zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
372
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/nonincrement_block_numbers", "",
A
Alexey Milovidov 已提交
373
										 acl, zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
374
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/leader_election", "",
A
Alexey Milovidov 已提交
375
										 acl, zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
376
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/temp", "",
A
Alexey Milovidov 已提交
377
										 acl, zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
378
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/replicas", "",
A
Alexey Milovidov 已提交
379
										 acl, zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
380 381 382 383

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

A
Merge  
Alexey Milovidov 已提交
386

M
Merge  
Michael Kolupaev 已提交
387 388
/** Проверить, что список столбцов и настройки таблицы совпадают с указанными в ZK (/metadata).
	* Если нет - бросить исключение.
M
Merge  
Michael Kolupaev 已提交
389
	*/
M
Merge  
Michael Kolupaev 已提交
390
void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bool allow_alter)
M
Merge  
Michael Kolupaev 已提交
391
{
A
Merge  
Alexey Milovidov 已提交
392 393
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
394
	String metadata_str = zookeeper->get(zookeeper_path + "/metadata");
M
Merge  
Michael Kolupaev 已提交
395 396 397 398 399 400 401 402 403 404 405 406 407
	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);
M
Merge  
Michael Kolupaev 已提交
408 409
	/// NOTE: Можно сделать менее строгую проверку совпадения выражений, чтобы таблицы не ломались от небольших изменений
	///       в коде formatAST.
M
Merge  
Michael Kolupaev 已提交
410
	assertString(formattedAST(data.primary_expr_ast), buf);
M
Merge  
Michael Kolupaev 已提交
411
	assertString("\n", buf);
M
Merge  
Michael Kolupaev 已提交
412
	assertEOF(buf);
M
Merge  
Michael Kolupaev 已提交
413 414

	zkutil::Stat stat;
415
	auto columns_desc = ColumnsDescription<true>::parse(zookeeper->get(zookeeper_path + "/columns", &stat));
416 417 418 419 420

	auto & columns = columns_desc.columns;
	auto & materialized_columns = columns_desc.materialized;
	auto & alias_columns = columns_desc.alias;
	auto & column_defaults = columns_desc.defaults;
M
Merge  
Michael Kolupaev 已提交
421
	columns_version = stat.version;
422

423 424 425 426
	if (columns != data.getColumnsListNonMaterialized() ||
		materialized_columns != data.materialized_columns ||
		alias_columns != data.alias_columns ||
		column_defaults != data.column_defaults)
M
Merge  
Michael Kolupaev 已提交
427
	{
428 429 430 431
		if (allow_alter &&
			(skip_sanity_checks ||
			 data.getColumnsListNonMaterialized().sizeOfDifference(columns) +
			 data.materialized_columns.sizeOfDifference(materialized_columns) <= 2))
M
Merge  
Michael Kolupaev 已提交
432
		{
M
Merge  
Michael Kolupaev 已提交
433
			LOG_WARNING(log, "Table structure in ZooKeeper is a little different from local table structure. Assuming ALTER.");
434 435

			/// Без всяких блокировок, потому что таблица еще не создана.
436 437
			InterpreterAlterQuery::updateMetadata(database_name, table_name, columns,
				materialized_columns, alias_columns, column_defaults, context);
438
			data.setColumnsList(columns);
439 440 441
			data.materialized_columns = std::move(materialized_columns);
			data.alias_columns = std::move(alias_columns);
			data.column_defaults = std::move(column_defaults);
M
Merge  
Michael Kolupaev 已提交
442
		}
M
Merge  
Michael Kolupaev 已提交
443
		else
M
Merge  
Michael Kolupaev 已提交
444
		{
M
Merge  
Michael Kolupaev 已提交
445
			throw Exception("Table structure in ZooKeeper is too different from local table structure.",
M
Merge  
Michael Kolupaev 已提交
446
							ErrorCodes::INCOMPATIBLE_COLUMNS);
M
Merge  
Michael Kolupaev 已提交
447
		}
M
Merge  
Michael Kolupaev 已提交
448 449
	}
}
M
Merge  
Michael Kolupaev 已提交
450

A
Merge  
Alexey Milovidov 已提交
451

452 453 454 455 456 457 458 459 460 461 462 463 464 465 466 467 468 469 470
/** При необходимости восстановить кусок, реплика сама добавляет в свою очередь запись на его получение.
  * Какое поставить время для этой записи в очереди? Время учитывается при расчёте отставания реплики.
  * Для этих целей имеет смысл использовать время создания недостающего куска
  *  (то есть, при расчёте отставания будет учитано, насколько старый кусок нам нужно восстановить).
  */
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;
}


M
Merge  
Michael Kolupaev 已提交
471 472
void StorageReplicatedMergeTree::createReplica()
{
A
Merge  
Alexey Milovidov 已提交
473 474
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
475
	LOG_DEBUG(log, "Creating replica " << replica_path);
M
Merge  
Michael Kolupaev 已提交
476

M
Merge  
Michael Kolupaev 已提交
477
	/// Создадим пустую реплику. Ноду columns создадим в конце - будем использовать ее в качестве признака, что создание реплики завершено.
A
Alexey Milovidov 已提交
478
	auto acl = zookeeper->getDefaultACL();
M
Merge  
Michael Kolupaev 已提交
479
	zkutil::Ops ops;
A
Alexey Milovidov 已提交
480 481 482 483 484 485
	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));
486 487 488 489 490 491 492 493 494 495 496 497

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

M
Merge  
Michael Kolupaev 已提交
499 500 501 502 503
	/** Нужно изменить данные ноды /replicas на что угодно, чтобы поток, удаляющий старые записи в логе,
	  *  споткнулся об это изменение и не удалил записи, которые мы еще не прочитали.
	  */
	zookeeper->set(zookeeper_path + "/replicas", "last added replica: " + replica_name);

M
Merge  
Michael Kolupaev 已提交
504
	Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
M
Merge  
Michael Kolupaev 已提交
505

M
Merge  
Michael Kolupaev 已提交
506 507 508 509
	/** "Эталонная" реплика, у которой мы возьмем информацию о множестве кусков, очередь и указатель на лог.
	  * Возьмем случайную из реплик, созданных раньше этой.
	  */
	String source_replica;
M
Merge  
Michael Kolupaev 已提交
510

M
Merge  
Michael Kolupaev 已提交
511 512 513
	Stat stat;
	zookeeper->exists(replica_path, &stat);
	auto my_create_time = stat.czxid;
M
Merge  
Michael Kolupaev 已提交
514

M
Merge  
Michael Kolupaev 已提交
515 516
	std::random_shuffle(replicas.begin(), replicas.end());
	for (const String & replica : replicas)
M
Merge  
Michael Kolupaev 已提交
517
	{
M
Merge  
Michael Kolupaev 已提交
518 519 520 521 522 523 524 525
		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;
		}
M
Merge  
Michael Kolupaev 已提交
526 527
	}

M
Merge  
Michael Kolupaev 已提交
528
	if (source_replica.empty())
M
Merge  
Michael Kolupaev 已提交
529
	{
M
Merge  
Michael Kolupaev 已提交
530
		LOG_INFO(log, "This is the first replica");
M
Merge  
Michael Kolupaev 已提交
531
	}
M
Merge  
Michael Kolupaev 已提交
532
	else
M
Merge  
Michael Kolupaev 已提交
533
	{
M
Merge  
Michael Kolupaev 已提交
534
		LOG_INFO(log, "Will mimic " << source_replica);
M
Merge  
Michael Kolupaev 已提交
535

M
Merge  
Michael Kolupaev 已提交
536
		String source_path = zookeeper_path + "/replicas/" + source_replica;
M
Merge  
Michael Kolupaev 已提交
537

M
Merge  
Michael Kolupaev 已提交
538 539 540 541 542 543 544 545 546 547 548 549 550 551 552 553 554 555 556 557 558 559 560 561 562 563 564 565 566 567 568 569 570 571 572 573 574 575 576
		/** Если эталонная реплика еще не до конца создана, подождем.
		  * 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");
577 578
		ActiveDataPartSet active_parts_set(parts);

M
Merge  
Michael Kolupaev 已提交
579 580 581 582 583 584 585
		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;
586
			log_entry.create_time = tryGetPartCreateTime(zookeeper, source_path, name);
587

M
Merge  
Michael Kolupaev 已提交
588 589 590 591 592 593 594 595 596
			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);
		}
597

598
		/// Далее оно будет загружено в переменную queue в методе queue.initialize.
599

M
Merge  
Michael Kolupaev 已提交
600
		LOG_DEBUG(log, "Copied " << source_queue.size() << " queue entries");
M
Merge  
Michael Kolupaev 已提交
601
	}
M
Merge  
Michael Kolupaev 已提交
602

603 604 605 606 607 608
	zookeeper->create(replica_path + "/columns", ColumnsDescription<false>{
			data.getColumnsListNonMaterialized(),
			data.materialized_columns,
			data.alias_columns,
			data.column_defaults
		}.toString(), zkutil::CreateMode::Persistent);
M
Merge  
Michael Kolupaev 已提交
609
}
M
Merge  
Michael Kolupaev 已提交
610 611


M
Merge  
Michael Kolupaev 已提交
612
void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
M
Merge  
Michael Kolupaev 已提交
613
{
A
Merge  
Alexey Milovidov 已提交
614 615
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
616
	Strings expected_parts_vec = zookeeper->getChildren(replica_path + "/parts");
M
Merge  
Michael Kolupaev 已提交
617 618

	/// Куски в ZK.
M
Merge  
Michael Kolupaev 已提交
619 620
	NameSet expected_parts(expected_parts_vec.begin(), expected_parts_vec.end());

M
Merge  
Michael Kolupaev 已提交
621
	MergeTreeData::DataParts parts = data.getAllDataParts();
M
Merge  
Michael Kolupaev 已提交
622

M
Merge  
Michael Kolupaev 已提交
623
	/// Локальные куски, которых нет в ZK.
M
Merge  
Michael Kolupaev 已提交
624
	MergeTreeData::DataParts unexpected_parts;
M
Merge  
Michael Kolupaev 已提交
625

M
Merge  
Michael Kolupaev 已提交
626 627 628 629 630
	for (const auto & part : parts)
	{
		if (expected_parts.count(part->name))
			expected_parts.erase(part->name);
		else
M
Merge  
Michael Kolupaev 已提交
631
			unexpected_parts.insert(part);
M
Merge  
Michael Kolupaev 已提交
632 633
	}

M
Merge  
Michael Kolupaev 已提交
634
	/// Какие локальные куски добавить в ZK.
M
Merge  
Michael Kolupaev 已提交
635
	MergeTreeData::DataPartsVector parts_to_add;
M
Merge  
Michael Kolupaev 已提交
636 637 638 639

	/// Какие куски нужно забрать с других реплик.
	Strings parts_to_fetch;

M
Merge  
Michael Kolupaev 已提交
640 641
	for (const String & missing_name : expected_parts)
	{
M
Merge  
Michael Kolupaev 已提交
642
		/// Если локально не хватает какого-то куска, но есть покрывающий его кусок, можно заменить в ZK недостающий покрывающим.
M
Merge  
Michael Kolupaev 已提交
643
		auto containing = data.getActiveContainingPart(missing_name);
M
Merge  
Michael Kolupaev 已提交
644 645 646 647 648 649 650 651 652 653
		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
M
Merge  
Michael Kolupaev 已提交
654
		{
M
Merge  
Michael Kolupaev 已提交
655
			LOG_ERROR(log, "Fetching missing part " << missing_name);
M
Merge  
Michael Kolupaev 已提交
656
			parts_to_fetch.push_back(missing_name);
M
Merge  
Michael Kolupaev 已提交
657 658
		}
	}
M
Merge  
Michael Kolupaev 已提交
659

M
Merge  
Michael Kolupaev 已提交
660 661 662
	for (const String & name : parts_to_fetch)
		expected_parts.erase(name);

663 664 665 666 667 668 669 670 671 672 673 674 675 676 677
	/** Для проверки адекватности, для кусков, которые есть в ФС, но нет в 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";
M
Merge  
Michael Kolupaev 已提交
678

679
	/** Можно автоматически синхронизировать данные,
680 681
	  *  если количество ошибок каждого из четырёх типов не больше соответствующих порогов,
	  *  или если отношение общего количества ошибок к общему количеству кусков (минимальному - в локальной файловой системе или в ZK)
682
	  *  не больше некоторого отношения (например 5%).
683 684 685 686
	  *
	  * Большое количество несовпадений в данных на файловой системе и ожидаемых данных
	  *  может свидетельствовать об ошибке конфигурации (сервер случайно подключили как реплику не от того шарда).
	  * В этом случае, защитный механизм не даёт стартовать серверу.
687 688 689
	  */

	size_t min_parts_local_or_expected = std::min(expected_parts_vec.size(), parts.size());
690
	size_t total_difference = parts_to_add.size() + unexpected_parts_nonnew + expected_parts.size() + parts_to_fetch.size();
691 692 693

	bool insane =
		(parts_to_add.size() > data.settings.replicated_max_unexpectedly_merged_parts
694
			|| unexpected_parts_nonnew > data.settings.replicated_max_unexpected_parts
695 696
			|| expected_parts.size() > data.settings.replicated_max_missing_obsolete_parts
			|| parts_to_fetch.size() > data.settings.replicated_max_missing_active_parts)
697
		&& (total_difference > min_parts_local_or_expected * data.settings.replicated_max_ratio_of_wrong_parts);
M
Merge  
Michael Kolupaev 已提交
698

699 700 701 702 703 704
	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);
M
Merge  
Michael Kolupaev 已提交
705

M
Merge  
Michael Kolupaev 已提交
706
	/// Добавим в ZK информацию о кусках, покрывающих недостающие куски.
707
	for (const MergeTreeData::DataPartPtr & part : parts_to_add)
M
Merge  
Michael Kolupaev 已提交
708
	{
M
Merge  
Michael Kolupaev 已提交
709
		LOG_ERROR(log, "Adding unexpected local part to ZooKeeper: " << part->name);
M
Merge  
Michael Kolupaev 已提交
710

M
Merge  
Michael Kolupaev 已提交
711
		zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
712
		checkPartAndAddToZooKeeper(part, ops);
M
Merge  
Michael Kolupaev 已提交
713
		zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
714
	}
M
Merge  
Michael Kolupaev 已提交
715

M
Merge  
Michael Kolupaev 已提交
716 717 718
	/// Удалим из ZK информацию о кусках, покрытых только что добавленными.
	for (const String & name : expected_parts)
	{
M
Merge  
Michael Kolupaev 已提交
719 720 721
		LOG_ERROR(log, "Removing unexpectedly merged local part from ZooKeeper: " << name);

		zkutil::Ops ops;
722
		removePartFromZooKeeper(name, ops);
M
Merge  
Michael Kolupaev 已提交
723 724 725 726 727 728 729 730 731 732
		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;
M
Merge  
Michael Kolupaev 已提交
733
		log_entry.source_replica = "";
M
Merge  
Michael Kolupaev 已提交
734
		log_entry.new_part_name = name;
735
		log_entry.create_time = tryGetPartCreateTime(zookeeper, replica_path, name);
M
Merge  
Michael Kolupaev 已提交
736

737
		/// Полагаемся, что это происходит до загрузки очереди (queue.initialize).
M
Merge  
Michael Kolupaev 已提交
738
		zkutil::Ops ops;
739
		removePartFromZooKeeper(name, ops);
M
Merge  
Michael Kolupaev 已提交
740 741
		ops.push_back(new zkutil::Op::Create(
			replica_path + "/queue/queue-", log_entry.toString(), zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
M
Merge  
Michael Kolupaev 已提交
742
		zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
743 744 745
	}

	/// Удалим лишние локальные куски.
746
	for (const MergeTreeData::DataPartPtr & part : unexpected_parts)
M
Merge  
Michael Kolupaev 已提交
747
	{
M
Merge  
Michael Kolupaev 已提交
748
		LOG_ERROR(log, "Renaming unexpected part " << part->name << " to ignored_" + part->name);
M
Merge  
Michael Kolupaev 已提交
749
		data.renameAndDetachPart(part, "ignored_", true);
M
Merge  
Michael Kolupaev 已提交
750 751
	}
}
M
Merge  
Michael Kolupaev 已提交
752

A
Merge  
Alexey Milovidov 已提交
753

754
void StorageReplicatedMergeTree::checkPartAndAddToZooKeeper(const MergeTreeData::DataPartPtr & part, zkutil::Ops & ops, String part_name)
M
Merge  
Michael Kolupaev 已提交
755
{
A
Merge  
Alexey Milovidov 已提交
756 757
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
758 759 760
	if (part_name.empty())
		part_name = part->name;

761
	check(part->columns);
M
Merge  
Michael Kolupaev 已提交
762
	int expected_columns_version = columns_version;
763

M
Merge  
Michael Kolupaev 已提交
764 765 766 767 768
	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)
M
Merge  
Michael Kolupaev 已提交
769
	{
M
Merge  
Michael Kolupaev 已提交
770 771
		zkutil::Stat stat_before, stat_after;
		String columns_str;
M
Merge  
Michael Kolupaev 已提交
772
		if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name + "/columns", columns_str, &stat_before))
M
Merge  
Michael Kolupaev 已提交
773 774 775
			continue;
		if (columns_str != expected_columns_str)
		{
M
Merge  
Michael Kolupaev 已提交
776
			LOG_INFO(log, "Not checking checksums of part " << part_name << " with replica " << replica
M
Merge  
Michael Kolupaev 已提交
777 778 779
				<< " because columns are different");
			continue;
		}
M
Merge  
Michael Kolupaev 已提交
780
		String checksums_str;
M
Merge  
Michael Kolupaev 已提交
781 782
		/// Проверим, что версия ноды со столбцами не изменилась, пока мы читали checksums.
		/// Это гарантирует, что столбцы и чексуммы относятся к одним и тем же данным.
M
Merge  
Michael Kolupaev 已提交
783 784
		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) ||
M
Merge  
Michael Kolupaev 已提交
785
			stat_before.version != stat_after.version)
M
Merge  
Michael Kolupaev 已提交
786
		{
M
Merge  
Michael Kolupaev 已提交
787
			LOG_INFO(log, "Not checking checksums of part " << part_name << " with replica " << replica
M
Merge  
Michael Kolupaev 已提交
788 789
				<< " because part changed while we were reading its checksums");
			continue;
M
Merge  
Michael Kolupaev 已提交
790
		}
M
Merge  
Michael Kolupaev 已提交
791 792 793

		auto checksums = MergeTreeData::DataPart::Checksums::parse(checksums_str);
		checksums.checkEqual(part->checksums, true);
M
Merge  
Michael Kolupaev 已提交
794 795
	}

M
Merge  
Michael Kolupaev 已提交
796
	if (zookeeper->exists(replica_path + "/parts/" + part_name))
M
Merge  
Michael Kolupaev 已提交
797
	{
M
Merge  
Michael Kolupaev 已提交
798
		LOG_ERROR(log, "checkPartAndAddToZooKeeper: node " << replica_path + "/parts/" + part_name << " already exists");
M
Merge  
Michael Kolupaev 已提交
799 800 801
		return;
	}

A
Alexey Milovidov 已提交
802 803
	auto acl = zookeeper->getDefaultACL();

804 805 806
	ops.push_back(new zkutil::Op::Check(
		zookeeper_path + "/columns",
		expected_columns_version));
M
Merge  
Michael Kolupaev 已提交
807
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
808
		replica_path + "/parts/" + part_name,
M
Merge  
Michael Kolupaev 已提交
809
		"",
A
Alexey Milovidov 已提交
810
		acl,
M
Merge  
Michael Kolupaev 已提交
811
		zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
812
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
813
		replica_path + "/parts/" + part_name + "/columns",
M
Merge  
Michael Kolupaev 已提交
814
		part->columns.toString(),
A
Alexey Milovidov 已提交
815
		acl,
M
Merge  
Michael Kolupaev 已提交
816
		zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
817
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
818
		replica_path + "/parts/" + part_name + "/checksums",
M
Merge  
Michael Kolupaev 已提交
819
		part->checksums.toString(),
A
Alexey Milovidov 已提交
820
		acl,
M
Merge  
Michael Kolupaev 已提交
821 822 823
		zkutil::CreateMode::Persistent));
}

A
Merge  
Alexey Milovidov 已提交
824

825 826 827 828 829 830 831 832 833 834 835 836 837 838 839 840 841 842 843 844 845 846 847 848 849 850 851 852 853 854 855 856
void StorageReplicatedMergeTree::addNewPartToZooKeeper(const MergeTreeData::DataPartPtr & part, zkutil::Ops & ops, String part_name)
{
	auto zookeeper = getZooKeeper();

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

	check(part->columns);

	auto acl = zookeeper->getDefaultACL();

	ops.push_back(new zkutil::Op::Check(
		zookeeper_path + "/columns",
		columns_version));
	ops.push_back(new zkutil::Op::Create(
		replica_path + "/parts/" + part_name,
		"",
		acl,
		zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(
		replica_path + "/parts/" + part_name + "/columns",
		part->columns.toString(),
		acl,
		zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(
		replica_path + "/parts/" + part_name + "/checksums",
		part->checksums.toString(),
		acl,
		zkutil::CreateMode::Persistent));
}


M
Merge  
Michael Kolupaev 已提交
857
void StorageReplicatedMergeTree::pullLogsToQueue(zkutil::EventPtr next_update_event)
M
Merge  
Michael Kolupaev 已提交
858
{
A
Alexey Milovidov 已提交
859
	if (queue.pullLogsToQueue(getZooKeeper(), next_update_event))
M
Merge  
Michael Kolupaev 已提交
860
	{
A
Alexey Milovidov 已提交
861 862
		if (queue_task_handle)
			queue_task_handle->wake();
M
Merge  
Michael Kolupaev 已提交
863
	}
M
Merge  
Michael Kolupaev 已提交
864 865
}

A
Merge  
Alexey Milovidov 已提交
866

M
Merge  
Michael Kolupaev 已提交
867
bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry, BackgroundProcessingPool::Context & pool_context)
M
Merge  
Michael Kolupaev 已提交
868
{
A
Merge  
Alexey Milovidov 已提交
869 870
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
871
	if (entry.type == LogEntry::DROP_RANGE)
M
Merge  
Michael Kolupaev 已提交
872 873 874 875
	{
		executeDropRange(entry);
		return true;
	}
M
Merge  
Michael Kolupaev 已提交
876

M
Merge  
Michael Kolupaev 已提交
877
	if (entry.type == LogEntry::GET_PART ||
M
Merge  
Michael Kolupaev 已提交
878 879
		entry.type == LogEntry::MERGE_PARTS ||
		entry.type == LogEntry::ATTACH_PART)
M
Merge  
Michael Kolupaev 已提交
880 881
	{
		/// Если у нас уже есть этот кусок или покрывающий его кусок, ничего делать не нужно.
M
Merge  
Michael Kolupaev 已提交
882
		MergeTreeData::DataPartPtr containing_part = data.getActiveContainingPart(entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
883

884
		/// Даже если кусок есть локально, его (в исключительных случаях) может не быть в zookeeper. Проверим, что он там есть.
M
Merge  
Michael Kolupaev 已提交
885
		if (containing_part && zookeeper->exists(replica_path + "/parts/" + containing_part->name))
M
Merge  
Michael Kolupaev 已提交
886
		{
M
Merge  
Michael Kolupaev 已提交
887
			if (!(entry.type == LogEntry::GET_PART && entry.source_replica == replica_name))
888
				LOG_DEBUG(log, "Skipping action for part " << entry.new_part_name << " - part already exists.");
M
Merge  
Michael Kolupaev 已提交
889
			return true;
M
Merge  
Michael Kolupaev 已提交
890
		}
M
Merge  
Michael Kolupaev 已提交
891 892
	}

M
Merge  
Michael Kolupaev 已提交
893
	if (entry.type == LogEntry::GET_PART && entry.source_replica == replica_name)
M
Merge  
Michael Kolupaev 已提交
894
		LOG_WARNING(log, "Part " << entry.new_part_name << " from own log doesn't exist.");
M
Merge  
Michael Kolupaev 已提交
895

896
	/// Возможно, этот кусок нам не нужен, так как при записи с кворумом, кворум пофейлился (см. ниже про /quorum/failed_parts).
897
	if (entry.quorum && zookeeper->exists(zookeeper_path + "/quorum/failed_parts/" + entry.new_part_name))
898 899 900 901 902
	{
		LOG_DEBUG(log, "Skipping action for part " << entry.new_part_name << " because quorum for that part was failed.");
		return true;	/// NOTE Удаление из virtual_parts не делается, но оно нужно только для мерджей.
	}

M
Merge  
Michael Kolupaev 已提交
903 904
	bool do_fetch = false;

M
Merge  
Michael Kolupaev 已提交
905 906
	if (entry.type == LogEntry::GET_PART)
	{
M
Merge  
Michael Kolupaev 已提交
907
		do_fetch = true;
M
Merge  
Michael Kolupaev 已提交
908
	}
M
Merge  
Michael Kolupaev 已提交
909 910 911 912
	else if (entry.type == LogEntry::ATTACH_PART)
	{
		do_fetch = !executeAttachPart(entry);
	}
M
Merge  
Michael Kolupaev 已提交
913 914
	else if (entry.type == LogEntry::MERGE_PARTS)
	{
915 916 917 918 919 920 921 922
		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());

M
Merge  
Michael Kolupaev 已提交
923
		MergeTreeData::DataPartsVector parts;
924
		bool have_all_parts = true;
M
Merge  
Michael Kolupaev 已提交
925 926
		for (const String & name : entry.parts_to_merge)
		{
M
Merge  
Michael Kolupaev 已提交
927
			MergeTreeData::DataPartPtr part = data.getActiveContainingPart(name);
M
Merge  
Michael Kolupaev 已提交
928 929 930 931 932 933 934
			if (!part)
			{
				have_all_parts = false;
				break;
			}
			if (part->name != name)
			{
M
Merge  
Michael Kolupaev 已提交
935 936
				LOG_WARNING(log, "Part " << name << " is covered by " << part->name
					<< " but should be merged into " << entry.new_part_name << ". This shouldn't happen often.");
M
Merge  
Michael Kolupaev 已提交
937 938 939
				have_all_parts = false;
				break;
			}
M
Merge  
Michael Kolupaev 已提交
940 941
			parts.push_back(part);
		}
M
Merge  
Michael Kolupaev 已提交
942

M
Merge  
Michael Kolupaev 已提交
943
		if (!have_all_parts)
M
Merge  
Michael Kolupaev 已提交
944
		{
M
Merge  
Michael Kolupaev 已提交
945 946 947
			/// Если нет всех нужных кусков, попробуем взять у кого-нибудь уже помердженный кусок.
			do_fetch = true;
			LOG_DEBUG(log, "Don't have all parts for merge " << entry.new_part_name << "; will try to fetch it instead");
M
Merge  
Michael Kolupaev 已提交
948
		}
M
Merge  
Michael Kolupaev 已提交
949 950
		else
		{
M
Merge  
Michael Kolupaev 已提交
951 952 953
			/// Если собираемся сливать большие куски, увеличим счетчик потоков, сливающих большие куски.
			for (const auto & part : parts)
			{
M
Merge  
Michael Kolupaev 已提交
954
				if (part->size_in_bytes > data.settings.max_bytes_to_merge_parts_small)
M
Merge  
Michael Kolupaev 已提交
955 956 957 958 959 960 961
				{
					pool_context.incrementCounter("big merges");
					pool_context.incrementCounter("replicated big merges");
					break;
				}
			}

A
Merge  
Alexey Milovidov 已提交
962
			size_t sum_parts_size_in_bytes = MergeTreeDataMerger::estimateDiskSpaceForMerge(parts);
963 964 965

			/// Может бросить исключение.
			DiskSpaceMonitor::ReservationPtr reserved_space = DiskSpaceMonitor::reserve(full_path, sum_parts_size_in_bytes);
A
Merge  
Alexey Milovidov 已提交
966

M
Merge  
Michael Kolupaev 已提交
967
			auto table_lock = lockStructure(false);
968

969
			const auto & merge_entry = context.getMergeList().insert(database_name, table_name, entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
970
			MergeTreeData::Transaction transaction;
A
Merge  
Alexey Arno 已提交
971
			size_t aio_threshold = context.getSettings().min_bytes_to_use_direct_io;
972 973 974

			MergeTreeData::DataPartPtr part = merger.mergeParts(
				parts, entry.new_part_name, *merge_entry, aio_threshold, &transaction, reserved_space);
M
Merge  
Michael Kolupaev 已提交
975 976

			zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
977
			checkPartAndAddToZooKeeper(part, ops);
M
Merge  
Michael Kolupaev 已提交
978

M
Merge  
Michael Kolupaev 已提交
979 980 981 982
			/** TODO: Переименование нового куска лучше делать здесь, а не пятью строчками выше,
			  *  чтобы оно было как можно ближе к zookeeper->multi.
			  */

M
Merge  
Michael Kolupaev 已提交
983
			zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
984

985 986 987
			/** Удаление старых кусков из ZK и с диска делается отложенно - см. ReplicatedMergeTreeCleanupThread, clearOldParts.
			  */

M
Merge  
Michael Kolupaev 已提交
988 989 990
			/** При ZCONNECTIONLOSS или ZOPERATIONTIMEOUT можем зря откатить локальные изменения кусков.
			  * Это не проблема, потому что в таком случае слияние останется в очереди, и мы попробуем снова.
			  */
M
Merge  
Michael Kolupaev 已提交
991
			transaction.commit();
M
Merge  
Michael Kolupaev 已提交
992
			merge_selecting_event.set();
M
Merge  
Michael Kolupaev 已提交
993 994 995

			ProfileEvents::increment(ProfileEvents::ReplicatedPartMerges);
		}
M
Merge  
Michael Kolupaev 已提交
996 997 998 999 1000
	}
	else
	{
		throw Exception("Unexpected log entry type: " + toString(static_cast<int>(entry.type)));
	}
M
Merge  
Michael Kolupaev 已提交
1001 1002 1003

	if (do_fetch)
	{
M
Merge  
Michael Kolupaev 已提交
1004 1005
		String replica;

M
Merge  
Michael Kolupaev 已提交
1006 1007
		try
		{
M
Merge  
Michael Kolupaev 已提交
1008
			replica = findReplicaHavingPart(entry.new_part_name, true);
A
Merge  
Alexey Milovidov 已提交
1009 1010 1011 1012 1013 1014 1015 1016 1017 1018 1019 1020 1021 1022 1023 1024 1025

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

M
Merge  
Michael Kolupaev 已提交
1026 1027
			if (replica.empty())
			{
1028 1029 1030 1031 1032 1033 1034 1035 1036 1037
				/** Если кусок должен быть записан с кворумом, и кворум ещё недостигнут,
				  *  то (из-за того, что кусок невозможно прямо сейчас скачать),
				  *  кворумную запись следует считать безуспешной.
				  * 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);

1038 1039 1040
					if (entry.block_id.empty())
						throw Exception("Logical error: log entry with quorum have empty block_id", ErrorCodes::LOGICAL_ERROR);

1041 1042 1043 1044 1045 1046 1047 1048
					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, чтобы разрешить мерджи через номер потерянного куска;
1049 1050
					  * - добавим кусок в список quorum/failed_parts;
					  * - если кусок ещё не удалён из списка для дедупликации blocks/block_num, то удалим его;
1051
					  *
1052 1053 1054 1055 1056 1057 1058 1059 1060 1061 1062 1063 1064 1065 1066 1067 1068 1069 1070 1071 1072 1073 1074
					  * Если что-то изменится, то ничего не сделаем - попадём сюда снова в следующий раз.
					  */

					/** Соберём версии узлов 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);

1075 1076
					/// Также за это время могла быть создана совсем новая реплика.
					/// Но если на старых не появится куска, то на новой его тоже не может быть.
1077 1078 1079 1080 1081 1082 1083 1084 1085 1086 1087 1088 1089 1090 1091 1092 1093 1094 1095 1096 1097

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

1098 1099
							zookeeper->createIfNotExists(zookeeper_path + "/nonincrement_block_numbers/" + partition_str, "");

A
Alexey Milovidov 已提交
1100 1101
							auto acl = zookeeper->getDefaultACL();

1102 1103 1104
							ops.push_back(new zkutil::Op::Create(
								zookeeper_path + "/nonincrement_block_numbers/" + partition_str + "/block-" + padIndex(part_info.left),
								"",
A
Alexey Milovidov 已提交
1105
								acl,
1106 1107 1108 1109 1110
								zkutil::CreateMode::Persistent));

							ops.push_back(new zkutil::Op::Create(
								zookeeper_path + "/quorum/failed_parts/" + entry.new_part_name,
								"",
A
Alexey Milovidov 已提交
1111
								acl,
1112 1113
								zkutil::CreateMode::Persistent));

1114 1115 1116 1117 1118 1119 1120 1121
							/// Удаление из blocks.
							if (zookeeper->exists(zookeeper_path + "/blocks/" + entry.block_id))
							{
								ops.push_back(new zkutil::Op::Remove(zookeeper_path + "/blocks/" + entry.block_id + "/number", -1));
								ops.push_back(new zkutil::Op::Remove(zookeeper_path + "/blocks/" + entry.block_id + "/checksum", -1));
								ops.push_back(new zkutil::Op::Remove(zookeeper_path + "/blocks/" + entry.block_id, -1));
							}

1122 1123 1124 1125 1126 1127 1128 1129 1130 1131
							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 "
1132
									<< entry.new_part_name << " as failed. Code: " << zerror(code));
1133 1134 1135 1136 1137 1138 1139 1140
							}
							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
1141 1142
								<< ". It means that part was successfully written to " << entry.quorum
								<< " replicas, but then all of them goes offline."
1143 1144 1145 1146 1147 1148 1149 1150 1151 1152
								<< " 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);
				}
M
Merge  
Michael Kolupaev 已提交
1153
			}
A
Merge  
Alexey Milovidov 已提交
1154

1155
			fetchPart(entry.new_part_name, zookeeper_path + "/replicas/" + replica, false, entry.quorum);
M
Merge  
Michael Kolupaev 已提交
1156 1157 1158 1159 1160 1161 1162 1163 1164 1165 1166 1167

			if (entry.type == LogEntry::MERGE_PARTS)
				ProfileEvents::increment(ProfileEvents::ReplicatedPartFetchesOfMerged);
		}
		catch (...)
		{
			/** Если не получилось скачать кусок, нужный для какого-то мерджа, лучше не пытаться получить другие куски для этого мерджа,
			  * а попытаться сразу получить помердженный кусок. Чтобы так получилось, переместим действия для получения остальных кусков
			  * для этого мерджа в конец очереди.
			  */
			try
			{
A
Alexey Milovidov 已提交
1168
				auto parts_for_merge = queue.moveSiblingPartsForMergeToEndOfQueue(entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
1169

A
Alexey Milovidov 已提交
1170
				if (!parts_for_merge.empty() && replica.empty())
M
Merge  
Michael Kolupaev 已提交
1171
				{
A
Alexey Milovidov 已提交
1172 1173
					LOG_INFO(log, "No active replica has part " << entry.new_part_name << ". Will fetch merged part instead.");
					return false;
M
Merge  
Michael Kolupaev 已提交
1174
				}
M
Merge  
Michael Kolupaev 已提交
1175

1176 1177 1178
				/** Если ни у какой активной реплики нет куска, и в очереди нет слияний с его участием,
				  * проверим, есть ли у любой (активной или неактивной) реплики такой кусок или покрывающий его.
				  */
M
Merge  
Michael Kolupaev 已提交
1179 1180
				if (replica.empty())
					enqueuePartForCheck(entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
1181 1182 1183 1184 1185 1186 1187 1188 1189
			}
			catch (...)
			{
				tryLogCurrentException(__PRETTY_FUNCTION__);
			}

			throw;
		}
	}
M
Merge  
Michael Kolupaev 已提交
1190 1191

	return true;
M
Merge  
Michael Kolupaev 已提交
1192 1193
}

A
Merge  
Alexey Milovidov 已提交
1194

M
Merge  
Michael Kolupaev 已提交
1195
void StorageReplicatedMergeTree::executeDropRange(const StorageReplicatedMergeTree::LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
1196
{
A
Merge  
Alexey Milovidov 已提交
1197 1198
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
1199
	LOG_INFO(log, (entry.detach ? "Detaching" : "Removing") << " parts inside " << entry.new_part_name << ".");
M
Merge  
Michael Kolupaev 已提交
1200

A
Alexey Milovidov 已提交
1201
	queue.removeGetsAndMergesInRange(zookeeper, entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
1202

M
Merge  
Michael Kolupaev 已提交
1203
	LOG_DEBUG(log, (entry.detach ? "Detaching" : "Removing") << " parts.");
M
Merge  
Michael Kolupaev 已提交
1204 1205 1206 1207 1208 1209 1210 1211 1212 1213 1214
	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;

M
Merge  
Michael Kolupaev 已提交
1215 1216 1217 1218
		/// Если кусок удалять не нужно, надежнее переместить директорию до изменений в ZooKeeper.
		if (entry.detach)
			data.renameAndDetachPart(part);

M
Merge  
Michael Kolupaev 已提交
1219
		zkutil::Ops ops;
1220
		removePartFromZooKeeper(part->name, ops);
M
Merge  
Michael Kolupaev 已提交
1221 1222
		zookeeper->multi(ops);

M
Merge  
Michael Kolupaev 已提交
1223
		/// Если кусок нужно удалить, надежнее удалить директорию после изменений в ZooKeeper.
M
Merge  
Michael Kolupaev 已提交
1224
		if (!entry.detach)
M
Merge  
Michael Kolupaev 已提交
1225
			data.replaceParts({part}, {}, true);
M
Merge  
Michael Kolupaev 已提交
1226 1227
	}

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

A
Merge  
Alexey Milovidov 已提交
1231

M
Merge  
Michael Kolupaev 已提交
1232 1233
bool StorageReplicatedMergeTree::executeAttachPart(const StorageReplicatedMergeTree::LogEntry & entry)
{
A
Merge  
Alexey Milovidov 已提交
1234 1235
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
1236 1237 1238 1239
	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);

M
Merge  
Michael Kolupaev 已提交
1240
	if (!Poco::File(data.getFullPath() + source_path).exists())
M
Merge  
Michael Kolupaev 已提交
1241
	{
M
Merge  
Michael Kolupaev 已提交
1242
		LOG_INFO(log, "No part at " << source_path << ". Will fetch it instead");
M
Merge  
Michael Kolupaev 已提交
1243 1244 1245 1246 1247 1248 1249 1250 1251 1252 1253 1254 1255 1256 1257 1258 1259 1260 1261 1262 1263 1264 1265 1266 1267 1268 1269 1270
		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);
M
Merge  
Michael Kolupaev 已提交
1271

M
Merge  
Michael Kolupaev 已提交
1272 1273 1274
	/// На месте удаленных кусков могут появиться новые, с другими данными.
	context.resetCaches();

M
Merge  
Michael Kolupaev 已提交
1275 1276 1277
	return true;
}

A
Merge  
Alexey Milovidov 已提交
1278

M
Merge  
Michael Kolupaev 已提交
1279 1280
void StorageReplicatedMergeTree::queueUpdatingThread()
{
1281 1282
	setThreadName("ReplMTQueueUpd");

M
Merge  
Michael Kolupaev 已提交
1283 1284
	while (!shutdown_called)
	{
M
Merge  
Michael Kolupaev 已提交
1285 1286
		try
		{
M
Merge  
Michael Kolupaev 已提交
1287
			pullLogsToQueue(queue_updating_event);
1288
			queue_updating_event->tryWait(WAIT_FOR_NEW_LOGS_SLEEP_MS);
M
Merge  
Michael Kolupaev 已提交
1289
		}
1290
		catch (const zkutil::KeeperException & e)
M
Merge  
Michael Kolupaev 已提交
1291 1292
		{
			if (e.code == ZINVALIDSTATE)
1293
				restarting_thread->wakeup();
M
Merge  
Michael Kolupaev 已提交
1294 1295 1296 1297

			tryLogCurrentException(__PRETTY_FUNCTION__);
			queue_updating_event->tryWait(ERROR_SLEEP_MS);
		}
M
Merge  
Michael Kolupaev 已提交
1298 1299 1300
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);
M
Merge  
Michael Kolupaev 已提交
1301 1302
			queue_updating_event->tryWait(ERROR_SLEEP_MS);
		}
M
Merge  
Michael Kolupaev 已提交
1303
	}
M
Merge  
Michael Kolupaev 已提交
1304

1305
	LOG_DEBUG(log, "Queue updating thread finished");
M
Merge  
Michael Kolupaev 已提交
1306
}
M
Merge  
Michael Kolupaev 已提交
1307

A
Merge  
Alexey Milovidov 已提交
1308

M
Merge  
Michael Kolupaev 已提交
1309
bool StorageReplicatedMergeTree::queueTask(BackgroundProcessingPool::Context & pool_context)
M
Merge  
Michael Kolupaev 已提交
1310
{
M
Merge  
Michael Kolupaev 已提交
1311
	LogEntryPtr entry;
M
Merge  
Michael Kolupaev 已提交
1312

M
Merge  
Michael Kolupaev 已提交
1313 1314
	try
	{
A
Alexey Milovidov 已提交
1315
		entry = queue.selectEntryToProcess(merger);
M
Merge  
Michael Kolupaev 已提交
1316 1317 1318 1319 1320
	}
	catch (...)
	{
		tryLogCurrentException(__PRETTY_FUNCTION__);
	}
M
Merge  
Michael Kolupaev 已提交
1321

M
Merge  
Michael Kolupaev 已提交
1322
	if (!entry)
M
Merge  
Michael Kolupaev 已提交
1323
		return false;
M
Merge  
Michael Kolupaev 已提交
1324

1325 1326
	time_t prev_attempt_time = entry->last_attempt_time;

A
Alexey Milovidov 已提交
1327
	bool res = queue.processEntry(getZooKeeper(), entry, [&](LogEntryPtr & entry)
M
Merge  
Michael Kolupaev 已提交
1328
	{
1329
		try
M
Merge  
Michael Kolupaev 已提交
1330
		{
A
Alexey Milovidov 已提交
1331 1332 1333 1334 1335
			return executeLogEntry(*entry, pool_context);
		}
		catch (const Exception & e)
		{
			if (e.code() == ErrorCodes::NO_REPLICA_HAS_PART)
1336
			{
A
Alexey Milovidov 已提交
1337 1338
				/// Если ни у кого нет нужного куска, наверно, просто не все реплики работают; не будем писать в лог с уровнем Error.
				LOG_INFO(log, e.displayText());
1339
			}
A
Alexey Milovidov 已提交
1340 1341 1342 1343 1344 1345 1346
			else if (e.code() == ErrorCodes::ABORTED)
			{
				/// Прерванный мердж или скачивание куска - не ошибка.
				LOG_INFO(log, e.message());
			}
			else
				tryLogCurrentException(__PRETTY_FUNCTION__);
1347

1348 1349 1350 1351
			/** Это исключение будет записано в элемент очереди, и его можно будет посмотреть с помощью таблицы system.replication_queue.
			  * Поток, выполняющий это действие, будет спать несколько секунд после исключения.
			  * См. функцию queue.processEntry.
			  */
1352
			throw;
1353 1354 1355
		}
		catch (...)
		{
M
Merge  
Michael Kolupaev 已提交
1356
			tryLogCurrentException(__PRETTY_FUNCTION__);
1357
			throw;
M
Merge  
Michael Kolupaev 已提交
1358
		}
A
Alexey Milovidov 已提交
1359
	});
M
Merge  
Michael Kolupaev 已提交
1360

1361 1362 1363
	/// Будем спать, если обработка прошла неуспешно и если мы недавно уже обрабатывали эту запись.
	bool need_sleep = !res && (entry->last_attempt_time - prev_attempt_time < 10);

M
Merge  
Michael Kolupaev 已提交
1364
	/// Если не было исключения, не нужно спать.
1365
	return !need_sleep;
M
Merge  
Michael Kolupaev 已提交
1366 1367
}

A
Merge  
Alexey Milovidov 已提交
1368

M
Merge  
Michael Kolupaev 已提交
1369 1370
void StorageReplicatedMergeTree::mergeSelectingThread()
{
1371 1372
	setThreadName("ReplMTMergeSel");

M
Merge  
Michael Kolupaev 已提交
1373
	bool need_pull = true;
M
Merge  
Michael Kolupaev 已提交
1374

A
Merge  
Alexey Milovidov 已提交
1375 1376 1377 1378 1379 1380 1381
	/** Может много времени тратиться на определение, можно ли мерджить два рядом стоящих куска.
	  * Два рядом стоящих куска можно мерджить, если все номера блоков между их номерами не используются ("заброшены", abandoned).
	  * Это значит, что между этими кусками не может быть вставлен другой кусок.
	  *
	  * Но если номера соседних блоков отличаются достаточно сильно (обычно, если между ними много "заброшенных" блоков),
	  *  то делается слишком много чтений из ZooKeeper, чтобы узнать, можно ли их мерджить.
	  *
1382 1383
	  * Воспользуемся утверждением, что если пару кусков было можно мерджить, и их мердж ещё не запланирован,
	  *  то и сейчас их можно мерджить, и будем запоминать это состояние, чтобы не делать много раз одинаковые запросы в ZooKeeper.
A
Merge  
Alexey Milovidov 已提交
1384 1385 1386 1387 1388 1389 1390 1391
	  *
	  * 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
	{
1392
		/// Если какой-то из кусков уже собираются слить в больший, не соглашаемся его сливать.
A
Alexey Milovidov 已提交
1393
		if (queue.partWillBeMergedOrMergesDisabled(left->name) || queue.partWillBeMergedOrMergesDisabled(right->name))
1394 1395
			return false;

A
Merge  
Alexey Milovidov 已提交
1396 1397 1398 1399
		auto key = std::make_pair(left->name, right->name);
		if (memoized_parts_that_could_be_merged.count(key))
			return true;

1400
		String month_name = left->name.substr(0, 6);
A
Merge  
Alexey Milovidov 已提交
1401
		auto zookeeper = getZooKeeper();
1402

1403 1404 1405 1406 1407 1408 1409 1410 1411 1412 1413 1414 1415 1416 1417 1418 1419 1420
		/// Нельзя сливать куски, среди которых находится кусок, для которого неудовлетворён кворум.
		/// Замечание: теоретически, это можно было бы разрешить. Но это сделает логику более сложной.
		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;
		}

1421
		/// Можно слить куски, если все номера между ними заброшены - не соответствуют никаким блокам.
A
Merge  
Alexey Milovidov 已提交
1422 1423
		/// Номера до RESERVED_BLOCK_NUMBERS всегда не соответствуют никаким блокам.
		for (Int64 number = std::max(RESERVED_BLOCK_NUMBERS, left->right + 1); number <= right->left - 1; ++number)
1424 1425 1426
		{
			String path1 = zookeeper_path +              "/block_numbers/" + month_name + "/block-" + padIndex(number);
			String path2 = zookeeper_path + "/nonincrement_block_numbers/" + month_name + "/block-" + padIndex(number);
A
Merge  
Alexey Milovidov 已提交
1427

1428 1429 1430 1431 1432 1433 1434
			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;
A
Merge  
Alexey Milovidov 已提交
1435 1436
	};

M
Merge  
Michael Kolupaev 已提交
1437 1438
	while (!shutdown_called && is_leader_node)
	{
M
Michael Kolupaev 已提交
1439
		bool success = false;
M
Merge  
Michael Kolupaev 已提交
1440

M
Michael Kolupaev 已提交
1441
		try
M
Merge  
Michael Kolupaev 已提交
1442
		{
M
Merge  
Michael Kolupaev 已提交
1443 1444
			if (need_pull)
			{
A
Alexey Milovidov 已提交
1445 1446
				/// Нужно загрузить новые записи в очередь перед тем, как выбирать куски для слияния.
				///  (чтобы мы знали, какие куски уже собираются сливать).
M
Merge  
Michael Kolupaev 已提交
1447 1448 1449 1450
				pullLogsToQueue();
				need_pull = false;
			}

A
Alexey Milovidov 已提交
1451 1452
			std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);

1453 1454 1455 1456 1457 1458 1459
			/** Сколько в очереди или в фоновом потоке мерджей крупных кусков.
			  * Если их больше половины от размера пула потоков для мерджа, то можно мерджить только мелкие куски.
			  */
			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;
M
Michael Kolupaev 已提交
1460
			size_t merges_queued = 0;
1461
			size_t big_merges_queued = 0;
M
Merge  
Michael Kolupaev 已提交
1462

1463
			if (big_merges_current < max_number_of_big_merges)
M
Michael Kolupaev 已提交
1464
			{
A
Alexey Milovidov 已提交
1465 1466
				queue.countMerges(merges_queued, big_merges_queued, max_number_of_big_merges - big_merges_current,
					[&](const String & name)
M
Merge  
Michael Kolupaev 已提交
1467
					{
A
Alexey Milovidov 已提交
1468 1469 1470
						MergeTreeData::DataPartPtr part = data.getActiveContainingPart(name);
						if (!part || part->name != name)
							return false;
M
Merge  
Michael Kolupaev 已提交
1471

A
Alexey Milovidov 已提交
1472 1473
						return part->size_in_bytes > data.settings.max_bytes_to_merge_parts_small;
					});
M
Michael Kolupaev 已提交
1474
			}
M
Merge  
Michael Kolupaev 已提交
1475

1476 1477
			bool only_small = big_merges_current + big_merges_queued >= max_number_of_big_merges;

1478 1479 1480 1481 1482 1483
			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." : "."));
1484

M
Merge  
Michael Kolupaev 已提交
1485
			do
M
Michael Kolupaev 已提交
1486
			{
M
Merge  
Michael Kolupaev 已提交
1487
				if (merges_queued >= data.settings.max_replicated_merges_in_queue)
1488
				{
1489 1490 1491
					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.");
M
Merge  
Michael Kolupaev 已提交
1492
					break;
1493
				}
M
Merge  
Michael Kolupaev 已提交
1494

M
Merge  
Michael Kolupaev 已提交
1495
				MergeTreeData::DataPartsVector parts;
M
Merge  
Michael Kolupaev 已提交
1496

M
Merge  
Michael Kolupaev 已提交
1497 1498
				String merged_name;

A
Merge  
Alexey Milovidov 已提交
1499 1500 1501 1502
				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))
1503
				{
M
Merge  
Michael Kolupaev 已提交
1504
					break;
1505
				}
M
Merge  
Michael Kolupaev 已提交
1506

A
Alexey Milovidov 已提交
1507 1508
				auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
1509 1510 1511 1512 1513 1514 1515
				bool all_in_zk = true;
				for (const auto & part : parts)
				{
					/// Если о каком-то из кусков нет информации в ZK, не будем сливать.
					if (!zookeeper->exists(replica_path + "/parts/" + part->name))
					{
						all_in_zk = false;
1516 1517 1518 1519 1520 1521 1522 1523 1524

						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);
						}
M
Merge  
Michael Kolupaev 已提交
1525 1526 1527 1528 1529
					}
				}
				if (!all_in_zk)
					break;

M
Merge  
Michael Kolupaev 已提交
1530 1531 1532 1533
				LogEntry entry;
				entry.type = LogEntry::MERGE_PARTS;
				entry.source_replica = replica_name;
				entry.new_part_name = merged_name;
1534
				entry.create_time = time(0);
M
Merge  
Michael Kolupaev 已提交
1535

M
Merge  
Michael Kolupaev 已提交
1536 1537
				for (const auto & part : parts)
					entry.parts_to_merge.push_back(part->name);
M
Merge  
Michael Kolupaev 已提交
1538

M
Merge  
Michael Kolupaev 已提交
1539
				need_pull = true;
M
Merge  
Michael Kolupaev 已提交
1540

M
Merge  
Michael Kolupaev 已提交
1541
				zookeeper->create(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential);
M
Merge  
Michael Kolupaev 已提交
1542 1543 1544

				String month_name = parts[0]->name.substr(0, 6);
				for (size_t i = 0; i + 1 < parts.size(); ++i)
M
Merge  
Michael Kolupaev 已提交
1545
				{
M
Merge  
Michael Kolupaev 已提交
1546
					/// Уберем больше не нужные отметки о несуществующих блоках.
A
Merge  
Alexey Milovidov 已提交
1547
					for (Int64 number = std::max(RESERVED_BLOCK_NUMBERS, parts[i]->right + 1); number <= parts[i + 1]->left - 1; ++number)
M
Merge  
Michael Kolupaev 已提交
1548
					{
M
Merge  
Michael Kolupaev 已提交
1549 1550
						zookeeper->tryRemove(zookeeper_path +              "/block_numbers/" + month_name + "/block-" + padIndex(number));
						zookeeper->tryRemove(zookeeper_path + "/nonincrement_block_numbers/" + month_name + "/block-" + padIndex(number));
M
Merge  
Michael Kolupaev 已提交
1551
					}
M
Merge  
Michael Kolupaev 已提交
1552
				}
M
Merge  
Michael Kolupaev 已提交
1553 1554

				success = true;
M
Merge  
Michael Kolupaev 已提交
1555
			}
1556
			while (false);
M
Merge  
Michael Kolupaev 已提交
1557 1558 1559 1560 1561 1562
		}
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);
		}

M
Merge  
Michael Kolupaev 已提交
1563
		if (shutdown_called || !is_leader_node)
M
Merge  
Michael Kolupaev 已提交
1564 1565
			break;

M
Merge  
Michael Kolupaev 已提交
1566
		if (!success)
M
Merge  
Michael Kolupaev 已提交
1567
			merge_selecting_event.tryWait(MERGE_SELECTING_SLEEP_MS);
M
Merge  
Michael Kolupaev 已提交
1568
	}
M
Merge  
Michael Kolupaev 已提交
1569

1570
	LOG_DEBUG(log, "Merge selecting thread finished");
M
Merge  
Michael Kolupaev 已提交
1571 1572
}

M
Merge  
Michael Kolupaev 已提交
1573

M
Merge  
Michael Kolupaev 已提交
1574 1575
void StorageReplicatedMergeTree::alterThread()
{
1576 1577
	setThreadName("ReplMTAlter");

M
Merge  
Michael Kolupaev 已提交
1578 1579 1580 1581 1582 1583
	bool force_recheck_parts = true;

	while (!shutdown_called)
	{
		try
		{
1584 1585 1586 1587 1588 1589 1590 1591 1592 1593 1594 1595 1596 1597 1598 1599 1600 1601 1602 1603 1604 1605 1606
			/** Имеем описание столбцов в 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: Слишком сложно, всё переделать.
			  */

A
Merge  
Alexey Milovidov 已提交
1607 1608
			auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
1609
			zkutil::Stat stat;
1610
			const String columns_str = zookeeper->get(zookeeper_path + "/columns", &stat, alter_thread_event);
1611
			auto columns_desc = ColumnsDescription<true>::parse(columns_str);
1612 1613 1614 1615 1616

			auto & columns = columns_desc.columns;
			auto & materialized_columns = columns_desc.materialized;
			auto & alias_columns = columns_desc.alias;
			auto & column_defaults = columns_desc.defaults;
M
Merge  
Michael Kolupaev 已提交
1617

1618
			bool changed_version = (stat.version != columns_version);
M
Merge  
Michael Kolupaev 已提交
1619 1620

			{
1621 1622 1623
				/// Если потребуется блокировать структуру таблицы, то приостановим мерджи.
				std::unique_ptr<MergeTreeMergeBlocker> merge_blocker;
				std::unique_ptr<MergeTreeMergeBlocker> unreplicated_merge_blocker;
1624

1625
				if (changed_version || force_recheck_parts)
M
Merge  
Michael Kolupaev 已提交
1626
				{
1627 1628 1629 1630
					merge_blocker = std::make_unique<MergeTreeMergeBlocker>(merger);
					if (unreplicated_merger)
						unreplicated_merge_blocker = std::make_unique<MergeTreeMergeBlocker>(*unreplicated_merger);
				}
1631

1632
				MergeTreeData::DataParts parts;
1633

1634 1635 1636 1637
				/// Если описание столбцов изменилось, обновим структуру таблицы локально.
				if (changed_version)
				{
					LOG_INFO(log, "Changed version of 'columns' node in ZooKeeper. Waiting for structure write lock.");
1638

1639
					auto table_lock = lockStructureForAlter();
1640

1641 1642 1643 1644
					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;
					const auto column_defaults_changed = column_defaults != data.column_defaults;
1645

1646 1647
					if (columns_changed || materialized_columns_changed || alias_columns_changed ||
						column_defaults_changed)
1648
					{
1649
						LOG_INFO(log, "Columns list changed in ZooKeeper. Applying changes locally.");
1650

1651 1652
						InterpreterAlterQuery::updateMetadata(database_name, table_name, columns,
							materialized_columns, alias_columns, column_defaults, context);
1653

1654 1655 1656
						if (columns_changed)
						{
							data.setColumnsList(columns);
1657

1658 1659 1660
							if (unreplicated_data)
								unreplicated_data->setColumnsList(columns);
						}
1661

1662 1663 1664 1665 1666
						if (materialized_columns_changed)
						{
							this->materialized_columns = materialized_columns;
							data.materialized_columns = std::move(materialized_columns);
						}
M
Merge  
Michael Kolupaev 已提交
1667

1668 1669 1670 1671 1672
						if (alias_columns_changed)
						{
							this->alias_columns = alias_columns;
							data.alias_columns = std::move(alias_columns);
						}
1673

1674 1675 1676 1677 1678
						if (column_defaults_changed)
						{
							this->column_defaults = column_defaults;
							data.column_defaults = std::move(column_defaults);
						}
M
Merge  
Michael Kolupaev 已提交
1679

1680 1681 1682 1683 1684 1685
						LOG_INFO(log, "Applied changes to table.");
					}
					else
					{
						LOG_INFO(log, "Columns version changed in ZooKeeper, but data wasn't changed. It's like cyclic ALTERs.");
					}
M
Merge  
Michael Kolupaev 已提交
1686

1687
					/// Нужно получить список кусков под блокировкой таблицы, чтобы избежать race condition с мерджем.
M
Merge  
Michael Kolupaev 已提交
1688 1689
					parts = data.getDataParts();

1690 1691
					columns_version = stat.version;
				}
1692

1693 1694
				/// Обновим куски.
				if (changed_version || force_recheck_parts)
M
Merge  
Michael Kolupaev 已提交
1695
				{
1696
					auto table_lock = lockStructure(false);
M
Merge  
Michael Kolupaev 已提交
1697

1698 1699
					if (changed_version)
						LOG_INFO(log, "ALTER-ing parts");
M
Merge  
Michael Kolupaev 已提交
1700

1701
					int changed_parts = 0;
M
Merge  
Michael Kolupaev 已提交
1702

1703 1704
					if (!changed_version)
						parts = data.getDataParts();
M
Merge  
Michael Kolupaev 已提交
1705

1706
					const auto columns_plus_materialized = data.getColumnsList();
M
Merge  
Michael Kolupaev 已提交
1707 1708 1709

					for (const MergeTreeData::DataPartPtr & part : parts)
					{
1710 1711 1712 1713
						/// Обновим кусок и запишем результат во временные файлы.
						/// TODO: Можно пропускать проверку на слишком большие изменения, если в ZooKeeper есть, например,
						///  нода /flags/force_alter.
						auto transaction = data.alterDataPart(part, columns_plus_materialized);
M
Merge  
Michael Kolupaev 已提交
1714 1715 1716 1717 1718 1719

						if (!transaction)
							continue;

						++changed_parts;

1720 1721 1722 1723 1724 1725 1726 1727 1728
						/// Обновим метаданные куска в ZooKeeper.
						zkutil::Ops ops;
						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));
						zookeeper->multi(ops);

						/// Применим изменения файлов.
M
Merge  
Michael Kolupaev 已提交
1729 1730
						transaction->commit();
					}
M
Merge  
Michael Kolupaev 已提交
1731

1732 1733 1734 1735
					/// То же самое для нереплицируемых данных.
					if (unreplicated_data)
					{
						parts = unreplicated_data->getDataParts();
M
Merge  
Michael Kolupaev 已提交
1736

1737 1738 1739 1740 1741 1742 1743 1744 1745 1746 1747 1748 1749 1750 1751 1752 1753 1754 1755 1756 1757 1758 1759 1760 1761
						for (const MergeTreeData::DataPartPtr & part : parts)
						{
							auto transaction = unreplicated_data->alterDataPart(part, columns_plus_materialized);

							if (!transaction)
								continue;

							++changed_parts;

							transaction->commit();
						}
					}

					/// Список столбцов для конкретной реплики.
					zookeeper->set(replica_path + "/columns", columns_str);

					if (changed_version)
					{
						if (changed_parts != 0)
							LOG_INFO(log, "ALTER-ed " << changed_parts << " parts");
						else
							LOG_INFO(log, "No parts ALTER-ed");
					}

					force_recheck_parts = false;
1762 1763
				}

1764
				/// Важно, что уничтожается parts и merge_blocker перед wait-ом.
M
Merge  
Michael Kolupaev 已提交
1765 1766
			}

1767
			alter_thread_event->tryWait(WAIT_FOR_ALTER_SLEEP_MS);
M
Merge  
Michael Kolupaev 已提交
1768 1769 1770 1771 1772 1773 1774 1775 1776 1777 1778
		}
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);

			force_recheck_parts = true;

			alter_thread_event->tryWait(ERROR_SLEEP_MS);
		}
	}

1779
	LOG_DEBUG(log, "Alter thread finished");
M
Merge  
Michael Kolupaev 已提交
1780 1781
}

A
Merge  
Alexey Milovidov 已提交
1782

1783 1784 1785 1786 1787 1788 1789 1790 1791 1792
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));
}


M
Merge  
Michael Kolupaev 已提交
1793 1794
void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_name)
{
A
Merge  
Alexey Milovidov 已提交
1795 1796
	auto zookeeper = getZooKeeper();

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

M
Merge  
Michael Kolupaev 已提交
1799 1800
	LogEntryPtr log_entry = new LogEntry;
	log_entry->type = LogEntry::GET_PART;
1801
	log_entry->create_time = tryGetPartCreateTime(zookeeper, replica_path, part_name);
M
Merge  
Michael Kolupaev 已提交
1802 1803
	log_entry->source_replica = "";
	log_entry->new_part_name = part_name;
M
Merge  
Michael Kolupaev 已提交
1804 1805 1806

	zkutil::Ops ops;
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
1807
		replica_path + "/queue/queue-", log_entry->toString(), zookeeper->getDefaultACL(),
M
Merge  
Michael Kolupaev 已提交
1808
		zkutil::CreateMode::PersistentSequential));
1809 1810 1811

	removePartFromZooKeeper(part_name, ops);

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

A
Alexey Milovidov 已提交
1814 1815
	String path_created = dynamic_cast<zkutil::Op::Create &>(ops[0]).getPathCreated();
	log_entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
1816
	queue.insert(zookeeper, log_entry);
M
Merge  
Michael Kolupaev 已提交
1817 1818
}

A
Merge  
Alexey Milovidov 已提交
1819

M
Merge  
Michael Kolupaev 已提交
1820 1821
void StorageReplicatedMergeTree::enqueuePartForCheck(const String & name)
{
1822
	std::lock_guard<std::mutex> lock(parts_to_check_mutex);
M
Merge  
Michael Kolupaev 已提交
1823 1824 1825 1826 1827 1828 1829 1830

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

1831 1832

void StorageReplicatedMergeTree::searchForMissingPart(const String & part_name)
M
Merge  
Michael Kolupaev 已提交
1833
{
1834 1835 1836 1837 1838
	auto zookeeper = getZooKeeper();
	String part_path = replica_path + "/parts/" + part_name;

	/// Если кусок есть в ZooKeeper, удалим его оттуда и добавим в очередь задание скачать его.
	if (zookeeper->exists(part_path))
M
Merge  
Michael Kolupaev 已提交
1839
	{
1840 1841 1842
		LOG_WARNING(log, "Checker: Part " << part_name << " exists in ZooKeeper but not locally. "
			"Removing from ZooKeeper and queueing a fetch.");
		ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
A
Merge  
Alexey Milovidov 已提交
1843

1844 1845 1846 1847 1848 1849 1850 1851 1852 1853 1854 1855 1856 1857 1858 1859 1860 1861 1862 1863 1864 1865 1866 1867 1868 1869 1870 1871 1872 1873 1874 1875
		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))
M
Merge  
Michael Kolupaev 已提交
1876
			{
1877 1878 1879
				found = true;
				LOG_WARNING(log, "Checker: Found part " << part_on_replica << " on " << replica);
				break;
M
Merge  
Michael Kolupaev 已提交
1880
			}
1881 1882

			if (ActiveDataPartSet::contains(part_name, part_on_replica))
M
Merge  
Michael Kolupaev 已提交
1883
			{
1884 1885 1886
				ActiveDataPartSet::Part part_on_replica_info;
				ActiveDataPartSet::parsePartName(part_on_replica, part_on_replica_info);

1887
				for (auto block_num = part_on_replica_info.left; block_num <= part_on_replica_info.right; ++block_num)
A
Alexey Milovidov 已提交
1888
					found_blocks.at(block_num - part_info.left) = 1;
M
Merge  
Michael Kolupaev 已提交
1889
			}
1890 1891 1892 1893
		}
		if (found)
			break;
	}
M
Merge  
Michael Kolupaev 已提交
1894

1895 1896 1897 1898 1899
	if (found)
	{
		/// На какой-то живой или мёртвой реплике есть нужный кусок или покрывающий его.
		return;
	}
M
Merge  
Michael Kolupaev 已提交
1900

1901 1902 1903
	size_t num_found_blocks = 0;
	for (auto found_block : found_blocks)
		num_found_blocks += (found_block == 1);
M
Merge  
Michael Kolupaev 已提交
1904

1905 1906 1907 1908 1909 1910
	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;
	}
1911

1912 1913
	/// Ни у кого нет такого куска.
	LOG_ERROR(log, "Checker: No replica has part covering " << part_name);
M
Merge  
Michael Kolupaev 已提交
1914

1915 1916 1917
	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.");
M
Merge  
Michael Kolupaev 已提交
1918

1919
	ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
1920

1921
	/// Есть ли он в очереди репликации? Если есть - удалим, так как задачу невозможно обработать.
A
Alexey Milovidov 已提交
1922
	if (!queue.remove(zookeeper, part_name))
1923 1924 1925 1926 1927
	{
		/// Куска не было в нашей очереди. С чего бы это?
		LOG_ERROR(log, "Checker: Missing part " << part_name << " is not in our queue.");
		return;
	}
M
Merge  
Michael Kolupaev 已提交
1928

1929 1930 1931 1932 1933 1934 1935 1936 1937 1938 1939 1940 1941
	/** Такая ситуация возможна, если на всех репликах, где был кусок, он испортился.
		* Например, у реплики, которая только что его записала, отключили питание, и данные не записались из кеша на диск.
		*/
	LOG_ERROR(log, "Checker: Part " << part_name << " is lost forever.");
	ProfileEvents::increment(ProfileEvents::ReplicatedDataLoss);

	/** Нужно добавить отсутствующий кусок в block_numbers, чтобы он не мешал слияниям.
		* Вот только в сам block_numbers мы его добавить не можем - если так сделать,
		*  ZooKeeper зачем-то пропустит один номер для автоинкремента,
		*  и в номерах блоков все равно останется дырка.
		* Специально из-за этого приходится отдельно иметь nonincrement_block_numbers.
		*
		* Кстати, если мы здесь сдохнем, то слияния не будут делаться сквозь эти отсутствующие куски.
1942 1943 1944 1945 1946
		*
		* А ещё, не будем добавлять, если:
		* - потребовалось бы создать слишком много (больше 1000) узлов;
		* - кусок является первым в партиции или был при-ATTACH-ен.
		* NOTE Возможно, добавить также условие, если запись в очереди очень старая.
1947
		*/
1948 1949 1950 1951 1952 1953 1954 1955 1956 1957 1958 1959 1960

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

1961 1962 1963 1964 1965 1966 1967 1968 1969
	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/" + partition_str, "");
		AbandonableLockInZooKeeper::createAbandonedIfNotExists(
			zookeeper_path + "/nonincrement_block_numbers/" + partition_str + "/block-" + padIndex(i),
			*zookeeper);
	}
}
M
Merge  
Michael Kolupaev 已提交
1970 1971


1972 1973 1974 1975
void StorageReplicatedMergeTree::checkPart(const String & part_name)
{
	LOG_WARNING(log, "Checker: Checking part " << part_name);
	ProfileEvents::increment(ProfileEvents::ReplicatedPartChecks);
M
Merge  
Michael Kolupaev 已提交
1976

1977
	auto part = data.getActiveContainingPart(part_name);
M
Merge  
Michael Kolupaev 已提交
1978

1979 1980 1981 1982 1983 1984 1985 1986 1987 1988
	/// Этого или покрывающего куска у нас нет.
	if (!part)
	{
		searchForMissingPart(part_name);
	}
	/// У нас есть этот кусок, и он активен. Будем проверять, нужен ли нам этот кусок и правильные ли у него данные.
	else if (part->name == part_name)
	{
		auto zookeeper = getZooKeeper();
		auto table_lock = lockStructure(false);
M
Merge  
Michael Kolupaev 已提交
1989

1990 1991 1992 1993 1994 1995 1996 1997 1998 1999 2000 2001 2002 2003 2004 2005 2006 2007 2008 2009 2010
		/// Если кусок есть в 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(
2011
					data.getFullPath() + part_name, settings, data.primary_key_data_types);
2012 2013

				LOG_INFO(log, "Checker: Part " << part_name << " looks good.");
M
Merge  
Michael Kolupaev 已提交
2014
			}
2015
			catch (...)
M
Merge  
Michael Kolupaev 已提交
2016
			{
2017
				tryLogCurrentException(__PRETTY_FUNCTION__);
M
Merge  
Michael Kolupaev 已提交
2018

2019 2020
				LOG_ERROR(log, "Checker: Part " << part_name << " looks broken. Removing it and queueing a fetch.");
				ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
M
Merge  
Michael Kolupaev 已提交
2021

2022
				removePartAndEnqueueFetch(part_name);
M
Merge  
Michael Kolupaev 已提交
2023

2024 2025 2026 2027
				/// Удалим кусок локально.
				data.renameAndDetachPart(part, "broken_");
			}
		}
2028
		else if (part->modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER < time(0))
2029 2030 2031 2032 2033 2034
		{
			/// Если куска нет в ZooKeeper, удалим его локально.
			/// Возможно, кусок кто-то только что записал, и еще не успел добавить в ZK.
			/// Поэтому удаляем только если кусок старый (не очень надежно).
			ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);

2035
			LOG_ERROR(log, "Checker: Unexpected part " << part_name << " in filesystem. Removing.");
2036 2037
			data.renameAndDetachPart(part, "unexpected_");
		}
2038 2039 2040 2041 2042 2043
		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.");
		}
2044 2045 2046 2047 2048 2049 2050 2051
	}
	else
	{
		/// Если у нас есть покрывающий кусок, игнорируем все проблемы с этим куском.
		/// В худшем случае в лог еще old_parts_lifetime секунд будут валиться ошибки, пока кусок не удалится как старый.
		LOG_WARNING(log, "Checker: We have part " << part->name << " covering part " << part_name);
	}
}
M
Merge  
Michael Kolupaev 已提交
2052 2053


2054 2055
void StorageReplicatedMergeTree::partCheckThread()
{
2056 2057
	setThreadName("ReplMTPartCheck");

2058 2059 2060 2061 2062 2063 2064 2065 2066 2067 2068 2069 2070 2071
	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();
M
Merge  
Michael Kolupaev 已提交
2072
					}
M
Merge  
Michael Kolupaev 已提交
2073
				}
2074
				else
M
Merge  
Michael Kolupaev 已提交
2075
				{
2076
					part_name = parts_to_check_queue.front();
M
Merge  
Michael Kolupaev 已提交
2077 2078
				}
			}
2079 2080

			if (part_name.empty())
M
Merge  
Michael Kolupaev 已提交
2081
			{
2082 2083
				parts_to_check_event.wait();
				continue;
M
Merge  
Michael Kolupaev 已提交
2084 2085
			}

2086 2087 2088
			checkPart(part_name);

			/// Удалим кусок из очереди проверок.
M
Merge  
Michael Kolupaev 已提交
2089
			{
2090
				std::lock_guard<std::mutex> lock(parts_to_check_mutex);
M
Merge  
Michael Kolupaev 已提交
2091 2092
				if (parts_to_check_queue.empty() || parts_to_check_queue.front() != part_name)
				{
2093
					LOG_ERROR(log, "Checker: Someone changed parts_to_check_queue.front(). This is a bug.");
M
Merge  
Michael Kolupaev 已提交
2094 2095 2096 2097 2098 2099 2100 2101 2102 2103 2104 2105 2106 2107 2108 2109 2110
				}
				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);
		}
	}
}


M
Merge  
Michael Kolupaev 已提交
2111 2112
void StorageReplicatedMergeTree::becomeLeader()
{
M
Merge  
Michael Kolupaev 已提交
2113
	LOG_INFO(log, "Became leader");
M
Merge  
Michael Kolupaev 已提交
2114 2115 2116 2117
	is_leader_node = true;
	merge_selecting_thread = std::thread(&StorageReplicatedMergeTree::mergeSelectingThread, this);
}

A
Merge  
Alexey Milovidov 已提交
2118

M
Merge  
Michael Kolupaev 已提交
2119
String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_name, bool active)
M
Merge  
Michael Kolupaev 已提交
2120
{
A
Merge  
Alexey Milovidov 已提交
2121
	auto zookeeper = getZooKeeper();
M
Merge  
Michael Kolupaev 已提交
2122
	Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
M
Merge  
Michael Kolupaev 已提交
2123 2124 2125 2126 2127 2128

	/// Из реплик, у которых есть кусок, выберем одну равновероятно.
	std::random_shuffle(replicas.begin(), replicas.end());

	for (const String & replica : replicas)
	{
M
Merge  
Michael Kolupaev 已提交
2129 2130
		if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name) &&
			(!active || zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")))
M
Merge  
Michael Kolupaev 已提交
2131
			return replica;
2132 2133

		/// Конечно, реплика может перестать быть активной или даже перестать существовать после возврата из этой функции.
M
Merge  
Michael Kolupaev 已提交
2134 2135
	}

M
Merge  
Michael Kolupaev 已提交
2136
	return "";
M
Merge  
Michael Kolupaev 已提交
2137 2138
}

A
Merge  
Alexey Milovidov 已提交
2139

2140 2141
/** Если для куска отслеживается кворум, то обновить информацию о нём в ZK.
  */
2142
void StorageReplicatedMergeTree::updateQuorum(const String & part_name)
2143
{
2144
	auto zookeeper = getZooKeeper();
2145

2146
	/// Информация, на какие реплики был добавлен кусок, если кворум ещё не достигнут.
2147
	const String quorum_status_path = zookeeper_path + "/quorum/status";
2148 2149 2150
	/// Имя предыдущего куска, для которого был достигнут кворум.
	const String quorum_last_part_path = zookeeper_path + "/quorum/last_part";

2151 2152 2153 2154 2155 2156 2157 2158 2159 2160 2161 2162 2163 2164 2165 2166 2167 2168 2169
	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)
		{
2170 2171 2172 2173 2174 2175
			/// Кворум достигнут. Удаляем узел, а также обновляем информацию о последнем куске, который был успешно записан с кворумом.

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

2177 2178 2179 2180 2181
			if (code == ZOK)
			{
				break;
			}
			else if (code == ZNONODE)
2182 2183 2184 2185 2186 2187 2188 2189 2190 2191 2192 2193 2194 2195 2196 2197 2198
			{
				/// Кворум уже был достигнут.
				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);

2199 2200 2201 2202 2203
			if (code == ZOK)
			{
				break;
			}
			else if (code == ZNONODE)
2204 2205 2206 2207 2208 2209 2210 2211 2212 2213 2214 2215 2216 2217 2218 2219 2220
			{
				/// Кворум уже был достигнут.
				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)
M
Merge  
Michael Kolupaev 已提交
2221
{
A
Merge  
Alexey Milovidov 已提交
2222 2223
	auto zookeeper = getZooKeeper();

2224
	LOG_DEBUG(log, "Fetching part " << part_name << " from " << replica_path);
M
Merge  
Michael Kolupaev 已提交
2225

2226 2227 2228
	TableStructureReadLockPtr table_lock;
	if (!to_detached)
		table_lock = lockStructure(true);
M
Merge  
Michael Kolupaev 已提交
2229

2230
	ReplicatedMergeTreeAddress address(zookeeper->get(replica_path + "/host"));
M
Merge  
Michael Kolupaev 已提交
2231

2232
	MergeTreeData::MutableDataPartPtr part = fetcher.fetchPart(part_name, replica_path, address.host, address.replication_port, to_detached);
M
Merge  
Michael Kolupaev 已提交
2233

2234 2235 2236 2237
	if (!to_detached)
	{
		zkutil::Ops ops;
		checkPartAndAddToZooKeeper(part, ops, part_name);
M
Merge  
Michael Kolupaev 已提交
2238

2239 2240
		MergeTreeData::Transaction transaction;
		auto removed_parts = data.renameTempPartAndReplace(part, nullptr, &transaction);
M
Merge  
Michael Kolupaev 已提交
2241

2242 2243
		zookeeper->multi(ops);
		transaction.commit();
2244 2245

		/** Если для этого куска отслеживается кворум, то надо его обновить.
2246
		  * Если не успеем, в случае потери сессии, при перезапуске сервера - см. метод ReplicatedMergeTreeRestartingThread::updateQuorumIfWeHavePart.
2247
		  */
2248 2249
		if (quorum)
			updateQuorum(part_name);
2250

2251
		merge_selecting_event.set();
M
Merge  
Michael Kolupaev 已提交
2252

2253 2254 2255 2256 2257 2258 2259
		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
M
Michael Kolupaev 已提交
2260
	{
2261
		Poco::File(data.getFullPath() + "detached/tmp_" + part_name).renameTo(data.getFullPath() + "detached/" + part_name);
M
Michael Kolupaev 已提交
2262 2263
	}

M
Merge  
Michael Kolupaev 已提交
2264 2265
	ProfileEvents::increment(ProfileEvents::ReplicatedPartFetches);

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

A
Merge  
Alexey Milovidov 已提交
2269

M
Merge  
Michael Kolupaev 已提交
2270 2271
void StorageReplicatedMergeTree::shutdown()
{
2272
	if (restarting_thread)
M
Merge  
Michael Kolupaev 已提交
2273
	{
2274 2275
		restarting_thread->stop();
		restarting_thread.reset();
M
Merge  
Michael Kolupaev 已提交
2276
	}
M
Merge  
Michael Kolupaev 已提交
2277 2278

	endpoint_holder = nullptr;
2279
	fetcher.cancel();
M
Merge  
Michael Kolupaev 已提交
2280 2281 2282
}


M
Merge  
Michael Kolupaev 已提交
2283 2284 2285 2286 2287 2288 2289 2290
StorageReplicatedMergeTree::~StorageReplicatedMergeTree()
{
	try
	{
		shutdown();
	}
	catch(...)
	{
2291
		tryLogCurrentException(__PRETTY_FUNCTION__);
M
Merge  
Michael Kolupaev 已提交
2292 2293 2294
	}
}

A
Merge  
Alexey Milovidov 已提交
2295

M
Merge  
Michael Kolupaev 已提交
2296
BlockInputStreams StorageReplicatedMergeTree::read(
2297 2298 2299 2300 2301 2302 2303
	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)
M
Merge  
Michael Kolupaev 已提交
2304
{
2305 2306 2307 2308 2309 2310 2311 2312
	/** У таблицы может быть два вида данных:
	  * - реплицируемые данные;
	  * - старые, нереплицируемые данные - они лежат отдельно и их целостность никак не контролируется.
	  * А ещё движок таблицы предоставляет возможность использовать "виртуальные столбцы".
	  * Один из них - _replicated позволяет определить, из какой части прочитаны данные,
	  *  или, при использовании в WHERE - выбрать данные только из одной части.
	  */

M
Merge  
Michael Kolupaev 已提交
2313 2314 2315 2316 2317 2318 2319 2320
	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);

2321
	auto & select = typeid_cast<const ASTSelectQuery &>(*query);
A
Merge  
Andrey Mironov 已提交
2322 2323 2324 2325

	/// 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)
2326
			MergeTreeWhereOptimizer{query, context, data, real_column_names, log};
A
Merge  
Andrey Mironov 已提交
2327

M
Merge  
Michael Kolupaev 已提交
2328 2329 2330 2331 2332
	Block virtual_columns_block;
	ColumnUInt8 * column = new ColumnUInt8(2);
	ColumnPtr column_ptr = column;
	column->getData()[0] = 0;
	column->getData()[1] = 1;
2333
	virtual_columns_block.insert(ColumnWithTypeAndName(column_ptr, new DataTypeUInt8, "_replicated"));
M
Merge  
Michael Kolupaev 已提交
2334

2335
	/// Если запрошен столбец _replicated, пробуем индексировать.
M
Merge  
Michael Kolupaev 已提交
2336
	if (!virt_column_names.empty())
2337
		VirtualColumnUtils::filterBlockWithQuery(query, virtual_columns_block, context);
M
Merge  
Michael Kolupaev 已提交
2338

2339
	std::multiset<UInt8> values = VirtualColumnUtils::extractSingleValueFromBlock<UInt8>(virtual_columns_block, "_replicated");
M
Merge  
Michael Kolupaev 已提交
2340

M
Merge  
Michael Kolupaev 已提交
2341 2342
	BlockInputStreams res;

M
Merge  
Michael Kolupaev 已提交
2343 2344
	size_t part_index = 0;

2345 2346 2347 2348 2349 2350
	/** Настройки parallel_replica_offset и parallel_replicas_count позволяют читать с одной реплики одну часть данных, а с другой - другую.
	  * Для реплицируемых, данные разбиваются таким же механизмом, как работает секция SAMPLE.
	  * А для нереплицируемых данных, так как их целостность между репликами не контролируется,
	  *  с первой (settings.parallel_replica_offset == 0) реплики выбираются все данные, а с остальных - никакие.
	  */

2351
	if ((settings.parallel_replica_offset == 0) && unreplicated_reader && values.count(0))
M
Merge  
Michael Kolupaev 已提交
2352
	{
2353 2354
		res = unreplicated_reader->read(real_column_names, query,
										context, settings, processed_stage,
2355
										max_block_size, threads, &part_index, 0);
M
Merge  
Michael Kolupaev 已提交
2356 2357 2358 2359 2360 2361

		for (auto & virtual_column : virt_column_names)
		{
			if (virtual_column == "_replicated")
			{
				for (auto & stream : res)
M
Merge  
Michael Kolupaev 已提交
2362
					stream = new AddingConstColumnBlockInputStream<UInt8>(stream, new DataTypeUInt8, 0, "_replicated");
M
Merge  
Michael Kolupaev 已提交
2363 2364 2365 2366
			}
		}
	}

M
Merge  
Michael Kolupaev 已提交
2367
	if (values.count(1))
M
Merge  
Michael Kolupaev 已提交
2368
	{
2369 2370 2371 2372 2373 2374 2375 2376 2377
		/** Настройка select_sequential_consistency имеет два смысла:
		  * 1. Кидать исключение, если на реплике есть не все куски, которые были записаны на кворум остальных реплик.
		  * 2. Не читать куски, которые ещё не были записаны на кворум реплик.
		  * Для этого приходится синхронно сходить в ZooKeeper.
		  */
		Int64 max_block_number_to_read = 0;
		if (settings.select_sequential_consistency)
		{
			auto zookeeper = getZooKeeper();
A
Merge  
Alexey Milovidov 已提交
2378

2379 2380 2381 2382 2383 2384 2385 2386 2387 2388 2389 2390 2391 2392 2393 2394 2395 2396 2397 2398 2399 2400 2401 2402 2403 2404 2405 2406 2407
			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);
M
Merge  
Michael Kolupaev 已提交
2408 2409 2410 2411 2412 2413

		for (auto & virtual_column : virt_column_names)
		{
			if (virtual_column == "_replicated")
			{
				for (auto & stream : res2)
M
Merge  
Michael Kolupaev 已提交
2414
					stream = new AddingConstColumnBlockInputStream<UInt8>(stream, new DataTypeUInt8, 1, "_replicated");
M
Merge  
Michael Kolupaev 已提交
2415 2416 2417
			}
		}

M
Merge  
Michael Kolupaev 已提交
2418
		res.insert(res.end(), res2.begin(), res2.end());
M
Merge  
Michael Kolupaev 已提交
2419 2420 2421
	}

	return res;
M
Merge  
Michael Kolupaev 已提交
2422 2423
}

A
Merge  
Alexey Milovidov 已提交
2424

A
Merge  
Alexey Milovidov 已提交
2425
void StorageReplicatedMergeTree::assertNotReadonly() const
M
Merge  
Michael Kolupaev 已提交
2426
{
2427 2428
	if (is_readonly)
		throw Exception("Table is in readonly mode", ErrorCodes::TABLE_IS_READ_ONLY);
A
Merge  
Alexey Milovidov 已提交
2429 2430 2431 2432 2433 2434
}


BlockOutputStreamPtr StorageReplicatedMergeTree::write(ASTPtr query, const Settings & settings)
{
	assertNotReadonly();
M
Merge  
Michael Kolupaev 已提交
2435

M
Merge  
Michael Kolupaev 已提交
2436
	String insert_id;
M
Merge  
Michael Kolupaev 已提交
2437 2438 2439
	if (query)
		if (ASTInsertQuery * insert = typeid_cast<ASTInsertQuery *>(&*query))
			insert_id = insert->insert_id;
M
Merge  
Michael Kolupaev 已提交
2440

2441 2442
	return new ReplicatedMergeTreeBlockOutputStream(*this, insert_id,
		settings.insert_quorum, settings.insert_quorum_timeout.totalMilliseconds());
M
Merge  
Michael Kolupaev 已提交
2443
}
M
Merge  
Michael Kolupaev 已提交
2444

A
Merge  
Alexey Milovidov 已提交
2445

2446
bool StorageReplicatedMergeTree::optimize(const Settings & settings)
M
Merge  
Michael Kolupaev 已提交
2447
{
2448 2449
	/// Померджим какие-нибудь куски из директории unreplicated.
	/// TODO: Мерджить реплицируемые куски тоже.
M
Merge  
Michael Kolupaev 已提交
2450 2451 2452 2453

	if (!unreplicated_data)
		return false;

2454
	std::lock_guard<std::mutex> lock(unreplicated_mutex);
M
Merge  
Michael Kolupaev 已提交
2455

M
Merge  
Michael Kolupaev 已提交
2456 2457 2458 2459
	unreplicated_data->clearOldParts();

	MergeTreeData::DataPartsVector parts;
	String merged_name;
2460
	auto always_can_merge = [](const MergeTreeData::DataPartPtr & a, const MergeTreeData::DataPartPtr & b) { return true; };
M
Merge  
Michael Kolupaev 已提交
2461 2462 2463
	if (!unreplicated_merger->selectPartsToMerge(parts, merged_name, 0, true, true, false, always_can_merge))
		return false;

2464
	const auto & merge_entry = context.getMergeList().insert(database_name, table_name, merged_name);
2465
	unreplicated_merger->mergeParts(parts, merged_name, *merge_entry, settings.min_bytes_to_use_direct_io);
2466

M
Merge  
Michael Kolupaev 已提交
2467 2468 2469
	return true;
}

A
Merge  
Alexey Milovidov 已提交
2470

M
Merge  
Michael Kolupaev 已提交
2471
void StorageReplicatedMergeTree::alter(const AlterCommands & params,
A
Merge  
Alexey Milovidov 已提交
2472
	const String & database_name, const String & table_name, Context & context)
M
Merge  
Michael Kolupaev 已提交
2473
{
A
Merge  
Alexey Milovidov 已提交
2474 2475
	assertNotReadonly();

A
Merge  
Alexey Milovidov 已提交
2476
	auto zookeeper = getZooKeeper();
2477 2478
	const MergeTreeMergeBlocker merge_blocker{merger};
	const auto unreplicated_merge_blocker = unreplicated_merger ?
2479
		std::make_unique<MergeTreeMergeBlocker>(*unreplicated_merger) : nullptr;
A
Merge  
Alexey Milovidov 已提交
2480

M
Merge  
Michael Kolupaev 已提交
2481 2482 2483
	LOG_DEBUG(log, "Doing ALTER");

	NamesAndTypesList new_columns;
2484 2485 2486
	NamesAndTypesList new_materialized_columns;
	NamesAndTypesList new_alias_columns;
	ColumnDefaults new_column_defaults;
M
Merge  
Michael Kolupaev 已提交
2487 2488 2489 2490 2491 2492 2493
	String new_columns_str;
	int new_columns_version;
	zkutil::Stat stat;

	{
		auto table_lock = lockStructureForAlter();

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

M
Merge  
Michael Kolupaev 已提交
2497 2498
		data.checkAlter(params);

2499 2500 2501 2502 2503
		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);
M
Merge  
Michael Kolupaev 已提交
2504

2505 2506 2507 2508
		new_columns_str = ColumnsDescription<false>{
			new_columns, new_materialized_columns,
			new_alias_columns, new_column_defaults
		}.toString();
M
Merge  
Michael Kolupaev 已提交
2509 2510 2511 2512 2513 2514 2515 2516 2517 2518 2519 2520 2521 2522

		/// Делаем 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);
2523

M
Merge  
Michael Kolupaev 已提交
2524 2525 2526 2527 2528 2529 2530 2531 2532 2533 2534 2535 2536 2537 2538 2539 2540 2541 2542 2543 2544 2545 2546 2547 2548 2549 2550 2551 2552 2553
	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);
2554

M
Merge  
Michael Kolupaev 已提交
2555 2556 2557 2558 2559 2560 2561
			if (stat.version != new_columns_version)
			{
				LOG_WARNING(log, zookeeper_path + "/columns changed before ALTER finished; "
					"overlapping ALTER-s are fine but use caution with nontransitive changes");
				return;
			}

M
Merge  
Michael Kolupaev 已提交
2562
			if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/columns", &stat, alter_query_event))
M
Merge  
Michael Kolupaev 已提交
2563 2564 2565 2566 2567 2568 2569 2570
			{
				LOG_WARNING(log, replica << " was removed");
				break;
			}

			if (stat.version != replica_columns_version)
				continue;

2571
			alter_query_event->tryWait(WAIT_FOR_ALTER_SLEEP_MS);
M
Merge  
Michael Kolupaev 已提交
2572 2573 2574 2575 2576 2577 2578 2579 2580
		}

		if (shutdown_called)
			break;
	}

	LOG_DEBUG(log, "ALTER finished");
}

M
Merge  
Michael Kolupaev 已提交
2581 2582 2583 2584 2585

/// Название воображаемого куска, покрывающего все возможные куски в указанном месяце с номерами в указанном диапазоне.
static String getFakePartNameForDrop(const String & month_name, UInt64 left, UInt64 right)
{
	/// Диапазон дат - весь месяц.
A
Merge  
Alexey Arno 已提交
2586
	const auto & lut = DateLUT::instance();
2587
	time_t start_time = lut.YYYYMMDDToDate(parse<UInt32>(month_name + "01"));
M
Merge  
Michael Kolupaev 已提交
2588 2589 2590 2591 2592 2593 2594
	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);
}

A
Merge  
Alexey Milovidov 已提交
2595

2596
void StorageReplicatedMergeTree::dropUnreplicatedPartition(const Field & partition, const bool detach, const Settings & settings)
M
Merge  
Michael Kolupaev 已提交
2597
{
A
Merge  
Andrey Mironov 已提交
2598 2599 2600 2601 2602 2603 2604 2605 2606 2607 2608 2609 2610 2611 2612
	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)
	{
A
Merge  
Alexey Milovidov 已提交
2613
		if (part->month != month)
A
Merge  
Andrey Mironov 已提交
2614 2615 2616 2617 2618
			continue;

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

2619 2620 2621 2622
		if (detach)
			unreplicated_data->renameAndDetachPart(part, "");
		else
			unreplicated_data->replaceParts({part}, {}, false);
A
Merge  
Andrey Mironov 已提交
2623 2624
	}

2625
	LOG_INFO(log, (detach ? "Detached " : "Removed ") << removed_parts << " unreplicated parts inside " << apply_visitor(FieldVisitorToString(), partition) << ".");
A
Merge  
Andrey Mironov 已提交
2626 2627 2628
}


2629
void StorageReplicatedMergeTree::dropPartition(ASTPtr query, const Field & field, bool detach, bool unreplicated, const Settings & settings)
A
Merge  
Andrey Mironov 已提交
2630 2631 2632
{
	if (unreplicated)
	{
2633
		dropUnreplicatedPartition(field, detach, settings);
A
Merge  
Andrey Mironov 已提交
2634 2635 2636
		return;
	}

A
Merge  
Alexey Milovidov 已提交
2637 2638
	assertNotReadonly();

A
Merge  
Alexey Milovidov 已提交
2639
	auto zookeeper = getZooKeeper();
A
Merge  
Alexey Milovidov 已提交
2640
	String month_name = MergeTreeData::getMonthName(field);
M
Merge  
Michael Kolupaev 已提交
2641 2642

	if (!is_leader_node)
2643 2644 2645 2646 2647 2648 2649
	{
		/// Проксируем запрос в лидера.

		auto live_replicas = zookeeper->getChildren(zookeeper_path + "/leader_election");
		if (live_replicas.empty())
			throw Exception("No active replicas", ErrorCodes::NO_ACTIVE_REPLICAS);

2650
		std::sort(live_replicas.begin(), live_replicas.end());
2651 2652 2653 2654 2655 2656 2657 2658 2659 2660 2661 2662 2663 2664 2665 2666 2667 2668 2669 2670 2671 2672 2673 2674 2675 2676 2677
		const auto leader = zookeeper->get(zookeeper_path + "/leader_election/" + live_replicas.front());

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

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

		auto new_query = query->clone();
		auto & alter = typeid_cast<ASTAlterQuery &>(*new_query);

		alter.database = leader_address.database;
		alter.table = leader_address.table;

		/// NOTE Работает только если есть доступ от пользователя default без пароля. Можно исправить с помощью добавления параметра в конфиг сервера.

		Connection connection(
			leader_address.host,
			leader_address.queries_port,
			leader_address.database,
			"", "", "ClickHouse replica");

		RemoteBlockInputStream stream(connection, formattedAST(new_query), &settings);
		NullBlockOutputStream output;

		copyData(stream, output);
		return;
	}
M
Merge  
Michael Kolupaev 已提交
2678 2679 2680 2681 2682 2683 2684

	/** Пропустим один номер в block_numbers для удаляемого месяца, и будем удалять только куски до этого номера.
	  * Это запретит мерджи удаляемых кусков с новыми вставляемыми данными.
	  * Инвариант: в логе не появятся слияния удаляемых кусков с другими кусками.
	  * NOTE: Если понадобится аналогично поддержать запрос DROP PART, для него придется придумать какой-нибудь новый механизм,
	  *        чтобы гарантировать этот инвариант.
	  */
A
Merge  
Alexey Milovidov 已提交
2685
	Int64 right;
M
Merge  
Michael Kolupaev 已提交
2686 2687

	{
A
Merge  
Alexey Milovidov 已提交
2688
		AbandonableLockInZooKeeper block_number_lock = allocateBlockNumber(month_name);
M
Merge  
Michael Kolupaev 已提交
2689 2690 2691 2692 2693 2694
		right = block_number_lock.getNumber();
		block_number_lock.unlock();
	}

	/// Такого никогда не должно происходить.
	if (right == 0)
2695
		throw Exception("Logical error: just allocated block number is zero", ErrorCodes::LOGICAL_ERROR);
M
Merge  
Michael Kolupaev 已提交
2696 2697
	--right;

A
Merge  
Alexey Milovidov 已提交
2698
	String fake_part_name = getFakePartNameForDrop(month_name, 0, right);
M
Merge  
Michael Kolupaev 已提交
2699

A
Alexey Milovidov 已提交
2700
	/** Запретим выбирать для слияния удаляемые куски.
M
Merge  
Michael Kolupaev 已提交
2701 2702 2703
	  * Инвариант: после появления в логе записи DROP_RANGE, в логе не появятся слияния удаляемых кусков.
	  */
	{
2704
		std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
M
Merge  
Michael Kolupaev 已提交
2705

A
Alexey Milovidov 已提交
2706
		queue.disableMergesInRange(fake_part_name);
M
Merge  
Michael Kolupaev 已提交
2707 2708
	}

A
Merge  
Alexey Milovidov 已提交
2709 2710 2711 2712 2713 2714 2715 2716 2717
	/// Наконец, добившись нужных инвариантов, можно положить запись в лог.
	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);
	entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);
	entry.create_time = time(0);
A
Merge  
Alexey Arno 已提交
2718

A
Merge  
Alexey Milovidov 已提交
2719 2720 2721 2722 2723 2724 2725 2726
	/// Если надо - дожидаемся выполнения операции на себе или на всех репликах.
	if (settings.replication_alter_partitions_sync != 0)
	{
		if (settings.replication_alter_partitions_sync == 1)
			waitForReplicaToProcessLogEntry(replica_name, entry);
		else
			waitForAllReplicasToProcessLogEntry(entry);
	}
A
Merge  
Alexey Arno 已提交
2727
}
A
Merge  
Alexey Milovidov 已提交
2728

A
Merge  
Alexey Milovidov 已提交
2729

2730
void StorageReplicatedMergeTree::attachPartition(ASTPtr query, const Field & field, bool unreplicated, bool attach_part, const Settings & settings)
M
Merge  
Michael Kolupaev 已提交
2731
{
A
Merge  
Alexey Milovidov 已提交
2732 2733
	assertNotReadonly();

A
Merge  
Alexey Milovidov 已提交
2734
	auto zookeeper = getZooKeeper();
A
Merge  
Alexey Milovidov 已提交
2735
	String partition;
M
Merge  
Michael Kolupaev 已提交
2736

A
Merge  
Alexey Milovidov 已提交
2737
	if (attach_part)
A
Merge  
Alexey Milovidov 已提交
2738
		partition = field.safeGet<String>();
A
Merge  
Alexey Milovidov 已提交
2739 2740
	else
		partition = MergeTreeData::getMonthName(field);
M
Merge  
Michael Kolupaev 已提交
2741 2742 2743 2744 2745 2746 2747 2748 2749 2750 2751 2752 2753 2754 2755 2756 2757 2758

	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;
2759
			if (0 != name.compare(0, partition.size(), partition))
M
Merge  
Michael Kolupaev 已提交
2760 2761 2762 2763 2764 2765 2766 2767 2768 2769 2770 2771 2772 2773 2774 2775 2776 2777
				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 одновременно нельзя.
A
Merge  
Alexey Milovidov 已提交
2778
	Int64 min_used_number = RESERVED_BLOCK_NUMBERS;
2779
	DayNum_t month = DateLUT::instance().makeDayNum(parse<UInt16>(partition.substr(0, 4)), parse<UInt8>(partition.substr(4, 2)), 1);
M
Merge  
Michael Kolupaev 已提交
2780 2781 2782 2783

	{
		auto existing_parts = data.getDataParts();
		for (const auto & part : existing_parts)
A
Merge  
Alexey Milovidov 已提交
2784 2785
			if (part->month == month)
				min_used_number = std::min(min_used_number, part->left);
M
Merge  
Michael Kolupaev 已提交
2786 2787 2788 2789 2790 2791 2792 2793 2794 2795 2796 2797 2798 2799 2800 2801 2802 2803 2804 2805 2806 2807
	}

	/// Добавим записи в лог.
	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;
2808 2809
		entry.create_time = time(0);

M
Merge  
Michael Kolupaev 已提交
2810 2811 2812 2813
		ops.push_back(new zkutil::Op::Create(
			zookeeper_path + "/log/log-", entry.toString(), zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
	}

A
Merge  
Alexey Arno 已提交
2814
	LOG_DEBUG(log, "Adding attaches to log");
M
Merge  
Michael Kolupaev 已提交
2815
	zookeeper->multi(ops);
2816

2817 2818
	/// Если надо - дожидаемся выполнения операции на себе или на всех репликах.
	if (settings.replication_alter_partitions_sync != 0)
M
Merge  
Michael Kolupaev 已提交
2819
	{
2820 2821 2822 2823 2824 2825 2826 2827 2828 2829
		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);
M
Merge  
Michael Kolupaev 已提交
2830

2831 2832
			++i;
		}
M
Merge  
Michael Kolupaev 已提交
2833
	}
M
Merge  
Michael Kolupaev 已提交
2834 2835
}

A
Merge  
Alexey Milovidov 已提交
2836

M
Merge  
Michael Kolupaev 已提交
2837 2838
void StorageReplicatedMergeTree::drop()
{
A
Merge  
Alexey Milovidov 已提交
2839 2840 2841
	auto zookeeper = tryGetZooKeeper();

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

M
Merge  
Michael Kolupaev 已提交
2844 2845
	shutdown();

2846 2847 2848
	if (zookeeper->expired())
		throw Exception("Table was not dropped because ZooKeeper session has been expired.", ErrorCodes::TABLE_WAS_NOT_DROPPED);

M
Merge  
Michael Kolupaev 已提交
2849
	LOG_INFO(log, "Removing replica " << replica_path);
M
Merge  
Michael Kolupaev 已提交
2850
	replica_is_active_node = nullptr;
M
Merge  
Michael Kolupaev 已提交
2851 2852
	zookeeper->tryRemoveRecursive(replica_path);

M
Merge  
Michael Kolupaev 已提交
2853
	/// Проверяем, что zookeeper_path существует: его могла удалить другая реплика после выполнения предыдущей строки.
M
Merge  
Michael Kolupaev 已提交
2854 2855
	Strings replicas;
	if (zookeeper->tryGetChildren(zookeeper_path + "/replicas", replicas) == ZOK && replicas.empty())
M
Merge  
Michael Kolupaev 已提交
2856 2857
	{
		LOG_INFO(log, "Removing table " << zookeeper_path << " (this might take several minutes)");
M
Merge  
Michael Kolupaev 已提交
2858
		zookeeper->tryRemoveRecursive(zookeeper_path);
M
Merge  
Michael Kolupaev 已提交
2859
	}
M
Merge  
Michael Kolupaev 已提交
2860 2861

	data.dropAllData();
M
Merge  
Michael Kolupaev 已提交
2862 2863
}

A
Merge  
Alexey Milovidov 已提交
2864

M
Merge  
Michael Kolupaev 已提交
2865 2866 2867 2868 2869 2870 2871 2872 2873 2874 2875 2876 2877 2878 2879
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: Можно обновить названия логгеров.
}

A
Merge  
Alexey Milovidov 已提交
2880

2881 2882 2883 2884 2885 2886 2887 2888 2889 2890 2891 2892 2893 2894 2895 2896 2897 2898 2899 2900 2901
bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path)
{
	{
		std::lock_guard<std::mutex> lock(existing_nodes_cache_mutex);
		if (existing_nodes_cache.count(path))
			return true;
	}

	auto zookeeper = getZooKeeper();
	bool res = zookeeper->exists(path);

	if (res)
	{
		std::lock_guard<std::mutex> lock(existing_nodes_cache_mutex);
		existing_nodes_cache.insert(path);
	}

	return res;
}


M
Merge  
Michael Kolupaev 已提交
2902 2903
AbandonableLockInZooKeeper StorageReplicatedMergeTree::allocateBlockNumber(const String & month_name)
{
A
Merge  
Alexey Milovidov 已提交
2904 2905
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
2906
	String month_path = zookeeper_path + "/block_numbers/" + month_name;
2907
	if (!existsNodeCached(month_path))
M
Merge  
Michael Kolupaev 已提交
2908
	{
A
Merge  
Alexey Milovidov 已提交
2909
		/// Создадим в block_numbers ноду для месяца и пропустим в ней 200 значений инкремента.
M
Merge  
Michael Kolupaev 已提交
2910 2911 2912 2913
		/// Нужно, чтобы в будущем при необходимости можно было добавить данные в начало.
		zkutil::Ops ops;
		auto acl = zookeeper->getDefaultACL();
		ops.push_back(new zkutil::Op::Create(month_path, "", acl, zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
2914
		for (size_t i = 0; i < RESERVED_BLOCK_NUMBERS; ++i)
M
Merge  
Michael Kolupaev 已提交
2915 2916 2917 2918 2919 2920 2921 2922 2923 2924 2925 2926 2927
		{
			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);
}

A
Merge  
Alexey Milovidov 已提交
2928

M
Merge  
Michael Kolupaev 已提交
2929
void StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry(const LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
2930
{
A
Merge  
Alexey Milovidov 已提交
2931
	auto zookeeper = getZooKeeper();
M
Merge  
Michael Kolupaev 已提交
2932 2933
	LOG_DEBUG(log, "Waiting for all replicas to process " << entry.znode_name);

A
Merge  
Alexey Milovidov 已提交
2934 2935 2936 2937 2938 2939 2940 2941 2942 2943
	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)
{
A
Merge  
Alexey Milovidov 已提交
2944 2945
	auto zookeeper = getZooKeeper();

2946 2947
	String entry_str = entry.toString();
	String log_node_name;
M
Merge  
Michael Kolupaev 已提交
2948

2949 2950 2951 2952 2953 2954 2955 2956 2957
	/** В эту функцию могут передать entry двух видов:
	  * 1. (более часто) Из директории log - общего лога, откуда реплики копируют записи в свою queue.
	  * 2. Из директории queue одной из реплик.
	  *
	  * Проблема в том, что номера (sequential нод) элементов очереди в log и в queue не совпадают.
	  * (И в queue не совпадают номера у одного и того же элемента лога для разных реплик.)
	  *
	  * Поэтому следует рассматривать эти случаи по-отдельности.
	  */
M
Merge  
Michael Kolupaev 已提交
2958

2959 2960 2961 2962 2963 2964 2965
	/** Первое - нужно дождаться, пока реплика возьмёт к себе в queue элемент очереди из log,
	  *  если она ещё этого не сделала (см. функцию pullLogsToQueue).
	  *
	  * Для этого проверяем её узел log_pointer - максимальный номер взятого элемента из log плюс единица.
	  */

	if (0 == entry.znode_name.compare(0, strlen("log-"), "log-"))
A
Merge  
Alexey Milovidov 已提交
2966
	{
2967 2968
		/** В этом случае просто берём номер из имени ноды log-xxxxxxxxxx.
		  */
M
Merge  
Michael Kolupaev 已提交
2969

2970 2971
		UInt64 log_index = parse<UInt64>(entry.znode_name.substr(entry.znode_name.size() - 10));
		log_node_name = entry.znode_name;
M
Merge  
Michael Kolupaev 已提交
2972

2973 2974 2975 2976 2977 2978 2979 2980 2981 2982 2983
		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;

2984
			event->tryWait(WAIT_FOR_REPLICA_QUEUE_MS);
2985
		}
A
Merge  
Alexey Milovidov 已提交
2986
	}
2987 2988 2989 2990 2991 2992 2993
	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");
M
Merge  
Michael Kolupaev 已提交
2994

2995 2996 2997 2998 2999 3000 3001 3002 3003 3004 3005 3006 3007 3008 3009 3010 3011 3012 3013 3014 3015 3016 3017 3018 3019 3020 3021 3022 3023 3024 3025 3026 3027 3028
		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;

3029
				event->tryWait(WAIT_FOR_REPLICA_QUEUE_MS);
3030 3031 3032 3033 3034 3035 3036 3037 3038 3039 3040 3041 3042 3043 3044
			}
		}
	}
	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-узлом у текущей реплики (у нас).
	  * Поэтому, ищем путём сравнения содержимого.
	  */
M
Merge  
Michael Kolupaev 已提交
3045

A
Merge  
Alexey Milovidov 已提交
3046
	Strings queue_entries = zookeeper->getChildren(zookeeper_path + "/replicas/" + replica + "/queue");
3047
	String queue_entry_to_wait_for;
M
Merge  
Michael Kolupaev 已提交
3048

A
Merge  
Alexey Milovidov 已提交
3049 3050 3051 3052
	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);
3053
		if (exists && queue_entry_str == entry_str)
M
Merge  
Michael Kolupaev 已提交
3054
		{
3055
			queue_entry_to_wait_for = entry_name;
A
Merge  
Alexey Milovidov 已提交
3056
			break;
M
Merge  
Michael Kolupaev 已提交
3057
		}
A
Merge  
Alexey Milovidov 已提交
3058
	}
M
Merge  
Michael Kolupaev 已提交
3059

A
Merge  
Alexey Milovidov 已提交
3060
	/// Пока искали запись, ее уже выполнили и удалили.
3061 3062 3063
	if (queue_entry_to_wait_for.empty())
	{
		LOG_DEBUG(log, "No corresponding node found. Assuming it has been already processed.");
A
Merge  
Alexey Milovidov 已提交
3064
		return;
3065 3066 3067
	}

	LOG_DEBUG(log, "Waiting for " << queue_entry_to_wait_for << " to disappear from " << replica << " queue");
M
Merge  
Michael Kolupaev 已提交
3068

3069
	/// Третье - дождемся, пока запись исчезнет из очереди реплики.
3070
	zookeeper->waitForDisappear(zookeeper_path + "/replicas/" + replica + "/queue/" + queue_entry_to_wait_for);
M
Merge  
Michael Kolupaev 已提交
3071 3072 3073
}


3074
void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields)
3075
{
A
Merge  
Alexey Milovidov 已提交
3076
	auto zookeeper = tryGetZooKeeper();
A
Merge  
Alexey Milovidov 已提交
3077

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

A
Alexey Milovidov 已提交
3082
	res.queue = queue.getStatus();
3083 3084 3085 3086 3087 3088 3089 3090 3091 3092 3093

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

3094
	if (res.is_session_expired || !with_zk_fields)
3095 3096 3097 3098 3099 3100 3101 3102 3103 3104
	{
		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");

3105 3106 3107 3108 3109 3110 3111 3112 3113 3114 3115 3116
		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);
3117 3118 3119 3120 3121 3122 3123 3124 3125 3126 3127

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

3128

3129 3130 3131
void StorageReplicatedMergeTree::getQueue(LogEntriesData & res, String & replica_name_)
{
	replica_name_ = replica_name;
A
Alexey Milovidov 已提交
3132
	queue.getEntries(res);
3133 3134 3135
}


3136
void StorageReplicatedMergeTree::getReplicaDelays(time_t & out_absolute_delay, time_t & out_relative_delay)
3137
{
3138 3139 3140 3141 3142 3143 3144 3145 3146 3147 3148 3149 3150 3151 3152 3153 3154 3155 3156 3157 3158 3159 3160 3161 3162 3163 3164
	assertNotReadonly();

	/** Абсолютная задержка - задержка отставания текущей реплики от реального времени.
	  */

	time_t min_unprocessed_insert_time = 0;
	time_t max_processed_insert_time = 0;
	queue.getInsertTimes(min_unprocessed_insert_time, max_processed_insert_time);

	time_t current_time = time(0);
	out_absolute_delay = 0;
	out_relative_delay = 0;

	if (min_unprocessed_insert_time)
		out_absolute_delay = current_time - min_unprocessed_insert_time;

	/** Относительная задержка - максимальная разница абсолютной задержки от какой-либо другой реплики,
	  *  (если эта реплика отстаёт от какой-либо другой реплики, или ноль, иначе).
	  * Вычисляется только если абсолютная задержка достаточно большая.
	  */

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

	auto zookeeper = getZooKeeper();

	time_t max_replicas_unprocessed_insert_time = 0;
3165 3166
	bool have_replica_with_nothing_unprocessed = false;

3167 3168 3169 3170 3171 3172 3173 3174 3175 3176 3177 3178
	Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");

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

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

		time_t replica_time = value.empty() ? 0 : parse<time_t>(value);
3179 3180 3181 3182 3183 3184 3185

		if (replica_time == 0)
		{
			have_replica_with_nothing_unprocessed = true;
			break;
		}

3186 3187 3188
		if (replica_time > max_replicas_unprocessed_insert_time)
			max_replicas_unprocessed_insert_time = replica_time;
	}
3189

3190 3191 3192
	if (have_replica_with_nothing_unprocessed)
		out_relative_delay = out_absolute_delay;
	else if (max_replicas_unprocessed_insert_time > min_unprocessed_insert_time)
3193
		out_relative_delay = max_replicas_unprocessed_insert_time - min_unprocessed_insert_time;
3194 3195 3196
}


3197
void StorageReplicatedMergeTree::fetchPartition(const Field & partition, const String & from_, const Settings & settings)
3198
{
A
Merge  
Alexey Milovidov 已提交
3199 3200
	auto zookeeper = getZooKeeper();

3201 3202
	String partition_str = MergeTreeData::getMonthName(partition);

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

3207 3208
	LOG_INFO(log, "Will fetch partition " << partition_str << " from shard " << from_);

3209 3210 3211 3212 3213 3214 3215 3216
	/** Проверим, что в директории 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);

3217 3218 3219 3220 3221 3222 3223 3224 3225 3226 3227 3228
	/// Список реплик шарда-источника.
	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())
3229
		throw Exception("No active replicas for shard " + from, ErrorCodes::NO_ACTIVE_REPLICAS);
3230 3231

	/** Надо выбрать лучшую (наиболее актуальную) реплику.
3232
	  * Это реплика с максимальным log_pointer, затем с минимальным размером queue.
3233 3234
	  * NOTE Это не совсем лучший критерий. Для скачивания старых партиций это не имеет смысла,
	  *  и было бы неплохо уметь выбирать реплику, ближайшую по сети.
3235
	  * NOTE Разумеется, здесь есть data race-ы. Можно решить ретраями.
3236
	  */
3237 3238 3239 3240 3241 3242
	Int64 max_log_pointer = -1;
	UInt64 min_queue_size = std::numeric_limits<UInt64>::max();
	String best_replica;

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

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

		zkutil::Stat stat;
3249
		zookeeper->get(current_replica_path + "/queue", &stat);
3250 3251 3252 3253 3254 3255 3256 3257 3258 3259 3260 3261 3262 3263 3264 3265 3266
		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.");

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

3269
	/// Выясним, какие куски есть на лучшей реплике.
3270

3271 3272 3273 3274 3275 3276 3277 3278 3279 3280 3281 3282 3283 3284 3285 3286 3287 3288 3289 3290 3291 3292 3293 3294 3295 3296 3297 3298 3299 3300
	/** Пытаемся скачать эти куски.
	  * Часть из них могла удалиться из-за мерджа.
	  * В этом случае, обновляем информацию о доступных кусках и пробуем снова.
	  */

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

			if (parts_to_fetch.empty())
				throw Exception("Partition " + partition_str + " on " + best_replica_path + " doesn't exist", ErrorCodes::PARTITION_DOESNT_EXIST);
3304 3305 3306 3307 3308 3309 3310 3311 3312 3313 3314 3315 3316 3317 3318 3319 3320 3321 3322 3323
		}
		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
			{
3324
				fetchPart(part, best_replica_path, true, 0);
3325 3326 3327 3328 3329 3330 3331 3332 3333 3334
			}
			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);
			}
		}
3335

3336 3337
		++try_no;
	} while (!missing_parts.empty());
3338 3339 3340
}


3341 3342 3343 3344 3345 3346 3347 3348 3349 3350 3351 3352 3353
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);
}


M
Merge  
Michael Kolupaev 已提交
3354
}