ClickHouse/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

411 lines
13 KiB
C++
Raw Normal View History

#include <IO/Operators.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h>
#include <Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h>
#include <Storages/MergeTree/ReplicatedMergeTreeAddress.h>
#include <Interpreters/Context.h>
2018-04-03 17:35:48 +00:00
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/randomSeed.h>
#include <boost/algorithm/string/replace.hpp>
namespace ProfileEvents
{
extern const Event ReplicaPartialShutdown;
}
namespace CurrentMetrics
{
extern const Metric ReadonlyReplica;
}
namespace DB
{
namespace ErrorCodes
{
extern const int REPLICA_IS_ALREADY_ACTIVE;
extern const int REPLICA_STATUS_CHANGED;
}
namespace
{
constexpr auto retry_period_ms = 10 * 1000;
}
2017-03-12 19:18:07 +00:00
/// Used to check whether it's us who set node `is_active`, or not.
static String generateActiveNodeIdentifier()
{
2016-07-31 03:53:16 +00:00
return "pid: " + toString(getpid()) + ", random: " + toString(randomSeed());
}
ReplicatedMergeTreeRestartingThread::ReplicatedMergeTreeRestartingThread(StorageReplicatedMergeTree & storage_)
: 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))
, active_node_identifier(generateActiveNodeIdentifier())
{
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;
task = storage.getContext()->getSchedulePool().createTask(log_name, [this]{ run(); });
}
void ReplicatedMergeTreeRestartingThread::run()
{
if (need_stop)
return;
2022-02-03 10:10:05 +00:00
size_t reschedule_period_ms = check_period_ms;
try
{
2022-02-03 10:10:05 +00:00
bool replica_is_active = runImpl();
if (!replica_is_active)
reschedule_period_ms = retry_period_ms;
}
catch (const Exception & e)
{
/// We couldn't activate table let's set it into readonly mode
partialShutdown();
tryLogCurrentException(log, __PRETTY_FUNCTION__);
if (e.code() == ErrorCodes::REPLICA_STATUS_CHANGED)
2022-02-03 10:10:05 +00:00
reschedule_period_ms = 0;
}
catch (...)
{
partialShutdown();
tryLogCurrentException(log, __PRETTY_FUNCTION__);
}
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;
}
2022-02-03 10:10:05 +00:00
if (need_stop)
return;
2022-02-03 10:10:05 +00:00
if (reschedule_period_ms)
task->scheduleAfter(reschedule_period_ms);
else
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;
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();
}
else
2022-02-03 10:10:05 +00:00
{
__builtin_unreachable();
}
2022-02-03 10:10:05 +00:00
try
{
storage.setZooKeeper();
}
2022-02-03 10:10:05 +00:00
catch (const Coordination::Exception &)
{
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.
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;
}
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();
storage.part_check_thread.start();
2022-02-03 10:10:05 +00:00
return true;
}
bool ReplicatedMergeTreeRestartingThread::tryStartup()
{
try
{
removeFailedQuorumParts();
activateReplica();
2018-08-27 19:06:32 +00:00
const auto & zookeeper = storage.getZooKeeper();
2019-08-26 14:24:29 +00:00
const auto storage_settings = storage.getSettings();
storage.cloneReplicaIfNeeded(zookeeper);
try
{
storage.queue.initialize(zookeeper);
storage.queue.load(zookeeper);
2021-10-18 20:16:02 +00:00
storage.queue.createLogEntriesToFetchBrokenParts();
/// 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.
storage.queue.pullLogsToQueue(zookeeper, {}, ReplicatedMergeTreeQueue::LOAD);
}
catch (...)
{
std::lock_guard lock(storage.last_queue_update_exception_lock);
storage.last_queue_update_exception = getCurrentExceptionMessage(false);
throw;
}
storage.queue.removeCurrentPartsFromMutations();
storage.last_queue_update_finish_time.store(time(nullptr));
updateQuorumIfWeHavePart();
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.
storage.partial_shutdown_called = false;
storage.partial_shutdown_event.reset();
return true;
}
catch (...)
{
2018-11-26 00:56:50 +00:00
storage.replica_is_active_node = nullptr;
try
{
throw;
}
catch (const Coordination::Exception & e)
{
LOG_ERROR(log, "Couldn't start replication (table will be in readonly mode): {}. {}", e.what(), DB::getCurrentExceptionMessage(true));
return false;
}
catch (const Exception & e)
{
if (e.code() != ErrorCodes::REPLICA_IS_ALREADY_ACTIVE)
throw;
LOG_ERROR(log, "Couldn't start replication (table will be in readonly mode): {}. {}", e.what(), DB::getCurrentExceptionMessage(true));
return false;
}
}
}
void ReplicatedMergeTreeRestartingThread::removeFailedQuorumParts()
{
auto zookeeper = storage.getZooKeeper();
Strings failed_parts;
if (zookeeper->tryGetChildren(storage.zookeeper_path + "/quorum/failed_parts", failed_parts) != Coordination::Error::ZOK)
return;
/// Firstly, remove parts from ZooKeeper
RFC: Throw exception if removing parts from ZooKeeper fails. This is used for removing part metadata from ZooKeeper when executing queue events like `DROP_RANGE` triggered when a user tries to drop a part or a partition. There are other uses but I'll focus only on this one. Before this change the method was giving up silently if it was unable to remove parts from ZooKeeper and this behaviour seems to be problematic. It could lead to operation being reported as successful at first but data reappearing later (very rarely) or "stuck" events in replication queue. Here is one particular scenario which I think we've hit: * Execute a DETACH PARTITION * DROP_RANGE event put in the queue * Replicas try to execute dropRange but some of them get disconnected from ZK and 5 retries aren't enough (ZK is miss-behaving), return code (false) is ignored and log pointer advances. * One of the replica where dropRange failed is restarted. * checkParts is executed and it finds parts that weren't removed from ZK, logs `Removing locally missing part from ZooKeeper and queueing a fetch` and puts GET_PART on the queue. * Few things can happen from here: * There is a lagging replica that din't execute DROP_RANGE yet: part will be fetched. The other replica will execute DROP_RANGE later and we'll get diverging set of parts on replicas. * Another replica also silently failed to remove parts from ZK: both of them are left with GET_PART in the queue and none of them can make progress, logging: `No active replica has part ... or covering part`.
2021-03-05 09:50:26 +00:00
storage.removePartsFromZooKeeperWithRetries(failed_parts);
2020-03-09 02:55:28 +00:00
for (const auto & part_name : failed_parts)
{
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});
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);
storage.forgetPartAndMoveToDetached(part, "noquorum");
2021-05-30 21:29:37 +00:00
storage.queue.removeFailedQuorumPart(part->info);
}
}
}
void ReplicatedMergeTreeRestartingThread::updateQuorumIfWeHavePart()
{
auto zookeeper = storage.getZooKeeper();
String quorum_str;
2021-05-08 10:59:55 +00:00
if (zookeeper->tryGet(fs::path(storage.zookeeper_path) / "quorum" / "status", quorum_str))
{
ReplicatedMergeTreeQuorumEntry quorum_entry(quorum_str);
if (!quorum_entry.replicas.contains(storage.replica_name)
&& storage.getActiveContainingPart(quorum_entry.part_name))
{
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);
storage.updateQuorum(quorum_entry.part_name, false);
}
}
2020-10-04 19:55:39 +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";
if (zookeeper->tryGetChildren(parallel_quorum_parts_path, part_names) == Coordination::Error::ZOK)
2020-10-04 19:55:39 +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
{
ReplicatedMergeTreeQuorumEntry quorum_entry(quorum_str);
if (!quorum_entry.replicas.contains(storage.replica_name)
2020-10-09 12:11:21 +00:00
&& storage.getActiveContainingPart(part_name))
{
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);
storage.updateQuorum(part_name, true);
}
2020-10-04 19:55:39 +00:00
}
}
}
}
void ReplicatedMergeTreeRestartingThread::activateReplica()
{
2014-12-12 20:50:32 +00:00
auto zookeeper = storage.getZooKeeper();
/// How other replicas can access this one.
ReplicatedMergeTreeAddress address = storage.getReplicatedMergeTreeAddress();
2021-05-08 10:59:55 +00:00
String is_active_path = fs::path(storage.replica_path) / "is_active";
zookeeper->waitForEphemeralToDisappearIfAny(is_active_path);
2017-03-13 18:01:46 +00:00
/// Simultaneously declare that this replica is active, and update the host.
Coordination::Requests ops;
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));
try
{
2014-12-12 20:50:32 +00:00
zookeeper->multi(ops);
}
catch (const Coordination::Exception & e)
{
String existing_replica_host;
2021-05-08 10:59:55 +00:00
zookeeper->tryGet(fs::path(storage.replica_path) / "host", existing_replica_host);
if (existing_replica_host.empty())
existing_replica_host = "without host node";
else
boost::replace_all(existing_replica_host, "\n", ", ");
if (e.code == Coordination::Error::ZNODEEXISTS)
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);
throw;
}
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.
storage.replica_is_active_node = zkutil::EphemeralNodeHolder::existing(is_active_path, *storage.current_zookeeper);
}
2022-02-03 10:10:05 +00:00
void ReplicatedMergeTreeRestartingThread::partialShutdown(bool part_of_full_shutdown)
{
2022-02-03 10:10:05 +00:00
setReadonly(part_of_full_shutdown);
ProfileEvents::increment(ProfileEvents::ReplicaPartialShutdown);
storage.partial_shutdown_called = true;
storage.partial_shutdown_event.set();
storage.replica_is_active_node = nullptr;
2020-05-23 22:24:01 +00:00
LOG_TRACE(log, "Waiting for threads to finish");
storage.merge_selecting_task->deactivate();
storage.queue_updating_task->deactivate();
storage.mutations_updating_task->deactivate();
storage.mutations_finalizing_task->deactivate();
storage.cleanup_thread.stop();
storage.part_check_thread.stop();
2021-05-30 21:29:37 +00:00
/// Stop queue processing
{
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-05-30 21:29:37 +00:00
2020-05-23 22:24:01 +00:00
LOG_TRACE(log, "Threads finished");
}
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");
/// Stop other tasks.
2022-02-03 10:10:05 +00:00
partialShutdown(/* part_of_full_shutdown */ true);
}
2022-02-03 10:10:05 +00:00
void ReplicatedMergeTreeRestartingThread::setReadonly(bool on_shutdown)
{
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)
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)
CurrentMetrics::sub(CurrentMetrics::ReadonlyReplica);
2022-02-03 10:10:05 +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);
}
}