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

22

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

M
Merge  
Michael Kolupaev 已提交
26

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

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

M
Merge  
Michael Kolupaev 已提交
32

33 34 35 36 37 38 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
/** Для каждого куска есть сразу три места, где он должен быть:
  * 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 已提交
70 71 72
StorageReplicatedMergeTree::StorageReplicatedMergeTree(
	const String & zookeeper_path_,
	const String & replica_name_,
M
Merge  
Michael Kolupaev 已提交
73
	bool attach,
M
Merge  
Michael Kolupaev 已提交
74 75
	const String & path_, const String & database_name_, const String & name_,
	NamesAndTypesListPtr columns_,
76
	const NamesAndTypesList & materialized_columns_,
77 78
	const NamesAndTypesList & alias_columns_,
	const ColumnDefaults & column_defaults_,
M
Merge  
Michael Kolupaev 已提交
79
	Context & context_,
M
Merge  
Michael Kolupaev 已提交
80 81 82 83 84 85
	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 已提交
86
	const Names & columns_to_sum_,
M
Merge  
Michael Kolupaev 已提交
87
	const MergeTreeSettings & settings_)
88
    : IStorage{materialized_columns_, alias_columns_, column_defaults_}, context(context_),
A
Merge  
Alexey Milovidov 已提交
89
	current_zookeeper(context.getZooKeeper()), database_name(database_name_),
M
Merge  
Michael Kolupaev 已提交
90 91 92
	table_name(name_), full_path(path_ + escapeForFileName(table_name) + '/'),
	zookeeper_path(context.getMacros().expand(zookeeper_path_)),
	replica_name(context.getMacros().expand(replica_name_)),
93 94 95
	data(full_path, columns_,
		 materialized_columns_, alias_columns_, column_defaults_,
		 context_, primary_expr_ast_, date_column_name_,
A
Merge  
Alexey Milovidov 已提交
96
		 sampling_expression_, index_granularity_, mode_, sign_column_, columns_to_sum_,
97 98
		 settings_, database_name_ + "." + table_name, true,
		 std::bind(&StorageReplicatedMergeTree::enqueuePartForCheck, this, std::placeholders::_1)),
99 100
	reader(data), writer(data), merger(data), fetcher(data), shutdown_event(false),
	log(&Logger::get(database_name + "." + table_name + " (StorageReplicatedMergeTree)"))
M
Merge  
Michael Kolupaev 已提交
101
{
102 103
	if (!zookeeper_path.empty() && zookeeper_path.back() == '/')
		zookeeper_path.resize(zookeeper_path.size() - 1);
M
Merge  
Michael Kolupaev 已提交
104 105 106 107
	replica_path = zookeeper_path + "/replicas/" + replica_name;

	bool skip_sanity_checks = false;

108
	try
M
Merge  
Michael Kolupaev 已提交
109
	{
A
Merge  
Alexey Milovidov 已提交
110
		if (current_zookeeper && current_zookeeper->exists(replica_path + "/flags/force_restore_data"))
111 112
		{
			skip_sanity_checks = true;
A
Merge  
Alexey Milovidov 已提交
113
			current_zookeeper->remove(replica_path + "/flags/force_restore_data");
M
Merge  
Michael Kolupaev 已提交
114

115 116 117 118 119 120 121 122 123 124
			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 已提交
125
			current_zookeeper = nullptr;
126 127 128
		}
		else
			throw;
M
Merge  
Michael Kolupaev 已提交
129 130 131 132
	}

	data.loadDataParts(skip_sanity_checks);

A
Merge  
Alexey Milovidov 已提交
133
	if (!current_zookeeper)
M
Merge  
Michael Kolupaev 已提交
134
	{
M
Merge  
Michael Kolupaev 已提交
135 136 137
		if (!attach)
			throw Exception("Can't create replicated table without ZooKeeper", ErrorCodes::NO_ZOOKEEPER);

138
		/// Не активируем реплику. Она будет в режиме readonly.
M
Merge  
Michael Kolupaev 已提交
139 140 141
		return;
	}

M
Merge  
Michael Kolupaev 已提交
142 143
	if (!attach)
	{
M
Merge  
Michael Kolupaev 已提交
144
		createTableIfNotExists();
M
Merge  
Michael Kolupaev 已提交
145

M
Merge  
Michael Kolupaev 已提交
146
		checkTableStructure(false, false);
M
Merge  
Michael Kolupaev 已提交
147
		createReplica();
M
Merge  
Michael Kolupaev 已提交
148 149 150
	}
	else
	{
M
Merge  
Michael Kolupaev 已提交
151
		checkTableStructure(skip_sanity_checks, true);
M
Merge  
Michael Kolupaev 已提交
152
		checkParts(skip_sanity_checks);
M
Merge  
Michael Kolupaev 已提交
153
	}
M
Merge  
Michael Kolupaev 已提交
154

M
Merge  
Michael Kolupaev 已提交
155
	initVirtualParts();
M
Merge  
Michael Kolupaev 已提交
156 157 158 159 160

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

162 163
		unreplicated_data.reset(new MergeTreeData(unreplicated_path, columns_,
			materialized_columns_, alias_columns_, column_defaults_,
164
			context_, primary_expr_ast_,
A
Merge  
Alexey Milovidov 已提交
165
			date_column_name_, sampling_expression_, index_granularity_, mode_, sign_column_, columns_to_sum_, settings_,
M
Merge  
Michael Kolupaev 已提交
166
			database_name_ + "." + table_name + "[unreplicated]", false));
167 168 169

		unreplicated_data->loadDataParts(skip_sanity_checks);

M
Merge  
Michael Kolupaev 已提交
170
		unreplicated_reader.reset(new MergeTreeDataSelectExecutor(*unreplicated_data));
M
Merge  
Michael Kolupaev 已提交
171
		unreplicated_merger.reset(new MergeTreeDataMerger(*unreplicated_data));
M
Merge  
Michael Kolupaev 已提交
172
	}
M
Merge  
Michael Kolupaev 已提交
173

174
	loadQueue();
M
Merge  
Michael Kolupaev 已提交
175

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

180

M
Merge  
Michael Kolupaev 已提交
181 182 183
StoragePtr StorageReplicatedMergeTree::create(
	const String & zookeeper_path_,
	const String & replica_name_,
M
Merge  
Michael Kolupaev 已提交
184
	bool attach,
M
Merge  
Michael Kolupaev 已提交
185 186
	const String & path_, const String & database_name_, const String & name_,
	NamesAndTypesListPtr columns_,
187
	const NamesAndTypesList & materialized_columns_,
188 189
	const NamesAndTypesList & alias_columns_,
	const ColumnDefaults & column_defaults_,
M
Merge  
Michael Kolupaev 已提交
190
	Context & context_,
M
Merge  
Michael Kolupaev 已提交
191 192 193 194 195 196
	ASTPtr & primary_expr_ast_,
	const String & date_column_name_,
	const ASTPtr & sampling_expression_,
	size_t index_granularity_,
	MergeTreeData::Mode mode_,
	const String & sign_column_,
197
	const Names & columns_to_sum_,
M
Merge  
Michael Kolupaev 已提交
198 199
	const MergeTreeSettings & settings_)
{
200 201 202
	auto res = new StorageReplicatedMergeTree{
		zookeeper_path_, replica_name_, attach,
		path_, database_name_, name_,
203
		columns_, materialized_columns_, alias_columns_, column_defaults_,
204 205
		context_, primary_expr_ast_, date_column_name_,
		sampling_expression_, index_granularity_, mode_,
A
Merge  
Alexey Milovidov 已提交
206 207
		sign_column_, columns_to_sum_, settings_};

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

A
Merge  
Alexey Milovidov 已提交
210
	if (res->getZooKeeper())
M
Merge  
Michael Kolupaev 已提交
211 212
	{
		String endpoint_name = "ReplicatedMergeTree:" + res->replica_path;
M
Merge  
Michael Kolupaev 已提交
213
		InterserverIOEndpointPtr endpoint = new ReplicatedMergeTreePartsServer(res->data, *res);
M
Merge  
Michael Kolupaev 已提交
214 215
		res->endpoint_holder = new InterserverIOEndpointHolder(endpoint_name, endpoint, res->context.getInterserverIOHandler());
	}
216

M
Merge  
Michael Kolupaev 已提交
217
	return res_ptr;
M
Merge  
Michael Kolupaev 已提交
218 219
}

A
Merge  
Alexey Milovidov 已提交
220

M
Merge  
Michael Kolupaev 已提交
221 222 223 224 225 226 227 228
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 已提交
229

A
Merge  
Alexey Milovidov 已提交
230

M
Merge  
Michael Kolupaev 已提交
231
void StorageReplicatedMergeTree::createTableIfNotExists()
M
Merge  
Michael Kolupaev 已提交
232
{
A
Merge  
Alexey Milovidov 已提交
233 234
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
235 236
	if (zookeeper->exists(zookeeper_path))
		return;
M
Merge  
Michael Kolupaev 已提交
237

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

M
Merge  
Michael Kolupaev 已提交
240 241
	zookeeper->createAncestors(zookeeper_path);

M
Merge  
Michael Kolupaev 已提交
242
	/// Запишем метаданные таблицы, чтобы реплики могли сверять с ними параметры таблицы.
M
Merge  
Michael Kolupaev 已提交
243 244 245 246 247 248 249 250
	std::stringstream metadata;
	metadata << "metadata format version: 1" << std::endl;
	metadata << "date column: " << data.date_column_name << std::endl;
	metadata << "sampling expression: " << formattedAST(data.sampling_expression) << std::endl;
	metadata << "index granularity: " << data.index_granularity << std::endl;
	metadata << "mode: " << static_cast<int>(data.mode) << std::endl;
	metadata << "sign column: " << data.sign_column << std::endl;
	metadata << "primary key: " << formattedAST(data.primary_expr_ast) << std::endl;
M
Merge  
Michael Kolupaev 已提交
251

M
Merge  
Michael Kolupaev 已提交
252 253 254 255 256
	zkutil::Ops ops;
	ops.push_back(new zkutil::Op::Create(zookeeper_path, "",
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/metadata", metadata.str(),
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
257 258 259
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/columns", ColumnsDescription<false>{
				data.getColumnsListNonMaterialized(), data.materialized_columns,
				data.alias_columns, data.column_defaults}.toString(),
M
Merge  
Michael Kolupaev 已提交
260 261 262 263 264 265 266 267 268 269 270 271 272 273 274 275 276 277 278
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/log", "",
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/blocks", "",
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/block_numbers", "",
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/nonincrement_block_numbers", "",
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/leader_election", "",
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/temp", "",
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(zookeeper_path + "/replicas", "",
										 zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));

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

A
Merge  
Alexey Milovidov 已提交
281

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

M
Merge  
Michael Kolupaev 已提交
289
	String metadata_str = zookeeper->get(zookeeper_path + "/metadata");
M
Merge  
Michael Kolupaev 已提交
290 291 292 293 294 295 296 297 298 299 300 301 302
	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 已提交
303 304
	/// NOTE: Можно сделать менее строгую проверку совпадения выражений, чтобы таблицы не ломались от небольших изменений
	///       в коде formatAST.
M
Merge  
Michael Kolupaev 已提交
305
	assertString(formattedAST(data.primary_expr_ast), buf);
M
Merge  
Michael Kolupaev 已提交
306
	assertString("\n", buf);
M
Merge  
Michael Kolupaev 已提交
307
	assertEOF(buf);
M
Merge  
Michael Kolupaev 已提交
308 309

	zkutil::Stat stat;
310
	auto columns_desc = ColumnsDescription<true>::parse(zookeeper->get(zookeeper_path + "/columns", &stat));
311 312 313 314 315

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

318 319 320 321
	if (columns != data.getColumnsListNonMaterialized() ||
		materialized_columns != data.materialized_columns ||
		alias_columns != data.alias_columns ||
		column_defaults != data.column_defaults)
M
Merge  
Michael Kolupaev 已提交
322
	{
323 324 325 326
		if (allow_alter &&
			(skip_sanity_checks ||
			 data.getColumnsListNonMaterialized().sizeOfDifference(columns) +
			 data.materialized_columns.sizeOfDifference(materialized_columns) <= 2))
M
Merge  
Michael Kolupaev 已提交
327
		{
M
Merge  
Michael Kolupaev 已提交
328
			LOG_WARNING(log, "Table structure in ZooKeeper is a little different from local table structure. Assuming ALTER.");
329 330

			/// Без всяких блокировок, потому что таблица еще не создана.
331 332
			InterpreterAlterQuery::updateMetadata(database_name, table_name, columns,
				materialized_columns, alias_columns, column_defaults, context);
333
			data.setColumnsList(columns);
334 335 336
			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 已提交
337
		}
M
Merge  
Michael Kolupaev 已提交
338
		else
M
Merge  
Michael Kolupaev 已提交
339
		{
M
Merge  
Michael Kolupaev 已提交
340
			throw Exception("Table structure in ZooKeeper is too different from local table structure.",
M
Merge  
Michael Kolupaev 已提交
341
							ErrorCodes::INCOMPATIBLE_COLUMNS);
M
Merge  
Michael Kolupaev 已提交
342
		}
M
Merge  
Michael Kolupaev 已提交
343 344
	}
}
M
Merge  
Michael Kolupaev 已提交
345

A
Merge  
Alexey Milovidov 已提交
346

347 348 349 350 351 352 353 354 355 356 357 358 359 360 361 362 363 364 365
/** При необходимости восстановить кусок, реплика сама добавляет в свою очередь запись на его получение.
  * Какое поставить время для этой записи в очереди? Время учитывается при расчёте отставания реплики.
  * Для этих целей имеет смысл использовать время создания недостающего куска
  *  (то есть, при расчёте отставания будет учитано, насколько старый кусок нам нужно восстановить).
  */
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 已提交
366 367
void StorageReplicatedMergeTree::createReplica()
{
A
Merge  
Alexey Milovidov 已提交
368 369
	auto zookeeper = getZooKeeper();

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

M
Merge  
Michael Kolupaev 已提交
372 373 374 375 376 377 378 379
	/// Создадим пустую реплику. Ноду columns создадим в конце - будем использовать ее в качестве признака, что создание реплики завершено.
	zkutil::Ops ops;
	ops.push_back(new zkutil::Op::Create(replica_path, "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(replica_path + "/host", "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(replica_path + "/log_pointer", "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(replica_path + "/queue", "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(replica_path + "/parts", "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
	ops.push_back(new zkutil::Op::Create(replica_path + "/flags", "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
380 381 382 383 384 385 386 387 388 389 390 391

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

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

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

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

M
Merge  
Michael Kolupaev 已提交
405 406 407
	Stat stat;
	zookeeper->exists(replica_path, &stat);
	auto my_create_time = stat.czxid;
M
Merge  
Michael Kolupaev 已提交
408

M
Merge  
Michael Kolupaev 已提交
409 410
	std::random_shuffle(replicas.begin(), replicas.end());
	for (const String & replica : replicas)
M
Merge  
Michael Kolupaev 已提交
411
	{
M
Merge  
Michael Kolupaev 已提交
412 413 414 415 416 417 418 419
		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 已提交
420 421
	}

M
Merge  
Michael Kolupaev 已提交
422
	if (source_replica.empty())
M
Merge  
Michael Kolupaev 已提交
423
	{
M
Merge  
Michael Kolupaev 已提交
424
		LOG_INFO(log, "This is the first replica");
M
Merge  
Michael Kolupaev 已提交
425
	}
M
Merge  
Michael Kolupaev 已提交
426
	else
M
Merge  
Michael Kolupaev 已提交
427
	{
M
Merge  
Michael Kolupaev 已提交
428
		LOG_INFO(log, "Will mimic " << source_replica);
M
Merge  
Michael Kolupaev 已提交
429

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

M
Merge  
Michael Kolupaev 已提交
432 433 434 435 436 437 438 439 440 441 442 443 444 445 446 447 448 449 450 451 452 453 454 455 456 457 458 459 460 461 462 463 464 465 466 467 468 469 470
		/** Если эталонная реплика еще не до конца создана, подождем.
		  * 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");
471 472
		ActiveDataPartSet active_parts_set(parts);

M
Merge  
Michael Kolupaev 已提交
473 474 475 476 477 478 479
		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;
480
			log_entry.create_time = tryGetPartCreateTime(zookeeper, source_path, name);
481

M
Merge  
Michael Kolupaev 已提交
482 483 484 485 486 487 488 489 490
			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);
		}
491 492 493

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

M
Merge  
Michael Kolupaev 已提交
494
		LOG_DEBUG(log, "Copied " << source_queue.size() << " queue entries");
M
Merge  
Michael Kolupaev 已提交
495
	}
M
Merge  
Michael Kolupaev 已提交
496

497 498 499 500 501 502
	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 已提交
503
}
M
Merge  
Michael Kolupaev 已提交
504 505


M
Merge  
Michael Kolupaev 已提交
506
void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
M
Merge  
Michael Kolupaev 已提交
507
{
A
Merge  
Alexey Milovidov 已提交
508 509
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
510
	Strings expected_parts_vec = zookeeper->getChildren(replica_path + "/parts");
M
Merge  
Michael Kolupaev 已提交
511 512

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

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

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

M
Merge  
Michael Kolupaev 已提交
520 521 522 523 524
	for (const auto & part : parts)
	{
		if (expected_parts.count(part->name))
			expected_parts.erase(part->name);
		else
M
Merge  
Michael Kolupaev 已提交
525
			unexpected_parts.insert(part);
M
Merge  
Michael Kolupaev 已提交
526 527
	}

M
Merge  
Michael Kolupaev 已提交
528
	/// Какие локальные куски добавить в ZK.
M
Merge  
Michael Kolupaev 已提交
529
	MergeTreeData::DataPartsVector parts_to_add;
M
Merge  
Michael Kolupaev 已提交
530 531 532 533

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

M
Merge  
Michael Kolupaev 已提交
534 535
	for (const String & missing_name : expected_parts)
	{
M
Merge  
Michael Kolupaev 已提交
536
		/// Если локально не хватает какого-то куска, но есть покрывающий его кусок, можно заменить в ZK недостающий покрывающим.
M
Merge  
Michael Kolupaev 已提交
537
		auto containing = data.getActiveContainingPart(missing_name);
M
Merge  
Michael Kolupaev 已提交
538 539 540 541 542 543 544 545 546 547
		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 已提交
548
		{
M
Merge  
Michael Kolupaev 已提交
549
			LOG_ERROR(log, "Fetching missing part " << missing_name);
M
Merge  
Michael Kolupaev 已提交
550
			parts_to_fetch.push_back(missing_name);
M
Merge  
Michael Kolupaev 已提交
551 552
		}
	}
M
Merge  
Michael Kolupaev 已提交
553

M
Merge  
Michael Kolupaev 已提交
554 555 556 557 558 559
	for (const String & name : parts_to_fetch)
		expected_parts.erase(name);

	String sanity_report =
		"There are " + toString(unexpected_parts.size()) + " unexpected parts, "
					 + toString(parts_to_add.size()) + " unexpectedly merged parts, "
M
Merge  
Michael Kolupaev 已提交
560
					 + toString(expected_parts.size()) + " missing obsolete parts, "
M
Merge  
Michael Kolupaev 已提交
561 562
					 + toString(parts_to_fetch.size()) + " missing parts";

563 564 565 566 567 568 569 570 571 572 573 574 575 576 577
	/** Можно автоматически синхронизировать данные,
	  * если количество ошибок каждого из четырёх типов не больше соответствующих порогов,
	  * или если отношение общего количества ошибок к общему количеству кусков (минимальному - в локальной файловой системе или в ZK)
	  *  не больше некоторого отношения (например 5%).
	  */

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

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

M
Merge  
Michael Kolupaev 已提交
579 580
	if (insane)
	{
581 582 583 584 585
		if (skip_sanity_checks)
			LOG_WARNING(log, sanity_report);
		else
			throw Exception("The local set of parts of table " + getTableName() + " doesn't look like the set of parts in ZooKeeper. "
				+ sanity_report, ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS);
M
Merge  
Michael Kolupaev 已提交
586 587
	}

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

M
Merge  
Michael Kolupaev 已提交
593
		zkutil::Ops ops;
M
Merge  
Michael Kolupaev 已提交
594
		checkPartAndAddToZooKeeper(part, ops);
M
Merge  
Michael Kolupaev 已提交
595
		zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
596
	}
M
Merge  
Michael Kolupaev 已提交
597

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

		zkutil::Ops ops;
604
		removePartFromZooKeeper(name, ops);
M
Merge  
Michael Kolupaev 已提交
605 606 607 608 609 610 611 612 613 614
		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 已提交
615
		log_entry.source_replica = "";
M
Merge  
Michael Kolupaev 已提交
616
		log_entry.new_part_name = name;
617
		log_entry.create_time = tryGetPartCreateTime(zookeeper, replica_path, name);
M
Merge  
Michael Kolupaev 已提交
618 619

		/// Полагаемся, что это происходит до загрузки очереди (loadQueue).
M
Merge  
Michael Kolupaev 已提交
620
		zkutil::Ops ops;
621
		removePartFromZooKeeper(name, ops);
M
Merge  
Michael Kolupaev 已提交
622 623
		ops.push_back(new zkutil::Op::Create(
			replica_path + "/queue/queue-", log_entry.toString(), zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
M
Merge  
Michael Kolupaev 已提交
624
		zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
625 626 627
	}

	/// Удалим лишние локальные куски.
628
	for (const MergeTreeData::DataPartPtr & part : unexpected_parts)
M
Merge  
Michael Kolupaev 已提交
629
	{
M
Merge  
Michael Kolupaev 已提交
630
		LOG_ERROR(log, "Renaming unexpected part " << part->name << " to ignored_" + part->name);
M
Merge  
Michael Kolupaev 已提交
631
		data.renameAndDetachPart(part, "ignored_", true);
M
Merge  
Michael Kolupaev 已提交
632 633
	}
}
M
Merge  
Michael Kolupaev 已提交
634

A
Merge  
Alexey Milovidov 已提交
635

M
Merge  
Michael Kolupaev 已提交
636 637 638 639 640 641 642
void StorageReplicatedMergeTree::initVirtualParts()
{
	auto parts = data.getDataParts();
	for (const auto & part : parts)
		virtual_parts.add(part->name);
}

A
Merge  
Alexey Milovidov 已提交
643

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

M
Merge  
Michael Kolupaev 已提交
648 649 650
	if (part_name.empty())
		part_name = part->name;

651
	check(part->columns);
M
Merge  
Michael Kolupaev 已提交
652
	int expected_columns_version = columns_version;
653

M
Merge  
Michael Kolupaev 已提交
654 655 656 657 658
	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 已提交
659
	{
M
Merge  
Michael Kolupaev 已提交
660 661
		zkutil::Stat stat_before, stat_after;
		String columns_str;
M
Merge  
Michael Kolupaev 已提交
662
		if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name + "/columns", columns_str, &stat_before))
M
Merge  
Michael Kolupaev 已提交
663 664 665
			continue;
		if (columns_str != expected_columns_str)
		{
M
Merge  
Michael Kolupaev 已提交
666
			LOG_INFO(log, "Not checking checksums of part " << part_name << " with replica " << replica
M
Merge  
Michael Kolupaev 已提交
667 668 669
				<< " because columns are different");
			continue;
		}
M
Merge  
Michael Kolupaev 已提交
670
		String checksums_str;
M
Merge  
Michael Kolupaev 已提交
671 672
		/// Проверим, что версия ноды со столбцами не изменилась, пока мы читали checksums.
		/// Это гарантирует, что столбцы и чексуммы относятся к одним и тем же данным.
M
Merge  
Michael Kolupaev 已提交
673 674
		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 已提交
675
			stat_before.version != stat_after.version)
M
Merge  
Michael Kolupaev 已提交
676
		{
M
Merge  
Michael Kolupaev 已提交
677
			LOG_INFO(log, "Not checking checksums of part " << part_name << " with replica " << replica
M
Merge  
Michael Kolupaev 已提交
678 679
				<< " because part changed while we were reading its checksums");
			continue;
M
Merge  
Michael Kolupaev 已提交
680
		}
M
Merge  
Michael Kolupaev 已提交
681 682 683

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

M
Merge  
Michael Kolupaev 已提交
686
	if (zookeeper->exists(replica_path + "/parts/" + part_name))
M
Merge  
Michael Kolupaev 已提交
687
	{
M
Merge  
Michael Kolupaev 已提交
688
		LOG_ERROR(log, "checkPartAndAddToZooKeeper: node " << replica_path + "/parts/" + part_name << " already exists");
M
Merge  
Michael Kolupaev 已提交
689 690 691
		return;
	}

692 693 694
	ops.push_back(new zkutil::Op::Check(
		zookeeper_path + "/columns",
		expected_columns_version));
M
Merge  
Michael Kolupaev 已提交
695
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
696
		replica_path + "/parts/" + part_name,
M
Merge  
Michael Kolupaev 已提交
697
		"",
M
Merge  
Michael Kolupaev 已提交
698
		zookeeper->getDefaultACL(),
M
Merge  
Michael Kolupaev 已提交
699
		zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
700
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
701
		replica_path + "/parts/" + part_name + "/columns",
M
Merge  
Michael Kolupaev 已提交
702 703 704
		part->columns.toString(),
		zookeeper->getDefaultACL(),
		zkutil::CreateMode::Persistent));
M
Merge  
Michael Kolupaev 已提交
705
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
706
		replica_path + "/parts/" + part_name + "/checksums",
M
Merge  
Michael Kolupaev 已提交
707
		part->checksums.toString(),
M
Merge  
Michael Kolupaev 已提交
708
		zookeeper->getDefaultACL(),
M
Merge  
Michael Kolupaev 已提交
709 710 711
		zkutil::CreateMode::Persistent));
}

A
Merge  
Alexey Milovidov 已提交
712

M
Merge  
Michael Kolupaev 已提交
713 714
void StorageReplicatedMergeTree::loadQueue()
{
A
Merge  
Alexey Milovidov 已提交
715 716
	auto zookeeper = getZooKeeper();

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

M
Merge  
Michael Kolupaev 已提交
719
	Strings children = zookeeper->getChildren(replica_path + "/queue");
M
Merge  
Michael Kolupaev 已提交
720 721 722
	std::sort(children.begin(), children.end());
	for (const String & child : children)
	{
A
Merge  
Alexey Milovidov 已提交
723 724
		zkutil::Stat stat;
		String s = zookeeper->get(replica_path + "/queue/" + child, &stat);
725
		LogEntryPtr entry = LogEntry::parse(s, stat);
M
Merge  
Michael Kolupaev 已提交
726 727
		entry->znode_name = child;
		entry->addResultToVirtualParts(*this);
M
Merge  
Michael Kolupaev 已提交
728
		queue.push_back(entry);
M
Merge  
Michael Kolupaev 已提交
729 730 731
	}
}

A
Merge  
Alexey Milovidov 已提交
732

M
Merge  
Michael Kolupaev 已提交
733
void StorageReplicatedMergeTree::pullLogsToQueue(zkutil::EventPtr next_update_event)
M
Merge  
Michael Kolupaev 已提交
734
{
A
Merge  
Alexey Milovidov 已提交
735 736
	auto zookeeper = getZooKeeper();

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

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

M
Merge  
Michael Kolupaev 已提交
742
	if (index_str.empty())
M
Merge  
Michael Kolupaev 已提交
743
	{
M
Merge  
Michael Kolupaev 已提交
744 745 746
		/// Если у нас еще нет указателя на лог, поставим указатель на первую запись в нем.
		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 已提交
747

M
Merge  
Michael Kolupaev 已提交
748
		zookeeper->set(replica_path + "/log_pointer", toString(index));
M
Merge  
Michael Kolupaev 已提交
749 750
	}
	else
M
Merge  
Michael Kolupaev 已提交
751
	{
M
Merge  
Michael Kolupaev 已提交
752
		index = parse<UInt64>(index_str);
M
Merge  
Michael Kolupaev 已提交
753
	}
M
Merge  
Michael Kolupaev 已提交
754

M
Merge  
Michael Kolupaev 已提交
755 756
	UInt64 first_index = index;

M
Merge  
Michael Kolupaev 已提交
757
	size_t count = 0;
M
Merge  
Michael Kolupaev 已提交
758
	String entry_str;
A
Merge  
Alexey Milovidov 已提交
759 760
	zkutil::Stat stat;
	while (zookeeper->tryGet(zookeeper_path + "/log/log-" + padIndex(index), entry_str, &stat))
M
Merge  
Michael Kolupaev 已提交
761
	{
M
Merge  
Michael Kolupaev 已提交
762
		++count;
M
Merge  
Michael Kolupaev 已提交
763
		++index;
M
Merge  
Michael Kolupaev 已提交
764

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

M
Merge  
Michael Kolupaev 已提交
767 768 769
		/// Одновременно добавим запись в очередь и продвинем указатель на лог.
		zkutil::Ops ops;
		ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
770
			replica_path + "/queue/queue-", entry_str, zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
M
Merge  
Michael Kolupaev 已提交
771
		ops.push_back(new zkutil::Op::SetData(
M
Merge  
Michael Kolupaev 已提交
772
			replica_path + "/log_pointer", toString(index), -1));
M
Merge  
Michael Kolupaev 已提交
773
		auto results = zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
774

775
		String path_created = dynamic_cast<zkutil::Op::Create &>(ops[0]).getPathCreated();
M
Merge  
Michael Kolupaev 已提交
776 777
		entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
		entry->addResultToVirtualParts(*this);
M
Merge  
Michael Kolupaev 已提交
778
		queue.push_back(entry);
M
Merge  
Michael Kolupaev 已提交
779
	}
M
Merge  
Michael Kolupaev 已提交
780

A
Alexey Milovidov 已提交
781 782
	last_queue_update = time(0);

M
Merge  
Michael Kolupaev 已提交
783 784 785 786
	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 已提交
787
	}
M
Merge  
Michael Kolupaev 已提交
788

M
Merge  
Michael Kolupaev 已提交
789 790 791
	if (!count)
		return;

M
Merge  
Michael Kolupaev 已提交
792 793
	if (queue_task_handle)
		queue_task_handle->wake();
M
Merge  
Michael Kolupaev 已提交
794

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

A
Merge  
Alexey Milovidov 已提交
798

M
Merge  
Michael Kolupaev 已提交
799
bool StorageReplicatedMergeTree::shouldExecuteLogEntry(const LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
800
{
A
Merge  
Alexey Milovidov 已提交
801 802 803
	/// queue_mutex уже захвачен. Функция вызывается только из queueTask.

	if (entry.type == LogEntry::MERGE_PARTS || entry.type == LogEntry::GET_PART || entry.type == LogEntry::ATTACH_PART)
M
Merge  
Michael Kolupaev 已提交
804
	{
A
Merge  
Alexey Milovidov 已提交
805 806 807 808 809 810 811 812 813 814 815 816 817 818 819 820 821 822 823 824 825 826 827 828 829 830 831 832 833 834 835
		/// Проверим, не создаётся ли сейчас этот же кусок другим действием.
		if (future_parts.count(entry.new_part_name))
		{
			LOG_DEBUG(log, "Not executing log entry for part " << entry.new_part_name
				<< " because another log entry for the same part is being processed. This shouldn't happen often.");
			return false;

			/** Когда соответствующее действие завершится, то 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))
			{
				LOG_DEBUG(log, "Not executing log entry for part " << entry.new_part_name
					<< " because another log entry for covering part " << future_part_name << " is being processed.");
				return false;
			}
		}
M
Merge  
Michael Kolupaev 已提交
836 837
	}

M
Merge  
Michael Kolupaev 已提交
838 839 840 841 842 843 844 845 846 847 848
	if (entry.type == LogEntry::MERGE_PARTS)
	{
		/** Если какая-то из нужных частей сейчас передается или мерджится, подождем окончания этой операции.
		  * Иначе, даже если всех нужных частей для мерджа нет, нужно попытаться сделать мердж.
		  * Если каких-то частей не хватает, вместо мерджа будет попытка скачать кусок.
		  * Такая ситуация возможна, если получение какого-то куска пофейлилось, и его переместили в конец очереди.
		  */
		for (const auto & name : entry.parts_to_merge)
		{
			if (future_parts.count(name))
			{
A
Merge  
Alexey Milovidov 已提交
849 850
				LOG_TRACE(log, "Not merging into part " << entry.new_part_name
					<< " because part " << name << " is not ready yet (log entry for that part is being processed).");
M
Merge  
Michael Kolupaev 已提交
851 852 853
				return false;
			}
		}
854 855 856 857 858 859

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

	return true;
M
Merge  
Michael Kolupaev 已提交
863 864
}

A
Merge  
Alexey Milovidov 已提交
865

M
Merge  
Michael Kolupaev 已提交
866
bool StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry, BackgroundProcessingPool::Context & pool_context)
M
Merge  
Michael Kolupaev 已提交
867
{
A
Merge  
Alexey Milovidov 已提交
868 869
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
870
	if (entry.type == LogEntry::DROP_RANGE)
M
Merge  
Michael Kolupaev 已提交
871 872 873 874
	{
		executeDropRange(entry);
		return true;
	}
M
Merge  
Michael Kolupaev 已提交
875

M
Merge  
Michael Kolupaev 已提交
876
	if (entry.type == LogEntry::GET_PART ||
M
Merge  
Michael Kolupaev 已提交
877 878
		entry.type == LogEntry::MERGE_PARTS ||
		entry.type == LogEntry::ATTACH_PART)
M
Merge  
Michael Kolupaev 已提交
879 880
	{
		/// Если у нас уже есть этот кусок или покрывающий его кусок, ничего делать не нужно.
M
Merge  
Michael Kolupaev 已提交
881
		MergeTreeData::DataPartPtr containing_part = data.getActiveContainingPart(entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
882

883
		/// Даже если кусок есть локально, его (в исключительных случаях) может не быть в zookeeper. Проверим, что он там есть.
M
Merge  
Michael Kolupaev 已提交
884
		if (containing_part && zookeeper->exists(replica_path + "/parts/" + containing_part->name))
M
Merge  
Michael Kolupaev 已提交
885
		{
M
Merge  
Michael Kolupaev 已提交
886
			if (!(entry.type == LogEntry::GET_PART && entry.source_replica == replica_name))
887
				LOG_DEBUG(log, "Skipping action for part " << entry.new_part_name << " - part already exists.");
M
Merge  
Michael Kolupaev 已提交
888
			return true;
M
Merge  
Michael Kolupaev 已提交
889
		}
M
Merge  
Michael Kolupaev 已提交
890 891
	}

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

895
	/// Возможно, этот кусок нам не нужен, так как при записи с кворумом, кворум пофейлился (см. ниже про /quorum/failed_parts).
896
	if (entry.quorum && zookeeper->exists(zookeeper_path + "/quorum/failed_parts/" + entry.new_part_name))
897 898 899 900 901
	{
		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 已提交
902 903
	bool do_fetch = false;

M
Merge  
Michael Kolupaev 已提交
904 905
	if (entry.type == LogEntry::GET_PART)
	{
M
Merge  
Michael Kolupaev 已提交
906
		do_fetch = true;
M
Merge  
Michael Kolupaev 已提交
907
	}
M
Merge  
Michael Kolupaev 已提交
908 909 910 911
	else if (entry.type == LogEntry::ATTACH_PART)
	{
		do_fetch = !executeAttachPart(entry);
	}
M
Merge  
Michael Kolupaev 已提交
912 913
	else if (entry.type == LogEntry::MERGE_PARTS)
	{
914 915 916 917 918 919 920 921
		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 已提交
922
		MergeTreeData::DataPartsVector parts;
923
		bool have_all_parts = true;
M
Merge  
Michael Kolupaev 已提交
924 925
		for (const String & name : entry.parts_to_merge)
		{
M
Merge  
Michael Kolupaev 已提交
926
			MergeTreeData::DataPartPtr part = data.getActiveContainingPart(name);
M
Merge  
Michael Kolupaev 已提交
927 928 929 930 931 932 933
			if (!part)
			{
				have_all_parts = false;
				break;
			}
			if (part->name != name)
			{
M
Merge  
Michael Kolupaev 已提交
934 935
				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 已提交
936 937 938
				have_all_parts = false;
				break;
			}
M
Merge  
Michael Kolupaev 已提交
939 940
			parts.push_back(part);
		}
M
Merge  
Michael Kolupaev 已提交
941

M
Merge  
Michael Kolupaev 已提交
942
		if (!have_all_parts)
M
Merge  
Michael Kolupaev 已提交
943
		{
M
Merge  
Michael Kolupaev 已提交
944 945 946
			/// Если нет всех нужных кусков, попробуем взять у кого-нибудь уже помердженный кусок.
			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 已提交
947
		}
M
Merge  
Michael Kolupaev 已提交
948 949
		else
		{
M
Merge  
Michael Kolupaev 已提交
950 951 952
			/// Если собираемся сливать большие куски, увеличим счетчик потоков, сливающих большие куски.
			for (const auto & part : parts)
			{
M
Merge  
Michael Kolupaev 已提交
953
				if (part->size_in_bytes > data.settings.max_bytes_to_merge_parts_small)
M
Merge  
Michael Kolupaev 已提交
954 955 956 957 958 959 960
				{
					pool_context.incrementCounter("big merges");
					pool_context.incrementCounter("replicated big merges");
					break;
				}
			}

A
Merge  
Alexey Milovidov 已提交
961
			size_t sum_parts_size_in_bytes = MergeTreeDataMerger::estimateDiskSpaceForMerge(parts);
962 963 964

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

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

968
			const auto & merge_entry = context.getMergeList().insert(database_name, table_name, entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
969
			MergeTreeData::Transaction transaction;
A
Merge  
Alexey Arno 已提交
970
			size_t aio_threshold = context.getSettings().min_bytes_to_use_direct_io;
971 972 973

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

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

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

M
Merge  
Michael Kolupaev 已提交
982
			zookeeper->multi(ops);
M
Merge  
Michael Kolupaev 已提交
983

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

M
Merge  
Michael Kolupaev 已提交
987 988 989
			/** При ZCONNECTIONLOSS или ZOPERATIONTIMEOUT можем зря откатить локальные изменения кусков.
			  * Это не проблема, потому что в таком случае слияние останется в очереди, и мы попробуем снова.
			  */
M
Merge  
Michael Kolupaev 已提交
990
			transaction.commit();
M
Merge  
Michael Kolupaev 已提交
991
			merge_selecting_event.set();
M
Merge  
Michael Kolupaev 已提交
992 993 994

			ProfileEvents::increment(ProfileEvents::ReplicatedPartMerges);
		}
M
Merge  
Michael Kolupaev 已提交
995 996 997 998 999
	}
	else
	{
		throw Exception("Unexpected log entry type: " + toString(static_cast<int>(entry.type)));
	}
M
Merge  
Michael Kolupaev 已提交
1000 1001 1002

	if (do_fetch)
	{
M
Merge  
Michael Kolupaev 已提交
1003 1004
		String replica;

M
Merge  
Michael Kolupaev 已提交
1005 1006
		try
		{
M
Merge  
Michael Kolupaev 已提交
1007
			replica = findReplicaHavingPart(entry.new_part_name, true);
A
Merge  
Alexey Milovidov 已提交
1008 1009 1010 1011 1012 1013 1014 1015 1016 1017 1018 1019 1020 1021 1022 1023 1024

			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 已提交
1025 1026
			if (replica.empty())
			{
1027 1028 1029 1030 1031 1032 1033 1034 1035 1036 1037 1038 1039 1040 1041 1042 1043 1044 1045 1046
				/** Если кусок должен быть записан с кворумом, и кворум ещё недостигнут,
				  *  то (из-за того, что кусок невозможно прямо сейчас скачать),
				  *  кворумную запись следует считать безуспешной.
				  * 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);

					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, чтобы разрешить мерджи через номер потерянного куска;
					  * - добавим кусок в список quorum/failed_parts.
					  *
1047 1048
					  * TODO Удаление из blocks.
					  *
1049 1050 1051 1052 1053 1054 1055 1056 1057 1058 1059 1060 1061 1062 1063 1064 1065 1066 1067 1068 1069 1070 1071 1072 1073 1074 1075 1076 1077 1078 1079 1080 1081 1082 1083 1084 1085 1086 1087 1088 1089 1090 1091 1092 1093 1094 1095 1096 1097 1098 1099 1100 1101 1102 1103 1104 1105 1106 1107 1108 1109 1110 1111 1112 1113 1114 1115 1116 1117 1118 1119 1120 1121 1122 1123 1124 1125 1126 1127 1128 1129 1130 1131 1132 1133 1134 1135
					  * Если что-то изменится, то ничего не сделаем - попадём сюда снова в следующий раз.
					  */

					/** Соберём версии узлов 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);

							ops.push_back(new zkutil::Op::Create(
								zookeeper_path + "/nonincrement_block_numbers/" + partition_str + "/block-" + padIndex(part_info.left),
								"",
								zookeeper->getDefaultACL(),
								zkutil::CreateMode::Persistent));

							ops.push_back(new zkutil::Op::Create(
								zookeeper_path + "/quorum/failed_parts/" + entry.new_part_name,
								"",
								zookeeper->getDefaultACL(),
								zkutil::CreateMode::Persistent));

							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 已提交
1136
			}
A
Merge  
Alexey Milovidov 已提交
1137

1138
			fetchPart(entry.new_part_name, zookeeper_path + "/replicas/" + replica, false, entry.quorum);
M
Merge  
Michael Kolupaev 已提交
1139 1140 1141 1142 1143 1144 1145 1146 1147 1148 1149 1150

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

				/// Найдем действие по объединению этого куска с другими. Запомним других.
				StringSet parts_for_merge;
				LogEntries::iterator merge_entry;
				for (LogEntries::iterator it = queue.begin(); it != queue.end(); ++it)
				{
M
Merge  
Michael Kolupaev 已提交
1158
					if ((*it)->type == LogEntry::MERGE_PARTS)
M
Merge  
Michael Kolupaev 已提交
1159
					{
M
Merge  
Michael Kolupaev 已提交
1160 1161
						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 已提交
1162
						{
M
Merge  
Michael Kolupaev 已提交
1163
							parts_for_merge = StringSet((*it)->parts_to_merge.begin(), (*it)->parts_to_merge.end());
M
Merge  
Michael Kolupaev 已提交
1164 1165 1166 1167 1168 1169 1170 1171 1172 1173 1174 1175 1176 1177 1178 1179 1180
							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 已提交
1181 1182
						if (((*it0)->type == LogEntry::MERGE_PARTS || (*it0)->type == LogEntry::GET_PART)
							&& parts_for_merge.count((*it0)->new_part_name))
M
Merge  
Michael Kolupaev 已提交
1183 1184 1185 1186
						{
							queue.splice(queue.end(), queue, it0, it);
						}
					}
M
Merge  
Michael Kolupaev 已提交
1187 1188 1189 1190 1191 1192

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

1198 1199 1200
				/** Если ни у какой активной реплики нет куска, и в очереди нет слияний с его участием,
				  * проверим, есть ли у любой (активной или неактивной) реплики такой кусок или покрывающий его.
				  */
M
Merge  
Michael Kolupaev 已提交
1201 1202
				if (replica.empty())
					enqueuePartForCheck(entry.new_part_name);
M
Merge  
Michael Kolupaev 已提交
1203 1204 1205 1206 1207 1208 1209 1210 1211
			}
			catch (...)
			{
				tryLogCurrentException(__PRETTY_FUNCTION__);
			}

			throw;
		}
	}
M
Merge  
Michael Kolupaev 已提交
1212 1213

	return true;
M
Merge  
Michael Kolupaev 已提交
1214 1215
}

A
Merge  
Alexey Milovidov 已提交
1216

M
Merge  
Michael Kolupaev 已提交
1217
void StorageReplicatedMergeTree::executeDropRange(const StorageReplicatedMergeTree::LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
1218
{
A
Merge  
Alexey Milovidov 已提交
1219 1220
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
1221
	LOG_INFO(log, (entry.detach ? "Detaching" : "Removing") << " parts inside " << entry.new_part_name << ".");
M
Merge  
Michael Kolupaev 已提交
1222 1223 1224 1225 1226 1227 1228 1229 1230 1231 1232 1233 1234 1235 1236 1237 1238 1239 1240 1241 1242 1243 1244 1245 1246 1247 1248 1249 1250

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

M
Merge  
Michael Kolupaev 已提交
1255
	LOG_DEBUG(log, (entry.detach ? "Detaching" : "Removing") << " parts.");
M
Merge  
Michael Kolupaev 已提交
1256 1257 1258 1259 1260 1261 1262 1263 1264 1265 1266
	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 已提交
1267 1268 1269 1270
		/// Если кусок удалять не нужно, надежнее переместить директорию до изменений в ZooKeeper.
		if (entry.detach)
			data.renameAndDetachPart(part);

M
Merge  
Michael Kolupaev 已提交
1271
		zkutil::Ops ops;
1272
		removePartFromZooKeeper(part->name, ops);
M
Merge  
Michael Kolupaev 已提交
1273 1274
		zookeeper->multi(ops);

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

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

A
Merge  
Alexey Milovidov 已提交
1283

M
Merge  
Michael Kolupaev 已提交
1284 1285
bool StorageReplicatedMergeTree::executeAttachPart(const StorageReplicatedMergeTree::LogEntry & entry)
{
A
Merge  
Alexey Milovidov 已提交
1286 1287
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
1288 1289 1290 1291
	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 已提交
1292
	if (!Poco::File(data.getFullPath() + source_path).exists())
M
Merge  
Michael Kolupaev 已提交
1293
	{
M
Merge  
Michael Kolupaev 已提交
1294
		LOG_INFO(log, "No part at " << source_path << ". Will fetch it instead");
M
Merge  
Michael Kolupaev 已提交
1295 1296 1297 1298 1299 1300 1301 1302 1303 1304 1305 1306 1307 1308 1309 1310 1311 1312 1313 1314 1315 1316 1317 1318 1319 1320 1321 1322
		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 已提交
1323

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

M
Merge  
Michael Kolupaev 已提交
1327 1328 1329
	return true;
}

A
Merge  
Alexey Milovidov 已提交
1330

M
Merge  
Michael Kolupaev 已提交
1331 1332 1333 1334
void StorageReplicatedMergeTree::queueUpdatingThread()
{
	while (!shutdown_called)
	{
M
Merge  
Michael Kolupaev 已提交
1335 1336
		try
		{
M
Merge  
Michael Kolupaev 已提交
1337 1338
			pullLogsToQueue(queue_updating_event);
			queue_updating_event->wait();
M
Merge  
Michael Kolupaev 已提交
1339
		}
1340
		catch (const zkutil::KeeperException & e)
M
Merge  
Michael Kolupaev 已提交
1341 1342
		{
			if (e.code == ZINVALIDSTATE)
1343
				restarting_thread->wakeup();
M
Merge  
Michael Kolupaev 已提交
1344 1345 1346 1347 1348

			tryLogCurrentException(__PRETTY_FUNCTION__);

			queue_updating_event->tryWait(ERROR_SLEEP_MS);
		}
M
Merge  
Michael Kolupaev 已提交
1349 1350 1351 1352
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);

M
Merge  
Michael Kolupaev 已提交
1353 1354
			queue_updating_event->tryWait(ERROR_SLEEP_MS);
		}
M
Merge  
Michael Kolupaev 已提交
1355
	}
M
Merge  
Michael Kolupaev 已提交
1356

1357
	LOG_DEBUG(log, "Queue updating thread finished");
M
Merge  
Michael Kolupaev 已提交
1358
}
M
Merge  
Michael Kolupaev 已提交
1359

A
Merge  
Alexey Milovidov 已提交
1360

M
Merge  
Michael Kolupaev 已提交
1361
bool StorageReplicatedMergeTree::queueTask(BackgroundProcessingPool::Context & pool_context)
M
Merge  
Michael Kolupaev 已提交
1362
{
M
Merge  
Michael Kolupaev 已提交
1363
	LogEntryPtr entry;
M
Merge  
Michael Kolupaev 已提交
1364

M
Merge  
Michael Kolupaev 已提交
1365 1366
	try
	{
1367
		std::lock_guard<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1368 1369
		bool empty = queue.empty();
		if (!empty)
M
Merge  
Michael Kolupaev 已提交
1370
		{
M
Merge  
Michael Kolupaev 已提交
1371
			for (LogEntries::iterator it = queue.begin(); it != queue.end(); ++it)
M
Merge  
Michael Kolupaev 已提交
1372
			{
M
Merge  
Michael Kolupaev 已提交
1373
				if (!(*it)->currently_executing && shouldExecuteLogEntry(**it))
M
Merge  
Michael Kolupaev 已提交
1374
				{
M
Merge  
Michael Kolupaev 已提交
1375
					entry = *it;
M
Merge  
Michael Kolupaev 已提交
1376
					entry->tagPartAsFuture(*this);
M
Merge  
Michael Kolupaev 已提交
1377
					queue.splice(queue.end(), queue, it);
M
Merge  
Michael Kolupaev 已提交
1378
					entry->currently_executing = true;
1379 1380
					++entry->num_tries;
					entry->last_attempt_time = time(0);
M
Merge  
Michael Kolupaev 已提交
1381
					break;
M
Merge  
Michael Kolupaev 已提交
1382
				}
M
Merge  
Michael Kolupaev 已提交
1383 1384
			}
		}
M
Merge  
Michael Kolupaev 已提交
1385 1386 1387 1388 1389
	}
	catch (...)
	{
		tryLogCurrentException(__PRETTY_FUNCTION__);
	}
M
Merge  
Michael Kolupaev 已提交
1390

M
Merge  
Michael Kolupaev 已提交
1391
	if (!entry)
M
Merge  
Michael Kolupaev 已提交
1392
		return false;
M
Merge  
Michael Kolupaev 已提交
1393

1394
	bool was_exception = true;
M
Merge  
Michael Kolupaev 已提交
1395
	bool success = false;
1396
	ExceptionPtr saved_exception;
M
Merge  
Michael Kolupaev 已提交
1397

M
Merge  
Michael Kolupaev 已提交
1398 1399
	try
	{
1400
		try
M
Merge  
Michael Kolupaev 已提交
1401
		{
1402 1403 1404 1405
			if (executeLogEntry(*entry, pool_context))
			{
				auto zookeeper = getZooKeeper();
				auto code = zookeeper->tryRemove(replica_path + "/queue/" + entry->znode_name);
M
Merge  
Michael Kolupaev 已提交
1406

1407 1408 1409
				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 已提交
1410

1411 1412 1413 1414 1415 1416 1417
				success = true;
			}
		}
		catch (...)
		{
			saved_exception = cloneCurrentException();
			throw;
M
Merge  
Michael Kolupaev 已提交
1418
		}
M
Merge  
Michael Kolupaev 已提交
1419

1420
		was_exception = false;
M
Merge  
Michael Kolupaev 已提交
1421
	}
1422
	catch (const Exception & e)
M
Merge  
Michael Kolupaev 已提交
1423 1424
	{
		if (e.code() == ErrorCodes::NO_REPLICA_HAS_PART)
1425
		{
M
Merge  
Michael Kolupaev 已提交
1426
			/// Если ни у кого нет нужного куска, наверно, просто не все реплики работают; не будем писать в лог с уровнем Error.
M
Merge  
Michael Kolupaev 已提交
1427
			LOG_INFO(log, e.displayText());
1428 1429 1430 1431 1432 1433
		}
		else if (e.code() == ErrorCodes::ABORTED)
		{
			/// Прерванный мердж - не ошибка.
			LOG_INFO(log, "Merge cancelled");
		}
M
Merge  
Michael Kolupaev 已提交
1434
		else
M
Merge  
Michael Kolupaev 已提交
1435
			tryLogCurrentException(__PRETTY_FUNCTION__);
M
Merge  
Michael Kolupaev 已提交
1436 1437 1438 1439 1440
	}
	catch (...)
	{
		tryLogCurrentException(__PRETTY_FUNCTION__);
	}
M
Merge  
Michael Kolupaev 已提交
1441

M
Merge  
Michael Kolupaev 已提交
1442 1443
	entry->future_part_tagger = nullptr;

1444
	std::lock_guard<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1445 1446

	entry->currently_executing = false;
1447
	entry->exception = saved_exception;
M
Merge  
Michael Kolupaev 已提交
1448 1449 1450
	entry->execution_complete.notify_all();

	if (success)
M
Merge  
Michael Kolupaev 已提交
1451
	{
M
Merge  
Michael Kolupaev 已提交
1452 1453 1454
		/// Удалим задание из очереди.
		/// Нельзя просто обратиться по заранее сохраненному итератору, потому что задание мог успеть удалить кто-то другой.
		for (LogEntries::iterator it = queue.end(); it != queue.begin();)
M
Merge  
Michael Kolupaev 已提交
1455
		{
M
Merge  
Michael Kolupaev 已提交
1456 1457 1458
			--it;
			if (*it == entry)
			{
M
Merge  
Michael Kolupaev 已提交
1459
				queue.erase(it);
M
Merge  
Michael Kolupaev 已提交
1460 1461
				break;
			}
M
Merge  
Michael Kolupaev 已提交
1462
		}
M
Merge  
Michael Kolupaev 已提交
1463
	}
M
Merge  
Michael Kolupaev 已提交
1464

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

A
Merge  
Alexey Milovidov 已提交
1469

M
Merge  
Michael Kolupaev 已提交
1470 1471
void StorageReplicatedMergeTree::mergeSelectingThread()
{
M
Merge  
Michael Kolupaev 已提交
1472
	bool need_pull = true;
M
Merge  
Michael Kolupaev 已提交
1473

A
Merge  
Alexey Milovidov 已提交
1474 1475 1476 1477 1478 1479 1480
	/** Может много времени тратиться на определение, можно ли мерджить два рядом стоящих куска.
	  * Два рядом стоящих куска можно мерджить, если все номера блоков между их номерами не используются ("заброшены", abandoned).
	  * Это значит, что между этими кусками не может быть вставлен другой кусок.
	  *
	  * Но если номера соседних блоков отличаются достаточно сильно (обычно, если между ними много "заброшенных" блоков),
	  *  то делается слишком много чтений из ZooKeeper, чтобы узнать, можно ли их мерджить.
	  *
1481 1482
	  * Воспользуемся утверждением, что если пару кусков было можно мерджить, и их мердж ещё не запланирован,
	  *  то и сейчас их можно мерджить, и будем запоминать это состояние, чтобы не делать много раз одинаковые запросы в ZooKeeper.
A
Merge  
Alexey Milovidov 已提交
1483 1484 1485 1486 1487 1488 1489 1490
	  *
	  * 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
	{
1491 1492 1493 1494 1495
		/// Если какой-то из кусков уже собираются слить в больший, не соглашаемся его сливать.
		if (virtual_parts.getContainingPart(left->name) != left->name ||
			virtual_parts.getContainingPart(right->name) != right->name)
			return false;

A
Merge  
Alexey Milovidov 已提交
1496 1497 1498 1499
		auto key = std::make_pair(left->name, right->name);
		if (memoized_parts_that_could_be_merged.count(key))
			return true;

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

1503 1504 1505 1506 1507 1508 1509 1510 1511 1512 1513 1514 1515 1516 1517 1518 1519 1520
		/// Нельзя сливать куски, среди которых находится кусок, для которого неудовлетворён кворум.
		/// Замечание: теоретически, это можно было бы разрешить. Но это сделает логику более сложной.
		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;
		}

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

1528 1529 1530 1531 1532 1533 1534
			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 已提交
1535 1536
	};

M
Merge  
Michael Kolupaev 已提交
1537 1538
	while (!shutdown_called && is_leader_node)
	{
M
Michael Kolupaev 已提交
1539
		bool success = false;
M
Merge  
Michael Kolupaev 已提交
1540

M
Michael Kolupaev 已提交
1541
		try
M
Merge  
Michael Kolupaev 已提交
1542
		{
1543
			std::lock_guard<std::mutex> merge_selecting_lock(merge_selecting_mutex);
M
Merge  
Michael Kolupaev 已提交
1544

M
Merge  
Michael Kolupaev 已提交
1545 1546 1547 1548 1549 1550 1551 1552
			if (need_pull)
			{
				/// Нужно загрузить новую запись в очередь перед тем, как выбирать куски для слияния.
				///  (чтобы кусок добавился в virtual_parts).
				pullLogsToQueue();
				need_pull = false;
			}

1553 1554 1555 1556 1557 1558 1559
			/** Сколько в очереди или в фоновом потоке мерджей крупных кусков.
			  * Если их больше половины от размера пула потоков для мерджа, то можно мерджить только мелкие куски.
			  */
			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 已提交
1560
			size_t merges_queued = 0;
1561
			size_t big_merges_queued = 0;
M
Merge  
Michael Kolupaev 已提交
1562

1563
			if (big_merges_current < max_number_of_big_merges)
M
Michael Kolupaev 已提交
1564
			{
1565
				std::lock_guard<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1566

M
Michael Kolupaev 已提交
1567
				for (const auto & entry : queue)
M
Merge  
Michael Kolupaev 已提交
1568
				{
M
Merge  
Michael Kolupaev 已提交
1569
					if (entry->type == LogEntry::MERGE_PARTS)
M
Merge  
Michael Kolupaev 已提交
1570
					{
M
Michael Kolupaev 已提交
1571
						++merges_queued;
M
Merge  
Michael Kolupaev 已提交
1572

1573
						if (big_merges_current + big_merges_queued < max_number_of_big_merges)
M
Merge  
Michael Kolupaev 已提交
1574
						{
M
Merge  
Michael Kolupaev 已提交
1575
							for (const String & name : entry->parts_to_merge)
M
Merge  
Michael Kolupaev 已提交
1576
							{
M
Merge  
Michael Kolupaev 已提交
1577
								MergeTreeData::DataPartPtr part = data.getActiveContainingPart(name);
M
Merge  
Michael Kolupaev 已提交
1578 1579
								if (!part || part->name != name)
									continue;
1580

M
Merge  
Michael Kolupaev 已提交
1581
								if (part->size_in_bytes > data.settings.max_bytes_to_merge_parts_small)
M
Merge  
Michael Kolupaev 已提交
1582
								{
1583
									++big_merges_queued;
M
Merge  
Michael Kolupaev 已提交
1584 1585 1586 1587 1588 1589
									break;
								}
							}
						}
					}
				}
M
Michael Kolupaev 已提交
1590
			}
M
Merge  
Michael Kolupaev 已提交
1591

1592 1593
			bool only_small = big_merges_current + big_merges_queued >= max_number_of_big_merges;

1594 1595 1596 1597 1598 1599
			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." : "."));
1600

M
Merge  
Michael Kolupaev 已提交
1601
			do
M
Michael Kolupaev 已提交
1602
			{
A
Merge  
Alexey Milovidov 已提交
1603 1604
				auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
1605
				if (merges_queued >= data.settings.max_replicated_merges_in_queue)
1606
				{
1607 1608 1609
					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 已提交
1610
					break;
1611
				}
M
Merge  
Michael Kolupaev 已提交
1612

M
Merge  
Michael Kolupaev 已提交
1613
				MergeTreeData::DataPartsVector parts;
M
Merge  
Michael Kolupaev 已提交
1614

M
Merge  
Michael Kolupaev 已提交
1615 1616
				String merged_name;

A
Merge  
Alexey Milovidov 已提交
1617 1618 1619 1620
				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))
1621
				{
M
Merge  
Michael Kolupaev 已提交
1622
					break;
1623
				}
M
Merge  
Michael Kolupaev 已提交
1624

M
Merge  
Michael Kolupaev 已提交
1625 1626 1627 1628 1629 1630 1631
				bool all_in_zk = true;
				for (const auto & part : parts)
				{
					/// Если о каком-то из кусков нет информации в ZK, не будем сливать.
					if (!zookeeper->exists(replica_path + "/parts/" + part->name))
					{
						all_in_zk = false;
1632 1633 1634 1635 1636 1637 1638 1639 1640

						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 已提交
1641 1642 1643 1644 1645
					}
				}
				if (!all_in_zk)
					break;

M
Merge  
Michael Kolupaev 已提交
1646 1647 1648 1649
				LogEntry entry;
				entry.type = LogEntry::MERGE_PARTS;
				entry.source_replica = replica_name;
				entry.new_part_name = merged_name;
M
Merge  
Michael Kolupaev 已提交
1650

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

M
Merge  
Michael Kolupaev 已提交
1654
				need_pull = true;
M
Merge  
Michael Kolupaev 已提交
1655

M
Merge  
Michael Kolupaev 已提交
1656
				zookeeper->create(zookeeper_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential);
M
Merge  
Michael Kolupaev 已提交
1657 1658 1659

				String month_name = parts[0]->name.substr(0, 6);
				for (size_t i = 0; i + 1 < parts.size(); ++i)
M
Merge  
Michael Kolupaev 已提交
1660
				{
M
Merge  
Michael Kolupaev 已提交
1661
					/// Уберем больше не нужные отметки о несуществующих блоках.
A
Merge  
Alexey Milovidov 已提交
1662
					for (Int64 number = std::max(RESERVED_BLOCK_NUMBERS, parts[i]->right + 1); number <= parts[i + 1]->left - 1; ++number)
M
Merge  
Michael Kolupaev 已提交
1663
					{
M
Merge  
Michael Kolupaev 已提交
1664 1665
						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 已提交
1666
					}
M
Merge  
Michael Kolupaev 已提交
1667
				}
M
Merge  
Michael Kolupaev 已提交
1668 1669

				success = true;
M
Merge  
Michael Kolupaev 已提交
1670
			}
1671
			while (false);
M
Merge  
Michael Kolupaev 已提交
1672 1673 1674 1675 1676 1677
		}
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);
		}

M
Merge  
Michael Kolupaev 已提交
1678
		if (shutdown_called || !is_leader_node)
M
Merge  
Michael Kolupaev 已提交
1679 1680
			break;

M
Merge  
Michael Kolupaev 已提交
1681
		if (!success)
M
Merge  
Michael Kolupaev 已提交
1682
			merge_selecting_event.tryWait(MERGE_SELECTING_SLEEP_MS);
M
Merge  
Michael Kolupaev 已提交
1683
	}
M
Merge  
Michael Kolupaev 已提交
1684

1685
	LOG_DEBUG(log, "Merge selecting thread finished");
M
Merge  
Michael Kolupaev 已提交
1686 1687
}

M
Merge  
Michael Kolupaev 已提交
1688

M
Merge  
Michael Kolupaev 已提交
1689 1690 1691 1692 1693 1694 1695 1696
void StorageReplicatedMergeTree::alterThread()
{
	bool force_recheck_parts = true;

	while (!shutdown_called)
	{
		try
		{
1697 1698 1699 1700 1701 1702 1703 1704 1705 1706 1707 1708 1709 1710 1711 1712 1713 1714 1715 1716 1717 1718 1719
			/** Имеем описание столбцов в 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 已提交
1720 1721
			auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
1722
			zkutil::Stat stat;
1723
			const String columns_str = zookeeper->get(zookeeper_path + "/columns", &stat, alter_thread_event);
1724
			auto columns_desc = ColumnsDescription<true>::parse(columns_str);
1725 1726 1727 1728 1729

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

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

M
Merge  
Michael Kolupaev 已提交
1733 1734
			MergeTreeData::DataParts parts;

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

M
Merge  
Michael Kolupaev 已提交
1740
				auto table_lock = lockStructureForAlter();
1741

1742 1743 1744
				const auto columns_changed = columns != data.getColumnsListNonMaterialized();
				const auto materialized_columns_changed = materialized_columns != data.materialized_columns;
				const auto alias_columns_changed = alias_columns != data.alias_columns;
A
Merge  
Andrey Mironov 已提交
1745
				const auto column_defaults_changed = column_defaults != data.column_defaults;
1746 1747 1748

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

1752 1753 1754
					InterpreterAlterQuery::updateMetadata(database_name, table_name, columns,
						materialized_columns, alias_columns, column_defaults, context);

1755 1756 1757 1758 1759 1760 1761 1762 1763 1764 1765 1766 1767 1768 1769 1770 1771 1772 1773 1774 1775 1776 1777 1778 1779 1780
					if (columns_changed)
					{
						data.setColumnsList(columns);

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

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

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

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

M
Merge  
Michael Kolupaev 已提交
1781 1782 1783 1784
					LOG_INFO(log, "Applied changes to table.");
				}
				else
				{
1785
					LOG_INFO(log, "Columns version changed in ZooKeeper, but data wasn't changed. It's like cyclic ALTERs.");
M
Merge  
Michael Kolupaev 已提交
1786
				}
1787 1788 1789 1790 1791

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

				columns_version = stat.version;
M
Merge  
Michael Kolupaev 已提交
1792 1793 1794
			}

			/// Обновим куски.
1795
			if (changed_version || force_recheck_parts)
M
Merge  
Michael Kolupaev 已提交
1796
			{
1797 1798 1799
				auto table_lock = lockStructure(false);

				if (changed_version)
M
Merge  
Michael Kolupaev 已提交
1800 1801 1802 1803
					LOG_INFO(log, "ALTER-ing parts");

				int changed_parts = 0;

1804
				if (!changed_version)
M
Merge  
Michael Kolupaev 已提交
1805 1806
					parts = data.getDataParts();

1807 1808
				const auto columns_plus_materialized = data.getColumnsList();

M
Merge  
Michael Kolupaev 已提交
1809 1810 1811
				for (const MergeTreeData::DataPartPtr & part : parts)
				{
					/// Обновим кусок и запишем результат во временные файлы.
M
Merge  
Michael Kolupaev 已提交
1812 1813
					/// TODO: Можно пропускать проверку на слишком большие изменения, если в ZooKeeper есть, например,
					///  нода /flags/force_alter.
1814
					auto transaction = data.alterDataPart(part, columns_plus_materialized);
M
Merge  
Michael Kolupaev 已提交
1815 1816 1817 1818 1819 1820 1821 1822

					if (!transaction)
						continue;

					++changed_parts;

					/// Обновим метаданные куска в ZooKeeper.
					zkutil::Ops ops;
1823 1824 1825 1826
					ops.push_back(new zkutil::Op::SetData(
						replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1));
					ops.push_back(new zkutil::Op::SetData(
						replica_path + "/parts/" + part->name + "/checksums", transaction->getNewChecksums().toString(), -1));
M
Merge  
Michael Kolupaev 已提交
1827 1828 1829 1830 1831 1832
					zookeeper->multi(ops);

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

M
Merge  
Michael Kolupaev 已提交
1833 1834 1835 1836 1837 1838 1839
				/// То же самое для нереплицируемых данных.
				if (unreplicated_data)
				{
					parts = unreplicated_data->getDataParts();

					for (const MergeTreeData::DataPartPtr & part : parts)
					{
1840
						auto transaction = unreplicated_data->alterDataPart(part, columns_plus_materialized);
M
Merge  
Michael Kolupaev 已提交
1841 1842 1843 1844 1845 1846 1847 1848 1849

						if (!transaction)
							continue;

						++changed_parts;

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

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

1854 1855 1856 1857 1858 1859 1860 1861
				if (changed_version)
				{
					if (changed_parts != 0)
						LOG_INFO(log, "ALTER-ed " << changed_parts << " parts");
					else
						LOG_INFO(log, "No parts ALTER-ed");
				}

M
Merge  
Michael Kolupaev 已提交
1862 1863 1864
				force_recheck_parts = false;
			}

1865
			parts.clear();
M
Merge  
Michael Kolupaev 已提交
1866 1867 1868 1869 1870 1871 1872 1873 1874 1875 1876 1877
			alter_thread_event->wait();
		}
		catch (...)
		{
			tryLogCurrentException(__PRETTY_FUNCTION__);

			force_recheck_parts = true;

			alter_thread_event->tryWait(ERROR_SLEEP_MS);
		}
	}

1878
	LOG_DEBUG(log, "Alter thread finished");
M
Merge  
Michael Kolupaev 已提交
1879 1880
}

A
Merge  
Alexey Milovidov 已提交
1881

1882 1883 1884 1885 1886 1887 1888 1889 1890 1891
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 已提交
1892 1893
void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_name)
{
A
Merge  
Alexey Milovidov 已提交
1894 1895
	auto zookeeper = getZooKeeper();

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

M
Merge  
Michael Kolupaev 已提交
1898 1899
	LogEntryPtr log_entry = new LogEntry;
	log_entry->type = LogEntry::GET_PART;
1900
	log_entry->create_time = tryGetPartCreateTime(zookeeper, replica_path, part_name);
M
Merge  
Michael Kolupaev 已提交
1901 1902
	log_entry->source_replica = "";
	log_entry->new_part_name = part_name;
M
Merge  
Michael Kolupaev 已提交
1903 1904 1905

	zkutil::Ops ops;
	ops.push_back(new zkutil::Op::Create(
M
Merge  
Michael Kolupaev 已提交
1906
		replica_path + "/queue/queue-", log_entry->toString(), zookeeper->getDefaultACL(),
M
Merge  
Michael Kolupaev 已提交
1907
		zkutil::CreateMode::PersistentSequential));
1908 1909 1910

	removePartFromZooKeeper(part_name, ops);

M
Merge  
Michael Kolupaev 已提交
1911 1912 1913
	auto results = zookeeper->multi(ops);

	{
1914
		std::lock_guard<std::mutex> lock(queue_mutex);
M
Merge  
Michael Kolupaev 已提交
1915 1916

		String path_created = dynamic_cast<zkutil::Op::Create &>(ops[0]).getPathCreated();
M
Merge  
Michael Kolupaev 已提交
1917 1918
		log_entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
		log_entry->addResultToVirtualParts(*this);
M
Merge  
Michael Kolupaev 已提交
1919 1920 1921 1922
		queue.push_back(log_entry);
	}
}

A
Merge  
Alexey Milovidov 已提交
1923

M
Merge  
Michael Kolupaev 已提交
1924 1925
void StorageReplicatedMergeTree::enqueuePartForCheck(const String & name)
{
1926
	std::lock_guard<std::mutex> lock(parts_to_check_mutex);
M
Merge  
Michael Kolupaev 已提交
1927 1928 1929 1930 1931 1932 1933 1934

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

1935 1936

void StorageReplicatedMergeTree::searchForMissingPart(const String & part_name)
M
Merge  
Michael Kolupaev 已提交
1937
{
1938 1939 1940 1941 1942
	auto zookeeper = getZooKeeper();
	String part_path = replica_path + "/parts/" + part_name;

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

1948 1949 1950 1951 1952 1953 1954 1955 1956 1957 1958 1959 1960 1961 1962 1963 1964 1965 1966 1967 1968 1969 1970 1971 1972 1973 1974 1975 1976 1977 1978 1979
		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 已提交
1980
			{
1981 1982 1983
				found = true;
				LOG_WARNING(log, "Checker: Found part " << part_on_replica << " on " << replica);
				break;
M
Merge  
Michael Kolupaev 已提交
1984
			}
1985 1986

			if (ActiveDataPartSet::contains(part_name, part_on_replica))
M
Merge  
Michael Kolupaev 已提交
1987
			{
1988 1989 1990
				ActiveDataPartSet::Part part_on_replica_info;
				ActiveDataPartSet::parsePartName(part_on_replica, part_on_replica_info);

1991
				for (auto block_num = part_on_replica_info.left; block_num <= part_on_replica_info.right; ++block_num)
A
Alexey Milovidov 已提交
1992
					found_blocks.at(block_num - part_info.left) = 1;
M
Merge  
Michael Kolupaev 已提交
1993
			}
1994 1995 1996 1997
		}
		if (found)
			break;
	}
M
Merge  
Michael Kolupaev 已提交
1998

1999 2000 2001 2002 2003
	if (found)
	{
		/// На какой-то живой или мёртвой реплике есть нужный кусок или покрывающий его.
		return;
	}
M
Merge  
Michael Kolupaev 已提交
2004

2005 2006 2007
	size_t num_found_blocks = 0;
	for (auto found_block : found_blocks)
		num_found_blocks += (found_block == 1);
M
Merge  
Michael Kolupaev 已提交
2008

2009 2010 2011 2012 2013 2014
	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;
	}
2015

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

2019 2020 2021
	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 已提交
2022

2023
	ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);
2024

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

2030 2031 2032 2033 2034 2035 2036 2037 2038 2039 2040 2041
		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;
		}
	}
2042

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

2050 2051 2052 2053 2054 2055 2056 2057 2058 2059 2060 2061 2062 2063 2064 2065 2066 2067 2068 2069 2070 2071 2072 2073
	/** Такая ситуация возможна, если на всех репликах, где был кусок, он испортился.
		* Например, у реплики, которая только что его записала, отключили питание, и данные не записались из кеша на диск.
		*/
	LOG_ERROR(log, "Checker: Part " << part_name << " is lost forever.");
	ProfileEvents::increment(ProfileEvents::ReplicatedDataLoss);

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


2076 2077 2078 2079
void StorageReplicatedMergeTree::checkPart(const String & part_name)
{
	LOG_WARNING(log, "Checker: Checking part " << part_name);
	ProfileEvents::increment(ProfileEvents::ReplicatedPartChecks);
M
Merge  
Michael Kolupaev 已提交
2080

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

2083 2084 2085 2086 2087 2088 2089 2090 2091 2092
	/// Этого или покрывающего куска у нас нет.
	if (!part)
	{
		searchForMissingPart(part_name);
	}
	/// У нас есть этот кусок, и он активен. Будем проверять, нужен ли нам этот кусок и правильные ли у него данные.
	else if (part->name == part_name)
	{
		auto zookeeper = getZooKeeper();
		auto table_lock = lockStructure(false);
M
Merge  
Michael Kolupaev 已提交
2093

2094 2095 2096 2097 2098 2099 2100 2101 2102 2103 2104 2105 2106 2107 2108 2109 2110 2111 2112 2113 2114 2115 2116 2117
		/// Если кусок есть в ZooKeeper, сверим его данные с его чексуммами, а их с ZooKeeper.
		if (zookeeper->exists(replica_path + "/parts/" + part_name))
		{
			LOG_WARNING(log, "Checker: Checking data of part " << part_name << ".");

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

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

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

				LOG_INFO(log, "Checker: Part " << part_name << " looks good.");
M
Merge  
Michael Kolupaev 已提交
2118
			}
2119
			catch (...)
M
Merge  
Michael Kolupaev 已提交
2120
			{
2121
				tryLogCurrentException(__PRETTY_FUNCTION__);
M
Merge  
Michael Kolupaev 已提交
2122

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

2126
				removePartAndEnqueueFetch(part_name);
M
Merge  
Michael Kolupaev 已提交
2127

2128 2129 2130 2131
				/// Удалим кусок локально.
				data.renameAndDetachPart(part, "broken_");
			}
		}
2132
		else if (part->modification_time + MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER < time(0))
2133 2134 2135 2136 2137 2138
		{
			/// Если куска нет в ZooKeeper, удалим его локально.
			/// Возможно, кусок кто-то только что записал, и еще не успел добавить в ZK.
			/// Поэтому удаляем только если кусок старый (не очень надежно).
			ProfileEvents::increment(ProfileEvents::ReplicatedPartChecksFailed);

2139
			LOG_ERROR(log, "Checker: Unexpected part " << part_name << " in filesystem. Removing.");
2140 2141
			data.renameAndDetachPart(part, "unexpected_");
		}
2142 2143 2144 2145 2146 2147
		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.");
		}
2148 2149 2150 2151 2152 2153 2154 2155
	}
	else
	{
		/// Если у нас есть покрывающий кусок, игнорируем все проблемы с этим куском.
		/// В худшем случае в лог еще old_parts_lifetime секунд будут валиться ошибки, пока кусок не удалится как старый.
		LOG_WARNING(log, "Checker: We have part " << part->name << " covering part " << part_name);
	}
}
M
Merge  
Michael Kolupaev 已提交
2156 2157


2158 2159 2160 2161 2162 2163 2164 2165 2166 2167 2168 2169 2170 2171 2172 2173
void StorageReplicatedMergeTree::partCheckThread()
{
	while (!shutdown_called)
	{
		try
		{
			/// Достанем из очереди кусок для проверки.
			String part_name;
			{
				std::lock_guard<std::mutex> lock(parts_to_check_mutex);
				if (parts_to_check_queue.empty())
				{
					if (!parts_to_check_set.empty())
					{
						LOG_ERROR(log, "Checker: Non-empty parts_to_check_set with empty parts_to_check_queue. This is a bug.");
						parts_to_check_set.clear();
M
Merge  
Michael Kolupaev 已提交
2174
					}
M
Merge  
Michael Kolupaev 已提交
2175
				}
2176
				else
M
Merge  
Michael Kolupaev 已提交
2177
				{
2178
					part_name = parts_to_check_queue.front();
M
Merge  
Michael Kolupaev 已提交
2179 2180
				}
			}
2181 2182

			if (part_name.empty())
M
Merge  
Michael Kolupaev 已提交
2183
			{
2184 2185
				parts_to_check_event.wait();
				continue;
M
Merge  
Michael Kolupaev 已提交
2186 2187
			}

2188 2189 2190
			checkPart(part_name);

			/// Удалим кусок из очереди проверок.
M
Merge  
Michael Kolupaev 已提交
2191
			{
2192
				std::lock_guard<std::mutex> lock(parts_to_check_mutex);
M
Merge  
Michael Kolupaev 已提交
2193 2194
				if (parts_to_check_queue.empty() || parts_to_check_queue.front() != part_name)
				{
2195
					LOG_ERROR(log, "Checker: Someone changed parts_to_check_queue.front(). This is a bug.");
M
Merge  
Michael Kolupaev 已提交
2196 2197 2198 2199 2200 2201 2202 2203 2204 2205 2206 2207 2208 2209 2210 2211 2212
				}
				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 已提交
2213 2214
void StorageReplicatedMergeTree::becomeLeader()
{
M
Merge  
Michael Kolupaev 已提交
2215
	LOG_INFO(log, "Became leader");
M
Merge  
Michael Kolupaev 已提交
2216 2217 2218 2219
	is_leader_node = true;
	merge_selecting_thread = std::thread(&StorageReplicatedMergeTree::mergeSelectingThread, this);
}

A
Merge  
Alexey Milovidov 已提交
2220

M
Merge  
Michael Kolupaev 已提交
2221
String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_name, bool active)
M
Merge  
Michael Kolupaev 已提交
2222
{
A
Merge  
Alexey Milovidov 已提交
2223
	auto zookeeper = getZooKeeper();
M
Merge  
Michael Kolupaev 已提交
2224
	Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
M
Merge  
Michael Kolupaev 已提交
2225 2226 2227 2228 2229 2230

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

	for (const String & replica : replicas)
	{
M
Merge  
Michael Kolupaev 已提交
2231 2232
		if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name) &&
			(!active || zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")))
M
Merge  
Michael Kolupaev 已提交
2233
			return replica;
2234 2235

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

M
Merge  
Michael Kolupaev 已提交
2238
	return "";
M
Merge  
Michael Kolupaev 已提交
2239 2240
}

A
Merge  
Alexey Milovidov 已提交
2241

2242 2243
/** Если для куска отслеживается кворум, то обновить информацию о нём в ZK.
  */
2244
void StorageReplicatedMergeTree::updateQuorum(const String & part_name)
2245
{
2246
	auto zookeeper = getZooKeeper();
2247

2248
	/// Информация, на какие реплики был добавлен кусок, если кворум ещё не достигнут.
2249
	const String quorum_status_path = zookeeper_path + "/quorum/status";
2250 2251 2252
	/// Имя предыдущего куска, для которого был достигнут кворум.
	const String quorum_last_part_path = zookeeper_path + "/quorum/last_part";

2253 2254 2255 2256 2257 2258 2259 2260 2261 2262 2263 2264 2265 2266 2267 2268 2269 2270 2271
	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)
		{
2272 2273 2274 2275 2276 2277
			/// Кворум достигнут. Удаляем узел, а также обновляем информацию о последнем куске, который был успешно записан с кворумом.

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

2279 2280 2281 2282 2283
			if (code == ZOK)
			{
				break;
			}
			else if (code == ZNONODE)
2284 2285 2286 2287 2288 2289 2290 2291 2292 2293 2294 2295 2296 2297 2298 2299 2300
			{
				/// Кворум уже был достигнут.
				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);

2301 2302 2303 2304 2305
			if (code == ZOK)
			{
				break;
			}
			else if (code == ZNONODE)
2306 2307 2308 2309 2310 2311 2312 2313 2314 2315 2316 2317 2318 2319 2320 2321 2322
			{
				/// Кворум уже был достигнут.
				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 已提交
2323
{
A
Merge  
Alexey Milovidov 已提交
2324 2325
	auto zookeeper = getZooKeeper();

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

2328 2329 2330
	TableStructureReadLockPtr table_lock;
	if (!to_detached)
		table_lock = lockStructure(true);
M
Merge  
Michael Kolupaev 已提交
2331 2332 2333 2334

	String host;
	int port;

2335
	String host_port_str = zookeeper->get(replica_path + "/host");
M
Merge  
Michael Kolupaev 已提交
2336 2337 2338 2339 2340 2341 2342 2343
	ReadBufferFromString buf(host_port_str);
	assertString("host: ", buf);
	readString(host, buf);
	assertString("\nport: ", buf);
	readText(port, buf);
	assertString("\n", buf);
	assertEOF(buf);

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

2346 2347 2348 2349
	if (!to_detached)
	{
		zkutil::Ops ops;
		checkPartAndAddToZooKeeper(part, ops, part_name);
M
Merge  
Michael Kolupaev 已提交
2350

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

2354 2355
		zookeeper->multi(ops);
		transaction.commit();
2356 2357

		/** Если для этого куска отслеживается кворум, то надо его обновить.
2358
		  * Если не успеем, в случае потери сессии, при перезапуске сервера - см. метод ReplicatedMergeTreeRestartingThread::updateQuorumIfWeHavePart.
2359
		  */
2360 2361
		if (quorum)
			updateQuorum(part_name);
2362

2363
		merge_selecting_event.set();
M
Merge  
Michael Kolupaev 已提交
2364

2365 2366 2367 2368 2369 2370 2371
		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 已提交
2372
	{
2373
		Poco::File(data.getFullPath() + "detached/tmp_" + part_name).renameTo(data.getFullPath() + "detached/" + part_name);
M
Michael Kolupaev 已提交
2374 2375
	}

M
Merge  
Michael Kolupaev 已提交
2376 2377
	ProfileEvents::increment(ProfileEvents::ReplicatedPartFetches);

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

A
Merge  
Alexey Milovidov 已提交
2381

M
Merge  
Michael Kolupaev 已提交
2382 2383
void StorageReplicatedMergeTree::shutdown()
{
2384
	if (restarting_thread)
M
Merge  
Michael Kolupaev 已提交
2385
	{
2386 2387
		restarting_thread->stop();
		restarting_thread.reset();
M
Merge  
Michael Kolupaev 已提交
2388
	}
M
Merge  
Michael Kolupaev 已提交
2389 2390

	endpoint_holder = nullptr;
M
Merge  
Michael Kolupaev 已提交
2391 2392 2393
}


M
Merge  
Michael Kolupaev 已提交
2394 2395 2396 2397 2398 2399 2400 2401
StorageReplicatedMergeTree::~StorageReplicatedMergeTree()
{
	try
	{
		shutdown();
	}
	catch(...)
	{
2402
		tryLogCurrentException(__PRETTY_FUNCTION__);
M
Merge  
Michael Kolupaev 已提交
2403 2404 2405
	}
}

A
Merge  
Alexey Milovidov 已提交
2406

M
Merge  
Michael Kolupaev 已提交
2407
BlockInputStreams StorageReplicatedMergeTree::read(
2408 2409 2410 2411 2412 2413 2414
	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 已提交
2415
{
2416 2417 2418 2419 2420 2421 2422 2423
	/** У таблицы может быть два вида данных:
	  * - реплицируемые данные;
	  * - старые, нереплицируемые данные - они лежат отдельно и их целостность никак не контролируется.
	  * А ещё движок таблицы предоставляет возможность использовать "виртуальные столбцы".
	  * Один из них - _replicated позволяет определить, из какой части прочитаны данные,
	  *  или, при использовании в WHERE - выбрать данные только из одной части.
	  */

M
Merge  
Michael Kolupaev 已提交
2424 2425 2426 2427 2428 2429 2430 2431
	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);

2432
	auto & select = typeid_cast<const ASTSelectQuery &>(*query);
A
Merge  
Andrey Mironov 已提交
2433 2434 2435 2436

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

M
Merge  
Michael Kolupaev 已提交
2439 2440 2441 2442 2443
	Block virtual_columns_block;
	ColumnUInt8 * column = new ColumnUInt8(2);
	ColumnPtr column_ptr = column;
	column->getData()[0] = 0;
	column->getData()[1] = 1;
2444
	virtual_columns_block.insert(ColumnWithTypeAndName(column_ptr, new DataTypeUInt8, "_replicated"));
M
Merge  
Michael Kolupaev 已提交
2445

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

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

M
Merge  
Michael Kolupaev 已提交
2452 2453
	BlockInputStreams res;

M
Merge  
Michael Kolupaev 已提交
2454 2455
	size_t part_index = 0;

2456 2457 2458 2459 2460 2461
	/** Настройки parallel_replica_offset и parallel_replicas_count позволяют читать с одной реплики одну часть данных, а с другой - другую.
	  * Для реплицируемых, данные разбиваются таким же механизмом, как работает секция SAMPLE.
	  * А для нереплицируемых данных, так как их целостность между репликами не контролируется,
	  *  с первой (settings.parallel_replica_offset == 0) реплики выбираются все данные, а с остальных - никакие.
	  */

2462
	if ((settings.parallel_replica_offset == 0) && unreplicated_reader && values.count(0))
M
Merge  
Michael Kolupaev 已提交
2463
	{
2464 2465
		res = unreplicated_reader->read(real_column_names, query,
										context, settings, processed_stage,
2466
										max_block_size, threads, &part_index, 0);
M
Merge  
Michael Kolupaev 已提交
2467 2468 2469 2470 2471 2472

		for (auto & virtual_column : virt_column_names)
		{
			if (virtual_column == "_replicated")
			{
				for (auto & stream : res)
M
Merge  
Michael Kolupaev 已提交
2473
					stream = new AddingConstColumnBlockInputStream<UInt8>(stream, new DataTypeUInt8, 0, "_replicated");
M
Merge  
Michael Kolupaev 已提交
2474 2475 2476 2477
			}
		}
	}

M
Merge  
Michael Kolupaev 已提交
2478
	if (values.count(1))
M
Merge  
Michael Kolupaev 已提交
2479
	{
2480 2481 2482 2483 2484 2485 2486 2487 2488 2489 2490 2491 2492 2493 2494 2495 2496 2497 2498 2499 2500 2501 2502 2503 2504 2505 2506 2507 2508 2509 2510 2511 2512 2513 2514 2515 2516 2517
		/** Настройка 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 已提交
2518 2519 2520 2521 2522 2523

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

M
Merge  
Michael Kolupaev 已提交
2528
		res.insert(res.end(), res2.begin(), res2.end());
M
Merge  
Michael Kolupaev 已提交
2529 2530 2531
	}

	return res;
M
Merge  
Michael Kolupaev 已提交
2532 2533
}

A
Merge  
Alexey Milovidov 已提交
2534

2535
BlockOutputStreamPtr StorageReplicatedMergeTree::write(ASTPtr query, const Settings & settings)
M
Merge  
Michael Kolupaev 已提交
2536
{
2537 2538
	if (is_readonly)
		throw Exception("Table is in readonly mode", ErrorCodes::TABLE_IS_READ_ONLY);
M
Merge  
Michael Kolupaev 已提交
2539

M
Merge  
Michael Kolupaev 已提交
2540
	String insert_id;
M
Merge  
Michael Kolupaev 已提交
2541 2542 2543
	if (query)
		if (ASTInsertQuery * insert = typeid_cast<ASTInsertQuery *>(&*query))
			insert_id = insert->insert_id;
M
Merge  
Michael Kolupaev 已提交
2544

2545
	return new ReplicatedMergeTreeBlockOutputStream(*this, insert_id, settings.insert_quorum);
M
Merge  
Michael Kolupaev 已提交
2546
}
M
Merge  
Michael Kolupaev 已提交
2547

A
Merge  
Alexey Milovidov 已提交
2548

2549
bool StorageReplicatedMergeTree::optimize(const Settings & settings)
M
Merge  
Michael Kolupaev 已提交
2550
{
2551 2552
	/// Померджим какие-нибудь куски из директории unreplicated.
	/// TODO: Мерджить реплицируемые куски тоже.
M
Merge  
Michael Kolupaev 已提交
2553 2554 2555 2556

	if (!unreplicated_data)
		return false;

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

M
Merge  
Michael Kolupaev 已提交
2559 2560 2561 2562
	unreplicated_data->clearOldParts();

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

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

M
Merge  
Michael Kolupaev 已提交
2570 2571 2572
	return true;
}

A
Merge  
Alexey Milovidov 已提交
2573

M
Merge  
Michael Kolupaev 已提交
2574 2575
void StorageReplicatedMergeTree::alter(const AlterCommands & params,
	const String & database_name, const String & table_name, Context & context)
M
Merge  
Michael Kolupaev 已提交
2576
{
A
Merge  
Alexey Milovidov 已提交
2577
	auto zookeeper = getZooKeeper();
2578 2579
	const MergeTreeMergeBlocker merge_blocker{merger};
	const auto unreplicated_merge_blocker = unreplicated_merger ?
2580
		std::make_unique<MergeTreeMergeBlocker>(*unreplicated_merger) : nullptr;
A
Merge  
Alexey Milovidov 已提交
2581

M
Merge  
Michael Kolupaev 已提交
2582 2583 2584
	LOG_DEBUG(log, "Doing ALTER");

	NamesAndTypesList new_columns;
2585 2586 2587
	NamesAndTypesList new_materialized_columns;
	NamesAndTypesList new_alias_columns;
	ColumnDefaults new_column_defaults;
M
Merge  
Michael Kolupaev 已提交
2588 2589 2590 2591 2592 2593 2594
	String new_columns_str;
	int new_columns_version;
	zkutil::Stat stat;

	{
		auto table_lock = lockStructureForAlter();

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

M
Merge  
Michael Kolupaev 已提交
2598 2599
		data.checkAlter(params);

2600 2601 2602 2603 2604
		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 已提交
2605

2606 2607 2608 2609
		new_columns_str = ColumnsDescription<false>{
			new_columns, new_materialized_columns,
			new_alias_columns, new_column_defaults
		}.toString();
M
Merge  
Michael Kolupaev 已提交
2610 2611 2612 2613 2614 2615 2616 2617 2618 2619 2620 2621 2622 2623

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

M
Merge  
Michael Kolupaev 已提交
2625 2626 2627 2628 2629 2630 2631 2632 2633 2634 2635 2636 2637 2638 2639 2640 2641 2642 2643 2644 2645 2646 2647 2648 2649 2650 2651 2652 2653 2654
	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);
2655

M
Merge  
Michael Kolupaev 已提交
2656 2657 2658 2659 2660 2661 2662
			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 已提交
2663
			if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/columns", &stat, alter_query_event))
M
Merge  
Michael Kolupaev 已提交
2664 2665 2666 2667 2668 2669 2670 2671 2672 2673 2674 2675 2676 2677 2678 2679 2680 2681
			{
				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 已提交
2682 2683 2684 2685 2686

/// Название воображаемого куска, покрывающего все возможные куски в указанном месяце с номерами в указанном диапазоне.
static String getFakePartNameForDrop(const String & month_name, UInt64 left, UInt64 right)
{
	/// Диапазон дат - весь месяц.
A
Merge  
Alexey Arno 已提交
2687
	const auto & lut = DateLUT::instance();
2688
	time_t start_time = lut.YYYYMMDDToDate(parse<UInt32>(month_name + "01"));
M
Merge  
Michael Kolupaev 已提交
2689 2690 2691 2692 2693 2694 2695
	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 已提交
2696

2697
void StorageReplicatedMergeTree::dropUnreplicatedPartition(const Field & partition, const bool detach, const Settings & settings)
M
Merge  
Michael Kolupaev 已提交
2698
{
A
Merge  
Andrey Mironov 已提交
2699 2700 2701 2702 2703 2704 2705 2706 2707 2708 2709 2710 2711 2712 2713
	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 已提交
2714
		if (part->month != month)
A
Merge  
Andrey Mironov 已提交
2715 2716 2717 2718 2719
			continue;

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

2720 2721 2722 2723
		if (detach)
			unreplicated_data->renameAndDetachPart(part, "");
		else
			unreplicated_data->replaceParts({part}, {}, false);
A
Merge  
Andrey Mironov 已提交
2724 2725
	}

2726
	LOG_INFO(log, (detach ? "Detached " : "Removed ") << removed_parts << " unreplicated parts inside " << apply_visitor(FieldVisitorToString(), partition) << ".");
A
Merge  
Andrey Mironov 已提交
2727 2728 2729 2730 2731 2732 2733
}


void StorageReplicatedMergeTree::dropPartition(const Field & field, bool detach, bool unreplicated, const Settings & settings)
{
	if (unreplicated)
	{
2734
		dropUnreplicatedPartition(field, detach, settings);
A
Merge  
Andrey Mironov 已提交
2735 2736 2737 2738

		return;
	}

A
Merge  
Alexey Milovidov 已提交
2739
	auto zookeeper = getZooKeeper();
A
Merge  
Alexey Milovidov 已提交
2740
	String month_name = MergeTreeData::getMonthName(field);
M
Merge  
Michael Kolupaev 已提交
2741 2742 2743

	/// TODO: Делать запрос в лидера по TCP.
	if (!is_leader_node)
2744
		throw Exception(String(detach ? "DETACH" : "DROP") + " PARTITION can only be done on leader replica.", ErrorCodes::NOT_LEADER);
M
Merge  
Michael Kolupaev 已提交
2745 2746 2747 2748 2749 2750 2751

	/** Пропустим один номер в block_numbers для удаляемого месяца, и будем удалять только куски до этого номера.
	  * Это запретит мерджи удаляемых кусков с новыми вставляемыми данными.
	  * Инвариант: в логе не появятся слияния удаляемых кусков с другими кусками.
	  * NOTE: Если понадобится аналогично поддержать запрос DROP PART, для него придется придумать какой-нибудь новый механизм,
	  *        чтобы гарантировать этот инвариант.
	  */
A
Merge  
Alexey Milovidov 已提交
2752
	Int64 right;
M
Merge  
Michael Kolupaev 已提交
2753 2754 2755 2756 2757 2758 2759 2760 2761 2762 2763 2764 2765 2766 2767 2768 2769 2770

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

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

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

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

		virtual_parts.add(fake_part_name);
	}

M
Merge  
Michael Kolupaev 已提交
2776
	/// Наконец, добившись нужных инвариантов, можно положить запись в лог.
M
Merge  
Michael Kolupaev 已提交
2777 2778 2779 2780 2781 2782
	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 已提交
2783
	entry.znode_name = log_znode_path.substr(log_znode_path.find_last_of('/') + 1);
M
Merge  
Michael Kolupaev 已提交
2784

2785 2786 2787 2788 2789 2790 2791 2792
	/// Если надо - дожидаемся выполнения операции на себе или на всех репликах.
	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 已提交
2793 2794
}

A
Merge  
Alexey Milovidov 已提交
2795

2796
void StorageReplicatedMergeTree::attachPartition(const Field & field, bool unreplicated, bool attach_part, const Settings & settings)
M
Merge  
Michael Kolupaev 已提交
2797
{
A
Merge  
Alexey Milovidov 已提交
2798
	auto zookeeper = getZooKeeper();
A
Merge  
Alexey Milovidov 已提交
2799
	String partition;
M
Merge  
Michael Kolupaev 已提交
2800

A
Merge  
Alexey Milovidov 已提交
2801
	if (attach_part)
A
Merge  
Alexey Milovidov 已提交
2802
		partition = field.safeGet<String>();
A
Merge  
Alexey Milovidov 已提交
2803 2804
	else
		partition = MergeTreeData::getMonthName(field);
M
Merge  
Michael Kolupaev 已提交
2805 2806 2807 2808 2809 2810 2811 2812 2813 2814 2815 2816 2817 2818 2819 2820 2821 2822

	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;
2823
			if (0 != name.compare(0, partition.size(), partition))
M
Merge  
Michael Kolupaev 已提交
2824 2825 2826 2827 2828 2829 2830 2831 2832 2833 2834 2835 2836 2837 2838 2839 2840 2841
				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 已提交
2842 2843
	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 已提交
2844 2845 2846 2847

	{
		auto existing_parts = data.getDataParts();
		for (const auto & part : existing_parts)
A
Merge  
Alexey Milovidov 已提交
2848 2849
			if (part->month == month)
				min_used_number = std::min(min_used_number, part->left);
M
Merge  
Michael Kolupaev 已提交
2850 2851 2852 2853 2854 2855 2856 2857 2858 2859 2860 2861 2862 2863 2864 2865 2866 2867 2868 2869 2870 2871 2872 2873 2874 2875 2876 2877
	}

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

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

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

	LOG_DEBUG(log, "Adding attaches to log");
	zookeeper->multi(ops);
2878

2879 2880
	/// Если надо - дожидаемся выполнения операции на себе или на всех репликах.
	if (settings.replication_alter_partitions_sync != 0)
M
Merge  
Michael Kolupaev 已提交
2881
	{
2882 2883 2884 2885 2886 2887 2888 2889 2890 2891
		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 已提交
2892

2893 2894
			++i;
		}
M
Merge  
Michael Kolupaev 已提交
2895
	}
M
Merge  
Michael Kolupaev 已提交
2896 2897
}

A
Merge  
Alexey Milovidov 已提交
2898

M
Merge  
Michael Kolupaev 已提交
2899 2900
void StorageReplicatedMergeTree::drop()
{
2901 2902
	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 已提交
2903

A
Merge  
Alexey Milovidov 已提交
2904 2905
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
2906 2907
	shutdown();

M
Merge  
Michael Kolupaev 已提交
2908
	LOG_INFO(log, "Removing replica " << replica_path);
M
Merge  
Michael Kolupaev 已提交
2909
	replica_is_active_node = nullptr;
M
Merge  
Michael Kolupaev 已提交
2910 2911
	zookeeper->tryRemoveRecursive(replica_path);

M
Merge  
Michael Kolupaev 已提交
2912
	/// Проверяем, что zookeeper_path существует: его могла удалить другая реплика после выполнения предыдущей строки.
M
Merge  
Michael Kolupaev 已提交
2913 2914
	Strings replicas;
	if (zookeeper->tryGetChildren(zookeeper_path + "/replicas", replicas) == ZOK && replicas.empty())
M
Merge  
Michael Kolupaev 已提交
2915 2916
	{
		LOG_INFO(log, "Removing table " << zookeeper_path << " (this might take several minutes)");
M
Merge  
Michael Kolupaev 已提交
2917
		zookeeper->tryRemoveRecursive(zookeeper_path);
M
Merge  
Michael Kolupaev 已提交
2918
	}
M
Merge  
Michael Kolupaev 已提交
2919 2920

	data.dropAllData();
M
Merge  
Michael Kolupaev 已提交
2921 2922
}

A
Merge  
Alexey Milovidov 已提交
2923

M
Merge  
Michael Kolupaev 已提交
2924 2925 2926 2927 2928 2929 2930 2931 2932 2933 2934 2935 2936 2937 2938
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 已提交
2939

M
Merge  
Michael Kolupaev 已提交
2940 2941
AbandonableLockInZooKeeper StorageReplicatedMergeTree::allocateBlockNumber(const String & month_name)
{
A
Merge  
Alexey Milovidov 已提交
2942 2943
	auto zookeeper = getZooKeeper();

M
Merge  
Michael Kolupaev 已提交
2944 2945 2946 2947 2948 2949 2950 2951
	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 已提交
2952
		for (size_t i = 0; i < RESERVED_BLOCK_NUMBERS; ++i)
M
Merge  
Michael Kolupaev 已提交
2953 2954 2955 2956 2957 2958 2959 2960 2961 2962 2963 2964 2965
		{
			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 已提交
2966

M
Merge  
Michael Kolupaev 已提交
2967
void StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry(const LogEntry & entry)
M
Merge  
Michael Kolupaev 已提交
2968
{
A
Merge  
Alexey Milovidov 已提交
2969
	auto zookeeper = getZooKeeper();
M
Merge  
Michael Kolupaev 已提交
2970 2971
	LOG_DEBUG(log, "Waiting for all replicas to process " << entry.znode_name);

A
Merge  
Alexey Milovidov 已提交
2972 2973 2974 2975 2976 2977 2978 2979 2980 2981
	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 已提交
2982 2983
	auto zookeeper = getZooKeeper();

2984 2985
	String entry_str = entry.toString();
	String log_node_name;
M
Merge  
Michael Kolupaev 已提交
2986

2987 2988 2989 2990 2991 2992 2993 2994 2995
	/** В эту функцию могут передать entry двух видов:
	  * 1. (более часто) Из директории log - общего лога, откуда реплики копируют записи в свою queue.
	  * 2. Из директории queue одной из реплик.
	  *
	  * Проблема в том, что номера (sequential нод) элементов очереди в log и в queue не совпадают.
	  * (И в queue не совпадают номера у одного и того же элемента лога для разных реплик.)
	  *
	  * Поэтому следует рассматривать эти случаи по-отдельности.
	  */
M
Merge  
Michael Kolupaev 已提交
2996

2997 2998 2999 3000 3001 3002 3003
	/** Первое - нужно дождаться, пока реплика возьмёт к себе в queue элемент очереди из log,
	  *  если она ещё этого не сделала (см. функцию pullLogsToQueue).
	  *
	  * Для этого проверяем её узел log_pointer - максимальный номер взятого элемента из log плюс единица.
	  */

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

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

3011 3012 3013 3014 3015 3016 3017 3018 3019 3020 3021 3022 3023
		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 已提交
3024
	}
3025 3026 3027 3028 3029 3030 3031
	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 已提交
3032

3033 3034 3035 3036 3037 3038 3039 3040 3041 3042 3043 3044 3045 3046 3047 3048 3049 3050 3051 3052 3053 3054 3055 3056 3057 3058 3059 3060 3061 3062 3063 3064 3065 3066 3067 3068 3069 3070 3071 3072 3073 3074 3075 3076 3077 3078 3079 3080 3081 3082
		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 已提交
3083

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

A
Merge  
Alexey Milovidov 已提交
3087 3088 3089 3090
	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);
3091
		if (exists && queue_entry_str == entry_str)
M
Merge  
Michael Kolupaev 已提交
3092
		{
3093
			queue_entry_to_wait_for = entry_name;
A
Merge  
Alexey Milovidov 已提交
3094
			break;
M
Merge  
Michael Kolupaev 已提交
3095
		}
A
Merge  
Alexey Milovidov 已提交
3096
	}
M
Merge  
Michael Kolupaev 已提交
3097

A
Merge  
Alexey Milovidov 已提交
3098
	/// Пока искали запись, ее уже выполнили и удалили.
3099 3100 3101
	if (queue_entry_to_wait_for.empty())
	{
		LOG_DEBUG(log, "No corresponding node found. Assuming it has been already processed.");
A
Merge  
Alexey Milovidov 已提交
3102
		return;
3103 3104 3105
	}

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

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


3112
void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields)
3113
{
A
Merge  
Alexey Milovidov 已提交
3114 3115
	auto zookeeper = getZooKeeper();

3116
	res.is_leader = is_leader_node;
3117
	res.is_readonly = is_readonly;
3118
	res.is_session_expired = !zookeeper || zookeeper->expired();
3119 3120 3121 3122 3123

	{
		std::lock_guard<std::mutex> lock(queue_mutex);
		res.future_parts = future_parts.size();
		res.queue_size = queue.size();
3124
		res.last_queue_update = last_queue_update;
3125 3126 3127

		res.inserts_in_queue = 0;
		res.merges_in_queue = 0;
A
Merge  
Alexey Milovidov 已提交
3128
		res.queue_oldest_time = 0;
3129 3130
		res.inserts_oldest_time = 0;
		res.merges_oldest_time = 0;
3131 3132 3133

		for (const LogEntryPtr & entry : queue)
		{
3134 3135 3136
			if (entry->create_time && (!res.queue_oldest_time || entry->create_time < res.queue_oldest_time))
				res.queue_oldest_time = entry->create_time;

3137
			if (entry->type == LogEntry::GET_PART)
3138
			{
3139
				++res.inserts_in_queue;
3140 3141

				if (entry->create_time && (!res.inserts_oldest_time || entry->create_time < res.inserts_oldest_time))
3142
				{
3143
					res.inserts_oldest_time = entry->create_time;
3144 3145
					res.oldest_part_to_get = entry->new_part_name;
				}
3146 3147
			}

3148
			if (entry->type == LogEntry::MERGE_PARTS)
3149
			{
3150
				++res.merges_in_queue;
A
Merge  
Alexey Milovidov 已提交
3151

3152
				if (entry->create_time && (!res.merges_oldest_time || entry->create_time < res.merges_oldest_time))
3153
				{
3154
					res.merges_oldest_time = entry->create_time;
3155 3156
					res.oldest_part_to_merge_to = entry->new_part_name;
				}
3157
			}
3158 3159 3160 3161 3162 3163 3164 3165 3166 3167 3168 3169 3170
		}
	}

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

3171
	if (res.is_session_expired || !with_zk_fields)
3172 3173 3174 3175 3176 3177 3178 3179 3180 3181
	{
		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");

3182 3183 3184 3185 3186 3187 3188 3189 3190 3191 3192 3193
		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);
3194 3195 3196 3197 3198 3199 3200 3201 3202 3203 3204

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

3205

3206
void StorageReplicatedMergeTree::fetchPartition(const Field & partition, const String & from_, const Settings & settings)
3207
{
A
Merge  
Alexey Milovidov 已提交
3208 3209
	auto zookeeper = getZooKeeper();

3210 3211
	String partition_str = MergeTreeData::getMonthName(partition);

3212
	String from = from_;
3213 3214
	if (from.back() == '/')
		from.resize(from.size() - 1);
3215

3216 3217
	LOG_INFO(log, "Will fetch partition " << partition_str << " from shard " << from_);

3218 3219 3220 3221 3222 3223 3224 3225
	/** Проверим, что в директории 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);

3226 3227 3228 3229 3230 3231 3232 3233 3234 3235 3236 3237
	/// Список реплик шарда-источника.
	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())
3238
		throw Exception("No active replicas for shard " + from, ErrorCodes::NO_ACTIVE_REPLICAS);
3239 3240

	/** Надо выбрать лучшую (наиболее актуальную) реплику.
3241
	  * Это реплика с максимальным log_pointer, затем с минимальным размером queue.
3242 3243
	  * NOTE Это не совсем лучший критерий. Для скачивания старых партиций это не имеет смысла,
	  *  и было бы неплохо уметь выбирать реплику, ближайшую по сети.
3244
	  * NOTE Разумеется, здесь есть data race-ы. Можно решить ретраями.
3245
	  */
3246 3247 3248 3249 3250 3251
	Int64 max_log_pointer = -1;
	UInt64 min_queue_size = std::numeric_limits<UInt64>::max();
	String best_replica;

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

3254
		String log_pointer_str = zookeeper->get(current_replica_path + "/log_pointer");
3255 3256 3257
		Int64 log_pointer = log_pointer_str.empty() ? 0 : parse<UInt64>(log_pointer_str);

		zkutil::Stat stat;
3258
		zookeeper->get(current_replica_path + "/queue", &stat);
3259 3260 3261 3262 3263 3264 3265 3266 3267 3268 3269 3270 3271 3272 3273 3274 3275
		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.");

3276 3277
	String best_replica_path = from + "/replicas/" + best_replica;

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

3280 3281 3282 3283 3284 3285 3286 3287 3288 3289 3290 3291 3292 3293 3294 3295 3296 3297 3298 3299 3300 3301 3302 3303 3304 3305 3306 3307 3308 3309
	/** Пытаемся скачать эти куски.
	  * Часть из них могла удалиться из-за мерджа.
	  * В этом случае, обновляем информацию о доступных кусках и пробуем снова.
	  */

	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);
3310 3311 3312

			if (parts_to_fetch.empty())
				throw Exception("Partition " + partition_str + " on " + best_replica_path + " doesn't exist", ErrorCodes::PARTITION_DOESNT_EXIST);
3313 3314 3315 3316 3317 3318 3319 3320 3321 3322 3323 3324 3325 3326 3327 3328 3329 3330 3331 3332
		}
		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
			{
3333
				fetchPart(part, best_replica_path, true, 0);
3334 3335 3336 3337 3338 3339 3340 3341 3342 3343
			}
			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);
			}
		}
3344

3345 3346
		++try_no;
	} while (!missing_parts.empty());
3347 3348 3349
}


3350 3351 3352 3353 3354 3355 3356 3357 3358 3359 3360 3361 3362
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 已提交
3363
}