mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-30 19:42:00 +00:00
09e3134bf4
This reverts commit 431a735efe
.
58 lines
1.4 KiB
C++
58 lines
1.4 KiB
C++
#pragma once
|
|
|
|
#include <unordered_map>
|
|
#include <unordered_set>
|
|
#include <mutex>
|
|
#include <memory>
|
|
#include <experimental/optional>
|
|
#include <Poco/Event.h>
|
|
#include <zkutil/ZooKeeper.h>
|
|
#include <zkutil/Common.h>
|
|
|
|
namespace DB
|
|
{
|
|
namespace ErrorCodes
|
|
{
|
|
extern const int NO_ZOOKEEPER;
|
|
}
|
|
}
|
|
|
|
namespace zkutil
|
|
{
|
|
|
|
/// This class allows querying the contents of ZooKeeper nodes and caching the results.
|
|
/// Watches are set for cached nodes and for nodes that were nonexistent at the time of query.
|
|
/// After a watch fires, a notification is generated for the change event.
|
|
/// NOTE: methods of this class are not thread-safe.
|
|
class ZooKeeperNodeCache
|
|
{
|
|
public:
|
|
ZooKeeperNodeCache(GetZooKeeper get_zookeeper);
|
|
|
|
ZooKeeperNodeCache(const ZooKeeperNodeCache &) = delete;
|
|
ZooKeeperNodeCache(ZooKeeperNodeCache &&) = default;
|
|
|
|
std::experimental::optional<std::string> get(const std::string & path);
|
|
|
|
Poco::Event & getChangedEvent() { return context->changed_event; }
|
|
|
|
private:
|
|
GetZooKeeper get_zookeeper;
|
|
|
|
struct Context
|
|
{
|
|
Poco::Event changed_event;
|
|
|
|
std::mutex mutex;
|
|
zkutil::ZooKeeperPtr zookeeper;
|
|
std::unordered_set<std::string> invalidated_paths;
|
|
};
|
|
|
|
std::shared_ptr<Context> context;
|
|
|
|
std::unordered_set<std::string> nonexistent_nodes;
|
|
std::unordered_map<std::string, std::string> node_cache;
|
|
};
|
|
|
|
}
|