mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 00:22:29 +00:00
Remove zkutil namespace from TestKeeperStorage
This commit is contained in:
parent
61fe49194b
commit
4aa11b3494
@ -8,10 +8,10 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
zkutil::TestKeeperStorage::RequestForSession parseRequest(nuraft::buffer & data)
|
||||
TestKeeperStorage::RequestForSession parseRequest(nuraft::buffer & data)
|
||||
{
|
||||
ReadBufferFromNuraftBuffer buffer(data);
|
||||
zkutil::TestKeeperStorage::RequestForSession request_for_session;
|
||||
TestKeeperStorage::RequestForSession request_for_session;
|
||||
readIntBinary(request_for_session.session_id, buffer);
|
||||
|
||||
int32_t length;
|
||||
@ -29,7 +29,7 @@ zkutil::TestKeeperStorage::RequestForSession parseRequest(nuraft::buffer & data)
|
||||
return request_for_session;
|
||||
}
|
||||
|
||||
nuraft::ptr<nuraft::buffer> writeResponses(zkutil::TestKeeperStorage::ResponsesForSessions & responses)
|
||||
nuraft::ptr<nuraft::buffer> writeResponses(TestKeeperStorage::ResponsesForSessions & responses)
|
||||
{
|
||||
WriteBufferFromNuraftBuffer buffer;
|
||||
for (const auto & response_and_session : responses)
|
||||
@ -52,7 +52,7 @@ nuraft::ptr<nuraft::buffer> NuKeeperStateMachine::commit(const size_t log_idx, n
|
||||
{
|
||||
LOG_DEBUG(log, "Commiting logidx {}", log_idx);
|
||||
auto request_for_session = parseRequest(data);
|
||||
zkutil::TestKeeperStorage::ResponsesForSessions responses_for_sessions;
|
||||
TestKeeperStorage::ResponsesForSessions responses_for_sessions;
|
||||
{
|
||||
std::lock_guard lock(storage_lock);
|
||||
responses_for_sessions = storage.processRequest(request_for_session.request, request_for_session.session_id);
|
||||
@ -107,7 +107,7 @@ NuKeeperStateMachine::StorageSnapshotPtr NuKeeperStateMachine::readSnapshot(nura
|
||||
TestKeeperStorageSerializer serializer;
|
||||
|
||||
ReadBufferFromNuraftBuffer reader(in);
|
||||
zkutil::TestKeeperStorage new_storage;
|
||||
TestKeeperStorage new_storage;
|
||||
serializer.deserialize(new_storage, reader);
|
||||
return std::make_shared<StorageSnapshot>(ss, new_storage);
|
||||
}
|
||||
|
@ -42,7 +42,7 @@ public:
|
||||
nuraft::ptr<nuraft::buffer> & data_out,
|
||||
bool & is_last_obj) override;
|
||||
|
||||
zkutil::TestKeeperStorage & getStorage()
|
||||
TestKeeperStorage & getStorage()
|
||||
{
|
||||
return storage;
|
||||
}
|
||||
@ -50,13 +50,13 @@ public:
|
||||
private:
|
||||
struct StorageSnapshot
|
||||
{
|
||||
StorageSnapshot(const nuraft::ptr<nuraft::snapshot> & s, const zkutil::TestKeeperStorage & storage_)
|
||||
StorageSnapshot(const nuraft::ptr<nuraft::snapshot> & s, const TestKeeperStorage & storage_)
|
||||
: snapshot(s)
|
||||
, storage(storage_)
|
||||
{}
|
||||
|
||||
nuraft::ptr<nuraft::snapshot> snapshot;
|
||||
zkutil::TestKeeperStorage storage;
|
||||
TestKeeperStorage storage;
|
||||
};
|
||||
|
||||
using StorageSnapshotPtr = std::shared_ptr<StorageSnapshot>;
|
||||
@ -67,7 +67,7 @@ private:
|
||||
|
||||
void writeSnapshot(const StorageSnapshotPtr & snapshot, nuraft::ptr<nuraft::buffer> & out) const;
|
||||
|
||||
zkutil::TestKeeperStorage storage;
|
||||
TestKeeperStorage storage;
|
||||
/// Mutex for snapshots
|
||||
std::mutex snapshots_lock;
|
||||
|
||||
|
@ -17,13 +17,6 @@ namespace ErrorCodes
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
namespace zkutil
|
||||
{
|
||||
|
||||
using namespace DB;
|
||||
|
||||
static String parentPath(const String & path)
|
||||
{
|
||||
auto rslash_pos = path.rfind('/');
|
||||
|
@ -8,7 +8,7 @@
|
||||
#include <unordered_set>
|
||||
#include <vector>
|
||||
|
||||
namespace zkutil
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using namespace DB;
|
||||
|
@ -11,10 +11,6 @@ namespace ErrorCodes
|
||||
extern const int TIMEOUT_EXCEEDED;
|
||||
}
|
||||
|
||||
}
|
||||
namespace zkutil
|
||||
{
|
||||
|
||||
void TestKeeperStorageDispatcher::processingThread()
|
||||
{
|
||||
setThreadName("TestKeeperSProc");
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <Coordination/TestKeeperStorage.h>
|
||||
#include <functional>
|
||||
|
||||
namespace zkutil
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using ZooKeeperResponseCallback = std::function<void(const Coordination::ZooKeeperResponsePtr & response)>;
|
||||
|
@ -8,7 +8,7 @@ namespace DB
|
||||
|
||||
namespace
|
||||
{
|
||||
void writeNode(const zkutil::TestKeeperStorage::Node & node, WriteBuffer & out)
|
||||
void writeNode(const TestKeeperStorage::Node & node, WriteBuffer & out)
|
||||
{
|
||||
Coordination::write(node.data, out);
|
||||
Coordination::write(node.acls, out);
|
||||
@ -18,7 +18,7 @@ namespace
|
||||
Coordination::write(node.seq_num, out);
|
||||
}
|
||||
|
||||
void readNode(zkutil::TestKeeperStorage::Node & node, ReadBuffer & in)
|
||||
void readNode(TestKeeperStorage::Node & node, ReadBuffer & in)
|
||||
{
|
||||
Coordination::read(node.data, in);
|
||||
Coordination::read(node.acls, in);
|
||||
@ -29,7 +29,7 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
void TestKeeperStorageSerializer::serialize(const zkutil::TestKeeperStorage & storage, WriteBuffer & out) const
|
||||
void TestKeeperStorageSerializer::serialize(const TestKeeperStorage & storage, WriteBuffer & out) const
|
||||
{
|
||||
Coordination::write(storage.zxid, out);
|
||||
Coordination::write(storage.session_id_counter, out);
|
||||
@ -49,7 +49,7 @@ void TestKeeperStorageSerializer::serialize(const zkutil::TestKeeperStorage & st
|
||||
}
|
||||
}
|
||||
|
||||
void TestKeeperStorageSerializer::deserialize(zkutil::TestKeeperStorage & storage, ReadBuffer & in) const
|
||||
void TestKeeperStorageSerializer::deserialize(TestKeeperStorage & storage, ReadBuffer & in) const
|
||||
{
|
||||
int64_t session_id_counter, zxid;
|
||||
Coordination::read(zxid, in);
|
||||
@ -63,7 +63,7 @@ void TestKeeperStorageSerializer::deserialize(zkutil::TestKeeperStorage & storag
|
||||
{
|
||||
std::string path;
|
||||
Coordination::read(path, in);
|
||||
zkutil::TestKeeperStorage::Node node;
|
||||
TestKeeperStorage::Node node;
|
||||
readNode(node, in);
|
||||
storage.container[path] = node;
|
||||
}
|
||||
|
@ -9,9 +9,9 @@ namespace DB
|
||||
class TestKeeperStorageSerializer
|
||||
{
|
||||
public:
|
||||
void serialize(const zkutil::TestKeeperStorage & storage, WriteBuffer & out) const;
|
||||
void serialize(const TestKeeperStorage & storage, WriteBuffer & out) const;
|
||||
|
||||
void deserialize(zkutil::TestKeeperStorage & storage, ReadBuffer & in) const;
|
||||
void deserialize(TestKeeperStorage & storage, ReadBuffer & in) const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -276,9 +276,9 @@ nuraft::ptr<nuraft::buffer> getZooKeeperLogEntry(int64_t session_id, const Coord
|
||||
return buf.getBuffer();
|
||||
}
|
||||
|
||||
zkutil::TestKeeperStorage::ResponsesForSessions getZooKeeperResponses(nuraft::ptr<nuraft::buffer> & buffer, const Coordination::ZooKeeperRequestPtr & request)
|
||||
DB::TestKeeperStorage::ResponsesForSessions getZooKeeperResponses(nuraft::ptr<nuraft::buffer> & buffer, const Coordination::ZooKeeperRequestPtr & request)
|
||||
{
|
||||
zkutil::TestKeeperStorage::ResponsesForSessions results;
|
||||
DB::TestKeeperStorage::ResponsesForSessions results;
|
||||
DB::ReadBufferFromNuraftBuffer buf(buffer);
|
||||
while (!buf.eof())
|
||||
{
|
||||
@ -296,28 +296,28 @@ zkutil::TestKeeperStorage::ResponsesForSessions getZooKeeperResponses(nuraft::pt
|
||||
Coordination::read(err, buf);
|
||||
auto response = request->makeResponse();
|
||||
response->readImpl(buf);
|
||||
results.push_back(zkutil::TestKeeperStorage::ResponseForSession{session_id, response});
|
||||
results.push_back(DB::TestKeeperStorage::ResponseForSession{session_id, response});
|
||||
}
|
||||
return results;
|
||||
}
|
||||
|
||||
TEST(CoordinationTest, TestStorageSerialization)
|
||||
{
|
||||
zkutil::TestKeeperStorage storage;
|
||||
storage.container["/hello"] = zkutil::TestKeeperStorage::Node{.data="world"};
|
||||
storage.container["/hello/somepath"] = zkutil::TestKeeperStorage::Node{.data="somedata"};
|
||||
DB::TestKeeperStorage storage;
|
||||
storage.container["/hello"] = DB::TestKeeperStorage::Node{.data="world"};
|
||||
storage.container["/hello/somepath"] = DB::TestKeeperStorage::Node{.data="somedata"};
|
||||
storage.session_id_counter = 5;
|
||||
storage.zxid = 156;
|
||||
storage.ephemerals[3] = {"/hello", "/"};
|
||||
storage.ephemerals[1] = {"/hello/somepath"};
|
||||
|
||||
DB::WriteBufferFromOwnString buffer;
|
||||
zkutil::TestKeeperStorageSerializer serializer;
|
||||
DB::TestKeeperStorageSerializer serializer;
|
||||
serializer.serialize(storage, buffer);
|
||||
std::string serialized = buffer.str();
|
||||
EXPECT_NE(serialized.size(), 0);
|
||||
DB::ReadBufferFromString read(serialized);
|
||||
zkutil::TestKeeperStorage new_storage;
|
||||
DB::TestKeeperStorage new_storage;
|
||||
serializer.deserialize(new_storage, read);
|
||||
|
||||
EXPECT_EQ(new_storage.container.size(), 3);
|
||||
|
0
src/Coordination/ya.make
Normal file
0
src/Coordination/ya.make
Normal file
@ -306,7 +306,7 @@ struct ContextShared
|
||||
ConfigurationPtr zookeeper_config; /// Stores zookeeper configs
|
||||
|
||||
mutable std::mutex test_keeper_storage_dispatcher_mutex;
|
||||
mutable std::shared_ptr<zkutil::TestKeeperStorageDispatcher> test_keeper_storage_dispatcher;
|
||||
mutable std::shared_ptr<TestKeeperStorageDispatcher> test_keeper_storage_dispatcher;
|
||||
mutable std::mutex auxiliary_zookeepers_mutex;
|
||||
mutable std::map<String, zkutil::ZooKeeperPtr> auxiliary_zookeepers; /// Map for auxiliary ZooKeeper clients.
|
||||
ConfigurationPtr auxiliary_zookeepers_config; /// Stores auxiliary zookeepers configs
|
||||
@ -1531,11 +1531,11 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const
|
||||
return shared->zookeeper;
|
||||
}
|
||||
|
||||
std::shared_ptr<zkutil::TestKeeperStorageDispatcher> & Context::getTestKeeperStorageDispatcher() const
|
||||
std::shared_ptr<TestKeeperStorageDispatcher> & Context::getTestKeeperStorageDispatcher() const
|
||||
{
|
||||
std::lock_guard lock(shared->test_keeper_storage_dispatcher_mutex);
|
||||
if (!shared->test_keeper_storage_dispatcher)
|
||||
shared->test_keeper_storage_dispatcher = std::make_shared<zkutil::TestKeeperStorageDispatcher>();
|
||||
shared->test_keeper_storage_dispatcher = std::make_shared<TestKeeperStorageDispatcher>();
|
||||
|
||||
return shared->test_keeper_storage_dispatcher;
|
||||
}
|
||||
|
@ -40,7 +40,6 @@ namespace Poco
|
||||
namespace zkutil
|
||||
{
|
||||
class ZooKeeper;
|
||||
class TestKeeperStorageDispatcher;
|
||||
}
|
||||
|
||||
|
||||
@ -107,6 +106,7 @@ using StoragePolicyPtr = std::shared_ptr<const StoragePolicy>;
|
||||
using StoragePoliciesMap = std::map<String, StoragePolicyPtr>;
|
||||
class StoragePolicySelector;
|
||||
using StoragePolicySelectorPtr = std::shared_ptr<const StoragePolicySelector>;
|
||||
class TestKeeperStorageDispatcher;
|
||||
|
||||
class IOutputFormat;
|
||||
using OutputFormatPtr = std::shared_ptr<IOutputFormat>;
|
||||
@ -513,7 +513,7 @@ public:
|
||||
std::shared_ptr<zkutil::ZooKeeper> getAuxiliaryZooKeeper(const String & name) const;
|
||||
|
||||
|
||||
std::shared_ptr<zkutil::TestKeeperStorageDispatcher> & getTestKeeperStorageDispatcher() const;
|
||||
std::shared_ptr<TestKeeperStorageDispatcher> & getTestKeeperStorageDispatcher() const;
|
||||
|
||||
/// Set auxiliary zookeepers configuration at server starting or configuration reloading.
|
||||
void reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & config);
|
||||
|
@ -28,7 +28,7 @@ private:
|
||||
IServer & server;
|
||||
Poco::Logger * log;
|
||||
Context global_context;
|
||||
std::shared_ptr<zkutil::TestKeeperStorageDispatcher> test_keeper_storage_dispatcher;
|
||||
std::shared_ptr<TestKeeperStorageDispatcher> test_keeper_storage_dispatcher;
|
||||
Poco::Timespan operation_timeout;
|
||||
Poco::Timespan session_timeout;
|
||||
int64_t session_id;
|
||||
|
Loading…
Reference in New Issue
Block a user