operations(new Operations<>(*this)),
exclusive_lock(nullptr), object_map(nullptr),
io_work_queue(nullptr), op_work_queue(nullptr),
- asok_hook(nullptr)
+ asok_hook(nullptr),
+ trace_endpoint("librbd")
{
md_ctx.dup(p);
data_ctx.dup(p);
pname += snap_name;
}
+ trace_endpoint.copy_name(pname);
perf_start(pname);
if (cache) {
#include "common/Readahead.h"
#include "common/RWLock.h"
#include "common/snap_types.h"
+#include "common/zipkin_trace.h"
#include "include/buffer_fwd.h"
#include "include/rbd/librbd.hpp"
class ThreadPool;
class SafeTimer;
-namespace ZTracer { struct Trace; }
-
namespace librbd {
class AsyncOperation;
exclusive_lock::Policy *exclusive_lock_policy = nullptr;
journal::Policy *journal_policy = nullptr;
+ ZTracer::Endpoint trace_endpoint;
+
static bool _filter_metadata_confs(const string &prefix, std::map<string, bool> &configs,
const map<string, bufferlist> &pairs, map<string, bufferlist> *res);
uint64_t off;
bufferlist bl;
SnapContext snapc;
- Context *req_comp;
uint64_t journal_tid;
- bool request_sent;
+ ZTracer::Trace trace;
+ Context *req_comp;
+ bool request_sent = false;
C_WriteJournalCommit(ImageCtx *_image_ctx, const std::string &_oid,
uint64_t _object_no, uint64_t _off,
const bufferlist &_bl, const SnapContext& _snapc,
- Context *_req_comp, uint64_t _journal_tid)
+ uint64_t _journal_tid,
+ const ZTracer::Trace &trace, Context *_req_comp)
: image_ctx(_image_ctx), oid(_oid), object_no(_object_no), off(_off),
- bl(_bl), snapc(_snapc), req_comp(_req_comp), journal_tid(_journal_tid),
- request_sent(false) {
+ bl(_bl), snapc(_snapc), journal_tid(_journal_tid),
+ trace(trace), req_comp(_req_comp) {
CephContext *cct = image_ctx->cct;
ldout(cct, 20) << this << " C_WriteJournalCommit: "
<< "delaying write until journal tid "
request_sent = true;
auto req = new io::ObjectWriteRequest(image_ctx, oid, object_no, off,
- bl, snapc, this, 0);
+ bl, snapc, 0, trace, this);
req->send();
}
};
{
ZTracer::Trace trace;
if (parent_trace.valid()) {
- trace.init("", nullptr, &parent_trace);
+ trace.init("", &m_ictx->trace_endpoint, &parent_trace);
trace.copy_name("writeback " + oid.name);
trace.event("start");
}
assert(journal_tid == 0 || m_ictx->journal != NULL);
if (journal_tid != 0) {
m_ictx->journal->flush_event(
- journal_tid, new C_WriteJournalCommit(m_ictx, oid.name, object_no, off,
- bl, snapc, req_comp,
- journal_tid));
+ journal_tid, new C_WriteJournalCommit(
+ m_ictx, oid.name, object_no, off, bl, snapc, journal_tid, trace,
+ req_comp));
} else {
- auto req = new io::ObjectWriteRequest(m_ictx, oid.name, object_no,
- off, bl, snapc, req_comp, 0);
+ auto req = new io::ObjectWriteRequest(
+ m_ictx, oid.name, object_no, off, bl, snapc, 0, trace, req_comp);
req->send();
}
return ++m_tid;
#include "include/rados/librados.hpp"
#include "include/rbd_types.h"
#include "include/Context.h"
+#include "common/zipkin_trace.h"
#include <atomic>
#include <type_traits>
size_t *write_offset,
size_t *write_length,
size_t *offset);
+
+template <typename I>
+inline ZTracer::Trace create_trace(const I &image_ctx, const char *trace_name,
+ const ZTracer::Trace &parent_trace) {
+ if (parent_trace.valid()) {
+ return ZTracer::Trace(trace_name, &image_ctx.trace_endpoint, &parent_trace);
+ }
+ return ZTracer::Trace();
+}
+
} // namespace util
} // namespace librbd
auto aio_comp = io::AioCompletion::create_and_start(on_finish, &m_image_ctx,
io::AIO_TYPE_READ);
io::ImageReadRequest<I> req(m_image_ctx, aio_comp, std::move(image_extents),
- io::ReadResult{bl}, fadvise_flags);
+ io::ReadResult{bl}, fadvise_flags, {});
req.set_bypass_image_cache();
req.send();
}
auto aio_comp = io::AioCompletion::create_and_start(on_finish, &m_image_ctx,
io::AIO_TYPE_WRITE);
io::ImageWriteRequest<I> req(m_image_ctx, aio_comp, std::move(image_extents),
- std::move(bl), fadvise_flags);
+ std::move(bl), fadvise_flags, {});
req.set_bypass_image_cache();
req.send();
}
template <typename I>
void ImageWriteback<I>::aio_discard(uint64_t offset, uint64_t length,
- bool skip_partial_discard, Context *on_finish) {
+ bool skip_partial_discard,
+ Context *on_finish) {
CephContext *cct = m_image_ctx.cct;
ldout(cct, 20) << "offset=" << offset << ", "
<< "length=" << length << ", "
auto aio_comp = io::AioCompletion::create_and_start(on_finish, &m_image_ctx,
io::AIO_TYPE_DISCARD);
io::ImageDiscardRequest<I> req(m_image_ctx, aio_comp, offset, length,
- skip_partial_discard);
+ skip_partial_discard, {});
req.set_bypass_image_cache();
req.send();
}
auto aio_comp = io::AioCompletion::create_and_start(on_finish, &m_image_ctx,
io::AIO_TYPE_FLUSH);
- io::ImageFlushRequest<I> req(m_image_ctx, aio_comp);
+ io::ImageFlushRequest<I> req(m_image_ctx, aio_comp, {});
req.set_bypass_image_cache();
req.send();
}
auto aio_comp = io::AioCompletion::create_and_start(on_finish, &m_image_ctx,
io::AIO_TYPE_WRITESAME);
io::ImageWriteSameRequest<I> req(m_image_ctx, aio_comp, offset, length,
- std::move(bl), fadvise_flags);
+ std::move(bl), fadvise_flags, {});
req.set_bypass_image_cache();
req.send();
}
m_dest->io_work_queue->aio_write(comp, m_offset + write_offset,
write_length,
std::move(*write_bl),
- LIBRADOS_OP_FLAG_FADVISE_DONTNEED);
+ LIBRADOS_OP_FLAG_FADVISE_DONTNEED,
+ std::move(read_trace));
write_offset = offset;
write_length = 0;
}
gather_ctx->activate();
}
+ ZTracer::Trace read_trace;
+
private:
SimpleThrottle *m_throttle;
ImageCtx *m_dest;
}
}
+ ZTracer::Trace trace;
+ if (cct->_conf->rbd_blkin_trace_all) {
+ trace.init("copy", &src->trace_endpoint);
+ }
+
RWLock::RLocker owner_lock(src->owner_lock);
SimpleThrottle throttle(src->concurrent_management_ops, false);
uint64_t period = src->get_stripe_period();
- unsigned fadvise_flags = LIBRADOS_OP_FLAG_FADVISE_SEQUENTIAL | LIBRADOS_OP_FLAG_FADVISE_NOCACHE;
+ unsigned fadvise_flags = LIBRADOS_OP_FLAG_FADVISE_SEQUENTIAL |
+ LIBRADOS_OP_FLAG_FADVISE_NOCACHE;
for (uint64_t offset = 0; offset < src_size; offset += period) {
if (throttle.pending_error()) {
return throttle.wait_for_ret();
uint64_t len = min(period, src_size - offset);
bufferlist *bl = new bufferlist();
- Context *ctx = new C_CopyRead(&throttle, dest, offset, bl, sparse_size);
- auto comp = io::AioCompletion::create_and_start(ctx, src,
- io::AIO_TYPE_READ);
- io::ImageRequest<>::aio_read(src, comp, {{offset, len}},
- io::ReadResult{bl}, fadvise_flags);
+ auto ctx = new C_CopyRead(&throttle, dest, offset, bl, sparse_size);
+ auto comp = io::AioCompletion::create_and_start<Context>(
+ ctx, src, io::AIO_TYPE_READ);
+
+ io::ImageReadRequest<> req(*src, comp, {{offset, len}},
+ io::ReadResult{bl}, fadvise_flags,
+ std::move(trace));
+ ctx->read_trace = req.get_trace();
+
+ req.send();
prog_ctx.update_progress(offset, src_size);
}
uint64_t period = ictx->get_stripe_period();
uint64_t left = mylen;
+ ZTracer::Trace trace;
+ if (ictx->cct->_conf->rbd_blkin_trace_all) {
+ trace.init("read_iterate", &ictx->trace_endpoint);
+ }
+
RWLock::RLocker owner_locker(ictx->owner_lock);
start_time = ceph_clock_now();
while (left > 0) {
auto c = io::AioCompletion::create_and_start(&ctx, ictx,
io::AIO_TYPE_READ);
io::ImageRequest<>::aio_read(ictx, c, {{off, read_len}},
- io::ReadResult{&bl}, 0);
+ io::ReadResult{&bl}, 0, std::move(trace));
int ret = ctx.wait();
if (ret < 0) {
CopyupRequest::CopyupRequest(ImageCtx *ictx, const std::string &oid,
- uint64_t objectno, Extents &&image_extents)
+ uint64_t objectno, Extents &&image_extents,
+ const ZTracer::Trace &parent_trace)
: m_ictx(ictx), m_oid(oid), m_object_no(objectno),
- m_image_extents(image_extents), m_state(STATE_READ_FROM_PARENT)
+ m_image_extents(image_extents),
+ m_trace(util::create_trace(*m_ictx, "copy-up", parent_trace)),
+ m_state(STATE_READ_FROM_PARENT)
{
m_async_op.start_op(*m_ictx);
}
r = rados.ioctx_create2(m_ictx->data_ctx.get_id(), m_data_ctx);
assert(r == 0);
- r = m_data_ctx.aio_operate(m_oid, comp, ©up_op, 0, snaps);
+ r = m_data_ctx.aio_operate(
+ m_oid, comp, ©up_op, 0, snaps,
+ (m_trace.valid() ? m_trace.get_info() : nullptr));
assert(r == 0);
comp->release();
}
snaps.insert(snaps.end(), snapc.snaps.begin(), snapc.snaps.end());
librados::AioCompletion *comp = util::create_rados_callback(this);
- r = m_ictx->data_ctx.aio_operate(m_oid, comp, &write_op);
+ r = m_ictx->data_ctx.aio_operate(
+ m_oid, comp, &write_op, snapc.seq, snaps,
+ (m_trace.valid() ? m_trace.get_info() : nullptr));
assert(r == 0);
comp->release();
}
<< ", extents " << m_image_extents
<< dendl;
ImageRequest<>::aio_read(m_ictx->parent, comp, std::move(m_image_extents),
- ReadResult{&m_copyup_data}, 0);
+ ReadResult{&m_copyup_data}, 0, m_trace);
}
void CopyupRequest::complete(int r)
#include "include/int_types.h"
#include "include/rados/librados.hpp"
#include "include/buffer.h"
+#include "common/zipkin_trace.h"
#include "librbd/io/Types.h"
#include <string>
#include <vector>
#include <atomic>
+namespace ZTracer { struct Trace; }
+
namespace librbd {
struct ImageCtx;
class CopyupRequest {
public:
CopyupRequest(ImageCtx *ictx, const std::string &oid, uint64_t objectno,
- Extents &&image_extents);
+ Extents &&image_extents, const ZTracer::Trace &parent_trace);
~CopyupRequest();
void append_request(ObjectRequest<ImageCtx> *req);
std::string m_oid;
uint64_t m_object_no;
Extents m_image_extents;
+ ZTracer::Trace m_trace;
+
State m_state;
ceph::bufferlist m_copyup_data;
std::vector<ObjectRequest<ImageCtx> *> m_pending_requests;
template <typename I>
void ImageRequest<I>::aio_read(I *ictx, AioCompletion *c,
Extents &&image_extents,
- ReadResult &&read_result, int op_flags) {
+ ReadResult &&read_result, int op_flags,
+ const ZTracer::Trace &parent_trace) {
ImageReadRequest<I> req(*ictx, c, std::move(image_extents),
- std::move(read_result), op_flags);
+ std::move(read_result), op_flags, parent_trace);
req.send();
}
template <typename I>
void ImageRequest<I>::aio_write(I *ictx, AioCompletion *c,
Extents &&image_extents, bufferlist &&bl,
- int op_flags) {
+ int op_flags,
+ const ZTracer::Trace &parent_trace) {
ImageWriteRequest<I> req(*ictx, c, std::move(image_extents), std::move(bl),
- op_flags);
+ op_flags, parent_trace);
req.send();
}
template <typename I>
void ImageRequest<I>::aio_discard(I *ictx, AioCompletion *c,
uint64_t off, uint64_t len,
- bool skip_partial_discard) {
- ImageDiscardRequest<I> req(*ictx, c, off, len, skip_partial_discard);
+ bool skip_partial_discard,
+ const ZTracer::Trace &parent_trace) {
+ ImageDiscardRequest<I> req(*ictx, c, off, len, skip_partial_discard,
+ parent_trace);
req.send();
}
template <typename I>
-void ImageRequest<I>::aio_flush(I *ictx, AioCompletion *c) {
- ImageFlushRequest<I> req(*ictx, c);
+void ImageRequest<I>::aio_flush(I *ictx, AioCompletion *c,
+ const ZTracer::Trace &parent_trace) {
+ ImageFlushRequest<I> req(*ictx, c, parent_trace);
req.send();
}
template <typename I>
void ImageRequest<I>::aio_writesame(I *ictx, AioCompletion *c,
uint64_t off, uint64_t len,
- bufferlist &&bl,
- int op_flags) {
- ImageWriteSameRequest<I> req(*ictx, c, off, len, std::move(bl), op_flags);
+ bufferlist &&bl, int op_flags,
+ const ZTracer::Trace &parent_trace) {
+ ImageWriteSameRequest<I> req(*ictx, c, off, len, std::move(bl), op_flags,
+ parent_trace);
req.send();
}
template <typename I>
ImageReadRequest<I>::ImageReadRequest(I &image_ctx, AioCompletion *aio_comp,
Extents &&image_extents,
- ReadResult &&read_result,
- int op_flags)
- : ImageRequest<I>(image_ctx, aio_comp, std::move(image_extents)),
+ ReadResult &&read_result, int op_flags,
+ const ZTracer::Trace &parent_trace)
+ : ImageRequest<I>(image_ctx, aio_comp, std::move(image_extents), "read",
+ parent_trace),
m_op_flags(op_flags) {
aio_comp->read_result = std::move(read_result);
}
aio_comp);
ObjectReadRequest<I> *req = ObjectReadRequest<I>::create(
&image_ctx, extent.oid.name, extent.objectno, extent.offset,
- extent.length, extent.buffer_extents, snap_id, true, req_comp,
- m_op_flags);
+ extent.length, extent.buffer_extents, snap_id, true, m_op_flags,
+ this->m_trace, req_comp);
req_comp->request = req;
if (image_ctx.object_cacher) {
C_ObjectCacheRead<I> *cache_comp = new C_ObjectCacheRead<I>(image_ctx,
req);
- image_ctx.aio_read_from_cache(extent.oid, extent.objectno,
- &req->data(), extent.length,
- extent.offset, cache_comp, m_op_flags,
- nullptr);
+ image_ctx.aio_read_from_cache(
+ extent.oid, extent.objectno, &req->data(), extent.length,
+ extent.offset, cache_comp, m_op_flags,
+ (this->m_trace.valid() ? &this->m_trace : nullptr));
} else {
req->send();
}
AioCompletion *aio_comp = this->m_aio_comp;
C_AioRequest *req_comp = new C_AioRequest(aio_comp);
- image_ctx.write_to_cache(object_extent.oid, bl, object_extent.length,
- object_extent.offset, req_comp, m_op_flags,
- journal_tid, nullptr);
+ image_ctx.write_to_cache(
+ object_extent.oid, bl, object_extent.length, object_extent.offset,
+ req_comp, m_op_flags, journal_tid,
+ (this->m_trace.valid() ? &this->m_trace : nullptr));
}
}
assemble_extent(object_extent, &bl);
ObjectRequest<I> *req = ObjectRequest<I>::create_write(
&image_ctx, object_extent.oid.name, object_extent.objectno,
- object_extent.offset, bl, snapc, on_finish, m_op_flags);
+ object_extent.offset, bl, snapc, m_op_flags, this->m_trace, on_finish);
return req;
}
if (object_extent.length == image_ctx.layout.object_size) {
req = ObjectRequest<I>::create_remove(
&image_ctx, object_extent.oid.name, object_extent.objectno, snapc,
- on_finish);
+ this->m_trace, on_finish);
} else if (object_extent.offset + object_extent.length ==
image_ctx.layout.object_size) {
req = ObjectRequest<I>::create_truncate(
&image_ctx, object_extent.oid.name, object_extent.objectno,
- object_extent.offset, snapc, on_finish);
+ object_extent.offset, snapc, this->m_trace, on_finish);
} else {
req = ObjectRequest<I>::create_zero(
&image_ctx, object_extent.oid.name, object_extent.objectno,
- object_extent.offset, object_extent.length, snapc, on_finish);
+ object_extent.offset, object_extent.length, snapc,
+ this->m_trace, on_finish);
}
return req;
}
AioCompletion *aio_comp = this->m_aio_comp;
C_AioRequest *req_comp = new C_AioRequest(aio_comp);
- image_ctx.write_to_cache(object_extent.oid, bl, object_extent.length,
- object_extent.offset, req_comp, m_op_flags,
- journal_tid, nullptr);
+ image_ctx.write_to_cache(
+ object_extent.oid, bl, object_extent.length, object_extent.offset,
+ req_comp, m_op_flags, journal_tid,
+ (this->m_trace.valid() ? &this->m_trace : nullptr));
}
}
req = ObjectRequest<I>::create_writesame(
&image_ctx, object_extent.oid.name, object_extent.objectno,
object_extent.offset, object_extent.length,
- bl, snapc, on_finish, m_op_flags);
+ bl, snapc, m_op_flags, this->m_trace, on_finish);
return req;
}
req = ObjectRequest<I>::create_write(
&image_ctx, object_extent.oid.name, object_extent.objectno,
- object_extent.offset,
- bl, snapc, on_finish, m_op_flags);
+ object_extent.offset, bl, snapc, m_op_flags, this->m_trace, on_finish);
return req;
}
#include "include/int_types.h"
#include "include/buffer_fwd.h"
#include "common/snap_types.h"
+#include "common/zipkin_trace.h"
#include "osd/osd_types.h"
+#include "librbd/Utils.h"
#include "librbd/io/Types.h"
#include <list>
#include <utility>
public:
typedef std::vector<std::pair<uint64_t,uint64_t> > Extents;
- virtual ~ImageRequest() {}
+ virtual ~ImageRequest() {
+ m_trace.event("finish");
+ }
static void aio_read(ImageCtxT *ictx, AioCompletion *c,
Extents &&image_extents, ReadResult &&read_result,
- int op_flags);
+ int op_flags, const ZTracer::Trace &parent_trace);
static void aio_write(ImageCtxT *ictx, AioCompletion *c,
- Extents &&image_extents, bufferlist &&bl, int op_flags);
+ Extents &&image_extents, bufferlist &&bl, int op_flags,
+ const ZTracer::Trace &parent_trace);
static void aio_discard(ImageCtxT *ictx, AioCompletion *c, uint64_t off,
- uint64_t len, bool skip_partial_discard);
- static void aio_flush(ImageCtxT *ictx, AioCompletion *c);
+ uint64_t len, bool skip_partial_discard,
+ const ZTracer::Trace &parent_trace);
+ static void aio_flush(ImageCtxT *ictx, AioCompletion *c,
+ const ZTracer::Trace &parent_trace);
static void aio_writesame(ImageCtxT *ictx, AioCompletion *c, uint64_t off,
- uint64_t len, bufferlist &&bl, int op_flags);
+ uint64_t len, bufferlist &&bl, int op_flags,
+ const ZTracer::Trace &parent_trace);
virtual bool is_write_op() const {
return false;
m_bypass_image_cache = true;
}
+ inline const ZTracer::Trace &get_trace() const {
+ return m_trace;
+ }
+
protected:
typedef std::list<ObjectRequestHandle *> ObjectRequests;
ImageCtxT &m_image_ctx;
AioCompletion *m_aio_comp;
Extents m_image_extents;
+ ZTracer::Trace m_trace;
bool m_bypass_image_cache = false;
ImageRequest(ImageCtxT &image_ctx, AioCompletion *aio_comp,
- Extents &&image_extents)
+ Extents &&image_extents, const char *trace_name,
+ const ZTracer::Trace &parent_trace)
: m_image_ctx(image_ctx), m_aio_comp(aio_comp),
- m_image_extents(image_extents) {
+ m_image_extents(std::move(image_extents)),
+ m_trace(util::create_trace(image_ctx, trace_name, parent_trace)) {
+ m_trace.event("start");
}
+
virtual int clip_request();
virtual void send_request() = 0;
ImageReadRequest(ImageCtxT &image_ctx, AioCompletion *aio_comp,
Extents &&image_extents, ReadResult &&read_result,
- int op_flags);
+ int op_flags, const ZTracer::Trace &parent_trace);
protected:
int clip_request() override;
typedef std::vector<ObjectExtent> ObjectExtents;
AbstractImageWriteRequest(ImageCtxT &image_ctx, AioCompletion *aio_comp,
- Extents &&image_extents)
- : ImageRequest<ImageCtxT>(image_ctx, aio_comp, std::move(image_extents)),
+ Extents &&image_extents, const char *trace_name,
+ const ZTracer::Trace &parent_trace)
+ : ImageRequest<ImageCtxT>(image_ctx, aio_comp, std::move(image_extents),
+ trace_name, parent_trace),
m_synchronous(false) {
}
using typename ImageRequest<ImageCtxT>::Extents;
ImageWriteRequest(ImageCtxT &image_ctx, AioCompletion *aio_comp,
- Extents &&image_extents, bufferlist &&bl, int op_flags)
- : AbstractImageWriteRequest<ImageCtxT>(image_ctx, aio_comp,
- std::move(image_extents)),
+ Extents &&image_extents, bufferlist &&bl, int op_flags,
+ const ZTracer::Trace &parent_trace)
+ : AbstractImageWriteRequest<ImageCtxT>(
+ image_ctx, aio_comp, std::move(image_extents), "write", parent_trace),
m_bl(std::move(bl)), m_op_flags(op_flags) {
}
class ImageDiscardRequest : public AbstractImageWriteRequest<ImageCtxT> {
public:
ImageDiscardRequest(ImageCtxT &image_ctx, AioCompletion *aio_comp,
- uint64_t off, uint64_t len, bool skip_partial_discard)
- : AbstractImageWriteRequest<ImageCtxT>(image_ctx, aio_comp, {{off, len}}),
+ uint64_t off, uint64_t len, bool skip_partial_discard,
+ const ZTracer::Trace &parent_trace)
+ : AbstractImageWriteRequest<ImageCtxT>(
+ image_ctx, aio_comp, {{off, len}}, "discard", parent_trace),
m_skip_partial_discard(skip_partial_discard) {
}
template <typename ImageCtxT = ImageCtx>
class ImageFlushRequest : public ImageRequest<ImageCtxT> {
public:
- ImageFlushRequest(ImageCtxT &image_ctx, AioCompletion *aio_comp)
- : ImageRequest<ImageCtxT>(image_ctx, aio_comp, {}) {
+ ImageFlushRequest(ImageCtxT &image_ctx, AioCompletion *aio_comp,
+ const ZTracer::Trace &parent_trace)
+ : ImageRequest<ImageCtxT>(image_ctx, aio_comp, {}, "flush", parent_trace) {
}
bool is_write_op() const override {
public:
ImageWriteSameRequest(ImageCtxT &image_ctx, AioCompletion *aio_comp,
uint64_t off, uint64_t len, bufferlist &&bl,
- int op_flags)
- : AbstractImageWriteRequest<ImageCtxT>(image_ctx, aio_comp, {{off, len}}),
+ int op_flags, const ZTracer::Trace &parent_trace)
+ : AbstractImageWriteRequest<ImageCtxT>(
+ image_ctx, aio_comp, {{off, len}}, "writesame", parent_trace),
m_data_bl(std::move(bl)), m_op_flags(op_flags) {
}
#include "librbd/io/ImageRequestWQ.h"
#include "common/errno.h"
+#include "common/zipkin_trace.h"
#include "librbd/ExclusiveLock.h"
#include "librbd/ImageCtx.h"
#include "librbd/ImageState.h"
void ImageRequestWQ::aio_read(AioCompletion *c, uint64_t off, uint64_t len,
ReadResult &&read_result, int op_flags,
bool native_async) {
- c->init_time(&m_image_ctx, AIO_TYPE_READ);
CephContext *cct = m_image_ctx.cct;
+ ZTracer::Trace trace;
+ if (cct->_conf->rbd_blkin_trace_all) {
+ trace.init("wq: read", &m_image_ctx.trace_endpoint);
+ trace.event("start");
+ }
+
+ c->init_time(&m_image_ctx, AIO_TYPE_READ);
ldout(cct, 20) << "ictx=" << &m_image_ctx << ", "
<< "completion=" << c << ", off=" << off << ", "
<< "len=" << len << ", " << "flags=" << op_flags << dendl;
if (m_image_ctx.non_blocking_aio || writes_blocked() || !writes_empty() ||
lock_required) {
queue(new ImageReadRequest<>(m_image_ctx, c, {{off, len}},
- std::move(read_result), op_flags));
+ std::move(read_result), op_flags, trace));
} else {
c->start_op();
ImageRequest<>::aio_read(&m_image_ctx, c, {{off, len}},
- std::move(read_result), op_flags);
+ std::move(read_result), op_flags, trace);
finish_in_flight_op();
}
+ trace.event("finish");
}
void ImageRequestWQ::aio_write(AioCompletion *c, uint64_t off, uint64_t len,
bufferlist &&bl, int op_flags,
bool native_async) {
- c->init_time(&m_image_ctx, AIO_TYPE_WRITE);
CephContext *cct = m_image_ctx.cct;
+ ZTracer::Trace trace;
+ if (cct->_conf->rbd_blkin_trace_all) {
+ trace.init("wq: write", &m_image_ctx.trace_endpoint);
+ trace.event("init");
+ }
+
+ c->init_time(&m_image_ctx, AIO_TYPE_WRITE);
ldout(cct, 20) << "ictx=" << &m_image_ctx << ", "
<< "completion=" << c << ", off=" << off << ", "
<< "len=" << len << ", flags=" << op_flags << dendl;
RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
if (m_image_ctx.non_blocking_aio || writes_blocked()) {
queue(new ImageWriteRequest<>(m_image_ctx, c, {{off, len}},
- std::move(bl), op_flags));
+ std::move(bl), op_flags, trace));
} else {
c->start_op();
ImageRequest<>::aio_write(&m_image_ctx, c, {{off, len}},
- std::move(bl), op_flags);
+ std::move(bl), op_flags, trace);
finish_in_flight_op();
}
+ trace.event("finish");
}
void ImageRequestWQ::aio_discard(AioCompletion *c, uint64_t off,
uint64_t len, bool skip_partial_discard,
bool native_async) {
- c->init_time(&m_image_ctx, AIO_TYPE_DISCARD);
CephContext *cct = m_image_ctx.cct;
+ ZTracer::Trace trace;
+ if (cct->_conf->rbd_blkin_trace_all) {
+ trace.init("wq: discard", &m_image_ctx.trace_endpoint);
+ trace.event("init");
+ }
+
+ c->init_time(&m_image_ctx, AIO_TYPE_DISCARD);
ldout(cct, 20) << "ictx=" << &m_image_ctx << ", "
<< "completion=" << c << ", off=" << off << ", len=" << len
<< dendl;
RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
if (m_image_ctx.non_blocking_aio || writes_blocked()) {
- queue(new ImageDiscardRequest<>(m_image_ctx, c, off, len, skip_partial_discard));
+ queue(new ImageDiscardRequest<>(m_image_ctx, c, off, len,
+ skip_partial_discard, trace));
} else {
c->start_op();
- ImageRequest<>::aio_discard(&m_image_ctx, c, off, len, skip_partial_discard);
+ ImageRequest<>::aio_discard(&m_image_ctx, c, off, len,
+ skip_partial_discard, trace);
finish_in_flight_op();
}
+ trace.event("finish");
}
void ImageRequestWQ::aio_flush(AioCompletion *c, bool native_async) {
- c->init_time(&m_image_ctx, AIO_TYPE_FLUSH);
CephContext *cct = m_image_ctx.cct;
+ ZTracer::Trace trace;
+ if (cct->_conf->rbd_blkin_trace_all) {
+ trace.init("wq: flush", &m_image_ctx.trace_endpoint);
+ trace.event("init");
+ }
+
+ c->init_time(&m_image_ctx, AIO_TYPE_FLUSH);
ldout(cct, 20) << "ictx=" << &m_image_ctx << ", "
<< "completion=" << c << dendl;
RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
if (m_image_ctx.non_blocking_aio || writes_blocked() || !writes_empty()) {
- queue(new ImageFlushRequest<>(m_image_ctx, c));
+ queue(new ImageFlushRequest<>(m_image_ctx, c, trace));
} else {
- ImageRequest<>::aio_flush(&m_image_ctx, c);
+ ImageRequest<>::aio_flush(&m_image_ctx, c, trace);
finish_in_flight_op();
}
+ trace.event("finish");
}
void ImageRequestWQ::aio_writesame(AioCompletion *c, uint64_t off, uint64_t len,
bufferlist &&bl, int op_flags,
bool native_async) {
- c->init_time(&m_image_ctx, AIO_TYPE_WRITESAME);
CephContext *cct = m_image_ctx.cct;
+ ZTracer::Trace trace;
+ if (cct->_conf->rbd_blkin_trace_all) {
+ trace.init("wq: writesame", &m_image_ctx.trace_endpoint);
+ trace.event("init");
+ }
+
+ c->init_time(&m_image_ctx, AIO_TYPE_WRITESAME);
ldout(cct, 20) << "ictx=" << &m_image_ctx << ", "
<< "completion=" << c << ", off=" << off << ", "
<< "len=" << len << ", data_len = " << bl.length() << ", "
RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
if (m_image_ctx.non_blocking_aio || writes_blocked()) {
queue(new ImageWriteSameRequest<>(m_image_ctx, c, off, len, std::move(bl),
- op_flags));
+ op_flags, trace));
} else {
c->start_op();
ImageRequest<>::aio_writesame(&m_image_ctx, c, off, len, std::move(bl),
- op_flags);
+ op_flags, trace);
finish_in_flight_op();
}
+ trace.event("finish");
}
void ImageRequestWQ::shut_down(Context *on_shutdown) {
ObjectRequest<I>::create_remove(I *ictx, const std::string &oid,
uint64_t object_no,
const ::SnapContext &snapc,
+ const ZTracer::Trace &parent_trace,
Context *completion) {
return new ObjectRemoveRequest(util::get_image_ctx(ictx), oid, object_no,
- snapc, completion);
+ snapc, parent_trace, completion);
}
template <typename I>
ObjectRequest<I>::create_truncate(I *ictx, const std::string &oid,
uint64_t object_no, uint64_t object_off,
const ::SnapContext &snapc,
- Context *completion) {
+ const ZTracer::Trace &parent_trace,
+ Context *completion) {
return new ObjectTruncateRequest(util::get_image_ctx(ictx), oid, object_no,
- object_off, snapc, completion);
+ object_off, snapc, parent_trace, completion);
}
template <typename I>
ObjectRequest<I>::create_write(I *ictx, const std::string &oid,
uint64_t object_no, uint64_t object_off,
const ceph::bufferlist &data,
- const ::SnapContext &snapc,
- Context *completion, int op_flags) {
+ const ::SnapContext &snapc, int op_flags,
+ const ZTracer::Trace &parent_trace,
+ Context *completion) {
return new ObjectWriteRequest(util::get_image_ctx(ictx), oid, object_no,
- object_off, data, snapc, completion, op_flags);
+ object_off, data, snapc, op_flags, parent_trace,
+ completion);
}
template <typename I>
uint64_t object_no, uint64_t object_off,
uint64_t object_len,
const ::SnapContext &snapc,
+ const ZTracer::Trace &parent_trace,
Context *completion) {
return new ObjectZeroRequest(util::get_image_ctx(ictx), oid, object_no,
- object_off, object_len, snapc, completion);
+ object_off, object_len, snapc, parent_trace,
+ completion);
}
template <typename I>
uint64_t object_no, uint64_t object_off,
uint64_t object_len,
const ceph::bufferlist &data,
- const ::SnapContext &snapc,
- Context *completion, int op_flags) {
+ const ::SnapContext &snapc, int op_flags,
+ const ZTracer::Trace &parent_trace,
+ Context *completion) {
return new ObjectWriteSameRequest(util::get_image_ctx(ictx), oid, object_no,
object_off, object_len, data, snapc,
- completion, op_flags);
+ op_flags, parent_trace, completion);
}
template <typename I>
ObjectRequest<I>::ObjectRequest(ImageCtx *ictx, const std::string &oid,
uint64_t objectno, uint64_t off,
uint64_t len, librados::snap_t snap_id,
- Context *completion, bool hide_enoent)
+ bool hide_enoent, const char *trace_name,
+ const ZTracer::Trace &trace,
+ Context *completion)
: m_ictx(ictx), m_oid(oid), m_object_no(objectno), m_object_off(off),
m_object_len(len), m_snap_id(snap_id), m_completion(completion),
- m_hide_enoent(hide_enoent) {
+ m_hide_enoent(hide_enoent),
+ m_trace(util::create_trace(*ictx, "", trace)) {
+ if (m_trace.valid()) {
+ m_trace.copy_name(trace_name + std::string(" ") + oid);
+ m_trace.event("start");
+ }
Striper::extent_to_file(m_ictx->cct, &m_ictx->layout, m_object_no,
0, m_ictx->layout.object_size, m_parent_extents);
uint64_t objectno, uint64_t offset,
uint64_t len, Extents& be,
librados::snap_t snap_id, bool sparse,
- Context *completion, int op_flags)
+ int op_flags,
+ const ZTracer::Trace &parent_trace,
+ Context *completion)
: ObjectRequest<I>(util::get_image_ctx(ictx), oid, objectno, offset, len,
- snap_id, completion, false),
+ snap_id, false, "read", parent_trace, completion),
m_buffer_extents(be), m_tried_parent(false), m_sparse(sparse),
m_op_flags(op_flags), m_state(LIBRBD_AIO_READ_FLAT) {
guard_read();
librados::AioCompletion *rados_completion =
util::create_rados_callback(this);
- int r = image_ctx->data_ctx.aio_operate(this->m_oid, rados_completion, &op,
- flags, nullptr);
+ int r = image_ctx->data_ctx.aio_operate(
+ this->m_oid, rados_completion, &op, flags, nullptr,
+ (this->m_trace.valid() ? this->m_trace.get_info() : nullptr));
assert(r == 0);
rados_completion->release();
// create and kick off a CopyupRequest
CopyupRequest *new_req = new CopyupRequest(
image_ctx, this->m_oid, this->m_object_no,
- std::move(this->m_parent_extents));
+ std::move(this->m_parent_extents), this->m_trace);
this->m_parent_extents.clear();
image_ctx->copyup_list[this->m_object_no] = new_req;
<< " extents " << parent_extents << dendl;
ImageRequest<>::aio_read(image_ctx->parent, parent_completion,
std::move(parent_extents),
- ReadResult{&m_read_data}, 0);
+ ReadResult{&m_read_data}, 0, this->m_trace);
}
/** write **/
uint64_t object_off,
uint64_t len,
const ::SnapContext &snapc,
- Context *completion,
- bool hide_enoent)
+ bool hide_enoent,
+ const char *trace_name,
+ const ZTracer::Trace &parent_trace,
+ Context *completion)
: ObjectRequest(ictx, oid, object_no, object_off, len, CEPH_NOSNAP,
- completion, hide_enoent),
+ hide_enoent, trace_name, parent_trace, completion),
m_state(LIBRBD_AIO_WRITE_FLAT), m_snap_seq(snapc.seq.val)
{
m_snaps.insert(m_snaps.end(), snapc.snaps.begin(), snapc.snaps.end());
if (it == m_ictx->copyup_list.end()) {
CopyupRequest *new_req = new CopyupRequest(m_ictx, m_oid,
m_object_no,
- std::move(m_parent_extents));
+ std::move(m_parent_extents),
+ this->m_trace);
m_parent_extents.clear();
// make sure to wait on this CopyupRequest
librados::AioCompletion *rados_completion =
util::create_rados_callback(this);
- int r = m_ictx->data_ctx.aio_operate(m_oid, rados_completion, &m_write,
- m_snap_seq, m_snaps);
+ int r = m_ictx->data_ctx.aio_operate(
+ m_oid, rados_completion, &m_write, m_snap_seq, m_snaps,
+ (this->m_trace.valid() ? this->m_trace.get_info() : nullptr));
assert(r == 0);
rados_completion->release();
}
#define CEPH_LIBRBD_IO_OBJECT_REQUEST_H
#include "include/int_types.h"
-
-#include <map>
-
-#include "common/snap_types.h"
#include "include/buffer.h"
#include "include/rados/librados.hpp"
+#include "common/snap_types.h"
+#include "common/zipkin_trace.h"
#include "librbd/ObjectMap.h"
+#include <map>
class Context;
const std::string &oid,
uint64_t object_no,
const ::SnapContext &snapc,
+ const ZTracer::Trace &parent_trace,
Context *completion);
static ObjectRequest* create_truncate(ImageCtxT *ictx,
const std::string &oid,
uint64_t object_no,
uint64_t object_off,
const ::SnapContext &snapc,
+ const ZTracer::Trace &parent_trace,
Context *completion);
static ObjectRequest* create_write(ImageCtxT *ictx, const std::string &oid,
uint64_t object_no,
uint64_t object_off,
const ceph::bufferlist &data,
- const ::SnapContext &snapc,
- Context *completion, int op_flags);
+ const ::SnapContext &snapc, int op_flags,
+ const ZTracer::Trace &parent_trace,
+ Context *completion);
static ObjectRequest* create_zero(ImageCtxT *ictx, const std::string &oid,
uint64_t object_no, uint64_t object_off,
uint64_t object_len,
const ::SnapContext &snapc,
+ const ZTracer::Trace &parent_trace,
Context *completion);
static ObjectRequest* create_writesame(ImageCtxT *ictx,
const std::string &oid,
uint64_t object_len,
const ceph::bufferlist &data,
const ::SnapContext &snapc,
- Context *completion, int op_flags);
+ int op_flags,
+ const ZTracer::Trace &parent_trace,
+ Context *completion);
ObjectRequest(ImageCtx *ictx, const std::string &oid,
uint64_t objectno, uint64_t off, uint64_t len,
- librados::snap_t snap_id,
- Context *completion, bool hide_enoent);
- ~ObjectRequest() override {}
+ librados::snap_t snap_id, bool hide_enoent,
+ const char *trace_name, const ZTracer::Trace &parent_trace,
+ Context *completion);
+ ~ObjectRequest() override {
+ m_trace.event("finish");
+ }
virtual void add_copyup_ops(librados::ObjectWriteOperation *wr,
bool set_hints) {
Context *m_completion;
Extents m_parent_extents;
bool m_hide_enoent;
+ ZTracer::Trace m_trace;
private:
bool m_has_parent = false;
uint64_t objectno, uint64_t offset,
uint64_t len, Extents &buffer_extents,
librados::snap_t snap_id, bool sparse,
- Context *completion, int op_flags) {
+ int op_flags,
+ const ZTracer::Trace &parent_trace,
+ Context *completion) {
return new ObjectReadRequest(ictx, oid, objectno, offset, len,
- buffer_extents, snap_id, sparse, completion,
- op_flags);
+ buffer_extents, snap_id, sparse, op_flags,
+ parent_trace, completion);
}
ObjectReadRequest(ImageCtxT *ictx, const std::string &oid,
uint64_t objectno, uint64_t offset, uint64_t len,
Extents& buffer_extents, librados::snap_t snap_id,
- bool sparse, Context *completion, int op_flags);
+ bool sparse, int op_flags,
+ const ZTracer::Trace &parent_trace, Context *completion);
bool should_complete(int r) override;
void send() override;
AbstractObjectWriteRequest(ImageCtx *ictx, const std::string &oid,
uint64_t object_no, uint64_t object_off,
uint64_t len, const ::SnapContext &snapc,
- Context *completion, bool hide_enoent);
+ bool hide_enoent, const char *trace_name,
+ const ZTracer::Trace &parent_trace,
+ Context *completion);
void add_copyup_ops(librados::ObjectWriteOperation *wr,
bool set_hints) override
public:
ObjectWriteRequest(ImageCtx *ictx, const std::string &oid, uint64_t object_no,
uint64_t object_off, const ceph::bufferlist &data,
- const ::SnapContext &snapc, Context *completion,
- int op_flags)
+ const ::SnapContext &snapc, int op_flags,
+ const ZTracer::Trace &parent_trace, Context *completion)
: AbstractObjectWriteRequest(ictx, oid, object_no, object_off,
- data.length(), snapc, completion, false),
+ data.length(), snapc, false, "write",
+ parent_trace, completion),
m_write_data(data), m_op_flags(op_flags) {
}
public:
ObjectRemoveRequest(ImageCtx *ictx, const std::string &oid,
uint64_t object_no, const ::SnapContext &snapc,
- Context *completion)
- : AbstractObjectWriteRequest(ictx, oid, object_no, 0, 0, snapc, completion,
- true),
+ const ZTracer::Trace &parent_trace, Context *completion)
+ : AbstractObjectWriteRequest(ictx, oid, object_no, 0, 0, snapc, true,
+ "remote", parent_trace, completion),
m_object_state(OBJECT_NONEXISTENT) {
}
// update is needed. pre update is decided as usual (by checking
// the state of the object in the map).
ObjectTrimRequest(ImageCtx *ictx, const std::string &oid, uint64_t object_no,
- const ::SnapContext &snapc, Context *completion,
- bool post_object_map_update)
- : AbstractObjectWriteRequest(ictx, oid, object_no, 0, 0, snapc, completion,
- true),
+ const ::SnapContext &snapc, bool post_object_map_update,
+ Context *completion)
+ : AbstractObjectWriteRequest(ictx, oid, object_no, 0, 0, snapc, true,
+ "trim", {}, completion),
m_post_object_map_update(post_object_map_update) {
}
public:
ObjectTruncateRequest(ImageCtx *ictx, const std::string &oid,
uint64_t object_no, uint64_t object_off,
- const ::SnapContext &snapc, Context *completion)
+ const ::SnapContext &snapc,
+ const ZTracer::Trace &parent_trace, Context *completion)
: AbstractObjectWriteRequest(ictx, oid, object_no, object_off, 0, snapc,
- completion, true) {
+ true, "truncate", parent_trace, completion) {
}
const char* get_op_type() const override {
public:
ObjectZeroRequest(ImageCtx *ictx, const std::string &oid, uint64_t object_no,
uint64_t object_off, uint64_t object_len,
- const ::SnapContext &snapc, Context *completion)
+ const ::SnapContext &snapc,
+ const ZTracer::Trace &parent_trace, Context *completion)
: AbstractObjectWriteRequest(ictx, oid, object_no, object_off, object_len,
- snapc, completion, true) {
+ snapc, true, "zero", parent_trace,
+ completion) {
}
const char* get_op_type() const override {
class ObjectWriteSameRequest : public AbstractObjectWriteRequest {
public:
- ObjectWriteSameRequest(ImageCtx *ictx, const std::string &oid, uint64_t object_no,
- uint64_t object_off, uint64_t object_len,
- const ceph::bufferlist &data,
- const ::SnapContext &snapc, Context *completion,
- int op_flags)
+ ObjectWriteSameRequest(ImageCtx *ictx, const std::string &oid,
+ uint64_t object_no, uint64_t object_off,
+ uint64_t object_len, const ceph::bufferlist &data,
+ const ::SnapContext &snapc, int op_flags,
+ const ZTracer::Trace &parent_trace,
+ Context *completion)
: AbstractObjectWriteRequest(ictx, oid, object_no, object_off,
- object_len, snapc, completion, false),
+ object_len, snapc, false, "writesame",
+ parent_trace, completion),
m_write_data(data), m_op_flags(op_flags) {
}
// execute the following outside of lock scope
if (flush_comp != nullptr) {
RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
- io::ImageRequest<I>::aio_flush(&m_image_ctx, flush_comp);
+ io::ImageRequest<I>::aio_flush(&m_image_ctx, flush_comp, {});
}
if (on_finish != nullptr) {
on_finish->complete(0);
}
RWLock::RLocker owner_locker(m_image_ctx.owner_lock);
- io::ImageRequest<I>::aio_flush(&m_image_ctx, aio_comp);
+ io::ImageRequest<I>::aio_flush(&m_image_ctx, aio_comp, {});
}
template <typename I>
io::AIO_TYPE_DISCARD,
&flush_required);
io::ImageRequest<I>::aio_discard(&m_image_ctx, aio_comp, event.offset,
- event.length, event.skip_partial_discard);
+ event.length, event.skip_partial_discard,
+ {});
if (flush_required) {
m_lock.Lock();
auto flush_comp = create_aio_flush_completion(nullptr);
m_lock.Unlock();
- io::ImageRequest<I>::aio_flush(&m_image_ctx, flush_comp);
+ io::ImageRequest<I>::aio_flush(&m_image_ctx, flush_comp, {});
}
}
&flush_required);
io::ImageRequest<I>::aio_write(&m_image_ctx, aio_comp,
{{event.offset, event.length}},
- std::move(data), 0);
+ std::move(data), 0, {});
if (flush_required) {
m_lock.Lock();
auto flush_comp = create_aio_flush_completion(nullptr);
m_lock.Unlock();
- io::ImageRequest<I>::aio_flush(&m_image_ctx, flush_comp);
+ io::ImageRequest<I>::aio_flush(&m_image_ctx, flush_comp, {});
}
}
Mutex::Locker locker(m_lock);
aio_comp = create_aio_flush_completion(on_safe);
}
- io::ImageRequest<I>::aio_flush(&m_image_ctx, aio_comp);
+ io::ImageRequest<I>::aio_flush(&m_image_ctx, aio_comp, {});
on_ready->complete(0);
}
io::AIO_TYPE_WRITESAME,
&flush_required);
io::ImageRequest<I>::aio_writesame(&m_image_ctx, aio_comp, event.offset,
- event.length, std::move(data), 0);
+ event.length, std::move(data), 0, {});
if (flush_required) {
m_lock.Lock();
auto flush_comp = create_aio_flush_completion(nullptr);
m_lock.Unlock();
- io::ImageRequest<I>::aio_flush(&m_image_ctx, flush_comp);
+ io::ImageRequest<I>::aio_flush(&m_image_ctx, flush_comp, {});
}
}
bufferlist bl;
string oid = image_ctx.get_object_name(m_object_no);
auto req = new io::ObjectWriteRequest(&image_ctx, oid, m_object_no, 0,
- bl, m_snapc, this, 0);
+ bl, m_snapc, 0, {}, this);
if (!req->has_parent()) {
// stop early if the parent went away - it just means
// another flatten finished first or the image was resized
ldout(image_ctx.cct, 10) << "removing (with copyup) " << oid << dendl;
auto req = new io::ObjectTrimRequest(&image_ctx, oid, m_object_no,
- m_snapc, this, false);
+ m_snapc, false, this);
req->send();
return 0;
}
io::ObjectRequest<> *req;
if (p->offset == 0) {
req = new io::ObjectTrimRequest(&image_ctx, p->oid.name, p->objectno,
- snapc, req_comp, true);
+ snapc, true, req_comp);
} else {
req = new io::ObjectTruncateRequest(&image_ctx, p->oid.name, p->objectno,
- p->offset, snapc, req_comp);
+ p->offset, snapc, {}, req_comp);
}
req->send();
}
const std::string &oid,
uint64_t object_no,
const ::SnapContext &snapc,
+ const ZTracer::Trace &parent_trace,
Context *completion) {
assert(s_instance != nullptr);
s_instance->on_finish = completion;
uint64_t object_no,
uint64_t object_off,
const ::SnapContext &snapc,
+ const ZTracer::Trace &parent_trace,
Context *completion) {
assert(s_instance != nullptr);
s_instance->on_finish = completion;
uint64_t object_no,
uint64_t object_off,
const ceph::bufferlist &data,
- const ::SnapContext &snapc,
- Context *completion, int op_flags) {
+ const ::SnapContext &snapc, int op_flags,
+ const ZTracer::Trace &parent_trace,
+ Context *completion) {
assert(s_instance != nullptr);
s_instance->on_finish = completion;
return s_instance;
uint64_t object_no, uint64_t object_off,
uint64_t object_len,
const ::SnapContext &snapc,
+ const ZTracer::Trace &parent_trace,
Context *completion) {
assert(s_instance != nullptr);
s_instance->on_finish = completion;
uint64_t object_len,
const ceph::bufferlist &data,
const ::SnapContext &snapc,
- Context *completion, int op_flags) {
+ int op_flags,
+ const ZTracer::Trace &parent_trace,
+ Context *completion) {
assert(s_instance != nullptr);
s_instance->on_finish = completion;
return s_instance;
uint64_t objectno, uint64_t offset,
uint64_t len, Extents &buffer_extents,
librados::snap_t snap_id, bool sparse,
- Context *completion, int op_flags) {
+ int op_flags,
+ const ZTracer::Trace &parent_trace,
+ Context *completion) {
assert(s_instance != nullptr);
s_instance->on_finish = completion;
return s_instance;
bufferlist bl;
bl.append("1");
MockImageWriteRequest mock_aio_image_write(mock_image_ctx, aio_comp,
- {{0, 1}}, std::move(bl), 0);
+ {{0, 1}}, std::move(bl), 0, {});
{
RWLock::RLocker owner_locker(mock_image_ctx.owner_lock);
mock_aio_image_write.send();
AioCompletion *aio_comp = AioCompletion::create_and_start(
&aio_comp_ctx, ictx, AIO_TYPE_DISCARD);
MockImageDiscardRequest mock_aio_image_discard(mock_image_ctx, aio_comp,
- 0, 1, ictx->skip_partial_discard);
+ 0, 1,
+ ictx->skip_partial_discard,
+ {});
{
RWLock::RLocker owner_locker(mock_image_ctx.owner_lock);
mock_aio_image_discard.send();
C_SaferCond aio_comp_ctx;
AioCompletion *aio_comp = AioCompletion::create_and_start(
&aio_comp_ctx, ictx, AIO_TYPE_FLUSH);
- MockImageFlushRequest mock_aio_image_flush(mock_image_ctx, aio_comp);
+ MockImageFlushRequest mock_aio_image_flush(mock_image_ctx, aio_comp, {});
{
RWLock::RLocker owner_locker(mock_image_ctx.owner_lock);
mock_aio_image_flush.send();
bufferlist bl;
bl.append("1");
MockImageWriteSameRequest mock_aio_image_writesame(mock_image_ctx, aio_comp,
- 0, 1, std::move(bl), 0);
+ 0, 1, std::move(bl), 0,
+ {});
{
RWLock::RLocker owner_locker(mock_image_ctx.owner_lock);
mock_aio_image_writesame.send();
const bufferlist &bl, int op_flags));
static void aio_write(MockReplayImageCtx *ictx, AioCompletion *c,
Extents &&image_extents, bufferlist &&bl,
- int op_flags) {
+ int op_flags, const ZTracer::Trace &parent_trace) {
assert(s_instance != nullptr);
s_instance->aio_write(c, image_extents, bl, op_flags);
}
MOCK_METHOD4(aio_discard, void(AioCompletion *c, uint64_t off, uint64_t len,
bool skip_partial_discard));
static void aio_discard(MockReplayImageCtx *ictx, AioCompletion *c,
- uint64_t off, uint64_t len, bool skip_partial_discard) {
+ uint64_t off, uint64_t len,
+ bool skip_partial_discard,
+ const ZTracer::Trace &parent_trace) {
assert(s_instance != nullptr);
s_instance->aio_discard(c, off, len, skip_partial_discard);
}
MOCK_METHOD1(aio_flush, void(AioCompletion *c));
- static void aio_flush(MockReplayImageCtx *ictx, AioCompletion *c) {
+ static void aio_flush(MockReplayImageCtx *ictx, AioCompletion *c,
+ const ZTracer::Trace &parent_trace) {
assert(s_instance != nullptr);
s_instance->aio_flush(c);
}
const bufferlist &bl, int op_flags));
static void aio_writesame(MockReplayImageCtx *ictx, AioCompletion *c,
uint64_t off, uint64_t len, bufferlist &&bl,
- int op_flags) {
+ int op_flags, const ZTracer::Trace &parent_trace) {
assert(s_instance != nullptr);
s_instance->aio_writesame(c, off, len, bl, op_flags);
}
state(new MockImageState()),
image_watcher(NULL), object_map(NULL),
exclusive_lock(NULL), journal(NULL),
+ trace_endpoint(image_ctx.trace_endpoint),
concurrent_management_ops(image_ctx.concurrent_management_ops),
blacklist_on_break_lock(image_ctx.blacklist_on_break_lock),
blacklist_expire_seconds(image_ctx.blacklist_expire_seconds),
MockExclusiveLock *exclusive_lock;
MockJournal *journal;
+ ZTracer::Endpoint trace_endpoint;
+
int concurrent_management_ops;
bool blacklist_on_break_lock;
uint32_t blacklist_expire_seconds;
C_SaferCond object_request_ctx;
auto object_request = new io::ObjectRemoveRequest(
- ictx, "oid", 0, {}, &object_request_ctx);
+ ictx, "oid", 0, {}, {}, &object_request_ctx);
::journal::MockFuture mock_future;
Context *on_journal_safe;
C_SaferCond object_request_ctx;
auto object_request = new io::ObjectRemoveRequest(
- ictx, "oid", 0, {}, &object_request_ctx);
+ ictx, "oid", 0, {}, {}, &object_request_ctx);
::journal::MockFuture mock_future;
Context *on_journal_safe;