StorageReplicatedMergeTree.cpp 106.7 KB
Newer Older
1
#include <statdaemons/ext/range.hpp>
M
Merge  
Michael Kolupaev 已提交
2
#include <DB/Storages/StorageReplicatedMergeTree.h>
M
Merge  
Michael Kolupaev 已提交
3
#include <DB/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h>
M
Merge  
Michael Kolupaev 已提交
4
#include <DB/Storages/MergeTree/ReplicatedMergeTreePartsExchange.h>
M
Merge  
Michael Kolupaev 已提交
5
#include <DB/Storages/MergeTree/MergeTreePartChecker.h>
M
Merge  
Michael Kolupaev 已提交
6 7 8
#include <DB/Parsers/formatAST.h>
#include <DB/IO/WriteBufferFromOStream.h>
#include <DB/IO/ReadBufferFromString.h>
M
Merge  
Michael Kolupaev 已提交
9
#include <DB/Interpreters/InterpreterAlterQuery.h>
M
Merge  
Michael Kolupaev 已提交
10
#include <DB/Common/VirtualColumnUtils.h>
M
Merge  
Michael Kolupaev 已提交
11
#include <time.h>
M
Merge  
Michael Kolupaev 已提交
12 13 14 15

namespace DB
{

M
Merge  
Michael Kolupaev 已提交
16

M
Merge  
Michael Kolupaev 已提交
17
const auto ERROR_SLEEP_MS = 1000;
M
Merge  
Michael Kolupaev 已提交
18
const auto MERGE_SELECTING_SLEEP_MS = 5 * 1000;
M
Merge  
Michael Kolupaev 已提交
19
const auto CLEANUP_SLEEP_MS = 30 * 1000;
M
Merge  
Michael Kolupaev 已提交
20

M
Merge  
Michael Kolupaev 已提交
21 22
const auto RESERVED_BLOCK_NUMBERS = 200;

M
Merge  
Michael Kolupaev 已提交
23 24 25 26 27 28 29 30 31
/// Преобразовать число в строку формате суффиксов автоинкрементных нод в ZooKeeper.
static String padIndex(UInt64 index)
{
	String index_str = toString(index);
	while (index_str.size() < 10)
		index_str = '0' + index_str;
	return index_str;
}

M
Merge  
Michael Kolupaev 已提交
32

33 34 35 36 37 38 39 40 41 42
/// Используется для проверки, выставили ли ноду is_active мы, или нет.
static String generateActiveNodeIdentifier()
{
	struct timespec times;
	if (clock_gettime(CLOCK_THREAD_CPUTIME_ID, &times))
		throwFromErrno("Cannot clock_gettime.", ErrorCodes::CANNOT_CLOCK_GETTIME);
	return toString(times.tv_nsec + times.tv_sec + getpid());
}


M
Merge  
Michael Kolupaev 已提交
43 44 45
StorageReplicatedMergeTree::StorageReplicatedMergeTree(
	const String & zookeeper_path_,
	const String & replica_name_,
M
Merge  
Michael Kolupaev 已提交
46
	bool attach,
M
Merge  
Michael Kolupaev 已提交
47 48
	const String & path_, const String & database_name_, const String & name_,
	NamesAndTypesListPtr columns_,
49
	const NamesAndTypesList & materialized_columns_,
50 51
	const NamesAndTypesList & alias_columns_,
	const ColumnDefaults & column_defaults_,
M
Merge  
Michael Kolupaev 已提交
52
	Context & context_,
M
Merge  
Michael Kolupaev 已提交
53 54 55 56 57 58 59
	ASTPtr & primary_expr_ast_,
	const String & date_column_name_,
	const ASTPtr & sampling_expression_,
	size_t index_granularity_,
	MergeTreeData::Mode mode_,
	const String & sign_column_,
	const MergeTreeSettings & settings_)
60
    : IStorage{materialized_columns_, alias_columns_, column_defaults_}, context(context_),
61
	zookeeper(context.getZooKeeper()), database_name(database_name_),
M
Merge  
Michael Kolupaev 已提交
62 63 64
	table_name(name_), full_path(path_ + escapeForFileName(table_name) + '/'),
	zookeeper_path(context.getMacros().expand(zookeeper_path_)),
	replica_name(context.getMacros().expand(replica_name_)),
65 66 67 68 69 70
	data(full_path, columns_,
		 materialized_columns_, alias_columns_, column_defaults_,
		 context_, primary_expr_ast_, date_column_name_,
		 sampling_expression_, index_granularity_, mode_, sign_column_,
		 settings_, database_name_ + "." + table_name, true,
		 std::bind(&StorageReplicatedMergeTree::enqueuePartForCheck, this, std::placeholders::_1)),
M
Merge  
Michael Kolupaev 已提交
71
	reader(data), writer(data), merger(data), fetcher(data),
M
Merge  
Michael Kolupaev 已提交
72
	log(&Logger::get(database_name_ + "." + table_name + " (StorageReplicatedMergeTree)")),
M
Merge  
Michael Kolupaev 已提交
73
	shutdown_event(false)
M
Merge  
Michael Kolupaev 已提交
74
{
M
Merge  
Michael Kolupaev 已提交
75 76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91
	if (!zookeeper_path.empty() && *zookeeper_path.rbegin() == '/')
		zookeeper_path.erase(zookeeper_path.end() - 1);
	replica_path = zookeeper_path + "/replicas/" + replica_name;

	bool skip_sanity_checks = false;

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

		LOG_WARNING(log, "Skipping the limits on severity of changes to data parts and columns (flag "
			<< replica_path << "/flags/force_restore_data).");
	}

	data.loadDataParts(skip_sanity_checks);

M
Merge  
Michael Kolupaev 已提交
92 93
	if (!zookeeper)
	{
M
Merge  
Michael Kolupaev 已提交
94 95 96
		if (!attach)
			throw Exception("Can't create replicated table without ZooKeeper", ErrorCodes::NO_ZOOKEEPER);

M
Merge  
Michael Kolupaev 已提交
97
		goReadOnlyPermanently();
M
Merge  
Michael Kolupaev 已提交
98 99 100
		return;
	}

M
Merge  
Michael Kolupaev 已提交
101 102
	if (!attach)
	{
M
Merge  
Michael Kolupaev 已提交
103
		createTableIfNotExists();
M
Merge  
Michael Kolupaev 已提交
104

M
Merge  
Michael Kolupaev 已提交
105
		checkTableStructure(false, false);
M
Merge  
Michael Kolupaev 已提交
106
		createReplica();
M
Merge  
Michael Kolupaev 已提交
107 108 109
	}
	else
	{
M
Merge  
Michael Kolupaev 已提交
110
		checkTableStructure(skip_sanity_checks, true);
M
Merge  
Michael Kolupaev 已提交
111
		checkParts(skip_sanity_checks);
M
Merge  
Michael Kolupaev 已提交
112
	}
M
Merge  
Michael Kolupaev 已提交
113

M
Merge  
Michael Kolupaev 已提交
114
	initVirtualParts();
M
Merge  
Michael Kolupaev 已提交
115
	loadQueue();
M
Merge  
Michael Kolupaev 已提交
116 117 118 119 120

	String unreplicated_path = full_path + "unreplicated/";
	if (Poco::File(unreplicated_path).exists())
	{
		LOG_INFO(log, "Have unreplicated data");
121

122 123
		unreplicated_data.reset(new MergeTreeData(unreplicated_path, columns_,
			materialized_columns_, alias_columns_, column_defaults_,
124
			context_, primary_expr_ast_,
M
Merge  
Michael Kolupaev 已提交
125
			date_column_name_, sampling_expression_, index_granularity_, mode_, sign_column_, settings_,
M
Merge  
Michael Kolupaev 已提交
126
			database_name_ + "." + table_name + "[unreplicated]", false));
127 128 129

		unreplicated_data->loadDataParts(skip_sanity_checks);

M
Merge  
Michael Kolupaev 已提交
130
		unreplicated_reader.reset(new MergeTreeDataSelectExecutor(*unreplicated_data));
M
Merge  
Michael Kolupaev 已提交
131
		unreplicated_merger.reset(new MergeTreeDataMerger(*unreplicated_data));
M
Merge  
Michael Kolupaev 已提交
132
	}
M
Merge  
Michael Kolupaev 已提交
133

M
Merge  
Michael Kolupaev 已提交
134
	/// Сгенерируем этому экземпляру случайный идентификатор.
135
	active_node_identifier = generateActiveNodeIdentifier();
M
Merge  
Michael Kolupaev 已提交
136

137
	/// В этом потоке реплика будет активирована.
M
Merge  
Michael Kolupaev 已提交
138
	restarting_thread = std::thread(&StorageReplicatedMergeTree::restartingThread, this);
M
Merge  
Michael Kolupaev 已提交
139 140 141 142 143
}

StoragePtr StorageReplicatedMergeTree::create(
	const String & zookeeper_path_,
	const String & replica_name_,
M
Merge  
Michael Kolupaev 已提交
144
	bool attach,
M
Merge  
Michael Kolupaev 已提交
145 146
	const String & path_, const String & database_name_, const String & name_,
	NamesAndTypesListPtr columns_,
147
	const NamesAndTypesList & materialized_columns_,
148 149
	const NamesAndTypesList & alias_columns_,
	const ColumnDefaults & column_defaults_,
M
Merge  
Michael Kolupaev 已提交
150
	Context & context_,
M
Merge  
Michael Kolupaev 已提交
151 152 153 154 155 156 157 158
	ASTPtr & primary_expr_ast_,
	const String & date_column_name_,
	const ASTPtr & sampling_expression_,
	size_t index_granularity_,
	MergeTreeData::Mode mode_,
	const String & sign_column_,
	const MergeTreeSettings & settings_)
{
159 160 161
	auto res = new StorageReplicatedMergeTree{
		zookeeper_path_, replica_name_, attach,
		path_, database_name_, name_,
162
		columns_, materialized_columns_, alias_columns_, column_defaults_,
163 164 165 166
		context_, primary_expr_ast_, date_column_name_,
		sampling_expression_, index_granularity_, mode_,
		sign_column_, settings_
	};
M
Merge  
Michael Kolupaev 已提交
167
	StoragePtr res_ptr = res->thisPtr();
M
Merge  
Michael Kolupaev 已提交
168 169 170
	if (!res->is_read_only)
	{
		String endpoint_name = "ReplicatedMergeTree:" + res->replica_path;
M
Merge  
Michael Kolupaev 已提交
171
		InterserverIOEndpointPtr endpoint = new ReplicatedMergeTreePartsServer(res->data, *res);
M
Merge  
Michael Kolupaev 已提交
172 173
		res->endpoint_holder = new InterserverIOEndpointHolder(endpoint_name, endpoint, res->context.getInterserverIOHandler());
	}
M
Merge  
Michael Kolupaev 已提交
174
	return res_ptr;
M
Merge  
Michael Kolupaev 已提交
175 176
}

M
Merge  
Michael Kolupaev 已提交
177 178 179 180 181 182 183 184
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 已提交
185

M
Merge  
Michael Kolupaev 已提交
186
void StorageReplicatedMergeTree::createTableIfNotExists()
M
Merge  
Michael Kolupaev 已提交
187
{
M
Merge  
Michael Kolupaev 已提交
188 189
	if (zookeeper->exists(zookeeper_path))
		return;
M
Merge  
Michael Kolupaev 已提交
190

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

M
Merge  
Michael Kolupaev 已提交
193 194
	zookeeper->createAncestors(zookeeper_path);

M
Merge  
Michael Kolupaev 已提交
195
	/// Запишем метаданные таблицы, чтобы реплики могли сверять с ними параметры таблицы.
M
Merge  
Michael Kolupaev 已提交
196 197 198 199 200 201 202 203
	std::stringstream metadata;
	metadata << "metadata format version: 1" << std::endl;
	metadata << "date column: " << data.date_column_name << std::endl;
	metadata << "sampling expression: " << formattedAST(data.sampling_expression) << std::endl;
	metadata << "index granularity: " << data.index_granularity << std::endl;
	metadata << "mode: " << static_cast<int>(data.mode) << std::endl;
	metadata << "sign column: " << data.sign_column << std::endl;
	metadata << "primary key: " << formattedAST(data.primary_expr_ast) << std::endl;
M
Merge  
Michael Kolupaev 已提交
204

M
Merge  
Michael Kolupaev 已提交
205 206 207 208 209
	zkutil::Ops ops;
	ops.push_back(new zkutil::Op::Create(zookeeper_path, "",
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/metadata", metadata.str(),
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
210
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/columns", data.getColumnsListNonMaterialized().toString(),
M
Merge  
Michael Kolupaev 已提交
211
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
212 213 214 215 216 217
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/materialized_columns", data.materialized_columns.toString(),
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/alias_columns", data.alias_columns.toString(),
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/column_defaults", data.column_defaults.toString(),
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
218 219 220 221 222 223 224 225 226 227 228 229 230 231 232 233 234 235
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/log", "",
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/blocks", "",
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/block_numbers", "",
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/nonincrement_block_numbers", "",
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/leader_election", "",
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/temp", "",
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/replicas", "",
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));

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

M
Merge  
Michael Kolupaev 已提交
238 239
/** Проверить, что список столбцов и настройки таблицы совпадают с указанными в ZK (/metadata).
	* Если нет - бросить исключение.
M
Merge  
Michael Kolupaev 已提交
240
	*/
M
Merge  
Michael Kolupaev 已提交
241
void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bool allow_alter)
M
Merge  
Michael Kolupaev 已提交
242
{
M
Merge  
Michael Kolupaev 已提交
243
	String metadata_str = zookeeper->get(zookeeper_path + "/metadata");
M
Merge  
Michael Kolupaev 已提交
244 245 246 247 248 249 250 251 252 253 254 255 256
	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 已提交
257 258
	/// NOTE: Можно сделать менее строгую проверку совпадения выражений, чтобы таблицы не ломались от небольших изменений
	///       в коде formatAST.
M
Merge  
Michael Kolupaev 已提交
259
	assertString(formattedAST(data.primary_expr_ast), buf);
M
Merge  
Michael Kolupaev 已提交
260
	assertString("\n", buf);
M
Merge  
Michael Kolupaev 已提交
261
	assertEOF(buf);
M
Merge  
Michael Kolupaev 已提交
262 263 264

	zkutil::Stat stat;
	auto columns = NamesAndTypesList::parse(zookeeper->get(zookeeper_path + "/columns", &stat), context.getDataTypeFactory());
265 266 267 268 269
	auto materialized_columns = NamesAndTypesList::parse(
		zookeeper->get(zookeeper_path + "/materialized_columns", &stat), context.getDataTypeFactory());
	auto alias_columns = NamesAndTypesList::parse(
		zookeeper->get(zookeeper_path + "/alias_columns", &stat), context.getDataTypeFactory());
	auto column_defaults = ColumnDefaults::parse(zookeeper->get(zookeeper_path + "/column_defaults", &stat));
M
Merge  
Michael Kolupaev 已提交
270
	columns_version = stat.version;
271 272 273 274 275 276 277 278 279
	if (columns != data.getColumnsListNonMaterialized() ||
		materialized_columns != data.materialized_columns ||
		alias_columns != data.alias_columns ||
		column_defaults != data.column_defaults)
	{
		if (allow_alter &&
			(skip_sanity_checks ||
			 data.getColumnsListNonMaterialized().sizeOfDifference(columns) +
			 data.materialized_columns.sizeOfDifference(materialized_columns) <= 2))
M
Merge  
Michael Kolupaev 已提交
280
		{
M
Merge  
Michael Kolupaev 已提交
281
			LOG_WARNING(log, "Table structure in ZooKeeper is a little different from local table structure. Assuming ALTER.");
282 283

			/// Без всяких блокировок, потому что таблица еще не создана.
284 285
			InterpreterAlterQuery::updateMetadata(database_name, table_name, columns,
				materialized_columns, alias_columns, column_defaults, context);
286
			data.setColumnsList(columns);
287 288 289
			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 已提交
290
		}
M
Merge  
Michael Kolupaev 已提交
291
		else
M
Merge  
Michael Kolupaev 已提交
292
		{
M
Merge  
Michael Kolupaev 已提交
293
			throw Exception("Table structure in ZooKeeper is too different from local table structure.",
M
Merge  
Michael Kolupaev 已提交
294
							ErrorCodes::INCOMPATIBLE_COLUMNS);
M
Merge  
Michael Kolupaev 已提交
295
		}
M
Merge  
Michael Kolupaev 已提交
296 297
	}
}
M
Merge  
Michael Kolupaev 已提交
298

M
Merge  
Michael Kolupaev 已提交
299 300
void StorageReplicatedMergeTree::createReplica()
{
M
Merge  
Michael Kolupaev 已提交
301
	LOG_DEBUG(log, "Creating replica " << replica_path);
M
Merge  
Michael Kolupaev 已提交
302

M
Merge  
Michael Kolupaev 已提交
303 304 305 306 307 308 309 310
	/// Создадим пустую реплику. Ноду columns создадим в конце - будем использовать ее в качестве признака, что создание реплики завершено.
	zkutil::Ops ops;
	ops.push_back(new zkutil::Op::Create(replica_path, "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(replica_path + "/host", "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(replica_path + "/log_pointer", "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(replica_path + "/queue", "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(replica_path + "/parts", "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(replica_path + "/flags", "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
311 312 313 314 315 316 317 318 319 320 321 322

	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 已提交
323

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

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

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

M
Merge  
Michael Kolupaev 已提交
336 337 338
	Stat stat;
	zookeeper->exists(replica_path, &stat);
	auto my_create_time = stat.czxid;
M
Merge  
Michael Kolupaev 已提交
339

M
Merge  
Michael Kolupaev 已提交
340 341
	std::random_shuffle(replicas.begin(), replicas.end());
	for (const String & replica : replicas)
M
Merge  
Michael Kolupaev 已提交
342
	{
M
Merge  
Michael Kolupaev 已提交
343 344 345 346 347 348 349 350
		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 已提交
351 352
	}

M
Merge  
Michael Kolupaev 已提交
353
	if (source_replica.empty())
M
Merge  
Michael Kolupaev 已提交
354
	{
M
Merge  
Michael Kolupaev 已提交
355
		LOG_INFO(log, "This is the first replica");
M
Merge  
Michael Kolupaev 已提交
356
	}
M
Merge  
Michael Kolupaev 已提交
357
	else
M
Merge  
Michael Kolupaev 已提交
358
	{
M
Merge  
Michael Kolupaev 已提交
359
		LOG_INFO(log, "Will mimic " << source_replica);
M
Merge  
Michael Kolupaev 已提交
360

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

M
Merge  
Michael Kolupaev 已提交
363 364 365 366 367 368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402 403 404 405 406 407 408 409 410 411 412 413 414 415 416 417 418 419 420 421 422 423 424
		/** Если эталонная реплика еще не до конца создана, подождем.
		  * 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");
		ActiveDataPartSet active_parts_set;
		for (const String & part : parts)
		{
			active_parts_set.add(part);
		}
		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;

			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);
		}
		LOG_DEBUG(log, "Copied " << source_queue.size() << " queue entries");
M
Merge  
Michael Kolupaev 已提交
425
	}
M
Merge  
Michael Kolupaev 已提交
426

427
	zookeeper->create(replica_path + "/columns", data.getColumnsListNonMaterialized().toString(), zkutil::CreateMode::Persistent);
428 429 430
	zookeeper->create(replica_path + "/materialized_columns", data.materialized_columns.toString(), zkutil::CreateMode::Persistent);
	zookeeper->create(replica_path + "/alias_columns", data.alias_columns.toString(), zkutil::CreateMode::Persistent);
	zookeeper->create(replica_path + "/column_defaults", data.column_defaults.toString(), zkutil::CreateMode::Persistent);
M
Merge  
Michael Kolupaev 已提交
431
}
M
Merge  
Michael Kolupaev 已提交
432

M
Merge  
Michael Kolupaev 已提交
433 434 435 436 437
void StorageReplicatedMergeTree::activateReplica()
{
	std::stringstream host;
	host << "host: " << context.getInterserverIOHost() << std::endl;
	host << "port: " << context.getInterserverIOPort() << std::endl;
M
Merge  
Michael Kolupaev 已提交
438

M
Merge  
Michael Kolupaev 已提交
439 440 441 442 443 444 445 446 447
	/** Если нода отмечена как активная, но отметка сделана в этом же экземпляре, удалим ее.
	  * Такое возможно только при истечении сессии в ZooKeeper.
	  * Здесь есть небольшой race condition (можем удалить не ту ноду, для которой сделали tryGet),
	  *  но он крайне маловероятен при нормальном использовании.
	  */
	String data;
	if (zookeeper->tryGet(replica_path + "/is_active", data) && data == active_node_identifier)
		zookeeper->tryRemove(replica_path + "/is_active");

M
Merge  
Michael Kolupaev 已提交
448
	/// Одновременно объявим, что эта реплика активна, и обновим хост.
M
Merge  
Michael Kolupaev 已提交
449
	zkutil::Ops ops;
450 451
	ops.push_back(new zkutil::Op::Create(replica_path + "/is_active",
		active_node_identifier, zookeeper->getDefaultACL(), zkutil::CreateMode::Ephemeral));
M
Merge  
Michael Kolupaev 已提交
452
	ops.push_back(new zkutil::Op::SetData(replica_path + "/host", host.str(), -1));
M
Merge  
Michael Kolupaev 已提交
453 454 455

	try
	{
M
Merge  
Michael Kolupaev 已提交
456
		zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
457
	}
458
	catch (const zkutil::KeeperException & e)
M
Merge  
Michael Kolupaev 已提交
459
	{
460
		if (e.code == ZNODEEXISTS)
M
Merge  
Michael Kolupaev 已提交
461 462 463 464 465
			throw Exception("Replica " + replica_path + " appears to be already active. If you're sure it's not, "
				"try again in a minute or remove znode " + replica_path + "/is_active manually", ErrorCodes::REPLICA_IS_ALREADY_ACTIVE);

		throw;
	}
M
Merge  
Michael Kolupaev 已提交
466

M
Merge  
Michael Kolupaev 已提交
467
	replica_is_active_node = zkutil::EphemeralNodeHolder::existing(replica_path + "/is_active", *zookeeper);
M
Merge  
Michael Kolupaev 已提交
468
}
M
Merge  
Michael Kolupaev 已提交
469

M
Merge  
Michael Kolupaev 已提交
470
void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
M
Merge  
Michael Kolupaev 已提交
471
{
M
Merge  
Michael Kolupaev 已提交
472
	Strings expected_parts_vec = zookeeper->getChildren(replica_path + "/parts");
M
Merge  
Michael Kolupaev 已提交
473 474

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

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

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

M
Merge  
Michael Kolupaev 已提交
482 483 484 485 486 487 488 489
	for (const auto & part : parts)
	{
		if (expected_parts.count(part->name))
		{
			expected_parts.erase(part->name);
		}
		else
		{
M
Merge  
Michael Kolupaev 已提交
490
			unexpected_parts.insert(part);
M
Merge  
Michael Kolupaev 已提交
491 492 493
		}
	}

M
Merge  
Michael Kolupaev 已提交
494
	/// Какие локальные куски добавить в ZK.
M
Merge  
Michael Kolupaev 已提交
495
	MergeTreeData::DataPartsVector parts_to_add;
M
Merge  
Michael Kolupaev 已提交
496 497 498 499

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

M
Merge  
Michael Kolupaev 已提交
500 501
	for (const String & missing_name : expected_parts)
	{
M
Merge  
Michael Kolupaev 已提交
502
		/// Если локально не хватает какого-то куска, но есть покрывающий его кусок, можно заменить в ZK недостающий покрывающим.
M
Merge  
Michael Kolupaev 已提交
503
		auto containing = data.getActiveContainingPart(missing_name);
M
Merge  
Michael Kolupaev 已提交
504 505 506 507 508 509 510 511 512 513
		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 已提交
514
		{
M
Merge  
Michael Kolupaev 已提交
515
			LOG_ERROR(log, "Fetching missing part " << missing_name);
M
Merge  
Michael Kolupaev 已提交
516
			parts_to_fetch.push_back(missing_name);
M
Merge  
Michael Kolupaev 已提交
517 518
		}
	}
M
Merge  
Michael Kolupaev 已提交
519

M
Merge  
Michael Kolupaev 已提交
520 521 522 523 524 525
	for (const String & name : parts_to_fetch)
		expected_parts.erase(name);

	String sanity_report =
		"There are " + toString(unexpected_parts.size()) + " unexpected parts, "
					 + toString(parts_to_add.size()) + " unexpectedly merged parts, "
M
Merge  
Michael Kolupaev 已提交
526
					 + toString(expected_parts.size()) + " missing obsolete parts, "
M
Merge  
Michael Kolupaev 已提交
527 528
					 + toString(parts_to_fetch.size()) + " missing parts";
	bool insane =
M
Merge  
Michael Kolupaev 已提交
529 530 531 532
		parts_to_add.size() > data.settings.replicated_max_unexpectedly_merged_parts ||
		unexpected_parts.size() > data.settings.replicated_max_unexpected_parts ||
		expected_parts.size() > data.settings.replicated_max_missing_obsolete_parts ||
		parts_to_fetch.size() > data.settings.replicated_max_missing_active_parts;
M
Merge  
Michael Kolupaev 已提交
533

M
Merge  
Michael Kolupaev 已提交
534
	if (insane && !skip_sanity_checks)
M
Merge  
Michael Kolupaev 已提交
535 536 537
	{
		throw Exception("The local set of parts of table " + getTableName() + " doesn't look like the set of parts in ZooKeeper. "
			+ sanity_report,
M
Merge  
Michael Kolupaev 已提交
538
			ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS);
M
Merge  
Michael Kolupaev 已提交
539 540
	}

M
Merge  
Michael Kolupaev 已提交
541 542 543 544 545
	if (insane)
	{
		LOG_WARNING(log, sanity_report);
	}

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

M
Merge  
Michael Kolupaev 已提交
551
		zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
552
		checkPartAndAddToZooKeeper(part, ops);
M
Merge  
Michael Kolupaev 已提交
553
		zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
554
	}
M
Merge  
Michael Kolupaev 已提交
555

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

		zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
562
		ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + name + "/columns", -1));
M
Merge  
Michael Kolupaev 已提交
563 564 565 566 567 568 569 570 571 572 573 574
		ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + name + "/checksums", -1));
		ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + name, -1));
		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 已提交
575
		log_entry.source_replica = "";
M
Merge  
Michael Kolupaev 已提交
576 577 578
		log_entry.new_part_name = name;

		/// Полагаемся, что это происходит до загрузки очереди (loadQueue).
M
Merge  
Michael Kolupaev 已提交
579
		zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
580
		ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + name + "/columns", -1));
M
Merge  
Michael Kolupaev 已提交
581 582
		ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + name + "/checksums", -1));
		ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + name, -1));
M
Merge  
Michael Kolupaev 已提交
583 584
		ops.push_back(new zkutil::Op::Create(
			replica_path + "/queue/queue-", log_entry.toString(), zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
M
Merge  
Michael Kolupaev 已提交
585
		zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
586 587 588
	}

	/// Удалим лишние локальные куски.
589
	for (const MergeTreeData::DataPartPtr & part : unexpected_parts)
M
Merge  
Michael Kolupaev 已提交
590
	{
M
Merge  
Michael Kolupaev 已提交
591
		LOG_ERROR(log, "Renaming unexpected part " << part->name << " to ignored_" + part->name);
M
Merge  
Michael Kolupaev 已提交
592
		data.renameAndDetachPart(part, "ignored_", true);
M
Merge  
Michael Kolupaev 已提交
593 594
	}
}
M
Merge  
Michael Kolupaev 已提交
595

M
Merge  
Michael Kolupaev 已提交
596 597 598 599 600 601 602
void StorageReplicatedMergeTree::initVirtualParts()
{
	auto parts = data.getDataParts();
	for (const auto & part : parts)
		virtual_parts.add(part->name);
}

603
void StorageReplicatedMergeTree::checkPartAndAddToZooKeeper(const MergeTreeData::DataPartPtr & part, zkutil::Ops & ops, String part_name)
M
Merge  
Michael Kolupaev 已提交
604
{
M
Merge  
Michael Kolupaev 已提交
605 606 607
	if (part_name.empty())
		part_name = part->name;

608
	check(part->columns);
M
Merge  
Michael Kolupaev 已提交
609
	int expected_columns_version = columns_version;
610

M
Merge  
Michael Kolupaev 已提交
611 612 613 614 615
	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 已提交
616
	{
M
Merge  
Michael Kolupaev 已提交
617 618
		zkutil::Stat stat_before, stat_after;
		String columns_str;
M
Merge  
Michael Kolupaev 已提交
619
		if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name + "/columns", columns_str, &stat_before))
M
Merge  
Michael Kolupaev 已提交
620 621 622
			continue;
		if (columns_str != expected_columns_str)
		{
M
Merge  
Michael Kolupaev 已提交
623
			LOG_INFO(log, "Not checking checksums of part " << part_name << " with replica " << replica
M
Merge  
Michael Kolupaev 已提交
624 625 626
				<< " because columns are different");
			continue;
		}
M
Merge  
Michael Kolupaev 已提交
627
		String checksums_str;
M
Merge  
Michael Kolupaev 已提交
628 629
		/// Проверим, что версия ноды со столбцами не изменилась, пока мы читали checksums.
		/// Это гарантирует, что столбцы и чексуммы относятся к одним и тем же данным.
M
Merge  
Michael Kolupaev 已提交
630 631
		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 已提交
632
			stat_before.version != stat_after.version)
M
Merge  
Michael Kolupaev 已提交
633
		{
M
Merge  
Michael Kolupaev 已提交
634
			LOG_INFO(log, "Not checking checksums of part " << part_name << " with replica " << replica
M
Merge  
Michael Kolupaev 已提交
635 636
				<< " because part changed while we were reading its checksums");
			continue;
M
Merge  
Michael Kolupaev 已提交
637
		}
M
Merge  
Michael Kolupaev 已提交
638 639 640

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

M
Merge  
Michael Kolupaev 已提交
643
	if (zookeeper->exists(replica_path + "/parts/" + part_name))
M
Merge  
Michael Kolupaev 已提交
644
	{
M
Merge  
Michael Kolupaev 已提交
645
		LOG_ERROR(log, "checkPartAndAddToZooKeeper: node " << replica_path + "/parts/" + part_name << " already exists");
M
Merge  
Michael Kolupaev 已提交
646 647 648
		return;
	}

649 650 651
	ops.push_back(new zkutil::Op::Check(
		zookeeper_path + "/columns",
		expected_columns_version));
652 653 654 655 656 657 658 659 660
	ops.push_back(new zkutil::Op::Check(
		zookeeper_path + "/materialized_columns",
		expected_columns_version));
	ops.push_back(new zkutil::Op::Check(
		zookeeper_path + "/alias_columns",
		expected_columns_version));
	ops.push_back(new zkutil::Op::Check(
		zookeeper_path + "/column_defaults",
		expected_columns_version));
M
Merge  
Michael Kolupaev 已提交
661
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
662
		replica_path + "/parts/" + part_name,
M
Merge  
Michael Kolupaev 已提交
663
		"",
M
Merge  
Michael Kolupaev 已提交
664
		zookeeper->getDefaultACL(),
M
Merge  
Michael Kolupaev 已提交
665
		zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
666
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
667
		replica_path + "/parts/" + part_name + "/columns",
M
Merge  
Michael Kolupaev 已提交
668 669 670
		part->columns.toString(),
		zookeeper->getDefaultACL(),
		zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
671
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
672
		replica_path + "/parts/" + part_name + "/checksums",
M
Merge  
Michael Kolupaev 已提交
673
		part->checksums.toString(),
M
Merge  
Michael Kolupaev 已提交
674
		zookeeper->getDefaultACL(),
M
Merge  
Michael Kolupaev 已提交
675 676 677
		zkutil::CreateMode::Persistent));
}

M
Merge  
Michael Kolupaev 已提交
678 679
void StorageReplicatedMergeTree::clearOldParts()
{
M
Merge  
Michael Kolupaev 已提交
680 681
	auto table_lock = lockStructure(false);

M
Merge  
Michael Kolupaev 已提交
682 683
	MergeTreeData::DataPartsVector parts = data.grabOldParts();
	size_t count = parts.size();
M
Merge  
Michael Kolupaev 已提交
684

M
Merge  
Michael Kolupaev 已提交
685
	if (!count)
686 687
	{
		LOG_TRACE(log, "No old parts");
M
Merge  
Michael Kolupaev 已提交
688
		return;
689
	}
M
Merge  
Michael Kolupaev 已提交
690 691

	try
M
Merge  
Michael Kolupaev 已提交
692
	{
M
Merge  
Michael Kolupaev 已提交
693 694
		while (!parts.empty())
		{
695
			MergeTreeData::DataPartPtr & part = parts.back();
M
Merge  
Michael Kolupaev 已提交
696

M
Merge  
Michael Kolupaev 已提交
697
			LOG_DEBUG(log, "Removing " << part->name);
M
Merge  
Michael Kolupaev 已提交
698

M
Merge  
Michael Kolupaev 已提交
699 700 701 702
			zkutil::Ops ops;
			ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + part->name + "/columns", -1));
			ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + part->name + "/checksums", -1));
			ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + part->name, -1));
M
Merge  
Michael Kolupaev 已提交
703 704 705
			auto code = zookeeper->tryMulti(ops);
			if (code != ZOK)
				LOG_WARNING(log, "Couldn't remove " << part->name << " from ZooKeeper: " << zkutil::ZooKeeper::error2string(code));
M
Merge  
Michael Kolupaev 已提交
706 707 708 709 710 711 712

			part->remove();
			parts.pop_back();
		}
	}
	catch (...)
	{
713
		tryLogCurrentException(__PRETTY_FUNCTION__);
M
Merge  
Michael Kolupaev 已提交
714 715
		data.addOldParts(parts);
		throw;
M
Merge  
Michael Kolupaev 已提交
716 717
	}

M
Merge  
Michael Kolupaev 已提交
718
	LOG_DEBUG(log, "Removed " << count << " old parts");
M
Merge  
Michael Kolupaev 已提交
719 720
}

M
Merge  
Michael Kolupaev 已提交
721 722
void StorageReplicatedMergeTree::clearOldLogs()
{
M
Merge  
Michael Kolupaev 已提交
723 724 725 726 727 728 729 730 731 732 733
	zkutil::Stat stat;
	if (!zookeeper->exists(zookeeper_path + "/log", &stat))
		throw Exception(zookeeper_path + "/log doesn't exist", ErrorCodes::NOT_FOUND_NODE);

	int children_count = stat.numChildren;

	/// Будем ждать, пока накопятся в 1.1 раза больше записей, чем нужно.
	if (static_cast<double>(children_count) < data.settings.replicated_logs_to_keep * 1.1)
		return;

	Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas", &stat);
M
Merge  
Michael Kolupaev 已提交
734 735 736
	UInt64 min_pointer = std::numeric_limits<UInt64>::max();
	for (const String & replica : replicas)
	{
M
Merge  
Michael Kolupaev 已提交
737 738
		String pointer = zookeeper->get(zookeeper_path + "/replicas/" + replica + "/log_pointer");
		if (pointer.empty())
M
Merge  
Michael Kolupaev 已提交
739 740 741 742
			return;
		min_pointer = std::min(min_pointer, parse<UInt64>(pointer));
	}

M
Merge  
Michael Kolupaev 已提交
743
	Strings entries = zookeeper->getChildren(zookeeper_path + "/log");
M
Merge  
Michael Kolupaev 已提交
744
	std::sort(entries.begin(), entries.end());
M
Merge  
Michael Kolupaev 已提交
745

M
Merge  
Michael Kolupaev 已提交
746 747
	/// Не будем трогать последние replicated_logs_to_keep записей.
	entries.erase(entries.end() - std::min(entries.size(), data.settings.replicated_logs_to_keep), entries.end());
M
Merge  
Michael Kolupaev 已提交
748 749 750 751 752
	/// Не будем трогать записи, не меньшие min_pointer.
	entries.erase(std::lower_bound(entries.begin(), entries.end(), "log-" + padIndex(min_pointer)), entries.end());

	if (entries.empty())
		return;
M
Merge  
Michael Kolupaev 已提交
753

M
Merge  
Michael Kolupaev 已提交
754
	zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
755
	for (size_t i = 0; i < entries.size(); ++i)
M
Merge  
Michael Kolupaev 已提交
756
	{
M
Merge  
Michael Kolupaev 已提交
757 758 759
		ops.push_back(new zkutil::Op::Remove(zookeeper_path + "/log/" + entries[i], -1));

		if (ops.size() > 400 || i + 1 == entries.size())
M
Merge  
Michael Kolupaev 已提交
760 761 762 763 764 765 766 767
		{
			/// Одновременно с очисткой лога проверим, не добавилась ли реплика с тех пор, как мы получили список реплик.
			ops.push_back(new zkutil::Op::Check(zookeeper_path + "/replicas", stat.version));
			zookeeper->multi(ops);
			ops.clear();
		}
	}

M
Merge  
Michael Kolupaev 已提交
768
	LOG_DEBUG(log, "Removed " << entries.size() << " old log entries: " << entries.front() << " - " << entries.back());
M
Merge  
Michael Kolupaev 已提交
769 770 771 772 773
}

void StorageReplicatedMergeTree::clearOldBlocks()
{
	zkutil::Stat stat;
M
Merge  
Michael Kolupaev 已提交
774
	if (!zookeeper->exists(zookeeper_path + "/blocks", &stat))
M
Merge  
Michael Kolupaev 已提交
775 776
		throw Exception(zookeeper_path + "/blocks doesn't exist", ErrorCodes::NOT_FOUND_NODE);

777
	int children_count = stat.numChildren;
M
Merge  
Michael Kolupaev 已提交
778

M
Merge  
Michael Kolupaev 已提交
779
	/// Чтобы делать "асимптотически" меньше запросов exists, будем ждать, пока накопятся в 1.1 раза больше блоков, чем нужно.
M
Merge  
Michael Kolupaev 已提交
780
	if (static_cast<double>(children_count) < data.settings.replicated_deduplication_window * 1.1)
M
Merge  
Michael Kolupaev 已提交
781 782
		return;

M
Merge  
Michael Kolupaev 已提交
783
	LOG_TRACE(log, "Clearing about " << static_cast<size_t>(children_count) - data.settings.replicated_deduplication_window
M
Merge  
Michael Kolupaev 已提交
784
		<< " old blocks from ZooKeeper. This might take several minutes.");
M
Merge  
Michael Kolupaev 已提交
785

M
Merge  
Michael Kolupaev 已提交
786
	Strings blocks = zookeeper->getChildren(zookeeper_path + "/blocks");
M
Merge  
Michael Kolupaev 已提交
787 788 789 790 791 792

	std::vector<std::pair<Int64, String> > timed_blocks;

	for (const String & block : blocks)
	{
		zkutil::Stat stat;
M
Merge  
Michael Kolupaev 已提交
793
		zookeeper->exists(zookeeper_path + "/blocks/" + block, &stat);
794
		timed_blocks.push_back(std::make_pair(stat.czxid, block));
M
Merge  
Michael Kolupaev 已提交
795 796
	}

M
Merge  
Michael Kolupaev 已提交
797
	zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
798
	std::sort(timed_blocks.begin(), timed_blocks.end(), std::greater<std::pair<Int64, String>>());
M
Merge  
Michael Kolupaev 已提交
799 800
	for (size_t i = data.settings.replicated_deduplication_window; i <  timed_blocks.size(); ++i)
	{
M
Merge  
Michael Kolupaev 已提交
801
		ops.push_back(new zkutil::Op::Remove(zookeeper_path + "/blocks/" + timed_blocks[i].second + "/number", -1));
M
Merge  
Michael Kolupaev 已提交
802
		ops.push_back(new zkutil::Op::Remove(zookeeper_path + "/blocks/" + timed_blocks[i].second + "/columns", -1));
M
Merge  
Michael Kolupaev 已提交
803 804
		ops.push_back(new zkutil::Op::Remove(zookeeper_path + "/blocks/" + timed_blocks[i].second + "/checksums", -1));
		ops.push_back(new zkutil::Op::Remove(zookeeper_path + "/blocks/" + timed_blocks[i].second, -1));
M
Merge  
Michael Kolupaev 已提交
805
		if (ops.size() > 400 || i + 1 == timed_blocks.size())
M
Merge  
Michael Kolupaev 已提交
806 807 808 809
		{
			zookeeper->multi(ops);
			ops.clear();
		}
M
Merge  
Michael Kolupaev 已提交
810 811 812 813 814
	}

	LOG_TRACE(log, "Cleared " << blocks.size() - data.settings.replicated_deduplication_window << " old blocks from ZooKeeper");
}

M
Merge  
Michael Kolupaev 已提交
815 816
void StorageReplicatedMergeTree::loadQueue()
{
M
Merge  
Michael Kolupaev 已提交
817
	std::unique_lock<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
818

M
Merge  
Michael Kolupaev 已提交
819
	Strings children = zookeeper->getChildren(replica_path + "/queue");
M
Merge  
Michael Kolupaev 已提交
820 821 822
	std::sort(children.begin(), children.end());
	for (const String & child : children)
	{
M
Merge  
Michael Kolupaev 已提交
823
		String s = zookeeper->get(replica_path + "/queue/" + child);
M
Merge  
Michael Kolupaev 已提交
824 825 826
		LogEntryPtr entry = LogEntry::parse(s);
		entry->znode_name = child;
		entry->addResultToVirtualParts(*this);
M
Merge  
Michael Kolupaev 已提交
827
		queue.push_back(entry);
M
Merge  
Michael Kolupaev 已提交
828 829 830
	}
}

M
Merge  
Michael Kolupaev 已提交
831
void StorageReplicatedMergeTree::pullLogsToQueue(zkutil::EventPtr next_update_event)
M
Merge  
Michael Kolupaev 已提交
832
{
M
Merge  
Michael Kolupaev 已提交
833
	std::unique_lock<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
834

M
Merge  
Michael Kolupaev 已提交
835 836
	String index_str = zookeeper->get(replica_path + "/log_pointer");
	UInt64 index;
M
Merge  
Michael Kolupaev 已提交
837

M
Merge  
Michael Kolupaev 已提交
838
	if (index_str.empty())
M
Merge  
Michael Kolupaev 已提交
839
	{
M
Merge  
Michael Kolupaev 已提交
840 841 842
		/// Если у нас еще нет указателя на лог, поставим указатель на первую запись в нем.
		Strings entries = zookeeper->getChildren(zookeeper_path + "/log");
		index = entries.empty() ? 0 : parse<UInt64>(std::min_element(entries.begin(), entries.end())->substr(strlen("log-")));
M
Merge  
Michael Kolupaev 已提交
843

M
Merge  
Michael Kolupaev 已提交
844
		zookeeper->set(replica_path + "/log_pointer", toString(index));
M
Merge  
Michael Kolupaev 已提交
845 846
	}
	else
M
Merge  
Michael Kolupaev 已提交
847
	{
M
Merge  
Michael Kolupaev 已提交
848
		index = parse<UInt64>(index_str);
M
Merge  
Michael Kolupaev 已提交
849
	}
M
Merge  
Michael Kolupaev 已提交
850

M
Merge  
Michael Kolupaev 已提交
851 852
	UInt64 first_index = index;

M
Merge  
Michael Kolupaev 已提交
853
	size_t count = 0;
M
Merge  
Michael Kolupaev 已提交
854 855
	String entry_str;
	while (zookeeper->tryGet(zookeeper_path + "/log/log-" + padIndex(index), entry_str))
M
Merge  
Michael Kolupaev 已提交
856
	{
M
Merge  
Michael Kolupaev 已提交
857
		++count;
M
Merge  
Michael Kolupaev 已提交
858
		++index;
M
Merge  
Michael Kolupaev 已提交
859

M
Merge  
Michael Kolupaev 已提交
860
		LogEntryPtr entry = LogEntry::parse(entry_str);
M
Merge  
Michael Kolupaev 已提交
861

M
Merge  
Michael Kolupaev 已提交
862 863 864
		/// Одновременно добавим запись в очередь и продвинем указатель на лог.
		zkutil::Ops ops;
		ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
865
			replica_path + "/queue/queue-", entry_str, zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
M
Merge  
Michael Kolupaev 已提交
866
		ops.push_back(new zkutil::Op::SetData(
M
Merge  
Michael Kolupaev 已提交
867
			replica_path + "/log_pointer", toString(index), -1));
M
Merge  
Michael Kolupaev 已提交
868
		auto results = zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
869

870
		String path_created = dynamic_cast<zkutil::Op::Create &>(ops[0]).getPathCreated();
M
Merge  
Michael Kolupaev 已提交
871 872
		entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
		entry->addResultToVirtualParts(*this);
M
Merge  
Michael Kolupaev 已提交
873
		queue.push_back(entry);
M
Merge  
Michael Kolupaev 已提交
874
	}
M
Merge  
Michael Kolupaev 已提交
875

M
Merge  
Michael Kolupaev 已提交
876 877 878 879
	if (next_update_event)
	{
		if (zookeeper->exists(zookeeper_path + "/log/log-" + padIndex(index), nullptr, next_update_event))
			next_update_event->set();
M
Merge  
Michael Kolupaev 已提交
880
	}
M
Merge  
Michael Kolupaev 已提交
881

M
Merge  
Michael Kolupaev 已提交
882 883 884
	if (!count)
		return;

M
Merge  
Michael Kolupaev 已提交
885 886
	if (queue_task_handle)
		queue_task_handle->wake();
M
Merge  
Michael Kolupaev 已提交
887

M
Merge  
Michael Kolupaev 已提交
888
	LOG_DEBUG(log, "Pulled " << count << " entries to queue: log-" << padIndex(first_index) << " - log-" << padIndex(index - 1));
M
Merge  
Michael Kolupaev 已提交
889 890
}

M
Merge  
Michael Kolupaev 已提交
891
bool StorageReplicatedMergeTree::shouldExecuteLogEntry(const LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
892
{
M
Merge  
Michael Kolupaev 已提交
893 894
	if ((entry.type == LogEntry::MERGE_PARTS || entry.type == LogEntry::GET_PART || entry.type == LogEntry::ATTACH_PART)
		&& future_parts.count(entry.new_part_name))
M
Merge  
Michael Kolupaev 已提交
895 896 897 898 899 900
	{
		LOG_DEBUG(log, "Not executing log entry for part " << entry.new_part_name <<
			" because another log entry for the same part is being processed. This shouldn't happen often.");
		return false;
	}

M
Merge  
Michael Kolupaev 已提交
901 902 903 904 905 906 907 908 909 910 911
	if (entry.type == LogEntry::MERGE_PARTS)
	{
		/** Если какая-то из нужных частей сейчас передается или мерджится, подождем окончания этой операции.
		  * Иначе, даже если всех нужных частей для мерджа нет, нужно попытаться сделать мердж.
		  * Если каких-то частей не хватает, вместо мерджа будет попытка скачать кусок.
		  * Такая ситуация возможна, если получение какого-то куска пофейлилось, и его переместили в конец очереди.
		  */
		for (const auto & name : entry.parts_to_merge)
		{
			if (future_parts.count(name))
			{
M
Merge  
Michael Kolupaev 已提交
912
				LOG_TRACE(log, "Not merging into part " << entry.new_part_name << " because part " << name << " is not ready yet.");
M
Merge  
Michael Kolupaev 已提交
913 914 915 916 917 918
				return false;
			}
		}
	}

	return true;
M
Merge  
Michael Kolupaev 已提交
919 920
}

M
Merge  
Michael Kolupaev 已提交
921
bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry, BackgroundProcessingPool::Context & pool_context)
M
Merge  
Michael Kolupaev 已提交
922
{
M
Merge  
Michael Kolupaev 已提交
923
	if (entry.type == LogEntry::DROP_RANGE)
M
Merge  
Michael Kolupaev 已提交
924 925 926 927
	{
		executeDropRange(entry);
		return true;
	}
M
Merge  
Michael Kolupaev 已提交
928

M
Merge  
Michael Kolupaev 已提交
929
	if (entry.type == LogEntry::GET_PART ||
M
Merge  
Michael Kolupaev 已提交
930 931
		entry.type == LogEntry::MERGE_PARTS ||
		entry.type == LogEntry::ATTACH_PART)
M
Merge  
Michael Kolupaev 已提交
932 933
	{
		/// Если у нас уже есть этот кусок или покрывающий его кусок, ничего делать не нужно.
M
Merge  
Michael Kolupaev 已提交
934
		MergeTreeData::DataPartPtr containing_part = data.getActiveContainingPart(entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
935 936

		/// Даже если кусок есть локально, его (в исключительных случаях) может не быть в zookeeper.
M
Merge  
Michael Kolupaev 已提交
937
		if (containing_part && zookeeper->exists(replica_path + "/parts/" + containing_part->name))
M
Merge  
Michael Kolupaev 已提交
938
		{
M
Merge  
Michael Kolupaev 已提交
939 940
			if (!(entry.type == LogEntry::GET_PART && entry.source_replica == replica_name))
				LOG_DEBUG(log, "Skipping action for part " + entry.new_part_name + " - part already exists");
M
Merge  
Michael Kolupaev 已提交
941
			return true;
M
Merge  
Michael Kolupaev 已提交
942
		}
M
Merge  
Michael Kolupaev 已提交
943 944
	}

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

	bool do_fetch = false;

M
Merge  
Michael Kolupaev 已提交
950 951
	if (entry.type == LogEntry::GET_PART)
	{
M
Merge  
Michael Kolupaev 已提交
952
		do_fetch = true;
M
Merge  
Michael Kolupaev 已提交
953
	}
M
Merge  
Michael Kolupaev 已提交
954 955 956 957
	else if (entry.type == LogEntry::ATTACH_PART)
	{
		do_fetch = !executeAttachPart(entry);
	}
M
Merge  
Michael Kolupaev 已提交
958 959
	else if (entry.type == LogEntry::MERGE_PARTS)
	{
960 961 962 963 964 965 966 967
		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 已提交
968
		MergeTreeData::DataPartsVector parts;
969
		bool have_all_parts = true;
M
Merge  
Michael Kolupaev 已提交
970 971
		for (const String & name : entry.parts_to_merge)
		{
M
Merge  
Michael Kolupaev 已提交
972
			MergeTreeData::DataPartPtr part = data.getActiveContainingPart(name);
M
Merge  
Michael Kolupaev 已提交
973 974 975 976 977 978 979
			if (!part)
			{
				have_all_parts = false;
				break;
			}
			if (part->name != name)
			{
M
Merge  
Michael Kolupaev 已提交
980 981
				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 已提交
982 983 984
				have_all_parts = false;
				break;
			}
M
Merge  
Michael Kolupaev 已提交
985 986
			parts.push_back(part);
		}
M
Merge  
Michael Kolupaev 已提交
987

M
Merge  
Michael Kolupaev 已提交
988
		if (!have_all_parts)
M
Merge  
Michael Kolupaev 已提交
989
		{
M
Merge  
Michael Kolupaev 已提交
990 991 992
			/// Если нет всех нужных кусков, попробуем взять у кого-нибудь уже помердженный кусок.
			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 已提交
993
		}
M
Merge  
Michael Kolupaev 已提交
994 995
		else
		{
M
Merge  
Michael Kolupaev 已提交
996 997 998
			/// Если собираемся сливать большие куски, увеличим счетчик потоков, сливающих большие куски.
			for (const auto & part : parts)
			{
M
Merge  
Michael Kolupaev 已提交
999
				if (part->size_in_bytes > data.settings.max_bytes_to_merge_parts_small)
M
Merge  
Michael Kolupaev 已提交
1000 1001 1002 1003 1004 1005 1006
				{
					pool_context.incrementCounter("big merges");
					pool_context.incrementCounter("replicated big merges");
					break;
				}
			}

M
Merge  
Michael Kolupaev 已提交
1007
			auto table_lock = lockStructure(false);
1008

1009
			const auto & merge_entry = context.getMergeList().insert(database_name, table_name, entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
1010
			MergeTreeData::Transaction transaction;
1011
			MergeTreeData::DataPartPtr part = merger.mergeParts(parts, entry.new_part_name, *merge_entry, &transaction);
M
Merge  
Michael Kolupaev 已提交
1012 1013

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

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

M
Merge  
Michael Kolupaev 已提交
1020
			zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
1021 1022 1023 1024

			/** При ZCONNECTIONLOSS или ZOPERATIONTIMEOUT можем зря откатить локальные изменения кусков.
			  * Это не проблема, потому что в таком случае слияние останется в очереди, и мы попробуем снова.
			  */
M
Merge  
Michael Kolupaev 已提交
1025
			transaction.commit();
M
Merge  
Michael Kolupaev 已提交
1026
			merge_selecting_event.set();
M
Merge  
Michael Kolupaev 已提交
1027 1028 1029

			ProfileEvents::increment(ProfileEvents::ReplicatedPartMerges);
		}
M
Merge  
Michael Kolupaev 已提交
1030 1031 1032 1033 1034
	}
	else
	{
		throw Exception("Unexpected log entry type: " + toString(static_cast<int>(entry.type)));
	}
M
Merge  
Michael Kolupaev 已提交
1035 1036 1037

	if (do_fetch)
	{
M
Merge  
Michael Kolupaev 已提交
1038 1039
		String replica;

M
Merge  
Michael Kolupaev 已提交
1040 1041
		try
		{
M
Merge  
Michael Kolupaev 已提交
1042
			replica = findReplicaHavingPart(entry.new_part_name, true);
M
Merge  
Michael Kolupaev 已提交
1043 1044 1045 1046 1047
			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 已提交
1048 1049 1050 1051 1052 1053 1054 1055 1056 1057 1058 1059 1060
			fetchPart(entry.new_part_name, replica);

			if (entry.type == LogEntry::MERGE_PARTS)
				ProfileEvents::increment(ProfileEvents::ReplicatedPartFetchesOfMerged);
		}
		catch (...)
		{
			/** Если не получилось скачать кусок, нужный для какого-то мерджа, лучше не пытаться получить другие куски для этого мерджа,
			  * а попытаться сразу получить помердженный кусок. Чтобы так получилось, переместим действия для получения остальных кусков
			  * для этого мерджа в конец очереди.
			  */
			try
			{
M
Merge  
Michael Kolupaev 已提交
1061
				std::unique_lock<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1062 1063 1064 1065 1066 1067

				/// Найдем действие по объединению этого куска с другими. Запомним других.
				StringSet parts_for_merge;
				LogEntries::iterator merge_entry;
				for (LogEntries::iterator it = queue.begin(); it != queue.end(); ++it)
				{
M
Merge  
Michael Kolupaev 已提交
1068
					if ((*it)->type == LogEntry::MERGE_PARTS)
M
Merge  
Michael Kolupaev 已提交
1069
					{
M
Merge  
Michael Kolupaev 已提交
1070 1071
						if (std::find((*it)->parts_to_merge.begin(), (*it)->parts_to_merge.end(), entry.new_part_name)
							!= (*it)->parts_to_merge.end())
M
Merge  
Michael Kolupaev 已提交
1072
						{
M
Merge  
Michael Kolupaev 已提交
1073
							parts_for_merge = StringSet((*it)->parts_to_merge.begin(), (*it)->parts_to_merge.end());
M
Merge  
Michael Kolupaev 已提交
1074 1075 1076 1077 1078 1079 1080 1081 1082 1083 1084 1085 1086 1087 1088 1089 1090
							merge_entry = it;
							break;
						}
					}
				}

				if (!parts_for_merge.empty())
				{
					/// Переместим в конец очереди действия, получающие parts_for_merge.
					for (LogEntries::iterator it = queue.begin(); it != queue.end();)
					{
						auto it0 = it;
						++it;

						if (it0 == merge_entry)
							break;

M
Merge  
Michael Kolupaev 已提交
1091 1092
						if (((*it0)->type == LogEntry::MERGE_PARTS || (*it0)->type == LogEntry::GET_PART)
							&& parts_for_merge.count((*it0)->new_part_name))
M
Merge  
Michael Kolupaev 已提交
1093 1094 1095 1096
						{
							queue.splice(queue.end(), queue, it0, it);
						}
					}
M
Merge  
Michael Kolupaev 已提交
1097 1098 1099 1100 1101 1102 1103 1104 1105

					/** Если этого куска ни у кого нет, но в очереди упоминается мердж с его участием, то наверно этот кусок такой старый,
					  *  что его все померджили и удалили. Не будем бросать исключение, чтобы queueTask лишний раз не спала.
					  */
					if (replica.empty())
					{
						LOG_INFO(log, "No replica has part " << entry.new_part_name << ". Will fetch merged part instead.");
						return false;
					}
M
Merge  
Michael Kolupaev 已提交
1106
				}
M
Merge  
Michael Kolupaev 已提交
1107 1108 1109 1110

				/// Если ни у кого нет куска, и в очереди нет слияний с его участием, проверим, есть ли у кого-то покрывающий его.
				if (replica.empty())
					enqueuePartForCheck(entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
1111 1112 1113 1114 1115 1116 1117 1118 1119
			}
			catch (...)
			{
				tryLogCurrentException(__PRETTY_FUNCTION__);
			}

			throw;
		}
	}
M
Merge  
Michael Kolupaev 已提交
1120 1121

	return true;
M
Merge  
Michael Kolupaev 已提交
1122 1123
}

M
Merge  
Michael Kolupaev 已提交
1124
void StorageReplicatedMergeTree::executeDropRange(const StorageReplicatedMergeTree::LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
1125
{
M
Merge  
Michael Kolupaev 已提交
1126
	LOG_INFO(log, (entry.detach ? "Detaching" : "Removing") << " parts inside " << entry.new_part_name << ".");
M
Merge  
Michael Kolupaev 已提交
1127 1128 1129 1130 1131 1132 1133 1134 1135 1136 1137 1138 1139 1140 1141 1142 1143 1144 1145 1146 1147 1148 1149 1150 1151 1152 1153 1154 1155

	{
		LogEntries to_wait;
		size_t removed_entries = 0;

		/// Удалим из очереди операции с кусками, содержащимися в удаляемом диапазоне.
		std::unique_lock<std::mutex> lock(queue_mutex);
		for (LogEntries::iterator it = queue.begin(); it != queue.end();)
		{
			if (((*it)->type == LogEntry::GET_PART || (*it)->type == LogEntry::MERGE_PARTS) &&
				ActiveDataPartSet::contains(entry.new_part_name, (*it)->new_part_name))
			{
				if ((*it)->currently_executing)
					to_wait.push_back(*it);
				auto code = zookeeper->tryRemove(replica_path + "/queue/" + (*it)->znode_name);
				if (code != ZOK)
					LOG_INFO(log, "Couldn't remove " << replica_path + "/queue/" + (*it)->znode_name << ": "
						<< zkutil::ZooKeeper::error2string(code));
				queue.erase(it++);
				++removed_entries;
			}
			else
				++it;
		}

		LOG_DEBUG(log, "Removed " << removed_entries << " entries from queue. "
			"Waiting for " << to_wait.size() << " entries that are currently executing.");

		/// Дождемся завершения операций с кусками, содержащимися в удаляемом диапазоне.
M
Merge  
Michael Kolupaev 已提交
1156 1157
		for (LogEntryPtr & entry : to_wait)
			entry->execution_complete.wait(lock, [&entry] { return !entry->currently_executing; });
M
Merge  
Michael Kolupaev 已提交
1158 1159
	}

M
Merge  
Michael Kolupaev 已提交
1160
	LOG_DEBUG(log, (entry.detach ? "Detaching" : "Removing") << " parts.");
M
Merge  
Michael Kolupaev 已提交
1161 1162 1163 1164 1165 1166 1167 1168 1169 1170 1171
	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 已提交
1172 1173 1174 1175
		/// Если кусок удалять не нужно, надежнее переместить директорию до изменений в ZooKeeper.
		if (entry.detach)
			data.renameAndDetachPart(part);

M
Merge  
Michael Kolupaev 已提交
1176 1177 1178 1179 1180 1181
		zkutil::Ops ops;
		ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + part->name + "/columns", -1));
		ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + part->name + "/checksums", -1));
		ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + part->name, -1));
		zookeeper->multi(ops);

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

M
Merge  
Michael Kolupaev 已提交
1187
	LOG_INFO(log, (entry.detach ? "Detached " : "Removed ") << removed_parts << " parts inside " << entry.new_part_name << ".");
M
Merge  
Michael Kolupaev 已提交
1188 1189 1190

	if (unreplicated_data)
	{
M
Merge  
Michael Kolupaev 已提交
1191 1192
		Poco::ScopedLock<Poco::FastMutex> unreplicated_lock(unreplicated_mutex);

M
Merge  
Michael Kolupaev 已提交
1193 1194 1195 1196 1197 1198 1199 1200 1201 1202 1203 1204 1205 1206 1207
		removed_parts = 0;
		parts = unreplicated_data->getDataParts();
		for (const auto & part : parts)
		{
			if (!ActiveDataPartSet::contains(entry.new_part_name, part->name))
				continue;
			LOG_DEBUG(log, "Removing unreplicated part " << part->name);
			++removed_parts;

			if (entry.detach)
				unreplicated_data->renameAndDetachPart(part, "");
			else
				unreplicated_data->replaceParts({part}, {}, false);
		}
	}
M
Merge  
Michael Kolupaev 已提交
1208 1209 1210 1211 1212 1213 1214 1215
}

bool StorageReplicatedMergeTree::executeAttachPart(const StorageReplicatedMergeTree::LogEntry & entry)
{
	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 已提交
1216
	if (!Poco::File(data.getFullPath() + source_path).exists())
M
Merge  
Michael Kolupaev 已提交
1217
	{
M
Merge  
Michael Kolupaev 已提交
1218
		LOG_INFO(log, "No part at " << source_path << ". Will fetch it instead");
M
Merge  
Michael Kolupaev 已提交
1219 1220 1221 1222 1223 1224 1225 1226 1227 1228 1229 1230 1231 1232 1233 1234 1235 1236 1237 1238 1239 1240 1241 1242 1243 1244 1245 1246
		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 已提交
1247

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

M
Merge  
Michael Kolupaev 已提交
1251 1252 1253
	return true;
}

M
Merge  
Michael Kolupaev 已提交
1254 1255 1256 1257
void StorageReplicatedMergeTree::queueUpdatingThread()
{
	while (!shutdown_called)
	{
M
Merge  
Michael Kolupaev 已提交
1258 1259
		try
		{
M
Merge  
Michael Kolupaev 已提交
1260
			pullLogsToQueue(queue_updating_event);
M
Merge  
Michael Kolupaev 已提交
1261

M
Merge  
Michael Kolupaev 已提交
1262
			queue_updating_event->wait();
M
Merge  
Michael Kolupaev 已提交
1263
		}
1264
		catch (const zkutil::KeeperException & e)
M
Merge  
Michael Kolupaev 已提交
1265 1266 1267 1268 1269 1270 1271 1272
		{
			if (e.code == ZINVALIDSTATE)
				restarting_event.set();

			tryLogCurrentException(__PRETTY_FUNCTION__);

			queue_updating_event->tryWait(ERROR_SLEEP_MS);
		}
M
Merge  
Michael Kolupaev 已提交
1273 1274 1275 1276
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);

M
Merge  
Michael Kolupaev 已提交
1277 1278
			queue_updating_event->tryWait(ERROR_SLEEP_MS);
		}
M
Merge  
Michael Kolupaev 已提交
1279
	}
M
Merge  
Michael Kolupaev 已提交
1280 1281

	LOG_DEBUG(log, "queue updating thread finished");
M
Merge  
Michael Kolupaev 已提交
1282
}
M
Merge  
Michael Kolupaev 已提交
1283

M
Merge  
Michael Kolupaev 已提交
1284
bool StorageReplicatedMergeTree::queueTask(BackgroundProcessingPool::Context & pool_context)
M
Merge  
Michael Kolupaev 已提交
1285
{
M
Merge  
Michael Kolupaev 已提交
1286
	LogEntryPtr entry;
M
Merge  
Michael Kolupaev 已提交
1287

M
Merge  
Michael Kolupaev 已提交
1288 1289
	try
	{
M
Merge  
Michael Kolupaev 已提交
1290
		std::unique_lock<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1291 1292
		bool empty = queue.empty();
		if (!empty)
M
Merge  
Michael Kolupaev 已提交
1293
		{
M
Merge  
Michael Kolupaev 已提交
1294
			for (LogEntries::iterator it = queue.begin(); it != queue.end(); ++it)
M
Merge  
Michael Kolupaev 已提交
1295
			{
M
Merge  
Michael Kolupaev 已提交
1296
				if (!(*it)->currently_executing && shouldExecuteLogEntry(**it))
M
Merge  
Michael Kolupaev 已提交
1297
				{
M
Merge  
Michael Kolupaev 已提交
1298
					entry = *it;
M
Merge  
Michael Kolupaev 已提交
1299
					entry->tagPartAsFuture(*this);
M
Merge  
Michael Kolupaev 已提交
1300
					queue.splice(queue.end(), queue, it);
M
Merge  
Michael Kolupaev 已提交
1301
					entry->currently_executing = true;
M
Merge  
Michael Kolupaev 已提交
1302
					break;
M
Merge  
Michael Kolupaev 已提交
1303
				}
M
Merge  
Michael Kolupaev 已提交
1304 1305
			}
		}
M
Merge  
Michael Kolupaev 已提交
1306 1307 1308 1309 1310
	}
	catch (...)
	{
		tryLogCurrentException(__PRETTY_FUNCTION__);
	}
M
Merge  
Michael Kolupaev 已提交
1311

M
Merge  
Michael Kolupaev 已提交
1312
	if (!entry)
M
Merge  
Michael Kolupaev 已提交
1313
		return false;
M
Merge  
Michael Kolupaev 已提交
1314

M
Merge  
Michael Kolupaev 已提交
1315
	bool exception = true;
M
Merge  
Michael Kolupaev 已提交
1316
	bool success = false;
M
Merge  
Michael Kolupaev 已提交
1317

M
Merge  
Michael Kolupaev 已提交
1318 1319
	try
	{
M
Merge  
Michael Kolupaev 已提交
1320
		if (executeLogEntry(*entry, pool_context))
M
Merge  
Michael Kolupaev 已提交
1321
		{
M
Merge  
Michael Kolupaev 已提交
1322
			auto code = zookeeper->tryRemove(replica_path + "/queue/" + entry->znode_name);
M
Merge  
Michael Kolupaev 已提交
1323

M
Merge  
Michael Kolupaev 已提交
1324
			if (code != ZOK)
M
Merge  
Michael Kolupaev 已提交
1325
				LOG_ERROR(log, "Couldn't remove " << replica_path + "/queue/" + entry->znode_name << ": "
M
Merge  
Michael Kolupaev 已提交
1326 1327 1328
					<< zkutil::ZooKeeper::error2string(code) + ". This shouldn't happen often.");

			success = true;
M
Merge  
Michael Kolupaev 已提交
1329
		}
M
Merge  
Michael Kolupaev 已提交
1330

M
Merge  
Michael Kolupaev 已提交
1331
		exception = false;
M
Merge  
Michael Kolupaev 已提交
1332 1333 1334 1335
	}
	catch (Exception & e)
	{
		if (e.code() == ErrorCodes::NO_REPLICA_HAS_PART)
M
Merge  
Michael Kolupaev 已提交
1336
			/// Если ни у кого нет нужного куска, наверно, просто не все реплики работают; не будем писать в лог с уровнем Error.
M
Merge  
Michael Kolupaev 已提交
1337 1338
			LOG_INFO(log, e.displayText());
		else
M
Merge  
Michael Kolupaev 已提交
1339
			tryLogCurrentException(__PRETTY_FUNCTION__);
M
Merge  
Michael Kolupaev 已提交
1340 1341 1342 1343 1344
	}
	catch (...)
	{
		tryLogCurrentException(__PRETTY_FUNCTION__);
	}
M
Merge  
Michael Kolupaev 已提交
1345

M
Merge  
Michael Kolupaev 已提交
1346 1347 1348 1349 1350 1351 1352 1353
	entry->future_part_tagger = nullptr;

	std::unique_lock<std::mutex> lock(queue_mutex);

	entry->currently_executing = false;
	entry->execution_complete.notify_all();

	if (success)
M
Merge  
Michael Kolupaev 已提交
1354
	{
M
Merge  
Michael Kolupaev 已提交
1355 1356 1357
		/// Удалим задание из очереди.
		/// Нельзя просто обратиться по заранее сохраненному итератору, потому что задание мог успеть удалить кто-то другой.
		for (LogEntries::iterator it = queue.end(); it != queue.begin();)
M
Merge  
Michael Kolupaev 已提交
1358
		{
M
Merge  
Michael Kolupaev 已提交
1359 1360 1361
			--it;
			if (*it == entry)
			{
M
Merge  
Michael Kolupaev 已提交
1362
				queue.erase(it);
M
Merge  
Michael Kolupaev 已提交
1363 1364
				break;
			}
M
Merge  
Michael Kolupaev 已提交
1365
		}
M
Merge  
Michael Kolupaev 已提交
1366
	}
M
Merge  
Michael Kolupaev 已提交
1367

M
Merge  
Michael Kolupaev 已提交
1368 1369
	/// Если не было исключения, не нужно спать.
	return !exception;
M
Merge  
Michael Kolupaev 已提交
1370 1371
}

M
Merge  
Michael Kolupaev 已提交
1372 1373
void StorageReplicatedMergeTree::mergeSelectingThread()
{
M
Merge  
Michael Kolupaev 已提交
1374
	bool need_pull = true;
M
Merge  
Michael Kolupaev 已提交
1375

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

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

1402 1403 1404 1405 1406 1407 1408
		String month_name = left->name.substr(0, 6);

		/// Можно слить куски, если все номера между ними заброшены - не соответствуют никаким блокам.
		for (UInt64 number = left->right + 1; number <= right->left - 1; ++number)	/// Номера блоков больше нуля.
		{
			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 已提交
1409

1410 1411 1412 1413 1414 1415 1416
			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 已提交
1417 1418
	};

M
Merge  
Michael Kolupaev 已提交
1419 1420
	while (!shutdown_called && is_leader_node)
	{
M
Michael Kolupaev 已提交
1421
		bool success = false;
M
Merge  
Michael Kolupaev 已提交
1422

M
Michael Kolupaev 已提交
1423
		try
M
Merge  
Michael Kolupaev 已提交
1424
		{
M
Merge  
Michael Kolupaev 已提交
1425 1426
			std::unique_lock<std::mutex> merge_selecting_lock(merge_selecting_mutex);

M
Merge  
Michael Kolupaev 已提交
1427 1428 1429 1430 1431 1432 1433 1434
			if (need_pull)
			{
				/// Нужно загрузить новую запись в очередь перед тем, как выбирать куски для слияния.
				///  (чтобы кусок добавился в virtual_parts).
				pullLogsToQueue();
				need_pull = false;
			}

1435 1436 1437 1438 1439 1440 1441
			/** Сколько в очереди или в фоновом потоке мерджей крупных кусков.
			  * Если их больше половины от размера пула потоков для мерджа, то можно мерджить только мелкие куски.
			  */
			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 已提交
1442
			size_t merges_queued = 0;
1443
			size_t big_merges_queued = 0;
M
Merge  
Michael Kolupaev 已提交
1444

1445
			if (big_merges_current < max_number_of_big_merges)
M
Michael Kolupaev 已提交
1446
			{
M
Merge  
Michael Kolupaev 已提交
1447
				std::unique_lock<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1448

M
Michael Kolupaev 已提交
1449
				for (const auto & entry : queue)
M
Merge  
Michael Kolupaev 已提交
1450
				{
M
Merge  
Michael Kolupaev 已提交
1451
					if (entry->type == LogEntry::MERGE_PARTS)
M
Merge  
Michael Kolupaev 已提交
1452
					{
M
Michael Kolupaev 已提交
1453
						++merges_queued;
M
Merge  
Michael Kolupaev 已提交
1454

1455
						if (big_merges_current + big_merges_queued < max_number_of_big_merges)
M
Merge  
Michael Kolupaev 已提交
1456
						{
M
Merge  
Michael Kolupaev 已提交
1457
							for (const String & name : entry->parts_to_merge)
M
Merge  
Michael Kolupaev 已提交
1458
							{
M
Merge  
Michael Kolupaev 已提交
1459
								MergeTreeData::DataPartPtr part = data.getActiveContainingPart(name);
M
Merge  
Michael Kolupaev 已提交
1460 1461
								if (!part || part->name != name)
									continue;
1462

M
Merge  
Michael Kolupaev 已提交
1463
								if (part->size_in_bytes > data.settings.max_bytes_to_merge_parts_small)
M
Merge  
Michael Kolupaev 已提交
1464
								{
1465
									++big_merges_queued;
M
Merge  
Michael Kolupaev 已提交
1466 1467 1468 1469 1470 1471
									break;
								}
							}
						}
					}
				}
M
Michael Kolupaev 已提交
1472
			}
M
Merge  
Michael Kolupaev 已提交
1473

1474 1475 1476 1477 1478 1479 1480 1481
			bool only_small = big_merges_current + big_merges_queued >= max_number_of_big_merges;

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

M
Merge  
Michael Kolupaev 已提交
1482
			do
M
Michael Kolupaev 已提交
1483
			{
M
Merge  
Michael Kolupaev 已提交
1484
				if (merges_queued >= data.settings.max_replicated_merges_in_queue)
1485 1486
				{
					LOG_TRACE(log, "Number of queued merges is greater than max_replicated_merges_in_queue, so won't select new parts to merge.");
M
Merge  
Michael Kolupaev 已提交
1487
					break;
1488
				}
M
Merge  
Michael Kolupaev 已提交
1489

M
Merge  
Michael Kolupaev 已提交
1490
				MergeTreeData::DataPartsVector parts;
M
Merge  
Michael Kolupaev 已提交
1491

M
Merge  
Michael Kolupaev 已提交
1492 1493
				String merged_name;

1494 1495 1496 1497
				if (   !merger.selectPartsToMerge(parts, merged_name, MergeTreeDataMerger::NO_LIMIT, false, false, only_small, can_merge)
					&& !merger.selectPartsToMerge(parts, merged_name, MergeTreeDataMerger::NO_LIMIT, true, false, only_small, can_merge))
				{
					LOG_INFO(log, "No parts to merge");
M
Merge  
Michael Kolupaev 已提交
1498
					break;
1499
				}
M
Merge  
Michael Kolupaev 已提交
1500

M
Merge  
Michael Kolupaev 已提交
1501 1502 1503 1504 1505 1506 1507 1508 1509 1510 1511 1512 1513 1514
				bool all_in_zk = true;
				for (const auto & part : parts)
				{
					/// Если о каком-то из кусков нет информации в ZK, не будем сливать.
					if (!zookeeper->exists(replica_path + "/parts/" + part->name))
					{
						LOG_WARNING(log, "Part " << part->name << " exists locally but not in ZooKeeper.");
						enqueuePartForCheck(part->name);
						all_in_zk = false;
					}
				}
				if (!all_in_zk)
					break;

M
Merge  
Michael Kolupaev 已提交
1515 1516 1517 1518
				LogEntry entry;
				entry.type = LogEntry::MERGE_PARTS;
				entry.source_replica = replica_name;
				entry.new_part_name = merged_name;
M
Merge  
Michael Kolupaev 已提交
1519

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

M
Merge  
Michael Kolupaev 已提交
1523
				need_pull = true;
M
Merge  
Michael Kolupaev 已提交
1524

M
Merge  
Michael Kolupaev 已提交
1525
				zookeeper->create(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential);
M
Merge  
Michael Kolupaev 已提交
1526 1527 1528

				String month_name = parts[0]->name.substr(0, 6);
				for (size_t i = 0; i + 1 < parts.size(); ++i)
M
Merge  
Michael Kolupaev 已提交
1529
				{
M
Merge  
Michael Kolupaev 已提交
1530 1531 1532
					/// Уберем больше не нужные отметки о несуществующих блоках.
					for (UInt64 number = parts[i]->right + 1; number <= parts[i + 1]->left - 1; ++number)
					{
M
Merge  
Michael Kolupaev 已提交
1533 1534
						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 已提交
1535
					}
M
Merge  
Michael Kolupaev 已提交
1536
				}
M
Merge  
Michael Kolupaev 已提交
1537 1538

				success = true;
M
Merge  
Michael Kolupaev 已提交
1539
			}
1540
			while (false);
M
Merge  
Michael Kolupaev 已提交
1541 1542 1543 1544 1545 1546
		}
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);
		}

M
Merge  
Michael Kolupaev 已提交
1547
		if (shutdown_called || !is_leader_node)
M
Merge  
Michael Kolupaev 已提交
1548 1549
			break;

M
Merge  
Michael Kolupaev 已提交
1550
		if (!success)
M
Merge  
Michael Kolupaev 已提交
1551
			merge_selecting_event.tryWait(MERGE_SELECTING_SLEEP_MS);
M
Merge  
Michael Kolupaev 已提交
1552
	}
M
Merge  
Michael Kolupaev 已提交
1553

1554
	LOG_DEBUG(log, "Merge selecting thread finished");
M
Merge  
Michael Kolupaev 已提交
1555 1556
}

M
Merge  
Michael Kolupaev 已提交
1557
void StorageReplicatedMergeTree::cleanupThread()
M
Merge  
Michael Kolupaev 已提交
1558
{
M
Merge  
Michael Kolupaev 已提交
1559
	while (!shutdown_called)
M
Merge  
Michael Kolupaev 已提交
1560 1561
	{
		try
M
Merge  
Michael Kolupaev 已提交
1562
		{
M
Merge  
Michael Kolupaev 已提交
1563 1564
			clearOldParts();

M
Merge  
Michael Kolupaev 已提交
1565 1566 1567
			if (unreplicated_data)
				unreplicated_data->clearOldParts();

M
Merge  
Michael Kolupaev 已提交
1568 1569 1570 1571 1572
			if (is_leader_node)
			{
				clearOldLogs();
				clearOldBlocks();
			}
M
Merge  
Michael Kolupaev 已提交
1573
		}
M
Merge  
Michael Kolupaev 已提交
1574 1575 1576 1577
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);
		}
M
Merge  
Michael Kolupaev 已提交
1578

M
Merge  
Michael Kolupaev 已提交
1579
		shutdown_event.tryWait(CLEANUP_SLEEP_MS);
M
Merge  
Michael Kolupaev 已提交
1580
	}
M
Merge  
Michael Kolupaev 已提交
1581

1582
	LOG_DEBUG(log, "Cleanup thread finished");
M
Merge  
Michael Kolupaev 已提交
1583 1584
}

M
Merge  
Michael Kolupaev 已提交
1585 1586 1587 1588 1589 1590 1591 1592
void StorageReplicatedMergeTree::alterThread()
{
	bool force_recheck_parts = true;

	while (!shutdown_called)
	{
		try
		{
1593 1594 1595 1596 1597 1598 1599 1600 1601 1602 1603 1604 1605 1606 1607 1608 1609 1610 1611 1612 1613 1614 1615
			/** Имеем описание столбцов в 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: Слишком сложно, всё переделать.
			  */

M
Merge  
Michael Kolupaev 已提交
1616
			zkutil::Stat stat;
1617 1618 1619 1620 1621
			const String columns_str = zookeeper->get(zookeeper_path + "/columns", &stat, alter_thread_event);
			const String materialized_columns_str = zookeeper->get(zookeeper_path + "/materialized_columns",
				&stat, alter_thread_event);
			const String alias_columns_str = zookeeper->get(zookeeper_path + "/alias_columns",
				&stat, alter_thread_event);
A
Merge  
Andrey Mironov 已提交
1622 1623
			const String column_defaults_str = zookeeper->get(zookeeper_path + "/column_defaults",
				&stat, alter_thread_event);
M
Merge  
Michael Kolupaev 已提交
1624
			NamesAndTypesList columns = NamesAndTypesList::parse(columns_str, context.getDataTypeFactory());
1625 1626 1627
			NamesAndTypesList materialized_columns = NamesAndTypesList::parse(
				materialized_columns_str, context.getDataTypeFactory());
			NamesAndTypesList alias_columns = NamesAndTypesList::parse(alias_columns_str, context.getDataTypeFactory());
A
Merge  
Andrey Mironov 已提交
1628
			ColumnDefaults column_defaults = ColumnDefaults::parse(column_defaults_str);
M
Merge  
Michael Kolupaev 已提交
1629

1630
			bool changed_version = (stat.version != columns_version);
M
Merge  
Michael Kolupaev 已提交
1631

M
Merge  
Michael Kolupaev 已提交
1632 1633
			MergeTreeData::DataParts parts;

M
Merge  
Michael Kolupaev 已提交
1634
			/// Если описание столбцов изменилось, обновим структуру таблицы локально.
1635
			if (changed_version)
M
Merge  
Michael Kolupaev 已提交
1636 1637
			{
				auto table_lock = lockStructureForAlter();
1638

1639 1640 1641
				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;
A
Merge  
Andrey Mironov 已提交
1642
				const auto column_defaults_changed = column_defaults != data.column_defaults;
1643 1644 1645

				if (columns_changed || materialized_columns_changed || alias_columns_changed ||
					column_defaults_changed)
M
Merge  
Michael Kolupaev 已提交
1646 1647
				{
					LOG_INFO(log, "Columns list changed in ZooKeeper. Applying changes locally.");
1648

1649 1650 1651
					InterpreterAlterQuery::updateMetadata(database_name, table_name, columns,
						materialized_columns, alias_columns, column_defaults, context);

1652 1653 1654 1655 1656 1657 1658 1659 1660 1661 1662 1663 1664 1665 1666 1667 1668 1669 1670 1671 1672 1673 1674 1675 1676 1677
					if (columns_changed)
					{
						data.setColumnsList(columns);

						if (unreplicated_data)
							unreplicated_data->setColumnsList(columns);
					}

					if (materialized_columns_changed)
					{
						this->materialized_columns = materialized_columns;
						data.materialized_columns = std::move(materialized_columns);
					}

					if (alias_columns_changed)
					{
						this->alias_columns = alias_columns;
						data.alias_columns = std::move(alias_columns);
					}

					if (column_defaults_changed)
					{
						this->column_defaults = column_defaults;
						data.column_defaults = std::move(column_defaults);
					}

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

				/// Нужно получить список кусков под блокировкой таблицы, чтобы избежать race condition с мерджем.
				parts = data.getDataParts();

				columns_version = stat.version;
M
Merge  
Michael Kolupaev 已提交
1689 1690 1691
			}

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

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

				int changed_parts = 0;

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

1704 1705
				const auto columns_plus_materialized = data.getColumnsList();

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

					if (!transaction)
						continue;

					++changed_parts;

					/// Обновим метаданные куска в ZooKeeper.
					zkutil::Ops ops;
					ops.push_back(new zkutil::Op::SetData(replica_path + "/parts/" + part->name + "/columns", part->columns.toString(), -1));
					ops.push_back(new zkutil::Op::SetData(replica_path + "/parts/" + part->name + "/checksums", part->checksums.toString(), -1));
					zookeeper->multi(ops);

					/// Применим изменения файлов.
					transaction->commit();
				}

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

					for (const MergeTreeData::DataPartPtr & part : parts)
					{
1735
						auto transaction = unreplicated_data->alterDataPart(part, columns_plus_materialized);
M
Merge  
Michael Kolupaev 已提交
1736 1737 1738 1739 1740 1741 1742 1743 1744

						if (!transaction)
							continue;

						++changed_parts;

						transaction->commit();
					}
				}
M
Merge  
Michael Kolupaev 已提交
1745

1746
				/// Список столбцов для конкретной реплики.
M
Merge  
Michael Kolupaev 已提交
1747
				zookeeper->set(replica_path + "/columns", columns.toString());
1748 1749 1750
				zookeeper->set(replica_path + "/materialized_columns", materialized_columns.toString());
				zookeeper->set(replica_path + "/alias_columns", alias_columns.toString());
				zookeeper->set(replica_path + "/column_defaults", column_defaults.toString());
M
Merge  
Michael Kolupaev 已提交
1751

1752 1753 1754 1755 1756 1757 1758 1759
				if (changed_version)
				{
					if (changed_parts != 0)
						LOG_INFO(log, "ALTER-ed " << changed_parts << " parts");
					else
						LOG_INFO(log, "No parts ALTER-ed");
				}

M
Merge  
Michael Kolupaev 已提交
1760 1761 1762
				force_recheck_parts = false;
			}

1763
			parts.clear();
M
Merge  
Michael Kolupaev 已提交
1764 1765 1766 1767 1768 1769 1770 1771 1772 1773 1774 1775
			alter_thread_event->wait();
		}
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);

			force_recheck_parts = true;

			alter_thread_event->tryWait(ERROR_SLEEP_MS);
		}
	}

1776
	LOG_DEBUG(log, "Alter thread finished");
M
Merge  
Michael Kolupaev 已提交
1777 1778
}

M
Merge  
Michael Kolupaev 已提交
1779 1780 1781 1782
void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_name)
{
	String part_path = replica_path + "/parts/" + part_name;

M
Merge  
Michael Kolupaev 已提交
1783 1784 1785 1786
	LogEntryPtr log_entry = new LogEntry;
	log_entry->type = LogEntry::GET_PART;
	log_entry->source_replica = "";
	log_entry->new_part_name = part_name;
M
Merge  
Michael Kolupaev 已提交
1787 1788 1789

	zkutil::Ops ops;
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
1790
		replica_path + "/queue/queue-", log_entry->toString(), zookeeper->getDefaultACL(),
M
Merge  
Michael Kolupaev 已提交
1791 1792 1793 1794 1795 1796 1797
		zkutil::CreateMode::PersistentSequential));
	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));
	auto results = zookeeper->multi(ops);

	{
M
Merge  
Michael Kolupaev 已提交
1798
		std::unique_lock<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1799 1800

		String path_created = dynamic_cast<zkutil::Op::Create &>(ops[0]).getPathCreated();
M
Merge  
Michael Kolupaev 已提交
1801 1802
		log_entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
		log_entry->addResultToVirtualParts(*this);
M
Merge  
Michael Kolupaev 已提交
1803 1804 1805 1806 1807 1808 1809 1810 1811 1812 1813 1814 1815 1816 1817
		queue.push_back(log_entry);
	}
}

void StorageReplicatedMergeTree::enqueuePartForCheck(const String & name)
{
	Poco::ScopedLock<Poco::FastMutex> lock(parts_to_check_mutex);

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

M
Merge  
Michael Kolupaev 已提交
1818 1819 1820 1821 1822 1823 1824 1825 1826 1827 1828 1829 1830 1831 1832 1833 1834 1835 1836 1837 1838 1839 1840
void StorageReplicatedMergeTree::partCheckThread()
{
	while (!shutdown_called)
	{
		try
		{
			/// Достанем из очереди кусок для проверки.
			String part_name;
			{
				Poco::ScopedLock<Poco::FastMutex> lock(parts_to_check_mutex);
				if (parts_to_check_queue.empty())
				{
					if (!parts_to_check_set.empty())
					{
						LOG_ERROR(log, "Non-empty parts_to_check_set with empty parts_to_check_queue. This is a bug.");
						parts_to_check_set.clear();
					}
				}
				else
				{
					part_name = parts_to_check_queue.front();
				}
			}
1841
			if (part_name.empty())	/// TODO Здесь race condition?
M
Merge  
Michael Kolupaev 已提交
1842 1843 1844 1845 1846 1847
			{
				parts_to_check_event.wait();
				continue;
			}

			LOG_WARNING(log, "Checking part " << part_name);
M
Merge  
Michael Kolupaev 已提交
1848
			ProfileEvents::increment(ProfileEvents::ReplicatedPartChecks);
M
Merge  
Michael Kolupaev 已提交
1849

M
Merge  
Michael Kolupaev 已提交
1850
			auto part = data.getActiveContainingPart(part_name);
M
Merge  
Michael Kolupaev 已提交
1851 1852 1853 1854 1855 1856 1857 1858 1859 1860
			String part_path = replica_path + "/parts/" + part_name;

			/// Этого или покрывающего куска у нас нет.
			if (!part)
			{
				/// Если кусок есть в ZooKeeper, удалим его оттуда и добавим в очередь задание скачать его.
				if (zookeeper->exists(part_path))
				{
					LOG_WARNING(log, "Part " << part_name << " exists in ZooKeeper but not locally. "
						"Removing from ZooKeeper and queueing a fetch.");
M
Merge  
Michael Kolupaev 已提交
1861
					ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
M
Merge  
Michael Kolupaev 已提交
1862

M
Merge  
Michael Kolupaev 已提交
1863
					removePartAndEnqueueFetch(part_name);
M
Merge  
Michael Kolupaev 已提交
1864 1865 1866 1867
				}
				/// Если куска нет в ZooKeeper, проверим есть ли он хоть у кого-то.
				else
				{
M
Merge  
Michael Kolupaev 已提交
1868 1869
					ActiveDataPartSet::Part part_info;
					ActiveDataPartSet::parsePartName(part_name, part_info);
M
Merge  
Michael Kolupaev 已提交
1870

M
Merge  
Michael Kolupaev 已提交
1871
					/** Будем проверять только куски, не полученные в результате слияния.
M
Merge  
Michael Kolupaev 已提交
1872
					  * Для кусков, полученных в результате слияния, такая проверка была бы некорректной,
M
Merge  
Michael Kolupaev 已提交
1873 1874
					  *  потому что слитого куска может еще ни у кого не быть.
					  */
M
Merge  
Michael Kolupaev 已提交
1875
					if (part_info.left == part_info.right)
M
Merge  
Michael Kolupaev 已提交
1876
					{
M
Merge  
Michael Kolupaev 已提交
1877 1878 1879 1880 1881
						LOG_WARNING(log, "Checking if anyone has part covering " << part_name << ".");

						bool found = false;
						Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
						for (const String & replica : replicas)
M
Merge  
Michael Kolupaev 已提交
1882
						{
M
Merge  
Michael Kolupaev 已提交
1883 1884
							Strings parts = zookeeper->getChildren(zookeeper_path + "/replicas/" + replica + "/parts");
							for (const String & part_on_replica : parts)
M
Merge  
Michael Kolupaev 已提交
1885
							{
M
Merge  
Michael Kolupaev 已提交
1886 1887 1888 1889 1890 1891
								if (part_on_replica == part_name || ActiveDataPartSet::contains(part_on_replica, part_name))
								{
									found = true;
									LOG_WARNING(log, "Found part " << part_on_replica << " on " << replica);
									break;
								}
M
Merge  
Michael Kolupaev 已提交
1892
							}
M
Merge  
Michael Kolupaev 已提交
1893 1894
							if (found)
								break;
M
Merge  
Michael Kolupaev 已提交
1895 1896
						}

M
Merge  
Michael Kolupaev 已提交
1897
						if (!found)
M
Merge  
Michael Kolupaev 已提交
1898
						{
M
Merge  
Michael Kolupaev 已提交
1899
							LOG_ERROR(log, "No replica has part covering " << part_name);
M
Merge  
Michael Kolupaev 已提交
1900 1901
							ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);

M
Merge  
Michael Kolupaev 已提交
1902
							/// Если ни у кого нет такого куска, удалим его из нашей очереди.
M
Merge  
Michael Kolupaev 已提交
1903

M
Merge  
Michael Kolupaev 已提交
1904 1905
							bool was_in_queue = false;

M
Merge  
Michael Kolupaev 已提交
1906
							{
M
Merge  
Michael Kolupaev 已提交
1907
								std::unique_lock<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1908 1909

								for (LogEntries::iterator it = queue.begin(); it != queue.end(); )
M
Merge  
Michael Kolupaev 已提交
1910
								{
M
Merge  
Michael Kolupaev 已提交
1911
									if ((*it)->new_part_name == part_name)
M
Merge  
Michael Kolupaev 已提交
1912
									{
M
Merge  
Michael Kolupaev 已提交
1913
										zookeeper->tryRemove(replica_path + "/queue/" + (*it)->znode_name);
M
Merge  
Michael Kolupaev 已提交
1914
										queue.erase(it++);
M
Merge  
Michael Kolupaev 已提交
1915
										was_in_queue = true;
M
Merge  
Michael Kolupaev 已提交
1916 1917 1918 1919 1920
									}
									else
									{
										++it;
									}
M
Merge  
Michael Kolupaev 已提交
1921 1922
								}
							}
M
Merge  
Michael Kolupaev 已提交
1923 1924

							if (was_in_queue)
M
Merge  
Michael Kolupaev 已提交
1925
							{
M
Merge  
Michael Kolupaev 已提交
1926 1927 1928 1929
								/** Такая ситуация возможна, если на всех репликах, где был кусок, он испортился.
								  * Например, у реплики, которая только что его записала, отключили питание, и данные не записались из кеша на диск.
								  */
								LOG_ERROR(log, "Part " << part_name << " is lost forever. Say goodbye to a piece of data!");
M
Merge  
Michael Kolupaev 已提交
1930 1931 1932 1933 1934 1935 1936 1937 1938 1939 1940 1941 1942

								/** Нужно добавить отсутствующий кусок в block_numbers, чтобы он не мешал слияниям.
								  * Вот только в сам block_numbers мы его добавить не можем - если так сделать,
								  *  ZooKeeper зачем-то пропустит один номер для автоинкремента,
								  *  и в номерах блоков все равно останется дырка.
								  * Специально из-за этого приходится отдельно иметь nonincrement_block_numbers.
								  */
								zookeeper->createIfNotExists(zookeeper_path + "/nonincrement_block_numbers", "");
								zookeeper->createIfNotExists(zookeeper_path + "/nonincrement_block_numbers/" + part_name.substr(0, 6), "");
								AbandonableLockInZooKeeper::createAbandonedIfNotExists(
									zookeeper_path + "/nonincrement_block_numbers/" + part_name.substr(0, 6) + "/block-" + padIndex(part_info.left),
									*zookeeper);
							}
M
Merge  
Michael Kolupaev 已提交
1943 1944
						}
					}
M
Merge  
Michael Kolupaev 已提交
1945 1946 1947 1948 1949
				}
			}
			/// У нас есть этот кусок, и он активен.
			else if (part->name == part_name)
			{
M
Merge  
Michael Kolupaev 已提交
1950 1951
				auto table_lock = lockStructure(false);

M
Merge  
Michael Kolupaev 已提交
1952 1953 1954
				/// Если кусок есть в ZooKeeper, сверим его данные с его чексуммами, а их с ZooKeeper.
				if (zookeeper->exists(replica_path + "/parts/" + part_name))
				{
M
Merge  
Michael Kolupaev 已提交
1955 1956 1957 1958 1959 1960 1961 1962 1963 1964 1965 1966 1967
					LOG_WARNING(log, "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"), context.getDataTypeFactory());
						if (part->columns != zk_columns)
							throw Exception("Columns of local part " + part_name + " are different from ZooKeeper");

M
Merge  
Michael Kolupaev 已提交
1968 1969 1970 1971
						MergeTreePartChecker::Settings settings;
						settings.setIndexGranularity(data.index_granularity);
						settings.setRequireChecksums(true);
						settings.setRequireColumnFiles(true);
M
Merge  
Michael Kolupaev 已提交
1972
						MergeTreePartChecker::checkDataPart(
M
Merge  
Michael Kolupaev 已提交
1973
							data.getFullPath() + part_name, settings, context.getDataTypeFactory());
M
Merge  
Michael Kolupaev 已提交
1974

M
Merge  
Michael Kolupaev 已提交
1975 1976 1977 1978 1979 1980
						LOG_INFO(log, "Part " << part_name << " looks good.");
					}
					catch (...)
					{
						tryLogCurrentException(__PRETTY_FUNCTION__);

M
Merge  
Michael Kolupaev 已提交
1981 1982
						LOG_ERROR(log, "Part " << part_name << " looks broken. Removing it and queueing a fetch.");
						ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
M
Merge  
Michael Kolupaev 已提交
1983 1984 1985 1986

						removePartAndEnqueueFetch(part_name);

						/// Удалим кусок локально.
M
Merge  
Michael Kolupaev 已提交
1987
						data.renameAndDetachPart(part, "broken_");
M
Merge  
Michael Kolupaev 已提交
1988
					}
M
Merge  
Michael Kolupaev 已提交
1989 1990
				}
				/// Если куска нет в ZooKeeper, удалим его локально.
M
Merge  
Michael Kolupaev 已提交
1991 1992 1993
				/// Возможно, кусок кто-то только что записал, и еще не успел добавить в ZK.
				/// Поэтому удаляем только если кусок старый (не очень надежно).
				else if (part->modification_time + 5 * 60 < time(0))
M
Merge  
Michael Kolupaev 已提交
1994
				{
M
Merge  
Michael Kolupaev 已提交
1995 1996
					ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);

M
Merge  
Michael Kolupaev 已提交
1997 1998
					LOG_ERROR(log, "Unexpected part " << part_name << ". Removing.");
					data.renameAndDetachPart(part, "unexpected_");
M
Merge  
Michael Kolupaev 已提交
1999 2000 2001 2002 2003 2004 2005 2006 2007 2008 2009 2010 2011 2012 2013 2014 2015 2016 2017 2018 2019 2020 2021 2022 2023 2024 2025 2026 2027 2028 2029
				}
			}
			else
			{
				/// Если у нас есть покрывающий кусок, игнорируем все проблемы с этим куском.
				/// В худшем случае в лог еще old_parts_lifetime секунд будут валиться ошибки, пока кусок не удалится как старый.
			}

			/// Удалим кусок из очереди.
			{
				Poco::ScopedLock<Poco::FastMutex> lock(parts_to_check_mutex);
				if (parts_to_check_queue.empty() || parts_to_check_queue.front() != part_name)
				{
					LOG_ERROR(log, "Someone changed parts_to_check_queue.front(). This is a bug.");
				}
				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 已提交
2030 2031
void StorageReplicatedMergeTree::becomeLeader()
{
M
Merge  
Michael Kolupaev 已提交
2032
	LOG_INFO(log, "Became leader");
M
Merge  
Michael Kolupaev 已提交
2033 2034 2035 2036
	is_leader_node = true;
	merge_selecting_thread = std::thread(&StorageReplicatedMergeTree::mergeSelectingThread, this);
}

M
Merge  
Michael Kolupaev 已提交
2037
String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_name, bool active)
M
Merge  
Michael Kolupaev 已提交
2038
{
M
Merge  
Michael Kolupaev 已提交
2039
	Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
M
Merge  
Michael Kolupaev 已提交
2040 2041 2042 2043 2044 2045

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

	for (const String & replica : replicas)
	{
M
Merge  
Michael Kolupaev 已提交
2046 2047
		if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name) &&
			(!active || zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")))
M
Merge  
Michael Kolupaev 已提交
2048 2049 2050
			return replica;
	}

M
Merge  
Michael Kolupaev 已提交
2051
	return "";
M
Merge  
Michael Kolupaev 已提交
2052 2053 2054 2055
}

void StorageReplicatedMergeTree::fetchPart(const String & part_name, const String & replica_name)
{
M
Merge  
Michael Kolupaev 已提交
2056 2057
	LOG_DEBUG(log, "Fetching part " << part_name << " from " << replica_name);

M
Merge  
Michael Kolupaev 已提交
2058 2059 2060 2061 2062
	auto table_lock = lockStructure(true);

	String host;
	int port;

M
Merge  
Michael Kolupaev 已提交
2063
	String host_port_str = zookeeper->get(zookeeper_path + "/replicas/" + replica_name + "/host");
M
Merge  
Michael Kolupaev 已提交
2064 2065 2066 2067 2068 2069 2070 2071
	ReadBufferFromString buf(host_port_str);
	assertString("host: ", buf);
	readString(host, buf);
	assertString("\nport: ", buf);
	readText(port, buf);
	assertString("\n", buf);
	assertEOF(buf);

M
Merge  
Michael Kolupaev 已提交
2072
	MergeTreeData::MutableDataPartPtr part = fetcher.fetchPart(part_name, zookeeper_path + "/replicas/" + replica_name, host, port);
M
Merge  
Michael Kolupaev 已提交
2073

M
Merge  
Michael Kolupaev 已提交
2074 2075 2076
	zkutil::Ops ops;
	checkPartAndAddToZooKeeper(part, ops, part_name);

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

M
Merge  
Michael Kolupaev 已提交
2080
	zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
2081
	transaction.commit();
M
Merge  
Michael Kolupaev 已提交
2082
	merge_selecting_event.set();
M
Merge  
Michael Kolupaev 已提交
2083

M
Michael Kolupaev 已提交
2084 2085 2086 2087 2088 2089
	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);
	}

M
Merge  
Michael Kolupaev 已提交
2090 2091
	ProfileEvents::increment(ProfileEvents::ReplicatedPartFetches);

M
Merge  
Michael Kolupaev 已提交
2092
	LOG_DEBUG(log, "Fetched part " << part_name << " from " << replica_name);
M
Merge  
Michael Kolupaev 已提交
2093
}
M
Merge  
Michael Kolupaev 已提交
2094

M
Merge  
Michael Kolupaev 已提交
2095 2096
void StorageReplicatedMergeTree::shutdown()
{
M
Merge  
Michael Kolupaev 已提交
2097
	if (permanent_shutdown_called)
M
Merge  
Michael Kolupaev 已提交
2098 2099 2100
	{
		if (restarting_thread.joinable())
			restarting_thread.join();
M
Merge  
Michael Kolupaev 已提交
2101
		return;
M
Merge  
Michael Kolupaev 已提交
2102
	}
M
Merge  
Michael Kolupaev 已提交
2103

M
Merge  
Michael Kolupaev 已提交
2104
	permanent_shutdown_called = true;
M
Merge  
Michael Kolupaev 已提交
2105
	restarting_event.set();
M
Merge  
Michael Kolupaev 已提交
2106
	restarting_thread.join();
M
Merge  
Michael Kolupaev 已提交
2107 2108

	endpoint_holder = nullptr;
M
Merge  
Michael Kolupaev 已提交
2109 2110 2111 2112
}

void StorageReplicatedMergeTree::partialShutdown()
{
M
Merge  
Michael Kolupaev 已提交
2113
	leader_election = nullptr;
M
Merge  
Michael Kolupaev 已提交
2114
	shutdown_called = true;
M
Merge  
Michael Kolupaev 已提交
2115
	shutdown_event.set();
M
Merge  
Michael Kolupaev 已提交
2116 2117 2118
	merge_selecting_event.set();
	queue_updating_event->set();
	alter_thread_event->set();
M
Merge  
Michael Kolupaev 已提交
2119
	alter_query_event->set();
M
Merge  
Michael Kolupaev 已提交
2120
	parts_to_check_event.set();
M
Merge  
Michael Kolupaev 已提交
2121 2122
	replica_is_active_node = nullptr;

M
Merge  
Michael Kolupaev 已提交
2123 2124 2125 2126
	merger.cancelAll();
	if (unreplicated_merger)
		unreplicated_merger->cancelAll();

M
Merge  
Michael Kolupaev 已提交
2127
	LOG_TRACE(log, "Waiting for threads to finish");
M
Merge  
Michael Kolupaev 已提交
2128
	if (is_leader_node)
M
Merge  
Michael Kolupaev 已提交
2129
	{
M
Merge  
Michael Kolupaev 已提交
2130
		is_leader_node = false;
M
Merge  
Michael Kolupaev 已提交
2131 2132
		if (merge_selecting_thread.joinable())
			merge_selecting_thread.join();
M
Merge  
Michael Kolupaev 已提交
2133
	}
M
Merge  
Michael Kolupaev 已提交
2134 2135
	if (queue_updating_thread.joinable())
		queue_updating_thread.join();
M
Merge  
Michael Kolupaev 已提交
2136 2137 2138 2139
	if (cleanup_thread.joinable())
		cleanup_thread.join();
	if (alter_thread.joinable())
		alter_thread.join();
M
Merge  
Michael Kolupaev 已提交
2140 2141
	if (part_check_thread.joinable())
		part_check_thread.join();
M
Merge  
Michael Kolupaev 已提交
2142 2143
	if (queue_task_handle)
		context.getBackgroundPool().removeTask(queue_task_handle);
M
Merge  
Michael Kolupaev 已提交
2144
	queue_task_handle.reset();
M
Merge  
Michael Kolupaev 已提交
2145
	LOG_TRACE(log, "Threads finished");
M
Merge  
Michael Kolupaev 已提交
2146 2147
}

M
Merge  
Michael Kolupaev 已提交
2148
void StorageReplicatedMergeTree::goReadOnlyPermanently()
M
Merge  
Michael Kolupaev 已提交
2149 2150 2151 2152 2153
{
	LOG_INFO(log, "Going to read-only mode");

	is_read_only = true;
	permanent_shutdown_called = true;
M
Merge  
Michael Kolupaev 已提交
2154
	restarting_event.set();
M
Merge  
Michael Kolupaev 已提交
2155

M
Merge  
Michael Kolupaev 已提交
2156
	partialShutdown();
M
Merge  
Michael Kolupaev 已提交
2157 2158
}

M
Merge  
Michael Kolupaev 已提交
2159
bool StorageReplicatedMergeTree::tryStartup()
M
Merge  
Michael Kolupaev 已提交
2160
{
M
Merge  
Michael Kolupaev 已提交
2161 2162 2163
	try
	{
		activateReplica();
M
Merge  
Michael Kolupaev 已提交
2164

M
Merge  
Michael Kolupaev 已提交
2165 2166 2167 2168 2169
		leader_election = new zkutil::LeaderElection(zookeeper_path + "/leader_election", *zookeeper,
			std::bind(&StorageReplicatedMergeTree::becomeLeader, this), replica_name);

		/// Все, что выше, может бросить KeeperException, если что-то не так с ZK.
		/// Все, что ниже, не должно бросать исключений.
M
Merge  
Michael Kolupaev 已提交
2170

M
Merge  
Michael Kolupaev 已提交
2171 2172
		shutdown_called = false;
		shutdown_event.reset();
M
Merge  
Michael Kolupaev 已提交
2173

M
Merge  
Michael Kolupaev 已提交
2174 2175 2176
		merger.uncancelAll();
		if (unreplicated_merger)
			unreplicated_merger->uncancelAll();
M
Merge  
Michael Kolupaev 已提交
2177

M
Merge  
Michael Kolupaev 已提交
2178 2179 2180 2181 2182 2183
		queue_updating_thread = std::thread(&StorageReplicatedMergeTree::queueUpdatingThread, this);
		cleanup_thread = std::thread(&StorageReplicatedMergeTree::cleanupThread, this);
		alter_thread = std::thread(&StorageReplicatedMergeTree::alterThread, this);
		part_check_thread = std::thread(&StorageReplicatedMergeTree::partCheckThread, this);
		queue_task_handle = context.getBackgroundPool().addTask(
			std::bind(&StorageReplicatedMergeTree::queueTask, this, std::placeholders::_1));
M
Merge  
Michael Kolupaev 已提交
2184
		queue_task_handle->wake();
M
Merge  
Michael Kolupaev 已提交
2185 2186
		return true;
	}
2187
	catch (const zkutil::KeeperException & e)
M
Merge  
Michael Kolupaev 已提交
2188 2189 2190 2191 2192 2193 2194
	{
		replica_is_active_node = nullptr;
		leader_election = nullptr;
		LOG_ERROR(log, "Couldn't start replication: " << e.what() << ", " << e.displayText() << ", stack trace:\n"
			<< e.getStackTrace().toString());
		return false;
	}
2195 2196 2197 2198 2199 2200 2201 2202 2203 2204 2205
	catch (const Exception & e)
	{
		if (e.code() != ErrorCodes::REPLICA_IS_ALREADY_ACTIVE)
			throw;

		replica_is_active_node = nullptr;
		leader_election = nullptr;
		LOG_ERROR(log, "Couldn't start replication: " << e.what() << ", " << e.displayText() << ", stack trace:\n"
			<< e.getStackTrace().toString());
		return false;
	}
M
Merge  
Michael Kolupaev 已提交
2206 2207 2208 2209 2210 2211
	catch (...)
	{
		replica_is_active_node = nullptr;
		leader_election = nullptr;
		throw;
	}
M
Merge  
Michael Kolupaev 已提交
2212 2213
}

M
Merge  
Michael Kolupaev 已提交
2214 2215
void StorageReplicatedMergeTree::restartingThread()
{
M
Merge  
Michael Kolupaev 已提交
2216
	try
M
Merge  
Michael Kolupaev 已提交
2217
	{
2218
		/// Запуск реплики при старте сервера/создании таблицы.
M
Merge  
Michael Kolupaev 已提交
2219 2220
		while (!permanent_shutdown_called && !tryStartup())
			restarting_event.tryWait(10 * 1000);
M
Merge  
Michael Kolupaev 已提交
2221

2222
		/// Цикл перезапуска реплики при истечении сессии с ZK.
M
Merge  
Michael Kolupaev 已提交
2223
		while (!permanent_shutdown_called)
M
Merge  
Michael Kolupaev 已提交
2224
		{
M
Merge  
Michael Kolupaev 已提交
2225 2226 2227
			if (zookeeper->expired())
			{
				LOG_WARNING(log, "ZooKeeper session has expired. Switching to a new session.");
M
Merge  
Michael Kolupaev 已提交
2228

2229 2230 2231
				partialShutdown();
				zookeeper = context.getZooKeeper();
				is_read_only = true;
M
Merge  
Michael Kolupaev 已提交
2232 2233 2234 2235 2236 2237

				while (!permanent_shutdown_called && !tryStartup())
					restarting_event.tryWait(10 * 1000);

				if (permanent_shutdown_called)
					break;
M
Merge  
Michael Kolupaev 已提交
2238

2239
				is_read_only = false;
M
Merge  
Michael Kolupaev 已提交
2240
			}
M
Merge  
Michael Kolupaev 已提交
2241

M
Merge  
Michael Kolupaev 已提交
2242
			restarting_event.tryWait(60 * 1000);
M
Merge  
Michael Kolupaev 已提交
2243 2244 2245 2246 2247
		}
	}
	catch (...)
	{
		tryLogCurrentException("StorageReplicatedMergeTree::restartingThread");
M
Merge  
Michael Kolupaev 已提交
2248 2249
		LOG_ERROR(log, "Unexpected exception in restartingThread. The storage will be read-only until server restart.");
		goReadOnlyPermanently();
M
Merge  
Michael Kolupaev 已提交
2250
		LOG_DEBUG(log, "restarting thread finished");
M
Merge  
Michael Kolupaev 已提交
2251
		return;
M
Merge  
Michael Kolupaev 已提交
2252 2253
	}

M
Merge  
Michael Kolupaev 已提交
2254 2255 2256 2257 2258 2259 2260 2261 2262
	try
	{
		endpoint_holder = nullptr;
		partialShutdown();
	}
	catch (...)
	{
		tryLogCurrentException("StorageReplicatedMergeTree::restartingThread");
	}
M
Merge  
Michael Kolupaev 已提交
2263 2264

	LOG_DEBUG(log, "restarting thread finished");
M
Merge  
Michael Kolupaev 已提交
2265 2266
}

M
Merge  
Michael Kolupaev 已提交
2267 2268 2269 2270 2271 2272 2273 2274 2275 2276 2277 2278 2279 2280 2281 2282 2283 2284 2285 2286
StorageReplicatedMergeTree::~StorageReplicatedMergeTree()
{
	try
	{
		shutdown();
	}
	catch(...)
	{
		tryLogCurrentException("~StorageReplicatedMergeTree");
	}
}

BlockInputStreams StorageReplicatedMergeTree::read(
		const Names & column_names,
		ASTPtr query,
		const Settings & settings,
		QueryProcessingStage::Enum & processed_stage,
		size_t max_block_size,
		unsigned threads)
{
M
Merge  
Michael Kolupaev 已提交
2287 2288 2289 2290 2291 2292 2293 2294 2295 2296 2297 2298 2299 2300 2301 2302 2303
	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);

	Block virtual_columns_block;
	ColumnUInt8 * column = new ColumnUInt8(2);
	ColumnPtr column_ptr = column;
	column->getData()[0] = 0;
	column->getData()[1] = 1;
	virtual_columns_block.insert(ColumnWithNameAndType(column_ptr, new DataTypeUInt8, "_replicated"));

	/// Если запрошен хотя бы один виртуальный столбец, пробуем индексировать
	if (!virt_column_names.empty())
2304
		VirtualColumnUtils::filterBlockWithQuery(query->clone(), virtual_columns_block, context);
M
Merge  
Michael Kolupaev 已提交
2305

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

M
Merge  
Michael Kolupaev 已提交
2308 2309
	BlockInputStreams res;

M
Merge  
Michael Kolupaev 已提交
2310 2311 2312
	size_t part_index = 0;

	if (unreplicated_reader && values.count(0))
M
Merge  
Michael Kolupaev 已提交
2313
	{
M
Merge  
Michael Kolupaev 已提交
2314 2315
		res = unreplicated_reader->read(
			real_column_names, query, settings, processed_stage, max_block_size, threads, &part_index);
M
Merge  
Michael Kolupaev 已提交
2316 2317 2318 2319 2320 2321

		for (auto & virtual_column : virt_column_names)
		{
			if (virtual_column == "_replicated")
			{
				for (auto & stream : res)
M
Merge  
Michael Kolupaev 已提交
2322
					stream = new AddingConstColumnBlockInputStream<UInt8>(stream, new DataTypeUInt8, 0, "_replicated");
M
Merge  
Michael Kolupaev 已提交
2323 2324 2325 2326
			}
		}
	}

M
Merge  
Michael Kolupaev 已提交
2327
	if (values.count(1))
M
Merge  
Michael Kolupaev 已提交
2328
	{
M
Merge  
Michael Kolupaev 已提交
2329
		auto res2 = reader.read(real_column_names, query, settings, processed_stage, max_block_size, threads, &part_index);
M
Merge  
Michael Kolupaev 已提交
2330 2331 2332 2333 2334 2335

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

M
Merge  
Michael Kolupaev 已提交
2340
		res.insert(res.end(), res2.begin(), res2.end());
M
Merge  
Michael Kolupaev 已提交
2341 2342 2343
	}

	return res;
M
Merge  
Michael Kolupaev 已提交
2344 2345
}

M
Merge  
Michael Kolupaev 已提交
2346 2347
BlockOutputStreamPtr StorageReplicatedMergeTree::write(ASTPtr query)
{
M
Merge  
Michael Kolupaev 已提交
2348 2349 2350
	if (is_read_only)
		throw Exception("Table is in read only mode", ErrorCodes::TABLE_IS_READ_ONLY);

M
Merge  
Michael Kolupaev 已提交
2351
	String insert_id;
M
Merge  
Michael Kolupaev 已提交
2352 2353 2354
	if (query)
		if (ASTInsertQuery * insert = typeid_cast<ASTInsertQuery *>(&*query))
			insert_id = insert->insert_id;
M
Merge  
Michael Kolupaev 已提交
2355 2356 2357

	return new ReplicatedMergeTreeBlockOutputStream(*this, insert_id);
}
M
Merge  
Michael Kolupaev 已提交
2358

M
Merge  
Michael Kolupaev 已提交
2359 2360
bool StorageReplicatedMergeTree::optimize()
{
2361 2362
	/// Померджим какие-нибудь куски из директории unreplicated.
	/// TODO: Мерджить реплицируемые куски тоже.
M
Merge  
Michael Kolupaev 已提交
2363 2364 2365 2366

	if (!unreplicated_data)
		return false;

M
Merge  
Michael Kolupaev 已提交
2367 2368
	Poco::ScopedLock<Poco::FastMutex> lock(unreplicated_mutex);

M
Merge  
Michael Kolupaev 已提交
2369 2370 2371 2372 2373 2374 2375 2376
	unreplicated_data->clearOldParts();

	MergeTreeData::DataPartsVector parts;
	String merged_name;
	auto always_can_merge = [](const MergeTreeData::DataPartPtr &a, const MergeTreeData::DataPartPtr &b) { return true; };
	if (!unreplicated_merger->selectPartsToMerge(parts, merged_name, 0, true, true, false, always_can_merge))
		return false;

2377 2378
	const auto & merge_entry = context.getMergeList().insert(database_name, table_name, merged_name);
	unreplicated_merger->mergeParts(parts, merged_name, *merge_entry);
M
Merge  
Michael Kolupaev 已提交
2379 2380 2381
	return true;
}

M
Merge  
Michael Kolupaev 已提交
2382 2383
void StorageReplicatedMergeTree::alter(const AlterCommands & params,
	const String & database_name, const String & table_name, Context & context)
M
Merge  
Michael Kolupaev 已提交
2384 2385 2386 2387
{
	LOG_DEBUG(log, "Doing ALTER");

	NamesAndTypesList new_columns;
2388 2389 2390
	NamesAndTypesList new_materialized_columns;
	NamesAndTypesList new_alias_columns;
	ColumnDefaults new_column_defaults;
M
Merge  
Michael Kolupaev 已提交
2391
	String new_columns_str;
2392 2393 2394
	String new_materialized_columns_str;
	String new_alias_columns_str;
	String new_column_defaults_str;
M
Merge  
Michael Kolupaev 已提交
2395 2396 2397 2398 2399 2400
	int new_columns_version;
	zkutil::Stat stat;

	{
		auto table_lock = lockStructureForAlter();

M
Merge  
Michael Kolupaev 已提交
2401 2402 2403
		if (is_read_only)
			throw Exception("Can't ALTER read-only table", ErrorCodes::TABLE_IS_READ_ONLY);

M
Merge  
Michael Kolupaev 已提交
2404 2405
		data.checkAlter(params);

2406 2407 2408 2409 2410
		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 已提交
2411 2412

		new_columns_str = new_columns.toString();
2413 2414 2415
		new_materialized_columns_str = new_materialized_columns.toString();
		new_alias_columns_str = new_alias_columns.toString();
		new_column_defaults_str = new_column_defaults.toString();
M
Merge  
Michael Kolupaev 已提交
2416 2417 2418

		/// Делаем ALTER.
		zookeeper->set(zookeeper_path + "/columns", new_columns_str, -1, &stat);
2419 2420 2421
		zookeeper->set(zookeeper_path + "/materialized_columns", new_materialized_columns_str, -1, &stat);
		zookeeper->set(zookeeper_path + "/alias_columns", new_alias_columns_str, -1, &stat);
		zookeeper->set(zookeeper_path + "/column_defaults", new_column_defaults_str, -1, &stat);
M
Merge  
Michael Kolupaev 已提交
2422 2423 2424 2425 2426 2427 2428 2429 2430 2431 2432

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

M
Merge  
Michael Kolupaev 已提交
2434 2435 2436 2437 2438 2439 2440 2441 2442 2443 2444 2445 2446 2447 2448
	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;
A
Merge  
Andrey Mironov 已提交
2449 2450 2451
			String replica_materialized_columns_str;
			String replica_alias_columns_str;
			String replica_column_defaults_str;
M
Merge  
Michael Kolupaev 已提交
2452 2453

			/// Реплику могли успеть удалить.
A
Merge  
Andrey Mironov 已提交
2454 2455 2456 2457 2458 2459 2460
			if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/columns", replica_columns_str, &stat) ||
				!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/materialized_columns",
								   replica_materialized_columns_str, &stat) ||
				!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/alias_columns",
								   replica_alias_columns_str, &stat) ||
				!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/column_defaults",
								   replica_column_defaults_str, &stat))
M
Merge  
Michael Kolupaev 已提交
2461 2462 2463 2464 2465 2466 2467
			{
				LOG_WARNING(log, replica << " was removed");
				break;
			}

			int replica_columns_version = stat.version;

A
Merge  
Andrey Mironov 已提交
2468 2469 2470 2471
			if (replica_columns_str == new_columns_str &&
				replica_materialized_columns_str == new_materialized_columns_str &&
				replica_alias_columns_str == new_alias_columns_str &&
				replica_column_defaults_str == new_column_defaults_str)
M
Merge  
Michael Kolupaev 已提交
2472 2473 2474 2475
				break;

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

A
Merge  
Andrey Mironov 已提交
2477 2478 2479 2480 2481 2482 2483 2484 2485
			if (!zookeeper->exists(zookeeper_path + "/materialized_columns", &stat))
				throw Exception(zookeeper_path + "/materialized_columns doesn't exist", ErrorCodes::NOT_FOUND_NODE);

			if (!zookeeper->exists(zookeeper_path + "/alias_columns", &stat))
				throw Exception(zookeeper_path + "/alias_columns doesn't exist", ErrorCodes::NOT_FOUND_NODE);

			if (!zookeeper->exists(zookeeper_path + "/column_defaults", &stat))
				throw Exception(zookeeper_path + "/column_defaults doesn't exist", ErrorCodes::NOT_FOUND_NODE);

M
Merge  
Michael Kolupaev 已提交
2486 2487 2488 2489 2490 2491 2492
			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;
			}

A
Merge  
Andrey Mironov 已提交
2493 2494 2495 2496 2497 2498 2499
			if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/columns", &stat, alter_query_event) ||
				!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/materialized_columns",
								   &stat, alter_query_event) ||
				!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/alias_columns",
								   &stat, alter_query_event) ||
				!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/column_defaults",
								   &stat, alter_query_event))
M
Merge  
Michael Kolupaev 已提交
2500 2501 2502 2503 2504 2505 2506 2507 2508 2509 2510 2511 2512 2513 2514 2515 2516 2517
			{
				LOG_WARNING(log, replica << " was removed");
				break;
			}

			if (stat.version != replica_columns_version)
				continue;

			alter_query_event->wait();
		}

		if (shutdown_called)
			break;
	}

	LOG_DEBUG(log, "ALTER finished");
}

M
Merge  
Michael Kolupaev 已提交
2518 2519 2520 2521 2522 2523 2524 2525 2526 2527 2528 2529 2530 2531 2532 2533 2534 2535 2536 2537 2538 2539 2540 2541 2542 2543
static bool isValidMonthName(const String & s)
{
	if (s.size() != 6)
		return false;
	if (!std::all_of(s.begin(), s.end(), isdigit))
		return false;
	DayNum_t date = DateLUT::instance().toDayNum(OrderedIdentifier2Date(s + "01"));
	/// Не можем просто сравнить date с нулем, потому что 0 тоже валидный DayNum.
	return s == toString(Date2OrderedIdentifier(DateLUT::instance().fromDayNum(date)) / 100);
}

/// Название воображаемого куска, покрывающего все возможные куски в указанном месяце с номерами в указанном диапазоне.
static String getFakePartNameForDrop(const String & month_name, UInt64 left, UInt64 right)
{
	/// Диапазон дат - весь месяц.
	DateLUT & lut = DateLUT::instance();
	time_t start_time = OrderedIdentifier2Date(month_name + "01");
	DayNum_t left_date = lut.toDayNum(start_time);
	DayNum_t right_date = DayNum_t(static_cast<size_t>(left_date) + lut.daysInMonth(start_time) - 1);

	/// Уровень - right-left+1: кусок не мог образоваться в результате такого или большего количества слияний.
	return ActiveDataPartSet::getPartName(left_date, right_date, left, right, right - left + 1);
}

void StorageReplicatedMergeTree::dropPartition(const Field & field, bool detach)
{
M
Merge  
Michael Kolupaev 已提交
2544
	String month_name = field.getType() == Field::Types::UInt64 ? toString(field.get<UInt64>()) : field.safeGet<String>();
M
Merge  
Michael Kolupaev 已提交
2545 2546 2547 2548 2549 2550 2551 2552 2553 2554 2555 2556 2557 2558 2559 2560 2561 2562 2563 2564 2565 2566 2567 2568 2569 2570 2571 2572 2573 2574 2575 2576 2577 2578 2579 2580 2581 2582 2583 2584

	if (!isValidMonthName(month_name))
		throw Exception("Invalid partition format: " + month_name + ". Partition should consist of 6 digits: YYYYMM",
						ErrorCodes::INVALID_PARTITION_NAME);

	/// TODO: Делать запрос в лидера по TCP.
	if (!is_leader_node)
		throw Exception("DROP PARTITION can only be done on leader replica.", ErrorCodes::NOT_LEADER);


	/** Пропустим один номер в block_numbers для удаляемого месяца, и будем удалять только куски до этого номера.
	  * Это запретит мерджи удаляемых кусков с новыми вставляемыми данными.
	  * Инвариант: в логе не появятся слияния удаляемых кусков с другими кусками.
	  * NOTE: Если понадобится аналогично поддержать запрос DROP PART, для него придется придумать какой-нибудь новый механизм,
	  *        чтобы гарантировать этот инвариант.
	  */
	UInt64 right;

	{
		AbandonableLockInZooKeeper block_number_lock = allocateBlockNumber(month_name);
		right = block_number_lock.getNumber();
		block_number_lock.unlock();
	}

	/// Такого никогда не должно происходить.
	if (right == 0)
		return;
	--right;

	String fake_part_name = getFakePartNameForDrop(month_name, 0, right);

	/** Запретим выбирать для слияния удаляемые куски - сделаем вид, что их всех уже собираются слить в fake_part_name.
	  * Инвариант: после появления в логе записи DROP_RANGE, в логе не появятся слияния удаляемых кусков.
	  */
	{
		std::unique_lock<std::mutex> merge_selecting_lock(merge_selecting_mutex);

		virtual_parts.add(fake_part_name);
	}

M
Merge  
Michael Kolupaev 已提交
2585
	/// Наконец, добившись нужных инвариантов, можно положить запись в лог.
M
Merge  
Michael Kolupaev 已提交
2586 2587 2588 2589 2590 2591
	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);
M
Merge  
Michael Kolupaev 已提交
2592
	entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);
M
Merge  
Michael Kolupaev 已提交
2593 2594

	/// Дождемся, пока все реплики выполнят дроп.
M
Merge  
Michael Kolupaev 已提交
2595
	waitForAllReplicasToProcessLogEntry(entry);
M
Merge  
Michael Kolupaev 已提交
2596 2597
}

M
Merge  
Michael Kolupaev 已提交
2598
void StorageReplicatedMergeTree::attachPartition(const Field & field, bool unreplicated, bool attach_part)
M
Merge  
Michael Kolupaev 已提交
2599
{
M
Merge  
Michael Kolupaev 已提交
2600 2601 2602 2603 2604 2605 2606 2607 2608 2609 2610 2611 2612 2613 2614 2615 2616 2617 2618 2619 2620 2621 2622 2623 2624 2625 2626 2627 2628 2629 2630 2631 2632 2633 2634 2635 2636 2637 2638 2639 2640 2641 2642 2643 2644 2645 2646 2647 2648 2649 2650 2651 2652 2653 2654 2655 2656 2657 2658 2659 2660 2661 2662 2663 2664 2665 2666 2667 2668 2669 2670 2671 2672 2673 2674 2675 2676 2677 2678 2679 2680 2681 2682 2683 2684 2685 2686 2687 2688 2689
	String partition = field.getType() == Field::Types::UInt64 ? toString(field.get<UInt64>()) : field.safeGet<String>();

	if (!attach_part && !isValidMonthName(partition))
		throw Exception("Invalid partition format: " + partition + ". Partition should consist of 6 digits: YYYYMM",
						ErrorCodes::INVALID_PARTITION_NAME);

	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;
			if (name.substr(0, partition.size()) != partition)
				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 одновременно нельзя.
	UInt64 min_used_number = RESERVED_BLOCK_NUMBERS;

	{
		auto existing_parts = data.getDataParts();
		for (const auto & part : existing_parts)
		{
			min_used_number = std::min(min_used_number, part->left);
		}
	}

	if (parts.size() > min_used_number)
		throw Exception("Not enough free small block numbers for attaching parts: "
			+ toString(parts.size()) + " needed, " + toString(min_used_number) + " available", ErrorCodes::NOT_ENOUGH_BLOCK_NUMBERS);

	/// Добавим записи в лог.
	std::reverse(parts.begin(), parts.end());
	std::list<LogEntry> entries;
	zkutil::Ops ops;
	for (const String & part_name : parts)
	{
		ActiveDataPartSet::Part part;
		ActiveDataPartSet::parsePartName(part_name, part);
		part.left = part.right = --min_used_number;
		String new_part_name = ActiveDataPartSet::getPartName(part.left_date, part.right_date, part.left, part.right, part.level);

		LOG_INFO(log, "Will attach " << part_name << " as " << new_part_name);

		entries.emplace_back();
		LogEntry & entry = entries.back();
		entry.type = LogEntry::ATTACH_PART;
		entry.source_replica = replica_name;
		entry.source_part_name = part_name;
		entry.new_part_name = new_part_name;
		entry.attach_unreplicated = unreplicated;
		ops.push_back(new zkutil::Op::Create(
			zookeeper_path + "/log/log-", entry.toString(), zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
	}

	LOG_DEBUG(log, "Adding attaches to log");
	zookeeper->multi(ops);
	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);

		waitForAllReplicasToProcessLogEntry(entry);
	}
M
Merge  
Michael Kolupaev 已提交
2690 2691
}

M
Merge  
Michael Kolupaev 已提交
2692 2693
void StorageReplicatedMergeTree::drop()
{
M
Merge  
Michael Kolupaev 已提交
2694 2695
	if (is_read_only)
		throw Exception("Can't drop read-only replicated table (need to drop data in ZooKeeper as well)", ErrorCodes::TABLE_IS_READ_ONLY);
M
Merge  
Michael Kolupaev 已提交
2696

M
Merge  
Michael Kolupaev 已提交
2697 2698
	shutdown();

M
Merge  
Michael Kolupaev 已提交
2699
	LOG_INFO(log, "Removing replica " << replica_path);
M
Merge  
Michael Kolupaev 已提交
2700
	replica_is_active_node = nullptr;
M
Merge  
Michael Kolupaev 已提交
2701 2702
	zookeeper->tryRemoveRecursive(replica_path);

M
Merge  
Michael Kolupaev 已提交
2703
	/// Проверяем, что zookeeper_path существует: его могла удалить другая реплика после выполнения предыдущей строки.
M
Merge  
Michael Kolupaev 已提交
2704 2705
	Strings replicas;
	if (zookeeper->tryGetChildren(zookeeper_path + "/replicas", replicas) == ZOK && replicas.empty())
M
Merge  
Michael Kolupaev 已提交
2706 2707
	{
		LOG_INFO(log, "Removing table " << zookeeper_path << " (this might take several minutes)");
M
Merge  
Michael Kolupaev 已提交
2708
		zookeeper->tryRemoveRecursive(zookeeper_path);
M
Merge  
Michael Kolupaev 已提交
2709
	}
M
Merge  
Michael Kolupaev 已提交
2710 2711

	data.dropAllData();
M
Merge  
Michael Kolupaev 已提交
2712 2713
}

M
Merge  
Michael Kolupaev 已提交
2714 2715 2716 2717 2718 2719 2720 2721 2722 2723 2724 2725 2726 2727 2728
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: Можно обновить названия логгеров.
}

M
Merge  
Michael Kolupaev 已提交
2729 2730 2731 2732 2733 2734 2735 2736 2737 2738
AbandonableLockInZooKeeper StorageReplicatedMergeTree::allocateBlockNumber(const String & month_name)
{
	String month_path = zookeeper_path + "/block_numbers/" + month_name;
	if (!zookeeper->exists(month_path))
	{
		/// Создадим в block_numbers ноду для месяца и пропустим в ней 200 значений инкремента.
		/// Нужно, чтобы в будущем при необходимости можно было добавить данные в начало.
		zkutil::Ops ops;
		auto acl = zookeeper->getDefaultACL();
		ops.push_back(new zkutil::Op::Create(month_path, "", acl, zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
2739
		for (size_t i = 0; i < RESERVED_BLOCK_NUMBERS; ++i)
M
Merge  
Michael Kolupaev 已提交
2740 2741 2742 2743 2744 2745 2746 2747 2748 2749 2750 2751 2752
		{
			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);
}

M
Merge  
Michael Kolupaev 已提交
2753
void StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry(const LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
2754
{
M
Merge  
Michael Kolupaev 已提交
2755 2756
	LOG_DEBUG(log, "Waiting for all replicas to process " << entry.znode_name);

M
Merge  
Michael Kolupaev 已提交
2757
	UInt64 log_index = parse<UInt64>(entry.znode_name.substr(entry.znode_name.size() - 10));
M
Merge  
Michael Kolupaev 已提交
2758 2759 2760 2761 2762
	String log_entry_str = entry.toString();

	Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
	for (const String & replica : replicas)
	{
M
Merge  
Michael Kolupaev 已提交
2763 2764
		LOG_DEBUG(log, "Waiting for " << replica << " to pull " << entry.znode_name << " to queue");

M
Merge  
Michael Kolupaev 已提交
2765 2766 2767 2768 2769 2770 2771 2772 2773 2774 2775 2776
		/// Дождемся, пока запись попадет в очередь реплики.
		while (true)
		{
			zkutil::EventPtr event = new Poco::Event;

			String pointer = zookeeper->get(zookeeper_path + "/replicas/" + replica + "/log_pointer", nullptr, event);
			if (!pointer.empty() && parse<UInt64>(pointer) > log_index)
				break;

			event->wait();
		}

M
Merge  
Michael Kolupaev 已提交
2777 2778
		LOG_DEBUG(log, "Looking for " << entry.znode_name << " in " << replica << " queue");

M
Merge  
Michael Kolupaev 已提交
2779 2780 2781 2782 2783 2784 2785
		/// Найдем запись в очереди реплики.
		Strings queue_entries = zookeeper->getChildren(zookeeper_path + "/replicas/" + replica + "/queue");
		String entry_to_wait_for;

		for (const String & entry_name : queue_entries)
		{
			String queue_entry_str;
M
Merge  
Michael Kolupaev 已提交
2786 2787
			bool exists = zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/queue/" + entry_name, queue_entry_str);
			if (exists && queue_entry_str == log_entry_str)
M
Merge  
Michael Kolupaev 已提交
2788 2789 2790 2791 2792 2793 2794 2795 2796 2797
			{
				entry_to_wait_for = entry_name;
				break;
			}
		}

		/// Пока искали запись, ее уже выполнили и удалили.
		if (entry_to_wait_for.empty())
			continue;

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

M
Merge  
Michael Kolupaev 已提交
2800 2801 2802 2803 2804 2805 2806
		/// Дождемся, пока запись исчезнет из очереди реплики.
		while (true)
		{
			zkutil::EventPtr event = new Poco::Event;

			String unused;
			/// get вместо exists, чтобы не утек watch, если ноды уже нет.
M
Merge  
Michael Kolupaev 已提交
2807
			if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/queue/" + entry_to_wait_for, unused, nullptr, event))
M
Merge  
Michael Kolupaev 已提交
2808 2809 2810 2811 2812
				break;

			event->wait();
		}
	}
M
Merge  
Michael Kolupaev 已提交
2813 2814

	LOG_DEBUG(log, "Finished waiting for all replicas to process " << entry.znode_name);
M
Merge  
Michael Kolupaev 已提交
2815 2816 2817
}


M
Merge  
Michael Kolupaev 已提交
2818 2819 2820
void StorageReplicatedMergeTree::LogEntry::writeText(WriteBuffer & out) const
{
	writeString("format version: 1\n", out);
M
Merge  
Michael Kolupaev 已提交
2821 2822 2823
	writeString("source replica: ", out);
	writeString(source_replica, out);
	writeString("\n", out);
M
Merge  
Michael Kolupaev 已提交
2824 2825 2826 2827 2828 2829 2830 2831 2832 2833 2834 2835 2836 2837 2838 2839
	switch (type)
	{
		case GET_PART:
			writeString("get\n", out);
			writeString(new_part_name, out);
			break;
		case MERGE_PARTS:
			writeString("merge\n", out);
			for (const String & s : parts_to_merge)
			{
				writeString(s, out);
				writeString("\n", out);
			}
			writeString("into\n", out);
			writeString(new_part_name, out);
			break;
M
Merge  
Michael Kolupaev 已提交
2840
		case DROP_RANGE:
M
Merge  
Michael Kolupaev 已提交
2841 2842 2843 2844
			if (detach)
				writeString("detach\n", out);
			else
				writeString("drop\n", out);
M
Merge  
Michael Kolupaev 已提交
2845 2846
			writeString(new_part_name, out);
			break;
M
Merge  
Michael Kolupaev 已提交
2847 2848 2849 2850 2851 2852 2853 2854 2855 2856
		case ATTACH_PART:
			writeString("attach\n", out);
			if (attach_unreplicated)
				writeString("unreplicated\n", out);
			else
				writeString("detached\n", out);
			writeString(source_part_name, out);
			writeString("\ninto\n", out);
			writeString(new_part_name, out);
			break;
M
Merge  
Michael Kolupaev 已提交
2857 2858 2859 2860 2861 2862 2863 2864 2865
	}
	writeString("\n", out);
}

void StorageReplicatedMergeTree::LogEntry::readText(ReadBuffer & in)
{
	String type_str;

	assertString("format version: 1\n", in);
M
Michael Kolupaev 已提交
2866
	assertString("source replica: ", in);
M
Merge  
Michael Kolupaev 已提交
2867
	readString(source_replica, in);
M
Merge  
Michael Kolupaev 已提交
2868
	assertString("\n", in);
M
Michael Kolupaev 已提交
2869 2870
	readString(type_str, in);
	assertString("\n", in);
M
Merge  
Michael Kolupaev 已提交
2871 2872 2873 2874 2875 2876 2877 2878 2879 2880 2881 2882 2883 2884 2885 2886 2887 2888 2889 2890

	if (type_str == "get")
	{
		type = GET_PART;
		readString(new_part_name, in);
	}
	else if (type_str == "merge")
	{
		type = MERGE_PARTS;
		while (true)
		{
			String s;
			readString(s, in);
			assertString("\n", in);
			if (s == "into")
				break;
			parts_to_merge.push_back(s);
		}
		readString(new_part_name, in);
	}
M
Merge  
Michael Kolupaev 已提交
2891
	else if (type_str == "drop" || type_str == "detach")
M
Merge  
Michael Kolupaev 已提交
2892 2893
	{
		type = DROP_RANGE;
M
Merge  
Michael Kolupaev 已提交
2894
		detach = type_str == "detach";
M
Merge  
Michael Kolupaev 已提交
2895 2896
		readString(new_part_name, in);
	}
M
Merge  
Michael Kolupaev 已提交
2897 2898 2899 2900 2901 2902 2903 2904 2905 2906 2907 2908 2909 2910 2911 2912
	else if (type_str == "attach")
	{
		type = ATTACH_PART;
		String source_type;
		readString(source_type, in);
		if (source_type == "unreplicated")
			attach_unreplicated = true;
		else if (source_type == "detached")
			attach_unreplicated = false;
		else
			throw Exception("Bad format: expected 'unreplicated' or 'detached', found '" + source_type + "'", ErrorCodes::CANNOT_PARSE_TEXT);
		assertString("\n", in);
		readString(source_part_name, in);
		assertString("\ninto\n", in);
		readString(new_part_name, in);
	}
M
Merge  
Michael Kolupaev 已提交
2913 2914 2915
	assertString("\n", in);
}

M
Merge  
Michael Kolupaev 已提交
2916
}