2014-04-04 10:37:33 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <functional>
|
2016-05-28 17:31:50 +00:00
|
|
|
#include <memory>
|
2015-09-29 19:19:54 +00:00
|
|
|
#include <common/logger_useful.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Common/CurrentMetrics.h>
|
2020-06-12 19:04:18 +00:00
|
|
|
#include <Common/ZooKeeper/ZooKeeper.h>
|
|
|
|
#include <Common/ZooKeeper/KeeperException.h>
|
2018-08-20 15:34:37 +00:00
|
|
|
#include <Core/BackgroundSchedulePool.h>
|
2016-10-24 13:47:15 +00:00
|
|
|
|
|
|
|
|
2014-04-04 10:37:33 +00:00
|
|
|
namespace zkutil
|
|
|
|
{
|
|
|
|
|
2020-06-12 19:19:01 +00:00
|
|
|
/** Initially was used to implement leader election algorithm described here:
|
|
|
|
* http://zookeeper.apache.org/doc/r3.4.5/recipes.html#sc_leaderElection
|
|
|
|
*
|
|
|
|
* But then we decided to get rid of leader election, so every replica can become leader.
|
|
|
|
* For now, every replica can become leader if there is no leader among replicas with old version.
|
|
|
|
*
|
2020-06-12 20:32:31 +00:00
|
|
|
* It's tempting to remove this class at all, but we have to maintain it,
|
|
|
|
* to maintain compatibility when replicas with different versions work on the same cluster
|
|
|
|
* (this is allowed for short time period during cluster update).
|
|
|
|
*
|
2020-06-15 00:59:12 +00:00
|
|
|
* Replicas with new versions creates ephemeral sequential nodes with values like "replica_name (multiple leaders Ok)".
|
|
|
|
* If the first node belongs to a replica with new version, then all replicas with new versions become leaders.
|
2014-04-04 10:37:33 +00:00
|
|
|
*/
|
|
|
|
class LeaderElection
|
|
|
|
{
|
|
|
|
public:
|
2017-04-01 07:20:54 +00:00
|
|
|
using LeadershipHandler = std::function<void()>;
|
|
|
|
|
|
|
|
/** handler is called when this instance become leader.
|
|
|
|
*
|
|
|
|
* identifier - if not empty, must uniquely (within same path) identify participant of leader election.
|
|
|
|
* It means that different participants of leader election have different identifiers
|
2018-04-05 20:47:06 +00:00
|
|
|
* and existence of more than one ephemeral node with same identifier indicates an error.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2020-06-12 19:02:00 +00:00
|
|
|
LeaderElection(
|
|
|
|
DB::BackgroundSchedulePool & pool_,
|
|
|
|
const std::string & path_,
|
|
|
|
ZooKeeper & zookeeper_,
|
|
|
|
LeadershipHandler handler_,
|
|
|
|
const std::string & identifier_)
|
2020-06-15 00:59:12 +00:00
|
|
|
: pool(pool_), path(path_), zookeeper(zookeeper_), handler(handler_), identifier(identifier_ + suffix)
|
2018-05-31 13:05:05 +00:00
|
|
|
, log_name("LeaderElection (" + path + ")")
|
2020-05-30 21:57:37 +00:00
|
|
|
, log(&Poco::Logger::get(log_name))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-05-31 13:05:05 +00:00
|
|
|
task = pool.createTask(log_name, [this] { threadFunction(); });
|
2017-04-01 07:20:54 +00:00
|
|
|
createNode();
|
|
|
|
}
|
|
|
|
|
2018-04-06 16:06:07 +00:00
|
|
|
void shutdown()
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-04-06 16:06:07 +00:00
|
|
|
if (shutdown_called)
|
|
|
|
return;
|
|
|
|
|
|
|
|
shutdown_called = true;
|
2018-05-31 13:05:05 +00:00
|
|
|
task->deactivate();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
~LeaderElection()
|
|
|
|
{
|
2020-06-15 00:59:12 +00:00
|
|
|
releaseNode();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2014-04-04 10:37:33 +00:00
|
|
|
|
|
|
|
private:
|
2020-06-15 00:59:12 +00:00
|
|
|
static inline constexpr auto suffix = " (multiple leaders Ok)";
|
2017-12-29 22:32:04 +00:00
|
|
|
DB::BackgroundSchedulePool & pool;
|
2018-05-31 13:05:05 +00:00
|
|
|
DB::BackgroundSchedulePool::TaskHolder task;
|
2020-06-15 00:59:12 +00:00
|
|
|
std::string path;
|
2017-04-01 07:20:54 +00:00
|
|
|
ZooKeeper & zookeeper;
|
|
|
|
LeadershipHandler handler;
|
|
|
|
std::string identifier;
|
2018-05-31 13:05:05 +00:00
|
|
|
std::string log_name;
|
2020-05-30 21:57:37 +00:00
|
|
|
Poco::Logger * log;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-06-15 00:59:12 +00:00
|
|
|
EphemeralNodeHolderPtr node;
|
|
|
|
std::string node_name;
|
|
|
|
|
2018-04-06 16:06:07 +00:00
|
|
|
std::atomic<bool> shutdown_called {false};
|
2017-12-21 18:17:06 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void createNode()
|
|
|
|
{
|
2018-04-06 16:06:07 +00:00
|
|
|
shutdown_called = false;
|
2020-06-15 00:59:12 +00:00
|
|
|
node = EphemeralNodeHolder::createSequential(path + "/leader_election-", zookeeper, identifier);
|
2020-06-12 20:32:31 +00:00
|
|
|
|
2020-06-15 00:59:12 +00:00
|
|
|
std::string node_path = node->getPath();
|
|
|
|
node_name = node_path.substr(node_path.find_last_of('/') + 1);
|
2020-06-12 20:32:31 +00:00
|
|
|
|
2018-08-22 13:43:27 +00:00
|
|
|
task->activateAndSchedule();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2020-06-15 00:59:12 +00:00
|
|
|
void releaseNode()
|
|
|
|
{
|
|
|
|
shutdown();
|
|
|
|
node = nullptr;
|
|
|
|
}
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
void threadFunction()
|
|
|
|
{
|
2020-06-15 00:59:12 +00:00
|
|
|
bool success = false;
|
|
|
|
|
2017-12-29 22:32:04 +00:00
|
|
|
try
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-12-29 22:32:04 +00:00
|
|
|
Strings children = zookeeper.getChildren(path);
|
2020-06-12 19:19:01 +00:00
|
|
|
std::sort(children.begin(), children.end());
|
|
|
|
|
2020-06-15 00:59:12 +00:00
|
|
|
auto my_node_it = std::lower_bound(children.begin(), children.end(), node_name);
|
|
|
|
if (my_node_it == children.end() || *my_node_it != node_name)
|
|
|
|
throw Poco::Exception("Assertion failed in LeaderElection");
|
|
|
|
|
|
|
|
String value = zookeeper.get(path + "/" + children.front());
|
2020-06-12 19:19:01 +00:00
|
|
|
|
2020-06-15 00:59:12 +00:00
|
|
|
if (value.ends_with(suffix))
|
2017-12-21 18:17:06 +00:00
|
|
|
{
|
2017-12-29 22:32:04 +00:00
|
|
|
handler();
|
|
|
|
return;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2021-01-03 16:20:09 +00:00
|
|
|
|
2020-06-15 00:59:12 +00:00
|
|
|
if (my_node_it == children.begin())
|
|
|
|
throw Poco::Exception("Assertion failed in LeaderElection");
|
|
|
|
|
|
|
|
/// Watch for the node in front of us.
|
|
|
|
--my_node_it;
|
2020-09-10 09:01:01 +00:00
|
|
|
std::string get_path_value;
|
|
|
|
if (!zookeeper.tryGetWatch(path + "/" + *my_node_it, get_path_value, nullptr, task->getWatchCallback()))
|
2020-06-15 00:59:12 +00:00
|
|
|
task->schedule();
|
|
|
|
|
|
|
|
success = true;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2018-04-24 17:11:59 +00:00
|
|
|
catch (const KeeperException & e)
|
|
|
|
{
|
2018-05-31 13:05:05 +00:00
|
|
|
DB::tryLogCurrentException(log);
|
2018-04-24 17:11:59 +00:00
|
|
|
|
2020-06-12 15:09:12 +00:00
|
|
|
if (e.code == Coordination::Error::ZSESSIONEXPIRED)
|
2018-04-24 17:11:59 +00:00
|
|
|
return;
|
|
|
|
}
|
2017-12-29 22:32:04 +00:00
|
|
|
catch (...)
|
|
|
|
{
|
2018-05-31 13:05:05 +00:00
|
|
|
DB::tryLogCurrentException(log);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2017-12-29 22:32:04 +00:00
|
|
|
|
2020-06-15 00:59:12 +00:00
|
|
|
if (!success)
|
|
|
|
task->scheduleAfter(10 * 1000);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2014-04-04 10:37:33 +00:00
|
|
|
};
|
|
|
|
|
2016-05-28 17:31:50 +00:00
|
|
|
using LeaderElectionPtr = std::shared_ptr<LeaderElection>;
|
2014-04-04 10:37:33 +00:00
|
|
|
|
|
|
|
}
|