}
void Client::dump_mds_requests(Formatter *f)
{
- for (map<tid_t, MetaRequest*>::iterator p = mds_requests.begin();
+ for (map<ceph_tid_t, MetaRequest*>::iterator p = mds_requests.begin();
p != mds_requests.end();
++p) {
f->open_object_section("request");
int r = 0;
// assign a unique tid
- tid_t tid = ++last_tid;
+ ceph_tid_t tid = ++last_tid;
request->set_tid(tid);
// make note
mds_requests[tid] = request->get();
fwd->put();
return;
}
- tid_t tid = fwd->get_tid();
+ ceph_tid_t tid = fwd->get_tid();
if (mds_requests.count(tid) == 0) {
ldout(cct, 10) << "handle_client_request_forward no pending request on tid " << tid << dendl;
return;
}
- tid_t tid = reply->get_tid();
+ ceph_tid_t tid = reply->get_tid();
bool is_safe = reply->is_safe();
if (mds_requests.count(tid) == 0) {
void Client::kick_requests(MetaSession *session)
{
ldout(cct, 10) << "kick_requests for mds." << session->mds_num << dendl;
- for (map<tid_t, MetaRequest*>::iterator p = mds_requests.begin();
+ for (map<ceph_tid_t, MetaRequest*>::iterator p = mds_requests.begin();
p != mds_requests.end();
++p) {
if (p->second->mds == session->mds_num) {
void Client::kick_requests_closed(MetaSession *session)
{
ldout(cct, 10) << "kick_requests_closed for mds." << session->mds_num << dendl;
- for (map<tid_t, MetaRequest*>::iterator p = mds_requests.begin();
+ for (map<ceph_tid_t, MetaRequest*>::iterator p = mds_requests.begin();
p != mds_requests.end();
++p) {
if (p->second->mds == session->mds_num) {
int r = 0;
Inode *in = f->inode;
- tid_t wait_on_flush = 0;
+ ceph_tid_t wait_on_flush = 0;
bool flushed_metadata = false;
Mutex lock("Client::_fsync::lock");
Cond cond;
void resend_unsafe_requests(MetaSession *s);
// mds requests
- tid_t last_tid, last_flush_seq;
- map<tid_t, MetaRequest*> mds_requests;
+ ceph_tid_t last_tid, last_flush_seq;
+ map<ceph_tid_t, MetaRequest*> mds_requests;
set<int> failed_mds;
void dump_mds_requests(Formatter *f);
int snap_caps, snap_cap_refs;
utime_t hold_caps_until;
xlist<Inode*>::item cap_item, flushing_cap_item;
- tid_t last_flush_tid;
+ ceph_tid_t last_flush_tid;
SnapRealm *snaprealm;
xlist<Inode*>::item snaprealm_item;
}
// normal fields
- void set_tid(tid_t t) { tid = t; }
- void set_oldest_client_tid(tid_t t) { head.oldest_client_tid = t; }
+ void set_tid(ceph_tid_t t) { tid = t; }
+ void set_oldest_client_tid(ceph_tid_t t) { head.oldest_client_tid = t; }
void inc_num_fwd() { head.num_fwd = head.num_fwd + 1; }
void set_retry_attempt(int a) { head.num_retry = a; }
void set_filepath(const filepath& fp) { path = fp; }
head.flags = head.flags | CEPH_MDS_FLAG_WANT_DENTRY;
}
int get_op() { return head.op; }
- tid_t get_tid() { return tid; }
+ ceph_tid_t get_tid() { return tid; }
filepath& get_filepath() { return path; }
filepath& get_filepath2() { return path2; }
return false;
}
- virtual tid_t write(const object_t& oid, const object_locator_t& oloc,
+ virtual ceph_tid_t write(const object_t& oid, const object_locator_t& oloc,
uint64_t off, uint64_t len, const SnapContext& snapc,
const bufferlist &bl, utime_t mtime, uint64_t trunc_size,
__u32 trunc_seq, Context *oncommit) {
trunc_size, trunc_seq, NULL, oncommit);
}
- virtual tid_t lock(const object_t& oid, const object_locator_t& oloc, int op,
+ virtual ceph_tid_t lock(const object_t& oid, const object_locator_t& oloc, int op,
int flags, Context *onack, Context *oncommit) {
return m_objecter->lock(oid, oloc, op, flags, onack, oncommit);
}
// some basic types
// NOTE: these must match ceph_fs.h typedefs
-typedef uint64_t tid_t; // transaction id
+typedef uint64_t ceph_tid_t; // transaction id
typedef uint64_t version_t;
typedef __u32 epoch_t; // map epoch (32bits -> 13 epochs/second for 10 years)
unsigned maxlen;
IoCtxImpl *io;
- tid_t aio_write_seq;
+ ceph_tid_t aio_write_seq;
xlist<AioCompletionImpl*>::item aio_write_list_item;
AioCompletionImpl() : lock("AioCompletionImpl lock", false, false),
assert(c->io == this);
c->aio_write_list_item.remove_myself();
- map<tid_t, std::list<AioCompletionImpl*> >::iterator waiters = aio_write_waiters.begin();
+ map<ceph_tid_t, std::list<AioCompletionImpl*> >::iterator waiters = aio_write_waiters.begin();
while (waiters != aio_write_waiters.end()) {
if (!aio_write_list.empty() &&
aio_write_list.front()->aio_write_seq <= waiters->first) {
ldout(client->cct, 20) << "flush_aio_writes_async " << this
<< " completion " << c << dendl;
Mutex::Locker l(aio_write_list_lock);
- tid_t seq = aio_write_seq;
+ ceph_tid_t seq = aio_write_seq;
if (aio_write_list.empty()) {
ldout(client->cct, 20) << "flush_aio_writes_async no writes. (tid "
<< seq << ")" << dendl;
{
ldout(client->cct, 20) << "flush_aio_writes" << dendl;
aio_write_list_lock.Lock();
- tid_t seq = aio_write_seq;
+ ceph_tid_t seq = aio_write_seq;
while (!aio_write_list.empty() &&
aio_write_list.front()->aio_write_seq <= seq)
aio_write_cond.Wait(aio_write_list_lock);
object_locator_t oloc;
Mutex aio_write_list_lock;
- tid_t aio_write_seq;
+ ceph_tid_t aio_write_seq;
Cond aio_write_cond;
xlist<AioCompletionImpl*> aio_write_list;
- map<tid_t, std::list<AioCompletionImpl*> > aio_write_waiters;
+ map<ceph_tid_t, std::list<AioCompletionImpl*> > aio_write_waiters;
Mutex *lock;
Objecter *objecter;
Cond cond;
bool done;
int ret;
- tid_t tid;
+ ceph_tid_t tid;
if (osd < 0)
return -EINVAL;
Cond cond;
bool done;
int ret;
- tid_t tid;
+ ceph_tid_t tid;
lock.Lock();
int r = objecter->pg_command(pgid, cmd, inbl, &tid, poutbl, prs,
new C_SafeCond(&mylock, &cond, &done, &ret));
return may;
}
- tid_t LibrbdWriteback::write(const object_t& oid,
+ ceph_tid_t LibrbdWriteback::write(const object_t& oid,
const object_locator_t& oloc,
uint64_t off, uint64_t len,
const SnapContext& snapc,
virtual bool may_copy_on_write(const object_t& oid, uint64_t read_off, uint64_t read_len, snapid_t snapid);
// Note that oloc, trunc_size, and trunc_seq are ignored
- virtual tid_t write(const object_t& oid, const object_locator_t& oloc,
+ virtual ceph_tid_t write(const object_t& oid, const object_locator_t& oloc,
uint64_t off, uint64_t len, const SnapContext& snapc,
const bufferlist &bl, utime_t mtime, uint64_t trunc_size,
__u32 trunc_seq, Context *oncommit);
private:
void complete_writes(const std::string& oid);
- tid_t m_tid;
+ ceph_tid_t m_tid;
Mutex& m_lock;
librbd::ImageCtx *m_ictx;
ceph::unordered_map<std::string, std::queue<write_result_d*> > m_writes;
set<dirfrag_t> uncommitted_fragments;
// client request ids
- map<int, tid_t> last_client_tids;
+ map<int, ceph_tid_t> last_client_tids;
// table version
version_t inotablev;
dout(5) << "find_ino_peers " << ino << " hint " << hint << dendl;
assert(!have_inode(ino));
- tid_t tid = ++find_ino_peer_last_tid;
+ ceph_tid_t tid = ++find_ino_peer_last_tid;
find_ino_peer_info_t& fip = find_ino_peer[tid];
fip.ino = ino;
fip.tid = tid;
void MDCache::handle_find_ino_reply(MMDSFindInoReply *m)
{
- map<tid_t, find_ino_peer_info_t>::iterator p = find_ino_peer.find(m->tid);
+ map<ceph_tid_t, find_ino_peer_info_t>::iterator p = find_ino_peer.find(m->tid);
if (p != find_ino_peer.end()) {
dout(10) << "handle_find_ino_reply " << *m << dendl;
find_ino_peer_info_t& fip = p->second;
void MDCache::kick_find_ino_peers(int who)
{
// find_ino_peers requests we should move on from
- for (map<tid_t,find_ino_peer_info_t>::iterator p = find_ino_peer.begin();
+ for (map<ceph_tid_t,find_ino_peer_info_t>::iterator p = find_ino_peer.begin();
p != find_ino_peer.end();
++p) {
find_ino_peer_info_t& fip = p->second;
void MDCache::kick_discovers(int who)
{
- for (map<tid_t,discover_info_t>::iterator p = discovers.begin();
+ for (map<ceph_tid_t,discover_info_t>::iterator p = discovers.begin();
p != discovers.end();
++p) {
if (p->second.mds != who)
// decrement discover counters
if (m->get_tid()) {
- map<tid_t,discover_info_t>::iterator p = discovers.find(m->get_tid());
+ map<ceph_tid_t,discover_info_t>::iterator p = discovers.find(m->get_tid());
if (p != discovers.end()) {
dout(10) << " found tid " << m->get_tid() << dendl;
discovers.erase(p);
// -- discover --
struct discover_info_t {
- tid_t tid;
+ ceph_tid_t tid;
int mds;
inodeno_t ino;
frag_t frag;
discover_info_t() : tid(0), mds(-1), snap(CEPH_NOSNAP), want_base_dir(false), want_xlocked(false) {}
};
- map<tid_t, discover_info_t> discovers;
- tid_t discover_last_tid;
+ map<ceph_tid_t, discover_info_t> discovers;
+ ceph_tid_t discover_last_tid;
void _send_discover(discover_info_t& dis);
discover_info_t& _create_discover(int mds) {
- tid_t t = ++discover_last_tid;
+ ceph_tid_t t = ++discover_last_tid;
discover_info_t& d = discovers[t];
d.tid = t;
d.mds = mds;
open_ino_info_t() : checking(-1), auth_hint(-1),
check_peers(true), fetch_backtrace(true), discover(false) {}
};
- tid_t open_ino_last_tid;
+ ceph_tid_t open_ino_last_tid;
map<inodeno_t,open_ino_info_t> opening_inodes;
void _open_ino_backtrace_fetched(inodeno_t ino, bufferlist& bl, int err);
// -- find_ino_peer --
struct find_ino_peer_info_t {
inodeno_t ino;
- tid_t tid;
+ ceph_tid_t tid;
Context *fin;
int hint;
int checking;
find_ino_peer_info_t() : tid(0), fin(NULL), hint(-1), checking(-1) {}
};
- map<tid_t, find_ino_peer_info_t> find_ino_peer;
- tid_t find_ino_peer_last_tid;
+ map<ceph_tid_t, find_ino_peer_info_t> find_ino_peer;
+ ceph_tid_t find_ino_peer_last_tid;
void find_ino_peers(inodeno_t ino, Context *c, int hint=-1);
void _do_find_ino_peer(find_ino_peer_info_t& fip);
map<int,version_t> peer_mdsmap_epoch;
- tid_t last_tid; // for mds-initiated requests (e.g. stray rename)
+ ceph_tid_t last_tid; // for mds-initiated requests (e.g. stray rename)
public:
void wait_for_active(Context *c) {
void request_state(int s);
- tid_t issue_tid() { return ++last_tid; }
+ ceph_tid_t issue_tid() { return ++last_tid; }
// -- waiters --
public:
- void add_completed_request(tid_t t, inodeno_t created) {
+ void add_completed_request(ceph_tid_t t, inodeno_t created) {
info.completed_requests[t] = created;
}
- void trim_completed_requests(tid_t mintid) {
+ void trim_completed_requests(ceph_tid_t mintid) {
// trim
while (!info.completed_requests.empty() &&
(mintid == 0 || info.completed_requests.begin()->first < mintid))
info.completed_requests.erase(info.completed_requests.begin());
}
- bool have_completed_request(tid_t tid, inodeno_t *pcreated) const {
- map<tid_t,inodeno_t>::const_iterator p = info.completed_requests.find(tid);
+ bool have_completed_request(ceph_tid_t tid, inodeno_t *pcreated) const {
+ map<ceph_tid_t,inodeno_t>::const_iterator p = info.completed_requests.find(tid);
if (p == info.completed_requests.end())
return false;
if (pcreated)
Session *session = get_session(rid.name);
return session && session->have_completed_request(rid.tid, NULL);
}
- void trim_completed_requests(entity_name_t c, tid_t tid) {
+ void trim_completed_requests(entity_name_t c, ceph_tid_t tid) {
Session *session = get_session(c);
assert(session);
session->trim_completed_requests(tid);
DECODE_START_LEGACY_COMPAT_LEN(3, 2, 2, p);
::decode(inst, p);
if (struct_v <= 2) {
- set<tid_t> s;
+ set<ceph_tid_t> s;
::decode(s, p);
while (!s.empty()) {
completed_requests[*s.begin()] = inodeno_t();
f->dump_stream("inst") << inst;
f->open_array_section("completed_requests");
- for (map<tid_t,inodeno_t>::const_iterator p = completed_requests.begin();
+ for (map<ceph_tid_t,inodeno_t>::const_iterator p = completed_requests.begin();
p != completed_requests.end();
++p) {
f->open_object_section("request");
struct session_info_t {
entity_inst_t inst;
- map<tid_t,inodeno_t> completed_requests;
+ map<ceph_tid_t,inodeno_t> completed_requests;
interval_set<inodeno_t> prealloc_inos; // preallocated, ready to use.
interval_set<inodeno_t> used_inos; // journaling use
entity_name_t name;
uint64_t tid;
metareqid_t() : tid(0) {}
- metareqid_t(entity_name_t n, tid_t t) : name(n), tid(t) {}
+ metareqid_t(entity_name_t n, ceph_tid_t t) : name(n), tid(t) {}
void encode(bufferlist& bl) const {
::encode(name, bl);
::encode(tid, bl);
}
// normal fields
- void set_oldest_client_tid(tid_t t) { head.oldest_client_tid = t; }
+ void set_oldest_client_tid(ceph_tid_t t) { head.oldest_client_tid = t; }
void inc_num_fwd() { head.num_fwd = head.num_fwd + 1; }
void set_retry_attempt(int a) { head.num_retry = a; }
void set_filepath(const filepath& fp) { path = fp; }
head.flags = head.flags | CEPH_MDS_FLAG_REPLAY;
}
- tid_t get_oldest_client_tid() const { return head.oldest_client_tid; }
+ ceph_tid_t get_oldest_client_tid() const { return head.oldest_client_tid; }
int get_num_fwd() const { return head.num_fwd; }
int get_retry_attempt() const { return head.num_retry; }
int get_op() const { return head.op; }
MClientRequestForward()
: Message(CEPH_MSG_CLIENT_REQUEST_FORWARD),
dest_mds(-1), num_fwd(-1), client_must_resend(false) {}
- MClientRequestForward(tid_t t, int dm, int nf, bool cmr) :
+ MClientRequestForward(ceph_tid_t t, int dm, int nf, bool cmr) :
Message(CEPH_MSG_CLIENT_REQUEST_FORWARD),
dest_mds(dm), num_fwd(nf), client_must_resend(cmr) {
assert(client_must_resend);
list<string> pools;
MGetPoolStats() : PaxosServiceMessage(MSG_GETPOOLSTATS, 0) {}
- MGetPoolStats(const uuid_d& f, tid_t t, list<string>& ls, version_t l) :
+ MGetPoolStats(const uuid_d& f, ceph_tid_t t, list<string>& ls, version_t l) :
PaxosServiceMessage(MSG_GETPOOLSTATS, l),
fsid(f), pools(ls) {
set_tid(t);
map<string,pool_stat_t> pool_stats;
MGetPoolStatsReply() : PaxosServiceMessage(MSG_GETPOOLSTATSREPLY, 0) {}
- MGetPoolStatsReply(uuid_d& f, tid_t t, version_t v) :
+ MGetPoolStatsReply(uuid_d& f, ceph_tid_t t, version_t v) :
PaxosServiceMessage(MSG_GETPOOLSTATSREPLY, v),
fsid(f) {
set_tid(t);
#include "include/filepath.h"
struct MMDSFindIno : public Message {
- tid_t tid;
+ ceph_tid_t tid;
inodeno_t ino;
MMDSFindIno() : Message(MSG_MDS_FINDINO) {}
- MMDSFindIno(tid_t t, inodeno_t i) : Message(MSG_MDS_FINDINO), tid(t), ino(i) {}
+ MMDSFindIno(ceph_tid_t t, inodeno_t i) : Message(MSG_MDS_FINDINO), tid(t), ino(i) {}
const char *get_type_name() const { return "findino"; }
void print(ostream &out) const {
#include "include/filepath.h"
struct MMDSFindInoReply : public Message {
- tid_t tid;
+ ceph_tid_t tid;
filepath path;
MMDSFindInoReply() : Message(MSG_MDS_FINDINOREPLY) {}
- MMDSFindInoReply(tid_t t) : Message(MSG_MDS_FINDINOREPLY), tid(t) {}
+ MMDSFindInoReply(ceph_tid_t t) : Message(MSG_MDS_FINDINOREPLY), tid(t) {}
const char *get_type_name() const { return "findinoreply"; }
void print(ostream &out) const {
vector<inode_backpointer_t> ancestors;
MMDSOpenIno() : Message(MSG_MDS_OPENINO) {}
- MMDSOpenIno(tid_t t, inodeno_t i, vector<inode_backpointer_t>& a) :
+ MMDSOpenIno(ceph_tid_t t, inodeno_t i, vector<inode_backpointer_t>& a) :
Message(MSG_MDS_OPENINO), ino(i), ancestors(a) {
header.tid = t;
}
int32_t error;
MMDSOpenInoReply() : Message(MSG_MDS_OPENINOREPLY) {}
- MMDSOpenInoReply(tid_t t, inodeno_t i, int h=-1, int e=0) :
+ MMDSOpenInoReply(ceph_tid_t t, inodeno_t i, int h=-1, int e=0) :
Message(MSG_MDS_OPENINOREPLY), ino(i), hint(h), error(e) {
header.tid = t;
}
::decode(what, p);
}
- tid_t handle;
+ ceph_tid_t handle;
string what;
private:
::decode(oldest_version, p);
}
- tid_t handle;
+ ceph_tid_t handle;
version_t version;
version_t oldest_version;
header.tid);
}
int get_client_inc() { return client_inc; }
- tid_t get_client_tid() { return header.tid; }
+ ceph_tid_t get_client_tid() { return header.tid; }
object_t& get_oid() { return oid; }
: Message(MSG_OSD_SUBOP, HEAD_VERSION, COMPAT_VERSION) { }
MOSDSubOp(osd_reqid_t r, pg_shard_t from,
spg_t p, const hobject_t& po, bool noop_, int aw,
- epoch_t mape, tid_t rtid, eversion_t v)
+ epoch_t mape, ceph_tid_t rtid, eversion_t v)
: Message(MSG_OSD_SUBOP, HEAD_VERSION, COMPAT_VERSION),
map_epoch(mape),
reqid(r),
MPoolOp()
: PaxosServiceMessage(CEPH_MSG_POOLOP, 0, HEAD_VERSION, COMPAT_VERSION) { }
- MPoolOp(const uuid_d& f, tid_t t, int p, string& n, int o, version_t v)
+ MPoolOp(const uuid_d& f, ceph_tid_t t, int p, string& n, int o, version_t v)
: PaxosServiceMessage(CEPH_MSG_POOLOP, v, HEAD_VERSION, COMPAT_VERSION),
fsid(f), pool(p), name(n), op(o),
auid(0), snapid(0), crush_rule(0) {
set_tid(t);
}
- MPoolOp(const uuid_d& f, tid_t t, int p, string& n,
+ MPoolOp(const uuid_d& f, ceph_tid_t t, int p, string& n,
int o, uint64_t uid, version_t v)
: PaxosServiceMessage(CEPH_MSG_POOLOP, v, HEAD_VERSION, COMPAT_VERSION),
fsid(f), pool(p), name(n), op(o),
MPoolOpReply() : PaxosServiceMessage(CEPH_MSG_POOLOP_REPLY, 0)
{}
- MPoolOpReply( uuid_d& f, tid_t t, int rc, int e, version_t v) :
+ MPoolOpReply( uuid_d& f, ceph_tid_t t, int rc, int e, version_t v) :
PaxosServiceMessage(CEPH_MSG_POOLOP_REPLY, v),
fsid(f),
replyCode(rc),
epoch(e) {
set_tid(t);
}
- MPoolOpReply( uuid_d& f, tid_t t, int rc, int e, version_t v,
+ MPoolOpReply( uuid_d& f, ceph_tid_t t, int rc, int e, version_t v,
bufferlist *blp) :
PaxosServiceMessage(CEPH_MSG_POOLOP_REPLY, v),
fsid(f),
uuid_d fsid;
MStatfs() : PaxosServiceMessage(CEPH_MSG_STATFS, 0) {}
- MStatfs(const uuid_d& f, tid_t t, version_t v) :
+ MStatfs(const uuid_d& f, ceph_tid_t t, version_t v) :
PaxosServiceMessage(CEPH_MSG_STATFS, v), fsid(f) {
set_tid(t);
}
struct ceph_mon_statfs_reply h;
MStatfsReply() : Message(CEPH_MSG_STATFS_REPLY) {}
- MStatfsReply(uuid_d &f, tid_t t, epoch_t epoch) : Message(CEPH_MSG_STATFS_REPLY) {
+ MStatfsReply(uuid_d &f, ceph_tid_t t, epoch_t epoch) : Message(CEPH_MSG_STATFS_REPLY) {
memcpy(&h.fsid, f.uuid, sizeof(h.fsid));
header.tid = t;
h.version = epoch;
{
assert(monc_lock.is_locked());
- map<tid_t, MonCommand*>::iterator it = mon_commands.find(tid);
+ map<ceph_tid_t, MonCommand*>::iterator it = mon_commands.find(tid);
if (it == mon_commands.end()) {
ldout(cct, 10) << __func__ << " tid " << tid << " dne" << dendl;
return -ENOENT;
void MonClient::handle_get_version_reply(MMonGetVersionReply* m)
{
assert(monc_lock.is_locked());
- map<tid_t, version_req_d*>::iterator iter = version_requests.find(m->handle);
+ map<ceph_tid_t, version_req_d*>::iterator iter = version_requests.find(m->handle);
if (iter == version_requests.end()) {
ldout(cct, 0) << __func__ << " version request with handle " << m->handle
<< " not found" << dendl;
version_req_d(Context *con, version_t *n, version_t *o) : context(con),newest(n), oldest(o) {}
};
- map<tid_t, version_req_d*> version_requests;
- tid_t version_req_id;
+ map<ceph_tid_t, version_req_d*> version_requests;
+ ceph_tid_t version_req_id;
void handle_get_version_reply(MMonGetVersionReply* m);
bool failed; /// true if we are a lossy connection that has failed.
int rx_buffers_version;
- map<tid_t,pair<bufferlist,int> > rx_buffers;
+ map<ceph_tid_t,pair<bufferlist,int> > rx_buffers;
friend class boost::intrusive_ptr<Connection>;
void set_features(uint64_t f) { features = f; }
void set_feature(uint64_t f) { features |= f; }
- void post_rx_buffer(tid_t tid, bufferlist& bl) {
+ void post_rx_buffer(ceph_tid_t tid, bufferlist& bl) {
Mutex::Locker l(lock);
++rx_buffers_version;
rx_buffers[tid] = pair<bufferlist,int>(bl, rx_buffers_version);
}
- void revoke_rx_buffer(tid_t tid) {
+ void revoke_rx_buffer(ceph_tid_t tid) {
Mutex::Locker l(lock);
rx_buffers.erase(tid);
}
// get a buffer
connection_state->lock.Lock();
- map<tid_t,pair<bufferlist,int> >::iterator p = connection_state->rx_buffers.find(header.tid);
+ map<ceph_tid_t,pair<bufferlist,int> >::iterator p = connection_state->rx_buffers.find(header.tid);
if (p != connection_state->rx_buffers.end()) {
if (rxbuf.length() == 0 || p->second.second != rxbuf_version) {
ldout(msgr->cct,10) << "reader seleting rx buffer v " << p->second.second
struct SubWriteCommitted : public Context {
ECBackend *pg;
OpRequestRef msg;
- tid_t tid;
+ ceph_tid_t tid;
eversion_t version;
eversion_t last_complete;
SubWriteCommitted(
ECBackend *pg,
OpRequestRef msg,
- tid_t tid,
+ ceph_tid_t tid,
eversion_t version,
eversion_t last_complete)
: pg(pg), msg(msg), tid(tid),
}
};
void ECBackend::sub_write_committed(
- tid_t tid, eversion_t version, eversion_t last_complete) {
+ ceph_tid_t tid, eversion_t version, eversion_t last_complete) {
if (get_parent()->pgb_is_primary()) {
ECSubWriteReply reply;
reply.tid = tid;
struct SubWriteApplied : public Context {
ECBackend *pg;
OpRequestRef msg;
- tid_t tid;
+ ceph_tid_t tid;
eversion_t version;
SubWriteApplied(
ECBackend *pg,
OpRequestRef msg,
- tid_t tid,
+ ceph_tid_t tid,
eversion_t version)
: pg(pg), msg(msg), tid(tid), version(version) {}
void finish(int) {
}
};
void ECBackend::sub_write_applied(
- tid_t tid, eversion_t version) {
+ ceph_tid_t tid, eversion_t version) {
parent->op_applied(version);
if (get_parent()->pgb_is_primary()) {
ECSubWriteReply reply;
pg_shard_t from,
ECSubWriteReply &op)
{
- map<tid_t, Op>::iterator i = tid_to_op_map.find(op.tid);
+ map<ceph_tid_t, Op>::iterator i = tid_to_op_map.find(op.tid);
assert(i != tid_to_op_map.end());
if (op.committed) {
assert(i->second.pending_commit.count(from));
RecoveryMessages *m)
{
dout(10) << __func__ << ": reply " << op << dendl;
- map<tid_t, ReadOp>::iterator iter = tid_to_read_map.find(op.tid);
+ map<ceph_tid_t, ReadOp>::iterator iter = tid_to_read_map.find(op.tid);
if (iter == tid_to_read_map.end()) {
//canceled
return;
rop.complete[i->first].r = i->second;
}
- map<pg_shard_t, set<tid_t> >::iterator siter = shard_to_read_map.find(from);
+ map<pg_shard_t, set<ceph_tid_t> >::iterator siter =
+shard_to_read_map.find(from);
assert(siter != shard_to_read_map.end());
assert(siter->second.count(op.tid));
siter->second.erase(op.tid);
struct FinishReadOp : public GenContext<ThreadPool::TPHandle&> {
ECBackend *ec;
- tid_t tid;
- FinishReadOp(ECBackend *ec, tid_t tid) : ec(ec), tid(tid) {}
+ ceph_tid_t tid;
+ FinishReadOp(ECBackend *ec, ceph_tid_t tid) : ec(ec), tid(tid) {}
void finish(ThreadPool::TPHandle &handle) {
assert(ec->tid_to_read_map.count(tid));
int priority = ec->tid_to_read_map[tid].priority;
void ECBackend::check_recovery_sources(const OSDMapRef osdmap)
{
- set<tid_t> tids_to_filter;
- for (map<pg_shard_t, set<tid_t> >::iterator i = shard_to_read_map.begin();
+ set<ceph_tid_t> tids_to_filter;
+ for (map<pg_shard_t, set<ceph_tid_t> >::iterator
+ i = shard_to_read_map.begin();
i != shard_to_read_map.end();
) {
if (osdmap->is_down(i->first.osd)) {
++i;
}
}
- for (set<tid_t>::iterator i = tids_to_filter.begin();
+ for (set<ceph_tid_t>::iterator i = tids_to_filter.begin();
i != tids_to_filter.end();
++i) {
- map<tid_t, ReadOp>::iterator j = tid_to_read_map.find(*i);
+ map<ceph_tid_t, ReadOp>::iterator j = tid_to_read_map.find(*i);
assert(j != tid_to_read_map.end());
filter_read_op(osdmap, j->second);
}
{
writing.clear();
tid_to_op_map.clear();
- for (map<tid_t, ReadOp>::iterator i = tid_to_read_map.begin();
+ for (map<ceph_tid_t, ReadOp>::iterator i = tid_to_read_map.begin();
i != tid_to_read_map.end();
++i) {
dout(10) << __func__ << ": cancelling " << i->second << dendl;
}
f->close_section();
f->open_array_section("read_ops");
- for (map<tid_t, ReadOp>::const_iterator i = tid_to_read_map.begin();
+ for (map<ceph_tid_t, ReadOp>::const_iterator i = tid_to_read_map.begin();
i != tid_to_read_map.end();
++i) {
f->open_object_section("read_op");
Context *on_local_applied_sync,
Context *on_all_applied,
Context *on_all_commit,
- tid_t tid,
+ ceph_tid_t tid,
osd_reqid_t reqid,
OpRequestRef client_op
)
map<hobject_t, read_request_t> &to_read,
OpRequestRef _op)
{
- tid_t tid = get_parent()->get_tid();
+ ceph_tid_t tid = get_parent()->get_tid();
assert(!tid_to_read_map.count(tid));
ReadOp &op(tid_to_read_map[tid]);
op.priority = priority;
writing.pop_front();
tid_to_op_map.erase(op->tid);
}
- for (map<tid_t, Op>::iterator i = tid_to_op_map.begin();
+ for (map<ceph_tid_t, Op>::iterator i = tid_to_op_map.begin();
i != tid_to_op_map.end();
++i) {
dout(20) << __func__ << " tid " << i->first <<": " << i->second << dendl;
friend struct SubWriteApplied;
friend struct SubWriteCommitted;
void sub_write_applied(
- tid_t tid, eversion_t version);
+ ceph_tid_t tid, eversion_t version);
void sub_write_committed(
- tid_t tid, eversion_t version, eversion_t last_complete);
+ ceph_tid_t tid, eversion_t version, eversion_t last_complete);
void handle_sub_write(
pg_shard_t from,
OpRequestRef msg,
Context *on_local_applied_sync,
Context *on_all_applied,
Context *on_all_commit,
- tid_t tid,
+ ceph_tid_t tid,
osd_reqid_t reqid,
OpRequestRef op
);
struct ReadOp {
int priority;
- tid_t tid;
+ ceph_tid_t tid;
OpRequestRef op; // may be null if not on behalf of a client
map<hobject_t, read_request_t> to_read;
ReadOp &op);
void complete_read_op(ReadOp &rop, RecoveryMessages *m);
friend ostream &operator<<(ostream &lhs, const ReadOp &rhs);
- map<tid_t, ReadOp> tid_to_read_map;
- map<pg_shard_t, set<tid_t> > shard_to_read_map;
+ map<ceph_tid_t, ReadOp> tid_to_read_map;
+ map<pg_shard_t, set<ceph_tid_t> > shard_to_read_map;
void start_read_op(
int priority,
map<hobject_t, read_request_t> &to_read,
Context *on_local_applied_sync;
Context *on_all_applied;
Context *on_all_commit;
- tid_t tid;
+ ceph_tid_t tid;
osd_reqid_t reqid;
OpRequestRef client_op;
pg_shard_t from,
RecoveryMessages *m);
- map<tid_t, Op> tid_to_op_map; /// lists below point into here
+ map<ceph_tid_t, Op> tid_to_op_map; /// lists below point into here
list<Op*> writing;
CephContext *cct;
struct ECSubWrite {
pg_shard_t from;
- tid_t tid;
+ ceph_tid_t tid;
osd_reqid_t reqid;
hobject_t soid;
pg_stat_t stats;
ECSubWrite() {}
ECSubWrite(
pg_shard_t from,
- tid_t tid,
+ ceph_tid_t tid,
osd_reqid_t reqid,
hobject_t soid,
const pg_stat_t &stats,
struct ECSubWriteReply {
pg_shard_t from;
- tid_t tid;
+ ceph_tid_t tid;
eversion_t last_complete;
bool committed;
bool applied;
struct ECSubRead {
pg_shard_t from;
- tid_t tid;
+ ceph_tid_t tid;
map<hobject_t, list<pair<uint64_t, uint64_t> > > to_read;
set<hobject_t> attrs_to_read;
void encode(bufferlist &bl) const;
struct ECSubReadReply {
pg_shard_t from;
- tid_t tid;
+ ceph_tid_t tid;
map<hobject_t, list<pair<uint64_t, bufferlist> > > buffers_read;
map<hobject_t, map<string, bufferlist> > attrs_read;
map<hobject_t, int> errors;
"osd", "rw", "cli,rest")
};
-void OSD::do_command(Connection *con, tid_t tid, vector<string>& cmd, bufferlist& data)
+void OSD::do_command(Connection *con, ceph_tid_t tid, vector<string>& cmd, bufferlist& data)
{
int r = 0;
stringstream ss, ds;
// -- tids --
// for ops i issue
- tid_t last_tid;
+ ceph_tid_t last_tid;
Mutex tid_lock;
- tid_t get_tid() {
- tid_t t;
+ ceph_tid_t get_tid() {
+ ceph_tid_t t;
tid_lock.Lock();
t = ++last_tid;
tid_lock.Unlock();
pg_stat_queue_lock.Unlock();
}
- tid_t get_tid() {
+ ceph_tid_t get_tid() {
return service.get_tid();
}
// -- commands --
struct Command {
vector<string> cmd;
- tid_t tid;
+ ceph_tid_t tid;
bufferlist indata;
ConnectionRef con;
- Command(vector<string>& c, tid_t t, bufferlist& bl, Connection *co)
+ Command(vector<string>& c, ceph_tid_t t, bufferlist& bl, Connection *co)
: cmd(c), tid(t), indata(bl), con(co) {}
};
list<Command*> command_queue;
void handle_command(class MMonCommand *m);
void handle_command(class MCommand *m);
- void do_command(Connection *con, tid_t tid, vector<string>& cmd, bufferlist& data);
+ void do_command(Connection *con, ceph_tid_t tid, vector<string>& cmd, bufferlist& data);
// -- pg recovery --
xlist<PG*> recovery_queue;
*/
struct osd_reqid_t {
entity_name_t name; // who
- tid_t tid;
+ ceph_tid_t tid;
int32_t inc; // incarnation
osd_reqid_t()
: tid(0), inc(0) {}
- osd_reqid_t(const entity_name_t& a, int i, tid_t t)
+ osd_reqid_t(const entity_name_t& a, int i, ceph_tid_t t)
: name(a), tid(t), inc(i) {}
void encode(bufferlist &bl) const;
virtual PerfCounters *get_logger() = 0;
- virtual tid_t get_tid() = 0;
+ virtual ceph_tid_t get_tid() = 0;
virtual LogClientTemp clog_error() = 0;
const eversion_t &trim_to, ///< [in] trim log to here
vector<pg_log_entry_t> &log_entries, ///< [in] log entries for t
Context *on_local_applied_sync, ///< [in] called when applied locally
- Context *on_all_applied, ///< [in] called when all acked
+ Context *on_all_applied, ///< [in] called when all acked
Context *on_all_commit, ///< [in] called when all commit
- tid_t tid, ///< [in] tid
+ ceph_tid_t tid, ///< [in] tid
osd_reqid_t reqid, ///< [in] reqid
OpRequestRef op ///< [in] op
) = 0;
void ReplicatedBackend::_on_change(ObjectStore::Transaction *t)
{
- for (map<tid_t, InProgressOp>::iterator i = in_progress_ops.begin();
+ for (map<ceph_tid_t, InProgressOp>::iterator i = in_progress_ops.begin();
i != in_progress_ops.end();
in_progress_ops.erase(i++)) {
if (i->second.on_commit)
Context *on_local_applied_sync,
Context *on_all_acked,
Context *on_all_commit,
- tid_t tid,
+ ceph_tid_t tid,
osd_reqid_t reqid,
OpRequestRef orig_op)
{
op->mark_started();
// must be replication.
- tid_t rep_tid = r->get_tid();
+ ceph_tid_t rep_tid = r->get_tid();
pg_shard_t from = r->from;
if (in_progress_ops.count(rep_tid)) {
- map<tid_t, InProgressOp>::iterator iter =
+ map<ceph_tid_t, InProgressOp>::iterator iter =
in_progress_ops.find(rep_tid);
InProgressOp &ip_op = iter->second;
MOSDOp *m = NULL;
* Client IO
*/
struct InProgressOp {
- tid_t tid;
+ ceph_tid_t tid;
set<pg_shard_t> waiting_for_commit;
set<pg_shard_t> waiting_for_applied;
Context *on_commit;
OpRequestRef op;
eversion_t v;
InProgressOp(
- tid_t tid, Context *on_commit, Context *on_applied,
+ ceph_tid_t tid, Context *on_commit, Context *on_applied,
OpRequestRef op, eversion_t v)
: tid(tid), on_commit(on_commit), on_applied(on_applied),
op(op), v(v) {}
waiting_for_applied.empty();
}
};
- map<tid_t, InProgressOp> in_progress_ops;
+ map<ceph_tid_t, InProgressOp> in_progress_ops;
public:
PGTransaction *get_transaction();
friend class C_OSD_OnOpCommit;
Context *on_local_applied_sync,
Context *on_all_applied,
Context *on_all_commit,
- tid_t tid,
+ ceph_tid_t tid,
osd_reqid_t reqid,
OpRequestRef op
);
void issue_op(
const hobject_t &soid,
const eversion_t &at_version,
- tid_t tid,
+ ceph_tid_t tid,
osd_reqid_t reqid,
eversion_t pg_trim_to,
hobject_t new_temp_oid,
// verify that we are doing this in order?
if (cct->_conf->osd_debug_op_order && m->get_source().is_client()) {
- map<client_t,tid_t>& cm = debug_op_order[obc->obs.oi.soid];
- tid_t t = m->get_tid();
+ map<client_t,ceph_tid_t>& cm = debug_op_order[obc->obs.oi.soid];
+ ceph_tid_t t = m->get_tid();
client_t n = m->get_source().num();
- map<client_t,tid_t>::iterator p = cm.find(n);
+ map<client_t,ceph_tid_t>::iterator p = cm.find(n);
if (p == cm.end()) {
dout(20) << " op order client." << n << " tid " << t << " (first)" << dendl;
cm[n] = t;
}
// issue replica writes
- tid_t rep_tid = osd->get_tid();
+ ceph_tid_t rep_tid = osd->get_tid();
RepGather *repop = new_repop(ctx, obc, rep_tid); // new repop claims our obc, src_obc refs
// note: repop now owns ctx AND ctx->op
ReplicatedPGRef pg;
hobject_t oid;
epoch_t last_peering_reset;
- tid_t tid;
+ ceph_tid_t tid;
C_Copyfrom(ReplicatedPG *p, hobject_t o, epoch_t lpr)
: pg(p), oid(o), last_peering_reset(lpr),
tid(0)
ObjectOperation op;
op.list_snaps(&cop->results.snapset, NULL);
osd->objecter_lock.Lock();
- tid_t tid = osd->objecter->read(cop->src.oid, cop->oloc, op,
+ ceph_tid_t tid = osd->objecter->read(cop->src.oid, cop->oloc, op,
CEPH_SNAPDIR, NULL,
flags, gather.new_sub(), NULL);
cop->objecter_tid2 = tid;
&osd->objecter_finisher));
osd->objecter_lock.Lock();
- tid_t tid = osd->objecter->read(cop->src.oid, cop->oloc, op,
+ ceph_tid_t tid = osd->objecter->read(cop->src.oid, cop->oloc, op,
cop->src.snap, NULL,
flags,
gather.new_sub(),
osd->objecter_lock.Unlock();
}
-void ReplicatedPG::process_copy_chunk(hobject_t oid, tid_t tid, int r)
+void ReplicatedPG::process_copy_chunk(hobject_t oid, ceph_tid_t tid, int r)
{
dout(10) << __func__ << " " << oid << " tid " << tid
<< " " << cpp_strerror(r) << dendl;
ReplicatedPGRef pg;
hobject_t oid;
epoch_t last_peering_reset;
- tid_t tid;
+ ceph_tid_t tid;
C_Flush(ReplicatedPG *p, hobject_t o, epoch_t lpr)
: pg(p), oid(o), last_peering_reset(lpr),
tid(0)
}
osd->objecter_lock.Lock();
- tid_t tid = osd->objecter->mutate(soid.oid, base_oloc, o, snapc, oi.mtime,
+ ceph_tid_t tid = osd->objecter->mutate(soid.oid, base_oloc, o, snapc, oi.mtime,
CEPH_OSD_FLAG_IGNORE_OVERLAY,
NULL,
new C_OnFinisher(fin,
return -EINPROGRESS;
}
-void ReplicatedPG::finish_flush(hobject_t oid, tid_t tid, int r)
+void ReplicatedPG::finish_flush(hobject_t oid, ceph_tid_t tid, int r)
{
dout(10) << __func__ << " " << oid << " tid " << tid
<< " " << cpp_strerror(r) << dendl;
}
dout(10) << __func__ << " clearing DIRTY flag for " << oid << dendl;
- tid_t rep_tid = osd->get_tid();
+ ceph_tid_t rep_tid = osd->get_tid();
RepGather *repop = new_repop(fop->ctx, obc, rep_tid);
OpContext *ctx = fop->ctx;
if (!fop->blocking) {
void ReplicatedBackend::issue_op(
const hobject_t &soid,
const eversion_t &at_version,
- tid_t tid,
+ ceph_tid_t tid,
osd_reqid_t reqid,
eversion_t pg_trim_to,
hobject_t new_temp_oid,
}
ReplicatedPG::RepGather *ReplicatedPG::new_repop(OpContext *ctx, ObjectContextRef obc,
- tid_t rep_tid)
+ ceph_tid_t rep_tid)
{
if (ctx->op)
dout(10) << "new_repop rep_tid " << rep_tid << " on " << *ctx->op->get_req() << dendl;
{
dout(20) << __func__ << " " << obc->obs.oi.soid << dendl;
vector<OSDOp> ops;
- tid_t rep_tid = osd->get_tid();
+ ceph_tid_t rep_tid = osd->get_tid();
osd_reqid_t reqid(osd->get_cluster_msgr_name(), 0, rep_tid);
OpContext *ctx = new OpContext(OpRequestRef(), reqid, ops,
&obc->obs, obc->ssc, this);
watch->remove();
vector<OSDOp> ops;
- tid_t rep_tid = osd->get_tid();
+ ceph_tid_t rep_tid = osd->get_tid();
osd_reqid_t reqid(osd->get_cluster_msgr_name(), 0, rep_tid);
OpContext *ctx = new OpContext(OpRequestRef(), reqid, ops,
&obc->obs, obc->ssc, this);
void ReplicatedPG::send_remove_op(
const hobject_t& oid, eversion_t v, pg_shard_t peer)
{
- tid_t tid = osd->get_tid();
+ ceph_tid_t tid = osd->get_tid();
osd_reqid_t rid(osd->get_cluster_msgr_name(), 0, tid);
dout(10) << "send_remove_op " << oid << " from osd." << peer
ObjectRecoveryProgress progress)
{
// send op
- tid_t tid = get_parent()->get_tid();
+ ceph_tid_t tid = get_parent()->get_tid();
osd_reqid_t rid(get_parent()->get_cluster_msgr_name(), 0, tid);
dout(10) << "send_pull_op " << recovery_info.soid << " "
int ReplicatedBackend::send_push_op_legacy(int prio, pg_shard_t peer, PushOp &pop)
{
- tid_t tid = get_parent()->get_tid();
+ ceph_tid_t tid = get_parent()->get_tid();
osd_reqid_t rid(get_parent()->get_cluster_msgr_name(), 0, tid);
MOSDSubOp *subop = new MOSDSubOp(
rid, parent->whoami_shard(),
// ages we expect.
vector<OSDOp> ops;
- tid_t rep_tid = osd->get_tid();
+ ceph_tid_t rep_tid = osd->get_tid();
osd_reqid_t reqid(osd->get_cluster_msgr_name(), 0, rep_tid);
OpContext *ctx = new OpContext(OpRequestRef(), reqid, ops,
&obc->obs, obc->ssc, this);
CopyResults results;
- tid_t objecter_tid;
- tid_t objecter_tid2;
+ ceph_tid_t objecter_tid;
+ ceph_tid_t objecter_tid2;
object_copy_cursor_t cursor;
map<string,bufferlist> attrs;
OpContext *ctx; ///< the parent OpContext
list<OpRequestRef> dup_ops; ///< dup flush requests
version_t flushed_version; ///< user version we are flushing
- tid_t objecter_tid; ///< copy-from request tid
+ ceph_tid_t objecter_tid; ///< copy-from request tid
int rval; ///< copy-from result
bool blocking; ///< whether we are blocking updates
bool removal; ///< we are removing the backend object
PerfCounters *get_logger();
- tid_t get_tid() { return osd->get_tid(); }
+ ceph_tid_t get_tid() { return osd->get_tid(); }
LogClientTemp clog_error() { return osd->clog.error(); }
ObjectContextRef obc;
map<hobject_t,ObjectContextRef> src_obc;
- tid_t rep_tid;
+ ceph_tid_t rep_tid;
bool rep_aborted, rep_done;
Context *on_applied;
- RepGather(OpContext *c, ObjectContextRef pi, tid_t rt,
+ RepGather(OpContext *c, ObjectContextRef pi, ceph_tid_t rt,
eversion_t lc) :
queue_item(this),
nref(1),
// replica ops
// [primary|tail]
xlist<RepGather*> repop_queue;
- map<tid_t, RepGather*> repop_map;
+ map<ceph_tid_t, RepGather*> repop_map;
friend class C_OSD_RepopApplied;
friend class C_OSD_RepopCommit;
void repop_all_committed(RepGather *repop);
void eval_repop(RepGather*);
void issue_repop(RepGather *repop, utime_t now);
- RepGather *new_repop(OpContext *ctx, ObjectContextRef obc, tid_t rep_tid);
+ RepGather *new_repop(OpContext *ctx, ObjectContextRef obc, ceph_tid_t rep_tid);
void remove_repop(RepGather *repop);
RepGather *simple_repop_create(ObjectContextRef obc);
Mutex snapset_contexts_lock;
// debug order that client ops are applied
- map<hobject_t, map<client_t, tid_t> > debug_op_order;
+ map<hobject_t, map<client_t, ceph_tid_t> > debug_op_order;
void populate_obc_watchers(ObjectContextRef obc);
void check_blacklisted_obc_watchers(ObjectContextRef obc);
void start_copy(CopyCallback *cb, ObjectContextRef obc, hobject_t src,
object_locator_t oloc, version_t version, unsigned flags,
bool mirror_snapset);
- void process_copy_chunk(hobject_t oid, tid_t tid, int r);
+ void process_copy_chunk(hobject_t oid, ceph_tid_t tid, int r);
void _write_copy_chunk(CopyOpRef cop, PGBackend::PGTransaction *t);
uint64_t get_copy_chunk_size() const {
uint64_t size = cct->_conf->osd_copyfrom_max_chunk;
map<hobject_t, FlushOpRef> flush_ops;
int start_flush(OpContext *ctx, bool blocking);
- void finish_flush(hobject_t oid, tid_t tid, int r);
+ void finish_flush(hobject_t oid, ceph_tid_t tid, int r);
int try_flush_mark_clean(FlushOpRef fop);
void cancel_flush(FlushOpRef fop, bool requeue);
void cancel_flush_ops(bool requeue);
++reads_outstanding;
}
-void ObjectCacher::bh_read_finish(int64_t poolid, sobject_t oid, tid_t tid,
+void ObjectCacher::bh_read_finish(int64_t poolid, sobject_t oid, ceph_tid_t tid,
loff_t start, uint64_t length,
bufferlist &bl, int r,
bool trust_enoent)
C_WriteCommit *oncommit = new C_WriteCommit(this, bh->ob->oloc.pool,
bh->ob->get_soid(), bh->start(), bh->length());
// go
- tid_t tid = writeback_handler.write(bh->ob->get_oid(), bh->ob->get_oloc(),
+ ceph_tid_t tid = writeback_handler.write(bh->ob->get_oid(), bh->ob->get_oloc(),
bh->start(), bh->length(),
bh->snapc, bh->bl, bh->last_write,
bh->ob->truncate_size, bh->ob->truncate_seq,
}
void ObjectCacher::bh_write_commit(int64_t poolid, sobject_t oid, loff_t start,
- uint64_t length, tid_t tid, int r)
+ uint64_t length, ceph_tid_t tid, int r)
{
assert(lock.is_locked());
ldout(cct, 7) << "bh_write_commit "
public:
Object *ob;
bufferlist bl;
- tid_t last_write_tid; // version of bh (if non-zero)
- tid_t last_read_tid; // tid of last read op (if any)
+ ceph_tid_t last_write_tid; // version of bh (if non-zero)
+ ceph_tid_t last_read_tid; // tid of last read op (if any)
utime_t last_write;
SnapContext snapc;
int error; // holds return value for failed reads
public:
map<loff_t, BufferHead*> data;
- tid_t last_write_tid; // version of bh (if non-zero)
- tid_t last_commit_tid; // last update commited.
+ ceph_tid_t last_write_tid; // version of bh (if non-zero)
+ ceph_tid_t last_commit_tid; // last update commited.
int dirty_or_tx;
- map< tid_t, list<Context*> > waitfor_commit;
+ map< ceph_tid_t, list<Context*> > waitfor_commit;
xlist<C_ReadFinish*> reads;
public:
vector<ceph::unordered_map<sobject_t, Object*> > objects; // indexed by pool_id
- tid_t last_read_tid;
+ ceph_tid_t last_read_tid;
set<BufferHead*> dirty_bh;
LRU bh_lru_dirty, bh_lru_rest;
bool external_call);
public:
- void bh_read_finish(int64_t poolid, sobject_t oid, tid_t tid,
+ void bh_read_finish(int64_t poolid, sobject_t oid, ceph_tid_t tid,
loff_t offset, uint64_t length,
bufferlist &bl, int r,
bool trust_enoent);
void bh_write_commit(int64_t poolid, sobject_t oid, loff_t offset,
- uint64_t length, tid_t t, int r);
+ uint64_t length, ceph_tid_t t, int r);
class C_ReadFinish : public Context {
ObjectCacher *oc;
uint64_t length;
xlist<C_ReadFinish*>::item set_item;
bool trust_enoent;
- tid_t tid;
+ ceph_tid_t tid;
public:
bufferlist bl;
- C_ReadFinish(ObjectCacher *c, Object *ob, tid_t t, loff_t s, uint64_t l) :
+ C_ReadFinish(ObjectCacher *c, Object *ob, ceph_tid_t t, loff_t s, uint64_t l) :
oc(c), poolid(ob->oloc.pool), oid(ob->get_soid()), start(s), length(l),
set_item(this), trust_enoent(true),
tid(t) {
loff_t start;
uint64_t length;
public:
- tid_t tid;
+ ceph_tid_t tid;
C_WriteCommit(ObjectCacher *c, int64_t _poolid, sobject_t o, loff_t s, uint64_t l) :
oc(c), poolid(_poolid), oid(o), start(s), length(l), tid(0) {}
void finish(int r) {
}
}
-tid_t Objecter::linger_mutate(const object_t& oid, const object_locator_t& oloc,
+ceph_tid_t Objecter::linger_mutate(const object_t& oid, const object_locator_t& oloc,
ObjectOperation& op,
const SnapContext& snapc, utime_t mtime,
bufferlist& inbl, int flags,
return info->linger_id;
}
-tid_t Objecter::linger_read(const object_t& oid, const object_locator_t& oloc,
+ceph_tid_t Objecter::linger_read(const object_t& oid, const object_locator_t& oloc,
ObjectOperation& op,
snapid_t snap, bufferlist& inbl, bufferlist *poutbl, int flags,
Context *onfinish,
void Objecter::scan_requests(bool force_resend,
bool force_resend_writes,
- map<tid_t, Op*>& need_resend,
+ map<ceph_tid_t, Op*>& need_resend,
list<LingerOp*>& need_resend_linger,
- map<tid_t, CommandOp*>& need_resend_command)
+ map<ceph_tid_t, CommandOp*>& need_resend_command)
{
// check for changed linger mappings (_before_ regular ops)
- map<tid_t,LingerOp*>::iterator lp = linger_ops.begin();
+ map<ceph_tid_t,LingerOp*>::iterator lp = linger_ops.begin();
while (lp != linger_ops.end()) {
LingerOp *op = lp->second;
++lp; // check_linger_pool_dne() may touch linger_ops; prevent iterator invalidation
}
// check for changed request mappings
- map<tid_t,Op*>::iterator p = ops.begin();
+ map<ceph_tid_t,Op*>::iterator p = ops.begin();
while (p != ops.end()) {
Op *op = p->second;
++p; // check_op_pool_dne() may touch ops; prevent iterator invalidation
}
// commands
- map<tid_t,CommandOp*>::iterator cp = command_ops.begin();
+ map<ceph_tid_t,CommandOp*>::iterator cp = command_ops.begin();
while (cp != command_ops.end()) {
CommandOp *c = cp->second;
++cp;
bool was_pausewr = osdmap->test_flag(CEPH_OSDMAP_PAUSEWR) || was_full;
list<LingerOp*> need_resend_linger;
- map<tid_t, Op*> need_resend;
- map<tid_t, CommandOp*> need_resend_command;
+ map<ceph_tid_t, Op*> need_resend;
+ map<ceph_tid_t, CommandOp*> need_resend_command;
if (m->get_last() <= osdmap->get_epoch()) {
ldout(cct, 3) << "handle_osd_map ignoring epochs ["
maybe_request_map();
// resend requests
- for (map<tid_t, Op*>::iterator p = need_resend.begin(); p != need_resend.end(); ++p) {
+ for (map<ceph_tid_t, Op*>::iterator p = need_resend.begin(); p != need_resend.end(); ++p) {
Op *op = p->second;
if (op->should_resend) {
if (op->session && !op->paused) {
send_linger(op);
}
}
- for (map<tid_t,CommandOp*>::iterator p = need_resend_command.begin(); p != need_resend_command.end(); ++p) {
+ for (map<ceph_tid_t,CommandOp*>::iterator p = need_resend_command.begin(); p != need_resend_command.end(); ++p) {
CommandOp *c = p->second;
if (c->session) {
_send_command(c);
Mutex::Locker l(objecter->client_lock);
- map<tid_t, Op*>::iterator iter =
+ map<ceph_tid_t, Op*>::iterator iter =
objecter->check_latest_map_ops.find(tid);
if (iter == objecter->check_latest_map_ops.end()) {
lgeneric_subdout(objecter->cct, objecter, 10) << "op_map_latest op " << tid << " not found" << dendl;
void Objecter::op_cancel_map_check(Op *op)
{
assert(client_lock.is_locked());
- map<tid_t, Op*>::iterator iter =
+ map<ceph_tid_t, Op*>::iterator iter =
check_latest_map_ops.find(op->tid);
if (iter != check_latest_map_ops.end()) {
check_latest_map_ops.erase(iter);
ldout(cct, 10) << "kick_requests for osd." << session->osd << dendl;
// resend ops
- map<tid_t,Op*> resend; // resend in tid order
+ map<ceph_tid_t,Op*> resend; // resend in tid order
for (xlist<Op*>::iterator p = session->ops.begin(); !p.end();) {
Op *op = *p;
++p;
cutoff -= cct->_conf->objecter_timeout; // timeout
unsigned laggy_ops = 0;
- for (map<tid_t,Op*>::iterator p = ops.begin();
+ for (map<ceph_tid_t,Op*>::iterator p = ops.begin();
p != ops.end();
++p) {
Op *op = p->second;
assert(client_lock.is_locked());
ldout(cct, 10) << "resend_mon_ops" << dendl;
- for (map<tid_t,PoolStatOp*>::iterator p = poolstat_ops.begin(); p!=poolstat_ops.end(); ++p) {
+ for (map<ceph_tid_t,PoolStatOp*>::iterator p = poolstat_ops.begin(); p!=poolstat_ops.end(); ++p) {
poolstat_submit(p->second);
logger->inc(l_osdc_poolstat_resend);
}
- for (map<tid_t,StatfsOp*>::iterator p = statfs_ops.begin(); p!=statfs_ops.end(); ++p) {
+ for (map<ceph_tid_t,StatfsOp*>::iterator p = statfs_ops.begin(); p!=statfs_ops.end(); ++p) {
fs_stats_submit(p->second);
logger->inc(l_osdc_statfs_resend);
}
- for (map<tid_t,PoolOp*>::iterator p = pool_ops.begin(); p!=pool_ops.end(); ++p) {
+ for (map<ceph_tid_t,PoolOp*>::iterator p = pool_ops.begin(); p!=pool_ops.end(); ++p) {
_pool_op_submit(p->second);
logger->inc(l_osdc_poolop_resend);
}
- for (map<tid_t, Op*>::iterator p = check_latest_map_ops.begin();
+ for (map<ceph_tid_t, Op*>::iterator p = check_latest_map_ops.begin();
p != check_latest_map_ops.end();
++p) {
C_Op_Map_Latest *c = new C_Op_Map_Latest(this, p->second->tid);
}
};
-tid_t Objecter::op_submit(Op *op)
+ceph_tid_t Objecter::op_submit(Op *op)
{
assert(client_lock.is_locked());
assert(initialized);
return _op_submit(op);
}
-tid_t Objecter::_op_submit(Op *op)
+ceph_tid_t Objecter::_op_submit(Op *op)
{
// pick tid
- tid_t mytid = ++last_tid;
+ ceph_tid_t mytid = ++last_tid;
op->tid = mytid;
assert(client_inc >= 0);
return op->tid;
}
-int Objecter::op_cancel(tid_t tid, int r)
+int Objecter::op_cancel(ceph_tid_t tid, int r)
{
assert(client_lock.is_locked());
assert(initialized);
- map<tid_t, Op*>::iterator p = ops.find(tid);
+ map<ceph_tid_t, Op*>::iterator p = ops.find(tid);
if (p == ops.end()) {
ldout(cct, 10) << __func__ << " tid " << tid << " dne" << dendl;
return -ENOENT;
ldout(cct, 10) << "in handle_osd_op_reply" << dendl;
// get pio
- tid_t tid = m->get_tid();
+ ceph_tid_t tid = m->get_tid();
if (ops.count(tid) == 0) {
ldout(cct, 7) << "handle_osd_op_reply " << tid
class C_CancelPoolOp : public Context
{
- tid_t tid;
+ ceph_tid_t tid;
Objecter *objecter;
public:
- C_CancelPoolOp(tid_t tid, Objecter *objecter) : tid(tid),
- objecter(objecter) {}
+ C_CancelPoolOp(ceph_tid_t tid, Objecter *objecter) : tid(tid),
+ objecter(objecter) {}
void finish(int r) {
// note that objecter lock == timer lock, and is already held
objecter->pool_op_cancel(tid, -ETIMEDOUT);
assert(client_lock.is_locked());
assert(initialized);
ldout(cct, 10) << "handle_pool_op_reply " << *m << dendl;
- tid_t tid = m->get_tid();
+ ceph_tid_t tid = m->get_tid();
if (pool_ops.count(tid)) {
PoolOp *op = pool_ops[tid];
ldout(cct, 10) << "have request " << tid << " at " << op << " Op: " << ceph_pool_op_name(op->pool_op) << dendl;
m->put();
}
-int Objecter::pool_op_cancel(tid_t tid, int r)
+int Objecter::pool_op_cancel(ceph_tid_t tid, int r)
{
assert(client_lock.is_locked());
assert(initialized);
- map<tid_t, PoolOp*>::iterator it = pool_ops.find(tid);
+ map<ceph_tid_t, PoolOp*>::iterator it = pool_ops.find(tid);
if (it == pool_ops.end()) {
ldout(cct, 10) << __func__ << " tid " << tid << " dne" << dendl;
return -ENOENT;
class C_CancelPoolStatOp : public Context
{
- tid_t tid;
+ ceph_tid_t tid;
Objecter *objecter;
public:
- C_CancelPoolStatOp(tid_t tid, Objecter *objecter) : tid(tid),
- objecter(objecter) {}
+ C_CancelPoolStatOp(ceph_tid_t tid, Objecter *objecter) : tid(tid),
+ objecter(objecter) {}
void finish(int r) {
// note that objecter lock == timer lock, and is already held
objecter->pool_stat_op_cancel(tid, -ETIMEDOUT);
assert(client_lock.is_locked());
assert(initialized);
ldout(cct, 10) << "handle_get_pool_stats_reply " << *m << dendl;
- tid_t tid = m->get_tid();
+ ceph_tid_t tid = m->get_tid();
if (poolstat_ops.count(tid)) {
PoolStatOp *op = poolstat_ops[tid];
m->put();
}
-int Objecter::pool_stat_op_cancel(tid_t tid, int r)
+int Objecter::pool_stat_op_cancel(ceph_tid_t tid, int r)
{
assert(client_lock.is_locked());
assert(initialized);
- map<tid_t, PoolStatOp*>::iterator it = poolstat_ops.find(tid);
+ map<ceph_tid_t, PoolStatOp*>::iterator it = poolstat_ops.find(tid);
if (it == poolstat_ops.end()) {
ldout(cct, 10) << __func__ << " tid " << tid << " dne" << dendl;
return -ENOENT;
class C_CancelStatfsOp : public Context
{
- tid_t tid;
+ ceph_tid_t tid;
Objecter *objecter;
public:
- C_CancelStatfsOp(tid_t tid, Objecter *objecter) : tid(tid),
- objecter(objecter) {}
+ C_CancelStatfsOp(ceph_tid_t tid, Objecter *objecter) : tid(tid),
+ objecter(objecter) {}
void finish(int r) {
// note that objecter lock == timer lock, and is already held
objecter->statfs_op_cancel(tid, -ETIMEDOUT);
assert(client_lock.is_locked());
assert(initialized);
ldout(cct, 10) << "handle_fs_stats_reply " << *m << dendl;
- tid_t tid = m->get_tid();
+ ceph_tid_t tid = m->get_tid();
if (statfs_ops.count(tid)) {
StatfsOp *op = statfs_ops[tid];
m->put();
}
-int Objecter::statfs_op_cancel(tid_t tid, int r)
+int Objecter::statfs_op_cancel(ceph_tid_t tid, int r)
{
assert(client_lock.is_locked());
assert(initialized);
- map<tid_t, StatfsOp*>::iterator it = statfs_ops.find(tid);
+ map<ceph_tid_t, StatfsOp*>::iterator it = statfs_ops.find(tid);
if (it == statfs_ops.end()) {
ldout(cct, 10) << __func__ << " tid " << tid << " dne" << dendl;
return -ENOENT;
void Objecter::dump_active()
{
ldout(cct, 20) << "dump_active .. " << num_homeless_ops << " homeless" << dendl;
- for (map<tid_t,Op*>::iterator p = ops.begin(); p != ops.end(); ++p) {
+ for (map<ceph_tid_t,Op*>::iterator p = ops.begin(); p != ops.end(); ++p) {
Op *op = p->second;
ldout(cct, 20) << op->tid << "\t" << op->pgid << "\tosd." << (op->session ? op->session->osd : -1)
<< "\t" << op->base_oid << "\t" << op->ops << dendl;
void Objecter::dump_ops(Formatter *fmt) const
{
fmt->open_array_section("ops");
- for (map<tid_t,Op*>::const_iterator p = ops.begin();
+ for (map<ceph_tid_t,Op*>::const_iterator p = ops.begin();
p != ops.end();
++p) {
Op *op = p->second;
void Objecter::dump_pool_ops(Formatter *fmt) const
{
fmt->open_array_section("pool_ops");
- for (map<tid_t, PoolOp*>::const_iterator p = pool_ops.begin();
+ for (map<ceph_tid_t, PoolOp*>::const_iterator p = pool_ops.begin();
p != pool_ops.end();
++p) {
PoolOp *op = p->second;
void Objecter::dump_pool_stat_ops(Formatter *fmt) const
{
fmt->open_array_section("pool_stat_ops");
- for (map<tid_t, PoolStatOp*>::const_iterator p = poolstat_ops.begin();
+ for (map<ceph_tid_t, PoolStatOp*>::const_iterator p = poolstat_ops.begin();
p != poolstat_ops.end();
++p) {
PoolStatOp *op = p->second;
void Objecter::dump_statfs_ops(Formatter *fmt) const
{
fmt->open_array_section("statfs_ops");
- for (map<tid_t, StatfsOp*>::const_iterator p = statfs_ops.begin();
+ for (map<ceph_tid_t, StatfsOp*>::const_iterator p = statfs_ops.begin();
p != statfs_ops.end();
++p) {
StatfsOp *op = p->second;
void Objecter::handle_command_reply(MCommandReply *m)
{
- map<tid_t,CommandOp*>::iterator p = command_ops.find(m->get_tid());
+ map<ceph_tid_t,CommandOp*>::iterator p = command_ops.find(m->get_tid());
if (p == command_ops.end()) {
ldout(cct, 10) << "handle_command_reply tid " << m->get_tid() << " not found" << dendl;
m->put();
class C_CancelCommandOp : public Context
{
- tid_t tid;
+ ceph_tid_t tid;
Objecter *objecter;
public:
- C_CancelCommandOp(tid_t tid, Objecter *objecter) : tid(tid),
- objecter(objecter) {}
+ C_CancelCommandOp(ceph_tid_t tid, Objecter *objecter) : tid(tid),
+ objecter(objecter) {}
void finish(int r) {
// note that objecter lock == timer lock, and is already held
objecter->command_op_cancel(tid, -ETIMEDOUT);
}
};
-int Objecter::_submit_command(CommandOp *c, tid_t *ptid)
+int Objecter::_submit_command(CommandOp *c, ceph_tid_t *ptid)
{
- tid_t tid = ++last_tid;
+ ceph_tid_t tid = ++last_tid;
ldout(cct, 10) << "_submit_command " << tid << " " << c->cmd << dendl;
c->tid = tid;
if (osd_timeout > 0) {
logger->inc(l_osdc_command_send);
}
-int Objecter::command_op_cancel(tid_t tid, int r)
+int Objecter::command_op_cancel(ceph_tid_t tid, int r)
{
assert(client_lock.is_locked());
assert(initialized);
- map<tid_t, CommandOp*>::iterator it = command_ops.find(tid);
+ map<ceph_tid_t, CommandOp*>::iterator it = command_ops.find(tid);
if (it == command_ops.end()) {
ldout(cct, 10) << __func__ << " tid " << tid << " dne" << dendl;
return -ENOENT;
bool initialized;
private:
- tid_t last_tid;
+ ceph_tid_t last_tid;
int client_inc;
uint64_t max_linger_id;
int num_unacked;
int flags, priority;
Context *onack, *oncommit, *ontimeout;
- tid_t tid;
+ ceph_tid_t tid;
eversion_t replay_version; // for op replay
int attempts;
struct C_Op_Map_Latest : public Context {
Objecter *objecter;
- tid_t tid;
+ ceph_tid_t tid;
version_t latest;
- C_Op_Map_Latest(Objecter *o, tid_t t) : objecter(o), tid(t), latest(0) {}
+ C_Op_Map_Latest(Objecter *o, ceph_tid_t t) : objecter(o), tid(t), latest(0) {}
void finish(int r);
};
Objecter *objecter;
uint64_t tid;
version_t latest;
- C_Command_Map_Latest(Objecter *o, tid_t t) : objecter(o), tid(t), latest(0) {}
+ C_Command_Map_Latest(Objecter *o, ceph_tid_t t) : objecter(o), tid(t), latest(0) {}
void finish(int r);
};
};
struct PoolStatOp {
- tid_t tid;
+ ceph_tid_t tid;
list<string> pools;
map<string,pool_stat_t> *pool_stats;
};
struct StatfsOp {
- tid_t tid;
+ ceph_tid_t tid;
struct ceph_statfs *stats;
Context *onfinish, *ontimeout;
};
struct PoolOp {
- tid_t tid;
+ ceph_tid_t tid;
int64_t pool;
string name;
Context *onfinish, *ontimeout;
struct CommandOp : public RefCountedObject {
xlist<CommandOp*>::item session_item;
OSDSession *session;
- tid_t tid;
+ ceph_tid_t tid;
vector<string> cmd;
bufferlist inbl;
bufferlist *poutbl;
onfinish(NULL), ontimeout(NULL) {}
};
- int _submit_command(CommandOp *c, tid_t *ptid);
+ int _submit_command(CommandOp *c, ceph_tid_t *ptid);
int recalc_command_target(CommandOp *c);
void _send_command(CommandOp *c);
- int command_op_cancel(tid_t tid, int r);
+ int command_op_cancel(ceph_tid_t tid, int r);
void _finish_command(CommandOp *c, int r, string rs);
void handle_command_reply(MCommandReply *m);
OSDSession *session;
xlist<LingerOp*>::item session_item;
- tid_t register_tid;
+ ceph_tid_t register_tid;
epoch_t map_dne_bound;
LingerOp() : linger_id(0), primary(-1),
private:
// pending ops
- map<tid_t,Op*> ops;
+ map<ceph_tid_t,Op*> ops;
int num_homeless_ops;
map<uint64_t, LingerOp*> linger_ops;
- map<tid_t,PoolStatOp*> poolstat_ops;
- map<tid_t,StatfsOp*> statfs_ops;
- map<tid_t,PoolOp*> pool_ops;
- map<tid_t,CommandOp*> command_ops;
+ map<ceph_tid_t,PoolStatOp*> poolstat_ops;
+ map<ceph_tid_t,StatfsOp*> statfs_ops;
+ map<ceph_tid_t,PoolOp*> pool_ops;
+ map<ceph_tid_t,CommandOp*> command_ops;
// ops waiting for an osdmap with a new pool or confirmation that
// the pool does not exist (may be expanded to other uses later)
- map<uint64_t, LingerOp*> check_latest_map_lingers;
- map<tid_t, Op*> check_latest_map_ops;
- map<tid_t, CommandOp*> check_latest_map_commands;
+ map<uint64_t, LingerOp*> check_latest_map_lingers;
+ map<ceph_tid_t, Op*> check_latest_map_ops;
+ map<ceph_tid_t, CommandOp*> check_latest_map_commands;
map<epoch_t,list< pair<Context*, int> > > waiting_for_map;
void scan_requests(bool force_resend,
bool force_resend_writes,
- map<tid_t, Op*>& need_resend,
+ map<ceph_tid_t, Op*>& need_resend,
list<LingerOp*>& need_resend_linger,
- map<tid_t, CommandOp*>& need_resend_command);
+ map<ceph_tid_t, CommandOp*>& need_resend_command);
int64_t get_object_hash_position(int64_t pool, const string& key, const string& ns);
int64_t get_object_pg_hash_position(int64_t pool, const string& key, const string& ns);
private:
// low-level
- tid_t _op_submit(Op *op);
+ ceph_tid_t _op_submit(Op *op);
inline void unregister_op(Op *op);
// public interface
public:
- tid_t op_submit(Op *op);
+ ceph_tid_t op_submit(Op *op);
bool is_active() {
return !(ops.empty() && linger_ops.empty() && poolstat_ops.empty() && statfs_ops.empty());
}
void clear_global_op_flag(int flags) { global_op_flags &= ~flags; }
/// cancel an in-progress request with the given return code
- int op_cancel(tid_t tid, int r);
+ int op_cancel(ceph_tid_t tid, int r);
// commands
int osd_command(int osd, vector<string>& cmd,
- const bufferlist& inbl, tid_t *ptid,
+ const bufferlist& inbl, ceph_tid_t *ptid,
bufferlist *poutbl, string *prs, Context *onfinish) {
assert(osd >= 0);
CommandOp *c = new CommandOp;
return _submit_command(c, ptid);
}
int pg_command(pg_t pgid, vector<string>& cmd,
- const bufferlist& inbl, tid_t *ptid,
+ const bufferlist& inbl, ceph_tid_t *ptid,
bufferlist *poutbl, string *prs, Context *onfinish) {
CommandOp *c = new CommandOp;
c->cmd = cmd;
}
// mid-level helpers
- Op *prepare_mutate_op(const object_t& oid, const object_locator_t& oloc,
+ Op *prepare_mutate_op(const object_t& oid, const object_locator_t& oloc,
ObjectOperation& op,
const SnapContext& snapc, utime_t mtime, int flags,
Context *onack, Context *oncommit, version_t *objver = NULL) {
o->out_rval.swap(op.out_rval);
return o;
}
- tid_t mutate(const object_t& oid, const object_locator_t& oloc,
+ ceph_tid_t mutate(const object_t& oid, const object_locator_t& oloc,
ObjectOperation& op,
const SnapContext& snapc, utime_t mtime, int flags,
Context *onack, Context *oncommit, version_t *objver = NULL) {
o->out_rval.swap(op.out_rval);
return o;
}
- tid_t read(const object_t& oid, const object_locator_t& oloc,
+ ceph_tid_t read(const object_t& oid, const object_locator_t& oloc,
ObjectOperation& op,
snapid_t snapid, bufferlist *pbl, int flags,
Context *onack, version_t *objver = NULL) {
Op *o = prepare_read_op(oid, oloc, op, snapid, pbl, flags, onack, objver);
return op_submit(o);
}
- tid_t pg_read(uint32_t hash, object_locator_t oloc,
+ ceph_tid_t pg_read(uint32_t hash, object_locator_t oloc,
ObjectOperation& op,
bufferlist *pbl, int flags,
Context *onack,
o->reply_epoch = reply_epoch;
return op_submit(o);
}
- tid_t linger_mutate(const object_t& oid, const object_locator_t& oloc,
+ ceph_tid_t linger_mutate(const object_t& oid, const object_locator_t& oloc,
ObjectOperation& op,
const SnapContext& snapc, utime_t mtime,
bufferlist& inbl, int flags,
Context *onack, Context *onfinish,
version_t *objver);
- tid_t linger_read(const object_t& oid, const object_locator_t& oloc,
+ ceph_tid_t linger_read(const object_t& oid, const object_locator_t& oloc,
ObjectOperation& op,
snapid_t snap, bufferlist& inbl, bufferlist *poutbl, int flags,
Context *onack,
// high-level helpers
- tid_t stat(const object_t& oid, const object_locator_t& oloc, snapid_t snap,
+ ceph_tid_t stat(const object_t& oid, const object_locator_t& oloc, snapid_t snap,
uint64_t *psize, utime_t *pmtime, int flags,
Context *onfinish,
version_t *objver = NULL, ObjectOperation *extra_ops = NULL) {
return op_submit(o);
}
- tid_t read(const object_t& oid, const object_locator_t& oloc,
+ ceph_tid_t read(const object_t& oid, const object_locator_t& oloc,
uint64_t off, uint64_t len, snapid_t snap, bufferlist *pbl, int flags,
Context *onfinish,
version_t *objver = NULL, ObjectOperation *extra_ops = NULL) {
return op_submit(o);
}
- tid_t read_trunc(const object_t& oid, const object_locator_t& oloc,
+ ceph_tid_t read_trunc(const object_t& oid, const object_locator_t& oloc,
uint64_t off, uint64_t len, snapid_t snap, bufferlist *pbl, int flags,
uint64_t trunc_size, __u32 trunc_seq,
Context *onfinish,
o->outbl = pbl;
return op_submit(o);
}
- tid_t mapext(const object_t& oid, const object_locator_t& oloc,
+ ceph_tid_t mapext(const object_t& oid, const object_locator_t& oloc,
uint64_t off, uint64_t len, snapid_t snap, bufferlist *pbl, int flags,
Context *onfinish,
version_t *objver = NULL, ObjectOperation *extra_ops = NULL) {
o->outbl = pbl;
return op_submit(o);
}
- tid_t getxattr(const object_t& oid, const object_locator_t& oloc,
+ ceph_tid_t getxattr(const object_t& oid, const object_locator_t& oloc,
const char *name, snapid_t snap, bufferlist *pbl, int flags,
Context *onfinish,
version_t *objver = NULL, ObjectOperation *extra_ops = NULL) {
return op_submit(o);
}
- tid_t getxattrs(const object_t& oid, const object_locator_t& oloc, snapid_t snap,
+ ceph_tid_t getxattrs(const object_t& oid, const object_locator_t& oloc, snapid_t snap,
map<string,bufferlist>& attrset,
int flags, Context *onfinish,
version_t *objver = NULL, ObjectOperation *extra_ops = NULL) {
return op_submit(o);
}
- tid_t read_full(const object_t& oid, const object_locator_t& oloc,
+ ceph_tid_t read_full(const object_t& oid, const object_locator_t& oloc,
snapid_t snap, bufferlist *pbl, int flags,
Context *onfinish,
version_t *objver = NULL, ObjectOperation *extra_ops = NULL) {
// writes
- tid_t _modify(const object_t& oid, const object_locator_t& oloc,
+ ceph_tid_t _modify(const object_t& oid, const object_locator_t& oloc,
vector<OSDOp>& ops, utime_t mtime,
const SnapContext& snapc, int flags,
Context *onack, Context *oncommit,
o->snapc = snapc;
return op_submit(o);
}
- tid_t write(const object_t& oid, const object_locator_t& oloc,
+ ceph_tid_t write(const object_t& oid, const object_locator_t& oloc,
uint64_t off, uint64_t len, const SnapContext& snapc, const bufferlist &bl,
utime_t mtime, int flags,
Context *onack, Context *oncommit,
o->snapc = snapc;
return op_submit(o);
}
- tid_t append(const object_t& oid, const object_locator_t& oloc,
+ ceph_tid_t append(const object_t& oid, const object_locator_t& oloc,
uint64_t len, const SnapContext& snapc, const bufferlist &bl,
utime_t mtime, int flags,
Context *onack, Context *oncommit,
o->snapc = snapc;
return op_submit(o);
}
- tid_t write_trunc(const object_t& oid, const object_locator_t& oloc,
+ ceph_tid_t write_trunc(const object_t& oid, const object_locator_t& oloc,
uint64_t off, uint64_t len, const SnapContext& snapc, const bufferlist &bl,
utime_t mtime, int flags,
uint64_t trunc_size, __u32 trunc_seq,
o->snapc = snapc;
return op_submit(o);
}
- tid_t write_full(const object_t& oid, const object_locator_t& oloc,
+ ceph_tid_t write_full(const object_t& oid, const object_locator_t& oloc,
const SnapContext& snapc, const bufferlist &bl, utime_t mtime, int flags,
Context *onack, Context *oncommit,
version_t *objver = NULL, ObjectOperation *extra_ops = NULL) {
o->snapc = snapc;
return op_submit(o);
}
- tid_t trunc(const object_t& oid, const object_locator_t& oloc,
+ ceph_tid_t trunc(const object_t& oid, const object_locator_t& oloc,
const SnapContext& snapc,
utime_t mtime, int flags,
uint64_t trunc_size, __u32 trunc_seq,
o->snapc = snapc;
return op_submit(o);
}
- tid_t zero(const object_t& oid, const object_locator_t& oloc,
+ ceph_tid_t zero(const object_t& oid, const object_locator_t& oloc,
uint64_t off, uint64_t len, const SnapContext& snapc, utime_t mtime, int flags,
Context *onack, Context *oncommit,
version_t *objver = NULL, ObjectOperation *extra_ops = NULL) {
o->snapc = snapc;
return op_submit(o);
}
- tid_t rollback_object(const object_t& oid, const object_locator_t& oloc,
+ ceph_tid_t rollback_object(const object_t& oid, const object_locator_t& oloc,
const SnapContext& snapc, snapid_t snapid,
utime_t mtime, Context *onack, Context *oncommit,
version_t *objver = NULL, ObjectOperation *extra_ops = NULL) {
o->snapc = snapc;
return op_submit(o);
}
- tid_t create(const object_t& oid, const object_locator_t& oloc,
+ ceph_tid_t create(const object_t& oid, const object_locator_t& oloc,
const SnapContext& snapc, utime_t mtime,
int global_flags, int create_flags,
Context *onack, Context *oncommit,
o->snapc = snapc;
return op_submit(o);
}
- tid_t remove(const object_t& oid, const object_locator_t& oloc,
+ ceph_tid_t remove(const object_t& oid, const object_locator_t& oloc,
const SnapContext& snapc, utime_t mtime, int flags,
Context *onack, Context *oncommit,
version_t *objver = NULL, ObjectOperation *extra_ops = NULL) {
return op_submit(o);
}
- tid_t lock(const object_t& oid, const object_locator_t& oloc, int op, int flags,
+ ceph_tid_t lock(const object_t& oid, const object_locator_t& oloc, int op, int flags,
Context *onack, Context *oncommit, version_t *objver = NULL, ObjectOperation *extra_ops = NULL) {
SnapContext snapc; // no snapc for lock ops
vector<OSDOp> ops;
o->snapc = snapc;
return op_submit(o);
}
- tid_t setxattr(const object_t& oid, const object_locator_t& oloc,
+ ceph_tid_t setxattr(const object_t& oid, const object_locator_t& oloc,
const char *name, const SnapContext& snapc, const bufferlist &bl,
utime_t mtime, int flags,
Context *onack, Context *oncommit,
o->snapc = snapc;
return op_submit(o);
}
- tid_t removexattr(const object_t& oid, const object_locator_t& oloc,
+ ceph_tid_t removexattr(const object_t& oid, const object_locator_t& oloc,
const char *name, const SnapContext& snapc,
utime_t mtime, int flags,
Context *onack, Context *oncommit,
int change_pool_auid(int64_t pool, Context *onfinish, uint64_t auid);
void handle_pool_op_reply(MPoolOpReply *m);
- int pool_op_cancel(tid_t tid, int r);
+ int pool_op_cancel(ceph_tid_t tid, int r);
void finish_pool_op(PoolOp *op);
// --------------------------
void handle_get_pool_stats_reply(MGetPoolStatsReply *m);
void get_pool_stats(list<string>& pools, map<string,pool_stat_t> *result,
Context *onfinish);
- int pool_stat_op_cancel(tid_t tid, int r);
+ int pool_stat_op_cancel(ceph_tid_t tid, int r);
void finish_pool_stat_op(PoolStatOp *op);
// ---------------------------
public:
void handle_fs_stats_reply(MStatfsReply *m);
void get_fs_stats(struct ceph_statfs& result, Context *onfinish);
- int statfs_op_cancel(tid_t tid, int r);
+ int statfs_op_cancel(ceph_tid_t tid, int r);
void finish_statfs_op(StatfsOp *op);
// ---------------------------
* @param snapid read snapid
*/
virtual bool may_copy_on_write(const object_t& oid, uint64_t read_off, uint64_t read_len, snapid_t snapid) = 0;
- virtual tid_t write(const object_t& oid, const object_locator_t& oloc,
- uint64_t off, uint64_t len, const SnapContext& snapc,
- const bufferlist &bl, utime_t mtime, uint64_t trunc_size,
- __u32 trunc_seq, Context *oncommit) = 0;
- virtual tid_t lock(const object_t& oid, const object_locator_t& oloc, int op,
- int flags, Context *onack, Context *oncommit) {
+ virtual ceph_tid_t write(const object_t& oid, const object_locator_t& oloc,
+ uint64_t off, uint64_t len, const SnapContext& snapc,
+ const bufferlist &bl, utime_t mtime,
+ uint64_t trunc_size, __u32 trunc_seq,
+ Context *oncommit) = 0;
+ virtual ceph_tid_t lock(const object_t& oid, const object_locator_t& oloc,
+ int op, int flags, Context *onack, Context *oncommit) {
assert(0 == "this WritebackHandler does not support the lock operation");
}
};
m_finisher->queue(wrapper, len);
}
-tid_t FakeWriteback::write(const object_t& oid,
+ceph_tid_t FakeWriteback::write(const object_t& oid,
const object_locator_t& oloc,
uint64_t off, uint64_t len,
const SnapContext& snapc,
bufferlist *pbl, uint64_t trunc_size, __u32 trunc_seq,
Context *onfinish);
- virtual tid_t write(const object_t& oid, const object_locator_t& oloc,
- uint64_t off, uint64_t len, const SnapContext& snapc,
- const bufferlist &bl, utime_t mtime, uint64_t trunc_size,
- __u32 trunc_seq, Context *oncommit);
+ virtual ceph_tid_t write(const object_t& oid, const object_locator_t& oloc,
+ uint64_t off, uint64_t len,
+ const SnapContext& snapc, const bufferlist &bl,
+ utime_t mtime, uint64_t trunc_size,
+ __u32 trunc_seq, Context *oncommit);
virtual bool may_copy_on_write(const object_t&, uint64_t, uint64_t, snapid_t);
private: