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

namespace DB
{

M
Merge  
Michael Kolupaev 已提交
16

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

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

M
Merge  
Michael Kolupaev 已提交
22

M
Merge  
Michael Kolupaev 已提交
23 24 25
StorageReplicatedMergeTree::StorageReplicatedMergeTree(
	const String & zookeeper_path_,
	const String & replica_name_,
M
Merge  
Michael Kolupaev 已提交
26
	bool attach,
M
Merge  
Michael Kolupaev 已提交
27 28
	const String & path_, const String & database_name_, const String & name_,
	NamesAndTypesListPtr columns_,
29
	const NamesAndTypesList & materialized_columns_,
30 31
	const NamesAndTypesList & alias_columns_,
	const ColumnDefaults & column_defaults_,
M
Merge  
Michael Kolupaev 已提交
32
	Context & context_,
M
Merge  
Michael Kolupaev 已提交
33 34 35 36 37 38
	ASTPtr & primary_expr_ast_,
	const String & date_column_name_,
	const ASTPtr & sampling_expression_,
	size_t index_granularity_,
	MergeTreeData::Mode mode_,
	const String & sign_column_,
A
Merge  
Alexey Milovidov 已提交
39
	const Names & columns_to_sum_,
M
Merge  
Michael Kolupaev 已提交
40
	const MergeTreeSettings & settings_)
41
    : IStorage{materialized_columns_, alias_columns_, column_defaults_}, context(context_),
A
Merge  
Alexey Milovidov 已提交
42
	current_zookeeper(context.getZooKeeper()), database_name(database_name_),
M
Merge  
Michael Kolupaev 已提交
43 44 45
	table_name(name_), full_path(path_ + escapeForFileName(table_name) + '/'),
	zookeeper_path(context.getMacros().expand(zookeeper_path_)),
	replica_name(context.getMacros().expand(replica_name_)),
46 47 48
	data(full_path, columns_,
		 materialized_columns_, alias_columns_, column_defaults_,
		 context_, primary_expr_ast_, date_column_name_,
A
Merge  
Alexey Milovidov 已提交
49
		 sampling_expression_, index_granularity_, mode_, sign_column_, columns_to_sum_,
50 51
		 settings_, database_name_ + "." + table_name, true,
		 std::bind(&StorageReplicatedMergeTree::enqueuePartForCheck, this, std::placeholders::_1)),
M
Merge  
Michael Kolupaev 已提交
52
	reader(data), writer(data), merger(data), fetcher(data),
53
	log(&Logger::get(database_name + "." + table_name + " (StorageReplicatedMergeTree)")),
M
Merge  
Michael Kolupaev 已提交
54
	shutdown_event(false)
M
Merge  
Michael Kolupaev 已提交
55
{
56 57
	if (!zookeeper_path.empty() && zookeeper_path.back() == '/')
		zookeeper_path.resize(zookeeper_path.size() - 1);
M
Merge  
Michael Kolupaev 已提交
58 59 60 61
	replica_path = zookeeper_path + "/replicas/" + replica_name;

	bool skip_sanity_checks = false;

62
	try
M
Merge  
Michael Kolupaev 已提交
63
	{
A
Merge  
Alexey Milovidov 已提交
64
		if (current_zookeeper && current_zookeeper->exists(replica_path + "/flags/force_restore_data"))
65 66
		{
			skip_sanity_checks = true;
A
Merge  
Alexey Milovidov 已提交
67
			current_zookeeper->remove(replica_path + "/flags/force_restore_data");
M
Merge  
Michael Kolupaev 已提交
68

69 70 71 72 73 74 75 76 77 78
			LOG_WARNING(log, "Skipping the limits on severity of changes to data parts and columns (flag "
				<< replica_path << "/flags/force_restore_data).");
		}
	}
	catch (const zkutil::KeeperException & e)
	{
		/// Не удалось соединиться с ZK (об этом стало известно при попытке выполнить первую операцию).
		if (e.code == ZCONNECTIONLOSS)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);
A
Merge  
Alexey Milovidov 已提交
79
			current_zookeeper = nullptr;
80 81 82
		}
		else
			throw;
M
Merge  
Michael Kolupaev 已提交
83 84 85 86
	}

	data.loadDataParts(skip_sanity_checks);

A
Merge  
Alexey Milovidov 已提交
87
	if (!current_zookeeper)
M
Merge  
Michael Kolupaev 已提交
88
	{
M
Merge  
Michael Kolupaev 已提交
89 90 91
		if (!attach)
			throw Exception("Can't create replicated table without ZooKeeper", ErrorCodes::NO_ZOOKEEPER);

92
		/// Не активируем реплику. Она будет в режиме readonly.
M
Merge  
Michael Kolupaev 已提交
93 94 95
		return;
	}

M
Merge  
Michael Kolupaev 已提交
96 97
	if (!attach)
	{
M
Merge  
Michael Kolupaev 已提交
98
		createTableIfNotExists();
M
Merge  
Michael Kolupaev 已提交
99

M
Merge  
Michael Kolupaev 已提交
100
		checkTableStructure(false, false);
M
Merge  
Michael Kolupaev 已提交
101
		createReplica();
M
Merge  
Michael Kolupaev 已提交
102 103 104
	}
	else
	{
M
Merge  
Michael Kolupaev 已提交
105
		checkTableStructure(skip_sanity_checks, true);
M
Merge  
Michael Kolupaev 已提交
106
		checkParts(skip_sanity_checks);
M
Merge  
Michael Kolupaev 已提交
107
	}
M
Merge  
Michael Kolupaev 已提交
108

M
Merge  
Michael Kolupaev 已提交
109
	initVirtualParts();
M
Merge  
Michael Kolupaev 已提交
110 111 112 113 114

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

116 117
		unreplicated_data.reset(new MergeTreeData(unreplicated_path, columns_,
			materialized_columns_, alias_columns_, column_defaults_,
118
			context_, primary_expr_ast_,
A
Merge  
Alexey Milovidov 已提交
119
			date_column_name_, sampling_expression_, index_granularity_, mode_, sign_column_, columns_to_sum_, settings_,
M
Merge  
Michael Kolupaev 已提交
120
			database_name_ + "." + table_name + "[unreplicated]", false));
121 122 123

		unreplicated_data->loadDataParts(skip_sanity_checks);

M
Merge  
Michael Kolupaev 已提交
124
		unreplicated_reader.reset(new MergeTreeDataSelectExecutor(*unreplicated_data));
M
Merge  
Michael Kolupaev 已提交
125
		unreplicated_merger.reset(new MergeTreeDataMerger(*unreplicated_data));
M
Merge  
Michael Kolupaev 已提交
126
	}
M
Merge  
Michael Kolupaev 已提交
127

128
	loadQueue();
M
Merge  
Michael Kolupaev 已提交
129

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

134

M
Merge  
Michael Kolupaev 已提交
135 136 137
StoragePtr StorageReplicatedMergeTree::create(
	const String & zookeeper_path_,
	const String & replica_name_,
M
Merge  
Michael Kolupaev 已提交
138
	bool attach,
M
Merge  
Michael Kolupaev 已提交
139 140
	const String & path_, const String & database_name_, const String & name_,
	NamesAndTypesListPtr columns_,
141
	const NamesAndTypesList & materialized_columns_,
142 143
	const NamesAndTypesList & alias_columns_,
	const ColumnDefaults & column_defaults_,
M
Merge  
Michael Kolupaev 已提交
144
	Context & context_,
M
Merge  
Michael Kolupaev 已提交
145 146 147 148 149 150
	ASTPtr & primary_expr_ast_,
	const String & date_column_name_,
	const ASTPtr & sampling_expression_,
	size_t index_granularity_,
	MergeTreeData::Mode mode_,
	const String & sign_column_,
A
Merge  
Alexey Milovidov 已提交
151
	const Names & columns_to_sum_ = Names(),
M
Merge  
Michael Kolupaev 已提交
152 153
	const MergeTreeSettings & settings_)
{
154 155 156
	auto res = new StorageReplicatedMergeTree{
		zookeeper_path_, replica_name_, attach,
		path_, database_name_, name_,
157
		columns_, materialized_columns_, alias_columns_, column_defaults_,
158 159
		context_, primary_expr_ast_, date_column_name_,
		sampling_expression_, index_granularity_, mode_,
A
Merge  
Alexey Milovidov 已提交
160 161
		sign_column_, columns_to_sum_, settings_};

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

A
Merge  
Alexey Milovidov 已提交
164
	if (res->getZooKeeper())
M
Merge  
Michael Kolupaev 已提交
165 166
	{
		String endpoint_name = "ReplicatedMergeTree:" + res->replica_path;
M
Merge  
Michael Kolupaev 已提交
167
		InterserverIOEndpointPtr endpoint = new ReplicatedMergeTreePartsServer(res->data, *res);
M
Merge  
Michael Kolupaev 已提交
168 169
		res->endpoint_holder = new InterserverIOEndpointHolder(endpoint_name, endpoint, res->context.getInterserverIOHandler());
	}
170

M
Merge  
Michael Kolupaev 已提交
171
	return res_ptr;
M
Merge  
Michael Kolupaev 已提交
172 173
}

A
Merge  
Alexey Milovidov 已提交
174

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

A
Merge  
Alexey Milovidov 已提交
184

M
Merge  
Michael Kolupaev 已提交
185
void StorageReplicatedMergeTree::createTableIfNotExists()
M
Merge  
Michael Kolupaev 已提交
186
{
A
Merge  
Alexey Milovidov 已提交
187 188
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
189 190
	if (zookeeper->exists(zookeeper_path))
		return;
M
Merge  
Michael Kolupaev 已提交
191

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

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

M
Merge  
Michael Kolupaev 已提交
196
	/// Запишем метаданные таблицы, чтобы реплики могли сверять с ними параметры таблицы.
M
Merge  
Michael Kolupaev 已提交
197 198 199 200 201 202 203 204
	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 已提交
205

M
Merge  
Michael Kolupaev 已提交
206 207 208 209 210
	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));
211 212 213
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/columns", ColumnsDescription<false>{
				data.getColumnsListNonMaterialized(), data.materialized_columns,
				data.alias_columns, data.column_defaults}.toString(),
M
Merge  
Michael Kolupaev 已提交
214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230 231 232
										 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 已提交
233
}
M
Merge  
Michael Kolupaev 已提交
234

A
Merge  
Alexey Milovidov 已提交
235

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

M
Merge  
Michael Kolupaev 已提交
243
	String metadata_str = zookeeper->get(zookeeper_path + "/metadata");
M
Merge  
Michael Kolupaev 已提交
244 245 246 247 248 249 250 251 252 253 254 255 256
	ReadBufferFromString buf(metadata_str);
	assertString("metadata format version: 1", buf);
	assertString("\ndate column: ", buf);
	assertString(data.date_column_name, buf);
	assertString("\nsampling expression: ", buf);
	assertString(formattedAST(data.sampling_expression), buf);
	assertString("\nindex granularity: ", buf);
	assertString(toString(data.index_granularity), buf);
	assertString("\nmode: ", buf);
	assertString(toString(static_cast<int>(data.mode)), buf);
	assertString("\nsign column: ", buf);
	assertString(data.sign_column, buf);
	assertString("\nprimary key: ", buf);
M
Merge  
Michael Kolupaev 已提交
257 258
	/// NOTE: Можно сделать менее строгую проверку совпадения выражений, чтобы таблицы не ломались от небольших изменений
	///       в коде formatAST.
M
Merge  
Michael Kolupaev 已提交
259
	assertString(formattedAST(data.primary_expr_ast), buf);
M
Merge  
Michael Kolupaev 已提交
260
	assertString("\n", buf);
M
Merge  
Michael Kolupaev 已提交
261
	assertEOF(buf);
M
Merge  
Michael Kolupaev 已提交
262 263

	zkutil::Stat stat;
264 265 266 267 268 269 270
	auto columns_desc = ColumnsDescription<true>::parse(
		zookeeper->get(zookeeper_path + "/columns", &stat), context.getDataTypeFactory());

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

273 274 275 276
	if (columns != data.getColumnsListNonMaterialized() ||
		materialized_columns != data.materialized_columns ||
		alias_columns != data.alias_columns ||
		column_defaults != data.column_defaults)
M
Merge  
Michael Kolupaev 已提交
277
	{
278 279 280 281
		if (allow_alter &&
			(skip_sanity_checks ||
			 data.getColumnsListNonMaterialized().sizeOfDifference(columns) +
			 data.materialized_columns.sizeOfDifference(materialized_columns) <= 2))
M
Merge  
Michael Kolupaev 已提交
282
		{
M
Merge  
Michael Kolupaev 已提交
283
			LOG_WARNING(log, "Table structure in ZooKeeper is a little different from local table structure. Assuming ALTER.");
284 285

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

A
Merge  
Alexey Milovidov 已提交
301

M
Merge  
Michael Kolupaev 已提交
302 303
void StorageReplicatedMergeTree::createReplica()
{
A
Merge  
Alexey Milovidov 已提交
304 305
	auto zookeeper = getZooKeeper();

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

M
Merge  
Michael Kolupaev 已提交
308 309 310 311 312 313 314 315
	/// Создадим пустую реплику. Ноду 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));
316 317 318 319 320 321 322 323 324 325 326 327

	try
	{
		zookeeper->multi(ops);
	}
	catch (const zkutil::KeeperException & e)
	{
		if (e.code == ZNODEEXISTS)
			throw Exception("Replica " + replica_path + " is already exist.", ErrorCodes::REPLICA_IS_ALREADY_EXIST);

		throw;
	}
M
Merge  
Michael Kolupaev 已提交
328

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

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

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

M
Merge  
Michael Kolupaev 已提交
341 342 343
	Stat stat;
	zookeeper->exists(replica_path, &stat);
	auto my_create_time = stat.czxid;
M
Merge  
Michael Kolupaev 已提交
344

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

M
Merge  
Michael Kolupaev 已提交
358
	if (source_replica.empty())
M
Merge  
Michael Kolupaev 已提交
359
	{
M
Merge  
Michael Kolupaev 已提交
360
		LOG_INFO(log, "This is the first replica");
M
Merge  
Michael Kolupaev 已提交
361
	}
M
Merge  
Michael Kolupaev 已提交
362
	else
M
Merge  
Michael Kolupaev 已提交
363
	{
M
Merge  
Michael Kolupaev 已提交
364
		LOG_INFO(log, "Will mimic " << source_replica);
M
Merge  
Michael Kolupaev 已提交
365

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

M
Merge  
Michael Kolupaev 已提交
368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402 403 404 405 406
		/** Если эталонная реплика еще не до конца создана, подождем.
		  * 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");
407 408
		ActiveDataPartSet active_parts_set(parts);

M
Merge  
Michael Kolupaev 已提交
409 410 411 412 413 414 415 416 417 418 419 420 421 422 423 424 425 426
		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 已提交
427
	}
M
Merge  
Michael Kolupaev 已提交
428

429 430 431 432 433 434
	zookeeper->create(replica_path + "/columns", ColumnsDescription<false>{
			data.getColumnsListNonMaterialized(),
			data.materialized_columns,
			data.alias_columns,
			data.column_defaults
		}.toString(), zkutil::CreateMode::Persistent);
M
Merge  
Michael Kolupaev 已提交
435
}
M
Merge  
Michael Kolupaev 已提交
436 437


M
Merge  
Michael Kolupaev 已提交
438
void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
M
Merge  
Michael Kolupaev 已提交
439
{
A
Merge  
Alexey Milovidov 已提交
440 441
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
442
	Strings expected_parts_vec = zookeeper->getChildren(replica_path + "/parts");
M
Merge  
Michael Kolupaev 已提交
443 444

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

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

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

M
Merge  
Michael Kolupaev 已提交
452 453 454 455 456 457 458 459
	for (const auto & part : parts)
	{
		if (expected_parts.count(part->name))
		{
			expected_parts.erase(part->name);
		}
		else
		{
M
Merge  
Michael Kolupaev 已提交
460
			unexpected_parts.insert(part);
M
Merge  
Michael Kolupaev 已提交
461 462 463
		}
	}

M
Merge  
Michael Kolupaev 已提交
464
	/// Какие локальные куски добавить в ZK.
M
Merge  
Michael Kolupaev 已提交
465
	MergeTreeData::DataPartsVector parts_to_add;
M
Merge  
Michael Kolupaev 已提交
466 467 468 469

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

M
Merge  
Michael Kolupaev 已提交
470 471
	for (const String & missing_name : expected_parts)
	{
M
Merge  
Michael Kolupaev 已提交
472
		/// Если локально не хватает какого-то куска, но есть покрывающий его кусок, можно заменить в ZK недостающий покрывающим.
M
Merge  
Michael Kolupaev 已提交
473
		auto containing = data.getActiveContainingPart(missing_name);
M
Merge  
Michael Kolupaev 已提交
474 475 476 477 478 479 480 481 482 483
		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 已提交
484
		{
M
Merge  
Michael Kolupaev 已提交
485
			LOG_ERROR(log, "Fetching missing part " << missing_name);
M
Merge  
Michael Kolupaev 已提交
486
			parts_to_fetch.push_back(missing_name);
M
Merge  
Michael Kolupaev 已提交
487 488
		}
	}
M
Merge  
Michael Kolupaev 已提交
489

M
Merge  
Michael Kolupaev 已提交
490 491 492 493 494 495
	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 已提交
496
					 + toString(expected_parts.size()) + " missing obsolete parts, "
M
Merge  
Michael Kolupaev 已提交
497 498
					 + toString(parts_to_fetch.size()) + " missing parts";

499 500 501 502 503 504 505 506 507 508 509 510 511 512 513
	/** Можно автоматически синхронизировать данные,
	  * если количество ошибок каждого из четырёх типов не больше соответствующих порогов,
	  * или если отношение общего количества ошибок к общему количеству кусков (минимальному - в локальной файловой системе или в ZK)
	  *  не больше некоторого отношения (например 5%).
	  */

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

	bool insane =
		(parts_to_add.size() > data.settings.replicated_max_unexpectedly_merged_parts
			|| unexpected_parts.size() > data.settings.replicated_max_unexpected_parts
			|| expected_parts.size() > data.settings.replicated_max_missing_obsolete_parts
			|| parts_to_fetch.size() > data.settings.replicated_max_missing_active_parts)
		&& ((parts_to_add.size() + unexpected_parts.size() + expected_parts.size() + parts_to_fetch.size())
			> min_parts_local_or_expected * data.settings.replicated_max_ratio_of_wrong_parts);
M
Merge  
Michael Kolupaev 已提交
514

M
Merge  
Michael Kolupaev 已提交
515 516
	if (insane)
	{
517 518 519 520 521
		if (skip_sanity_checks)
			LOG_WARNING(log, sanity_report);
		else
			throw Exception("The local set of parts of table " + getTableName() + " doesn't look like the set of parts in ZooKeeper. "
				+ sanity_report, ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS);
M
Merge  
Michael Kolupaev 已提交
522 523
	}

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

M
Merge  
Michael Kolupaev 已提交
529
		zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
530
		checkPartAndAddToZooKeeper(part, ops);
M
Merge  
Michael Kolupaev 已提交
531
		zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
532
	}
M
Merge  
Michael Kolupaev 已提交
533

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

		zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
540
		ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + name + "/columns", -1));
M
Merge  
Michael Kolupaev 已提交
541 542 543 544 545 546 547 548 549 550 551 552
		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 已提交
553
		log_entry.source_replica = "";
M
Merge  
Michael Kolupaev 已提交
554 555 556
		log_entry.new_part_name = name;

		/// Полагаемся, что это происходит до загрузки очереди (loadQueue).
M
Merge  
Michael Kolupaev 已提交
557
		zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
558
		ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + name + "/columns", -1));
M
Merge  
Michael Kolupaev 已提交
559 560
		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 已提交
561 562
		ops.push_back(new zkutil::Op::Create(
			replica_path + "/queue/queue-", log_entry.toString(), zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
M
Merge  
Michael Kolupaev 已提交
563
		zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
564 565 566
	}

	/// Удалим лишние локальные куски.
567
	for (const MergeTreeData::DataPartPtr & part : unexpected_parts)
M
Merge  
Michael Kolupaev 已提交
568
	{
M
Merge  
Michael Kolupaev 已提交
569
		LOG_ERROR(log, "Renaming unexpected part " << part->name << " to ignored_" + part->name);
M
Merge  
Michael Kolupaev 已提交
570
		data.renameAndDetachPart(part, "ignored_", true);
M
Merge  
Michael Kolupaev 已提交
571 572
	}
}
M
Merge  
Michael Kolupaev 已提交
573

A
Merge  
Alexey Milovidov 已提交
574

M
Merge  
Michael Kolupaev 已提交
575 576 577 578 579 580 581
void StorageReplicatedMergeTree::initVirtualParts()
{
	auto parts = data.getDataParts();
	for (const auto & part : parts)
		virtual_parts.add(part->name);
}

A
Merge  
Alexey Milovidov 已提交
582

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

M
Merge  
Michael Kolupaev 已提交
587 588 589
	if (part_name.empty())
		part_name = part->name;

590
	check(part->columns);
M
Merge  
Michael Kolupaev 已提交
591
	int expected_columns_version = columns_version;
592

M
Merge  
Michael Kolupaev 已提交
593 594 595 596 597
	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 已提交
598
	{
M
Merge  
Michael Kolupaev 已提交
599 600
		zkutil::Stat stat_before, stat_after;
		String columns_str;
M
Merge  
Michael Kolupaev 已提交
601
		if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name + "/columns", columns_str, &stat_before))
M
Merge  
Michael Kolupaev 已提交
602 603 604
			continue;
		if (columns_str != expected_columns_str)
		{
M
Merge  
Michael Kolupaev 已提交
605
			LOG_INFO(log, "Not checking checksums of part " << part_name << " with replica " << replica
M
Merge  
Michael Kolupaev 已提交
606 607 608
				<< " because columns are different");
			continue;
		}
M
Merge  
Michael Kolupaev 已提交
609
		String checksums_str;
M
Merge  
Michael Kolupaev 已提交
610 611
		/// Проверим, что версия ноды со столбцами не изменилась, пока мы читали checksums.
		/// Это гарантирует, что столбцы и чексуммы относятся к одним и тем же данным.
M
Merge  
Michael Kolupaev 已提交
612 613
		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 已提交
614
			stat_before.version != stat_after.version)
M
Merge  
Michael Kolupaev 已提交
615
		{
M
Merge  
Michael Kolupaev 已提交
616
			LOG_INFO(log, "Not checking checksums of part " << part_name << " with replica " << replica
M
Merge  
Michael Kolupaev 已提交
617 618
				<< " because part changed while we were reading its checksums");
			continue;
M
Merge  
Michael Kolupaev 已提交
619
		}
M
Merge  
Michael Kolupaev 已提交
620 621 622

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

M
Merge  
Michael Kolupaev 已提交
625
	if (zookeeper->exists(replica_path + "/parts/" + part_name))
M
Merge  
Michael Kolupaev 已提交
626
	{
M
Merge  
Michael Kolupaev 已提交
627
		LOG_ERROR(log, "checkPartAndAddToZooKeeper: node " << replica_path + "/parts/" + part_name << " already exists");
M
Merge  
Michael Kolupaev 已提交
628 629 630
		return;
	}

631 632 633
	ops.push_back(new zkutil::Op::Check(
		zookeeper_path + "/columns",
		expected_columns_version));
M
Merge  
Michael Kolupaev 已提交
634
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
635
		replica_path + "/parts/" + part_name,
M
Merge  
Michael Kolupaev 已提交
636
		"",
M
Merge  
Michael Kolupaev 已提交
637
		zookeeper->getDefaultACL(),
M
Merge  
Michael Kolupaev 已提交
638
		zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
639
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
640
		replica_path + "/parts/" + part_name + "/columns",
M
Merge  
Michael Kolupaev 已提交
641 642 643
		part->columns.toString(),
		zookeeper->getDefaultACL(),
		zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
644
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
645
		replica_path + "/parts/" + part_name + "/checksums",
M
Merge  
Michael Kolupaev 已提交
646
		part->checksums.toString(),
M
Merge  
Michael Kolupaev 已提交
647
		zookeeper->getDefaultACL(),
M
Merge  
Michael Kolupaev 已提交
648 649 650
		zkutil::CreateMode::Persistent));
}

A
Merge  
Alexey Milovidov 已提交
651

M
Merge  
Michael Kolupaev 已提交
652 653
void StorageReplicatedMergeTree::loadQueue()
{
A
Merge  
Alexey Milovidov 已提交
654 655
	auto zookeeper = getZooKeeper();

656
	std::lock_guard<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
657

M
Merge  
Michael Kolupaev 已提交
658
	Strings children = zookeeper->getChildren(replica_path + "/queue");
M
Merge  
Michael Kolupaev 已提交
659 660 661
	std::sort(children.begin(), children.end());
	for (const String & child : children)
	{
A
Merge  
Alexey Milovidov 已提交
662 663
		zkutil::Stat stat;
		String s = zookeeper->get(replica_path + "/queue/" + child, &stat);
M
Merge  
Michael Kolupaev 已提交
664
		LogEntryPtr entry = LogEntry::parse(s);
A
Merge  
Alexey Milovidov 已提交
665
		entry->create_time = stat.ctime / 1000;
M
Merge  
Michael Kolupaev 已提交
666 667
		entry->znode_name = child;
		entry->addResultToVirtualParts(*this);
M
Merge  
Michael Kolupaev 已提交
668
		queue.push_back(entry);
M
Merge  
Michael Kolupaev 已提交
669 670 671
	}
}

A
Merge  
Alexey Milovidov 已提交
672

M
Merge  
Michael Kolupaev 已提交
673
void StorageReplicatedMergeTree::pullLogsToQueue(zkutil::EventPtr next_update_event)
M
Merge  
Michael Kolupaev 已提交
674
{
A
Merge  
Alexey Milovidov 已提交
675 676
	auto zookeeper = getZooKeeper();

677
	std::lock_guard<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
678

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

M
Merge  
Michael Kolupaev 已提交
682
	if (index_str.empty())
M
Merge  
Michael Kolupaev 已提交
683
	{
M
Merge  
Michael Kolupaev 已提交
684 685 686
		/// Если у нас еще нет указателя на лог, поставим указатель на первую запись в нем.
		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 已提交
687

M
Merge  
Michael Kolupaev 已提交
688
		zookeeper->set(replica_path + "/log_pointer", toString(index));
M
Merge  
Michael Kolupaev 已提交
689 690
	}
	else
M
Merge  
Michael Kolupaev 已提交
691
	{
M
Merge  
Michael Kolupaev 已提交
692
		index = parse<UInt64>(index_str);
M
Merge  
Michael Kolupaev 已提交
693
	}
M
Merge  
Michael Kolupaev 已提交
694

M
Merge  
Michael Kolupaev 已提交
695 696
	UInt64 first_index = index;

M
Merge  
Michael Kolupaev 已提交
697
	size_t count = 0;
M
Merge  
Michael Kolupaev 已提交
698
	String entry_str;
A
Merge  
Alexey Milovidov 已提交
699 700
	zkutil::Stat stat;
	while (zookeeper->tryGet(zookeeper_path + "/log/log-" + padIndex(index), entry_str, &stat))
M
Merge  
Michael Kolupaev 已提交
701
	{
M
Merge  
Michael Kolupaev 已提交
702
		++count;
M
Merge  
Michael Kolupaev 已提交
703
		++index;
M
Merge  
Michael Kolupaev 已提交
704

M
Merge  
Michael Kolupaev 已提交
705
		LogEntryPtr entry = LogEntry::parse(entry_str);
A
Merge  
Alexey Milovidov 已提交
706
		entry->create_time = stat.ctime / 1000;
M
Merge  
Michael Kolupaev 已提交
707

M
Merge  
Michael Kolupaev 已提交
708 709 710
		/// Одновременно добавим запись в очередь и продвинем указатель на лог.
		zkutil::Ops ops;
		ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
711
			replica_path + "/queue/queue-", entry_str, zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
M
Merge  
Michael Kolupaev 已提交
712
		ops.push_back(new zkutil::Op::SetData(
M
Merge  
Michael Kolupaev 已提交
713
			replica_path + "/log_pointer", toString(index), -1));
M
Merge  
Michael Kolupaev 已提交
714
		auto results = zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
715

716
		String path_created = dynamic_cast<zkutil::Op::Create &>(ops[0]).getPathCreated();
M
Merge  
Michael Kolupaev 已提交
717 718
		entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
		entry->addResultToVirtualParts(*this);
M
Merge  
Michael Kolupaev 已提交
719
		queue.push_back(entry);
M
Merge  
Michael Kolupaev 已提交
720
	}
M
Merge  
Michael Kolupaev 已提交
721

M
Merge  
Michael Kolupaev 已提交
722 723 724 725
	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 已提交
726
	}
M
Merge  
Michael Kolupaev 已提交
727

M
Merge  
Michael Kolupaev 已提交
728 729 730
	if (!count)
		return;

M
Merge  
Michael Kolupaev 已提交
731 732
	if (queue_task_handle)
		queue_task_handle->wake();
M
Merge  
Michael Kolupaev 已提交
733

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

A
Merge  
Alexey Milovidov 已提交
737

M
Merge  
Michael Kolupaev 已提交
738
bool StorageReplicatedMergeTree::shouldExecuteLogEntry(const LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
739
{
M
Merge  
Michael Kolupaev 已提交
740 741
	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 已提交
742 743 744 745 746 747
	{
		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 已提交
748 749 750 751 752 753 754 755 756 757 758
	if (entry.type == LogEntry::MERGE_PARTS)
	{
		/** Если какая-то из нужных частей сейчас передается или мерджится, подождем окончания этой операции.
		  * Иначе, даже если всех нужных частей для мерджа нет, нужно попытаться сделать мердж.
		  * Если каких-то частей не хватает, вместо мерджа будет попытка скачать кусок.
		  * Такая ситуация возможна, если получение какого-то куска пофейлилось, и его переместили в конец очереди.
		  */
		for (const auto & name : entry.parts_to_merge)
		{
			if (future_parts.count(name))
			{
M
Merge  
Michael Kolupaev 已提交
759
				LOG_TRACE(log, "Not merging into part " << entry.new_part_name << " because part " << name << " is not ready yet.");
M
Merge  
Michael Kolupaev 已提交
760 761 762 763 764 765
				return false;
			}
		}
	}

	return true;
M
Merge  
Michael Kolupaev 已提交
766 767
}

A
Merge  
Alexey Milovidov 已提交
768

M
Merge  
Michael Kolupaev 已提交
769
bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry, BackgroundProcessingPool::Context & pool_context)
M
Merge  
Michael Kolupaev 已提交
770
{
A
Merge  
Alexey Milovidov 已提交
771 772
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
773
	if (entry.type == LogEntry::DROP_RANGE)
M
Merge  
Michael Kolupaev 已提交
774 775 776 777
	{
		executeDropRange(entry);
		return true;
	}
M
Merge  
Michael Kolupaev 已提交
778

M
Merge  
Michael Kolupaev 已提交
779
	if (entry.type == LogEntry::GET_PART ||
M
Merge  
Michael Kolupaev 已提交
780 781
		entry.type == LogEntry::MERGE_PARTS ||
		entry.type == LogEntry::ATTACH_PART)
M
Merge  
Michael Kolupaev 已提交
782 783
	{
		/// Если у нас уже есть этот кусок или покрывающий его кусок, ничего делать не нужно.
M
Merge  
Michael Kolupaev 已提交
784
		MergeTreeData::DataPartPtr containing_part = data.getActiveContainingPart(entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
785 786

		/// Даже если кусок есть локально, его (в исключительных случаях) может не быть в zookeeper.
M
Merge  
Michael Kolupaev 已提交
787
		if (containing_part && zookeeper->exists(replica_path + "/parts/" + containing_part->name))
M
Merge  
Michael Kolupaev 已提交
788
		{
M
Merge  
Michael Kolupaev 已提交
789 790
			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 已提交
791
			return true;
M
Merge  
Michael Kolupaev 已提交
792
		}
M
Merge  
Michael Kolupaev 已提交
793 794
	}

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

	bool do_fetch = false;

M
Merge  
Michael Kolupaev 已提交
800 801
	if (entry.type == LogEntry::GET_PART)
	{
M
Merge  
Michael Kolupaev 已提交
802
		do_fetch = true;
M
Merge  
Michael Kolupaev 已提交
803
	}
M
Merge  
Michael Kolupaev 已提交
804 805 806 807
	else if (entry.type == LogEntry::ATTACH_PART)
	{
		do_fetch = !executeAttachPart(entry);
	}
M
Merge  
Michael Kolupaev 已提交
808 809
	else if (entry.type == LogEntry::MERGE_PARTS)
	{
810 811 812 813 814 815 816 817
		std::stringstream log_message;
		log_message << "Executing log entry to merge parts ";
		for (auto i : ext::range(0, entry.parts_to_merge.size()))
			log_message << (i != 0 ? ", " : "") << entry.parts_to_merge[i];
		log_message << " to " << entry.new_part_name;

		LOG_TRACE(log, log_message.rdbuf());

M
Merge  
Michael Kolupaev 已提交
818
		MergeTreeData::DataPartsVector parts;
819
		bool have_all_parts = true;
M
Merge  
Michael Kolupaev 已提交
820 821
		for (const String & name : entry.parts_to_merge)
		{
M
Merge  
Michael Kolupaev 已提交
822
			MergeTreeData::DataPartPtr part = data.getActiveContainingPart(name);
M
Merge  
Michael Kolupaev 已提交
823 824 825 826 827 828 829
			if (!part)
			{
				have_all_parts = false;
				break;
			}
			if (part->name != name)
			{
M
Merge  
Michael Kolupaev 已提交
830 831
				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 已提交
832 833 834
				have_all_parts = false;
				break;
			}
M
Merge  
Michael Kolupaev 已提交
835 836
			parts.push_back(part);
		}
M
Merge  
Michael Kolupaev 已提交
837

M
Merge  
Michael Kolupaev 已提交
838
		if (!have_all_parts)
M
Merge  
Michael Kolupaev 已提交
839
		{
M
Merge  
Michael Kolupaev 已提交
840 841 842
			/// Если нет всех нужных кусков, попробуем взять у кого-нибудь уже помердженный кусок.
			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 已提交
843
		}
M
Merge  
Michael Kolupaev 已提交
844 845
		else
		{
M
Merge  
Michael Kolupaev 已提交
846 847 848
			/// Если собираемся сливать большие куски, увеличим счетчик потоков, сливающих большие куски.
			for (const auto & part : parts)
			{
M
Merge  
Michael Kolupaev 已提交
849
				if (part->size_in_bytes > data.settings.max_bytes_to_merge_parts_small)
M
Merge  
Michael Kolupaev 已提交
850 851 852 853 854 855 856
				{
					pool_context.incrementCounter("big merges");
					pool_context.incrementCounter("replicated big merges");
					break;
				}
			}

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

859
			const auto & merge_entry = context.getMergeList().insert(database_name, table_name, entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
860
			MergeTreeData::Transaction transaction;
861
			MergeTreeData::DataPartPtr part = merger.mergeParts(parts, entry.new_part_name, *merge_entry, &transaction);
M
Merge  
Michael Kolupaev 已提交
862 863

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

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

M
Merge  
Michael Kolupaev 已提交
870
			zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
871 872 873 874

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

			ProfileEvents::increment(ProfileEvents::ReplicatedPartMerges);
		}
M
Merge  
Michael Kolupaev 已提交
880 881 882 883 884
	}
	else
	{
		throw Exception("Unexpected log entry type: " + toString(static_cast<int>(entry.type)));
	}
M
Merge  
Michael Kolupaev 已提交
885 886 887

	if (do_fetch)
	{
M
Merge  
Michael Kolupaev 已提交
888 889
		String replica;

M
Merge  
Michael Kolupaev 已提交
890 891
		try
		{
M
Merge  
Michael Kolupaev 已提交
892
			replica = findReplicaHavingPart(entry.new_part_name, true);
A
Merge  
Alexey Milovidov 已提交
893 894 895 896 897 898 899 900 901 902 903 904 905 906 907 908 909

			if (replica.empty() && entry.type == LogEntry::ATTACH_PART)
			{
				/** Если ATTACH - куска может не быть, потому что реплика, на которой кусок есть, ещё сама не успела его прицепить.
				  * В таком случае, надо подождать этого.
				  */

				/// Кусок должен быть на реплике-инициаторе.
				if (entry.source_replica.empty() || entry.source_replica == replica_name)
					throw Exception("Logical error: no source replica specified for ATTACH_PART log entry;"
						" or trying to fetch part on source replica", ErrorCodes::LOGICAL_ERROR);

				/// Подождём, пока реплика-инициатор подцепит кусок.
				waitForReplicaToProcessLogEntry(entry.source_replica, entry);
				replica = findReplicaHavingPart(entry.new_part_name, true);
			}

M
Merge  
Michael Kolupaev 已提交
910 911 912 913 914
			if (replica.empty())
			{
				ProfileEvents::increment(ProfileEvents::ReplicatedPartFailedFetches);
				throw Exception("No active replica has part " + entry.new_part_name, ErrorCodes::NO_REPLICA_HAS_PART);
			}
A
Merge  
Alexey Milovidov 已提交
915

916
			fetchPart(entry.new_part_name, zookeeper_path + "/replicas/" + replica);
M
Merge  
Michael Kolupaev 已提交
917 918 919 920 921 922 923 924 925 926 927 928

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

				/// Найдем действие по объединению этого куска с другими. Запомним других.
				StringSet parts_for_merge;
				LogEntries::iterator merge_entry;
				for (LogEntries::iterator it = queue.begin(); it != queue.end(); ++it)
				{
M
Merge  
Michael Kolupaev 已提交
936
					if ((*it)->type == LogEntry::MERGE_PARTS)
M
Merge  
Michael Kolupaev 已提交
937
					{
M
Merge  
Michael Kolupaev 已提交
938 939
						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 已提交
940
						{
M
Merge  
Michael Kolupaev 已提交
941
							parts_for_merge = StringSet((*it)->parts_to_merge.begin(), (*it)->parts_to_merge.end());
M
Merge  
Michael Kolupaev 已提交
942 943 944 945 946 947 948 949 950 951 952 953 954 955 956 957 958
							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 已提交
959 960
						if (((*it0)->type == LogEntry::MERGE_PARTS || (*it0)->type == LogEntry::GET_PART)
							&& parts_for_merge.count((*it0)->new_part_name))
M
Merge  
Michael Kolupaev 已提交
961 962 963 964
						{
							queue.splice(queue.end(), queue, it0, it);
						}
					}
M
Merge  
Michael Kolupaev 已提交
965 966 967 968 969 970 971 972 973

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

				/// Если ни у кого нет куска, и в очереди нет слияний с его участием, проверим, есть ли у кого-то покрывающий его.
				if (replica.empty())
					enqueuePartForCheck(entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
979 980 981 982 983 984 985 986 987
			}
			catch (...)
			{
				tryLogCurrentException(__PRETTY_FUNCTION__);
			}

			throw;
		}
	}
M
Merge  
Michael Kolupaev 已提交
988 989

	return true;
M
Merge  
Michael Kolupaev 已提交
990 991
}

A
Merge  
Alexey Milovidov 已提交
992

M
Merge  
Michael Kolupaev 已提交
993
void StorageReplicatedMergeTree::executeDropRange(const StorageReplicatedMergeTree::LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
994
{
A
Merge  
Alexey Milovidov 已提交
995 996
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
997
	LOG_INFO(log, (entry.detach ? "Detaching" : "Removing") << " parts inside " << entry.new_part_name << ".");
M
Merge  
Michael Kolupaev 已提交
998 999 1000 1001 1002 1003 1004 1005 1006 1007 1008 1009 1010 1011 1012 1013 1014 1015 1016 1017 1018 1019 1020 1021 1022 1023 1024 1025 1026

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

M
Merge  
Michael Kolupaev 已提交
1031
	LOG_DEBUG(log, (entry.detach ? "Detaching" : "Removing") << " parts.");
M
Merge  
Michael Kolupaev 已提交
1032 1033 1034 1035 1036 1037 1038 1039 1040 1041 1042
	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 已提交
1043 1044 1045 1046
		/// Если кусок удалять не нужно, надежнее переместить директорию до изменений в ZooKeeper.
		if (entry.detach)
			data.renameAndDetachPart(part);

M
Merge  
Michael Kolupaev 已提交
1047 1048 1049 1050 1051 1052
		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 已提交
1053
		/// Если кусок нужно удалить, надежнее удалить директорию после изменений в ZooKeeper.
M
Merge  
Michael Kolupaev 已提交
1054
		if (!entry.detach)
M
Merge  
Michael Kolupaev 已提交
1055
			data.replaceParts({part}, {}, true);
M
Merge  
Michael Kolupaev 已提交
1056 1057
	}

M
Merge  
Michael Kolupaev 已提交
1058
	LOG_INFO(log, (entry.detach ? "Detached " : "Removed ") << removed_parts << " parts inside " << entry.new_part_name << ".");
M
Merge  
Michael Kolupaev 已提交
1059 1060 1061

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

M
Merge  
Michael Kolupaev 已提交
1064 1065 1066 1067 1068 1069 1070 1071 1072 1073 1074 1075 1076 1077
		removed_parts = 0;
		parts = unreplicated_data->getDataParts();
		for (const auto & part : parts)
		{
			if (!ActiveDataPartSet::contains(entry.new_part_name, part->name))
				continue;
			LOG_DEBUG(log, "Removing unreplicated part " << part->name);
			++removed_parts;

			if (entry.detach)
				unreplicated_data->renameAndDetachPart(part, "");
			else
				unreplicated_data->replaceParts({part}, {}, false);
		}
A
Merge  
Alexey Milovidov 已提交
1078 1079

		LOG_INFO(log, (entry.detach ? "Detached " : "Removed ") << removed_parts << " parts inside " << entry.new_part_name << " (in unreplicated data).");
M
Merge  
Michael Kolupaev 已提交
1080
	}
M
Merge  
Michael Kolupaev 已提交
1081 1082
}

A
Merge  
Alexey Milovidov 已提交
1083

M
Merge  
Michael Kolupaev 已提交
1084 1085
bool StorageReplicatedMergeTree::executeAttachPart(const StorageReplicatedMergeTree::LogEntry & entry)
{
A
Merge  
Alexey Milovidov 已提交
1086 1087
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
1088 1089 1090 1091
	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 已提交
1092
	if (!Poco::File(data.getFullPath() + source_path).exists())
M
Merge  
Michael Kolupaev 已提交
1093
	{
M
Merge  
Michael Kolupaev 已提交
1094
		LOG_INFO(log, "No part at " << source_path << ". Will fetch it instead");
M
Merge  
Michael Kolupaev 已提交
1095 1096 1097 1098 1099 1100 1101 1102 1103 1104 1105 1106 1107 1108 1109 1110 1111 1112 1113 1114 1115 1116 1117 1118 1119 1120 1121 1122
		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 已提交
1123

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

M
Merge  
Michael Kolupaev 已提交
1127 1128 1129
	return true;
}

A
Merge  
Alexey Milovidov 已提交
1130

M
Merge  
Michael Kolupaev 已提交
1131 1132 1133 1134
void StorageReplicatedMergeTree::queueUpdatingThread()
{
	while (!shutdown_called)
	{
M
Merge  
Michael Kolupaev 已提交
1135 1136
		try
		{
M
Merge  
Michael Kolupaev 已提交
1137
			pullLogsToQueue(queue_updating_event);
M
Merge  
Michael Kolupaev 已提交
1138

M
Merge  
Michael Kolupaev 已提交
1139
			queue_updating_event->wait();
M
Merge  
Michael Kolupaev 已提交
1140
		}
1141
		catch (const zkutil::KeeperException & e)
M
Merge  
Michael Kolupaev 已提交
1142 1143
		{
			if (e.code == ZINVALIDSTATE)
1144
				restarting_thread->wakeup();
M
Merge  
Michael Kolupaev 已提交
1145 1146 1147 1148 1149

			tryLogCurrentException(__PRETTY_FUNCTION__);

			queue_updating_event->tryWait(ERROR_SLEEP_MS);
		}
M
Merge  
Michael Kolupaev 已提交
1150 1151 1152 1153
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);

M
Merge  
Michael Kolupaev 已提交
1154 1155
			queue_updating_event->tryWait(ERROR_SLEEP_MS);
		}
M
Merge  
Michael Kolupaev 已提交
1156
	}
M
Merge  
Michael Kolupaev 已提交
1157

1158
	LOG_DEBUG(log, "Queue updating thread finished");
M
Merge  
Michael Kolupaev 已提交
1159
}
M
Merge  
Michael Kolupaev 已提交
1160

A
Merge  
Alexey Milovidov 已提交
1161

M
Merge  
Michael Kolupaev 已提交
1162
bool StorageReplicatedMergeTree::queueTask(BackgroundProcessingPool::Context & pool_context)
M
Merge  
Michael Kolupaev 已提交
1163
{
M
Merge  
Michael Kolupaev 已提交
1164
	LogEntryPtr entry;
M
Merge  
Michael Kolupaev 已提交
1165

M
Merge  
Michael Kolupaev 已提交
1166 1167
	try
	{
1168
		std::lock_guard<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1169 1170
		bool empty = queue.empty();
		if (!empty)
M
Merge  
Michael Kolupaev 已提交
1171
		{
M
Merge  
Michael Kolupaev 已提交
1172
			for (LogEntries::iterator it = queue.begin(); it != queue.end(); ++it)
M
Merge  
Michael Kolupaev 已提交
1173
			{
M
Merge  
Michael Kolupaev 已提交
1174
				if (!(*it)->currently_executing && shouldExecuteLogEntry(**it))
M
Merge  
Michael Kolupaev 已提交
1175
				{
M
Merge  
Michael Kolupaev 已提交
1176
					entry = *it;
M
Merge  
Michael Kolupaev 已提交
1177
					entry->tagPartAsFuture(*this);
M
Merge  
Michael Kolupaev 已提交
1178
					queue.splice(queue.end(), queue, it);
M
Merge  
Michael Kolupaev 已提交
1179
					entry->currently_executing = true;
M
Merge  
Michael Kolupaev 已提交
1180
					break;
M
Merge  
Michael Kolupaev 已提交
1181
				}
M
Merge  
Michael Kolupaev 已提交
1182 1183
			}
		}
M
Merge  
Michael Kolupaev 已提交
1184 1185 1186 1187 1188
	}
	catch (...)
	{
		tryLogCurrentException(__PRETTY_FUNCTION__);
	}
M
Merge  
Michael Kolupaev 已提交
1189

M
Merge  
Michael Kolupaev 已提交
1190
	if (!entry)
M
Merge  
Michael Kolupaev 已提交
1191
		return false;
M
Merge  
Michael Kolupaev 已提交
1192

M
Merge  
Michael Kolupaev 已提交
1193
	bool exception = true;
M
Merge  
Michael Kolupaev 已提交
1194
	bool success = false;
M
Merge  
Michael Kolupaev 已提交
1195

M
Merge  
Michael Kolupaev 已提交
1196 1197
	try
	{
M
Merge  
Michael Kolupaev 已提交
1198
		if (executeLogEntry(*entry, pool_context))
M
Merge  
Michael Kolupaev 已提交
1199
		{
A
Merge  
Alexey Milovidov 已提交
1200
			auto zookeeper = getZooKeeper();
M
Merge  
Michael Kolupaev 已提交
1201
			auto code = zookeeper->tryRemove(replica_path + "/queue/" + entry->znode_name);
M
Merge  
Michael Kolupaev 已提交
1202

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

			success = true;
M
Merge  
Michael Kolupaev 已提交
1208
		}
M
Merge  
Michael Kolupaev 已提交
1209

M
Merge  
Michael Kolupaev 已提交
1210
		exception = false;
M
Merge  
Michael Kolupaev 已提交
1211 1212 1213 1214
	}
	catch (Exception & e)
	{
		if (e.code() == ErrorCodes::NO_REPLICA_HAS_PART)
M
Merge  
Michael Kolupaev 已提交
1215
			/// Если ни у кого нет нужного куска, наверно, просто не все реплики работают; не будем писать в лог с уровнем Error.
M
Merge  
Michael Kolupaev 已提交
1216 1217
			LOG_INFO(log, e.displayText());
		else
M
Merge  
Michael Kolupaev 已提交
1218
			tryLogCurrentException(__PRETTY_FUNCTION__);
M
Merge  
Michael Kolupaev 已提交
1219 1220 1221 1222 1223
	}
	catch (...)
	{
		tryLogCurrentException(__PRETTY_FUNCTION__);
	}
M
Merge  
Michael Kolupaev 已提交
1224

M
Merge  
Michael Kolupaev 已提交
1225 1226
	entry->future_part_tagger = nullptr;

1227
	std::lock_guard<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1228 1229 1230 1231 1232

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

	if (success)
M
Merge  
Michael Kolupaev 已提交
1233
	{
M
Merge  
Michael Kolupaev 已提交
1234 1235 1236
		/// Удалим задание из очереди.
		/// Нельзя просто обратиться по заранее сохраненному итератору, потому что задание мог успеть удалить кто-то другой.
		for (LogEntries::iterator it = queue.end(); it != queue.begin();)
M
Merge  
Michael Kolupaev 已提交
1237
		{
M
Merge  
Michael Kolupaev 已提交
1238 1239 1240
			--it;
			if (*it == entry)
			{
M
Merge  
Michael Kolupaev 已提交
1241
				queue.erase(it);
M
Merge  
Michael Kolupaev 已提交
1242 1243
				break;
			}
M
Merge  
Michael Kolupaev 已提交
1244
		}
M
Merge  
Michael Kolupaev 已提交
1245
	}
M
Merge  
Michael Kolupaev 已提交
1246

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

A
Merge  
Alexey Milovidov 已提交
1251

M
Merge  
Michael Kolupaev 已提交
1252 1253
void StorageReplicatedMergeTree::mergeSelectingThread()
{
M
Merge  
Michael Kolupaev 已提交
1254
	bool need_pull = true;
M
Merge  
Michael Kolupaev 已提交
1255

A
Merge  
Alexey Milovidov 已提交
1256 1257 1258 1259 1260 1261 1262
	/** Может много времени тратиться на определение, можно ли мерджить два рядом стоящих куска.
	  * Два рядом стоящих куска можно мерджить, если все номера блоков между их номерами не используются ("заброшены", abandoned).
	  * Это значит, что между этими кусками не может быть вставлен другой кусок.
	  *
	  * Но если номера соседних блоков отличаются достаточно сильно (обычно, если между ними много "заброшенных" блоков),
	  *  то делается слишком много чтений из ZooKeeper, чтобы узнать, можно ли их мерджить.
	  *
1263 1264
	  * Воспользуемся утверждением, что если пару кусков было можно мерджить, и их мердж ещё не запланирован,
	  *  то и сейчас их можно мерджить, и будем запоминать это состояние, чтобы не делать много раз одинаковые запросы в ZooKeeper.
A
Merge  
Alexey Milovidov 已提交
1265 1266 1267 1268 1269 1270 1271 1272
	  *
	  * TODO Интересно, как это сочетается с DROP PARTITION и затем ATTACH PARTITION.
	  */
	std::set<std::pair<std::string, std::string>> memoized_parts_that_could_be_merged;

	auto can_merge = [&memoized_parts_that_could_be_merged, this]
		(const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right) -> bool
	{
1273 1274 1275 1276 1277
		/// Если какой-то из кусков уже собираются слить в больший, не соглашаемся его сливать.
		if (virtual_parts.getContainingPart(left->name) != left->name ||
			virtual_parts.getContainingPart(right->name) != right->name)
			return false;

A
Merge  
Alexey Milovidov 已提交
1278 1279 1280 1281
		auto key = std::make_pair(left->name, right->name);
		if (memoized_parts_that_could_be_merged.count(key))
			return true;

1282
		String month_name = left->name.substr(0, 6);
A
Merge  
Alexey Milovidov 已提交
1283
		auto zookeeper = getZooKeeper();
1284 1285 1286 1287 1288 1289

		/// Можно слить куски, если все номера между ними заброшены - не соответствуют никаким блокам.
		for (UInt64 number = left->right + 1; number <= right->left - 1; ++number)	/// Номера блоков больше нуля.
		{
			String path1 = zookeeper_path +              "/block_numbers/" + month_name + "/block-" + padIndex(number);
			String path2 = zookeeper_path + "/nonincrement_block_numbers/" + month_name + "/block-" + padIndex(number);
A
Merge  
Alexey Milovidov 已提交
1290

1291 1292 1293 1294 1295 1296 1297
			if (AbandonableLockInZooKeeper::check(path1, *zookeeper) != AbandonableLockInZooKeeper::ABANDONED &&
				AbandonableLockInZooKeeper::check(path2, *zookeeper) != AbandonableLockInZooKeeper::ABANDONED)
				return false;
		}

		memoized_parts_that_could_be_merged.insert(key);
		return true;
A
Merge  
Alexey Milovidov 已提交
1298 1299
	};

M
Merge  
Michael Kolupaev 已提交
1300 1301
	while (!shutdown_called && is_leader_node)
	{
M
Michael Kolupaev 已提交
1302
		bool success = false;
M
Merge  
Michael Kolupaev 已提交
1303

M
Michael Kolupaev 已提交
1304
		try
M
Merge  
Michael Kolupaev 已提交
1305
		{
1306
			std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
M
Merge  
Michael Kolupaev 已提交
1307

M
Merge  
Michael Kolupaev 已提交
1308 1309 1310 1311 1312 1313 1314 1315
			if (need_pull)
			{
				/// Нужно загрузить новую запись в очередь перед тем, как выбирать куски для слияния.
				///  (чтобы кусок добавился в virtual_parts).
				pullLogsToQueue();
				need_pull = false;
			}

1316 1317 1318 1319 1320 1321 1322
			/** Сколько в очереди или в фоновом потоке мерджей крупных кусков.
			  * Если их больше половины от размера пула потоков для мерджа, то можно мерджить только мелкие куски.
			  */
			auto & background_pool = context.getBackgroundPool();

			size_t big_merges_current = background_pool.getCounter("replicated big merges");
			size_t max_number_of_big_merges = background_pool.getNumberOfThreads() / 2;
M
Michael Kolupaev 已提交
1323
			size_t merges_queued = 0;
1324
			size_t big_merges_queued = 0;
M
Merge  
Michael Kolupaev 已提交
1325

1326
			if (big_merges_current < max_number_of_big_merges)
M
Michael Kolupaev 已提交
1327
			{
1328
				std::lock_guard<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1329

M
Michael Kolupaev 已提交
1330
				for (const auto & entry : queue)
M
Merge  
Michael Kolupaev 已提交
1331
				{
M
Merge  
Michael Kolupaev 已提交
1332
					if (entry->type == LogEntry::MERGE_PARTS)
M
Merge  
Michael Kolupaev 已提交
1333
					{
M
Michael Kolupaev 已提交
1334
						++merges_queued;
M
Merge  
Michael Kolupaev 已提交
1335

1336
						if (big_merges_current + big_merges_queued < max_number_of_big_merges)
M
Merge  
Michael Kolupaev 已提交
1337
						{
M
Merge  
Michael Kolupaev 已提交
1338
							for (const String & name : entry->parts_to_merge)
M
Merge  
Michael Kolupaev 已提交
1339
							{
M
Merge  
Michael Kolupaev 已提交
1340
								MergeTreeData::DataPartPtr part = data.getActiveContainingPart(name);
M
Merge  
Michael Kolupaev 已提交
1341 1342
								if (!part || part->name != name)
									continue;
1343

M
Merge  
Michael Kolupaev 已提交
1344
								if (part->size_in_bytes > data.settings.max_bytes_to_merge_parts_small)
M
Merge  
Michael Kolupaev 已提交
1345
								{
1346
									++big_merges_queued;
M
Merge  
Michael Kolupaev 已提交
1347 1348 1349 1350 1351 1352
									break;
								}
							}
						}
					}
				}
M
Michael Kolupaev 已提交
1353
			}
M
Merge  
Michael Kolupaev 已提交
1354

1355 1356 1357 1358 1359 1360 1361 1362
			bool only_small = big_merges_current + big_merges_queued >= max_number_of_big_merges;

			LOG_TRACE(log, "Currently executing big merges: " << big_merges_current
				<< ". Queued big merges: " << big_merges_queued
				<< ". All merges in queue: " << merges_queued
				<< ". Max number of big merges: " << max_number_of_big_merges
				<< (only_small ? ". So, will select only small parts to merge." : "."));

M
Merge  
Michael Kolupaev 已提交
1363
			do
M
Michael Kolupaev 已提交
1364
			{
A
Merge  
Alexey Milovidov 已提交
1365 1366
				auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
1367
				if (merges_queued >= data.settings.max_replicated_merges_in_queue)
1368 1369
				{
					LOG_TRACE(log, "Number of queued merges is greater than max_replicated_merges_in_queue, so won't select new parts to merge.");
M
Merge  
Michael Kolupaev 已提交
1370
					break;
1371
				}
M
Merge  
Michael Kolupaev 已提交
1372

M
Merge  
Michael Kolupaev 已提交
1373
				MergeTreeData::DataPartsVector parts;
M
Merge  
Michael Kolupaev 已提交
1374

M
Merge  
Michael Kolupaev 已提交
1375 1376
				String merged_name;

1377 1378 1379 1380
				if (   !merger.selectPartsToMerge(parts, merged_name, MergeTreeDataMerger::NO_LIMIT, false, false, only_small, can_merge)
					&& !merger.selectPartsToMerge(parts, merged_name, MergeTreeDataMerger::NO_LIMIT, true, false, only_small, can_merge))
				{
					LOG_INFO(log, "No parts to merge");
M
Merge  
Michael Kolupaev 已提交
1381
					break;
1382
				}
M
Merge  
Michael Kolupaev 已提交
1383

M
Merge  
Michael Kolupaev 已提交
1384 1385 1386 1387 1388 1389 1390 1391 1392 1393 1394 1395 1396 1397
				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 已提交
1398 1399 1400 1401
				LogEntry entry;
				entry.type = LogEntry::MERGE_PARTS;
				entry.source_replica = replica_name;
				entry.new_part_name = merged_name;
M
Merge  
Michael Kolupaev 已提交
1402

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

M
Merge  
Michael Kolupaev 已提交
1406
				need_pull = true;
M
Merge  
Michael Kolupaev 已提交
1407

M
Merge  
Michael Kolupaev 已提交
1408
				zookeeper->create(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential);
M
Merge  
Michael Kolupaev 已提交
1409 1410 1411

				String month_name = parts[0]->name.substr(0, 6);
				for (size_t i = 0; i + 1 < parts.size(); ++i)
M
Merge  
Michael Kolupaev 已提交
1412
				{
M
Merge  
Michael Kolupaev 已提交
1413 1414 1415
					/// Уберем больше не нужные отметки о несуществующих блоках.
					for (UInt64 number = parts[i]->right + 1; number <= parts[i + 1]->left - 1; ++number)
					{
M
Merge  
Michael Kolupaev 已提交
1416 1417
						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 已提交
1418
					}
M
Merge  
Michael Kolupaev 已提交
1419
				}
M
Merge  
Michael Kolupaev 已提交
1420 1421

				success = true;
M
Merge  
Michael Kolupaev 已提交
1422
			}
1423
			while (false);
M
Merge  
Michael Kolupaev 已提交
1424 1425 1426 1427 1428 1429
		}
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);
		}

M
Merge  
Michael Kolupaev 已提交
1430
		if (shutdown_called || !is_leader_node)
M
Merge  
Michael Kolupaev 已提交
1431 1432
			break;

M
Merge  
Michael Kolupaev 已提交
1433
		if (!success)
M
Merge  
Michael Kolupaev 已提交
1434
			merge_selecting_event.tryWait(MERGE_SELECTING_SLEEP_MS);
M
Merge  
Michael Kolupaev 已提交
1435
	}
M
Merge  
Michael Kolupaev 已提交
1436

1437
	LOG_DEBUG(log, "Merge selecting thread finished");
M
Merge  
Michael Kolupaev 已提交
1438 1439
}

M
Merge  
Michael Kolupaev 已提交
1440

M
Merge  
Michael Kolupaev 已提交
1441 1442 1443 1444 1445 1446 1447 1448
void StorageReplicatedMergeTree::alterThread()
{
	bool force_recheck_parts = true;

	while (!shutdown_called)
	{
		try
		{
1449 1450 1451 1452 1453 1454 1455 1456 1457 1458 1459 1460 1461 1462 1463 1464 1465 1466 1467 1468 1469 1470 1471
			/** Имеем описание столбцов в ZooKeeper, общее для всех реплик (Пример: /clickhouse/tables/02-06/visits/columns),
			  *  а также описание столбцов в локальном файле с метаданными (data.getColumnsList()).
			  *
			  * Если эти описания отличаются - нужно сделать ALTER.
			  *
			  * Если запомненная версия ноды (columns_version) отличается от версии в ZK,
			  *  то описание столбцов в ZK не обязательно отличается от локального
			  *  - такое может быть при цикле из ALTER-ов, который в целом, ничего не меняет.
			  * В этом случае, надо обновить запомненный номер версии,
			  *  а также всё-равно проверить структуру кусков, и, при необходимости, сделать ALTER.
			  *
			  * Запомненный номер версии нужно обновить после обновления метаданных, под блокировкой.
			  * Этот номер версии проверяется на соответствие актуальному при INSERT-е.
			  * То есть, так добиваемся, чтобы вставлялись блоки с правильной структурой.
			  *
			  * При старте сервера, мог быть не завершён предыдущий ALTER.
			  * Поэтому, в первый раз, независимо от изменений, проверяем структуру всех part-ов,
			  *  (Пример: /clickhouse/tables/02-06/visits/replicas/example02-06-1.yandex.ru/parts/20140806_20140831_131664_134988_3296/columns)
			  *  и делаем ALTER, если необходимо.
			  *
			  * TODO: Слишком сложно, всё переделать.
			  */

A
Merge  
Alexey Milovidov 已提交
1472 1473
			auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
1474
			zkutil::Stat stat;
1475
			const String columns_str = zookeeper->get(zookeeper_path + "/columns", &stat, alter_thread_event);
1476 1477 1478 1479 1480 1481
			auto columns_desc = ColumnsDescription<true>::parse(columns_str, context.getDataTypeFactory());

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

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

M
Merge  
Michael Kolupaev 已提交
1485 1486
			MergeTreeData::DataParts parts;

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

1492 1493 1494
				const auto columns_changed = columns != data.getColumnsListNonMaterialized();
				const auto materialized_columns_changed = materialized_columns != data.materialized_columns;
				const auto alias_columns_changed = alias_columns != data.alias_columns;
A
Merge  
Andrey Mironov 已提交
1495
				const auto column_defaults_changed = column_defaults != data.column_defaults;
1496 1497 1498

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

1502 1503 1504
					InterpreterAlterQuery::updateMetadata(database_name, table_name, columns,
						materialized_columns, alias_columns, column_defaults, context);

1505 1506 1507 1508 1509 1510 1511 1512 1513 1514 1515 1516 1517 1518 1519 1520 1521 1522 1523 1524 1525 1526 1527 1528 1529 1530
					if (columns_changed)
					{
						data.setColumnsList(columns);

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

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

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

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

M
Merge  
Michael Kolupaev 已提交
1531 1532 1533 1534
					LOG_INFO(log, "Applied changes to table.");
				}
				else
				{
1535
					LOG_INFO(log, "Columns version changed in ZooKeeper, but data wasn't changed. It's like cyclic ALTERs.");
M
Merge  
Michael Kolupaev 已提交
1536
				}
1537 1538 1539 1540 1541

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

				columns_version = stat.version;
M
Merge  
Michael Kolupaev 已提交
1542 1543 1544
			}

			/// Обновим куски.
1545
			if (changed_version || force_recheck_parts)
M
Merge  
Michael Kolupaev 已提交
1546
			{
1547 1548 1549
				auto table_lock = lockStructure(false);

				if (changed_version)
M
Merge  
Michael Kolupaev 已提交
1550 1551 1552 1553
					LOG_INFO(log, "ALTER-ing parts");

				int changed_parts = 0;

1554
				if (!changed_version)
M
Merge  
Michael Kolupaev 已提交
1555 1556
					parts = data.getDataParts();

1557 1558
				const auto columns_plus_materialized = data.getColumnsList();

M
Merge  
Michael Kolupaev 已提交
1559 1560 1561
				for (const MergeTreeData::DataPartPtr & part : parts)
				{
					/// Обновим кусок и запишем результат во временные файлы.
M
Merge  
Michael Kolupaev 已提交
1562 1563
					/// TODO: Можно пропускать проверку на слишком большие изменения, если в ZooKeeper есть, например,
					///  нода /flags/force_alter.
1564
					auto transaction = data.alterDataPart(part, columns_plus_materialized);
M
Merge  
Michael Kolupaev 已提交
1565 1566 1567 1568 1569 1570 1571 1572 1573 1574 1575 1576 1577 1578 1579 1580

					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 已提交
1581 1582 1583 1584 1585 1586 1587
				/// То же самое для нереплицируемых данных.
				if (unreplicated_data)
				{
					parts = unreplicated_data->getDataParts();

					for (const MergeTreeData::DataPartPtr & part : parts)
					{
1588
						auto transaction = unreplicated_data->alterDataPart(part, columns_plus_materialized);
M
Merge  
Michael Kolupaev 已提交
1589 1590 1591 1592 1593 1594 1595 1596 1597

						if (!transaction)
							continue;

						++changed_parts;

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

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

1602 1603 1604 1605 1606 1607 1608 1609
				if (changed_version)
				{
					if (changed_parts != 0)
						LOG_INFO(log, "ALTER-ed " << changed_parts << " parts");
					else
						LOG_INFO(log, "No parts ALTER-ed");
				}

M
Merge  
Michael Kolupaev 已提交
1610 1611 1612
				force_recheck_parts = false;
			}

1613
			parts.clear();
M
Merge  
Michael Kolupaev 已提交
1614 1615 1616 1617 1618 1619 1620 1621 1622 1623 1624 1625
			alter_thread_event->wait();
		}
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);

			force_recheck_parts = true;

			alter_thread_event->tryWait(ERROR_SLEEP_MS);
		}
	}

1626
	LOG_DEBUG(log, "Alter thread finished");
M
Merge  
Michael Kolupaev 已提交
1627 1628
}

A
Merge  
Alexey Milovidov 已提交
1629

M
Merge  
Michael Kolupaev 已提交
1630 1631
void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_name)
{
A
Merge  
Alexey Milovidov 已提交
1632 1633
	auto zookeeper = getZooKeeper();

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

M
Merge  
Michael Kolupaev 已提交
1636 1637
	LogEntryPtr log_entry = new LogEntry;
	log_entry->type = LogEntry::GET_PART;
A
Merge  
Alexey Milovidov 已提交
1638
	log_entry->create_time = time(0);
M
Merge  
Michael Kolupaev 已提交
1639 1640
	log_entry->source_replica = "";
	log_entry->new_part_name = part_name;
M
Merge  
Michael Kolupaev 已提交
1641 1642 1643

	zkutil::Ops ops;
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
1644
		replica_path + "/queue/queue-", log_entry->toString(), zookeeper->getDefaultACL(),
M
Merge  
Michael Kolupaev 已提交
1645 1646 1647 1648 1649 1650 1651
		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);

	{
1652
		std::lock_guard<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1653 1654

		String path_created = dynamic_cast<zkutil::Op::Create &>(ops[0]).getPathCreated();
M
Merge  
Michael Kolupaev 已提交
1655 1656
		log_entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
		log_entry->addResultToVirtualParts(*this);
M
Merge  
Michael Kolupaev 已提交
1657 1658 1659 1660
		queue.push_back(log_entry);
	}
}

A
Merge  
Alexey Milovidov 已提交
1661

M
Merge  
Michael Kolupaev 已提交
1662 1663
void StorageReplicatedMergeTree::enqueuePartForCheck(const String & name)
{
1664
	std::lock_guard<std::mutex> lock(parts_to_check_mutex);
M
Merge  
Michael Kolupaev 已提交
1665 1666 1667 1668 1669 1670 1671 1672

	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 已提交
1673 1674 1675 1676 1677 1678
void StorageReplicatedMergeTree::partCheckThread()
{
	while (!shutdown_called)
	{
		try
		{
A
Merge  
Alexey Milovidov 已提交
1679 1680
			auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
1681 1682 1683
			/// Достанем из очереди кусок для проверки.
			String part_name;
			{
1684
				std::lock_guard<std::mutex> lock(parts_to_check_mutex);
M
Merge  
Michael Kolupaev 已提交
1685 1686 1687 1688 1689 1690 1691 1692 1693 1694 1695 1696 1697
				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();
				}
			}
1698
			if (part_name.empty())	/// TODO Здесь race condition?
M
Merge  
Michael Kolupaev 已提交
1699 1700 1701 1702 1703 1704
			{
				parts_to_check_event.wait();
				continue;
			}

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

M
Merge  
Michael Kolupaev 已提交
1707
			auto part = data.getActiveContainingPart(part_name);
M
Merge  
Michael Kolupaev 已提交
1708 1709 1710 1711 1712 1713 1714 1715 1716 1717
			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 已提交
1718
					ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
M
Merge  
Michael Kolupaev 已提交
1719

M
Merge  
Michael Kolupaev 已提交
1720
					removePartAndEnqueueFetch(part_name);
M
Merge  
Michael Kolupaev 已提交
1721 1722 1723 1724
				}
				/// Если куска нет в ZooKeeper, проверим есть ли он хоть у кого-то.
				else
				{
M
Merge  
Michael Kolupaev 已提交
1725 1726
					ActiveDataPartSet::Part part_info;
					ActiveDataPartSet::parsePartName(part_name, part_info);
M
Merge  
Michael Kolupaev 已提交
1727

M
Merge  
Michael Kolupaev 已提交
1728
					/** Будем проверять только куски, не полученные в результате слияния.
M
Merge  
Michael Kolupaev 已提交
1729
					  * Для кусков, полученных в результате слияния, такая проверка была бы некорректной,
M
Merge  
Michael Kolupaev 已提交
1730 1731
					  *  потому что слитого куска может еще ни у кого не быть.
					  */
M
Merge  
Michael Kolupaev 已提交
1732
					if (part_info.left == part_info.right)
M
Merge  
Michael Kolupaev 已提交
1733
					{
M
Merge  
Michael Kolupaev 已提交
1734 1735 1736 1737 1738
						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 已提交
1739
						{
M
Merge  
Michael Kolupaev 已提交
1740 1741
							Strings parts = zookeeper->getChildren(zookeeper_path + "/replicas/" + replica + "/parts");
							for (const String & part_on_replica : parts)
M
Merge  
Michael Kolupaev 已提交
1742
							{
M
Merge  
Michael Kolupaev 已提交
1743 1744 1745 1746 1747 1748
								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 已提交
1749
							}
M
Merge  
Michael Kolupaev 已提交
1750 1751
							if (found)
								break;
M
Merge  
Michael Kolupaev 已提交
1752 1753
						}

M
Merge  
Michael Kolupaev 已提交
1754
						if (!found)
M
Merge  
Michael Kolupaev 已提交
1755
						{
M
Merge  
Michael Kolupaev 已提交
1756
							LOG_ERROR(log, "No replica has part covering " << part_name);
M
Merge  
Michael Kolupaev 已提交
1757 1758
							ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);

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

M
Merge  
Michael Kolupaev 已提交
1761 1762
							bool was_in_queue = false;

M
Merge  
Michael Kolupaev 已提交
1763
							{
1764
								std::lock_guard<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1765 1766

								for (LogEntries::iterator it = queue.begin(); it != queue.end(); )
M
Merge  
Michael Kolupaev 已提交
1767
								{
M
Merge  
Michael Kolupaev 已提交
1768
									if ((*it)->new_part_name == part_name)
M
Merge  
Michael Kolupaev 已提交
1769
									{
M
Merge  
Michael Kolupaev 已提交
1770
										zookeeper->tryRemove(replica_path + "/queue/" + (*it)->znode_name);
M
Merge  
Michael Kolupaev 已提交
1771
										queue.erase(it++);
M
Merge  
Michael Kolupaev 已提交
1772
										was_in_queue = true;
M
Merge  
Michael Kolupaev 已提交
1773 1774 1775 1776 1777
									}
									else
									{
										++it;
									}
M
Merge  
Michael Kolupaev 已提交
1778 1779
								}
							}
M
Merge  
Michael Kolupaev 已提交
1780 1781

							if (was_in_queue)
M
Merge  
Michael Kolupaev 已提交
1782
							{
M
Merge  
Michael Kolupaev 已提交
1783 1784 1785
								/** Такая ситуация возможна, если на всех репликах, где был кусок, он испортился.
								  * Например, у реплики, которая только что его записала, отключили питание, и данные не записались из кеша на диск.
								  */
A
Merge  
Alexey Milovidov 已提交
1786 1787
								LOG_ERROR(log, "Part " << part_name << " is lost forever.");
								ProfileEvents::increment(ProfileEvents::ReplicatedDataLoss);
M
Merge  
Michael Kolupaev 已提交
1788 1789 1790 1791 1792 1793 1794 1795 1796 1797 1798 1799 1800

								/** Нужно добавить отсутствующий кусок в 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 已提交
1801 1802
						}
					}
M
Merge  
Michael Kolupaev 已提交
1803 1804 1805 1806 1807
				}
			}
			/// У нас есть этот кусок, и он активен.
			else if (part->name == part_name)
			{
M
Merge  
Michael Kolupaev 已提交
1808 1809
				auto table_lock = lockStructure(false);

M
Merge  
Michael Kolupaev 已提交
1810 1811 1812
				/// Если кусок есть в ZooKeeper, сверим его данные с его чексуммами, а их с ZooKeeper.
				if (zookeeper->exists(replica_path + "/parts/" + part_name))
				{
M
Merge  
Michael Kolupaev 已提交
1813 1814 1815 1816 1817 1818 1819 1820 1821 1822 1823 1824 1825
					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 已提交
1826 1827 1828 1829
						MergeTreePartChecker::Settings settings;
						settings.setIndexGranularity(data.index_granularity);
						settings.setRequireChecksums(true);
						settings.setRequireColumnFiles(true);
M
Merge  
Michael Kolupaev 已提交
1830
						MergeTreePartChecker::checkDataPart(
M
Merge  
Michael Kolupaev 已提交
1831
							data.getFullPath() + part_name, settings, context.getDataTypeFactory());
M
Merge  
Michael Kolupaev 已提交
1832

M
Merge  
Michael Kolupaev 已提交
1833 1834 1835 1836 1837 1838
						LOG_INFO(log, "Part " << part_name << " looks good.");
					}
					catch (...)
					{
						tryLogCurrentException(__PRETTY_FUNCTION__);

M
Merge  
Michael Kolupaev 已提交
1839 1840
						LOG_ERROR(log, "Part " << part_name << " looks broken. Removing it and queueing a fetch.");
						ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
M
Merge  
Michael Kolupaev 已提交
1841 1842 1843 1844

						removePartAndEnqueueFetch(part_name);

						/// Удалим кусок локально.
M
Merge  
Michael Kolupaev 已提交
1845
						data.renameAndDetachPart(part, "broken_");
M
Merge  
Michael Kolupaev 已提交
1846
					}
M
Merge  
Michael Kolupaev 已提交
1847 1848
				}
				/// Если куска нет в ZooKeeper, удалим его локально.
M
Merge  
Michael Kolupaev 已提交
1849 1850 1851
				/// Возможно, кусок кто-то только что записал, и еще не успел добавить в ZK.
				/// Поэтому удаляем только если кусок старый (не очень надежно).
				else if (part->modification_time + 5 * 60 < time(0))
M
Merge  
Michael Kolupaev 已提交
1852
				{
M
Merge  
Michael Kolupaev 已提交
1853 1854
					ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);

M
Merge  
Michael Kolupaev 已提交
1855 1856
					LOG_ERROR(log, "Unexpected part " << part_name << ". Removing.");
					data.renameAndDetachPart(part, "unexpected_");
M
Merge  
Michael Kolupaev 已提交
1857 1858 1859 1860 1861 1862 1863 1864 1865 1866
				}
			}
			else
			{
				/// Если у нас есть покрывающий кусок, игнорируем все проблемы с этим куском.
				/// В худшем случае в лог еще old_parts_lifetime секунд будут валиться ошибки, пока кусок не удалится как старый.
			}

			/// Удалим кусок из очереди.
			{
1867
				std::lock_guard<std::mutex> lock(parts_to_check_mutex);
M
Merge  
Michael Kolupaev 已提交
1868 1869 1870 1871 1872 1873 1874 1875 1876 1877 1878 1879 1880 1881 1882 1883 1884 1885 1886 1887
				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 已提交
1888 1889
void StorageReplicatedMergeTree::becomeLeader()
{
M
Merge  
Michael Kolupaev 已提交
1890
	LOG_INFO(log, "Became leader");
M
Merge  
Michael Kolupaev 已提交
1891 1892 1893 1894
	is_leader_node = true;
	merge_selecting_thread = std::thread(&StorageReplicatedMergeTree::mergeSelectingThread, this);
}

A
Merge  
Alexey Milovidov 已提交
1895

M
Merge  
Michael Kolupaev 已提交
1896
String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_name, bool active)
M
Merge  
Michael Kolupaev 已提交
1897
{
A
Merge  
Alexey Milovidov 已提交
1898
	auto zookeeper = getZooKeeper();
M
Merge  
Michael Kolupaev 已提交
1899
	Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
M
Merge  
Michael Kolupaev 已提交
1900 1901 1902 1903 1904 1905

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

	for (const String & replica : replicas)
	{
M
Merge  
Michael Kolupaev 已提交
1906 1907
		if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name) &&
			(!active || zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")))
M
Merge  
Michael Kolupaev 已提交
1908 1909 1910
			return replica;
	}

M
Merge  
Michael Kolupaev 已提交
1911
	return "";
M
Merge  
Michael Kolupaev 已提交
1912 1913
}

A
Merge  
Alexey Milovidov 已提交
1914

1915
void StorageReplicatedMergeTree::fetchPart(const String & part_name, const String & replica_path, bool to_detached)
M
Merge  
Michael Kolupaev 已提交
1916
{
A
Merge  
Alexey Milovidov 已提交
1917 1918
	auto zookeeper = getZooKeeper();

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

1921 1922 1923
	TableStructureReadLockPtr table_lock;
	if (!to_detached)
		table_lock = lockStructure(true);
M
Merge  
Michael Kolupaev 已提交
1924 1925 1926 1927

	String host;
	int port;

1928
	String host_port_str = zookeeper->get(replica_path + "/host");
M
Merge  
Michael Kolupaev 已提交
1929 1930 1931 1932 1933 1934 1935 1936
	ReadBufferFromString buf(host_port_str);
	assertString("host: ", buf);
	readString(host, buf);
	assertString("\nport: ", buf);
	readText(port, buf);
	assertString("\n", buf);
	assertEOF(buf);

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

1939 1940 1941 1942
	if (!to_detached)
	{
		zkutil::Ops ops;
		checkPartAndAddToZooKeeper(part, ops, part_name);
M
Merge  
Michael Kolupaev 已提交
1943

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

1947 1948 1949
		zookeeper->multi(ops);
		transaction.commit();
		merge_selecting_event.set();
M
Merge  
Michael Kolupaev 已提交
1950

1951 1952 1953 1954 1955 1956 1957
		for (const auto & removed_part : removed_parts)
		{
			LOG_DEBUG(log, "Part " << removed_part->name << " is rendered obsolete by fetching part " << part_name);
			ProfileEvents::increment(ProfileEvents::ObsoleteReplicatedParts);
		}
	}
	else
M
Michael Kolupaev 已提交
1958
	{
1959
		Poco::File(data.getFullPath() + "detached/tmp_" + part_name).renameTo(data.getFullPath() + "detached/" + part_name);
M
Michael Kolupaev 已提交
1960 1961
	}

M
Merge  
Michael Kolupaev 已提交
1962 1963
	ProfileEvents::increment(ProfileEvents::ReplicatedPartFetches);

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

A
Merge  
Alexey Milovidov 已提交
1967

M
Merge  
Michael Kolupaev 已提交
1968 1969
void StorageReplicatedMergeTree::shutdown()
{
1970
	if (restarting_thread)
M
Merge  
Michael Kolupaev 已提交
1971
	{
1972 1973
		restarting_thread->stop();
		restarting_thread.reset();
M
Merge  
Michael Kolupaev 已提交
1974
	}
M
Merge  
Michael Kolupaev 已提交
1975 1976

	endpoint_holder = nullptr;
M
Merge  
Michael Kolupaev 已提交
1977 1978 1979
}


M
Merge  
Michael Kolupaev 已提交
1980 1981 1982 1983 1984 1985 1986 1987
StorageReplicatedMergeTree::~StorageReplicatedMergeTree()
{
	try
	{
		shutdown();
	}
	catch(...)
	{
1988
		tryLogCurrentException(__PRETTY_FUNCTION__);
M
Merge  
Michael Kolupaev 已提交
1989 1990 1991
	}
}

A
Merge  
Alexey Milovidov 已提交
1992

M
Merge  
Michael Kolupaev 已提交
1993 1994 1995 1996 1997 1998 1999 2000
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 已提交
2001 2002 2003 2004 2005 2006 2007 2008 2009 2010 2011 2012 2013 2014 2015 2016 2017
	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())
2018
		VirtualColumnUtils::filterBlockWithQuery(query, virtual_columns_block, context);
M
Merge  
Michael Kolupaev 已提交
2019

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

M
Merge  
Michael Kolupaev 已提交
2022 2023
	BlockInputStreams res;

M
Merge  
Michael Kolupaev 已提交
2024 2025 2026
	size_t part_index = 0;

	if (unreplicated_reader && values.count(0))
M
Merge  
Michael Kolupaev 已提交
2027
	{
M
Merge  
Michael Kolupaev 已提交
2028 2029
		res = unreplicated_reader->read(
			real_column_names, query, settings, processed_stage, max_block_size, threads, &part_index);
M
Merge  
Michael Kolupaev 已提交
2030 2031 2032 2033 2034 2035

		for (auto & virtual_column : virt_column_names)
		{
			if (virtual_column == "_replicated")
			{
				for (auto & stream : res)
M
Merge  
Michael Kolupaev 已提交
2036
					stream = new AddingConstColumnBlockInputStream<UInt8>(stream, new DataTypeUInt8, 0, "_replicated");
M
Merge  
Michael Kolupaev 已提交
2037 2038 2039 2040
			}
		}
	}

M
Merge  
Michael Kolupaev 已提交
2041
	if (values.count(1))
M
Merge  
Michael Kolupaev 已提交
2042
	{
M
Merge  
Michael Kolupaev 已提交
2043
		auto res2 = reader.read(real_column_names, query, settings, processed_stage, max_block_size, threads, &part_index);
M
Merge  
Michael Kolupaev 已提交
2044 2045 2046 2047 2048 2049

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

M
Merge  
Michael Kolupaev 已提交
2054
		res.insert(res.end(), res2.begin(), res2.end());
M
Merge  
Michael Kolupaev 已提交
2055 2056 2057
	}

	return res;
M
Merge  
Michael Kolupaev 已提交
2058 2059
}

A
Merge  
Alexey Milovidov 已提交
2060

M
Merge  
Michael Kolupaev 已提交
2061 2062
BlockOutputStreamPtr StorageReplicatedMergeTree::write(ASTPtr query)
{
2063 2064
	if (is_readonly)
		throw Exception("Table is in readonly mode", ErrorCodes::TABLE_IS_READ_ONLY);
M
Merge  
Michael Kolupaev 已提交
2065

M
Merge  
Michael Kolupaev 已提交
2066
	String insert_id;
M
Merge  
Michael Kolupaev 已提交
2067 2068 2069
	if (query)
		if (ASTInsertQuery * insert = typeid_cast<ASTInsertQuery *>(&*query))
			insert_id = insert->insert_id;
M
Merge  
Michael Kolupaev 已提交
2070 2071 2072

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

A
Merge  
Alexey Milovidov 已提交
2074

M
Merge  
Michael Kolupaev 已提交
2075 2076
bool StorageReplicatedMergeTree::optimize()
{
2077 2078
	/// Померджим какие-нибудь куски из директории unreplicated.
	/// TODO: Мерджить реплицируемые куски тоже.
M
Merge  
Michael Kolupaev 已提交
2079 2080 2081 2082

	if (!unreplicated_data)
		return false;

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

M
Merge  
Michael Kolupaev 已提交
2085 2086 2087 2088 2089 2090 2091 2092
	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;

2093 2094
	const auto & merge_entry = context.getMergeList().insert(database_name, table_name, merged_name);
	unreplicated_merger->mergeParts(parts, merged_name, *merge_entry);
M
Merge  
Michael Kolupaev 已提交
2095 2096 2097
	return true;
}

A
Merge  
Alexey Milovidov 已提交
2098

M
Merge  
Michael Kolupaev 已提交
2099 2100
void StorageReplicatedMergeTree::alter(const AlterCommands & params,
	const String & database_name, const String & table_name, Context & context)
M
Merge  
Michael Kolupaev 已提交
2101
{
A
Merge  
Alexey Milovidov 已提交
2102 2103
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
2104 2105 2106
	LOG_DEBUG(log, "Doing ALTER");

	NamesAndTypesList new_columns;
2107 2108 2109
	NamesAndTypesList new_materialized_columns;
	NamesAndTypesList new_alias_columns;
	ColumnDefaults new_column_defaults;
M
Merge  
Michael Kolupaev 已提交
2110 2111 2112 2113 2114 2115 2116
	String new_columns_str;
	int new_columns_version;
	zkutil::Stat stat;

	{
		auto table_lock = lockStructureForAlter();

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

M
Merge  
Michael Kolupaev 已提交
2120 2121
		data.checkAlter(params);

2122 2123 2124 2125 2126
		new_columns = data.getColumnsListNonMaterialized();
		new_materialized_columns = data.materialized_columns;
		new_alias_columns = data.alias_columns;
		new_column_defaults = data.column_defaults;
		params.apply(new_columns, new_materialized_columns, new_alias_columns, new_column_defaults);
M
Merge  
Michael Kolupaev 已提交
2127

2128 2129 2130 2131
		new_columns_str = ColumnsDescription<false>{
			new_columns, new_materialized_columns,
			new_alias_columns, new_column_defaults
		}.toString();
M
Merge  
Michael Kolupaev 已提交
2132 2133 2134 2135 2136 2137 2138 2139 2140 2141 2142 2143 2144 2145

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

M
Merge  
Michael Kolupaev 已提交
2147 2148 2149 2150 2151 2152 2153 2154 2155 2156 2157 2158 2159 2160 2161 2162 2163 2164 2165 2166 2167 2168 2169 2170 2171 2172 2173 2174 2175 2176
	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);
2177

M
Merge  
Michael Kolupaev 已提交
2178 2179 2180 2181 2182 2183 2184
			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 已提交
2185
			if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/columns", &stat, alter_query_event))
M
Merge  
Michael Kolupaev 已提交
2186 2187 2188 2189 2190 2191 2192 2193 2194 2195 2196 2197 2198 2199 2200 2201 2202 2203
			{
				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 已提交
2204 2205 2206 2207 2208 2209

/// Название воображаемого куска, покрывающего все возможные куски в указанном месяце с номерами в указанном диапазоне.
static String getFakePartNameForDrop(const String & month_name, UInt64 left, UInt64 right)
{
	/// Диапазон дат - весь месяц.
	DateLUT & lut = DateLUT::instance();
2210
	time_t start_time = DateLUT::instance().YYYYMMDDToDate(parse<UInt32>(month_name + "01"));
M
Merge  
Michael Kolupaev 已提交
2211 2212 2213 2214 2215 2216 2217
	DayNum_t left_date = lut.toDayNum(start_time);
	DayNum_t right_date = DayNum_t(static_cast<size_t>(left_date) + lut.daysInMonth(start_time) - 1);

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

A
Merge  
Alexey Milovidov 已提交
2218

2219
void StorageReplicatedMergeTree::dropPartition(const Field & field, bool detach, const Settings & settings)
M
Merge  
Michael Kolupaev 已提交
2220
{
A
Merge  
Alexey Milovidov 已提交
2221
	auto zookeeper = getZooKeeper();
A
Merge  
Alexey Milovidov 已提交
2222
	String month_name = MergeTreeData::getMonthName(field);
M
Merge  
Michael Kolupaev 已提交
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

	/// 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, в логе не появятся слияния удаляемых кусков.
	  */
	{
2253
		std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
M
Merge  
Michael Kolupaev 已提交
2254 2255 2256 2257

		virtual_parts.add(fake_part_name);
	}

M
Merge  
Michael Kolupaev 已提交
2258
	/// Наконец, добившись нужных инвариантов, можно положить запись в лог.
M
Merge  
Michael Kolupaev 已提交
2259 2260 2261 2262 2263 2264
	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 已提交
2265
	entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);
M
Merge  
Michael Kolupaev 已提交
2266

2267 2268 2269 2270 2271 2272 2273 2274
	/// Если надо - дожидаемся выполнения операции на себе или на всех репликах.
	if (settings.replication_alter_partitions_sync != 0)
	{
		if (settings.replication_alter_partitions_sync == 1)
			waitForReplicaToProcessLogEntry(replica_name, entry);
		else
			waitForAllReplicasToProcessLogEntry(entry);
	}
M
Merge  
Michael Kolupaev 已提交
2275 2276
}

A
Merge  
Alexey Milovidov 已提交
2277

2278
void StorageReplicatedMergeTree::attachPartition(const Field & field, bool unreplicated, bool attach_part, const Settings & settings)
M
Merge  
Michael Kolupaev 已提交
2279
{
A
Merge  
Alexey Milovidov 已提交
2280
	auto zookeeper = getZooKeeper();
A
Merge  
Alexey Milovidov 已提交
2281
	String partition;
M
Merge  
Michael Kolupaev 已提交
2282

A
Merge  
Alexey Milovidov 已提交
2283 2284 2285 2286
	if (attach_part)
		partition = field.getType() == Field::Types::UInt64 ? toString(field.get<UInt64>()) : field.safeGet<String>();
	else
		partition = MergeTreeData::getMonthName(field);
M
Merge  
Michael Kolupaev 已提交
2287 2288 2289 2290 2291 2292 2293 2294 2295 2296 2297 2298 2299 2300 2301 2302 2303 2304

	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;
2305
			if (0 != name.compare(0, partition.size(), partition))
M
Merge  
Michael Kolupaev 已提交
2306 2307 2308 2309 2310 2311 2312 2313 2314 2315 2316 2317 2318 2319 2320 2321 2322 2323 2324 2325 2326
				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;

	{
2327
		/// TODO Это необходимо лишь в пределах одного месяца.
M
Merge  
Michael Kolupaev 已提交
2328 2329 2330 2331 2332 2333 2334 2335 2336 2337 2338 2339 2340 2341 2342 2343 2344 2345 2346 2347 2348 2349 2350 2351 2352 2353 2354 2355 2356 2357 2358 2359 2360 2361 2362
		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);
2363

2364 2365
	/// Если надо - дожидаемся выполнения операции на себе или на всех репликах.
	if (settings.replication_alter_partitions_sync != 0)
M
Merge  
Michael Kolupaev 已提交
2366
	{
2367 2368 2369 2370 2371 2372 2373 2374 2375 2376
		size_t i = 0;
		for (LogEntry & entry : entries)
		{
			String log_znode_path = dynamic_cast<zkutil::Op::Create &>(ops[i]).getPathCreated();
			entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);

			if (settings.replication_alter_partitions_sync == 1)
				waitForReplicaToProcessLogEntry(replica_name, entry);
			else
				waitForAllReplicasToProcessLogEntry(entry);
M
Merge  
Michael Kolupaev 已提交
2377

2378 2379
			++i;
		}
M
Merge  
Michael Kolupaev 已提交
2380
	}
M
Merge  
Michael Kolupaev 已提交
2381 2382
}

A
Merge  
Alexey Milovidov 已提交
2383

M
Merge  
Michael Kolupaev 已提交
2384 2385
void StorageReplicatedMergeTree::drop()
{
2386 2387
	if (is_readonly)
		throw Exception("Can't drop readonly replicated table (need to drop data in ZooKeeper as well)", ErrorCodes::TABLE_IS_READ_ONLY);
M
Merge  
Michael Kolupaev 已提交
2388

A
Merge  
Alexey Milovidov 已提交
2389 2390
	auto zookeeper = getZooKeeper();

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

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

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

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

A
Merge  
Alexey Milovidov 已提交
2408

M
Merge  
Michael Kolupaev 已提交
2409 2410 2411 2412 2413 2414 2415 2416 2417 2418 2419 2420 2421 2422 2423
void StorageReplicatedMergeTree::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name)
{
	std::string new_full_path = new_path_to_db + escapeForFileName(new_table_name) + '/';

	data.setPath(new_full_path, true);
	if (unreplicated_data)
		unreplicated_data->setPath(new_full_path + "unreplicated/", false);

	database_name = new_database_name;
	table_name = new_table_name;
	full_path = new_full_path;

	/// TODO: Можно обновить названия логгеров.
}

A
Merge  
Alexey Milovidov 已提交
2424

M
Merge  
Michael Kolupaev 已提交
2425 2426
AbandonableLockInZooKeeper StorageReplicatedMergeTree::allocateBlockNumber(const String & month_name)
{
A
Merge  
Alexey Milovidov 已提交
2427 2428
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
2429 2430 2431 2432 2433 2434 2435 2436
	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 已提交
2437
		for (size_t i = 0; i < RESERVED_BLOCK_NUMBERS; ++i)
M
Merge  
Michael Kolupaev 已提交
2438 2439 2440 2441 2442 2443 2444 2445 2446 2447 2448 2449 2450
		{
			ops.push_back(new zkutil::Op::Create(month_path + "/skip_increment", "", acl, zkutil::CreateMode::Persistent));
			ops.push_back(new zkutil::Op::Remove(month_path + "/skip_increment", -1));
		}
		/// Игнорируем ошибки - не получиться могло только если кто-то еще выполнил эту строчку раньше нас.
		zookeeper->tryMulti(ops);
	}

	return AbandonableLockInZooKeeper(
		zookeeper_path + "/block_numbers/" + month_name + "/block-",
		zookeeper_path + "/temp", *zookeeper);
}

A
Merge  
Alexey Milovidov 已提交
2451

M
Merge  
Michael Kolupaev 已提交
2452
void StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry(const LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
2453
{
A
Merge  
Alexey Milovidov 已提交
2454
	auto zookeeper = getZooKeeper();
M
Merge  
Michael Kolupaev 已提交
2455 2456
	LOG_DEBUG(log, "Waiting for all replicas to process " << entry.znode_name);

A
Merge  
Alexey Milovidov 已提交
2457 2458 2459 2460 2461 2462 2463 2464 2465 2466
	Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
	for (const String & replica : replicas)
		waitForReplicaToProcessLogEntry(replica, entry);

	LOG_DEBUG(log, "Finished waiting for all replicas to process " << entry.znode_name);
}


void StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry(const String & replica, const LogEntry & entry)
{
A
Merge  
Alexey Milovidov 已提交
2467 2468
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
2469
	UInt64 log_index = parse<UInt64>(entry.znode_name.substr(entry.znode_name.size() - 10));
M
Merge  
Michael Kolupaev 已提交
2470 2471
	String log_entry_str = entry.toString();

A
Merge  
Alexey Milovidov 已提交
2472
	LOG_DEBUG(log, "Waiting for " << replica << " to pull " << entry.znode_name << " to queue");
M
Merge  
Michael Kolupaev 已提交
2473

A
Merge  
Alexey Milovidov 已提交
2474 2475 2476 2477
	/// Дождемся, пока запись попадет в очередь реплики.
	while (true)
	{
		zkutil::EventPtr event = new Poco::Event;
M
Merge  
Michael Kolupaev 已提交
2478

A
Merge  
Alexey Milovidov 已提交
2479 2480 2481
		String pointer = zookeeper->get(zookeeper_path + "/replicas/" + replica + "/log_pointer", nullptr, event);
		if (!pointer.empty() && parse<UInt64>(pointer) > log_index)
			break;
M
Merge  
Michael Kolupaev 已提交
2482

A
Merge  
Alexey Milovidov 已提交
2483 2484
		event->wait();
	}
M
Merge  
Michael Kolupaev 已提交
2485

A
Merge  
Alexey Milovidov 已提交
2486
	LOG_DEBUG(log, "Looking for " << entry.znode_name << " in " << replica << " queue");
M
Merge  
Michael Kolupaev 已提交
2487

A
Merge  
Alexey Milovidov 已提交
2488 2489 2490
	/// Найдем запись в очереди реплики.
	Strings queue_entries = zookeeper->getChildren(zookeeper_path + "/replicas/" + replica + "/queue");
	String entry_to_wait_for;
M
Merge  
Michael Kolupaev 已提交
2491

A
Merge  
Alexey Milovidov 已提交
2492 2493 2494 2495 2496
	for (const String & entry_name : queue_entries)
	{
		String queue_entry_str;
		bool exists = zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/queue/" + entry_name, queue_entry_str);
		if (exists && queue_entry_str == log_entry_str)
M
Merge  
Michael Kolupaev 已提交
2497
		{
A
Merge  
Alexey Milovidov 已提交
2498 2499
			entry_to_wait_for = entry_name;
			break;
M
Merge  
Michael Kolupaev 已提交
2500
		}
A
Merge  
Alexey Milovidov 已提交
2501
	}
M
Merge  
Michael Kolupaev 已提交
2502

A
Merge  
Alexey Milovidov 已提交
2503 2504 2505
	/// Пока искали запись, ее уже выполнили и удалили.
	if (entry_to_wait_for.empty())
		return;
M
Merge  
Michael Kolupaev 已提交
2506

A
Merge  
Alexey Milovidov 已提交
2507
	LOG_DEBUG(log, "Waiting for " << entry_to_wait_for << " to disappear from " << replica << " queue");
M
Merge  
Michael Kolupaev 已提交
2508

A
Merge  
Alexey Milovidov 已提交
2509 2510 2511 2512
	/// Дождемся, пока запись исчезнет из очереди реплики.
	while (true)
	{
		zkutil::EventPtr event = new Poco::Event;
M
Merge  
Michael Kolupaev 已提交
2513

A
Merge  
Alexey Milovidov 已提交
2514 2515 2516 2517
		String unused;
		/// get вместо exists, чтобы не утек watch, если ноды уже нет.
		if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/queue/" + entry_to_wait_for, unused, nullptr, event))
			break;
M
Merge  
Michael Kolupaev 已提交
2518

A
Merge  
Alexey Milovidov 已提交
2519
		event->wait();
M
Merge  
Michael Kolupaev 已提交
2520 2521 2522 2523
	}
}


2524
void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields)
2525
{
A
Merge  
Alexey Milovidov 已提交
2526 2527
	auto zookeeper = getZooKeeper();

2528
	res.is_leader = is_leader_node;
2529
	res.is_readonly = is_readonly;
2530
	res.is_session_expired = !zookeeper || zookeeper->expired();
2531 2532 2533 2534 2535 2536 2537 2538

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

		res.inserts_in_queue = 0;
		res.merges_in_queue = 0;
A
Merge  
Alexey Milovidov 已提交
2539
		res.queue_oldest_time = 0;
2540 2541 2542 2543 2544 2545 2546

		for (const LogEntryPtr & entry : queue)
		{
			if (entry->type == LogEntry::GET_PART)
				++res.inserts_in_queue;
			if (entry->type == LogEntry::MERGE_PARTS)
				++res.merges_in_queue;
A
Merge  
Alexey Milovidov 已提交
2547 2548 2549

			if (entry->create_time && (!res.queue_oldest_time || entry->create_time < res.queue_oldest_time))
				res.queue_oldest_time = entry->create_time;
2550 2551 2552 2553 2554 2555 2556 2557 2558 2559 2560 2561 2562
		}
	}

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

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

2563
	if (res.is_session_expired || !with_zk_fields)
2564 2565 2566 2567 2568 2569 2570 2571 2572 2573
	{
		res.log_max_index = 0;
		res.log_pointer = 0;
		res.total_replicas = 0;
		res.active_replicas = 0;
	}
	else
	{
		auto log_entries = zookeeper->getChildren(zookeeper_path + "/log");

2574 2575 2576 2577 2578 2579 2580 2581 2582 2583 2584 2585
		if (log_entries.empty())
		{
			res.log_max_index = 0;
		}
		else
		{
			const String & last_log_entry = *std::max_element(log_entries.begin(), log_entries.end());
			res.log_max_index = parse<UInt64>(last_log_entry.substr(strlen("log-")));
		}

		String log_pointer_str = zookeeper->get(replica_path + "/log_pointer");
		res.log_pointer = log_pointer_str.empty() ? 0 : parse<UInt64>(log_pointer_str);
2586 2587 2588 2589 2590 2591 2592 2593 2594 2595 2596

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

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

2597

2598
void StorageReplicatedMergeTree::fetchPartition(const Field & partition, const String & from_, const Settings & settings)
2599
{
A
Merge  
Alexey Milovidov 已提交
2600 2601
	auto zookeeper = getZooKeeper();

2602 2603
	String partition_str = MergeTreeData::getMonthName(partition);

2604
	String from = from_;
2605 2606
	if (from.back() == '/')
		from.resize(from.size() - 1);
2607

2608 2609
	LOG_INFO(log, "Will fetch partition " << partition_str << " from shard " << from_);

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

2618 2619 2620 2621 2622 2623 2624 2625 2626 2627 2628 2629
	/// Список реплик шарда-источника.
	zkutil::Strings replicas = zookeeper->getChildren(from + "/replicas");

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

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

	if (active_replicas.empty())
2630
		throw Exception("No active replicas for shard " + from, ErrorCodes::NO_ACTIVE_REPLICAS);
2631 2632

	/** Надо выбрать лучшую (наиболее актуальную) реплику.
2633
	  * Это реплика с максимальным log_pointer, затем с минимальным размером queue.
2634 2635
	  * NOTE Это не совсем лучший критерий. Для скачивания старых партиций это не имеет смысла,
	  *  и было бы неплохо уметь выбирать реплику, ближайшую по сети.
2636
	  * NOTE Разумеется, здесь есть data race-ы. Можно решить ретраями.
2637
	  */
2638 2639 2640 2641 2642 2643
	Int64 max_log_pointer = -1;
	UInt64 min_queue_size = std::numeric_limits<UInt64>::max();
	String best_replica;

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

2646
		String log_pointer_str = zookeeper->get(current_replica_path + "/log_pointer");
2647 2648 2649
		Int64 log_pointer = log_pointer_str.empty() ? 0 : parse<UInt64>(log_pointer_str);

		zkutil::Stat stat;
2650
		zookeeper->get(current_replica_path + "/queue", &stat);
2651 2652 2653 2654 2655 2656 2657 2658 2659 2660 2661 2662 2663 2664 2665 2666 2667
		size_t queue_size = stat.numChildren;

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

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

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

2668 2669
	String best_replica_path = from + "/replicas/" + best_replica;

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

2672 2673 2674 2675 2676 2677 2678 2679 2680 2681 2682 2683 2684 2685 2686 2687 2688 2689 2690 2691 2692 2693 2694 2695 2696 2697 2698 2699 2700 2701
	/** Пытаемся скачать эти куски.
	  * Часть из них могла удалиться из-за мерджа.
	  * В этом случае, обновляем информацию о доступных кусках и пробуем снова.
	  */

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

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

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

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

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

			parts_to_fetch = std::move(parts_to_fetch_partition);
2702 2703 2704

			if (parts_to_fetch.empty())
				throw Exception("Partition " + partition_str + " on " + best_replica_path + " doesn't exist", ErrorCodes::PARTITION_DOESNT_EXIST);
2705 2706 2707 2708 2709 2710 2711 2712 2713 2714 2715 2716 2717 2718 2719 2720 2721 2722 2723 2724 2725 2726 2727 2728 2729 2730 2731 2732 2733 2734 2735
		}
		else
		{
			for (const String & missing_part : missing_parts)
			{
				String containing_part = active_parts_set.getContainingPart(missing_part);
				if (!containing_part.empty())
					parts_to_fetch.push_back(containing_part);
				else
					LOG_WARNING(log, "Part " << missing_part << " on replica " << best_replica_path << " has been vanished.");
			}
		}

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

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

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

2737 2738
		++try_no;
	} while (!missing_parts.empty());
2739 2740 2741
}


2742 2743 2744 2745 2746 2747 2748 2749 2750 2751 2752 2753 2754
void StorageReplicatedMergeTree::freezePartition(const Field & partition, const Settings & settings)
{
	/// Префикс может быть произвольным. Не обязательно месяц - можно указать лишь год.
	String prefix = partition.getType() == Field::Types::UInt64
		? toString(partition.get<UInt64>())
		: partition.safeGet<String>();

	data.freezePartition(prefix);
	if (unreplicated_data)
		unreplicated_data->freezePartition(prefix);
}


M
Merge  
Michael Kolupaev 已提交
2755
}