local_mirror_uuid,
{"remote mirror uuid",
"remote mirror peer uuid"},
- nullptr, nullptr,
+ nullptr, nullptr, nullptr,
&m_mock_state_builder,
&m_do_resync, on_finish);
}
#include "tools/rbd_mirror/ImageReplayer.h"
#include "tools/rbd_mirror/InstanceWatcher.h"
#include "tools/rbd_mirror/MirrorStatusUpdater.h"
+#include "tools/rbd_mirror/PoolMetaCache.h"
#include "tools/rbd_mirror/Threads.h"
#include "tools/rbd_mirror/Throttler.h"
#include "tools/rbd_mirror/Types.h"
m_replayer = new ImageReplayerT(m_local_ioctx, m_local_mirror_uuid,
m_global_image_id, m_threads.get(),
m_instance_watcher, m_local_status_updater,
- nullptr);
+ nullptr, &m_pool_meta_cache);
m_replayer->add_peer({"peer uuid", m_remote_ioctx, {}, nullptr});
}
static int _image_number;
+ rbd::mirror::PoolMetaCache m_pool_meta_cache{g_ceph_context};
+
std::shared_ptr<librados::Rados> m_local_cluster;
std::unique_ptr<rbd::mirror::Threads<>> m_threads;
std::unique_ptr<rbd::mirror::Throttler<>> m_image_sync_throttler;
const std::string &local_mirror_uuid,
const RemotePoolMeta& remote_pool_meta,
::journal::CacheManagerHandler *cache_manager_handler,
+ PoolMetaCache* pool_meta_cache,
rbd::mirror::ProgressContext *progress_ctx,
StateBuilder<librbd::MockTestImageCtx>** state_builder,
bool *do_resync, Context *on_finish) {
void create_image_replayer(MockThreads &mock_threads) {
m_image_replayer = new MockImageReplayer(
m_local_io_ctx, "local_mirror_uuid", "global image id",
- &mock_threads, &m_instance_watcher, &m_local_status_updater, nullptr);
+ &mock_threads, &m_instance_watcher, &m_local_status_updater, nullptr,
+ nullptr);
m_image_replayer->add_peer({"peer_uuid", m_remote_io_ctx,
{"remote mirror uuid",
"remote mirror peer uuid"},
Threads<librbd::MockTestImageCtx> *threads,
InstanceWatcher<librbd::MockTestImageCtx> *instance_watcher,
MirrorStatusUpdater<librbd::MockTestImageCtx>* local_status_updater,
- journal::CacheManagerHandler *cache_manager_handler) {
+ journal::CacheManagerHandler *cache_manager_handler,
+ PoolMetaCache* pool_meta_cache) {
ceph_assert(s_instance != nullptr);
s_instance->global_image_id = global_image_id;
return s_instance;
MockImageReplayer mock_image_replayer;
MockInstanceReplayer instance_replayer(
m_local_io_ctx, "local_mirror_uuid",
- &mock_threads, &mock_service_daemon, &mock_status_updater, nullptr);
+ &mock_threads, &mock_service_daemon, &mock_status_updater, nullptr,
+ nullptr);
std::string global_image_id("global_image_id");
EXPECT_CALL(mock_image_replayer, get_global_image_id())
MockImageReplayer mock_image_replayer;
MockInstanceReplayer instance_replayer(
m_local_io_ctx, "local_mirror_uuid",
- &mock_threads, &mock_service_daemon, &mock_status_updater, nullptr);
+ &mock_threads, &mock_service_daemon, &mock_status_updater, nullptr,
+ nullptr);
std::string global_image_id("global_image_id");
EXPECT_CALL(mock_image_replayer, get_global_image_id())
MockImageReplayer mock_image_replayer;
MockInstanceReplayer instance_replayer(
m_local_io_ctx, "local_mirror_uuid",
- &mock_threads, &mock_service_daemon, &mock_status_updater, nullptr);
+ &mock_threads, &mock_service_daemon, &mock_status_updater, nullptr,
+ nullptr);
std::string global_image_id("global_image_id");
EXPECT_CALL(mock_image_replayer, get_global_image_id())
Threads<librbd::MockTestImageCtx> *threads,
ServiceDaemon<librbd::MockTestImageCtx> *service_daemon,
MirrorStatusUpdater<librbd::MockTestImageCtx>* local_status_updater,
- journal::CacheManagerHandler *cache_manager_handler) {
+ journal::CacheManagerHandler *cache_manager_handler,
+ PoolMetaCache* pool_meta_cache) {
ceph_assert(s_instance != nullptr);
return s_instance;
}
MockNamespaceReplayer namespace_replayer(
{}, m_local_io_ctx, m_remote_io_ctx, "local mirror uuid",
"local peer uuid", {"remote mirror uuid", ""}, m_mock_threads,
- nullptr, nullptr, nullptr, nullptr);
+ nullptr, nullptr, nullptr, nullptr, nullptr);
C_SaferCond on_init;
namespace_replayer.init(&on_init);
MockNamespaceReplayer namespace_replayer(
{}, m_local_io_ctx, m_remote_io_ctx, "local mirror uuid",
"local peer uuid", {"remote mirror uuid", ""}, m_mock_threads,
- nullptr, nullptr, nullptr, nullptr);
+ nullptr, nullptr, nullptr, nullptr, nullptr);
C_SaferCond on_init;
namespace_replayer.init(&on_init);
MockNamespaceReplayer namespace_replayer(
{}, m_local_io_ctx, m_remote_io_ctx, "local mirror uuid",
"local peer uuid", {"remote mirror uuid", ""}, m_mock_threads,
- nullptr, nullptr, nullptr, nullptr);
+ nullptr, nullptr, nullptr, nullptr, nullptr);
C_SaferCond on_init;
namespace_replayer.init(&on_init);
MockNamespaceReplayer namespace_replayer(
{}, m_local_io_ctx, m_remote_io_ctx, "local mirror uuid",
"local peer uuid", {"remote mirror uuid", ""}, m_mock_threads,
- nullptr, nullptr, nullptr, nullptr);
+ nullptr, nullptr, nullptr, nullptr, nullptr);
C_SaferCond on_init;
namespace_replayer.init(&on_init);
MockNamespaceReplayer namespace_replayer(
{}, m_local_io_ctx, m_remote_io_ctx, "local mirror uuid",
"local peer uuid", {"remote mirror uuid", ""}, m_mock_threads,
- nullptr, nullptr, &mock_service_daemon, nullptr);
+ nullptr, nullptr, &mock_service_daemon, nullptr, nullptr);
C_SaferCond on_init;
namespace_replayer.init(&on_init);
MockNamespaceReplayer namespace_replayer(
{}, m_local_io_ctx, m_remote_io_ctx, "local mirror uuid",
"local peer uuid", {"remote mirror uuid", ""}, m_mock_threads,
- nullptr, nullptr, &mock_service_daemon, nullptr);
+ nullptr, nullptr, &mock_service_daemon, nullptr, nullptr);
C_SaferCond on_init;
namespace_replayer.init(&on_init);
#include "tools/rbd_mirror/Throttler.h"
#include "tools/rbd_mirror/LeaderWatcher.h"
#include "tools/rbd_mirror/NamespaceReplayer.h"
+#include "tools/rbd_mirror/PoolMetaCache.h"
#include "tools/rbd_mirror/PoolReplayer.h"
#include "tools/rbd_mirror/RemotePoolPoller.h"
#include "tools/rbd_mirror/ServiceDaemon.h"
Throttler<librbd::MockTestImageCtx> *image_sync_throttler,
Throttler<librbd::MockTestImageCtx> *image_deletion_throttler,
ServiceDaemon<librbd::MockTestImageCtx> *service_daemon,
- journal::CacheManagerHandler *cache_manager_handler) {
+ journal::CacheManagerHandler *cache_manager_handler,
+ PoolMetaCache* pool_meta_cache) {
ceph_assert(s_instances.count(name));
auto namespace_replayer = s_instances[name];
s_instances.erase(name);
expect_service_daemon_add_or_update_attribute(
mock_service_daemon, "instance_id", {instance_id});
}
+
+ PoolMetaCache m_pool_meta_cache{g_ceph_context};
};
TEST_F(TestMockPoolReplayer, ConfigKeyOverride) {
mock_service_daemon, instance_id);
MockPoolReplayer pool_replayer(&mock_threads, &mock_service_daemon, nullptr,
+ &m_pool_meta_cache,
m_local_io_ctx.get_id(), peer_spec, {});
pool_replayer.init("siteA");
mock_service_daemon, instance_id);
MockPoolReplayer pool_replayer(&mock_threads, &mock_service_daemon, nullptr,
+ &m_pool_meta_cache,
m_local_io_ctx.get_id(), peer_spec, {});
pool_replayer.init("siteA");
mock_service_daemon, instance_id);
MockPoolReplayer pool_replayer(&mock_threads, &mock_service_daemon, nullptr,
+ &m_pool_meta_cache,
m_local_io_ctx.get_id(), peer_spec, {});
pool_replayer.init("siteA");
mock_service_daemon, instance_id);
MockPoolReplayer pool_replayer(&mock_threads, &mock_service_daemon, nullptr,
+ &m_pool_meta_cache,
m_local_io_ctx.get_id(), peer_spec, {});
pool_replayer.init("siteA");
MirrorStatusUpdater.cc
MirrorStatusWatcher.cc
NamespaceReplayer.cc
+ PoolMetaCache.cc
PoolReplayer.cc
PoolWatcher.cc
RemotePoolPoller.cc
const std::string &global_image_id, Threads<I> *threads,
InstanceWatcher<I> *instance_watcher,
MirrorStatusUpdater<I>* local_status_updater,
- journal::CacheManagerHandler *cache_manager_handler) :
+ journal::CacheManagerHandler *cache_manager_handler,
+ PoolMetaCache* pool_meta_cache) :
m_local_io_ctx(local_io_ctx), m_local_mirror_uuid(local_mirror_uuid),
m_global_image_id(global_image_id), m_threads(threads),
m_instance_watcher(instance_watcher),
m_local_status_updater(local_status_updater),
m_cache_manager_handler(cache_manager_handler),
+ m_pool_meta_cache(pool_meta_cache),
m_local_image_name(global_image_id),
m_lock(ceph::make_mutex("rbd::mirror::ImageReplayer " +
stringify(local_io_ctx.get_id()) + " " + global_image_id)),
m_threads, m_local_io_ctx, m_remote_image_peer.io_ctx, m_instance_watcher,
m_global_image_id, m_local_mirror_uuid,
m_remote_image_peer.remote_pool_meta, m_cache_manager_handler,
- &m_progress_cxt, &m_state_builder, &m_resync_requested, ctx);
+ m_pool_meta_cache, &m_progress_cxt, &m_state_builder, &m_resync_requested,
+ ctx);
request->get();
m_bootstrap_request = request;
template <typename> struct InstanceWatcher;
template <typename> struct MirrorStatusUpdater;
+struct PoolMetaCache;
template <typename> struct Threads;
namespace image_replayer {
const std::string &global_image_id, Threads<ImageCtxT> *threads,
InstanceWatcher<ImageCtxT> *instance_watcher,
MirrorStatusUpdater<ImageCtxT>* local_status_updater,
- journal::CacheManagerHandler *cache_manager_handler) {
+ journal::CacheManagerHandler *cache_manager_handler,
+ PoolMetaCache* pool_meta_cache) {
return new ImageReplayer(local_io_ctx, local_mirror_uuid, global_image_id,
threads, instance_watcher, local_status_updater,
- cache_manager_handler);
+ cache_manager_handler, pool_meta_cache);
}
void destroy() {
delete this;
Threads<ImageCtxT> *threads,
InstanceWatcher<ImageCtxT> *instance_watcher,
MirrorStatusUpdater<ImageCtxT>* local_status_updater,
- journal::CacheManagerHandler *cache_manager_handler);
+ journal::CacheManagerHandler *cache_manager_handler,
+ PoolMetaCache* pool_meta_cache);
virtual ~ImageReplayer();
ImageReplayer(const ImageReplayer&) = delete;
ImageReplayer& operator=(const ImageReplayer&) = delete;
InstanceWatcher<ImageCtxT> *m_instance_watcher;
MirrorStatusUpdater<ImageCtxT>* m_local_status_updater;
journal::CacheManagerHandler *m_cache_manager_handler;
+ PoolMetaCache* m_pool_meta_cache;
Peers m_peers;
Peer<ImageCtxT> m_remote_image_peer;
librados::IoCtx &local_io_ctx, const std::string &local_mirror_uuid,
Threads<I> *threads, ServiceDaemon<I>* service_daemon,
MirrorStatusUpdater<I>* local_status_updater,
- journal::CacheManagerHandler *cache_manager_handler)
+ journal::CacheManagerHandler *cache_manager_handler,
+ PoolMetaCache* pool_meta_cache)
: m_local_io_ctx(local_io_ctx), m_local_mirror_uuid(local_mirror_uuid),
m_threads(threads), m_service_daemon(service_daemon),
m_local_status_updater(local_status_updater),
m_cache_manager_handler(cache_manager_handler),
+ m_pool_meta_cache(pool_meta_cache),
m_lock(ceph::make_mutex("rbd::mirror::InstanceReplayer " +
stringify(local_io_ctx.get_id()))) {
}
auto image_replayer = ImageReplayer<I>::create(
m_local_io_ctx, m_local_mirror_uuid, global_image_id,
m_threads, instance_watcher, m_local_status_updater,
- m_cache_manager_handler);
+ m_cache_manager_handler, m_pool_meta_cache);
dout(10) << global_image_id << ": creating replayer " << image_replayer
<< dendl;
template <typename> class ImageReplayer;
template <typename> class InstanceWatcher;
template <typename> class MirrorStatusUpdater;
+struct PoolMetaCache;
template <typename> class ServiceDaemon;
template <typename> struct Threads;
librados::IoCtx &local_io_ctx, const std::string &local_mirror_uuid,
Threads<ImageCtxT> *threads, ServiceDaemon<ImageCtxT> *service_daemon,
MirrorStatusUpdater<ImageCtxT>* local_status_updater,
- journal::CacheManagerHandler *cache_manager_handler) {
+ journal::CacheManagerHandler *cache_manager_handler,
+ PoolMetaCache* pool_meta_cache) {
return new InstanceReplayer(local_io_ctx, local_mirror_uuid, threads,
service_daemon, local_status_updater,
- cache_manager_handler);
+ cache_manager_handler, pool_meta_cache);
}
void destroy() {
delete this;
Threads<ImageCtxT> *threads,
ServiceDaemon<ImageCtxT> *service_daemon,
MirrorStatusUpdater<ImageCtxT>* local_status_updater,
- journal::CacheManagerHandler *cache_manager_handler);
+ journal::CacheManagerHandler *cache_manager_handler,
+ PoolMetaCache* pool_meta_cache);
~InstanceReplayer();
int init();
ServiceDaemon<ImageCtxT> *m_service_daemon;
MirrorStatusUpdater<ImageCtxT>* m_local_status_updater;
journal::CacheManagerHandler *m_cache_manager_handler;
+ PoolMetaCache* m_pool_meta_cache;
ceph::mutex m_lock;
AsyncOpTracker m_async_op_tracker;
#include "librbd/ImageCtx.h"
#include "perfglue/heap_profiler.h"
#include "Mirror.h"
+#include "PoolMetaCache.h"
#include "ServiceDaemon.h"
#include "Threads.h"
m_args(args),
m_local(new librados::Rados()),
m_cache_manager_handler(new CacheManagerHandler(cct)),
+ m_pool_meta_cache(new PoolMetaCache(cct)),
m_asok_hook(new MirrorAdminSocketHook(cct, this))
{
m_threads =
dout(20) << "starting pool replayer for " << peer << dendl;
unique_ptr<PoolReplayer<>> pool_replayer(
new PoolReplayer<>(m_threads, m_service_daemon.get(),
- m_cache_manager_handler.get(), kv.first, peer,
+ m_cache_manager_handler.get(),
+ m_pool_meta_cache.get(), kv.first, peer,
m_args));
// TODO: make async
template <typename> struct Threads;
class CacheManagerHandler;
class MirrorAdminSocketHook;
+class PoolMetaCache;
/**
* Contains the main loop and overall state for rbd-mirror.
// monitor local cluster for config changes in peers
std::unique_ptr<ClusterWatcher> m_local_cluster_watcher;
std::unique_ptr<CacheManagerHandler> m_cache_manager_handler;
+ std::unique_ptr<PoolMetaCache> m_pool_meta_cache;
std::map<PoolPeer, std::unique_ptr<PoolReplayer<>>> m_pool_replayers;
std::atomic<bool> m_stopping = { false };
bool m_manual_stop = false;
librados::IoCtx &local_io_ctx, librados::IoCtx &remote_io_ctx,
const std::string &local_mirror_uuid,
const std::string& local_mirror_peer_uuid,
- const RemotePoolMeta& remote_pool_meta, Threads<I> *threads,
- Throttler<I> *image_sync_throttler, Throttler<I> *image_deletion_throttler,
+ const RemotePoolMeta& remote_pool_meta,
+ Threads<I> *threads,
+ Throttler<I> *image_sync_throttler,
+ Throttler<I> *image_deletion_throttler,
ServiceDaemon<I> *service_daemon,
- journal::CacheManagerHandler *cache_manager_handler) :
+ journal::CacheManagerHandler *cache_manager_handler,
+ PoolMetaCache* pool_meta_cache) :
m_namespace_name(name),
m_local_mirror_uuid(local_mirror_uuid),
m_local_mirror_peer_uuid(local_mirror_peer_uuid),
m_image_deletion_throttler(image_deletion_throttler),
m_service_daemon(service_daemon),
m_cache_manager_handler(cache_manager_handler),
+ m_pool_meta_cache(pool_meta_cache),
m_lock(ceph::make_mutex(librbd::util::unique_lock_name(
"rbd::mirror::NamespaceReplayer " + name, this))),
m_local_pool_watcher_listener(this, true),
m_instance_replayer.reset(InstanceReplayer<I>::create(
m_local_io_ctx, m_local_mirror_uuid, m_threads, m_service_daemon,
- m_local_status_updater.get(), m_cache_manager_handler));
+ m_local_status_updater.get(), m_cache_manager_handler,
+ m_pool_meta_cache));
auto ctx = create_context_callback<NamespaceReplayer<I>,
&NamespaceReplayer<I>::handle_init_instance_replayer>(this);
namespace rbd {
namespace mirror {
+struct PoolMetaCache;
template <typename> class ServiceDaemon;
template <typename> class Throttler;
template <typename> struct Threads;
Throttler<ImageCtxT> *image_sync_throttler,
Throttler<ImageCtxT> *image_deletion_throttler,
ServiceDaemon<ImageCtxT> *service_daemon,
- journal::CacheManagerHandler *cache_manager_handler) {
+ journal::CacheManagerHandler *cache_manager_handler,
+ PoolMetaCache* pool_meta_cache) {
return new NamespaceReplayer(name, local_ioctx, remote_ioctx,
local_mirror_uuid, local_mirror_peer_uuid,
remote_pool_meta, threads,
image_sync_throttler, image_deletion_throttler,
- service_daemon, cache_manager_handler);
+ service_daemon, cache_manager_handler,
+ pool_meta_cache);
}
NamespaceReplayer(const std::string &name,
Throttler<ImageCtxT> *image_sync_throttler,
Throttler<ImageCtxT> *image_deletion_throttler,
ServiceDaemon<ImageCtxT> *service_daemon,
- journal::CacheManagerHandler *cache_manager_handler);
+ journal::CacheManagerHandler *cache_manager_handler,
+ PoolMetaCache* pool_meta_cache);
NamespaceReplayer(const NamespaceReplayer&) = delete;
NamespaceReplayer& operator=(const NamespaceReplayer&) = delete;
Throttler<ImageCtxT> *m_image_deletion_throttler;
ServiceDaemon<ImageCtxT> *m_service_daemon;
journal::CacheManagerHandler *m_cache_manager_handler;
+ PoolMetaCache* m_pool_meta_cache;
mutable ceph::mutex m_lock;
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#include "common/debug.h"
+#include "common/dout.h"
+#include "tools/rbd_mirror/PoolMetaCache.h"
+#include <shared_mutex>
+
+#define dout_context g_ceph_context
+#define dout_subsys ceph_subsys_rbd_mirror
+#undef dout_prefix
+#define dout_prefix *_dout << "rbd::mirror::PoolMetaCache: " \
+ << this << " " << __func__ << ": "
+
+namespace rbd {
+namespace mirror {
+
+int PoolMetaCache::get_local_pool_meta(
+ int64_t pool_id,
+ LocalPoolMeta* local_pool_meta) const {
+ dout(15) << "pool_id=" << pool_id << dendl;
+
+ std::shared_lock locker{m_lock};
+ auto it = m_local_pool_metas.find(pool_id);
+ if (it == m_local_pool_metas.end()) {
+ return -ENOENT;
+ }
+
+ *local_pool_meta = it->second;
+ return 0;
+}
+
+void PoolMetaCache::set_local_pool_meta(
+ int64_t pool_id,
+ const LocalPoolMeta& local_pool_meta) {
+ dout(15) << "pool_id=" << pool_id << ", "
+ << "local_pool_meta=" << local_pool_meta << dendl;
+
+ std::unique_lock locker(m_lock);
+ m_local_pool_metas[pool_id] = local_pool_meta;
+}
+
+void PoolMetaCache::remove_local_pool_meta(int64_t pool_id) {
+ dout(15) << "pool_id=" << pool_id << dendl;
+
+ std::unique_lock locker(m_lock);
+ m_local_pool_metas.erase(pool_id);
+}
+
+int PoolMetaCache::get_remote_pool_meta(
+ int64_t pool_id,
+ RemotePoolMeta* remote_pool_meta) const {
+ dout(15) << "pool_id=" << pool_id << dendl;
+
+ std::shared_lock locker{m_lock};
+ auto it = m_remote_pool_metas.find(pool_id);
+ if (it == m_remote_pool_metas.end()) {
+ return -ENOENT;
+ }
+
+ *remote_pool_meta = it->second;
+ return 0;
+}
+
+void PoolMetaCache::set_remote_pool_meta(
+ int64_t pool_id,
+ const RemotePoolMeta& remote_pool_meta) {
+ dout(15) << "pool_id=" << pool_id << ", "
+ << "remote_pool_meta=" << remote_pool_meta << dendl;
+
+ std::unique_lock locker(m_lock);
+ m_remote_pool_metas[pool_id] = remote_pool_meta;
+}
+
+void PoolMetaCache::remove_remote_pool_meta(int64_t pool_id) {
+ dout(15) << "pool_id=" << pool_id << dendl;
+
+ std::unique_lock locker(m_lock);
+ m_remote_pool_metas.erase(pool_id);
+}
+
+} // namespace mirror
+} // namespace rbd
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+
+#ifndef CEPH_RBD_MIRROR_POOL_META_CACHE_H
+#define CEPH_RBD_MIRROR_POOL_META_CACHE_H
+
+#include "include/int_types.h"
+#include "common/ceph_mutex.h"
+#include "tools/rbd_mirror/Types.h"
+#include <map>
+
+struct CephContext;
+
+namespace rbd {
+namespace mirror {
+
+class PoolMetaCache {
+public:
+ PoolMetaCache(CephContext* cct)
+ : m_cct(cct) {
+ }
+ PoolMetaCache(const PoolMetaCache&) = delete;
+ PoolMetaCache& operator=(const PoolMetaCache&) = delete;
+
+ int get_local_pool_meta(int64_t pool_id,
+ LocalPoolMeta* local_pool_meta) const;
+ void set_local_pool_meta(int64_t pool_id,
+ const LocalPoolMeta& local_pool_meta);
+ void remove_local_pool_meta(int64_t pool_id);
+
+ int get_remote_pool_meta(int64_t pool_id,
+ RemotePoolMeta* remote_pool_meta) const;
+ void set_remote_pool_meta(int64_t pool_id,
+ const RemotePoolMeta& remote_pool_meta);
+ void remove_remote_pool_meta(int64_t pool_id);
+
+private:
+ CephContext* m_cct;
+
+ mutable ceph::shared_mutex m_lock =
+ ceph::make_shared_mutex("rbd::mirror::PoolMetaCache::m_lock");
+ std::map<int64_t, LocalPoolMeta> m_local_pool_metas;
+ std::map<int64_t, RemotePoolMeta> m_remote_pool_metas;
+};
+
+} // namespace mirror
+} // namespace rbd
+
+#endif // CEPH_RBD_MIRROR_POOL_META_CACHE_H
#include "global/global_context.h"
#include "librbd/api/Config.h"
#include "librbd/api/Namespace.h"
+#include "PoolMetaCache.h"
#include "RemotePoolPoller.h"
#include "ServiceDaemon.h"
#include "Threads.h"
template <typename I>
PoolReplayer<I>::PoolReplayer(
Threads<I> *threads, ServiceDaemon<I> *service_daemon,
- journal::CacheManagerHandler *cache_manager_handler, int64_t local_pool_id,
+ journal::CacheManagerHandler *cache_manager_handler,
+ PoolMetaCache* pool_meta_cache, int64_t local_pool_id,
const PeerSpec &peer, const std::vector<const char*> &args) :
m_threads(threads),
m_service_daemon(service_daemon),
m_cache_manager_handler(cache_manager_handler),
+ m_pool_meta_cache(pool_meta_cache),
m_local_pool_id(local_pool_id),
m_peer(peer),
m_args(args),
return;
}
ceph_assert(!m_remote_pool_meta.mirror_uuid.empty());
+ m_pool_meta_cache->set_remote_pool_meta(
+ m_remote_io_ctx.get_id(), m_remote_pool_meta);
+ m_pool_meta_cache->set_local_pool_meta(
+ m_local_io_ctx.get_id(), {m_local_mirror_uuid});
m_default_namespace_replayer.reset(NamespaceReplayer<I>::create(
"", m_local_io_ctx, m_remote_io_ctx, m_local_mirror_uuid, m_peer.uuid,
m_remote_pool_meta, m_threads, m_image_sync_throttler.get(),
m_image_deletion_throttler.get(), m_service_daemon,
- m_cache_manager_handler));
+ m_cache_manager_handler, m_pool_meta_cache));
C_SaferCond on_init;
m_default_namespace_replayer->init(&on_init);
C_SaferCond ctx;
m_remote_pool_poller->shut_down(&ctx);
ctx.wait();
+
+ m_pool_meta_cache->remove_remote_pool_meta(m_remote_io_ctx.get_id());
+ m_pool_meta_cache->remove_local_pool_meta(m_local_io_ctx.get_id());
}
m_remote_pool_poller.reset();
m_remote_pool_poller_listener.reset();
name, m_local_io_ctx, m_remote_io_ctx, m_local_mirror_uuid, m_peer.uuid,
m_remote_pool_meta, m_threads, m_image_sync_throttler.get(),
m_image_deletion_throttler.get(), m_service_daemon,
- m_cache_manager_handler);
+ m_cache_manager_handler, m_pool_meta_cache);
auto on_init = new LambdaContext(
[this, namespace_replayer, name, &mirroring_namespaces,
ctx=gather_ctx->new_sub()](int r) {
template <typename> class RemotePoolPoller;
namespace remote_pool_poller { struct Listener; }
+struct PoolMetaCache;
template <typename> class ServiceDaemon;
template <typename> struct Threads;
PoolReplayer(Threads<ImageCtxT> *threads,
ServiceDaemon<ImageCtxT> *service_daemon,
journal::CacheManagerHandler *cache_manager_handler,
+ PoolMetaCache* pool_meta_cache,
int64_t local_pool_id, const PeerSpec &peer,
const std::vector<const char*> &args);
~PoolReplayer();
Threads<ImageCtxT> *m_threads;
ServiceDaemon<ImageCtxT> *m_service_daemon;
journal::CacheManagerHandler *m_cache_manager_handler;
+ PoolMetaCache* m_pool_meta_cache;
int64_t m_local_pool_id = -1;
PeerSpec m_peer;
std::vector<const char*> m_args;
<< "id=" << image_id.id;
}
+std::ostream& operator<<(std::ostream& lhs,
+ const LocalPoolMeta& rhs) {
+ return lhs << "mirror_uuid=" << rhs.mirror_uuid;
+}
+
std::ostream& operator<<(std::ostream& lhs,
const RemotePoolMeta& rhs) {
return lhs << "mirror_uuid=" << rhs.mirror_uuid << ", "
typedef std::set<ImageId> ImageIds;
+struct LocalPoolMeta {
+ LocalPoolMeta() {}
+ LocalPoolMeta(const std::string& mirror_uuid)
+ : mirror_uuid(mirror_uuid) {
+ }
+
+ std::string mirror_uuid;
+};
+
+std::ostream& operator<<(std::ostream& lhs,
+ const LocalPoolMeta& local_pool_meta);
+
struct RemotePoolMeta {
RemotePoolMeta() {}
RemotePoolMeta(const std::string& mirror_uuid,
const std::string& local_mirror_uuid,
const RemotePoolMeta& remote_pool_meta,
::journal::CacheManagerHandler* cache_manager_handler,
+ PoolMetaCache* pool_meta_cache,
ProgressContext* progress_ctx,
StateBuilder<I>** state_builder,
bool* do_resync,
m_local_mirror_uuid(local_mirror_uuid),
m_remote_pool_meta(remote_pool_meta),
m_cache_manager_handler(cache_manager_handler),
+ m_pool_meta_cache(pool_meta_cache),
m_progress_ctx(progress_ctx),
m_state_builder(state_builder),
m_do_resync(do_resync),
template <typename> class ImageSync;
template <typename> class InstanceWatcher;
+struct PoolMetaCache;
template <typename> struct Threads;
namespace image_replayer {
const std::string& local_mirror_uuid,
const RemotePoolMeta& remote_pool_meta,
::journal::CacheManagerHandler* cache_manager_handler,
+ PoolMetaCache* pool_meta_cache,
ProgressContext* progress_ctx,
StateBuilder<ImageCtxT>** state_builder,
bool* do_resync,
Context* on_finish) {
return new BootstrapRequest(
threads, local_io_ctx, remote_io_ctx, instance_watcher, global_image_id,
- local_mirror_uuid, remote_pool_meta, cache_manager_handler, progress_ctx,
- state_builder, do_resync, on_finish);
+ local_mirror_uuid, remote_pool_meta, cache_manager_handler,
+ pool_meta_cache, progress_ctx, state_builder, do_resync, on_finish);
}
BootstrapRequest(
const std::string& local_mirror_uuid,
const RemotePoolMeta& remote_pool_meta,
::journal::CacheManagerHandler* cache_manager_handler,
+ PoolMetaCache* pool_meta_cache,
ProgressContext* progress_ctx,
StateBuilder<ImageCtxT>** state_builder,
bool* do_resync,
std::string m_local_mirror_uuid;
RemotePoolMeta m_remote_pool_meta;
::journal::CacheManagerHandler *m_cache_manager_handler;
+ PoolMetaCache* m_pool_meta_cache;
ProgressContext *m_progress_ctx;
StateBuilder<ImageCtxT>** m_state_builder;
bool *m_do_resync;