namespace librbd {
-class AsyncFlattenObjectContext : public C_AsyncObjectThrottle {
+class AsyncFlattenObjectContext : public C_AsyncObjectThrottle<> {
public:
- AsyncFlattenObjectContext(AsyncObjectThrottle &throttle, ImageCtx *image_ctx,
- uint64_t object_size, ::SnapContext snapc,
- uint64_t object_no)
+ AsyncFlattenObjectContext(AsyncObjectThrottle<> &throttle,
+ ImageCtx *image_ctx, uint64_t object_size,
+ ::SnapContext snapc, uint64_t object_no)
: C_AsyncObjectThrottle(throttle, *image_ctx), m_object_size(object_size),
m_snapc(snapc), m_object_no(object_no)
{
ldout(cct, 5) << this << " send" << dendl;
m_state = STATE_FLATTEN_OBJECTS;
- AsyncObjectThrottle::ContextFactory context_factory(
+ AsyncObjectThrottle<>::ContextFactory context_factory(
boost::lambda::bind(boost::lambda::new_ptr<AsyncFlattenObjectContext>(),
boost::lambda::_1, &m_image_ctx, m_object_size, m_snapc,
boost::lambda::_2));
- AsyncObjectThrottle *throttle = new AsyncObjectThrottle(
+ AsyncObjectThrottle<> *throttle = new AsyncObjectThrottle<>(
this, m_image_ctx, context_factory, create_callback_context(), &m_prog_ctx,
0, m_overlap_objects);
throttle->start_ops(m_image_ctx.concurrent_management_ops);
class ImageCtx;
class ProgressContext;
-class AsyncFlattenRequest : public AsyncRequest
+class AsyncFlattenRequest : public AsyncRequest<>
{
public:
AsyncFlattenRequest(ImageCtx &image_ctx, Context *on_finish,
#include "librbd/AsyncObjectThrottle.h"
#include "include/rbd/librbd.hpp"
#include "common/RWLock.h"
+#include "common/WorkQueue.h"
#include "librbd/AsyncRequest.h"
#include "librbd/ImageCtx.h"
#include "librbd/internal.h"
namespace librbd
{
-void C_AsyncObjectThrottle::finish(int r) {
- RWLock::RLocker l(m_image_ctx.owner_lock);
- m_finisher.finish_op(r);
-}
-
-AsyncObjectThrottle::AsyncObjectThrottle(const AsyncRequest* async_request,
- ImageCtx &image_ctx,
- const ContextFactory& context_factory,
- Context *ctx, ProgressContext *prog_ctx,
- uint64_t object_no,
- uint64_t end_object_no)
+template <typename T>
+AsyncObjectThrottle<T>::AsyncObjectThrottle(
+ const AsyncRequest<T>* async_request, T &image_ctx,
+ const ContextFactory& context_factory, Context *ctx,
+ ProgressContext *prog_ctx, uint64_t object_no, uint64_t end_object_no)
: m_lock(unique_lock_name("librbd::AsyncThrottle::m_lock", this)),
m_async_request(async_request), m_image_ctx(image_ctx),
m_context_factory(context_factory), m_ctx(ctx), m_prog_ctx(prog_ctx),
{
}
-void AsyncObjectThrottle::start_ops(uint64_t max_concurrent) {
+template <typename T>
+void AsyncObjectThrottle<T>::start_ops(uint64_t max_concurrent) {
assert(m_image_ctx.owner_lock.is_locked());
bool complete;
{
}
}
-void AsyncObjectThrottle::finish_op(int r) {
+template <typename T>
+void AsyncObjectThrottle<T>::finish_op(int r) {
assert(m_image_ctx.owner_lock.is_locked());
bool complete;
{
- Mutex::Locker l(m_lock);
+ Mutex::Locker locker(m_lock);
--m_current_ops;
if (r < 0 && r != -ENOENT && m_ret == 0) {
m_ret = r;
}
}
-void AsyncObjectThrottle::start_next_op() {
+template <typename T>
+void AsyncObjectThrottle<T>::start_next_op() {
bool done = false;
while (!done) {
if (m_async_request != NULL && m_async_request->is_canceled() &&
}
uint64_t ono = m_object_no++;
- C_AsyncObjectThrottle *ctx = m_context_factory(*this, ono);
+ C_AsyncObjectThrottle<T> *ctx = m_context_factory(*this, ono);
int r = ctx->send();
if (r < 0) {
}
} // namespace librbd
+
+template class librbd::AsyncObjectThrottle<librbd::ImageCtx>;
#include "include/int_types.h"
#include "include/Context.h"
+#include "common/RWLock.h"
#include <boost/function.hpp>
#include "include/assert.h"
namespace librbd
{
-class AsyncRequest;
+template <typename ImageCtxT> class AsyncRequest;
class ProgressContext;
struct ImageCtx;
virtual void finish_op(int r) = 0;
};
+template <typename ImageCtxT = ImageCtx>
class C_AsyncObjectThrottle : public Context {
public:
C_AsyncObjectThrottle(AsyncObjectThrottleFinisher &finisher,
- ImageCtx &image_ctx)
- : m_image_ctx(image_ctx), m_finisher(finisher)
- {
+ ImageCtxT &image_ctx)
+ : m_image_ctx(image_ctx), m_finisher(finisher) {
}
virtual int send() = 0;
protected:
- ImageCtx &m_image_ctx;
+ ImageCtxT &m_image_ctx;
- virtual void finish(int r);
+ virtual void finish(int r) {
+ RWLock::RLocker locker(m_image_ctx.owner_lock);
+ m_finisher.finish_op(r);
+ }
private:
AsyncObjectThrottleFinisher &m_finisher;
};
+template <typename ImageCtxT = ImageCtx>
class AsyncObjectThrottle : public AsyncObjectThrottleFinisher {
public:
- typedef boost::function<C_AsyncObjectThrottle*(AsyncObjectThrottle&,
- uint64_t)> ContextFactory;
+ typedef boost::function<
+ C_AsyncObjectThrottle<ImageCtxT>* (AsyncObjectThrottle&,
+ uint64_t)> ContextFactory;
- AsyncObjectThrottle(const AsyncRequest *async_request, ImageCtx &image_ctx,
+ AsyncObjectThrottle(const AsyncRequest<ImageCtxT> *async_request,
+ ImageCtxT &image_ctx,
const ContextFactory& context_factory, Context *ctx,
ProgressContext *prog_ctx, uint64_t object_no,
uint64_t end_object_no);
private:
Mutex m_lock;
- const AsyncRequest *m_async_request;
- ImageCtx &m_image_ctx;
+ const AsyncRequest<ImageCtxT> *m_async_request;
+ ImageCtxT &m_image_ctx;
ContextFactory m_context_factory;
Context *m_ctx;
ProgressContext *m_prog_ctx;
} // namespace librbd
+extern template class librbd::AsyncObjectThrottle<librbd::ImageCtx>;
+
#endif // CEPH_LIBRBD_ASYNC_OBJECT_THROTTLE_H
#include "librbd/AsyncRequest.h"
#include "librbd/ImageCtx.h"
#include "librbd/internal.h"
+#include "common/WorkQueue.h"
#include <boost/bind.hpp>
namespace librbd
{
-AsyncRequest::AsyncRequest(ImageCtx &image_ctx, Context *on_finish)
+template <typename T>
+AsyncRequest<T>::AsyncRequest(T &image_ctx, Context *on_finish)
: m_image_ctx(image_ctx), m_on_finish(on_finish), m_canceled(false),
m_xlist_item(this) {
+ assert(m_on_finish != NULL);
Mutex::Locker l(m_image_ctx.async_ops_lock);
m_image_ctx.async_requests.push_back(&m_xlist_item);
}
-AsyncRequest::~AsyncRequest() {
+template <typename T>
+AsyncRequest<T>::~AsyncRequest() {
Mutex::Locker l(m_image_ctx.async_ops_lock);
assert(m_xlist_item.remove_myself());
m_image_ctx.async_requests_cond.Signal();
}
-void AsyncRequest::async_complete(int r) {
+template <typename T>
+void AsyncRequest<T>::async_complete(int r) {
m_image_ctx.op_work_queue->queue(create_callback_context(), r);
}
-librados::AioCompletion *AsyncRequest::create_callback_completion() {
+template <typename T>
+librados::AioCompletion *AsyncRequest<T>::create_callback_completion() {
return librados::Rados::aio_create_completion(create_callback_context(),
NULL, rados_ctx_cb);
}
-Context *AsyncRequest::create_callback_context() {
- return new FunctionContext(boost::bind(&AsyncRequest::complete, this, _1));
+template <typename T>
+Context *AsyncRequest<T>::create_callback_context() {
+ return new FunctionContext(boost::bind(&AsyncRequest<T>::complete, this, _1));
}
-Context *AsyncRequest::create_async_callback_context() {
- return new FunctionContext(boost::bind(&AsyncRequest::async_complete, this,
+template <typename T>
+Context *AsyncRequest<T>::create_async_callback_context() {
+ return new FunctionContext(boost::bind(&AsyncRequest<T>::async_complete, this,
_1));;
}
} // namespace librbd
+
+template class librbd::AsyncRequest<librbd::ImageCtx>;
#include "include/rados/librados.hpp"
#include "include/xlist.h"
-/* DARWIN Missing ERESTART */
-#include "include/compat.h"
-
namespace librbd {
class ImageCtx;
+template <typename ImageCtxT = ImageCtx>
class AsyncRequest
{
public:
- AsyncRequest(ImageCtx &image_ctx, Context *on_finish);
+ AsyncRequest(ImageCtxT &image_ctx, Context *on_finish);
virtual ~AsyncRequest();
void complete(int r) {
}
protected:
- ImageCtx &m_image_ctx;
+ ImageCtxT &m_image_ctx;
Context *m_on_finish;
librados::AioCompletion *create_callback_completion();
}
private:
bool m_canceled;
- xlist<AsyncRequest *>::item m_xlist_item;
-};
-
-class C_AsyncRequest : public Context
-{
-public:
- C_AsyncRequest(AsyncRequest *req)
- : m_req(req)
- {
- }
-
-protected:
- virtual void finish(int r) {
- m_req->complete(r);
- }
-
-private:
- AsyncRequest *m_req;
+ typename xlist<AsyncRequest<ImageCtxT> *>::item m_xlist_item;
};
} // namespace librbd
+extern template class librbd::AsyncRequest<librbd::ImageCtx>;
+
#endif //CEPH_LIBRBD_ASYNC_REQUEST_H
class ImageCtx;
class ProgressContext;
-class AsyncResizeRequest : public AsyncRequest
+class AsyncResizeRequest : public AsyncRequest<>
{
public:
AsyncResizeRequest(ImageCtx &image_ctx, Context *on_finish, uint64_t new_size,
namespace librbd
{
-class C_CopyupObject : public C_AsyncObjectThrottle {
+class C_CopyupObject : public C_AsyncObjectThrottle<> {
public:
- C_CopyupObject(AsyncObjectThrottle &throttle, ImageCtx *image_ctx,
+ C_CopyupObject(AsyncObjectThrottle<> &throttle, ImageCtx *image_ctx,
::SnapContext snapc, uint64_t object_no)
: C_AsyncObjectThrottle(throttle, *image_ctx), m_snapc(snapc),
m_object_no(object_no)
uint64_t m_object_no;
};
-class C_RemoveObject : public C_AsyncObjectThrottle {
+class C_RemoveObject : public C_AsyncObjectThrottle<> {
public:
- C_RemoveObject(AsyncObjectThrottle &throttle, ImageCtx *image_ctx,
+ C_RemoveObject(AsyncObjectThrottle<> &throttle, ImageCtx *image_ctx,
uint64_t object_no)
: C_AsyncObjectThrottle(throttle, *image_ctx), m_object_no(object_no)
{
m_state = STATE_COPYUP_OBJECTS;
Context *ctx = create_callback_context();
- AsyncObjectThrottle::ContextFactory context_factory(
+ AsyncObjectThrottle<>::ContextFactory context_factory(
boost::lambda::bind(boost::lambda::new_ptr<C_CopyupObject>(),
boost::lambda::_1, &m_image_ctx, snapc, boost::lambda::_2));
- AsyncObjectThrottle *throttle = new AsyncObjectThrottle(
+ AsyncObjectThrottle<> *throttle = new AsyncObjectThrottle<>(
this, m_image_ctx, context_factory, ctx, &m_prog_ctx, copyup_start,
copyup_end);
throttle->start_ops(m_image_ctx.concurrent_management_ops);
m_state = STATE_REMOVE_OBJECTS;
Context *ctx = create_callback_context();
- AsyncObjectThrottle::ContextFactory context_factory(
+ AsyncObjectThrottle<>::ContextFactory context_factory(
boost::lambda::bind(boost::lambda::new_ptr<C_RemoveObject>(),
boost::lambda::_1, &m_image_ctx, boost::lambda::_2));
- AsyncObjectThrottle *throttle = new AsyncObjectThrottle(
+ AsyncObjectThrottle<> *throttle = new AsyncObjectThrottle<>(
this, m_image_ctx, context_factory, ctx, &m_prog_ctx, m_delete_start,
m_num_objects);
throttle->start_ops(m_image_ctx.concurrent_management_ops);
class ImageCtx;
class ProgressContext;
-class AsyncTrimRequest : public AsyncRequest
+class AsyncTrimRequest : public AsyncRequest<>
{
public:
AsyncTrimRequest(ImageCtx &image_ctx, Context *on_finish,
namespace {
-class UpdateObjectMap : public C_AsyncObjectThrottle {
+class UpdateObjectMap : public C_AsyncObjectThrottle<> {
public:
- UpdateObjectMap(AsyncObjectThrottle &throttle, ImageCtx *image_ctx,
+ UpdateObjectMap(AsyncObjectThrottle<> &throttle, ImageCtx *image_ctx,
uint64_t object_no, const std::vector<uint64_t> *snap_ids,
size_t snap_id_idx)
: C_AsyncObjectThrottle(throttle, *image_ctx),
m_state = STATE_OBJECT_MAP;
RWLock::RLocker owner_locker(m_ictx->owner_lock);
- AsyncObjectThrottle::ContextFactory context_factory(
+ AsyncObjectThrottle<>::ContextFactory context_factory(
boost::lambda::bind(boost::lambda::new_ptr<UpdateObjectMap>(),
boost::lambda::_1, m_ictx, m_object_no, &m_snap_ids,
boost::lambda::_2));
- AsyncObjectThrottle *throttle = new AsyncObjectThrottle(
+ AsyncObjectThrottle<> *throttle = new AsyncObjectThrottle<>(
NULL, *m_ictx, context_factory, create_callback_context(), NULL, 0,
m_snap_ids.size());
throttle->start_ops(m_ictx->concurrent_management_ops);
ldout(cct, 10) << "canceling async requests: count="
<< async_requests.size() << dendl;
- for (xlist<AsyncRequest*>::iterator it = async_requests.begin();
+ for (xlist<AsyncRequest<>*>::iterator it = async_requests.begin();
!it.end(); ++it) {
ldout(cct, 10) << "canceling async request: " << *it << dendl;
(*it)->cancel();
namespace librbd {
class AsyncOperation;
- class AsyncRequest;
+ template <typename ImageCtxT> class AsyncRequest;
class AsyncResizeRequest;
class CopyupRequest;
class ImageWatcher;
std::map<uint64_t, CopyupRequest*> copyup_list;
xlist<AsyncOperation*> async_ops;
- xlist<AsyncRequest*> async_requests;
+ xlist<AsyncRequest<>*> async_requests;
Cond async_requests_cond;
ObjectMap object_map;
private:
- class Request : public AsyncRequest {
+ class Request : public AsyncRequest<> {
public:
Request(ImageCtx &image_ctx, uint64_t snap_id, Context *on_finish)
: AsyncRequest(image_ctx, on_finish), m_snap_id(snap_id),
namespace {
-class C_VerifyObject : public C_AsyncObjectThrottle {
+class C_VerifyObject : public C_AsyncObjectThrottle<> {
public:
- C_VerifyObject(AsyncObjectThrottle &throttle, ImageCtx *image_ctx,
+ C_VerifyObject(AsyncObjectThrottle<> &throttle, ImageCtx *image_ctx,
uint64_t snap_id, uint64_t object_no)
: C_AsyncObjectThrottle(throttle, *image_ctx), m_snap_id(snap_id),
m_object_no(object_no), m_oid(m_image_ctx.get_object_name(m_object_no))
m_state = STATE_VERIFY_OBJECTS;
ldout(cct, 5) << this << " send_verify_objects" << dendl;
- AsyncObjectThrottle::ContextFactory context_factory(
+ AsyncObjectThrottle<>::ContextFactory context_factory(
boost::lambda::bind(boost::lambda::new_ptr<C_VerifyObject>(),
boost::lambda::_1, &m_image_ctx, snap_id, boost::lambda::_2));
- AsyncObjectThrottle *throttle = new AsyncObjectThrottle(
+ AsyncObjectThrottle<> *throttle = new AsyncObjectThrottle<>(
this, m_image_ctx, context_factory, create_callback_context(), &m_prog_ctx,
0, num_objects);
throttle->start_ops(cct->_conf->rbd_concurrent_management_ops);
class ImageCtx;
class ProgressContext;
-class RebuildObjectMapRequest : public AsyncRequest {
+class RebuildObjectMapRequest : public AsyncRequest<> {
public:
RebuildObjectMapRequest(ImageCtx &image_ctx, Context *on_finish,