ClickHouse/libs/libzkutil/include/zkutil/LeaderElection.h

146 lines
3.5 KiB
C++
Raw Normal View History

2014-04-04 10:37:33 +00:00
#pragma once
#include <zkutil/ZooKeeper.h>
#include <functional>
#include <memory>
2015-09-29 19:19:54 +00:00
#include <common/logger_useful.h>
2014-04-04 10:37:33 +00:00
namespace zkutil
{
2016-10-23 03:46:51 +00:00
/** Implements leader election algorithm described here: http://zookeeper.apache.org/doc/r3.4.5/recipes.html#sc_leaderElection
2014-04-04 10:37:33 +00:00
*/
class LeaderElection
{
public:
using LeadershipHandler = std::function<void()>;
2014-04-04 10:37:33 +00:00
2016-10-23 03:46:51 +00:00
/** 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
* and existence of more than one ephemeral node with same identifier indicates an error
* (see cleanOldEphemeralNodes).
2014-04-04 10:37:33 +00:00
*/
LeaderElection(const std::string & path_, ZooKeeper & zookeeper_, LeadershipHandler handler_, const std::string & identifier_ = "")
2016-10-23 03:46:51 +00:00
: path(path_), zookeeper(zookeeper_), handler(handler_), identifier(identifier_)
2014-04-04 10:37:33 +00:00
{
2016-10-23 03:46:51 +00:00
createNode();
2014-04-04 10:37:33 +00:00
}
2016-10-23 03:46:51 +00:00
void yield()
{
2016-10-23 03:46:51 +00:00
releaseNode();
createNode();
}
2014-04-04 10:37:33 +00:00
~LeaderElection()
{
2016-10-23 03:46:51 +00:00
releaseNode();
2014-04-04 10:37:33 +00:00
}
private:
std::string path;
ZooKeeper & zookeeper;
LeadershipHandler handler;
std::string identifier;
EphemeralNodeHolderPtr node;
std::string node_name;
std::thread thread;
2016-10-23 03:46:51 +00:00
std::atomic<bool> shutdown {false};
zkutil::EventPtr event = std::make_shared<Poco::Event>();
2014-04-04 10:37:33 +00:00
2016-10-23 03:46:51 +00:00
void createNode()
{
shutdown = false;
node = EphemeralNodeHolder::createSequential(path + "/leader_election-", zookeeper, identifier);
std::string node_path = node->getPath();
node_name = node_path.substr(node_path.find_last_of('/') + 1);
cleanOldEphemeralNodes();
2016-10-23 03:46:51 +00:00
thread = std::thread(&LeaderElection::threadFunction, this);
}
void cleanOldEphemeralNodes()
{
if (identifier.empty())
return;
/** If there are nodes with same identifier, remove them.
* Such nodes could still be alive after failed attempt of removal,
* if it was temporary communication failure, that was continued for more than session timeout,
* but ZK session is still alive for unknown reason, and someone still holds that ZK session.
* See comments in destructor of EphemeralNodeHolder.
*/
Strings brothers = zookeeper.getChildren(path);
for (const auto & brother : brothers)
{
if (brother == node_name)
continue;
std::string brother_path = path + "/" + brother;
std::string brother_identifier = zookeeper.get(brother_path);
if (brother_identifier == identifier)
{
LOG_WARNING(&Logger::get("LeaderElection"), "Found obsolete ephemeral node for identifier "
+ identifier + ", removing: " + brother_path);
zookeeper.tryRemoveWithRetries(brother_path);
}
}
}
2016-10-23 03:46:51 +00:00
void releaseNode()
{
shutdown = true;
event->set();
if (thread.joinable())
thread.join();
node = nullptr;
}
2014-04-04 10:37:33 +00:00
void threadFunction()
{
while (!shutdown)
2014-04-04 10:37:33 +00:00
{
2014-07-01 15:58:25 +00:00
bool success = false;
try
2014-04-04 10:37:33 +00:00
{
Strings children = zookeeper.getChildren(path);
std::sort(children.begin(), children.end());
auto it = std::lower_bound(children.begin(), children.end(), node_name);
if (it == children.end() || *it != node_name)
throw Poco::Exception("Assertion failed in LeaderElection");
if (it == children.begin())
{
handler();
return;
}
if (zookeeper.exists(path + "/" + *(it - 1), nullptr, event))
2014-07-14 11:45:34 +00:00
event->wait();
2014-07-01 15:58:25 +00:00
success = true;
}
catch (...)
{
2016-03-29 15:31:24 +00:00
DB::tryLogCurrentException("LeaderElection");
2014-04-04 10:37:33 +00:00
}
2014-07-01 15:58:25 +00:00
if (!success)
2014-07-24 16:29:37 +00:00
event->tryWait(10 * 1000);
2014-04-04 10:37:33 +00:00
}
}
};
using LeaderElectionPtr = std::shared_ptr<LeaderElection>;
2014-04-04 10:37:33 +00:00
}