StorageReplicatedMergeTree.cpp 99.1 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

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

M
Merge  
Michael Kolupaev 已提交
22

23 24 25 26 27 28 29 30 31 32
/// Используется для проверки, выставили ли ноду 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 已提交
33 34 35
StorageReplicatedMergeTree::StorageReplicatedMergeTree(
	const String & zookeeper_path_,
	const String & replica_name_,
M
Merge  
Michael Kolupaev 已提交
36
	bool attach,
M
Merge  
Michael Kolupaev 已提交
37 38
	const String & path_, const String & database_name_, const String & name_,
	NamesAndTypesListPtr columns_,
M
Merge  
Michael Kolupaev 已提交
39
	Context & context_,
M
Merge  
Michael Kolupaev 已提交
40 41 42 43 44 45 46 47
	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_)
	:
48
	context(context_), zookeeper(context.getZooKeeper()), database_name(database_name_),
M
Merge  
Michael Kolupaev 已提交
49 50 51
	table_name(name_), full_path(path_ + escapeForFileName(table_name) + '/'),
	zookeeper_path(context.getMacros().expand(zookeeper_path_)),
	replica_name(context.getMacros().expand(replica_name_)),
M
Merge  
Michael Kolupaev 已提交
52
	data(	full_path, columns_, context_, primary_expr_ast_, date_column_name_, sampling_expression_,
53
			index_granularity_, mode_, sign_column_, settings_, database_name + "." + table_name, true,
M
Merge  
Michael Kolupaev 已提交
54
			std::bind(&StorageReplicatedMergeTree::enqueuePartForCheck, this, std::placeholders::_1)),
M
Merge  
Michael Kolupaev 已提交
55
	reader(data), writer(data), merger(data), fetcher(data),
56
	log(&Logger::get(database_name + "." + table_name + " (StorageReplicatedMergeTree)")),
M
Merge  
Michael Kolupaev 已提交
57
	shutdown_event(false)
M
Merge  
Michael Kolupaev 已提交
58
{
59 60
	if (!zookeeper_path.empty() && zookeeper_path.back() == '/')
		zookeeper_path.resize(zookeeper_path.size() - 1);
M
Merge  
Michael Kolupaev 已提交
61 62 63 64 65 66 67 68 69 70 71 72 73 74 75
	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 已提交
76 77
	if (!zookeeper)
	{
M
Merge  
Michael Kolupaev 已提交
78 79 80
		if (!attach)
			throw Exception("Can't create replicated table without ZooKeeper", ErrorCodes::NO_ZOOKEEPER);

M
Merge  
Michael Kolupaev 已提交
81
		goReadOnlyPermanently();
M
Merge  
Michael Kolupaev 已提交
82 83 84
		return;
	}

M
Merge  
Michael Kolupaev 已提交
85 86
	if (!attach)
	{
M
Merge  
Michael Kolupaev 已提交
87
		createTableIfNotExists();
M
Merge  
Michael Kolupaev 已提交
88

M
Merge  
Michael Kolupaev 已提交
89
		checkTableStructure(false, false);
M
Merge  
Michael Kolupaev 已提交
90
		createReplica();
M
Merge  
Michael Kolupaev 已提交
91 92 93
	}
	else
	{
M
Merge  
Michael Kolupaev 已提交
94
		checkTableStructure(skip_sanity_checks, true);
M
Merge  
Michael Kolupaev 已提交
95
		checkParts(skip_sanity_checks);
M
Merge  
Michael Kolupaev 已提交
96
	}
M
Merge  
Michael Kolupaev 已提交
97

M
Merge  
Michael Kolupaev 已提交
98
	initVirtualParts();
M
Merge  
Michael Kolupaev 已提交
99
	loadQueue();
M
Merge  
Michael Kolupaev 已提交
100 101 102 103 104

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

M
Merge  
Michael Kolupaev 已提交
106
		unreplicated_data.reset(new MergeTreeData(unreplicated_path, columns_, context_, primary_expr_ast_,
M
Merge  
Michael Kolupaev 已提交
107
			date_column_name_, sampling_expression_, index_granularity_, mode_, sign_column_, settings_,
M
Merge  
Michael Kolupaev 已提交
108
			database_name_ + "." + table_name + "[unreplicated]", false));
109 110 111

		unreplicated_data->loadDataParts(skip_sanity_checks);

M
Merge  
Michael Kolupaev 已提交
112
		unreplicated_reader.reset(new MergeTreeDataSelectExecutor(*unreplicated_data));
M
Merge  
Michael Kolupaev 已提交
113
		unreplicated_merger.reset(new MergeTreeDataMerger(*unreplicated_data));
M
Merge  
Michael Kolupaev 已提交
114
	}
M
Merge  
Michael Kolupaev 已提交
115

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

119
	/// В этом потоке реплика будет активирована.
M
Merge  
Michael Kolupaev 已提交
120
	restarting_thread = std::thread(&StorageReplicatedMergeTree::restartingThread, this);
M
Merge  
Michael Kolupaev 已提交
121 122 123 124 125
}

StoragePtr StorageReplicatedMergeTree::create(
	const String & zookeeper_path_,
	const String & replica_name_,
M
Merge  
Michael Kolupaev 已提交
126
	bool attach,
M
Merge  
Michael Kolupaev 已提交
127 128
	const String & path_, const String & database_name_, const String & name_,
	NamesAndTypesListPtr columns_,
M
Merge  
Michael Kolupaev 已提交
129
	Context & context_,
M
Merge  
Michael Kolupaev 已提交
130 131 132 133 134 135 136 137
	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_)
{
M
Merge  
Michael Kolupaev 已提交
138
	StorageReplicatedMergeTree * res = new StorageReplicatedMergeTree(zookeeper_path_, replica_name_, attach,
M
Merge  
Michael Kolupaev 已提交
139
		path_, database_name_, name_, columns_, context_, primary_expr_ast_, date_column_name_, sampling_expression_,
M
Merge  
Michael Kolupaev 已提交
140 141
		index_granularity_, mode_, sign_column_, settings_);
	StoragePtr res_ptr = res->thisPtr();
M
Merge  
Michael Kolupaev 已提交
142 143 144
	if (!res->is_read_only)
	{
		String endpoint_name = "ReplicatedMergeTree:" + res->replica_path;
M
Merge  
Michael Kolupaev 已提交
145
		InterserverIOEndpointPtr endpoint = new ReplicatedMergeTreePartsServer(res->data, *res);
M
Merge  
Michael Kolupaev 已提交
146 147
		res->endpoint_holder = new InterserverIOEndpointHolder(endpoint_name, endpoint, res->context.getInterserverIOHandler());
	}
M
Merge  
Michael Kolupaev 已提交
148
	return res_ptr;
M
Merge  
Michael Kolupaev 已提交
149 150
}

M
Merge  
Michael Kolupaev 已提交
151 152 153 154 155 156 157 158
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 已提交
159

M
Merge  
Michael Kolupaev 已提交
160
void StorageReplicatedMergeTree::createTableIfNotExists()
M
Merge  
Michael Kolupaev 已提交
161
{
M
Merge  
Michael Kolupaev 已提交
162 163
	if (zookeeper->exists(zookeeper_path))
		return;
M
Merge  
Michael Kolupaev 已提交
164

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

M
Merge  
Michael Kolupaev 已提交
167 168
	zookeeper->createAncestors(zookeeper_path);

M
Merge  
Michael Kolupaev 已提交
169
	/// Запишем метаданные таблицы, чтобы реплики могли сверять с ними параметры таблицы.
M
Merge  
Michael Kolupaev 已提交
170 171 172 173 174 175 176 177
	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 已提交
178

M
Merge  
Michael Kolupaev 已提交
179 180 181 182 183 184 185 186 187 188 189 190 191 192 193 194 195 196 197 198 199 200 201 202 203
	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));
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/columns", data.getColumnsList().toString(),
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	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 已提交
204
}
M
Merge  
Michael Kolupaev 已提交
205

M
Merge  
Michael Kolupaev 已提交
206 207
/** Проверить, что список столбцов и настройки таблицы совпадают с указанными в ZK (/metadata).
	* Если нет - бросить исключение.
M
Merge  
Michael Kolupaev 已提交
208
	*/
M
Merge  
Michael Kolupaev 已提交
209
void StorageReplicatedMergeTree::checkTableStructure(bool skip_sanity_checks, bool allow_alter)
M
Merge  
Michael Kolupaev 已提交
210
{
M
Merge  
Michael Kolupaev 已提交
211
	String metadata_str = zookeeper->get(zookeeper_path + "/metadata");
M
Merge  
Michael Kolupaev 已提交
212 213 214 215 216 217 218 219 220 221 222 223 224
	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 已提交
225 226
	/// NOTE: Можно сделать менее строгую проверку совпадения выражений, чтобы таблицы не ломались от небольших изменений
	///       в коде formatAST.
M
Merge  
Michael Kolupaev 已提交
227
	assertString(formattedAST(data.primary_expr_ast), buf);
M
Merge  
Michael Kolupaev 已提交
228
	assertString("\n", buf);
M
Merge  
Michael Kolupaev 已提交
229
	assertEOF(buf);
M
Merge  
Michael Kolupaev 已提交
230 231 232 233

	zkutil::Stat stat;
	auto columns = NamesAndTypesList::parse(zookeeper->get(zookeeper_path + "/columns", &stat), context.getDataTypeFactory());
	columns_version = stat.version;
M
Merge  
Michael Kolupaev 已提交
234
	if (columns != data.getColumnsList())
M
Merge  
Michael Kolupaev 已提交
235
	{
M
Merge  
Michael Kolupaev 已提交
236
		if (allow_alter && (data.getColumnsList().sizeOfDifference(columns) <= 2 || skip_sanity_checks))
M
Merge  
Michael Kolupaev 已提交
237
		{
M
Merge  
Michael Kolupaev 已提交
238
			LOG_WARNING(log, "Table structure in ZooKeeper is a little different from local table structure. Assuming ALTER.");
239 240 241 242

			/// Без всяких блокировок, потому что таблица еще не создана.
			InterpreterAlterQuery::updateMetadata(database_name, table_name, columns, context);
			data.setColumnsList(columns);
M
Merge  
Michael Kolupaev 已提交
243
		}
M
Merge  
Michael Kolupaev 已提交
244
		else
M
Merge  
Michael Kolupaev 已提交
245
		{
M
Merge  
Michael Kolupaev 已提交
246
			throw Exception("Table structure in ZooKeeper is too different from local table structure.",
M
Merge  
Michael Kolupaev 已提交
247
							ErrorCodes::INCOMPATIBLE_COLUMNS);
M
Merge  
Michael Kolupaev 已提交
248
		}
M
Merge  
Michael Kolupaev 已提交
249 250
	}
}
M
Merge  
Michael Kolupaev 已提交
251

M
Merge  
Michael Kolupaev 已提交
252 253
void StorageReplicatedMergeTree::createReplica()
{
M
Merge  
Michael Kolupaev 已提交
254
	LOG_DEBUG(log, "Creating replica " << replica_path);
M
Merge  
Michael Kolupaev 已提交
255

M
Merge  
Michael Kolupaev 已提交
256 257 258 259 260 261 262 263
	/// Создадим пустую реплику. Ноду 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));
264 265 266 267 268 269 270 271 272 273 274 275

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

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

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

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

M
Merge  
Michael Kolupaev 已提交
289 290 291
	Stat stat;
	zookeeper->exists(replica_path, &stat);
	auto my_create_time = stat.czxid;
M
Merge  
Michael Kolupaev 已提交
292

M
Merge  
Michael Kolupaev 已提交
293 294
	std::random_shuffle(replicas.begin(), replicas.end());
	for (const String & replica : replicas)
M
Merge  
Michael Kolupaev 已提交
295
	{
M
Merge  
Michael Kolupaev 已提交
296 297 298 299 300 301 302 303
		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 已提交
304 305
	}

M
Merge  
Michael Kolupaev 已提交
306
	if (source_replica.empty())
M
Merge  
Michael Kolupaev 已提交
307
	{
M
Merge  
Michael Kolupaev 已提交
308
		LOG_INFO(log, "This is the first replica");
M
Merge  
Michael Kolupaev 已提交
309
	}
M
Merge  
Michael Kolupaev 已提交
310
	else
M
Merge  
Michael Kolupaev 已提交
311
	{
M
Merge  
Michael Kolupaev 已提交
312
		LOG_INFO(log, "Will mimic " << source_replica);
M
Merge  
Michael Kolupaev 已提交
313

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

M
Merge  
Michael Kolupaev 已提交
316 317 318 319 320 321 322 323 324 325 326 327 328 329 330 331 332 333 334 335 336 337 338 339 340 341 342 343 344 345 346 347 348 349 350 351 352 353 354
		/** Если эталонная реплика еще не до конца создана, подождем.
		  * 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");
355 356
		ActiveDataPartSet active_parts_set(parts);

M
Merge  
Michael Kolupaev 已提交
357 358 359 360 361 362 363 364 365 366 367 368 369 370 371 372 373 374
		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 已提交
375
	}
M
Merge  
Michael Kolupaev 已提交
376 377

	zookeeper->create(replica_path + "/columns", data.getColumnsList().toString(), zkutil::CreateMode::Persistent);
M
Merge  
Michael Kolupaev 已提交
378
}
M
Merge  
Michael Kolupaev 已提交
379

M
Merge  
Michael Kolupaev 已提交
380 381 382 383 384
void StorageReplicatedMergeTree::activateReplica()
{
	std::stringstream host;
	host << "host: " << context.getInterserverIOHost() << std::endl;
	host << "port: " << context.getInterserverIOPort() << std::endl;
M
Merge  
Michael Kolupaev 已提交
385

M
Merge  
Michael Kolupaev 已提交
386 387 388 389 390 391 392 393 394
	/** Если нода отмечена как активная, но отметка сделана в этом же экземпляре, удалим ее.
	  * Такое возможно только при истечении сессии в 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 已提交
395
	/// Одновременно объявим, что эта реплика активна, и обновим хост.
M
Merge  
Michael Kolupaev 已提交
396
	zkutil::Ops ops;
397 398
	ops.push_back(new zkutil::Op::Create(replica_path + "/is_active",
		active_node_identifier, zookeeper->getDefaultACL(), zkutil::CreateMode::Ephemeral));
M
Merge  
Michael Kolupaev 已提交
399
	ops.push_back(new zkutil::Op::SetData(replica_path + "/host", host.str(), -1));
M
Merge  
Michael Kolupaev 已提交
400 401 402

	try
	{
M
Merge  
Michael Kolupaev 已提交
403
		zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
404
	}
405
	catch (const zkutil::KeeperException & e)
M
Merge  
Michael Kolupaev 已提交
406
	{
407
		if (e.code == ZNODEEXISTS)
M
Merge  
Michael Kolupaev 已提交
408 409 410 411 412
			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 已提交
413

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

M
Merge  
Michael Kolupaev 已提交
417
void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
M
Merge  
Michael Kolupaev 已提交
418
{
M
Merge  
Michael Kolupaev 已提交
419
	Strings expected_parts_vec = zookeeper->getChildren(replica_path + "/parts");
M
Merge  
Michael Kolupaev 已提交
420 421

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

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

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

M
Merge  
Michael Kolupaev 已提交
429 430 431 432 433 434 435 436
	for (const auto & part : parts)
	{
		if (expected_parts.count(part->name))
		{
			expected_parts.erase(part->name);
		}
		else
		{
M
Merge  
Michael Kolupaev 已提交
437
			unexpected_parts.insert(part);
M
Merge  
Michael Kolupaev 已提交
438 439 440
		}
	}

M
Merge  
Michael Kolupaev 已提交
441
	/// Какие локальные куски добавить в ZK.
M
Merge  
Michael Kolupaev 已提交
442
	MergeTreeData::DataPartsVector parts_to_add;
M
Merge  
Michael Kolupaev 已提交
443 444 445 446

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

M
Merge  
Michael Kolupaev 已提交
447 448
	for (const String & missing_name : expected_parts)
	{
M
Merge  
Michael Kolupaev 已提交
449
		/// Если локально не хватает какого-то куска, но есть покрывающий его кусок, можно заменить в ZK недостающий покрывающим.
M
Merge  
Michael Kolupaev 已提交
450
		auto containing = data.getActiveContainingPart(missing_name);
M
Merge  
Michael Kolupaev 已提交
451 452 453 454 455 456 457 458 459 460
		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 已提交
461
		{
M
Merge  
Michael Kolupaev 已提交
462
			LOG_ERROR(log, "Fetching missing part " << missing_name);
M
Merge  
Michael Kolupaev 已提交
463
			parts_to_fetch.push_back(missing_name);
M
Merge  
Michael Kolupaev 已提交
464 465
		}
	}
M
Merge  
Michael Kolupaev 已提交
466

M
Merge  
Michael Kolupaev 已提交
467 468 469 470 471 472
	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 已提交
473
					 + toString(expected_parts.size()) + " missing obsolete parts, "
M
Merge  
Michael Kolupaev 已提交
474 475
					 + toString(parts_to_fetch.size()) + " missing parts";

476 477 478 479 480 481 482 483 484 485 486 487 488 489 490
	/** Можно автоматически синхронизировать данные,
	  * если количество ошибок каждого из четырёх типов не больше соответствующих порогов,
	  * или если отношение общего количества ошибок к общему количеству кусков (минимальному - в локальной файловой системе или в ZK)
	  *  не больше некоторого отношения (например 5%).
	  */

	size_t min_parts_local_or_expected = std::min(expected_parts_vec.size(), parts.size());

	bool insane =
		(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)
		&& ((parts_to_add.size() + unexpected_parts.size() + expected_parts.size() + parts_to_fetch.size())
			> min_parts_local_or_expected * data.settings.replicated_max_ratio_of_wrong_parts);
M
Merge  
Michael Kolupaev 已提交
491

M
Merge  
Michael Kolupaev 已提交
492 493
	if (insane)
	{
494 495 496 497 498
		if (skip_sanity_checks)
			LOG_WARNING(log, sanity_report);
		else
			throw Exception("The local set of parts of table " + getTableName() + " doesn't look like the set of parts in ZooKeeper. "
				+ sanity_report, ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS);
M
Merge  
Michael Kolupaev 已提交
499 500
	}

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

M
Merge  
Michael Kolupaev 已提交
506
		zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
507
		checkPartAndAddToZooKeeper(part, ops);
M
Merge  
Michael Kolupaev 已提交
508
		zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
509
	}
M
Merge  
Michael Kolupaev 已提交
510

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

		zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
517
		ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + name + "/columns", -1));
M
Merge  
Michael Kolupaev 已提交
518 519 520 521 522 523 524 525 526 527 528 529
		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 已提交
530
		log_entry.source_replica = "";
M
Merge  
Michael Kolupaev 已提交
531 532 533
		log_entry.new_part_name = name;

		/// Полагаемся, что это происходит до загрузки очереди (loadQueue).
M
Merge  
Michael Kolupaev 已提交
534
		zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
535
		ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + name + "/columns", -1));
M
Merge  
Michael Kolupaev 已提交
536 537
		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 已提交
538 539
		ops.push_back(new zkutil::Op::Create(
			replica_path + "/queue/queue-", log_entry.toString(), zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
M
Merge  
Michael Kolupaev 已提交
540
		zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
541 542 543
	}

	/// Удалим лишние локальные куски.
544
	for (const MergeTreeData::DataPartPtr & part : unexpected_parts)
M
Merge  
Michael Kolupaev 已提交
545
	{
M
Merge  
Michael Kolupaev 已提交
546
		LOG_ERROR(log, "Renaming unexpected part " << part->name << " to ignored_" + part->name);
M
Merge  
Michael Kolupaev 已提交
547
		data.renameAndDetachPart(part, "ignored_", true);
M
Merge  
Michael Kolupaev 已提交
548 549
	}
}
M
Merge  
Michael Kolupaev 已提交
550

M
Merge  
Michael Kolupaev 已提交
551 552 553 554 555 556 557
void StorageReplicatedMergeTree::initVirtualParts()
{
	auto parts = data.getDataParts();
	for (const auto & part : parts)
		virtual_parts.add(part->name);
}

558
void StorageReplicatedMergeTree::checkPartAndAddToZooKeeper(const MergeTreeData::DataPartPtr & part, zkutil::Ops & ops, String part_name)
M
Merge  
Michael Kolupaev 已提交
559
{
M
Merge  
Michael Kolupaev 已提交
560 561 562
	if (part_name.empty())
		part_name = part->name;

563
	check(part->columns);
M
Merge  
Michael Kolupaev 已提交
564
	int expected_columns_version = columns_version;
565

M
Merge  
Michael Kolupaev 已提交
566 567 568 569 570
	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 已提交
571
	{
M
Merge  
Michael Kolupaev 已提交
572 573
		zkutil::Stat stat_before, stat_after;
		String columns_str;
M
Merge  
Michael Kolupaev 已提交
574
		if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name + "/columns", columns_str, &stat_before))
M
Merge  
Michael Kolupaev 已提交
575 576 577
			continue;
		if (columns_str != expected_columns_str)
		{
M
Merge  
Michael Kolupaev 已提交
578
			LOG_INFO(log, "Not checking checksums of part " << part_name << " with replica " << replica
M
Merge  
Michael Kolupaev 已提交
579 580 581
				<< " because columns are different");
			continue;
		}
M
Merge  
Michael Kolupaev 已提交
582
		String checksums_str;
M
Merge  
Michael Kolupaev 已提交
583 584
		/// Проверим, что версия ноды со столбцами не изменилась, пока мы читали checksums.
		/// Это гарантирует, что столбцы и чексуммы относятся к одним и тем же данным.
M
Merge  
Michael Kolupaev 已提交
585 586
		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 已提交
587
			stat_before.version != stat_after.version)
M
Merge  
Michael Kolupaev 已提交
588
		{
M
Merge  
Michael Kolupaev 已提交
589
			LOG_INFO(log, "Not checking checksums of part " << part_name << " with replica " << replica
M
Merge  
Michael Kolupaev 已提交
590 591
				<< " because part changed while we were reading its checksums");
			continue;
M
Merge  
Michael Kolupaev 已提交
592
		}
M
Merge  
Michael Kolupaev 已提交
593 594 595

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

M
Merge  
Michael Kolupaev 已提交
598
	if (zookeeper->exists(replica_path + "/parts/" + part_name))
M
Merge  
Michael Kolupaev 已提交
599
	{
M
Merge  
Michael Kolupaev 已提交
600
		LOG_ERROR(log, "checkPartAndAddToZooKeeper: node " << replica_path + "/parts/" + part_name << " already exists");
M
Merge  
Michael Kolupaev 已提交
601 602 603
		return;
	}

604 605 606
	ops.push_back(new zkutil::Op::Check(
		zookeeper_path + "/columns",
		expected_columns_version));
M
Merge  
Michael Kolupaev 已提交
607
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
608
		replica_path + "/parts/" + part_name,
M
Merge  
Michael Kolupaev 已提交
609
		"",
M
Merge  
Michael Kolupaev 已提交
610
		zookeeper->getDefaultACL(),
M
Merge  
Michael Kolupaev 已提交
611
		zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
612
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
613
		replica_path + "/parts/" + part_name + "/columns",
M
Merge  
Michael Kolupaev 已提交
614 615 616
		part->columns.toString(),
		zookeeper->getDefaultACL(),
		zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
617
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
618
		replica_path + "/parts/" + part_name + "/checksums",
M
Merge  
Michael Kolupaev 已提交
619
		part->checksums.toString(),
M
Merge  
Michael Kolupaev 已提交
620
		zookeeper->getDefaultACL(),
M
Merge  
Michael Kolupaev 已提交
621 622 623
		zkutil::CreateMode::Persistent));
}

M
Merge  
Michael Kolupaev 已提交
624 625
void StorageReplicatedMergeTree::loadQueue()
{
M
Merge  
Michael Kolupaev 已提交
626
	std::unique_lock<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
627

M
Merge  
Michael Kolupaev 已提交
628
	Strings children = zookeeper->getChildren(replica_path + "/queue");
M
Merge  
Michael Kolupaev 已提交
629 630 631
	std::sort(children.begin(), children.end());
	for (const String & child : children)
	{
M
Merge  
Michael Kolupaev 已提交
632
		String s = zookeeper->get(replica_path + "/queue/" + child);
M
Merge  
Michael Kolupaev 已提交
633 634 635
		LogEntryPtr entry = LogEntry::parse(s);
		entry->znode_name = child;
		entry->addResultToVirtualParts(*this);
M
Merge  
Michael Kolupaev 已提交
636
		queue.push_back(entry);
M
Merge  
Michael Kolupaev 已提交
637 638 639
	}
}

M
Merge  
Michael Kolupaev 已提交
640
void StorageReplicatedMergeTree::pullLogsToQueue(zkutil::EventPtr next_update_event)
M
Merge  
Michael Kolupaev 已提交
641
{
M
Merge  
Michael Kolupaev 已提交
642
	std::unique_lock<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
643

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

M
Merge  
Michael Kolupaev 已提交
647
	if (index_str.empty())
M
Merge  
Michael Kolupaev 已提交
648
	{
M
Merge  
Michael Kolupaev 已提交
649 650 651
		/// Если у нас еще нет указателя на лог, поставим указатель на первую запись в нем.
		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 已提交
652

M
Merge  
Michael Kolupaev 已提交
653
		zookeeper->set(replica_path + "/log_pointer", toString(index));
M
Merge  
Michael Kolupaev 已提交
654 655
	}
	else
M
Merge  
Michael Kolupaev 已提交
656
	{
M
Merge  
Michael Kolupaev 已提交
657
		index = parse<UInt64>(index_str);
M
Merge  
Michael Kolupaev 已提交
658
	}
M
Merge  
Michael Kolupaev 已提交
659

M
Merge  
Michael Kolupaev 已提交
660 661
	UInt64 first_index = index;

M
Merge  
Michael Kolupaev 已提交
662
	size_t count = 0;
M
Merge  
Michael Kolupaev 已提交
663 664
	String entry_str;
	while (zookeeper->tryGet(zookeeper_path + "/log/log-" + padIndex(index), entry_str))
M
Merge  
Michael Kolupaev 已提交
665
	{
M
Merge  
Michael Kolupaev 已提交
666
		++count;
M
Merge  
Michael Kolupaev 已提交
667
		++index;
M
Merge  
Michael Kolupaev 已提交
668

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

M
Merge  
Michael Kolupaev 已提交
671 672 673
		/// Одновременно добавим запись в очередь и продвинем указатель на лог.
		zkutil::Ops ops;
		ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
674
			replica_path + "/queue/queue-", entry_str, zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
M
Merge  
Michael Kolupaev 已提交
675
		ops.push_back(new zkutil::Op::SetData(
M
Merge  
Michael Kolupaev 已提交
676
			replica_path + "/log_pointer", toString(index), -1));
M
Merge  
Michael Kolupaev 已提交
677
		auto results = zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
678

679
		String path_created = dynamic_cast<zkutil::Op::Create &>(ops[0]).getPathCreated();
M
Merge  
Michael Kolupaev 已提交
680 681
		entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
		entry->addResultToVirtualParts(*this);
M
Merge  
Michael Kolupaev 已提交
682
		queue.push_back(entry);
M
Merge  
Michael Kolupaev 已提交
683
	}
M
Merge  
Michael Kolupaev 已提交
684

M
Merge  
Michael Kolupaev 已提交
685 686 687 688
	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 已提交
689
	}
M
Merge  
Michael Kolupaev 已提交
690

M
Merge  
Michael Kolupaev 已提交
691 692 693
	if (!count)
		return;

M
Merge  
Michael Kolupaev 已提交
694 695
	if (queue_task_handle)
		queue_task_handle->wake();
M
Merge  
Michael Kolupaev 已提交
696

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

M
Merge  
Michael Kolupaev 已提交
700
bool StorageReplicatedMergeTree::shouldExecuteLogEntry(const LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
701
{
M
Merge  
Michael Kolupaev 已提交
702 703
	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 已提交
704 705 706 707 708 709
	{
		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 已提交
710 711 712 713 714 715 716 717 718 719 720
	if (entry.type == LogEntry::MERGE_PARTS)
	{
		/** Если какая-то из нужных частей сейчас передается или мерджится, подождем окончания этой операции.
		  * Иначе, даже если всех нужных частей для мерджа нет, нужно попытаться сделать мердж.
		  * Если каких-то частей не хватает, вместо мерджа будет попытка скачать кусок.
		  * Такая ситуация возможна, если получение какого-то куска пофейлилось, и его переместили в конец очереди.
		  */
		for (const auto & name : entry.parts_to_merge)
		{
			if (future_parts.count(name))
			{
M
Merge  
Michael Kolupaev 已提交
721
				LOG_TRACE(log, "Not merging into part " << entry.new_part_name << " because part " << name << " is not ready yet.");
M
Merge  
Michael Kolupaev 已提交
722 723 724 725 726 727
				return false;
			}
		}
	}

	return true;
M
Merge  
Michael Kolupaev 已提交
728 729
}

M
Merge  
Michael Kolupaev 已提交
730
bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry, BackgroundProcessingPool::Context & pool_context)
M
Merge  
Michael Kolupaev 已提交
731
{
M
Merge  
Michael Kolupaev 已提交
732
	if (entry.type == LogEntry::DROP_RANGE)
M
Merge  
Michael Kolupaev 已提交
733 734 735 736
	{
		executeDropRange(entry);
		return true;
	}
M
Merge  
Michael Kolupaev 已提交
737

M
Merge  
Michael Kolupaev 已提交
738
	if (entry.type == LogEntry::GET_PART ||
M
Merge  
Michael Kolupaev 已提交
739 740
		entry.type == LogEntry::MERGE_PARTS ||
		entry.type == LogEntry::ATTACH_PART)
M
Merge  
Michael Kolupaev 已提交
741 742
	{
		/// Если у нас уже есть этот кусок или покрывающий его кусок, ничего делать не нужно.
M
Merge  
Michael Kolupaev 已提交
743
		MergeTreeData::DataPartPtr containing_part = data.getActiveContainingPart(entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
744 745

		/// Даже если кусок есть локально, его (в исключительных случаях) может не быть в zookeeper.
M
Merge  
Michael Kolupaev 已提交
746
		if (containing_part && zookeeper->exists(replica_path + "/parts/" + containing_part->name))
M
Merge  
Michael Kolupaev 已提交
747
		{
M
Merge  
Michael Kolupaev 已提交
748 749
			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 已提交
750
			return true;
M
Merge  
Michael Kolupaev 已提交
751
		}
M
Merge  
Michael Kolupaev 已提交
752 753
	}

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

	bool do_fetch = false;

M
Merge  
Michael Kolupaev 已提交
759 760
	if (entry.type == LogEntry::GET_PART)
	{
M
Merge  
Michael Kolupaev 已提交
761
		do_fetch = true;
M
Merge  
Michael Kolupaev 已提交
762
	}
M
Merge  
Michael Kolupaev 已提交
763 764 765 766
	else if (entry.type == LogEntry::ATTACH_PART)
	{
		do_fetch = !executeAttachPart(entry);
	}
M
Merge  
Michael Kolupaev 已提交
767 768
	else if (entry.type == LogEntry::MERGE_PARTS)
	{
769 770 771 772 773 774 775 776
		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 已提交
777
		MergeTreeData::DataPartsVector parts;
778
		bool have_all_parts = true;
M
Merge  
Michael Kolupaev 已提交
779 780
		for (const String & name : entry.parts_to_merge)
		{
M
Merge  
Michael Kolupaev 已提交
781
			MergeTreeData::DataPartPtr part = data.getActiveContainingPart(name);
M
Merge  
Michael Kolupaev 已提交
782 783 784 785 786 787 788
			if (!part)
			{
				have_all_parts = false;
				break;
			}
			if (part->name != name)
			{
M
Merge  
Michael Kolupaev 已提交
789 790
				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 已提交
791 792 793
				have_all_parts = false;
				break;
			}
M
Merge  
Michael Kolupaev 已提交
794 795
			parts.push_back(part);
		}
M
Merge  
Michael Kolupaev 已提交
796

M
Merge  
Michael Kolupaev 已提交
797
		if (!have_all_parts)
M
Merge  
Michael Kolupaev 已提交
798
		{
M
Merge  
Michael Kolupaev 已提交
799 800 801
			/// Если нет всех нужных кусков, попробуем взять у кого-нибудь уже помердженный кусок.
			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 已提交
802
		}
M
Merge  
Michael Kolupaev 已提交
803 804
		else
		{
M
Merge  
Michael Kolupaev 已提交
805 806 807
			/// Если собираемся сливать большие куски, увеличим счетчик потоков, сливающих большие куски.
			for (const auto & part : parts)
			{
M
Merge  
Michael Kolupaev 已提交
808
				if (part->size_in_bytes > data.settings.max_bytes_to_merge_parts_small)
M
Merge  
Michael Kolupaev 已提交
809 810 811 812 813 814 815
				{
					pool_context.incrementCounter("big merges");
					pool_context.incrementCounter("replicated big merges");
					break;
				}
			}

M
Merge  
Michael Kolupaev 已提交
816
			auto table_lock = lockStructure(false);
817

818
			const auto & merge_entry = context.getMergeList().insert(database_name, table_name, entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
819
			MergeTreeData::Transaction transaction;
820
			MergeTreeData::DataPartPtr part = merger.mergeParts(parts, entry.new_part_name, *merge_entry, &transaction);
M
Merge  
Michael Kolupaev 已提交
821 822

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

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

M
Merge  
Michael Kolupaev 已提交
829
			zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
830 831 832 833

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

			ProfileEvents::increment(ProfileEvents::ReplicatedPartMerges);
		}
M
Merge  
Michael Kolupaev 已提交
839 840 841 842 843
	}
	else
	{
		throw Exception("Unexpected log entry type: " + toString(static_cast<int>(entry.type)));
	}
M
Merge  
Michael Kolupaev 已提交
844 845 846

	if (do_fetch)
	{
M
Merge  
Michael Kolupaev 已提交
847 848
		String replica;

M
Merge  
Michael Kolupaev 已提交
849 850
		try
		{
M
Merge  
Michael Kolupaev 已提交
851
			replica = findReplicaHavingPart(entry.new_part_name, true);
M
Merge  
Michael Kolupaev 已提交
852 853 854 855 856
			if (replica.empty())
			{
				ProfileEvents::increment(ProfileEvents::ReplicatedPartFailedFetches);
				throw Exception("No active replica has part " + entry.new_part_name, ErrorCodes::NO_REPLICA_HAS_PART);
			}
857
			fetchPart(entry.new_part_name, zookeeper_path + "/replicas/" + replica);
M
Merge  
Michael Kolupaev 已提交
858 859 860 861 862 863 864 865 866 867 868 869

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

				/// Найдем действие по объединению этого куска с другими. Запомним других.
				StringSet parts_for_merge;
				LogEntries::iterator merge_entry;
				for (LogEntries::iterator it = queue.begin(); it != queue.end(); ++it)
				{
M
Merge  
Michael Kolupaev 已提交
877
					if ((*it)->type == LogEntry::MERGE_PARTS)
M
Merge  
Michael Kolupaev 已提交
878
					{
M
Merge  
Michael Kolupaev 已提交
879 880
						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 已提交
881
						{
M
Merge  
Michael Kolupaev 已提交
882
							parts_for_merge = StringSet((*it)->parts_to_merge.begin(), (*it)->parts_to_merge.end());
M
Merge  
Michael Kolupaev 已提交
883 884 885 886 887 888 889 890 891 892 893 894 895 896 897 898 899
							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 已提交
900 901
						if (((*it0)->type == LogEntry::MERGE_PARTS || (*it0)->type == LogEntry::GET_PART)
							&& parts_for_merge.count((*it0)->new_part_name))
M
Merge  
Michael Kolupaev 已提交
902 903 904 905
						{
							queue.splice(queue.end(), queue, it0, it);
						}
					}
M
Merge  
Michael Kolupaev 已提交
906 907 908 909 910 911 912 913 914

					/** Если этого куска ни у кого нет, но в очереди упоминается мердж с его участием, то наверно этот кусок такой старый,
					  *  что его все померджили и удалили. Не будем бросать исключение, чтобы 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 已提交
915
				}
M
Merge  
Michael Kolupaev 已提交
916 917 918 919

				/// Если ни у кого нет куска, и в очереди нет слияний с его участием, проверим, есть ли у кого-то покрывающий его.
				if (replica.empty())
					enqueuePartForCheck(entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
920 921 922 923 924 925 926 927 928
			}
			catch (...)
			{
				tryLogCurrentException(__PRETTY_FUNCTION__);
			}

			throw;
		}
	}
M
Merge  
Michael Kolupaev 已提交
929 930

	return true;
M
Merge  
Michael Kolupaev 已提交
931 932
}

M
Merge  
Michael Kolupaev 已提交
933
void StorageReplicatedMergeTree::executeDropRange(const StorageReplicatedMergeTree::LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
934
{
M
Merge  
Michael Kolupaev 已提交
935
	LOG_INFO(log, (entry.detach ? "Detaching" : "Removing") << " parts inside " << entry.new_part_name << ".");
M
Merge  
Michael Kolupaev 已提交
936 937 938 939 940 941 942 943 944 945 946 947 948 949 950 951 952 953 954 955 956 957 958 959 960 961 962 963 964

	{
		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 已提交
965 966
		for (LogEntryPtr & entry : to_wait)
			entry->execution_complete.wait(lock, [&entry] { return !entry->currently_executing; });
M
Merge  
Michael Kolupaev 已提交
967 968
	}

M
Merge  
Michael Kolupaev 已提交
969
	LOG_DEBUG(log, (entry.detach ? "Detaching" : "Removing") << " parts.");
M
Merge  
Michael Kolupaev 已提交
970 971 972 973 974 975 976 977 978 979 980
	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 已提交
981 982 983 984
		/// Если кусок удалять не нужно, надежнее переместить директорию до изменений в ZooKeeper.
		if (entry.detach)
			data.renameAndDetachPart(part);

M
Merge  
Michael Kolupaev 已提交
985 986 987 988 989 990
		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 已提交
991
		/// Если кусок нужно удалить, надежнее удалить директорию после изменений в ZooKeeper.
M
Merge  
Michael Kolupaev 已提交
992
		if (!entry.detach)
M
Merge  
Michael Kolupaev 已提交
993
			data.replaceParts({part}, {}, true);
M
Merge  
Michael Kolupaev 已提交
994 995
	}

M
Merge  
Michael Kolupaev 已提交
996
	LOG_INFO(log, (entry.detach ? "Detached " : "Removed ") << removed_parts << " parts inside " << entry.new_part_name << ".");
M
Merge  
Michael Kolupaev 已提交
997 998 999

	if (unreplicated_data)
	{
1000
		std::lock_guard<std::mutex> unreplicated_lock(unreplicated_mutex);
M
Merge  
Michael Kolupaev 已提交
1001

M
Merge  
Michael Kolupaev 已提交
1002 1003 1004 1005 1006 1007 1008 1009 1010 1011 1012 1013 1014 1015
		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);
		}
A
Merge  
Alexey Milovidov 已提交
1016 1017

		LOG_INFO(log, (entry.detach ? "Detached " : "Removed ") << removed_parts << " parts inside " << entry.new_part_name << " (in unreplicated data).");
M
Merge  
Michael Kolupaev 已提交
1018
	}
M
Merge  
Michael Kolupaev 已提交
1019 1020 1021 1022 1023 1024 1025 1026
}

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 已提交
1027
	if (!Poco::File(data.getFullPath() + source_path).exists())
M
Merge  
Michael Kolupaev 已提交
1028
	{
M
Merge  
Michael Kolupaev 已提交
1029
		LOG_INFO(log, "No part at " << source_path << ". Will fetch it instead");
M
Merge  
Michael Kolupaev 已提交
1030 1031 1032 1033 1034 1035 1036 1037 1038 1039 1040 1041 1042 1043 1044 1045 1046 1047 1048 1049 1050 1051 1052 1053 1054 1055 1056 1057
		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 已提交
1058

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

M
Merge  
Michael Kolupaev 已提交
1062 1063 1064
	return true;
}

M
Merge  
Michael Kolupaev 已提交
1065 1066 1067 1068
void StorageReplicatedMergeTree::queueUpdatingThread()
{
	while (!shutdown_called)
	{
M
Merge  
Michael Kolupaev 已提交
1069 1070
		try
		{
M
Merge  
Michael Kolupaev 已提交
1071
			pullLogsToQueue(queue_updating_event);
M
Merge  
Michael Kolupaev 已提交
1072

M
Merge  
Michael Kolupaev 已提交
1073
			queue_updating_event->wait();
M
Merge  
Michael Kolupaev 已提交
1074
		}
1075
		catch (const zkutil::KeeperException & e)
M
Merge  
Michael Kolupaev 已提交
1076 1077 1078 1079 1080 1081 1082 1083
		{
			if (e.code == ZINVALIDSTATE)
				restarting_event.set();

			tryLogCurrentException(__PRETTY_FUNCTION__);

			queue_updating_event->tryWait(ERROR_SLEEP_MS);
		}
M
Merge  
Michael Kolupaev 已提交
1084 1085 1086 1087
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);

M
Merge  
Michael Kolupaev 已提交
1088 1089
			queue_updating_event->tryWait(ERROR_SLEEP_MS);
		}
M
Merge  
Michael Kolupaev 已提交
1090
	}
M
Merge  
Michael Kolupaev 已提交
1091

1092
	LOG_DEBUG(log, "Queue updating thread finished");
M
Merge  
Michael Kolupaev 已提交
1093
}
M
Merge  
Michael Kolupaev 已提交
1094

M
Merge  
Michael Kolupaev 已提交
1095
bool StorageReplicatedMergeTree::queueTask(BackgroundProcessingPool::Context & pool_context)
M
Merge  
Michael Kolupaev 已提交
1096
{
M
Merge  
Michael Kolupaev 已提交
1097
	LogEntryPtr entry;
M
Merge  
Michael Kolupaev 已提交
1098

M
Merge  
Michael Kolupaev 已提交
1099 1100
	try
	{
M
Merge  
Michael Kolupaev 已提交
1101
		std::unique_lock<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1102 1103
		bool empty = queue.empty();
		if (!empty)
M
Merge  
Michael Kolupaev 已提交
1104
		{
M
Merge  
Michael Kolupaev 已提交
1105
			for (LogEntries::iterator it = queue.begin(); it != queue.end(); ++it)
M
Merge  
Michael Kolupaev 已提交
1106
			{
M
Merge  
Michael Kolupaev 已提交
1107
				if (!(*it)->currently_executing && shouldExecuteLogEntry(**it))
M
Merge  
Michael Kolupaev 已提交
1108
				{
M
Merge  
Michael Kolupaev 已提交
1109
					entry = *it;
M
Merge  
Michael Kolupaev 已提交
1110
					entry->tagPartAsFuture(*this);
M
Merge  
Michael Kolupaev 已提交
1111
					queue.splice(queue.end(), queue, it);
M
Merge  
Michael Kolupaev 已提交
1112
					entry->currently_executing = true;
M
Merge  
Michael Kolupaev 已提交
1113
					break;
M
Merge  
Michael Kolupaev 已提交
1114
				}
M
Merge  
Michael Kolupaev 已提交
1115 1116
			}
		}
M
Merge  
Michael Kolupaev 已提交
1117 1118 1119 1120 1121
	}
	catch (...)
	{
		tryLogCurrentException(__PRETTY_FUNCTION__);
	}
M
Merge  
Michael Kolupaev 已提交
1122

M
Merge  
Michael Kolupaev 已提交
1123
	if (!entry)
M
Merge  
Michael Kolupaev 已提交
1124
		return false;
M
Merge  
Michael Kolupaev 已提交
1125

M
Merge  
Michael Kolupaev 已提交
1126
	bool exception = true;
M
Merge  
Michael Kolupaev 已提交
1127
	bool success = false;
M
Merge  
Michael Kolupaev 已提交
1128

M
Merge  
Michael Kolupaev 已提交
1129 1130
	try
	{
M
Merge  
Michael Kolupaev 已提交
1131
		if (executeLogEntry(*entry, pool_context))
M
Merge  
Michael Kolupaev 已提交
1132
		{
M
Merge  
Michael Kolupaev 已提交
1133
			auto code = zookeeper->tryRemove(replica_path + "/queue/" + entry->znode_name);
M
Merge  
Michael Kolupaev 已提交
1134

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

			success = true;
M
Merge  
Michael Kolupaev 已提交
1140
		}
M
Merge  
Michael Kolupaev 已提交
1141

M
Merge  
Michael Kolupaev 已提交
1142
		exception = false;
M
Merge  
Michael Kolupaev 已提交
1143 1144 1145 1146
	}
	catch (Exception & e)
	{
		if (e.code() == ErrorCodes::NO_REPLICA_HAS_PART)
M
Merge  
Michael Kolupaev 已提交
1147
			/// Если ни у кого нет нужного куска, наверно, просто не все реплики работают; не будем писать в лог с уровнем Error.
M
Merge  
Michael Kolupaev 已提交
1148 1149
			LOG_INFO(log, e.displayText());
		else
M
Merge  
Michael Kolupaev 已提交
1150
			tryLogCurrentException(__PRETTY_FUNCTION__);
M
Merge  
Michael Kolupaev 已提交
1151 1152 1153 1154 1155
	}
	catch (...)
	{
		tryLogCurrentException(__PRETTY_FUNCTION__);
	}
M
Merge  
Michael Kolupaev 已提交
1156

M
Merge  
Michael Kolupaev 已提交
1157 1158 1159 1160 1161 1162 1163 1164
	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 已提交
1165
	{
M
Merge  
Michael Kolupaev 已提交
1166 1167 1168
		/// Удалим задание из очереди.
		/// Нельзя просто обратиться по заранее сохраненному итератору, потому что задание мог успеть удалить кто-то другой.
		for (LogEntries::iterator it = queue.end(); it != queue.begin();)
M
Merge  
Michael Kolupaev 已提交
1169
		{
M
Merge  
Michael Kolupaev 已提交
1170 1171 1172
			--it;
			if (*it == entry)
			{
M
Merge  
Michael Kolupaev 已提交
1173
				queue.erase(it);
M
Merge  
Michael Kolupaev 已提交
1174 1175
				break;
			}
M
Merge  
Michael Kolupaev 已提交
1176
		}
M
Merge  
Michael Kolupaev 已提交
1177
	}
M
Merge  
Michael Kolupaev 已提交
1178

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

M
Merge  
Michael Kolupaev 已提交
1183 1184
void StorageReplicatedMergeTree::mergeSelectingThread()
{
M
Merge  
Michael Kolupaev 已提交
1185
	bool need_pull = true;
M
Merge  
Michael Kolupaev 已提交
1186

A
Merge  
Alexey Milovidov 已提交
1187 1188 1189 1190 1191 1192 1193
	/** Может много времени тратиться на определение, можно ли мерджить два рядом стоящих куска.
	  * Два рядом стоящих куска можно мерджить, если все номера блоков между их номерами не используются ("заброшены", abandoned).
	  * Это значит, что между этими кусками не может быть вставлен другой кусок.
	  *
	  * Но если номера соседних блоков отличаются достаточно сильно (обычно, если между ними много "заброшенных" блоков),
	  *  то делается слишком много чтений из ZooKeeper, чтобы узнать, можно ли их мерджить.
	  *
1194 1195
	  * Воспользуемся утверждением, что если пару кусков было можно мерджить, и их мердж ещё не запланирован,
	  *  то и сейчас их можно мерджить, и будем запоминать это состояние, чтобы не делать много раз одинаковые запросы в ZooKeeper.
A
Merge  
Alexey Milovidov 已提交
1196 1197 1198 1199 1200 1201 1202 1203
	  *
	  * 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
	{
1204 1205 1206 1207 1208
		/// Если какой-то из кусков уже собираются слить в больший, не соглашаемся его сливать.
		if (virtual_parts.getContainingPart(left->name) != left->name ||
			virtual_parts.getContainingPart(right->name) != right->name)
			return false;

A
Merge  
Alexey Milovidov 已提交
1209 1210 1211 1212
		auto key = std::make_pair(left->name, right->name);
		if (memoized_parts_that_could_be_merged.count(key))
			return true;

1213 1214 1215 1216 1217 1218 1219
		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 已提交
1220

1221 1222 1223 1224 1225 1226 1227
			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 已提交
1228 1229
	};

M
Merge  
Michael Kolupaev 已提交
1230 1231
	while (!shutdown_called && is_leader_node)
	{
M
Michael Kolupaev 已提交
1232
		bool success = false;
M
Merge  
Michael Kolupaev 已提交
1233

M
Michael Kolupaev 已提交
1234
		try
M
Merge  
Michael Kolupaev 已提交
1235
		{
M
Merge  
Michael Kolupaev 已提交
1236 1237
			std::unique_lock<std::mutex> merge_selecting_lock(merge_selecting_mutex);

M
Merge  
Michael Kolupaev 已提交
1238 1239 1240 1241 1242 1243 1244 1245
			if (need_pull)
			{
				/// Нужно загрузить новую запись в очередь перед тем, как выбирать куски для слияния.
				///  (чтобы кусок добавился в virtual_parts).
				pullLogsToQueue();
				need_pull = false;
			}

1246 1247 1248 1249 1250 1251 1252
			/** Сколько в очереди или в фоновом потоке мерджей крупных кусков.
			  * Если их больше половины от размера пула потоков для мерджа, то можно мерджить только мелкие куски.
			  */
			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 已提交
1253
			size_t merges_queued = 0;
1254
			size_t big_merges_queued = 0;
M
Merge  
Michael Kolupaev 已提交
1255

1256
			if (big_merges_current < max_number_of_big_merges)
M
Michael Kolupaev 已提交
1257
			{
M
Merge  
Michael Kolupaev 已提交
1258
				std::unique_lock<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1259

M
Michael Kolupaev 已提交
1260
				for (const auto & entry : queue)
M
Merge  
Michael Kolupaev 已提交
1261
				{
M
Merge  
Michael Kolupaev 已提交
1262
					if (entry->type == LogEntry::MERGE_PARTS)
M
Merge  
Michael Kolupaev 已提交
1263
					{
M
Michael Kolupaev 已提交
1264
						++merges_queued;
M
Merge  
Michael Kolupaev 已提交
1265

1266
						if (big_merges_current + big_merges_queued < max_number_of_big_merges)
M
Merge  
Michael Kolupaev 已提交
1267
						{
M
Merge  
Michael Kolupaev 已提交
1268
							for (const String & name : entry->parts_to_merge)
M
Merge  
Michael Kolupaev 已提交
1269
							{
M
Merge  
Michael Kolupaev 已提交
1270
								MergeTreeData::DataPartPtr part = data.getActiveContainingPart(name);
M
Merge  
Michael Kolupaev 已提交
1271 1272
								if (!part || part->name != name)
									continue;
1273

M
Merge  
Michael Kolupaev 已提交
1274
								if (part->size_in_bytes > data.settings.max_bytes_to_merge_parts_small)
M
Merge  
Michael Kolupaev 已提交
1275
								{
1276
									++big_merges_queued;
M
Merge  
Michael Kolupaev 已提交
1277 1278 1279 1280 1281 1282
									break;
								}
							}
						}
					}
				}
M
Michael Kolupaev 已提交
1283
			}
M
Merge  
Michael Kolupaev 已提交
1284

1285 1286 1287 1288 1289 1290 1291 1292
			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 已提交
1293
			do
M
Michael Kolupaev 已提交
1294
			{
M
Merge  
Michael Kolupaev 已提交
1295
				if (merges_queued >= data.settings.max_replicated_merges_in_queue)
1296 1297
				{
					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 已提交
1298
					break;
1299
				}
M
Merge  
Michael Kolupaev 已提交
1300

M
Merge  
Michael Kolupaev 已提交
1301
				MergeTreeData::DataPartsVector parts;
M
Merge  
Michael Kolupaev 已提交
1302

M
Merge  
Michael Kolupaev 已提交
1303 1304
				String merged_name;

1305 1306 1307 1308
				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 已提交
1309
					break;
1310
				}
M
Merge  
Michael Kolupaev 已提交
1311

M
Merge  
Michael Kolupaev 已提交
1312 1313 1314 1315 1316 1317 1318 1319 1320 1321 1322 1323 1324 1325
				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 已提交
1326 1327 1328 1329
				LogEntry entry;
				entry.type = LogEntry::MERGE_PARTS;
				entry.source_replica = replica_name;
				entry.new_part_name = merged_name;
M
Merge  
Michael Kolupaev 已提交
1330

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

M
Merge  
Michael Kolupaev 已提交
1334
				need_pull = true;
M
Merge  
Michael Kolupaev 已提交
1335

M
Merge  
Michael Kolupaev 已提交
1336
				zookeeper->create(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential);
M
Merge  
Michael Kolupaev 已提交
1337 1338 1339

				String month_name = parts[0]->name.substr(0, 6);
				for (size_t i = 0; i + 1 < parts.size(); ++i)
M
Merge  
Michael Kolupaev 已提交
1340
				{
M
Merge  
Michael Kolupaev 已提交
1341 1342 1343
					/// Уберем больше не нужные отметки о несуществующих блоках.
					for (UInt64 number = parts[i]->right + 1; number <= parts[i + 1]->left - 1; ++number)
					{
M
Merge  
Michael Kolupaev 已提交
1344 1345
						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 已提交
1346
					}
M
Merge  
Michael Kolupaev 已提交
1347
				}
M
Merge  
Michael Kolupaev 已提交
1348 1349

				success = true;
M
Merge  
Michael Kolupaev 已提交
1350
			}
1351
			while (false);
M
Merge  
Michael Kolupaev 已提交
1352 1353 1354 1355 1356 1357
		}
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);
		}

M
Merge  
Michael Kolupaev 已提交
1358
		if (shutdown_called || !is_leader_node)
M
Merge  
Michael Kolupaev 已提交
1359 1360
			break;

M
Merge  
Michael Kolupaev 已提交
1361
		if (!success)
M
Merge  
Michael Kolupaev 已提交
1362
			merge_selecting_event.tryWait(MERGE_SELECTING_SLEEP_MS);
M
Merge  
Michael Kolupaev 已提交
1363
	}
M
Merge  
Michael Kolupaev 已提交
1364

1365
	LOG_DEBUG(log, "Merge selecting thread finished");
M
Merge  
Michael Kolupaev 已提交
1366 1367
}

M
Merge  
Michael Kolupaev 已提交
1368

M
Merge  
Michael Kolupaev 已提交
1369 1370 1371 1372 1373 1374 1375 1376
void StorageReplicatedMergeTree::alterThread()
{
	bool force_recheck_parts = true;

	while (!shutdown_called)
	{
		try
		{
1377 1378 1379 1380 1381 1382 1383 1384 1385 1386 1387 1388 1389 1390 1391 1392 1393 1394 1395 1396 1397 1398 1399
			/** Имеем описание столбцов в 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 已提交
1400 1401 1402 1403
			zkutil::Stat stat;
			String columns_str = zookeeper->get(zookeeper_path + "/columns", &stat, alter_thread_event);
			NamesAndTypesList columns = NamesAndTypesList::parse(columns_str, context.getDataTypeFactory());

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

M
Merge  
Michael Kolupaev 已提交
1406 1407
			MergeTreeData::DataParts parts;

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

M
Merge  
Michael Kolupaev 已提交
1413 1414 1415
				if (columns != data.getColumnsList())
				{
					LOG_INFO(log, "Columns list changed in ZooKeeper. Applying changes locally.");
1416

M
Merge  
Michael Kolupaev 已提交
1417 1418
					InterpreterAlterQuery::updateMetadata(database_name, table_name, columns, context);
					data.setColumnsList(columns);
M
Merge  
Michael Kolupaev 已提交
1419 1420
					if (unreplicated_data)
						unreplicated_data->setColumnsList(columns);
M
Merge  
Michael Kolupaev 已提交
1421 1422 1423 1424
					LOG_INFO(log, "Applied changes to table.");
				}
				else
				{
1425
					LOG_INFO(log, "Columns version changed in ZooKeeper, but data wasn't changed. It's like cyclic ALTERs.");
M
Merge  
Michael Kolupaev 已提交
1426
				}
1427 1428 1429 1430 1431

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

				columns_version = stat.version;
M
Merge  
Michael Kolupaev 已提交
1432 1433 1434
			}

			/// Обновим куски.
1435
			if (changed_version || force_recheck_parts)
M
Merge  
Michael Kolupaev 已提交
1436
			{
1437 1438 1439
				auto table_lock = lockStructure(false);

				if (changed_version)
M
Merge  
Michael Kolupaev 已提交
1440 1441 1442 1443
					LOG_INFO(log, "ALTER-ing parts");

				int changed_parts = 0;

1444
				if (!changed_version)
M
Merge  
Michael Kolupaev 已提交
1445 1446
					parts = data.getDataParts();

M
Merge  
Michael Kolupaev 已提交
1447 1448 1449
				for (const MergeTreeData::DataPartPtr & part : parts)
				{
					/// Обновим кусок и запишем результат во временные файлы.
M
Merge  
Michael Kolupaev 已提交
1450 1451
					/// TODO: Можно пропускать проверку на слишком большие изменения, если в ZooKeeper есть, например,
					///  нода /flags/force_alter.
M
Merge  
Michael Kolupaev 已提交
1452 1453 1454 1455 1456 1457 1458 1459 1460 1461 1462 1463 1464 1465 1466 1467 1468
					auto transaction = data.alterDataPart(part, columns);

					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 已提交
1469 1470 1471 1472 1473 1474 1475 1476 1477 1478 1479 1480 1481 1482 1483 1484 1485
				/// То же самое для нереплицируемых данных.
				if (unreplicated_data)
				{
					parts = unreplicated_data->getDataParts();

					for (const MergeTreeData::DataPartPtr & part : parts)
					{
						auto transaction = unreplicated_data->alterDataPart(part, columns);

						if (!transaction)
							continue;

						++changed_parts;

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

1487
				/// Список столбцов для конкретной реплики.
M
Merge  
Michael Kolupaev 已提交
1488 1489
				zookeeper->set(replica_path + "/columns", columns.toString());

1490 1491 1492 1493 1494 1495 1496 1497
				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 已提交
1498 1499 1500
				force_recheck_parts = false;
			}

1501
			parts.clear();
M
Merge  
Michael Kolupaev 已提交
1502 1503 1504 1505 1506 1507 1508 1509 1510 1511 1512 1513
			alter_thread_event->wait();
		}
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);

			force_recheck_parts = true;

			alter_thread_event->tryWait(ERROR_SLEEP_MS);
		}
	}

1514
	LOG_DEBUG(log, "Alter thread finished");
M
Merge  
Michael Kolupaev 已提交
1515 1516
}

M
Merge  
Michael Kolupaev 已提交
1517 1518 1519 1520
void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_name)
{
	String part_path = replica_path + "/parts/" + part_name;

M
Merge  
Michael Kolupaev 已提交
1521 1522 1523 1524
	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 已提交
1525 1526 1527

	zkutil::Ops ops;
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
1528
		replica_path + "/queue/queue-", log_entry->toString(), zookeeper->getDefaultACL(),
M
Merge  
Michael Kolupaev 已提交
1529 1530 1531 1532 1533 1534 1535
		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 已提交
1536
		std::unique_lock<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1537 1538

		String path_created = dynamic_cast<zkutil::Op::Create &>(ops[0]).getPathCreated();
M
Merge  
Michael Kolupaev 已提交
1539 1540
		log_entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
		log_entry->addResultToVirtualParts(*this);
M
Merge  
Michael Kolupaev 已提交
1541 1542 1543 1544 1545 1546
		queue.push_back(log_entry);
	}
}

void StorageReplicatedMergeTree::enqueuePartForCheck(const String & name)
{
1547
	std::lock_guard<std::mutex> lock(parts_to_check_mutex);
M
Merge  
Michael Kolupaev 已提交
1548 1549 1550 1551 1552 1553 1554 1555

	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 已提交
1556 1557 1558 1559 1560 1561 1562 1563 1564
void StorageReplicatedMergeTree::partCheckThread()
{
	while (!shutdown_called)
	{
		try
		{
			/// Достанем из очереди кусок для проверки.
			String part_name;
			{
1565
				std::lock_guard<std::mutex> lock(parts_to_check_mutex);
M
Merge  
Michael Kolupaev 已提交
1566 1567 1568 1569 1570 1571 1572 1573 1574 1575 1576 1577 1578
				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();
				}
			}
1579
			if (part_name.empty())	/// TODO Здесь race condition?
M
Merge  
Michael Kolupaev 已提交
1580 1581 1582 1583 1584 1585
			{
				parts_to_check_event.wait();
				continue;
			}

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

M
Merge  
Michael Kolupaev 已提交
1588
			auto part = data.getActiveContainingPart(part_name);
M
Merge  
Michael Kolupaev 已提交
1589 1590 1591 1592 1593 1594 1595 1596 1597 1598
			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 已提交
1599
					ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
M
Merge  
Michael Kolupaev 已提交
1600

M
Merge  
Michael Kolupaev 已提交
1601
					removePartAndEnqueueFetch(part_name);
M
Merge  
Michael Kolupaev 已提交
1602 1603 1604 1605
				}
				/// Если куска нет в ZooKeeper, проверим есть ли он хоть у кого-то.
				else
				{
M
Merge  
Michael Kolupaev 已提交
1606 1607
					ActiveDataPartSet::Part part_info;
					ActiveDataPartSet::parsePartName(part_name, part_info);
M
Merge  
Michael Kolupaev 已提交
1608

M
Merge  
Michael Kolupaev 已提交
1609
					/** Будем проверять только куски, не полученные в результате слияния.
M
Merge  
Michael Kolupaev 已提交
1610
					  * Для кусков, полученных в результате слияния, такая проверка была бы некорректной,
M
Merge  
Michael Kolupaev 已提交
1611 1612
					  *  потому что слитого куска может еще ни у кого не быть.
					  */
M
Merge  
Michael Kolupaev 已提交
1613
					if (part_info.left == part_info.right)
M
Merge  
Michael Kolupaev 已提交
1614
					{
M
Merge  
Michael Kolupaev 已提交
1615 1616 1617 1618 1619
						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 已提交
1620
						{
M
Merge  
Michael Kolupaev 已提交
1621 1622
							Strings parts = zookeeper->getChildren(zookeeper_path + "/replicas/" + replica + "/parts");
							for (const String & part_on_replica : parts)
M
Merge  
Michael Kolupaev 已提交
1623
							{
M
Merge  
Michael Kolupaev 已提交
1624 1625 1626 1627 1628 1629
								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 已提交
1630
							}
M
Merge  
Michael Kolupaev 已提交
1631 1632
							if (found)
								break;
M
Merge  
Michael Kolupaev 已提交
1633 1634
						}

M
Merge  
Michael Kolupaev 已提交
1635
						if (!found)
M
Merge  
Michael Kolupaev 已提交
1636
						{
M
Merge  
Michael Kolupaev 已提交
1637
							LOG_ERROR(log, "No replica has part covering " << part_name);
M
Merge  
Michael Kolupaev 已提交
1638 1639
							ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);

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

M
Merge  
Michael Kolupaev 已提交
1642 1643
							bool was_in_queue = false;

M
Merge  
Michael Kolupaev 已提交
1644
							{
M
Merge  
Michael Kolupaev 已提交
1645
								std::unique_lock<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1646 1647

								for (LogEntries::iterator it = queue.begin(); it != queue.end(); )
M
Merge  
Michael Kolupaev 已提交
1648
								{
M
Merge  
Michael Kolupaev 已提交
1649
									if ((*it)->new_part_name == part_name)
M
Merge  
Michael Kolupaev 已提交
1650
									{
M
Merge  
Michael Kolupaev 已提交
1651
										zookeeper->tryRemove(replica_path + "/queue/" + (*it)->znode_name);
M
Merge  
Michael Kolupaev 已提交
1652
										queue.erase(it++);
M
Merge  
Michael Kolupaev 已提交
1653
										was_in_queue = true;
M
Merge  
Michael Kolupaev 已提交
1654 1655 1656 1657 1658
									}
									else
									{
										++it;
									}
M
Merge  
Michael Kolupaev 已提交
1659 1660
								}
							}
M
Merge  
Michael Kolupaev 已提交
1661 1662

							if (was_in_queue)
M
Merge  
Michael Kolupaev 已提交
1663
							{
M
Merge  
Michael Kolupaev 已提交
1664 1665 1666 1667
								/** Такая ситуация возможна, если на всех репликах, где был кусок, он испортился.
								  * Например, у реплики, которая только что его записала, отключили питание, и данные не записались из кеша на диск.
								  */
								LOG_ERROR(log, "Part " << part_name << " is lost forever. Say goodbye to a piece of data!");
M
Merge  
Michael Kolupaev 已提交
1668 1669 1670 1671 1672 1673 1674 1675 1676 1677 1678 1679 1680

								/** Нужно добавить отсутствующий кусок в 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 已提交
1681 1682
						}
					}
M
Merge  
Michael Kolupaev 已提交
1683 1684 1685 1686 1687
				}
			}
			/// У нас есть этот кусок, и он активен.
			else if (part->name == part_name)
			{
M
Merge  
Michael Kolupaev 已提交
1688 1689
				auto table_lock = lockStructure(false);

M
Merge  
Michael Kolupaev 已提交
1690 1691 1692
				/// Если кусок есть в ZooKeeper, сверим его данные с его чексуммами, а их с ZooKeeper.
				if (zookeeper->exists(replica_path + "/parts/" + part_name))
				{
M
Merge  
Michael Kolupaev 已提交
1693 1694 1695 1696 1697 1698 1699 1700 1701 1702 1703 1704 1705
					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 已提交
1706 1707 1708 1709
						MergeTreePartChecker::Settings settings;
						settings.setIndexGranularity(data.index_granularity);
						settings.setRequireChecksums(true);
						settings.setRequireColumnFiles(true);
M
Merge  
Michael Kolupaev 已提交
1710
						MergeTreePartChecker::checkDataPart(
M
Merge  
Michael Kolupaev 已提交
1711
							data.getFullPath() + part_name, settings, context.getDataTypeFactory());
M
Merge  
Michael Kolupaev 已提交
1712

M
Merge  
Michael Kolupaev 已提交
1713 1714 1715 1716 1717 1718
						LOG_INFO(log, "Part " << part_name << " looks good.");
					}
					catch (...)
					{
						tryLogCurrentException(__PRETTY_FUNCTION__);

M
Merge  
Michael Kolupaev 已提交
1719 1720
						LOG_ERROR(log, "Part " << part_name << " looks broken. Removing it and queueing a fetch.");
						ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
M
Merge  
Michael Kolupaev 已提交
1721 1722 1723 1724

						removePartAndEnqueueFetch(part_name);

						/// Удалим кусок локально.
M
Merge  
Michael Kolupaev 已提交
1725
						data.renameAndDetachPart(part, "broken_");
M
Merge  
Michael Kolupaev 已提交
1726
					}
M
Merge  
Michael Kolupaev 已提交
1727 1728
				}
				/// Если куска нет в ZooKeeper, удалим его локально.
M
Merge  
Michael Kolupaev 已提交
1729 1730 1731
				/// Возможно, кусок кто-то только что записал, и еще не успел добавить в ZK.
				/// Поэтому удаляем только если кусок старый (не очень надежно).
				else if (part->modification_time + 5 * 60 < time(0))
M
Merge  
Michael Kolupaev 已提交
1732
				{
M
Merge  
Michael Kolupaev 已提交
1733 1734
					ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);

M
Merge  
Michael Kolupaev 已提交
1735 1736
					LOG_ERROR(log, "Unexpected part " << part_name << ". Removing.");
					data.renameAndDetachPart(part, "unexpected_");
M
Merge  
Michael Kolupaev 已提交
1737 1738 1739 1740 1741 1742 1743 1744 1745 1746
				}
			}
			else
			{
				/// Если у нас есть покрывающий кусок, игнорируем все проблемы с этим куском.
				/// В худшем случае в лог еще old_parts_lifetime секунд будут валиться ошибки, пока кусок не удалится как старый.
			}

			/// Удалим кусок из очереди.
			{
1747
				std::lock_guard<std::mutex> lock(parts_to_check_mutex);
M
Merge  
Michael Kolupaev 已提交
1748 1749 1750 1751 1752 1753 1754 1755 1756 1757 1758 1759 1760 1761 1762 1763 1764 1765 1766 1767
				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 已提交
1768 1769
void StorageReplicatedMergeTree::becomeLeader()
{
M
Merge  
Michael Kolupaev 已提交
1770
	LOG_INFO(log, "Became leader");
M
Merge  
Michael Kolupaev 已提交
1771 1772 1773 1774
	is_leader_node = true;
	merge_selecting_thread = std::thread(&StorageReplicatedMergeTree::mergeSelectingThread, this);
}

M
Merge  
Michael Kolupaev 已提交
1775
String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_name, bool active)
M
Merge  
Michael Kolupaev 已提交
1776
{
M
Merge  
Michael Kolupaev 已提交
1777
	Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
M
Merge  
Michael Kolupaev 已提交
1778 1779 1780 1781 1782 1783

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

	for (const String & replica : replicas)
	{
M
Merge  
Michael Kolupaev 已提交
1784 1785
		if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name) &&
			(!active || zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")))
M
Merge  
Michael Kolupaev 已提交
1786 1787 1788
			return replica;
	}

M
Merge  
Michael Kolupaev 已提交
1789
	return "";
M
Merge  
Michael Kolupaev 已提交
1790 1791
}

1792
void StorageReplicatedMergeTree::fetchPart(const String & part_name, const String & replica_path, bool to_detached)
M
Merge  
Michael Kolupaev 已提交
1793
{
1794
	LOG_DEBUG(log, "Fetching part " << part_name << " from " << replica_path);
M
Merge  
Michael Kolupaev 已提交
1795

1796 1797 1798
	TableStructureReadLockPtr table_lock;
	if (!to_detached)
		table_lock = lockStructure(true);
M
Merge  
Michael Kolupaev 已提交
1799 1800 1801 1802

	String host;
	int port;

1803
	String host_port_str = zookeeper->get(replica_path + "/host");
M
Merge  
Michael Kolupaev 已提交
1804 1805 1806 1807 1808 1809 1810 1811
	ReadBufferFromString buf(host_port_str);
	assertString("host: ", buf);
	readString(host, buf);
	assertString("\nport: ", buf);
	readText(port, buf);
	assertString("\n", buf);
	assertEOF(buf);

1812
	MergeTreeData::MutableDataPartPtr part = fetcher.fetchPart(part_name, replica_path, host, port, to_detached);
M
Merge  
Michael Kolupaev 已提交
1813

1814 1815 1816 1817
	if (!to_detached)
	{
		zkutil::Ops ops;
		checkPartAndAddToZooKeeper(part, ops, part_name);
M
Merge  
Michael Kolupaev 已提交
1818

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

1822 1823 1824
		zookeeper->multi(ops);
		transaction.commit();
		merge_selecting_event.set();
M
Merge  
Michael Kolupaev 已提交
1825

1826 1827 1828 1829 1830 1831 1832
		for (const auto & removed_part : removed_parts)
		{
			LOG_DEBUG(log, "Part " << removed_part->name << " is rendered obsolete by fetching part " << part_name);
			ProfileEvents::increment(ProfileEvents::ObsoleteReplicatedParts);
		}
	}
	else
M
Michael Kolupaev 已提交
1833
	{
1834
		Poco::File(data.getFullPath() + "detached/tmp_" + part_name).renameTo(data.getFullPath() + "detached/" + part_name);
M
Michael Kolupaev 已提交
1835 1836
	}

M
Merge  
Michael Kolupaev 已提交
1837 1838
	ProfileEvents::increment(ProfileEvents::ReplicatedPartFetches);

1839
	LOG_DEBUG(log, "Fetched part " << part_name << " from " << replica_name << (to_detached ? " (to 'detached' directory)" : ""));
M
Merge  
Michael Kolupaev 已提交
1840
}
M
Merge  
Michael Kolupaev 已提交
1841

M
Merge  
Michael Kolupaev 已提交
1842 1843
void StorageReplicatedMergeTree::shutdown()
{
M
Merge  
Michael Kolupaev 已提交
1844
	if (permanent_shutdown_called)
M
Merge  
Michael Kolupaev 已提交
1845 1846 1847
	{
		if (restarting_thread.joinable())
			restarting_thread.join();
M
Merge  
Michael Kolupaev 已提交
1848
		return;
M
Merge  
Michael Kolupaev 已提交
1849
	}
M
Merge  
Michael Kolupaev 已提交
1850

M
Merge  
Michael Kolupaev 已提交
1851
	permanent_shutdown_called = true;
M
Merge  
Michael Kolupaev 已提交
1852
	restarting_event.set();
M
Merge  
Michael Kolupaev 已提交
1853
	restarting_thread.join();
M
Merge  
Michael Kolupaev 已提交
1854 1855

	endpoint_holder = nullptr;
M
Merge  
Michael Kolupaev 已提交
1856 1857 1858 1859
}

void StorageReplicatedMergeTree::partialShutdown()
{
M
Merge  
Michael Kolupaev 已提交
1860
	leader_election = nullptr;
M
Merge  
Michael Kolupaev 已提交
1861
	shutdown_called = true;
M
Merge  
Michael Kolupaev 已提交
1862
	shutdown_event.set();
M
Merge  
Michael Kolupaev 已提交
1863 1864 1865
	merge_selecting_event.set();
	queue_updating_event->set();
	alter_thread_event->set();
M
Merge  
Michael Kolupaev 已提交
1866
	alter_query_event->set();
M
Merge  
Michael Kolupaev 已提交
1867
	parts_to_check_event.set();
M
Merge  
Michael Kolupaev 已提交
1868 1869
	replica_is_active_node = nullptr;

M
Merge  
Michael Kolupaev 已提交
1870 1871 1872 1873
	merger.cancelAll();
	if (unreplicated_merger)
		unreplicated_merger->cancelAll();

M
Merge  
Michael Kolupaev 已提交
1874
	LOG_TRACE(log, "Waiting for threads to finish");
M
Merge  
Michael Kolupaev 已提交
1875
	if (is_leader_node)
M
Merge  
Michael Kolupaev 已提交
1876
	{
M
Merge  
Michael Kolupaev 已提交
1877
		is_leader_node = false;
M
Merge  
Michael Kolupaev 已提交
1878 1879
		if (merge_selecting_thread.joinable())
			merge_selecting_thread.join();
M
Merge  
Michael Kolupaev 已提交
1880
	}
M
Merge  
Michael Kolupaev 已提交
1881 1882
	if (queue_updating_thread.joinable())
		queue_updating_thread.join();
1883 1884 1885

	cleanup_thread.reset();

M
Merge  
Michael Kolupaev 已提交
1886 1887
	if (alter_thread.joinable())
		alter_thread.join();
M
Merge  
Michael Kolupaev 已提交
1888 1889
	if (part_check_thread.joinable())
		part_check_thread.join();
M
Merge  
Michael Kolupaev 已提交
1890 1891
	if (queue_task_handle)
		context.getBackgroundPool().removeTask(queue_task_handle);
M
Merge  
Michael Kolupaev 已提交
1892
	queue_task_handle.reset();
M
Merge  
Michael Kolupaev 已提交
1893
	LOG_TRACE(log, "Threads finished");
M
Merge  
Michael Kolupaev 已提交
1894 1895
}

M
Merge  
Michael Kolupaev 已提交
1896
void StorageReplicatedMergeTree::goReadOnlyPermanently()
M
Merge  
Michael Kolupaev 已提交
1897 1898 1899 1900 1901
{
	LOG_INFO(log, "Going to read-only mode");

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

M
Merge  
Michael Kolupaev 已提交
1904
	partialShutdown();
M
Merge  
Michael Kolupaev 已提交
1905 1906
}

M
Merge  
Michael Kolupaev 已提交
1907
bool StorageReplicatedMergeTree::tryStartup()
M
Merge  
Michael Kolupaev 已提交
1908
{
M
Merge  
Michael Kolupaev 已提交
1909 1910 1911
	try
	{
		activateReplica();
M
Merge  
Michael Kolupaev 已提交
1912

M
Merge  
Michael Kolupaev 已提交
1913 1914 1915 1916 1917
		leader_election = new zkutil::LeaderElection(zookeeper_path + "/leader_election", *zookeeper,
			std::bind(&StorageReplicatedMergeTree::becomeLeader, this), replica_name);

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

M
Merge  
Michael Kolupaev 已提交
1919 1920
		shutdown_called = false;
		shutdown_event.reset();
M
Merge  
Michael Kolupaev 已提交
1921

M
Merge  
Michael Kolupaev 已提交
1922 1923 1924
		merger.uncancelAll();
		if (unreplicated_merger)
			unreplicated_merger->uncancelAll();
M
Merge  
Michael Kolupaev 已提交
1925

M
Merge  
Michael Kolupaev 已提交
1926
		queue_updating_thread = std::thread(&StorageReplicatedMergeTree::queueUpdatingThread, this);
1927
		cleanup_thread.reset(new ReplicatedMergeTreeCleanupThread(*this));
M
Merge  
Michael Kolupaev 已提交
1928 1929 1930 1931
		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 已提交
1932
		queue_task_handle->wake();
M
Merge  
Michael Kolupaev 已提交
1933 1934
		return true;
	}
1935
	catch (const zkutil::KeeperException & e)
M
Merge  
Michael Kolupaev 已提交
1936 1937 1938 1939 1940 1941 1942
	{
		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;
	}
1943 1944 1945 1946 1947 1948 1949 1950 1951 1952 1953
	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 已提交
1954 1955 1956 1957 1958 1959
	catch (...)
	{
		replica_is_active_node = nullptr;
		leader_election = nullptr;
		throw;
	}
M
Merge  
Michael Kolupaev 已提交
1960 1961
}

M
Merge  
Michael Kolupaev 已提交
1962 1963
void StorageReplicatedMergeTree::restartingThread()
{
M
Merge  
Michael Kolupaev 已提交
1964
	try
M
Merge  
Michael Kolupaev 已提交
1965
	{
1966
		/// Запуск реплики при старте сервера/создании таблицы.
M
Merge  
Michael Kolupaev 已提交
1967 1968
		while (!permanent_shutdown_called && !tryStartup())
			restarting_event.tryWait(10 * 1000);
M
Merge  
Michael Kolupaev 已提交
1969

1970
		/// Цикл перезапуска реплики при истечении сессии с ZK.
M
Merge  
Michael Kolupaev 已提交
1971
		while (!permanent_shutdown_called)
M
Merge  
Michael Kolupaev 已提交
1972
		{
M
Merge  
Michael Kolupaev 已提交
1973 1974 1975
			if (zookeeper->expired())
			{
				LOG_WARNING(log, "ZooKeeper session has expired. Switching to a new session.");
M
Merge  
Michael Kolupaev 已提交
1976

1977 1978 1979
				partialShutdown();
				zookeeper = context.getZooKeeper();
				is_read_only = true;
M
Merge  
Michael Kolupaev 已提交
1980 1981 1982 1983 1984 1985

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

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

1987
				is_read_only = false;
M
Merge  
Michael Kolupaev 已提交
1988
			}
M
Merge  
Michael Kolupaev 已提交
1989

M
Merge  
Michael Kolupaev 已提交
1990
			restarting_event.tryWait(60 * 1000);
M
Merge  
Michael Kolupaev 已提交
1991 1992 1993 1994 1995
		}
	}
	catch (...)
	{
		tryLogCurrentException("StorageReplicatedMergeTree::restartingThread");
M
Merge  
Michael Kolupaev 已提交
1996 1997
		LOG_ERROR(log, "Unexpected exception in restartingThread. The storage will be read-only until server restart.");
		goReadOnlyPermanently();
1998
		LOG_DEBUG(log, "Restarting thread finished");
M
Merge  
Michael Kolupaev 已提交
1999
		return;
M
Merge  
Michael Kolupaev 已提交
2000 2001
	}

M
Merge  
Michael Kolupaev 已提交
2002 2003 2004 2005 2006 2007 2008 2009 2010
	try
	{
		endpoint_holder = nullptr;
		partialShutdown();
	}
	catch (...)
	{
		tryLogCurrentException("StorageReplicatedMergeTree::restartingThread");
	}
M
Merge  
Michael Kolupaev 已提交
2011

2012
	LOG_DEBUG(log, "Restarting thread finished");
M
Merge  
Michael Kolupaev 已提交
2013 2014
}

M
Merge  
Michael Kolupaev 已提交
2015 2016 2017 2018 2019 2020 2021 2022 2023 2024 2025 2026 2027 2028 2029 2030 2031 2032 2033 2034
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 已提交
2035 2036 2037 2038 2039 2040 2041 2042 2043 2044 2045 2046 2047 2048 2049 2050 2051
	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())
2052
		VirtualColumnUtils::filterBlockWithQuery(query, virtual_columns_block, context);
M
Merge  
Michael Kolupaev 已提交
2053

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

M
Merge  
Michael Kolupaev 已提交
2056 2057
	BlockInputStreams res;

M
Merge  
Michael Kolupaev 已提交
2058 2059 2060
	size_t part_index = 0;

	if (unreplicated_reader && values.count(0))
M
Merge  
Michael Kolupaev 已提交
2061
	{
M
Merge  
Michael Kolupaev 已提交
2062 2063
		res = unreplicated_reader->read(
			real_column_names, query, settings, processed_stage, max_block_size, threads, &part_index);
M
Merge  
Michael Kolupaev 已提交
2064 2065 2066 2067 2068 2069

		for (auto & virtual_column : virt_column_names)
		{
			if (virtual_column == "_replicated")
			{
				for (auto & stream : res)
M
Merge  
Michael Kolupaev 已提交
2070
					stream = new AddingConstColumnBlockInputStream<UInt8>(stream, new DataTypeUInt8, 0, "_replicated");
M
Merge  
Michael Kolupaev 已提交
2071 2072 2073 2074
			}
		}
	}

M
Merge  
Michael Kolupaev 已提交
2075
	if (values.count(1))
M
Merge  
Michael Kolupaev 已提交
2076
	{
M
Merge  
Michael Kolupaev 已提交
2077
		auto res2 = reader.read(real_column_names, query, settings, processed_stage, max_block_size, threads, &part_index);
M
Merge  
Michael Kolupaev 已提交
2078 2079 2080 2081 2082 2083

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

M
Merge  
Michael Kolupaev 已提交
2088
		res.insert(res.end(), res2.begin(), res2.end());
M
Merge  
Michael Kolupaev 已提交
2089 2090 2091
	}

	return res;
M
Merge  
Michael Kolupaev 已提交
2092 2093
}

M
Merge  
Michael Kolupaev 已提交
2094 2095
BlockOutputStreamPtr StorageReplicatedMergeTree::write(ASTPtr query)
{
M
Merge  
Michael Kolupaev 已提交
2096 2097 2098
	if (is_read_only)
		throw Exception("Table is in read only mode", ErrorCodes::TABLE_IS_READ_ONLY);

M
Merge  
Michael Kolupaev 已提交
2099
	String insert_id;
M
Merge  
Michael Kolupaev 已提交
2100 2101 2102
	if (query)
		if (ASTInsertQuery * insert = typeid_cast<ASTInsertQuery *>(&*query))
			insert_id = insert->insert_id;
M
Merge  
Michael Kolupaev 已提交
2103 2104 2105

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

M
Merge  
Michael Kolupaev 已提交
2107 2108
bool StorageReplicatedMergeTree::optimize()
{
2109 2110
	/// Померджим какие-нибудь куски из директории unreplicated.
	/// TODO: Мерджить реплицируемые куски тоже.
M
Merge  
Michael Kolupaev 已提交
2111 2112 2113 2114

	if (!unreplicated_data)
		return false;

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

M
Merge  
Michael Kolupaev 已提交
2117 2118 2119 2120 2121 2122 2123 2124
	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;

2125 2126
	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 已提交
2127 2128 2129
	return true;
}

M
Merge  
Michael Kolupaev 已提交
2130 2131
void StorageReplicatedMergeTree::alter(const AlterCommands & params,
	const String & database_name, const String & table_name, Context & context)
M
Merge  
Michael Kolupaev 已提交
2132 2133 2134 2135 2136 2137 2138 2139 2140 2141 2142
{
	LOG_DEBUG(log, "Doing ALTER");

	NamesAndTypesList new_columns;
	String new_columns_str;
	int new_columns_version;
	zkutil::Stat stat;

	{
		auto table_lock = lockStructureForAlter();

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

M
Merge  
Michael Kolupaev 已提交
2146 2147 2148 2149 2150 2151 2152 2153 2154 2155 2156 2157 2158 2159 2160 2161 2162 2163 2164 2165
		data.checkAlter(params);

		new_columns = data.getColumnsList();
		params.apply(new_columns);

		new_columns_str = new_columns.toString();

		/// Делаем ALTER.
		zookeeper->set(zookeeper_path + "/columns", new_columns_str, -1, &stat);

		new_columns_version = stat.version;
	}

	LOG_DEBUG(log, "Updated columns in ZooKeeper. Waiting for replicas to apply changes.");

	/// Ждем, пока все реплики обновят данные.

	/// Подпишемся на изменения столбцов, чтобы перестать ждать, если кто-то еще сделает ALTER.
	if (!zookeeper->exists(zookeeper_path + "/columns", &stat, alter_query_event))
		throw Exception(zookeeper_path + "/columns doesn't exist", ErrorCodes::NOT_FOUND_NODE);
2166

M
Merge  
Michael Kolupaev 已提交
2167 2168 2169 2170 2171 2172 2173 2174 2175 2176 2177 2178 2179 2180 2181 2182 2183 2184 2185 2186 2187 2188 2189 2190 2191 2192 2193 2194 2195 2196
	if (stat.version != new_columns_version)
	{
		LOG_WARNING(log, zookeeper_path + "/columns changed before this ALTER finished; "
			"overlapping ALTER-s are fine but use caution with nontransitive changes");
		return;
	}

	Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
	for (const String & replica : replicas)
	{
		LOG_DEBUG(log, "Waiting for " << replica << " to apply changes");

		while (!shutdown_called)
		{
			String replica_columns_str;

			/// Реплику могли успеть удалить.
			if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/columns", replica_columns_str, &stat))
			{
				LOG_WARNING(log, replica << " was removed");
				break;
			}

			int replica_columns_version = stat.version;

			if (replica_columns_str == new_columns_str)
				break;

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

M
Merge  
Michael Kolupaev 已提交
2198 2199 2200 2201 2202 2203 2204
			if (stat.version != new_columns_version)
			{
				LOG_WARNING(log, zookeeper_path + "/columns changed before ALTER finished; "
					"overlapping ALTER-s are fine but use caution with nontransitive changes");
				return;
			}

M
Merge  
Michael Kolupaev 已提交
2205
			if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/columns", &stat, alter_query_event))
M
Merge  
Michael Kolupaev 已提交
2206 2207 2208 2209 2210 2211 2212 2213 2214 2215 2216 2217 2218 2219 2220 2221 2222 2223
			{
				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 已提交
2224 2225 2226 2227 2228 2229 2230 2231 2232 2233 2234 2235 2236 2237 2238 2239

/// Название воображаемого куска, покрывающего все возможные куски в указанном месяце с номерами в указанном диапазоне.
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)
{
A
Merge  
Alexey Milovidov 已提交
2240
	String month_name = MergeTreeData::getMonthName(field);
M
Merge  
Michael Kolupaev 已提交
2241 2242 2243 2244 2245 2246 2247 2248 2249 2250 2251 2252 2253 2254 2255 2256 2257 2258 2259 2260 2261 2262 2263 2264 2265 2266 2267 2268 2269 2270 2271 2272 2273 2274 2275 2276

	/// 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 已提交
2277
	/// Наконец, добившись нужных инвариантов, можно положить запись в лог.
M
Merge  
Michael Kolupaev 已提交
2278 2279 2280 2281 2282 2283
	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 已提交
2284
	entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);
M
Merge  
Michael Kolupaev 已提交
2285 2286

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

M
Merge  
Michael Kolupaev 已提交
2290
void StorageReplicatedMergeTree::attachPartition(const Field & field, bool unreplicated, bool attach_part)
M
Merge  
Michael Kolupaev 已提交
2291
{
A
Merge  
Alexey Milovidov 已提交
2292
	String partition;
M
Merge  
Michael Kolupaev 已提交
2293

A
Merge  
Alexey Milovidov 已提交
2294 2295 2296 2297
	if (attach_part)
		partition = field.getType() == Field::Types::UInt64 ? toString(field.get<UInt64>()) : field.safeGet<String>();
	else
		partition = MergeTreeData::getMonthName(field);
M
Merge  
Michael Kolupaev 已提交
2298 2299 2300 2301 2302 2303 2304 2305 2306 2307 2308 2309 2310 2311 2312 2313 2314 2315 2316 2317 2318 2319 2320 2321 2322 2323 2324 2325 2326 2327 2328 2329 2330 2331 2332 2333 2334 2335 2336 2337 2338 2339 2340 2341 2342 2343 2344 2345 2346 2347 2348 2349 2350 2351 2352 2353 2354 2355 2356 2357 2358 2359 2360 2361 2362 2363 2364 2365 2366 2367 2368 2369 2370 2371 2372 2373 2374 2375 2376 2377 2378 2379 2380 2381 2382

	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 已提交
2383 2384
}

M
Merge  
Michael Kolupaev 已提交
2385 2386
void StorageReplicatedMergeTree::drop()
{
M
Merge  
Michael Kolupaev 已提交
2387 2388
	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 已提交
2389

M
Merge  
Michael Kolupaev 已提交
2390 2391
	shutdown();

M
Merge  
Michael Kolupaev 已提交
2392
	LOG_INFO(log, "Removing replica " << replica_path);
M
Merge  
Michael Kolupaev 已提交
2393
	replica_is_active_node = nullptr;
M
Merge  
Michael Kolupaev 已提交
2394 2395
	zookeeper->tryRemoveRecursive(replica_path);

M
Merge  
Michael Kolupaev 已提交
2396
	/// Проверяем, что zookeeper_path существует: его могла удалить другая реплика после выполнения предыдущей строки.
M
Merge  
Michael Kolupaev 已提交
2397 2398
	Strings replicas;
	if (zookeeper->tryGetChildren(zookeeper_path + "/replicas", replicas) == ZOK && replicas.empty())
M
Merge  
Michael Kolupaev 已提交
2399 2400
	{
		LOG_INFO(log, "Removing table " << zookeeper_path << " (this might take several minutes)");
M
Merge  
Michael Kolupaev 已提交
2401
		zookeeper->tryRemoveRecursive(zookeeper_path);
M
Merge  
Michael Kolupaev 已提交
2402
	}
M
Merge  
Michael Kolupaev 已提交
2403 2404

	data.dropAllData();
M
Merge  
Michael Kolupaev 已提交
2405 2406
}

M
Merge  
Michael Kolupaev 已提交
2407 2408 2409 2410 2411 2412 2413 2414 2415 2416 2417 2418 2419 2420 2421
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 已提交
2422 2423 2424 2425 2426 2427 2428 2429 2430 2431
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 已提交
2432
		for (size_t i = 0; i < RESERVED_BLOCK_NUMBERS; ++i)
M
Merge  
Michael Kolupaev 已提交
2433 2434 2435 2436 2437 2438 2439 2440 2441 2442 2443 2444 2445
		{
			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 已提交
2446
void StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry(const LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
2447
{
M
Merge  
Michael Kolupaev 已提交
2448 2449
	LOG_DEBUG(log, "Waiting for all replicas to process " << entry.znode_name);

M
Merge  
Michael Kolupaev 已提交
2450
	UInt64 log_index = parse<UInt64>(entry.znode_name.substr(entry.znode_name.size() - 10));
M
Merge  
Michael Kolupaev 已提交
2451 2452 2453 2454 2455
	String log_entry_str = entry.toString();

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

M
Merge  
Michael Kolupaev 已提交
2458 2459 2460 2461 2462 2463 2464 2465 2466 2467 2468 2469
		/// Дождемся, пока запись попадет в очередь реплики.
		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 已提交
2470 2471
		LOG_DEBUG(log, "Looking for " << entry.znode_name << " in " << replica << " queue");

M
Merge  
Michael Kolupaev 已提交
2472 2473 2474 2475 2476 2477 2478
		/// Найдем запись в очереди реплики.
		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 已提交
2479 2480
			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 已提交
2481 2482 2483 2484 2485 2486 2487 2488 2489 2490
			{
				entry_to_wait_for = entry_name;
				break;
			}
		}

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

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

M
Merge  
Michael Kolupaev 已提交
2493 2494 2495 2496 2497 2498 2499
		/// Дождемся, пока запись исчезнет из очереди реплики.
		while (true)
		{
			zkutil::EventPtr event = new Poco::Event;

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

			event->wait();
		}
	}
M
Merge  
Michael Kolupaev 已提交
2506 2507

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


2511
void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields)
2512 2513 2514 2515 2516 2517 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
{
	res.is_leader = is_leader_node;
	res.is_readonly = is_read_only;
	res.is_session_expired = zookeeper->expired();

	{
		std::lock_guard<std::mutex> lock(queue_mutex);
		res.future_parts = future_parts.size();
		res.queue_size = queue.size();

		res.inserts_in_queue = 0;
		res.merges_in_queue = 0;

		for (const LogEntryPtr & entry : queue)
		{
			if (entry->type == LogEntry::GET_PART)
				++res.inserts_in_queue;
			if (entry->type == LogEntry::MERGE_PARTS)
				++res.merges_in_queue;
		}
	}

	{
		std::lock_guard<std::mutex> lock(parts_to_check_mutex);
		res.parts_to_check = parts_to_check_set.size();
	}

	res.zookeeper_path = zookeeper_path;
	res.replica_name = replica_name;
	res.replica_path = replica_path;
	res.columns_version = columns_version;

2544
	if (res.is_session_expired || !with_zk_fields)
2545 2546 2547 2548 2549 2550 2551 2552 2553 2554
	{
		res.log_max_index = 0;
		res.log_pointer = 0;
		res.total_replicas = 0;
		res.active_replicas = 0;
	}
	else
	{
		auto log_entries = zookeeper->getChildren(zookeeper_path + "/log");

2555 2556 2557 2558 2559 2560 2561 2562 2563 2564 2565 2566
		if (log_entries.empty())
		{
			res.log_max_index = 0;
		}
		else
		{
			const String & last_log_entry = *std::max_element(log_entries.begin(), log_entries.end());
			res.log_max_index = parse<UInt64>(last_log_entry.substr(strlen("log-")));
		}

		String log_pointer_str = zookeeper->get(replica_path + "/log_pointer");
		res.log_pointer = log_pointer_str.empty() ? 0 : parse<UInt64>(log_pointer_str);
2567 2568 2569 2570 2571 2572 2573 2574 2575 2576 2577

		auto all_replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
		res.total_replicas = all_replicas.size();

		res.active_replicas = 0;
		for (const String & replica : all_replicas)
			if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))
				++res.active_replicas;
	}
}

2578 2579 2580

void StorageReplicatedMergeTree::fetchPartition(const Field & partition, bool unreplicated, const String & from_)
{
2581 2582
	String partition_str = MergeTreeData::getMonthName(partition);

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

2587 2588
	LOG_INFO(log, "Will fetch partition " << partition_str << " from shard " << from_);

2589 2590 2591 2592 2593 2594 2595 2596
	/** Проверим, что в директории detached (куда мы будем записывать скаченные куски) ещё нет такой партиции.
	  * Ненадёжно (есть race condition) - такая партиция может появиться чуть позже.
	  */
	Poco::DirectoryIterator dir_end;
	for (Poco::DirectoryIterator dir_it{data.getFullPath() + "detached/"}; dir_it != dir_end; ++dir_it)
		if (0 == dir_it.name().compare(0, partition_str.size(), partition_str))
			throw Exception("Detached partition " + partition_str + " is already exists.", ErrorCodes::PARTITION_ALREADY_EXISTS);

2597 2598 2599 2600 2601 2602 2603 2604 2605 2606 2607 2608 2609 2610 2611
    if (unreplicated)
		throw Exception("Not implemented", ErrorCodes::NOT_IMPLEMENTED);	/// TODO

	/// Список реплик шарда-источника.
	zkutil::Strings replicas = zookeeper->getChildren(from + "/replicas");

	/// Оставим только активные реплики.
	zkutil::Strings active_replicas;
	active_replicas.reserve(replicas.size());

	for (const String & replica : replicas)
		if (zookeeper->exists(from + "/replicas/" + replica + "/is_active"))
			active_replicas.push_back(replica);

	if (active_replicas.empty())
2612
		throw Exception("No active replicas for shard " + from, ErrorCodes::NO_ACTIVE_REPLICAS);
2613 2614

	/** Надо выбрать лучшую (наиболее актуальную) реплику.
2615
	  * Это реплика с максимальным log_pointer, затем с минимальным размером queue.
2616 2617
	  * NOTE Это не совсем лучший критерий. Для скачивания старых партиций это не имеет смысла,
	  *  и было бы неплохо уметь выбирать реплику, ближайшую по сети.
2618
	  * NOTE Разумеется, здесь есть data race-ы. Можно решить ретраями.
2619
	  */
2620 2621 2622 2623 2624 2625
	Int64 max_log_pointer = -1;
	UInt64 min_queue_size = std::numeric_limits<UInt64>::max();
	String best_replica;

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

2628
		String log_pointer_str = zookeeper->get(current_replica_path + "/log_pointer");
2629 2630 2631
		Int64 log_pointer = log_pointer_str.empty() ? 0 : parse<UInt64>(log_pointer_str);

		zkutil::Stat stat;
2632
		zookeeper->get(current_replica_path + "/queue", &stat);
2633 2634 2635 2636 2637 2638 2639 2640 2641 2642 2643 2644 2645 2646 2647 2648 2649
		size_t queue_size = stat.numChildren;

		if (log_pointer > max_log_pointer
			|| (log_pointer == max_log_pointer && queue_size < min_queue_size))
		{
			max_log_pointer = log_pointer;
			min_queue_size = queue_size;
			best_replica = replica;
		}
	}

	if (best_replica.empty())
		throw Exception("Logical error: cannot choose best replica.", ErrorCodes::LOGICAL_ERROR);

	LOG_INFO(log, "Found " << replicas.size() << " replicas, " << active_replicas.size() << " of them are active."
		<< " Selected " << best_replica << " to fetch from.");

2650 2651
	String best_replica_path = from + "/replicas/" + best_replica;

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
	/** Пытаемся скачать эти куски.
	  * Часть из них могла удалиться из-за мерджа.
	  * В этом случае, обновляем информацию о доступных кусках и пробуем снова.
	  */

	unsigned try_no = 0;
	Strings missing_parts;
	do
	{
		if (try_no)
			LOG_INFO(log, "Some of parts (" << missing_parts.size() << ") are missing. Will try to fetch covering parts.");

		if (try_no >= 5)
			throw Exception("Too much retries to fetch parts from " + best_replica_path, ErrorCodes::TOO_MUCH_RETRIES_TO_FETCH_PARTS);

		Strings parts = zookeeper->getChildren(best_replica_path + "/parts");
		ActiveDataPartSet active_parts_set(parts);
		Strings parts_to_fetch;

		if (missing_parts.empty())
		{
			parts_to_fetch = active_parts_set.getParts();

			/// Оставляем только куски нужной партиции.
			Strings parts_to_fetch_partition;
			for (const String & part : parts_to_fetch)
				if (0 == part.compare(0, partition_str.size(), partition_str))
					parts_to_fetch_partition.push_back(part);

			parts_to_fetch = std::move(parts_to_fetch_partition);
2684 2685 2686

			if (parts_to_fetch.empty())
				throw Exception("Partition " + partition_str + " on " + best_replica_path + " doesn't exist", ErrorCodes::PARTITION_DOESNT_EXIST);
2687 2688 2689 2690 2691 2692 2693 2694 2695 2696 2697 2698 2699 2700 2701 2702 2703 2704 2705 2706 2707 2708 2709 2710 2711 2712 2713 2714 2715 2716 2717
		}
		else
		{
			for (const String & missing_part : missing_parts)
			{
				String containing_part = active_parts_set.getContainingPart(missing_part);
				if (!containing_part.empty())
					parts_to_fetch.push_back(containing_part);
				else
					LOG_WARNING(log, "Part " << missing_part << " on replica " << best_replica_path << " has been vanished.");
			}
		}

		LOG_INFO(log, "Parts to fetch: " << parts_to_fetch.size());

		missing_parts.clear();
		for (const String & part : parts_to_fetch)
		{
			try
			{
				fetchPart(part, best_replica_path, true);
			}
			catch (const DB::Exception & e)
			{
				if (e.code() != ErrorCodes::RECEIVED_ERROR_FROM_REMOTE_IO_SERVER)
					throw;

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

2719 2720
		++try_no;
	} while (!missing_parts.empty());
2721 2722 2723
}


M
Merge  
Michael Kolupaev 已提交
2724
}