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

namespace DB
{

M
Merge  
Michael Kolupaev 已提交
15

M
Merge  
Michael Kolupaev 已提交
16
const auto ERROR_SLEEP_MS = 1000;
M
Merge  
Michael Kolupaev 已提交
17
const auto MERGE_SELECTING_SLEEP_MS = 5 * 1000;
M
Merge  
Michael Kolupaev 已提交
18
const auto CLEANUP_SLEEP_MS = 30 * 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
/// Преобразовать число в строку формате суффиксов автоинкрементных нод в ZooKeeper.
static String padIndex(UInt64 index)
{
	String index_str = toString(index);
	while (index_str.size() < 10)
		index_str = '0' + index_str;
	return index_str;
}

M
Merge  
Michael Kolupaev 已提交
31

M
Merge  
Michael Kolupaev 已提交
32 33 34
StorageReplicatedMergeTree::StorageReplicatedMergeTree(
	const String & zookeeper_path_,
	const String & replica_name_,
M
Merge  
Michael Kolupaev 已提交
35
	bool attach,
M
Merge  
Michael Kolupaev 已提交
36 37
	const String & path_, const String & database_name_, const String & name_,
	NamesAndTypesListPtr columns_,
M
Merge  
Michael Kolupaev 已提交
38
	Context & context_,
M
Merge  
Michael Kolupaev 已提交
39 40 41 42 43 44 45 46
	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_)
	:
47
	context(context_), zookeeper(context.getZooKeeper()), database_name(database_name_),
M
Merge  
Michael Kolupaev 已提交
48 49 50
	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 已提交
51
	data(	full_path, columns_, context_, primary_expr_ast_, date_column_name_, sampling_expression_,
M
Merge  
Michael Kolupaev 已提交
52 53
			index_granularity_, mode_, sign_column_, settings_, database_name_ + "." + table_name, true,
			std::bind(&StorageReplicatedMergeTree::enqueuePartForCheck, this, std::placeholders::_1)),
M
Merge  
Michael Kolupaev 已提交
54
	reader(data), writer(data), merger(data), fetcher(data),
M
Merge  
Michael Kolupaev 已提交
55
	log(&Logger::get(database_name_ + "." + table_name + " (StorageReplicatedMergeTree)")),
M
Merge  
Michael Kolupaev 已提交
56
	shutdown_event(false)
M
Merge  
Michael Kolupaev 已提交
57
{
M
Merge  
Michael Kolupaev 已提交
58 59 60 61 62 63 64 65 66 67 68 69 70 71 72 73 74
	if (!zookeeper_path.empty() && *zookeeper_path.rbegin() == '/')
		zookeeper_path.erase(zookeeper_path.end() - 1);
	replica_path = zookeeper_path + "/replicas/" + replica_name;

	bool skip_sanity_checks = false;

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

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

	data.loadDataParts(skip_sanity_checks);

M
Merge  
Michael Kolupaev 已提交
75 76
	if (!zookeeper)
	{
M
Merge  
Michael Kolupaev 已提交
77 78 79
		if (!attach)
			throw Exception("Can't create replicated table without ZooKeeper", ErrorCodes::NO_ZOOKEEPER);

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

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

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

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

	String unreplicated_path = full_path + "unreplicated/";
	if (Poco::File(unreplicated_path).exists())
	{
		LOG_INFO(log, "Have unreplicated data");
		unreplicated_data.reset(new MergeTreeData(unreplicated_path, columns_, context_, primary_expr_ast_,
M
Merge  
Michael Kolupaev 已提交
105
			date_column_name_, sampling_expression_, index_granularity_, mode_, sign_column_, settings_,
M
Merge  
Michael Kolupaev 已提交
106
			database_name_ + "." + table_name + "[unreplicated]", false));
M
Merge  
Michael Kolupaev 已提交
107
		unreplicated_reader.reset(new MergeTreeDataSelectExecutor(*unreplicated_data));
M
Merge  
Michael Kolupaev 已提交
108
		unreplicated_merger.reset(new MergeTreeDataMerger(*unreplicated_data));
M
Merge  
Michael Kolupaev 已提交
109
	}
M
Merge  
Michael Kolupaev 已提交
110

M
Merge  
Michael Kolupaev 已提交
111 112 113 114 115 116
	/// Сгенерируем этому экземпляру случайный идентификатор.
	struct timespec times;
	if (clock_gettime(CLOCK_THREAD_CPUTIME_ID, &times))
		throwFromErrno("Cannot clock_gettime.", ErrorCodes::CANNOT_CLOCK_GETTIME);
	active_node_identifier = toString(times.tv_nsec);

M
Merge  
Michael Kolupaev 已提交
117
	restarting_thread = std::thread(&StorageReplicatedMergeTree::restartingThread, this);
M
Merge  
Michael Kolupaev 已提交
118 119 120 121 122
}

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

M
Merge  
Michael Kolupaev 已提交
148 149 150 151 152 153 154 155
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 已提交
156

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

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

M
Merge  
Michael Kolupaev 已提交
164 165
	zookeeper->createAncestors(zookeeper_path);

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

M
Merge  
Michael Kolupaev 已提交
176 177 178 179 180 181 182 183 184 185 186 187 188 189 190 191 192 193 194 195 196 197 198 199 200
	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 已提交
201
}
M
Merge  
Michael Kolupaev 已提交
202

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

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

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

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

M
Merge  
Michael Kolupaev 已提交
253 254 255 256 257 258 259 260 261
	/// Создадим пустую реплику. Ноду 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));
	zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
262

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

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

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

M
Merge  
Michael Kolupaev 已提交
275 276 277
	Stat stat;
	zookeeper->exists(replica_path, &stat);
	auto my_create_time = stat.czxid;
M
Merge  
Michael Kolupaev 已提交
278

M
Merge  
Michael Kolupaev 已提交
279 280
	std::random_shuffle(replicas.begin(), replicas.end());
	for (const String & replica : replicas)
M
Merge  
Michael Kolupaev 已提交
281
	{
M
Merge  
Michael Kolupaev 已提交
282 283 284 285 286 287 288 289
		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 已提交
290 291
	}

M
Merge  
Michael Kolupaev 已提交
292
	if (source_replica.empty())
M
Merge  
Michael Kolupaev 已提交
293
	{
M
Merge  
Michael Kolupaev 已提交
294
		LOG_INFO(log, "This is the first replica");
M
Merge  
Michael Kolupaev 已提交
295
	}
M
Merge  
Michael Kolupaev 已提交
296
	else
M
Merge  
Michael Kolupaev 已提交
297
	{
M
Merge  
Michael Kolupaev 已提交
298
		LOG_INFO(log, "Will mimic " << source_replica);
M
Merge  
Michael Kolupaev 已提交
299

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

M
Merge  
Michael Kolupaev 已提交
302 303 304 305 306 307 308 309 310 311 312 313 314 315 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 355 356 357 358 359 360 361 362 363
		/** Если эталонная реплика еще не до конца создана, подождем.
		  * NOTE: Если при ее создании что-то пошло не так, можем провисеть тут вечно.
		  *       Можно создавать на время создания эфемерную ноду, чтобы быть уверенным, что реплика создается, а не заброшена.
		  *       То же можно делать и для таблицы. Можно автоматически удалять ноду реплики/таблицы,
		  *        если видно, что она создана не до конца, а создающий ее умер.
		  */
		while (!zookeeper->exists(source_path + "/columns"))
		{
			LOG_INFO(log, "Waiting for replica " << source_path << " to be fully created");

			zkutil::EventPtr event = new Poco::Event;
			if (zookeeper->exists(source_path + "/columns", nullptr, event))
			{
				LOG_WARNING(log, "Oops, a watch has leaked");
				break;
			}

			event->wait();
		}

		/// Порядок следующих трех действий важен. Записи в логе могут продублироваться, но не могут потеряться.

		/// Скопируем у эталонной реплики ссылку на лог.
		zookeeper->set(replica_path + "/log_pointer", zookeeper->get(source_path + "/log_pointer"));

		/// Запомним очередь эталонной реплики.
		Strings source_queue_names = zookeeper->getChildren(source_path + "/queue");
		std::sort(source_queue_names.begin(), source_queue_names.end());
		Strings source_queue;
		for (const String & entry_name : source_queue_names)
		{
			String entry;
			if (!zookeeper->tryGet(source_path + "/queue/" + entry_name, entry))
				continue;
			source_queue.push_back(entry);
		}

		/// Добавим в очередь задания на получение всех активных кусков, которые есть у эталонной реплики.
		Strings parts = zookeeper->getChildren(source_path + "/parts");
		ActiveDataPartSet active_parts_set;
		for (const String & part : parts)
		{
			active_parts_set.add(part);
		}
		Strings active_parts = active_parts_set.getParts();
		for (const String & name : active_parts)
		{
			LogEntry log_entry;
			log_entry.type = LogEntry::GET_PART;
			log_entry.source_replica = "";
			log_entry.new_part_name = name;

			zookeeper->create(replica_path + "/queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential);
		}
		LOG_DEBUG(log, "Queued " << active_parts.size() << " parts to be fetched");

		/// Добавим в очередь содержимое очереди эталонной реплики.
		for (const String & entry : source_queue)
		{
			zookeeper->create(replica_path + "/queue/queue-", entry, zkutil::CreateMode::PersistentSequential);
		}
		LOG_DEBUG(log, "Copied " << source_queue.size() << " queue entries");
M
Merge  
Michael Kolupaev 已提交
364
	}
M
Merge  
Michael Kolupaev 已提交
365 366

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

M
Merge  
Michael Kolupaev 已提交
369 370 371 372 373
void StorageReplicatedMergeTree::activateReplica()
{
	std::stringstream host;
	host << "host: " << context.getInterserverIOHost() << std::endl;
	host << "port: " << context.getInterserverIOPort() << std::endl;
M
Merge  
Michael Kolupaev 已提交
374

M
Merge  
Michael Kolupaev 已提交
375 376 377 378 379 380 381 382 383
	/** Если нода отмечена как активная, но отметка сделана в этом же экземпляре, удалим ее.
	  * Такое возможно только при истечении сессии в 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 已提交
384
	/// Одновременно объявим, что эта реплика активна, и обновим хост.
M
Merge  
Michael Kolupaev 已提交
385
	zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
386
	ops.push_back(new zkutil::Op::Create(replica_path + "/is_active", "", zookeeper->getDefaultACL(), zkutil::CreateMode::Ephemeral));
M
Merge  
Michael Kolupaev 已提交
387
	ops.push_back(new zkutil::Op::SetData(replica_path + "/host", host.str(), -1));
M
Merge  
Michael Kolupaev 已提交
388 389 390

	try
	{
M
Merge  
Michael Kolupaev 已提交
391
		zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
392 393 394
	}
	catch (zkutil::KeeperException & e)
	{
395
		if (e.code == ZNODEEXISTS)
M
Merge  
Michael Kolupaev 已提交
396 397 398 399 400
			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 已提交
401

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

M
Merge  
Michael Kolupaev 已提交
405
void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
M
Merge  
Michael Kolupaev 已提交
406
{
M
Merge  
Michael Kolupaev 已提交
407
	Strings expected_parts_vec = zookeeper->getChildren(replica_path + "/parts");
M
Merge  
Michael Kolupaev 已提交
408 409

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

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

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

M
Merge  
Michael Kolupaev 已提交
417 418 419 420 421 422 423 424
	for (const auto & part : parts)
	{
		if (expected_parts.count(part->name))
		{
			expected_parts.erase(part->name);
		}
		else
		{
M
Merge  
Michael Kolupaev 已提交
425
			unexpected_parts.insert(part);
M
Merge  
Michael Kolupaev 已提交
426 427 428
		}
	}

M
Merge  
Michael Kolupaev 已提交
429
	/// Какие локальные куски добавить в ZK.
M
Merge  
Michael Kolupaev 已提交
430
	MergeTreeData::DataPartsVector parts_to_add;
M
Merge  
Michael Kolupaev 已提交
431 432 433 434

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

M
Merge  
Michael Kolupaev 已提交
435 436
	for (const String & missing_name : expected_parts)
	{
M
Merge  
Michael Kolupaev 已提交
437
		/// Если локально не хватает какого-то куска, но есть покрывающий его кусок, можно заменить в ZK недостающий покрывающим.
M
Merge  
Michael Kolupaev 已提交
438
		auto containing = data.getActiveContainingPart(missing_name);
M
Merge  
Michael Kolupaev 已提交
439 440 441 442 443 444 445 446 447 448
		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 已提交
449
		{
M
Merge  
Michael Kolupaev 已提交
450
			LOG_ERROR(log, "Fetching missing part " << missing_name);
M
Merge  
Michael Kolupaev 已提交
451
			parts_to_fetch.push_back(missing_name);
M
Merge  
Michael Kolupaev 已提交
452 453
		}
	}
M
Merge  
Michael Kolupaev 已提交
454

M
Merge  
Michael Kolupaev 已提交
455 456 457 458 459 460
	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 已提交
461
					 + toString(expected_parts.size()) + " missing obsolete parts, "
M
Merge  
Michael Kolupaev 已提交
462 463
					 + toString(parts_to_fetch.size()) + " missing parts";
	bool insane =
M
Merge  
Michael Kolupaev 已提交
464 465 466 467
		parts_to_add.size() > data.settings.replicated_max_unexpectedly_merged_parts ||
		unexpected_parts.size() > data.settings.replicated_max_unexpected_parts ||
		expected_parts.size() > data.settings.replicated_max_missing_obsolete_parts ||
		parts_to_fetch.size() > data.settings.replicated_max_missing_active_parts;
M
Merge  
Michael Kolupaev 已提交
468

M
Merge  
Michael Kolupaev 已提交
469
	if (insane && !skip_sanity_checks)
M
Merge  
Michael Kolupaev 已提交
470 471 472
	{
		throw Exception("The local set of parts of table " + getTableName() + " doesn't look like the set of parts in ZooKeeper. "
			+ sanity_report,
M
Merge  
Michael Kolupaev 已提交
473
			ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS);
M
Merge  
Michael Kolupaev 已提交
474 475
	}

M
Merge  
Michael Kolupaev 已提交
476 477 478 479 480
	if (insane)
	{
		LOG_WARNING(log, sanity_report);
	}

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

M
Merge  
Michael Kolupaev 已提交
486
		zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
487
		checkPartAndAddToZooKeeper(part, ops);
M
Merge  
Michael Kolupaev 已提交
488
		zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
489
	}
M
Merge  
Michael Kolupaev 已提交
490

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

		zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
497
		ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + name + "/columns", -1));
M
Merge  
Michael Kolupaev 已提交
498 499 500 501 502 503 504 505 506 507 508 509
		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 已提交
510
		log_entry.source_replica = "";
M
Merge  
Michael Kolupaev 已提交
511 512 513
		log_entry.new_part_name = name;

		/// Полагаемся, что это происходит до загрузки очереди (loadQueue).
M
Merge  
Michael Kolupaev 已提交
514
		zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
515
		ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + name + "/columns", -1));
M
Merge  
Michael Kolupaev 已提交
516 517
		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 已提交
518 519
		ops.push_back(new zkutil::Op::Create(
			replica_path + "/queue/queue-", log_entry.toString(), zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
M
Merge  
Michael Kolupaev 已提交
520
		zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
521 522 523
	}

	/// Удалим лишние локальные куски.
M
Merge  
Michael Kolupaev 已提交
524 525
	for (MergeTreeData::DataPartPtr part : unexpected_parts)
	{
M
Merge  
Michael Kolupaev 已提交
526
		LOG_ERROR(log, "Renaming unexpected part " << part->name << " to ignored_" + part->name);
M
Merge  
Michael Kolupaev 已提交
527
		data.renameAndDetachPart(part, "ignored_", true);
M
Merge  
Michael Kolupaev 已提交
528 529
	}
}
M
Merge  
Michael Kolupaev 已提交
530

M
Merge  
Michael Kolupaev 已提交
531 532 533 534 535 536 537 538 539
void StorageReplicatedMergeTree::initVirtualParts()
{
	auto parts = data.getDataParts();
	for (const auto & part : parts)
	{
		virtual_parts.add(part->name);
	}
}

M
Merge  
Michael Kolupaev 已提交
540
void StorageReplicatedMergeTree::checkPartAndAddToZooKeeper(MergeTreeData::DataPartPtr part, zkutil::Ops & ops, String part_name)
M
Merge  
Michael Kolupaev 已提交
541
{
M
Merge  
Michael Kolupaev 已提交
542 543 544
	if (part_name.empty())
		part_name = part->name;

545
	check(part->columns);
M
Merge  
Michael Kolupaev 已提交
546
	int expected_columns_version = columns_version;
547

M
Merge  
Michael Kolupaev 已提交
548 549 550 551 552
	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 已提交
553
	{
M
Merge  
Michael Kolupaev 已提交
554 555
		zkutil::Stat stat_before, stat_after;
		String columns_str;
M
Merge  
Michael Kolupaev 已提交
556
		if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name + "/columns", columns_str, &stat_before))
M
Merge  
Michael Kolupaev 已提交
557 558 559
			continue;
		if (columns_str != expected_columns_str)
		{
M
Merge  
Michael Kolupaev 已提交
560
			LOG_INFO(log, "Not checking checksums of part " << part_name << " with replica " << replica
M
Merge  
Michael Kolupaev 已提交
561 562 563
				<< " because columns are different");
			continue;
		}
M
Merge  
Michael Kolupaev 已提交
564
		String checksums_str;
M
Merge  
Michael Kolupaev 已提交
565 566
		/// Проверим, что версия ноды со столбцами не изменилась, пока мы читали checksums.
		/// Это гарантирует, что столбцы и чексуммы относятся к одним и тем же данным.
M
Merge  
Michael Kolupaev 已提交
567 568
		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 已提交
569
			stat_before.version != stat_after.version)
M
Merge  
Michael Kolupaev 已提交
570
		{
M
Merge  
Michael Kolupaev 已提交
571
			LOG_INFO(log, "Not checking checksums of part " << part_name << " with replica " << replica
M
Merge  
Michael Kolupaev 已提交
572 573
				<< " because part changed while we were reading its checksums");
			continue;
M
Merge  
Michael Kolupaev 已提交
574
		}
M
Merge  
Michael Kolupaev 已提交
575 576 577

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

M
Merge  
Michael Kolupaev 已提交
580
	if (zookeeper->exists(replica_path + "/parts/" + part_name))
M
Merge  
Michael Kolupaev 已提交
581
	{
M
Merge  
Michael Kolupaev 已提交
582
		LOG_ERROR(log, "checkPartAndAddToZooKeeper: node " << replica_path + "/parts/" + part_name << " already exists");
M
Merge  
Michael Kolupaev 已提交
583 584 585
		return;
	}

586 587 588
	ops.push_back(new zkutil::Op::Check(
		zookeeper_path + "/columns",
		expected_columns_version));
M
Merge  
Michael Kolupaev 已提交
589
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
590
		replica_path + "/parts/" + part_name,
M
Merge  
Michael Kolupaev 已提交
591
		"",
M
Merge  
Michael Kolupaev 已提交
592
		zookeeper->getDefaultACL(),
M
Merge  
Michael Kolupaev 已提交
593
		zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
594
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
595
		replica_path + "/parts/" + part_name + "/columns",
M
Merge  
Michael Kolupaev 已提交
596 597 598
		part->columns.toString(),
		zookeeper->getDefaultACL(),
		zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
599
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
600
		replica_path + "/parts/" + part_name + "/checksums",
M
Merge  
Michael Kolupaev 已提交
601
		part->checksums.toString(),
M
Merge  
Michael Kolupaev 已提交
602
		zookeeper->getDefaultACL(),
M
Merge  
Michael Kolupaev 已提交
603 604 605
		zkutil::CreateMode::Persistent));
}

M
Merge  
Michael Kolupaev 已提交
606 607
void StorageReplicatedMergeTree::clearOldParts()
{
M
Merge  
Michael Kolupaev 已提交
608 609
	auto table_lock = lockStructure(false);

M
Merge  
Michael Kolupaev 已提交
610 611
	MergeTreeData::DataPartsVector parts = data.grabOldParts();
	size_t count = parts.size();
M
Merge  
Michael Kolupaev 已提交
612

M
Merge  
Michael Kolupaev 已提交
613 614 615 616
	if (!count)
		return;

	try
M
Merge  
Michael Kolupaev 已提交
617
	{
M
Merge  
Michael Kolupaev 已提交
618 619 620 621
		while (!parts.empty())
		{
			MergeTreeData::DataPartPtr part = parts.back();

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

M
Merge  
Michael Kolupaev 已提交
624 625 626 627
			zkutil::Ops ops;
			ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + part->name + "/columns", -1));
			ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + part->name + "/checksums", -1));
			ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + part->name, -1));
M
Merge  
Michael Kolupaev 已提交
628 629 630
			auto code = zookeeper->tryMulti(ops);
			if (code != ZOK)
				LOG_WARNING(log, "Couldn't remove " << part->name << " from ZooKeeper: " << zkutil::ZooKeeper::error2string(code));
M
Merge  
Michael Kolupaev 已提交
631 632 633 634 635 636 637 638 639

			part->remove();
			parts.pop_back();
		}
	}
	catch (...)
	{
		data.addOldParts(parts);
		throw;
M
Merge  
Michael Kolupaev 已提交
640 641
	}

M
Merge  
Michael Kolupaev 已提交
642
	LOG_DEBUG(log, "Removed " << count << " old parts");
M
Merge  
Michael Kolupaev 已提交
643 644
}

M
Merge  
Michael Kolupaev 已提交
645 646
void StorageReplicatedMergeTree::clearOldLogs()
{
M
Merge  
Michael Kolupaev 已提交
647 648 649 650 651 652 653 654 655 656 657
	zkutil::Stat stat;
	if (!zookeeper->exists(zookeeper_path + "/log", &stat))
		throw Exception(zookeeper_path + "/log doesn't exist", ErrorCodes::NOT_FOUND_NODE);

	int children_count = stat.numChildren;

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

	Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas", &stat);
M
Merge  
Michael Kolupaev 已提交
658 659 660
	UInt64 min_pointer = std::numeric_limits<UInt64>::max();
	for (const String & replica : replicas)
	{
M
Merge  
Michael Kolupaev 已提交
661 662
		String pointer = zookeeper->get(zookeeper_path + "/replicas/" + replica + "/log_pointer");
		if (pointer.empty())
M
Merge  
Michael Kolupaev 已提交
663 664 665 666
			return;
		min_pointer = std::min(min_pointer, parse<UInt64>(pointer));
	}

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

M
Merge  
Michael Kolupaev 已提交
670 671
	/// Не будем трогать последние replicated_logs_to_keep записей.
	entries.erase(entries.end() - std::min(entries.size(), data.settings.replicated_logs_to_keep), entries.end());
M
Merge  
Michael Kolupaev 已提交
672 673 674 675 676
	/// Не будем трогать записи, не меньшие min_pointer.
	entries.erase(std::lower_bound(entries.begin(), entries.end(), "log-" + padIndex(min_pointer)), entries.end());

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

M
Merge  
Michael Kolupaev 已提交
678
	zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
679
	for (size_t i = 0; i < entries.size(); ++i)
M
Merge  
Michael Kolupaev 已提交
680
	{
M
Merge  
Michael Kolupaev 已提交
681 682 683
		ops.push_back(new zkutil::Op::Remove(zookeeper_path + "/log/" + entries[i], -1));

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

M
Merge  
Michael Kolupaev 已提交
692
	LOG_DEBUG(log, "Removed " << entries.size() << " old log entries: " << entries.front() << " - " << entries.back());
M
Merge  
Michael Kolupaev 已提交
693 694 695 696 697
}

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

701
	int children_count = stat.numChildren;
M
Merge  
Michael Kolupaev 已提交
702

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

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

M
Merge  
Michael Kolupaev 已提交
710
	Strings blocks = zookeeper->getChildren(zookeeper_path + "/blocks");
M
Merge  
Michael Kolupaev 已提交
711 712 713 714 715 716

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

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

M
Merge  
Michael Kolupaev 已提交
721
	zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
722
	std::sort(timed_blocks.begin(), timed_blocks.end(), std::greater<std::pair<Int64, String>>());
M
Merge  
Michael Kolupaev 已提交
723 724
	for (size_t i = data.settings.replicated_deduplication_window; i <  timed_blocks.size(); ++i)
	{
M
Merge  
Michael Kolupaev 已提交
725
		ops.push_back(new zkutil::Op::Remove(zookeeper_path + "/blocks/" + timed_blocks[i].second + "/number", -1));
M
Merge  
Michael Kolupaev 已提交
726
		ops.push_back(new zkutil::Op::Remove(zookeeper_path + "/blocks/" + timed_blocks[i].second + "/columns", -1));
M
Merge  
Michael Kolupaev 已提交
727 728
		ops.push_back(new zkutil::Op::Remove(zookeeper_path + "/blocks/" + timed_blocks[i].second + "/checksums", -1));
		ops.push_back(new zkutil::Op::Remove(zookeeper_path + "/blocks/" + timed_blocks[i].second, -1));
M
Merge  
Michael Kolupaev 已提交
729
		if (ops.size() > 400 || i + 1 == timed_blocks.size())
M
Merge  
Michael Kolupaev 已提交
730 731 732 733
		{
			zookeeper->multi(ops);
			ops.clear();
		}
M
Merge  
Michael Kolupaev 已提交
734 735 736 737 738
	}

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

M
Merge  
Michael Kolupaev 已提交
739 740
void StorageReplicatedMergeTree::loadQueue()
{
M
Merge  
Michael Kolupaev 已提交
741
	std::unique_lock<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
742

M
Merge  
Michael Kolupaev 已提交
743
	Strings children = zookeeper->getChildren(replica_path + "/queue");
M
Merge  
Michael Kolupaev 已提交
744 745 746
	std::sort(children.begin(), children.end());
	for (const String & child : children)
	{
M
Merge  
Michael Kolupaev 已提交
747
		String s = zookeeper->get(replica_path + "/queue/" + child);
M
Merge  
Michael Kolupaev 已提交
748 749 750
		LogEntryPtr entry = LogEntry::parse(s);
		entry->znode_name = child;
		entry->addResultToVirtualParts(*this);
M
Merge  
Michael Kolupaev 已提交
751
		queue.push_back(entry);
M
Merge  
Michael Kolupaev 已提交
752 753 754
	}
}

M
Merge  
Michael Kolupaev 已提交
755
void StorageReplicatedMergeTree::pullLogsToQueue(zkutil::EventPtr next_update_event)
M
Merge  
Michael Kolupaev 已提交
756
{
M
Merge  
Michael Kolupaev 已提交
757
	std::unique_lock<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
758

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

M
Merge  
Michael Kolupaev 已提交
762
	if (index_str.empty())
M
Merge  
Michael Kolupaev 已提交
763
	{
M
Merge  
Michael Kolupaev 已提交
764 765 766
		/// Если у нас еще нет указателя на лог, поставим указатель на первую запись в нем.
		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 已提交
767

M
Merge  
Michael Kolupaev 已提交
768
		zookeeper->set(replica_path + "/log_pointer", toString(index));
M
Merge  
Michael Kolupaev 已提交
769 770
	}
	else
M
Merge  
Michael Kolupaev 已提交
771
	{
M
Merge  
Michael Kolupaev 已提交
772
		index = parse<UInt64>(index_str);
M
Merge  
Michael Kolupaev 已提交
773
	}
M
Merge  
Michael Kolupaev 已提交
774

M
Merge  
Michael Kolupaev 已提交
775 776
	UInt64 first_index = index;

M
Merge  
Michael Kolupaev 已提交
777
	size_t count = 0;
M
Merge  
Michael Kolupaev 已提交
778 779
	String entry_str;
	while (zookeeper->tryGet(zookeeper_path + "/log/log-" + padIndex(index), entry_str))
M
Merge  
Michael Kolupaev 已提交
780
	{
M
Merge  
Michael Kolupaev 已提交
781
		++count;
M
Merge  
Michael Kolupaev 已提交
782
		++index;
M
Merge  
Michael Kolupaev 已提交
783

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

M
Merge  
Michael Kolupaev 已提交
786 787 788
		/// Одновременно добавим запись в очередь и продвинем указатель на лог.
		zkutil::Ops ops;
		ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
789
			replica_path + "/queue/queue-", entry_str, zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
M
Merge  
Michael Kolupaev 已提交
790
		ops.push_back(new zkutil::Op::SetData(
M
Merge  
Michael Kolupaev 已提交
791
			replica_path + "/log_pointer", toString(index), -1));
M
Merge  
Michael Kolupaev 已提交
792
		auto results = zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
793

794
		String path_created = dynamic_cast<zkutil::Op::Create &>(ops[0]).getPathCreated();
M
Merge  
Michael Kolupaev 已提交
795 796
		entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
		entry->addResultToVirtualParts(*this);
M
Merge  
Michael Kolupaev 已提交
797
		queue.push_back(entry);
M
Merge  
Michael Kolupaev 已提交
798
	}
M
Merge  
Michael Kolupaev 已提交
799

M
Merge  
Michael Kolupaev 已提交
800 801 802 803
	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 已提交
804
	}
M
Merge  
Michael Kolupaev 已提交
805

M
Merge  
Michael Kolupaev 已提交
806 807 808
	if (!count)
		return;

M
Merge  
Michael Kolupaev 已提交
809 810
	if (queue_task_handle)
		queue_task_handle->wake();
M
Merge  
Michael Kolupaev 已提交
811

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

M
Merge  
Michael Kolupaev 已提交
815
bool StorageReplicatedMergeTree::shouldExecuteLogEntry(const LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
816
{
M
Merge  
Michael Kolupaev 已提交
817 818
	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 已提交
819 820 821 822 823 824
	{
		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 已提交
825 826 827 828 829 830 831 832 833 834 835
	if (entry.type == LogEntry::MERGE_PARTS)
	{
		/** Если какая-то из нужных частей сейчас передается или мерджится, подождем окончания этой операции.
		  * Иначе, даже если всех нужных частей для мерджа нет, нужно попытаться сделать мердж.
		  * Если каких-то частей не хватает, вместо мерджа будет попытка скачать кусок.
		  * Такая ситуация возможна, если получение какого-то куска пофейлилось, и его переместили в конец очереди.
		  */
		for (const auto & name : entry.parts_to_merge)
		{
			if (future_parts.count(name))
			{
M
Merge  
Michael Kolupaev 已提交
836
				LOG_TRACE(log, "Not merging into part " << entry.new_part_name << " because part " << name << " is not ready yet.");
M
Merge  
Michael Kolupaev 已提交
837 838 839 840 841 842
				return false;
			}
		}
	}

	return true;
M
Merge  
Michael Kolupaev 已提交
843 844
}

M
Merge  
Michael Kolupaev 已提交
845
bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry, BackgroundProcessingPool::Context & pool_context)
M
Merge  
Michael Kolupaev 已提交
846
{
M
Merge  
Michael Kolupaev 已提交
847
	if (entry.type == LogEntry::DROP_RANGE)
M
Merge  
Michael Kolupaev 已提交
848 849 850 851
	{
		executeDropRange(entry);
		return true;
	}
M
Merge  
Michael Kolupaev 已提交
852

M
Merge  
Michael Kolupaev 已提交
853
	if (entry.type == LogEntry::GET_PART ||
M
Merge  
Michael Kolupaev 已提交
854 855
		entry.type == LogEntry::MERGE_PARTS ||
		entry.type == LogEntry::ATTACH_PART)
M
Merge  
Michael Kolupaev 已提交
856 857
	{
		/// Если у нас уже есть этот кусок или покрывающий его кусок, ничего делать не нужно.
M
Merge  
Michael Kolupaev 已提交
858
		MergeTreeData::DataPartPtr containing_part = data.getActiveContainingPart(entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
859 860

		/// Даже если кусок есть локально, его (в исключительных случаях) может не быть в zookeeper.
M
Merge  
Michael Kolupaev 已提交
861
		if (containing_part && zookeeper->exists(replica_path + "/parts/" + containing_part->name))
M
Merge  
Michael Kolupaev 已提交
862
		{
M
Merge  
Michael Kolupaev 已提交
863 864
			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 已提交
865
			return true;
M
Merge  
Michael Kolupaev 已提交
866
		}
M
Merge  
Michael Kolupaev 已提交
867 868
	}

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

	bool do_fetch = false;

M
Merge  
Michael Kolupaev 已提交
874 875
	if (entry.type == LogEntry::GET_PART)
	{
M
Merge  
Michael Kolupaev 已提交
876
		do_fetch = true;
M
Merge  
Michael Kolupaev 已提交
877
	}
M
Merge  
Michael Kolupaev 已提交
878 879 880 881
	else if (entry.type == LogEntry::ATTACH_PART)
	{
		do_fetch = !executeAttachPart(entry);
	}
M
Merge  
Michael Kolupaev 已提交
882 883
	else if (entry.type == LogEntry::MERGE_PARTS)
	{
M
Merge  
Michael Kolupaev 已提交
884
		MergeTreeData::DataPartsVector parts;
885
		bool have_all_parts = true;
M
Merge  
Michael Kolupaev 已提交
886 887
		for (const String & name : entry.parts_to_merge)
		{
M
Merge  
Michael Kolupaev 已提交
888
			MergeTreeData::DataPartPtr part = data.getActiveContainingPart(name);
M
Merge  
Michael Kolupaev 已提交
889 890 891 892 893 894 895
			if (!part)
			{
				have_all_parts = false;
				break;
			}
			if (part->name != name)
			{
M
Merge  
Michael Kolupaev 已提交
896 897
				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 已提交
898 899 900
				have_all_parts = false;
				break;
			}
M
Merge  
Michael Kolupaev 已提交
901 902
			parts.push_back(part);
		}
M
Merge  
Michael Kolupaev 已提交
903

M
Merge  
Michael Kolupaev 已提交
904
		if (!have_all_parts)
M
Merge  
Michael Kolupaev 已提交
905
		{
M
Merge  
Michael Kolupaev 已提交
906 907 908
			/// Если нет всех нужных кусков, попробуем взять у кого-нибудь уже помердженный кусок.
			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 已提交
909
		}
M
Merge  
Michael Kolupaev 已提交
910 911
		else
		{
M
Merge  
Michael Kolupaev 已提交
912 913 914
			/// Если собираемся сливать большие куски, увеличим счетчик потоков, сливающих большие куски.
			for (const auto & part : parts)
			{
M
Merge  
Michael Kolupaev 已提交
915
				if (part->size_in_bytes > data.settings.max_bytes_to_merge_parts_small)
M
Merge  
Michael Kolupaev 已提交
916 917 918 919 920 921 922
				{
					pool_context.incrementCounter("big merges");
					pool_context.incrementCounter("replicated big merges");
					break;
				}
			}

M
Merge  
Michael Kolupaev 已提交
923
			auto table_lock = lockStructure(false);
924

M
Merge  
Michael Kolupaev 已提交
925 926
			MergeTreeData::Transaction transaction;
			MergeTreeData::DataPartPtr part = merger.mergeParts(parts, entry.new_part_name, &transaction);
M
Merge  
Michael Kolupaev 已提交
927 928

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

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

M
Merge  
Michael Kolupaev 已提交
935
			zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
936 937 938 939

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

			ProfileEvents::increment(ProfileEvents::ReplicatedPartMerges);
		}
M
Merge  
Michael Kolupaev 已提交
945 946 947 948 949
	}
	else
	{
		throw Exception("Unexpected log entry type: " + toString(static_cast<int>(entry.type)));
	}
M
Merge  
Michael Kolupaev 已提交
950 951 952

	if (do_fetch)
	{
M
Merge  
Michael Kolupaev 已提交
953 954
		String replica;

M
Merge  
Michael Kolupaev 已提交
955 956
		try
		{
M
Merge  
Michael Kolupaev 已提交
957
			replica = findReplicaHavingPart(entry.new_part_name, true);
M
Merge  
Michael Kolupaev 已提交
958 959 960 961 962
			if (replica.empty())
			{
				ProfileEvents::increment(ProfileEvents::ReplicatedPartFailedFetches);
				throw Exception("No active replica has part " + entry.new_part_name, ErrorCodes::NO_REPLICA_HAS_PART);
			}
M
Merge  
Michael Kolupaev 已提交
963 964 965 966 967 968 969 970 971 972 973 974 975 976
			fetchPart(entry.new_part_name, replica);

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

				/// Найдем действие по объединению этого куска с другими. Запомним других.
				StringSet parts_for_merge;
				LogEntries::iterator merge_entry;
				for (LogEntries::iterator it = queue.begin(); it != queue.end(); ++it)
				{
M
Merge  
Michael Kolupaev 已提交
984
					if ((*it)->type == LogEntry::MERGE_PARTS)
M
Merge  
Michael Kolupaev 已提交
985
					{
M
Merge  
Michael Kolupaev 已提交
986 987
						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 已提交
988
						{
M
Merge  
Michael Kolupaev 已提交
989
							parts_for_merge = StringSet((*it)->parts_to_merge.begin(), (*it)->parts_to_merge.end());
M
Merge  
Michael Kolupaev 已提交
990 991 992 993 994 995 996 997 998 999 1000 1001 1002 1003 1004 1005 1006
							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 已提交
1007 1008
						if (((*it0)->type == LogEntry::MERGE_PARTS || (*it0)->type == LogEntry::GET_PART)
							&& parts_for_merge.count((*it0)->new_part_name))
M
Merge  
Michael Kolupaev 已提交
1009 1010 1011 1012
						{
							queue.splice(queue.end(), queue, it0, it);
						}
					}
M
Merge  
Michael Kolupaev 已提交
1013 1014 1015 1016 1017 1018 1019 1020 1021

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

				/// Если ни у кого нет куска, и в очереди нет слияний с его участием, проверим, есть ли у кого-то покрывающий его.
				if (replica.empty())
					enqueuePartForCheck(entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
1027 1028 1029 1030 1031 1032 1033 1034 1035
			}
			catch (...)
			{
				tryLogCurrentException(__PRETTY_FUNCTION__);
			}

			throw;
		}
	}
M
Merge  
Michael Kolupaev 已提交
1036 1037

	return true;
M
Merge  
Michael Kolupaev 已提交
1038 1039
}

M
Merge  
Michael Kolupaev 已提交
1040
void StorageReplicatedMergeTree::executeDropRange(const StorageReplicatedMergeTree::LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
1041
{
M
Merge  
Michael Kolupaev 已提交
1042
	LOG_INFO(log, (entry.detach ? "Detaching" : "Removing") << " parts inside " << entry.new_part_name << ".");
M
Merge  
Michael Kolupaev 已提交
1043 1044 1045 1046 1047 1048 1049 1050 1051 1052 1053 1054 1055 1056 1057 1058 1059 1060 1061 1062 1063 1064 1065 1066 1067 1068 1069 1070 1071

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

M
Merge  
Michael Kolupaev 已提交
1076
	LOG_DEBUG(log, (entry.detach ? "Detaching" : "Removing") << " parts.");
M
Merge  
Michael Kolupaev 已提交
1077 1078 1079 1080 1081 1082 1083 1084 1085 1086 1087
	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 已提交
1088 1089 1090 1091
		/// Если кусок удалять не нужно, надежнее переместить директорию до изменений в ZooKeeper.
		if (entry.detach)
			data.renameAndDetachPart(part);

M
Merge  
Michael Kolupaev 已提交
1092 1093 1094 1095 1096 1097
		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 已提交
1098
		/// Если кусок нужно удалить, надежнее удалить директорию после изменений в ZooKeeper.
M
Merge  
Michael Kolupaev 已提交
1099
		if (!entry.detach)
M
Merge  
Michael Kolupaev 已提交
1100
			data.replaceParts({part}, {}, true);
M
Merge  
Michael Kolupaev 已提交
1101 1102
	}

M
Merge  
Michael Kolupaev 已提交
1103
	LOG_INFO(log, (entry.detach ? "Detached " : "Removed ") << removed_parts << " parts inside " << entry.new_part_name << ".");
M
Merge  
Michael Kolupaev 已提交
1104 1105 1106

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

M
Merge  
Michael Kolupaev 已提交
1109 1110 1111 1112 1113 1114 1115 1116 1117 1118 1119 1120 1121 1122 1123
		removed_parts = 0;
		parts = unreplicated_data->getDataParts();
		for (const auto & part : parts)
		{
			if (!ActiveDataPartSet::contains(entry.new_part_name, part->name))
				continue;
			LOG_DEBUG(log, "Removing unreplicated part " << part->name);
			++removed_parts;

			if (entry.detach)
				unreplicated_data->renameAndDetachPart(part, "");
			else
				unreplicated_data->replaceParts({part}, {}, false);
		}
	}
M
Merge  
Michael Kolupaev 已提交
1124 1125 1126 1127 1128 1129 1130 1131
}

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 已提交
1132
	if (!Poco::File(data.getFullPath() + source_path).exists())
M
Merge  
Michael Kolupaev 已提交
1133
	{
M
Merge  
Michael Kolupaev 已提交
1134
		LOG_INFO(log, "No part at " << source_path << ". Will fetch it instead");
M
Merge  
Michael Kolupaev 已提交
1135 1136 1137 1138 1139 1140 1141 1142 1143 1144 1145 1146 1147 1148 1149 1150 1151 1152 1153 1154 1155 1156 1157 1158 1159 1160 1161 1162
		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 已提交
1163

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

M
Merge  
Michael Kolupaev 已提交
1167 1168 1169
	return true;
}

M
Merge  
Michael Kolupaev 已提交
1170 1171 1172 1173
void StorageReplicatedMergeTree::queueUpdatingThread()
{
	while (!shutdown_called)
	{
M
Merge  
Michael Kolupaev 已提交
1174 1175
		try
		{
M
Merge  
Michael Kolupaev 已提交
1176
			pullLogsToQueue(queue_updating_event);
M
Merge  
Michael Kolupaev 已提交
1177

M
Merge  
Michael Kolupaev 已提交
1178
			queue_updating_event->wait();
M
Merge  
Michael Kolupaev 已提交
1179
		}
M
Merge  
Michael Kolupaev 已提交
1180 1181 1182 1183 1184 1185 1186 1187 1188
		catch (zkutil::KeeperException & e)
		{
			if (e.code == ZINVALIDSTATE)
				restarting_event.set();

			tryLogCurrentException(__PRETTY_FUNCTION__);

			queue_updating_event->tryWait(ERROR_SLEEP_MS);
		}
M
Merge  
Michael Kolupaev 已提交
1189 1190 1191 1192
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);

M
Merge  
Michael Kolupaev 已提交
1193 1194
			queue_updating_event->tryWait(ERROR_SLEEP_MS);
		}
M
Merge  
Michael Kolupaev 已提交
1195
	}
M
Merge  
Michael Kolupaev 已提交
1196 1197

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

M
Merge  
Michael Kolupaev 已提交
1200
bool StorageReplicatedMergeTree::queueTask(BackgroundProcessingPool::Context & pool_context)
M
Merge  
Michael Kolupaev 已提交
1201
{
M
Merge  
Michael Kolupaev 已提交
1202
	LogEntryPtr entry;
M
Merge  
Michael Kolupaev 已提交
1203

M
Merge  
Michael Kolupaev 已提交
1204 1205
	try
	{
M
Merge  
Michael Kolupaev 已提交
1206
		std::unique_lock<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1207 1208
		bool empty = queue.empty();
		if (!empty)
M
Merge  
Michael Kolupaev 已提交
1209
		{
M
Merge  
Michael Kolupaev 已提交
1210
			for (LogEntries::iterator it = queue.begin(); it != queue.end(); ++it)
M
Merge  
Michael Kolupaev 已提交
1211
			{
M
Merge  
Michael Kolupaev 已提交
1212
				if (!(*it)->currently_executing && shouldExecuteLogEntry(**it))
M
Merge  
Michael Kolupaev 已提交
1213
				{
M
Merge  
Michael Kolupaev 已提交
1214
					entry = *it;
M
Merge  
Michael Kolupaev 已提交
1215
					entry->tagPartAsFuture(*this);
M
Merge  
Michael Kolupaev 已提交
1216
					queue.splice(queue.end(), queue, it);
M
Merge  
Michael Kolupaev 已提交
1217
					entry->currently_executing = true;
M
Merge  
Michael Kolupaev 已提交
1218
					break;
M
Merge  
Michael Kolupaev 已提交
1219
				}
M
Merge  
Michael Kolupaev 已提交
1220 1221
			}
		}
M
Merge  
Michael Kolupaev 已提交
1222 1223 1224 1225 1226
	}
	catch (...)
	{
		tryLogCurrentException(__PRETTY_FUNCTION__);
	}
M
Merge  
Michael Kolupaev 已提交
1227

M
Merge  
Michael Kolupaev 已提交
1228
	if (!entry)
M
Merge  
Michael Kolupaev 已提交
1229
		return false;
M
Merge  
Michael Kolupaev 已提交
1230

M
Merge  
Michael Kolupaev 已提交
1231
	bool exception = true;
M
Merge  
Michael Kolupaev 已提交
1232
	bool success = false;
M
Merge  
Michael Kolupaev 已提交
1233

M
Merge  
Michael Kolupaev 已提交
1234 1235
	try
	{
M
Merge  
Michael Kolupaev 已提交
1236
		if (executeLogEntry(*entry, pool_context))
M
Merge  
Michael Kolupaev 已提交
1237
		{
M
Merge  
Michael Kolupaev 已提交
1238
			auto code = zookeeper->tryRemove(replica_path + "/queue/" + entry->znode_name);
M
Merge  
Michael Kolupaev 已提交
1239

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

			success = true;
M
Merge  
Michael Kolupaev 已提交
1245
		}
M
Merge  
Michael Kolupaev 已提交
1246

M
Merge  
Michael Kolupaev 已提交
1247
		exception = false;
M
Merge  
Michael Kolupaev 已提交
1248 1249 1250 1251
	}
	catch (Exception & e)
	{
		if (e.code() == ErrorCodes::NO_REPLICA_HAS_PART)
M
Merge  
Michael Kolupaev 已提交
1252
			/// Если ни у кого нет нужного куска, наверно, просто не все реплики работают; не будем писать в лог с уровнем Error.
M
Merge  
Michael Kolupaev 已提交
1253 1254
			LOG_INFO(log, e.displayText());
		else
M
Merge  
Michael Kolupaev 已提交
1255
			tryLogCurrentException(__PRETTY_FUNCTION__);
M
Merge  
Michael Kolupaev 已提交
1256 1257 1258 1259 1260
	}
	catch (...)
	{
		tryLogCurrentException(__PRETTY_FUNCTION__);
	}
M
Merge  
Michael Kolupaev 已提交
1261

M
Merge  
Michael Kolupaev 已提交
1262 1263 1264 1265 1266 1267 1268 1269
	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 已提交
1270
	{
M
Merge  
Michael Kolupaev 已提交
1271 1272 1273
		/// Удалим задание из очереди.
		/// Нельзя просто обратиться по заранее сохраненному итератору, потому что задание мог успеть удалить кто-то другой.
		for (LogEntries::iterator it = queue.end(); it != queue.begin();)
M
Merge  
Michael Kolupaev 已提交
1274
		{
M
Merge  
Michael Kolupaev 已提交
1275 1276 1277
			--it;
			if (*it == entry)
			{
M
Merge  
Michael Kolupaev 已提交
1278
				queue.erase(it);
M
Merge  
Michael Kolupaev 已提交
1279 1280
				break;
			}
M
Merge  
Michael Kolupaev 已提交
1281
		}
M
Merge  
Michael Kolupaev 已提交
1282
	}
M
Merge  
Michael Kolupaev 已提交
1283

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

M
Merge  
Michael Kolupaev 已提交
1288 1289
void StorageReplicatedMergeTree::mergeSelectingThread()
{
M
Merge  
Michael Kolupaev 已提交
1290
	bool need_pull = true;
M
Merge  
Michael Kolupaev 已提交
1291 1292 1293

	while (!shutdown_called && is_leader_node)
	{
M
Michael Kolupaev 已提交
1294
		bool success = false;
M
Merge  
Michael Kolupaev 已提交
1295

M
Michael Kolupaev 已提交
1296
		try
M
Merge  
Michael Kolupaev 已提交
1297
		{
M
Merge  
Michael Kolupaev 已提交
1298 1299
			std::unique_lock<std::mutex> merge_selecting_lock(merge_selecting_mutex);

M
Merge  
Michael Kolupaev 已提交
1300 1301 1302 1303 1304 1305 1306 1307
			if (need_pull)
			{
				/// Нужно загрузить новую запись в очередь перед тем, как выбирать куски для слияния.
				///  (чтобы кусок добавился в virtual_parts).
				pullLogsToQueue();
				need_pull = false;
			}

M
Michael Kolupaev 已提交
1308
			size_t merges_queued = 0;
M
Merge  
Michael Kolupaev 已提交
1309
			/// Есть ли в очереди или в фоновом потоке мердж крупных кусков.
M
Merge  
Michael Kolupaev 已提交
1310
			bool has_big_merge = context.getBackgroundPool().getCounter("replicated big merges") > 0;
M
Merge  
Michael Kolupaev 已提交
1311

M
Merge  
Michael Kolupaev 已提交
1312
			if (!has_big_merge)
M
Michael Kolupaev 已提交
1313
			{
M
Merge  
Michael Kolupaev 已提交
1314
				std::unique_lock<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1315

M
Michael Kolupaev 已提交
1316
				for (const auto & entry : queue)
M
Merge  
Michael Kolupaev 已提交
1317
				{
M
Merge  
Michael Kolupaev 已提交
1318
					if (entry->type == LogEntry::MERGE_PARTS)
M
Merge  
Michael Kolupaev 已提交
1319
					{
M
Michael Kolupaev 已提交
1320
						++merges_queued;
M
Merge  
Michael Kolupaev 已提交
1321 1322 1323

						if (!has_big_merge)
						{
M
Merge  
Michael Kolupaev 已提交
1324
							for (const String & name : entry->parts_to_merge)
M
Merge  
Michael Kolupaev 已提交
1325
							{
M
Merge  
Michael Kolupaev 已提交
1326
								MergeTreeData::DataPartPtr part = data.getActiveContainingPart(name);
M
Merge  
Michael Kolupaev 已提交
1327 1328
								if (!part || part->name != name)
									continue;
M
Merge  
Michael Kolupaev 已提交
1329
								if (part->size_in_bytes > data.settings.max_bytes_to_merge_parts_small)
M
Merge  
Michael Kolupaev 已提交
1330 1331 1332 1333 1334 1335 1336 1337
								{
									has_big_merge = true;
									break;
								}
							}
						}
					}
				}
M
Michael Kolupaev 已提交
1338
			}
M
Merge  
Michael Kolupaev 已提交
1339

M
Merge  
Michael Kolupaev 已提交
1340
			do
M
Michael Kolupaev 已提交
1341
			{
M
Merge  
Michael Kolupaev 已提交
1342
				if (merges_queued >= data.settings.max_replicated_merges_in_queue)
M
Merge  
Michael Kolupaev 已提交
1343
					break;
M
Merge  
Michael Kolupaev 已提交
1344

M
Merge  
Michael Kolupaev 已提交
1345
				MergeTreeData::DataPartsVector parts;
M
Merge  
Michael Kolupaev 已提交
1346

M
Merge  
Michael Kolupaev 已提交
1347 1348 1349 1350
				String merged_name;
				auto can_merge = std::bind(
					&StorageReplicatedMergeTree::canMergeParts, this, std::placeholders::_1, std::placeholders::_2);

M
Merge  
Michael Kolupaev 已提交
1351 1352 1353
				if (!merger.selectPartsToMerge(parts, merged_name, MergeTreeDataMerger::NO_LIMIT,
												false, false, has_big_merge, can_merge) &&
					!merger.selectPartsToMerge(parts, merged_name, MergeTreeDataMerger::NO_LIMIT,
M
Merge  
Michael Kolupaev 已提交
1354
												true, false, has_big_merge, can_merge))
M
Merge  
Michael Kolupaev 已提交
1355
					break;
M
Merge  
Michael Kolupaev 已提交
1356

M
Merge  
Michael Kolupaev 已提交
1357 1358 1359 1360 1361 1362 1363 1364 1365 1366 1367 1368 1369 1370
				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 已提交
1371 1372 1373 1374
				LogEntry entry;
				entry.type = LogEntry::MERGE_PARTS;
				entry.source_replica = replica_name;
				entry.new_part_name = merged_name;
M
Merge  
Michael Kolupaev 已提交
1375

M
Merge  
Michael Kolupaev 已提交
1376 1377 1378
				for (const auto & part : parts)
				{
					entry.parts_to_merge.push_back(part->name);
M
Merge  
Michael Kolupaev 已提交
1379 1380
				}

M
Merge  
Michael Kolupaev 已提交
1381
				need_pull = true;
M
Merge  
Michael Kolupaev 已提交
1382

M
Merge  
Michael Kolupaev 已提交
1383
				zookeeper->create(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential);
M
Merge  
Michael Kolupaev 已提交
1384 1385 1386

				String month_name = parts[0]->name.substr(0, 6);
				for (size_t i = 0; i + 1 < parts.size(); ++i)
M
Merge  
Michael Kolupaev 已提交
1387
				{
M
Merge  
Michael Kolupaev 已提交
1388 1389 1390
					/// Уберем больше не нужные отметки о несуществующих блоках.
					for (UInt64 number = parts[i]->right + 1; number <= parts[i + 1]->left - 1; ++number)
					{
M
Merge  
Michael Kolupaev 已提交
1391 1392
						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 已提交
1393
					}
M
Merge  
Michael Kolupaev 已提交
1394
				}
M
Merge  
Michael Kolupaev 已提交
1395 1396

				success = true;
M
Merge  
Michael Kolupaev 已提交
1397
			}
M
Merge  
Michael Kolupaev 已提交
1398
			while(false);
M
Merge  
Michael Kolupaev 已提交
1399 1400 1401 1402 1403 1404
		}
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);
		}

M
Merge  
Michael Kolupaev 已提交
1405
		if (shutdown_called || !is_leader_node)
M
Merge  
Michael Kolupaev 已提交
1406 1407
			break;

M
Merge  
Michael Kolupaev 已提交
1408
		if (!success)
M
Merge  
Michael Kolupaev 已提交
1409
			merge_selecting_event.tryWait(MERGE_SELECTING_SLEEP_MS);
M
Merge  
Michael Kolupaev 已提交
1410
	}
M
Merge  
Michael Kolupaev 已提交
1411 1412

	LOG_DEBUG(log, "merge selecting thread finished");
M
Merge  
Michael Kolupaev 已提交
1413 1414
}

M
Merge  
Michael Kolupaev 已提交
1415
void StorageReplicatedMergeTree::cleanupThread()
M
Merge  
Michael Kolupaev 已提交
1416
{
M
Merge  
Michael Kolupaev 已提交
1417
	while (!shutdown_called)
M
Merge  
Michael Kolupaev 已提交
1418 1419
	{
		try
M
Merge  
Michael Kolupaev 已提交
1420
		{
M
Merge  
Michael Kolupaev 已提交
1421 1422
			clearOldParts();

M
Merge  
Michael Kolupaev 已提交
1423 1424 1425
			if (unreplicated_data)
				unreplicated_data->clearOldParts();

M
Merge  
Michael Kolupaev 已提交
1426 1427 1428 1429 1430
			if (is_leader_node)
			{
				clearOldLogs();
				clearOldBlocks();
			}
M
Merge  
Michael Kolupaev 已提交
1431
		}
M
Merge  
Michael Kolupaev 已提交
1432 1433 1434 1435
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);
		}
M
Merge  
Michael Kolupaev 已提交
1436

M
Merge  
Michael Kolupaev 已提交
1437
		shutdown_event.tryWait(CLEANUP_SLEEP_MS);
M
Merge  
Michael Kolupaev 已提交
1438
	}
M
Merge  
Michael Kolupaev 已提交
1439 1440

	LOG_DEBUG(log, "cleanup thread finished");
M
Merge  
Michael Kolupaev 已提交
1441 1442
}

M
Merge  
Michael Kolupaev 已提交
1443 1444 1445 1446 1447 1448 1449 1450 1451 1452 1453 1454 1455 1456 1457 1458 1459 1460 1461 1462 1463 1464
void StorageReplicatedMergeTree::alterThread()
{
	bool force_recheck_parts = true;

	while (!shutdown_called)
	{
		try
		{
			zkutil::Stat stat;
			String columns_str = zookeeper->get(zookeeper_path + "/columns", &stat, alter_thread_event);
			NamesAndTypesList columns = NamesAndTypesList::parse(columns_str, context.getDataTypeFactory());

			bool changed = false;

			/// Проверим, что описание столбцов изменилось.
			/// Чтобы не останавливать лишний раз все запросы в таблицу, проверим сначала под локом на чтение.
			{
				auto table_lock = lockStructure(false);
				if (columns != data.getColumnsList())
					changed = true;
			}

M
Merge  
Michael Kolupaev 已提交
1465 1466
			MergeTreeData::DataParts parts;

M
Merge  
Michael Kolupaev 已提交
1467 1468 1469 1470 1471 1472 1473 1474 1475
			/// Если описание столбцов изменилось, обновим структуру таблицы локально.
			if (changed)
			{
				auto table_lock = lockStructureForAlter();
				if (columns != data.getColumnsList())
				{
					LOG_INFO(log, "Columns list changed in ZooKeeper. Applying changes locally.");
					InterpreterAlterQuery::updateMetadata(database_name, table_name, columns, context);
					data.setColumnsList(columns);
M
Merge  
Michael Kolupaev 已提交
1476 1477
					if (unreplicated_data)
						unreplicated_data->setColumnsList(columns);
M
Merge  
Michael Kolupaev 已提交
1478 1479
					columns_version = stat.version;
					LOG_INFO(log, "Applied changes to table.");
M
Merge  
Michael Kolupaev 已提交
1480 1481 1482

					/// Нужно получить список кусков под блокировкой таблицы, чтобы избежать race condition с мерджем.
					parts = data.getDataParts();
M
Merge  
Michael Kolupaev 已提交
1483 1484 1485 1486 1487 1488 1489 1490 1491 1492 1493 1494 1495 1496 1497
				}
				else
				{
					changed = false;
				}
			}

			/// Обновим куски.
			if (changed || force_recheck_parts)
			{
				if (changed)
					LOG_INFO(log, "ALTER-ing parts");

				int changed_parts = 0;

M
Merge  
Michael Kolupaev 已提交
1498 1499 1500 1501
				if (!changed)
					parts = data.getDataParts();

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

M
Merge  
Michael Kolupaev 已提交
1503 1504 1505
				for (const MergeTreeData::DataPartPtr & part : parts)
				{
					/// Обновим кусок и запишем результат во временные файлы.
M
Merge  
Michael Kolupaev 已提交
1506 1507
					/// TODO: Можно пропускать проверку на слишком большие изменения, если в ZooKeeper есть, например,
					///  нода /flags/force_alter.
M
Merge  
Michael Kolupaev 已提交
1508 1509 1510 1511 1512 1513 1514 1515 1516 1517 1518 1519 1520 1521 1522 1523 1524
					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 已提交
1525 1526 1527 1528 1529 1530 1531 1532 1533 1534 1535 1536 1537 1538 1539 1540 1541
				/// То же самое для нереплицируемых данных.
				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 已提交
1542 1543 1544

				zookeeper->set(replica_path + "/columns", columns.toString());

M
Merge  
Michael Kolupaev 已提交
1545 1546 1547 1548 1549 1550 1551 1552 1553 1554 1555 1556 1557 1558 1559 1560 1561 1562 1563 1564
				if (changed || changed_parts != 0)
					LOG_INFO(log, "ALTER-ed " << changed_parts << " parts");
				force_recheck_parts = false;
			}

			alter_thread_event->wait();
		}
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);

			force_recheck_parts = true;

			alter_thread_event->tryWait(ERROR_SLEEP_MS);
		}
	}

	LOG_DEBUG(log, "alter thread finished");
}

M
Merge  
Michael Kolupaev 已提交
1565 1566 1567 1568
void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_name)
{
	String part_path = replica_path + "/parts/" + part_name;

M
Merge  
Michael Kolupaev 已提交
1569 1570 1571 1572
	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 已提交
1573 1574 1575

	zkutil::Ops ops;
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
1576
		replica_path + "/queue/queue-", log_entry->toString(), zookeeper->getDefaultACL(),
M
Merge  
Michael Kolupaev 已提交
1577 1578 1579 1580 1581 1582 1583
		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 已提交
1584
		std::unique_lock<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1585 1586

		String path_created = dynamic_cast<zkutil::Op::Create &>(ops[0]).getPathCreated();
M
Merge  
Michael Kolupaev 已提交
1587 1588
		log_entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
		log_entry->addResultToVirtualParts(*this);
M
Merge  
Michael Kolupaev 已提交
1589 1590 1591 1592 1593 1594 1595 1596 1597 1598 1599 1600 1601 1602 1603
		queue.push_back(log_entry);
	}
}

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

	if (parts_to_check_set.count(name))
		return;
	parts_to_check_queue.push_back(name);
	parts_to_check_set.insert(name);
	parts_to_check_event.set();
}

M
Merge  
Michael Kolupaev 已提交
1604 1605 1606 1607 1608 1609 1610 1611 1612 1613 1614 1615 1616 1617 1618 1619 1620 1621 1622 1623 1624 1625 1626 1627 1628 1629 1630 1631 1632 1633
void StorageReplicatedMergeTree::partCheckThread()
{
	while (!shutdown_called)
	{
		try
		{
			/// Достанем из очереди кусок для проверки.
			String part_name;
			{
				Poco::ScopedLock<Poco::FastMutex> lock(parts_to_check_mutex);
				if (parts_to_check_queue.empty())
				{
					if (!parts_to_check_set.empty())
					{
						LOG_ERROR(log, "Non-empty parts_to_check_set with empty parts_to_check_queue. This is a bug.");
						parts_to_check_set.clear();
					}
				}
				else
				{
					part_name = parts_to_check_queue.front();
				}
			}
			if (part_name.empty())
			{
				parts_to_check_event.wait();
				continue;
			}

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

M
Merge  
Michael Kolupaev 已提交
1636
			auto part = data.getActiveContainingPart(part_name);
M
Merge  
Michael Kolupaev 已提交
1637 1638 1639 1640 1641 1642 1643 1644 1645 1646
			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 已提交
1647
					ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
M
Merge  
Michael Kolupaev 已提交
1648

M
Merge  
Michael Kolupaev 已提交
1649
					removePartAndEnqueueFetch(part_name);
M
Merge  
Michael Kolupaev 已提交
1650 1651 1652 1653
				}
				/// Если куска нет в ZooKeeper, проверим есть ли он хоть у кого-то.
				else
				{
M
Merge  
Michael Kolupaev 已提交
1654 1655
					ActiveDataPartSet::Part part_info;
					ActiveDataPartSet::parsePartName(part_name, part_info);
M
Merge  
Michael Kolupaev 已提交
1656

M
Merge  
Michael Kolupaev 已提交
1657
					/** Будем проверять только куски, не полученные в результате слияния.
M
Merge  
Michael Kolupaev 已提交
1658
					  * Для кусков, полученных в результате слияния, такая проверка была бы некорректной,
M
Merge  
Michael Kolupaev 已提交
1659 1660
					  *  потому что слитого куска может еще ни у кого не быть.
					  */
M
Merge  
Michael Kolupaev 已提交
1661
					if (part_info.left == part_info.right)
M
Merge  
Michael Kolupaev 已提交
1662
					{
M
Merge  
Michael Kolupaev 已提交
1663 1664 1665 1666 1667
						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 已提交
1668
						{
M
Merge  
Michael Kolupaev 已提交
1669 1670
							Strings parts = zookeeper->getChildren(zookeeper_path + "/replicas/" + replica + "/parts");
							for (const String & part_on_replica : parts)
M
Merge  
Michael Kolupaev 已提交
1671
							{
M
Merge  
Michael Kolupaev 已提交
1672 1673 1674 1675 1676 1677
								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 已提交
1678
							}
M
Merge  
Michael Kolupaev 已提交
1679 1680
							if (found)
								break;
M
Merge  
Michael Kolupaev 已提交
1681 1682
						}

M
Merge  
Michael Kolupaev 已提交
1683
						if (!found)
M
Merge  
Michael Kolupaev 已提交
1684
						{
M
Merge  
Michael Kolupaev 已提交
1685
							LOG_ERROR(log, "No replica has part covering " << part_name);
M
Merge  
Michael Kolupaev 已提交
1686 1687
							ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);

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

M
Merge  
Michael Kolupaev 已提交
1690 1691
							bool was_in_queue = false;

M
Merge  
Michael Kolupaev 已提交
1692
							{
M
Merge  
Michael Kolupaev 已提交
1693
								std::unique_lock<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1694 1695

								for (LogEntries::iterator it = queue.begin(); it != queue.end(); )
M
Merge  
Michael Kolupaev 已提交
1696
								{
M
Merge  
Michael Kolupaev 已提交
1697
									if ((*it)->new_part_name == part_name)
M
Merge  
Michael Kolupaev 已提交
1698
									{
M
Merge  
Michael Kolupaev 已提交
1699
										zookeeper->tryRemove(replica_path + "/queue/" + (*it)->znode_name);
M
Merge  
Michael Kolupaev 已提交
1700
										queue.erase(it++);
M
Merge  
Michael Kolupaev 已提交
1701
										was_in_queue = true;
M
Merge  
Michael Kolupaev 已提交
1702 1703 1704 1705 1706
									}
									else
									{
										++it;
									}
M
Merge  
Michael Kolupaev 已提交
1707 1708
								}
							}
M
Merge  
Michael Kolupaev 已提交
1709 1710

							if (was_in_queue)
M
Merge  
Michael Kolupaev 已提交
1711
							{
M
Merge  
Michael Kolupaev 已提交
1712 1713 1714 1715
								/** Такая ситуация возможна, если на всех репликах, где был кусок, он испортился.
								  * Например, у реплики, которая только что его записала, отключили питание, и данные не записались из кеша на диск.
								  */
								LOG_ERROR(log, "Part " << part_name << " is lost forever. Say goodbye to a piece of data!");
M
Merge  
Michael Kolupaev 已提交
1716 1717 1718 1719 1720 1721 1722 1723 1724 1725 1726 1727 1728

								/** Нужно добавить отсутствующий кусок в 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 已提交
1729 1730
						}
					}
M
Merge  
Michael Kolupaev 已提交
1731 1732 1733 1734 1735
				}
			}
			/// У нас есть этот кусок, и он активен.
			else if (part->name == part_name)
			{
M
Merge  
Michael Kolupaev 已提交
1736 1737
				auto table_lock = lockStructure(false);

M
Merge  
Michael Kolupaev 已提交
1738 1739 1740
				/// Если кусок есть в ZooKeeper, сверим его данные с его чексуммами, а их с ZooKeeper.
				if (zookeeper->exists(replica_path + "/parts/" + part_name))
				{
M
Merge  
Michael Kolupaev 已提交
1741 1742 1743 1744 1745 1746 1747 1748 1749 1750 1751 1752 1753
					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 已提交
1754 1755 1756 1757
						MergeTreePartChecker::Settings settings;
						settings.setIndexGranularity(data.index_granularity);
						settings.setRequireChecksums(true);
						settings.setRequireColumnFiles(true);
M
Merge  
Michael Kolupaev 已提交
1758
						MergeTreePartChecker::checkDataPart(
M
Merge  
Michael Kolupaev 已提交
1759
							data.getFullPath() + part_name, settings, context.getDataTypeFactory());
M
Merge  
Michael Kolupaev 已提交
1760

M
Merge  
Michael Kolupaev 已提交
1761 1762 1763 1764 1765 1766
						LOG_INFO(log, "Part " << part_name << " looks good.");
					}
					catch (...)
					{
						tryLogCurrentException(__PRETTY_FUNCTION__);

M
Merge  
Michael Kolupaev 已提交
1767 1768
						LOG_ERROR(log, "Part " << part_name << " looks broken. Removing it and queueing a fetch.");
						ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
M
Merge  
Michael Kolupaev 已提交
1769 1770 1771 1772

						removePartAndEnqueueFetch(part_name);

						/// Удалим кусок локально.
M
Merge  
Michael Kolupaev 已提交
1773
						data.renameAndDetachPart(part, "broken_");
M
Merge  
Michael Kolupaev 已提交
1774
					}
M
Merge  
Michael Kolupaev 已提交
1775 1776
				}
				/// Если куска нет в ZooKeeper, удалим его локально.
M
Merge  
Michael Kolupaev 已提交
1777 1778 1779
				/// Возможно, кусок кто-то только что записал, и еще не успел добавить в ZK.
				/// Поэтому удаляем только если кусок старый (не очень надежно).
				else if (part->modification_time + 5 * 60 < time(0))
M
Merge  
Michael Kolupaev 已提交
1780
				{
M
Merge  
Michael Kolupaev 已提交
1781 1782
					ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);

M
Merge  
Michael Kolupaev 已提交
1783 1784
					LOG_ERROR(log, "Unexpected part " << part_name << ". Removing.");
					data.renameAndDetachPart(part, "unexpected_");
M
Merge  
Michael Kolupaev 已提交
1785 1786 1787 1788 1789 1790 1791 1792 1793 1794 1795 1796 1797 1798 1799 1800 1801 1802 1803 1804 1805 1806 1807 1808 1809 1810 1811 1812 1813 1814 1815
				}
			}
			else
			{
				/// Если у нас есть покрывающий кусок, игнорируем все проблемы с этим куском.
				/// В худшем случае в лог еще old_parts_lifetime секунд будут валиться ошибки, пока кусок не удалится как старый.
			}

			/// Удалим кусок из очереди.
			{
				Poco::ScopedLock<Poco::FastMutex> lock(parts_to_check_mutex);
				if (parts_to_check_queue.empty() || parts_to_check_queue.front() != part_name)
				{
					LOG_ERROR(log, "Someone changed parts_to_check_queue.front(). This is a bug.");
				}
				else
				{
					parts_to_check_queue.pop_front();
					parts_to_check_set.erase(part_name);
				}
			}
		}
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);
			parts_to_check_event.tryWait(ERROR_SLEEP_MS);
		}
	}
}


M
Merge  
Michael Kolupaev 已提交
1816 1817
bool StorageReplicatedMergeTree::canMergeParts(const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right)
{
M
Merge  
Michael Kolupaev 已提交
1818 1819 1820
	/// Если какой-то из кусков уже собираются слить в больший, не соглашаемся его сливать.
	if (virtual_parts.getContainingPart(left->name) != left->name ||
		virtual_parts.getContainingPart(right->name) != right->name)
M
Merge  
Michael Kolupaev 已提交
1821 1822
		return false;

M
Merge  
Michael Kolupaev 已提交
1823 1824
	String month_name = left->name.substr(0, 6);

M
Merge  
Michael Kolupaev 已提交
1825 1826 1827
	/// Можно слить куски, если все номера между ними заброшены - не соответствуют никаким блокам.
	for (UInt64 number = left->right + 1; number <= right->left - 1; ++number)
	{
M
Merge  
Michael Kolupaev 已提交
1828 1829
		String path1 = zookeeper_path +              "/block_numbers/" + month_name + "/block-" + padIndex(number);
		String path2 = zookeeper_path + "/nonincrement_block_numbers/" + month_name + "/block-" + padIndex(number);
M
Merge  
Michael Kolupaev 已提交
1830

M
Merge  
Michael Kolupaev 已提交
1831 1832
		if (AbandonableLockInZooKeeper::check(path1, *zookeeper) != AbandonableLockInZooKeeper::ABANDONED &&
			AbandonableLockInZooKeeper::check(path2, *zookeeper) != AbandonableLockInZooKeeper::ABANDONED)
M
Merge  
Michael Kolupaev 已提交
1833 1834 1835 1836 1837 1838 1839 1840
			return false;
	}

	return true;
}

void StorageReplicatedMergeTree::becomeLeader()
{
M
Merge  
Michael Kolupaev 已提交
1841
	LOG_INFO(log, "Became leader");
M
Merge  
Michael Kolupaev 已提交
1842 1843 1844 1845
	is_leader_node = true;
	merge_selecting_thread = std::thread(&StorageReplicatedMergeTree::mergeSelectingThread, this);
}

M
Merge  
Michael Kolupaev 已提交
1846
String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_name, bool active)
M
Merge  
Michael Kolupaev 已提交
1847
{
M
Merge  
Michael Kolupaev 已提交
1848
	Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
M
Merge  
Michael Kolupaev 已提交
1849 1850 1851 1852 1853 1854

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

	for (const String & replica : replicas)
	{
M
Merge  
Michael Kolupaev 已提交
1855 1856
		if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name) &&
			(!active || zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")))
M
Merge  
Michael Kolupaev 已提交
1857 1858 1859
			return replica;
	}

M
Merge  
Michael Kolupaev 已提交
1860
	return "";
M
Merge  
Michael Kolupaev 已提交
1861 1862 1863 1864
}

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

M
Merge  
Michael Kolupaev 已提交
1867 1868 1869 1870 1871
	auto table_lock = lockStructure(true);

	String host;
	int port;

M
Merge  
Michael Kolupaev 已提交
1872
	String host_port_str = zookeeper->get(zookeeper_path + "/replicas/" + replica_name + "/host");
M
Merge  
Michael Kolupaev 已提交
1873 1874 1875 1876 1877 1878 1879 1880
	ReadBufferFromString buf(host_port_str);
	assertString("host: ", buf);
	readString(host, buf);
	assertString("\nport: ", buf);
	readText(port, buf);
	assertString("\n", buf);
	assertEOF(buf);

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

M
Merge  
Michael Kolupaev 已提交
1883 1884 1885
	zkutil::Ops ops;
	checkPartAndAddToZooKeeper(part, ops, part_name);

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

M
Merge  
Michael Kolupaev 已提交
1889
	zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
1890
	transaction.commit();
M
Merge  
Michael Kolupaev 已提交
1891
	merge_selecting_event.set();
M
Merge  
Michael Kolupaev 已提交
1892

M
Michael Kolupaev 已提交
1893 1894 1895 1896 1897 1898
	for (const auto & removed_part : removed_parts)
	{
		LOG_DEBUG(log, "Part " << removed_part->name << " is rendered obsolete by fetching part " << part_name);
		ProfileEvents::increment(ProfileEvents::ObsoleteReplicatedParts);
	}

M
Merge  
Michael Kolupaev 已提交
1899 1900
	ProfileEvents::increment(ProfileEvents::ReplicatedPartFetches);

M
Merge  
Michael Kolupaev 已提交
1901
	LOG_DEBUG(log, "Fetched part " << part_name << " from " << replica_name);
M
Merge  
Michael Kolupaev 已提交
1902
}
M
Merge  
Michael Kolupaev 已提交
1903

M
Merge  
Michael Kolupaev 已提交
1904 1905
void StorageReplicatedMergeTree::shutdown()
{
M
Merge  
Michael Kolupaev 已提交
1906
	if (permanent_shutdown_called)
M
Merge  
Michael Kolupaev 已提交
1907 1908 1909
	{
		if (restarting_thread.joinable())
			restarting_thread.join();
M
Merge  
Michael Kolupaev 已提交
1910
		return;
M
Merge  
Michael Kolupaev 已提交
1911
	}
M
Merge  
Michael Kolupaev 已提交
1912

M
Merge  
Michael Kolupaev 已提交
1913
	permanent_shutdown_called = true;
M
Merge  
Michael Kolupaev 已提交
1914
	restarting_event.set();
M
Merge  
Michael Kolupaev 已提交
1915
	restarting_thread.join();
M
Merge  
Michael Kolupaev 已提交
1916 1917

	endpoint_holder = nullptr;
M
Merge  
Michael Kolupaev 已提交
1918 1919 1920 1921
}

void StorageReplicatedMergeTree::partialShutdown()
{
M
Merge  
Michael Kolupaev 已提交
1922
	leader_election = nullptr;
M
Merge  
Michael Kolupaev 已提交
1923
	shutdown_called = true;
M
Merge  
Michael Kolupaev 已提交
1924
	shutdown_event.set();
M
Merge  
Michael Kolupaev 已提交
1925 1926 1927
	merge_selecting_event.set();
	queue_updating_event->set();
	alter_thread_event->set();
M
Merge  
Michael Kolupaev 已提交
1928
	alter_query_event->set();
M
Merge  
Michael Kolupaev 已提交
1929
	parts_to_check_event.set();
M
Merge  
Michael Kolupaev 已提交
1930 1931
	replica_is_active_node = nullptr;

M
Merge  
Michael Kolupaev 已提交
1932 1933 1934 1935
	merger.cancelAll();
	if (unreplicated_merger)
		unreplicated_merger->cancelAll();

M
Merge  
Michael Kolupaev 已提交
1936
	LOG_TRACE(log, "Waiting for threads to finish");
M
Merge  
Michael Kolupaev 已提交
1937
	if (is_leader_node)
M
Merge  
Michael Kolupaev 已提交
1938
	{
M
Merge  
Michael Kolupaev 已提交
1939
		is_leader_node = false;
M
Merge  
Michael Kolupaev 已提交
1940 1941
		if (merge_selecting_thread.joinable())
			merge_selecting_thread.join();
M
Merge  
Michael Kolupaev 已提交
1942
	}
M
Merge  
Michael Kolupaev 已提交
1943 1944
	if (queue_updating_thread.joinable())
		queue_updating_thread.join();
M
Merge  
Michael Kolupaev 已提交
1945 1946 1947 1948
	if (cleanup_thread.joinable())
		cleanup_thread.join();
	if (alter_thread.joinable())
		alter_thread.join();
M
Merge  
Michael Kolupaev 已提交
1949 1950
	if (part_check_thread.joinable())
		part_check_thread.join();
M
Merge  
Michael Kolupaev 已提交
1951 1952
	if (queue_task_handle)
		context.getBackgroundPool().removeTask(queue_task_handle);
M
Merge  
Michael Kolupaev 已提交
1953
	queue_task_handle.reset();
M
Merge  
Michael Kolupaev 已提交
1954
	LOG_TRACE(log, "Threads finished");
M
Merge  
Michael Kolupaev 已提交
1955 1956
}

M
Merge  
Michael Kolupaev 已提交
1957
void StorageReplicatedMergeTree::goReadOnlyPermanently()
M
Merge  
Michael Kolupaev 已提交
1958 1959 1960 1961 1962
{
	LOG_INFO(log, "Going to read-only mode");

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

M
Merge  
Michael Kolupaev 已提交
1965
	partialShutdown();
M
Merge  
Michael Kolupaev 已提交
1966 1967
}

M
Merge  
Michael Kolupaev 已提交
1968
bool StorageReplicatedMergeTree::tryStartup()
M
Merge  
Michael Kolupaev 已提交
1969
{
M
Merge  
Michael Kolupaev 已提交
1970 1971 1972
	try
	{
		activateReplica();
M
Merge  
Michael Kolupaev 已提交
1973

M
Merge  
Michael Kolupaev 已提交
1974 1975 1976 1977 1978
		leader_election = new zkutil::LeaderElection(zookeeper_path + "/leader_election", *zookeeper,
			std::bind(&StorageReplicatedMergeTree::becomeLeader, this), replica_name);

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

M
Merge  
Michael Kolupaev 已提交
1980 1981
		shutdown_called = false;
		shutdown_event.reset();
M
Merge  
Michael Kolupaev 已提交
1982

M
Merge  
Michael Kolupaev 已提交
1983 1984 1985
		merger.uncancelAll();
		if (unreplicated_merger)
			unreplicated_merger->uncancelAll();
M
Merge  
Michael Kolupaev 已提交
1986

M
Merge  
Michael Kolupaev 已提交
1987 1988 1989 1990 1991 1992
		queue_updating_thread = std::thread(&StorageReplicatedMergeTree::queueUpdatingThread, this);
		cleanup_thread = std::thread(&StorageReplicatedMergeTree::cleanupThread, this);
		alter_thread = std::thread(&StorageReplicatedMergeTree::alterThread, this);
		part_check_thread = std::thread(&StorageReplicatedMergeTree::partCheckThread, this);
		queue_task_handle = context.getBackgroundPool().addTask(
			std::bind(&StorageReplicatedMergeTree::queueTask, this, std::placeholders::_1));
M
Merge  
Michael Kolupaev 已提交
1993
		queue_task_handle->wake();
M
Merge  
Michael Kolupaev 已提交
1994 1995 1996 1997 1998 1999 2000 2001 2002 2003 2004 2005 2006 2007 2008 2009
		return true;
	}
	catch (zkutil::KeeperException & e)
	{
		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;
	}
	catch (...)
	{
		replica_is_active_node = nullptr;
		leader_election = nullptr;
		throw;
	}
M
Merge  
Michael Kolupaev 已提交
2010 2011
}

M
Merge  
Michael Kolupaev 已提交
2012 2013
void StorageReplicatedMergeTree::restartingThread()
{
M
Merge  
Michael Kolupaev 已提交
2014
	try
M
Merge  
Michael Kolupaev 已提交
2015
	{
M
Merge  
Michael Kolupaev 已提交
2016 2017
		while (!permanent_shutdown_called && !tryStartup())
			restarting_event.tryWait(10 * 1000);
M
Merge  
Michael Kolupaev 已提交
2018 2019

		while (!permanent_shutdown_called)
M
Merge  
Michael Kolupaev 已提交
2020
		{
M
Merge  
Michael Kolupaev 已提交
2021 2022 2023
			if (zookeeper->expired())
			{
				LOG_WARNING(log, "ZooKeeper session has expired. Switching to a new session.");
M
Merge  
Michael Kolupaev 已提交
2024

M
Merge  
Michael Kolupaev 已提交
2025 2026 2027 2028 2029
				{
					/// Запретим писать в таблицу, пока подменяем zookeeper.
					auto structure_lock = lockDataForAlter();

					partialShutdown();
M
Merge  
Michael Kolupaev 已提交
2030

M
Merge  
Michael Kolupaev 已提交
2031
					zookeeper = context.getZooKeeper();
M
Merge  
Michael Kolupaev 已提交
2032

M
Merge  
Michael Kolupaev 已提交
2033 2034 2035 2036 2037 2038 2039 2040
					is_read_only = true;
				}

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

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

M
Merge  
Michael Kolupaev 已提交
2042 2043 2044 2045 2046
				{
					auto structure_lock = lockDataForAlter();

					is_read_only = false;
				}
M
Merge  
Michael Kolupaev 已提交
2047
			}
M
Merge  
Michael Kolupaev 已提交
2048

M
Merge  
Michael Kolupaev 已提交
2049
			restarting_event.tryWait(60 * 1000);
M
Merge  
Michael Kolupaev 已提交
2050 2051 2052 2053 2054
		}
	}
	catch (...)
	{
		tryLogCurrentException("StorageReplicatedMergeTree::restartingThread");
M
Merge  
Michael Kolupaev 已提交
2055 2056
		LOG_ERROR(log, "Unexpected exception in restartingThread. The storage will be read-only until server restart.");
		goReadOnlyPermanently();
M
Merge  
Michael Kolupaev 已提交
2057
		LOG_DEBUG(log, "restarting thread finished");
M
Merge  
Michael Kolupaev 已提交
2058
		return;
M
Merge  
Michael Kolupaev 已提交
2059 2060
	}

M
Merge  
Michael Kolupaev 已提交
2061 2062 2063 2064 2065 2066 2067 2068 2069
	try
	{
		endpoint_holder = nullptr;
		partialShutdown();
	}
	catch (...)
	{
		tryLogCurrentException("StorageReplicatedMergeTree::restartingThread");
	}
M
Merge  
Michael Kolupaev 已提交
2070 2071

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

M
Merge  
Michael Kolupaev 已提交
2074 2075 2076 2077 2078 2079 2080 2081 2082 2083 2084 2085 2086 2087 2088 2089 2090 2091 2092 2093
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 已提交
2094 2095 2096 2097 2098 2099 2100 2101 2102 2103 2104 2105 2106 2107 2108 2109 2110
	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())
2111
		VirtualColumnUtils::filterBlockWithQuery(query->clone(), virtual_columns_block, context);
M
Merge  
Michael Kolupaev 已提交
2112

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

M
Merge  
Michael Kolupaev 已提交
2115 2116
	BlockInputStreams res;

M
Merge  
Michael Kolupaev 已提交
2117 2118 2119
	size_t part_index = 0;

	if (unreplicated_reader && values.count(0))
M
Merge  
Michael Kolupaev 已提交
2120
	{
M
Merge  
Michael Kolupaev 已提交
2121 2122
		res = unreplicated_reader->read(
			real_column_names, query, settings, processed_stage, max_block_size, threads, &part_index);
M
Merge  
Michael Kolupaev 已提交
2123 2124 2125 2126 2127 2128

		for (auto & virtual_column : virt_column_names)
		{
			if (virtual_column == "_replicated")
			{
				for (auto & stream : res)
M
Merge  
Michael Kolupaev 已提交
2129
					stream = new AddingConstColumnBlockInputStream<UInt8>(stream, new DataTypeUInt8, 0, "_replicated");
M
Merge  
Michael Kolupaev 已提交
2130 2131 2132 2133
			}
		}
	}

M
Merge  
Michael Kolupaev 已提交
2134
	if (values.count(1))
M
Merge  
Michael Kolupaev 已提交
2135
	{
M
Merge  
Michael Kolupaev 已提交
2136
		auto res2 = reader.read(real_column_names, query, settings, processed_stage, max_block_size, threads, &part_index);
M
Merge  
Michael Kolupaev 已提交
2137 2138 2139 2140 2141 2142

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

M
Merge  
Michael Kolupaev 已提交
2147
		res.insert(res.end(), res2.begin(), res2.end());
M
Merge  
Michael Kolupaev 已提交
2148 2149 2150
	}

	return res;
M
Merge  
Michael Kolupaev 已提交
2151 2152
}

M
Merge  
Michael Kolupaev 已提交
2153 2154
BlockOutputStreamPtr StorageReplicatedMergeTree::write(ASTPtr query)
{
M
Merge  
Michael Kolupaev 已提交
2155 2156 2157
	if (is_read_only)
		throw Exception("Table is in read only mode", ErrorCodes::TABLE_IS_READ_ONLY);

M
Merge  
Michael Kolupaev 已提交
2158
	String insert_id;
M
Merge  
Michael Kolupaev 已提交
2159 2160 2161
	if (query)
		if (ASTInsertQuery * insert = typeid_cast<ASTInsertQuery *>(&*query))
			insert_id = insert->insert_id;
M
Merge  
Michael Kolupaev 已提交
2162 2163 2164

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

M
Merge  
Michael Kolupaev 已提交
2166 2167
bool StorageReplicatedMergeTree::optimize()
{
2168 2169
	/// Померджим какие-нибудь куски из директории unreplicated.
	/// TODO: Мерджить реплицируемые куски тоже.
M
Merge  
Michael Kolupaev 已提交
2170 2171 2172 2173

	if (!unreplicated_data)
		return false;

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

M
Merge  
Michael Kolupaev 已提交
2176 2177 2178 2179 2180 2181 2182 2183 2184 2185 2186 2187
	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;

	unreplicated_merger->mergeParts(parts, merged_name);
	return true;
}

M
Merge  
Michael Kolupaev 已提交
2188 2189
void StorageReplicatedMergeTree::alter(const AlterCommands & params,
	const String & database_name, const String & table_name, Context & context)
M
Merge  
Michael Kolupaev 已提交
2190 2191 2192 2193 2194 2195 2196 2197 2198 2199 2200
{
	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 已提交
2201 2202 2203
		if (is_read_only)
			throw Exception("Can't ALTER read-only table", ErrorCodes::TABLE_IS_READ_ONLY);

M
Merge  
Michael Kolupaev 已提交
2204 2205 2206 2207 2208 2209 2210 2211 2212 2213 2214 2215 2216 2217 2218 2219 2220 2221 2222 2223 2224 2225 2226 2227 2228 2229 2230 2231 2232 2233 2234 2235 2236 2237 2238 2239 2240 2241 2242 2243 2244 2245 2246 2247 2248 2249 2250 2251 2252 2253 2254 2255 2256 2257 2258 2259 2260
		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);
	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);
			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 已提交
2261
			if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/columns", &stat, alter_query_event))
M
Merge  
Michael Kolupaev 已提交
2262 2263 2264 2265 2266 2267 2268 2269 2270 2271 2272 2273 2274 2275 2276 2277 2278 2279
			{
				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 已提交
2280 2281 2282 2283 2284 2285 2286 2287 2288 2289 2290 2291 2292 2293 2294 2295 2296 2297 2298 2299 2300 2301 2302 2303 2304 2305
static bool isValidMonthName(const String & s)
{
	if (s.size() != 6)
		return false;
	if (!std::all_of(s.begin(), s.end(), isdigit))
		return false;
	DayNum_t date = DateLUT::instance().toDayNum(OrderedIdentifier2Date(s + "01"));
	/// Не можем просто сравнить date с нулем, потому что 0 тоже валидный DayNum.
	return s == toString(Date2OrderedIdentifier(DateLUT::instance().fromDayNum(date)) / 100);
}

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

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

void StorageReplicatedMergeTree::dropPartition(const Field & field, bool detach)
{
M
Merge  
Michael Kolupaev 已提交
2306
	String month_name = field.getType() == Field::Types::UInt64 ? toString(field.get<UInt64>()) : field.safeGet<String>();
M
Merge  
Michael Kolupaev 已提交
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

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

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


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

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

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

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

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

		virtual_parts.add(fake_part_name);
	}

M
Merge  
Michael Kolupaev 已提交
2347
	/// Наконец, добившись нужных инвариантов, можно положить запись в лог.
M
Merge  
Michael Kolupaev 已提交
2348 2349 2350 2351 2352 2353
	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 已提交
2354
	entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);
M
Merge  
Michael Kolupaev 已提交
2355 2356

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

M
Merge  
Michael Kolupaev 已提交
2360
void StorageReplicatedMergeTree::attachPartition(const Field & field, bool unreplicated, bool attach_part)
M
Merge  
Michael Kolupaev 已提交
2361
{
M
Merge  
Michael Kolupaev 已提交
2362 2363 2364 2365 2366 2367 2368 2369 2370 2371 2372 2373 2374 2375 2376 2377 2378 2379 2380 2381 2382 2383 2384 2385 2386 2387 2388 2389 2390 2391 2392 2393 2394 2395 2396 2397 2398 2399 2400 2401 2402 2403 2404 2405 2406 2407 2408 2409 2410 2411 2412 2413 2414 2415 2416 2417 2418 2419 2420 2421 2422 2423 2424 2425 2426 2427 2428 2429 2430 2431 2432 2433 2434 2435 2436 2437 2438 2439 2440 2441 2442 2443 2444 2445 2446 2447 2448 2449 2450 2451
	String partition = field.getType() == Field::Types::UInt64 ? toString(field.get<UInt64>()) : field.safeGet<String>();

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

	String source_dir = (unreplicated ? "unreplicated/" : "detached/");

	/// Составим список кусков, которые нужно добавить.
	Strings parts;
	if (attach_part)
	{
		parts.push_back(partition);
	}
	else
	{
		LOG_DEBUG(log, "Looking for parts for partition " << partition << " in " << source_dir);
		ActiveDataPartSet active_parts;
		for (Poco::DirectoryIterator it = Poco::DirectoryIterator(full_path + source_dir); it != Poco::DirectoryIterator(); ++it)
		{
			String name = it.name();
			if (!ActiveDataPartSet::isPartDirectory(name))
				continue;
			if (name.substr(0, partition.size()) != partition)
				continue;
			LOG_DEBUG(log, "Found part " << name);
			active_parts.add(name);
		}
		LOG_DEBUG(log, active_parts.size() << " of them are active");
		parts = active_parts.getParts();
	}

	/// Синхронно проверим, что добавляемые куски существуют и не испорчены хотя бы на этой реплике. Запишем checksums.txt, если его нет.
	LOG_DEBUG(log, "Checking parts");
	for (const String & part : parts)
	{
		LOG_DEBUG(log, "Checking part " << part);
		data.loadPartAndFixMetadata(source_dir + part);
	}

	/// Выделим добавляемым кускам максимальные свободные номера, меньшие RESERVED_BLOCK_NUMBERS.
	/// NOTE: Проверка свободности номеров никак не синхронизируется. Выполнять несколько запросов ATTACH/DETACH/DROP одновременно нельзя.
	UInt64 min_used_number = RESERVED_BLOCK_NUMBERS;

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

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

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

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

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

	LOG_DEBUG(log, "Adding attaches to log");
	zookeeper->multi(ops);
	size_t i = 0;
	for (LogEntry & entry : entries)
	{
		String log_znode_path = dynamic_cast<zkutil::Op::Create &>(ops[i++]).getPathCreated();
		entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);

		waitForAllReplicasToProcessLogEntry(entry);
	}
M
Merge  
Michael Kolupaev 已提交
2452 2453
}

M
Merge  
Michael Kolupaev 已提交
2454 2455
void StorageReplicatedMergeTree::drop()
{
M
Merge  
Michael Kolupaev 已提交
2456 2457
	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 已提交
2458

M
Merge  
Michael Kolupaev 已提交
2459 2460
	shutdown();

M
Merge  
Michael Kolupaev 已提交
2461
	LOG_INFO(log, "Removing replica " << replica_path);
M
Merge  
Michael Kolupaev 已提交
2462
	replica_is_active_node = nullptr;
M
Merge  
Michael Kolupaev 已提交
2463 2464
	zookeeper->tryRemoveRecursive(replica_path);

M
Merge  
Michael Kolupaev 已提交
2465
	/// Проверяем, что zookeeper_path существует: его могла удалить другая реплика после выполнения предыдущей строки.
M
Merge  
Michael Kolupaev 已提交
2466 2467
	Strings replicas;
	if (zookeeper->tryGetChildren(zookeeper_path + "/replicas", replicas) == ZOK && replicas.empty())
M
Merge  
Michael Kolupaev 已提交
2468 2469
	{
		LOG_INFO(log, "Removing table " << zookeeper_path << " (this might take several minutes)");
M
Merge  
Michael Kolupaev 已提交
2470
		zookeeper->tryRemoveRecursive(zookeeper_path);
M
Merge  
Michael Kolupaev 已提交
2471
	}
M
Merge  
Michael Kolupaev 已提交
2472 2473

	data.dropAllData();
M
Merge  
Michael Kolupaev 已提交
2474 2475
}

M
Merge  
Michael Kolupaev 已提交
2476 2477 2478 2479 2480 2481 2482 2483 2484 2485 2486 2487 2488 2489 2490
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 已提交
2491 2492 2493 2494 2495 2496 2497 2498 2499 2500
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 已提交
2501
		for (size_t i = 0; i < RESERVED_BLOCK_NUMBERS; ++i)
M
Merge  
Michael Kolupaev 已提交
2502 2503 2504 2505 2506 2507 2508 2509 2510 2511 2512 2513 2514
		{
			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 已提交
2515
void StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry(const LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
2516
{
M
Merge  
Michael Kolupaev 已提交
2517 2518
	LOG_DEBUG(log, "Waiting for all replicas to process " << entry.znode_name);

M
Merge  
Michael Kolupaev 已提交
2519
	UInt64 log_index = parse<UInt64>(entry.znode_name.substr(entry.znode_name.size() - 10));
M
Merge  
Michael Kolupaev 已提交
2520 2521 2522 2523 2524
	String log_entry_str = entry.toString();

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

M
Merge  
Michael Kolupaev 已提交
2527 2528 2529 2530 2531 2532 2533 2534 2535 2536 2537 2538
		/// Дождемся, пока запись попадет в очередь реплики.
		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 已提交
2539 2540
		LOG_DEBUG(log, "Looking for " << entry.znode_name << " in " << replica << " queue");

M
Merge  
Michael Kolupaev 已提交
2541 2542 2543 2544 2545 2546 2547
		/// Найдем запись в очереди реплики.
		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 已提交
2548 2549
			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 已提交
2550 2551 2552 2553 2554 2555 2556 2557 2558 2559
			{
				entry_to_wait_for = entry_name;
				break;
			}
		}

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

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

M
Merge  
Michael Kolupaev 已提交
2562 2563 2564 2565 2566 2567 2568
		/// Дождемся, пока запись исчезнет из очереди реплики.
		while (true)
		{
			zkutil::EventPtr event = new Poco::Event;

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

			event->wait();
		}
	}
M
Merge  
Michael Kolupaev 已提交
2575 2576

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


M
Merge  
Michael Kolupaev 已提交
2580 2581 2582
void StorageReplicatedMergeTree::LogEntry::writeText(WriteBuffer & out) const
{
	writeString("format version: 1\n", out);
M
Merge  
Michael Kolupaev 已提交
2583 2584 2585
	writeString("source replica: ", out);
	writeString(source_replica, out);
	writeString("\n", out);
M
Merge  
Michael Kolupaev 已提交
2586 2587 2588 2589 2590 2591 2592 2593 2594 2595 2596 2597 2598 2599 2600 2601
	switch (type)
	{
		case GET_PART:
			writeString("get\n", out);
			writeString(new_part_name, out);
			break;
		case MERGE_PARTS:
			writeString("merge\n", out);
			for (const String & s : parts_to_merge)
			{
				writeString(s, out);
				writeString("\n", out);
			}
			writeString("into\n", out);
			writeString(new_part_name, out);
			break;
M
Merge  
Michael Kolupaev 已提交
2602
		case DROP_RANGE:
M
Merge  
Michael Kolupaev 已提交
2603 2604 2605 2606
			if (detach)
				writeString("detach\n", out);
			else
				writeString("drop\n", out);
M
Merge  
Michael Kolupaev 已提交
2607 2608
			writeString(new_part_name, out);
			break;
M
Merge  
Michael Kolupaev 已提交
2609 2610 2611 2612 2613 2614 2615 2616 2617 2618
		case ATTACH_PART:
			writeString("attach\n", out);
			if (attach_unreplicated)
				writeString("unreplicated\n", out);
			else
				writeString("detached\n", out);
			writeString(source_part_name, out);
			writeString("\ninto\n", out);
			writeString(new_part_name, out);
			break;
M
Merge  
Michael Kolupaev 已提交
2619 2620 2621 2622 2623 2624 2625 2626 2627
	}
	writeString("\n", out);
}

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

	assertString("format version: 1\n", in);
M
Michael Kolupaev 已提交
2628
	assertString("source replica: ", in);
M
Merge  
Michael Kolupaev 已提交
2629
	readString(source_replica, in);
M
Merge  
Michael Kolupaev 已提交
2630
	assertString("\n", in);
M
Michael Kolupaev 已提交
2631 2632
	readString(type_str, in);
	assertString("\n", in);
M
Merge  
Michael Kolupaev 已提交
2633 2634 2635 2636 2637 2638 2639 2640 2641 2642 2643 2644 2645 2646 2647 2648 2649 2650 2651 2652

	if (type_str == "get")
	{
		type = GET_PART;
		readString(new_part_name, in);
	}
	else if (type_str == "merge")
	{
		type = MERGE_PARTS;
		while (true)
		{
			String s;
			readString(s, in);
			assertString("\n", in);
			if (s == "into")
				break;
			parts_to_merge.push_back(s);
		}
		readString(new_part_name, in);
	}
M
Merge  
Michael Kolupaev 已提交
2653
	else if (type_str == "drop" || type_str == "detach")
M
Merge  
Michael Kolupaev 已提交
2654 2655
	{
		type = DROP_RANGE;
M
Merge  
Michael Kolupaev 已提交
2656
		detach = type_str == "detach";
M
Merge  
Michael Kolupaev 已提交
2657 2658
		readString(new_part_name, in);
	}
M
Merge  
Michael Kolupaev 已提交
2659 2660 2661 2662 2663 2664 2665 2666 2667 2668 2669 2670 2671 2672 2673 2674
	else if (type_str == "attach")
	{
		type = ATTACH_PART;
		String source_type;
		readString(source_type, in);
		if (source_type == "unreplicated")
			attach_unreplicated = true;
		else if (source_type == "detached")
			attach_unreplicated = false;
		else
			throw Exception("Bad format: expected 'unreplicated' or 'detached', found '" + source_type + "'", ErrorCodes::CANNOT_PARSE_TEXT);
		assertString("\n", in);
		readString(source_part_name, in);
		assertString("\ninto\n", in);
		readString(new_part_name, in);
	}
M
Merge  
Michael Kolupaev 已提交
2675 2676 2677
	assertString("\n", in);
}

M
Merge  
Michael Kolupaev 已提交
2678
}