StorageReplicatedMergeTree.cpp 113.4 KB
Newer Older
1
#include <statdaemons/ext/range.hpp>
2
#include <DB/Storages/ColumnsDescription.h>
M
Merge  
Michael Kolupaev 已提交
3
#include <DB/Storages/StorageReplicatedMergeTree.h>
M
Merge  
Michael Kolupaev 已提交
4
#include <DB/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h>
M
Merge  
Michael Kolupaev 已提交
5
#include <DB/Storages/MergeTree/ReplicatedMergeTreePartsExchange.h>
6
#include <DB/Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h>
M
Merge  
Michael Kolupaev 已提交
7
#include <DB/Storages/MergeTree/MergeTreePartChecker.h>
8
#include <DB/Storages/MergeTree/MergeList.h>
A
Merge  
Andrey Mironov 已提交
9
#include <DB/Storages/MergeTree/MergeTreeWhereOptimizer.h>
M
Merge  
Michael Kolupaev 已提交
10 11 12
#include <DB/Parsers/formatAST.h>
#include <DB/IO/WriteBufferFromOStream.h>
#include <DB/IO/ReadBufferFromString.h>
M
Merge  
Michael Kolupaev 已提交
13
#include <DB/Interpreters/InterpreterAlterQuery.h>
M
Merge  
Michael Kolupaev 已提交
14
#include <DB/Common/VirtualColumnUtils.h>
15
#include <DB/Parsers/ASTInsertQuery.h>
16
#include <DB/DataStreams/AddingConstColumnBlockInputStream.h>
17
#include <DB/Common/Macros.h>
A
Merge  
Alexey Milovidov 已提交
18
#include <DB/Common/formatReadable.h>
19
#include <Poco/DirectoryIterator.h>
M
Merge  
Michael Kolupaev 已提交
20
#include <time.h>
M
Merge  
Michael Kolupaev 已提交
21

22

M
Merge  
Michael Kolupaev 已提交
23 24 25
namespace DB
{

M
Merge  
Michael Kolupaev 已提交
26

M
Merge  
Michael Kolupaev 已提交
27
const auto ERROR_SLEEP_MS = 1000;
M
Merge  
Michael Kolupaev 已提交
28
const auto MERGE_SELECTING_SLEEP_MS = 5 * 1000;
M
Merge  
Michael Kolupaev 已提交
29

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

M
Merge  
Michael Kolupaev 已提交
32

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

	bool skip_sanity_checks = false;

71
	try
M
Merge  
Michael Kolupaev 已提交
72
	{
A
Merge  
Alexey Milovidov 已提交
73
		if (current_zookeeper && current_zookeeper->exists(replica_path + "/flags/force_restore_data"))
74 75
		{
			skip_sanity_checks = true;
A
Merge  
Alexey Milovidov 已提交
76
			current_zookeeper->remove(replica_path + "/flags/force_restore_data");
M
Merge  
Michael Kolupaev 已提交
77

78 79 80 81 82 83 84 85 86 87
			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 已提交
88
			current_zookeeper = nullptr;
89 90 91
		}
		else
			throw;
M
Merge  
Michael Kolupaev 已提交
92 93 94 95
	}

	data.loadDataParts(skip_sanity_checks);

A
Merge  
Alexey Milovidov 已提交
96
	if (!current_zookeeper)
M
Merge  
Michael Kolupaev 已提交
97
	{
M
Merge  
Michael Kolupaev 已提交
98 99 100
		if (!attach)
			throw Exception("Can't create replicated table without ZooKeeper", ErrorCodes::NO_ZOOKEEPER);

101
		/// Не активируем реплику. Она будет в режиме readonly.
M
Merge  
Michael Kolupaev 已提交
102 103 104
		return;
	}

M
Merge  
Michael Kolupaev 已提交
105 106
	if (!attach)
	{
M
Merge  
Michael Kolupaev 已提交
107
		createTableIfNotExists();
M
Merge  
Michael Kolupaev 已提交
108

M
Merge  
Michael Kolupaev 已提交
109
		checkTableStructure(false, false);
M
Merge  
Michael Kolupaev 已提交
110
		createReplica();
M
Merge  
Michael Kolupaev 已提交
111 112 113
	}
	else
	{
M
Merge  
Michael Kolupaev 已提交
114
		checkTableStructure(skip_sanity_checks, true);
M
Merge  
Michael Kolupaev 已提交
115
		checkParts(skip_sanity_checks);
M
Merge  
Michael Kolupaev 已提交
116
	}
M
Merge  
Michael Kolupaev 已提交
117

M
Merge  
Michael Kolupaev 已提交
118
	initVirtualParts();
M
Merge  
Michael Kolupaev 已提交
119 120 121 122 123

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

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

		unreplicated_data->loadDataParts(skip_sanity_checks);

M
Merge  
Michael Kolupaev 已提交
133
		unreplicated_reader.reset(new MergeTreeDataSelectExecutor(*unreplicated_data));
M
Merge  
Michael Kolupaev 已提交
134
		unreplicated_merger.reset(new MergeTreeDataMerger(*unreplicated_data));
M
Merge  
Michael Kolupaev 已提交
135
	}
M
Merge  
Michael Kolupaev 已提交
136

137
	loadQueue();
M
Merge  
Michael Kolupaev 已提交
138

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

143

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

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

A
Merge  
Alexey Milovidov 已提交
173
	if (res->getZooKeeper())
M
Merge  
Michael Kolupaev 已提交
174 175
	{
		String endpoint_name = "ReplicatedMergeTree:" + res->replica_path;
M
Merge  
Michael Kolupaev 已提交
176
		InterserverIOEndpointPtr endpoint = new ReplicatedMergeTreePartsServer(res->data, *res);
M
Merge  
Michael Kolupaev 已提交
177 178
		res->endpoint_holder = new InterserverIOEndpointHolder(endpoint_name, endpoint, res->context.getInterserverIOHandler());
	}
179

M
Merge  
Michael Kolupaev 已提交
180
	return res_ptr;
M
Merge  
Michael Kolupaev 已提交
181 182
}

A
Merge  
Alexey Milovidov 已提交
183

M
Merge  
Michael Kolupaev 已提交
184 185 186 187 188 189 190 191
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 已提交
192

A
Merge  
Alexey Milovidov 已提交
193

M
Merge  
Michael Kolupaev 已提交
194
void StorageReplicatedMergeTree::createTableIfNotExists()
M
Merge  
Michael Kolupaev 已提交
195
{
A
Merge  
Alexey Milovidov 已提交
196 197
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
198 199
	if (zookeeper->exists(zookeeper_path))
		return;
M
Merge  
Michael Kolupaev 已提交
200

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

M
Merge  
Michael Kolupaev 已提交
203 204
	zookeeper->createAncestors(zookeeper_path);

M
Merge  
Michael Kolupaev 已提交
205
	/// Запишем метаданные таблицы, чтобы реплики могли сверять с ними параметры таблицы.
M
Merge  
Michael Kolupaev 已提交
206 207 208 209 210 211 212 213
	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 已提交
214

M
Merge  
Michael Kolupaev 已提交
215 216 217 218 219
	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));
220 221 222
	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 已提交
223 224 225 226 227 228 229 230 231 232 233 234 235 236 237 238 239 240 241
										 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 已提交
242
}
M
Merge  
Michael Kolupaev 已提交
243

A
Merge  
Alexey Milovidov 已提交
244

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

M
Merge  
Michael Kolupaev 已提交
252
	String metadata_str = zookeeper->get(zookeeper_path + "/metadata");
M
Merge  
Michael Kolupaev 已提交
253 254 255 256 257 258 259 260 261 262 263 264 265
	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 已提交
266 267
	/// NOTE: Можно сделать менее строгую проверку совпадения выражений, чтобы таблицы не ломались от небольших изменений
	///       в коде formatAST.
M
Merge  
Michael Kolupaev 已提交
268
	assertString(formattedAST(data.primary_expr_ast), buf);
M
Merge  
Michael Kolupaev 已提交
269
	assertString("\n", buf);
M
Merge  
Michael Kolupaev 已提交
270
	assertEOF(buf);
M
Merge  
Michael Kolupaev 已提交
271 272

	zkutil::Stat stat;
273
	auto columns_desc = ColumnsDescription<true>::parse(zookeeper->get(zookeeper_path + "/columns", &stat));
274 275 276 277 278

	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 已提交
279
	columns_version = stat.version;
280

281 282 283 284
	if (columns != data.getColumnsListNonMaterialized() ||
		materialized_columns != data.materialized_columns ||
		alias_columns != data.alias_columns ||
		column_defaults != data.column_defaults)
M
Merge  
Michael Kolupaev 已提交
285
	{
286 287 288 289
		if (allow_alter &&
			(skip_sanity_checks ||
			 data.getColumnsListNonMaterialized().sizeOfDifference(columns) +
			 data.materialized_columns.sizeOfDifference(materialized_columns) <= 2))
M
Merge  
Michael Kolupaev 已提交
290
		{
M
Merge  
Michael Kolupaev 已提交
291
			LOG_WARNING(log, "Table structure in ZooKeeper is a little different from local table structure. Assuming ALTER.");
292 293

			/// Без всяких блокировок, потому что таблица еще не создана.
294 295
			InterpreterAlterQuery::updateMetadata(database_name, table_name, columns,
				materialized_columns, alias_columns, column_defaults, context);
296
			data.setColumnsList(columns);
297 298 299
			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 已提交
300
		}
M
Merge  
Michael Kolupaev 已提交
301
		else
M
Merge  
Michael Kolupaev 已提交
302
		{
M
Merge  
Michael Kolupaev 已提交
303
			throw Exception("Table structure in ZooKeeper is too different from local table structure.",
M
Merge  
Michael Kolupaev 已提交
304
							ErrorCodes::INCOMPATIBLE_COLUMNS);
M
Merge  
Michael Kolupaev 已提交
305
		}
M
Merge  
Michael Kolupaev 已提交
306 307
	}
}
M
Merge  
Michael Kolupaev 已提交
308

A
Merge  
Alexey Milovidov 已提交
309

M
Merge  
Michael Kolupaev 已提交
310 311
void StorageReplicatedMergeTree::createReplica()
{
A
Merge  
Alexey Milovidov 已提交
312 313
	auto zookeeper = getZooKeeper();

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

M
Merge  
Michael Kolupaev 已提交
316 317 318 319 320 321 322 323
	/// Создадим пустую реплику. Ноду 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));
324 325 326 327 328 329 330 331 332 333 334 335

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

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

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

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

M
Merge  
Michael Kolupaev 已提交
349 350 351
	Stat stat;
	zookeeper->exists(replica_path, &stat);
	auto my_create_time = stat.czxid;
M
Merge  
Michael Kolupaev 已提交
352

M
Merge  
Michael Kolupaev 已提交
353 354
	std::random_shuffle(replicas.begin(), replicas.end());
	for (const String & replica : replicas)
M
Merge  
Michael Kolupaev 已提交
355
	{
M
Merge  
Michael Kolupaev 已提交
356 357 358 359 360 361 362 363
		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 已提交
364 365
	}

M
Merge  
Michael Kolupaev 已提交
366
	if (source_replica.empty())
M
Merge  
Michael Kolupaev 已提交
367
	{
M
Merge  
Michael Kolupaev 已提交
368
		LOG_INFO(log, "This is the first replica");
M
Merge  
Michael Kolupaev 已提交
369
	}
M
Merge  
Michael Kolupaev 已提交
370
	else
M
Merge  
Michael Kolupaev 已提交
371
	{
M
Merge  
Michael Kolupaev 已提交
372
		LOG_INFO(log, "Will mimic " << source_replica);
M
Merge  
Michael Kolupaev 已提交
373

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

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

M
Merge  
Michael Kolupaev 已提交
417 418 419 420 421 422 423 424
		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;

425 426 427 428 429 430 431 432 433 434 435 436
			/// Узнаем время создания part-а, если он ещё не удалён (не был, например, смерджен).
			{
				zkutil::Stat stat;
				String unused;
				if (zookeeper->tryGet(source_path + "/parts/" + name, unused, &stat))
					log_entry.create_time = stat.ctime / 1000;

				/** Иначе временем создания будет текущее время.
				  * Это время используется для измерения отставания реплик.
				  */
			}

M
Merge  
Michael Kolupaev 已提交
437 438 439 440 441 442 443 444 445 446
			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 已提交
447
	}
M
Merge  
Michael Kolupaev 已提交
448

449 450 451 452 453 454
	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 已提交
455
}
M
Merge  
Michael Kolupaev 已提交
456 457


M
Merge  
Michael Kolupaev 已提交
458
void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
M
Merge  
Michael Kolupaev 已提交
459
{
A
Merge  
Alexey Milovidov 已提交
460 461
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
462
	Strings expected_parts_vec = zookeeper->getChildren(replica_path + "/parts");
M
Merge  
Michael Kolupaev 已提交
463 464

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

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

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

M
Merge  
Michael Kolupaev 已提交
472 473 474 475 476
	for (const auto & part : parts)
	{
		if (expected_parts.count(part->name))
			expected_parts.erase(part->name);
		else
M
Merge  
Michael Kolupaev 已提交
477
			unexpected_parts.insert(part);
M
Merge  
Michael Kolupaev 已提交
478 479
	}

M
Merge  
Michael Kolupaev 已提交
480
	/// Какие локальные куски добавить в ZK.
M
Merge  
Michael Kolupaev 已提交
481
	MergeTreeData::DataPartsVector parts_to_add;
M
Merge  
Michael Kolupaev 已提交
482 483 484 485

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

M
Merge  
Michael Kolupaev 已提交
486 487
	for (const String & missing_name : expected_parts)
	{
M
Merge  
Michael Kolupaev 已提交
488
		/// Если локально не хватает какого-то куска, но есть покрывающий его кусок, можно заменить в ZK недостающий покрывающим.
M
Merge  
Michael Kolupaev 已提交
489
		auto containing = data.getActiveContainingPart(missing_name);
M
Merge  
Michael Kolupaev 已提交
490 491 492 493 494 495 496 497 498 499
		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 已提交
500
		{
M
Merge  
Michael Kolupaev 已提交
501
			LOG_ERROR(log, "Fetching missing part " << missing_name);
M
Merge  
Michael Kolupaev 已提交
502
			parts_to_fetch.push_back(missing_name);
M
Merge  
Michael Kolupaev 已提交
503 504
		}
	}
M
Merge  
Michael Kolupaev 已提交
505

M
Merge  
Michael Kolupaev 已提交
506 507 508 509 510 511
	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 已提交
512
					 + toString(expected_parts.size()) + " missing obsolete parts, "
M
Merge  
Michael Kolupaev 已提交
513 514
					 + toString(parts_to_fetch.size()) + " missing parts";

515 516 517 518 519 520 521 522 523 524 525 526 527 528 529
	/** Можно автоматически синхронизировать данные,
	  * если количество ошибок каждого из четырёх типов не больше соответствующих порогов,
	  * или если отношение общего количества ошибок к общему количеству кусков (минимальному - в локальной файловой системе или в 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 已提交
530

M
Merge  
Michael Kolupaev 已提交
531 532
	if (insane)
	{
533 534 535 536 537
		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 已提交
538 539
	}

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

M
Merge  
Michael Kolupaev 已提交
545
		zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
546
		checkPartAndAddToZooKeeper(part, ops);
M
Merge  
Michael Kolupaev 已提交
547
		zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
548
	}
M
Merge  
Michael Kolupaev 已提交
549

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

		zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
556
		ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + name + "/columns", -1));
M
Merge  
Michael Kolupaev 已提交
557 558 559 560 561 562 563 564 565 566 567 568
		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 已提交
569
		log_entry.source_replica = "";
M
Merge  
Michael Kolupaev 已提交
570 571 572
		log_entry.new_part_name = name;

		/// Полагаемся, что это происходит до загрузки очереди (loadQueue).
M
Merge  
Michael Kolupaev 已提交
573
		zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
574
		ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + name + "/columns", -1));
M
Merge  
Michael Kolupaev 已提交
575 576
		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 已提交
577 578
		ops.push_back(new zkutil::Op::Create(
			replica_path + "/queue/queue-", log_entry.toString(), zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
M
Merge  
Michael Kolupaev 已提交
579
		zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
580 581 582
	}

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

A
Merge  
Alexey Milovidov 已提交
590

M
Merge  
Michael Kolupaev 已提交
591 592 593 594 595 596 597
void StorageReplicatedMergeTree::initVirtualParts()
{
	auto parts = data.getDataParts();
	for (const auto & part : parts)
		virtual_parts.add(part->name);
}

A
Merge  
Alexey Milovidov 已提交
598

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

M
Merge  
Michael Kolupaev 已提交
603 604 605
	if (part_name.empty())
		part_name = part->name;

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

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

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

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

647 648 649
	ops.push_back(new zkutil::Op::Check(
		zookeeper_path + "/columns",
		expected_columns_version));
M
Merge  
Michael Kolupaev 已提交
650
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
651
		replica_path + "/parts/" + part_name,
M
Merge  
Michael Kolupaev 已提交
652
		"",
M
Merge  
Michael Kolupaev 已提交
653
		zookeeper->getDefaultACL(),
M
Merge  
Michael Kolupaev 已提交
654
		zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
655
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
656
		replica_path + "/parts/" + part_name + "/columns",
M
Merge  
Michael Kolupaev 已提交
657 658 659
		part->columns.toString(),
		zookeeper->getDefaultACL(),
		zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
660
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
661
		replica_path + "/parts/" + part_name + "/checksums",
M
Merge  
Michael Kolupaev 已提交
662
		part->checksums.toString(),
M
Merge  
Michael Kolupaev 已提交
663
		zookeeper->getDefaultACL(),
M
Merge  
Michael Kolupaev 已提交
664 665 666
		zkutil::CreateMode::Persistent));
}

A
Merge  
Alexey Milovidov 已提交
667

M
Merge  
Michael Kolupaev 已提交
668 669
void StorageReplicatedMergeTree::loadQueue()
{
A
Merge  
Alexey Milovidov 已提交
670 671
	auto zookeeper = getZooKeeper();

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

M
Merge  
Michael Kolupaev 已提交
674
	Strings children = zookeeper->getChildren(replica_path + "/queue");
M
Merge  
Michael Kolupaev 已提交
675 676 677
	std::sort(children.begin(), children.end());
	for (const String & child : children)
	{
A
Merge  
Alexey Milovidov 已提交
678 679
		zkutil::Stat stat;
		String s = zookeeper->get(replica_path + "/queue/" + child, &stat);
680
		LogEntryPtr entry = LogEntry::parse(s, stat);
M
Merge  
Michael Kolupaev 已提交
681 682
		entry->znode_name = child;
		entry->addResultToVirtualParts(*this);
M
Merge  
Michael Kolupaev 已提交
683
		queue.push_back(entry);
M
Merge  
Michael Kolupaev 已提交
684 685 686
	}
}

A
Merge  
Alexey Milovidov 已提交
687

M
Merge  
Michael Kolupaev 已提交
688
void StorageReplicatedMergeTree::pullLogsToQueue(zkutil::EventPtr next_update_event)
M
Merge  
Michael Kolupaev 已提交
689
{
A
Merge  
Alexey Milovidov 已提交
690 691
	auto zookeeper = getZooKeeper();

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

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

M
Merge  
Michael Kolupaev 已提交
697
	if (index_str.empty())
M
Merge  
Michael Kolupaev 已提交
698
	{
M
Merge  
Michael Kolupaev 已提交
699 700 701
		/// Если у нас еще нет указателя на лог, поставим указатель на первую запись в нем.
		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 已提交
702

M
Merge  
Michael Kolupaev 已提交
703
		zookeeper->set(replica_path + "/log_pointer", toString(index));
M
Merge  
Michael Kolupaev 已提交
704 705
	}
	else
M
Merge  
Michael Kolupaev 已提交
706
	{
M
Merge  
Michael Kolupaev 已提交
707
		index = parse<UInt64>(index_str);
M
Merge  
Michael Kolupaev 已提交
708
	}
M
Merge  
Michael Kolupaev 已提交
709

M
Merge  
Michael Kolupaev 已提交
710 711
	UInt64 first_index = index;

M
Merge  
Michael Kolupaev 已提交
712
	size_t count = 0;
M
Merge  
Michael Kolupaev 已提交
713
	String entry_str;
A
Merge  
Alexey Milovidov 已提交
714 715
	zkutil::Stat stat;
	while (zookeeper->tryGet(zookeeper_path + "/log/log-" + padIndex(index), entry_str, &stat))
M
Merge  
Michael Kolupaev 已提交
716
	{
M
Merge  
Michael Kolupaev 已提交
717
		++count;
M
Merge  
Michael Kolupaev 已提交
718
		++index;
M
Merge  
Michael Kolupaev 已提交
719

720
		LogEntryPtr entry = LogEntry::parse(entry_str, stat);
M
Merge  
Michael Kolupaev 已提交
721

M
Merge  
Michael Kolupaev 已提交
722 723 724
		/// Одновременно добавим запись в очередь и продвинем указатель на лог.
		zkutil::Ops ops;
		ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
725
			replica_path + "/queue/queue-", entry_str, zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
M
Merge  
Michael Kolupaev 已提交
726
		ops.push_back(new zkutil::Op::SetData(
M
Merge  
Michael Kolupaev 已提交
727
			replica_path + "/log_pointer", toString(index), -1));
M
Merge  
Michael Kolupaev 已提交
728
		auto results = zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
729

730
		String path_created = dynamic_cast<zkutil::Op::Create &>(ops[0]).getPathCreated();
M
Merge  
Michael Kolupaev 已提交
731 732
		entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
		entry->addResultToVirtualParts(*this);
M
Merge  
Michael Kolupaev 已提交
733
		queue.push_back(entry);
M
Merge  
Michael Kolupaev 已提交
734
	}
M
Merge  
Michael Kolupaev 已提交
735

M
Merge  
Michael Kolupaev 已提交
736 737 738 739
	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 已提交
740
	}
M
Merge  
Michael Kolupaev 已提交
741

M
Merge  
Michael Kolupaev 已提交
742 743 744
	if (!count)
		return;

M
Merge  
Michael Kolupaev 已提交
745 746
	if (queue_task_handle)
		queue_task_handle->wake();
M
Merge  
Michael Kolupaev 已提交
747

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

A
Merge  
Alexey Milovidov 已提交
751

M
Merge  
Michael Kolupaev 已提交
752
bool StorageReplicatedMergeTree::shouldExecuteLogEntry(const LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
753
{
M
Merge  
Michael Kolupaev 已提交
754 755
	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 已提交
756
	{
757 758
		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.");
M
Merge  
Michael Kolupaev 已提交
759 760 761
		return false;
	}

M
Merge  
Michael Kolupaev 已提交
762 763 764 765 766 767 768 769 770 771 772
	if (entry.type == LogEntry::MERGE_PARTS)
	{
		/** Если какая-то из нужных частей сейчас передается или мерджится, подождем окончания этой операции.
		  * Иначе, даже если всех нужных частей для мерджа нет, нужно попытаться сделать мердж.
		  * Если каких-то частей не хватает, вместо мерджа будет попытка скачать кусок.
		  * Такая ситуация возможна, если получение какого-то куска пофейлилось, и его переместили в конец очереди.
		  */
		for (const auto & name : entry.parts_to_merge)
		{
			if (future_parts.count(name))
			{
M
Merge  
Michael Kolupaev 已提交
773
				LOG_TRACE(log, "Not merging into part " << entry.new_part_name << " because part " << name << " is not ready yet.");
M
Merge  
Michael Kolupaev 已提交
774 775 776
				return false;
			}
		}
777 778 779 780 781 782

		if (merger.isCancelled())
		{
			LOG_DEBUG(log, "Not executing log entry for part " << entry.new_part_name << " because merges are cancelled now.");
			return false;
		}
M
Merge  
Michael Kolupaev 已提交
783 784 785
	}

	return true;
M
Merge  
Michael Kolupaev 已提交
786 787
}

A
Merge  
Alexey Milovidov 已提交
788

M
Merge  
Michael Kolupaev 已提交
789
bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry, BackgroundProcessingPool::Context & pool_context)
M
Merge  
Michael Kolupaev 已提交
790
{
A
Merge  
Alexey Milovidov 已提交
791 792
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
793
	if (entry.type == LogEntry::DROP_RANGE)
M
Merge  
Michael Kolupaev 已提交
794 795 796 797
	{
		executeDropRange(entry);
		return true;
	}
M
Merge  
Michael Kolupaev 已提交
798

M
Merge  
Michael Kolupaev 已提交
799
	if (entry.type == LogEntry::GET_PART ||
M
Merge  
Michael Kolupaev 已提交
800 801
		entry.type == LogEntry::MERGE_PARTS ||
		entry.type == LogEntry::ATTACH_PART)
M
Merge  
Michael Kolupaev 已提交
802 803
	{
		/// Если у нас уже есть этот кусок или покрывающий его кусок, ничего делать не нужно.
M
Merge  
Michael Kolupaev 已提交
804
		MergeTreeData::DataPartPtr containing_part = data.getActiveContainingPart(entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
805

806
		/// Даже если кусок есть локально, его (в исключительных случаях) может не быть в zookeeper. Проверим, что он там есть.
M
Merge  
Michael Kolupaev 已提交
807
		if (containing_part && zookeeper->exists(replica_path + "/parts/" + containing_part->name))
M
Merge  
Michael Kolupaev 已提交
808
		{
M
Merge  
Michael Kolupaev 已提交
809 810
			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 已提交
811
			return true;
M
Merge  
Michael Kolupaev 已提交
812
		}
M
Merge  
Michael Kolupaev 已提交
813 814
	}

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

	bool do_fetch = false;

M
Merge  
Michael Kolupaev 已提交
820 821
	if (entry.type == LogEntry::GET_PART)
	{
M
Merge  
Michael Kolupaev 已提交
822
		do_fetch = true;
M
Merge  
Michael Kolupaev 已提交
823
	}
M
Merge  
Michael Kolupaev 已提交
824 825 826 827
	else if (entry.type == LogEntry::ATTACH_PART)
	{
		do_fetch = !executeAttachPart(entry);
	}
M
Merge  
Michael Kolupaev 已提交
828 829
	else if (entry.type == LogEntry::MERGE_PARTS)
	{
830 831 832 833 834 835 836 837
		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 已提交
838
		MergeTreeData::DataPartsVector parts;
839
		bool have_all_parts = true;
M
Merge  
Michael Kolupaev 已提交
840 841
		for (const String & name : entry.parts_to_merge)
		{
M
Merge  
Michael Kolupaev 已提交
842
			MergeTreeData::DataPartPtr part = data.getActiveContainingPart(name);
M
Merge  
Michael Kolupaev 已提交
843 844 845 846 847 848 849
			if (!part)
			{
				have_all_parts = false;
				break;
			}
			if (part->name != name)
			{
M
Merge  
Michael Kolupaev 已提交
850 851
				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 已提交
852 853 854
				have_all_parts = false;
				break;
			}
M
Merge  
Michael Kolupaev 已提交
855 856
			parts.push_back(part);
		}
M
Merge  
Michael Kolupaev 已提交
857

M
Merge  
Michael Kolupaev 已提交
858
		if (!have_all_parts)
M
Merge  
Michael Kolupaev 已提交
859
		{
M
Merge  
Michael Kolupaev 已提交
860 861 862
			/// Если нет всех нужных кусков, попробуем взять у кого-нибудь уже помердженный кусок.
			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 已提交
863
		}
M
Merge  
Michael Kolupaev 已提交
864 865
		else
		{
M
Merge  
Michael Kolupaev 已提交
866 867 868
			/// Если собираемся сливать большие куски, увеличим счетчик потоков, сливающих большие куски.
			for (const auto & part : parts)
			{
M
Merge  
Michael Kolupaev 已提交
869
				if (part->size_in_bytes > data.settings.max_bytes_to_merge_parts_small)
M
Merge  
Michael Kolupaev 已提交
870 871 872 873 874 875 876
				{
					pool_context.incrementCounter("big merges");
					pool_context.incrementCounter("replicated big merges");
					break;
				}
			}

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

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

882
			const auto & merge_entry = context.getMergeList().insert(database_name, table_name, entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
883
			MergeTreeData::Transaction transaction;
A
Merge  
Alexey Arno 已提交
884
			size_t aio_threshold = context.getSettings().min_bytes_to_use_direct_io;
A
Merge  
Alexey Milovidov 已提交
885
			MergeTreeData::DataPartPtr part = merger.mergeParts(parts, entry.new_part_name, *merge_entry, aio_threshold, &transaction, reserved_space);
M
Merge  
Michael Kolupaev 已提交
886 887

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

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

M
Merge  
Michael Kolupaev 已提交
894
			zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
895

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

M
Merge  
Michael Kolupaev 已提交
899 900 901
			/** При ZCONNECTIONLOSS или ZOPERATIONTIMEOUT можем зря откатить локальные изменения кусков.
			  * Это не проблема, потому что в таком случае слияние останется в очереди, и мы попробуем снова.
			  */
M
Merge  
Michael Kolupaev 已提交
902
			transaction.commit();
M
Merge  
Michael Kolupaev 已提交
903
			merge_selecting_event.set();
M
Merge  
Michael Kolupaev 已提交
904 905 906

			ProfileEvents::increment(ProfileEvents::ReplicatedPartMerges);
		}
M
Merge  
Michael Kolupaev 已提交
907 908 909 910 911
	}
	else
	{
		throw Exception("Unexpected log entry type: " + toString(static_cast<int>(entry.type)));
	}
M
Merge  
Michael Kolupaev 已提交
912 913 914

	if (do_fetch)
	{
M
Merge  
Michael Kolupaev 已提交
915 916
		String replica;

M
Merge  
Michael Kolupaev 已提交
917 918
		try
		{
M
Merge  
Michael Kolupaev 已提交
919
			replica = findReplicaHavingPart(entry.new_part_name, true);
A
Merge  
Alexey Milovidov 已提交
920 921 922 923 924 925 926 927 928 929 930 931 932 933 934 935 936

			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 已提交
937 938 939 940 941
			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 已提交
942

943
			fetchPart(entry.new_part_name, zookeeper_path + "/replicas/" + replica, false, entry.quorum);
M
Merge  
Michael Kolupaev 已提交
944 945 946 947 948 949 950 951 952 953 954 955

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

				/// Найдем действие по объединению этого куска с другими. Запомним других.
				StringSet parts_for_merge;
				LogEntries::iterator merge_entry;
				for (LogEntries::iterator it = queue.begin(); it != queue.end(); ++it)
				{
M
Merge  
Michael Kolupaev 已提交
963
					if ((*it)->type == LogEntry::MERGE_PARTS)
M
Merge  
Michael Kolupaev 已提交
964
					{
M
Merge  
Michael Kolupaev 已提交
965 966
						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 已提交
967
						{
M
Merge  
Michael Kolupaev 已提交
968
							parts_for_merge = StringSet((*it)->parts_to_merge.begin(), (*it)->parts_to_merge.end());
M
Merge  
Michael Kolupaev 已提交
969 970 971 972 973 974 975 976 977 978 979 980 981 982 983 984 985
							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 已提交
986 987
						if (((*it0)->type == LogEntry::MERGE_PARTS || (*it0)->type == LogEntry::GET_PART)
							&& parts_for_merge.count((*it0)->new_part_name))
M
Merge  
Michael Kolupaev 已提交
988 989 990 991
						{
							queue.splice(queue.end(), queue, it0, it);
						}
					}
M
Merge  
Michael Kolupaev 已提交
992 993 994 995 996 997

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

1003 1004 1005
				/** Если ни у какой активной реплики нет куска, и в очереди нет слияний с его участием,
				  * проверим, есть ли у любой (активной или неактивной) реплики такой кусок или покрывающий его.
				  */
M
Merge  
Michael Kolupaev 已提交
1006 1007
				if (replica.empty())
					enqueuePartForCheck(entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
1008 1009 1010 1011 1012 1013 1014 1015 1016
			}
			catch (...)
			{
				tryLogCurrentException(__PRETTY_FUNCTION__);
			}

			throw;
		}
	}
M
Merge  
Michael Kolupaev 已提交
1017 1018

	return true;
M
Merge  
Michael Kolupaev 已提交
1019 1020
}

A
Merge  
Alexey Milovidov 已提交
1021

M
Merge  
Michael Kolupaev 已提交
1022
void StorageReplicatedMergeTree::executeDropRange(const StorageReplicatedMergeTree::LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
1023
{
A
Merge  
Alexey Milovidov 已提交
1024 1025
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
1026
	LOG_INFO(log, (entry.detach ? "Detaching" : "Removing") << " parts inside " << entry.new_part_name << ".");
M
Merge  
Michael Kolupaev 已提交
1027 1028 1029 1030 1031 1032 1033 1034 1035 1036 1037 1038 1039 1040 1041 1042 1043 1044 1045 1046 1047 1048 1049 1050 1051 1052 1053 1054 1055

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

M
Merge  
Michael Kolupaev 已提交
1060
	LOG_DEBUG(log, (entry.detach ? "Detaching" : "Removing") << " parts.");
M
Merge  
Michael Kolupaev 已提交
1061 1062 1063 1064 1065 1066 1067 1068 1069 1070 1071
	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 已提交
1072 1073 1074 1075
		/// Если кусок удалять не нужно, надежнее переместить директорию до изменений в ZooKeeper.
		if (entry.detach)
			data.renameAndDetachPart(part);

M
Merge  
Michael Kolupaev 已提交
1076 1077 1078 1079 1080 1081
		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 已提交
1082
		/// Если кусок нужно удалить, надежнее удалить директорию после изменений в ZooKeeper.
M
Merge  
Michael Kolupaev 已提交
1083
		if (!entry.detach)
M
Merge  
Michael Kolupaev 已提交
1084
			data.replaceParts({part}, {}, true);
M
Merge  
Michael Kolupaev 已提交
1085 1086
	}

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

A
Merge  
Alexey Milovidov 已提交
1090

M
Merge  
Michael Kolupaev 已提交
1091 1092
bool StorageReplicatedMergeTree::executeAttachPart(const StorageReplicatedMergeTree::LogEntry & entry)
{
A
Merge  
Alexey Milovidov 已提交
1093 1094
	auto zookeeper = getZooKeeper();

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

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

M
Merge  
Michael Kolupaev 已提交
1134 1135 1136
	return true;
}

A
Merge  
Alexey Milovidov 已提交
1137

M
Merge  
Michael Kolupaev 已提交
1138 1139 1140 1141
void StorageReplicatedMergeTree::queueUpdatingThread()
{
	while (!shutdown_called)
	{
M
Merge  
Michael Kolupaev 已提交
1142 1143
		try
		{
M
Merge  
Michael Kolupaev 已提交
1144
			pullLogsToQueue(queue_updating_event);
M
Merge  
Michael Kolupaev 已提交
1145

M
Merge  
Michael Kolupaev 已提交
1146
			queue_updating_event->wait();
M
Merge  
Michael Kolupaev 已提交
1147
		}
1148
		catch (const zkutil::KeeperException & e)
M
Merge  
Michael Kolupaev 已提交
1149 1150
		{
			if (e.code == ZINVALIDSTATE)
1151
				restarting_thread->wakeup();
M
Merge  
Michael Kolupaev 已提交
1152 1153 1154 1155 1156

			tryLogCurrentException(__PRETTY_FUNCTION__);

			queue_updating_event->tryWait(ERROR_SLEEP_MS);
		}
M
Merge  
Michael Kolupaev 已提交
1157 1158 1159 1160
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);

M
Merge  
Michael Kolupaev 已提交
1161 1162
			queue_updating_event->tryWait(ERROR_SLEEP_MS);
		}
M
Merge  
Michael Kolupaev 已提交
1163
	}
M
Merge  
Michael Kolupaev 已提交
1164

1165
	LOG_DEBUG(log, "Queue updating thread finished");
M
Merge  
Michael Kolupaev 已提交
1166
}
M
Merge  
Michael Kolupaev 已提交
1167

A
Merge  
Alexey Milovidov 已提交
1168

M
Merge  
Michael Kolupaev 已提交
1169
bool StorageReplicatedMergeTree::queueTask(BackgroundProcessingPool::Context & pool_context)
M
Merge  
Michael Kolupaev 已提交
1170
{
M
Merge  
Michael Kolupaev 已提交
1171
	LogEntryPtr entry;
M
Merge  
Michael Kolupaev 已提交
1172

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

M
Merge  
Michael Kolupaev 已提交
1197
	if (!entry)
M
Merge  
Michael Kolupaev 已提交
1198
		return false;
M
Merge  
Michael Kolupaev 已提交
1199

M
Merge  
Michael Kolupaev 已提交
1200
	bool exception = true;
M
Merge  
Michael Kolupaev 已提交
1201
	bool success = false;
M
Merge  
Michael Kolupaev 已提交
1202

M
Merge  
Michael Kolupaev 已提交
1203 1204
	try
	{
M
Merge  
Michael Kolupaev 已提交
1205
		if (executeLogEntry(*entry, pool_context))
M
Merge  
Michael Kolupaev 已提交
1206
		{
A
Merge  
Alexey Milovidov 已提交
1207
			auto zookeeper = getZooKeeper();
M
Merge  
Michael Kolupaev 已提交
1208
			auto code = zookeeper->tryRemove(replica_path + "/queue/" + entry->znode_name);
M
Merge  
Michael Kolupaev 已提交
1209

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

			success = true;
M
Merge  
Michael Kolupaev 已提交
1215
		}
M
Merge  
Michael Kolupaev 已提交
1216

M
Merge  
Michael Kolupaev 已提交
1217
		exception = false;
M
Merge  
Michael Kolupaev 已提交
1218
	}
1219
	catch (const Exception & e)
M
Merge  
Michael Kolupaev 已提交
1220 1221
	{
		if (e.code() == ErrorCodes::NO_REPLICA_HAS_PART)
1222
		{
M
Merge  
Michael Kolupaev 已提交
1223
			/// Если ни у кого нет нужного куска, наверно, просто не все реплики работают; не будем писать в лог с уровнем Error.
M
Merge  
Michael Kolupaev 已提交
1224
			LOG_INFO(log, e.displayText());
1225 1226 1227 1228 1229 1230
		}
		else if (e.code() == ErrorCodes::ABORTED)
		{
			/// Прерванный мердж - не ошибка.
			LOG_INFO(log, "Merge cancelled");
		}
M
Merge  
Michael Kolupaev 已提交
1231
		else
M
Merge  
Michael Kolupaev 已提交
1232
			tryLogCurrentException(__PRETTY_FUNCTION__);
M
Merge  
Michael Kolupaev 已提交
1233 1234 1235 1236 1237
	}
	catch (...)
	{
		tryLogCurrentException(__PRETTY_FUNCTION__);
	}
M
Merge  
Michael Kolupaev 已提交
1238

M
Merge  
Michael Kolupaev 已提交
1239 1240
	entry->future_part_tagger = nullptr;

1241
	std::lock_guard<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1242 1243 1244 1245 1246

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

	if (success)
M
Merge  
Michael Kolupaev 已提交
1247
	{
M
Merge  
Michael Kolupaev 已提交
1248 1249 1250
		/// Удалим задание из очереди.
		/// Нельзя просто обратиться по заранее сохраненному итератору, потому что задание мог успеть удалить кто-то другой.
		for (LogEntries::iterator it = queue.end(); it != queue.begin();)
M
Merge  
Michael Kolupaev 已提交
1251
		{
M
Merge  
Michael Kolupaev 已提交
1252 1253 1254
			--it;
			if (*it == entry)
			{
M
Merge  
Michael Kolupaev 已提交
1255
				queue.erase(it);
M
Merge  
Michael Kolupaev 已提交
1256 1257
				break;
			}
M
Merge  
Michael Kolupaev 已提交
1258
		}
M
Merge  
Michael Kolupaev 已提交
1259
	}
M
Merge  
Michael Kolupaev 已提交
1260

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

A
Merge  
Alexey Milovidov 已提交
1265

M
Merge  
Michael Kolupaev 已提交
1266 1267
void StorageReplicatedMergeTree::mergeSelectingThread()
{
M
Merge  
Michael Kolupaev 已提交
1268
	bool need_pull = true;
M
Merge  
Michael Kolupaev 已提交
1269

A
Merge  
Alexey Milovidov 已提交
1270 1271 1272 1273 1274 1275 1276
	/** Может много времени тратиться на определение, можно ли мерджить два рядом стоящих куска.
	  * Два рядом стоящих куска можно мерджить, если все номера блоков между их номерами не используются ("заброшены", abandoned).
	  * Это значит, что между этими кусками не может быть вставлен другой кусок.
	  *
	  * Но если номера соседних блоков отличаются достаточно сильно (обычно, если между ними много "заброшенных" блоков),
	  *  то делается слишком много чтений из ZooKeeper, чтобы узнать, можно ли их мерджить.
	  *
1277 1278
	  * Воспользуемся утверждением, что если пару кусков было можно мерджить, и их мердж ещё не запланирован,
	  *  то и сейчас их можно мерджить, и будем запоминать это состояние, чтобы не делать много раз одинаковые запросы в ZooKeeper.
A
Merge  
Alexey Milovidov 已提交
1279 1280 1281 1282 1283 1284 1285 1286
	  *
	  * 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
	{
1287 1288 1289 1290 1291
		/// Если какой-то из кусков уже собираются слить в больший, не соглашаемся его сливать.
		if (virtual_parts.getContainingPart(left->name) != left->name ||
			virtual_parts.getContainingPart(right->name) != right->name)
			return false;

A
Merge  
Alexey Milovidov 已提交
1292 1293 1294 1295
		auto key = std::make_pair(left->name, right->name);
		if (memoized_parts_that_could_be_merged.count(key))
			return true;

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

1299 1300 1301 1302 1303 1304 1305 1306 1307 1308 1309 1310 1311 1312 1313 1314 1315 1316
		/// Нельзя сливать куски, среди которых находится кусок, для которого неудовлетворён кворум.
		/// Замечание: теоретически, это можно было бы разрешить. Но это сделает логику более сложной.
		String quorum_node_value;
		if (zookeeper->tryGet(zookeeper_path + "/quorum/status", quorum_node_value))
		{
			ReplicatedMergeTreeQuorumEntry quorum_entry;
			quorum_entry.fromString(quorum_node_value);

			ActiveDataPartSet::Part part_info;
			ActiveDataPartSet::parsePartName(quorum_entry.part_name, part_info);

			if (part_info.left != part_info.right)
				throw Exception("Logical error: part written with quorum covers more than one block numbers", ErrorCodes::LOGICAL_ERROR);

			if (left->right <= part_info.left && right->left >= part_info.right)
				return false;
		}

1317
		/// Можно слить куски, если все номера между ними заброшены - не соответствуют никаким блокам.
A
Merge  
Alexey Milovidov 已提交
1318 1319
		/// Номера до RESERVED_BLOCK_NUMBERS всегда не соответствуют никаким блокам.
		for (Int64 number = std::max(RESERVED_BLOCK_NUMBERS, left->right + 1); number <= right->left - 1; ++number)
1320 1321 1322
		{
			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 已提交
1323

1324 1325 1326 1327 1328 1329 1330
			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 已提交
1331 1332
	};

M
Merge  
Michael Kolupaev 已提交
1333 1334
	while (!shutdown_called && is_leader_node)
	{
M
Michael Kolupaev 已提交
1335
		bool success = false;
M
Merge  
Michael Kolupaev 已提交
1336

M
Michael Kolupaev 已提交
1337
		try
M
Merge  
Michael Kolupaev 已提交
1338
		{
1339
			std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
M
Merge  
Michael Kolupaev 已提交
1340

M
Merge  
Michael Kolupaev 已提交
1341 1342 1343 1344 1345 1346 1347 1348
			if (need_pull)
			{
				/// Нужно загрузить новую запись в очередь перед тем, как выбирать куски для слияния.
				///  (чтобы кусок добавился в virtual_parts).
				pullLogsToQueue();
				need_pull = false;
			}

1349 1350 1351 1352 1353 1354 1355
			/** Сколько в очереди или в фоновом потоке мерджей крупных кусков.
			  * Если их больше половины от размера пула потоков для мерджа, то можно мерджить только мелкие куски.
			  */
			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 已提交
1356
			size_t merges_queued = 0;
1357
			size_t big_merges_queued = 0;
M
Merge  
Michael Kolupaev 已提交
1358

1359
			if (big_merges_current < max_number_of_big_merges)
M
Michael Kolupaev 已提交
1360
			{
1361
				std::lock_guard<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1362

M
Michael Kolupaev 已提交
1363
				for (const auto & entry : queue)
M
Merge  
Michael Kolupaev 已提交
1364
				{
M
Merge  
Michael Kolupaev 已提交
1365
					if (entry->type == LogEntry::MERGE_PARTS)
M
Merge  
Michael Kolupaev 已提交
1366
					{
M
Michael Kolupaev 已提交
1367
						++merges_queued;
M
Merge  
Michael Kolupaev 已提交
1368

1369
						if (big_merges_current + big_merges_queued < max_number_of_big_merges)
M
Merge  
Michael Kolupaev 已提交
1370
						{
M
Merge  
Michael Kolupaev 已提交
1371
							for (const String & name : entry->parts_to_merge)
M
Merge  
Michael Kolupaev 已提交
1372
							{
M
Merge  
Michael Kolupaev 已提交
1373
								MergeTreeData::DataPartPtr part = data.getActiveContainingPart(name);
M
Merge  
Michael Kolupaev 已提交
1374 1375
								if (!part || part->name != name)
									continue;
1376

M
Merge  
Michael Kolupaev 已提交
1377
								if (part->size_in_bytes > data.settings.max_bytes_to_merge_parts_small)
M
Merge  
Michael Kolupaev 已提交
1378
								{
1379
									++big_merges_queued;
M
Merge  
Michael Kolupaev 已提交
1380 1381 1382 1383 1384 1385
									break;
								}
							}
						}
					}
				}
M
Michael Kolupaev 已提交
1386
			}
M
Merge  
Michael Kolupaev 已提交
1387

1388 1389
			bool only_small = big_merges_current + big_merges_queued >= max_number_of_big_merges;

1390 1391 1392 1393 1394 1395
			if (big_merges_current || merges_queued)
				LOG_TRACE(log, "Currently executing big merges: " << big_merges_current
					<< ". Queued big merges: " << big_merges_queued
					<< ". All merges in queue: " << merges_queued
					<< ". Max number of big merges: " << max_number_of_big_merges
					<< (only_small ? ". So, will select only small parts to merge." : "."));
1396

M
Merge  
Michael Kolupaev 已提交
1397
			do
M
Michael Kolupaev 已提交
1398
			{
A
Merge  
Alexey Milovidov 已提交
1399 1400
				auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
1401
				if (merges_queued >= data.settings.max_replicated_merges_in_queue)
1402 1403
				{
					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 已提交
1404
					break;
1405
				}
M
Merge  
Michael Kolupaev 已提交
1406

M
Merge  
Michael Kolupaev 已提交
1407
				MergeTreeData::DataPartsVector parts;
M
Merge  
Michael Kolupaev 已提交
1408

M
Merge  
Michael Kolupaev 已提交
1409 1410
				String merged_name;

A
Merge  
Alexey Milovidov 已提交
1411 1412 1413 1414
				size_t disk_space = DiskSpaceMonitor::getUnreservedFreeSpace(full_path);

				if (   !merger.selectPartsToMerge(parts, merged_name, disk_space, false, false, only_small, can_merge)
					&& !merger.selectPartsToMerge(parts, merged_name, disk_space, true, false, only_small, can_merge))
1415
				{
M
Merge  
Michael Kolupaev 已提交
1416
					break;
1417
				}
M
Merge  
Michael Kolupaev 已提交
1418

M
Merge  
Michael Kolupaev 已提交
1419 1420 1421 1422 1423 1424 1425 1426 1427 1428 1429 1430 1431 1432
				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 已提交
1433 1434 1435 1436
				LogEntry entry;
				entry.type = LogEntry::MERGE_PARTS;
				entry.source_replica = replica_name;
				entry.new_part_name = merged_name;
M
Merge  
Michael Kolupaev 已提交
1437

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

M
Merge  
Michael Kolupaev 已提交
1441
				need_pull = true;
M
Merge  
Michael Kolupaev 已提交
1442

M
Merge  
Michael Kolupaev 已提交
1443
				zookeeper->create(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential);
M
Merge  
Michael Kolupaev 已提交
1444 1445 1446

				String month_name = parts[0]->name.substr(0, 6);
				for (size_t i = 0; i + 1 < parts.size(); ++i)
M
Merge  
Michael Kolupaev 已提交
1447
				{
M
Merge  
Michael Kolupaev 已提交
1448
					/// Уберем больше не нужные отметки о несуществующих блоках.
A
Merge  
Alexey Milovidov 已提交
1449
					for (Int64 number = std::max(RESERVED_BLOCK_NUMBERS, parts[i]->right + 1); number <= parts[i + 1]->left - 1; ++number)
M
Merge  
Michael Kolupaev 已提交
1450
					{
M
Merge  
Michael Kolupaev 已提交
1451 1452
						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 已提交
1453
					}
M
Merge  
Michael Kolupaev 已提交
1454
				}
M
Merge  
Michael Kolupaev 已提交
1455 1456

				success = true;
M
Merge  
Michael Kolupaev 已提交
1457
			}
1458
			while (false);
M
Merge  
Michael Kolupaev 已提交
1459 1460 1461 1462 1463 1464
		}
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);
		}

M
Merge  
Michael Kolupaev 已提交
1465
		if (shutdown_called || !is_leader_node)
M
Merge  
Michael Kolupaev 已提交
1466 1467
			break;

M
Merge  
Michael Kolupaev 已提交
1468
		if (!success)
M
Merge  
Michael Kolupaev 已提交
1469
			merge_selecting_event.tryWait(MERGE_SELECTING_SLEEP_MS);
M
Merge  
Michael Kolupaev 已提交
1470
	}
M
Merge  
Michael Kolupaev 已提交
1471

1472
	LOG_DEBUG(log, "Merge selecting thread finished");
M
Merge  
Michael Kolupaev 已提交
1473 1474
}

M
Merge  
Michael Kolupaev 已提交
1475

M
Merge  
Michael Kolupaev 已提交
1476 1477 1478 1479 1480 1481 1482 1483
void StorageReplicatedMergeTree::alterThread()
{
	bool force_recheck_parts = true;

	while (!shutdown_called)
	{
		try
		{
1484 1485 1486 1487 1488 1489 1490 1491 1492 1493 1494 1495 1496 1497 1498 1499 1500 1501 1502 1503 1504 1505 1506
			/** Имеем описание столбцов в 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 已提交
1507 1508
			auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
1509
			zkutil::Stat stat;
1510
			const String columns_str = zookeeper->get(zookeeper_path + "/columns", &stat, alter_thread_event);
1511
			auto columns_desc = ColumnsDescription<true>::parse(columns_str);
1512 1513 1514 1515 1516

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

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

M
Merge  
Michael Kolupaev 已提交
1520 1521
			MergeTreeData::DataParts parts;

M
Merge  
Michael Kolupaev 已提交
1522
			/// Если описание столбцов изменилось, обновим структуру таблицы локально.
1523
			if (changed_version)
M
Merge  
Michael Kolupaev 已提交
1524
			{
1525 1526
				LOG_INFO(log, "Changed version of 'columns' node in ZooKeeper. Waiting for structure write lock.");

M
Merge  
Michael Kolupaev 已提交
1527
				auto table_lock = lockStructureForAlter();
1528

1529 1530 1531
				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 已提交
1532
				const auto column_defaults_changed = column_defaults != data.column_defaults;
1533 1534 1535

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

1539 1540 1541
					InterpreterAlterQuery::updateMetadata(database_name, table_name, columns,
						materialized_columns, alias_columns, column_defaults, context);

1542 1543 1544 1545 1546 1547 1548 1549 1550 1551 1552 1553 1554 1555 1556 1557 1558 1559 1560 1561 1562 1563 1564 1565 1566 1567
					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 已提交
1568 1569 1570 1571
					LOG_INFO(log, "Applied changes to table.");
				}
				else
				{
1572
					LOG_INFO(log, "Columns version changed in ZooKeeper, but data wasn't changed. It's like cyclic ALTERs.");
M
Merge  
Michael Kolupaev 已提交
1573
				}
1574 1575 1576 1577 1578

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

				columns_version = stat.version;
M
Merge  
Michael Kolupaev 已提交
1579 1580 1581
			}

			/// Обновим куски.
1582
			if (changed_version || force_recheck_parts)
M
Merge  
Michael Kolupaev 已提交
1583
			{
1584 1585 1586
				auto table_lock = lockStructure(false);

				if (changed_version)
M
Merge  
Michael Kolupaev 已提交
1587 1588 1589 1590
					LOG_INFO(log, "ALTER-ing parts");

				int changed_parts = 0;

1591
				if (!changed_version)
M
Merge  
Michael Kolupaev 已提交
1592 1593
					parts = data.getDataParts();

1594 1595
				const auto columns_plus_materialized = data.getColumnsList();

M
Merge  
Michael Kolupaev 已提交
1596 1597 1598
				for (const MergeTreeData::DataPartPtr & part : parts)
				{
					/// Обновим кусок и запишем результат во временные файлы.
M
Merge  
Michael Kolupaev 已提交
1599 1600
					/// TODO: Можно пропускать проверку на слишком большие изменения, если в ZooKeeper есть, например,
					///  нода /flags/force_alter.
1601
					auto transaction = data.alterDataPart(part, columns_plus_materialized);
M
Merge  
Michael Kolupaev 已提交
1602 1603 1604 1605 1606 1607 1608 1609

					if (!transaction)
						continue;

					++changed_parts;

					/// Обновим метаданные куска в ZooKeeper.
					zkutil::Ops ops;
1610 1611 1612 1613
					ops.push_back(new zkutil::Op::SetData(
						replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1));
					ops.push_back(new zkutil::Op::SetData(
						replica_path + "/parts/" + part->name + "/checksums", transaction->getNewChecksums().toString(), -1));
M
Merge  
Michael Kolupaev 已提交
1614 1615 1616 1617 1618 1619
					zookeeper->multi(ops);

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

M
Merge  
Michael Kolupaev 已提交
1620 1621 1622 1623 1624 1625 1626
				/// То же самое для нереплицируемых данных.
				if (unreplicated_data)
				{
					parts = unreplicated_data->getDataParts();

					for (const MergeTreeData::DataPartPtr & part : parts)
					{
1627
						auto transaction = unreplicated_data->alterDataPart(part, columns_plus_materialized);
M
Merge  
Michael Kolupaev 已提交
1628 1629 1630 1631 1632 1633 1634 1635 1636

						if (!transaction)
							continue;

						++changed_parts;

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

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

1641 1642 1643 1644 1645 1646 1647 1648
				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 已提交
1649 1650 1651
				force_recheck_parts = false;
			}

1652
			parts.clear();
M
Merge  
Michael Kolupaev 已提交
1653 1654 1655 1656 1657 1658 1659 1660 1661 1662 1663 1664
			alter_thread_event->wait();
		}
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);

			force_recheck_parts = true;

			alter_thread_event->tryWait(ERROR_SLEEP_MS);
		}
	}

1665
	LOG_DEBUG(log, "Alter thread finished");
M
Merge  
Michael Kolupaev 已提交
1666 1667
}

A
Merge  
Alexey Milovidov 已提交
1668

M
Merge  
Michael Kolupaev 已提交
1669 1670
void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_name)
{
A
Merge  
Alexey Milovidov 已提交
1671 1672
	auto zookeeper = getZooKeeper();

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

M
Merge  
Michael Kolupaev 已提交
1675 1676
	LogEntryPtr log_entry = new LogEntry;
	log_entry->type = LogEntry::GET_PART;
A
Merge  
Alexey Milovidov 已提交
1677
	log_entry->create_time = time(0);
M
Merge  
Michael Kolupaev 已提交
1678 1679
	log_entry->source_replica = "";
	log_entry->new_part_name = part_name;
M
Merge  
Michael Kolupaev 已提交
1680 1681 1682

	zkutil::Ops ops;
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
1683
		replica_path + "/queue/queue-", log_entry->toString(), zookeeper->getDefaultACL(),
M
Merge  
Michael Kolupaev 已提交
1684 1685 1686 1687 1688 1689 1690
		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);

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

		String path_created = dynamic_cast<zkutil::Op::Create &>(ops[0]).getPathCreated();
M
Merge  
Michael Kolupaev 已提交
1694 1695
		log_entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
		log_entry->addResultToVirtualParts(*this);
M
Merge  
Michael Kolupaev 已提交
1696 1697 1698 1699
		queue.push_back(log_entry);
	}
}

A
Merge  
Alexey Milovidov 已提交
1700

M
Merge  
Michael Kolupaev 已提交
1701 1702
void StorageReplicatedMergeTree::enqueuePartForCheck(const String & name)
{
1703
	std::lock_guard<std::mutex> lock(parts_to_check_mutex);
M
Merge  
Michael Kolupaev 已提交
1704 1705 1706 1707 1708 1709 1710 1711

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

1712 1713

void StorageReplicatedMergeTree::searchForMissingPart(const String & part_name)
M
Merge  
Michael Kolupaev 已提交
1714
{
1715 1716 1717 1718 1719
	auto zookeeper = getZooKeeper();
	String part_path = replica_path + "/parts/" + part_name;

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

1725 1726 1727 1728 1729 1730 1731 1732 1733 1734 1735 1736 1737 1738 1739 1740 1741 1742 1743 1744 1745 1746 1747 1748 1749 1750 1751 1752 1753 1754 1755 1756
		removePartAndEnqueueFetch(part_name);
		return;
	}

	/// Если куска нет в ZooKeeper, проверим есть ли он хоть у кого-то.
	ActiveDataPartSet::Part part_info;
	ActiveDataPartSet::parsePartName(part_name, part_info);

	/** Логика такая:
		* - если у какой-то живой или неактивной реплики есть такой кусок, или покрывающий его кусок
		*   - всё Ок, ничего делать не нужно, он скачается затем при обработке очереди, когда реплика оживёт;
		*   - или, если реплика никогда не оживёт, то администратор удалит или создаст новую реплику с тем же адресом и см. всё сначала;
		* - если ни у кого нет такого или покрывающего его куска, то
		*   - если у кого-то есть все составляющие куски, то ничего делать не будем - это просто значит, что другие реплики ещё недоделали мердж
		*   - если ни у кого нет всех составляющих кусков, то признаем кусок навечно потерянным,
		*     и удалим запись из очереди репликации.
		*/

	LOG_WARNING(log, "Checker: Checking if anyone has part covering " << part_name << ".");

	bool found = false;

	size_t part_length_in_blocks = part_info.right + 1 - part_info.left;
	std::vector<char> found_blocks(part_length_in_blocks);

	Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
	for (const String & replica : replicas)
	{
		Strings parts = zookeeper->getChildren(zookeeper_path + "/replicas/" + replica + "/parts");
		for (const String & part_on_replica : parts)
		{
			if (part_on_replica == part_name || ActiveDataPartSet::contains(part_on_replica, part_name))
M
Merge  
Michael Kolupaev 已提交
1757
			{
1758 1759 1760
				found = true;
				LOG_WARNING(log, "Checker: Found part " << part_on_replica << " on " << replica);
				break;
M
Merge  
Michael Kolupaev 已提交
1761
			}
1762 1763

			if (ActiveDataPartSet::contains(part_name, part_on_replica))
M
Merge  
Michael Kolupaev 已提交
1764
			{
1765 1766 1767
				ActiveDataPartSet::Part part_on_replica_info;
				ActiveDataPartSet::parsePartName(part_on_replica, part_on_replica_info);

1768
				for (auto block_num = part_on_replica_info.left; block_num <= part_on_replica_info.right; ++block_num)
A
Alexey Milovidov 已提交
1769
					found_blocks.at(block_num - part_info.left) = 1;
M
Merge  
Michael Kolupaev 已提交
1770
			}
1771 1772 1773 1774
		}
		if (found)
			break;
	}
M
Merge  
Michael Kolupaev 已提交
1775

1776 1777 1778 1779 1780
	if (found)
	{
		/// На какой-то живой или мёртвой реплике есть нужный кусок или покрывающий его.
		return;
	}
M
Merge  
Michael Kolupaev 已提交
1781

1782 1783 1784
	size_t num_found_blocks = 0;
	for (auto found_block : found_blocks)
		num_found_blocks += (found_block == 1);
M
Merge  
Michael Kolupaev 已提交
1785

1786 1787 1788 1789 1790 1791
	if (num_found_blocks == part_length_in_blocks)
	{
		/// На совокупности живых или мёртвых реплик есть все куски, из которых можно составить нужный кусок. Ничего делать не будем.
		LOG_WARNING(log, "Checker: Found all blocks for missing part. Will wait for them to be merged.");
		return;
	}
1792

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

1796 1797 1798
	if (num_found_blocks != 0)
		LOG_WARNING(log, "When looking for smaller parts, that is covered by " << part_name
			<< ", we found just " << num_found_blocks << " of " << part_length_in_blocks << " blocks.");
M
Merge  
Michael Kolupaev 已提交
1799

1800
	ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
1801

1802 1803 1804 1805
	/// Есть ли он в очереди репликации? Если есть - удалим, так как задачу невозможно обработать.
	bool was_in_queue = false;
	{
		std::lock_guard<std::mutex> lock(queue_mutex);
1806

1807 1808 1809 1810 1811 1812 1813 1814 1815 1816 1817 1818
		for (LogEntries::iterator it = queue.begin(); it != queue.end();)
		{
			if ((*it)->new_part_name == part_name)
			{
				zookeeper->tryRemove(replica_path + "/queue/" + (*it)->znode_name);
				queue.erase(it++);
				was_in_queue = true;
			}
			else
				++it;
		}
	}
1819

1820 1821 1822 1823 1824 1825
	if (!was_in_queue)
	{
		/// Куска не было в нашей очереди. С чего бы это?
		LOG_ERROR(log, "Checker: Missing part " << part_name << " is not in our queue.");
		return;
	}
M
Merge  
Michael Kolupaev 已提交
1826

1827 1828 1829 1830 1831 1832 1833 1834 1835 1836 1837 1838 1839 1840 1841 1842 1843 1844 1845 1846 1847 1848 1849 1850
	/** Такая ситуация возможна, если на всех репликах, где был кусок, он испортился.
		* Например, у реплики, которая только что его записала, отключили питание, и данные не записались из кеша на диск.
		*/
	LOG_ERROR(log, "Checker: Part " << part_name << " is lost forever.");
	ProfileEvents::increment(ProfileEvents::ReplicatedDataLoss);

	/** Нужно добавить отсутствующий кусок в block_numbers, чтобы он не мешал слияниям.
		* Вот только в сам block_numbers мы его добавить не можем - если так сделать,
		*  ZooKeeper зачем-то пропустит один номер для автоинкремента,
		*  и в номерах блоков все равно останется дырка.
		* Специально из-за этого приходится отдельно иметь nonincrement_block_numbers.
		*
		* Кстати, если мы здесь сдохнем, то слияния не будут делаться сквозь эти отсутствующие куски.
		*/
	const auto partition_str = part_name.substr(0, 6);
	for (auto i = part_info.left; i <= part_info.right; ++i)
	{
		zookeeper->createIfNotExists(zookeeper_path + "/nonincrement_block_numbers", "");
		zookeeper->createIfNotExists(zookeeper_path + "/nonincrement_block_numbers/" + partition_str, "");
		AbandonableLockInZooKeeper::createAbandonedIfNotExists(
			zookeeper_path + "/nonincrement_block_numbers/" + partition_str + "/block-" + padIndex(i),
			*zookeeper);
	}
}
M
Merge  
Michael Kolupaev 已提交
1851 1852


1853 1854 1855 1856
void StorageReplicatedMergeTree::checkPart(const String & part_name)
{
	LOG_WARNING(log, "Checker: Checking part " << part_name);
	ProfileEvents::increment(ProfileEvents::ReplicatedPartChecks);
M
Merge  
Michael Kolupaev 已提交
1857

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

1860 1861 1862 1863 1864 1865 1866 1867 1868 1869
	/// Этого или покрывающего куска у нас нет.
	if (!part)
	{
		searchForMissingPart(part_name);
	}
	/// У нас есть этот кусок, и он активен. Будем проверять, нужен ли нам этот кусок и правильные ли у него данные.
	else if (part->name == part_name)
	{
		auto zookeeper = getZooKeeper();
		auto table_lock = lockStructure(false);
M
Merge  
Michael Kolupaev 已提交
1870

1871 1872 1873 1874 1875 1876 1877 1878 1879 1880 1881 1882 1883 1884 1885 1886 1887 1888 1889 1890 1891 1892 1893 1894
		/// Если кусок есть в ZooKeeper, сверим его данные с его чексуммами, а их с ZooKeeper.
		if (zookeeper->exists(replica_path + "/parts/" + part_name))
		{
			LOG_WARNING(log, "Checker: Checking data of part " << part_name << ".");

			try
			{
				auto zk_checksums = MergeTreeData::DataPart::Checksums::parse(
					zookeeper->get(replica_path + "/parts/" + part_name + "/checksums"));
				zk_checksums.checkEqual(part->checksums, true);

				auto zk_columns = NamesAndTypesList::parse(
					zookeeper->get(replica_path + "/parts/" + part_name + "/columns"));
				if (part->columns != zk_columns)
					throw Exception("Columns of local part " + part_name + " are different from ZooKeeper");

				MergeTreePartChecker::Settings settings;
				settings.setIndexGranularity(data.index_granularity);
				settings.setRequireChecksums(true);
				settings.setRequireColumnFiles(true);
				MergeTreePartChecker::checkDataPart(
					data.getFullPath() + part_name, settings, data.primary_key_sample);

				LOG_INFO(log, "Checker: Part " << part_name << " looks good.");
M
Merge  
Michael Kolupaev 已提交
1895
			}
1896
			catch (...)
M
Merge  
Michael Kolupaev 已提交
1897
			{
1898
				tryLogCurrentException(__PRETTY_FUNCTION__);
M
Merge  
Michael Kolupaev 已提交
1899

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

1903
				removePartAndEnqueueFetch(part_name);
M
Merge  
Michael Kolupaev 已提交
1904

1905 1906 1907 1908 1909 1910 1911 1912 1913 1914 1915 1916 1917 1918 1919 1920 1921 1922 1923 1924 1925 1926
				/// Удалим кусок локально.
				data.renameAndDetachPart(part, "broken_");
			}
		}
		else if (part->modification_time + 5 * 60 < time(0))
		{
			/// Если куска нет в ZooKeeper, удалим его локально.
			/// Возможно, кусок кто-то только что записал, и еще не успел добавить в ZK.
			/// Поэтому удаляем только если кусок старый (не очень надежно).
			ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);

			LOG_ERROR(log, "Checker: Unexpected part " << part_name << ". Removing.");
			data.renameAndDetachPart(part, "unexpected_");
		}
	}
	else
	{
		/// Если у нас есть покрывающий кусок, игнорируем все проблемы с этим куском.
		/// В худшем случае в лог еще old_parts_lifetime секунд будут валиться ошибки, пока кусок не удалится как старый.
		LOG_WARNING(log, "Checker: We have part " << part->name << " covering part " << part_name);
	}
}
M
Merge  
Michael Kolupaev 已提交
1927 1928


1929 1930 1931 1932 1933 1934 1935 1936 1937 1938 1939 1940 1941 1942 1943 1944
void StorageReplicatedMergeTree::partCheckThread()
{
	while (!shutdown_called)
	{
		try
		{
			/// Достанем из очереди кусок для проверки.
			String part_name;
			{
				std::lock_guard<std::mutex> lock(parts_to_check_mutex);
				if (parts_to_check_queue.empty())
				{
					if (!parts_to_check_set.empty())
					{
						LOG_ERROR(log, "Checker: Non-empty parts_to_check_set with empty parts_to_check_queue. This is a bug.");
						parts_to_check_set.clear();
M
Merge  
Michael Kolupaev 已提交
1945
					}
M
Merge  
Michael Kolupaev 已提交
1946
				}
1947
				else
M
Merge  
Michael Kolupaev 已提交
1948
				{
1949
					part_name = parts_to_check_queue.front();
M
Merge  
Michael Kolupaev 已提交
1950 1951
				}
			}
1952 1953

			if (part_name.empty())
M
Merge  
Michael Kolupaev 已提交
1954
			{
1955 1956
				parts_to_check_event.wait();
				continue;
M
Merge  
Michael Kolupaev 已提交
1957 1958
			}

1959 1960 1961
			checkPart(part_name);

			/// Удалим кусок из очереди проверок.
M
Merge  
Michael Kolupaev 已提交
1962
			{
1963
				std::lock_guard<std::mutex> lock(parts_to_check_mutex);
M
Merge  
Michael Kolupaev 已提交
1964 1965
				if (parts_to_check_queue.empty() || parts_to_check_queue.front() != part_name)
				{
1966
					LOG_ERROR(log, "Checker: Someone changed parts_to_check_queue.front(). This is a bug.");
M
Merge  
Michael Kolupaev 已提交
1967 1968 1969 1970 1971 1972 1973 1974 1975 1976 1977 1978 1979 1980 1981 1982 1983
				}
				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 已提交
1984 1985
void StorageReplicatedMergeTree::becomeLeader()
{
M
Merge  
Michael Kolupaev 已提交
1986
	LOG_INFO(log, "Became leader");
M
Merge  
Michael Kolupaev 已提交
1987 1988 1989 1990
	is_leader_node = true;
	merge_selecting_thread = std::thread(&StorageReplicatedMergeTree::mergeSelectingThread, this);
}

A
Merge  
Alexey Milovidov 已提交
1991

M
Merge  
Michael Kolupaev 已提交
1992
String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_name, bool active)
M
Merge  
Michael Kolupaev 已提交
1993
{
A
Merge  
Alexey Milovidov 已提交
1994
	auto zookeeper = getZooKeeper();
M
Merge  
Michael Kolupaev 已提交
1995
	Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
M
Merge  
Michael Kolupaev 已提交
1996 1997 1998 1999 2000 2001

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

	for (const String & replica : replicas)
	{
M
Merge  
Michael Kolupaev 已提交
2002 2003
		if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name) &&
			(!active || zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")))
M
Merge  
Michael Kolupaev 已提交
2004
			return replica;
2005 2006

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

M
Merge  
Michael Kolupaev 已提交
2009
	return "";
M
Merge  
Michael Kolupaev 已提交
2010 2011
}

A
Merge  
Alexey Milovidov 已提交
2012

2013 2014 2015 2016 2017 2018 2019 2020 2021 2022 2023 2024 2025 2026 2027 2028 2029 2030 2031 2032 2033 2034 2035 2036 2037 2038 2039 2040 2041 2042 2043 2044 2045 2046 2047 2048 2049 2050 2051
/** Если для куска отслеживается кворум, то обновить информацию о нём в ZK.
  */
static void updateQuorum(
	zkutil::ZooKeeperPtr & zookeeper,
	const String & zookeeper_path,
	const String & replica_name,
	const String & part_name,
	size_t quorum)
{
	if (!quorum)
		return;

	const String quorum_status_path = zookeeper_path + "/quorum/status";
	String value;
	zkutil::Stat stat;

	/// Если узла нет, значит по всем кворумным INSERT-ам уже был достигнут кворум, и ничего делать не нужно.
	while (zookeeper->tryGet(quorum_status_path, value, &stat))
	{
		ReplicatedMergeTreeQuorumEntry quorum_entry;
		quorum_entry.fromString(value);

		if (quorum_entry.part_name != part_name)
		{
			/// Кворум уже был достигнут. Более того, уже начался другой INSERT с кворумом.
			break;
		}

		if (quorum_entry.required_number_of_replicas != quorum)
			throw Exception("Logical error: quorum size in log entry is different than quorum size in node /quorum/status",
				ErrorCodes::LOGICAL_ERROR);

		quorum_entry.replicas.insert(replica_name);

		if (quorum_entry.replicas.size() >= quorum_entry.required_number_of_replicas)
		{
			/// Кворум достигнут. Удаляем узел.
			auto code = zookeeper->tryRemove(quorum_status_path, stat.version);

2052 2053 2054 2055 2056
			if (code == ZOK)
			{
				break;
			}
			else if (code == ZNONODE)
2057 2058 2059 2060 2061 2062 2063 2064 2065 2066 2067 2068 2069 2070 2071 2072 2073
			{
				/// Кворум уже был достигнут.
				break;
			}
			else if (code == ZBADVERSION)
			{
				/// Узел успели обновить. Надо заново его прочитать и повторить все действия.
				continue;
			}
			else
				throw zkutil::KeeperException(code, quorum_status_path);
		}
		else
		{
			/// Обновляем узел, прописывая туда на одну реплику больше.
			auto code = zookeeper->trySet(quorum_status_path, quorum_entry.toString(), stat.version);

2074 2075 2076 2077 2078
			if (code == ZOK)
			{
				break;
			}
			else if (code == ZNONODE)
2079 2080 2081 2082 2083 2084 2085 2086 2087 2088 2089 2090 2091 2092 2093 2094 2095
			{
				/// Кворум уже был достигнут.
				break;
			}
			else if (code == ZBADVERSION)
			{
				/// Узел успели обновить. Надо заново его прочитать и повторить все действия.
				continue;
			}
			else
				throw zkutil::KeeperException(code, quorum_status_path);
		}
	}
}


void StorageReplicatedMergeTree::fetchPart(const String & part_name, const String & replica_path, bool to_detached, size_t quorum)
M
Merge  
Michael Kolupaev 已提交
2096
{
A
Merge  
Alexey Milovidov 已提交
2097 2098
	auto zookeeper = getZooKeeper();

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

2101 2102 2103
	TableStructureReadLockPtr table_lock;
	if (!to_detached)
		table_lock = lockStructure(true);
M
Merge  
Michael Kolupaev 已提交
2104 2105 2106 2107

	String host;
	int port;

2108
	String host_port_str = zookeeper->get(replica_path + "/host");
M
Merge  
Michael Kolupaev 已提交
2109 2110 2111 2112 2113 2114 2115 2116
	ReadBufferFromString buf(host_port_str);
	assertString("host: ", buf);
	readString(host, buf);
	assertString("\nport: ", buf);
	readText(port, buf);
	assertString("\n", buf);
	assertEOF(buf);

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

2119 2120 2121 2122
	if (!to_detached)
	{
		zkutil::Ops ops;
		checkPartAndAddToZooKeeper(part, ops, part_name);
M
Merge  
Michael Kolupaev 已提交
2123

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

2127 2128
		zookeeper->multi(ops);
		transaction.commit();
2129 2130 2131 2132 2133 2134

		/** Если для этого куска отслеживается кворум, то надо его обновить.
		  * TODO Обработка в случае неизвестной ошибки, потери сессии, при перезапуске сервера.
		  */
		updateQuorum(zookeeper, zookeeper_path, replica_name, part_name, quorum);

2135
		merge_selecting_event.set();
M
Merge  
Michael Kolupaev 已提交
2136

2137 2138 2139 2140 2141 2142 2143
		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 已提交
2144
	{
2145
		Poco::File(data.getFullPath() + "detached/tmp_" + part_name).renameTo(data.getFullPath() + "detached/" + part_name);
M
Michael Kolupaev 已提交
2146 2147
	}

M
Merge  
Michael Kolupaev 已提交
2148 2149
	ProfileEvents::increment(ProfileEvents::ReplicatedPartFetches);

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

A
Merge  
Alexey Milovidov 已提交
2153

M
Merge  
Michael Kolupaev 已提交
2154 2155
void StorageReplicatedMergeTree::shutdown()
{
2156
	if (restarting_thread)
M
Merge  
Michael Kolupaev 已提交
2157
	{
2158 2159
		restarting_thread->stop();
		restarting_thread.reset();
M
Merge  
Michael Kolupaev 已提交
2160
	}
M
Merge  
Michael Kolupaev 已提交
2161 2162

	endpoint_holder = nullptr;
M
Merge  
Michael Kolupaev 已提交
2163 2164 2165
}


M
Merge  
Michael Kolupaev 已提交
2166 2167 2168 2169 2170 2171 2172 2173
StorageReplicatedMergeTree::~StorageReplicatedMergeTree()
{
	try
	{
		shutdown();
	}
	catch(...)
	{
2174
		tryLogCurrentException(__PRETTY_FUNCTION__);
M
Merge  
Michael Kolupaev 已提交
2175 2176 2177
	}
}

A
Merge  
Alexey Milovidov 已提交
2178

M
Merge  
Michael Kolupaev 已提交
2179
BlockInputStreams StorageReplicatedMergeTree::read(
2180 2181 2182 2183 2184 2185 2186
	const Names & column_names,
	ASTPtr query,
	const Context & context,
	const Settings & settings,
	QueryProcessingStage::Enum & processed_stage,
	const size_t max_block_size,
	const unsigned threads)
M
Merge  
Michael Kolupaev 已提交
2187
{
M
Merge  
Michael Kolupaev 已提交
2188 2189 2190 2191 2192 2193 2194 2195
	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);

A
Merge  
Andrey Mironov 已提交
2196 2197 2198 2199 2200 2201 2202
	ASTSelectQuery & select = *typeid_cast<ASTSelectQuery*>(&*query);

	/// Try transferring some condition from WHERE to PREWHERE if enabled and viable
	if (settings.optimize_move_to_prewhere)
		if (select.where_expression && !select.prewhere_expression)
			MergeTreeWhereOptimizer{select, data, real_column_names, log};

M
Merge  
Michael Kolupaev 已提交
2203 2204 2205 2206 2207
	Block virtual_columns_block;
	ColumnUInt8 * column = new ColumnUInt8(2);
	ColumnPtr column_ptr = column;
	column->getData()[0] = 0;
	column->getData()[1] = 1;
2208
	virtual_columns_block.insert(ColumnWithTypeAndName(column_ptr, new DataTypeUInt8, "_replicated"));
M
Merge  
Michael Kolupaev 已提交
2209 2210 2211

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

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

M
Merge  
Michael Kolupaev 已提交
2216 2217
	BlockInputStreams res;

M
Merge  
Michael Kolupaev 已提交
2218 2219
	size_t part_index = 0;

2220
	if ((settings.parallel_replica_offset == 0) && unreplicated_reader && values.count(0))
M
Merge  
Michael Kolupaev 已提交
2221
	{
2222 2223 2224
		res = unreplicated_reader->read(real_column_names, query,
										context, settings, processed_stage,
										max_block_size, threads, &part_index);
M
Merge  
Michael Kolupaev 已提交
2225 2226 2227 2228 2229 2230

		for (auto & virtual_column : virt_column_names)
		{
			if (virtual_column == "_replicated")
			{
				for (auto & stream : res)
M
Merge  
Michael Kolupaev 已提交
2231
					stream = new AddingConstColumnBlockInputStream<UInt8>(stream, new DataTypeUInt8, 0, "_replicated");
M
Merge  
Michael Kolupaev 已提交
2232 2233 2234 2235
			}
		}
	}

M
Merge  
Michael Kolupaev 已提交
2236
	if (values.count(1))
M
Merge  
Michael Kolupaev 已提交
2237
	{
2238
		auto res2 = reader.read(real_column_names, query, context, settings, processed_stage, max_block_size, threads, &part_index);
M
Merge  
Michael Kolupaev 已提交
2239 2240 2241 2242 2243 2244

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

M
Merge  
Michael Kolupaev 已提交
2249
		res.insert(res.end(), res2.begin(), res2.end());
M
Merge  
Michael Kolupaev 已提交
2250 2251 2252
	}

	return res;
M
Merge  
Michael Kolupaev 已提交
2253 2254
}

A
Merge  
Alexey Milovidov 已提交
2255

2256
BlockOutputStreamPtr StorageReplicatedMergeTree::write(ASTPtr query, const Settings & settings)
M
Merge  
Michael Kolupaev 已提交
2257
{
2258 2259
	if (is_readonly)
		throw Exception("Table is in readonly mode", ErrorCodes::TABLE_IS_READ_ONLY);
M
Merge  
Michael Kolupaev 已提交
2260

M
Merge  
Michael Kolupaev 已提交
2261
	String insert_id;
M
Merge  
Michael Kolupaev 已提交
2262 2263 2264
	if (query)
		if (ASTInsertQuery * insert = typeid_cast<ASTInsertQuery *>(&*query))
			insert_id = insert->insert_id;
M
Merge  
Michael Kolupaev 已提交
2265

2266
	return new ReplicatedMergeTreeBlockOutputStream(*this, insert_id, settings.insert_quorum);
M
Merge  
Michael Kolupaev 已提交
2267
}
M
Merge  
Michael Kolupaev 已提交
2268

A
Merge  
Alexey Milovidov 已提交
2269

2270
bool StorageReplicatedMergeTree::optimize(const Settings & settings)
M
Merge  
Michael Kolupaev 已提交
2271
{
2272 2273
	/// Померджим какие-нибудь куски из директории unreplicated.
	/// TODO: Мерджить реплицируемые куски тоже.
M
Merge  
Michael Kolupaev 已提交
2274 2275 2276 2277

	if (!unreplicated_data)
		return false;

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

M
Merge  
Michael Kolupaev 已提交
2280 2281 2282 2283
	unreplicated_data->clearOldParts();

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

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

M
Merge  
Michael Kolupaev 已提交
2291 2292 2293
	return true;
}

A
Merge  
Alexey Milovidov 已提交
2294

M
Merge  
Michael Kolupaev 已提交
2295 2296
void StorageReplicatedMergeTree::alter(const AlterCommands & params,
	const String & database_name, const String & table_name, Context & context)
M
Merge  
Michael Kolupaev 已提交
2297
{
A
Merge  
Alexey Milovidov 已提交
2298
	auto zookeeper = getZooKeeper();
2299 2300
	const MergeTreeMergeBlocker merge_blocker{merger};
	const auto unreplicated_merge_blocker = unreplicated_merger ?
2301
		std::make_unique<MergeTreeMergeBlocker>(*unreplicated_merger) : nullptr;
A
Merge  
Alexey Milovidov 已提交
2302

M
Merge  
Michael Kolupaev 已提交
2303 2304 2305
	LOG_DEBUG(log, "Doing ALTER");

	NamesAndTypesList new_columns;
2306 2307 2308
	NamesAndTypesList new_materialized_columns;
	NamesAndTypesList new_alias_columns;
	ColumnDefaults new_column_defaults;
M
Merge  
Michael Kolupaev 已提交
2309 2310 2311 2312 2313 2314 2315
	String new_columns_str;
	int new_columns_version;
	zkutil::Stat stat;

	{
		auto table_lock = lockStructureForAlter();

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

M
Merge  
Michael Kolupaev 已提交
2319 2320
		data.checkAlter(params);

2321 2322 2323 2324 2325
		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 已提交
2326

2327 2328 2329 2330
		new_columns_str = ColumnsDescription<false>{
			new_columns, new_materialized_columns,
			new_alias_columns, new_column_defaults
		}.toString();
M
Merge  
Michael Kolupaev 已提交
2331 2332 2333 2334 2335 2336 2337 2338 2339 2340 2341 2342 2343 2344

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

M
Merge  
Michael Kolupaev 已提交
2346 2347 2348 2349 2350 2351 2352 2353 2354 2355 2356 2357 2358 2359 2360 2361 2362 2363 2364 2365 2366 2367 2368 2369 2370 2371 2372 2373 2374 2375
	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);
2376

M
Merge  
Michael Kolupaev 已提交
2377 2378 2379 2380 2381 2382 2383
			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 已提交
2384
			if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/columns", &stat, alter_query_event))
M
Merge  
Michael Kolupaev 已提交
2385 2386 2387 2388 2389 2390 2391 2392 2393 2394 2395 2396 2397 2398 2399 2400 2401 2402
			{
				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 已提交
2403 2404 2405 2406 2407

/// Название воображаемого куска, покрывающего все возможные куски в указанном месяце с номерами в указанном диапазоне.
static String getFakePartNameForDrop(const String & month_name, UInt64 left, UInt64 right)
{
	/// Диапазон дат - весь месяц.
A
Merge  
Alexey Arno 已提交
2408
	const auto & lut = DateLUT::instance();
2409
	time_t start_time = lut.YYYYMMDDToDate(parse<UInt32>(month_name + "01"));
M
Merge  
Michael Kolupaev 已提交
2410 2411 2412 2413 2414 2415 2416
	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 已提交
2417

2418
void StorageReplicatedMergeTree::dropUnreplicatedPartition(const Field & partition, const bool detach, const Settings & settings)
M
Merge  
Michael Kolupaev 已提交
2419
{
A
Merge  
Andrey Mironov 已提交
2420 2421 2422 2423 2424 2425 2426 2427 2428 2429 2430 2431 2432 2433 2434
	if (!unreplicated_data)
		return;

	/// Просит завершить мерджи и не позволяет им начаться.
	/// Это защищает от "оживания" данных за удалённую партицию после завершения мерджа.
	const MergeTreeMergeBlocker merge_blocker{*unreplicated_merger};
	auto structure_lock = lockStructure(true);

	const DayNum_t month = MergeTreeData::getMonthDayNum(partition);

	size_t removed_parts = 0;
	MergeTreeData::DataParts parts = unreplicated_data->getDataParts();

	for (const auto & part : parts)
	{
A
Merge  
Alexey Milovidov 已提交
2435
		if (part->month != month)
A
Merge  
Andrey Mironov 已提交
2436 2437 2438 2439 2440
			continue;

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

2441 2442 2443 2444
		if (detach)
			unreplicated_data->renameAndDetachPart(part, "");
		else
			unreplicated_data->replaceParts({part}, {}, false);
A
Merge  
Andrey Mironov 已提交
2445 2446
	}

2447
	LOG_INFO(log, (detach ? "Detached " : "Removed ") << removed_parts << " unreplicated parts inside " << apply_visitor(FieldVisitorToString(), partition) << ".");
A
Merge  
Andrey Mironov 已提交
2448 2449 2450 2451 2452 2453 2454
}


void StorageReplicatedMergeTree::dropPartition(const Field & field, bool detach, bool unreplicated, const Settings & settings)
{
	if (unreplicated)
	{
2455
		dropUnreplicatedPartition(field, detach, settings);
A
Merge  
Andrey Mironov 已提交
2456 2457 2458 2459

		return;
	}

A
Merge  
Alexey Milovidov 已提交
2460
	auto zookeeper = getZooKeeper();
A
Merge  
Alexey Milovidov 已提交
2461
	String month_name = MergeTreeData::getMonthName(field);
M
Merge  
Michael Kolupaev 已提交
2462 2463 2464

	/// TODO: Делать запрос в лидера по TCP.
	if (!is_leader_node)
2465
		throw Exception(String(detach ? "DETACH" : "DROP") + " PARTITION can only be done on leader replica.", ErrorCodes::NOT_LEADER);
M
Merge  
Michael Kolupaev 已提交
2466 2467 2468 2469 2470 2471 2472

	/** Пропустим один номер в block_numbers для удаляемого месяца, и будем удалять только куски до этого номера.
	  * Это запретит мерджи удаляемых кусков с новыми вставляемыми данными.
	  * Инвариант: в логе не появятся слияния удаляемых кусков с другими кусками.
	  * NOTE: Если понадобится аналогично поддержать запрос DROP PART, для него придется придумать какой-нибудь новый механизм,
	  *        чтобы гарантировать этот инвариант.
	  */
A
Merge  
Alexey Milovidov 已提交
2473
	Int64 right;
M
Merge  
Michael Kolupaev 已提交
2474 2475 2476 2477 2478 2479 2480 2481 2482 2483 2484 2485 2486 2487 2488 2489 2490 2491

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

		virtual_parts.add(fake_part_name);
	}

M
Merge  
Michael Kolupaev 已提交
2497
	/// Наконец, добившись нужных инвариантов, можно положить запись в лог.
M
Merge  
Michael Kolupaev 已提交
2498 2499 2500 2501 2502 2503
	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 已提交
2504
	entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);
M
Merge  
Michael Kolupaev 已提交
2505

2506 2507 2508 2509 2510 2511 2512 2513
	/// Если надо - дожидаемся выполнения операции на себе или на всех репликах.
	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 已提交
2514 2515
}

A
Merge  
Alexey Milovidov 已提交
2516

2517
void StorageReplicatedMergeTree::attachPartition(const Field & field, bool unreplicated, bool attach_part, const Settings & settings)
M
Merge  
Michael Kolupaev 已提交
2518
{
A
Merge  
Alexey Milovidov 已提交
2519
	auto zookeeper = getZooKeeper();
A
Merge  
Alexey Milovidov 已提交
2520
	String partition;
M
Merge  
Michael Kolupaev 已提交
2521

A
Merge  
Alexey Milovidov 已提交
2522
	if (attach_part)
A
Merge  
Alexey Milovidov 已提交
2523
		partition = field.safeGet<String>();
A
Merge  
Alexey Milovidov 已提交
2524 2525
	else
		partition = MergeTreeData::getMonthName(field);
M
Merge  
Michael Kolupaev 已提交
2526 2527 2528 2529 2530 2531 2532 2533 2534 2535 2536 2537 2538 2539 2540 2541 2542 2543

	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;
2544
			if (0 != name.compare(0, partition.size(), partition))
M
Merge  
Michael Kolupaev 已提交
2545 2546 2547 2548 2549 2550 2551 2552 2553 2554 2555 2556 2557 2558 2559 2560 2561 2562
				continue;
			LOG_DEBUG(log, "Found part " << name);
			active_parts.add(name);
		}
		LOG_DEBUG(log, active_parts.size() << " of them are active");
		parts = active_parts.getParts();
	}

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

	/// Выделим добавляемым кускам максимальные свободные номера, меньшие RESERVED_BLOCK_NUMBERS.
	/// NOTE: Проверка свободности номеров никак не синхронизируется. Выполнять несколько запросов ATTACH/DETACH/DROP одновременно нельзя.
A
Merge  
Alexey Milovidov 已提交
2563 2564
	Int64 min_used_number = RESERVED_BLOCK_NUMBERS;
	DayNum_t month = DateLUT::instance().makeDayNum(parse<UInt16>(partition.substr(0, 4)), parse<UInt8>(partition.substr(4, 2)), 0);
M
Merge  
Michael Kolupaev 已提交
2565 2566 2567 2568

	{
		auto existing_parts = data.getDataParts();
		for (const auto & part : existing_parts)
A
Merge  
Alexey Milovidov 已提交
2569 2570
			if (part->month == month)
				min_used_number = std::min(min_used_number, part->left);
M
Merge  
Michael Kolupaev 已提交
2571 2572 2573 2574 2575 2576 2577 2578 2579 2580 2581 2582 2583 2584 2585 2586 2587 2588 2589 2590 2591 2592 2593 2594 2595 2596 2597 2598
	}

	/// Добавим записи в лог.
	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);
2599

2600 2601
	/// Если надо - дожидаемся выполнения операции на себе или на всех репликах.
	if (settings.replication_alter_partitions_sync != 0)
M
Merge  
Michael Kolupaev 已提交
2602
	{
2603 2604 2605 2606 2607 2608 2609 2610 2611 2612
		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 已提交
2613

2614 2615
			++i;
		}
M
Merge  
Michael Kolupaev 已提交
2616
	}
M
Merge  
Michael Kolupaev 已提交
2617 2618
}

A
Merge  
Alexey Milovidov 已提交
2619

M
Merge  
Michael Kolupaev 已提交
2620 2621
void StorageReplicatedMergeTree::drop()
{
2622 2623
	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 已提交
2624

A
Merge  
Alexey Milovidov 已提交
2625 2626
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
2627 2628
	shutdown();

M
Merge  
Michael Kolupaev 已提交
2629
	LOG_INFO(log, "Removing replica " << replica_path);
M
Merge  
Michael Kolupaev 已提交
2630
	replica_is_active_node = nullptr;
M
Merge  
Michael Kolupaev 已提交
2631 2632
	zookeeper->tryRemoveRecursive(replica_path);

M
Merge  
Michael Kolupaev 已提交
2633
	/// Проверяем, что zookeeper_path существует: его могла удалить другая реплика после выполнения предыдущей строки.
M
Merge  
Michael Kolupaev 已提交
2634 2635
	Strings replicas;
	if (zookeeper->tryGetChildren(zookeeper_path + "/replicas", replicas) == ZOK && replicas.empty())
M
Merge  
Michael Kolupaev 已提交
2636 2637
	{
		LOG_INFO(log, "Removing table " << zookeeper_path << " (this might take several minutes)");
M
Merge  
Michael Kolupaev 已提交
2638
		zookeeper->tryRemoveRecursive(zookeeper_path);
M
Merge  
Michael Kolupaev 已提交
2639
	}
M
Merge  
Michael Kolupaev 已提交
2640 2641

	data.dropAllData();
M
Merge  
Michael Kolupaev 已提交
2642 2643
}

A
Merge  
Alexey Milovidov 已提交
2644

M
Merge  
Michael Kolupaev 已提交
2645 2646 2647 2648 2649 2650 2651 2652 2653 2654 2655 2656 2657 2658 2659
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 已提交
2660

M
Merge  
Michael Kolupaev 已提交
2661 2662
AbandonableLockInZooKeeper StorageReplicatedMergeTree::allocateBlockNumber(const String & month_name)
{
A
Merge  
Alexey Milovidov 已提交
2663 2664
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
2665 2666 2667 2668 2669 2670 2671 2672
	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 已提交
2673
		for (size_t i = 0; i < RESERVED_BLOCK_NUMBERS; ++i)
M
Merge  
Michael Kolupaev 已提交
2674 2675 2676 2677 2678 2679 2680 2681 2682 2683 2684 2685 2686
		{
			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 已提交
2687

M
Merge  
Michael Kolupaev 已提交
2688
void StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry(const LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
2689
{
A
Merge  
Alexey Milovidov 已提交
2690
	auto zookeeper = getZooKeeper();
M
Merge  
Michael Kolupaev 已提交
2691 2692
	LOG_DEBUG(log, "Waiting for all replicas to process " << entry.znode_name);

A
Merge  
Alexey Milovidov 已提交
2693 2694 2695 2696 2697 2698 2699 2700 2701 2702
	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 已提交
2703 2704
	auto zookeeper = getZooKeeper();

2705 2706
	String entry_str = entry.toString();
	String log_node_name;
M
Merge  
Michael Kolupaev 已提交
2707

2708 2709 2710 2711 2712 2713 2714 2715 2716
	/** В эту функцию могут передать entry двух видов:
	  * 1. (более часто) Из директории log - общего лога, откуда реплики копируют записи в свою queue.
	  * 2. Из директории queue одной из реплик.
	  *
	  * Проблема в том, что номера (sequential нод) элементов очереди в log и в queue не совпадают.
	  * (И в queue не совпадают номера у одного и того же элемента лога для разных реплик.)
	  *
	  * Поэтому следует рассматривать эти случаи по-отдельности.
	  */
M
Merge  
Michael Kolupaev 已提交
2717

2718 2719 2720 2721 2722 2723 2724
	/** Первое - нужно дождаться, пока реплика возьмёт к себе в queue элемент очереди из log,
	  *  если она ещё этого не сделала (см. функцию pullLogsToQueue).
	  *
	  * Для этого проверяем её узел log_pointer - максимальный номер взятого элемента из log плюс единица.
	  */

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

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

2732 2733 2734 2735 2736 2737 2738 2739 2740 2741 2742 2743 2744
		LOG_DEBUG(log, "Waiting for " << replica << " to pull " << log_node_name << " to queue");

		/// Дождемся, пока запись попадет в очередь реплики.
		while (true)
		{
			zkutil::EventPtr event = new Poco::Event;

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

			event->wait();
		}
A
Merge  
Alexey Milovidov 已提交
2745
	}
2746 2747 2748 2749 2750 2751 2752
	else if (0 == entry.znode_name.compare(0, strlen("queue-"), "queue-"))
	{
		/** В этом случае номер log-ноды неизвестен. Нужно просмотреть все от log_pointer до конца,
		  *  ища ноду с таким же содержимым. И если мы её не найдём - значит реплика уже взяла эту запись в свою queue.
		  */

		String log_pointer = zookeeper->get(zookeeper_path + "/replicas/" + replica + "/log_pointer");
M
Merge  
Michael Kolupaev 已提交
2753

2754 2755 2756 2757 2758 2759 2760 2761 2762 2763 2764 2765 2766 2767 2768 2769 2770 2771 2772 2773 2774 2775 2776 2777 2778 2779 2780 2781 2782 2783 2784 2785 2786 2787 2788 2789 2790 2791 2792 2793 2794 2795 2796 2797 2798 2799 2800 2801 2802 2803
		Strings log_entries = zookeeper->getChildren(zookeeper_path + "/log");
		UInt64 log_index = 0;
		bool found = false;

		for (const String & log_entry_name : log_entries)
		{
			log_index = parse<UInt64>(log_entry_name.substr(log_entry_name.size() - 10));

			if (!log_pointer.empty() && log_index < parse<UInt64>(log_pointer))
				continue;

			String log_entry_str;
			bool exists = zookeeper->tryGet(zookeeper_path + "/log/" + log_entry_name, log_entry_str);
			if (exists && entry_str == log_entry_str)
			{
				found = true;
				log_node_name = log_entry_name;
				break;
			}
		}

		if (found)
		{
			LOG_DEBUG(log, "Waiting for " << replica << " to pull " << log_node_name << " to queue");

			/// Дождемся, пока запись попадет в очередь реплики.
			while (true)
			{
				zkutil::EventPtr event = new Poco::Event;

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

				event->wait();
			}
		}
	}
	else
		throw Exception("Logical error: unexpected name of log node: " + entry.znode_name, ErrorCodes::LOGICAL_ERROR);

	if (!log_node_name.empty())
		LOG_DEBUG(log, "Looking for node corresponding to " << log_node_name << " in " << replica << " queue");
	else
		LOG_DEBUG(log, "Looking for corresponding node in " << replica << " queue");

	/** Второе - найдем соответствующую запись в очереди указанной реплики (replica).
	  * Её номер может не совпадать ни с log-узлом, ни с queue-узлом у текущей реплики (у нас).
	  * Поэтому, ищем путём сравнения содержимого.
	  */
M
Merge  
Michael Kolupaev 已提交
2804

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

A
Merge  
Alexey Milovidov 已提交
2808 2809 2810 2811
	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);
2812
		if (exists && queue_entry_str == entry_str)
M
Merge  
Michael Kolupaev 已提交
2813
		{
2814
			queue_entry_to_wait_for = entry_name;
A
Merge  
Alexey Milovidov 已提交
2815
			break;
M
Merge  
Michael Kolupaev 已提交
2816
		}
A
Merge  
Alexey Milovidov 已提交
2817
	}
M
Merge  
Michael Kolupaev 已提交
2818

A
Merge  
Alexey Milovidov 已提交
2819
	/// Пока искали запись, ее уже выполнили и удалили.
2820 2821 2822
	if (queue_entry_to_wait_for.empty())
	{
		LOG_DEBUG(log, "No corresponding node found. Assuming it has been already processed.");
A
Merge  
Alexey Milovidov 已提交
2823
		return;
2824 2825 2826
	}

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

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


2833
void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields)
2834
{
A
Merge  
Alexey Milovidov 已提交
2835 2836
	auto zookeeper = getZooKeeper();

2837
	res.is_leader = is_leader_node;
2838
	res.is_readonly = is_readonly;
2839
	res.is_session_expired = !zookeeper || zookeeper->expired();
2840 2841 2842 2843 2844 2845 2846 2847

	{
		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 已提交
2848
		res.queue_oldest_time = 0;
2849 2850
		res.inserts_oldest_time = 0;
		res.merges_oldest_time = 0;
2851 2852 2853

		for (const LogEntryPtr & entry : queue)
		{
2854 2855 2856
			if (entry->create_time && (!res.queue_oldest_time || entry->create_time < res.queue_oldest_time))
				res.queue_oldest_time = entry->create_time;

2857
			if (entry->type == LogEntry::GET_PART)
2858
			{
2859
				++res.inserts_in_queue;
2860 2861 2862 2863 2864

				if (entry->create_time && (!res.inserts_oldest_time || entry->create_time < res.inserts_oldest_time))
					res.inserts_oldest_time = entry->create_time;
			}

2865
			if (entry->type == LogEntry::MERGE_PARTS)
2866
			{
2867
				++res.merges_in_queue;
A
Merge  
Alexey Milovidov 已提交
2868

2869 2870 2871
				if (entry->create_time && (!res.merges_oldest_time || entry->create_time < res.merges_oldest_time))
					res.merges_oldest_time = entry->create_time;
			}
2872 2873 2874 2875 2876 2877 2878 2879 2880 2881 2882 2883 2884
		}
	}

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

2885
	if (res.is_session_expired || !with_zk_fields)
2886 2887 2888 2889 2890 2891 2892 2893 2894 2895
	{
		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");

2896 2897 2898 2899 2900 2901 2902 2903 2904 2905 2906 2907
		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);
2908 2909 2910 2911 2912 2913 2914 2915 2916 2917 2918

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

2919

2920
void StorageReplicatedMergeTree::fetchPartition(const Field & partition, const String & from_, const Settings & settings)
2921
{
A
Merge  
Alexey Milovidov 已提交
2922 2923
	auto zookeeper = getZooKeeper();

2924 2925
	String partition_str = MergeTreeData::getMonthName(partition);

2926
	String from = from_;
2927 2928
	if (from.back() == '/')
		from.resize(from.size() - 1);
2929

2930 2931
	LOG_INFO(log, "Will fetch partition " << partition_str << " from shard " << from_);

2932 2933 2934 2935 2936 2937 2938 2939
	/** Проверим, что в директории 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);

2940 2941 2942 2943 2944 2945 2946 2947 2948 2949 2950 2951
	/// Список реплик шарда-источника.
	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())
2952
		throw Exception("No active replicas for shard " + from, ErrorCodes::NO_ACTIVE_REPLICAS);
2953 2954

	/** Надо выбрать лучшую (наиболее актуальную) реплику.
2955
	  * Это реплика с максимальным log_pointer, затем с минимальным размером queue.
2956 2957
	  * NOTE Это не совсем лучший критерий. Для скачивания старых партиций это не имеет смысла,
	  *  и было бы неплохо уметь выбирать реплику, ближайшую по сети.
2958
	  * NOTE Разумеется, здесь есть data race-ы. Можно решить ретраями.
2959
	  */
2960 2961 2962 2963 2964 2965
	Int64 max_log_pointer = -1;
	UInt64 min_queue_size = std::numeric_limits<UInt64>::max();
	String best_replica;

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

2968
		String log_pointer_str = zookeeper->get(current_replica_path + "/log_pointer");
2969 2970 2971
		Int64 log_pointer = log_pointer_str.empty() ? 0 : parse<UInt64>(log_pointer_str);

		zkutil::Stat stat;
2972
		zookeeper->get(current_replica_path + "/queue", &stat);
2973 2974 2975 2976 2977 2978 2979 2980 2981 2982 2983 2984 2985 2986 2987 2988 2989
		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.");

2990 2991
	String best_replica_path = from + "/replicas/" + best_replica;

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

2994 2995 2996 2997 2998 2999 3000 3001 3002 3003 3004 3005 3006 3007 3008 3009 3010 3011 3012 3013 3014 3015 3016 3017 3018 3019 3020 3021 3022 3023
	/** Пытаемся скачать эти куски.
	  * Часть из них могла удалиться из-за мерджа.
	  * В этом случае, обновляем информацию о доступных кусках и пробуем снова.
	  */

	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);
3024 3025 3026

			if (parts_to_fetch.empty())
				throw Exception("Partition " + partition_str + " on " + best_replica_path + " doesn't exist", ErrorCodes::PARTITION_DOESNT_EXIST);
3027 3028 3029 3030 3031 3032 3033 3034 3035 3036 3037 3038 3039 3040 3041 3042 3043 3044 3045 3046
		}
		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
			{
3047
				fetchPart(part, best_replica_path, true, 0);
3048 3049 3050 3051 3052 3053 3054 3055 3056 3057
			}
			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);
			}
		}
3058

3059 3060
		++try_no;
	} while (!missing_parts.empty());
3061 3062 3063
}


3064 3065 3066 3067 3068 3069 3070 3071 3072 3073 3074 3075 3076
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 已提交
3077
}