}
}
- auto m = MClientSession::create(CEPH_SESSION_REQUEST_OPEN);
+ auto m = make_message<MClientSession>(CEPH_SESSION_REQUEST_OPEN);
m->metadata = metadata;
m->supported_features = feature_bitset_t(CEPHFS_FEATURES_CLIENT_SUPPORTED);
session->con->send_message2(std::move(m));
{
ldout(cct, 2) << __func__ << " mds." << s->mds_num << " seq " << s->seq << dendl;
s->state = MetaSession::STATE_CLOSING;
- s->con->send_message2(MClientSession::create(CEPH_SESSION_REQUEST_CLOSE, s->seq));
+ s->con->send_message2(make_message<MClientSession>(CEPH_SESSION_REQUEST_CLOSE, s->seq));
}
void Client::_closed_mds_session(MetaSession *s)
if (auto& m = session->release; m) {
session->con->send_message2(std::move(m));
}
- session->con->send_message2(MClientSession::create(CEPH_SESSION_FLUSHMSG_ACK, m->get_seq()));
+ session->con->send_message2(make_message<MClientSession>(CEPH_SESSION_FLUSHMSG_ACK, m->get_seq()));
break;
case CEPH_SESSION_FORCE_RO:
ref_t<MClientRequest> Client::build_client_request(MetaRequest *request)
{
- auto req = MClientRequest::create(request->get_op());
+ auto req = make_message<MClientRequest>(request->get_op());
req->set_tid(request->tid);
req->set_stamp(request->op_stamp);
memcpy(&req->head, &request->head, sizeof(ceph_mds_request_head));
early_kick_flushing_caps(session);
- auto m = MClientReconnect::create();
+ auto m = make_message<MClientReconnect>();
bool allow_multi = session->mds_features.test(CEPHFS_FEATURE_MULTI_RECONNECT);
// i have an open session.
m->mark_more();
session->con->send_message2(std::move(m));
- m = MClientReconnect::create();
+ m = make_message<MClientReconnect>();
}
Cap &cap = it->second;
s->seq++;
ldout(cct, 10) << " mds." << s->mds_num << " seq now " << s->seq << dendl;
if (s->state == MetaSession::STATE_CLOSING) {
- s->con->send_message2(MClientSession::create(CEPH_SESSION_REQUEST_CLOSE, s->seq));
+ s->con->send_message2(make_message<MClientSession>(CEPH_SESSION_REQUEST_CLOSE, s->seq));
}
}
revoke:
{
- auto reply = MClientLease::create(CEPH_MDS_LEASE_RELEASE, seq, m->get_mask(), m->get_ino(), m->get_first(), m->get_last(), m->dname);
+ auto reply = make_message<MClientLease>(CEPH_MDS_LEASE_RELEASE, seq,
+ m->get_mask(), m->get_ino(),
+ m->get_first(), m->get_last(), m->dname);
m->get_connection()->send_message2(std::move(reply));
}
}
if (flush)
follows = in->snaprealm->get_snap_context().seq;
- auto m = MClientCaps::create(op,
+ auto m = make_message<MClientCaps>(op,
in->ino,
0,
cap->cap_id, cap->seq,
session->flushing_caps_tids.insert(capsnap.flush_tid);
}
- auto m = MClientCaps::create(CEPH_CAP_OP_FLUSHSNAP, in->ino, in->snaprealm->ino, 0, mseq,
+ auto m = make_message<MClientCaps>(CEPH_CAP_OP_FLUSHSNAP, in->ino, in->snaprealm->ino, 0, mseq,
cap_epoch_barrier);
m->caller_uid = capsnap.cap_dirtier_uid;
m->caller_gid = capsnap.cap_dirtier_gid;
// will crash if they see an unknown CEPH_SESSION_* value in this msg.
const uint64_t features = session->con->get_features();
if (HAVE_FEATURE(features, SERVER_LUMINOUS)) {
- auto m = MClientSession::create(CEPH_SESSION_REQUEST_FLUSH_MDLOG);
+ auto m = make_message<MClientSession>(CEPH_SESSION_REQUEST_FLUSH_MDLOG);
session->con->send_message2(std::move(m));
}
}
ldout(cct, 10) << "renew_caps mds." << session->mds_num << dendl;
session->last_cap_renew_request = ceph_clock_now();
uint64_t seq = ++session->cap_renew_seq;
- session->con->send_message2(MClientSession::create(CEPH_SESSION_REQUEST_RENEWCAPS, seq));
+ session->con->send_message2(make_message<MClientSession>(CEPH_SESSION_REQUEST_RENEWCAPS, seq));
}
if (session->reclaim_state == MetaSession::RECLAIM_NULL ||
session->reclaim_state == MetaSession::RECLAIMING) {
session->reclaim_state = MetaSession::RECLAIMING;
- auto m = MClientReclaim::create(uuid, flags);
+ auto m = make_message<MClientReclaim>(uuid, flags);
session->con->send_message2(std::move(m));
wait_on_list(waiting_for_reclaim);
} else if (session->reclaim_state == MetaSession::RECLAIM_FAIL) {
for (auto &p : mds_sessions) {
p.second.reclaim_state = MetaSession::RECLAIM_NULL;
- auto m = MClientReclaim::create("", MClientReclaim::FLAG_FINISH);
+ auto m = make_message<MClientReclaim>("", MClientReclaim::FLAG_FINISH);
p.second.con->send_message2(std::move(m));
}
MCommand::ref get_message(const uuid_d &fsid) const
{
- auto m = MCommand::create(fsid);
+ auto m = make_message<MCommand>(fsid);
m->cmd = cmd;
m->set_data(inbl);
m->set_tid(tid);
seastar::logger& logger() {
return ceph::get_logger(ceph_subsys_osd);
}
-
- template<typename Message, typename... Args>
- Ref<Message> make_message(Args&&... args)
- {
- return {new Message{std::forward<Args>(args)...}, false};
- }
}
Heartbeat::Heartbeat(const OSDMapService& service,
seastar::logger& logger() {
return ceph::get_logger(ceph_subsys_osd);
}
-
- template<typename Message, typename... Args>
- Ref<Message> make_message(Args&&... args)
- {
- return {new Message{std::forward<Args>(args)...}, false};
- }
static constexpr int TICK_INTERVAL = 1;
}
seastar::logger& logger() {
return ceph::get_logger(ceph_subsys_osd);
}
- template<typename Message, typename... Args>
- Ref<Message> make_message(Args&&... args)
- {
- return {new Message{std::forward<Args>(args)...}, false};
- }
}
using recovery::AdvMap;
ceph_assert(want_state != MDSMap::STATE_NULL);
- auto beacon = MMDSBeacon::create(
+ auto beacon = make_message<MMDSBeacon>(
monc->get_fsid(), mds_gid_t(monc->get_global_id()),
name,
epoch,
mempool_inode *pi = get_projected_inode();
fnode_t *pf = dir->project_fnode();
- std::string_view ename = 0;
+ std::string_view ename;
switch (lock->get_type()) {
case CEPH_LOCK_IFILE:
pf->fragstat.version = pi->dirstat.version;
if (mds->is_cluster_degraded() &&
mds->mdsmap->get_state(it.first) < MDSMap::STATE_REJOIN)
continue;
- auto m = MLock::create(lock, msg, mds->get_nodeid());
+ auto m = make_message<MLock>(lock, msg, mds->get_nodeid());
mds->send_message_mds(m, it.first);
}
}
if (mds->is_cluster_degraded() &&
mds->mdsmap->get_state(it.first) < MDSMap::STATE_REJOIN)
continue;
- auto m = MLock::create(lock, msg, mds->get_nodeid());
+ auto m = make_message<MLock>(lock, msg, mds->get_nodeid());
m->set_data(data);
mds->send_message_mds(m, it.first);
}
return false;
}
- auto req = MMDSSlaveRequest::create(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_AUTHPIN);
+ auto req = make_message<MMDSSlaveRequest>(mdr->reqid, mdr->attempt,
+ MMDSSlaveRequest::OP_AUTHPIN);
for (set<MDSCacheObject*>::iterator q = p->second.begin();
q != p->second.end();
++q) {
if (!mds->is_cluster_degraded() ||
mds->mdsmap->get_state(*p) >= MDSMap::STATE_REJOIN) {
dout(10) << "_drop_non_rdlocks dropping remote locks on mds." << *p << dendl;
- auto slavereq = MMDSSlaveRequest::create(mut->reqid, mut->attempt, MMDSSlaveRequest::OP_DROPLOCKS);
+ auto slavereq = make_message<MMDSSlaveRequest>(mut->reqid, mut->attempt,
+ MMDSSlaveRequest::OP_DROPLOCKS);
mds->send_message_mds(slavereq, *p);
}
}
mds->mdsmap->get_state(auth) >= MDSMap::STATE_REJOIN) {
switch (lock->get_state()) {
case LOCK_SYNC_LOCK:
- mds->send_message_mds(MLock::create(lock, LOCK_AC_LOCKACK, mds->get_nodeid()), auth);
+ mds->send_message_mds(make_message<MLock>(lock, LOCK_AC_LOCKACK, mds->get_nodeid()), auth);
break;
case LOCK_MIX_SYNC:
{
- auto reply = MLock::create(lock, LOCK_AC_SYNCACK, mds->get_nodeid());
+ auto reply = make_message<MLock>(lock, LOCK_AC_SYNCACK, mds->get_nodeid());
lock->encode_locked_state(reply->get_data());
mds->send_message_mds(reply, auth);
next = LOCK_MIX_SYNC2;
case LOCK_SYNC_MIX:
{
- auto reply = MLock::create(lock, LOCK_AC_MIXACK, mds->get_nodeid());
+ auto reply = make_message<MLock>(lock, LOCK_AC_MIXACK, mds->get_nodeid());
mds->send_message_mds(reply, auth);
next = LOCK_SYNC_MIX2;
}
{
bufferlist data;
lock->encode_locked_state(data);
- mds->send_message_mds(MLock::create(lock, LOCK_AC_LOCKACK, mds->get_nodeid(), data), auth);
+ mds->send_message_mds(make_message<MLock>(lock, LOCK_AC_LOCKACK, mds->get_nodeid(), data), auth);
(static_cast<ScatterLock *>(lock))->start_flush();
// we'll get an AC_LOCKFLUSHED to complete
}
mds->mdsmap->is_clientreplay_or_active_or_stopping(auth)) {
dout(10) << "requesting rdlock from auth on "
<< *lock << " on " << *lock->get_parent() << dendl;
- mds->send_message_mds(MLock::create(lock, LOCK_AC_REQRDLOCK, mds->get_nodeid()), auth);
+ mds->send_message_mds(make_message<MLock>(lock, LOCK_AC_REQRDLOCK, mds->get_nodeid()), auth);
}
return false;
}
mds->mdsmap->is_clientreplay_or_active_or_stopping(auth)) {
dout(10) << "requesting scatter from auth on "
<< *lock << " on " << *lock->get_parent() << dendl;
- mds->send_message_mds(MLock::create(lock, LOCK_AC_REQSCATTER, mds->get_nodeid()), auth);
+ mds->send_message_mds(make_message<MLock>(lock, LOCK_AC_REQSCATTER, mds->get_nodeid()), auth);
}
break;
}
// send lock request
mut->start_locking(lock, target);
mut->more()->slaves.insert(target);
- auto r = MMDSSlaveRequest::create(mut->reqid, mut->attempt, MMDSSlaveRequest::OP_WRLOCK);
+ auto r = make_message<MMDSSlaveRequest>(mut->reqid, mut->attempt, MMDSSlaveRequest::OP_WRLOCK);
r->set_lock_type(lock->get_type());
lock->get_parent()->set_object_info(r->get_object_info());
mds->send_message_mds(r, target);
<< " " << *lock->get_parent() << dendl;
if (!mds->is_cluster_degraded() ||
mds->mdsmap->get_state(target) >= MDSMap::STATE_REJOIN) {
- auto slavereq = MMDSSlaveRequest::create(mut->reqid, mut->attempt, MMDSSlaveRequest::OP_UNWRLOCK);
+ auto slavereq = make_message<MMDSSlaveRequest>(mut->reqid, mut->attempt, MMDSSlaveRequest::OP_UNWRLOCK);
slavereq->set_lock_type(lock->get_type());
lock->get_parent()->set_object_info(slavereq->get_object_info());
mds->send_message_mds(slavereq, target);
// send lock request
mut->more()->slaves.insert(auth);
mut->start_locking(lock, auth);
- auto r = MMDSSlaveRequest::create(mut->reqid, mut->attempt, MMDSSlaveRequest::OP_XLOCK);
+ auto r = make_message<MMDSSlaveRequest>(mut->reqid, mut->attempt, MMDSSlaveRequest::OP_XLOCK);
r->set_lock_type(lock->get_type());
lock->get_parent()->set_object_info(r->get_object_info());
mds->send_message_mds(r, auth);
mds_rank_t auth = lock->get_parent()->authority().first;
if (!mds->is_cluster_degraded() ||
mds->mdsmap->get_state(auth) >= MDSMap::STATE_REJOIN) {
- auto slavereq = MMDSSlaveRequest::create(mut->reqid, mut->attempt, MMDSSlaveRequest::OP_UNXLOCK);
+ auto slavereq = make_message<MMDSSlaveRequest>(mut->reqid, mut->attempt, MMDSSlaveRequest::OP_UNXLOCK);
slavereq->set_lock_type(lock->get_type());
lock->get_parent()->set_object_info(slavereq->get_object_info());
mds->send_message_mds(slavereq, auth);
cap->reset_num_revoke_warnings();
}
- auto m = MClientCaps::create(op, in->ino(),
+ auto m = make_message<MClientCaps>(op, in->ino(),
in->find_snaprealm()->inode->ino(),
cap->get_cap_id(),
cap->get_last_seq(),
for (auto &p : in->client_caps) {
Capability *cap = &p.second;
- auto m = MClientCaps::create(CEPH_CAP_OP_TRUNC,
+ auto m = make_message<MClientCaps>(CEPH_CAP_OP_TRUNC,
in->ino(),
in->find_snaprealm()->inode->ino(),
cap->get_cap_id(), cap->get_last_seq(),
if (!mds->is_cluster_degraded() ||
mds->mdsmap->is_clientreplay_or_active_or_stopping(auth))
- mds->send_message_mds(MInodeFileCaps::create(in->ino(), in->replica_caps_wanted), auth);
+ mds->send_message_mds(make_message<MInodeFileCaps>(in->ino(), in->replica_caps_wanted), auth);
}
}
if (cap->pending() & (CEPH_CAP_FILE_WR|CEPH_CAP_FILE_BUFFER)) {
dout(10) << "share_inode_max_size with client." << client << dendl;
cap->inc_last_seq();
- auto m = MClientCaps::create(CEPH_CAP_OP_GRANT,
+ auto m = make_message<MClientCaps>(CEPH_CAP_OP_GRANT,
in->ino(),
in->find_snaprealm()->inode->ino(),
cap->get_cap_id(),
<< " for client." << client << dendl;
ref_t<MClientCaps> ack;
if (op == CEPH_CAP_OP_FLUSHSNAP) {
- ack = MClientCaps::create(CEPH_CAP_OP_FLUSHSNAP_ACK, m->get_ino(), 0, 0, 0, 0, 0, dirty, 0, mds->get_osd_epoch_barrier());
+ ack = make_message<MClientCaps>(CEPH_CAP_OP_FLUSHSNAP_ACK, m->get_ino(), 0, 0, 0, 0, 0, dirty, 0, mds->get_osd_epoch_barrier());
} else {
- ack = MClientCaps::create(CEPH_CAP_OP_FLUSH_ACK, m->get_ino(), 0, m->get_cap_id(), m->get_seq(), m->get_caps(), 0, dirty, 0, mds->get_osd_epoch_barrier());
+ ack = make_message<MClientCaps>(CEPH_CAP_OP_FLUSH_ACK, m->get_ino(), 0, m->get_cap_id(), m->get_seq(), m->get_caps(), 0, dirty, 0, mds->get_osd_epoch_barrier());
}
ack->set_snap_follows(follows);
ack->set_client_tid(m->get_client_tid());
// case we get a dup response, so whatever.)
ref_t<MClientCaps> ack;
if (dirty) {
- ack = MClientCaps::create(CEPH_CAP_OP_FLUSHSNAP_ACK, in->ino(), 0, 0, 0, 0, 0, dirty, 0, mds->get_osd_epoch_barrier());
+ ack = make_message<MClientCaps>(CEPH_CAP_OP_FLUSHSNAP_ACK, in->ino(), 0, 0, 0, 0, 0, dirty, 0, mds->get_osd_epoch_barrier());
ack->set_snap_follows(follows);
ack->set_client_tid(m->get_client_tid());
ack->set_oldest_flush_tid(m->get_oldest_flush_tid());
if (dirty && in->is_auth()) {
dout(7) << " flush client." << client << " dirty " << ccap_string(dirty)
<< " seq " << m->get_seq() << " on " << *in << dendl;
- ack = MClientCaps::create(CEPH_CAP_OP_FLUSH_ACK, in->ino(), 0, cap->get_cap_id(), m->get_seq(),
+ ack = make_message<MClientCaps>(CEPH_CAP_OP_FLUSH_ACK, in->ino(), 0, cap->get_cap_id(), m->get_seq(),
m->get_caps(), 0, dirty, 0, mds->get_osd_epoch_barrier());
ack->set_client_tid(m->get_client_tid());
ack->set_oldest_flush_tid(m->get_oldest_flush_tid());
dout(7) << "handle_client_lease client." << client << " renew on " << *dn
<< (!dn->lock.can_lease(client)?", revoking lease":"") << dendl;
if (dn->lock.can_lease(client)) {
- auto reply = MClientLease::create(*m);
+ auto reply = make_message<MClientLease>(*m);
int pool = 1; // fixme.. do something smart!
reply->h.duration_ms = (int)(1000 * mdcache->client_lease_durations[pool]);
reply->h.seq = ++l->seq;
// i should also revoke the dir ICONTENT lease, if they have it!
CInode *diri = dn->get_dir()->get_inode();
- auto lease = MClientLease::create(CEPH_MDS_LEASE_REVOKE, l->seq, mask, diri->ino(), diri->first, CEPH_NOSNAP, dn->get_name());
+ auto lease = make_message<MClientLease>(CEPH_MDS_LEASE_REVOKE, l->seq, mask, diri->ino(), diri->first, CEPH_NOSNAP, dn->get_name());
mds->send_message_client_counted(lease, l->client);
}
}
// request unscatter?
mds_rank_t auth = lock->get_parent()->authority().first;
if (!mds->is_cluster_degraded() || mds->mdsmap->is_clientreplay_or_active_or_stopping(auth)) {
- mds->send_message_mds(MLock::create(lock, LOCK_AC_NUDGE, mds->get_nodeid()), auth);
+ mds->send_message_mds(make_message<MLock>(lock, LOCK_AC_NUDGE, mds->get_nodeid()), auth);
}
// wait...
for (const auto& r : up) {
if (r == mds->get_nodeid())
continue;
- auto hb = MHeartbeat::create(load, beat_epoch);
+ auto hb = make_message<MHeartbeat>(load, beat_epoch);
hb->get_import_map() = import_map;
mds->send_message_mds(hb, r);
}
cap->last_rsize = i->rstat.rsize();
cap->last_rbytes = i->rstat.rbytes;
- auto msg = MClientQuota::create();
+ auto msg = make_message<MClientQuota>();
msg->ino = in->ino();
msg->rstat = i->rstat;
msg->quota = i->quota;
mds->send_message_client_counted(msg, cap->get_session());
}
for (const auto &it : in->get_replicas()) {
- auto msg = MGatherCaps::create();
+ auto msg = make_message<MGatherCaps>();
msg->ino = in->ino();
mds->send_message_mds(msg, it.first);
}
dout(10) << "_logged_slave_commit from mds." << from << " " << reqid << dendl;
// send a message
- auto req = MMDSSlaveRequest::create(reqid, 0, MMDSSlaveRequest::OP_COMMITTED);
+ auto req = make_message<MMDSSlaveRequest>(reqid, 0, MMDSSlaveRequest::OP_COMMITTED);
mds->send_message_mds(req, from);
}
for (map<mds_rank_t, map<metareqid_t, MDSlaveUpdate*> >::iterator p = uncommitted_slave_updates.begin();
p != uncommitted_slave_updates.end();
++p) {
- resolves[p->first] = MMDSResolve::create();
+ resolves[p->first] = make_message<MMDSResolve>();
for (map<metareqid_t, MDSlaveUpdate*>::iterator q = p->second.begin();
q != p->second.end();
++q) {
if (resolve_set.count(master) || is_ambiguous_slave_update(p->first, master)) {
dout(10) << " including uncommitted " << *mdr << dendl;
if (!resolves.count(master))
- resolves[master] = MMDSResolve::create();
+ resolves[master] = make_message<MMDSResolve>();
if (!mdr->committing &&
mdr->has_more() && mdr->more()->is_inode_exporter) {
// re-send cap exports
if (*p == mds->get_nodeid())
continue;
if (mds->is_resolve() || mds->mdsmap->is_resolve(*p))
- resolves[*p] = MMDSResolve::create();
+ resolves[*p] = make_message<MMDSResolve>();
}
map<dirfrag_t, vector<dirfrag_t> > my_subtrees;
}
}
- auto ack = MMDSResolveAck::create();
+ auto ack = make_message<MMDSResolveAck>();
for (const auto &p : m->slave_requests) {
if (uncommitted_masters.count(p.first)) { //mds->sessionmap.have_completed_request(p.first)) {
// COMMIT
if (*p == mds->get_nodeid()) continue; // nothing to myself!
if (rejoin_sent.count(*p)) continue; // already sent a rejoin to this node!
if (mds->is_rejoin())
- rejoins[*p] = MMDSCacheRejoin::create(MMDSCacheRejoin::OP_WEAK);
+ rejoins[*p] = make_message<MMDSCacheRejoin>(MMDSCacheRejoin::OP_WEAK);
else if (mds->mdsmap->is_rejoin(*p))
- rejoins[*p] = MMDSCacheRejoin::create(MMDSCacheRejoin::OP_STRONG);
+ rejoins[*p] = make_message<MMDSCacheRejoin>(MMDSCacheRejoin::OP_STRONG);
}
if (mds->is_rejoin()) {
if (mds->is_clientreplay() || mds->is_active() || mds->is_stopping()) {
survivor = true;
dout(10) << "i am a surivivor, and will ack immediately" << dendl;
- ack = MMDSCacheRejoin::create(MMDSCacheRejoin::OP_ACK);
+ ack = make_message<MMDSCacheRejoin>(MMDSCacheRejoin::OP_ACK);
map<inodeno_t,map<client_t,Capability::Import> > imported_caps;
}
// mark client caps stale.
- auto m = MClientCaps::create(CEPH_CAP_OP_EXPORT, p->first, 0,
+ auto m = make_message<MClientCaps>(CEPH_CAP_OP_EXPORT, p->first, 0,
r->second.capinfo.cap_id, 0,
mds->get_osd_epoch_barrier());
m->set_cap_peer(q->second.cap_id, q->second.issue_seq, q->second.mseq,
snap = it->second;
snap->head.op = CEPH_SNAP_OP_SPLIT;
} else {
- snap = MClientSnap::create(CEPH_SNAP_OP_SPLIT);
+ snap = make_message<MClientSnap>(CEPH_SNAP_OP_SPLIT);
splits.emplace(std::piecewise_construct, std::forward_as_tuple(client), std::forward_as_tuple(snap));
snap->head.split = realm->inode->ino();
snap->bl = realm->get_snap_trace();
ceph_assert(!p.second->empty());
auto em = splits.emplace(std::piecewise_construct, std::forward_as_tuple(p.first), std::forward_as_tuple());
if (em.second) {
- auto update = MClientSnap::create(CEPH_SNAP_OP_SPLIT);
+ auto update = make_message<MClientSnap>(CEPH_SNAP_OP_SPLIT);
update->head.split = parent_realm->inode->ino();
update->split_inos = split_inos;
update->split_realms = split_realms;
Session *session = mds->sessionmap.get_session(entity_name_t::CLIENT(q->first.v));
if (session) {
// mark client caps stale.
- auto stale = MClientCaps::create(CEPH_CAP_OP_EXPORT, p->first, 0, 0, 0, mds->get_osd_epoch_barrier());
+ auto stale = make_message<MClientCaps>(CEPH_CAP_OP_EXPORT, p->first,
+ 0, 0, 0,
+ mds->get_osd_epoch_barrier());
stale->set_cap_peer(0, 0, 0, -1, 0);
mds->send_message_client_counted(stale, q->first);
}
cap->set_last_issue();
cap->set_last_issue_stamp(ceph_clock_now());
cap->clear_new();
- auto reap = MClientCaps::create(CEPH_CAP_OP_IMPORT, in->ino(), realm->inode->ino(), cap->get_cap_id(), cap->get_last_seq(), cap->pending(), cap->wanted(), 0, cap->get_mseq(), mds->get_osd_epoch_barrier());
+ auto reap = make_message<MClientCaps>(
+ CEPH_CAP_OP_IMPORT, in->ino(), realm->inode->ino(), cap->get_cap_id(),
+ cap->get_last_seq(), cap->pending(), cap->wanted(), 0, cap->get_mseq(),
+ mds->get_osd_epoch_barrier());
in->encode_cap_message(reap, cap);
reap->snapbl = realm->get_snap_trace();
reap->set_cap_peer(p_cap_id, p_seq, p_mseq, peer, p_flags);
if (seq < realm->get_newest_seq()) {
dout(10) << "finish_snaprealm_reconnect client." << client << " has old seq " << seq << " < "
<< realm->get_newest_seq() << " on " << *realm << dendl;
- auto snap = MClientSnap::create(CEPH_SNAP_OP_UPDATE);
+ auto snap = make_message<MClientSnap>(CEPH_SNAP_OP_UPDATE);
snap->bl = realm->get_snap_trace();
for (const auto& child : realm->open_children)
snap->split_realms.push_back(child->inode->ino());
++p) {
if (rejoin_ack_sent.count(*p))
continue;
- acks[*p] = MMDSCacheRejoin::create(MMDSCacheRejoin::OP_ACK);
+ acks[*p] = make_message<MMDSCacheRejoin>(MMDSCacheRejoin::OP_ACK);
}
rejoin_ack_sent = recovery_set;
auto em = expiremap.emplace(std::piecewise_construct, std::forward_as_tuple(rank), std::forward_as_tuple());
if (em.second) {
- em.first->second = MCacheExpire::create(mds->get_nodeid());
+ em.first->second = make_message<MCacheExpire>(mds->get_nodeid());
}
dout(20) << __func__ << ": try expiring " << *mdsdir_in << " for stopping mds." << mds << dendl;
ceph_assert(a != mds->get_nodeid());
auto em = expiremap.emplace(std::piecewise_construct, std::forward_as_tuple(a), std::forward_as_tuple());
if (em.second)
- em.first->second = MCacheExpire::create(mds->get_nodeid());
+ em.first->second = make_message<MCacheExpire>(mds->get_nodeid());
em.first->second->add_dentry(con->dirfrag(), dir->dirfrag(), dn->get_name(), dn->last, dn->get_replica_nonce());
}
}
ceph_assert(a != mds->get_nodeid());
auto em = expiremap.emplace(std::piecewise_construct, std::forward_as_tuple(a), std::forward_as_tuple());
if (em.second)
- em.first->second = MCacheExpire::create(mds->get_nodeid()); /* new */
+ em.first->second = make_message<MCacheExpire>(mds->get_nodeid()); /* new */
em.first->second->add_dir(condf, dir->dirfrag(), dir->replica_nonce);
}
}
ceph_assert(a != mds->get_nodeid());
auto em = expiremap.emplace(std::piecewise_construct, std::forward_as_tuple(a), std::forward_as_tuple());
if (em.second)
- em.first->second = MCacheExpire::create(mds->get_nodeid()); /* new */
+ em.first->second = make_message<MCacheExpire>(mds->get_nodeid()); /* new */
em.first->second->add_inode(df, in->vino(), in->get_replica_nonce());
}
}
auto em = delayed_expire[parent_dir].emplace(std::piecewise_construct, std::forward_as_tuple(from), std::forward_as_tuple());
if (em.second)
- em.first->second = MCacheExpire::create(from); /* new */
+ em.first->second = make_message<MCacheExpire>(from); /* new */
// merge these expires into it
em.first->second->add_realm(p.first, p.second);
// got backtrace from peer or backtrace just fetched
if (info.discover || !info.fetch_backtrace)
pa = &info.ancestors;
- mds->send_message_mds(MMDSOpenIno::create(info.tid, ino, pa), peer);
+ mds->send_message_mds(make_message<MMDSOpenIno>(info.tid, ino, pa), peer);
if (mds->logger)
mds->logger->inc(l_mds_openino_peer_discover);
}
CInode *in = get_inode(ino);
if (in) {
dout(10) << " have " << *in << dendl;
- reply = MMDSOpenInoReply::create(m->get_tid(), ino, mds_rank_t(0));
+ reply = make_message<MMDSOpenInoReply>(m->get_tid(), ino, mds_rank_t(0));
if (in->is_auth()) {
touch_inode(in);
while (1) {
reply->hint = in->authority().first;
}
} else if (err < 0) {
- reply = MMDSOpenInoReply::create(m->get_tid(), ino, MDS_RANK_NONE, err);
+ reply = make_message<MMDSOpenInoReply>(m->get_tid(), ino, MDS_RANK_NONE, err);
} else {
mds_rank_t hint = MDS_RANK_NONE;
int ret = open_ino_traverse_dir(ino, m, m->ancestors, false, false, &hint);
if (ret > 0)
return;
- reply = MMDSOpenInoReply::create(m->get_tid(), ino, hint, ret);
+ reply = make_message<MMDSOpenInoReply>(m->get_tid(), ino, hint, ret);
}
mds->send_message_mds(reply, from);
}
}
} else {
fip.checking = m;
- mds->send_message_mds(MMDSFindIno::create(fip.tid, fip.ino), m);
+ mds->send_message_mds(make_message<MMDSFindIno>(fip.tid, fip.ino), m);
}
}
}
dout(10) << "handle_find_ino " << *m << dendl;
- auto r = MMDSFindInoReply::create(m->tid);
+ auto r = make_message<MMDSFindInoReply>(m->tid);
CInode *in = get_inode(m->ino);
if (in) {
in->make_path(r->path);
for (set<mds_rank_t>::iterator p = mdr->more()->slaves.begin();
p != mdr->more()->slaves.end();
++p) {
- auto r = MMDSSlaveRequest::create(mdr->reqid, mdr->attempt,
- MMDSSlaveRequest::OP_FINISH);
+ auto r = make_message<MMDSSlaveRequest>(mdr->reqid, mdr->attempt,
+ MMDSSlaveRequest::OP_FINISH);
if (mdr->killed && !mdr->committing) {
r->mark_abort();
auto em = updates.emplace(std::piecewise_construct, std::forward_as_tuple(client), std::forward_as_tuple());
if (em.second) {
- auto update = MClientSnap::create(CEPH_SNAP_OP_SPLIT);
+ auto update = make_message<MClientSnap>(CEPH_SNAP_OP_SPLIT);
update->head.split = in->ino();
update->split_inos = split_inos;
update->split_realms = split_realms;
in->encode_snap(snap_blob);
for (auto p : mds_set) {
- auto m = MMDSSnapUpdate::create(in->ino(), stid, snap_op);
+ auto m = make_message<MMDSSnapUpdate>(in->ino(), stid, snap_op);
m->snap_blob = snap_blob;
mds->send_message_mds(m, p);
}
for (auto &session : sessions) {
if (!session->is_open() && !session->is_stale())
continue;
- auto update = MClientSnap::create(snap_op);
+ auto update = make_message<MClientSnap>(snap_op);
update->head.split = global_snaprealm->inode->ino();
update->bl = global_snaprealm->get_snap_trace();
mds->send_message_client_counted(update, session);
void MDCache::_send_discover(discover_info_t& d)
{
- auto dis = MDiscover::create(d.ino, d.frag, d.snap, d.want_path, d.want_base_dir, d.want_xlocked);
+ auto dis = make_message<MDiscover>(d.ino, d.frag, d.snap, d.want_path,
+ d.want_base_dir, d.want_xlocked);
dis->set_tid(d.tid);
mds->send_message_mds(dis, d.mds);
}
CInode *cur = 0;
- auto reply = MDiscoverReply::create(*dis);
+ auto reply = make_message<MDiscoverReply>(*dis);
snapid_t snapid = dis->get_snapid();
for (const auto &r : dir->dir_rep_by) {
s.insert(r);
}
- mds->send_message_mds(MDirUpdate::create(mds->get_nodeid(), dir->dirfrag(), dir->dir_rep, s, path, bcast), *it);
+ mds->send_message_mds(make_message<MDirUpdate>(mds->get_nodeid(), dir->dirfrag(), dir->dir_rep, s, path, bcast), *it);
}
return 0;
rejoin_gather.count(p.first)))
continue;
CDentry::linkage_t *dnl = dn->get_linkage();
- auto m = MDentryLink::create(subtree->dirfrag(), dn->get_dir()->dirfrag(), dn->get_name(), dnl->is_primary());
+ auto m = make_message<MDentryLink>(subtree->dirfrag(), dn->get_dir()->dirfrag(), dn->get_name(), dnl->is_primary());
if (dnl->is_primary()) {
dout(10) << " primary " << *dnl->get_inode() << dendl;
replicate_inode(dnl->get_inode(), p.first, m->bl,
rejoin_gather.count(*it)))
continue;
- auto unlink = MDentryUnlink::create(dn->get_dir()->dirfrag(), dn->get_name());
+ auto unlink = make_message<MDentryUnlink>(dn->get_dir()->dirfrag(), dn->get_name());
if (straydn) {
replicate_stray(straydn, *it, unlink->straybl);
unlink->snapbl = snapbl;
rejoin_gather.count(p.first)))
continue;
- auto notify = MMDSFragmentNotify::create(basedirfrag, info.bits, mdr->reqid.tid);
+ auto notify = make_message<MMDSFragmentNotify>(basedirfrag, info.bits, mdr->reqid.tid);
if (diri_auth != CDIR_AUTH_UNKNOWN && // subtree root
diri_auth != p.first) { // not auth mds of diri
/*
}
if (notify->is_ack_wanted()) {
- auto ack = MMDSFragmentNotifyAck::create(notify->get_base_dirfrag(),
+ auto ack = make_message<MMDSFragmentNotifyAck>(notify->get_base_dirfrag(),
notify->get_bits(), notify->get_tid());
mds->send_message_mds(ack, from);
}
}
priv.reset();
- auto reply = MCommandReply::create(r, outs);
+ auto reply = make_message<MCommandReply>(r, outs);
reply->set_tid(m->get_tid());
reply->set_data(outbl);
m->get_connection()->send_message2(reply);
if (send) {
dout(15) << "updating export_targets, now " << new_map_targets.size() << " ranks are targets" << dendl;
- auto m = MMDSLoadTargets::create(mds_gid_t(monc->get_global_id()), new_map_targets);
+ auto m = make_message<MMDSLoadTargets>(mds_gid_t(monc->get_global_id()), new_map_targets);
monc->send_mon_message(m.detach());
}
}
// send mdsmap first?
if (mds != whoami && peer_mdsmap_epoch[mds] < mdsmap->get_epoch()) {
- auto _m = MMDSMap::create(monc->get_fsid(), *mdsmap);
+ auto _m = make_message<MMDSMap>(monc->get_fsid(), *mdsmap);
messenger->send_to_mds(_m.detach(), mdsmap->get_addrs(mds));
peer_mdsmap_epoch[mds] = mdsmap->get_epoch();
}
// tell the client where it should go
auto session = get_session(m);
- auto f = MClientRequestForward::create(m->get_tid(), mds, m->get_num_fwd()+1, client_must_resend);
+ auto f = make_message<MClientRequestForward>(m->get_tid(), mds, m->get_num_fwd()+1, client_must_resend);
send_message_client(f, session);
}
set<Session*> clients;
sessionmap.get_client_session_set(clients);
for (const auto &session : clients) {
- auto m = MMDSMap::create(monc->get_fsid(), *mdsmap);
+ auto m = make_message<MMDSMap>(monc->get_fsid(), *mdsmap);
session->get_connection()->send_message2(std::move(m));
}
last_client_mdsmap_bcast = mdsmap->get_epoch();
dout(10) << "stray agree on " << reqid << " tid " << tid
<< ", sending ROLLBACK" << dendl;
ceph_assert(!server_ready);
- auto req = MMDSTableRequest::create(table, TABLESERVER_OP_ROLLBACK, 0, tid);
+ auto req = make_message<MMDSTableRequest>(table, TABLESERVER_OP_ROLLBACK, 0, tid);
mds->send_message_mds(req, mds->get_mds_map()->get_tableserver());
}
break;
if (server_ready) {
// send message
- auto req = MMDSTableRequest::create(table, TABLESERVER_OP_PREPARE, reqid);
+ auto req = make_message<MMDSTableRequest>(table, TABLESERVER_OP_PREPARE, reqid);
req->bl = mutation;
mds->send_message_mds(req, mds->get_mds_map()->get_tableserver());
} else
if (server_ready) {
// send message
- auto req = MMDSTableRequest::create(table, TABLESERVER_OP_COMMIT, 0, tid);
+ auto req = make_message<MMDSTableRequest>(table, TABLESERVER_OP_COMMIT, 0, tid);
mds->send_message_mds(req, mds->get_mds_map()->get_tableserver());
} else
dout(10) << "tableserver is not ready yet, deferring request" << dendl;
p != pending_commit.end();
++p) {
dout(10) << "resending commit on " << p->first << dendl;
- auto req = MMDSTableRequest::create(table, TABLESERVER_OP_COMMIT, 0, p->first);
+ auto req = make_message<MMDSTableRequest>(table, TABLESERVER_OP_COMMIT, 0, p->first);
mds->send_message_mds(req, mds->get_mds_map()->get_tableserver());
}
}
p != pending_prepare.end();
++p) {
dout(10) << "resending prepare on " << p->first << dendl;
- auto req = MMDSTableRequest::create(table, TABLESERVER_OP_PREPARE, p->first);
+ auto req = make_message<MMDSTableRequest>(table, TABLESERVER_OP_PREPARE, p->first);
req->bl = p->second.mutation;
mds->send_message_mds(req, mds->get_mds_map()->get_tableserver());
}
_prepare(req->bl, req->reqid, from, out);
ceph_assert(version == tid);
- auto reply = MMDSTableRequest::create(table, TABLESERVER_OP_AGREE, req->reqid, tid);
+ auto reply = make_message<MMDSTableRequest>(table, TABLESERVER_OP_AGREE, req->reqid, tid);
reply->bl = std::move(out);
if (_notify_prep(tid)) {
else if (tid <= version) {
dout(0) << "got commit for tid " << tid << " <= " << version
<< ", already committed, sending ack." << dendl;
- auto reply = MMDSTableRequest::create(table, TABLESERVER_OP_ACK, req->reqid, tid);
+ auto reply = make_message<MMDSTableRequest>(table, TABLESERVER_OP_ACK, req->reqid, tid);
mds->send_message(reply, req->get_connection());
}
else {
_commit(tid, req);
_note_commit(tid);
- auto reply = MMDSTableRequest::create(table, TABLESERVER_OP_ACK, req->reqid, req->get_tid());
+ auto reply = make_message<MMDSTableRequest>(table, TABLESERVER_OP_ACK, req->reqid, req->get_tid());
mds->send_message_mds(reply, mds_rank_t(req->get_source().num()));
}
next_reqids[who] = p.second.reqid + 1;
version_t tid = p.second.tid;
- auto reply = MMDSTableRequest::create(table, TABLESERVER_OP_AGREE, p.second.reqid, tid);
+ auto reply = make_message<MMDSTableRequest>(table, TABLESERVER_OP_AGREE, p.second.reqid, tid);
_get_reply_buffer(tid, &reply->bl);
mds->send_message_mds(reply, who);
}
for (auto p : active_clients) {
- auto reply = MMDSTableRequest::create(table, TABLESERVER_OP_SERVER_READY, next_reqids[p]);
+ auto reply = make_message<MMDSTableRequest>(table, TABLESERVER_OP_SERVER_READY, next_reqids[p]);
mds->send_message_mds(reply, p);
}
recovered = true;
if (p->second.reqid >= next_reqid)
next_reqid = p->second.reqid + 1;
- auto reply = MMDSTableRequest::create(table, TABLESERVER_OP_AGREE, p->second.reqid, p->second.tid);
+ auto reply = make_message<MMDSTableRequest>(table, TABLESERVER_OP_AGREE, p->second.reqid, p->second.tid);
_get_reply_buffer(p->second.tid, &reply->bl);
mds->send_message_mds(reply, who);
}
- auto reply = MMDSTableRequest::create(table, TABLESERVER_OP_SERVER_READY, next_reqid);
+ auto reply = make_message<MMDSTableRequest>(table, TABLESERVER_OP_SERVER_READY, next_reqid);
mds->send_message_mds(reply, who);
}
if (notify_peer &&
(!mds->is_cluster_degraded() ||
mds->mdsmap->is_clientreplay_or_active_or_stopping(it->second.peer))) // tell them.
- mds->send_message_mds(MExportDirCancel::create(dir->dirfrag(), it->second.tid), it->second.peer);
+ mds->send_message_mds(make_message<MExportDirCancel>(dir->dirfrag(),
+ it->second.tid),
+ it->second.peer);
break;
case EXPORT_FREEZING:
if (notify_peer &&
(!mds->is_cluster_degraded() ||
mds->mdsmap->is_clientreplay_or_active_or_stopping(it->second.peer))) // tell them.
- mds->send_message_mds(MExportDirCancel::create(dir->dirfrag(), it->second.tid), it->second.peer);
+ mds->send_message_mds(make_message<MExportDirCancel>(dir->dirfrag(),
+ it->second.tid),
+ it->second.peer);
break;
// NOTE: state order reversal, warning comes after prepping
if (notify_peer &&
(!mds->is_cluster_degraded() ||
mds->mdsmap->is_clientreplay_or_active_or_stopping(it->second.peer))) // tell them.
- mds->send_message_mds(MExportDirCancel::create(dir->dirfrag(), it->second.tid), it->second.peer);
+ mds->send_message_mds(make_message<MExportDirCancel>(dir->dirfrag(),
+ it->second.tid),
+ it->second.peer);
break;
case EXPORT_EXPORTING:
// send ExportDirDiscover (ask target)
filepath path;
dir->inode->make_path(path);
- auto discover = MExportDirDiscover::create(dir->dirfrag(), path,
- mds->get_nodeid(), it->second.tid);
+ auto discover = make_message<MExportDirDiscover>(dir->dirfrag(), path,
+ mds->get_nodeid(),
+ it->second.tid);
mds->send_message_mds(discover, dest);
ceph_assert(g_conf()->mds_kill_export_at != 2);
cache->get_subtree_bounds(dir, bounds);
// generate prep message, log entry.
- auto prep = MExportDirPrep::create(dir->dirfrag(), it->second.tid);
+ auto prep = make_message<MExportDirPrep>(dir->dirfrag(), it->second.tid);
// include list of bystanders
for (const auto &p : dir->get_replicas()) {
it->second.warning_ack_waiting.insert(p.first);
it->second.notify_ack_waiting.insert(p.first); // we'll eventually get a notifyack, too!
- auto notify = MExportDirNotify::create(dir->dirfrag(), it->second.tid, true,
+ auto notify = make_message<MExportDirNotify>(dir->dirfrag(), it->second.tid, true,
mds_authority_t(mds->get_nodeid(),CDIR_AUTH_UNKNOWN),
mds_authority_t(mds->get_nodeid(),it->second.peer));
for (auto &cdir : bounds) {
mds->balancer->subtract_export(dir);
// fill export message with cache data
- auto req = MExportDir::create(dir->dirfrag(), it->second.tid);
+ auto req = make_message<MExportDir>(dir->dirfrag(), it->second.tid);
map<client_t,entity_inst_t> exported_client_map;
map<client_t,client_metadata_t> exported_client_metadata_map;
uint64_t num_exported_inodes = encode_export_dir(req->export_data,
const Capability *cap = &p.second;
dout(7) << "finish_export_inode_caps telling client." << p.first
<< " exported caps on " << *in << dendl;
- auto m = MClientCaps::create(CEPH_CAP_OP_EXPORT, in->ino(), 0,
- cap->get_cap_id(), cap->get_mseq(), mds->get_osd_epoch_barrier());
-
+ auto m = make_message<MClientCaps>(CEPH_CAP_OP_EXPORT, in->ino(), 0,
+ cap->get_cap_id(), cap->get_mseq(),
+ mds->get_osd_epoch_barrier());
map<client_t,Capability::Import>::iterator q = peer_imported.find(p.first);
ceph_assert(q != peer_imported.end());
m->set_cap_peer(q->second.cap_id, q->second.issue_seq, q->second.mseq,
for (set<mds_rank_t>::iterator p = stat.notify_ack_waiting.begin();
p != stat.notify_ack_waiting.end();
++p) {
- auto notify = MExportDirNotify::create(dir->dirfrag(), stat.tid, true,
+ auto notify = make_message<MExportDirNotify>(dir->dirfrag(), stat.tid, true,
pair<int,int>(mds->get_nodeid(), stat.peer),
pair<int,int>(mds->get_nodeid(), CDIR_AUTH_UNKNOWN));
for (set<CDir*>::iterator i = bounds.begin(); i != bounds.end(); ++i)
for (set<mds_rank_t>::iterator p = stat.notify_ack_waiting.begin();
p != stat.notify_ack_waiting.end();
++p) {
- auto notify = MExportDirNotify::create(dir->dirfrag(), stat.tid, true,
+ auto notify = make_message<MExportDirNotify>(dir->dirfrag(), stat.tid, true,
pair<int,int>(mds->get_nodeid(), stat.peer),
pair<int,int>(stat.peer, CDIR_AUTH_UNKNOWN));
// notify peer to send cap import messages to clients
if (!mds->is_cluster_degraded() ||
mds->mdsmap->is_clientreplay_or_active_or_stopping(stat.peer)) {
- mds->send_message_mds(MExportDirFinish::create(dir->dirfrag(), false, stat.tid), stat.peer);
+ mds->send_message_mds(make_message<MExportDirFinish>(dir->dirfrag(), false, stat.tid), stat.peer);
} else {
dout(7) << "not sending MExportDirFinish, dest has failed" << dendl;
}
// send finish/commit to new auth
if (!mds->is_cluster_degraded() ||
mds->mdsmap->is_clientreplay_or_active_or_stopping(it->second.peer)) {
- mds->send_message_mds(MExportDirFinish::create(dir->dirfrag(), true, it->second.tid), it->second.peer);
+ mds->send_message_mds(make_message<MExportDirFinish>(dir->dirfrag(), true, it->second.tid), it->second.peer);
} else {
dout(7) << "not sending MExportDirFinish last, dest has failed" << dendl;
}
if (!mds->is_active()) {
dout(7) << " not active, send NACK " << dendl;
- mds->send_message_mds(MExportDirDiscoverAck::create(df, m->get_tid(), false), from);
+ mds->send_message_mds(make_message<MExportDirDiscoverAck>(df, m->get_tid(), false), from);
return;
}
// reply
dout(7) << " sending export_discover_ack on " << *in << dendl;
- mds->send_message_mds(MExportDirDiscoverAck::create(df, m->get_tid()), p_state->peer);
+ mds->send_message_mds(make_message<MExportDirDiscoverAck>(df, m->get_tid()), p_state->peer);
assert (g_conf()->mds_kill_import_at != 2);
}
// ok!
dout(7) << " sending export_prep_ack on " << *dir << dendl;
- mds->send_message(MExportDirPrepAck::create(dir->dirfrag(), success, m->get_tid()), m->get_connection());
+ mds->send_message(make_message<MExportDirPrepAck>(dir->dirfrag(), success, m->get_tid()), m->get_connection());
ceph_assert(g_conf()->mds_kill_import_at != 4);
}
for (set<mds_rank_t>::iterator p = stat.bystanders.begin();
p != stat.bystanders.end();
++p) {
- auto notify = MExportDirNotify::create(dir->dirfrag(), stat.tid, false,
+ auto notify = make_message<MExportDirNotify>(dir->dirfrag(), stat.tid, false,
pair<int,int>(stat.peer, mds->get_nodeid()),
pair<int,int>(mds->get_nodeid(), CDIR_AUTH_UNKNOWN));
for (set<CDir*>::iterator i = bounds.begin(); i != bounds.end(); ++i)
stat.bystanders.erase(p++);
continue;
}
- auto notify = MExportDirNotify::create(dir->dirfrag(), stat.tid, true,
+ auto notify = make_message<MExportDirNotify>(dir->dirfrag(), stat.tid, true,
mds_authority_t(stat.peer, mds->get_nodeid()),
mds_authority_t(stat.peer, CDIR_AUTH_UNKNOWN));
for (set<CDir*>::iterator i = bounds.begin(); i != bounds.end(); ++i)
// test surviving observer of a failed migration that did not complete
//assert(dir->replica_map.size() < 2 || mds->get_nodeid() != 0);
- auto ack = MExportDirAck::create(dir->dirfrag(), it->second.tid);
+ auto ack = make_message<MExportDirAck>(dir->dirfrag(), it->second.tid);
encode(imported_caps, ack->imported_caps);
mds->send_message_mds(ack, from);
// send ack
if (m->wants_ack()) {
- mds->send_message_mds(MExportDirNotifyAck::create(m->get_dirfrag(), m->get_tid(), m->get_new_auth()), from);
+ mds->send_message_mds(make_message<MExportDirNotifyAck>(m->get_dirfrag(), m->get_tid(), m->get_new_auth()), from);
} else {
// aborted. no ack.
dout(7) << "handle_export_notify no ack requested" << dendl;
ceph_assert(!in->is_ambiguous_auth());
ceph_assert(!in->state_test(CInode::STATE_EXPORTINGCAPS));
- auto ex = MExportCaps::create();
+ auto ex = make_message<MExportCaps>();
ex->ino = in->ino();
encode_export_inode_caps(in, false, ex->cap_bl, ex->client_map, ex->client_metadata_map);
dout(7) << __func__ << " telling client." << it.first
<< " exported caps on " << *in << dendl;
- auto m = MClientCaps::create(CEPH_CAP_OP_EXPORT, in->ino(), 0,
+ auto m = make_message<MClientCaps>(CEPH_CAP_OP_EXPORT, in->ino(), 0,
cap->get_cap_id(), cap->get_mseq(),
mds->get_osd_epoch_barrier());
m->set_cap_peer(it.second.cap_id, it.second.issue_seq, it.second.mseq, from, 0);
mds->locker->eval(in, CEPH_CAP_LOCKS, true);
if (!imported_caps.empty()) {
- auto ack = MExportCapsAck::create(in->ino());
+ auto ack = make_message<MExportCapsAck>(in->ino());
map<client_t,uint64_t> peer_caps_ids;
for (auto &p : imported_caps )
peer_caps_ids[p.first] = it->second.at(p.first).cap_id;
return;
}
- auto reply = MClientReclaimReply::create(0);
+ auto reply = make_message<MClientReclaimReply>(0);
if (m->get_uuid().empty()) {
dout(10) << __func__ << " invalid message (no uuid)" << dendl;
reply->set_result(-EINVAL);
};
auto send_reject_message = [this, &session, &log_session_status](std::string_view err_str) {
- auto m = MClientSession::create(CEPH_SESSION_REJECT);
+ auto m = make_message<MClientSession>(CEPH_SESSION_REJECT);
if (session->info.has_feature(CEPHFS_FEATURE_MIMIC))
m->metadata["error_string"] = err_str;
mds->send_message_client(m, session);
mds->locker->resume_stale_caps(session);
mds->sessionmap.touch_session(session);
}
- auto reply = MClientSession::create(CEPH_SESSION_RENEWCAPS, m->get_seq());
+ auto reply = make_message<MClientSession>(CEPH_SESSION_RENEWCAPS, m->get_seq());
mds->send_message_client(reply, session);
} else {
dout(10) << "ignoring renewcaps on non open|stale session (" << session->get_state_name() << ")" << dendl;
version_t seq = session->wait_for_flush(gather->new_sub());
mds->send_message_client(
- MClientSession::create(CEPH_SESSION_FLUSHMSG, seq), session);
+ make_message<MClientSession>(CEPH_SESSION_FLUSHMSG, seq), session);
}
void Server::flush_client_sessions(set<client_t>& client_set, MDSGatherBuilder& gather)
mds->sessionmap.set_state(session, Session::STATE_OPEN);
mds->sessionmap.touch_session(session);
ceph_assert(session->get_connection());
- auto reply = MClientSession::create(CEPH_SESSION_OPEN);
+ auto reply = make_message<MClientSession>(CEPH_SESSION_OPEN);
if (session->info.has_feature(CEPHFS_FEATURE_MIMIC))
reply->supported_features = supported_features;
mds->send_message_client(reply, session);
if (mdcache->is_readonly()) {
- auto m = MClientSession::create(CEPH_SESSION_FORCE_RO);
+ auto m = make_message<MClientSession>(CEPH_SESSION_FORCE_RO);
mds->send_message_client(m, session);
}
} else if (session->is_closing() ||
}
// reset session
- mds->send_message_client(MClientSession::create(CEPH_SESSION_CLOSE), session);
+ mds->send_message_client(make_message<MClientSession>(CEPH_SESSION_CLOSE), session);
mds->sessionmap.set_state(session, Session::STATE_CLOSED);
session->clear();
mds->sessionmap.remove_session(session);
mds->sessionmap.set_state(session, Session::STATE_OPEN);
mds->sessionmap.touch_session(session);
- auto reply = MClientSession::create(CEPH_SESSION_OPEN);
+ auto reply = make_message<MClientSession>(CEPH_SESSION_OPEN);
if (session->info.has_feature(CEPHFS_FEATURE_MIMIC))
reply->supported_features = supported_features;
mds->send_message_client(reply, session);
if (mdcache->is_readonly())
- mds->send_message_client(MClientSession::create(CEPH_SESSION_FORCE_RO), session);
+ mds->send_message_client(make_message<MClientSession>(CEPH_SESSION_FORCE_RO), session);
}
} else {
dout(10) << "force_open_sessions skipping already-open " << session->info.inst << dendl;
mds->sessionmap.set_state(session, Session::STATE_STALE);
mds->locker->revoke_stale_caps(session);
mds->locker->remove_stale_leases(session);
- mds->send_message_client(MClientSession::create(CEPH_SESSION_STALE, session->get_push_seq()), session);
+ mds->send_message_client(make_message<MClientSession>(CEPH_SESSION_STALE, session->get_push_seq()), session);
finish_flush_session(session, session->get_push_seq());
}
}
}
if (deny) {
- auto r = MClientSession::create(CEPH_SESSION_CLOSE);
+ auto r = make_message<MClientSession>(CEPH_SESSION_CLOSE);
mds->send_message_client(r, session);
if (session->is_open())
kill_session(session, nullptr);
if (!m->has_more()) {
// notify client of success with an OPEN
- auto reply = MClientSession::create(CEPH_SESSION_OPEN);
+ auto reply = make_message<MClientSession>(CEPH_SESSION_OPEN);
if (session->info.has_feature(CEPHFS_FEATURE_MIMIC))
reply->supported_features = supported_features;
mds->send_message_client(reply, session);
dout(7) << " recalling " << recall << " caps; session_recall_throttle = " << session_recall_throttle << "; global_recall_throttle = " << global_recall_throttle << dendl;
- auto m = MClientSession::create(CEPH_SESSION_RECALL_STATE);
+ auto m = make_message<MClientSession>(CEPH_SESSION_RECALL_STATE);
m->head.max_caps = newlim;
mds->send_message_client(m, session);
if (gather) {
if (!session->info.inst.name.is_client() ||
!(session->is_open() || session->is_stale()))
continue;
- mds->send_message_client(MClientSession::create(CEPH_SESSION_FORCE_RO), session);
+ mds->send_message_client(make_message<MClientSession>(CEPH_SESSION_FORCE_RO), session);
}
}
void Server::respond_to_request(MDRequestRef& mdr, int r)
{
if (mdr->client_request) {
- reply_client_request(mdr, MClientReply::create(*mdr->client_request, r));
+ reply_client_request(mdr, make_message<MClientReply>(*mdr->client_request, r));
} else if (mdr->internal_op > -1) {
dout(10) << "respond_to_request on internal request " << mdr << dendl;
if (!mdr->internal_op_finish)
}
- auto reply = MClientReply::create(*req, 0);
+ auto reply = make_message<MClientReply>(*req, 0);
reply->set_unsafe();
// mark xlocks "done", indicating that we are exposing uncommitted changes.
req->get_op() != CEPH_MDS_OP_OPEN &&
req->get_op() != CEPH_MDS_OP_CREATE)) {
dout(5) << "already completed " << req->get_reqid() << dendl;
- auto reply = MClientReply::create(*req, 0);
+ auto reply = make_message<MClientReply>(*req, 0);
if (created != inodeno_t()) {
bufferlist extra;
encode(created, extra);
// the purpose of rename notify is enforcing causal message ordering. making sure
// bystanders have received all messages from rename srcdn's auth MDS.
if (m->get_op() == MMDSSlaveRequest::OP_RENAMENOTIFY) {
- auto reply = MMDSSlaveRequest::create(m->get_reqid(), m->get_attempt(), MMDSSlaveRequest::OP_RENAMENOTIFYACK);
+ auto reply = make_message<MMDSSlaveRequest>(m->get_reqid(), m->get_attempt(), MMDSSlaveRequest::OP_RENAMENOTIFYACK);
mds->send_message(reply, m->get_connection());
return;
}
return;
// ack
- auto r = MMDSSlaveRequest::create(mdr->reqid, mdr->attempt, replycode);
+ auto r = make_message<MMDSSlaveRequest>(mdr->reqid, mdr->attempt, replycode);
r->set_lock_type(lock->get_type());
lock->get_parent()->set_object_info(r->get_object_info());
if (replycode == MMDSSlaveRequest::OP_XLOCKACK)
}
// ack!
- auto reply = MMDSSlaveRequest::create(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_AUTHPINACK);
+ auto reply = make_message<MMDSSlaveRequest>(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_AUTHPINACK);
// return list of my auth_pins (if any)
for (const auto &p : mdr->auth_pins) {
{
dout(10) << __func__ << " " << *in << dendl;
- auto req = MClientRequest::create(CEPH_MDS_OP_SETXATTR);
+ auto req = make_message<MClientRequest>(CEPH_MDS_OP_SETXATTR);
req->set_filepath(filepath(in->ino()));
req->set_string2("ceph.quota");
// empty vxattr value
op = MMDSSlaveRequest::OP_LINKPREP;
else
op = MMDSSlaveRequest::OP_UNLINKPREP;
- auto req = MMDSSlaveRequest::create(mdr->reqid, mdr->attempt, op);
+ auto req = make_message<MMDSSlaveRequest>(mdr->reqid, mdr->attempt, op);
targeti->set_object_info(req->get_object_info());
req->op_stamp = mdr->get_op_stamp();
if (auto& desti_srnode = mdr->more()->desti_srnode)
// ack
if (!mdr->aborted) {
- auto reply = MMDSSlaveRequest::create(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_LINKPREPACK);
+ auto reply = make_message<MMDSSlaveRequest>(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_LINKPREPACK);
mds->send_message_mds(reply, mdr->slave_to_mds);
} else {
dout(10) << " abort flag set, finishing" << dendl;
ceph_assert(g_conf()->mds_kill_link_at != 8);
- auto req = MMDSSlaveRequest::create(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_COMMITTED);
+ auto req = make_message<MMDSSlaveRequest>(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_COMMITTED);
mds->send_message_mds(req, mdr->slave_to_mds);
mdcache->request_finish(mdr);
}
}
dout(10) << "_rmdir_prepare_witness mds." << who << dendl;
- auto req = MMDSSlaveRequest::create(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RMDIRPREP);
+ auto req = make_message<MMDSSlaveRequest>(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RMDIRPREP);
req->srcdnpath = filepath(trace.front()->get_dir()->ino());
for (auto dn : trace)
req->srcdnpath.push_dentry(dn->get_name());
mdr->straydn = 0;
if (!mdr->aborted) {
- auto reply = MMDSSlaveRequest::create(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RMDIRPREPACK);
+ auto reply = make_message<MMDSSlaveRequest>(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RMDIRPREPACK);
if (!mdr->more()->slave_update_journaled)
reply->mark_not_journaled();
mds->send_message_mds(reply, mdr->slave_to_mds);
}
dout(10) << "_rename_prepare_witness mds." << who << dendl;
- auto req = MMDSSlaveRequest::create(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RENAMEPREP);
+ auto req = make_message<MMDSSlaveRequest>(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RENAMEPREP);
req->srcdnpath = filepath(srctrace.front()->get_dir()->ino());
for (auto dn : srctrace)
if (mdr->slave_request->is_interrupted()) {
dout(10) << " slave request interrupted, sending noop reply" << dendl;
- auto reply = MMDSSlaveRequest::create(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RENAMEPREPACK);
+ auto reply = make_message<MMDSSlaveRequest>(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RENAMEPREPACK);
reply->mark_interrupted();
mds->send_message_mds(reply, mdr->slave_to_mds);
mdr->reset_slave_request();
(mds->is_cluster_degraded() &&
!mds->mdsmap->is_clientreplay_or_active_or_stopping(*p)))
continue;
- auto notify = MMDSSlaveRequest::create(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RENAMENOTIFY);
+ auto notify = make_message<MMDSSlaveRequest>(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RENAMENOTIFY);
mds->send_message_mds(notify, *p);
mdr->more()->waiting_on_slave.insert(*p);
}
if (reply_witness) {
ceph_assert(!srcdnrep.empty());
- auto reply = MMDSSlaveRequest::create(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RENAMEPREPACK);
+ auto reply = make_message<MMDSSlaveRequest>(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RENAMEPREPACK);
reply->witnesses.swap(srcdnrep);
mds->send_message_mds(reply, mdr->slave_to_mds);
mdr->reset_slave_request();
// prepare ack
ref_t<MMDSSlaveRequest> reply;
if (!mdr->aborted) {
- reply = MMDSSlaveRequest::create(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RENAMEPREPACK);
+ reply = make_message<MMDSSlaveRequest>(mdr->reqid, mdr->attempt, MMDSSlaveRequest::OP_RENAMEPREPACK);
if (!mdr->more()->slave_update_journaled)
reply->mark_not_journaled();
}
{
dout(10) << __func__ << " " << *m << dendl;
handle_query_result(m);
- auto ack = MMDSTableRequest::create(table, TABLESERVER_OP_NOTIFY_ACK, 0, m->get_tid());
+ auto ack = make_message<MMDSTableRequest>(table, TABLESERVER_OP_NOTIFY_ACK, 0, m->get_tid());
mds->send_message(ack, m->get_connection());
}
return;
mds_rank_t ts = mds->mdsmap->get_tableserver();
- auto req = MMDSTableRequest::create(table, TABLESERVER_OP_QUERY, ++last_reqid, 0);
+ auto req = make_message<MMDSTableRequest>(table, TABLESERVER_OP_QUERY, ++last_reqid, 0);
using ceph::encode;
char op = 'F';
encode(op, req->bl);
ceph_assert(version == tid);
for (auto &p : active_clients) {
- auto m = MMDSTableRequest::create(table, TABLESERVER_OP_NOTIFY_PREP, 0, version);
+ auto m = make_message<MMDSTableRequest>(table, TABLESERVER_OP_NOTIFY_PREP, 0, version);
m->bl = bl;
mds->send_message_mds(m, p);
}
auto p = req->bl.cbegin();
decode(op, p);
- auto reply = MMDSTableRequest::create(table, TABLESERVER_OP_QUERY_REPLY, req->reqid, version);
+ auto reply = make_message<MMDSTableRequest>(table, TABLESERVER_OP_QUERY_REPLY, req->reqid, version);
switch (op) {
case 'F': // full
if (!all_purge.empty()) {
dout(10) << "requesting removal of " << all_purge << dendl;
- auto m = MRemoveSnaps::create(all_purge);
+ auto m = make_message<MRemoveSnaps>(all_purge);
mon_client->send_mon_message(m.detach());
}
filepath dst;
rdn->make_path(dst);
- auto req = MClientRequest::create(CEPH_MDS_OP_RENAME);
+ auto req = make_message<MClientRequest>(CEPH_MDS_OP_RENAME);
req->set_filepath(dst);
req->set_filepath2(src);
req->set_tid(mds->issue_tid());
dst.push_dentry(src[0]);
dst.push_dentry(src[1]);
- auto req = MClientRequest::create(CEPH_MDS_OP_RENAME);
+ auto req = make_message<MClientRequest>(CEPH_MDS_OP_RENAME);
req->set_filepath(dst);
req->set_filepath2(src);
req->set_tid(mds->issue_tid());
#include "msg/MessageRef.h"
#include "messages/PaxosServiceMessage.h"
-class MAuth : public MessageInstance<MAuth, PaxosServiceMessage> {
+class MAuth : public PaxosServiceMessage {
public:
- friend factory;
-
__u32 protocol;
ceph::buffer::list auth_payload;
epoch_t monmap_epoch;
/* if protocol == 0, then auth_payload is a set<__u32> listing protocols the client supports */
- MAuth() : MessageInstance(CEPH_MSG_AUTH, 0), protocol(0), monmap_epoch(0) { }
+ MAuth() : PaxosServiceMessage{CEPH_MSG_AUTH, 0}, protocol(0), monmap_epoch(0) { }
private:
~MAuth() override {}
#include "msg/Message.h"
#include "common/errno.h"
-class MAuthReply : public MessageInstance<MAuthReply> {
+class MAuthReply : public Message {
public:
- friend factory;
-
__u32 protocol;
errorcode32_t result;
uint64_t global_id; // if zero, meaningless
std::string result_msg;
ceph::buffer::list result_bl;
- MAuthReply() : MessageInstance(CEPH_MSG_AUTH_REPLY), protocol(0), result(0), global_id(0) {}
+ MAuthReply() : Message(CEPH_MSG_AUTH_REPLY), protocol(0), result(0), global_id(0) {}
MAuthReply(__u32 p, ceph::buffer::list *bl = NULL, int r = 0, uint64_t gid=0, const char *msg = "") :
- MessageInstance(CEPH_MSG_AUTH_REPLY),
+ Message(CEPH_MSG_AUTH_REPLY),
protocol(p), result(r), global_id(gid),
result_msg(msg) {
if (bl)
#include "msg/Message.h"
#include "messages/MOSDPeeringOp.h"
-class MBackfillReserve : public MessageInstance<MBackfillReserve, MOSDPeeringOp> {
-public:
- friend factory;
+class MBackfillReserve : public MOSDPeeringOp {
private:
static constexpr int HEAD_VERSION = 5;
static constexpr int COMPAT_VERSION = 4;
}
MBackfillReserve()
- : MessageInstance(MSG_OSD_BACKFILL_RESERVE, HEAD_VERSION, COMPAT_VERSION),
+ : MOSDPeeringOp{MSG_OSD_BACKFILL_RESERVE, HEAD_VERSION, COMPAT_VERSION},
query_epoch(0), type(-1), priority(-1), primary_num_bytes(0),
shard_num_bytes(0) {}
MBackfillReserve(int type,
epoch_t query_epoch, unsigned prio = -1,
int64_t primary_num_bytes = 0,
int64_t shard_num_bytes = 0)
- : MessageInstance(MSG_OSD_BACKFILL_RESERVE, HEAD_VERSION, COMPAT_VERSION),
+ : MOSDPeeringOp{MSG_OSD_BACKFILL_RESERVE, HEAD_VERSION, COMPAT_VERSION},
pgid(pgid), query_epoch(query_epoch),
type(type), priority(prio), primary_num_bytes(primary_num_bytes),
shard_num_bytes(shard_num_bytes) {}
#include "mds/mdstypes.h"
-class MCacheExpire : public MessageInstance<MCacheExpire> {
-public:
- friend factory;
+class MCacheExpire : public Message {
private:
__s32 from;
int get_from() const { return from; }
protected:
- MCacheExpire() : MessageInstance(MSG_MDS_CACHEEXPIRE), from(-1) {}
+ MCacheExpire() : Message{MSG_MDS_CACHEEXPIRE}, from(-1) {}
MCacheExpire(int f) :
- MessageInstance(MSG_MDS_CACHEEXPIRE),
+ Message{MSG_MDS_CACHEEXPIRE},
from(f) { }
~MCacheExpire() override {}
encode(from, payload);
encode(realms, payload);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
WRITE_CLASS_ENCODER(MCacheExpire::realm)
#include "msg/Message.h"
-class MClientCapRelease : public MessageInstance<MClientCapRelease> {
-public:
- friend factory;
-
+class MClientCapRelease : public Message {
private:
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 1;
epoch_t osd_epoch_barrier;
MClientCapRelease() :
- MessageInstance(CEPH_MSG_CLIENT_CAPRELEASE, HEAD_VERSION, COMPAT_VERSION),
+ Message{CEPH_MSG_CLIENT_CAPRELEASE, HEAD_VERSION, COMPAT_VERSION},
osd_epoch_barrier(0)
{
memset(&head, 0, sizeof(head));
#include "mds/mdstypes.h"
#include "include/ceph_features.h"
-class MClientCaps : public MessageInstance<MClientCaps> {
-public:
- friend factory;
+class MClientCaps : public Message {
private:
static constexpr int HEAD_VERSION = 11;
protected:
MClientCaps()
- : MessageInstance(CEPH_MSG_CLIENT_CAPS, HEAD_VERSION, COMPAT_VERSION) {}
+ : Message{CEPH_MSG_CLIENT_CAPS, HEAD_VERSION, COMPAT_VERSION} {}
MClientCaps(int op,
inodeno_t ino,
inodeno_t realm,
int dirty,
int mseq,
epoch_t oeb)
- : MessageInstance(CEPH_MSG_CLIENT_CAPS, HEAD_VERSION, COMPAT_VERSION),
+ : Message{CEPH_MSG_CLIENT_CAPS, HEAD_VERSION, COMPAT_VERSION},
osd_epoch_barrier(oeb) {
memset(&head, 0, sizeof(head));
head.op = op;
MClientCaps(int op,
inodeno_t ino, inodeno_t realm,
uint64_t id, int mseq, epoch_t oeb)
- : MessageInstance(CEPH_MSG_CLIENT_CAPS, HEAD_VERSION, COMPAT_VERSION),
+ : Message{CEPH_MSG_CLIENT_CAPS, HEAD_VERSION, COMPAT_VERSION},
osd_epoch_barrier(oeb) {
memset(&head, 0, sizeof(head));
head.op = op;
encode(nfiles, payload);
encode(nsubdirs, payload);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MClientLease : public MessageInstance<MClientLease> {
+class MClientLease : public Message {
public:
- friend factory;
-
struct ceph_mds_lease h;
std::string dname;
snapid_t get_last() const { return snapid_t(h.last); }
protected:
- MClientLease() : MessageInstance(CEPH_MSG_CLIENT_LEASE) {}
+ MClientLease() : Message(CEPH_MSG_CLIENT_LEASE) {}
MClientLease(const MClientLease& m) :
- MessageInstance(CEPH_MSG_CLIENT_LEASE),
+ Message(CEPH_MSG_CLIENT_LEASE),
h(m.h),
dname(m.dname) {}
MClientLease(int ac, ceph_seq_t seq, int m, uint64_t i, uint64_t sf, uint64_t sl) :
- MessageInstance(CEPH_MSG_CLIENT_LEASE) {
+ Message(CEPH_MSG_CLIENT_LEASE) {
h.action = ac;
h.seq = seq;
h.mask = m;
h.duration_ms = 0;
}
MClientLease(int ac, ceph_seq_t seq, int m, uint64_t i, uint64_t sf, uint64_t sl, std::string_view d) :
- MessageInstance(CEPH_MSG_CLIENT_LEASE),
+ Message(CEPH_MSG_CLIENT_LEASE),
dname(d) {
h.action = ac;
h.seq = seq;
encode(dname, payload);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MClientQuota : public MessageInstance<MClientQuota> {
+class MClientQuota : public Message {
public:
- friend factory;
-
inodeno_t ino;
nest_info_t rstat;
quota_info_t quota;
protected:
MClientQuota() :
- MessageInstance(CEPH_MSG_CLIENT_QUOTA),
+ Message{CEPH_MSG_CLIENT_QUOTA},
ino(0)
{}
~MClientQuota() override {}
decode(quota, p);
ceph_assert(p.end());
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MClientReclaim: public MessageInstance<MClientReclaim> {
+class MClientReclaim: public Message {
public:
static constexpr int HEAD_VERSION = 1;
static constexpr int COMPAT_VERSION = 1;
}
protected:
- friend factory;
MClientReclaim() :
- MessageInstance(CEPH_MSG_CLIENT_RECLAIM, HEAD_VERSION, COMPAT_VERSION) {}
+ Message{CEPH_MSG_CLIENT_RECLAIM, HEAD_VERSION, COMPAT_VERSION} {}
MClientReclaim(std::string_view _uuid, uint32_t _flags) :
- MessageInstance(CEPH_MSG_CLIENT_RECLAIM, HEAD_VERSION, COMPAT_VERSION),
+ Message{CEPH_MSG_CLIENT_RECLAIM, HEAD_VERSION, COMPAT_VERSION},
uuid(_uuid), flags(_flags) {}
private:
~MClientReclaim() override {}
std::string uuid;
uint32_t flags = 0;
+
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MClientReclaimReply: public MessageInstance<MClientReclaimReply> {
+class MClientReclaimReply: public Message {
public:
static constexpr int HEAD_VERSION = 1;
static constexpr int COMPAT_VERSION = 1;
}
protected:
- friend factory;
MClientReclaimReply() :
- MessageInstance(CEPH_MSG_CLIENT_RECLAIM_REPLY, HEAD_VERSION, COMPAT_VERSION) {}
+ MClientReclaimReply{0, 0}
+ {}
MClientReclaimReply(int r, epoch_t e=0) :
- MessageInstance(CEPH_MSG_CLIENT_RECLAIM_REPLY, HEAD_VERSION, COMPAT_VERSION),
+ Message{CEPH_MSG_CLIENT_RECLAIM_REPLY, HEAD_VERSION, COMPAT_VERSION},
result(r), epoch(e) {}
private:
int32_t result;
epoch_t epoch;
entity_addrvec_t addrs;
+
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "include/ceph_features.h"
-class MClientReconnect : public MessageInstance<MClientReconnect> {
-public:
- friend factory;
+class MClientReconnect : public Message {
private:
static constexpr int HEAD_VERSION = 5;
static constexpr int COMPAT_VERSION = 4;
vector<snaprealm_reconnect_t> realms;
bool more = false;
- MClientReconnect() :
- MessageInstance(CEPH_MSG_CLIENT_RECONNECT, HEAD_VERSION, COMPAT_VERSION) {}
private:
+ MClientReconnect() :
+ Message{CEPH_MSG_CLIENT_RECONNECT, HEAD_VERSION, COMPAT_VERSION} {}
~MClientReconnect() override {}
size_t cap_size = 0;
}
}
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
};
-class MClientReply : public MessageInstance<MClientReply> {
+class MClientReply : public Message {
public:
- friend factory;
-
// reply data
struct ceph_mds_reply_head head {};
bufferlist trace_bl;
bool is_safe() const { return head.safe; }
protected:
- MClientReply() : MessageInstance(CEPH_MSG_CLIENT_REPLY) {}
+ MClientReply() : Message{CEPH_MSG_CLIENT_REPLY} {}
MClientReply(const MClientRequest &req, int result = 0) :
- MessageInstance(CEPH_MSG_CLIENT_REPLY) {
+ Message{CEPH_MSG_CLIENT_REPLY} {
memset(&head, 0, sizeof(head));
header.tid = req.get_tid();
head.op = req.get_op();
const bufferlist& get_trace_bl() const {
return trace_bl;
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
// metadata ops.
-class MClientRequest : public MessageInstance<MClientRequest> {
-public:
- friend factory;
+class MClientRequest : public Message {
private:
static constexpr int HEAD_VERSION = 4;
static constexpr int COMPAT_VERSION = 1;
protected:
// cons
MClientRequest()
- : MessageInstance(CEPH_MSG_CLIENT_REQUEST, HEAD_VERSION, COMPAT_VERSION) {}
+ : Message(CEPH_MSG_CLIENT_REQUEST, HEAD_VERSION, COMPAT_VERSION) {}
MClientRequest(int op)
- : MessageInstance(CEPH_MSG_CLIENT_REQUEST, HEAD_VERSION, COMPAT_VERSION) {
+ : Message(CEPH_MSG_CLIENT_REQUEST, HEAD_VERSION, COMPAT_VERSION) {
memset(&head, 0, sizeof(head));
head.op = op;
}
out << '}'
<< ")";
}
-
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
WRITE_CLASS_ENCODER(MClientRequest::Release)
#include "msg/Message.h"
-class MClientRequestForward : public MessageInstance<MClientRequestForward> {
-public:
- friend factory;
+class MClientRequestForward : public Message {
private:
int32_t dest_mds;
int32_t num_fwd;
protected:
MClientRequestForward()
- : MessageInstance(CEPH_MSG_CLIENT_REQUEST_FORWARD),
+ : Message{CEPH_MSG_CLIENT_REQUEST_FORWARD},
dest_mds(-1), num_fwd(-1), client_must_resend(false) {}
MClientRequestForward(ceph_tid_t t, int dm, int nf, bool cmr) :
- MessageInstance(CEPH_MSG_CLIENT_REQUEST_FORWARD),
+ Message{CEPH_MSG_CLIENT_REQUEST_FORWARD},
dest_mds(dm), num_fwd(nf), client_must_resend(cmr) {
ceph_assert(client_must_resend);
header.tid = t;
decode(num_fwd, p);
decode(client_must_resend, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
#include "mds/mdstypes.h"
-class MClientSession : public MessageInstance<MClientSession> {
-public:
- friend factory;
+class MClientSession : public Message {
private:
static constexpr int HEAD_VERSION = 3;
static constexpr int COMPAT_VERSION = 1;
int get_max_leases() const { return head.max_leases; }
protected:
- MClientSession() : MessageInstance(CEPH_MSG_CLIENT_SESSION, HEAD_VERSION, COMPAT_VERSION) { }
+ MClientSession() : Message{CEPH_MSG_CLIENT_SESSION, HEAD_VERSION, COMPAT_VERSION} { }
MClientSession(int o, version_t s=0) :
- MessageInstance(CEPH_MSG_CLIENT_SESSION, HEAD_VERSION, COMPAT_VERSION) {
+ Message{CEPH_MSG_CLIENT_SESSION, HEAD_VERSION, COMPAT_VERSION} {
memset(&head, 0, sizeof(head));
head.op = o;
head.seq = s;
}
MClientSession(int o, utime_t st) :
- MessageInstance(CEPH_MSG_CLIENT_SESSION, HEAD_VERSION, COMPAT_VERSION) {
+ Message{CEPH_MSG_CLIENT_SESSION, HEAD_VERSION, COMPAT_VERSION} {
memset(&head, 0, sizeof(head));
head.op = o;
head.seq = 0;
encode(supported_features, payload);
}
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MClientSnap : public MessageInstance<MClientSnap> {
+class MClientSnap : public Message {
public:
- friend factory;
-
ceph_mds_snap_head head;
bufferlist bl;
protected:
MClientSnap(int o=0) :
- MessageInstance(CEPH_MSG_CLIENT_SNAP) {
+ Message{CEPH_MSG_CLIENT_SNAP} {
memset(&head, 0, sizeof(head));
head.op = o;
}
decode_nohead(head.trace_len, bl, p);
ceph_assert(p.end());
}
-
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MCommand : public MessageInstance<MCommand> {
+class MCommand : public Message {
public:
- friend factory;
-
uuid_d fsid;
std::vector<std::string> cmd;
MCommand()
- : MessageInstance(MSG_COMMAND) {}
+ : Message{MSG_COMMAND} {}
MCommand(const uuid_d &f)
- : MessageInstance(MSG_COMMAND),
+ : Message{MSG_COMMAND},
fsid(f) { }
private:
#include "msg/Message.h"
#include "MCommand.h"
-class MCommandReply : public MessageInstance<MCommandReply> {
+class MCommandReply : public Message {
public:
- friend factory;
-
errorcode32_t r;
std::string rs;
MCommandReply()
- : MessageInstance(MSG_COMMAND_REPLY) {}
+ : Message{MSG_COMMAND_REPLY} {}
MCommandReply(MCommand *m, int _r)
- : MessageInstance(MSG_COMMAND_REPLY), r(_r) {
+ : Message{MSG_COMMAND_REPLY}, r(_r) {
header.tid = m->get_tid();
}
MCommandReply(int _r, std::string_view s)
- : MessageInstance(MSG_COMMAND_REPLY),
+ : Message{MSG_COMMAND_REPLY},
r(_r), rs(s) { }
private:
~MCommandReply() override {}
#include "msg/Message.h"
-class MConfig : public MessageInstance<MConfig> {
+class MConfig : public Message {
public:
- friend factory;
-
static constexpr int HEAD_VERSION = 1;
static constexpr int COMPAT_VERSION = 1;
// use transparent comparator so we can lookup in it by std::string_view keys
std::map<std::string,std::string,std::less<>> config;
- MConfig() : MessageInstance(MSG_CONFIG, HEAD_VERSION, COMPAT_VERSION) { }
+ MConfig() : Message{MSG_CONFIG, HEAD_VERSION, COMPAT_VERSION} { }
MConfig(const std::map<std::string,std::string,std::less<>>& c)
- : MessageInstance(MSG_CONFIG, HEAD_VERSION, COMPAT_VERSION),
+ : Message{MSG_CONFIG, HEAD_VERSION, COMPAT_VERSION},
config{c} {}
MConfig(std::map<std::string,std::string,std::less<>>&& c)
- : MessageInstance(MSG_CONFIG, HEAD_VERSION, COMPAT_VERSION),
+ : Message{MSG_CONFIG, HEAD_VERSION, COMPAT_VERSION},
config{std::move(c)} {}
std::string_view get_type_name() const override {
#include "msg/Message.h"
-class MDentryLink : public MessageInstance<MDentryLink> {
-public:
- friend factory;
+class MDentryLink : public Message {
private:
dirfrag_t subtree;
dirfrag_t dirfrag;
protected:
MDentryLink() :
- MessageInstance(MSG_MDS_DENTRYLINK) { }
+ Message{MSG_MDS_DENTRYLINK} { }
MDentryLink(dirfrag_t r, dirfrag_t df, std::string_view n, bool p) :
- MessageInstance(MSG_MDS_DENTRYLINK),
+ Message{MSG_MDS_DENTRYLINK},
subtree(r),
dirfrag(df),
dn(n),
encode(is_primary, payload);
encode(bl, payload);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MDentryUnlink : public MessageInstance<MDentryUnlink> {
-public:
- friend factory;
+class MDentryUnlink : public Message {
private:
-
dirfrag_t dirfrag;
string dn;
protected:
MDentryUnlink() :
- MessageInstance(MSG_MDS_DENTRYUNLINK) { }
+ Message{MSG_MDS_DENTRYUNLINK} { }
MDentryUnlink(dirfrag_t df, std::string_view n) :
- MessageInstance(MSG_MDS_DENTRYUNLINK),
+ Message{MSG_MDS_DENTRYUNLINK},
dirfrag(df),
dn(n) {}
~MDentryUnlink() override {}
encode(dn, payload);
encode(straybl, payload);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MDirUpdate : public MessageInstance<MDirUpdate> {
+class MDirUpdate : public Message {
public:
- friend factory;
-
mds_rank_t get_source_mds() const { return from_mds; }
dirfrag_t get_dirfrag() const { return dirfrag; }
int get_dir_rep() const { return dir_rep; }
protected:
~MDirUpdate() {}
- MDirUpdate() : MessageInstance(MSG_MDS_DIRUPDATE) {}
+ MDirUpdate() : Message{MSG_MDS_DIRUPDATE} {}
MDirUpdate(mds_rank_t f,
dirfrag_t dirfrag,
int dir_rep,
const std::set<int32_t>& dir_rep_by,
filepath& path,
bool discover = false) :
- MessageInstance(MSG_MDS_DIRUPDATE), from_mds(f), dirfrag(dirfrag),
+ Message{MSG_MDS_DIRUPDATE}, from_mds(f), dirfrag(dirfrag),
dir_rep(dir_rep), dir_rep_by(dir_rep_by), path(path) {
this->discover = discover ? 5 : 0;
}
MDirUpdate(const MDirUpdate& m)
- : MessageInstance(MSG_MDS_DIRUPDATE),
+ : Message{MSG_MDS_DIRUPDATE},
from_mds(m.from_mds),
dirfrag(m.dirfrag),
dir_rep(m.dir_rep),
std::set<int32_t> dir_rep_by;
filepath path;
mutable int tried_discover = 0; // XXX HACK
+
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include <string>
-class MDiscover : public MessageInstance<MDiscover> {
-public:
- friend factory;
+class MDiscover : public Message {
private:
-
inodeno_t base_ino; // 1 -> root
frag_t base_dir_frag;
void set_base_dir_frag(frag_t f) { base_dir_frag = f; }
protected:
- MDiscover() : MessageInstance(MSG_MDS_DISCOVER) { }
+ MDiscover() : Message{MSG_MDS_DISCOVER} { }
MDiscover(inodeno_t base_ino_,
frag_t base_frag_,
snapid_t s,
filepath& want_path_,
bool want_base_dir_ = true,
bool discover_xlocks_ = false) :
- MessageInstance(MSG_MDS_DISCOVER),
+ Message{MSG_MDS_DISCOVER},
base_ino(base_ino_),
base_dir_frag(base_frag_),
snapid(s),
encode(want_base_dir, payload);
encode(want_xlocked, payload);
}
-
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
*
*/
-class MDiscoverReply : public MessageInstance<MDiscoverReply> {
-public:
- friend factory;
+class MDiscoverReply : public Message {
private:
static constexpr int HEAD_VERSION = 2;
void set_base_dir_frag(frag_t df) { base_dir_frag = df; }
protected:
- MDiscoverReply() : MessageInstance(MSG_MDS_DISCOVERREPLY, HEAD_VERSION) { }
+ MDiscoverReply() : Message{MSG_MDS_DISCOVERREPLY, HEAD_VERSION} { }
MDiscoverReply(const MDiscover &dis) :
- MessageInstance(MSG_MDS_DISCOVERREPLY, HEAD_VERSION),
+ Message{MSG_MDS_DISCOVERREPLY, HEAD_VERSION},
base_ino(dis.get_base_ino()),
base_dir_frag(dis.get_base_dir_frag()),
wanted_base_dir(dis.wants_base_dir()),
header.tid = dis.get_tid();
}
MDiscoverReply(dirfrag_t df) :
- MessageInstance(MSG_MDS_DISCOVERREPLY, HEAD_VERSION),
+ Message{MSG_MDS_DISCOVERREPLY, HEAD_VERSION},
base_ino(df.ino),
base_dir_frag(df.frag),
wanted_base_dir(false),
encode(starts_with, payload);
encode(trace, payload);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MExportCaps : public MessageInstance<MExportCaps> {
-public:
- friend factory;
+class MExportCaps : public Message {
private:
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 1;
protected:
MExportCaps() :
- MessageInstance(MSG_MDS_EXPORTCAPS, HEAD_VERSION, COMPAT_VERSION) {}
+ Message{MSG_MDS_EXPORTCAPS, HEAD_VERSION, COMPAT_VERSION} {}
~MExportCaps() override {}
public:
if (header.version >= 2)
decode(client_metadata_map, p);
}
-
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MExportCapsAck : public MessageInstance<MExportCapsAck> {
+class MExportCapsAck : public Message {
public:
- friend factory;
-
inodeno_t ino;
bufferlist cap_bl;
protected:
MExportCapsAck() :
- MessageInstance(MSG_MDS_EXPORTCAPSACK) {}
+ Message{MSG_MDS_EXPORTCAPSACK} {}
MExportCapsAck(inodeno_t i) :
- MessageInstance(MSG_MDS_EXPORTCAPSACK), ino(i) {}
+ Message{MSG_MDS_EXPORTCAPSACK}, ino(i) {}
~MExportCapsAck() override {}
public:
decode(ino, p);
decode(cap_bl, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MExportDir : public MessageInstance<MExportDir> {
+class MExportDir : public Message {
public:
- friend factory;
dirfrag_t dirfrag;
bufferlist export_data;
vector<dirfrag_t> bounds;
bufferlist client_map;
protected:
- MExportDir() : MessageInstance(MSG_MDS_EXPORTDIR) {}
+ MExportDir() : Message{MSG_MDS_EXPORTDIR} {}
MExportDir(dirfrag_t df, uint64_t tid) :
- MessageInstance(MSG_MDS_EXPORTDIR), dirfrag(df) {
+ Message{MSG_MDS_EXPORTDIR}, dirfrag(df) {
set_tid(tid);
}
~MExportDir() override {}
decode(export_data, p);
decode(client_map, p);
}
-
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "MExportDir.h"
#include "msg/Message.h"
-class MExportDirAck : public MessageInstance<MExportDirAck> {
+class MExportDirAck : public Message {
public:
- friend factory;
-
dirfrag_t dirfrag;
bufferlist imported_caps;
dirfrag_t get_dirfrag() const { return dirfrag; }
protected:
- MExportDirAck() : MessageInstance(MSG_MDS_EXPORTDIRACK) {}
+ MExportDirAck() : Message{MSG_MDS_EXPORTDIRACK} {}
MExportDirAck(dirfrag_t df, uint64_t tid) :
- MessageInstance(MSG_MDS_EXPORTDIRACK), dirfrag(df) {
+ Message{MSG_MDS_EXPORTDIRACK}, dirfrag(df) {
set_tid(tid);
}
~MExportDirAck() override {}
encode(dirfrag, payload);
encode(imported_caps, payload);
}
-
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
#include "include/types.h"
-class MExportDirCancel : public MessageInstance<MExportDirCancel> {
-public:
- friend factory;
+class MExportDirCancel : public Message {
private:
dirfrag_t dirfrag;
dirfrag_t get_dirfrag() const { return dirfrag; }
protected:
- MExportDirCancel() : MessageInstance(MSG_MDS_EXPORTDIRCANCEL) {}
+ MExportDirCancel() : Message{MSG_MDS_EXPORTDIRCANCEL} {}
MExportDirCancel(dirfrag_t df, uint64_t tid) :
- MessageInstance(MSG_MDS_EXPORTDIRCANCEL), dirfrag(df) {
+ Message{MSG_MDS_EXPORTDIRCANCEL}, dirfrag(df) {
set_tid(tid);
}
~MExportDirCancel() override {}
auto p = payload.cbegin();
decode(dirfrag, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
#include "include/types.h"
-class MExportDirDiscover : public MessageInstance<MExportDirDiscover> {
-public:
- friend factory;
+class MExportDirDiscover : public Message {
private:
mds_rank_t from = -1;
dirfrag_t dirfrag;
protected:
MExportDirDiscover() :
- MessageInstance(MSG_MDS_EXPORTDIRDISCOVER),
+ Message{MSG_MDS_EXPORTDIRDISCOVER},
started(false) { }
MExportDirDiscover(dirfrag_t df, filepath& p, mds_rank_t f, uint64_t tid) :
- MessageInstance(MSG_MDS_EXPORTDIRDISCOVER),
+ Message{MSG_MDS_EXPORTDIRDISCOVER},
from(f), dirfrag(df), path(p), started(false) {
set_tid(tid);
}
encode(dirfrag, payload);
encode(path, payload);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
#include "include/types.h"
-class MExportDirDiscoverAck : public MessageInstance<MExportDirDiscoverAck> {
-public:
- friend factory;
+class MExportDirDiscoverAck : public Message {
private:
dirfrag_t dirfrag;
bool success;
bool is_success() const { return success; }
protected:
- MExportDirDiscoverAck() : MessageInstance(MSG_MDS_EXPORTDIRDISCOVERACK) {}
+ MExportDirDiscoverAck() : Message{MSG_MDS_EXPORTDIRDISCOVERACK} {}
MExportDirDiscoverAck(dirfrag_t df, uint64_t tid, bool s=true) :
- MessageInstance(MSG_MDS_EXPORTDIRDISCOVERACK),
+ Message{MSG_MDS_EXPORTDIRDISCOVERACK},
dirfrag(df), success(s) {
set_tid(tid);
}
encode(dirfrag, payload);
encode(success, payload);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MExportDirFinish : public MessageInstance<MExportDirFinish> {
-public:
- friend factory;
+class MExportDirFinish : public Message {
private:
dirfrag_t dirfrag;
bool last;
protected:
MExportDirFinish() : last(false) {}
MExportDirFinish(dirfrag_t df, bool l, uint64_t tid) :
- MessageInstance(MSG_MDS_EXPORTDIRFINISH), dirfrag(df), last(l) {
+ Message{MSG_MDS_EXPORTDIRFINISH}, dirfrag(df), last(l) {
set_tid(tid);
}
~MExportDirFinish() override {}
decode(dirfrag, p);
decode(last, p);
}
-
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MExportDirNotify : public MessageInstance<MExportDirNotify> {
-public:
- friend factory;
+class MExportDirNotify : public Message {
private:
dirfrag_t base;
bool ack;
protected:
MExportDirNotify() {}
MExportDirNotify(dirfrag_t i, uint64_t tid, bool a, pair<__s32,__s32> oa, pair<__s32,__s32> na) :
- MessageInstance(MSG_MDS_EXPORTDIRNOTIFY),
+ Message{MSG_MDS_EXPORTDIRNOTIFY},
base(i), ack(a), old_auth(oa), new_auth(na) {
set_tid(tid);
}
decode(new_auth, p);
decode(bounds, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MExportDirNotifyAck : public MessageInstance<MExportDirNotifyAck> {
-public:
- friend factory;
+class MExportDirNotifyAck : public Message {
private:
dirfrag_t dirfrag;
pair<__s32,__s32> new_auth;
protected:
MExportDirNotifyAck() {}
MExportDirNotifyAck(dirfrag_t df, uint64_t tid, pair<__s32,__s32> na) :
- MessageInstance(MSG_MDS_EXPORTDIRNOTIFYACK), dirfrag(df), new_auth(na) {
+ Message{MSG_MDS_EXPORTDIRNOTIFYACK}, dirfrag(df), new_auth(na) {
set_tid(tid);
}
~MExportDirNotifyAck() override {}
decode(dirfrag, p);
decode(new_auth, p);
}
-
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
#include "include/types.h"
-class MExportDirPrep : public MessageInstance<MExportDirPrep> {
-public:
- friend factory;
+class MExportDirPrep : public Message {
private:
dirfrag_t dirfrag;
public:
list<bufferlist> traces;
private:
set<mds_rank_t> bystanders;
- bool b_did_assim;
+ bool b_did_assim = false;
public:
dirfrag_t get_dirfrag() const { return dirfrag; }
void mark_assim() { b_did_assim = true; }
protected:
- MExportDirPrep() {
- b_did_assim = false;
- }
+ MExportDirPrep() = default;
MExportDirPrep(dirfrag_t df, uint64_t tid) :
- MessageInstance(MSG_MDS_EXPORTDIRPREP),
- dirfrag(df), b_did_assim(false) {
+ Message{MSG_MDS_EXPORTDIRPREP},
+ dirfrag(df)
+ {
set_tid(tid);
}
~MExportDirPrep() override {}
encode(traces, payload);
encode(bystanders, payload);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
#include "include/types.h"
-class MExportDirPrepAck : public MessageInstance<MExportDirPrepAck> {
-public:
- friend factory;
+class MExportDirPrepAck : public Message {
private:
dirfrag_t dirfrag;
bool success = false;
protected:
MExportDirPrepAck() {}
MExportDirPrepAck(dirfrag_t df, bool s, uint64_t tid) :
- MessageInstance(MSG_MDS_EXPORTDIRPREPACK), dirfrag(df), success(s) {
+ Message{MSG_MDS_EXPORTDIRPREPACK}, dirfrag(df), success(s) {
set_tid(tid);
}
~MExportDirPrepAck() override {}
encode(dirfrag, payload);
encode(success, payload);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "mds/FSMap.h"
#include "include/ceph_features.h"
-class MFSMap : public MessageInstance<MFSMap> {
+class MFSMap : public Message {
public:
- friend factory;
-
epoch_t epoch;
bufferlist encoded;
const FSMap& get_fsmap() const {return fsmap;}
MFSMap() :
- MessageInstance(CEPH_MSG_FS_MAP), epoch(0) {}
+ Message{CEPH_MSG_FS_MAP}, epoch(0) {}
MFSMap(const uuid_d &f, const FSMap &fsmap_) :
- MessageInstance(CEPH_MSG_FS_MAP), epoch(fsmap_.get_epoch())
- {
- fsmap = fsmap_;
- }
+ Message{CEPH_MSG_FS_MAP},
+ epoch(fsmap_.get_epoch()),
+ fsmap{fsmap_}
+ {}
private:
FSMap fsmap;
#include "mds/FSMapUser.h"
#include "include/ceph_features.h"
-class MFSMapUser : public MessageInstance<MFSMapUser> {
+class MFSMapUser : public Message {
public:
- friend factory;
-
epoch_t epoch;
version_t get_epoch() const { return epoch; }
const FSMapUser& get_fsmap() const { return fsmap; }
MFSMapUser() :
- MessageInstance(CEPH_MSG_FS_MAP_USER), epoch(0) {}
+ Message{CEPH_MSG_FS_MAP_USER}, epoch(0) {}
MFSMapUser(const uuid_d &f, const FSMapUser &fsmap_) :
- MessageInstance(CEPH_MSG_FS_MAP_USER), epoch(fsmap_.epoch)
- {
- fsmap = fsmap_;
- }
+ Message{CEPH_MSG_FS_MAP_USER},
+ epoch(fsmap_.epoch),
+ fsmap{fsmap_}
+ {}
private:
FSMapUser fsmap;
encode(epoch, payload);
encode(fsmap, payload, features);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "include/encoding.h"
#include "include/stringify.h"
-class MForward : public MessageInstance<MForward> {
+class MForward : public Message {
public:
- friend factory;
-
uint64_t tid;
uint8_t client_type;
entity_addrvec_t client_addrs;
static constexpr int HEAD_VERSION = 4;
static constexpr int COMPAT_VERSION = 4;
- MForward() : MessageInstance(MSG_FORWARD, HEAD_VERSION, COMPAT_VERSION),
+ MForward() : Message{MSG_FORWARD, HEAD_VERSION, COMPAT_VERSION},
tid(0), con_features(0), msg(NULL) {}
MForward(uint64_t t, PaxosServiceMessage *m, uint64_t feat,
const MonCap& caps) :
- MessageInstance(MSG_FORWARD, HEAD_VERSION, COMPAT_VERSION),
+ Message{MSG_FORWARD, HEAD_VERSION, COMPAT_VERSION},
tid(t), client_caps(caps), msg(NULL) {
client_type = m->get_source().type();
client_addrs = m->get_source_addrs();
<< " tid " << tid
<< " con_features " << con_features << ")";
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MGatherCaps : public MessageInstance<MGatherCaps> {
+class MGatherCaps : public Message {
public:
- friend factory;
-
-
inodeno_t ino;
protected:
MGatherCaps() :
- MessageInstance(MSG_MDS_GATHERCAPS) {}
+ Message{MSG_MDS_GATHERCAPS} {}
~MGatherCaps() override {}
public:
auto p = payload.cbegin();
decode(ino, p);
}
-
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MGenericMessage : public MessageInstance<MGenericMessage> {
-public:
- friend factory;
+class MGenericMessage : public Message {
private:
char tname[20];
//long pcid;
public:
- MGenericMessage(int t=0) : MessageInstance(t) {
+ MGenericMessage(int t=0) : Message{t} {
snprintf(tname, sizeof(tname), "generic%d", get_type());
}
#include "msg/Message.h"
-class MGetConfig : public MessageInstance<MGetConfig> {
+class MGetConfig : public Message {
public:
- friend factory;
-
static constexpr int HEAD_VERSION = 1;
static constexpr int COMPAT_VERSION = 1;
std::string host; ///< our hostname
std::string device_class;
- MGetConfig() : MessageInstance(MSG_GET_CONFIG, HEAD_VERSION, COMPAT_VERSION) { }
+ MGetConfig() : Message{MSG_GET_CONFIG, HEAD_VERSION, COMPAT_VERSION} { }
MGetConfig(const EntityName& n, const std::string& h)
- : MessageInstance(MSG_GET_CONFIG, HEAD_VERSION, COMPAT_VERSION),
+ : Message{MSG_GET_CONFIG, HEAD_VERSION, COMPAT_VERSION},
name(n),
host(h) {}
#include "messages/PaxosServiceMessage.h"
-class MGetPoolStats : public MessageInstance<MGetPoolStats, PaxosServiceMessage> {
+class MGetPoolStats : public PaxosServiceMessage {
public:
- friend factory;
-
uuid_d fsid;
std::list<std::string> pools;
- MGetPoolStats() : MessageInstance(MSG_GETPOOLSTATS, 0) {}
+ MGetPoolStats() : PaxosServiceMessage{MSG_GETPOOLSTATS, 0} {}
MGetPoolStats(const uuid_d& f, ceph_tid_t t, std::list<std::string>& ls, version_t l) :
- MessageInstance(MSG_GETPOOLSTATS, l),
+ PaxosServiceMessage{MSG_GETPOOLSTATS, l},
fsid(f), pools(ls) {
set_tid(t);
}
#ifndef CEPH_MGETPOOLSTATSREPLY_H
#define CEPH_MGETPOOLSTATSREPLY_H
-class MGetPoolStatsReply : public MessageInstance<MGetPoolStatsReply, PaxosServiceMessage> {
+class MGetPoolStatsReply : public PaxosServiceMessage {
public:
- friend factory;
-
uuid_d fsid;
std::map<std::string,pool_stat_t> pool_stats;
- MGetPoolStatsReply() : MessageInstance(MSG_GETPOOLSTATSREPLY, 0) {}
+ MGetPoolStatsReply() : PaxosServiceMessage{MSG_GETPOOLSTATSREPLY, 0} {}
MGetPoolStatsReply(uuid_d& f, ceph_tid_t t, version_t v) :
- MessageInstance(MSG_GETPOOLSTATSREPLY, v),
+ PaxosServiceMessage{MSG_GETPOOLSTATSREPLY, v},
fsid(f) {
set_tid(t);
}
decode(fsid, p);
decode(pool_stats, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
#include "common/DecayCounter.h"
-class MHeartbeat : public MessageInstance<MHeartbeat> {
-public:
- friend factory;
+class MHeartbeat : public Message {
private:
mds_load_t load;
__s32 beat = 0;
map<mds_rank_t, float>& get_import_map() { return import_map; }
protected:
- MHeartbeat() : MessageInstance(MSG_MDS_HEARTBEAT), load(DecayRate()) {}
+ MHeartbeat() : Message(MSG_MDS_HEARTBEAT), load(DecayRate()) {}
MHeartbeat(mds_load_t& load, int beat)
- : MessageInstance(MSG_MDS_HEARTBEAT),
- load(load) {
- this->beat = beat;
- }
+ : Message(MSG_MDS_HEARTBEAT),
+ load(load),
+ beat(beat)
+ {}
~MHeartbeat() override {}
public:
decode(beat, p);
decode(import_map, p);
}
-
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MInodeFileCaps : public MessageInstance<MInodeFileCaps> {
-public:
- friend factory;
+class MInodeFileCaps : public Message {
private:
inodeno_t ino;
__u32 caps = 0;
int get_caps() const { return caps; }
protected:
- MInodeFileCaps() : MessageInstance(MSG_MDS_INODEFILECAPS) {}
+ MInodeFileCaps() : Message{MSG_MDS_INODEFILECAPS} {}
MInodeFileCaps(inodeno_t ino, int caps) :
- MessageInstance(MSG_MDS_INODEFILECAPS) {
+ Message{MSG_MDS_INODEFILECAPS} {
this->ino = ino;
this->caps = caps;
}
decode(ino, p);
decode(caps, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "mds/locks.h"
#include "mds/SimpleLock.h"
-class MLock : public MessageInstance<MLock> {
-public:
- friend factory;
+class MLock : public Message {
private:
int32_t action = 0; // action type
mds_rank_t asker = 0; // who is initiating this request
MDSCacheObjectInfo &get_object_info() { return object_info; }
protected:
- MLock() : MessageInstance(MSG_MDS_LOCK) {}
+ MLock() : Message{MSG_MDS_LOCK} {}
MLock(int ac, mds_rank_t as) :
- MessageInstance(MSG_MDS_LOCK),
+ Message{MSG_MDS_LOCK},
action(ac), asker(as),
lock_type(0) { }
MLock(SimpleLock *lock, int ac, mds_rank_t as) :
- MessageInstance(MSG_MDS_LOCK),
+ Message{MSG_MDS_LOCK},
action(ac), asker(as),
lock_type(lock->get_type()) {
lock->get_parent()->set_object_info(object_info);
}
MLock(SimpleLock *lock, int ac, mds_rank_t as, bufferlist& bl) :
- MessageInstance(MSG_MDS_LOCK),
+ Message{MSG_MDS_LOCK},
action(ac), asker(as), lock_type(lock->get_type()) {
lock->get_parent()->set_object_info(object_info);
lockdata.claim(bl);
encode(object_info, payload);
encode(lockdata, payload);
}
-
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include <deque>
-class MLog : public MessageInstance<MLog, PaxosServiceMessage> {
+class MLog : public PaxosServiceMessage {
public:
- friend factory;
-
uuid_d fsid;
std::deque<LogEntry> entries;
- MLog() : MessageInstance(MSG_LOG, 0) {}
+ MLog() : PaxosServiceMessage{MSG_LOG, 0} {}
MLog(const uuid_d& f, const std::deque<LogEntry>& e)
- : MessageInstance(MSG_LOG, 0), fsid(f), entries(e) { }
- MLog(const uuid_d& f) : MessageInstance(MSG_LOG, 0), fsid(f) { }
+ : PaxosServiceMessage{MSG_LOG, 0}, fsid(f), entries(e) { }
+ MLog(const uuid_d& f) : PaxosServiceMessage(MSG_LOG, 0), fsid(f) { }
private:
~MLog() override {}
#include "msg/Message.h"
-class MLogAck : public MessageInstance<MLogAck> {
+class MLogAck : public Message {
public:
- friend factory;
-
uuid_d fsid;
version_t last = 0;
std::string channel;
- MLogAck() : MessageInstance(MSG_LOGACK) {}
- MLogAck(uuid_d& f, version_t l) : MessageInstance(MSG_LOGACK), fsid(f), last(l) {}
+ MLogAck() : Message{MSG_LOGACK} {}
+ MLogAck(uuid_d& f, version_t l) : Message{MSG_LOGACK}, fsid(f), last(l) {}
private:
~MLogAck() override {}
if (!p.end())
decode(channel, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
WRITE_CLASS_ENCODER(MDSHealth)
-class MMDSBeacon : public MessageInstance<MMDSBeacon, PaxosServiceMessage> {
-public:
- friend factory;
+class MMDSBeacon : public PaxosServiceMessage {
private:
static constexpr int HEAD_VERSION = 7;
uint64_t mds_features = 0;
protected:
- MMDSBeacon() : MessageInstance(MSG_MDS_BEACON, 0, HEAD_VERSION, COMPAT_VERSION)
+ MMDSBeacon() : PaxosServiceMessage(MSG_MDS_BEACON, 0, HEAD_VERSION, COMPAT_VERSION)
{
set_priority(CEPH_MSG_PRIO_HIGH);
}
MMDSBeacon(const uuid_d &f, mds_gid_t g, const string& n, epoch_t les, MDSMap::DaemonState st, version_t se, uint64_t feat) :
- MessageInstance(MSG_MDS_BEACON, les, HEAD_VERSION, COMPAT_VERSION),
+ PaxosServiceMessage(MSG_MDS_BEACON, les, HEAD_VERSION, COMPAT_VERSION),
fsid(f), global_id(g), name(n), state(st), seq(se),
mds_features(feat) {
set_priority(CEPH_MSG_PRIO_HIGH);
state = MDSMap::STATE_STANDBY;
}
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
// sent from replica to auth
-class MMDSCacheRejoin : public MessageInstance<MMDSCacheRejoin> {
-public:
- friend factory;
+class MMDSCacheRejoin : public Message {
private:
-
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 1;
protected:
MMDSCacheRejoin() :
- MessageInstance(MSG_MDS_CACHEREJOIN, HEAD_VERSION, COMPAT_VERSION),
- op(0) {}
+ MMDSCacheRejoin{0}
+ {}
MMDSCacheRejoin(int o) :
- MessageInstance(MSG_MDS_CACHEREJOIN, HEAD_VERSION, COMPAT_VERSION),
+ Message{MSG_MDS_CACHEREJOIN, HEAD_VERSION, COMPAT_VERSION},
op(o) {}
~MMDSCacheRejoin() override {}
if (header.version >= 2)
decode(client_metadata_map, p);
}
-
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
WRITE_CLASS_ENCODER(MMDSCacheRejoin::inode_strong)
#include "msg/Message.h"
#include "include/filepath.h"
-class MMDSFindIno : public MessageInstance<MMDSFindIno> {
+class MMDSFindIno : public Message {
public:
- friend factory;
-
ceph_tid_t tid {0};
inodeno_t ino;
protected:
- MMDSFindIno() : MessageInstance(MSG_MDS_FINDINO) {}
- MMDSFindIno(ceph_tid_t t, inodeno_t i) : MessageInstance(MSG_MDS_FINDINO), tid(t), ino(i) {}
+ MMDSFindIno() : Message{MSG_MDS_FINDINO} {}
+ MMDSFindIno(ceph_tid_t t, inodeno_t i) : Message{MSG_MDS_FINDINO}, tid(t), ino(i) {}
~MMDSFindIno() override {}
public:
decode(tid, p);
decode(ino, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
#include "include/filepath.h"
-class MMDSFindInoReply : public MessageInstance<MMDSFindInoReply> {
+class MMDSFindInoReply : public Message {
public:
- friend factory;
-
ceph_tid_t tid = 0;
filepath path;
protected:
- MMDSFindInoReply() : MessageInstance(MSG_MDS_FINDINOREPLY) {}
- MMDSFindInoReply(ceph_tid_t t) : MessageInstance(MSG_MDS_FINDINOREPLY), tid(t) {}
+ MMDSFindInoReply() : Message{MSG_MDS_FINDINOREPLY} {}
+ MMDSFindInoReply(ceph_tid_t t) : Message{MSG_MDS_FINDINOREPLY}, tid(t) {}
~MMDSFindInoReply() override {}
public:
decode(tid, p);
decode(path, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MMDSFragmentNotify : public MessageInstance<MMDSFragmentNotify> {
-public:
- friend factory;
+class MMDSFragmentNotify : public Message {
private:
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 1;
protected:
MMDSFragmentNotify() :
- MessageInstance(MSG_MDS_FRAGMENTNOTIFY, HEAD_VERSION, COMPAT_VERSION) {}
+ Message{MSG_MDS_FRAGMENTNOTIFY, HEAD_VERSION, COMPAT_VERSION} {}
MMDSFragmentNotify(dirfrag_t df, int b, uint64_t tid) :
- MessageInstance(MSG_MDS_FRAGMENTNOTIFY, HEAD_VERSION, COMPAT_VERSION),
+ Message{MSG_MDS_FRAGMENTNOTIFY, HEAD_VERSION, COMPAT_VERSION},
base_dirfrag(df), bits(b) {
set_tid(tid);
}
if (header.version >= 2)
decode(ack_wanted, p);
}
-
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MMDSFragmentNotifyAck : public MessageInstance<MMDSFragmentNotifyAck> {
-public:
- friend factory;
+class MMDSFragmentNotifyAck : public Message {
private:
dirfrag_t base_dirfrag;
int8_t bits = 0;
bufferlist basebl;
protected:
- MMDSFragmentNotifyAck() : MessageInstance(MSG_MDS_FRAGMENTNOTIFYACK) {}
+ MMDSFragmentNotifyAck() : Message{MSG_MDS_FRAGMENTNOTIFYACK} {}
MMDSFragmentNotifyAck(dirfrag_t df, int b, uint64_t tid) :
- MessageInstance(MSG_MDS_FRAGMENTNOTIFYACK),
+ Message{MSG_MDS_FRAGMENTNOTIFYACK},
base_dirfrag(df), bits(b) {
set_tid(tid);
}
decode(base_dirfrag, p);
decode(bits, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include <map>
using std::map;
-class MMDSLoadTargets : public MessageInstance<MMDSLoadTargets, PaxosServiceMessage> {
+class MMDSLoadTargets : public PaxosServiceMessage {
public:
- friend factory;
-
mds_gid_t global_id;
set<mds_rank_t> targets;
protected:
- MMDSLoadTargets() : MessageInstance(MSG_MDS_OFFLOAD_TARGETS, 0) {}
+ MMDSLoadTargets() : PaxosServiceMessage(MSG_MDS_OFFLOAD_TARGETS, 0) {}
MMDSLoadTargets(mds_gid_t g, set<mds_rank_t>& mds_targets) :
- MessageInstance(MSG_MDS_OFFLOAD_TARGETS, 0),
+ PaxosServiceMessage(MSG_MDS_OFFLOAD_TARGETS, 0),
global_id(g), targets(mds_targets) {}
~MMDSLoadTargets() override {}
encode(global_id, payload);
encode(targets, payload);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "mds/MDSMap.h"
#include "include/ceph_features.h"
-class MMDSMap : public MessageInstance<MMDSMap> {
-public:
- friend factory;
+class MMDSMap : public Message {
private:
static constexpr int HEAD_VERSION = 1;
static constexpr int COMPAT_VERSION = 1;
protected:
MMDSMap() :
- MessageInstance(CEPH_MSG_MDS_MAP, HEAD_VERSION, COMPAT_VERSION) {}
+ Message{CEPH_MSG_MDS_MAP, HEAD_VERSION, COMPAT_VERSION} {}
MMDSMap(const uuid_d &f, const MDSMap &mm) :
- MessageInstance(CEPH_MSG_MDS_MAP, HEAD_VERSION, COMPAT_VERSION),
+ Message{CEPH_MSG_MDS_MAP, HEAD_VERSION, COMPAT_VERSION},
fsid(f) {
epoch = mm.get_epoch();
mm.encode(encoded, -1); // we will reencode with fewer features as necessary
}
encode(encoded, payload);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MMDSOpenIno : public MessageInstance<MMDSOpenIno> {
+class MMDSOpenIno : public Message {
public:
- friend factory;
-
inodeno_t ino;
vector<inode_backpointer_t> ancestors;
protected:
- MMDSOpenIno() : MessageInstance(MSG_MDS_OPENINO) {}
+ MMDSOpenIno() : Message{MSG_MDS_OPENINO} {}
MMDSOpenIno(ceph_tid_t t, inodeno_t i, vector<inode_backpointer_t>* pa) :
- MessageInstance(MSG_MDS_OPENINO), ino(i) {
+ Message{MSG_MDS_OPENINO}, ino(i) {
header.tid = t;
if (pa)
ancestors = *pa;
decode(ino, p);
decode(ancestors, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MMDSOpenInoReply : public MessageInstance<MMDSOpenInoReply> {
+class MMDSOpenInoReply : public Message {
public:
- friend factory;
-
inodeno_t ino;
vector<inode_backpointer_t> ancestors;
mds_rank_t hint;
int32_t error;
protected:
- MMDSOpenInoReply() : MessageInstance(MSG_MDS_OPENINOREPLY), error(0) {}
+ MMDSOpenInoReply() : Message{MSG_MDS_OPENINOREPLY}, error(0) {}
MMDSOpenInoReply(ceph_tid_t t, inodeno_t i, mds_rank_t h=MDS_RANK_NONE, int e=0) :
- MessageInstance(MSG_MDS_OPENINOREPLY), ino(i), hint(h), error(e) {
+ Message{MSG_MDS_OPENINOREPLY}, ino(i), hint(h), error(e) {
header.tid = t;
}
decode(hint, p);
decode(error, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "include/types.h"
-class MMDSResolve : public MessageInstance<MMDSResolve> {
+class MMDSResolve : public Message {
public:
- friend factory;
-
map<dirfrag_t, vector<dirfrag_t> > subtrees;
map<dirfrag_t, vector<dirfrag_t> > ambiguous_imports;
list<table_client> table_clients;
protected:
- MMDSResolve() : MessageInstance(MSG_MDS_RESOLVE) {}
+ MMDSResolve() : Message{MSG_MDS_RESOLVE}
+ {}
~MMDSResolve() override {}
public:
decode(slave_requests, p);
decode(table_clients, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
inline ostream& operator<<(ostream& out, const MMDSResolve::slave_request&) {
#include "include/types.h"
-class MMDSResolveAck : public MessageInstance<MMDSResolveAck> {
+class MMDSResolveAck : public Message {
public:
- friend factory;
-
map<metareqid_t, bufferlist> commit;
vector<metareqid_t> abort;
protected:
- MMDSResolveAck() : MessageInstance(MSG_MDS_RESOLVEACK) {}
+ MMDSResolveAck() : Message{MSG_MDS_RESOLVEACK} {}
~MMDSResolveAck() override {}
public:
decode(commit, p);
decode(abort, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
#include "mds/mdstypes.h"
-class MMDSSlaveRequest : public MessageInstance<MMDSSlaveRequest> {
+class MMDSSlaveRequest : public Message {
public:
- friend factory;
-
static constexpr int OP_XLOCK = 1;
static constexpr int OP_XLOCKACK = -1;
static constexpr int OP_UNXLOCK = 2;
bufferlist& get_lock_data() { return inode_export; }
protected:
- MMDSSlaveRequest() : MessageInstance(MSG_MDS_SLAVE_REQUEST) { }
+ MMDSSlaveRequest() : Message{MSG_MDS_SLAVE_REQUEST} { }
MMDSSlaveRequest(metareqid_t ri, __u32 att, int o) :
- MessageInstance(MSG_MDS_SLAVE_REQUEST),
+ Message{MSG_MDS_SLAVE_REQUEST},
reqid(ri), attempt(att), op(o), flags(0), lock_type(0),
inode_export_v(0), srcdn_auth(MDS_RANK_NONE) { }
~MMDSSlaveRequest() override {}
<< " " << get_opname(op)
<< ")";
}
-
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MMDSSnapUpdate : public MessageInstance<MMDSSnapUpdate> {
-public:
- friend factory;
+class MMDSSnapUpdate : public Message {
private:
-
inodeno_t ino;
__s16 snap_op;
bufferlist snap_blob;
protected:
- MMDSSnapUpdate() : MessageInstance(MSG_MDS_SNAPUPDATE) {}
+ MMDSSnapUpdate() : Message{MSG_MDS_SNAPUPDATE} {}
MMDSSnapUpdate(inodeno_t i, version_t tid, int op) :
- MessageInstance(MSG_MDS_SNAPUPDATE), ino(i), snap_op(op) {
+ Message{MSG_MDS_SNAPUPDATE}, ino(i), snap_op(op) {
set_tid(tid);
}
~MMDSSnapUpdate() override {}
decode(snap_op, p);
decode(snap_blob, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
#include "mds/mds_table_types.h"
-class MMDSTableRequest : public MessageInstance<MMDSTableRequest> {
+class MMDSTableRequest : public Message {
public:
- friend factory;
-
__u16 table = 0;
__s16 op = 0;
uint64_t reqid = 0;
bufferlist bl;
protected:
- MMDSTableRequest() : MessageInstance(MSG_MDS_TABLE_REQUEST) {}
+ MMDSTableRequest() : Message{MSG_MDS_TABLE_REQUEST} {}
MMDSTableRequest(int tab, int o, uint64_t r, version_t v=0) :
- MessageInstance(MSG_MDS_TABLE_REQUEST),
+ Message{MSG_MDS_TABLE_REQUEST},
table(tab), op(o), reqid(r) {
set_tid(v);
}
encode(reqid, payload);
encode(bl, payload);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "include/types.h"
-class MMgrBeacon : public MessageInstance<MMgrBeacon, PaxosServiceMessage> {
-public:
- friend factory;
+class MMgrBeacon : public PaxosServiceMessage {
private:
-
static constexpr int HEAD_VERSION = 8;
static constexpr int COMPAT_VERSION = 8;
public:
MMgrBeacon()
- : MessageInstance(MSG_MGR_BEACON, 0, HEAD_VERSION, COMPAT_VERSION),
+ : PaxosServiceMessage{MSG_MGR_BEACON, 0, HEAD_VERSION, COMPAT_VERSION},
gid(0), available(false)
- {
- }
+ {}
MMgrBeacon(const uuid_d& fsid_, uint64_t gid_, const std::string &name_,
entity_addrvec_t server_addrs_, bool available_,
std::vector<MgrMap::ModuleInfo>&& modules_,
map<string,string>&& metadata_)
- : MessageInstance(MSG_MGR_BEACON, 0, HEAD_VERSION, COMPAT_VERSION),
+ : PaxosServiceMessage{MSG_MGR_BEACON, 0, HEAD_VERSION, COMPAT_VERSION},
gid(gid_), server_addrs(server_addrs_), available(available_), name(name_),
fsid(fsid_), modules(std::move(modules_)), metadata(std::move(metadata_))
{
decode(modules, p);
}
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#include "msg/Message.h"
-class MMgrClose : public MessageInstance<MMgrClose> {
-public:
- friend factory;
+class MMgrClose : public Message {
private:
static constexpr int HEAD_VERSION = 1;
}
MMgrClose()
- : MessageInstance(MSG_MGR_CLOSE, HEAD_VERSION, COMPAT_VERSION)
+ : Message{MSG_MGR_CLOSE, HEAD_VERSION, COMPAT_VERSION}
{}
+
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
* This message is sent from ceph-mgr to MgrClient, instructing it
* it about what data to send back to ceph-mgr at what frequency.
*/
-class MMgrConfigure : public MessageInstance<MMgrConfigure> {
-public:
- friend factory;
+class MMgrConfigure : public Message {
private:
-
static constexpr int HEAD_VERSION = 3;
static constexpr int COMPAT_VERSION = 1;
<< ", threshold=" << stats_threshold << ")";
}
+private:
MMgrConfigure()
- : MessageInstance(MSG_MGR_CONFIGURE, HEAD_VERSION, COMPAT_VERSION)
+ : Message{MSG_MGR_CONFIGURE, HEAD_VERSION, COMPAT_VERSION}
{}
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
* The mgr digest is a way for the mgr to subscribe to things
* other than the cluster maps, which are needed by
*/
-class MMgrDigest : public MessageInstance<MMgrDigest> {
+class MMgrDigest : public Message {
public:
- friend factory;
-
bufferlist mon_status_json;
bufferlist health_json;
- MMgrDigest() :
- MessageInstance(MSG_MGR_DIGEST) {}
-
std::string_view get_type_name() const override { return "mgrdigest"; }
void print(ostream& out) const override {
out << get_type_name();
}
private:
+ MMgrDigest() :
+ Message{MSG_MGR_DIGEST} {}
~MMgrDigest() override {}
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
#include "mon/MgrMap.h"
-class MMgrMap : public MessageInstance<MMgrMap> {
-public:
- friend factory;
-
+class MMgrMap : public Message {
protected:
MgrMap map;
public:
const MgrMap & get_map() {return map;}
+private:
MMgrMap() :
- MessageInstance(MSG_MGR_MAP) {}
+ Message{MSG_MGR_MAP} {}
MMgrMap(const MgrMap &map_) :
- MessageInstance(MSG_MGR_MAP), map(map_)
- {
- }
-
-private:
+ Message{MSG_MGR_MAP}, map(map_)
+ {}
~MMgrMap() override {}
public:
using ceph::encode;
encode(map, payload, features);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MMgrOpen : public MessageInstance<MMgrOpen> {
-public:
- friend factory;
+class MMgrOpen : public Message {
private:
-
static constexpr int HEAD_VERSION = 3;
static constexpr int COMPAT_VERSION = 1;
out << ")";
}
+private:
MMgrOpen()
- : MessageInstance(MSG_MGR_OPEN, HEAD_VERSION, COMPAT_VERSION)
+ : Message{MSG_MGR_OPEN, HEAD_VERSION, COMPAT_VERSION}
{}
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
};
WRITE_CLASS_ENCODER(PerfCounterType)
-class MMgrReport : public MessageInstance<MMgrReport> {
-public:
- friend factory;
+class MMgrReport : public Message {
private:
-
static constexpr int HEAD_VERSION = 7;
static constexpr int COMPAT_VERSION = 1;
}
MMgrReport()
- : MessageInstance(MSG_MGR_REPORT, HEAD_VERSION, COMPAT_VERSION)
+ : Message{MSG_MGR_REPORT, HEAD_VERSION, COMPAT_VERSION}
{}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include <vector>
#include <string>
-class MMonCommand : public MessageInstance<MMonCommand, PaxosServiceMessage> {
+class MMonCommand : public PaxosServiceMessage {
public:
- friend factory;
-
uuid_d fsid;
std::vector<std::string> cmd;
- MMonCommand() : MessageInstance(MSG_MON_COMMAND, 0) {}
+ MMonCommand() : PaxosServiceMessage{MSG_MON_COMMAND, 0} {}
MMonCommand(const uuid_d &f)
- : MessageInstance(MSG_MON_COMMAND, 0),
+ : PaxosServiceMessage{MSG_MON_COMMAND, 0},
fsid(f)
{ }
decode(fsid, p);
decode(cmd, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "messages/PaxosServiceMessage.h"
-class MMonCommandAck : public MessageInstance<MMonCommandAck, PaxosServiceMessage> {
+class MMonCommandAck : public PaxosServiceMessage {
public:
- friend factory;
-
std::vector<std::string> cmd;
errorcode32_t r;
std::string rs;
- MMonCommandAck() : MessageInstance(MSG_MON_COMMAND_ACK, 0) {}
+ MMonCommandAck() : PaxosServiceMessage{MSG_MON_COMMAND_ACK, 0} {}
MMonCommandAck(std::vector<std::string>& c, int _r, std::string s, version_t v) :
- MessageInstance(MSG_MON_COMMAND_ACK, v),
+ PaxosServiceMessage{MSG_MON_COMMAND_ACK, v},
cmd(c), r(_r), rs(s) { }
private:
~MMonCommandAck() override {}
decode(rs, p);
decode(cmd, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "mon/MonMap.h"
#include "mon/mon_types.h"
-class MMonElection : public MessageInstance<MMonElection> {
-public:
- friend factory;
-
+class MMonElection : public Message {
private:
static constexpr int HEAD_VERSION = 8;
static constexpr int COMPAT_VERSION = 5;
bufferlist sharing_bl;
map<string,string> metadata;
- MMonElection() : MessageInstance(MSG_MON_ELECTION, HEAD_VERSION, COMPAT_VERSION),
+ MMonElection() : Message{MSG_MON_ELECTION, HEAD_VERSION, COMPAT_VERSION},
op(0), epoch(0),
quorum_features(0),
mon_features(0)
{ }
MMonElection(int o, epoch_t e, MonMap *m)
- : MessageInstance(MSG_MON_ELECTION, HEAD_VERSION, COMPAT_VERSION),
+ : Message{MSG_MON_ELECTION, HEAD_VERSION, COMPAT_VERSION},
fsid(m->fsid), op(o), epoch(e),
quorum_features(0),
mon_features(0)
else
mon_release = infer_ceph_release_from_mon_features(mon_features);
}
-
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "include/types.h"
-class MMonGetMap : public MessageInstance<MMonGetMap> {
+class MMonGetMap : public Message {
public:
- friend factory;
-
- MMonGetMap() : MessageInstance(CEPH_MSG_MON_GET_MAP) { }
+ MMonGetMap() : Message{CEPH_MSG_MON_GET_MAP} { }
private:
~MMonGetMap() override {}
void encode_payload(uint64_t features) override { }
void decode_payload() override { }
+
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "include/types.h"
-class MMonGetOSDMap : public MessageInstance<MMonGetOSDMap, PaxosServiceMessage> {
-public:
- friend factory;
+class MMonGetOSDMap : public PaxosServiceMessage {
private:
-
epoch_t full_first, full_last;
epoch_t inc_first, inc_last;
public:
MMonGetOSDMap()
- : MessageInstance(CEPH_MSG_MON_GET_OSDMAP, 0),
+ : PaxosServiceMessage{CEPH_MSG_MON_GET_OSDMAP, 0},
full_first(0),
full_last(0),
inc_first(0),
decode(inc_first, p);
decode(inc_last, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
* can be used to determine whether a pool actually does not exist, or
* if it may have been created but the map was not received yet.
*/
-class MMonGetVersion : public MessageInstance<MMonGetVersion> {
+class MMonGetVersion : public Message {
public:
- friend factory;
-
- MMonGetVersion() : MessageInstance(CEPH_MSG_MON_GET_VERSION) {}
+ MMonGetVersion() : Message{CEPH_MSG_MON_GET_VERSION} {}
std::string_view get_type_name() const override {
return "mon_get_version";
* MMonGetVersion. The latest version of the requested thing is sent
* back.
*/
-class MMonGetVersionReply : public MessageInstance<MMonGetVersionReply> {
-public:
- friend factory;
+class MMonGetVersionReply : public Message {
private:
static constexpr int HEAD_VERSION = 2;
public:
- MMonGetVersionReply() : MessageInstance(CEPH_MSG_MON_GET_VERSION_REPLY, HEAD_VERSION) { }
+ MMonGetVersionReply() : Message{CEPH_MSG_MON_GET_VERSION_REPLY, HEAD_VERSION} { }
std::string_view get_type_name() const override {
return "mon_get_version_reply";
#include "messages/PaxosServiceMessage.h"
-class MMonGlobalID : public MessageInstance<MMonGlobalID, PaxosServiceMessage> {
+class MMonGlobalID : public PaxosServiceMessage {
public:
- friend factory;
-
- uint64_t old_max_id;
- MMonGlobalID() : MessageInstance(MSG_MON_GLOBAL_ID, 0), old_max_id(0) { }
+ uint64_t old_max_id = 0;
+ MMonGlobalID() : PaxosServiceMessage{MSG_MON_GLOBAL_ID, 0}
+ {}
private:
~MMonGlobalID() override {}
paxos_encode();
encode(old_max_id, payload);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "messages/MMonQuorumService.h"
#include "mon/mon_types.h"
-class MMonHealth : public MessageInstance<MMonHealth, MMonQuorumService> {
+class MMonHealth : public MMonQuorumService {
public:
- friend factory;
-
static constexpr int HEAD_VERSION = 1;
int service_type = 0;
// service specific data
DataStats data_stats;
- MMonHealth() : MessageInstance(MSG_MON_HEALTH, HEAD_VERSION) { }
+ MMonHealth() : MMonQuorumService{MSG_MON_HEALTH, HEAD_VERSION} { }
private:
~MMonHealth() override { }
encode(service_op, payload);
encode(data_stats, payload);
}
-
};
#endif /* CEPH_MMON_HEALTH_H */
#include "messages/PaxosServiceMessage.h"
#include "mon/health_check.h"
-class MMonHealthChecks : public MessageInstance<MMonHealthChecks, PaxosServiceMessage> {
+class MMonHealthChecks : public PaxosServiceMessage {
public:
- friend factory;
-
static constexpr int HEAD_VERSION = 1;
static constexpr int COMPAT_VERSION = 1;
health_check_map_t health_checks;
MMonHealthChecks()
- : MessageInstance(MSG_MON_HEALTH_CHECKS, HEAD_VERSION, COMPAT_VERSION) {
+ : PaxosServiceMessage{MSG_MON_HEALTH_CHECKS, HEAD_VERSION, COMPAT_VERSION} {
}
MMonHealthChecks(health_check_map_t& m)
- : MessageInstance(MSG_MON_HEALTH_CHECKS, HEAD_VERSION, COMPAT_VERSION),
- health_checks(m) {
- }
+ : PaxosServiceMessage{MSG_MON_HEALTH_CHECKS, HEAD_VERSION, COMPAT_VERSION},
+ health_checks(m)
+ {}
private:
~MMonHealthChecks() override { }
paxos_encode();
encode(health_checks, payload);
}
-
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include <vector>
using std::vector;
-class MMonJoin : public MessageInstance<MMonJoin, PaxosServiceMessage> {
+class MMonJoin : public PaxosServiceMessage {
public:
- friend factory;
-
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 2;
string name;
entity_addrvec_t addrs;
- MMonJoin() : MessageInstance(MSG_MON_JOIN, 0, HEAD_VERSION, COMPAT_VERSION) {}
+ MMonJoin() : PaxosServiceMessage{MSG_MON_JOIN, 0, HEAD_VERSION, COMPAT_VERSION} {}
MMonJoin(uuid_d &f, string n, const entity_addrvec_t& av)
- : MessageInstance(MSG_MON_JOIN, 0, HEAD_VERSION, COMPAT_VERSION),
+ : PaxosServiceMessage{MSG_MON_JOIN, 0, HEAD_VERSION, COMPAT_VERSION},
fsid(f), name(n), addrs(av)
{ }
#include "msg/MessageRef.h"
#include "mon/MonMap.h"
-class MMonMap : public MessageInstance<MMonMap> {
+class MMonMap : public Message {
public:
- friend factory;
-
ceph::buffer::list monmapbl;
- MMonMap() : MessageInstance(CEPH_MSG_MON_MAP) { }
- explicit MMonMap(ceph::buffer::list &bl) : MessageInstance(CEPH_MSG_MON_MAP) {
+ MMonMap() : Message{CEPH_MSG_MON_MAP} { }
+ explicit MMonMap(ceph::buffer::list &bl) : Message{CEPH_MSG_MON_MAP} {
monmapbl.claim(bl);
}
private:
auto p = payload.cbegin();
decode(monmapbl, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "mon/mon_types.h"
#include "msg/Message.h"
-class MMonMetadata : public MessageInstance<MMonMetadata> {
+class MMonMetadata : public Message {
public:
- friend factory;
-
Metadata data;
private:
public:
MMonMetadata() :
- MessageInstance(CEPH_MSG_MON_METADATA)
+ Message{CEPH_MSG_MON_METADATA}
{}
MMonMetadata(const Metadata& metadata) :
- MessageInstance(CEPH_MSG_MON_METADATA, HEAD_VERSION),
+ Message{CEPH_MSG_MON_METADATA, HEAD_VERSION},
data(metadata)
{}
auto p = payload.cbegin();
decode(data, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "mon/health_check.h"
#include "mon/PGMap.h"
-class MMonMgrReport
- : public MessageInstance<MMonMgrReport, PaxosServiceMessage> {
-public:
- friend factory;
+class MMonMgrReport : public PaxosServiceMessage {
private:
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 1;
std::map<std::string,ProgressEvent> progress_events;
MMonMgrReport()
- : MessageInstance(MSG_MON_MGR_REPORT, 0, HEAD_VERSION, COMPAT_VERSION)
+ : PaxosServiceMessage{MSG_MON_MGR_REPORT, 0, HEAD_VERSION, COMPAT_VERSION}
{}
private:
~MMonMgrReport() override {}
decode(progress_events, p);
}
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "mon/mon_types.h"
#include "include/ceph_features.h"
-class MMonPaxos : public MessageInstance<MMonPaxos> {
-public:
- friend factory;
+class MMonPaxos : public Message {
private:
static constexpr int HEAD_VERSION = 4;
static constexpr int COMPAT_VERSION = 3;
bufferlist feature_map;
- MMonPaxos() : MessageInstance(MSG_MON_PAXOS, HEAD_VERSION, COMPAT_VERSION) { }
+ MMonPaxos() : Message{MSG_MON_PAXOS, HEAD_VERSION, COMPAT_VERSION} { }
MMonPaxos(epoch_t e, int o, utime_t now) :
- MessageInstance(MSG_MON_PAXOS, HEAD_VERSION, COMPAT_VERSION),
+ Message{MSG_MON_PAXOS, HEAD_VERSION, COMPAT_VERSION},
epoch(e),
op(o),
first_committed(0), last_committed(0), pn_from(0), pn(0), uncommitted_pn(0),
decode(feature_map, p);
}
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
#include "mon/MonMap.h"
-class MMonProbe : public MessageInstance<MMonProbe> {
+class MMonProbe : public Message {
public:
- friend factory;
-
static constexpr int HEAD_VERSION = 7;
static constexpr int COMPAT_VERSION = 5;
uint8_t mon_release = 0;
MMonProbe()
- : MessageInstance(MSG_MON_PROBE, HEAD_VERSION, COMPAT_VERSION) {}
+ : Message{MSG_MON_PROBE, HEAD_VERSION, COMPAT_VERSION} {}
MMonProbe(const uuid_d& f, int o, const string& n, bool hej, uint8_t mr)
- : MessageInstance(MSG_MON_PROBE, HEAD_VERSION, COMPAT_VERSION),
+ : Message{MSG_MON_PROBE, HEAD_VERSION, COMPAT_VERSION},
fsid(f),
op(o),
name(n),
else
mon_release = 0;
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
#include "mon/mon_types.h"
-class MMonScrub : public MessageInstance<MMonScrub> {
-public:
- friend factory;
+class MMonScrub : public Message {
private:
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 2;
pair<string,string> key;
MMonScrub()
- : MessageInstance(MSG_MON_SCRUB, HEAD_VERSION, COMPAT_VERSION),
+ : Message{MSG_MON_SCRUB, HEAD_VERSION, COMPAT_VERSION},
num_keys(-1)
{ }
MMonScrub(op_type_t op, version_t v, int32_t num_keys)
- : MessageInstance(MSG_MON_SCRUB, HEAD_VERSION, COMPAT_VERSION),
+ : Message{MSG_MON_SCRUB, HEAD_VERSION, COMPAT_VERSION},
op(op), version(v), num_keys(num_keys)
{ }
decode(num_keys, p);
decode(key, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif /* CEPH_MMONSCRUB_H */
WRITE_RAW_ENCODER(ceph_mon_subscribe_item_old)
-class MMonSubscribe : public MessageInstance<MMonSubscribe> {
+class MMonSubscribe : public Message {
public:
- friend factory;
-
static constexpr int HEAD_VERSION = 3;
static constexpr int COMPAT_VERSION = 1;
std::string hostname;
std::map<std::string, ceph_mon_subscribe_item> what;
- MMonSubscribe() : MessageInstance(CEPH_MSG_MON_SUBSCRIBE, HEAD_VERSION, COMPAT_VERSION) { }
+ MMonSubscribe() : Message{CEPH_MSG_MON_SUBSCRIBE, HEAD_VERSION, COMPAT_VERSION} { }
private:
~MMonSubscribe() override {}
encode(what, payload);
encode(hostname, payload);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MMonSubscribeAck : public MessageInstance<MMonSubscribeAck> {
+class MMonSubscribeAck : public Message {
public:
- friend factory;
-
__u32 interval;
uuid_d fsid;
- MMonSubscribeAck() : MessageInstance(CEPH_MSG_MON_SUBSCRIBE_ACK),
+ MMonSubscribeAck() : Message{CEPH_MSG_MON_SUBSCRIBE_ACK},
interval(0) {
}
- MMonSubscribeAck(uuid_d& f, int i) : MessageInstance(CEPH_MSG_MON_SUBSCRIBE_ACK),
+ MMonSubscribeAck(uuid_d& f, int i) : Message{CEPH_MSG_MON_SUBSCRIBE_ACK},
interval(i), fsid(f) { }
private:
~MMonSubscribeAck() override {}
encode(interval, payload);
encode(fsid, payload);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MMonSync : public MessageInstance<MMonSync> {
-public:
- friend factory;
+class MMonSync : public Message {
private:
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 2;
entity_inst_t reply_to;
MMonSync()
- : MessageInstance(MSG_MON_SYNC, HEAD_VERSION, COMPAT_VERSION)
+ : Message{MSG_MON_SYNC, HEAD_VERSION, COMPAT_VERSION}
{ }
MMonSync(uint32_t op, uint64_t c = 0)
- : MessageInstance(MSG_MON_SYNC, HEAD_VERSION, COMPAT_VERSION),
+ : Message{MSG_MON_SYNC, HEAD_VERSION, COMPAT_VERSION},
op(op),
cookie(c),
last_committed(0)
decode(chunk_bl, p);
decode(reply_to, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif /* CEPH_MMONSYNC_H */
#include "messages/PaxosServiceMessage.h"
-class MOSDAlive : public MessageInstance<MOSDAlive, PaxosServiceMessage> {
+class MOSDAlive : public PaxosServiceMessage {
public:
- friend factory;
-
epoch_t want = 0;
- MOSDAlive(epoch_t h, epoch_t w) : MessageInstance(MSG_OSD_ALIVE, h), want(w) { }
- MOSDAlive() : MessageInstance(MSG_OSD_ALIVE, 0) {}
+ MOSDAlive(epoch_t h, epoch_t w) : PaxosServiceMessage{MSG_OSD_ALIVE, h}, want(w) {}
+ MOSDAlive() : MOSDAlive{0, 0} {}
private:
~MOSDAlive() override {}
void print(ostream &out) const override {
out << "osd_alive(want up_thru " << want << " have " << version << ")";
}
-
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "MOSDFastDispatchOp.h"
#include "osd/osd_types.h"
-class MOSDBackoff : public MessageInstance<MOSDBackoff, MOSDFastDispatchOp> {
+class MOSDBackoff : public MOSDFastDispatchOp {
public:
- friend factory;
-
static constexpr int HEAD_VERSION = 1;
static constexpr int COMPAT_VERSION = 1;
}
MOSDBackoff()
- : MessageInstance(CEPH_MSG_OSD_BACKOFF, HEAD_VERSION, COMPAT_VERSION) {}
+ : MOSDFastDispatchOp{CEPH_MSG_OSD_BACKOFF, HEAD_VERSION, COMPAT_VERSION} {}
MOSDBackoff(spg_t pgid_, epoch_t ep, uint8_t op_, uint64_t id_,
hobject_t begin_, hobject_t end_)
- : MessageInstance(CEPH_MSG_OSD_BACKOFF, HEAD_VERSION, COMPAT_VERSION),
+ : MOSDFastDispatchOp{CEPH_MSG_OSD_BACKOFF, HEAD_VERSION, COMPAT_VERSION},
pgid(pgid_),
map_epoch(ep),
op(op_),
<< " [" << begin << "," << end << ")"
<< " e" << map_epoch << ")";
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "PaxosServiceMessage.h"
-class MOSDBeacon : public MessageInstance<MOSDBeacon, PaxosServiceMessage> {
+class MOSDBeacon : public PaxosServiceMessage {
public:
- friend factory;
-
std::vector<pg_t> pgs;
epoch_t min_last_epoch_clean = 0;
MOSDBeacon()
- : MessageInstance(MSG_OSD_BEACON, 0)
+ : PaxosServiceMessage{MSG_OSD_BEACON, 0}
{}
MOSDBeacon(epoch_t e, epoch_t min_lec)
- : MessageInstance(MSG_OSD_BEACON, e),
+ : PaxosServiceMessage{MSG_OSD_BEACON, e},
min_last_epoch_clean(min_lec)
{}
void encode_payload(uint64_t features) override {
<< " lec " << min_last_epoch_clean
<< " v" << version << ")";
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#include "include/types.h"
#include "osd/osd_types.h"
-class MOSDBoot : public MessageInstance<MOSDBoot, PaxosServiceMessage> {
-public:
- friend factory;
+class MOSDBoot : public PaxosServiceMessage {
private:
static constexpr int HEAD_VERSION = 7;
static constexpr int COMPAT_VERSION = 7;
uint64_t osd_features;
MOSDBoot()
- : MessageInstance(MSG_OSD_BOOT, 0, HEAD_VERSION, COMPAT_VERSION),
+ : PaxosServiceMessage{MSG_OSD_BOOT, 0, HEAD_VERSION, COMPAT_VERSION},
boot_epoch(0), osd_features(0)
{ }
MOSDBoot(OSDSuperblock& s, epoch_t e, epoch_t be,
const entity_addrvec_t& hb_front_addr_ref,
const entity_addrvec_t& cluster_addr_ref,
uint64_t feat)
- : MessageInstance(MSG_OSD_BOOT, e, HEAD_VERSION, COMPAT_VERSION),
+ : PaxosServiceMessage{MSG_OSD_BOOT, e, HEAD_VERSION, COMPAT_VERSION},
sb(s),
hb_back_addrs(hb_back_addr_ref),
hb_front_addrs(hb_front_addr_ref),
decode(metadata, p);
decode(osd_features, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "MOSDFastDispatchOp.h"
#include "osd/ECMsgTypes.h"
-class MOSDECSubOpRead : public MessageInstance<MOSDECSubOpRead, MOSDFastDispatchOp> {
-public:
- friend factory;
+class MOSDECSubOpRead : public MOSDFastDispatchOp {
private:
static constexpr int HEAD_VERSION = 3;
static constexpr int COMPAT_VERSION = 1;
}
MOSDECSubOpRead()
- : MessageInstance(MSG_OSD_EC_READ, HEAD_VERSION, COMPAT_VERSION)
+ : MOSDFastDispatchOp{MSG_OSD_EC_READ, HEAD_VERSION, COMPAT_VERSION}
{}
void decode_payload() override {
<< " " << op;
out << ")";
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "MOSDFastDispatchOp.h"
#include "osd/ECMsgTypes.h"
-class MOSDECSubOpReadReply : public MessageInstance<MOSDECSubOpReadReply, MOSDFastDispatchOp> {
-public:
- friend factory;
+class MOSDECSubOpReadReply : public MOSDFastDispatchOp {
private:
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 1;
}
MOSDECSubOpReadReply()
- : MessageInstance(MSG_OSD_EC_READ_REPLY, HEAD_VERSION, COMPAT_VERSION)
+ : MOSDFastDispatchOp{MSG_OSD_EC_READ_REPLY, HEAD_VERSION, COMPAT_VERSION}
{}
void decode_payload() override {
<< " " << op;
out << ")";
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "MOSDFastDispatchOp.h"
#include "osd/ECMsgTypes.h"
-class MOSDECSubOpWrite : public MessageInstance<MOSDECSubOpWrite, MOSDFastDispatchOp> {
-public:
- friend factory;
+class MOSDECSubOpWrite : public MOSDFastDispatchOp {
private:
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 1;
}
MOSDECSubOpWrite()
- : MessageInstance(MSG_OSD_EC_WRITE, HEAD_VERSION, COMPAT_VERSION)
+ : MOSDFastDispatchOp{MSG_OSD_EC_WRITE, HEAD_VERSION, COMPAT_VERSION}
{}
MOSDECSubOpWrite(ECSubWrite &in_op)
- : MessageInstance(MSG_OSD_EC_WRITE, HEAD_VERSION, COMPAT_VERSION) {
+ : MOSDFastDispatchOp{MSG_OSD_EC_WRITE, HEAD_VERSION, COMPAT_VERSION} {
op.claim(in_op);
}
op.t = ObjectStore::Transaction();
op.log_entries.clear();
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "MOSDFastDispatchOp.h"
#include "osd/ECMsgTypes.h"
-class MOSDECSubOpWriteReply : public MessageInstance<MOSDECSubOpWriteReply, MOSDFastDispatchOp> {
-public:
- friend factory;
+class MOSDECSubOpWriteReply : public MOSDFastDispatchOp {
private:
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 1;
}
MOSDECSubOpWriteReply()
- : MessageInstance(MSG_OSD_EC_WRITE_REPLY, HEAD_VERSION, COMPAT_VERSION)
- {}
+ : MOSDFastDispatchOp{MSG_OSD_EC_WRITE_REPLY, HEAD_VERSION, COMPAT_VERSION}
+ {}
void decode_payload() override {
auto p = payload.cbegin();
<< " " << op;
out << ")";
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "messages/PaxosServiceMessage.h"
-class MOSDFailure : public MessageInstance<MOSDFailure, PaxosServiceMessage> {
-public:
- friend factory;
+class MOSDFailure : public PaxosServiceMessage {
private:
static constexpr int HEAD_VERSION = 4;
static constexpr int COMPAT_VERSION = 4;
epoch_t epoch = 0;
int32_t failed_for = 0; // known to be failed since at least this long
- MOSDFailure() : MessageInstance(MSG_OSD_FAILURE, 0, HEAD_VERSION) { }
+ MOSDFailure() : PaxosServiceMessage(MSG_OSD_FAILURE, 0, HEAD_VERSION) { }
MOSDFailure(const uuid_d &fs, int osd, const entity_addrvec_t& av,
int duration, epoch_t e)
- : MessageInstance(MSG_OSD_FAILURE, e, HEAD_VERSION, COMPAT_VERSION),
+ : PaxosServiceMessage(MSG_OSD_FAILURE, e, HEAD_VERSION, COMPAT_VERSION),
fsid(fs),
target_osd(osd),
target_addrs(av),
MOSDFailure(const uuid_d &fs, int osd, const entity_addrvec_t& av,
int duration,
epoch_t e, __u8 extra_flags)
- : MessageInstance(MSG_OSD_FAILURE, e, HEAD_VERSION, COMPAT_VERSION),
+ : PaxosServiceMessage(MSG_OSD_FAILURE, e, HEAD_VERSION, COMPAT_VERSION),
fsid(fs),
target_osd(osd),
target_addrs(av),
<< " for " << failed_for << "sec e" << epoch
<< " v" << version << ")";
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
// cancel priority boost, requeue if necessary
static const int OFR_CANCEL = 4;
-class MOSDForceRecovery : public MessageInstance<MOSDForceRecovery> {
+class MOSDForceRecovery : public Message {
public:
- friend factory;
-
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 2;
vector<spg_t> forced_pgs;
uint8_t options = 0;
- MOSDForceRecovery() : MessageInstance(MSG_OSD_FORCE_RECOVERY, HEAD_VERSION, COMPAT_VERSION) {}
+ MOSDForceRecovery() : Message{MSG_OSD_FORCE_RECOVERY, HEAD_VERSION, COMPAT_VERSION} {}
MOSDForceRecovery(const uuid_d& f, char opts) :
- MessageInstance(MSG_OSD_FORCE_RECOVERY, HEAD_VERSION, COMPAT_VERSION),
+ Message{MSG_OSD_FORCE_RECOVERY, HEAD_VERSION, COMPAT_VERSION},
fsid(f), options(opts) {}
MOSDForceRecovery(const uuid_d& f, vector<spg_t>& pgs, char opts) :
- MessageInstance(MSG_OSD_FORCE_RECOVERY, HEAD_VERSION, COMPAT_VERSION),
+ Message{MSG_OSD_FORCE_RECOVERY, HEAD_VERSION, COMPAT_VERSION},
fsid(f), forced_pgs(pgs), options(opts) {}
private:
~MOSDForceRecovery() {}
decode(forced_pgs, p);
decode(options, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif /* CEPH_MOSDFORCERECOVERY_H_ */
// future this message could be generalized to other state bits, but
// for now name it for its sole application.
-class MOSDFull : public MessageInstance<MOSDFull, PaxosServiceMessage> {
+class MOSDFull : public PaxosServiceMessage {
public:
- friend factory;
-
epoch_t map_epoch = 0;
uint32_t state = 0;
public:
MOSDFull(epoch_t e, unsigned s)
- : MessageInstance(MSG_OSD_FULL, e), map_epoch(e), state(s) { }
+ : PaxosServiceMessage{MSG_OSD_FULL, e}, map_epoch(e), state(s) { }
MOSDFull()
- : MessageInstance(MSG_OSD_FULL, 0) {}
+ : PaxosServiceMessage{MSG_OSD_FULL, 0} {}
public:
void encode_payload(uint64_t features) {
OSDMap::calc_state_set(state, states);
out << "osd_full(e" << map_epoch << " " << states << " v" << version << ")";
}
-
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "osd/OSDMap.h"
#include "include/ceph_features.h"
-class MOSDMap : public MessageInstance<MOSDMap> {
-public:
- friend factory;
+class MOSDMap : public Message {
private:
static constexpr int HEAD_VERSION = 4;
static constexpr int COMPAT_VERSION = 3;
- public:
+public:
uuid_d fsid;
uint64_t encode_features = 0;
std::map<epoch_t, ceph::buffer::list> maps;
}
- MOSDMap() : MessageInstance(CEPH_MSG_OSD_MAP, HEAD_VERSION, COMPAT_VERSION) { }
+ MOSDMap() : Message{CEPH_MSG_OSD_MAP, HEAD_VERSION, COMPAT_VERSION} { }
MOSDMap(const uuid_d &f, const uint64_t features)
- : MessageInstance(CEPH_MSG_OSD_MAP, HEAD_VERSION, COMPAT_VERSION),
+ : Message{CEPH_MSG_OSD_MAP, HEAD_VERSION, COMPAT_VERSION},
fsid(f), encode_features(features),
oldest_map(0), newest_map(0) { }
private:
out << " +gap_removed_snaps";
out << ")";
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "messages/PaxosServiceMessage.h"
-class MOSDMarkMeDown : public MessageInstance<MOSDMarkMeDown, PaxosServiceMessage> {
-public:
- friend factory;
+class MOSDMarkMeDown : public PaxosServiceMessage {
private:
static constexpr int HEAD_VERSION = 3;
static constexpr int COMPAT_VERSION = 3;
bool request_ack = false; // ack requested
MOSDMarkMeDown()
- : MessageInstance(MSG_OSD_MARK_ME_DOWN, 0,
- HEAD_VERSION, COMPAT_VERSION) { }
+ : PaxosServiceMessage{MSG_OSD_MARK_ME_DOWN, 0,
+ HEAD_VERSION, COMPAT_VERSION} { }
MOSDMarkMeDown(const uuid_d &fs, int osd, const entity_addrvec_t& av,
epoch_t e, bool request_ack)
- : MessageInstance(MSG_OSD_MARK_ME_DOWN, e,
- HEAD_VERSION, COMPAT_VERSION),
+ : PaxosServiceMessage{MSG_OSD_MARK_ME_DOWN, e,
+ HEAD_VERSION, COMPAT_VERSION},
fsid(fs), target_osd(osd), target_addrs(av),
epoch(e), request_ack(request_ack) {}
private:
<< ", fsid=" << fsid
<< ")";
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
class OSD;
-class MOSDOp : public MessageInstance<MOSDOp, MOSDFastDispatchOp> {
-public:
- friend factory;
+class MOSDOp : public MOSDFastDispatchOp {
private:
static constexpr int HEAD_VERSION = 8;
static constexpr int COMPAT_VERSION = 3;
}
MOSDOp()
- : MessageInstance(CEPH_MSG_OSD_OP, HEAD_VERSION, COMPAT_VERSION),
+ : MOSDFastDispatchOp(CEPH_MSG_OSD_OP, HEAD_VERSION, COMPAT_VERSION),
partial_decode_needed(true),
final_decode_needed(true),
bdata_encode(false) { }
MOSDOp(int inc, long tid, const hobject_t& ho, spg_t& _pgid,
epoch_t _osdmap_epoch,
int _flags, uint64_t feat)
- : MessageInstance(CEPH_MSG_OSD_OP, HEAD_VERSION, COMPAT_VERSION),
+ : MOSDFastDispatchOp(CEPH_MSG_OSD_OP, HEAD_VERSION, COMPAT_VERSION),
client_inc(inc),
osdmap_epoch(_osdmap_epoch), flags(_flags), retry_attempt(-1),
hobj(ho),
}
out << ")";
}
+
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
*
*/
-class MOSDOpReply : public MessageInstance<MOSDOpReply> {
-public:
- friend factory;
+class MOSDOpReply : public Message {
private:
static constexpr int HEAD_VERSION = 8;
static constexpr int COMPAT_VERSION = 2;
public:
MOSDOpReply()
- : MessageInstance(CEPH_MSG_OSD_OPREPLY, HEAD_VERSION, COMPAT_VERSION),
+ : Message{CEPH_MSG_OSD_OPREPLY, HEAD_VERSION, COMPAT_VERSION},
bdata_encode(false) {
do_redirect = false;
}
MOSDOpReply(const MOSDOp *req, int r, epoch_t e, int acktype,
bool ignore_out_data)
- : MessageInstance(CEPH_MSG_OSD_OPREPLY, HEAD_VERSION, COMPAT_VERSION),
+ : Message{CEPH_MSG_OSD_OPREPLY, HEAD_VERSION, COMPAT_VERSION},
oid(req->hobj.oid), pgid(req->pgid.pgid), ops(req->ops),
bdata_encode(false) {
out << ")";
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#include "MOSDFastDispatchOp.h"
-class MOSDPGBackfill : public MessageInstance<MOSDPGBackfill, MOSDFastDispatchOp> {
-public:
- friend factory;
+class MOSDPGBackfill : public MOSDFastDispatchOp {
private:
static constexpr int HEAD_VERSION = 3;
static constexpr int COMPAT_VERSION = 3;
}
MOSDPGBackfill()
- : MessageInstance(MSG_OSD_PG_BACKFILL, HEAD_VERSION, COMPAT_VERSION) {}
+ : MOSDFastDispatchOp{MSG_OSD_PG_BACKFILL, HEAD_VERSION, COMPAT_VERSION} {}
MOSDPGBackfill(__u32 o, epoch_t e, epoch_t qe, spg_t p)
- : MessageInstance(MSG_OSD_PG_BACKFILL, HEAD_VERSION, COMPAT_VERSION),
+ : MOSDFastDispatchOp{MSG_OSD_PG_BACKFILL, HEAD_VERSION, COMPAT_VERSION},
op(o),
map_epoch(e), query_epoch(e),
pgid(p) {}
<< " lb " << last_backfill
<< ")";
}
+
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
* instruct non-primary to remove some objects during backfill
*/
-class MOSDPGBackfillRemove : public MessageInstance<MOSDPGBackfillRemove, MOSDFastDispatchOp> {
+class MOSDPGBackfillRemove : public MOSDFastDispatchOp {
public:
- friend factory;
-
static constexpr int HEAD_VERSION = 1;
static constexpr int COMPAT_VERSION = 1;
}
MOSDPGBackfillRemove()
- : MessageInstance(MSG_OSD_PG_BACKFILL_REMOVE, HEAD_VERSION,
- COMPAT_VERSION) {}
+ : MOSDFastDispatchOp{MSG_OSD_PG_BACKFILL_REMOVE, HEAD_VERSION,
+ COMPAT_VERSION}
+ {}
MOSDPGBackfillRemove(spg_t pgid, epoch_t map_epoch)
- : MessageInstance(MSG_OSD_PG_BACKFILL_REMOVE, HEAD_VERSION,
- COMPAT_VERSION),
+ : MOSDFastDispatchOp{MSG_OSD_PG_BACKFILL_REMOVE, HEAD_VERSION,
+ COMPAT_VERSION},
pgid(pgid),
map_epoch(map_epoch) {}
decode(map_epoch, p);
decode(ls, p);
}
-};
-
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
+};
#endif
* PGCreate - instruct an OSD to create a pg, if it doesn't already exist
*/
-class MOSDPGCreate : public MessageInstance<MOSDPGCreate> {
+class MOSDPGCreate : public Message {
public:
- friend factory;
-
static constexpr int HEAD_VERSION = 3;
static constexpr int COMPAT_VERSION = 3;
map<pg_t,utime_t> ctimes;
MOSDPGCreate()
- : MessageInstance(MSG_OSD_PG_CREATE, HEAD_VERSION, COMPAT_VERSION) {}
+ : MOSDPGCreate{0}
+ {}
MOSDPGCreate(epoch_t e)
- : MessageInstance(MSG_OSD_PG_CREATE, HEAD_VERSION, COMPAT_VERSION),
- epoch(e) { }
+ : Message{MSG_OSD_PG_CREATE, HEAD_VERSION, COMPAT_VERSION},
+ epoch(e)
+ {}
private:
~MOSDPGCreate() override {}
decode(mkpg, p);
decode(ctimes, p);
}
-
void print(ostream& out) const override {
out << "osd_pg_create(e" << epoch;
for (map<pg_t,pg_create_t>::const_iterator i = mkpg.begin();
}
out << ")";
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
* PGCreate2 - instruct an OSD to create some pgs
*/
-class MOSDPGCreate2 : public MessageInstance<MOSDPGCreate2> {
+class MOSDPGCreate2 : public Message {
public:
- friend factory;
-
static constexpr int HEAD_VERSION = 1;
static constexpr int COMPAT_VERSION = 1;
map<spg_t,pair<epoch_t,utime_t>> pgs;
MOSDPGCreate2()
- : MessageInstance(MSG_OSD_PG_CREATE2, HEAD_VERSION, COMPAT_VERSION) {}
+ : Message{MSG_OSD_PG_CREATE2, HEAD_VERSION, COMPAT_VERSION} {}
MOSDPGCreate2(epoch_t e)
- : MessageInstance(MSG_OSD_PG_CREATE2, HEAD_VERSION, COMPAT_VERSION),
+ : Message{MSG_OSD_PG_CREATE2, HEAD_VERSION, COMPAT_VERSION},
epoch(e) { }
private:
~MOSDPGCreate2() override {}
decode(epoch, p);
decode(pgs, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#include "osd/osd_types.h"
#include "messages/PaxosServiceMessage.h"
-class MOSDPGCreated : public MessageInstance<MOSDPGCreated, PaxosServiceMessage> {
+class MOSDPGCreated : public PaxosServiceMessage {
public:
- friend factory;
-
pg_t pgid;
MOSDPGCreated()
- : MessageInstance(MSG_OSD_PG_CREATED, 0)
+ : PaxosServiceMessage{MSG_OSD_PG_CREATED, 0}
{}
MOSDPGCreated(pg_t pgid)
- : MessageInstance(MSG_OSD_PG_CREATED, 0),
+ : PaxosServiceMessage{MSG_OSD_PG_CREATED, 0},
pgid(pgid)
{}
std::string_view get_type_name() const override { return "pg_created"; }
paxos_decode(p);
decode(pgid, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#include "msg/Message.h"
#include "osd/osd_types.h"
-class MOSDPGInfo : public MessageInstance<MOSDPGInfo> {
-public:
- friend factory;
+class MOSDPGInfo : public Message {
private:
static constexpr int HEAD_VERSION = 5;
static constexpr int COMPAT_VERSION = 5;
: MOSDPGInfo(mv, {})
{}
MOSDPGInfo(epoch_t mv, pg_list_t&& l)
- : MessageInstance(MSG_OSD_PG_INFO, HEAD_VERSION, COMPAT_VERSION),
+ : Message{MSG_OSD_PG_INFO, HEAD_VERSION, COMPAT_VERSION},
epoch{mv},
pg_list{std::move(l)}
{
decode(epoch, p);
decode(pg_list, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "messages/MOSDPeeringOp.h"
-class MOSDPGLog : public MessageInstance<MOSDPGLog, MOSDPeeringOp> {
-public:
- friend factory;
+class MOSDPGLog : public MOSDPeeringOp {
private:
static constexpr int HEAD_VERSION = 5;
static constexpr int COMPAT_VERSION = 5;
false));
}
- MOSDPGLog() : MessageInstance(MSG_OSD_PG_LOG, HEAD_VERSION, COMPAT_VERSION) {
+ MOSDPGLog() : MOSDPeeringOp{MSG_OSD_PG_LOG, HEAD_VERSION, COMPAT_VERSION} {
set_priority(CEPH_MSG_PRIO_HIGH);
}
MOSDPGLog(shard_id_t to, shard_id_t from,
version_t mv, const pg_info_t& i, epoch_t query_epoch)
- : MessageInstance(MSG_OSD_PG_LOG, HEAD_VERSION, COMPAT_VERSION),
+ : MOSDPeeringOp{MSG_OSD_PG_LOG, HEAD_VERSION, COMPAT_VERSION},
epoch(mv), query_epoch(query_epoch),
to(to), from(from),
info(i) {
decode(to, p);
decode(from, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
* PGNotify - notify primary of my PGs and versions.
*/
-class MOSDPGNotify : public MessageInstance<MOSDPGNotify> {
-public:
- friend factory;
+class MOSDPGNotify : public Message {
private:
static constexpr int HEAD_VERSION = 6;
static constexpr int COMPAT_VERSION = 6;
: MOSDPGNotify(0, {})
{}
MOSDPGNotify(epoch_t e, pg_list_t&& l)
- : MessageInstance(MSG_OSD_PG_NOTIFY, HEAD_VERSION, COMPAT_VERSION),
+ : Message{MSG_OSD_PG_NOTIFY, HEAD_VERSION, COMPAT_VERSION},
epoch(e),
pg_list(std::move(l)) {
set_priority(CEPH_MSG_PRIO_HIGH);
out << " epoch " << epoch
<< ")";
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "MOSDFastDispatchOp.h"
-class MOSDPGPull : public MessageInstance<MOSDPGPull, MOSDFastDispatchOp> {
-public:
- friend factory;
+class MOSDPGPull : public MOSDFastDispatchOp {
private:
static constexpr int HEAD_VERSION = 3;
static constexpr int COMPAT_VERSION = 2;
pg_shard_t from;
spg_t pgid;
epoch_t map_epoch = 0, min_epoch = 0;
- uint64_t cost;
+ uint64_t cost = 0;
epoch_t get_map_epoch() const override {
return map_epoch;
}
MOSDPGPull()
- : MessageInstance(MSG_OSD_PG_PULL, HEAD_VERSION, COMPAT_VERSION),
- cost(0)
- {}
+ : MOSDFastDispatchOp{MSG_OSD_PG_PULL, HEAD_VERSION, COMPAT_VERSION}
+ {}
void compute_cost(CephContext *cct) {
cost = 0;
<< " cost " << cost
<< ")";
}
+
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "MOSDFastDispatchOp.h"
-class MOSDPGPush : public MessageInstance<MOSDPGPush, MOSDFastDispatchOp> {
-public:
- friend factory;
+class MOSDPGPush : public MOSDFastDispatchOp {
private:
static constexpr int HEAD_VERSION = 4;
static constexpr int COMPAT_VERSION = 2;
bool is_repair = false;
private:
- uint64_t cost;
+ uint64_t cost = 0;
public:
void compute_cost(CephContext *cct) {
}
MOSDPGPush()
- : MessageInstance(MSG_OSD_PG_PUSH, HEAD_VERSION, COMPAT_VERSION),
- cost(0)
- {}
+ : MOSDFastDispatchOp{MSG_OSD_PG_PUSH, HEAD_VERSION, COMPAT_VERSION}
+ {}
void decode_payload() override {
auto p = payload.cbegin();
<< " " << pushes;
out << ")";
}
+
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "MOSDFastDispatchOp.h"
-class MOSDPGPushReply : public MessageInstance<MOSDPGPushReply, MOSDFastDispatchOp> {
-public:
- friend factory;
+class MOSDPGPushReply : public MOSDFastDispatchOp {
private:
static constexpr int HEAD_VERSION = 3;
static constexpr int COMPAT_VERSION = 2;
spg_t pgid;
epoch_t map_epoch = 0, min_epoch = 0;
vector<PushReplyOp> replies;
- uint64_t cost;
+ uint64_t cost = 0;
epoch_t get_map_epoch() const override {
return map_epoch;
}
MOSDPGPushReply()
- : MessageInstance(MSG_OSD_PG_PUSH_REPLY, HEAD_VERSION, COMPAT_VERSION),
- cost(0)
+ : MOSDFastDispatchOp{MSG_OSD_PG_PUSH_REPLY, HEAD_VERSION, COMPAT_VERSION}
{}
void compute_cost(CephContext *cct) {
* PGQuery - query another OSD as to the contents of their PGs
*/
-class MOSDPGQuery : public MessageInstance<MOSDPGQuery> {
-public:
- friend factory;
+class MOSDPGQuery : public Message {
private:
static constexpr int HEAD_VERSION = 4;
static constexpr int COMPAT_VERSION = 4;
using pg_list_t = std::map<spg_t, pg_query_t>;
pg_list_t pg_list;
- MOSDPGQuery() : MessageInstance(MSG_OSD_PG_QUERY,
+ MOSDPGQuery() : Message{MSG_OSD_PG_QUERY,
HEAD_VERSION,
- COMPAT_VERSION) {
+ COMPAT_VERSION} {
set_priority(CEPH_MSG_PRIO_HIGH);
}
MOSDPGQuery(epoch_t e, pg_list_t&& ls) :
- MessageInstance(MSG_OSD_PG_QUERY,
+ Message{MSG_OSD_PG_QUERY,
HEAD_VERSION,
- COMPAT_VERSION),
+ COMPAT_VERSION},
epoch(e),
pg_list(std::move(ls)) {
set_priority(CEPH_MSG_PRIO_HIGH);
decode(epoch, p);
decode(pg_list, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#pragma once
-class MOSDPGReadyToMerge
- : public MessageInstance<MOSDPGReadyToMerge, PaxosServiceMessage> {
+class MOSDPGReadyToMerge : public PaxosServiceMessage {
public:
pg_t pgid;
eversion_t source_version, target_version;
bool ready = true;
MOSDPGReadyToMerge()
- : MessageInstance(MSG_OSD_PG_READY_TO_MERGE, 0)
+ : PaxosServiceMessage{MSG_OSD_PG_READY_TO_MERGE, 0}
{}
MOSDPGReadyToMerge(pg_t p, eversion_t sv, eversion_t tv,
epoch_t les, epoch_t lec, bool r, epoch_t v)
- : MessageInstance(MSG_OSD_PG_READY_TO_MERGE, v),
+ : PaxosServiceMessage{MSG_OSD_PG_READY_TO_MERGE, v},
pgid(p),
source_version(sv),
target_version(tv),
<< (ready ? " ready" : " NOT READY")
<< " v" << version << ")";
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
* instruct non-primary to remove some objects during recovery
*/
-class MOSDPGRecoveryDelete : public MessageInstance<MOSDPGRecoveryDelete, MOSDFastDispatchOp> {
+class MOSDPGRecoveryDelete : public MOSDFastDispatchOp {
public:
- friend factory;
-
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 1;
list<pair<hobject_t, eversion_t> > objects; ///< objects to remove
private:
- uint64_t cost;
+ uint64_t cost = 0;
public:
int get_cost() const override {
}
MOSDPGRecoveryDelete()
- : MessageInstance(MSG_OSD_PG_RECOVERY_DELETE, HEAD_VERSION,
- COMPAT_VERSION), cost(0) {}
+ : MOSDFastDispatchOp{MSG_OSD_PG_RECOVERY_DELETE, HEAD_VERSION,
+ COMPAT_VERSION}
+ {}
MOSDPGRecoveryDelete(pg_shard_t from, spg_t pgid, epoch_t map_epoch,
epoch_t min_epoch)
- : MessageInstance(MSG_OSD_PG_RECOVERY_DELETE, HEAD_VERSION,
- COMPAT_VERSION),
+ : MOSDFastDispatchOp{MSG_OSD_PG_RECOVERY_DELETE, HEAD_VERSION,
+ COMPAT_VERSION},
from(from),
pgid(pgid),
map_epoch(map_epoch),
- min_epoch(min_epoch),
- cost(0) {}
+ min_epoch(min_epoch)
+ {}
private:
~MOSDPGRecoveryDelete() {}
decode(cost, p);
decode(objects, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
-
-
#endif
#include "MOSDFastDispatchOp.h"
-class MOSDPGRecoveryDeleteReply : public MessageInstance<MOSDPGRecoveryDeleteReply, MOSDFastDispatchOp> {
+class MOSDPGRecoveryDeleteReply : public MOSDFastDispatchOp {
public:
- friend factory;
-
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 1;
pg_shard_t from;
spg_t pgid;
- epoch_t map_epoch, min_epoch;
+ epoch_t map_epoch = 0;
+ epoch_t min_epoch = 0;
list<pair<hobject_t, eversion_t> > objects;
epoch_t get_map_epoch() const override {
}
MOSDPGRecoveryDeleteReply()
- : MessageInstance(MSG_OSD_PG_RECOVERY_DELETE_REPLY, HEAD_VERSION, COMPAT_VERSION),
- map_epoch(0), min_epoch(0)
- {}
+ : MOSDFastDispatchOp{MSG_OSD_PG_RECOVERY_DELETE_REPLY, HEAD_VERSION, COMPAT_VERSION}
+ {}
void decode_payload() override {
auto p = payload.cbegin();
}
std::string_view get_type_name() const override { return "recovery_delete_reply"; }
+
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MOSDPGRemove : public MessageInstance<MOSDPGRemove> {
-public:
- friend factory;
+class MOSDPGRemove : public Message {
private:
static constexpr int HEAD_VERSION = 3;
static constexpr int COMPAT_VERSION = 3;
epoch_t get_epoch() const { return epoch; }
MOSDPGRemove() :
- MessageInstance(MSG_OSD_PG_REMOVE, HEAD_VERSION, COMPAT_VERSION) {}
+ Message{MSG_OSD_PG_REMOVE, HEAD_VERSION, COMPAT_VERSION} {}
MOSDPGRemove(epoch_t e, vector<spg_t>& l) :
- MessageInstance(MSG_OSD_PG_REMOVE, HEAD_VERSION, COMPAT_VERSION) {
+ Message{MSG_OSD_PG_REMOVE, HEAD_VERSION, COMPAT_VERSION} {
this->epoch = e;
pg_list.swap(l);
}
}
out << ")";
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "MOSDFastDispatchOp.h"
-class MOSDPGScan : public MessageInstance<MOSDPGScan, MOSDFastDispatchOp> {
-public:
- friend factory;
+class MOSDPGScan : public MOSDFastDispatchOp {
private:
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 2;
}
MOSDPGScan()
- : MessageInstance(MSG_OSD_PG_SCAN, HEAD_VERSION, COMPAT_VERSION) {}
+ : MOSDFastDispatchOp{MSG_OSD_PG_SCAN, HEAD_VERSION, COMPAT_VERSION} {}
MOSDPGScan(__u32 o, pg_shard_t from,
epoch_t e, epoch_t qe, spg_t p, hobject_t be, hobject_t en)
- : MessageInstance(MSG_OSD_PG_SCAN, HEAD_VERSION, COMPAT_VERSION),
+ : MOSDFastDispatchOp{MSG_OSD_PG_SCAN, HEAD_VERSION, COMPAT_VERSION},
op(o),
map_epoch(e), query_epoch(qe),
from(from),
<< " e " << map_epoch << "/" << query_epoch
<< ")";
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "messages/PaxosServiceMessage.h"
-class MOSDPGTemp : public MessageInstance<MOSDPGTemp, PaxosServiceMessage> {
+class MOSDPGTemp : public PaxosServiceMessage {
public:
- friend factory;
-
epoch_t map_epoch = 0;
map<pg_t, vector<int32_t> > pg_temp;
bool forced = false;
MOSDPGTemp(epoch_t e)
- : MessageInstance(MSG_OSD_PGTEMP, e, HEAD_VERSION, COMPAT_VERSION),
+ : PaxosServiceMessage{MSG_OSD_PGTEMP, e, HEAD_VERSION, COMPAT_VERSION},
map_epoch(e)
{}
MOSDPGTemp()
private:
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 1;
+
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
#include "messages/MOSDPeeringOp.h"
-class MOSDPGTrim : public MessageInstance<MOSDPGTrim, MOSDPeeringOp> {
-public:
- friend factory;
+class MOSDPGTrim : public MOSDPeeringOp {
private:
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 2;
MTrim(epoch, get_source().num(), pgid.shard, trim_to));
}
- MOSDPGTrim() : MessageInstance(MSG_OSD_PG_TRIM, HEAD_VERSION, COMPAT_VERSION) {}
+ MOSDPGTrim() : MOSDPeeringOp{MSG_OSD_PG_TRIM, HEAD_VERSION, COMPAT_VERSION} {}
MOSDPGTrim(version_t mv, spg_t p, eversion_t tt) :
- MessageInstance(MSG_OSD_PG_TRIM, HEAD_VERSION, COMPAT_VERSION),
+ MOSDPeeringOp{MSG_OSD_PG_TRIM, HEAD_VERSION, COMPAT_VERSION},
epoch(mv), pgid(p), trim_to(tt) { }
private:
~MOSDPGTrim() override {}
decode(trim_to, p);
decode(pgid.shard, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "MOSDFastDispatchOp.h"
-class MOSDPGUpdateLogMissing : public MessageInstance<MOSDPGUpdateLogMissing, MOSDFastDispatchOp> {
-public:
- friend factory;
+class MOSDPGUpdateLogMissing : public MOSDFastDispatchOp {
private:
static constexpr int HEAD_VERSION = 3;
static constexpr int COMPAT_VERSION = 1;
-
public:
epoch_t map_epoch = 0, min_epoch = 0;
spg_t pgid;
}
MOSDPGUpdateLogMissing()
- : MessageInstance(MSG_OSD_PG_UPDATE_LOG_MISSING, HEAD_VERSION,
- COMPAT_VERSION) { }
+ : MOSDFastDispatchOp{MSG_OSD_PG_UPDATE_LOG_MISSING, HEAD_VERSION,
+ COMPAT_VERSION} {}
MOSDPGUpdateLogMissing(
const mempool::osd_pglog::list<pg_log_entry_t> &entries,
spg_t pgid,
ceph_tid_t rep_tid,
eversion_t pg_trim_to,
eversion_t pg_roll_forward_to)
- : MessageInstance(MSG_OSD_PG_UPDATE_LOG_MISSING, HEAD_VERSION,
- COMPAT_VERSION),
+ : MOSDFastDispatchOp{MSG_OSD_PG_UPDATE_LOG_MISSING, HEAD_VERSION,
+ COMPAT_VERSION},
map_epoch(epoch),
min_epoch(min_epoch),
pgid(pgid),
decode(pg_roll_forward_to, p);
}
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "MOSDFastDispatchOp.h"
-class MOSDPGUpdateLogMissingReply : public MessageInstance<MOSDPGUpdateLogMissingReply, MOSDFastDispatchOp> {
-public:
- friend factory;
+class MOSDPGUpdateLogMissingReply : public MOSDFastDispatchOp {
private:
static constexpr int HEAD_VERSION = 3;
static constexpr int COMPAT_VERSION = 1;
}
MOSDPGUpdateLogMissingReply()
- : MessageInstance(
- MSG_OSD_PG_UPDATE_LOG_MISSING_REPLY,
- HEAD_VERSION,
- COMPAT_VERSION)
- {}
+ : MOSDFastDispatchOp{MSG_OSD_PG_UPDATE_LOG_MISSING_REPLY, HEAD_VERSION,
+ COMPAT_VERSION}
+ {}
MOSDPGUpdateLogMissingReply(
spg_t pgid,
shard_id_t from,
epoch_t min_epoch,
ceph_tid_t rep_tid,
eversion_t last_complete_ondisk)
- : MessageInstance(
- MSG_OSD_PG_UPDATE_LOG_MISSING_REPLY,
- HEAD_VERSION,
- COMPAT_VERSION),
+ : MOSDFastDispatchOp{MSG_OSD_PG_UPDATE_LOG_MISSING_REPLY, HEAD_VERSION,
+ COMPAT_VERSION},
map_epoch(epoch),
min_epoch(min_epoch),
pgid(pgid),
decode(last_complete_ondisk, p);
}
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "osd/osd_types.h"
-class MOSDPing : public MessageInstance<MOSDPing> {
-public:
- friend factory;
+class MOSDPing : public Message {
private:
static constexpr int HEAD_VERSION = 4;
static constexpr int COMPAT_VERSION = 4;
epoch_t map_epoch = 0;
__u8 op = 0;
utime_t stamp;
- uint32_t min_message_size;
+ uint32_t min_message_size = 0;
MOSDPing(const uuid_d& f, epoch_t e, __u8 o, utime_t s, uint32_t min_message)
- : MessageInstance(MSG_OSD_PING, HEAD_VERSION, COMPAT_VERSION),
+ : Message{MSG_OSD_PING, HEAD_VERSION, COMPAT_VERSION},
fsid(f), map_epoch(e), op(o), stamp(s), min_message_size(min_message)
{ }
MOSDPing()
- : MessageInstance(MSG_OSD_PING, HEAD_VERSION, COMPAT_VERSION), min_message_size(0)
+ : Message{MSG_OSD_PING, HEAD_VERSION, COMPAT_VERSION}
{}
private:
~MOSDPing() override {}
<< " stamp " << stamp
<< ")";
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
* OSD sub op - for internal ops on pobjects between primary and replicas(/stripes/whatever)
*/
-class MOSDRepOp : public MessageInstance<MOSDRepOp, MOSDFastDispatchOp> {
-public:
- friend factory;
+class MOSDRepOp : public MOSDFastDispatchOp {
private:
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 1;
}
MOSDRepOp()
- : MessageInstance(MSG_OSD_REPOP, HEAD_VERSION, COMPAT_VERSION),
+ : MOSDFastDispatchOp{MSG_OSD_REPOP, HEAD_VERSION, COMPAT_VERSION},
map_epoch(0),
final_decode_needed(true), acks_wanted (0) {}
MOSDRepOp(osd_reqid_t r, pg_shard_t from,
spg_t p, const hobject_t& po, int aw,
epoch_t mape, epoch_t min_epoch, ceph_tid_t rtid, eversion_t v)
- : MessageInstance(MSG_OSD_REPOP, HEAD_VERSION, COMPAT_VERSION),
+ : MOSDFastDispatchOp{MSG_OSD_REPOP, HEAD_VERSION, COMPAT_VERSION},
map_epoch(mape),
min_epoch(min_epoch),
reqid(r),
}
out << ")";
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
-
#endif
*
*/
-class MOSDRepOpReply : public MessageInstance<MOSDRepOpReply, MOSDFastDispatchOp> {
-public:
- friend factory;
+class MOSDRepOpReply : public MOSDFastDispatchOp {
private:
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 1;
MOSDRepOpReply(
const MOSDRepOp *req, pg_shard_t from, int result_, epoch_t e, epoch_t mine,
int at) :
- MessageInstance(MSG_OSD_REPOPREPLY, HEAD_VERSION, COMPAT_VERSION),
+ MOSDFastDispatchOp{MSG_OSD_REPOPREPLY, HEAD_VERSION, COMPAT_VERSION},
map_epoch(e),
min_epoch(mine),
reqid(req->reqid),
set_tid(req->get_tid());
}
MOSDRepOpReply()
- : MessageInstance(MSG_OSD_REPOPREPLY, HEAD_VERSION, COMPAT_VERSION),
+ : MOSDFastDispatchOp{MSG_OSD_REPOPREPLY, HEAD_VERSION, COMPAT_VERSION},
map_epoch(0),
min_epoch(0),
ack_type(0), result(0),
out << ")";
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
-
#endif
* instruct an OSD initiate a replica scrub on a specific PG
*/
-class MOSDRepScrub : public MessageInstance<MOSDRepScrub, MOSDFastDispatchOp> {
+class MOSDRepScrub : public MOSDFastDispatchOp {
public:
- friend factory;
-
static constexpr int HEAD_VERSION = 9;
static constexpr int COMPAT_VERSION = 6;
}
MOSDRepScrub()
- : MessageInstance(MSG_OSD_REP_SCRUB, HEAD_VERSION, COMPAT_VERSION),
+ : MOSDFastDispatchOp{MSG_OSD_REP_SCRUB, HEAD_VERSION, COMPAT_VERSION},
chunky(false),
deep(false) { }
MOSDRepScrub(spg_t pgid, eversion_t scrub_to, epoch_t map_epoch, epoch_t min_epoch,
hobject_t start, hobject_t end, bool deep,
bool preemption, int prio, bool highprio)
- : MessageInstance(MSG_OSD_REP_SCRUB, HEAD_VERSION, COMPAT_VERSION),
+ : MOSDFastDispatchOp{MSG_OSD_REP_SCRUB, HEAD_VERSION, COMPAT_VERSION},
pgid(pgid),
scrub_to(scrub_to),
map_epoch(map_epoch),
* pass a ScrubMap from a shard back to the primary
*/
-class MOSDRepScrubMap : public MessageInstance<MOSDRepScrubMap, MOSDFastDispatchOp> {
+class MOSDRepScrubMap : public MOSDFastDispatchOp {
public:
- friend factory;
-
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 1;
}
MOSDRepScrubMap()
- : MessageInstance(MSG_OSD_REP_SCRUBMAP, HEAD_VERSION, COMPAT_VERSION) {}
+ : MOSDFastDispatchOp{MSG_OSD_REP_SCRUBMAP, HEAD_VERSION, COMPAT_VERSION} {}
MOSDRepScrubMap(spg_t pgid, epoch_t map_epoch, pg_shard_t from)
- : MessageInstance(MSG_OSD_REP_SCRUBMAP, HEAD_VERSION, COMPAT_VERSION),
+ : MOSDFastDispatchOp{MSG_OSD_REP_SCRUBMAP, HEAD_VERSION, COMPAT_VERSION},
pgid(pgid),
map_epoch(map_epoch),
from(from) {}
decode(preempted, p);
}
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
-
#endif
* instruct an OSD to scrub some or all pg(s)
*/
-class MOSDScrub : public MessageInstance<MOSDScrub> {
+class MOSDScrub : public Message {
public:
- friend factory;
-
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 2;
bool repair = false;
bool deep = false;
- MOSDScrub() : MessageInstance(MSG_OSD_SCRUB, HEAD_VERSION, COMPAT_VERSION) {}
+ MOSDScrub() : Message{MSG_OSD_SCRUB, HEAD_VERSION, COMPAT_VERSION} {}
MOSDScrub(const uuid_d& f, bool r, bool d) :
- MessageInstance(MSG_OSD_SCRUB, HEAD_VERSION, COMPAT_VERSION),
+ Message{MSG_OSD_SCRUB, HEAD_VERSION, COMPAT_VERSION},
fsid(f), repair(r), deep(d) {}
MOSDScrub(const uuid_d& f, vector<pg_t>& pgs, bool r, bool d) :
- MessageInstance(MSG_OSD_SCRUB, HEAD_VERSION, COMPAT_VERSION),
+ Message{MSG_OSD_SCRUB, HEAD_VERSION, COMPAT_VERSION},
fsid(f), scrub_pgs(pgs), repair(r), deep(d) {}
private:
~MOSDScrub() override {}
decode(repair, p);
decode(deep, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
* instruct an OSD to scrub some or all pg(s)
*/
-class MOSDScrub2 : public MessageInstance<MOSDScrub2> {
+class MOSDScrub2 : public Message {
public:
- friend factory;
-
static constexpr int HEAD_VERSION = 1;
static constexpr int COMPAT_VERSION = 1;
bool repair = false;
bool deep = false;
- MOSDScrub2() : MessageInstance(MSG_OSD_SCRUB2, HEAD_VERSION, COMPAT_VERSION) {}
+ MOSDScrub2() : Message{MSG_OSD_SCRUB2, HEAD_VERSION, COMPAT_VERSION} {}
MOSDScrub2(const uuid_d& f, epoch_t e, vector<spg_t>& pgs, bool r, bool d) :
- MessageInstance(MSG_OSD_SCRUB2, HEAD_VERSION, COMPAT_VERSION),
+ Message{MSG_OSD_SCRUB2, HEAD_VERSION, COMPAT_VERSION},
fsid(f), epoch(e), scrub_pgs(pgs), repair(r), deep(d) {}
private:
~MOSDScrub2() override {}
decode(repair, p);
decode(deep, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#include "MOSDFastDispatchOp.h"
-class MOSDScrubReserve : public MessageInstance<MOSDScrubReserve, MOSDFastDispatchOp> {
-public:
- friend factory;
+class MOSDScrubReserve : public MOSDFastDispatchOp {
private:
static constexpr int HEAD_VERSION = 1;
static constexpr int COMPAT_VERSION = 1;
}
MOSDScrubReserve()
- : MessageInstance(MSG_OSD_SCRUB_RESERVE, HEAD_VERSION, COMPAT_VERSION),
+ : MOSDFastDispatchOp{MSG_OSD_SCRUB_RESERVE, HEAD_VERSION, COMPAT_VERSION},
map_epoch(0), type(-1) {}
MOSDScrubReserve(spg_t pgid,
epoch_t map_epoch,
int type,
pg_shard_t from)
- : MessageInstance(MSG_OSD_SCRUB_RESERVE, HEAD_VERSION, COMPAT_VERSION),
+ : MOSDFastDispatchOp{MSG_OSD_SCRUB_RESERVE, HEAD_VERSION, COMPAT_VERSION},
pgid(pgid), map_epoch(map_epoch),
type(type), from(from) {}
encode(type, payload);
encode(from, payload);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "osd/osd_types.h"
#include "messages/PaxosServiceMessage.h"
-class MPGStats : public MessageInstance<MPGStats, PaxosServiceMessage> {
+class MPGStats : public PaxosServiceMessage {
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 1;
-public:
- friend factory;
+public:
uuid_d fsid;
std::map<pg_t, pg_stat_t> pg_stat;
osd_stat_t osd_stat;
std::map<int64_t, store_statfs_t> pool_stat;
epoch_t epoch = 0;
- MPGStats() : MessageInstance(MSG_PGSTATS, 0, HEAD_VERSION, COMPAT_VERSION) {}
+ MPGStats() : PaxosServiceMessage{MSG_PGSTATS, 0, HEAD_VERSION, COMPAT_VERSION} {}
MPGStats(const uuid_d& f, epoch_t e)
- : MessageInstance(MSG_PGSTATS, 0, HEAD_VERSION, COMPAT_VERSION),
+ : PaxosServiceMessage{MSG_PGSTATS, 0, HEAD_VERSION, COMPAT_VERSION},
fsid(f),
epoch(e)
{}
#include "osd/osd_types.h"
-class MPGStatsAck : public MessageInstance<MPGStatsAck> {
+class MPGStatsAck : public Message {
public:
- friend factory;
-
map<pg_t,pair<version_t,epoch_t> > pg_stat;
- MPGStatsAck() : MessageInstance(MSG_PGSTATSACK) {}
+ MPGStatsAck() : Message{MSG_PGSTATSACK} {}
private:
~MPGStatsAck() override {}
auto p = payload.cbegin();
decode(pg_stat, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
-class MPing : public MessageInstance<MPing> {
+class MPing : public Message {
public:
- friend factory;
-
- MPing() : MessageInstance(CEPH_MSG_PING) {}
+ MPing() : Message{CEPH_MSG_PING} {}
private:
~MPing() override {}
#include "messages/PaxosServiceMessage.h"
-class MPoolOp : public MessageInstance<MPoolOp, PaxosServiceMessage> {
-public:
- friend factory;
+class MPoolOp : public PaxosServiceMessage {
private:
static constexpr int HEAD_VERSION = 4;
static constexpr int COMPAT_VERSION = 2;
__s16 crush_rule = 0;
MPoolOp()
- : MessageInstance(CEPH_MSG_POOLOP, 0, HEAD_VERSION, COMPAT_VERSION) { }
+ : PaxosServiceMessage{CEPH_MSG_POOLOP, 0, HEAD_VERSION, COMPAT_VERSION} {}
MPoolOp(const uuid_d& f, ceph_tid_t t, int p, std::string& n, int o, version_t v)
- : MessageInstance(CEPH_MSG_POOLOP, v, HEAD_VERSION, COMPAT_VERSION),
+ : PaxosServiceMessage{CEPH_MSG_POOLOP, v, HEAD_VERSION, COMPAT_VERSION},
fsid(f), pool(p), name(n), op(o),
snapid(0), crush_rule(0) {
set_tid(t);
} else
crush_rule = -1;
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "common/errno.h"
-class MPoolOpReply : public MessageInstance<MPoolOpReply, PaxosServiceMessage> {
+class MPoolOpReply : public PaxosServiceMessage {
public:
- friend factory;
-
uuid_d fsid;
__u32 replyCode = 0;
epoch_t epoch = 0;
ceph::buffer::list response_data;
- MPoolOpReply() : MessageInstance(CEPH_MSG_POOLOP_REPLY, 0)
+ MPoolOpReply() : PaxosServiceMessage{CEPH_MSG_POOLOP_REPLY, 0}
{}
MPoolOpReply( uuid_d& f, ceph_tid_t t, int rc, int e, version_t v) :
- MessageInstance(CEPH_MSG_POOLOP_REPLY, v),
+ PaxosServiceMessage{CEPH_MSG_POOLOP_REPLY, v},
fsid(f),
replyCode(rc),
epoch(e) {
}
MPoolOpReply(uuid_d& f, ceph_tid_t t, int rc, int e, version_t v,
ceph::buffer::list *blp) :
- MessageInstance(CEPH_MSG_POOLOP_REPLY, v),
+ PaxosServiceMessage{CEPH_MSG_POOLOP_REPLY, v},
fsid(f),
replyCode(rc),
epoch(e) {
decode(response_data, p);
}
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
#include "messages/MOSDPeeringOp.h"
-class MRecoveryReserve : public MessageInstance<MRecoveryReserve, MOSDPeeringOp> {
-public:
- friend factory;
+class MRecoveryReserve : public MOSDPeeringOp {
private:
static constexpr int HEAD_VERSION = 3;
static constexpr int COMPAT_VERSION = 2;
}
MRecoveryReserve()
- : MessageInstance(MSG_OSD_RECOVERY_RESERVE, HEAD_VERSION, COMPAT_VERSION),
+ : MOSDPeeringOp{MSG_OSD_RECOVERY_RESERVE, HEAD_VERSION, COMPAT_VERSION},
query_epoch(0), type(-1) {}
MRecoveryReserve(int type,
spg_t pgid,
epoch_t query_epoch,
unsigned prio = 0)
- : MessageInstance(MSG_OSD_RECOVERY_RESERVE, HEAD_VERSION, COMPAT_VERSION),
+ : MOSDPeeringOp{MSG_OSD_RECOVERY_RESERVE, HEAD_VERSION, COMPAT_VERSION},
pgid(pgid), query_epoch(query_epoch),
type(type), priority(prio) {}
encode(pgid.shard, payload);
encode(priority, payload);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "messages/PaxosServiceMessage.h"
-class MRemoveSnaps : public MessageInstance<MRemoveSnaps, PaxosServiceMessage> {
+class MRemoveSnaps : public PaxosServiceMessage {
public:
- friend factory;
-
map<int, vector<snapid_t> > snaps;
protected:
MRemoveSnaps() :
- MessageInstance(MSG_REMOVE_SNAPS, 0) { }
+ PaxosServiceMessage{MSG_REMOVE_SNAPS, 0} { }
MRemoveSnaps(map<int, vector<snapid_t> >& s) :
- MessageInstance(MSG_REMOVE_SNAPS, 0) {
+ PaxosServiceMessage{MSG_REMOVE_SNAPS, 0} {
snaps.swap(s);
}
~MRemoveSnaps() override {}
decode(snaps, p);
ceph_assert(p.end());
}
-
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
#include "include/encoding.h"
-class MRoute : public MessageInstance<MRoute> {
+class MRoute : public Message {
public:
- friend factory;
-
static constexpr int HEAD_VERSION = 3;
static constexpr int COMPAT_VERSION = 3;
Message *msg;
epoch_t send_osdmap_first;
- MRoute() : MessageInstance(MSG_ROUTE, HEAD_VERSION, COMPAT_VERSION),
+ MRoute() : Message{MSG_ROUTE, HEAD_VERSION, COMPAT_VERSION},
session_mon_tid(0),
msg(NULL),
send_osdmap_first(0) {}
MRoute(uint64_t t, Message *m)
- : MessageInstance(MSG_ROUTE, HEAD_VERSION, COMPAT_VERSION),
+ : Message{MSG_ROUTE, HEAD_VERSION, COMPAT_VERSION},
session_mon_tid(t),
msg(m),
send_osdmap_first(0) {}
else
o << " tid (none)";
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#include "msg/Message.h"
#include "mgr/ServiceMap.h"
-class MServiceMap : public MessageInstance<MServiceMap> {
+class MServiceMap : public Message {
public:
- friend factory;
-
ServiceMap service_map;
- MServiceMap() : MessageInstance(MSG_SERVICE_MAP) { }
+ MServiceMap() : Message{MSG_SERVICE_MAP} { }
explicit MServiceMap(const ServiceMap& sm)
- : MessageInstance(MSG_SERVICE_MAP),
+ : Message{MSG_SERVICE_MAP},
service_map(sm) {
}
private:
auto p = payload.cbegin();
decode(service_map, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#include <sys/statvfs.h> /* or <sys/statfs.h> */
#include "messages/PaxosServiceMessage.h"
-class MStatfs : public MessageInstance<MStatfs, PaxosServiceMessage> {
-public:
- friend factory;
+class MStatfs : public PaxosServiceMessage {
private:
static constexpr int HEAD_VERSION = 2;
static constexpr int COMPAT_VERSION = 1;
uuid_d fsid;
boost::optional<int64_t> data_pool;
- MStatfs() : MessageInstance(CEPH_MSG_STATFS, 0, HEAD_VERSION, COMPAT_VERSION) {}
+ MStatfs() : PaxosServiceMessage{CEPH_MSG_STATFS, 0, HEAD_VERSION, COMPAT_VERSION} {}
MStatfs(const uuid_d& f, ceph_tid_t t, boost::optional<int64_t> _data_pool,
- version_t v) : MessageInstance(CEPH_MSG_STATFS, v,
- HEAD_VERSION, COMPAT_VERSION),
- fsid(f), data_pool(_data_pool) {
+ version_t v)
+ : PaxosServiceMessage{CEPH_MSG_STATFS, v, HEAD_VERSION, COMPAT_VERSION},
+ fsid(f), data_pool(_data_pool) {
set_tid(t);
}
data_pool = boost::optional<int64_t> ();
}
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#ifndef CEPH_MSTATFSREPLY_H
#define CEPH_MSTATFSREPLY_H
-class MStatfsReply : public MessageInstance<MStatfsReply> {
+class MStatfsReply : public Message {
public:
- friend factory;
-
struct ceph_mon_statfs_reply h{};
- MStatfsReply() : MessageInstance(CEPH_MSG_STATFS_REPLY) {}
- MStatfsReply(uuid_d &f, ceph_tid_t t, epoch_t epoch) : MessageInstance(CEPH_MSG_STATFS_REPLY) {
+ MStatfsReply() : 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.bytes(), sizeof(h.fsid));
header.tid = t;
h.version = epoch;
auto p = payload.cbegin();
decode(h, p);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
#ifndef CEPH_MTIMECHECK_H
#define CEPH_MTIMECHECK_H
-class MTimeCheck : public MessageInstance<MTimeCheck> {
+class MTimeCheck : public Message {
public:
- friend factory;
-
static constexpr int HEAD_VERSION = 1;
enum {
map<entity_inst_t, double> skews;
map<entity_inst_t, double> latencies;
- MTimeCheck() : MessageInstance(MSG_TIMECHECK, HEAD_VERSION) { }
+ MTimeCheck() : Message{MSG_TIMECHECK, HEAD_VERSION} {}
MTimeCheck(int op) :
- MessageInstance(MSG_TIMECHECK, HEAD_VERSION),
+ Message{MSG_TIMECHECK, HEAD_VERSION},
op(op)
- { }
+ {}
private:
- ~MTimeCheck() override { }
+ ~MTimeCheck() override {}
public:
std::string_view get_type_name() const override { return "time_check"; }
encode(skews, payload, features);
encode(latencies, payload, features);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif /* CEPH_MTIMECHECK_H */
#pragma once
-class MTimeCheck2 : public MessageInstance<MTimeCheck2> {
+class MTimeCheck2 : public Message {
public:
- friend factory;
-
static constexpr int HEAD_VERSION = 1;
static constexpr int COMPAT_VERSION = 1;
map<int, double> skews;
map<int, double> latencies;
- MTimeCheck2() : MessageInstance(MSG_TIMECHECK2, HEAD_VERSION, COMPAT_VERSION) { }
+ MTimeCheck2() : Message{MSG_TIMECHECK2, HEAD_VERSION, COMPAT_VERSION} { }
MTimeCheck2(int op) :
- MessageInstance(MSG_TIMECHECK2, HEAD_VERSION, COMPAT_VERSION),
+ Message{MSG_TIMECHECK2, HEAD_VERSION, COMPAT_VERSION},
op(op)
{ }
encode(skews, payload, features);
encode(latencies, payload, features);
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#include "msg/Message.h"
-class MWatchNotify : public MessageInstance<MWatchNotify> {
-public:
- friend factory;
+class MWatchNotify : public Message {
private:
static constexpr int HEAD_VERSION = 3;
static constexpr int COMPAT_VERSION = 1;
uint64_t notifier_gid; ///< who sent the notify
MWatchNotify()
- : MessageInstance(CEPH_MSG_WATCH_NOTIFY, HEAD_VERSION, COMPAT_VERSION) { }
+ : Message{CEPH_MSG_WATCH_NOTIFY, HEAD_VERSION, COMPAT_VERSION} { }
MWatchNotify(uint64_t c, uint64_t v, uint64_t i, uint8_t o, ceph::buffer::list b)
- : MessageInstance(CEPH_MSG_WATCH_NOTIFY, HEAD_VERSION, COMPAT_VERSION),
+ : Message{CEPH_MSG_WATCH_NOTIFY, HEAD_VERSION, COMPAT_VERSION},
cookie(c),
ver(v),
notify_id(i),
<< " ret " << return_code
<< ")";
}
+private:
+ template<class T, typename... Args>
+ friend boost::intrusive_ptr<T> ceph::make_message(Args&&... args);
};
#endif
{
ceph_assert(lock.is_locked_by_me());
- auto configure = new MMgrConfigure();
+ auto configure = make_message<MMgrConfigure>();
configure->stats_period = g_conf().get_val<int64_t>("mgr_stats_period");
configure->stats_threshold = g_conf().get_val<int64_t>("mgr_stats_threshold");
osd_perf_metric_collector.get_queries();
}
- c->send_message(configure);
+ c->send_message2(configure);
}
OSDPerfMetricQueryID DaemonServer::add_osd_perf_query(
void MgrClient::_send_open()
{
if (session && session->con) {
- auto open = new MMgrOpen();
+ auto open = make_message<MMgrOpen>();
if (!service_name.empty()) {
open->service_name = service_name;
open->daemon_name = daemon_name;
}
cct->_conf.get_config_bl(0, &open->config_bl, &last_config_bl_version);
cct->_conf.get_defaults_bl(&open->config_defaults_bl);
- session->con->send_message(open);
+ session->con->send_message2(open);
}
}
ceph_assert(session);
report_callback = nullptr;
- auto report = new MMgrReport();
+ auto report = make_message<MMgrReport>();
auto pcc = cct->get_perfcounters_collection();
pcc->with_counters([this, report](
get_perf_report_cb(&report->osd_perf_metric_reports);
}
- session->con->send_message(report);
+ session->con->send_message2(report);
}
void MgrClient::send_pgstats()
MDSMap null_map;
null_map.epoch = fsmap.epoch;
null_map.compat = fsmap.compat;
- auto m = MMDSMap::create(mon->monmap->fsid, null_map);
+ auto m = make_message<MMDSMap>(mon->monmap->fsid, null_map);
mon->send_reply(op, m.detach());
return true;
} else {
ceph_assert(effective_epoch > 0);
_note_beacon(m);
{
- auto beacon = MMDSBeacon::create(mon->monmap->fsid,
+ auto beacon = make_message<MMDSBeacon>(mon->monmap->fsid,
m->get_global_id(), m->get_name(), effective_epoch,
state, seq, CEPH_FEATURES_SUPPORTED_DEFAULT);
mon->send_reply(op, beacon.detach());
MDSMap null_map;
null_map.epoch = fsmap.epoch;
null_map.compat = fsmap.compat;
- auto m = MMDSMap::create(mon->monmap->fsid, null_map);
+ auto m = make_message<MMDSMap>(mon->monmap->fsid, null_map);
mon->send_reply(op, m.detach());
} else {
dispatch(op); // try again
last_beacon.erase(gid);
// Respond to MDS, so that it knows it can continue to shut down
- auto beacon = MMDSBeacon::create(
+ auto beacon = make_message<MMDSBeacon>(
mon->monmap->fsid, m->get_global_id(),
m->get_name(), pending.get_epoch(), state, seq,
CEPH_FEATURES_SUPPORTED_DEFAULT);
request_proposal(mon->osdmon());
// Respond to MDS, so that it knows it can continue to shut down
- auto beacon = MMDSBeacon::create(mon->monmap->fsid,
+ auto beacon = make_message<MMDSBeacon>(mon->monmap->fsid,
m->get_global_id(), m->get_name(), pending.get_epoch(), state, seq,
CEPH_FEATURES_SUPPORTED_DEFAULT);
mon->send_reply(op, beacon.detach());
MDSMap null_map;
null_map.epoch = fsmap.epoch;
null_map.compat = fsmap.compat;
- auto m = MMDSMap::create(mon->monmap->fsid, null_map);
+ auto m = make_message<MMDSMap>(mon->monmap->fsid, null_map);
mon->send_reply(op, m.detach());
} else {
- auto beacon = MMDSBeacon::create(mon->monmap->fsid,
+ auto beacon = make_message<MMDSBeacon>(mon->monmap->fsid,
m->get_global_id(), m->get_name(), fsmap.get_epoch(),
m->get_state(), m->get_seq(), CEPH_FEATURES_SUPPORTED_DEFAULT);
mon->send_reply(op, beacon.detach());
if (sub->next > mds_map->epoch) {
return;
}
- auto msg = MMDSMap::create(mon->monmap->fsid, *mds_map);
+ auto msg = make_message<MMDSMap>(mon->monmap->fsid, *mds_map);
sub->session->con->send_message(msg.detach());
if (sub->onetime) {
if (sub->next <= map.get_epoch()) {
dout(20) << "Sending map to subscriber " << sub->session->con
<< " " << sub->session->con->get_peer_addr() << dendl;
- sub->session->con->send_message(new MMgrMap(map));
+ sub->session->con->send_message2(make_message<MMgrMap>(map));
if (sub->onetime) {
mon->session_map.remove_sub(sub);
} else {
for (auto sub : *(mon->session_map.subs[type])) {
dout(10) << __func__ << " sending digest to subscriber " << sub->session->con
<< " " << sub->session->con->get_peer_addr() << dendl;
- MMgrDigest *mdigest = new MMgrDigest;
+ auto mdigest = make_message<MMgrDigest>();
JSONFormatter f;
mon->get_health_status(true, &f, nullptr, nullptr, nullptr);
f.flush(mdigest->mon_status_json);
f.reset();
- sub->session->con->send_message(mdigest);
+ sub->session->con->send_message2(mdigest);
}
timer:
// -- with payload --
case MSG_PGSTATS:
- m = MPGStats::create();
+ m = make_message<MPGStats>();
break;
case MSG_PGSTATSACK:
- m = MPGStatsAck::create();
+ m = make_message<MPGStatsAck>();
break;
case CEPH_MSG_STATFS:
- m = MStatfs::create();
+ m = make_message<MStatfs>();
break;
case CEPH_MSG_STATFS_REPLY:
- m = MStatfsReply::create();
+ m = make_message<MStatfsReply>();
break;
case MSG_GETPOOLSTATS:
- m = MGetPoolStats::create();
+ m = make_message<MGetPoolStats>();
break;
case MSG_GETPOOLSTATSREPLY:
- m = MGetPoolStatsReply::create();
+ m = make_message<MGetPoolStatsReply>();
break;
case CEPH_MSG_POOLOP:
- m = MPoolOp::create();
+ m = make_message<MPoolOp>();
break;
case CEPH_MSG_POOLOP_REPLY:
- m = MPoolOpReply::create();
+ m = make_message<MPoolOpReply>();
break;
case MSG_MON_COMMAND:
- m = MMonCommand::create();
+ m = make_message<MMonCommand>();
break;
case MSG_MON_COMMAND_ACK:
- m = MMonCommandAck::create();
+ m = make_message<MMonCommandAck>();
break;
case MSG_MON_PAXOS:
- m = MMonPaxos::create();
+ m = make_message<MMonPaxos>();
break;
case MSG_CONFIG:
- m = MConfig::create();
+ m = make_message<MConfig>();
break;
case MSG_GET_CONFIG:
- m = MGetConfig::create();
+ m = make_message<MGetConfig>();
break;
case MSG_MON_PROBE:
- m = MMonProbe::create();
+ m = make_message<MMonProbe>();
break;
case MSG_MON_JOIN:
- m = MMonJoin::create();
+ m = make_message<MMonJoin>();
break;
case MSG_MON_ELECTION:
- m = MMonElection::create();
+ m = make_message<MMonElection>();
break;
case MSG_MON_SYNC:
- m = MMonSync::create();
+ m = make_message<MMonSync>();
break;
case MSG_MON_SCRUB:
- m = MMonScrub::create();
+ m = make_message<MMonScrub>();
break;
case MSG_LOG:
- m = MLog::create();
+ m = make_message<MLog>();
break;
case MSG_LOGACK:
- m = MLogAck::create();
+ m = make_message<MLogAck>();
break;
case CEPH_MSG_PING:
- m = MPing::create();
+ m = make_message<MPing>();
break;
case MSG_COMMAND:
- m = MCommand::create();
+ m = make_message<MCommand>();
break;
case MSG_COMMAND_REPLY:
- m = MCommandReply::create();
+ m = make_message<MCommandReply>();
break;
case MSG_OSD_BACKFILL_RESERVE:
- m = MBackfillReserve::create();
+ m = make_message<MBackfillReserve>();
break;
case MSG_OSD_RECOVERY_RESERVE:
- m = MRecoveryReserve::create();
+ m = make_message<MRecoveryReserve>();
break;
case MSG_OSD_FORCE_RECOVERY:
- m = MOSDForceRecovery::create();
+ m = make_message<MOSDForceRecovery>();
break;
case MSG_ROUTE:
- m = MRoute::create();
+ m = make_message<MRoute>();
break;
case MSG_FORWARD:
- m = MForward::create();
+ m = make_message<MForward>();
break;
case CEPH_MSG_MON_MAP:
- m = MMonMap::create();
+ m = make_message<MMonMap>();
break;
case CEPH_MSG_MON_GET_MAP:
- m = MMonGetMap::create();
+ m = make_message<MMonGetMap>();
break;
case CEPH_MSG_MON_GET_OSDMAP:
- m = MMonGetOSDMap::create();
+ m = make_message<MMonGetOSDMap>();
break;
case CEPH_MSG_MON_GET_VERSION:
- m = MMonGetVersion::create();
+ m = make_message<MMonGetVersion>();
break;
case CEPH_MSG_MON_GET_VERSION_REPLY:
- m = MMonGetVersionReply::create();
+ m = make_message<MMonGetVersionReply>();
break;
case CEPH_MSG_MON_METADATA:
- m = MMonMetadata::create();
+ m = make_message<MMonMetadata>();
break;
case MSG_OSD_BOOT:
- m = MOSDBoot::create();
+ m = make_message<MOSDBoot>();
break;
case MSG_OSD_ALIVE:
- m = MOSDAlive::create();
+ m = make_message<MOSDAlive>();
break;
case MSG_OSD_BEACON:
- m = MOSDBeacon::create();
+ m = make_message<MOSDBeacon>();
break;
case MSG_OSD_PGTEMP:
- m = MOSDPGTemp::create();
+ m = make_message<MOSDPGTemp>();
break;
case MSG_OSD_FAILURE:
- m = MOSDFailure::create();
+ m = make_message<MOSDFailure>();
break;
case MSG_OSD_MARK_ME_DOWN:
- m = MOSDMarkMeDown::create();
+ m = make_message<MOSDMarkMeDown>();
break;
case MSG_OSD_FULL:
- m = MOSDFull::create();
+ m = make_message<MOSDFull>();
break;
case MSG_OSD_PING:
- m = MOSDPing::create();
+ m = make_message<MOSDPing>();
break;
case CEPH_MSG_OSD_OP:
- m = MOSDOp::create();
+ m = make_message<MOSDOp>();
break;
case CEPH_MSG_OSD_OPREPLY:
- m = MOSDOpReply::create();
+ m = make_message<MOSDOpReply>();
break;
case MSG_OSD_REPOP:
- m = MOSDRepOp::create();
+ m = make_message<MOSDRepOp>();
break;
case MSG_OSD_REPOPREPLY:
- m = MOSDRepOpReply::create();
+ m = make_message<MOSDRepOpReply>();
break;
case MSG_OSD_PG_CREATED:
- m = MOSDPGCreated::create();
+ m = make_message<MOSDPGCreated>();
break;
case MSG_OSD_PG_UPDATE_LOG_MISSING:
- m = MOSDPGUpdateLogMissing::create();
+ m = make_message<MOSDPGUpdateLogMissing>();
break;
case MSG_OSD_PG_UPDATE_LOG_MISSING_REPLY:
- m = MOSDPGUpdateLogMissingReply::create();
+ m = make_message<MOSDPGUpdateLogMissingReply>();
break;
case CEPH_MSG_OSD_BACKOFF:
- m = MOSDBackoff::create();
+ m = make_message<MOSDBackoff>();
break;
case CEPH_MSG_OSD_MAP:
- m = MOSDMap::create();
+ m = make_message<MOSDMap>();
break;
case CEPH_MSG_WATCH_NOTIFY:
- m = MWatchNotify::create();
+ m = make_message<MWatchNotify>();
break;
case MSG_OSD_PG_NOTIFY:
- m = MOSDPGNotify::create();
+ m = make_message<MOSDPGNotify>();
break;
case MSG_OSD_PG_QUERY:
- m = MOSDPGQuery::create();
+ m = make_message<MOSDPGQuery>();
break;
case MSG_OSD_PG_LOG:
- m = MOSDPGLog::create();
+ m = make_message<MOSDPGLog>();
break;
case MSG_OSD_PG_REMOVE:
- m = MOSDPGRemove::create();
+ m = make_message<MOSDPGRemove>();
break;
case MSG_OSD_PG_INFO:
- m = MOSDPGInfo::create();
+ m = make_message<MOSDPGInfo>();
break;
case MSG_OSD_PG_CREATE:
- m = MOSDPGCreate::create();
+ m = make_message<MOSDPGCreate>();
break;
case MSG_OSD_PG_CREATE2:
- m = MOSDPGCreate2::create();
+ m = make_message<MOSDPGCreate2>();
break;
case MSG_OSD_PG_TRIM:
- m = MOSDPGTrim::create();
+ m = make_message<MOSDPGTrim>();
break;
case MSG_OSD_SCRUB:
- m = MOSDScrub::create();
+ m = make_message<MOSDScrub>();
break;
case MSG_OSD_SCRUB2:
- m = MOSDScrub2::create();
+ m = make_message<MOSDScrub2>();
break;
case MSG_OSD_SCRUB_RESERVE:
- m = MOSDScrubReserve::create();
+ m = make_message<MOSDScrubReserve>();
break;
case MSG_REMOVE_SNAPS:
- m = MRemoveSnaps::create();
+ m = make_message<MRemoveSnaps>();
break;
case MSG_OSD_REP_SCRUB:
- m = MOSDRepScrub::create();
+ m = make_message<MOSDRepScrub>();
break;
case MSG_OSD_REP_SCRUBMAP:
- m = MOSDRepScrubMap::create();
+ m = make_message<MOSDRepScrubMap>();
break;
case MSG_OSD_PG_SCAN:
- m = MOSDPGScan::create();
+ m = make_message<MOSDPGScan>();
break;
case MSG_OSD_PG_BACKFILL:
- m = MOSDPGBackfill::create();
+ m = make_message<MOSDPGBackfill>();
break;
case MSG_OSD_PG_BACKFILL_REMOVE:
- m = MOSDPGBackfillRemove::create();
+ m = make_message<MOSDPGBackfillRemove>();
break;
case MSG_OSD_PG_PUSH:
- m = MOSDPGPush::create();
+ m = make_message<MOSDPGPush>();
break;
case MSG_OSD_PG_PULL:
- m = MOSDPGPull::create();
+ m = make_message<MOSDPGPull>();
break;
case MSG_OSD_PG_PUSH_REPLY:
- m = MOSDPGPushReply::create();
+ m = make_message<MOSDPGPushReply>();
break;
case MSG_OSD_PG_RECOVERY_DELETE:
- m = MOSDPGRecoveryDelete::create();
+ m = make_message<MOSDPGRecoveryDelete>();
break;
case MSG_OSD_PG_RECOVERY_DELETE_REPLY:
- m = MOSDPGRecoveryDeleteReply::create();
+ m = make_message<MOSDPGRecoveryDeleteReply>();
break;
case MSG_OSD_PG_READY_TO_MERGE:
- m = MOSDPGReadyToMerge::create();
+ m = make_message<MOSDPGReadyToMerge>();
break;
case MSG_OSD_EC_WRITE:
- m = MOSDECSubOpWrite::create();
+ m = make_message<MOSDECSubOpWrite>();
break;
case MSG_OSD_EC_WRITE_REPLY:
- m = MOSDECSubOpWriteReply::create();
+ m = make_message<MOSDECSubOpWriteReply>();
break;
case MSG_OSD_EC_READ:
- m = MOSDECSubOpRead::create();
+ m = make_message<MOSDECSubOpRead>();
break;
case MSG_OSD_EC_READ_REPLY:
- m = MOSDECSubOpReadReply::create();
+ m = make_message<MOSDECSubOpReadReply>();
break;
// auth
case CEPH_MSG_AUTH:
- m = MAuth::create();
+ m = make_message<MAuth>();
break;
case CEPH_MSG_AUTH_REPLY:
- m = MAuthReply::create();
+ m = make_message<MAuthReply>();
break;
case MSG_MON_GLOBAL_ID:
- m = MMonGlobalID::create();
+ m = make_message<MMonGlobalID>();
break;
// clients
case CEPH_MSG_MON_SUBSCRIBE:
- m = MMonSubscribe::create();
+ m = make_message<MMonSubscribe>();
break;
case CEPH_MSG_MON_SUBSCRIBE_ACK:
- m = MMonSubscribeAck::create();
+ m = make_message<MMonSubscribeAck>();
break;
case CEPH_MSG_CLIENT_SESSION:
- m = MClientSession::create();
+ m = make_message<MClientSession>();
break;
case CEPH_MSG_CLIENT_RECONNECT:
- m = MClientReconnect::create();
+ m = make_message<MClientReconnect>();
break;
case CEPH_MSG_CLIENT_REQUEST:
- m = MClientRequest::create();
+ m = make_message<MClientRequest>();
break;
case CEPH_MSG_CLIENT_REQUEST_FORWARD:
- m = MClientRequestForward::create();
+ m = make_message<MClientRequestForward>();
break;
case CEPH_MSG_CLIENT_REPLY:
- m = MClientReply::create();
+ m = make_message<MClientReply>();
break;
case CEPH_MSG_CLIENT_RECLAIM:
- m = MClientReclaim::create();
+ m = make_message<MClientReclaim>();
break;
case CEPH_MSG_CLIENT_RECLAIM_REPLY:
- m = MClientReclaimReply::create();
+ m = make_message<MClientReclaimReply>();
break;
case CEPH_MSG_CLIENT_CAPS:
- m = MClientCaps::create();
+ m = make_message<MClientCaps>();
break;
case CEPH_MSG_CLIENT_CAPRELEASE:
- m = MClientCapRelease::create();
+ m = make_message<MClientCapRelease>();
break;
case CEPH_MSG_CLIENT_LEASE:
- m = MClientLease::create();
+ m = make_message<MClientLease>();
break;
case CEPH_MSG_CLIENT_SNAP:
- m = MClientSnap::create();
+ m = make_message<MClientSnap>();
break;
case CEPH_MSG_CLIENT_QUOTA:
- m = MClientQuota::create();
+ m = make_message<MClientQuota>();
break;
// mds
case MSG_MDS_SLAVE_REQUEST:
- m = MMDSSlaveRequest::create();
+ m = make_message<MMDSSlaveRequest>();
break;
case CEPH_MSG_MDS_MAP:
- m = MMDSMap::create();
+ m = make_message<MMDSMap>();
break;
case CEPH_MSG_FS_MAP:
- m = MFSMap::create();
+ m = make_message<MFSMap>();
break;
case CEPH_MSG_FS_MAP_USER:
- m = MFSMapUser::create();
+ m = make_message<MFSMapUser>();
break;
case MSG_MDS_BEACON:
- m = MMDSBeacon::create();
+ m = make_message<MMDSBeacon>();
break;
case MSG_MDS_OFFLOAD_TARGETS:
- m = MMDSLoadTargets::create();
+ m = make_message<MMDSLoadTargets>();
break;
case MSG_MDS_RESOLVE:
- m = MMDSResolve::create();
+ m = make_message<MMDSResolve>();
break;
case MSG_MDS_RESOLVEACK:
- m = MMDSResolveAck::create();
+ m = make_message<MMDSResolveAck>();
break;
case MSG_MDS_CACHEREJOIN:
- m = MMDSCacheRejoin::create();
+ m = make_message<MMDSCacheRejoin>();
break;
case MSG_MDS_DIRUPDATE:
- m = MDirUpdate::create();
+ m = make_message<MDirUpdate>();
break;
case MSG_MDS_DISCOVER:
- m = MDiscover::create();
+ m = make_message<MDiscover>();
break;
case MSG_MDS_DISCOVERREPLY:
- m = MDiscoverReply::create();
+ m = make_message<MDiscoverReply>();
break;
case MSG_MDS_FINDINO:
- m = MMDSFindIno::create();
+ m = make_message<MMDSFindIno>();
break;
case MSG_MDS_FINDINOREPLY:
- m = MMDSFindInoReply::create();
+ m = make_message<MMDSFindInoReply>();
break;
case MSG_MDS_OPENINO:
- m = MMDSOpenIno::create();
+ m = make_message<MMDSOpenIno>();
break;
case MSG_MDS_OPENINOREPLY:
- m = MMDSOpenInoReply::create();
+ m = make_message<MMDSOpenInoReply>();
break;
case MSG_MDS_SNAPUPDATE:
- m = MMDSSnapUpdate::create();
+ m = make_message<MMDSSnapUpdate>();
break;
case MSG_MDS_FRAGMENTNOTIFY:
- m = MMDSFragmentNotify::create();
+ m = make_message<MMDSFragmentNotify>();
break;
case MSG_MDS_FRAGMENTNOTIFYACK:
- m = MMDSFragmentNotifyAck::create();
+ m = make_message<MMDSFragmentNotifyAck>();
break;
case MSG_MDS_EXPORTDIRDISCOVER:
- m = MExportDirDiscover::create();
+ m = make_message<MExportDirDiscover>();
break;
case MSG_MDS_EXPORTDIRDISCOVERACK:
- m = MExportDirDiscoverAck::create();
+ m = make_message<MExportDirDiscoverAck>();
break;
case MSG_MDS_EXPORTDIRCANCEL:
- m = MExportDirCancel::create();
+ m = make_message<MExportDirCancel>();
break;
case MSG_MDS_EXPORTDIR:
- m = MExportDir::create();
+ m = make_message<MExportDir>();
break;
case MSG_MDS_EXPORTDIRACK:
- m = MExportDirAck::create();
+ m = make_message<MExportDirAck>();
break;
case MSG_MDS_EXPORTDIRFINISH:
- m = MExportDirFinish::create();
+ m = make_message<MExportDirFinish>();
break;
case MSG_MDS_EXPORTDIRNOTIFY:
- m = MExportDirNotify::create();
+ m = make_message<MExportDirNotify>();
break;
case MSG_MDS_EXPORTDIRNOTIFYACK:
- m = MExportDirNotifyAck::create();
+ m = make_message<MExportDirNotifyAck>();
break;
case MSG_MDS_EXPORTDIRPREP:
- m = MExportDirPrep::create();
+ m = make_message<MExportDirPrep>();
break;
case MSG_MDS_EXPORTDIRPREPACK:
- m = MExportDirPrepAck::create();
+ m = make_message<MExportDirPrepAck>();
break;
case MSG_MDS_EXPORTCAPS:
- m = MExportCaps::create();
+ m = make_message<MExportCaps>();
break;
case MSG_MDS_EXPORTCAPSACK:
- m = MExportCapsAck::create();
+ m = make_message<MExportCapsAck>();
break;
case MSG_MDS_GATHERCAPS:
- m = MGatherCaps::create();
+ m = make_message<MGatherCaps>();
break;
case MSG_MDS_DENTRYUNLINK:
- m = MDentryUnlink::create();
+ m = make_message<MDentryUnlink>();
break;
case MSG_MDS_DENTRYLINK:
- m = MDentryLink::create();
+ m = make_message<MDentryLink>();
break;
case MSG_MDS_HEARTBEAT:
- m = MHeartbeat::create();
+ m = make_message<MHeartbeat>();
break;
case MSG_MDS_CACHEEXPIRE:
- m = MCacheExpire::create();
+ m = make_message<MCacheExpire>();
break;
case MSG_MDS_TABLE_REQUEST:
- m = MMDSTableRequest::create();
+ m = make_message<MMDSTableRequest>();
break;
/* case MSG_MDS_INODEUPDATE:
- m = MInodeUpdate::create();
+ m = make_message<MInodeUpdate>();
break;
*/
case MSG_MDS_INODEFILECAPS:
- m = MInodeFileCaps::create();
+ m = make_message<MInodeFileCaps>();
break;
case MSG_MDS_LOCK:
- m = MLock::create();
+ m = make_message<MLock>();
break;
case MSG_MGR_BEACON:
- m = MMgrBeacon::create();
+ m = make_message<MMgrBeacon>();
break;
case MSG_MON_MGR_REPORT:
- m = MMonMgrReport::create();
+ m = make_message<MMonMgrReport>();
break;
case MSG_SERVICE_MAP:
- m = MServiceMap::create();
+ m = make_message<MServiceMap>();
break;
case MSG_MGR_MAP:
- m = MMgrMap::create();
+ m = make_message<MMgrMap>();
break;
case MSG_MGR_DIGEST:
- m = MMgrDigest::create();
+ m = make_message<MMgrDigest>();
break;
case MSG_MGR_OPEN:
- m = MMgrOpen::create();
+ m = make_message<MMgrOpen>();
break;
case MSG_MGR_CLOSE:
- m = MMgrClose::create();
+ m = make_message<MMgrClose>();
break;
case MSG_MGR_REPORT:
- m = MMgrReport::create();
+ m = make_message<MMgrReport>();
break;
case MSG_MGR_CONFIGURE:
- m = MMgrConfigure::create();
+ m = make_message<MMgrConfigure>();
break;
case MSG_TIMECHECK:
- m = MTimeCheck::create();
+ m = make_message<MTimeCheck>();
break;
case MSG_TIMECHECK2:
- m = MTimeCheck2::create();
+ m = make_message<MTimeCheck2>();
break;
case MSG_MON_HEALTH:
- m = MMonHealth::create();
+ m = make_message<MMonHealth>();
break;
case MSG_MON_HEALTH_CHECKS:
- m = MMonHealthChecks::create();
+ m = make_message<MMonHealthChecks>();
break;
// -- simple messages without payload --
case CEPH_MSG_SHUTDOWN:
- m = MGenericMessage::create(type);
+ m = make_message<MGenericMessage>(type);
break;
default:
boost::intrusive_ptr<const T> ref_cast(const boost::intrusive_ptr<const U>& r) noexcept {
return static_cast<const T*>(r.get());
}
-template<class T, class U>
-boost::intrusive_ptr<const T> ref_cast(const boost::intrusive_ptr<const U>&& r) noexcept {
- return {static_cast<const T*>(r.detach()), false};
+template<class T, typename... Args>
+boost::intrusive_ptr<T> make_message(Args&&... args) {
+ return {new T(std::forward<Args>(args)...), false};
}
}
list<ref_t<T>> m_list;
public:
- MessageDencoderImpl() : m_object(T::create()) {}
+ MessageDencoderImpl() : m_object{make_message<T>()} {}
~MessageDencoderImpl() override {}
string decode(bufferlist bl, uint64_t seek) override {