2017-04-01 09:19:00 +00:00
|
|
|
#include <IO/Operators.h>
|
|
|
|
#include <Storages/StorageReplicatedMergeTree.h>
|
|
|
|
#include <Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h>
|
|
|
|
#include <Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h>
|
|
|
|
#include <Storages/MergeTree/ReplicatedMergeTreeAddress.h>
|
2020-05-20 20:16:32 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2018-04-03 17:35:48 +00:00
|
|
|
#include <Common/ZooKeeper/KeeperException.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Common/randomSeed.h>
|
2021-11-18 18:07:35 +00:00
|
|
|
#include <boost/algorithm/string/replace.hpp>
|
2014-10-17 01:05:51 +00:00
|
|
|
|
|
|
|
|
2016-10-24 02:02:37 +00:00
|
|
|
namespace ProfileEvents
|
|
|
|
{
|
|
|
|
extern const Event ReplicaPartialShutdown;
|
|
|
|
}
|
|
|
|
|
2016-10-24 04:06:27 +00:00
|
|
|
namespace CurrentMetrics
|
|
|
|
{
|
|
|
|
extern const Metric ReadonlyReplica;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2014-10-17 01:05:51 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int REPLICA_IS_ALREADY_ACTIVE;
|
2021-08-18 09:49:22 +00:00
|
|
|
extern const int REPLICA_STATUS_CHANGED;
|
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
2017-12-29 22:32:04 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
constexpr auto retry_period_ms = 10 * 1000;
|
|
|
|
}
|
2014-10-17 01:05:51 +00:00
|
|
|
|
2017-03-12 19:18:07 +00:00
|
|
|
/// Used to check whether it's us who set node `is_active`, or not.
|
2014-10-17 01:05:51 +00:00
|
|
|
static String generateActiveNodeIdentifier()
|
|
|
|
{
|
2016-07-31 03:53:16 +00:00
|
|
|
return "pid: " + toString(getpid()) + ", random: " + toString(randomSeed());
|
2014-10-17 01:05:51 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
ReplicatedMergeTreeRestartingThread::ReplicatedMergeTreeRestartingThread(StorageReplicatedMergeTree & storage_)
|
2018-05-31 13:05:05 +00:00
|
|
|
: storage(storage_)
|
2019-12-03 16:25:32 +00:00
|
|
|
, log_name(storage.getStorageID().getFullTableName() + " (ReplicatedMergeTreeRestartingThread)")
|
2020-05-30 21:57:37 +00:00
|
|
|
, log(&Poco::Logger::get(log_name))
|
2018-05-31 13:05:05 +00:00
|
|
|
, active_node_identifier(generateActiveNodeIdentifier())
|
2014-10-17 01:05:51 +00:00
|
|
|
{
|
2019-08-26 14:24:29 +00:00
|
|
|
const auto storage_settings = storage.getSettings();
|
2019-08-13 10:29:31 +00:00
|
|
|
check_period_ms = storage_settings->zookeeper_session_expiration_check_period.totalSeconds() * 1000;
|
2017-12-29 22:32:04 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
task = storage.getContext()->getSchedulePool().createTask(log_name, [this]{ run(); });
|
2017-12-29 22:32:04 +00:00
|
|
|
}
|
2014-10-17 01:05:51 +00:00
|
|
|
|
|
|
|
void ReplicatedMergeTreeRestartingThread::run()
|
|
|
|
{
|
2017-12-29 22:32:04 +00:00
|
|
|
if (need_stop)
|
|
|
|
return;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2022-02-03 10:10:05 +00:00
|
|
|
size_t reschedule_period_ms = check_period_ms;
|
|
|
|
|
2017-12-29 22:32:04 +00:00
|
|
|
try
|
|
|
|
{
|
2022-02-03 10:10:05 +00:00
|
|
|
bool replica_is_active = runImpl();
|
|
|
|
if (!replica_is_active)
|
|
|
|
reschedule_period_ms = retry_period_ms;
|
2017-12-21 18:17:06 +00:00
|
|
|
}
|
2021-08-18 09:49:22 +00:00
|
|
|
catch (const Exception & e)
|
2017-12-29 22:32:04 +00:00
|
|
|
{
|
2021-05-14 08:32:41 +00:00
|
|
|
/// We couldn't activate table let's set it into readonly mode
|
2021-08-18 09:49:22 +00:00
|
|
|
partialShutdown();
|
|
|
|
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-08-18 09:49:22 +00:00
|
|
|
if (e.code() == ErrorCodes::REPLICA_STATUS_CHANGED)
|
2022-02-03 10:10:05 +00:00
|
|
|
reschedule_period_ms = 0;
|
2021-08-18 09:49:22 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
partialShutdown();
|
2018-04-10 13:20:14 +00:00
|
|
|
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
2017-12-21 18:17:06 +00:00
|
|
|
}
|
|
|
|
|
2022-02-03 10:10:05 +00:00
|
|
|
if (first_time)
|
|
|
|
{
|
|
|
|
if (storage.is_readonly)
|
|
|
|
{
|
|
|
|
/// We failed to start replication, table is still readonly, so we should increment the metric. See also setNotReadonly().
|
|
|
|
CurrentMetrics::add(CurrentMetrics::ReadonlyReplica);
|
|
|
|
}
|
|
|
|
/// It does not matter if replication is actually started or not, just notify after the first attempt.
|
|
|
|
storage.startup_event.set();
|
|
|
|
first_time = false;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2022-02-03 10:10:05 +00:00
|
|
|
if (need_stop)
|
|
|
|
return;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2022-02-03 10:10:05 +00:00
|
|
|
if (reschedule_period_ms)
|
|
|
|
task->scheduleAfter(reschedule_period_ms);
|
|
|
|
else
|
2021-08-18 09:49:22 +00:00
|
|
|
task->schedule();
|
2022-02-03 10:10:05 +00:00
|
|
|
}
|
2017-11-01 13:58:02 +00:00
|
|
|
|
2022-02-03 10:10:05 +00:00
|
|
|
bool ReplicatedMergeTreeRestartingThread::runImpl()
|
|
|
|
{
|
2022-08-24 17:44:14 +00:00
|
|
|
if (!storage.is_readonly && !storage.getZooKeeper()->expired())
|
2022-02-03 10:10:05 +00:00
|
|
|
return true;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2022-02-03 10:10:05 +00:00
|
|
|
if (first_time)
|
|
|
|
{
|
|
|
|
LOG_DEBUG(log, "Activating replica.");
|
|
|
|
assert(storage.is_readonly);
|
|
|
|
}
|
|
|
|
else if (storage.is_readonly)
|
|
|
|
{
|
|
|
|
LOG_WARNING(log, "Table was in readonly mode. Will try to activate it.");
|
|
|
|
}
|
|
|
|
else if (storage.getZooKeeper()->expired())
|
|
|
|
{
|
|
|
|
LOG_WARNING(log, "ZooKeeper session has expired. Switching to a new session.");
|
|
|
|
partialShutdown();
|
|
|
|
}
|
2021-08-18 09:49:22 +00:00
|
|
|
else
|
2022-02-03 10:10:05 +00:00
|
|
|
{
|
|
|
|
__builtin_unreachable();
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2022-02-03 10:10:05 +00:00
|
|
|
try
|
|
|
|
{
|
|
|
|
storage.setZooKeeper();
|
2017-12-21 18:17:06 +00:00
|
|
|
}
|
2022-02-03 10:10:05 +00:00
|
|
|
catch (const Coordination::Exception &)
|
2017-12-29 22:32:04 +00:00
|
|
|
{
|
2022-02-03 10:10:05 +00:00
|
|
|
/// The exception when you try to zookeeper_init usually happens if DNS does not work. We will try to do it again.
|
2018-04-10 13:20:14 +00:00
|
|
|
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
2022-02-03 10:10:05 +00:00
|
|
|
assert(storage.is_readonly);
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (need_stop)
|
|
|
|
return false;
|
|
|
|
|
|
|
|
if (!tryStartup())
|
|
|
|
{
|
|
|
|
assert(storage.is_readonly);
|
|
|
|
return false;
|
2017-12-21 18:17:06 +00:00
|
|
|
}
|
2017-11-17 08:58:35 +00:00
|
|
|
|
2022-02-03 10:10:05 +00:00
|
|
|
setNotReadonly();
|
|
|
|
|
|
|
|
/// Start queue processing
|
|
|
|
storage.background_operations_assignee.start();
|
|
|
|
storage.queue_updating_task->activateAndSchedule();
|
|
|
|
storage.mutations_updating_task->activateAndSchedule();
|
|
|
|
storage.mutations_finalizing_task->activateAndSchedule();
|
|
|
|
storage.merge_selecting_task->activateAndSchedule();
|
|
|
|
storage.cleanup_thread.start();
|
2022-08-16 10:49:17 +00:00
|
|
|
storage.part_check_thread.start();
|
2022-02-03 10:10:05 +00:00
|
|
|
|
|
|
|
return true;
|
2017-12-29 22:32:04 +00:00
|
|
|
}
|
2017-11-17 08:58:35 +00:00
|
|
|
|
2017-12-21 18:17:06 +00:00
|
|
|
|
2014-10-17 01:05:51 +00:00
|
|
|
bool ReplicatedMergeTreeRestartingThread::tryStartup()
|
|
|
|
{
|
|
|
|
try
|
|
|
|
{
|
2015-09-20 11:02:59 +00:00
|
|
|
removeFailedQuorumParts();
|
2014-10-17 01:05:51 +00:00
|
|
|
activateReplica();
|
2018-08-27 19:06:32 +00:00
|
|
|
|
2018-08-09 15:06:39 +00:00
|
|
|
const auto & zookeeper = storage.getZooKeeper();
|
2019-08-26 14:24:29 +00:00
|
|
|
const auto storage_settings = storage.getSettings();
|
2018-08-09 15:06:39 +00:00
|
|
|
|
2018-08-22 14:01:54 +00:00
|
|
|
storage.cloneReplicaIfNeeded(zookeeper);
|
|
|
|
|
2021-07-27 14:35:20 +00:00
|
|
|
try
|
|
|
|
{
|
2021-09-13 08:00:07 +00:00
|
|
|
storage.queue.initialize(zookeeper);
|
|
|
|
|
2021-07-27 14:35:20 +00:00
|
|
|
storage.queue.load(zookeeper);
|
|
|
|
|
2021-10-18 20:16:02 +00:00
|
|
|
storage.queue.createLogEntriesToFetchBrokenParts();
|
|
|
|
|
2021-07-27 14:35:20 +00:00
|
|
|
/// pullLogsToQueue() after we mark replica 'is_active' (and after we repair if it was lost);
|
|
|
|
/// because cleanup_thread doesn't delete log_pointer of active replicas.
|
2021-08-19 09:29:16 +00:00
|
|
|
storage.queue.pullLogsToQueue(zookeeper, {}, ReplicatedMergeTreeQueue::LOAD);
|
2021-07-27 14:35:20 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
2022-06-28 19:19:06 +00:00
|
|
|
std::lock_guard lock(storage.last_queue_update_exception_lock);
|
2021-08-09 12:58:23 +00:00
|
|
|
storage.last_queue_update_exception = getCurrentExceptionMessage(false);
|
2021-07-27 14:35:20 +00:00
|
|
|
throw;
|
|
|
|
}
|
2018-08-09 15:06:39 +00:00
|
|
|
|
2020-02-18 19:57:48 +00:00
|
|
|
storage.queue.removeCurrentPartsFromMutations();
|
2018-08-08 13:53:06 +00:00
|
|
|
storage.last_queue_update_finish_time.store(time(nullptr));
|
2018-08-09 15:06:39 +00:00
|
|
|
|
2015-09-20 11:02:59 +00:00
|
|
|
updateQuorumIfWeHavePart();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-03-12 19:18:07 +00:00
|
|
|
/// Anything above can throw a KeeperException if something is wrong with ZK.
|
|
|
|
/// Anything below should not throw exceptions.
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-07-30 18:30:33 +00:00
|
|
|
storage.partial_shutdown_called = false;
|
|
|
|
storage.partial_shutdown_event.reset();
|
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
|
|
|
{
|
2018-11-26 00:56:50 +00:00
|
|
|
storage.replica_is_active_node = nullptr;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2014-12-21 01:20:07 +00:00
|
|
|
try
|
|
|
|
{
|
2014-10-17 01:05:51 +00:00
|
|
|
throw;
|
2014-12-21 01:20:07 +00:00
|
|
|
}
|
2018-08-25 01:58:14 +00:00
|
|
|
catch (const Coordination::Exception & e)
|
2014-12-21 01:20:07 +00:00
|
|
|
{
|
2021-05-14 08:32:41 +00:00
|
|
|
LOG_ERROR(log, "Couldn't start replication (table will be in readonly mode): {}. {}", e.what(), DB::getCurrentExceptionMessage(true));
|
2014-12-21 01:20:07 +00:00
|
|
|
return false;
|
|
|
|
}
|
|
|
|
catch (const Exception & e)
|
|
|
|
{
|
|
|
|
if (e.code() != ErrorCodes::REPLICA_IS_ALREADY_ACTIVE)
|
|
|
|
throw;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-05-14 08:32:41 +00:00
|
|
|
LOG_ERROR(log, "Couldn't start replication (table will be in readonly mode): {}. {}", e.what(), DB::getCurrentExceptionMessage(true));
|
2014-12-21 01:20:07 +00:00
|
|
|
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();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2015-09-20 11:02:59 +00:00
|
|
|
Strings failed_parts;
|
2020-06-12 15:09:12 +00:00
|
|
|
if (zookeeper->tryGetChildren(storage.zookeeper_path + "/quorum/failed_parts", failed_parts) != Coordination::Error::ZOK)
|
2015-09-20 11:02:59 +00:00
|
|
|
return;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-05-21 13:49:54 +00:00
|
|
|
/// Firstly, remove parts from ZooKeeper
|
2021-03-05 09:50:26 +00:00
|
|
|
storage.removePartsFromZooKeeperWithRetries(failed_parts);
|
2018-05-21 13:49:54 +00:00
|
|
|
|
2020-03-09 02:55:28 +00:00
|
|
|
for (const auto & part_name : failed_parts)
|
2015-09-20 11:02:59 +00:00
|
|
|
{
|
2019-05-03 02:00:57 +00:00
|
|
|
auto part = storage.getPartIfExists(
|
2021-12-30 14:27:22 +00:00
|
|
|
part_name, {MergeTreeDataPartState::PreActive, MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated});
|
2018-05-21 13:49:54 +00:00
|
|
|
|
2015-09-20 11:02:59 +00:00
|
|
|
if (part)
|
|
|
|
{
|
2020-05-23 22:24:01 +00:00
|
|
|
LOG_DEBUG(log, "Found part {} with failed quorum. Moving to detached. This shouldn't happen often.", part_name);
|
2019-05-20 16:24:36 +00:00
|
|
|
storage.forgetPartAndMoveToDetached(part, "noquorum");
|
2021-05-30 21:29:37 +00:00
|
|
|
storage.queue.removeFailedQuorumPart(part->info);
|
2015-09-20 11:02:59 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void ReplicatedMergeTreeRestartingThread::updateQuorumIfWeHavePart()
|
|
|
|
{
|
|
|
|
auto zookeeper = storage.getZooKeeper();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2015-09-20 11:02:59 +00:00
|
|
|
String quorum_str;
|
2021-05-08 10:59:55 +00:00
|
|
|
if (zookeeper->tryGet(fs::path(storage.zookeeper_path) / "quorum" / "status", quorum_str))
|
2015-09-20 11:02:59 +00:00
|
|
|
{
|
2020-10-06 21:49:48 +00:00
|
|
|
ReplicatedMergeTreeQuorumEntry quorum_entry(quorum_str);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2022-04-18 10:18:43 +00:00
|
|
|
if (!quorum_entry.replicas.contains(storage.replica_name)
|
2020-10-09 11:20:20 +00:00
|
|
|
&& storage.getActiveContainingPart(quorum_entry.part_name))
|
2015-09-20 11:02:59 +00:00
|
|
|
{
|
2020-05-23 22:24:01 +00:00
|
|
|
LOG_WARNING(log, "We have part {} but we is not in quorum. Updating quorum. This shouldn't happen often.", quorum_entry.part_name);
|
2020-10-07 11:28:48 +00:00
|
|
|
storage.updateQuorum(quorum_entry.part_name, false);
|
2015-09-20 11:02:59 +00:00
|
|
|
}
|
|
|
|
}
|
2020-10-04 19:55:39 +00:00
|
|
|
|
2020-10-09 11:20:20 +00:00
|
|
|
Strings part_names;
|
2021-05-08 10:59:55 +00:00
|
|
|
String parallel_quorum_parts_path = fs::path(storage.zookeeper_path) / "quorum" / "parallel";
|
2020-10-09 11:20:20 +00:00
|
|
|
if (zookeeper->tryGetChildren(parallel_quorum_parts_path, part_names) == Coordination::Error::ZOK)
|
2020-10-04 19:55:39 +00:00
|
|
|
{
|
2020-10-09 11:20:20 +00:00
|
|
|
for (auto & part_name : part_names)
|
2020-10-04 19:55:39 +00:00
|
|
|
{
|
2021-05-08 10:59:55 +00:00
|
|
|
if (zookeeper->tryGet(fs::path(parallel_quorum_parts_path) / part_name, quorum_str))
|
2020-10-04 19:55:39 +00:00
|
|
|
{
|
2020-10-06 21:49:48 +00:00
|
|
|
ReplicatedMergeTreeQuorumEntry quorum_entry(quorum_str);
|
2022-04-18 10:18:43 +00:00
|
|
|
if (!quorum_entry.replicas.contains(storage.replica_name)
|
2020-10-09 12:11:21 +00:00
|
|
|
&& storage.getActiveContainingPart(part_name))
|
2020-10-06 21:49:48 +00:00
|
|
|
{
|
2020-10-09 12:11:21 +00:00
|
|
|
LOG_WARNING(log, "We have part {} but we is not in quorum. Updating quorum. This shouldn't happen often.", part_name);
|
2020-10-09 11:20:20 +00:00
|
|
|
storage.updateQuorum(part_name, true);
|
2020-10-06 21:49:48 +00:00
|
|
|
}
|
2020-10-04 19:55:39 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2015-09-20 11:02:59 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2014-10-17 01:05:51 +00:00
|
|
|
void ReplicatedMergeTreeRestartingThread::activateReplica()
|
|
|
|
{
|
2014-12-12 20:50:32 +00:00
|
|
|
auto zookeeper = storage.getZooKeeper();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-04-17 17:59:42 +00:00
|
|
|
/// How other replicas can access this one.
|
|
|
|
ReplicatedMergeTreeAddress address = storage.getReplicatedMergeTreeAddress();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-05-08 10:59:55 +00:00
|
|
|
String is_active_path = fs::path(storage.replica_path) / "is_active";
|
2022-04-14 13:31:05 +00:00
|
|
|
zookeeper->waitForEphemeralToDisappearIfAny(is_active_path);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-03-13 18:01:46 +00:00
|
|
|
/// Simultaneously declare that this replica is active, and update the host.
|
2018-08-25 01:58:14 +00:00
|
|
|
Coordination::Requests ops;
|
2018-03-24 00:45:04 +00:00
|
|
|
ops.emplace_back(zkutil::makeCreateRequest(is_active_path, active_node_identifier, zkutil::CreateMode::Ephemeral));
|
2021-05-08 10:59:55 +00:00
|
|
|
ops.emplace_back(zkutil::makeSetRequest(fs::path(storage.replica_path) / "host", address.toString(), -1));
|
2017-04-01 07:20:54 +00:00
|
|
|
|
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
|
|
|
}
|
2018-08-25 01:58:14 +00:00
|
|
|
catch (const Coordination::Exception & e)
|
2014-10-17 01:05:51 +00:00
|
|
|
{
|
2020-06-27 13:55:00 +00:00
|
|
|
String existing_replica_host;
|
2021-05-08 10:59:55 +00:00
|
|
|
zookeeper->tryGet(fs::path(storage.replica_path) / "host", existing_replica_host);
|
2020-06-27 13:55:00 +00:00
|
|
|
|
|
|
|
if (existing_replica_host.empty())
|
|
|
|
existing_replica_host = "without host node";
|
|
|
|
else
|
|
|
|
boost::replace_all(existing_replica_host, "\n", ", ");
|
|
|
|
|
2020-06-12 15:09:12 +00:00
|
|
|
if (e.code == Coordination::Error::ZNODEEXISTS)
|
2020-06-27 13:55:00 +00:00
|
|
|
throw Exception(ErrorCodes::REPLICA_IS_ALREADY_ACTIVE,
|
|
|
|
"Replica {} appears to be already active ({}). If you're sure it's not, "
|
|
|
|
"try again in a minute or remove znode {}/is_active manually",
|
|
|
|
storage.replica_path, existing_replica_host, storage.replica_path);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2014-10-17 01:05:51 +00:00
|
|
|
throw;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-03-13 18:01:46 +00:00
|
|
|
/// `current_zookeeper` lives for the lifetime of `replica_is_active_node`,
|
|
|
|
/// since before changing `current_zookeeper`, `replica_is_active_node` object is destroyed in `partialShutdown` method.
|
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
|
|
|
}
|
|
|
|
|
|
|
|
|
2022-02-03 10:10:05 +00:00
|
|
|
void ReplicatedMergeTreeRestartingThread::partialShutdown(bool part_of_full_shutdown)
|
2014-10-17 01:05:51 +00:00
|
|
|
{
|
2022-02-03 10:10:05 +00:00
|
|
|
setReadonly(part_of_full_shutdown);
|
2016-01-21 01:47:28 +00:00
|
|
|
ProfileEvents::increment(ProfileEvents::ReplicaPartialShutdown);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-07-30 18:30:33 +00:00
|
|
|
storage.partial_shutdown_called = true;
|
|
|
|
storage.partial_shutdown_event.set();
|
2014-10-17 01:05:51 +00:00
|
|
|
storage.replica_is_active_node = nullptr;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-05-23 22:24:01 +00:00
|
|
|
LOG_TRACE(log, "Waiting for threads to finish");
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2022-01-28 12:50:58 +00:00
|
|
|
storage.merge_selecting_task->deactivate();
|
2018-05-31 13:05:05 +00:00
|
|
|
storage.queue_updating_task->deactivate();
|
|
|
|
storage.mutations_updating_task->deactivate();
|
2018-08-22 13:57:30 +00:00
|
|
|
storage.mutations_finalizing_task->deactivate();
|
2018-04-19 14:20:18 +00:00
|
|
|
|
2018-07-30 17:34:55 +00:00
|
|
|
storage.cleanup_thread.stop();
|
2022-08-16 10:49:17 +00:00
|
|
|
storage.part_check_thread.stop();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-05-30 21:29:37 +00:00
|
|
|
/// Stop queue processing
|
2021-06-15 21:48:38 +00:00
|
|
|
{
|
|
|
|
auto fetch_lock = storage.fetcher.blocker.cancel();
|
|
|
|
auto merge_lock = storage.merger_mutator.merges_blocker.cancel();
|
|
|
|
auto move_lock = storage.parts_mover.moves_blocker.cancel();
|
2021-09-08 00:21:21 +00:00
|
|
|
storage.background_operations_assignee.finish();
|
2021-06-15 21:48:38 +00:00
|
|
|
}
|
2021-05-30 21:29:37 +00:00
|
|
|
|
2020-05-23 22:24:01 +00:00
|
|
|
LOG_TRACE(log, "Threads finished");
|
2014-10-17 01:05:51 +00:00
|
|
|
}
|
|
|
|
|
2018-08-21 14:03:06 +00:00
|
|
|
|
|
|
|
void ReplicatedMergeTreeRestartingThread::shutdown()
|
|
|
|
{
|
|
|
|
/// Stop restarting_thread before stopping other tasks - so that it won't restart them again.
|
|
|
|
need_stop = true;
|
|
|
|
task->deactivate();
|
2020-05-23 22:24:01 +00:00
|
|
|
LOG_TRACE(log, "Restarting thread finished");
|
2018-08-21 14:03:06 +00:00
|
|
|
|
|
|
|
/// Stop other tasks.
|
2022-02-03 10:10:05 +00:00
|
|
|
partialShutdown(/* part_of_full_shutdown */ true);
|
2018-08-21 14:03:06 +00:00
|
|
|
}
|
|
|
|
|
2022-02-03 10:10:05 +00:00
|
|
|
void ReplicatedMergeTreeRestartingThread::setReadonly(bool on_shutdown)
|
2021-05-14 08:32:41 +00:00
|
|
|
{
|
|
|
|
bool old_val = false;
|
2022-02-03 10:10:05 +00:00
|
|
|
bool became_readonly = storage.is_readonly.compare_exchange_strong(old_val, true);
|
|
|
|
|
|
|
|
/// Do not increment the metric if replica became readonly due to shutdown.
|
|
|
|
if (became_readonly && on_shutdown)
|
|
|
|
return;
|
|
|
|
|
|
|
|
if (became_readonly)
|
2021-05-14 08:32:41 +00:00
|
|
|
CurrentMetrics::add(CurrentMetrics::ReadonlyReplica);
|
2022-02-03 10:10:05 +00:00
|
|
|
|
|
|
|
/// Replica was already readonly, but we should decrement the metric, because we are detaching/dropping table.
|
|
|
|
if (on_shutdown)
|
2020-10-04 07:14:22 +00:00
|
|
|
CurrentMetrics::sub(CurrentMetrics::ReadonlyReplica);
|
2022-02-03 10:10:05 +00:00
|
|
|
}
|
2020-10-04 07:14:22 +00:00
|
|
|
|
2022-02-03 10:10:05 +00:00
|
|
|
void ReplicatedMergeTreeRestartingThread::setNotReadonly()
|
|
|
|
{
|
|
|
|
bool old_val = true;
|
|
|
|
/// is_readonly is true on startup, but ReadonlyReplica metric is not incremented,
|
|
|
|
/// because we don't want to change this metric if replication is started successfully.
|
|
|
|
/// So we should not decrement it when replica stopped being readonly on startup.
|
|
|
|
if (storage.is_readonly.compare_exchange_strong(old_val, false) && !first_time)
|
|
|
|
CurrentMetrics::sub(CurrentMetrics::ReadonlyReplica);
|
2018-08-21 14:03:06 +00:00
|
|
|
}
|
|
|
|
|
2014-10-17 01:05:51 +00:00
|
|
|
}
|