Store only root system node

This commit is contained in:
Antonio Andelic 2022-07-22 08:07:38 +00:00
parent 7d30ab80c4
commit 75476d5110
3 changed files with 66 additions and 36 deletions

View File

@ -1,11 +1,10 @@
#pragma once #pragma once
#include <IO/WriteHelpers.h>
namespace DB namespace DB
{ {
const std::string keeper_system_path = "/keeper";
const std::string keeper_api_version_path = keeper_system_path + "/api_version";
enum class KeeperApiVersion : uint8_t enum class KeeperApiVersion : uint8_t
{ {
V0 = 0, // ZooKeeper compatible version V0 = 0, // ZooKeeper compatible version
@ -14,4 +13,13 @@ enum class KeeperApiVersion : uint8_t
inline constexpr auto current_keeper_api_version = KeeperApiVersion::V1; inline constexpr auto current_keeper_api_version = KeeperApiVersion::V1;
const std::string keeper_system_path = "/keeper";
const std::string keeper_api_version_path = keeper_system_path + "/api_version";
using PathWithData = std::pair<std::string_view, std::string>;
const std::vector<PathWithData> data_for_system_paths
{
{keeper_api_version_path, toString(static_cast<uint8_t>(current_keeper_api_version))}
};
} }

View File

@ -145,6 +145,16 @@ namespace
} }
} }
namespace
{
bool isChildSystemPath(const std::string_view path)
{
auto [first_it, second_it] = std::mismatch(path.begin(), path.end(), keeper_system_path.begin(), keeper_system_path.end());
return first_it != path.end() && *first_it == '/' && second_it == keeper_system_path.end();
}
}
void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out) void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out)
{ {
@ -183,11 +193,16 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr
} }
/// Serialize data tree /// Serialize data tree
writeBinary(snapshot.snapshot_container_size, out); writeBinary(snapshot.snapshot_container_size - data_for_system_paths.size(), out);
size_t counter = 0; size_t counter = 0;
for (auto it = snapshot.begin; counter < snapshot.snapshot_container_size; ++counter) for (auto it = snapshot.begin; counter < snapshot.snapshot_container_size; ++counter)
{ {
const auto & path = it->key; const auto & path = it->key;
// write only the root system path because of digest
if (isChildSystemPath(path.toView()))
continue;
const auto & node = it->value; const auto & node = it->value;
/// Benign race condition possible while taking snapshot: NuRaft decide to create snapshot at some log id /// Benign race condition possible while taking snapshot: NuRaft decide to create snapshot at some log id

View File

@ -244,39 +244,46 @@ void KeeperStorage::initializeSystemNodes()
if (initialized) if (initialized)
throw Exception(ErrorCodes::LOGICAL_ERROR, "KeeperStorage system nodes initialized twice"); throw Exception(ErrorCodes::LOGICAL_ERROR, "KeeperStorage system nodes initialized twice");
const auto create_system_node = [&](const auto & path, auto data) // insert root system path
Node system_node;
system_node.setData("");
container.insertOrReplace(keeper_system_path, system_node);
// store digest for the empty node because we won't update
// its stats
addDigest(system_node, keeper_system_path);
// update root and the digest based on it
auto current_root_it = container.find("/");
assert(current_root_it != container.end());
removeDigest(current_root_it->value, "/");
auto updated_root_it = container.updateValue(
"/",
[](auto & node)
{
++node.stat.numChildren;
node.addChild(keeper_system_path);
}
);
addDigest(updated_root_it->value, "/");
// insert child system nodes
for (const auto & [path, data] : data_for_system_paths)
{ {
auto node_it = container.find(path); assert(keeper_api_version_path.starts_with(keeper_system_path));
if (node_it == container.end()) Node child_system_node;
{ system_node.setData(data);
// we update numChildren during preprocessing so and createNode is called during auto [map_key, _] = container.insert(std::string{path}, child_system_node);
// commit so we need to update it manually here /// Take child path from key owned by map.
container.updateValue( auto child_path = getBaseName(map_key->getKey());
parentPath(path), container.updateValue(
[](KeeperStorage::Node & parent) parentPath(child_path),
{ [child_path](auto & parent)
++parent.stat.numChildren; {
} // don't update stats so digest is okay
); parent.addChild(child_path);
createNode(path, std::move(data), {}, false, {}); }
} );
else }
{
container.updateValue(
path,
[data = std::move(data)](KeeperStorage::Node & node)
{
node.setData(std::move(data));
}
);
}
};
create_system_node(keeper_system_path, "");
assert(keeper_api_version_path.starts_with(keeper_system_path));
auto api_version_data = toString(static_cast<uint8_t>(current_keeper_api_version));
create_system_node(keeper_api_version_path, std::move(api_version_data));
initialized = true; initialized = true;
} }