2014-11-19 20:40:51 +00:00
|
|
|
|
#include <DB/IO/Operators.h>
|
2014-10-17 01:05:51 +00:00
|
|
|
|
#include <DB/Storages/StorageReplicatedMergeTree.h>
|
|
|
|
|
#include <DB/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h>
|
2015-09-20 11:02:59 +00:00
|
|
|
|
#include <DB/Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h>
|
2015-10-02 18:33:46 +00:00
|
|
|
|
#include <DB/Storages/MergeTree/ReplicatedMergeTreeAddress.h>
|
2015-09-24 18:54:21 +00:00
|
|
|
|
#include <DB/Common/setThreadName.h>
|
2014-10-17 01:05:51 +00:00
|
|
|
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
|
{
|
|
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
|
namespace ErrorCodes
|
|
|
|
|
{
|
|
|
|
|
extern const int CANNOT_CLOCK_GETTIME;
|
|
|
|
|
extern const int REPLICA_IS_ALREADY_ACTIVE;
|
|
|
|
|
}
|
|
|
|
|
|
2014-10-17 01:05:51 +00:00
|
|
|
|
|
|
|
|
|
/// Используется для проверки, выставили ли ноду is_active мы, или нет.
|
|
|
|
|
static String generateActiveNodeIdentifier()
|
|
|
|
|
{
|
|
|
|
|
struct timespec times;
|
|
|
|
|
if (clock_gettime(CLOCK_THREAD_CPUTIME_ID, ×))
|
|
|
|
|
throwFromErrno("Cannot clock_gettime.", ErrorCodes::CANNOT_CLOCK_GETTIME);
|
2014-12-21 01:20:07 +00:00
|
|
|
|
return "pid: " + toString(getpid()) + ", random: " + toString(times.tv_nsec + times.tv_sec + getpid());
|
2014-10-17 01:05:51 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
ReplicatedMergeTreeRestartingThread::ReplicatedMergeTreeRestartingThread(StorageReplicatedMergeTree & storage_)
|
|
|
|
|
: storage(storage_),
|
2014-10-18 19:14:09 +00:00
|
|
|
|
log(&Logger::get(storage.database_name + "." + storage.table_name + " (StorageReplicatedMergeTree, RestartingThread)")),
|
2014-10-18 17:37:55 +00:00
|
|
|
|
active_node_identifier(generateActiveNodeIdentifier()),
|
2014-10-17 01:05:51 +00:00
|
|
|
|
thread([this] { run(); })
|
|
|
|
|
{
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
void ReplicatedMergeTreeRestartingThread::run()
|
|
|
|
|
{
|
2015-11-05 19:44:19 +00:00
|
|
|
|
constexpr auto retry_period_ms = 10 * 1000;
|
|
|
|
|
|
|
|
|
|
/// Периодичность проверки истечения сессии в ZK.
|
|
|
|
|
time_t check_period_ms = 60 * 1000;
|
|
|
|
|
|
|
|
|
|
/// Периодичность проверки величины отставания реплики.
|
|
|
|
|
if (check_period_ms > static_cast<time_t>(storage.data.settings.check_delay_period) * 1000)
|
|
|
|
|
check_period_ms = storage.data.settings.check_delay_period * 1000;
|
2014-12-07 22:59:00 +00:00
|
|
|
|
|
2015-09-24 18:54:21 +00:00
|
|
|
|
setThreadName("ReplMTRestart");
|
|
|
|
|
|
2014-10-17 01:05:51 +00:00
|
|
|
|
try
|
|
|
|
|
{
|
2015-11-05 19:44:19 +00:00
|
|
|
|
bool first_time = true; /// Активация реплики в первый раз.
|
|
|
|
|
bool need_restart = false; /// Перезапуск по собственной инициативе, чтобы отдать лидерство.
|
|
|
|
|
time_t prev_time_of_check_delay = 0;
|
2014-10-17 01:05:51 +00:00
|
|
|
|
|
2014-12-11 01:56:42 +00:00
|
|
|
|
/// Запуск реплики при старте сервера/создании таблицы. Перезапуск реплики при истечении сессии с ZK.
|
2014-10-17 01:05:51 +00:00
|
|
|
|
while (!need_stop)
|
|
|
|
|
{
|
2015-11-05 19:44:19 +00:00
|
|
|
|
if (first_time || need_restart || storage.getZooKeeper()->expired())
|
2014-10-17 01:05:51 +00:00
|
|
|
|
{
|
2014-12-11 01:56:42 +00:00
|
|
|
|
if (first_time)
|
|
|
|
|
{
|
|
|
|
|
LOG_DEBUG(log, "Activating replica.");
|
|
|
|
|
}
|
|
|
|
|
else
|
|
|
|
|
{
|
2015-11-05 19:44:19 +00:00
|
|
|
|
if (need_restart)
|
|
|
|
|
LOG_WARNING(log, "Will reactivate replica.");
|
|
|
|
|
else
|
|
|
|
|
LOG_WARNING(log, "ZooKeeper session has expired. Switching to a new session.");
|
2014-10-17 01:05:51 +00:00
|
|
|
|
|
2016-01-21 01:47:28 +00:00
|
|
|
|
if (!storage.is_readonly)
|
|
|
|
|
CurrentMetrics::add(CurrentMetrics::ReadonlyReplica);
|
2014-12-11 02:04:13 +00:00
|
|
|
|
storage.is_readonly = true;
|
2014-12-11 01:56:42 +00:00
|
|
|
|
partialShutdown();
|
|
|
|
|
}
|
2014-11-28 20:45:33 +00:00
|
|
|
|
|
2014-12-21 01:20:07 +00:00
|
|
|
|
while (true)
|
2014-11-28 20:45:33 +00:00
|
|
|
|
{
|
|
|
|
|
try
|
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
storage.setZooKeeper(storage.context.getZooKeeper());
|
2014-11-28 20:45:33 +00:00
|
|
|
|
}
|
|
|
|
|
catch (const zkutil::KeeperException & e)
|
|
|
|
|
{
|
|
|
|
|
/// Исключение при попытке zookeeper_init обычно бывает, если не работает DNS. Будем пытаться сделать это заново.
|
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
2014-12-21 01:20:07 +00:00
|
|
|
|
|
2015-11-05 19:44:19 +00:00
|
|
|
|
wakeup_event.tryWait(retry_period_ms);
|
2014-11-28 20:45:33 +00:00
|
|
|
|
continue;
|
|
|
|
|
}
|
2014-10-17 01:05:51 +00:00
|
|
|
|
|
2014-12-11 01:56:42 +00:00
|
|
|
|
if (!need_stop && !tryStartup())
|
|
|
|
|
{
|
2015-11-05 19:44:19 +00:00
|
|
|
|
wakeup_event.tryWait(retry_period_ms);
|
2014-12-11 01:56:42 +00:00
|
|
|
|
continue;
|
|
|
|
|
}
|
2014-12-21 01:20:07 +00:00
|
|
|
|
|
|
|
|
|
break;
|
|
|
|
|
}
|
2014-10-17 01:05:51 +00:00
|
|
|
|
|
2016-01-21 01:47:28 +00:00
|
|
|
|
if (storage.is_readonly)
|
|
|
|
|
CurrentMetrics::sub(CurrentMetrics::ReadonlyReplica);
|
2014-12-11 02:04:13 +00:00
|
|
|
|
storage.is_readonly = false;
|
2014-12-11 01:56:42 +00:00
|
|
|
|
first_time = false;
|
2015-11-05 19:44:19 +00:00
|
|
|
|
need_restart = false;
|
2014-10-17 01:05:51 +00:00
|
|
|
|
}
|
|
|
|
|
|
2015-11-05 19:44:19 +00:00
|
|
|
|
time_t current_time = time(0);
|
|
|
|
|
if (current_time >= prev_time_of_check_delay + static_cast<time_t>(storage.data.settings.check_delay_period))
|
|
|
|
|
{
|
|
|
|
|
/// Выясняем отставания реплик.
|
2016-01-17 13:00:42 +00:00
|
|
|
|
time_t absolute_delay = 0;
|
|
|
|
|
time_t relative_delay = 0;
|
2015-11-05 19:44:19 +00:00
|
|
|
|
|
2016-01-17 13:00:42 +00:00
|
|
|
|
bool error = false;
|
|
|
|
|
try
|
|
|
|
|
{
|
|
|
|
|
storage.getReplicaDelays(absolute_delay, relative_delay);
|
|
|
|
|
LOG_TRACE(log, "Absolute delay: " << absolute_delay << ". Relative delay: " << relative_delay << ".");
|
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
2016-01-21 01:47:28 +00:00
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__, "Cannot get replica delays");
|
2016-01-17 13:00:42 +00:00
|
|
|
|
error = true;
|
|
|
|
|
}
|
2015-11-05 19:44:19 +00:00
|
|
|
|
|
|
|
|
|
prev_time_of_check_delay = current_time;
|
2015-11-05 17:38:24 +00:00
|
|
|
|
|
2015-11-05 19:44:19 +00:00
|
|
|
|
/// Уступаем лидерство, если относительное отставание больше порога.
|
2016-01-17 13:00:42 +00:00
|
|
|
|
if (storage.is_leader_node
|
|
|
|
|
&& (error || relative_delay > static_cast<time_t>(storage.data.settings.min_relative_delay_to_yield_leadership)))
|
2015-11-05 19:44:19 +00:00
|
|
|
|
{
|
2016-01-17 13:00:42 +00:00
|
|
|
|
if (error)
|
|
|
|
|
LOG_INFO(log, "Will yield leadership.");
|
|
|
|
|
else
|
|
|
|
|
LOG_INFO(log, "Relative replica delay (" << relative_delay << " seconds) is bigger than threshold ("
|
|
|
|
|
<< storage.data.settings.min_relative_delay_to_yield_leadership << "). Will yield leadership.");
|
2015-11-05 19:44:19 +00:00
|
|
|
|
|
2016-01-20 20:43:42 +00:00
|
|
|
|
ProfileEvents::increment(ProfileEvents::ReplicaYieldLeadership);
|
|
|
|
|
|
2015-11-05 19:44:19 +00:00
|
|
|
|
need_restart = true;
|
|
|
|
|
continue;
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
wakeup_event.tryWait(check_period_ms);
|
2014-10-17 01:05:51 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException("StorageReplicatedMergeTree::restartingThread");
|
2014-12-11 02:04:13 +00:00
|
|
|
|
LOG_ERROR(log, "Unexpected exception in restartingThread. The storage will be readonly until server restart.");
|
2014-10-17 01:05:51 +00:00
|
|
|
|
goReadOnlyPermanently();
|
|
|
|
|
LOG_DEBUG(log, "Restarting thread finished");
|
|
|
|
|
return;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
try
|
|
|
|
|
{
|
2015-12-24 21:28:18 +00:00
|
|
|
|
storage.endpoint_holder->cancel();
|
2014-10-17 01:05:51 +00:00
|
|
|
|
storage.endpoint_holder = nullptr;
|
2016-01-28 01:00:27 +00:00
|
|
|
|
|
|
|
|
|
storage.disk_space_monitor_endpoint_holder->cancel();
|
|
|
|
|
storage.disk_space_monitor_endpoint_holder = nullptr;
|
|
|
|
|
|
2016-03-01 17:47:53 +00:00
|
|
|
|
storage.sharded_partition_uploader_endpoint_holder->cancel();
|
|
|
|
|
storage.sharded_partition_uploader_endpoint_holder = nullptr;
|
2016-01-28 01:00:27 +00:00
|
|
|
|
|
|
|
|
|
storage.remote_query_executor_endpoint_holder->cancel();
|
|
|
|
|
storage.remote_query_executor_endpoint_holder = nullptr;
|
|
|
|
|
|
2014-10-17 01:05:51 +00:00
|
|
|
|
partialShutdown();
|
|
|
|
|
}
|
|
|
|
|
catch (...)
|
|
|
|
|
{
|
|
|
|
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
LOG_DEBUG(log, "Restarting thread finished");
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
bool ReplicatedMergeTreeRestartingThread::tryStartup()
|
|
|
|
|
{
|
|
|
|
|
try
|
|
|
|
|
{
|
2015-09-20 11:02:59 +00:00
|
|
|
|
removeFailedQuorumParts();
|
2014-10-17 01:05:51 +00:00
|
|
|
|
activateReplica();
|
2015-09-20 11:02:59 +00:00
|
|
|
|
updateQuorumIfWeHavePart();
|
2014-10-17 01:05:51 +00:00
|
|
|
|
|
2014-12-11 01:56:42 +00:00
|
|
|
|
storage.leader_election = new zkutil::LeaderElection(
|
|
|
|
|
storage.zookeeper_path + "/leader_election",
|
2014-12-12 20:50:32 +00:00
|
|
|
|
*storage.current_zookeeper, /// current_zookeeper живёт в течение времени жизни leader_election,
|
|
|
|
|
/// так как до изменения current_zookeeper, объект leader_election уничтожается в методе partialShutdown.
|
2014-12-11 01:56:42 +00:00
|
|
|
|
[this] { storage.becomeLeader(); },
|
|
|
|
|
storage.replica_name);
|
2014-10-17 01:05:51 +00:00
|
|
|
|
|
|
|
|
|
/// Все, что выше, может бросить KeeperException, если что-то не так с ZK.
|
|
|
|
|
/// Все, что ниже, не должно бросать исключений.
|
|
|
|
|
|
|
|
|
|
storage.shutdown_called = false;
|
|
|
|
|
storage.shutdown_event.reset();
|
|
|
|
|
|
|
|
|
|
storage.queue_updating_thread = std::thread(&StorageReplicatedMergeTree::queueUpdatingThread, &storage);
|
|
|
|
|
storage.cleanup_thread.reset(new ReplicatedMergeTreeCleanupThread(storage));
|
|
|
|
|
storage.alter_thread = std::thread(&StorageReplicatedMergeTree::alterThread, &storage);
|
|
|
|
|
storage.part_check_thread = std::thread(&StorageReplicatedMergeTree::partCheckThread, &storage);
|
|
|
|
|
storage.queue_task_handle = storage.context.getBackgroundPool().addTask(
|
|
|
|
|
std::bind(&StorageReplicatedMergeTree::queueTask, &storage, std::placeholders::_1));
|
|
|
|
|
storage.queue_task_handle->wake();
|
2014-12-21 01:20:07 +00:00
|
|
|
|
|
2015-11-06 17:34:48 +00:00
|
|
|
|
storage.merger.uncancel();
|
|
|
|
|
if (storage.unreplicated_merger)
|
|
|
|
|
storage.unreplicated_merger->uncancel();
|
|
|
|
|
|
2014-10-17 01:05:51 +00:00
|
|
|
|
return true;
|
|
|
|
|
}
|
2014-12-21 01:20:07 +00:00
|
|
|
|
catch (...)
|
2014-10-17 01:05:51 +00:00
|
|
|
|
{
|
2014-12-21 01:20:07 +00:00
|
|
|
|
storage.replica_is_active_node = nullptr;
|
|
|
|
|
storage.leader_election = nullptr;
|
|
|
|
|
|
|
|
|
|
try
|
|
|
|
|
{
|
2014-10-17 01:05:51 +00:00
|
|
|
|
throw;
|
2014-12-21 01:20:07 +00:00
|
|
|
|
}
|
|
|
|
|
catch (const zkutil::KeeperException & e)
|
|
|
|
|
{
|
|
|
|
|
LOG_ERROR(log, "Couldn't start replication: " << e.what() << ", " << e.displayText() << ", stack trace:\n" << e.getStackTrace().toString());
|
|
|
|
|
return false;
|
|
|
|
|
}
|
|
|
|
|
catch (const Exception & e)
|
|
|
|
|
{
|
|
|
|
|
if (e.code() != ErrorCodes::REPLICA_IS_ALREADY_ACTIVE)
|
|
|
|
|
throw;
|
2014-10-17 01:05:51 +00:00
|
|
|
|
|
2014-12-21 01:20:07 +00:00
|
|
|
|
LOG_ERROR(log, "Couldn't start replication: " << e.what() << ", " << e.displayText() << ", stack trace:\n" << e.getStackTrace().toString());
|
|
|
|
|
return false;
|
|
|
|
|
}
|
2014-10-17 01:05:51 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2015-09-20 11:02:59 +00:00
|
|
|
|
void ReplicatedMergeTreeRestartingThread::removeFailedQuorumParts()
|
|
|
|
|
{
|
|
|
|
|
auto zookeeper = storage.getZooKeeper();
|
|
|
|
|
|
|
|
|
|
Strings failed_parts;
|
|
|
|
|
if (!zookeeper->tryGetChildren(storage.zookeeper_path + "/quorum/failed_parts", failed_parts))
|
|
|
|
|
return;
|
|
|
|
|
|
|
|
|
|
for (auto part_name : failed_parts)
|
|
|
|
|
{
|
|
|
|
|
auto part = storage.data.getPartIfExists(part_name);
|
|
|
|
|
if (part)
|
|
|
|
|
{
|
|
|
|
|
LOG_DEBUG(log, "Found part " << part_name << " with failed quorum. Moving to detached. This shouldn't happen often.");
|
|
|
|
|
|
|
|
|
|
zkutil::Ops ops;
|
|
|
|
|
storage.removePartFromZooKeeper(part_name, ops);
|
|
|
|
|
auto code = zookeeper->tryMulti(ops);
|
|
|
|
|
if (code == ZNONODE)
|
|
|
|
|
LOG_WARNING(log, "Part " << part_name << " with failed quorum is not in ZooKeeper. This shouldn't happen often.");
|
|
|
|
|
|
|
|
|
|
storage.data.renameAndDetachPart(part, "noquorum");
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
void ReplicatedMergeTreeRestartingThread::updateQuorumIfWeHavePart()
|
|
|
|
|
{
|
|
|
|
|
auto zookeeper = storage.getZooKeeper();
|
|
|
|
|
|
|
|
|
|
String quorum_str;
|
|
|
|
|
if (zookeeper->tryGet(storage.zookeeper_path + "/quorum/status", quorum_str))
|
|
|
|
|
{
|
|
|
|
|
ReplicatedMergeTreeQuorumEntry quorum_entry;
|
|
|
|
|
quorum_entry.fromString(quorum_str);
|
|
|
|
|
|
|
|
|
|
if (!quorum_entry.replicas.count(storage.replica_name)
|
|
|
|
|
&& zookeeper->exists(storage.replica_path + "/parts/" + quorum_entry.part_name))
|
|
|
|
|
{
|
|
|
|
|
LOG_WARNING(log, "We have part " << quorum_entry.part_name
|
|
|
|
|
<< " but we is not in quorum. Updating quorum. This shouldn't happen often.");
|
|
|
|
|
storage.updateQuorum(quorum_entry.part_name);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
2014-10-17 01:05:51 +00:00
|
|
|
|
void ReplicatedMergeTreeRestartingThread::activateReplica()
|
|
|
|
|
{
|
2014-11-19 20:40:51 +00:00
|
|
|
|
auto host_port = storage.context.getInterserverIOAddress();
|
2014-12-12 20:50:32 +00:00
|
|
|
|
auto zookeeper = storage.getZooKeeper();
|
2014-11-19 20:40:51 +00:00
|
|
|
|
|
2015-10-02 18:33:46 +00:00
|
|
|
|
/// Как другие реплики могут обращаться к данной.
|
|
|
|
|
ReplicatedMergeTreeAddress address;
|
|
|
|
|
address.host = host_port.first;
|
|
|
|
|
address.replication_port = host_port.second;
|
|
|
|
|
address.queries_port = storage.context.getTCPPort();
|
|
|
|
|
address.database = storage.database_name;
|
|
|
|
|
address.table = storage.table_name;
|
2014-10-17 01:05:51 +00:00
|
|
|
|
|
2015-09-20 11:02:59 +00:00
|
|
|
|
String is_active_path = storage.replica_path + "/is_active";
|
|
|
|
|
|
2014-10-17 01:05:51 +00:00
|
|
|
|
/** Если нода отмечена как активная, но отметка сделана в этом же экземпляре, удалим ее.
|
|
|
|
|
* Такое возможно только при истечении сессии в ZooKeeper.
|
|
|
|
|
*/
|
|
|
|
|
String data;
|
2015-09-20 11:02:59 +00:00
|
|
|
|
Stat stat;
|
|
|
|
|
bool has_is_active = zookeeper->tryGet(is_active_path, data, &stat);
|
|
|
|
|
if (has_is_active && data == active_node_identifier)
|
|
|
|
|
{
|
|
|
|
|
auto code = zookeeper->tryRemove(is_active_path, stat.version);
|
|
|
|
|
|
|
|
|
|
if (code == ZBADVERSION)
|
|
|
|
|
throw Exception("Another instance of replica " + storage.replica_path + " was created just now."
|
|
|
|
|
" You shouldn't run multiple instances of same replica. You need to check configuration files.",
|
|
|
|
|
ErrorCodes::REPLICA_IS_ALREADY_ACTIVE);
|
|
|
|
|
|
|
|
|
|
if (code != ZOK && code != ZNONODE)
|
|
|
|
|
throw zkutil::KeeperException(code, is_active_path);
|
|
|
|
|
}
|
2014-10-17 01:05:51 +00:00
|
|
|
|
|
|
|
|
|
/// Одновременно объявим, что эта реплика активна, и обновим хост.
|
|
|
|
|
zkutil::Ops ops;
|
2015-09-20 11:02:59 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::Create(is_active_path,
|
2014-12-12 20:50:32 +00:00
|
|
|
|
active_node_identifier, zookeeper->getDefaultACL(), zkutil::CreateMode::Ephemeral));
|
2015-10-02 18:33:46 +00:00
|
|
|
|
ops.push_back(new zkutil::Op::SetData(storage.replica_path + "/host", address.toString(), -1));
|
2014-10-17 01:05:51 +00:00
|
|
|
|
|
|
|
|
|
try
|
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
|
zookeeper->multi(ops);
|
2014-10-17 01:05:51 +00:00
|
|
|
|
}
|
|
|
|
|
catch (const zkutil::KeeperException & e)
|
|
|
|
|
{
|
|
|
|
|
if (e.code == ZNODEEXISTS)
|
|
|
|
|
throw Exception("Replica " + storage.replica_path + " appears to be already active. If you're sure it's not, "
|
|
|
|
|
"try again in a minute or remove znode " + storage.replica_path + "/is_active manually", ErrorCodes::REPLICA_IS_ALREADY_ACTIVE);
|
|
|
|
|
|
|
|
|
|
throw;
|
|
|
|
|
}
|
|
|
|
|
|
2014-12-12 20:50:32 +00:00
|
|
|
|
/// current_zookeeper живёт в течение времени жизни replica_is_active_node,
|
|
|
|
|
/// так как до изменения current_zookeeper, объект replica_is_active_node уничтожается в методе partialShutdown.
|
2015-09-20 11:02:59 +00:00
|
|
|
|
storage.replica_is_active_node = zkutil::EphemeralNodeHolder::existing(is_active_path, *storage.current_zookeeper);
|
2014-10-17 01:05:51 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
void ReplicatedMergeTreeRestartingThread::partialShutdown()
|
|
|
|
|
{
|
2016-01-21 01:47:28 +00:00
|
|
|
|
ProfileEvents::increment(ProfileEvents::ReplicaPartialShutdown);
|
|
|
|
|
|
2014-10-17 01:05:51 +00:00
|
|
|
|
storage.leader_election = nullptr;
|
|
|
|
|
storage.shutdown_called = true;
|
|
|
|
|
storage.shutdown_event.set();
|
|
|
|
|
storage.merge_selecting_event.set();
|
|
|
|
|
storage.queue_updating_event->set();
|
|
|
|
|
storage.alter_thread_event->set();
|
|
|
|
|
storage.alter_query_event->set();
|
|
|
|
|
storage.parts_to_check_event.set();
|
|
|
|
|
storage.replica_is_active_node = nullptr;
|
|
|
|
|
|
2015-11-06 17:34:48 +00:00
|
|
|
|
storage.merger.cancel();
|
2014-10-17 01:05:51 +00:00
|
|
|
|
if (storage.unreplicated_merger)
|
2015-11-06 17:34:48 +00:00
|
|
|
|
storage.unreplicated_merger->cancel();
|
2014-10-17 01:05:51 +00:00
|
|
|
|
|
|
|
|
|
LOG_TRACE(log, "Waiting for threads to finish");
|
|
|
|
|
if (storage.is_leader_node)
|
|
|
|
|
{
|
|
|
|
|
storage.is_leader_node = false;
|
|
|
|
|
if (storage.merge_selecting_thread.joinable())
|
|
|
|
|
storage.merge_selecting_thread.join();
|
|
|
|
|
}
|
|
|
|
|
if (storage.queue_updating_thread.joinable())
|
|
|
|
|
storage.queue_updating_thread.join();
|
|
|
|
|
|
|
|
|
|
storage.cleanup_thread.reset();
|
|
|
|
|
|
|
|
|
|
if (storage.alter_thread.joinable())
|
|
|
|
|
storage.alter_thread.join();
|
|
|
|
|
if (storage.part_check_thread.joinable())
|
|
|
|
|
storage.part_check_thread.join();
|
|
|
|
|
if (storage.queue_task_handle)
|
|
|
|
|
storage.context.getBackgroundPool().removeTask(storage.queue_task_handle);
|
|
|
|
|
storage.queue_task_handle.reset();
|
|
|
|
|
LOG_TRACE(log, "Threads finished");
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
void ReplicatedMergeTreeRestartingThread::goReadOnlyPermanently()
|
|
|
|
|
{
|
2014-12-11 02:04:13 +00:00
|
|
|
|
LOG_INFO(log, "Going to readonly mode");
|
2016-01-21 01:47:28 +00:00
|
|
|
|
ProfileEvents::increment(ProfileEvents::ReplicaPermanentlyReadonly);
|
2014-10-17 01:05:51 +00:00
|
|
|
|
|
2016-01-21 01:47:28 +00:00
|
|
|
|
if (!storage.is_readonly)
|
|
|
|
|
CurrentMetrics::add(CurrentMetrics::ReadonlyReplica);
|
2014-12-11 02:06:22 +00:00
|
|
|
|
storage.is_readonly = true;
|
2014-10-17 01:05:51 +00:00
|
|
|
|
stop();
|
|
|
|
|
|
|
|
|
|
partialShutdown();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
}
|