mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
better names in ZooKeeperNodeCache [#CLICKHOUSE-3859]
This commit is contained in:
parent
e085bc9816
commit
ceea7bde2e
@ -353,12 +353,12 @@ void ConfigProcessor::doIncludesRecursive(
|
||||
XMLDocumentPtr zk_document;
|
||||
auto get_zk_node = [&](const std::string & name) -> const Node *
|
||||
{
|
||||
zkutil::ZooKeeperNodeCache::GetResult result = zk_node_cache->get(name, zk_changed_event);
|
||||
if (!result.exists)
|
||||
zkutil::ZooKeeperNodeCache::ZNode znode = zk_node_cache->get(name, zk_changed_event);
|
||||
if (!znode.exists)
|
||||
return nullptr;
|
||||
|
||||
/// Enclose contents into a fake <from_zk> tag to allow pure text substitutions.
|
||||
zk_document = dom_parser.parseString("<from_zk>" + result.contents + "</from_zk>");
|
||||
zk_document = dom_parser.parseString("<from_zk>" + znode.contents + "</from_zk>");
|
||||
return getRootNode(zk_document.get());
|
||||
};
|
||||
|
||||
|
@ -9,7 +9,7 @@ ZooKeeperNodeCache::ZooKeeperNodeCache(GetZooKeeper get_zookeeper_)
|
||||
{
|
||||
}
|
||||
|
||||
ZooKeeperNodeCache::GetResult ZooKeeperNodeCache::get(const std::string & path, EventPtr watch_event)
|
||||
ZooKeeperNodeCache::ZNode ZooKeeperNodeCache::get(const std::string & path, EventPtr watch_event)
|
||||
{
|
||||
Coordination::WatchCallback watch_callback;
|
||||
if (watch_event)
|
||||
@ -18,7 +18,7 @@ ZooKeeperNodeCache::GetResult ZooKeeperNodeCache::get(const std::string & path,
|
||||
return get(path, watch_callback);
|
||||
}
|
||||
|
||||
ZooKeeperNodeCache::GetResult ZooKeeperNodeCache::get(const std::string & path, Coordination::WatchCallback caller_watch_callback)
|
||||
ZooKeeperNodeCache::ZNode ZooKeeperNodeCache::get(const std::string & path, Coordination::WatchCallback caller_watch_callback)
|
||||
{
|
||||
zkutil::ZooKeeperPtr zookeeper;
|
||||
std::unordered_set<std::string> invalidated_paths;
|
||||
@ -28,7 +28,7 @@ ZooKeeperNodeCache::GetResult ZooKeeperNodeCache::get(const std::string & path,
|
||||
if (!context->zookeeper)
|
||||
{
|
||||
/// Possibly, there was a previous session and it has expired. Clear the cache.
|
||||
node_cache.clear();
|
||||
path_to_cached_znode.clear();
|
||||
|
||||
context->zookeeper = get_zookeeper();
|
||||
}
|
||||
@ -41,10 +41,10 @@ ZooKeeperNodeCache::GetResult ZooKeeperNodeCache::get(const std::string & path,
|
||||
throw DB::Exception("Could not get znode: `" + path + "'. ZooKeeper not configured.", DB::ErrorCodes::NO_ZOOKEEPER);
|
||||
|
||||
for (const auto & invalidated_path : invalidated_paths)
|
||||
node_cache.erase(invalidated_path);
|
||||
path_to_cached_znode.erase(invalidated_path);
|
||||
|
||||
auto cache_it = node_cache.find(path);
|
||||
if (cache_it != node_cache.end())
|
||||
auto cache_it = path_to_cached_znode.find(path);
|
||||
if (cache_it != path_to_cached_znode.end())
|
||||
return cache_it->second;
|
||||
|
||||
std::weak_ptr<Context> weak_context(context);
|
||||
@ -74,12 +74,12 @@ ZooKeeperNodeCache::GetResult ZooKeeperNodeCache::get(const std::string & path,
|
||||
caller_watch_callback(response);
|
||||
};
|
||||
|
||||
GetResult result;
|
||||
ZNode result;
|
||||
|
||||
result.exists = zookeeper->tryGetWatch(path, result.contents, &result.stat, watch_callback);
|
||||
if (result.exists)
|
||||
{
|
||||
node_cache.emplace(path, result);
|
||||
path_to_cached_znode.emplace(path, result);
|
||||
return result;
|
||||
}
|
||||
|
||||
@ -88,14 +88,14 @@ ZooKeeperNodeCache::GetResult ZooKeeperNodeCache::get(const std::string & path,
|
||||
result.exists = zookeeper->existsWatch(path, &result.stat, watch_callback);
|
||||
if (!result.exists)
|
||||
{
|
||||
node_cache.emplace(path, result);
|
||||
path_to_cached_znode.emplace(path, result);
|
||||
return result;
|
||||
}
|
||||
|
||||
/// Node was created between the two previous calls, try again. Watch is already set.
|
||||
|
||||
result.exists = zookeeper->tryGet(path, result.contents, &result.stat);
|
||||
node_cache.emplace(path, result);
|
||||
path_to_cached_znode.emplace(path, result);
|
||||
return result;
|
||||
}
|
||||
|
||||
|
@ -37,15 +37,15 @@ public:
|
||||
ZooKeeperNodeCache(const ZooKeeperNodeCache &) = delete;
|
||||
ZooKeeperNodeCache(ZooKeeperNodeCache &&) = default;
|
||||
|
||||
struct GetResult
|
||||
struct ZNode
|
||||
{
|
||||
bool exists = false;
|
||||
std::string contents;
|
||||
Coordination::Stat stat;
|
||||
};
|
||||
|
||||
GetResult get(const std::string & path, EventPtr watch_event);
|
||||
GetResult get(const std::string & path, Coordination::WatchCallback watch_callback);
|
||||
ZNode get(const std::string & path, EventPtr watch_event);
|
||||
ZNode get(const std::string & path, Coordination::WatchCallback watch_callback);
|
||||
|
||||
private:
|
||||
GetZooKeeper get_zookeeper;
|
||||
@ -59,7 +59,7 @@ private:
|
||||
|
||||
std::shared_ptr<Context> context;
|
||||
|
||||
std::unordered_map<std::string, GetResult> node_cache;
|
||||
std::unordered_map<std::string, ZNode> path_to_cached_znode;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -62,16 +62,16 @@ void ReplicatedMergeTreeAlterThread::run()
|
||||
auto zookeeper = storage.getZooKeeper();
|
||||
|
||||
String columns_path = storage.zookeeper_path + "/columns";
|
||||
auto columns_result = zk_node_cache.get(columns_path, task->getWatchCallback());
|
||||
if (!columns_result.exists)
|
||||
auto columns_znode = zk_node_cache.get(columns_path, task->getWatchCallback());
|
||||
if (!columns_znode.exists)
|
||||
throw Exception(columns_path + " doesn't exist", ErrorCodes::NOT_FOUND_NODE);
|
||||
int32_t columns_version = columns_result.stat.version;
|
||||
int32_t columns_version = columns_znode.stat.version;
|
||||
|
||||
String metadata_path = storage.zookeeper_path + "/metadata";
|
||||
auto metadata_result = zk_node_cache.get(metadata_path, task->getWatchCallback());
|
||||
if (!metadata_result.exists)
|
||||
auto metadata_znode = zk_node_cache.get(metadata_path, task->getWatchCallback());
|
||||
if (!metadata_znode.exists)
|
||||
throw Exception(metadata_path + " doesn't exist", ErrorCodes::NOT_FOUND_NODE);
|
||||
int32_t metadata_version = metadata_result.stat.version;
|
||||
int32_t metadata_version = metadata_znode.stat.version;
|
||||
|
||||
const bool changed_columns_version = (columns_version != storage.columns_version);
|
||||
const bool changed_metadata_version = (metadata_version != storage.metadata_version);
|
||||
@ -79,10 +79,10 @@ void ReplicatedMergeTreeAlterThread::run()
|
||||
if (!(changed_columns_version || changed_metadata_version || force_recheck_parts))
|
||||
return;
|
||||
|
||||
const String & columns_str = columns_result.contents;
|
||||
const String & columns_str = columns_znode.contents;
|
||||
auto columns_in_zk = ColumnsDescription::parse(columns_str);
|
||||
|
||||
const String & metadata_str = metadata_result.contents;
|
||||
const String & metadata_str = metadata_znode.contents;
|
||||
auto metadata_in_zk = ReplicatedMergeTreeTableMetadata::parse(metadata_str);
|
||||
auto metadata_diff = ReplicatedMergeTreeTableMetadata(storage.data).checkAndFindDiff(metadata_in_zk, /* allow_alter = */ true);
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user