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

29

M
Merge  
Michael Kolupaev 已提交
30 31 32
namespace DB
{

M
Merge  
Michael Kolupaev 已提交
33

M
Merge  
Michael Kolupaev 已提交
34
const auto ERROR_SLEEP_MS = 1000;
M
Merge  
Michael Kolupaev 已提交
35
const auto MERGE_SELECTING_SLEEP_MS = 5 * 1000;
M
Merge  
Michael Kolupaev 已提交
36

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

M
Merge  
Michael Kolupaev 已提交
39

40 41 42 43 44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67 68 69 70 71 72 73 74 75 76
/** Для каждого куска есть сразу три места, где он должен быть:
  * 1. В оперативке (RAM), MergeTreeData::data_parts, all_data_parts.
  * 2. В файловой системе (FS), директория с данными таблицы.
  * 3. В ZooKeeper (ZK).
  *
  * При добавлении куска, его надо добавить сразу в эти три места.
  * Это делается так:
  * - [FS] сначала записываем кусок во временную директорию на файловой системе;
  * - [FS] переименовываем временный кусок в результирующий на файловой системе;
  * - [RAM] сразу же после этого добавляем его в data_parts, и удаляем из data_parts покрываемые им куски;
  * - [RAM] также устанавливаем объект Transaction, который в случае исключения (в следующем пункте),
  *   откатит изменения в data_parts (из предыдущего пункта) назад;
  * - [ZK] затем отправляем транзакцию (multi) на добавление куска в ZooKeeper (и ещё некоторых действий);
  * - [FS, ZK] кстати, удаление покрываемых (старых) кусков из файловой системы, из ZooKeeper и из all_data_parts
  *   делается отложенно, через несколько минут.
  *
  * Здесь нет никакой атомарности.
  * Можно было бы добиться атомарности с помощью undo/redo логов и флага в DataPart, когда он полностью готов.
  * Но это было бы неудобно - пришлось бы писать undo/redo логи для каждого Part-а в ZK, а это увеличило бы и без того большое количество взаимодействий.
  *
  * Вместо этого, мы вынуждены работать в ситуации, когда в любой момент времени
  *  (из другого потока, или после рестарта сервера) может наблюдаться недоделанная до конца транзакция.
  *  (заметим - для этого кусок должен быть в RAM)
  * Из этих случаев наиболее частый - когда кусок уже есть в data_parts, но его ещё нет в ZooKeeper.
  * Этот случай надо отличить от случая, когда такая ситуация достигается вследствие какого-то повреждения состояния.
  *
  * Делаем это с помощью порога на время.
  * Если кусок достаточно молодой, то его отсутствие в ZooKeeper будем воспринимать оптимистично - как будто он просто не успел ещё туда добавиться
  *  - как будто транзакция ещё не выполнена, но скоро выполнится.
  * А если кусок старый, то его отсутствие в ZooKeeper будем воспринимать как недоделанную транзакцию, которую нужно откатить.
  *
  * PS. Возможно, было бы лучше добавить в DataPart флаг о том, что кусок вставлен в ZK.
  * Но здесь уже слишком легко запутаться с консистентностью этого флага.
  */
const auto MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER = 5 * 60;


M
Merge  
Michael Kolupaev 已提交
77 78 79
StorageReplicatedMergeTree::StorageReplicatedMergeTree(
	const String & zookeeper_path_,
	const String & replica_name_,
M
Merge  
Michael Kolupaev 已提交
80
	bool attach,
M
Merge  
Michael Kolupaev 已提交
81 82
	const String & path_, const String & database_name_, const String & name_,
	NamesAndTypesListPtr columns_,
83
	const NamesAndTypesList & materialized_columns_,
84 85
	const NamesAndTypesList & alias_columns_,
	const ColumnDefaults & column_defaults_,
M
Merge  
Michael Kolupaev 已提交
86
	Context & context_,
M
Merge  
Michael Kolupaev 已提交
87 88 89 90 91 92
	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 已提交
93
	const Names & columns_to_sum_,
M
Merge  
Michael Kolupaev 已提交
94
	const MergeTreeSettings & settings_)
95
    : IStorage{materialized_columns_, alias_columns_, column_defaults_}, context(context_),
A
Merge  
Alexey Milovidov 已提交
96
	current_zookeeper(context.getZooKeeper()), database_name(database_name_),
M
Merge  
Michael Kolupaev 已提交
97 98 99
	table_name(name_), full_path(path_ + escapeForFileName(table_name) + '/'),
	zookeeper_path(context.getMacros().expand(zookeeper_path_)),
	replica_name(context.getMacros().expand(replica_name_)),
100 101 102
	data(full_path, columns_,
		 materialized_columns_, alias_columns_, column_defaults_,
		 context_, primary_expr_ast_, date_column_name_,
A
Merge  
Alexey Milovidov 已提交
103
		 sampling_expression_, index_granularity_, mode_, sign_column_, columns_to_sum_,
104 105
		 settings_, database_name_ + "." + table_name, true,
		 std::bind(&StorageReplicatedMergeTree::enqueuePartForCheck, this, std::placeholders::_1)),
106 107
	reader(data), writer(data), merger(data), fetcher(data), shutdown_event(false),
	log(&Logger::get(database_name + "." + table_name + " (StorageReplicatedMergeTree)"))
M
Merge  
Michael Kolupaev 已提交
108
{
109 110
	if (!zookeeper_path.empty() && zookeeper_path.back() == '/')
		zookeeper_path.resize(zookeeper_path.size() - 1);
M
Merge  
Michael Kolupaev 已提交
111 112 113 114
	replica_path = zookeeper_path + "/replicas/" + replica_name;

	bool skip_sanity_checks = false;

115
	try
M
Merge  
Michael Kolupaev 已提交
116
	{
A
Merge  
Alexey Milovidov 已提交
117
		if (current_zookeeper && current_zookeeper->exists(replica_path + "/flags/force_restore_data"))
118 119
		{
			skip_sanity_checks = true;
A
Merge  
Alexey Milovidov 已提交
120
			current_zookeeper->remove(replica_path + "/flags/force_restore_data");
M
Merge  
Michael Kolupaev 已提交
121

122 123 124 125 126 127 128 129 130 131
			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 已提交
132
			current_zookeeper = nullptr;
133 134 135
		}
		else
			throw;
M
Merge  
Michael Kolupaev 已提交
136 137 138 139
	}

	data.loadDataParts(skip_sanity_checks);

A
Merge  
Alexey Milovidov 已提交
140
	if (!current_zookeeper)
M
Merge  
Michael Kolupaev 已提交
141
	{
M
Merge  
Michael Kolupaev 已提交
142 143 144
		if (!attach)
			throw Exception("Can't create replicated table without ZooKeeper", ErrorCodes::NO_ZOOKEEPER);

145
		/// Не активируем реплику. Она будет в режиме readonly.
M
Merge  
Michael Kolupaev 已提交
146 147 148
		return;
	}

M
Merge  
Michael Kolupaev 已提交
149 150
	if (!attach)
	{
A
Merge  
Alexey Milovidov 已提交
151 152 153
		if (!data.getDataParts().empty())
			throw Exception("Data directory for table already containing data parts - probably it was unclean DROP table or manual intervention. You must either clear directory by hand or use ATTACH TABLE instead of CREATE TABLE if you need to use that parts.", ErrorCodes::INCORRECT_DATA);

M
Merge  
Michael Kolupaev 已提交
154
		createTableIfNotExists();
M
Merge  
Michael Kolupaev 已提交
155

M
Merge  
Michael Kolupaev 已提交
156
		checkTableStructure(false, false);
M
Merge  
Michael Kolupaev 已提交
157
		createReplica();
M
Merge  
Michael Kolupaev 已提交
158 159 160
	}
	else
	{
M
Merge  
Michael Kolupaev 已提交
161
		checkTableStructure(skip_sanity_checks, true);
M
Merge  
Michael Kolupaev 已提交
162
		checkParts(skip_sanity_checks);
M
Merge  
Michael Kolupaev 已提交
163
	}
M
Merge  
Michael Kolupaev 已提交
164

165 166
	createNewZooKeeperNodes();

M
Merge  
Michael Kolupaev 已提交
167
	initVirtualParts();
M
Merge  
Michael Kolupaev 已提交
168 169 170 171

	String unreplicated_path = full_path + "unreplicated/";
	if (Poco::File(unreplicated_path).exists())
	{
172 173
		unreplicated_data.reset(new MergeTreeData(unreplicated_path, columns_,
			materialized_columns_, alias_columns_, column_defaults_,
174
			context_, primary_expr_ast_,
A
Merge  
Alexey Milovidov 已提交
175
			date_column_name_, sampling_expression_, index_granularity_, mode_, sign_column_, columns_to_sum_, settings_,
M
Merge  
Michael Kolupaev 已提交
176
			database_name_ + "." + table_name + "[unreplicated]", false));
177 178 179

		unreplicated_data->loadDataParts(skip_sanity_checks);

180 181 182 183 184 185 186 187 188 189
		if (unreplicated_data->getDataPartsVector().empty())
		{
			unreplicated_data.reset();
		}
		else
		{
			LOG_INFO(log, "Have unreplicated data");
			unreplicated_reader.reset(new MergeTreeDataSelectExecutor(*unreplicated_data));
			unreplicated_merger.reset(new MergeTreeDataMerger(*unreplicated_data));
		}
M
Merge  
Michael Kolupaev 已提交
190
	}
M
Merge  
Michael Kolupaev 已提交
191

192
	loadQueue();
M
Merge  
Michael Kolupaev 已提交
193

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

198

199 200 201 202 203 204 205 206 207 208 209 210 211 212
void StorageReplicatedMergeTree::createNewZooKeeperNodes()
{
	auto zookeeper = getZooKeeper();

	/// Работа с кворумом.
	zookeeper->createIfNotExists(zookeeper_path + "/quorum", "");
	zookeeper->createIfNotExists(zookeeper_path + "/quorum/last_part", "");
	zookeeper->createIfNotExists(zookeeper_path + "/quorum/failed_parts", "");

	/// Отслеживание отставания реплик.
	zookeeper->createIfNotExists(replica_path + "/min_unprocessed_insert_time", "");
}


M
Merge  
Michael Kolupaev 已提交
213 214 215
StoragePtr StorageReplicatedMergeTree::create(
	const String & zookeeper_path_,
	const String & replica_name_,
M
Merge  
Michael Kolupaev 已提交
216
	bool attach,
M
Merge  
Michael Kolupaev 已提交
217 218
	const String & path_, const String & database_name_, const String & name_,
	NamesAndTypesListPtr columns_,
219
	const NamesAndTypesList & materialized_columns_,
220 221
	const NamesAndTypesList & alias_columns_,
	const ColumnDefaults & column_defaults_,
M
Merge  
Michael Kolupaev 已提交
222
	Context & context_,
M
Merge  
Michael Kolupaev 已提交
223 224 225 226 227 228
	ASTPtr & primary_expr_ast_,
	const String & date_column_name_,
	const ASTPtr & sampling_expression_,
	size_t index_granularity_,
	MergeTreeData::Mode mode_,
	const String & sign_column_,
229
	const Names & columns_to_sum_,
M
Merge  
Michael Kolupaev 已提交
230 231
	const MergeTreeSettings & settings_)
{
232 233 234
	auto res = new StorageReplicatedMergeTree{
		zookeeper_path_, replica_name_, attach,
		path_, database_name_, name_,
235
		columns_, materialized_columns_, alias_columns_, column_defaults_,
236 237
		context_, primary_expr_ast_, date_column_name_,
		sampling_expression_, index_granularity_, mode_,
A
Merge  
Alexey Milovidov 已提交
238 239
		sign_column_, columns_to_sum_, settings_};

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

A
Merge  
Alexey Milovidov 已提交
242
	if (res->getZooKeeper())
M
Merge  
Michael Kolupaev 已提交
243 244
	{
		String endpoint_name = "ReplicatedMergeTree:" + res->replica_path;
M
Merge  
Michael Kolupaev 已提交
245
		InterserverIOEndpointPtr endpoint = new ReplicatedMergeTreePartsServer(res->data, *res);
M
Merge  
Michael Kolupaev 已提交
246 247
		res->endpoint_holder = new InterserverIOEndpointHolder(endpoint_name, endpoint, res->context.getInterserverIOHandler());
	}
248

M
Merge  
Michael Kolupaev 已提交
249
	return res_ptr;
M
Merge  
Michael Kolupaev 已提交
250 251
}

A
Merge  
Alexey Milovidov 已提交
252

M
Merge  
Michael Kolupaev 已提交
253 254 255 256 257 258 259 260
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 已提交
261

A
Merge  
Alexey Milovidov 已提交
262

M
Merge  
Michael Kolupaev 已提交
263
void StorageReplicatedMergeTree::createTableIfNotExists()
M
Merge  
Michael Kolupaev 已提交
264
{
A
Merge  
Alexey Milovidov 已提交
265 266
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
267 268
	if (zookeeper->exists(zookeeper_path))
		return;
M
Merge  
Michael Kolupaev 已提交
269

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

M
Merge  
Michael Kolupaev 已提交
272 273
	zookeeper->createAncestors(zookeeper_path);

M
Merge  
Michael Kolupaev 已提交
274
	/// Запишем метаданные таблицы, чтобы реплики могли сверять с ними параметры таблицы.
A
Merge  
Alexey Milovidov 已提交
275 276 277 278 279 280 281 282 283 284 285
	std::string metadata;
	{
		WriteBufferFromString out(metadata);
		out << "metadata format version: 1" << "\n"
			<< "date column: " << data.date_column_name << "\n"
			<< "sampling expression: " << formattedAST(data.sampling_expression) << "\n"
			<< "index granularity: " << data.index_granularity << "\n"
			<< "mode: " << static_cast<int>(data.mode) << "\n"
			<< "sign column: " << data.sign_column << "\n"
			<< "primary key: " << formattedAST(data.primary_expr_ast) << "\n";
	}
M
Merge  
Michael Kolupaev 已提交
286

A
Alexey Milovidov 已提交
287 288
	auto acl = zookeeper->getDefaultACL();

M
Merge  
Michael Kolupaev 已提交
289 290
	zkutil::Ops ops;
	ops.push_back(new zkutil::Op::Create(zookeeper_path, "",
A
Alexey Milovidov 已提交
291
										 acl, zkutil::CreateMode::Persistent));
A
Merge  
Alexey Milovidov 已提交
292
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/metadata", metadata,
A
Alexey Milovidov 已提交
293
										 acl, zkutil::CreateMode::Persistent));
294 295 296
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/columns", ColumnsDescription<false>{
				data.getColumnsListNonMaterialized(), data.materialized_columns,
				data.alias_columns, data.column_defaults}.toString(),
A
Alexey Milovidov 已提交
297
										 acl, zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
298
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/log", "",
A
Alexey Milovidov 已提交
299
										 acl, zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
300
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/blocks", "",
A
Alexey Milovidov 已提交
301
										 acl, zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
302
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/block_numbers", "",
A
Alexey Milovidov 已提交
303
										 acl, zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
304
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/nonincrement_block_numbers", "",
A
Alexey Milovidov 已提交
305
										 acl, zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
306
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/leader_election", "",
A
Alexey Milovidov 已提交
307
										 acl, zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
308
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/temp", "",
A
Alexey Milovidov 已提交
309
										 acl, zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
310
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/replicas", "",
A
Alexey Milovidov 已提交
311
										 acl, zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
312 313 314 315

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

A
Merge  
Alexey Milovidov 已提交
318

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

M
Merge  
Michael Kolupaev 已提交
326
	String metadata_str = zookeeper->get(zookeeper_path + "/metadata");
M
Merge  
Michael Kolupaev 已提交
327 328 329 330 331 332 333 334 335 336 337 338 339
	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 已提交
340 341
	/// NOTE: Можно сделать менее строгую проверку совпадения выражений, чтобы таблицы не ломались от небольших изменений
	///       в коде formatAST.
M
Merge  
Michael Kolupaev 已提交
342
	assertString(formattedAST(data.primary_expr_ast), buf);
M
Merge  
Michael Kolupaev 已提交
343
	assertString("\n", buf);
M
Merge  
Michael Kolupaev 已提交
344
	assertEOF(buf);
M
Merge  
Michael Kolupaev 已提交
345 346

	zkutil::Stat stat;
347
	auto columns_desc = ColumnsDescription<true>::parse(zookeeper->get(zookeeper_path + "/columns", &stat));
348 349 350 351 352

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

355 356 357 358
	if (columns != data.getColumnsListNonMaterialized() ||
		materialized_columns != data.materialized_columns ||
		alias_columns != data.alias_columns ||
		column_defaults != data.column_defaults)
M
Merge  
Michael Kolupaev 已提交
359
	{
360 361 362 363
		if (allow_alter &&
			(skip_sanity_checks ||
			 data.getColumnsListNonMaterialized().sizeOfDifference(columns) +
			 data.materialized_columns.sizeOfDifference(materialized_columns) <= 2))
M
Merge  
Michael Kolupaev 已提交
364
		{
M
Merge  
Michael Kolupaev 已提交
365
			LOG_WARNING(log, "Table structure in ZooKeeper is a little different from local table structure. Assuming ALTER.");
366 367

			/// Без всяких блокировок, потому что таблица еще не создана.
368 369
			InterpreterAlterQuery::updateMetadata(database_name, table_name, columns,
				materialized_columns, alias_columns, column_defaults, context);
370
			data.setColumnsList(columns);
371 372 373
			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 已提交
374
		}
M
Merge  
Michael Kolupaev 已提交
375
		else
M
Merge  
Michael Kolupaev 已提交
376
		{
M
Merge  
Michael Kolupaev 已提交
377
			throw Exception("Table structure in ZooKeeper is too different from local table structure.",
M
Merge  
Michael Kolupaev 已提交
378
							ErrorCodes::INCOMPATIBLE_COLUMNS);
M
Merge  
Michael Kolupaev 已提交
379
		}
M
Merge  
Michael Kolupaev 已提交
380 381
	}
}
M
Merge  
Michael Kolupaev 已提交
382

A
Merge  
Alexey Milovidov 已提交
383

384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402
/** При необходимости восстановить кусок, реплика сама добавляет в свою очередь запись на его получение.
  * Какое поставить время для этой записи в очереди? Время учитывается при расчёте отставания реплики.
  * Для этих целей имеет смысл использовать время создания недостающего куска
  *  (то есть, при расчёте отставания будет учитано, насколько старый кусок нам нужно восстановить).
  */
static time_t tryGetPartCreateTime(zkutil::ZooKeeperPtr & zookeeper, const String & replica_path, const String & part_name)
{
	time_t res = 0;

	/// Узнаем время создания part-а, если он ещё существует (не был, например, смерджен).
	zkutil::Stat stat;
	String unused;
	if (zookeeper->tryGet(replica_path + "/parts/" + part_name, unused, &stat))
		res = stat.ctime / 1000;

	return res;
}


M
Merge  
Michael Kolupaev 已提交
403 404
void StorageReplicatedMergeTree::createReplica()
{
A
Merge  
Alexey Milovidov 已提交
405 406
	auto zookeeper = getZooKeeper();

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

M
Merge  
Michael Kolupaev 已提交
409
	/// Создадим пустую реплику. Ноду columns создадим в конце - будем использовать ее в качестве признака, что создание реплики завершено.
A
Alexey Milovidov 已提交
410
	auto acl = zookeeper->getDefaultACL();
M
Merge  
Michael Kolupaev 已提交
411
	zkutil::Ops ops;
A
Alexey Milovidov 已提交
412 413 414 415 416 417
	ops.push_back(new zkutil::Op::Create(replica_path, "", acl, zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(replica_path + "/host", "", acl, zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(replica_path + "/log_pointer", "", acl, zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(replica_path + "/queue", "", acl, zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(replica_path + "/parts", "", acl, zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(replica_path + "/flags", "", acl, zkutil::CreateMode::Persistent));
418 419 420 421 422 423 424 425 426 427 428 429

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

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

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

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

M
Merge  
Michael Kolupaev 已提交
443 444 445
	Stat stat;
	zookeeper->exists(replica_path, &stat);
	auto my_create_time = stat.czxid;
M
Merge  
Michael Kolupaev 已提交
446

M
Merge  
Michael Kolupaev 已提交
447 448
	std::random_shuffle(replicas.begin(), replicas.end());
	for (const String & replica : replicas)
M
Merge  
Michael Kolupaev 已提交
449
	{
M
Merge  
Michael Kolupaev 已提交
450 451 452 453 454 455 456 457
		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 已提交
458 459
	}

M
Merge  
Michael Kolupaev 已提交
460
	if (source_replica.empty())
M
Merge  
Michael Kolupaev 已提交
461
	{
M
Merge  
Michael Kolupaev 已提交
462
		LOG_INFO(log, "This is the first replica");
M
Merge  
Michael Kolupaev 已提交
463
	}
M
Merge  
Michael Kolupaev 已提交
464
	else
M
Merge  
Michael Kolupaev 已提交
465
	{
M
Merge  
Michael Kolupaev 已提交
466
		LOG_INFO(log, "Will mimic " << source_replica);
M
Merge  
Michael Kolupaev 已提交
467

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

M
Merge  
Michael Kolupaev 已提交
470 471 472 473 474 475 476 477 478 479 480 481 482 483 484 485 486 487 488 489 490 491 492 493 494 495 496 497 498 499 500 501 502 503 504 505 506 507 508
		/** Если эталонная реплика еще не до конца создана, подождем.
		  * 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");
509 510
		ActiveDataPartSet active_parts_set(parts);

M
Merge  
Michael Kolupaev 已提交
511 512 513 514 515 516 517
		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;
518
			log_entry.create_time = tryGetPartCreateTime(zookeeper, source_path, name);
519

M
Merge  
Michael Kolupaev 已提交
520 521 522 523 524 525 526 527 528
			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);
		}
529 530 531

		/// Далее оно будет загружено в переменную queue в методе loadQueue.

M
Merge  
Michael Kolupaev 已提交
532
		LOG_DEBUG(log, "Copied " << source_queue.size() << " queue entries");
M
Merge  
Michael Kolupaev 已提交
533
	}
M
Merge  
Michael Kolupaev 已提交
534

535 536 537 538 539 540
	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 已提交
541
}
M
Merge  
Michael Kolupaev 已提交
542 543


M
Merge  
Michael Kolupaev 已提交
544
void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
M
Merge  
Michael Kolupaev 已提交
545
{
A
Merge  
Alexey Milovidov 已提交
546 547
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
548
	Strings expected_parts_vec = zookeeper->getChildren(replica_path + "/parts");
M
Merge  
Michael Kolupaev 已提交
549 550

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

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

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

M
Merge  
Michael Kolupaev 已提交
558 559 560 561 562
	for (const auto & part : parts)
	{
		if (expected_parts.count(part->name))
			expected_parts.erase(part->name);
		else
M
Merge  
Michael Kolupaev 已提交
563
			unexpected_parts.insert(part);
M
Merge  
Michael Kolupaev 已提交
564 565
	}

M
Merge  
Michael Kolupaev 已提交
566
	/// Какие локальные куски добавить в ZK.
M
Merge  
Michael Kolupaev 已提交
567
	MergeTreeData::DataPartsVector parts_to_add;
M
Merge  
Michael Kolupaev 已提交
568 569 570 571

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

M
Merge  
Michael Kolupaev 已提交
572 573
	for (const String & missing_name : expected_parts)
	{
M
Merge  
Michael Kolupaev 已提交
574
		/// Если локально не хватает какого-то куска, но есть покрывающий его кусок, можно заменить в ZK недостающий покрывающим.
M
Merge  
Michael Kolupaev 已提交
575
		auto containing = data.getActiveContainingPart(missing_name);
M
Merge  
Michael Kolupaev 已提交
576 577 578 579 580 581 582 583 584 585
		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 已提交
586
		{
M
Merge  
Michael Kolupaev 已提交
587
			LOG_ERROR(log, "Fetching missing part " << missing_name);
M
Merge  
Michael Kolupaev 已提交
588
			parts_to_fetch.push_back(missing_name);
M
Merge  
Michael Kolupaev 已提交
589 590
		}
	}
M
Merge  
Michael Kolupaev 已提交
591

M
Merge  
Michael Kolupaev 已提交
592 593 594
	for (const String & name : parts_to_fetch)
		expected_parts.erase(name);

595 596 597 598 599 600 601 602 603 604 605 606 607 608 609
	/** Для проверки адекватности, для кусков, которые есть в ФС, но нет в ZK, будем учитывать только не самые новые куски.
	  * Потому что неожиданные новые куски обычно возникают лишь оттого, что они не успели записаться в ZK при грубом перезапуске сервера.
	  * Также это возникает от дедуплицированных кусков, которые не успели удалиться.
	  */
	size_t unexpected_parts_nonnew = 0;
	for (const auto & part : unexpected_parts)
		if (part->level > 0 || part->right < RESERVED_BLOCK_NUMBERS)
			++unexpected_parts_nonnew;

	String sanity_report = "There are "
			+ toString(unexpected_parts.size()) + " unexpected parts ("
			+ toString(unexpected_parts_nonnew) + " of them is not just-written), "
			+ toString(parts_to_add.size()) + " unexpectedly merged parts, "
			+ toString(expected_parts.size()) + " missing obsolete parts, "
			+ toString(parts_to_fetch.size()) + " missing parts";
M
Merge  
Michael Kolupaev 已提交
610

611
	/** Можно автоматически синхронизировать данные,
612 613
	  *  если количество ошибок каждого из четырёх типов не больше соответствующих порогов,
	  *  или если отношение общего количества ошибок к общему количеству кусков (минимальному - в локальной файловой системе или в ZK)
614
	  *  не больше некоторого отношения (например 5%).
615 616 617 618
	  *
	  * Большое количество несовпадений в данных на файловой системе и ожидаемых данных
	  *  может свидетельствовать об ошибке конфигурации (сервер случайно подключили как реплику не от того шарда).
	  * В этом случае, защитный механизм не даёт стартовать серверу.
619 620 621
	  */

	size_t min_parts_local_or_expected = std::min(expected_parts_vec.size(), parts.size());
622
	size_t total_difference = parts_to_add.size() + unexpected_parts_nonnew + expected_parts.size() + parts_to_fetch.size();
623 624 625

	bool insane =
		(parts_to_add.size() > data.settings.replicated_max_unexpectedly_merged_parts
626
			|| unexpected_parts_nonnew > data.settings.replicated_max_unexpected_parts
627 628
			|| expected_parts.size() > data.settings.replicated_max_missing_obsolete_parts
			|| parts_to_fetch.size() > data.settings.replicated_max_missing_active_parts)
629
		&& (total_difference > min_parts_local_or_expected * data.settings.replicated_max_ratio_of_wrong_parts);
M
Merge  
Michael Kolupaev 已提交
630

631 632 633 634 635 636
	if (insane && !skip_sanity_checks)
		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);

	if (total_difference > 0)
		LOG_WARNING(log, sanity_report);
M
Merge  
Michael Kolupaev 已提交
637

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

M
Merge  
Michael Kolupaev 已提交
643
		zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
644
		checkPartAndAddToZooKeeper(part, ops);
M
Merge  
Michael Kolupaev 已提交
645
		zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
646
	}
M
Merge  
Michael Kolupaev 已提交
647

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

		zkutil::Ops ops;
654
		removePartFromZooKeeper(name, ops);
M
Merge  
Michael Kolupaev 已提交
655 656 657 658 659 660 661 662 663 664
		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 已提交
665
		log_entry.source_replica = "";
M
Merge  
Michael Kolupaev 已提交
666
		log_entry.new_part_name = name;
667
		log_entry.create_time = tryGetPartCreateTime(zookeeper, replica_path, name);
M
Merge  
Michael Kolupaev 已提交
668 669

		/// Полагаемся, что это происходит до загрузки очереди (loadQueue).
M
Merge  
Michael Kolupaev 已提交
670
		zkutil::Ops ops;
671
		removePartFromZooKeeper(name, ops);
M
Merge  
Michael Kolupaev 已提交
672 673
		ops.push_back(new zkutil::Op::Create(
			replica_path + "/queue/queue-", log_entry.toString(), zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
M
Merge  
Michael Kolupaev 已提交
674
		zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
675 676 677
	}

	/// Удалим лишние локальные куски.
678
	for (const MergeTreeData::DataPartPtr & part : unexpected_parts)
M
Merge  
Michael Kolupaev 已提交
679
	{
M
Merge  
Michael Kolupaev 已提交
680
		LOG_ERROR(log, "Renaming unexpected part " << part->name << " to ignored_" + part->name);
M
Merge  
Michael Kolupaev 已提交
681
		data.renameAndDetachPart(part, "ignored_", true);
M
Merge  
Michael Kolupaev 已提交
682 683
	}
}
M
Merge  
Michael Kolupaev 已提交
684

A
Merge  
Alexey Milovidov 已提交
685

M
Merge  
Michael Kolupaev 已提交
686 687 688 689 690 691 692
void StorageReplicatedMergeTree::initVirtualParts()
{
	auto parts = data.getDataParts();
	for (const auto & part : parts)
		virtual_parts.add(part->name);
}

A
Merge  
Alexey Milovidov 已提交
693

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

M
Merge  
Michael Kolupaev 已提交
698 699 700
	if (part_name.empty())
		part_name = part->name;

701
	check(part->columns);
M
Merge  
Michael Kolupaev 已提交
702
	int expected_columns_version = columns_version;
703

M
Merge  
Michael Kolupaev 已提交
704 705 706 707 708
	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 已提交
709
	{
M
Merge  
Michael Kolupaev 已提交
710 711
		zkutil::Stat stat_before, stat_after;
		String columns_str;
M
Merge  
Michael Kolupaev 已提交
712
		if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name + "/columns", columns_str, &stat_before))
M
Merge  
Michael Kolupaev 已提交
713 714 715
			continue;
		if (columns_str != expected_columns_str)
		{
M
Merge  
Michael Kolupaev 已提交
716
			LOG_INFO(log, "Not checking checksums of part " << part_name << " with replica " << replica
M
Merge  
Michael Kolupaev 已提交
717 718 719
				<< " because columns are different");
			continue;
		}
M
Merge  
Michael Kolupaev 已提交
720
		String checksums_str;
M
Merge  
Michael Kolupaev 已提交
721 722
		/// Проверим, что версия ноды со столбцами не изменилась, пока мы читали checksums.
		/// Это гарантирует, что столбцы и чексуммы относятся к одним и тем же данным.
M
Merge  
Michael Kolupaev 已提交
723 724
		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 已提交
725
			stat_before.version != stat_after.version)
M
Merge  
Michael Kolupaev 已提交
726
		{
M
Merge  
Michael Kolupaev 已提交
727
			LOG_INFO(log, "Not checking checksums of part " << part_name << " with replica " << replica
M
Merge  
Michael Kolupaev 已提交
728 729
				<< " because part changed while we were reading its checksums");
			continue;
M
Merge  
Michael Kolupaev 已提交
730
		}
M
Merge  
Michael Kolupaev 已提交
731 732 733

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

M
Merge  
Michael Kolupaev 已提交
736
	if (zookeeper->exists(replica_path + "/parts/" + part_name))
M
Merge  
Michael Kolupaev 已提交
737
	{
M
Merge  
Michael Kolupaev 已提交
738
		LOG_ERROR(log, "checkPartAndAddToZooKeeper: node " << replica_path + "/parts/" + part_name << " already exists");
M
Merge  
Michael Kolupaev 已提交
739 740 741
		return;
	}

A
Alexey Milovidov 已提交
742 743
	auto acl = zookeeper->getDefaultACL();

744 745 746
	ops.push_back(new zkutil::Op::Check(
		zookeeper_path + "/columns",
		expected_columns_version));
M
Merge  
Michael Kolupaev 已提交
747
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
748
		replica_path + "/parts/" + part_name,
M
Merge  
Michael Kolupaev 已提交
749
		"",
A
Alexey Milovidov 已提交
750
		acl,
M
Merge  
Michael Kolupaev 已提交
751
		zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
752
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
753
		replica_path + "/parts/" + part_name + "/columns",
M
Merge  
Michael Kolupaev 已提交
754
		part->columns.toString(),
A
Alexey Milovidov 已提交
755
		acl,
M
Merge  
Michael Kolupaev 已提交
756
		zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
757
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
758
		replica_path + "/parts/" + part_name + "/checksums",
M
Merge  
Michael Kolupaev 已提交
759
		part->checksums.toString(),
A
Alexey Milovidov 已提交
760
		acl,
M
Merge  
Michael Kolupaev 已提交
761 762 763
		zkutil::CreateMode::Persistent));
}

A
Merge  
Alexey Milovidov 已提交
764

M
Merge  
Michael Kolupaev 已提交
765 766
void StorageReplicatedMergeTree::loadQueue()
{
A
Merge  
Alexey Milovidov 已提交
767 768
	auto zookeeper = getZooKeeper();

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

M
Merge  
Michael Kolupaev 已提交
771
	Strings children = zookeeper->getChildren(replica_path + "/queue");
M
Merge  
Michael Kolupaev 已提交
772
	std::sort(children.begin(), children.end());
773 774 775 776

	std::vector<std::pair<String, zkutil::ZooKeeper::GetFuture>> futures;
	futures.reserve(children.size());

M
Merge  
Michael Kolupaev 已提交
777
	for (const String & child : children)
778 779 780
		futures.emplace_back(child, zookeeper->asyncGet(replica_path + "/queue/" + child));

	for (auto & future : futures)
M
Merge  
Michael Kolupaev 已提交
781
	{
782 783 784 785
		zkutil::ZooKeeper::ValueAndStat res = future.second.get();
		LogEntryPtr entry = LogEntry::parse(res.value, res.stat);

		entry->znode_name = future.first;
M
Merge  
Michael Kolupaev 已提交
786
		entry->addResultToVirtualParts(*this);
M
Merge  
Michael Kolupaev 已提交
787
		queue.push_back(entry);
M
Merge  
Michael Kolupaev 已提交
788 789 790
	}
}

A
Merge  
Alexey Milovidov 已提交
791

M
Merge  
Michael Kolupaev 已提交
792
void StorageReplicatedMergeTree::pullLogsToQueue(zkutil::EventPtr next_update_event)
M
Merge  
Michael Kolupaev 已提交
793
{
A
Merge  
Alexey Milovidov 已提交
794 795
	auto zookeeper = getZooKeeper();

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

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

M
Merge  
Michael Kolupaev 已提交
801
	if (index_str.empty())
M
Merge  
Michael Kolupaev 已提交
802
	{
M
Merge  
Michael Kolupaev 已提交
803 804 805
		/// Если у нас еще нет указателя на лог, поставим указатель на первую запись в нем.
		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 已提交
806

M
Merge  
Michael Kolupaev 已提交
807
		zookeeper->set(replica_path + "/log_pointer", toString(index));
M
Merge  
Michael Kolupaev 已提交
808 809
	}
	else
M
Merge  
Michael Kolupaev 已提交
810
	{
M
Merge  
Michael Kolupaev 已提交
811
		index = parse<UInt64>(index_str);
M
Merge  
Michael Kolupaev 已提交
812
	}
M
Merge  
Michael Kolupaev 已提交
813

M
Merge  
Michael Kolupaev 已提交
814 815
	UInt64 first_index = index;

M
Merge  
Michael Kolupaev 已提交
816
	size_t count = 0;
M
Merge  
Michael Kolupaev 已提交
817
	String entry_str;
A
Merge  
Alexey Milovidov 已提交
818 819
	zkutil::Stat stat;
	while (zookeeper->tryGet(zookeeper_path + "/log/log-" + padIndex(index), entry_str, &stat))
M
Merge  
Michael Kolupaev 已提交
820
	{
M
Merge  
Michael Kolupaev 已提交
821
		++count;
M
Merge  
Michael Kolupaev 已提交
822
		++index;
M
Merge  
Michael Kolupaev 已提交
823

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

M
Merge  
Michael Kolupaev 已提交
826 827 828
		/// Одновременно добавим запись в очередь и продвинем указатель на лог.
		zkutil::Ops ops;
		ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
829
			replica_path + "/queue/queue-", entry_str, zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
M
Merge  
Michael Kolupaev 已提交
830
		ops.push_back(new zkutil::Op::SetData(
M
Merge  
Michael Kolupaev 已提交
831
			replica_path + "/log_pointer", toString(index), -1));
M
Merge  
Michael Kolupaev 已提交
832
		auto results = zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
833

834
		String path_created = dynamic_cast<zkutil::Op::Create &>(ops[0]).getPathCreated();
M
Merge  
Michael Kolupaev 已提交
835 836
		entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
		entry->addResultToVirtualParts(*this);
M
Merge  
Michael Kolupaev 已提交
837
		queue.push_back(entry);
M
Merge  
Michael Kolupaev 已提交
838
	}
M
Merge  
Michael Kolupaev 已提交
839

A
Alexey Milovidov 已提交
840 841
	last_queue_update = time(0);

M
Merge  
Michael Kolupaev 已提交
842 843 844 845
	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 已提交
846
	}
M
Merge  
Michael Kolupaev 已提交
847

M
Merge  
Michael Kolupaev 已提交
848 849 850
	if (!count)
		return;

M
Merge  
Michael Kolupaev 已提交
851 852
	if (queue_task_handle)
		queue_task_handle->wake();
M
Merge  
Michael Kolupaev 已提交
853

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

A
Merge  
Alexey Milovidov 已提交
857

858
bool StorageReplicatedMergeTree::shouldExecuteLogEntry(const LogEntry & entry, String & out_postpone_reason)
M
Merge  
Michael Kolupaev 已提交
859
{
A
Merge  
Alexey Milovidov 已提交
860 861 862
	/// queue_mutex уже захвачен. Функция вызывается только из queueTask.

	if (entry.type == LogEntry::MERGE_PARTS || entry.type == LogEntry::GET_PART || entry.type == LogEntry::ATTACH_PART)
M
Merge  
Michael Kolupaev 已提交
863
	{
A
Merge  
Alexey Milovidov 已提交
864 865 866
		/// Проверим, не создаётся ли сейчас этот же кусок другим действием.
		if (future_parts.count(entry.new_part_name))
		{
867 868 869 870
			String reason = "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.";
			LOG_DEBUG(log, reason);
			out_postpone_reason = reason;
A
Merge  
Alexey Milovidov 已提交
871 872 873 874 875 876 877 878 879 880 881 882 883 884 885 886 887 888 889 890 891
			return false;

			/** Когда соответствующее действие завершится, то shouldExecuteLogEntry, в следующий раз, пройдёт успешно,
			  *  и элемент очереди будет обработан. Сразу же в функции executeLogEntry будет выяснено, что кусок у нас уже есть,
			  *  и элемент очереди будет сразу считаться обработанным.
			  */
		}

		/// Более сложная проверка - не создаётся ли сейчас другим действием кусок, который покроет этот кусок.
		/// NOTE То, что выше - избыточно, но оставлено ради более удобного сообщения в логе.
		ActiveDataPartSet::Part result_part;
		ActiveDataPartSet::parsePartName(entry.new_part_name, result_part);

		/// Оно может тормозить при большом размере future_parts. Но он не может быть большим, так как ограничен BackgroundProcessingPool.
		for (const auto & future_part_name : future_parts)
		{
			ActiveDataPartSet::Part future_part;
			ActiveDataPartSet::parsePartName(future_part_name, future_part);

			if (future_part.contains(result_part))
			{
892 893 894 895
				String reason = "Not executing log entry for part " + entry.new_part_name
					+ " because another log entry for covering part " + future_part_name + " is being processed.";
				LOG_DEBUG(log, reason);
				out_postpone_reason = reason;
A
Merge  
Alexey Milovidov 已提交
896 897 898
				return false;
			}
		}
M
Merge  
Michael Kolupaev 已提交
899 900
	}

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

		if (merger.isCancelled())
		{
922 923 924
			String reason = "Not executing log entry for part " + entry.new_part_name + " because merges are cancelled now.";
			LOG_DEBUG(log, reason);
			out_postpone_reason = reason;
925 926
			return false;
		}
M
Merge  
Michael Kolupaev 已提交
927 928 929
	}

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

A
Merge  
Alexey Milovidov 已提交
932

M
Merge  
Michael Kolupaev 已提交
933
bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry, BackgroundProcessingPool::Context & pool_context)
M
Merge  
Michael Kolupaev 已提交
934
{
A
Merge  
Alexey Milovidov 已提交
935 936
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
937
	if (entry.type == LogEntry::DROP_RANGE)
M
Merge  
Michael Kolupaev 已提交
938 939 940 941
	{
		executeDropRange(entry);
		return true;
	}
M
Merge  
Michael Kolupaev 已提交
942

M
Merge  
Michael Kolupaev 已提交
943
	if (entry.type == LogEntry::GET_PART ||
M
Merge  
Michael Kolupaev 已提交
944 945
		entry.type == LogEntry::MERGE_PARTS ||
		entry.type == LogEntry::ATTACH_PART)
M
Merge  
Michael Kolupaev 已提交
946 947
	{
		/// Если у нас уже есть этот кусок или покрывающий его кусок, ничего делать не нужно.
M
Merge  
Michael Kolupaev 已提交
948
		MergeTreeData::DataPartPtr containing_part = data.getActiveContainingPart(entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
949

950
		/// Даже если кусок есть локально, его (в исключительных случаях) может не быть в zookeeper. Проверим, что он там есть.
M
Merge  
Michael Kolupaev 已提交
951
		if (containing_part && zookeeper->exists(replica_path + "/parts/" + containing_part->name))
M
Merge  
Michael Kolupaev 已提交
952
		{
M
Merge  
Michael Kolupaev 已提交
953
			if (!(entry.type == LogEntry::GET_PART && entry.source_replica == replica_name))
954
				LOG_DEBUG(log, "Skipping action for part " << entry.new_part_name << " - part already exists.");
M
Merge  
Michael Kolupaev 已提交
955
			return true;
M
Merge  
Michael Kolupaev 已提交
956
		}
M
Merge  
Michael Kolupaev 已提交
957 958
	}

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

962
	/// Возможно, этот кусок нам не нужен, так как при записи с кворумом, кворум пофейлился (см. ниже про /quorum/failed_parts).
963
	if (entry.quorum && zookeeper->exists(zookeeper_path + "/quorum/failed_parts/" + entry.new_part_name))
964 965 966 967 968
	{
		LOG_DEBUG(log, "Skipping action for part " << entry.new_part_name << " because quorum for that part was failed.");
		return true;	/// NOTE Удаление из virtual_parts не делается, но оно нужно только для мерджей.
	}

M
Merge  
Michael Kolupaev 已提交
969 970
	bool do_fetch = false;

M
Merge  
Michael Kolupaev 已提交
971 972
	if (entry.type == LogEntry::GET_PART)
	{
M
Merge  
Michael Kolupaev 已提交
973
		do_fetch = true;
M
Merge  
Michael Kolupaev 已提交
974
	}
M
Merge  
Michael Kolupaev 已提交
975 976 977 978
	else if (entry.type == LogEntry::ATTACH_PART)
	{
		do_fetch = !executeAttachPart(entry);
	}
M
Merge  
Michael Kolupaev 已提交
979 980
	else if (entry.type == LogEntry::MERGE_PARTS)
	{
981 982 983 984 985 986 987 988
		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 已提交
989
		MergeTreeData::DataPartsVector parts;
990
		bool have_all_parts = true;
M
Merge  
Michael Kolupaev 已提交
991 992
		for (const String & name : entry.parts_to_merge)
		{
M
Merge  
Michael Kolupaev 已提交
993
			MergeTreeData::DataPartPtr part = data.getActiveContainingPart(name);
M
Merge  
Michael Kolupaev 已提交
994 995 996 997 998 999 1000
			if (!part)
			{
				have_all_parts = false;
				break;
			}
			if (part->name != name)
			{
M
Merge  
Michael Kolupaev 已提交
1001 1002
				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 已提交
1003 1004 1005
				have_all_parts = false;
				break;
			}
M
Merge  
Michael Kolupaev 已提交
1006 1007
			parts.push_back(part);
		}
M
Merge  
Michael Kolupaev 已提交
1008

M
Merge  
Michael Kolupaev 已提交
1009
		if (!have_all_parts)
M
Merge  
Michael Kolupaev 已提交
1010
		{
M
Merge  
Michael Kolupaev 已提交
1011 1012 1013
			/// Если нет всех нужных кусков, попробуем взять у кого-нибудь уже помердженный кусок.
			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 已提交
1014
		}
M
Merge  
Michael Kolupaev 已提交
1015 1016
		else
		{
M
Merge  
Michael Kolupaev 已提交
1017 1018 1019
			/// Если собираемся сливать большие куски, увеличим счетчик потоков, сливающих большие куски.
			for (const auto & part : parts)
			{
M
Merge  
Michael Kolupaev 已提交
1020
				if (part->size_in_bytes > data.settings.max_bytes_to_merge_parts_small)
M
Merge  
Michael Kolupaev 已提交
1021 1022 1023 1024 1025 1026 1027
				{
					pool_context.incrementCounter("big merges");
					pool_context.incrementCounter("replicated big merges");
					break;
				}
			}

A
Merge  
Alexey Milovidov 已提交
1028
			size_t sum_parts_size_in_bytes = MergeTreeDataMerger::estimateDiskSpaceForMerge(parts);
1029 1030 1031

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

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

1035
			const auto & merge_entry = context.getMergeList().insert(database_name, table_name, entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
1036
			MergeTreeData::Transaction transaction;
A
Merge  
Alexey Arno 已提交
1037
			size_t aio_threshold = context.getSettings().min_bytes_to_use_direct_io;
1038 1039 1040

			MergeTreeData::DataPartPtr part = merger.mergeParts(
				parts, entry.new_part_name, *merge_entry, aio_threshold, &transaction, reserved_space);
M
Merge  
Michael Kolupaev 已提交
1041 1042

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

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

M
Merge  
Michael Kolupaev 已提交
1049
			zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
1050

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

M
Merge  
Michael Kolupaev 已提交
1054 1055 1056
			/** При ZCONNECTIONLOSS или ZOPERATIONTIMEOUT можем зря откатить локальные изменения кусков.
			  * Это не проблема, потому что в таком случае слияние останется в очереди, и мы попробуем снова.
			  */
M
Merge  
Michael Kolupaev 已提交
1057
			transaction.commit();
M
Merge  
Michael Kolupaev 已提交
1058
			merge_selecting_event.set();
M
Merge  
Michael Kolupaev 已提交
1059 1060 1061

			ProfileEvents::increment(ProfileEvents::ReplicatedPartMerges);
		}
M
Merge  
Michael Kolupaev 已提交
1062 1063 1064 1065 1066
	}
	else
	{
		throw Exception("Unexpected log entry type: " + toString(static_cast<int>(entry.type)));
	}
M
Merge  
Michael Kolupaev 已提交
1067 1068 1069

	if (do_fetch)
	{
M
Merge  
Michael Kolupaev 已提交
1070 1071
		String replica;

M
Merge  
Michael Kolupaev 已提交
1072 1073
		try
		{
M
Merge  
Michael Kolupaev 已提交
1074
			replica = findReplicaHavingPart(entry.new_part_name, true);
A
Merge  
Alexey Milovidov 已提交
1075 1076 1077 1078 1079 1080 1081 1082 1083 1084 1085 1086 1087 1088 1089 1090 1091

			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 已提交
1092 1093
			if (replica.empty())
			{
1094 1095 1096 1097 1098 1099 1100 1101 1102 1103
				/** Если кусок должен быть записан с кворумом, и кворум ещё недостигнут,
				  *  то (из-за того, что кусок невозможно прямо сейчас скачать),
				  *  кворумную запись следует считать безуспешной.
				  * TODO Сложный код, вынести отдельно.
				  */
				if (entry.quorum)
				{
					if (entry.type != LogEntry::GET_PART)
						throw Exception("Logical error: log entry with quorum but type is not GET_PART", ErrorCodes::LOGICAL_ERROR);

1104 1105 1106
					if (entry.block_id.empty())
						throw Exception("Logical error: log entry with quorum have empty block_id", ErrorCodes::LOGICAL_ERROR);

1107 1108 1109 1110 1111 1112 1113 1114
					LOG_DEBUG(log, "No active replica has part " << entry.new_part_name << " which needs to be written with quorum."
						" Will try to mark that quorum as failed.");

					/** Атомарно:
					  * - если реплики не стали активными;
					  * - если существует узел quorum с этим куском;
					  * - удалим узел quorum;
					  * - установим nonincrement_block_numbers, чтобы разрешить мерджи через номер потерянного куска;
1115 1116
					  * - добавим кусок в список quorum/failed_parts;
					  * - если кусок ещё не удалён из списка для дедупликации blocks/block_num, то удалим его;
1117
					  *
1118 1119 1120 1121 1122 1123 1124 1125 1126 1127 1128 1129 1130 1131 1132 1133 1134 1135 1136 1137 1138 1139 1140 1141 1142 1143 1144 1145 1146 1147 1148 1149 1150 1151 1152 1153 1154 1155 1156 1157 1158 1159 1160 1161 1162
					  * Если что-то изменится, то ничего не сделаем - попадём сюда снова в следующий раз.
					  */

					/** Соберём версии узлов host у реплик.
					  * Когда реплика становится активной, она в той же транзакции (с созданием is_active), меняет значение host.
					  * Это позволит проследить, что реплики не стали активными.
					  */

					Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");

					zkutil::Ops ops;

					for (size_t i = 0, size = replicas.size(); i < size; ++i)
					{
						Stat stat;
						String path = zookeeper_path + "/replicas/" + replicas[i] + "/host";
						zookeeper->get(path, &stat);
						ops.push_back(new zkutil::Op::Check(path, stat.version));
					}

					/// Проверяем, что пока мы собирали версии, не ожила реплика с нужным куском.
					replica = findReplicaHavingPart(entry.new_part_name, true);

					/// Также за это время могла быть создана совсем новая реплика. Но если на старых не появится куска, то на новой его тоже не может быть.

					if (replica.empty())
					{
						Stat quorum_stat;
						String quorum_path = zookeeper_path + "/quorum/status";
						String quorum_str = zookeeper->get(quorum_path, &quorum_stat);
						ReplicatedMergeTreeQuorumEntry quorum_entry;
						quorum_entry.fromString(quorum_str);

						if (quorum_entry.part_name == entry.new_part_name)
						{
							ops.push_back(new zkutil::Op::Remove(quorum_path, quorum_stat.version));

							const auto partition_str = entry.new_part_name.substr(0, 6);
							ActiveDataPartSet::Part part_info;
							ActiveDataPartSet::parsePartName(entry.new_part_name, part_info);

							if (part_info.left != part_info.right)
								throw Exception("Logical error: log entry with quorum for part covering more than one block number",
									ErrorCodes::LOGICAL_ERROR);

A
Alexey Milovidov 已提交
1163 1164
							auto acl = zookeeper->getDefaultACL();

1165 1166 1167
							ops.push_back(new zkutil::Op::Create(
								zookeeper_path + "/nonincrement_block_numbers/" + partition_str + "/block-" + padIndex(part_info.left),
								"",
A
Alexey Milovidov 已提交
1168
								acl,
1169 1170 1171 1172 1173
								zkutil::CreateMode::Persistent));

							ops.push_back(new zkutil::Op::Create(
								zookeeper_path + "/quorum/failed_parts/" + entry.new_part_name,
								"",
A
Alexey Milovidov 已提交
1174
								acl,
1175 1176
								zkutil::CreateMode::Persistent));

1177 1178 1179 1180 1181 1182 1183 1184
							/// Удаление из blocks.
							if (zookeeper->exists(zookeeper_path + "/blocks/" + entry.block_id))
							{
								ops.push_back(new zkutil::Op::Remove(zookeeper_path + "/blocks/" + entry.block_id + "/number", -1));
								ops.push_back(new zkutil::Op::Remove(zookeeper_path + "/blocks/" + entry.block_id + "/checksum", -1));
								ops.push_back(new zkutil::Op::Remove(zookeeper_path + "/blocks/" + entry.block_id, -1));
							}

1185 1186 1187 1188 1189 1190 1191 1192 1193 1194 1195 1196 1197 1198 1199 1200 1201 1202 1203 1204 1205 1206 1207 1208 1209 1210 1211 1212 1213 1214
							auto code = zookeeper->tryMulti(ops);

							if (code == ZOK)
							{
								LOG_DEBUG(log, "Marked quorum for part " << entry.new_part_name << " as failed.");
								return true;	/// NOTE Удаление из virtual_parts не делается, но оно нужно только для мерджей.
							}
							else if (code == ZBADVERSION || code == ZNONODE || code == ZNODEEXISTS)
							{
								LOG_DEBUG(log, "State was changed or isn't expected when trying to mark quorum for part "
									<< entry.new_part_name << " as failed.");
							}
							else
								throw zkutil::KeeperException(code);
						}
						else
						{
							LOG_WARNING(log, "No active replica has part " << entry.new_part_name
								<< ", but that part needs quorum and /quorum/status contains entry about another part " << quorum_entry.part_name
								<< ". It means that part was successfully written to " << entry.quorum << " replicas, but then all of them goes offline."
								<< " Or it is a bug.");
						}
					}
				}

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

1217
			fetchPart(entry.new_part_name, zookeeper_path + "/replicas/" + replica, false, entry.quorum);
M
Merge  
Michael Kolupaev 已提交
1218 1219 1220 1221 1222 1223 1224 1225 1226 1227 1228 1229

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

				/// Найдем действие по объединению этого куска с другими. Запомним других.
				StringSet parts_for_merge;
				LogEntries::iterator merge_entry;
				for (LogEntries::iterator it = queue.begin(); it != queue.end(); ++it)
				{
M
Merge  
Michael Kolupaev 已提交
1237
					if ((*it)->type == LogEntry::MERGE_PARTS)
M
Merge  
Michael Kolupaev 已提交
1238
					{
M
Merge  
Michael Kolupaev 已提交
1239 1240
						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 已提交
1241
						{
M
Merge  
Michael Kolupaev 已提交
1242
							parts_for_merge = StringSet((*it)->parts_to_merge.begin(), (*it)->parts_to_merge.end());
M
Merge  
Michael Kolupaev 已提交
1243 1244 1245 1246 1247 1248 1249 1250 1251 1252 1253 1254 1255 1256 1257 1258 1259
							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 已提交
1260 1261
						if (((*it0)->type == LogEntry::MERGE_PARTS || (*it0)->type == LogEntry::GET_PART)
							&& parts_for_merge.count((*it0)->new_part_name))
M
Merge  
Michael Kolupaev 已提交
1262 1263 1264 1265
						{
							queue.splice(queue.end(), queue, it0, it);
						}
					}
M
Merge  
Michael Kolupaev 已提交
1266 1267 1268 1269 1270 1271

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

1277 1278 1279
				/** Если ни у какой активной реплики нет куска, и в очереди нет слияний с его участием,
				  * проверим, есть ли у любой (активной или неактивной) реплики такой кусок или покрывающий его.
				  */
M
Merge  
Michael Kolupaev 已提交
1280 1281
				if (replica.empty())
					enqueuePartForCheck(entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
1282 1283 1284 1285 1286 1287 1288 1289 1290
			}
			catch (...)
			{
				tryLogCurrentException(__PRETTY_FUNCTION__);
			}

			throw;
		}
	}
M
Merge  
Michael Kolupaev 已提交
1291 1292

	return true;
M
Merge  
Michael Kolupaev 已提交
1293 1294
}

A
Merge  
Alexey Milovidov 已提交
1295

M
Merge  
Michael Kolupaev 已提交
1296
void StorageReplicatedMergeTree::executeDropRange(const StorageReplicatedMergeTree::LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
1297
{
A
Merge  
Alexey Milovidov 已提交
1298 1299
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
1300
	LOG_INFO(log, (entry.detach ? "Detaching" : "Removing") << " parts inside " << entry.new_part_name << ".");
M
Merge  
Michael Kolupaev 已提交
1301 1302 1303 1304 1305 1306 1307 1308 1309 1310 1311 1312 1313 1314 1315 1316 1317 1318 1319 1320 1321 1322 1323 1324 1325 1326 1327 1328 1329

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

M
Merge  
Michael Kolupaev 已提交
1334
	LOG_DEBUG(log, (entry.detach ? "Detaching" : "Removing") << " parts.");
M
Merge  
Michael Kolupaev 已提交
1335 1336 1337 1338 1339 1340 1341 1342 1343 1344 1345
	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 已提交
1346 1347 1348 1349
		/// Если кусок удалять не нужно, надежнее переместить директорию до изменений в ZooKeeper.
		if (entry.detach)
			data.renameAndDetachPart(part);

M
Merge  
Michael Kolupaev 已提交
1350
		zkutil::Ops ops;
1351
		removePartFromZooKeeper(part->name, ops);
M
Merge  
Michael Kolupaev 已提交
1352 1353
		zookeeper->multi(ops);

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

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

A
Merge  
Alexey Milovidov 已提交
1362

M
Merge  
Michael Kolupaev 已提交
1363 1364
bool StorageReplicatedMergeTree::executeAttachPart(const StorageReplicatedMergeTree::LogEntry & entry)
{
A
Merge  
Alexey Milovidov 已提交
1365 1366
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
1367 1368 1369 1370
	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 已提交
1371
	if (!Poco::File(data.getFullPath() + source_path).exists())
M
Merge  
Michael Kolupaev 已提交
1372
	{
M
Merge  
Michael Kolupaev 已提交
1373
		LOG_INFO(log, "No part at " << source_path << ". Will fetch it instead");
M
Merge  
Michael Kolupaev 已提交
1374 1375 1376 1377 1378 1379 1380 1381 1382 1383 1384 1385 1386 1387 1388 1389 1390 1391 1392 1393 1394 1395 1396 1397 1398 1399 1400 1401
		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 已提交
1402

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

M
Merge  
Michael Kolupaev 已提交
1406 1407 1408
	return true;
}

A
Merge  
Alexey Milovidov 已提交
1409

M
Merge  
Michael Kolupaev 已提交
1410 1411
void StorageReplicatedMergeTree::queueUpdatingThread()
{
1412 1413
	setThreadName("ReplMTQueueUpd");

M
Merge  
Michael Kolupaev 已提交
1414 1415
	while (!shutdown_called)
	{
M
Merge  
Michael Kolupaev 已提交
1416 1417
		try
		{
M
Merge  
Michael Kolupaev 已提交
1418 1419
			pullLogsToQueue(queue_updating_event);
			queue_updating_event->wait();
M
Merge  
Michael Kolupaev 已提交
1420
		}
1421
		catch (const zkutil::KeeperException & e)
M
Merge  
Michael Kolupaev 已提交
1422 1423
		{
			if (e.code == ZINVALIDSTATE)
1424
				restarting_thread->wakeup();
M
Merge  
Michael Kolupaev 已提交
1425 1426 1427 1428 1429

			tryLogCurrentException(__PRETTY_FUNCTION__);

			queue_updating_event->tryWait(ERROR_SLEEP_MS);
		}
M
Merge  
Michael Kolupaev 已提交
1430 1431 1432 1433
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);

M
Merge  
Michael Kolupaev 已提交
1434 1435
			queue_updating_event->tryWait(ERROR_SLEEP_MS);
		}
M
Merge  
Michael Kolupaev 已提交
1436
	}
M
Merge  
Michael Kolupaev 已提交
1437

1438
	LOG_DEBUG(log, "Queue updating thread finished");
M
Merge  
Michael Kolupaev 已提交
1439
}
M
Merge  
Michael Kolupaev 已提交
1440

A
Merge  
Alexey Milovidov 已提交
1441

M
Merge  
Michael Kolupaev 已提交
1442
bool StorageReplicatedMergeTree::queueTask(BackgroundProcessingPool::Context & pool_context)
M
Merge  
Michael Kolupaev 已提交
1443
{
M
Merge  
Michael Kolupaev 已提交
1444
	LogEntryPtr entry;
M
Merge  
Michael Kolupaev 已提交
1445

M
Merge  
Michael Kolupaev 已提交
1446 1447
	try
	{
1448
		std::lock_guard<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1449 1450
		bool empty = queue.empty();
		if (!empty)
M
Merge  
Michael Kolupaev 已提交
1451
		{
M
Merge  
Michael Kolupaev 已提交
1452
			for (LogEntries::iterator it = queue.begin(); it != queue.end(); ++it)
M
Merge  
Michael Kolupaev 已提交
1453
			{
1454 1455 1456 1457
				if ((*it)->currently_executing)
					continue;

				if (shouldExecuteLogEntry(**it, (*it)->postpone_reason))
M
Merge  
Michael Kolupaev 已提交
1458
				{
M
Merge  
Michael Kolupaev 已提交
1459
					entry = *it;
M
Merge  
Michael Kolupaev 已提交
1460
					entry->tagPartAsFuture(*this);
M
Merge  
Michael Kolupaev 已提交
1461
					queue.splice(queue.end(), queue, it);
M
Merge  
Michael Kolupaev 已提交
1462
					entry->currently_executing = true;
1463 1464
					++entry->num_tries;
					entry->last_attempt_time = time(0);
M
Merge  
Michael Kolupaev 已提交
1465
					break;
M
Merge  
Michael Kolupaev 已提交
1466
				}
1467 1468
				else
				{
1469 1470
					++(*it)->num_postponed;
					(*it)->last_postpone_time = time(0);
1471
				}
M
Merge  
Michael Kolupaev 已提交
1472 1473
			}
		}
M
Merge  
Michael Kolupaev 已提交
1474 1475 1476 1477 1478
	}
	catch (...)
	{
		tryLogCurrentException(__PRETTY_FUNCTION__);
	}
M
Merge  
Michael Kolupaev 已提交
1479

M
Merge  
Michael Kolupaev 已提交
1480
	if (!entry)
M
Merge  
Michael Kolupaev 已提交
1481
		return false;
M
Merge  
Michael Kolupaev 已提交
1482

1483
	bool was_exception = true;
M
Merge  
Michael Kolupaev 已提交
1484
	bool success = false;
A
Alexey Milovidov 已提交
1485
	std::exception_ptr saved_exception;
M
Merge  
Michael Kolupaev 已提交
1486

M
Merge  
Michael Kolupaev 已提交
1487 1488
	try
	{
1489
		try
M
Merge  
Michael Kolupaev 已提交
1490
		{
1491 1492 1493 1494
			if (executeLogEntry(*entry, pool_context))
			{
				auto zookeeper = getZooKeeper();
				auto code = zookeeper->tryRemove(replica_path + "/queue/" + entry->znode_name);
M
Merge  
Michael Kolupaev 已提交
1495

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

1500 1501 1502 1503 1504
				success = true;
			}
		}
		catch (...)
		{
A
Alexey Milovidov 已提交
1505
			saved_exception = std::current_exception();
1506
			throw;
M
Merge  
Michael Kolupaev 已提交
1507
		}
M
Merge  
Michael Kolupaev 已提交
1508

1509
		was_exception = false;
M
Merge  
Michael Kolupaev 已提交
1510
	}
1511
	catch (const Exception & e)
M
Merge  
Michael Kolupaev 已提交
1512 1513
	{
		if (e.code() == ErrorCodes::NO_REPLICA_HAS_PART)
1514
		{
M
Merge  
Michael Kolupaev 已提交
1515
			/// Если ни у кого нет нужного куска, наверно, просто не все реплики работают; не будем писать в лог с уровнем Error.
M
Merge  
Michael Kolupaev 已提交
1516
			LOG_INFO(log, e.displayText());
1517 1518 1519 1520 1521 1522
		}
		else if (e.code() == ErrorCodes::ABORTED)
		{
			/// Прерванный мердж - не ошибка.
			LOG_INFO(log, "Merge cancelled");
		}
M
Merge  
Michael Kolupaev 已提交
1523
		else
M
Merge  
Michael Kolupaev 已提交
1524
			tryLogCurrentException(__PRETTY_FUNCTION__);
M
Merge  
Michael Kolupaev 已提交
1525 1526 1527 1528 1529
	}
	catch (...)
	{
		tryLogCurrentException(__PRETTY_FUNCTION__);
	}
M
Merge  
Michael Kolupaev 已提交
1530

M
Merge  
Michael Kolupaev 已提交
1531 1532
	entry->future_part_tagger = nullptr;

1533
	std::lock_guard<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1534 1535

	entry->currently_executing = false;
1536
	entry->exception = saved_exception;
M
Merge  
Michael Kolupaev 已提交
1537 1538 1539
	entry->execution_complete.notify_all();

	if (success)
M
Merge  
Michael Kolupaev 已提交
1540
	{
M
Merge  
Michael Kolupaev 已提交
1541 1542 1543
		/// Удалим задание из очереди.
		/// Нельзя просто обратиться по заранее сохраненному итератору, потому что задание мог успеть удалить кто-то другой.
		for (LogEntries::iterator it = queue.end(); it != queue.begin();)
M
Merge  
Michael Kolupaev 已提交
1544
		{
M
Merge  
Michael Kolupaev 已提交
1545 1546 1547
			--it;
			if (*it == entry)
			{
M
Merge  
Michael Kolupaev 已提交
1548
				queue.erase(it);
M
Merge  
Michael Kolupaev 已提交
1549 1550
				break;
			}
M
Merge  
Michael Kolupaev 已提交
1551
		}
M
Merge  
Michael Kolupaev 已提交
1552
	}
M
Merge  
Michael Kolupaev 已提交
1553

M
Merge  
Michael Kolupaev 已提交
1554
	/// Если не было исключения, не нужно спать.
1555
	return !was_exception;
M
Merge  
Michael Kolupaev 已提交
1556 1557
}

A
Merge  
Alexey Milovidov 已提交
1558

M
Merge  
Michael Kolupaev 已提交
1559 1560
void StorageReplicatedMergeTree::mergeSelectingThread()
{
1561 1562
	setThreadName("ReplMTMergeSel");

M
Merge  
Michael Kolupaev 已提交
1563
	bool need_pull = true;
M
Merge  
Michael Kolupaev 已提交
1564

A
Merge  
Alexey Milovidov 已提交
1565 1566 1567 1568 1569 1570 1571
	/** Может много времени тратиться на определение, можно ли мерджить два рядом стоящих куска.
	  * Два рядом стоящих куска можно мерджить, если все номера блоков между их номерами не используются ("заброшены", abandoned).
	  * Это значит, что между этими кусками не может быть вставлен другой кусок.
	  *
	  * Но если номера соседних блоков отличаются достаточно сильно (обычно, если между ними много "заброшенных" блоков),
	  *  то делается слишком много чтений из ZooKeeper, чтобы узнать, можно ли их мерджить.
	  *
1572 1573
	  * Воспользуемся утверждением, что если пару кусков было можно мерджить, и их мердж ещё не запланирован,
	  *  то и сейчас их можно мерджить, и будем запоминать это состояние, чтобы не делать много раз одинаковые запросы в ZooKeeper.
A
Merge  
Alexey Milovidov 已提交
1574 1575 1576 1577 1578 1579 1580 1581
	  *
	  * 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
	{
1582 1583 1584 1585 1586
		/// Если какой-то из кусков уже собираются слить в больший, не соглашаемся его сливать.
		if (virtual_parts.getContainingPart(left->name) != left->name ||
			virtual_parts.getContainingPart(right->name) != right->name)
			return false;

A
Merge  
Alexey Milovidov 已提交
1587 1588 1589 1590
		auto key = std::make_pair(left->name, right->name);
		if (memoized_parts_that_could_be_merged.count(key))
			return true;

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

1594 1595 1596 1597 1598 1599 1600 1601 1602 1603 1604 1605 1606 1607 1608 1609 1610 1611
		/// Нельзя сливать куски, среди которых находится кусок, для которого неудовлетворён кворум.
		/// Замечание: теоретически, это можно было бы разрешить. Но это сделает логику более сложной.
		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;
		}

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

1619 1620 1621 1622 1623 1624 1625
			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 已提交
1626 1627
	};

M
Merge  
Michael Kolupaev 已提交
1628 1629
	while (!shutdown_called && is_leader_node)
	{
M
Michael Kolupaev 已提交
1630
		bool success = false;
M
Merge  
Michael Kolupaev 已提交
1631

M
Michael Kolupaev 已提交
1632
		try
M
Merge  
Michael Kolupaev 已提交
1633
		{
1634
			std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
M
Merge  
Michael Kolupaev 已提交
1635

M
Merge  
Michael Kolupaev 已提交
1636 1637 1638 1639 1640 1641 1642 1643
			if (need_pull)
			{
				/// Нужно загрузить новую запись в очередь перед тем, как выбирать куски для слияния.
				///  (чтобы кусок добавился в virtual_parts).
				pullLogsToQueue();
				need_pull = false;
			}

1644 1645 1646 1647 1648 1649 1650
			/** Сколько в очереди или в фоновом потоке мерджей крупных кусков.
			  * Если их больше половины от размера пула потоков для мерджа, то можно мерджить только мелкие куски.
			  */
			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 已提交
1651
			size_t merges_queued = 0;
1652
			size_t big_merges_queued = 0;
M
Merge  
Michael Kolupaev 已提交
1653

1654
			if (big_merges_current < max_number_of_big_merges)
M
Michael Kolupaev 已提交
1655
			{
1656
				std::lock_guard<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1657

M
Michael Kolupaev 已提交
1658
				for (const auto & entry : queue)
M
Merge  
Michael Kolupaev 已提交
1659
				{
M
Merge  
Michael Kolupaev 已提交
1660
					if (entry->type == LogEntry::MERGE_PARTS)
M
Merge  
Michael Kolupaev 已提交
1661
					{
M
Michael Kolupaev 已提交
1662
						++merges_queued;
M
Merge  
Michael Kolupaev 已提交
1663

1664
						if (big_merges_current + big_merges_queued < max_number_of_big_merges)
M
Merge  
Michael Kolupaev 已提交
1665
						{
M
Merge  
Michael Kolupaev 已提交
1666
							for (const String & name : entry->parts_to_merge)
M
Merge  
Michael Kolupaev 已提交
1667
							{
M
Merge  
Michael Kolupaev 已提交
1668
								MergeTreeData::DataPartPtr part = data.getActiveContainingPart(name);
M
Merge  
Michael Kolupaev 已提交
1669 1670
								if (!part || part->name != name)
									continue;
1671

M
Merge  
Michael Kolupaev 已提交
1672
								if (part->size_in_bytes > data.settings.max_bytes_to_merge_parts_small)
M
Merge  
Michael Kolupaev 已提交
1673
								{
1674
									++big_merges_queued;
M
Merge  
Michael Kolupaev 已提交
1675 1676 1677 1678 1679 1680
									break;
								}
							}
						}
					}
				}
M
Michael Kolupaev 已提交
1681
			}
M
Merge  
Michael Kolupaev 已提交
1682

1683 1684
			bool only_small = big_merges_current + big_merges_queued >= max_number_of_big_merges;

1685 1686 1687 1688 1689 1690
			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." : "."));
1691

M
Merge  
Michael Kolupaev 已提交
1692
			do
M
Michael Kolupaev 已提交
1693
			{
A
Merge  
Alexey Milovidov 已提交
1694 1695
				auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
1696
				if (merges_queued >= data.settings.max_replicated_merges_in_queue)
1697
				{
1698 1699 1700
					LOG_TRACE(log, "Number of queued merges (" << merges_queued
						<< ") is greater than max_replicated_merges_in_queue ("
						<< data.settings.max_replicated_merges_in_queue << "), so won't select new parts to merge.");
M
Merge  
Michael Kolupaev 已提交
1701
					break;
1702
				}
M
Merge  
Michael Kolupaev 已提交
1703

M
Merge  
Michael Kolupaev 已提交
1704
				MergeTreeData::DataPartsVector parts;
M
Merge  
Michael Kolupaev 已提交
1705

M
Merge  
Michael Kolupaev 已提交
1706 1707
				String merged_name;

A
Merge  
Alexey Milovidov 已提交
1708 1709 1710 1711
				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))
1712
				{
M
Merge  
Michael Kolupaev 已提交
1713
					break;
1714
				}
M
Merge  
Michael Kolupaev 已提交
1715

M
Merge  
Michael Kolupaev 已提交
1716 1717 1718 1719 1720 1721 1722
				bool all_in_zk = true;
				for (const auto & part : parts)
				{
					/// Если о каком-то из кусков нет информации в ZK, не будем сливать.
					if (!zookeeper->exists(replica_path + "/parts/" + part->name))
					{
						all_in_zk = false;
1723 1724 1725 1726 1727 1728 1729 1730 1731

						if (part->modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER < time(0))
						{
							LOG_WARNING(log, "Part " << part->name << " (that was selected for merge)"
								<< " with age " << (time(0) - part->modification_time)
								<< " seconds exists locally but not in ZooKeeper."
								<< " Won't do merge with that part and will check it.");
							enqueuePartForCheck(part->name);
						}
M
Merge  
Michael Kolupaev 已提交
1732 1733 1734 1735 1736
					}
				}
				if (!all_in_zk)
					break;

M
Merge  
Michael Kolupaev 已提交
1737 1738 1739 1740
				LogEntry entry;
				entry.type = LogEntry::MERGE_PARTS;
				entry.source_replica = replica_name;
				entry.new_part_name = merged_name;
1741
				entry.create_time = time(0);
M
Merge  
Michael Kolupaev 已提交
1742

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

M
Merge  
Michael Kolupaev 已提交
1746
				need_pull = true;
M
Merge  
Michael Kolupaev 已提交
1747

M
Merge  
Michael Kolupaev 已提交
1748
				zookeeper->create(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential);
M
Merge  
Michael Kolupaev 已提交
1749 1750 1751

				String month_name = parts[0]->name.substr(0, 6);
				for (size_t i = 0; i + 1 < parts.size(); ++i)
M
Merge  
Michael Kolupaev 已提交
1752
				{
M
Merge  
Michael Kolupaev 已提交
1753
					/// Уберем больше не нужные отметки о несуществующих блоках.
A
Merge  
Alexey Milovidov 已提交
1754
					for (Int64 number = std::max(RESERVED_BLOCK_NUMBERS, parts[i]->right + 1); number <= parts[i + 1]->left - 1; ++number)
M
Merge  
Michael Kolupaev 已提交
1755
					{
M
Merge  
Michael Kolupaev 已提交
1756 1757
						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 已提交
1758
					}
M
Merge  
Michael Kolupaev 已提交
1759
				}
M
Merge  
Michael Kolupaev 已提交
1760 1761

				success = true;
M
Merge  
Michael Kolupaev 已提交
1762
			}
1763
			while (false);
M
Merge  
Michael Kolupaev 已提交
1764 1765 1766 1767 1768 1769
		}
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);
		}

M
Merge  
Michael Kolupaev 已提交
1770
		if (shutdown_called || !is_leader_node)
M
Merge  
Michael Kolupaev 已提交
1771 1772
			break;

M
Merge  
Michael Kolupaev 已提交
1773
		if (!success)
M
Merge  
Michael Kolupaev 已提交
1774
			merge_selecting_event.tryWait(MERGE_SELECTING_SLEEP_MS);
M
Merge  
Michael Kolupaev 已提交
1775
	}
M
Merge  
Michael Kolupaev 已提交
1776

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

M
Merge  
Michael Kolupaev 已提交
1780

M
Merge  
Michael Kolupaev 已提交
1781 1782
void StorageReplicatedMergeTree::alterThread()
{
1783 1784
	setThreadName("ReplMTAlter");

M
Merge  
Michael Kolupaev 已提交
1785 1786 1787 1788 1789 1790
	bool force_recheck_parts = true;

	while (!shutdown_called)
	{
		try
		{
1791 1792 1793 1794 1795 1796 1797 1798 1799 1800 1801 1802 1803 1804 1805 1806 1807 1808 1809 1810 1811 1812 1813
			/** Имеем описание столбцов в 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 已提交
1814 1815
			auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
1816
			zkutil::Stat stat;
1817
			const String columns_str = zookeeper->get(zookeeper_path + "/columns", &stat, alter_thread_event);
1818
			auto columns_desc = ColumnsDescription<true>::parse(columns_str);
1819 1820 1821 1822 1823

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

1825
			bool changed_version = (stat.version != columns_version);
M
Merge  
Michael Kolupaev 已提交
1826 1827

			{
1828 1829 1830
				/// Если потребуется блокировать структуру таблицы, то приостановим мерджи.
				std::unique_ptr<MergeTreeMergeBlocker> merge_blocker;
				std::unique_ptr<MergeTreeMergeBlocker> unreplicated_merge_blocker;
1831

1832
				if (changed_version || force_recheck_parts)
M
Merge  
Michael Kolupaev 已提交
1833
				{
1834 1835 1836 1837
					merge_blocker = std::make_unique<MergeTreeMergeBlocker>(merger);
					if (unreplicated_merger)
						unreplicated_merge_blocker = std::make_unique<MergeTreeMergeBlocker>(*unreplicated_merger);
				}
1838

1839
				MergeTreeData::DataParts parts;
1840

1841 1842 1843 1844
				/// Если описание столбцов изменилось, обновим структуру таблицы локально.
				if (changed_version)
				{
					LOG_INFO(log, "Changed version of 'columns' node in ZooKeeper. Waiting for structure write lock.");
1845

1846
					auto table_lock = lockStructureForAlter();
1847

1848 1849 1850 1851
					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;
					const auto column_defaults_changed = column_defaults != data.column_defaults;
1852

1853 1854
					if (columns_changed || materialized_columns_changed || alias_columns_changed ||
						column_defaults_changed)
1855
					{
1856
						LOG_INFO(log, "Columns list changed in ZooKeeper. Applying changes locally.");
1857

1858 1859
						InterpreterAlterQuery::updateMetadata(database_name, table_name, columns,
							materialized_columns, alias_columns, column_defaults, context);
1860

1861 1862 1863
						if (columns_changed)
						{
							data.setColumnsList(columns);
1864

1865 1866 1867
							if (unreplicated_data)
								unreplicated_data->setColumnsList(columns);
						}
1868

1869 1870 1871 1872 1873
						if (materialized_columns_changed)
						{
							this->materialized_columns = materialized_columns;
							data.materialized_columns = std::move(materialized_columns);
						}
M
Merge  
Michael Kolupaev 已提交
1874

1875 1876 1877 1878 1879
						if (alias_columns_changed)
						{
							this->alias_columns = alias_columns;
							data.alias_columns = std::move(alias_columns);
						}
1880

1881 1882 1883 1884 1885
						if (column_defaults_changed)
						{
							this->column_defaults = column_defaults;
							data.column_defaults = std::move(column_defaults);
						}
M
Merge  
Michael Kolupaev 已提交
1886

1887 1888 1889 1890 1891 1892
						LOG_INFO(log, "Applied changes to table.");
					}
					else
					{
						LOG_INFO(log, "Columns version changed in ZooKeeper, but data wasn't changed. It's like cyclic ALTERs.");
					}
M
Merge  
Michael Kolupaev 已提交
1893

1894
					/// Нужно получить список кусков под блокировкой таблицы, чтобы избежать race condition с мерджем.
M
Merge  
Michael Kolupaev 已提交
1895 1896
					parts = data.getDataParts();

1897 1898
					columns_version = stat.version;
				}
1899

1900 1901
				/// Обновим куски.
				if (changed_version || force_recheck_parts)
M
Merge  
Michael Kolupaev 已提交
1902
				{
1903
					auto table_lock = lockStructure(false);
M
Merge  
Michael Kolupaev 已提交
1904

1905 1906
					if (changed_version)
						LOG_INFO(log, "ALTER-ing parts");
M
Merge  
Michael Kolupaev 已提交
1907

1908
					int changed_parts = 0;
M
Merge  
Michael Kolupaev 已提交
1909

1910 1911
					if (!changed_version)
						parts = data.getDataParts();
M
Merge  
Michael Kolupaev 已提交
1912

1913
					const auto columns_plus_materialized = data.getColumnsList();
M
Merge  
Michael Kolupaev 已提交
1914 1915 1916

					for (const MergeTreeData::DataPartPtr & part : parts)
					{
1917 1918 1919 1920
						/// Обновим кусок и запишем результат во временные файлы.
						/// TODO: Можно пропускать проверку на слишком большие изменения, если в ZooKeeper есть, например,
						///  нода /flags/force_alter.
						auto transaction = data.alterDataPart(part, columns_plus_materialized);
M
Merge  
Michael Kolupaev 已提交
1921 1922 1923 1924 1925 1926

						if (!transaction)
							continue;

						++changed_parts;

1927 1928 1929 1930 1931 1932 1933 1934 1935
						/// Обновим метаданные куска в ZooKeeper.
						zkutil::Ops ops;
						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));
						zookeeper->multi(ops);

						/// Применим изменения файлов.
M
Merge  
Michael Kolupaev 已提交
1936 1937
						transaction->commit();
					}
M
Merge  
Michael Kolupaev 已提交
1938

1939 1940 1941 1942
					/// То же самое для нереплицируемых данных.
					if (unreplicated_data)
					{
						parts = unreplicated_data->getDataParts();
M
Merge  
Michael Kolupaev 已提交
1943

1944 1945 1946 1947 1948 1949 1950 1951 1952 1953 1954 1955 1956 1957 1958 1959 1960 1961 1962 1963 1964 1965 1966 1967 1968
						for (const MergeTreeData::DataPartPtr & part : parts)
						{
							auto transaction = unreplicated_data->alterDataPart(part, columns_plus_materialized);

							if (!transaction)
								continue;

							++changed_parts;

							transaction->commit();
						}
					}

					/// Список столбцов для конкретной реплики.
					zookeeper->set(replica_path + "/columns", columns_str);

					if (changed_version)
					{
						if (changed_parts != 0)
							LOG_INFO(log, "ALTER-ed " << changed_parts << " parts");
						else
							LOG_INFO(log, "No parts ALTER-ed");
					}

					force_recheck_parts = false;
1969 1970
				}

1971
				/// Важно, что уничтожается parts и merge_blocker перед wait-ом.
M
Merge  
Michael Kolupaev 已提交
1972 1973 1974 1975 1976 1977 1978 1979 1980 1981 1982 1983 1984 1985
			}

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

			force_recheck_parts = true;

			alter_thread_event->tryWait(ERROR_SLEEP_MS);
		}
	}

1986
	LOG_DEBUG(log, "Alter thread finished");
M
Merge  
Michael Kolupaev 已提交
1987 1988
}

A
Merge  
Alexey Milovidov 已提交
1989

1990 1991 1992 1993 1994 1995 1996 1997 1998 1999
void StorageReplicatedMergeTree::removePartFromZooKeeper(const String & part_name, zkutil::Ops & ops)
{
	String part_path = replica_path + "/parts/" + part_name;

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


M
Merge  
Michael Kolupaev 已提交
2000 2001
void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_name)
{
A
Merge  
Alexey Milovidov 已提交
2002 2003
	auto zookeeper = getZooKeeper();

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

M
Merge  
Michael Kolupaev 已提交
2006 2007
	LogEntryPtr log_entry = new LogEntry;
	log_entry->type = LogEntry::GET_PART;
2008
	log_entry->create_time = tryGetPartCreateTime(zookeeper, replica_path, part_name);
M
Merge  
Michael Kolupaev 已提交
2009 2010
	log_entry->source_replica = "";
	log_entry->new_part_name = part_name;
M
Merge  
Michael Kolupaev 已提交
2011 2012 2013

	zkutil::Ops ops;
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
2014
		replica_path + "/queue/queue-", log_entry->toString(), zookeeper->getDefaultACL(),
M
Merge  
Michael Kolupaev 已提交
2015
		zkutil::CreateMode::PersistentSequential));
2016 2017 2018

	removePartFromZooKeeper(part_name, ops);

M
Merge  
Michael Kolupaev 已提交
2019 2020 2021
	auto results = zookeeper->multi(ops);

	{
2022
		std::lock_guard<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
2023 2024

		String path_created = dynamic_cast<zkutil::Op::Create &>(ops[0]).getPathCreated();
M
Merge  
Michael Kolupaev 已提交
2025 2026
		log_entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
		log_entry->addResultToVirtualParts(*this);
M
Merge  
Michael Kolupaev 已提交
2027 2028 2029 2030
		queue.push_back(log_entry);
	}
}

A
Merge  
Alexey Milovidov 已提交
2031

M
Merge  
Michael Kolupaev 已提交
2032 2033
void StorageReplicatedMergeTree::enqueuePartForCheck(const String & name)
{
2034
	std::lock_guard<std::mutex> lock(parts_to_check_mutex);
M
Merge  
Michael Kolupaev 已提交
2035 2036 2037 2038 2039 2040 2041 2042

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

2043 2044

void StorageReplicatedMergeTree::searchForMissingPart(const String & part_name)
M
Merge  
Michael Kolupaev 已提交
2045
{
2046 2047 2048 2049 2050
	auto zookeeper = getZooKeeper();
	String part_path = replica_path + "/parts/" + part_name;

	/// Если кусок есть в ZooKeeper, удалим его оттуда и добавим в очередь задание скачать его.
	if (zookeeper->exists(part_path))
M
Merge  
Michael Kolupaev 已提交
2051
	{
2052 2053 2054
		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 已提交
2055

2056 2057 2058 2059 2060 2061 2062 2063 2064 2065 2066 2067 2068 2069 2070 2071 2072 2073 2074 2075 2076 2077 2078 2079 2080 2081 2082 2083 2084 2085 2086 2087
		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 已提交
2088
			{
2089 2090 2091
				found = true;
				LOG_WARNING(log, "Checker: Found part " << part_on_replica << " on " << replica);
				break;
M
Merge  
Michael Kolupaev 已提交
2092
			}
2093 2094

			if (ActiveDataPartSet::contains(part_name, part_on_replica))
M
Merge  
Michael Kolupaev 已提交
2095
			{
2096 2097 2098
				ActiveDataPartSet::Part part_on_replica_info;
				ActiveDataPartSet::parsePartName(part_on_replica, part_on_replica_info);

2099
				for (auto block_num = part_on_replica_info.left; block_num <= part_on_replica_info.right; ++block_num)
A
Alexey Milovidov 已提交
2100
					found_blocks.at(block_num - part_info.left) = 1;
M
Merge  
Michael Kolupaev 已提交
2101
			}
2102 2103 2104 2105
		}
		if (found)
			break;
	}
M
Merge  
Michael Kolupaev 已提交
2106

2107 2108 2109 2110 2111
	if (found)
	{
		/// На какой-то живой или мёртвой реплике есть нужный кусок или покрывающий его.
		return;
	}
M
Merge  
Michael Kolupaev 已提交
2112

2113 2114 2115
	size_t num_found_blocks = 0;
	for (auto found_block : found_blocks)
		num_found_blocks += (found_block == 1);
M
Merge  
Michael Kolupaev 已提交
2116

2117 2118 2119 2120 2121 2122
	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;
	}
2123

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

2127 2128 2129
	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 已提交
2130

2131
	ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
2132

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

2138 2139 2140 2141 2142 2143 2144 2145 2146 2147 2148 2149
		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;
		}
	}
2150

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

2158 2159 2160 2161 2162 2163 2164 2165 2166 2167 2168 2169 2170
	/** Такая ситуация возможна, если на всех репликах, где был кусок, он испортился.
		* Например, у реплики, которая только что его записала, отключили питание, и данные не записались из кеша на диск.
		*/
	LOG_ERROR(log, "Checker: Part " << part_name << " is lost forever.");
	ProfileEvents::increment(ProfileEvents::ReplicatedDataLoss);

	/** Нужно добавить отсутствующий кусок в block_numbers, чтобы он не мешал слияниям.
		* Вот только в сам block_numbers мы его добавить не можем - если так сделать,
		*  ZooKeeper зачем-то пропустит один номер для автоинкремента,
		*  и в номерах блоков все равно останется дырка.
		* Специально из-за этого приходится отдельно иметь nonincrement_block_numbers.
		*
		* Кстати, если мы здесь сдохнем, то слияния не будут делаться сквозь эти отсутствующие куски.
2171 2172 2173 2174 2175
		*
		* А ещё, не будем добавлять, если:
		* - потребовалось бы создать слишком много (больше 1000) узлов;
		* - кусок является первым в партиции или был при-ATTACH-ен.
		* NOTE Возможно, добавить также условие, если запись в очереди очень старая.
2176
		*/
2177 2178 2179 2180 2181 2182 2183 2184 2185 2186 2187 2188 2189

	if (part_length_in_blocks > 1000)
	{
		LOG_ERROR(log, "Won't add nonincrement_block_numbers because part spans too much blocks (" << part_length_in_blocks << ")");
		return;
	}

	if (part_info.left <= RESERVED_BLOCK_NUMBERS)
	{
		LOG_ERROR(log, "Won't add nonincrement_block_numbers because part is one of first in partition");
		return;
	}

2190 2191 2192 2193 2194 2195 2196 2197 2198 2199
	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 已提交
2200 2201


2202 2203 2204 2205
void StorageReplicatedMergeTree::checkPart(const String & part_name)
{
	LOG_WARNING(log, "Checker: Checking part " << part_name);
	ProfileEvents::increment(ProfileEvents::ReplicatedPartChecks);
M
Merge  
Michael Kolupaev 已提交
2206

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

2209 2210 2211 2212 2213 2214 2215 2216 2217 2218
	/// Этого или покрывающего куска у нас нет.
	if (!part)
	{
		searchForMissingPart(part_name);
	}
	/// У нас есть этот кусок, и он активен. Будем проверять, нужен ли нам этот кусок и правильные ли у него данные.
	else if (part->name == part_name)
	{
		auto zookeeper = getZooKeeper();
		auto table_lock = lockStructure(false);
M
Merge  
Michael Kolupaev 已提交
2219

2220 2221 2222 2223 2224 2225 2226 2227 2228 2229 2230 2231 2232 2233 2234 2235 2236 2237 2238 2239 2240
		/// Если кусок есть в 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(
2241
					data.getFullPath() + part_name, settings, data.primary_key_data_types);
2242 2243

				LOG_INFO(log, "Checker: Part " << part_name << " looks good.");
M
Merge  
Michael Kolupaev 已提交
2244
			}
2245
			catch (...)
M
Merge  
Michael Kolupaev 已提交
2246
			{
2247
				tryLogCurrentException(__PRETTY_FUNCTION__);
M
Merge  
Michael Kolupaev 已提交
2248

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

2252
				removePartAndEnqueueFetch(part_name);
M
Merge  
Michael Kolupaev 已提交
2253

2254 2255 2256 2257
				/// Удалим кусок локально.
				data.renameAndDetachPart(part, "broken_");
			}
		}
2258
		else if (part->modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER < time(0))
2259 2260 2261 2262 2263 2264
		{
			/// Если куска нет в ZooKeeper, удалим его локально.
			/// Возможно, кусок кто-то только что записал, и еще не успел добавить в ZK.
			/// Поэтому удаляем только если кусок старый (не очень надежно).
			ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);

2265
			LOG_ERROR(log, "Checker: Unexpected part " << part_name << " in filesystem. Removing.");
2266 2267
			data.renameAndDetachPart(part, "unexpected_");
		}
2268 2269 2270 2271 2272 2273
		else
		{
			LOG_TRACE(log, "Checker: Young part " << part_name
				<< " with age " << (time(0) - part->modification_time)
				<< " seconds hasn't been added to ZooKeeper yet. It's ok.");
		}
2274 2275 2276 2277 2278 2279 2280 2281
	}
	else
	{
		/// Если у нас есть покрывающий кусок, игнорируем все проблемы с этим куском.
		/// В худшем случае в лог еще old_parts_lifetime секунд будут валиться ошибки, пока кусок не удалится как старый.
		LOG_WARNING(log, "Checker: We have part " << part->name << " covering part " << part_name);
	}
}
M
Merge  
Michael Kolupaev 已提交
2282 2283


2284 2285
void StorageReplicatedMergeTree::partCheckThread()
{
2286 2287
	setThreadName("ReplMTPartCheck");

2288 2289 2290 2291 2292 2293 2294 2295 2296 2297 2298 2299 2300 2301
	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 已提交
2302
					}
M
Merge  
Michael Kolupaev 已提交
2303
				}
2304
				else
M
Merge  
Michael Kolupaev 已提交
2305
				{
2306
					part_name = parts_to_check_queue.front();
M
Merge  
Michael Kolupaev 已提交
2307 2308
				}
			}
2309 2310

			if (part_name.empty())
M
Merge  
Michael Kolupaev 已提交
2311
			{
2312 2313
				parts_to_check_event.wait();
				continue;
M
Merge  
Michael Kolupaev 已提交
2314 2315
			}

2316 2317 2318
			checkPart(part_name);

			/// Удалим кусок из очереди проверок.
M
Merge  
Michael Kolupaev 已提交
2319
			{
2320
				std::lock_guard<std::mutex> lock(parts_to_check_mutex);
M
Merge  
Michael Kolupaev 已提交
2321 2322
				if (parts_to_check_queue.empty() || parts_to_check_queue.front() != part_name)
				{
2323
					LOG_ERROR(log, "Checker: Someone changed parts_to_check_queue.front(). This is a bug.");
M
Merge  
Michael Kolupaev 已提交
2324 2325 2326 2327 2328 2329 2330 2331 2332 2333 2334 2335 2336 2337 2338 2339 2340
				}
				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 已提交
2341 2342
void StorageReplicatedMergeTree::becomeLeader()
{
M
Merge  
Michael Kolupaev 已提交
2343
	LOG_INFO(log, "Became leader");
M
Merge  
Michael Kolupaev 已提交
2344 2345 2346 2347
	is_leader_node = true;
	merge_selecting_thread = std::thread(&StorageReplicatedMergeTree::mergeSelectingThread, this);
}

A
Merge  
Alexey Milovidov 已提交
2348

M
Merge  
Michael Kolupaev 已提交
2349
String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_name, bool active)
M
Merge  
Michael Kolupaev 已提交
2350
{
A
Merge  
Alexey Milovidov 已提交
2351
	auto zookeeper = getZooKeeper();
M
Merge  
Michael Kolupaev 已提交
2352
	Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
M
Merge  
Michael Kolupaev 已提交
2353 2354 2355 2356 2357 2358

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

	for (const String & replica : replicas)
	{
M
Merge  
Michael Kolupaev 已提交
2359 2360
		if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name) &&
			(!active || zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")))
M
Merge  
Michael Kolupaev 已提交
2361
			return replica;
2362 2363

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

M
Merge  
Michael Kolupaev 已提交
2366
	return "";
M
Merge  
Michael Kolupaev 已提交
2367 2368
}

A
Merge  
Alexey Milovidov 已提交
2369

2370 2371
/** Если для куска отслеживается кворум, то обновить информацию о нём в ZK.
  */
2372
void StorageReplicatedMergeTree::updateQuorum(const String & part_name)
2373
{
2374
	auto zookeeper = getZooKeeper();
2375

2376
	/// Информация, на какие реплики был добавлен кусок, если кворум ещё не достигнут.
2377
	const String quorum_status_path = zookeeper_path + "/quorum/status";
2378 2379 2380
	/// Имя предыдущего куска, для которого был достигнут кворум.
	const String quorum_last_part_path = zookeeper_path + "/quorum/last_part";

2381 2382 2383 2384 2385 2386 2387 2388 2389 2390 2391 2392 2393 2394 2395 2396 2397 2398 2399
	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;
		}

		quorum_entry.replicas.insert(replica_name);

		if (quorum_entry.replicas.size() >= quorum_entry.required_number_of_replicas)
		{
2400 2401 2402 2403 2404 2405
			/// Кворум достигнут. Удаляем узел, а также обновляем информацию о последнем куске, который был успешно записан с кворумом.

			zkutil::Ops ops;
			ops.push_back(new zkutil::Op::Remove(quorum_status_path, stat.version));
			ops.push_back(new zkutil::Op::SetData(quorum_last_part_path, part_name, -1));
			auto code = zookeeper->tryMulti(ops);
2406

2407 2408 2409 2410 2411
			if (code == ZOK)
			{
				break;
			}
			else if (code == ZNONODE)
2412 2413 2414 2415 2416 2417 2418 2419 2420 2421 2422 2423 2424 2425 2426 2427 2428
			{
				/// Кворум уже был достигнут.
				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);

2429 2430 2431 2432 2433
			if (code == ZOK)
			{
				break;
			}
			else if (code == ZNONODE)
2434 2435 2436 2437 2438 2439 2440 2441 2442 2443 2444 2445 2446 2447 2448 2449 2450
			{
				/// Кворум уже был достигнут.
				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 已提交
2451
{
A
Merge  
Alexey Milovidov 已提交
2452 2453
	auto zookeeper = getZooKeeper();

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

2456 2457 2458
	TableStructureReadLockPtr table_lock;
	if (!to_detached)
		table_lock = lockStructure(true);
M
Merge  
Michael Kolupaev 已提交
2459

2460
	ReplicatedMergeTreeAddress address(zookeeper->get(replica_path + "/host"));
M
Merge  
Michael Kolupaev 已提交
2461

2462
	MergeTreeData::MutableDataPartPtr part = fetcher.fetchPart(part_name, replica_path, address.host, address.replication_port, to_detached);
M
Merge  
Michael Kolupaev 已提交
2463

2464 2465 2466 2467
	if (!to_detached)
	{
		zkutil::Ops ops;
		checkPartAndAddToZooKeeper(part, ops, part_name);
M
Merge  
Michael Kolupaev 已提交
2468

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

2472 2473
		zookeeper->multi(ops);
		transaction.commit();
2474 2475

		/** Если для этого куска отслеживается кворум, то надо его обновить.
2476
		  * Если не успеем, в случае потери сессии, при перезапуске сервера - см. метод ReplicatedMergeTreeRestartingThread::updateQuorumIfWeHavePart.
2477
		  */
2478 2479
		if (quorum)
			updateQuorum(part_name);
2480

2481
		merge_selecting_event.set();
M
Merge  
Michael Kolupaev 已提交
2482

2483 2484 2485 2486 2487 2488 2489
		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 已提交
2490
	{
2491
		Poco::File(data.getFullPath() + "detached/tmp_" + part_name).renameTo(data.getFullPath() + "detached/" + part_name);
M
Michael Kolupaev 已提交
2492 2493
	}

M
Merge  
Michael Kolupaev 已提交
2494 2495
	ProfileEvents::increment(ProfileEvents::ReplicatedPartFetches);

2496
	LOG_DEBUG(log, "Fetched part " << part_name << " from " << replica_path << (to_detached ? " (to 'detached' directory)" : ""));
M
Merge  
Michael Kolupaev 已提交
2497
}
M
Merge  
Michael Kolupaev 已提交
2498

A
Merge  
Alexey Milovidov 已提交
2499

M
Merge  
Michael Kolupaev 已提交
2500 2501
void StorageReplicatedMergeTree::shutdown()
{
2502
	if (restarting_thread)
M
Merge  
Michael Kolupaev 已提交
2503
	{
2504 2505
		restarting_thread->stop();
		restarting_thread.reset();
M
Merge  
Michael Kolupaev 已提交
2506
	}
M
Merge  
Michael Kolupaev 已提交
2507 2508

	endpoint_holder = nullptr;
M
Merge  
Michael Kolupaev 已提交
2509 2510 2511
}


M
Merge  
Michael Kolupaev 已提交
2512 2513 2514 2515 2516 2517 2518 2519
StorageReplicatedMergeTree::~StorageReplicatedMergeTree()
{
	try
	{
		shutdown();
	}
	catch(...)
	{
2520
		tryLogCurrentException(__PRETTY_FUNCTION__);
M
Merge  
Michael Kolupaev 已提交
2521 2522 2523
	}
}

A
Merge  
Alexey Milovidov 已提交
2524

M
Merge  
Michael Kolupaev 已提交
2525
BlockInputStreams StorageReplicatedMergeTree::read(
2526 2527 2528 2529 2530 2531 2532
	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 已提交
2533
{
2534 2535 2536 2537 2538 2539 2540 2541
	/** У таблицы может быть два вида данных:
	  * - реплицируемые данные;
	  * - старые, нереплицируемые данные - они лежат отдельно и их целостность никак не контролируется.
	  * А ещё движок таблицы предоставляет возможность использовать "виртуальные столбцы".
	  * Один из них - _replicated позволяет определить, из какой части прочитаны данные,
	  *  или, при использовании в WHERE - выбрать данные только из одной части.
	  */

M
Merge  
Michael Kolupaev 已提交
2542 2543 2544 2545 2546 2547 2548 2549
	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);

2550
	auto & select = typeid_cast<const ASTSelectQuery &>(*query);
A
Merge  
Andrey Mironov 已提交
2551 2552 2553 2554

	/// 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)
2555
			MergeTreeWhereOptimizer{query, context, data, real_column_names, log};
A
Merge  
Andrey Mironov 已提交
2556

M
Merge  
Michael Kolupaev 已提交
2557 2558 2559 2560 2561
	Block virtual_columns_block;
	ColumnUInt8 * column = new ColumnUInt8(2);
	ColumnPtr column_ptr = column;
	column->getData()[0] = 0;
	column->getData()[1] = 1;
2562
	virtual_columns_block.insert(ColumnWithTypeAndName(column_ptr, new DataTypeUInt8, "_replicated"));
M
Merge  
Michael Kolupaev 已提交
2563

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

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

M
Merge  
Michael Kolupaev 已提交
2570 2571
	BlockInputStreams res;

M
Merge  
Michael Kolupaev 已提交
2572 2573
	size_t part_index = 0;

2574 2575 2576 2577 2578 2579
	/** Настройки parallel_replica_offset и parallel_replicas_count позволяют читать с одной реплики одну часть данных, а с другой - другую.
	  * Для реплицируемых, данные разбиваются таким же механизмом, как работает секция SAMPLE.
	  * А для нереплицируемых данных, так как их целостность между репликами не контролируется,
	  *  с первой (settings.parallel_replica_offset == 0) реплики выбираются все данные, а с остальных - никакие.
	  */

2580
	if ((settings.parallel_replica_offset == 0) && unreplicated_reader && values.count(0))
M
Merge  
Michael Kolupaev 已提交
2581
	{
2582 2583
		res = unreplicated_reader->read(real_column_names, query,
										context, settings, processed_stage,
2584
										max_block_size, threads, &part_index, 0);
M
Merge  
Michael Kolupaev 已提交
2585 2586 2587 2588 2589 2590

		for (auto & virtual_column : virt_column_names)
		{
			if (virtual_column == "_replicated")
			{
				for (auto & stream : res)
M
Merge  
Michael Kolupaev 已提交
2591
					stream = new AddingConstColumnBlockInputStream<UInt8>(stream, new DataTypeUInt8, 0, "_replicated");
M
Merge  
Michael Kolupaev 已提交
2592 2593 2594 2595
			}
		}
	}

M
Merge  
Michael Kolupaev 已提交
2596
	if (values.count(1))
M
Merge  
Michael Kolupaev 已提交
2597
	{
2598 2599 2600 2601 2602 2603 2604 2605 2606 2607 2608 2609 2610 2611 2612 2613 2614 2615 2616 2617 2618 2619 2620 2621 2622 2623 2624 2625 2626 2627 2628 2629 2630 2631 2632 2633 2634 2635
		/** Настройка select_sequential_consistency имеет два смысла:
		  * 1. Кидать исключение, если на реплике есть не все куски, которые были записаны на кворум остальных реплик.
		  * 2. Не читать куски, которые ещё не были записаны на кворум реплик.
		  * Для этого приходится синхронно сходить в ZooKeeper.
		  */
		Int64 max_block_number_to_read = 0;
		if (settings.select_sequential_consistency)
		{
			auto zookeeper = getZooKeeper();
			String last_part;
			zookeeper->tryGet(zookeeper_path + "/quorum/last_part", last_part);

			if (!last_part.empty() && !data.getPartIfExists(last_part))	/// TODO Отключение реплики при распределённых запросах.
				throw Exception("Replica doesn't have part " + last_part + " which was successfully written to quorum of other replicas."
					" Send query to another replica or disable 'select_sequential_consistency' setting.", ErrorCodes::REPLICA_IS_NOT_IN_QUORUM);

			if (last_part.empty())	/// Если ещё ни один кусок не был записан с кворумом.
			{
				String quorum_str;
				if (zookeeper->tryGet(zookeeper_path + "/quorum/status", quorum_str))
				{
					ReplicatedMergeTreeQuorumEntry quorum_entry;
					quorum_entry.fromString(quorum_str);
					ActiveDataPartSet::Part part_info;
					ActiveDataPartSet::parsePartName(quorum_entry.part_name, part_info);
					max_block_number_to_read = part_info.left - 1;
				}
			}
			else
			{
				ActiveDataPartSet::Part part_info;
				ActiveDataPartSet::parsePartName(last_part, part_info);
				max_block_number_to_read = part_info.right;
			}
		}

		auto res2 = reader.read(
			real_column_names, query, context, settings, processed_stage, max_block_size, threads, &part_index, max_block_number_to_read);
M
Merge  
Michael Kolupaev 已提交
2636 2637 2638 2639 2640 2641

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

M
Merge  
Michael Kolupaev 已提交
2646
		res.insert(res.end(), res2.begin(), res2.end());
M
Merge  
Michael Kolupaev 已提交
2647 2648 2649
	}

	return res;
M
Merge  
Michael Kolupaev 已提交
2650 2651
}

A
Merge  
Alexey Milovidov 已提交
2652

2653
BlockOutputStreamPtr StorageReplicatedMergeTree::write(ASTPtr query, const Settings & settings)
M
Merge  
Michael Kolupaev 已提交
2654
{
2655 2656
	if (is_readonly)
		throw Exception("Table is in readonly mode", ErrorCodes::TABLE_IS_READ_ONLY);
M
Merge  
Michael Kolupaev 已提交
2657

M
Merge  
Michael Kolupaev 已提交
2658
	String insert_id;
M
Merge  
Michael Kolupaev 已提交
2659 2660 2661
	if (query)
		if (ASTInsertQuery * insert = typeid_cast<ASTInsertQuery *>(&*query))
			insert_id = insert->insert_id;
M
Merge  
Michael Kolupaev 已提交
2662

2663
	return new ReplicatedMergeTreeBlockOutputStream(*this, insert_id, settings.insert_quorum);
M
Merge  
Michael Kolupaev 已提交
2664
}
M
Merge  
Michael Kolupaev 已提交
2665

A
Merge  
Alexey Milovidov 已提交
2666

2667
bool StorageReplicatedMergeTree::optimize(const Settings & settings)
M
Merge  
Michael Kolupaev 已提交
2668
{
2669 2670
	/// Померджим какие-нибудь куски из директории unreplicated.
	/// TODO: Мерджить реплицируемые куски тоже.
M
Merge  
Michael Kolupaev 已提交
2671 2672 2673 2674

	if (!unreplicated_data)
		return false;

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

M
Merge  
Michael Kolupaev 已提交
2677 2678 2679 2680
	unreplicated_data->clearOldParts();

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

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

M
Merge  
Michael Kolupaev 已提交
2688 2689 2690
	return true;
}

A
Merge  
Alexey Milovidov 已提交
2691

M
Merge  
Michael Kolupaev 已提交
2692 2693
void StorageReplicatedMergeTree::alter(const AlterCommands & params,
	const String & database_name, const String & table_name, Context & context)
M
Merge  
Michael Kolupaev 已提交
2694
{
A
Merge  
Alexey Milovidov 已提交
2695
	auto zookeeper = getZooKeeper();
2696 2697
	const MergeTreeMergeBlocker merge_blocker{merger};
	const auto unreplicated_merge_blocker = unreplicated_merger ?
2698
		std::make_unique<MergeTreeMergeBlocker>(*unreplicated_merger) : nullptr;
A
Merge  
Alexey Milovidov 已提交
2699

M
Merge  
Michael Kolupaev 已提交
2700 2701 2702
	LOG_DEBUG(log, "Doing ALTER");

	NamesAndTypesList new_columns;
2703 2704 2705
	NamesAndTypesList new_materialized_columns;
	NamesAndTypesList new_alias_columns;
	ColumnDefaults new_column_defaults;
M
Merge  
Michael Kolupaev 已提交
2706 2707 2708 2709 2710 2711 2712
	String new_columns_str;
	int new_columns_version;
	zkutil::Stat stat;

	{
		auto table_lock = lockStructureForAlter();

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

M
Merge  
Michael Kolupaev 已提交
2716 2717
		data.checkAlter(params);

2718 2719 2720 2721 2722
		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 已提交
2723

2724 2725 2726 2727
		new_columns_str = ColumnsDescription<false>{
			new_columns, new_materialized_columns,
			new_alias_columns, new_column_defaults
		}.toString();
M
Merge  
Michael Kolupaev 已提交
2728 2729 2730 2731 2732 2733 2734 2735 2736 2737 2738 2739 2740 2741

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

M
Merge  
Michael Kolupaev 已提交
2743 2744 2745 2746 2747 2748 2749 2750 2751 2752 2753 2754 2755 2756 2757 2758 2759 2760 2761 2762 2763 2764 2765 2766 2767 2768 2769 2770 2771 2772
	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);
2773

M
Merge  
Michael Kolupaev 已提交
2774 2775 2776 2777 2778 2779 2780
			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 已提交
2781
			if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/columns", &stat, alter_query_event))
M
Merge  
Michael Kolupaev 已提交
2782 2783 2784 2785 2786 2787 2788 2789 2790 2791 2792 2793 2794 2795 2796 2797 2798 2799
			{
				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 已提交
2800 2801 2802 2803 2804

/// Название воображаемого куска, покрывающего все возможные куски в указанном месяце с номерами в указанном диапазоне.
static String getFakePartNameForDrop(const String & month_name, UInt64 left, UInt64 right)
{
	/// Диапазон дат - весь месяц.
A
Merge  
Alexey Arno 已提交
2805
	const auto & lut = DateLUT::instance();
2806
	time_t start_time = lut.YYYYMMDDToDate(parse<UInt32>(month_name + "01"));
M
Merge  
Michael Kolupaev 已提交
2807 2808 2809 2810 2811 2812 2813
	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 已提交
2814

2815
void StorageReplicatedMergeTree::dropUnreplicatedPartition(const Field & partition, const bool detach, const Settings & settings)
M
Merge  
Michael Kolupaev 已提交
2816
{
A
Merge  
Andrey Mironov 已提交
2817 2818 2819 2820 2821 2822 2823 2824 2825 2826 2827 2828 2829 2830 2831
	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 已提交
2832
		if (part->month != month)
A
Merge  
Andrey Mironov 已提交
2833 2834 2835 2836 2837
			continue;

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

2838 2839 2840 2841
		if (detach)
			unreplicated_data->renameAndDetachPart(part, "");
		else
			unreplicated_data->replaceParts({part}, {}, false);
A
Merge  
Andrey Mironov 已提交
2842 2843
	}

2844
	LOG_INFO(log, (detach ? "Detached " : "Removed ") << removed_parts << " unreplicated parts inside " << apply_visitor(FieldVisitorToString(), partition) << ".");
A
Merge  
Andrey Mironov 已提交
2845 2846 2847
}


2848
void StorageReplicatedMergeTree::dropPartition(ASTPtr query, const Field & field, bool detach, bool unreplicated, const Settings & settings)
A
Merge  
Andrey Mironov 已提交
2849 2850 2851
{
	if (unreplicated)
	{
2852
		dropUnreplicatedPartition(field, detach, settings);
A
Merge  
Andrey Mironov 已提交
2853 2854 2855
		return;
	}

A
Merge  
Alexey Milovidov 已提交
2856
	auto zookeeper = getZooKeeper();
A
Merge  
Alexey Milovidov 已提交
2857
	String month_name = MergeTreeData::getMonthName(field);
M
Merge  
Michael Kolupaev 已提交
2858 2859

	if (!is_leader_node)
2860 2861 2862 2863 2864 2865 2866
	{
		/// Проксируем запрос в лидера.

		auto live_replicas = zookeeper->getChildren(zookeeper_path + "/leader_election");
		if (live_replicas.empty())
			throw Exception("No active replicas", ErrorCodes::NO_ACTIVE_REPLICAS);

2867
		std::sort(live_replicas.begin(), live_replicas.end());
2868 2869 2870 2871 2872 2873 2874 2875 2876 2877 2878 2879 2880 2881 2882 2883 2884 2885 2886 2887 2888 2889 2890 2891 2892 2893 2894
		const auto leader = zookeeper->get(zookeeper_path + "/leader_election/" + live_replicas.front());

		if (leader == replica_name)
			throw Exception("Leader was suddenly changed or logical error.", ErrorCodes::LEADERSHIP_CHANGED);

		ReplicatedMergeTreeAddress leader_address(zookeeper->get(zookeeper_path + "/replicas/" + leader + "/host"));

		auto new_query = query->clone();
		auto & alter = typeid_cast<ASTAlterQuery &>(*new_query);

		alter.database = leader_address.database;
		alter.table = leader_address.table;

		/// NOTE Работает только если есть доступ от пользователя default без пароля. Можно исправить с помощью добавления параметра в конфиг сервера.

		Connection connection(
			leader_address.host,
			leader_address.queries_port,
			leader_address.database,
			"", "", "ClickHouse replica");

		RemoteBlockInputStream stream(connection, formattedAST(new_query), &settings);
		NullBlockOutputStream output;

		copyData(stream, output);
		return;
	}
M
Merge  
Michael Kolupaev 已提交
2895 2896 2897 2898 2899 2900 2901

	/** Пропустим один номер в block_numbers для удаляемого месяца, и будем удалять только куски до этого номера.
	  * Это запретит мерджи удаляемых кусков с новыми вставляемыми данными.
	  * Инвариант: в логе не появятся слияния удаляемых кусков с другими кусками.
	  * NOTE: Если понадобится аналогично поддержать запрос DROP PART, для него придется придумать какой-нибудь новый механизм,
	  *        чтобы гарантировать этот инвариант.
	  */
A
Merge  
Alexey Milovidov 已提交
2902
	Int64 right;
M
Merge  
Michael Kolupaev 已提交
2903 2904 2905 2906 2907 2908 2909 2910 2911

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

	/// Такого никогда не должно происходить.
	if (right == 0)
2912
		throw Exception("Logical error: just allocated block number is zero", ErrorCodes::LOGICAL_ERROR);
M
Merge  
Michael Kolupaev 已提交
2913 2914 2915 2916 2917 2918 2919 2920
	--right;

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

	/** Запретим выбирать для слияния удаляемые куски - сделаем вид, что их всех уже собираются слить в fake_part_name.
	  * Инвариант: после появления в логе записи DROP_RANGE, в логе не появятся слияния удаляемых кусков.
	  */
	{
2921
		std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
M
Merge  
Michael Kolupaev 已提交
2922 2923 2924 2925

		virtual_parts.add(fake_part_name);
	}

M
Merge  
Michael Kolupaev 已提交
2926
	/// Наконец, добившись нужных инвариантов, можно положить запись в лог.
M
Merge  
Michael Kolupaev 已提交
2927 2928 2929 2930 2931 2932
	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 已提交
2933
	entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);
2934
	entry.create_time = time(0);
M
Merge  
Michael Kolupaev 已提交
2935

2936 2937 2938 2939 2940 2941 2942 2943
	/// Если надо - дожидаемся выполнения операции на себе или на всех репликах.
	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 已提交
2944 2945
}

A
Merge  
Alexey Milovidov 已提交
2946

2947
void StorageReplicatedMergeTree::attachPartition(ASTPtr query, const Field & field, bool unreplicated, bool attach_part, const Settings & settings)
M
Merge  
Michael Kolupaev 已提交
2948
{
A
Merge  
Alexey Milovidov 已提交
2949
	auto zookeeper = getZooKeeper();
A
Merge  
Alexey Milovidov 已提交
2950
	String partition;
M
Merge  
Michael Kolupaev 已提交
2951

A
Merge  
Alexey Milovidov 已提交
2952
	if (attach_part)
A
Merge  
Alexey Milovidov 已提交
2953
		partition = field.safeGet<String>();
A
Merge  
Alexey Milovidov 已提交
2954 2955
	else
		partition = MergeTreeData::getMonthName(field);
M
Merge  
Michael Kolupaev 已提交
2956 2957 2958 2959 2960 2961 2962 2963 2964 2965 2966 2967 2968 2969 2970 2971 2972 2973

	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;
2974
			if (0 != name.compare(0, partition.size(), partition))
M
Merge  
Michael Kolupaev 已提交
2975 2976 2977 2978 2979 2980 2981 2982 2983 2984 2985 2986 2987 2988 2989 2990 2991 2992
				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 已提交
2993 2994
	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 已提交
2995 2996 2997 2998

	{
		auto existing_parts = data.getDataParts();
		for (const auto & part : existing_parts)
A
Merge  
Alexey Milovidov 已提交
2999 3000
			if (part->month == month)
				min_used_number = std::min(min_used_number, part->left);
M
Merge  
Michael Kolupaev 已提交
3001 3002 3003 3004 3005 3006 3007 3008 3009 3010 3011 3012 3013 3014 3015 3016 3017 3018 3019 3020 3021 3022
	}

	/// Добавим записи в лог.
	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;
3023 3024
		entry.create_time = time(0);

M
Merge  
Michael Kolupaev 已提交
3025 3026 3027 3028 3029 3030
		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);
3031

3032 3033
	/// Если надо - дожидаемся выполнения операции на себе или на всех репликах.
	if (settings.replication_alter_partitions_sync != 0)
M
Merge  
Michael Kolupaev 已提交
3034
	{
3035 3036 3037 3038 3039 3040 3041 3042 3043 3044
		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 已提交
3045

3046 3047
			++i;
		}
M
Merge  
Michael Kolupaev 已提交
3048
	}
M
Merge  
Michael Kolupaev 已提交
3049 3050
}

A
Merge  
Alexey Milovidov 已提交
3051

M
Merge  
Michael Kolupaev 已提交
3052 3053
void StorageReplicatedMergeTree::drop()
{
3054 3055
	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 已提交
3056

M
Merge  
Michael Kolupaev 已提交
3057 3058
	shutdown();

3059 3060
	auto zookeeper = getZooKeeper();

3061 3062 3063
	if (zookeeper->expired())
		throw Exception("Table was not dropped because ZooKeeper session has been expired.", ErrorCodes::TABLE_WAS_NOT_DROPPED);

M
Merge  
Michael Kolupaev 已提交
3064
	LOG_INFO(log, "Removing replica " << replica_path);
M
Merge  
Michael Kolupaev 已提交
3065
	replica_is_active_node = nullptr;
M
Merge  
Michael Kolupaev 已提交
3066 3067
	zookeeper->tryRemoveRecursive(replica_path);

M
Merge  
Michael Kolupaev 已提交
3068
	/// Проверяем, что zookeeper_path существует: его могла удалить другая реплика после выполнения предыдущей строки.
M
Merge  
Michael Kolupaev 已提交
3069 3070
	Strings replicas;
	if (zookeeper->tryGetChildren(zookeeper_path + "/replicas", replicas) == ZOK && replicas.empty())
M
Merge  
Michael Kolupaev 已提交
3071 3072
	{
		LOG_INFO(log, "Removing table " << zookeeper_path << " (this might take several minutes)");
M
Merge  
Michael Kolupaev 已提交
3073
		zookeeper->tryRemoveRecursive(zookeeper_path);
M
Merge  
Michael Kolupaev 已提交
3074
	}
M
Merge  
Michael Kolupaev 已提交
3075 3076

	data.dropAllData();
M
Merge  
Michael Kolupaev 已提交
3077 3078
}

A
Merge  
Alexey Milovidov 已提交
3079

M
Merge  
Michael Kolupaev 已提交
3080 3081 3082 3083 3084 3085 3086 3087 3088 3089 3090 3091 3092 3093 3094
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 已提交
3095

M
Merge  
Michael Kolupaev 已提交
3096 3097
AbandonableLockInZooKeeper StorageReplicatedMergeTree::allocateBlockNumber(const String & month_name)
{
A
Merge  
Alexey Milovidov 已提交
3098 3099
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
3100 3101 3102 3103 3104 3105 3106 3107
	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 已提交
3108
		for (size_t i = 0; i < RESERVED_BLOCK_NUMBERS; ++i)
M
Merge  
Michael Kolupaev 已提交
3109 3110 3111 3112 3113 3114 3115 3116 3117 3118 3119 3120 3121
		{
			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 已提交
3122

M
Merge  
Michael Kolupaev 已提交
3123
void StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry(const LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
3124
{
A
Merge  
Alexey Milovidov 已提交
3125
	auto zookeeper = getZooKeeper();
M
Merge  
Michael Kolupaev 已提交
3126 3127
	LOG_DEBUG(log, "Waiting for all replicas to process " << entry.znode_name);

A
Merge  
Alexey Milovidov 已提交
3128 3129 3130 3131 3132 3133 3134 3135 3136 3137
	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 已提交
3138 3139
	auto zookeeper = getZooKeeper();

3140 3141
	String entry_str = entry.toString();
	String log_node_name;
M
Merge  
Michael Kolupaev 已提交
3142

3143 3144 3145 3146 3147 3148 3149 3150 3151
	/** В эту функцию могут передать entry двух видов:
	  * 1. (более часто) Из директории log - общего лога, откуда реплики копируют записи в свою queue.
	  * 2. Из директории queue одной из реплик.
	  *
	  * Проблема в том, что номера (sequential нод) элементов очереди в log и в queue не совпадают.
	  * (И в queue не совпадают номера у одного и того же элемента лога для разных реплик.)
	  *
	  * Поэтому следует рассматривать эти случаи по-отдельности.
	  */
M
Merge  
Michael Kolupaev 已提交
3152

3153 3154 3155 3156 3157 3158 3159
	/** Первое - нужно дождаться, пока реплика возьмёт к себе в queue элемент очереди из log,
	  *  если она ещё этого не сделала (см. функцию pullLogsToQueue).
	  *
	  * Для этого проверяем её узел log_pointer - максимальный номер взятого элемента из log плюс единица.
	  */

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

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

3167 3168 3169 3170 3171 3172 3173 3174 3175 3176 3177 3178 3179
		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 已提交
3180
	}
3181 3182 3183 3184 3185 3186 3187
	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 已提交
3188

3189 3190 3191 3192 3193 3194 3195 3196 3197 3198 3199 3200 3201 3202 3203 3204 3205 3206 3207 3208 3209 3210 3211 3212 3213 3214 3215 3216 3217 3218 3219 3220 3221 3222 3223 3224 3225 3226 3227 3228 3229 3230 3231 3232 3233 3234 3235 3236 3237 3238
		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 已提交
3239

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

A
Merge  
Alexey Milovidov 已提交
3243 3244 3245 3246
	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);
3247
		if (exists && queue_entry_str == entry_str)
M
Merge  
Michael Kolupaev 已提交
3248
		{
3249
			queue_entry_to_wait_for = entry_name;
A
Merge  
Alexey Milovidov 已提交
3250
			break;
M
Merge  
Michael Kolupaev 已提交
3251
		}
A
Merge  
Alexey Milovidov 已提交
3252
	}
M
Merge  
Michael Kolupaev 已提交
3253

A
Merge  
Alexey Milovidov 已提交
3254
	/// Пока искали запись, ее уже выполнили и удалили.
3255 3256 3257
	if (queue_entry_to_wait_for.empty())
	{
		LOG_DEBUG(log, "No corresponding node found. Assuming it has been already processed.");
A
Merge  
Alexey Milovidov 已提交
3258
		return;
3259 3260 3261
	}

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

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


3268
void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields)
3269
{
A
Merge  
Alexey Milovidov 已提交
3270 3271
	auto zookeeper = getZooKeeper();

3272
	res.is_leader = is_leader_node;
3273
	res.is_readonly = is_readonly;
3274
	res.is_session_expired = !zookeeper || zookeeper->expired();
3275 3276 3277 3278 3279

	{
		std::lock_guard<std::mutex> lock(queue_mutex);
		res.future_parts = future_parts.size();
		res.queue_size = queue.size();
3280
		res.last_queue_update = last_queue_update;
3281 3282 3283

		res.inserts_in_queue = 0;
		res.merges_in_queue = 0;
A
Merge  
Alexey Milovidov 已提交
3284
		res.queue_oldest_time = 0;
3285 3286
		res.inserts_oldest_time = 0;
		res.merges_oldest_time = 0;
3287 3288 3289

		for (const LogEntryPtr & entry : queue)
		{
3290 3291 3292
			if (entry->create_time && (!res.queue_oldest_time || entry->create_time < res.queue_oldest_time))
				res.queue_oldest_time = entry->create_time;

3293
			if (entry->type == LogEntry::GET_PART)
3294
			{
3295
				++res.inserts_in_queue;
3296 3297

				if (entry->create_time && (!res.inserts_oldest_time || entry->create_time < res.inserts_oldest_time))
3298
				{
3299
					res.inserts_oldest_time = entry->create_time;
3300 3301
					res.oldest_part_to_get = entry->new_part_name;
				}
3302 3303
			}

3304
			if (entry->type == LogEntry::MERGE_PARTS)
3305
			{
3306
				++res.merges_in_queue;
A
Merge  
Alexey Milovidov 已提交
3307

3308
				if (entry->create_time && (!res.merges_oldest_time || entry->create_time < res.merges_oldest_time))
3309
				{
3310
					res.merges_oldest_time = entry->create_time;
3311 3312
					res.oldest_part_to_merge_to = entry->new_part_name;
				}
3313
			}
3314 3315 3316 3317 3318 3319 3320 3321 3322 3323 3324 3325 3326
		}
	}

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

3327
	if (res.is_session_expired || !with_zk_fields)
3328 3329 3330 3331 3332 3333 3334 3335 3336 3337
	{
		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");

3338 3339 3340 3341 3342 3343 3344 3345 3346 3347 3348 3349
		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);
3350 3351 3352 3353 3354 3355 3356 3357 3358 3359 3360

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

3361

3362 3363 3364 3365 3366 3367 3368 3369 3370 3371 3372 3373
void StorageReplicatedMergeTree::getQueue(LogEntriesData & res, String & replica_name_)
{
	res.clear();
	replica_name_ = replica_name;

	std::lock_guard<std::mutex> lock(queue_mutex);
	res.reserve(queue.size());
	for (const auto & entry : queue)
		res.emplace_back(*entry);
}


3374 3375 3376 3377 3378 3379 3380 3381 3382
void StorageReplicatedMergeTree::getReplicaDelays(time_t & out_absolute_delay, time_t & out_relative_delay) const
{
	if (!restarting_thread)
		throw Exception("Table was shutted down or is in readonly mode.", ErrorCodes::TABLE_IS_READ_ONLY);

	restarting_thread->getReplicaDelays(out_absolute_delay, out_relative_delay);
}


3383
void StorageReplicatedMergeTree::fetchPartition(const Field & partition, const String & from_, const Settings & settings)
3384
{
A
Merge  
Alexey Milovidov 已提交
3385 3386
	auto zookeeper = getZooKeeper();

3387 3388
	String partition_str = MergeTreeData::getMonthName(partition);

3389
	String from = from_;
3390 3391
	if (from.back() == '/')
		from.resize(from.size() - 1);
3392

3393 3394
	LOG_INFO(log, "Will fetch partition " << partition_str << " from shard " << from_);

3395 3396 3397 3398 3399 3400 3401 3402
	/** Проверим, что в директории 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);

3403 3404 3405 3406 3407 3408 3409 3410 3411 3412 3413 3414
	/// Список реплик шарда-источника.
	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())
3415
		throw Exception("No active replicas for shard " + from, ErrorCodes::NO_ACTIVE_REPLICAS);
3416 3417

	/** Надо выбрать лучшую (наиболее актуальную) реплику.
3418
	  * Это реплика с максимальным log_pointer, затем с минимальным размером queue.
3419 3420
	  * NOTE Это не совсем лучший критерий. Для скачивания старых партиций это не имеет смысла,
	  *  и было бы неплохо уметь выбирать реплику, ближайшую по сети.
3421
	  * NOTE Разумеется, здесь есть data race-ы. Можно решить ретраями.
3422
	  */
3423 3424 3425 3426 3427 3428
	Int64 max_log_pointer = -1;
	UInt64 min_queue_size = std::numeric_limits<UInt64>::max();
	String best_replica;

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

3431
		String log_pointer_str = zookeeper->get(current_replica_path + "/log_pointer");
3432 3433 3434
		Int64 log_pointer = log_pointer_str.empty() ? 0 : parse<UInt64>(log_pointer_str);

		zkutil::Stat stat;
3435
		zookeeper->get(current_replica_path + "/queue", &stat);
3436 3437 3438 3439 3440 3441 3442 3443 3444 3445 3446 3447 3448 3449 3450 3451 3452
		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.");

3453 3454
	String best_replica_path = from + "/replicas/" + best_replica;

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

3457 3458 3459 3460 3461 3462 3463 3464 3465 3466 3467 3468 3469 3470 3471 3472 3473 3474 3475 3476 3477 3478 3479 3480 3481 3482 3483 3484 3485 3486
	/** Пытаемся скачать эти куски.
	  * Часть из них могла удалиться из-за мерджа.
	  * В этом случае, обновляем информацию о доступных кусках и пробуем снова.
	  */

	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);
3487 3488 3489

			if (parts_to_fetch.empty())
				throw Exception("Partition " + partition_str + " on " + best_replica_path + " doesn't exist", ErrorCodes::PARTITION_DOESNT_EXIST);
3490 3491 3492 3493 3494 3495 3496 3497 3498 3499 3500 3501 3502 3503 3504 3505 3506 3507 3508 3509
		}
		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
			{
3510
				fetchPart(part, best_replica_path, true, 0);
3511 3512 3513 3514 3515 3516 3517 3518 3519 3520
			}
			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);
			}
		}
3521

3522 3523
		++try_no;
	} while (!missing_parts.empty());
3524 3525 3526
}


3527 3528 3529 3530 3531 3532 3533 3534 3535 3536 3537 3538 3539
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 已提交
3540
}