2014-03-21 13:42:14 +00:00
|
|
|
|
#include <DB/Storages/StorageReplicatedMergeTree.h>
|
2014-04-02 10:10:37 +00:00
|
|
|
|
#include <DB/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h>
|
2014-04-02 13:45:39 +00:00
|
|
|
|
#include <DB/Storages/MergeTree/ReplicatedMergeTreePartsExchange.h>
|
2014-03-22 14:44:44 +00:00
|
|
|
|
#include <DB/Parsers/formatAST.h>
|
|
|
|
|
#include <DB/IO/WriteBufferFromOStream.h>
|
|
|
|
|
#include <DB/IO/ReadBufferFromString.h>
|
2014-05-13 10:10:26 +00:00
|
|
|
|
#include <time.h>
|
2014-03-21 13:42:14 +00:00
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
|
2014-04-03 11:48:28 +00:00
|
|
|
|
|
|
|
|
|
const auto QUEUE_UPDATE_SLEEP = std::chrono::seconds(5);
|
|
|
|
|
const auto QUEUE_NO_WORK_SLEEP = std::chrono::seconds(5);
|
|
|
|
|
const auto QUEUE_ERROR_SLEEP = std::chrono::seconds(1);
|
|
|
|
|
const auto QUEUE_AFTER_WORK_SLEEP = std::chrono::seconds(0);
|
2014-04-04 10:37:33 +00:00
|
|
|
|
const auto MERGE_SELECTING_SLEEP = std::chrono::seconds(5);
|
2014-04-03 11:48:28 +00:00
|
|
|
|
|
|
|
|
|
|
2014-03-21 13:42:14 +00:00
|
|
|
|
StorageReplicatedMergeTree::StorageReplicatedMergeTree(
|
|
|
|
|
const String & zookeeper_path_,
|
|
|
|
|
const String & replica_name_,
|
2014-03-21 19:17:59 +00:00
|
|
|
|
bool attach,
|
2014-05-08 07:12:01 +00:00
|
|
|
|
const String & path_, const String & database_name_, const String & name_,
|
|
|
|
|
NamesAndTypesListPtr columns_,
|
2014-03-22 14:44:44 +00:00
|
|
|
|
Context & context_,
|
2014-03-21 13:42:14 +00:00
|
|
|
|
ASTPtr & primary_expr_ast_,
|
|
|
|
|
const String & date_column_name_,
|
|
|
|
|
const ASTPtr & sampling_expression_,
|
|
|
|
|
size_t index_granularity_,
|
|
|
|
|
MergeTreeData::Mode mode_,
|
|
|
|
|
const String & sign_column_,
|
|
|
|
|
const MergeTreeSettings & settings_)
|
|
|
|
|
:
|
2014-03-22 14:44:44 +00:00
|
|
|
|
context(context_), zookeeper(context.getZooKeeper()),
|
2014-04-04 12:47:57 +00:00
|
|
|
|
table_name(name_), full_path(path_ + escapeForFileName(table_name) + '/'), zookeeper_path(zookeeper_path_),
|
2014-04-14 10:19:33 +00:00
|
|
|
|
replica_name(replica_name_),
|
2014-03-21 13:42:14 +00:00
|
|
|
|
data( full_path, columns_, context_, primary_expr_ast_, date_column_name_, sampling_expression_,
|
2014-05-08 07:12:01 +00:00
|
|
|
|
index_granularity_, mode_, sign_column_, settings_, database_name_ + "." + table_name),
|
2014-04-04 10:37:33 +00:00
|
|
|
|
reader(data), writer(data), merger(data), fetcher(data),
|
2014-05-08 07:12:01 +00:00
|
|
|
|
log(&Logger::get(database_name_ + "." + table_name + " (StorageReplicatedMergeTree)"))
|
2014-03-21 13:42:14 +00:00
|
|
|
|
{
|
2014-05-13 11:24:04 +00:00
|
|
|
|
if (!zookeeper)
|
|
|
|
|
{
|
|
|
|
|
goReadOnly();
|
|
|
|
|
return;
|
|
|
|
|
}
|
|
|
|
|
|
2014-03-22 14:44:44 +00:00
|
|
|
|
if (!zookeeper_path.empty() && *zookeeper_path.rbegin() == '/')
|
|
|
|
|
zookeeper_path.erase(zookeeper_path.end() - 1);
|
|
|
|
|
replica_path = zookeeper_path + "/replicas/" + replica_name;
|
2014-03-21 19:49:27 +00:00
|
|
|
|
|
2014-03-21 19:17:59 +00:00
|
|
|
|
if (!attach)
|
|
|
|
|
{
|
2014-05-13 10:10:26 +00:00
|
|
|
|
if (!zookeeper->exists(zookeeper_path))
|
2014-03-22 14:44:44 +00:00
|
|
|
|
createTable();
|
|
|
|
|
|
|
|
|
|
checkTableStructure();
|
|
|
|
|
createReplica();
|
2014-03-21 19:17:59 +00:00
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
checkTableStructure();
|
2014-04-02 07:59:43 +00:00
|
|
|
|
checkParts();
|
2014-03-21 19:17:59 +00:00
|
|
|
|
}
|
2014-03-22 14:44:44 +00:00
|
|
|
|
|
2014-05-26 11:40:22 +00:00
|
|
|
|
initVirtualParts();
|
2014-04-03 08:47:59 +00:00
|
|
|
|
loadQueue();
|
2014-04-24 10:20:02 +00:00
|
|
|
|
|
|
|
|
|
String unreplicated_path = full_path + "unreplicated/";
|
|
|
|
|
if (Poco::File(unreplicated_path).exists())
|
|
|
|
|
{
|
|
|
|
|
LOG_INFO(log, "Have unreplicated data");
|
|
|
|
|
unreplicated_data.reset(new MergeTreeData(unreplicated_path, columns_, context_, primary_expr_ast_,
|
2014-05-08 07:12:01 +00:00
|
|
|
|
date_column_name_, sampling_expression_, index_granularity_, mode_, sign_column_, settings_,
|
|
|
|
|
database_name_ + "." + table_name + "[unreplicated]"));
|
2014-04-24 10:20:02 +00:00
|
|
|
|
unreplicated_reader.reset(new MergeTreeDataSelectExecutor(*unreplicated_data));
|
2014-05-08 08:03:03 +00:00
|
|
|
|
unreplicated_merger.reset(new MergeTreeDataMerger(*unreplicated_data));
|
2014-04-24 10:20:02 +00:00
|
|
|
|
}
|
2014-04-25 13:55:15 +00:00
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
/// Сгенерируем этому экземпляру случайный идентификатор.
|
|
|
|
|
struct timespec times;
|
|
|
|
|
if (clock_gettime(CLOCK_THREAD_CPUTIME_ID, ×))
|
|
|
|
|
throwFromErrno("Cannot clock_gettime.", ErrorCodes::CANNOT_CLOCK_GETTIME);
|
|
|
|
|
active_node_identifier = toString(times.tv_nsec);
|
|
|
|
|
|
2014-04-25 13:55:15 +00:00
|
|
|
|
restarting_thread = std::thread(&StorageReplicatedMergeTree::restartingThread, this);
|
2014-03-21 13:42:14 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
StoragePtr StorageReplicatedMergeTree::create(
|
|
|
|
|
const String & zookeeper_path_,
|
|
|
|
|
const String & replica_name_,
|
2014-03-21 19:17:59 +00:00
|
|
|
|
bool attach,
|
2014-05-08 07:12:01 +00:00
|
|
|
|
const String & path_, const String & database_name_, const String & name_,
|
|
|
|
|
NamesAndTypesListPtr columns_,
|
2014-03-22 14:44:44 +00:00
|
|
|
|
Context & context_,
|
2014-03-21 13:42:14 +00:00
|
|
|
|
ASTPtr & primary_expr_ast_,
|
|
|
|
|
const String & date_column_name_,
|
|
|
|
|
const ASTPtr & sampling_expression_,
|
|
|
|
|
size_t index_granularity_,
|
|
|
|
|
MergeTreeData::Mode mode_,
|
|
|
|
|
const String & sign_column_,
|
|
|
|
|
const MergeTreeSettings & settings_)
|
|
|
|
|
{
|
2014-04-02 07:59:43 +00:00
|
|
|
|
StorageReplicatedMergeTree * res = new StorageReplicatedMergeTree(zookeeper_path_, replica_name_, attach,
|
2014-05-08 07:12:01 +00:00
|
|
|
|
path_, database_name_, name_, columns_, context_, primary_expr_ast_, date_column_name_, sampling_expression_,
|
2014-04-02 07:59:43 +00:00
|
|
|
|
index_granularity_, mode_, sign_column_, settings_);
|
|
|
|
|
StoragePtr res_ptr = res->thisPtr();
|
2014-05-13 11:24:04 +00:00
|
|
|
|
if (!res->is_read_only)
|
|
|
|
|
{
|
|
|
|
|
String endpoint_name = "ReplicatedMergeTree:" + res->replica_path;
|
|
|
|
|
InterserverIOEndpointPtr endpoint = new ReplicatedMergeTreePartsServer(res->data, res_ptr);
|
|
|
|
|
res->endpoint_holder = new InterserverIOEndpointHolder(endpoint_name, endpoint, res->context.getInterserverIOHandler());
|
|
|
|
|
}
|
2014-04-02 07:59:43 +00:00
|
|
|
|
return res_ptr;
|
2014-03-21 13:42:14 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-03-22 14:44:44 +00:00
|
|
|
|
static String formattedAST(const ASTPtr & ast)
|
|
|
|
|
{
|
|
|
|
|
if (!ast)
|
|
|
|
|
return "";
|
|
|
|
|
std::stringstream ss;
|
|
|
|
|
formatAST(*ast, ss, 0, false, true);
|
|
|
|
|
return ss.str();
|
|
|
|
|
}
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-03-22 14:44:44 +00:00
|
|
|
|
void StorageReplicatedMergeTree::createTable()
|
2014-03-21 19:17:59 +00:00
|
|
|
|
{
|
2014-05-26 18:27:25 +00:00
|
|
|
|
LOG_DEBUG(log, "Creating table " << zookeeper_path);
|
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper->create(zookeeper_path, "", zkutil::CreateMode::Persistent);
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-03-22 14:44:44 +00:00
|
|
|
|
/// Запишем метаданные таблицы, чтобы реплики могли сверять с ними свою локальную структуру таблицы.
|
|
|
|
|
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;
|
|
|
|
|
metadata << "columns:" << std::endl;
|
|
|
|
|
WriteBufferFromOStream buf(metadata);
|
|
|
|
|
for (auto & it : data.getColumnsList())
|
|
|
|
|
{
|
|
|
|
|
writeBackQuotedString(it.first, buf);
|
|
|
|
|
writeChar(' ', buf);
|
|
|
|
|
writeString(it.second->getName(), buf);
|
|
|
|
|
writeChar('\n', buf);
|
|
|
|
|
}
|
|
|
|
|
buf.next();
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper->create(zookeeper_path + "/metadata", metadata.str(), zkutil::CreateMode::Persistent);
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper->create(zookeeper_path + "/replicas", "", zkutil::CreateMode::Persistent);
|
|
|
|
|
zookeeper->create(zookeeper_path + "/blocks", "", zkutil::CreateMode::Persistent);
|
|
|
|
|
zookeeper->create(zookeeper_path + "/block_numbers", "", zkutil::CreateMode::Persistent);
|
|
|
|
|
zookeeper->create(zookeeper_path + "/leader_election", "", zkutil::CreateMode::Persistent);
|
|
|
|
|
zookeeper->create(zookeeper_path + "/temp", "", zkutil::CreateMode::Persistent);
|
2014-03-22 14:44:44 +00:00
|
|
|
|
}
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-03-22 14:44:44 +00:00
|
|
|
|
/** Проверить, что список столбцов и настройки таблицы совпадают с указанными в ZK (/metadata).
|
|
|
|
|
* Если нет - бросить исключение.
|
2014-03-21 19:17:59 +00:00
|
|
|
|
*/
|
2014-03-22 14:44:44 +00:00
|
|
|
|
void StorageReplicatedMergeTree::checkTableStructure()
|
|
|
|
|
{
|
2014-05-13 10:10:26 +00:00
|
|
|
|
String metadata_str = zookeeper->get(zookeeper_path + "/metadata");
|
2014-03-22 14:44:44 +00:00
|
|
|
|
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);
|
|
|
|
|
assertString(formattedAST(data.primary_expr_ast), buf);
|
|
|
|
|
assertString("\ncolumns:\n", buf);
|
|
|
|
|
for (auto & it : data.getColumnsList())
|
|
|
|
|
{
|
|
|
|
|
String name;
|
|
|
|
|
readBackQuotedString(name, buf);
|
|
|
|
|
if (name != it.first)
|
|
|
|
|
throw Exception("Unexpected column name in ZooKeeper: expected " + it.first + ", found " + name,
|
|
|
|
|
ErrorCodes::UNKNOWN_IDENTIFIER);
|
|
|
|
|
assertString(" ", buf);
|
|
|
|
|
assertString(it.second->getName(), buf);
|
|
|
|
|
assertString("\n", buf);
|
|
|
|
|
}
|
2014-04-02 07:59:43 +00:00
|
|
|
|
assertEOF(buf);
|
2014-03-22 14:44:44 +00:00
|
|
|
|
}
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-03-22 14:44:44 +00:00
|
|
|
|
void StorageReplicatedMergeTree::createReplica()
|
|
|
|
|
{
|
2014-05-26 18:29:29 +00:00
|
|
|
|
LOG_DEBUG(log, "Creating replica " << replica_path);
|
2014-05-26 18:27:25 +00:00
|
|
|
|
|
2014-05-26 18:14:52 +00:00
|
|
|
|
/** Запомним список других реплик.
|
|
|
|
|
* NOTE: Здесь есть race condition. Если почти одновременно добавить нескольких реплик, сразу же начиная в них писать,
|
|
|
|
|
* небольшая часть данных может не реплицироваться.
|
|
|
|
|
*/
|
|
|
|
|
Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
|
|
|
|
|
|
|
|
|
|
/// Создадим пустую реплику.
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper->create(replica_path, "", zkutil::CreateMode::Persistent);
|
|
|
|
|
zookeeper->create(replica_path + "/host", "", zkutil::CreateMode::Persistent);
|
|
|
|
|
zookeeper->create(replica_path + "/log", "", zkutil::CreateMode::Persistent);
|
|
|
|
|
zookeeper->create(replica_path + "/log_pointers", "", zkutil::CreateMode::Persistent);
|
|
|
|
|
zookeeper->create(replica_path + "/queue", "", zkutil::CreateMode::Persistent);
|
|
|
|
|
zookeeper->create(replica_path + "/parts", "", zkutil::CreateMode::Persistent);
|
2014-05-26 18:14:52 +00:00
|
|
|
|
|
|
|
|
|
/// Если таблица пуста, больше ничего делать не нужно.
|
|
|
|
|
if (replicas.empty())
|
2014-05-26 18:27:25 +00:00
|
|
|
|
{
|
|
|
|
|
LOG_DEBUG(log, "No other replicas");
|
2014-05-26 18:14:52 +00:00
|
|
|
|
return;
|
2014-05-26 18:27:25 +00:00
|
|
|
|
}
|
2014-05-26 18:14:52 +00:00
|
|
|
|
|
|
|
|
|
/// "Эталонная" реплика, у которой мы возьмем информацию о множестве кусков, очередь и указатели на логи.
|
|
|
|
|
String source_replica = replicas[0];
|
|
|
|
|
|
|
|
|
|
/** Дождемся, пока все активные реплики заметят появление этой реплики.
|
|
|
|
|
* Это не даст им удалять записи из своих логов, пока эта реплика их не скопирует.
|
|
|
|
|
*/
|
|
|
|
|
for (const String & replica : replicas)
|
|
|
|
|
{
|
2014-05-26 18:27:25 +00:00
|
|
|
|
LOG_DEBUG(log, "Waiting for " << replica << " to acknowledge me");
|
|
|
|
|
|
2014-05-26 18:14:52 +00:00
|
|
|
|
bool active = true;
|
|
|
|
|
while(true)
|
|
|
|
|
{
|
|
|
|
|
if (!zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active"))
|
|
|
|
|
{
|
|
|
|
|
active = false;
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/log_pointers/" + replica_name))
|
|
|
|
|
break;
|
|
|
|
|
std::this_thread::sleep_for(std::chrono::seconds(1));
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Будем предпочитать активную реплику в качестве эталонной.
|
|
|
|
|
if (active)
|
|
|
|
|
source_replica = replica;
|
|
|
|
|
}
|
|
|
|
|
|
2014-05-26 18:27:25 +00:00
|
|
|
|
LOG_INFO(log, "Will mimic " << source_replica);
|
|
|
|
|
|
2014-05-26 18:14:52 +00:00
|
|
|
|
String source_path = zookeeper_path + "/replicas/" + source_replica;
|
|
|
|
|
|
|
|
|
|
/// Порядок следующих трех действий важен. Записи в логе могут продублироваться, но не могут потеряться.
|
|
|
|
|
|
|
|
|
|
/// Скопируем у эталонной реплики ссылки на все логи.
|
|
|
|
|
for (const String & replica : replicas)
|
|
|
|
|
{
|
|
|
|
|
String pointer = zookeeper->get(source_path + "/log_pointers/" + replica);
|
|
|
|
|
zookeeper->create(replica_path + "/log_pointers/" + replica, pointer, zkutil::CreateMode::Persistent);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Запомним очередь эталонной реплики.
|
|
|
|
|
Strings source_queue_names = zookeeper->getChildren(source_path + "/queue");
|
|
|
|
|
std::sort(source_queue_names.begin(), source_queue_names.end());
|
|
|
|
|
Strings source_queue;
|
|
|
|
|
for (const String & entry_name : source_queue_names)
|
|
|
|
|
{
|
|
|
|
|
String entry;
|
|
|
|
|
if (!zookeeper->tryGet(source_path + "/queue/" + entry_name, entry))
|
|
|
|
|
continue;
|
|
|
|
|
source_queue.push_back(entry);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Добавим в очередь задания на получение всех активных кусков, которые есть у эталонной реплики.
|
|
|
|
|
Strings parts = zookeeper->getChildren(source_path + "/parts");
|
|
|
|
|
ActiveDataPartSet active_parts_set;
|
|
|
|
|
for (const String & part : parts)
|
|
|
|
|
{
|
|
|
|
|
active_parts_set.add(part);
|
|
|
|
|
}
|
|
|
|
|
Strings active_parts = active_parts_set.getParts();
|
|
|
|
|
for (const String & name : active_parts)
|
|
|
|
|
{
|
|
|
|
|
LogEntry log_entry;
|
|
|
|
|
log_entry.type = LogEntry::GET_PART;
|
|
|
|
|
log_entry.source_replica = "";
|
|
|
|
|
log_entry.new_part_name = name;
|
|
|
|
|
|
|
|
|
|
zookeeper->create(replica_path + "/queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential);
|
|
|
|
|
}
|
2014-05-26 18:27:25 +00:00
|
|
|
|
LOG_DEBUG(log, "Queued " << active_parts.size() << " parts to be fetched");
|
2014-05-26 18:14:52 +00:00
|
|
|
|
|
|
|
|
|
/// Добавим в очередь содержимое очереди эталонной реплики.
|
|
|
|
|
for (const String & entry : source_queue)
|
|
|
|
|
{
|
|
|
|
|
zookeeper->create(replica_path + "/queue/queue-", entry, zkutil::CreateMode::PersistentSequential);
|
|
|
|
|
}
|
2014-05-26 18:27:25 +00:00
|
|
|
|
LOG_DEBUG(log, "Copied " << source_queue.size() << " queue entries");
|
2014-03-22 14:44:44 +00:00
|
|
|
|
}
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-03-22 14:44:44 +00:00
|
|
|
|
void StorageReplicatedMergeTree::activateReplica()
|
|
|
|
|
{
|
|
|
|
|
std::stringstream host;
|
|
|
|
|
host << "host: " << context.getInterserverIOHost() << std::endl;
|
|
|
|
|
host << "port: " << context.getInterserverIOPort() << std::endl;
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
/** Если нода отмечена как активная, но отметка сделана в этом же экземпляре, удалим ее.
|
|
|
|
|
* Такое возможно только при истечении сессии в ZooKeeper.
|
|
|
|
|
* Здесь есть небольшой race condition (можем удалить не ту ноду, для которой сделали tryGet),
|
|
|
|
|
* но он крайне маловероятен при нормальном использовании.
|
|
|
|
|
*/
|
|
|
|
|
String data;
|
|
|
|
|
if (zookeeper->tryGet(replica_path + "/is_active", data) && data == active_node_identifier)
|
|
|
|
|
zookeeper->tryRemove(replica_path + "/is_active");
|
|
|
|
|
|
2014-04-03 08:47:59 +00:00
|
|
|
|
/// Одновременно объявим, что эта реплика активна, и обновим хост.
|
2014-03-22 14:44:44 +00:00
|
|
|
|
zkutil::Ops ops;
|
2014-05-13 10:10:26 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::Create(replica_path + "/is_active", "", zookeeper->getDefaultACL(), zkutil::CreateMode::Ephemeral));
|
2014-03-22 14:44:44 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::SetData(replica_path + "/host", host.str(), -1));
|
2014-04-03 13:11:11 +00:00
|
|
|
|
|
|
|
|
|
try
|
|
|
|
|
{
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper->multi(ops);
|
2014-04-03 13:11:11 +00:00
|
|
|
|
}
|
|
|
|
|
catch (zkutil::KeeperException & e)
|
|
|
|
|
{
|
2014-06-04 13:49:22 +00:00
|
|
|
|
if (e.code == ZNODEEXISTS)
|
2014-04-03 13:11:11 +00:00
|
|
|
|
throw Exception("Replica " + replica_path + " appears to be already active. If you're sure it's not, "
|
|
|
|
|
"try again in a minute or remove znode " + replica_path + "/is_active manually", ErrorCodes::REPLICA_IS_ALREADY_ACTIVE);
|
|
|
|
|
|
|
|
|
|
throw;
|
|
|
|
|
}
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
replica_is_active_node = zkutil::EphemeralNodeHolder::existing(replica_path + "/is_active", *zookeeper);
|
2014-03-22 14:44:44 +00:00
|
|
|
|
}
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-04-02 07:59:43 +00:00
|
|
|
|
void StorageReplicatedMergeTree::checkParts()
|
|
|
|
|
{
|
2014-05-13 10:10:26 +00:00
|
|
|
|
Strings expected_parts_vec = zookeeper->getChildren(replica_path + "/parts");
|
2014-05-22 10:37:17 +00:00
|
|
|
|
|
|
|
|
|
/// Куски в ZK.
|
2014-04-02 07:59:43 +00:00
|
|
|
|
NameSet expected_parts(expected_parts_vec.begin(), expected_parts_vec.end());
|
|
|
|
|
|
2014-04-09 16:32:32 +00:00
|
|
|
|
MergeTreeData::DataParts parts = data.getAllDataParts();
|
2014-04-02 07:59:43 +00:00
|
|
|
|
|
2014-05-22 10:37:17 +00:00
|
|
|
|
/// Локальные куски, которых нет в ZK.
|
2014-04-08 17:45:21 +00:00
|
|
|
|
MergeTreeData::DataParts unexpected_parts;
|
2014-05-22 10:37:17 +00:00
|
|
|
|
|
2014-04-02 07:59:43 +00:00
|
|
|
|
for (const auto & part : parts)
|
|
|
|
|
{
|
|
|
|
|
if (expected_parts.count(part->name))
|
|
|
|
|
{
|
|
|
|
|
expected_parts.erase(part->name);
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
2014-04-08 17:45:21 +00:00
|
|
|
|
unexpected_parts.insert(part);
|
2014-04-02 07:59:43 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2014-05-22 10:37:17 +00:00
|
|
|
|
/// Какие локальные куски добавить в ZK.
|
2014-04-08 17:45:21 +00:00
|
|
|
|
MergeTreeData::DataPartsVector parts_to_add;
|
2014-05-22 10:37:17 +00:00
|
|
|
|
|
|
|
|
|
/// Какие куски нужно забрать с других реплик.
|
|
|
|
|
Strings parts_to_fetch;
|
|
|
|
|
|
2014-04-08 17:45:21 +00:00
|
|
|
|
for (const String & missing_name : expected_parts)
|
|
|
|
|
{
|
2014-05-22 10:37:17 +00:00
|
|
|
|
/// Если локально не хватает какого-то куска, но есть покрывающий его кусок, можно заменить в ZK недостающий покрывающим.
|
2014-04-08 17:45:21 +00:00
|
|
|
|
auto containing = data.getContainingPart(missing_name);
|
2014-05-22 10:37:17 +00:00
|
|
|
|
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
|
2014-04-08 17:45:21 +00:00
|
|
|
|
{
|
2014-05-22 10:37:17 +00:00
|
|
|
|
parts_to_fetch.push_back(missing_name);
|
2014-04-08 17:45:21 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
2014-04-02 07:59:43 +00:00
|
|
|
|
|
2014-05-22 10:37:17 +00:00
|
|
|
|
for (const String & name : parts_to_fetch)
|
|
|
|
|
expected_parts.erase(name);
|
|
|
|
|
|
|
|
|
|
bool skip_sanity_check = false;
|
|
|
|
|
if (zookeeper->exists(replica_path + "/flags/force_restore_data"))
|
|
|
|
|
{
|
|
|
|
|
skip_sanity_check = true;
|
|
|
|
|
zookeeper->remove(replica_path + "/flags/force_restore_data");
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
String sanity_report =
|
|
|
|
|
"There are " + toString(unexpected_parts.size()) + " unexpected parts, "
|
|
|
|
|
+ toString(parts_to_add.size()) + " unexpectedly merged parts, "
|
|
|
|
|
+ toString(expected_parts.size()) + " unexpectedly obsolete parts, "
|
|
|
|
|
+ toString(parts_to_fetch.size()) + " missing parts";
|
|
|
|
|
bool insane =
|
|
|
|
|
parts_to_add.size() > 2 ||
|
2014-04-08 17:45:21 +00:00
|
|
|
|
unexpected_parts.size() > 2 ||
|
2014-05-22 10:37:17 +00:00
|
|
|
|
expected_parts.size() > 20 ||
|
|
|
|
|
parts_to_fetch.size() > 2;
|
|
|
|
|
|
|
|
|
|
if (skip_sanity_check)
|
2014-04-08 17:45:21 +00:00
|
|
|
|
{
|
2014-05-22 10:37:17 +00:00
|
|
|
|
LOG_WARNING(log, "Skipping the limits on severity of changes to data parts (flag "
|
|
|
|
|
<< replica_path << "/flags/force_restore_data). " << sanity_report);
|
|
|
|
|
}
|
|
|
|
|
else if (insane)
|
|
|
|
|
{
|
|
|
|
|
throw Exception("The local set of parts of table " + getTableName() + " doesn't look like the set of parts in ZooKeeper. "
|
|
|
|
|
+ sanity_report,
|
2014-04-02 07:59:43 +00:00
|
|
|
|
ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS);
|
2014-04-08 17:45:21 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Добавим в ZK информацию о кусках, покрывающих недостающие куски.
|
|
|
|
|
for (MergeTreeData::DataPartPtr part : parts_to_add)
|
|
|
|
|
{
|
2014-04-09 16:32:32 +00:00
|
|
|
|
LOG_ERROR(log, "Adding unexpected local part to ZooKeeper: " << part->name);
|
2014-05-22 10:37:17 +00:00
|
|
|
|
|
2014-04-08 17:45:21 +00:00
|
|
|
|
zkutil::Ops ops;
|
|
|
|
|
checkPartAndAddToZooKeeper(part, ops);
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper->multi(ops);
|
2014-04-08 17:45:21 +00:00
|
|
|
|
}
|
2014-04-02 07:59:43 +00:00
|
|
|
|
|
2014-04-08 17:45:21 +00:00
|
|
|
|
/// Удалим из ZK информацию о кусках, покрытых только что добавленными.
|
|
|
|
|
for (const String & name : expected_parts)
|
|
|
|
|
{
|
2014-05-22 10:37:17 +00:00
|
|
|
|
LOG_ERROR(log, "Removing unexpectedly merged local part from ZooKeeper: " << name);
|
|
|
|
|
|
|
|
|
|
zkutil::Ops ops;
|
|
|
|
|
ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + name + "/checksums", -1));
|
|
|
|
|
ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + name, -1));
|
|
|
|
|
zookeeper->multi(ops);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Добавим в очередь задание забрать недостающие куски с других реплик и уберем из ZK информацию, что они у нас есть.
|
|
|
|
|
for (const String & name : parts_to_fetch)
|
|
|
|
|
{
|
|
|
|
|
LOG_ERROR(log, "Removing missing part from ZooKeeper and queueing a fetch: " << name);
|
|
|
|
|
|
|
|
|
|
LogEntry log_entry;
|
|
|
|
|
log_entry.type = LogEntry::GET_PART;
|
2014-05-26 17:12:46 +00:00
|
|
|
|
log_entry.source_replica = "";
|
2014-05-22 10:37:17 +00:00
|
|
|
|
log_entry.new_part_name = name;
|
|
|
|
|
|
|
|
|
|
/// Полагаемся, что это происходит до загрузки очереди (loadQueue).
|
2014-04-08 17:45:21 +00:00
|
|
|
|
zkutil::Ops ops;
|
|
|
|
|
ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + name + "/checksums", -1));
|
|
|
|
|
ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + name, -1));
|
2014-05-22 10:37:17 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::Create(
|
|
|
|
|
replica_path + "/queue/queue-", log_entry.toString(), zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper->multi(ops);
|
2014-04-08 17:45:21 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Удалим лишние локальные куски.
|
2014-04-02 07:59:43 +00:00
|
|
|
|
for (MergeTreeData::DataPartPtr part : unexpected_parts)
|
|
|
|
|
{
|
2014-05-22 10:37:17 +00:00
|
|
|
|
LOG_ERROR(log, "Renaming unexpected part " << part->name << " to ignored_" + part->name);
|
2014-04-02 10:10:37 +00:00
|
|
|
|
data.renameAndDetachPart(part, "ignored_");
|
2014-04-02 07:59:43 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-05-26 11:40:22 +00:00
|
|
|
|
void StorageReplicatedMergeTree::initVirtualParts()
|
|
|
|
|
{
|
|
|
|
|
auto parts = data.getDataParts();
|
|
|
|
|
for (const auto & part : parts)
|
|
|
|
|
{
|
|
|
|
|
virtual_parts.add(part->name);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2014-04-08 17:45:21 +00:00
|
|
|
|
void StorageReplicatedMergeTree::checkPartAndAddToZooKeeper(MergeTreeData::DataPartPtr part, zkutil::Ops & ops)
|
|
|
|
|
{
|
|
|
|
|
String another_replica = findReplicaHavingPart(part->name, false);
|
|
|
|
|
if (!another_replica.empty())
|
|
|
|
|
{
|
2014-04-14 13:08:26 +00:00
|
|
|
|
String checksums_str;
|
2014-05-13 10:10:26 +00:00
|
|
|
|
if (zookeeper->tryGet(zookeeper_path + "/replicas/" + another_replica + "/parts/" + part->name + "/checksums", checksums_str))
|
2014-04-14 13:08:26 +00:00
|
|
|
|
{
|
|
|
|
|
auto checksums = MergeTreeData::DataPart::Checksums::parse(checksums_str);
|
|
|
|
|
checksums.checkEqual(part->checksums, true);
|
|
|
|
|
}
|
2014-04-08 17:45:21 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
ops.push_back(new zkutil::Op::Create(
|
|
|
|
|
replica_path + "/parts/" + part->name,
|
|
|
|
|
"",
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper->getDefaultACL(),
|
2014-04-08 17:45:21 +00:00
|
|
|
|
zkutil::CreateMode::Persistent));
|
|
|
|
|
ops.push_back(new zkutil::Op::Create(
|
|
|
|
|
replica_path + "/parts/" + part->name + "/checksums",
|
|
|
|
|
part->checksums.toString(),
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper->getDefaultACL(),
|
2014-04-08 17:45:21 +00:00
|
|
|
|
zkutil::CreateMode::Persistent));
|
|
|
|
|
}
|
|
|
|
|
|
2014-04-09 15:52:47 +00:00
|
|
|
|
void StorageReplicatedMergeTree::clearOldParts()
|
|
|
|
|
{
|
|
|
|
|
Strings parts = data.clearOldParts();
|
|
|
|
|
|
|
|
|
|
for (const String & name : parts)
|
|
|
|
|
{
|
|
|
|
|
zkutil::Ops ops;
|
|
|
|
|
ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + name + "/checksums", -1));
|
|
|
|
|
ops.push_back(new zkutil::Op::Remove(replica_path + "/parts/" + name, -1));
|
2014-06-04 13:49:22 +00:00
|
|
|
|
int32_t code = zookeeper->tryMulti(ops);
|
|
|
|
|
if (code != ZOK)
|
|
|
|
|
LOG_WARNING(log, "Couldn't remove part " << name << " from ZooKeeper: " << zkutil::ZooKeeper::error2string(code));
|
2014-04-09 15:52:47 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (!parts.empty())
|
|
|
|
|
LOG_DEBUG(log, "Removed " << parts.size() << " old parts");
|
|
|
|
|
}
|
|
|
|
|
|
2014-04-14 10:19:33 +00:00
|
|
|
|
void StorageReplicatedMergeTree::clearOldLogs()
|
|
|
|
|
{
|
2014-05-13 10:10:26 +00:00
|
|
|
|
Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
|
2014-04-14 10:19:33 +00:00
|
|
|
|
UInt64 min_pointer = std::numeric_limits<UInt64>::max();
|
|
|
|
|
for (const String & replica : replicas)
|
|
|
|
|
{
|
|
|
|
|
String pointer;
|
2014-05-13 10:10:26 +00:00
|
|
|
|
if (!zookeeper->tryGet(zookeeper_path + "/replicas/" + replica + "/log_pointers/" + replica_name, pointer))
|
2014-04-14 10:19:33 +00:00
|
|
|
|
return;
|
|
|
|
|
min_pointer = std::min(min_pointer, parse<UInt64>(pointer));
|
|
|
|
|
}
|
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
Strings entries = zookeeper->getChildren(replica_path + "/log");
|
2014-04-14 10:19:33 +00:00
|
|
|
|
std::sort(entries.begin(), entries.end());
|
|
|
|
|
size_t removed = 0;
|
|
|
|
|
|
|
|
|
|
for (const String & entry : entries)
|
|
|
|
|
{
|
|
|
|
|
UInt64 index = parse<UInt64>(entry.substr(strlen("log-")));
|
|
|
|
|
if (index >= min_pointer)
|
|
|
|
|
break;
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper->remove(replica_path + "/log/" + entry);
|
2014-04-14 10:19:33 +00:00
|
|
|
|
++removed;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (removed > 0)
|
|
|
|
|
LOG_DEBUG(log, "Removed " << removed << " old log entries");
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
void StorageReplicatedMergeTree::clearOldBlocks()
|
|
|
|
|
{
|
|
|
|
|
zkutil::Stat stat;
|
2014-05-13 10:10:26 +00:00
|
|
|
|
if (!zookeeper->exists(zookeeper_path + "/blocks", &stat))
|
2014-04-14 10:19:33 +00:00
|
|
|
|
throw Exception(zookeeper_path + "/blocks doesn't exist", ErrorCodes::NOT_FOUND_NODE);
|
|
|
|
|
|
2014-06-04 13:49:22 +00:00
|
|
|
|
int children_count = stat.numChildren;
|
2014-04-14 10:56:06 +00:00
|
|
|
|
|
2014-04-14 10:19:33 +00:00
|
|
|
|
/// Чтобы делать "асимптотически" меньше запросов exists, будем ждать, пока накопятся в 1.1 раза больше блоков, чем нужно.
|
2014-04-14 10:56:06 +00:00
|
|
|
|
if (static_cast<double>(children_count) < data.settings.replicated_deduplication_window * 1.1)
|
2014-04-14 10:19:33 +00:00
|
|
|
|
return;
|
|
|
|
|
|
2014-04-14 10:56:06 +00:00
|
|
|
|
LOG_TRACE(log, "Clearing about " << static_cast<size_t>(children_count) - data.settings.replicated_deduplication_window
|
|
|
|
|
<< " old blocks from ZooKeeper");
|
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
Strings blocks = zookeeper->getChildren(zookeeper_path + "/blocks");
|
2014-04-14 10:19:33 +00:00
|
|
|
|
|
|
|
|
|
std::vector<std::pair<Int64, String> > timed_blocks;
|
|
|
|
|
|
|
|
|
|
for (const String & block : blocks)
|
|
|
|
|
{
|
|
|
|
|
zkutil::Stat stat;
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper->exists(zookeeper_path + "/blocks/" + block, &stat);
|
2014-06-04 13:49:22 +00:00
|
|
|
|
timed_blocks.push_back(std::make_pair(stat.czxid, block));
|
2014-04-14 10:19:33 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-04-14 10:56:06 +00:00
|
|
|
|
std::sort(timed_blocks.begin(), timed_blocks.end(), std::greater<std::pair<Int64, String>>());
|
2014-04-14 10:19:33 +00:00
|
|
|
|
for (size_t i = data.settings.replicated_deduplication_window; i < timed_blocks.size(); ++i)
|
|
|
|
|
{
|
2014-04-14 10:56:06 +00:00
|
|
|
|
zkutil::Ops ops;
|
|
|
|
|
ops.push_back(new zkutil::Op::Remove(zookeeper_path + "/blocks/" + timed_blocks[i].second + "/number", -1));
|
|
|
|
|
ops.push_back(new zkutil::Op::Remove(zookeeper_path + "/blocks/" + timed_blocks[i].second + "/checksums", -1));
|
|
|
|
|
ops.push_back(new zkutil::Op::Remove(zookeeper_path + "/blocks/" + timed_blocks[i].second, -1));
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper->multi(ops);
|
2014-04-14 10:19:33 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
LOG_TRACE(log, "Cleared " << blocks.size() - data.settings.replicated_deduplication_window << " old blocks from ZooKeeper");
|
|
|
|
|
}
|
|
|
|
|
|
2014-04-03 08:47:59 +00:00
|
|
|
|
void StorageReplicatedMergeTree::loadQueue()
|
|
|
|
|
{
|
|
|
|
|
Poco::ScopedLock<Poco::FastMutex> lock(queue_mutex);
|
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
Strings children = zookeeper->getChildren(replica_path + "/queue");
|
2014-04-03 08:47:59 +00:00
|
|
|
|
std::sort(children.begin(), children.end());
|
|
|
|
|
for (const String & child : children)
|
|
|
|
|
{
|
2014-05-13 10:10:26 +00:00
|
|
|
|
String s = zookeeper->get(replica_path + "/queue/" + child);
|
2014-04-03 11:48:28 +00:00
|
|
|
|
LogEntry entry = LogEntry::parse(s);
|
|
|
|
|
entry.znode_name = child;
|
2014-05-26 11:40:22 +00:00
|
|
|
|
entry.addResultToVirtualParts(*this);
|
2014-04-03 11:48:28 +00:00
|
|
|
|
queue.push_back(entry);
|
2014-04-03 08:47:59 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
void StorageReplicatedMergeTree::pullLogsToQueue()
|
|
|
|
|
{
|
|
|
|
|
Poco::ScopedLock<Poco::FastMutex> lock(queue_mutex);
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-04-03 12:49:01 +00:00
|
|
|
|
/// Сольем все логи в хронологическом порядке.
|
|
|
|
|
|
|
|
|
|
struct LogIterator
|
|
|
|
|
{
|
|
|
|
|
String replica; /// Имя реплики.
|
|
|
|
|
UInt64 index; /// Номер записи в логе (суффикс имени ноды).
|
|
|
|
|
|
|
|
|
|
Int64 timestamp; /// Время (czxid) создания записи в логе.
|
|
|
|
|
String entry_str; /// Сама запись.
|
|
|
|
|
|
|
|
|
|
bool operator<(const LogIterator & rhs) const
|
|
|
|
|
{
|
2014-05-27 12:08:40 +00:00
|
|
|
|
/// Нужно доставать из очереди минимальный timestamp.
|
|
|
|
|
return timestamp > rhs.timestamp;
|
2014-04-03 12:49:01 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
bool readEntry(zkutil::ZooKeeper & zookeeper, const String & zookeeper_path)
|
|
|
|
|
{
|
|
|
|
|
String index_str = toString(index);
|
|
|
|
|
while (index_str.size() < 10)
|
|
|
|
|
index_str = '0' + index_str;
|
|
|
|
|
zkutil::Stat stat;
|
|
|
|
|
if (!zookeeper.tryGet(zookeeper_path + "/replicas/" + replica + "/log/log-" + index_str, entry_str, &stat))
|
|
|
|
|
return false;
|
2014-06-04 13:49:22 +00:00
|
|
|
|
timestamp = stat.czxid;
|
2014-04-03 12:49:01 +00:00
|
|
|
|
return true;
|
|
|
|
|
}
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
typedef std::priority_queue<LogIterator> PriorityQueue;
|
|
|
|
|
PriorityQueue priority_queue;
|
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
|
2014-04-03 12:49:01 +00:00
|
|
|
|
|
2014-04-03 08:47:59 +00:00
|
|
|
|
for (const String & replica : replicas)
|
|
|
|
|
{
|
2014-04-03 12:49:01 +00:00
|
|
|
|
String index_str;
|
|
|
|
|
UInt64 index;
|
2014-04-03 08:47:59 +00:00
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
if (zookeeper->tryGet(replica_path + "/log_pointers/" + replica, index_str))
|
2014-04-03 08:47:59 +00:00
|
|
|
|
{
|
2014-04-07 17:39:45 +00:00
|
|
|
|
index = parse<UInt64>(index_str);
|
2014-04-03 08:47:59 +00:00
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
/// Если у нас еще нет указателя на лог этой реплики, поставим указатель на первую запись в нем.
|
2014-05-13 10:10:26 +00:00
|
|
|
|
Strings entries = zookeeper->getChildren(zookeeper_path + "/replicas/" + replica + "/log");
|
2014-04-03 08:47:59 +00:00
|
|
|
|
std::sort(entries.begin(), entries.end());
|
2014-04-07 17:39:45 +00:00
|
|
|
|
index = entries.empty() ? 0 : parse<UInt64>(entries[0].substr(strlen("log-")));
|
2014-04-03 08:47:59 +00:00
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper->create(replica_path + "/log_pointers/" + replica, toString(index), zkutil::CreateMode::Persistent);
|
2014-04-03 08:47:59 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-04-03 12:49:01 +00:00
|
|
|
|
LogIterator iterator;
|
|
|
|
|
iterator.replica = replica;
|
|
|
|
|
iterator.index = index;
|
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
if (iterator.readEntry(*zookeeper, zookeeper_path))
|
2014-04-03 12:49:01 +00:00
|
|
|
|
priority_queue.push(iterator);
|
|
|
|
|
}
|
2014-04-03 08:47:59 +00:00
|
|
|
|
|
2014-04-04 12:47:57 +00:00
|
|
|
|
size_t count = 0;
|
|
|
|
|
|
2014-04-03 12:49:01 +00:00
|
|
|
|
while (!priority_queue.empty())
|
|
|
|
|
{
|
|
|
|
|
LogIterator iterator = priority_queue.top();
|
|
|
|
|
priority_queue.pop();
|
2014-04-04 12:47:57 +00:00
|
|
|
|
++count;
|
2014-04-03 11:48:28 +00:00
|
|
|
|
|
2014-04-03 12:49:01 +00:00
|
|
|
|
LogEntry entry = LogEntry::parse(iterator.entry_str);
|
2014-04-03 08:47:59 +00:00
|
|
|
|
|
2014-04-03 12:49:01 +00:00
|
|
|
|
/// Одновременно добавим запись в очередь и продвинем указатель на лог.
|
|
|
|
|
zkutil::Ops ops;
|
|
|
|
|
ops.push_back(new zkutil::Op::Create(
|
2014-05-13 10:10:26 +00:00
|
|
|
|
replica_path + "/queue/queue-", iterator.entry_str, zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
|
2014-04-03 12:49:01 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::SetData(
|
|
|
|
|
replica_path + "/log_pointers/" + iterator.replica, toString(iterator.index + 1), -1));
|
2014-05-13 10:10:26 +00:00
|
|
|
|
auto results = zookeeper->multi(ops);
|
2014-04-03 12:49:01 +00:00
|
|
|
|
|
2014-06-04 13:49:22 +00:00
|
|
|
|
String path_created = dynamic_cast<zkutil::Op::Create &>(ops[0]).getPathCreated();
|
2014-04-04 12:47:57 +00:00
|
|
|
|
entry.znode_name = path_created.substr(path_created.find_last_of('/') + 1);
|
2014-05-26 11:40:22 +00:00
|
|
|
|
entry.addResultToVirtualParts(*this);
|
2014-04-03 12:49:01 +00:00
|
|
|
|
queue.push_back(entry);
|
|
|
|
|
|
|
|
|
|
++iterator.index;
|
2014-05-13 10:10:26 +00:00
|
|
|
|
if (iterator.readEntry(*zookeeper, zookeeper_path))
|
2014-04-03 12:49:01 +00:00
|
|
|
|
priority_queue.push(iterator);
|
2014-04-03 08:47:59 +00:00
|
|
|
|
}
|
2014-04-04 12:47:57 +00:00
|
|
|
|
|
|
|
|
|
if (count > 0)
|
|
|
|
|
LOG_DEBUG(log, "Pulled " << count << " entries to queue");
|
2014-04-03 08:47:59 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-04-07 15:45:46 +00:00
|
|
|
|
bool StorageReplicatedMergeTree::shouldExecuteLogEntry(const LogEntry & entry)
|
2014-04-03 08:47:59 +00:00
|
|
|
|
{
|
2014-04-07 15:45:46 +00:00
|
|
|
|
if (entry.type == LogEntry::MERGE_PARTS)
|
|
|
|
|
{
|
|
|
|
|
/** Если какая-то из нужных частей сейчас передается или мерджится, подождем окончания этой операции.
|
|
|
|
|
* Иначе, даже если всех нужных частей для мерджа нет, нужно попытаться сделать мердж.
|
|
|
|
|
* Если каких-то частей не хватает, вместо мерджа будет попытка скачать кусок.
|
|
|
|
|
* Такая ситуация возможна, если получение какого-то куска пофейлилось, и его переместили в конец очереди.
|
|
|
|
|
*/
|
|
|
|
|
for (const auto & name : entry.parts_to_merge)
|
|
|
|
|
{
|
|
|
|
|
if (future_parts.count(name))
|
|
|
|
|
{
|
2014-05-07 14:12:15 +00:00
|
|
|
|
LOG_TRACE(log, "Not merging into part " << entry.new_part_name << " because part " << name << " is not ready yet.");
|
2014-04-07 15:45:46 +00:00
|
|
|
|
return false;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
return true;
|
2014-04-03 11:48:28 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
void StorageReplicatedMergeTree::executeLogEntry(const LogEntry & entry)
|
|
|
|
|
{
|
|
|
|
|
if (entry.type == LogEntry::GET_PART ||
|
|
|
|
|
entry.type == LogEntry::MERGE_PARTS)
|
|
|
|
|
{
|
|
|
|
|
/// Если у нас уже есть этот кусок или покрывающий его кусок, ничего делать не нужно.
|
2014-05-27 10:03:13 +00:00
|
|
|
|
MergeTreeData::DataPartPtr containing_part = data.getContainingPart(entry.new_part_name, true);
|
2014-04-03 11:48:28 +00:00
|
|
|
|
|
|
|
|
|
/// Даже если кусок есть локально, его (в исключительных случаях) может не быть в zookeeper.
|
2014-05-13 10:10:26 +00:00
|
|
|
|
if (containing_part && zookeeper->exists(replica_path + "/parts/" + containing_part->name))
|
2014-04-04 12:47:57 +00:00
|
|
|
|
{
|
2014-04-07 15:45:46 +00:00
|
|
|
|
if (!(entry.type == LogEntry::GET_PART && entry.source_replica == replica_name))
|
|
|
|
|
LOG_DEBUG(log, "Skipping action for part " + entry.new_part_name + " - part already exists");
|
2014-04-03 11:48:28 +00:00
|
|
|
|
return;
|
2014-04-04 12:47:57 +00:00
|
|
|
|
}
|
2014-04-03 11:48:28 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-04-07 15:45:46 +00:00
|
|
|
|
if (entry.type == LogEntry::GET_PART && entry.source_replica == replica_name)
|
2014-05-27 10:03:13 +00:00
|
|
|
|
LOG_WARNING(log, "Part " << entry.new_part_name << " from own log doesn't exist.");
|
2014-04-07 15:45:46 +00:00
|
|
|
|
|
|
|
|
|
bool do_fetch = false;
|
|
|
|
|
|
2014-04-04 12:47:57 +00:00
|
|
|
|
if (entry.type == LogEntry::GET_PART)
|
|
|
|
|
{
|
2014-04-07 15:45:46 +00:00
|
|
|
|
do_fetch = true;
|
2014-04-04 12:47:57 +00:00
|
|
|
|
}
|
|
|
|
|
else if (entry.type == LogEntry::MERGE_PARTS)
|
|
|
|
|
{
|
2014-04-04 13:27:47 +00:00
|
|
|
|
MergeTreeData::DataPartsVector parts;
|
2014-04-07 15:45:46 +00:00
|
|
|
|
bool have_all_parts = true;;
|
2014-04-04 13:27:47 +00:00
|
|
|
|
for (const String & name : entry.parts_to_merge)
|
|
|
|
|
{
|
|
|
|
|
MergeTreeData::DataPartPtr part = data.getContainingPart(name);
|
2014-04-07 15:45:46 +00:00
|
|
|
|
if (!part)
|
|
|
|
|
{
|
|
|
|
|
have_all_parts = false;
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
if (part->name != name)
|
|
|
|
|
{
|
|
|
|
|
LOG_ERROR(log, "Log and parts set look inconsistent: " << name << " is covered by " << part->name
|
|
|
|
|
<< " but should be merged into " << entry.new_part_name);
|
|
|
|
|
have_all_parts = false;
|
|
|
|
|
break;
|
|
|
|
|
}
|
2014-04-04 13:27:47 +00:00
|
|
|
|
parts.push_back(part);
|
|
|
|
|
}
|
2014-04-04 17:20:45 +00:00
|
|
|
|
|
2014-04-07 15:45:46 +00:00
|
|
|
|
if (!have_all_parts)
|
2014-04-04 13:27:47 +00:00
|
|
|
|
{
|
2014-04-07 15:45:46 +00:00
|
|
|
|
/// Если нет всех нужных кусков, попробуем взять у кого-нибудь уже помердженный кусок.
|
|
|
|
|
do_fetch = true;
|
|
|
|
|
LOG_DEBUG(log, "Don't have all parts for merge " << entry.new_part_name << "; will try to fetch it instead");
|
2014-04-04 13:27:47 +00:00
|
|
|
|
}
|
2014-04-07 15:45:46 +00:00
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
MergeTreeData::DataPartPtr part = merger.mergeParts(parts, entry.new_part_name);
|
|
|
|
|
|
|
|
|
|
zkutil::Ops ops;
|
2014-04-08 17:45:21 +00:00
|
|
|
|
checkPartAndAddToZooKeeper(part, ops);
|
2014-04-07 15:45:46 +00:00
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper->multi(ops);
|
2014-04-07 15:45:46 +00:00
|
|
|
|
|
|
|
|
|
ProfileEvents::increment(ProfileEvents::ReplicatedPartMerges);
|
|
|
|
|
}
|
2014-04-04 12:47:57 +00:00
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
throw Exception("Unexpected log entry type: " + toString(static_cast<int>(entry.type)));
|
|
|
|
|
}
|
2014-04-07 15:45:46 +00:00
|
|
|
|
|
|
|
|
|
if (do_fetch)
|
|
|
|
|
{
|
|
|
|
|
try
|
|
|
|
|
{
|
2014-04-08 17:45:21 +00:00
|
|
|
|
String replica = findReplicaHavingPart(entry.new_part_name, true);
|
|
|
|
|
if (replica.empty())
|
|
|
|
|
{
|
|
|
|
|
ProfileEvents::increment(ProfileEvents::ReplicatedPartFailedFetches);
|
|
|
|
|
throw Exception("No active replica has part " + entry.new_part_name, ErrorCodes::NO_REPLICA_HAS_PART);
|
|
|
|
|
}
|
2014-04-07 15:45:46 +00:00
|
|
|
|
fetchPart(entry.new_part_name, replica);
|
|
|
|
|
|
|
|
|
|
if (entry.type == LogEntry::MERGE_PARTS)
|
|
|
|
|
ProfileEvents::increment(ProfileEvents::ReplicatedPartFetchesOfMerged);
|
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
/** Если не получилось скачать кусок, нужный для какого-то мерджа, лучше не пытаться получить другие куски для этого мерджа,
|
|
|
|
|
* а попытаться сразу получить помердженный кусок. Чтобы так получилось, переместим действия для получения остальных кусков
|
|
|
|
|
* для этого мерджа в конец очереди.
|
|
|
|
|
*
|
|
|
|
|
*/
|
|
|
|
|
try
|
|
|
|
|
{
|
|
|
|
|
Poco::ScopedLock<Poco::FastMutex> lock(queue_mutex);
|
|
|
|
|
|
|
|
|
|
/// Найдем действие по объединению этого куска с другими. Запомним других.
|
|
|
|
|
StringSet parts_for_merge;
|
|
|
|
|
LogEntries::iterator merge_entry;
|
|
|
|
|
for (LogEntries::iterator it = queue.begin(); it != queue.end(); ++it)
|
|
|
|
|
{
|
|
|
|
|
if (it->type == LogEntry::MERGE_PARTS)
|
|
|
|
|
{
|
|
|
|
|
if (std::find(it->parts_to_merge.begin(), it->parts_to_merge.end(), entry.new_part_name)
|
|
|
|
|
!= it->parts_to_merge.end())
|
|
|
|
|
{
|
|
|
|
|
parts_for_merge = StringSet(it->parts_to_merge.begin(), it->parts_to_merge.end());
|
|
|
|
|
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;
|
|
|
|
|
|
|
|
|
|
if ((it0->type == LogEntry::MERGE_PARTS || it0->type == LogEntry::GET_PART)
|
|
|
|
|
&& parts_for_merge.count(it0->new_part_name))
|
|
|
|
|
{
|
|
|
|
|
queue.splice(queue.end(), queue, it0, it);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
throw;
|
|
|
|
|
}
|
|
|
|
|
}
|
2014-04-03 11:48:28 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
void StorageReplicatedMergeTree::queueUpdatingThread()
|
|
|
|
|
{
|
|
|
|
|
while (!shutdown_called)
|
|
|
|
|
{
|
2014-04-04 10:37:33 +00:00
|
|
|
|
try
|
|
|
|
|
{
|
|
|
|
|
pullLogsToQueue();
|
2014-04-09 15:52:47 +00:00
|
|
|
|
|
|
|
|
|
clearOldParts();
|
2014-04-14 10:19:33 +00:00
|
|
|
|
|
|
|
|
|
/// Каждую минуту выбрасываем ненужные записи из лога.
|
|
|
|
|
if (time(0) - clear_old_logs_time > 60)
|
|
|
|
|
{
|
|
|
|
|
clear_old_logs_time = time(0);
|
|
|
|
|
clearOldLogs();
|
|
|
|
|
}
|
2014-04-04 10:37:33 +00:00
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
|
}
|
|
|
|
|
|
2014-04-03 11:48:28 +00:00
|
|
|
|
std::this_thread::sleep_for(QUEUE_UPDATE_SLEEP);
|
|
|
|
|
}
|
2014-04-03 08:47:59 +00:00
|
|
|
|
}
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-04-03 11:48:28 +00:00
|
|
|
|
void StorageReplicatedMergeTree::queueThread()
|
|
|
|
|
{
|
|
|
|
|
while (!shutdown_called)
|
|
|
|
|
{
|
|
|
|
|
LogEntry entry;
|
2014-04-07 15:45:46 +00:00
|
|
|
|
bool have_work = false;
|
2014-04-03 11:48:28 +00:00
|
|
|
|
|
2014-04-07 15:45:46 +00:00
|
|
|
|
try
|
2014-04-03 11:48:28 +00:00
|
|
|
|
{
|
|
|
|
|
Poco::ScopedLock<Poco::FastMutex> lock(queue_mutex);
|
2014-04-07 15:45:46 +00:00
|
|
|
|
bool empty = queue.empty();
|
2014-04-03 11:48:28 +00:00
|
|
|
|
if (!empty)
|
|
|
|
|
{
|
2014-04-07 15:45:46 +00:00
|
|
|
|
for (LogEntries::iterator it = queue.begin(); it != queue.end(); ++it)
|
|
|
|
|
{
|
|
|
|
|
if (shouldExecuteLogEntry(*it))
|
|
|
|
|
{
|
|
|
|
|
entry = *it;
|
|
|
|
|
entry.tagPartAsFuture(*this);
|
|
|
|
|
queue.erase(it);
|
|
|
|
|
have_work = true;
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
}
|
2014-04-03 11:48:28 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
2014-04-07 15:45:46 +00:00
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
|
}
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-04-07 15:45:46 +00:00
|
|
|
|
if (!have_work)
|
2014-04-03 11:48:28 +00:00
|
|
|
|
{
|
|
|
|
|
std::this_thread::sleep_for(QUEUE_NO_WORK_SLEEP);
|
|
|
|
|
continue;
|
|
|
|
|
}
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-04-03 11:48:28 +00:00
|
|
|
|
bool success = false;
|
2014-04-02 07:59:43 +00:00
|
|
|
|
|
2014-04-03 11:48:28 +00:00
|
|
|
|
try
|
|
|
|
|
{
|
|
|
|
|
executeLogEntry(entry);
|
2014-04-07 15:45:46 +00:00
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
auto code = zookeeper->tryRemove(replica_path + "/queue/" + entry.znode_name);
|
2014-06-04 13:49:22 +00:00
|
|
|
|
if (code != ZOK)
|
2014-04-07 15:45:46 +00:00
|
|
|
|
LOG_ERROR(log, "Couldn't remove " << replica_path + "/queue/" + entry.znode_name << ": "
|
2014-06-04 13:49:22 +00:00
|
|
|
|
<< zkutil::ZooKeeper::error2string(code) + ". There must be a bug somewhere. Ignoring it.");
|
2014-04-03 11:48:28 +00:00
|
|
|
|
|
|
|
|
|
success = true;
|
|
|
|
|
}
|
2014-04-07 18:14:39 +00:00
|
|
|
|
catch (Exception & e)
|
|
|
|
|
{
|
|
|
|
|
if (e.code() == ErrorCodes::NO_REPLICA_HAS_PART)
|
|
|
|
|
/// Если ни у кого нет нужного куска, это нормальная ситуация; не будем писать в лог с уровнем Error.
|
|
|
|
|
LOG_INFO(log, e.displayText());
|
|
|
|
|
else
|
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
|
}
|
2014-04-03 11:48:28 +00:00
|
|
|
|
catch (...)
|
|
|
|
|
{
|
2014-04-04 10:37:33 +00:00
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
2014-04-03 11:48:28 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if (shutdown_called)
|
|
|
|
|
break;
|
|
|
|
|
|
|
|
|
|
if (success)
|
|
|
|
|
{
|
|
|
|
|
std::this_thread::sleep_for(QUEUE_AFTER_WORK_SLEEP);
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
|
|
|
|
{
|
|
|
|
|
/// Добавим действие, которое не получилось выполнить, в конец очереди.
|
2014-04-07 15:45:46 +00:00
|
|
|
|
entry.future_part_tagger = nullptr;
|
2014-04-03 11:48:28 +00:00
|
|
|
|
Poco::ScopedLock<Poco::FastMutex> lock(queue_mutex);
|
|
|
|
|
queue.push_back(entry);
|
|
|
|
|
}
|
|
|
|
|
std::this_thread::sleep_for(QUEUE_ERROR_SLEEP);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2014-04-04 10:37:33 +00:00
|
|
|
|
void StorageReplicatedMergeTree::mergeSelectingThread()
|
|
|
|
|
{
|
|
|
|
|
pullLogsToQueue();
|
|
|
|
|
|
|
|
|
|
while (!shutdown_called && is_leader_node)
|
|
|
|
|
{
|
2014-04-07 18:14:39 +00:00
|
|
|
|
bool success = false;
|
2014-04-04 10:37:33 +00:00
|
|
|
|
|
2014-04-07 18:14:39 +00:00
|
|
|
|
try
|
2014-04-04 10:37:33 +00:00
|
|
|
|
{
|
2014-04-07 18:14:39 +00:00
|
|
|
|
size_t merges_queued = 0;
|
2014-05-26 11:40:22 +00:00
|
|
|
|
/// Есть ли в очереди мердж крупных кусков.
|
|
|
|
|
/// TODO: Если мердж уже выполняется, его нет в очереди, но здесь нужно все равно как-то о нем узнать.
|
|
|
|
|
bool has_big_merge = false;
|
2014-04-04 10:37:33 +00:00
|
|
|
|
|
2014-04-07 18:14:39 +00:00
|
|
|
|
{
|
|
|
|
|
Poco::ScopedLock<Poco::FastMutex> lock(queue_mutex);
|
2014-04-04 10:37:33 +00:00
|
|
|
|
|
2014-04-07 18:14:39 +00:00
|
|
|
|
for (const auto & entry : queue)
|
2014-05-26 11:40:22 +00:00
|
|
|
|
{
|
2014-04-07 18:14:39 +00:00
|
|
|
|
if (entry.type == LogEntry::MERGE_PARTS)
|
2014-05-26 11:40:22 +00:00
|
|
|
|
{
|
2014-04-07 18:14:39 +00:00
|
|
|
|
++merges_queued;
|
2014-05-26 11:40:22 +00:00
|
|
|
|
|
|
|
|
|
if (!has_big_merge)
|
|
|
|
|
{
|
|
|
|
|
for (const String & name : entry.parts_to_merge)
|
|
|
|
|
{
|
|
|
|
|
MergeTreeData::DataPartPtr part = data.getContainingPart(name);
|
|
|
|
|
if (!part || part->name != name)
|
|
|
|
|
continue;
|
|
|
|
|
if (part->size * data.index_granularity > 25 * 1024 * 1024)
|
|
|
|
|
{
|
|
|
|
|
has_big_merge = true;
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
2014-04-07 18:14:39 +00:00
|
|
|
|
}
|
2014-04-04 10:37:33 +00:00
|
|
|
|
|
2014-04-07 18:14:39 +00:00
|
|
|
|
if (merges_queued >= data.settings.merging_threads)
|
|
|
|
|
{
|
|
|
|
|
std::this_thread::sleep_for(MERGE_SELECTING_SLEEP);
|
|
|
|
|
continue;
|
|
|
|
|
}
|
2014-04-04 10:37:33 +00:00
|
|
|
|
|
|
|
|
|
MergeTreeData::DataPartsVector parts;
|
2014-04-04 12:47:57 +00:00
|
|
|
|
|
2014-04-04 10:37:33 +00:00
|
|
|
|
{
|
2014-04-04 17:20:45 +00:00
|
|
|
|
String merged_name;
|
|
|
|
|
auto can_merge = std::bind(
|
|
|
|
|
&StorageReplicatedMergeTree::canMergeParts, this, std::placeholders::_1, std::placeholders::_2);
|
|
|
|
|
|
2014-05-14 17:51:37 +00:00
|
|
|
|
if (merger.selectPartsToMerge(parts, merged_name, MergeTreeDataMerger::NO_LIMIT,
|
|
|
|
|
false, false, has_big_merge, can_merge) ||
|
|
|
|
|
merger.selectPartsToMerge(parts, merged_name, MergeTreeDataMerger::NO_LIMIT,
|
|
|
|
|
true, false, has_big_merge, can_merge))
|
2014-04-04 10:37:33 +00:00
|
|
|
|
{
|
2014-04-04 17:20:45 +00:00
|
|
|
|
LogEntry entry;
|
|
|
|
|
entry.type = LogEntry::MERGE_PARTS;
|
2014-04-07 15:45:46 +00:00
|
|
|
|
entry.source_replica = replica_name;
|
2014-04-04 17:20:45 +00:00
|
|
|
|
entry.new_part_name = merged_name;
|
|
|
|
|
|
|
|
|
|
for (const auto & part : parts)
|
|
|
|
|
{
|
|
|
|
|
entry.parts_to_merge.push_back(part->name);
|
|
|
|
|
}
|
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper->create(replica_path + "/log/log-", entry.toString(), zkutil::CreateMode::PersistentSequential);
|
2014-04-04 17:20:45 +00:00
|
|
|
|
|
|
|
|
|
success = true;
|
2014-04-04 10:37:33 +00:00
|
|
|
|
}
|
2014-04-04 17:20:45 +00:00
|
|
|
|
}
|
2014-04-04 10:37:33 +00:00
|
|
|
|
|
2014-05-07 13:58:20 +00:00
|
|
|
|
if (success)
|
2014-04-04 17:20:45 +00:00
|
|
|
|
{
|
2014-05-07 13:58:20 +00:00
|
|
|
|
/// Нужно загрузить новую запись в очередь перед тем, как в следующий раз выбирать куски для слияния.
|
2014-05-26 11:40:22 +00:00
|
|
|
|
/// (чтобы кусок добавился в virtual_parts).
|
2014-05-07 13:58:20 +00:00
|
|
|
|
pullLogsToQueue();
|
|
|
|
|
|
|
|
|
|
String month_name = parts[0]->name.substr(0, 6);
|
|
|
|
|
for (size_t i = 0; i + 1 < parts.size(); ++i)
|
2014-04-04 17:20:45 +00:00
|
|
|
|
{
|
2014-05-07 13:58:20 +00:00
|
|
|
|
/// Уберем больше не нужные отметки о несуществующих блоках.
|
|
|
|
|
for (UInt64 number = parts[i]->right + 1; number <= parts[i + 1]->left - 1; ++number)
|
|
|
|
|
{
|
|
|
|
|
String number_str = toString(number);
|
|
|
|
|
while (number_str.size() < 10)
|
|
|
|
|
number_str = '0' + number_str;
|
|
|
|
|
String path = zookeeper_path + "/block_numbers/" + month_name + "/block-" + number_str;
|
2014-04-04 10:37:33 +00:00
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper->tryRemove(path);
|
2014-05-07 13:58:20 +00:00
|
|
|
|
}
|
2014-04-04 17:20:45 +00:00
|
|
|
|
}
|
2014-04-04 10:37:33 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
|
}
|
|
|
|
|
|
2014-04-14 10:56:06 +00:00
|
|
|
|
if (shutdown_called || !is_leader_node)
|
2014-04-04 10:37:33 +00:00
|
|
|
|
break;
|
|
|
|
|
|
2014-04-14 10:56:06 +00:00
|
|
|
|
if (!success)
|
|
|
|
|
std::this_thread::sleep_for(MERGE_SELECTING_SLEEP);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
void StorageReplicatedMergeTree::clearOldBlocksThread()
|
|
|
|
|
{
|
|
|
|
|
while (!shutdown_called && is_leader_node)
|
|
|
|
|
{
|
|
|
|
|
try
|
2014-04-14 10:19:33 +00:00
|
|
|
|
{
|
|
|
|
|
clearOldBlocks();
|
|
|
|
|
}
|
2014-04-14 10:56:06 +00:00
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
|
}
|
2014-04-14 10:19:33 +00:00
|
|
|
|
|
2014-04-14 10:56:06 +00:00
|
|
|
|
/// Спим минуту, но проверяем, нужно ли завершиться, каждую секунду.
|
|
|
|
|
/// TODO: Лучше во всех подобных местах использовать condition variable.
|
|
|
|
|
for (size_t i = 0; i < 60; ++i)
|
|
|
|
|
{
|
|
|
|
|
if (shutdown_called || !is_leader_node)
|
|
|
|
|
break;
|
|
|
|
|
std::this_thread::sleep_for(std::chrono::seconds(1));
|
|
|
|
|
}
|
2014-04-04 10:37:33 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
bool StorageReplicatedMergeTree::canMergeParts(const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right)
|
|
|
|
|
{
|
2014-05-26 11:40:22 +00:00
|
|
|
|
/// Если какой-то из кусков уже собираются слить в больший, не соглашаемся его сливать.
|
|
|
|
|
if (virtual_parts.getContainingPart(left->name) != left->name ||
|
|
|
|
|
virtual_parts.getContainingPart(right->name) != right->name)
|
2014-04-04 10:37:33 +00:00
|
|
|
|
return false;
|
|
|
|
|
|
2014-05-07 13:58:20 +00:00
|
|
|
|
String month_name = left->name.substr(0, 6);
|
|
|
|
|
|
2014-04-04 10:37:33 +00:00
|
|
|
|
/// Можно слить куски, если все номера между ними заброшены - не соответствуют никаким блокам.
|
|
|
|
|
for (UInt64 number = left->right + 1; number <= right->left - 1; ++number)
|
|
|
|
|
{
|
|
|
|
|
String number_str = toString(number);
|
|
|
|
|
while (number_str.size() < 10)
|
|
|
|
|
number_str = '0' + number_str;
|
2014-05-07 13:58:20 +00:00
|
|
|
|
String path = zookeeper_path + "/block_numbers/" + month_name + "/block-" + number_str;
|
2014-04-04 10:37:33 +00:00
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
if (AbandonableLockInZooKeeper::check(path, *zookeeper) != AbandonableLockInZooKeeper::ABANDONED)
|
2014-04-04 12:47:57 +00:00
|
|
|
|
{
|
|
|
|
|
LOG_DEBUG(log, "Can't merge parts " << left->name << " and " << right->name << " because block " << path << " exists");
|
2014-04-04 10:37:33 +00:00
|
|
|
|
return false;
|
2014-04-04 12:47:57 +00:00
|
|
|
|
}
|
2014-04-04 10:37:33 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
return true;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
void StorageReplicatedMergeTree::becomeLeader()
|
|
|
|
|
{
|
2014-04-04 12:47:57 +00:00
|
|
|
|
LOG_INFO(log, "Became leader");
|
2014-04-04 10:37:33 +00:00
|
|
|
|
is_leader_node = true;
|
|
|
|
|
merge_selecting_thread = std::thread(&StorageReplicatedMergeTree::mergeSelectingThread, this);
|
2014-04-14 10:56:06 +00:00
|
|
|
|
clear_old_blocks_thread = std::thread(&StorageReplicatedMergeTree::clearOldBlocksThread, this);
|
2014-04-04 10:37:33 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-04-08 17:45:21 +00:00
|
|
|
|
String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_name, bool active)
|
2014-04-03 11:48:28 +00:00
|
|
|
|
{
|
2014-05-13 10:10:26 +00:00
|
|
|
|
Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
|
2014-04-03 11:48:28 +00:00
|
|
|
|
|
|
|
|
|
/// Из реплик, у которых есть кусок, выберем одну равновероятно.
|
|
|
|
|
std::random_shuffle(replicas.begin(), replicas.end());
|
|
|
|
|
|
|
|
|
|
for (const String & replica : replicas)
|
|
|
|
|
{
|
2014-05-13 10:10:26 +00:00
|
|
|
|
if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/parts/" + part_name) &&
|
|
|
|
|
(!active || zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")))
|
2014-04-03 11:48:28 +00:00
|
|
|
|
return replica;
|
|
|
|
|
}
|
|
|
|
|
|
2014-04-08 17:45:21 +00:00
|
|
|
|
return "";
|
2014-04-03 11:48:28 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
void StorageReplicatedMergeTree::fetchPart(const String & part_name, const String & replica_name)
|
|
|
|
|
{
|
2014-04-04 12:47:57 +00:00
|
|
|
|
LOG_DEBUG(log, "Fetching part " << part_name << " from " << replica_name);
|
|
|
|
|
|
2014-04-03 11:48:28 +00:00
|
|
|
|
auto table_lock = lockStructure(true);
|
|
|
|
|
|
|
|
|
|
String host;
|
|
|
|
|
int port;
|
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
String host_port_str = zookeeper->get(zookeeper_path + "/replicas/" + replica_name + "/host");
|
2014-04-03 11:48:28 +00:00
|
|
|
|
ReadBufferFromString buf(host_port_str);
|
|
|
|
|
assertString("host: ", buf);
|
|
|
|
|
readString(host, buf);
|
|
|
|
|
assertString("\nport: ", buf);
|
|
|
|
|
readText(port, buf);
|
|
|
|
|
assertString("\n", buf);
|
|
|
|
|
assertEOF(buf);
|
|
|
|
|
|
2014-04-03 12:49:01 +00:00
|
|
|
|
MergeTreeData::MutableDataPartPtr part = fetcher.fetchPart(part_name, zookeeper_path + "/replicas/" + replica_name, host, port);
|
2014-04-07 15:45:46 +00:00
|
|
|
|
auto removed_parts = data.renameTempPartAndReplace(part);
|
|
|
|
|
|
2014-04-03 11:48:28 +00:00
|
|
|
|
zkutil::Ops ops;
|
2014-04-08 17:45:21 +00:00
|
|
|
|
checkPartAndAddToZooKeeper(part, ops);
|
2014-04-07 18:14:39 +00:00
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper->multi(ops);
|
2014-04-09 15:52:47 +00:00
|
|
|
|
|
2014-04-07 18:14:39 +00:00
|
|
|
|
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);
|
|
|
|
|
}
|
|
|
|
|
|
2014-04-07 15:45:46 +00:00
|
|
|
|
ProfileEvents::increment(ProfileEvents::ReplicatedPartFetches);
|
|
|
|
|
|
2014-04-04 12:47:57 +00:00
|
|
|
|
LOG_DEBUG(log, "Fetched part");
|
2014-04-03 11:48:28 +00:00
|
|
|
|
}
|
2014-03-21 19:17:59 +00:00
|
|
|
|
|
2014-03-22 14:44:44 +00:00
|
|
|
|
void StorageReplicatedMergeTree::shutdown()
|
|
|
|
|
{
|
2014-04-25 13:55:15 +00:00
|
|
|
|
if (permanent_shutdown_called)
|
2014-05-26 11:40:22 +00:00
|
|
|
|
{
|
|
|
|
|
if (restarting_thread.joinable())
|
|
|
|
|
restarting_thread.join();
|
2014-03-22 14:44:44 +00:00
|
|
|
|
return;
|
2014-05-26 11:40:22 +00:00
|
|
|
|
}
|
2014-04-25 13:55:15 +00:00
|
|
|
|
permanent_shutdown_called = true;
|
|
|
|
|
restarting_thread.join();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
void StorageReplicatedMergeTree::partialShutdown()
|
|
|
|
|
{
|
2014-04-04 10:37:33 +00:00
|
|
|
|
leader_election = nullptr;
|
2014-03-22 14:44:44 +00:00
|
|
|
|
shutdown_called = true;
|
|
|
|
|
replica_is_active_node = nullptr;
|
|
|
|
|
|
2014-05-08 08:03:03 +00:00
|
|
|
|
merger.cancelAll();
|
|
|
|
|
if (unreplicated_merger)
|
|
|
|
|
unreplicated_merger->cancelAll();
|
|
|
|
|
|
2014-04-03 11:48:28 +00:00
|
|
|
|
LOG_TRACE(log, "Waiting for threads to finish");
|
2014-04-04 10:37:33 +00:00
|
|
|
|
if (is_leader_node)
|
2014-04-14 10:56:06 +00:00
|
|
|
|
{
|
2014-04-22 10:57:19 +00:00
|
|
|
|
is_leader_node = false;
|
2014-05-13 11:24:04 +00:00
|
|
|
|
if (merge_selecting_thread.joinable())
|
|
|
|
|
merge_selecting_thread.join();
|
|
|
|
|
if (clear_old_blocks_thread.joinable())
|
|
|
|
|
clear_old_blocks_thread.join();
|
2014-04-14 10:56:06 +00:00
|
|
|
|
}
|
2014-05-13 11:24:04 +00:00
|
|
|
|
if (queue_updating_thread.joinable())
|
|
|
|
|
queue_updating_thread.join();
|
2014-04-03 11:48:28 +00:00
|
|
|
|
for (auto & thread : queue_threads)
|
|
|
|
|
thread.join();
|
2014-04-25 13:55:15 +00:00
|
|
|
|
queue_threads.clear();
|
2014-04-03 11:48:28 +00:00
|
|
|
|
LOG_TRACE(log, "Threads finished");
|
2014-03-22 14:44:44 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-05-13 11:24:04 +00:00
|
|
|
|
void StorageReplicatedMergeTree::goReadOnly()
|
|
|
|
|
{
|
|
|
|
|
LOG_INFO(log, "Going to read-only mode");
|
|
|
|
|
|
|
|
|
|
is_read_only = true;
|
|
|
|
|
shutdown_called = true;
|
|
|
|
|
permanent_shutdown_called = true;
|
|
|
|
|
|
|
|
|
|
leader_election = nullptr;
|
|
|
|
|
replica_is_active_node = nullptr;
|
|
|
|
|
merger.cancelAll();
|
|
|
|
|
|
|
|
|
|
endpoint_holder = nullptr;
|
2014-05-26 11:40:22 +00:00
|
|
|
|
|
|
|
|
|
LOG_TRACE(log, "Waiting for threads to finish");
|
|
|
|
|
if (is_leader_node)
|
|
|
|
|
{
|
|
|
|
|
is_leader_node = false;
|
|
|
|
|
if (merge_selecting_thread.joinable())
|
|
|
|
|
merge_selecting_thread.join();
|
|
|
|
|
if (clear_old_blocks_thread.joinable())
|
|
|
|
|
clear_old_blocks_thread.join();
|
|
|
|
|
}
|
|
|
|
|
if (queue_updating_thread.joinable())
|
|
|
|
|
queue_updating_thread.join();
|
|
|
|
|
for (auto & thread : queue_threads)
|
|
|
|
|
thread.join();
|
|
|
|
|
queue_threads.clear();
|
|
|
|
|
LOG_TRACE(log, "Threads finished");
|
2014-05-13 11:24:04 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-04-22 10:57:19 +00:00
|
|
|
|
void StorageReplicatedMergeTree::startup()
|
|
|
|
|
{
|
|
|
|
|
shutdown_called = false;
|
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
merger.uncancelAll();
|
|
|
|
|
if (unreplicated_merger)
|
|
|
|
|
unreplicated_merger->uncancelAll();
|
|
|
|
|
|
2014-04-22 10:57:19 +00:00
|
|
|
|
activateReplica();
|
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
leader_election = new zkutil::LeaderElection(zookeeper_path + "/leader_election", *zookeeper,
|
2014-04-22 10:57:19 +00:00
|
|
|
|
std::bind(&StorageReplicatedMergeTree::becomeLeader, this), replica_name);
|
|
|
|
|
|
|
|
|
|
queue_updating_thread = std::thread(&StorageReplicatedMergeTree::queueUpdatingThread, this);
|
|
|
|
|
for (size_t i = 0; i < data.settings.replication_threads; ++i)
|
|
|
|
|
queue_threads.push_back(std::thread(&StorageReplicatedMergeTree::queueThread, this));
|
|
|
|
|
}
|
|
|
|
|
|
2014-04-25 13:55:15 +00:00
|
|
|
|
void StorageReplicatedMergeTree::restartingThread()
|
|
|
|
|
{
|
2014-05-13 10:10:26 +00:00
|
|
|
|
try
|
2014-04-25 13:55:15 +00:00
|
|
|
|
{
|
2014-05-13 10:10:26 +00:00
|
|
|
|
startup();
|
|
|
|
|
|
|
|
|
|
while (!permanent_shutdown_called)
|
2014-04-25 13:55:15 +00:00
|
|
|
|
{
|
2014-05-13 10:10:26 +00:00
|
|
|
|
if (zookeeper->expired())
|
|
|
|
|
{
|
|
|
|
|
LOG_WARNING(log, "ZooKeeper session has expired. Switching to a new session.");
|
2014-04-25 13:55:15 +00:00
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
/// Запретим писать в таблицу, пока подменяем zookeeper.
|
|
|
|
|
LOG_TRACE(log, "Locking all operations");
|
|
|
|
|
auto structure_lock = lockDataForAlter();
|
|
|
|
|
LOG_TRACE(log, "Locked all operations");
|
2014-04-25 13:55:15 +00:00
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
partialShutdown();
|
2014-04-25 13:55:15 +00:00
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper = context.getZooKeeper();
|
|
|
|
|
|
|
|
|
|
startup();
|
|
|
|
|
}
|
2014-04-25 13:55:15 +00:00
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
std::this_thread::sleep_for(std::chrono::seconds(2));
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException("StorageReplicatedMergeTree::restartingThread");
|
2014-05-13 11:24:04 +00:00
|
|
|
|
LOG_ERROR(log, "Exception in restartingThread. The storage will be read-only until server restart.");
|
|
|
|
|
goReadOnly();
|
2014-05-13 10:10:26 +00:00
|
|
|
|
return;
|
2014-04-25 13:55:15 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-05-13 10:10:26 +00:00
|
|
|
|
try
|
|
|
|
|
{
|
|
|
|
|
endpoint_holder = nullptr;
|
|
|
|
|
partialShutdown();
|
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException("StorageReplicatedMergeTree::restartingThread");
|
|
|
|
|
}
|
2014-04-25 13:55:15 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-03-22 14:44:44 +00:00
|
|
|
|
StorageReplicatedMergeTree::~StorageReplicatedMergeTree()
|
|
|
|
|
{
|
|
|
|
|
try
|
|
|
|
|
{
|
|
|
|
|
shutdown();
|
|
|
|
|
}
|
|
|
|
|
catch(...)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException("~StorageReplicatedMergeTree");
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
BlockInputStreams StorageReplicatedMergeTree::read(
|
|
|
|
|
const Names & column_names,
|
|
|
|
|
ASTPtr query,
|
|
|
|
|
const Settings & settings,
|
|
|
|
|
QueryProcessingStage::Enum & processed_stage,
|
|
|
|
|
size_t max_block_size,
|
|
|
|
|
unsigned threads)
|
|
|
|
|
{
|
2014-04-24 10:20:02 +00:00
|
|
|
|
BlockInputStreams res = reader.read(column_names, query, settings, processed_stage, max_block_size, threads);
|
|
|
|
|
|
|
|
|
|
if (unreplicated_reader)
|
|
|
|
|
{
|
|
|
|
|
BlockInputStreams res2 = unreplicated_reader->read(column_names, query, settings, processed_stage, max_block_size, threads);
|
|
|
|
|
res.insert(res.begin(), res2.begin(), res2.end());
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
return res;
|
2014-03-22 14:44:44 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-04-02 10:10:37 +00:00
|
|
|
|
BlockOutputStreamPtr StorageReplicatedMergeTree::write(ASTPtr query)
|
|
|
|
|
{
|
2014-05-13 11:24:04 +00:00
|
|
|
|
if (is_read_only)
|
|
|
|
|
throw Exception("Table is in read only mode", ErrorCodes::TABLE_IS_READ_ONLY);
|
|
|
|
|
|
2014-04-02 10:10:37 +00:00
|
|
|
|
String insert_id;
|
|
|
|
|
if (ASTInsertQuery * insert = dynamic_cast<ASTInsertQuery *>(&*query))
|
|
|
|
|
insert_id = insert->insert_id;
|
|
|
|
|
|
|
|
|
|
return new ReplicatedMergeTreeBlockOutputStream(*this, insert_id);
|
|
|
|
|
}
|
2014-03-22 14:44:44 +00:00
|
|
|
|
|
2014-05-08 08:03:03 +00:00
|
|
|
|
bool StorageReplicatedMergeTree::optimize()
|
|
|
|
|
{
|
|
|
|
|
/// Померджим какие-нибудь куски из директории unreplicated. TODO: Мерджить реплицируемые куски тоже.
|
|
|
|
|
|
|
|
|
|
if (!unreplicated_data)
|
|
|
|
|
return false;
|
|
|
|
|
|
|
|
|
|
unreplicated_data->clearOldParts();
|
|
|
|
|
|
|
|
|
|
MergeTreeData::DataPartsVector parts;
|
|
|
|
|
String merged_name;
|
|
|
|
|
auto always_can_merge = [](const MergeTreeData::DataPartPtr &a, const MergeTreeData::DataPartPtr &b) { return true; };
|
|
|
|
|
if (!unreplicated_merger->selectPartsToMerge(parts, merged_name, 0, true, true, false, always_can_merge))
|
|
|
|
|
return false;
|
|
|
|
|
|
|
|
|
|
unreplicated_merger->mergeParts(parts, merged_name);
|
|
|
|
|
return true;
|
|
|
|
|
}
|
|
|
|
|
|
2014-03-22 14:44:44 +00:00
|
|
|
|
void StorageReplicatedMergeTree::drop()
|
|
|
|
|
{
|
2014-04-03 11:48:28 +00:00
|
|
|
|
shutdown();
|
|
|
|
|
|
2014-05-27 12:08:40 +00:00
|
|
|
|
LOG_INFO(log, "Removing replica " << replica_path);
|
2014-03-22 14:44:44 +00:00
|
|
|
|
replica_is_active_node = nullptr;
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper->removeRecursive(replica_path);
|
|
|
|
|
if (zookeeper->getChildren(zookeeper_path + "/replicas").empty())
|
2014-05-27 12:08:40 +00:00
|
|
|
|
{
|
|
|
|
|
LOG_INFO(log, "Removing table " << zookeeper_path << " (this might take several minutes)");
|
2014-05-13 10:10:26 +00:00
|
|
|
|
zookeeper->removeRecursive(zookeeper_path);
|
2014-05-27 12:08:40 +00:00
|
|
|
|
}
|
2014-03-22 14:44:44 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-04-02 10:10:37 +00:00
|
|
|
|
void StorageReplicatedMergeTree::LogEntry::writeText(WriteBuffer & out) const
|
|
|
|
|
{
|
|
|
|
|
writeString("format version: 1\n", out);
|
2014-04-07 15:45:46 +00:00
|
|
|
|
writeString("source replica: ", out);
|
|
|
|
|
writeString(source_replica, out);
|
|
|
|
|
writeString("\n", out);
|
2014-04-02 10:10:37 +00:00
|
|
|
|
switch (type)
|
|
|
|
|
{
|
|
|
|
|
case GET_PART:
|
|
|
|
|
writeString("get\n", out);
|
|
|
|
|
writeString(new_part_name, out);
|
|
|
|
|
break;
|
|
|
|
|
case MERGE_PARTS:
|
|
|
|
|
writeString("merge\n", out);
|
|
|
|
|
for (const String & s : parts_to_merge)
|
|
|
|
|
{
|
|
|
|
|
writeString(s, out);
|
|
|
|
|
writeString("\n", out);
|
|
|
|
|
}
|
|
|
|
|
writeString("into\n", out);
|
|
|
|
|
writeString(new_part_name, out);
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
writeString("\n", out);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
void StorageReplicatedMergeTree::LogEntry::readText(ReadBuffer & in)
|
|
|
|
|
{
|
|
|
|
|
String type_str;
|
|
|
|
|
|
|
|
|
|
assertString("format version: 1\n", in);
|
2014-04-07 17:39:45 +00:00
|
|
|
|
assertString("source replica: ", in);
|
2014-04-07 15:45:46 +00:00
|
|
|
|
readString(source_replica, in);
|
2014-04-02 10:10:37 +00:00
|
|
|
|
assertString("\n", in);
|
2014-04-07 17:39:45 +00:00
|
|
|
|
readString(type_str, in);
|
|
|
|
|
assertString("\n", in);
|
2014-04-02 10:10:37 +00:00
|
|
|
|
|
|
|
|
|
if (type_str == "get")
|
|
|
|
|
{
|
|
|
|
|
type = GET_PART;
|
|
|
|
|
readString(new_part_name, in);
|
|
|
|
|
}
|
|
|
|
|
else if (type_str == "merge")
|
|
|
|
|
{
|
|
|
|
|
type = MERGE_PARTS;
|
|
|
|
|
while (true)
|
|
|
|
|
{
|
|
|
|
|
String s;
|
|
|
|
|
readString(s, in);
|
|
|
|
|
assertString("\n", in);
|
|
|
|
|
if (s == "into")
|
|
|
|
|
break;
|
|
|
|
|
parts_to_merge.push_back(s);
|
|
|
|
|
}
|
|
|
|
|
readString(new_part_name, in);
|
|
|
|
|
}
|
|
|
|
|
assertString("\n", in);
|
|
|
|
|
}
|
|
|
|
|
|
2014-03-21 13:42:14 +00:00
|
|
|
|
}
|