if (!t.empty()) {
ceph::shared_ptr<ObjectStore::Sequencer> osr(
new ObjectStore::Sequencer("fuse"));
- fs->store->apply_transaction(&*osr, t);
+ fs->store->apply_transaction(&*osr, std::move(t));
C_SaferCond waiter;
if (!osr->flush_commit(&waiter))
waiter.wait();
if (!t.empty()) {
ceph::shared_ptr<ObjectStore::Sequencer> osr(
new ObjectStore::Sequencer("fuse"));
- fs->store->apply_transaction(&*osr, t);
+ fs->store->apply_transaction(&*osr, std::move(t));
C_SaferCond waiter;
if (!osr->flush_commit(&waiter))
waiter.wait();
ceph::shared_ptr<ObjectStore::Sequencer> osr(
new ObjectStore::Sequencer("fuse"));
- fs->store->apply_transaction(&*osr, t);
+ fs->store->apply_transaction(&*osr, std::move(t));
C_SaferCond waiter;
if (!osr->flush_commit(&waiter))
waiter.wait();
ceph::shared_ptr<ObjectStore::Sequencer> osr(
new ObjectStore::Sequencer("fuse"));
- fs->store->apply_transaction(&*osr, t);
+ fs->store->apply_transaction(&*osr, std::move(t));
C_SaferCond waiter;
if (!osr->flush_commit(&waiter))
waiter.wait();
t.truncate(cid, oid, size);
ceph::shared_ptr<ObjectStore::Sequencer> osr(
new ObjectStore::Sequencer("fuse"));
- fs->store->apply_transaction(&*osr, t);
+ fs->store->apply_transaction(&*osr, std::move(t));
C_SaferCond waiter;
if (!osr->flush_commit(&waiter))
waiter.wait();
return out << "osr(" << s.get_name() << " " << &s << ")";
}
+ostream& operator<<(ostream& out, const ObjectStore::Transaction& tx) {
+
+ return out << "Transaction(" << &tx << ")";
+}
+
unsigned ObjectStore::apply_transactions(Sequencer *osr,
- list<Transaction*> &tls,
+ vector<Transaction>& tls,
Context *ondisk)
{
// use op pool
int ObjectStore::queue_transactions(
Sequencer *osr,
- list<Transaction*>& tls,
+ vector<Transaction>& tls,
Context *onreadable,
Context *oncommit,
Context *onreadable_sync,
}
static void collect_contexts(
- list<Transaction *> &t,
+ vector<Transaction>& t,
Context **out_on_applied,
Context **out_on_commit,
Context **out_on_applied_sync) {
assert(out_on_commit);
assert(out_on_applied_sync);
list<Context *> on_applied, on_commit, on_applied_sync;
- for (list<Transaction *>::iterator i = t.begin();
+ for (vector<Transaction>::iterator i = t.begin();
i != t.end();
++i) {
- on_applied.splice(on_applied.end(), (*i)->on_applied);
- on_commit.splice(on_commit.end(), (*i)->on_commit);
- on_applied_sync.splice(on_applied_sync.end(), (*i)->on_applied_sync);
+ on_applied.splice(on_applied.end(), (*i).on_applied);
+ on_commit.splice(on_commit.end(), (*i).on_commit);
+ on_applied_sync.splice(on_applied_sync.end(), (*i).on_applied_sync);
}
*out_on_applied = C_Contexts::list_to_context(on_applied);
*out_on_commit = C_Contexts::list_to_context(on_commit);
static void generate_test_instances(list<Transaction*>& o);
};
- struct C_DeleteTransaction : public Context {
- ObjectStore::Transaction *t;
- C_DeleteTransaction(ObjectStore::Transaction *tt) : t(tt) {}
- void finish(int r) {
- delete t;
- }
- };
- template<class T>
- struct C_DeleteTransactionHolder : public Context {
- ObjectStore::Transaction *t;
- T obj;
- C_DeleteTransactionHolder(ObjectStore::Transaction *tt, T &obj) :
- t(tt), obj(obj) {}
- void finish(int r) {
- delete t;
- }
- };
-
// synchronous wrappers
- unsigned apply_transaction(Sequencer *osr, Transaction& t, Context *ondisk=0) {
- list<Transaction*> tls;
- tls.push_back(&t);
+ unsigned apply_transaction(Sequencer *osr, Transaction&& t, Context *ondisk=0) {
+ vector<Transaction> tls;
+ tls.push_back(std::move(t));
return apply_transactions(osr, tls, ondisk);
}
- unsigned apply_transactions(Sequencer *osr, list<Transaction*>& tls, Context *ondisk=0);
-
- int queue_transaction_and_cleanup(Sequencer *osr, Transaction* t,
- ThreadPool::TPHandle *handle = NULL) {
- list<Transaction *> tls;
- tls.push_back(t);
- return queue_transactions(osr, tls, new C_DeleteTransaction(t),
- NULL, NULL, TrackedOpRef(), handle);
- }
+ unsigned apply_transactions(Sequencer *osr, vector<Transaction>& tls, Context *ondisk=0);
- int queue_transaction(Sequencer *osr, Transaction *t, Context *onreadable, Context *ondisk=0,
+ int queue_transaction(Sequencer *osr, Transaction&& t, Context *onreadable, Context *ondisk=0,
Context *onreadable_sync=0,
TrackedOpRef op = TrackedOpRef(),
ThreadPool::TPHandle *handle = NULL) {
- list<Transaction*> tls;
- tls.push_back(t);
+ vector<Transaction> tls;
+ tls.push_back(std::move(t));
return queue_transactions(osr, tls, onreadable, ondisk, onreadable_sync,
op, handle);
}
- int queue_transactions(Sequencer *osr, list<Transaction*>& tls,
+ int queue_transactions(Sequencer *osr, vector<Transaction>& tls,
Context *onreadable, Context *ondisk=0,
Context *onreadable_sync=0,
TrackedOpRef op = TrackedOpRef(),
ThreadPool::TPHandle *handle = NULL) {
assert(!tls.empty());
- tls.back()->register_on_applied(onreadable);
- tls.back()->register_on_commit(ondisk);
- tls.back()->register_on_applied_sync(onreadable_sync);
+ tls.back().register_on_applied(onreadable);
+ tls.back().register_on_commit(ondisk);
+ tls.back().register_on_applied_sync(onreadable_sync);
return queue_transactions(osr, tls, op, handle);
}
virtual int queue_transactions(
- Sequencer *osr, list<Transaction*>& tls,
+ Sequencer *osr, vector<Transaction>& tls,
TrackedOpRef op = TrackedOpRef(),
ThreadPool::TPHandle *handle = NULL) = 0;
int queue_transactions(
Sequencer *osr,
- list<Transaction*>& tls,
+ vector<Transaction>& tls,
Context *onreadable,
Context *oncommit,
Context *onreadable_sync,
int queue_transaction(
Sequencer *osr,
- Transaction* t,
+ Transaction&& t,
Context *onreadable,
Context *oncommit,
Context *onreadable_sync,
Context *oncomplete,
TrackedOpRef op) {
- list<Transaction*> tls;
- tls.push_back(t);
+
+ vector<Transaction> tls;
+ tls.push_back(std::move(t));
return queue_transactions(
osr, tls, onreadable, oncommit, onreadable_sync, oncomplete, op);
}
}
ostream& operator<<(ostream& out, const ObjectStore::Sequencer& s);
+ostream& operator<<(ostream& out, const ObjectStore::Transaction& tx);
#endif
int BlueStore::queue_transactions(
Sequencer *posr,
- list<Transaction*>& tls,
+ vector<Transaction>& tls,
TrackedOpRef op,
ThreadPool::TPHandle *handle)
{
txc->onreadable_sync = onreadable_sync;
txc->oncommit = ondisk;
- for (list<Transaction*>::iterator p = tls.begin(); p != tls.end(); ++p) {
- (*p)->set_osr(osr);
- txc->ops += (*p)->get_num_ops();
- txc->bytes += (*p)->get_num_bytes();
- _txc_add_transaction(txc, *p);
+ for (vector<Transaction>::iterator p = tls.begin(); p != tls.end(); ++p) {
+ (*p).set_osr(osr);
+ txc->ops += (*p).get_num_ops();
+ txc->bytes += (*p).get_num_bytes();
+ _txc_add_transaction(txc, &(*p));
}
r = _txc_finalize(osr, txc);
int queue_transactions(
Sequencer *osr,
- list<Transaction*>& tls,
+ vector<Transaction>& tls,
TrackedOpRef op = TrackedOpRef(),
ThreadPool::TPHandle *handle = NULL);
}
#endif
-int FileJournal::prepare_entry(list<ObjectStore::Transaction*>& tls, bufferlist* tbl) {
+int FileJournal::prepare_entry(vector<ObjectStore::Transaction>& tls, bufferlist* tbl) {
dout(10) << "prepare_entry " << tls << dendl;
unsigned data_len = 0;
int data_align = -1; // -1 indicates that we don't care about the alignment
bufferlist bl;
- for (list<ObjectStore::Transaction*>::iterator p = tls.begin();
+ for (vector<ObjectStore::Transaction>::iterator p = tls.begin();
p != tls.end(); ++p) {
- ObjectStore::Transaction *t = *p;
- if (t->get_data_length() > data_len &&
- (int)t->get_data_length() >= g_conf->journal_align_min_size) {
- data_len = t->get_data_length();
- data_align = (t->get_data_alignment() - bl.length()) & ~CEPH_PAGE_MASK;
+ if ((*p).get_data_length() > data_len &&
+ (int)(*p).get_data_length() >= g_conf->journal_align_min_size) {
+ data_len = (*p).get_data_length();
+ data_align = ((*p).get_data_alignment() - bl.length()) & ~CEPH_PAGE_MASK;
}
- ::encode(*t, bl);
+ ::encode(*p, bl);
}
if (tbl->length()) {
bl.claim_append(*tbl);
completions.pop_front();
}
- int prepare_entry(list<ObjectStore::Transaction*>& tls, bufferlist* tbl);
+ int prepare_entry(vector<ObjectStore::Transaction>& tls, bufferlist* tbl);
void submit_entry(uint64_t seq, bufferlist& bl, uint32_t orig_len,
Context *oncommit,
/// -----------------------------
-FileStore::Op *FileStore::build_op(list<Transaction*>& tls,
+FileStore::Op *FileStore::build_op(vector<Transaction>& tls,
Context *onreadable,
Context *onreadable_sync,
TrackedOpRef osd_op)
{
uint64_t bytes = 0, ops = 0;
- for (list<Transaction*>::iterator p = tls.begin();
+ for (vector<Transaction>::iterator p = tls.begin();
p != tls.end();
++p) {
- bytes += (*p)->get_num_bytes();
- ops += (*p)->get_num_ops();
+ bytes += (*p).get_num_bytes();
+ ops += (*p).get_num_ops();
}
Op *o = new Op;
o->start = ceph_clock_now(g_ceph_context);
- o->tls.swap(tls);
+ o->tls = std::move(tls);
o->onreadable = onreadable;
o->onreadable_sync = onreadable_sync;
o->ops = ops;
apply_manager.op_apply_finish(o->op);
dout(10) << "_do_op " << o << " seq " << o->op << " r = " << r
<< ", finisher " << o->onreadable << " " << o->onreadable_sync << dendl;
+
+ o->tls.clear();
+
}
void FileStore::_finish_op(OpSequencer *osr)
}
};
-int FileStore::queue_transactions(Sequencer *posr, list<Transaction*> &tls,
+int FileStore::queue_transactions(Sequencer *posr, vector<Transaction>& tls,
TrackedOpRef osd_op,
ThreadPool::TPHandle *handle)
{
}
// used to include osr information in tracepoints during transaction apply
- for (list<ObjectStore::Transaction*>::iterator i = tls.begin(); i != tls.end(); ++i) {
- (*i)->set_osr(osr);
+ for (vector<Transaction>::iterator i = tls.begin(); i != tls.end(); ++i) {
+ (*i).set_osr(osr);
}
if (journal && journal->is_writeable() && !m_filestore_journal_trailing) {
}
int FileStore::_do_transactions(
- list<Transaction*> &tls,
+ vector<Transaction> &tls,
uint64_t op_seq,
ThreadPool::TPHandle *handle)
{
int trans_num = 0;
- for (list<Transaction*>::iterator p = tls.begin();
+ for (vector<Transaction>::iterator p = tls.begin();
p != tls.end();
++p, trans_num++) {
- _do_transaction(**p, op_seq, trans_num, handle);
+ _do_transaction(*p, op_seq, trans_num, handle);
if (handle)
handle->reset_tp_timeout();
}
}
}
-void FileStore::dump_transactions(list<ObjectStore::Transaction*>& ls, uint64_t seq, OpSequencer *osr)
+void FileStore::dump_transactions(vector<ObjectStore::Transaction>& ls, uint64_t seq, OpSequencer *osr)
{
m_filestore_dump_fmt.open_array_section("transactions");
unsigned trans_num = 0;
- for (list<ObjectStore::Transaction*>::iterator i = ls.begin(); i != ls.end(); ++i, ++trans_num) {
+ for (vector<ObjectStore::Transaction>::iterator i = ls.begin(); i != ls.end(); ++i, ++trans_num) {
m_filestore_dump_fmt.open_object_section("transaction");
m_filestore_dump_fmt.dump_string("osr", osr->get_name());
m_filestore_dump_fmt.dump_unsigned("seq", seq);
m_filestore_dump_fmt.dump_unsigned("trans_num", trans_num);
- (*i)->dump(&m_filestore_dump_fmt);
+ (*i).dump(&m_filestore_dump_fmt);
m_filestore_dump_fmt.close_section();
}
m_filestore_dump_fmt.close_section();
struct Op {
utime_t start;
uint64_t op;
- list<Transaction*> tls;
+ vector<Transaction> tls;
Context *onreadable, *onreadable_sync;
uint64_t ops, bytes;
TrackedOpRef osd_op;
void _do_op(OpSequencer *o, ThreadPool::TPHandle &handle);
void _finish_op(OpSequencer *o);
- Op *build_op(list<Transaction*>& tls,
+ Op *build_op(vector<Transaction>& tls,
Context *onreadable, Context *onreadable_sync,
TrackedOpRef osd_op);
void queue_op(OpSequencer *osr, Op *o);
int statfs(struct statfs *buf);
int _do_transactions(
- list<Transaction*> &tls, uint64_t op_seq,
+ vector<Transaction> &tls, uint64_t op_seq,
ThreadPool::TPHandle *handle);
- int do_transactions(list<Transaction*> &tls, uint64_t op_seq) {
+ int do_transactions(vector<Transaction> &tls, uint64_t op_seq) {
return _do_transactions(tls, op_seq, 0);
}
void _do_transaction(
Transaction& t, uint64_t op_seq, int trans_num,
ThreadPool::TPHandle *handle);
- int queue_transactions(Sequencer *osr, list<Transaction*>& tls,
+ int queue_transactions(Sequencer *osr, vector<Transaction>& tls,
TrackedOpRef op = TrackedOpRef(),
ThreadPool::TPHandle *handle = NULL);
void dump_start(const std::string& file);
void dump_stop();
- void dump_transactions(list<ObjectStore::Transaction*>& ls, uint64_t seq, OpSequencer *osr);
+ void dump_transactions(vector<Transaction>& ls, uint64_t seq, OpSequencer *osr);
private:
void _inject_failure();
virtual bool should_commit_now() = 0;
- virtual int prepare_entry(list<ObjectStore::Transaction*>& tls, bufferlist* tbl) = 0;
+ virtual int prepare_entry(vector<ObjectStore::Transaction>& tls, bufferlist* tbl) = 0;
// reads/recovery
dout(3) << "journal_replay: applying op seq " << seq << dendl;
bufferlist::iterator p = bl.begin();
- list<Transaction*> tls;
+ vector<ObjectStore::Transaction> tls;
while (!p.end()) {
- Transaction *t = new Transaction(p);
- tls.push_back(t);
+ tls.emplace_back(Transaction(p));
}
apply_manager.op_apply_start(seq);
op_seq = seq;
- while (!tls.empty()) {
- delete tls.front();
- tls.pop_front();
- }
-
dout(3) << "journal_replay: r = " << r << ", op_seq now " << op_seq << dendl;
}
void _op_journal_transactions(bufferlist& tls, uint32_t orig_len, uint64_t op,
Context *onjournal, TrackedOpRef osd_op);
- virtual int do_transactions(list<ObjectStore::Transaction*>& tls, uint64_t op_seq) = 0;
+ virtual int do_transactions(vector<ObjectStore::Transaction>& tls, uint64_t op_seq) = 0;
public:
bool is_committing() {
int KStore::queue_transactions(
Sequencer *posr,
- list<Transaction*>& tls,
+ vector<Transaction>& tls,
TrackedOpRef op,
ThreadPool::TPHandle *handle)
{
txc->onreadable_sync = onreadable_sync;
txc->oncommit = ondisk;
- for (list<Transaction*>::iterator p = tls.begin(); p != tls.end(); ++p) {
- (*p)->set_osr(osr);
- txc->ops += (*p)->get_num_ops();
- txc->bytes += (*p)->get_num_bytes();
- _txc_add_transaction(txc, *p);
+ for (vector<Transaction>::iterator p = tls.begin(); p != tls.end(); ++p) {
+ (*p).set_osr(osr);
+ txc->ops += (*p).get_num_ops();
+ txc->bytes += (*p).get_num_bytes();
+ _txc_add_transaction(txc, &(*p));
}
r = _txc_finalize(osr, txc);
int queue_transactions(
Sequencer *osr,
- list<Transaction*>& tls,
+ vector<Transaction>& tls,
TrackedOpRef op = TrackedOpRef(),
ThreadPool::TPHandle *handle = NULL);
// write operations
int MemStore::queue_transactions(Sequencer *osr,
- list<Transaction*>& tls,
+ vector<Transaction>& tls,
TrackedOpRef op,
ThreadPool::TPHandle *handle)
{
lock = std::unique_lock<std::mutex>((*seq)->mutex);
}
- for (list<Transaction*>::iterator p = tls.begin(); p != tls.end(); ++p) {
+ for (vector<Transaction>::iterator p = tls.begin(); p != tls.end(); ++p) {
// poke the TPHandle heartbeat just to exercise that code path
if (handle)
handle->reset_tp_timeout();
- _do_transaction(**p);
+ _do_transaction(*p);
}
Context *on_apply = NULL, *on_apply_sync = NULL, *on_commit = NULL;
objectstore_perf_stat_t get_cur_stats();
int queue_transactions(
- Sequencer *osr, list<Transaction*>& tls,
+ Sequencer *osr, vector<Transaction>& tls,
TrackedOpRef op = TrackedOpRef(),
ThreadPool::TPHandle *handle = NULL);
};
map<pg_shard_t, vector<PushOp> > pushes;
map<pg_shard_t, vector<PushReplyOp> > push_replies;
- ObjectStore::Transaction *t;
- RecoveryMessages() : t(NULL) {}
- ~RecoveryMessages() { assert(!t); }
+ ObjectStore::Transaction t;
+ RecoveryMessages() {}
+ ~RecoveryMessages(){}
};
void ECBackend::handle_recovery_push(
PushOp &op,
RecoveryMessages *m)
{
- assert(m->t);
bool oneshot = op.before_progress.first && op.after_progress.data_complete;
ghobject_t tobj;
}
if (op.before_progress.first) {
- m->t->remove(coll, tobj);
- m->t->touch(coll, tobj);
+ m->t.remove(coll, tobj);
+ m->t.touch(coll, tobj);
}
if (!op.data_included.empty()) {
uint64_t end = op.data_included.range_end();
assert(op.data.length() == (end - start));
- m->t->write(
+ m->t.write(
coll,
tobj,
start,
if (op.before_progress.first) {
assert(op.attrset.count(string("_")));
- m->t->setattrs(
+ m->t.setattrs(
coll,
tobj,
op.attrset);
dout(10) << __func__ << ": Removing oid "
<< tobj.hobj << " from the temp collection" << dendl;
clear_temp_obj(tobj.hobj);
- m->t->remove(coll, ghobject_t(
+ m->t.remove(coll, ghobject_t(
op.soid, ghobject_t::NO_GEN, get_parent()->whoami_shard().shard));
- m->t->collection_move_rename(
+ m->t.collection_move_rename(
coll, tobj,
coll, ghobject_t(
op.soid, ghobject_t::NO_GEN, get_parent()->whoami_shard().shard));
stats,
op.recovery_info,
recovery_ops[op.soid].obc,
- m->t);
+ &m->t);
} else {
get_parent()->on_local_recover(
op.soid,
object_stat_sum_t(),
op.recovery_info,
ObjectContextRef(),
- m->t);
+ &m->t);
}
}
m->push_replies[get_parent()->primary_shard()].push_back(PushReplyOp());
}
if (!replies.empty()) {
- m.t->register_on_complete(
+ (m.t).register_on_complete(
get_parent()->bless_context(
new SendPushReplies(
get_parent(),
get_parent()->get_epoch(),
replies)));
- m.t->register_on_applied(
- new ObjectStore::C_DeleteTransaction(m.t));
- get_parent()->queue_transaction(m.t);
- m.t = NULL;
- } else {
- assert(!m.t);
- }
+ get_parent()->queue_transaction(std::move(m.t));
+ }
if (m.reads.empty())
return;
case MSG_OSD_PG_PUSH: {
MOSDPGPush *op = static_cast<MOSDPGPush *>(_op->get_req());
RecoveryMessages rm;
- rm.t = new ObjectStore::Transaction;
- assert(rm.t);
for (vector<PushOp>::iterator i = op->pushes.begin();
i != op->pushes.end();
++i) {
assert(!get_parent()->get_log().get_missing().is_missing(op.soid));
if (!get_parent()->pgb_is_primary())
get_parent()->update_stats(op.stats);
- ObjectStore::Transaction *localt = new ObjectStore::Transaction;
- localt->set_use_tbl(op.t.get_use_tbl());
+ ObjectStore::Transaction localt;
+ localt.set_use_tbl(op.t.get_use_tbl());
if (!op.temp_added.empty()) {
add_temp_objs(op.temp_added);
}
++i) {
dout(10) << __func__ << ": removing object " << *i
<< " since we won't get the transaction" << dendl;
- localt->remove(
+ localt.remove(
coll,
ghobject_t(
*i,
op.trim_to,
op.trim_rollback_to,
!(op.t.empty()),
- localt);
+ &localt);
if (!(dynamic_cast<ReplicatedPG *>(get_parent())->is_undersized()) &&
(unsigned)get_parent()->whoami_shard().shard >= ec_impl->get_data_chunk_count())
if (on_local_applied_sync) {
dout(10) << "Queueing onreadable_sync: " << on_local_applied_sync << dendl;
- localt->register_on_applied_sync(on_local_applied_sync);
+ localt.register_on_applied_sync(on_local_applied_sync);
}
- localt->register_on_commit(
+ localt.register_on_commit(
get_parent()->bless_context(
new SubWriteCommitted(
this, msg, op.tid,
op.at_version,
get_parent()->get_info().last_complete)));
- localt->register_on_applied(
+ localt.register_on_applied(
get_parent()->bless_context(
new SubWriteApplied(this, msg, op.tid, op.at_version)));
- localt->register_on_applied(
- new ObjectStore::C_DeleteTransaction(localt));
- list<ObjectStore::Transaction*> tls;
- tls.push_back(localt);
- tls.push_back(new ObjectStore::Transaction);
- tls.back()->swap(op.t);
- tls.back()->register_on_complete(
- new ObjectStore::C_DeleteTransaction(tls.back()));
+ vector<ObjectStore::Transaction> tls;
+ tls.reserve(2);
+ tls.push_back(std::move(localt));
+ tls.push_back(std::move(op.t));
get_parent()->queue_transactions(tls, msg);
}
ObjectStore::Transaction t;
t.create_collection(coll_t::meta(), 0);
t.write(coll_t::meta(), OSD_SUPERBLOCK_POBJECT, 0, bl.length(), bl);
- ret = store->apply_transaction(osr.get(), t);
+ ret = store->apply_transaction(osr.get(), std::move(t));
if (ret) {
derr << "OSD::mkfs: error while writing OSD_SUPERBLOCK_POBJECT: "
<< "apply_transaction returned " << ret << dendl;
dout(5) << "Upgrading superblock adding: " << diff << dendl;
ObjectStore::Transaction t;
write_superblock(t);
- r = store->apply_transaction(service.meta_osr.get(), t);
+ r = store->apply_transaction(service.meta_osr.get(), std::move(t));
if (r < 0)
goto out;
}
dout(10) << "init creating/touching snapmapper object" << dendl;
ObjectStore::Transaction t;
t.touch(coll_t::meta(), OSD::make_snapmapper_oid());
- r = store->apply_transaction(service.meta_osr.get(), t);
+ r = store->apply_transaction(service.meta_osr.get(), std::move(t));
if (r < 0)
goto out;
}
superblock.clean_thru = osdmap->get_epoch();
ObjectStore::Transaction t;
write_superblock(t);
- int r = store->apply_transaction(service.meta_osr.get(), t);
+ int r = store->apply_transaction(service.meta_osr.get(), std::move(t));
if (r) {
derr << "OSD::shutdown: error writing superblock: "
<< cpp_strerror(r) << dendl;
dout(20) << " removing " << *p << " object " << *q << dendl;
t.remove(*p, *q);
}
- store->apply_transaction(service.meta_osr.get(), t);
+ store->apply_transaction(service.meta_osr.get(), std::move(t));
}
}
}
assert(0);
t.remove(tmp, *p);
if (removed > 300) {
- int r = store->apply_transaction(osr.get(), t);
+ int r = store->apply_transaction(osr.get(), std::move(t));
assert(r == 0);
t = ObjectStore::Transaction();
removed = 0;
}
}
t.remove_collection(tmp);
- int r = store->apply_transaction(osr.get(), t);
+ int r = store->apply_transaction(osr.get(), std::move(t));
assert(r == 0);
C_SaferCond waiter;
dout(1) << __func__ << " removing legacy infos object" << dendl;
ObjectStore::Transaction t;
t.remove(coll_t::meta(), OSD::make_infos_oid());
- int r = store->apply_transaction(service.meta_osr.get(), t);
+ int r = store->apply_transaction(service.meta_osr.get(), std::move(t));
if (r != 0) {
derr << __func__ << ": apply_transaction returned "
<< cpp_strerror(r) << dendl;
// don't let the transaction get too big
if (++num >= cct->_conf->osd_target_transaction_size) {
- store->apply_transaction(service.meta_osr.get(), t);
+ store->apply_transaction(service.meta_osr.get(), std::move(t));
t = ObjectStore::Transaction();
num = 0;
}
}
if (!t.empty())
- store->apply_transaction(service.meta_osr.get(), t);
+ store->apply_transaction(service.meta_osr.get(), std::move(t));
}
/*
val.append(valstr);
newattrs[key] = val;
t.omap_setkeys(coll_t(pgid), ghobject_t(obj), newattrs);
- r = store->apply_transaction(service->meta_osr.get(), t);
+ r = store->apply_transaction(service->meta_osr.get(), std::move(t));
if (r < 0)
ss << "error=" << r;
else
keys.insert(key);
t.omap_rmkeys(coll_t(pgid), ghobject_t(obj), keys);
- r = store->apply_transaction(service->meta_osr.get(), t);
+ r = store->apply_transaction(service->meta_osr.get(), std::move(t));
if (r < 0)
ss << "error=" << r;
else
cmd_getval(service->cct, cmdmap, "header", headerstr);
newheader.append(headerstr);
t.omap_setheader(coll_t(pgid), ghobject_t(obj), newheader);
- r = store->apply_transaction(service->meta_osr.get(), t);
+ r = store->apply_transaction(service->meta_osr.get(), std::move(t));
if (r < 0)
ss << "error=" << r;
else
int64_t trunclen;
cmd_getval(service->cct, cmdmap, "len", trunclen);
t.truncate(coll_t(pgid), ghobject_t(obj), trunclen);
- r = store->apply_transaction(service->meta_osr.get(), t);
+ r = store->apply_transaction(service->meta_osr.get(), std::move(t));
if (r < 0)
ss << "error=" << r;
else
t.remove(coll, *i);
if (num >= cct->_conf->osd_target_transaction_size) {
C_SaferCond waiter;
- store->queue_transaction(osr, &t, &waiter);
+ store->queue_transaction(osr, std::move(t), &waiter);
bool cont = dstate->pause_clearing();
handle.suspend_tp_timeout();
waiter.wait();
num = 0;
}
}
-
C_SaferCond waiter;
- store->queue_transaction(osr, &t, &waiter);
+ store->queue_transaction(osr, std::move(t), &waiter);
bool cont = dstate->pause_clearing();
handle.suspend_tp_timeout();
waiter.wait();
if (!item.second->start_deleting())
return;
- ObjectStore::Transaction *t = new ObjectStore::Transaction;
- PGLog::clear_info_log(pg->info.pgid, t);
+ ObjectStore::Transaction t;
+ PGLog::clear_info_log(pg->info.pgid, &t);
if (g_conf->osd_inject_failure_on_pg_removal) {
generic_derr << "osd_inject_failure_on_pg_removal" << dendl;
exit(1);
}
- t->remove_collection(coll);
+ t.remove_collection(coll);
// We need the sequencer to stick around until the op is complete
store->queue_transaction(
pg->osr.get(),
- t,
+ std::move(t),
0, // onapplied
0, // oncommit
0, // onreadable sync
- new ObjectStore::C_DeleteTransactionHolder<PGRef>(
- t, pg), // oncomplete
+ new ContainerContext<PGRef>(pg),
TrackedOpRef());
item.second->finish_deleting();
dout(1) << " bench count " << count
<< " bsize " << prettybyte_t(bsize) << dendl;
- ObjectStore::Transaction *cleanupt = new ObjectStore::Transaction;
+ ObjectStore::Transaction cleanupt;
if (osize && onum) {
bufferlist bl;
snprintf(nm, sizeof(nm), "disk_bw_test_%d", i);
object_t oid(nm);
hobject_t soid(sobject_t(oid, 0));
- ObjectStore::Transaction *t = new ObjectStore::Transaction;
- t->write(coll_t(), ghobject_t(soid), 0, osize, bl);
- store->queue_transaction_and_cleanup(osr.get(), t);
- cleanupt->remove(coll_t(), ghobject_t(soid));
+ ObjectStore::Transaction t;
+ t.write(coll_t(), ghobject_t(soid), 0, osize, bl);
+ store->queue_transaction(osr.get(), std::move(t), NULL);
+ cleanupt.remove(coll_t(), ghobject_t(soid));
}
}
}
object_t oid(nm);
hobject_t soid(sobject_t(oid, 0));
- ObjectStore::Transaction *t = new ObjectStore::Transaction;
- t->write(coll_t::meta(), ghobject_t(soid), offset, bsize, bl);
- store->queue_transaction_and_cleanup(osr.get(), t);
+ ObjectStore::Transaction t;
+ t.write(coll_t::meta(), ghobject_t(soid), offset, bsize, bl);
+ store->queue_transaction(osr.get(), std::move(t), NULL);
if (!onum || !osize)
- cleanupt->remove(coll_t::meta(), ghobject_t(soid));
+ cleanupt.remove(coll_t::meta(), ghobject_t(soid));
}
{
utime_t end = ceph_clock_now(cct);
// clean up
- store->queue_transaction_and_cleanup(osr.get(), cleanupt);
+ store->queue_transaction(osr.get(), std::move(cleanupt), NULL);
{
C_SaferCond waiter;
if (!osr->flush_commit(&waiter)) {
struct C_OnMapApply : public Context {
OSDService *service;
- boost::scoped_ptr<ObjectStore::Transaction> t;
list<OSDMapRef> pinned_maps;
epoch_t e;
C_OnMapApply(OSDService *service,
- ObjectStore::Transaction *t,
const list<OSDMapRef> &pinned_maps,
epoch_t e)
- : service(service), t(t), pinned_maps(pinned_maps), e(e) {}
+ : service(service), pinned_maps(pinned_maps), e(e) {}
void finish(int r) {
service->clear_map_bl_cache_pins(e);
}
skip_maps = true;
}
- ObjectStore::Transaction *_t = new ObjectStore::Transaction;
- ObjectStore::Transaction &t = *_t;
+ ObjectStore::Transaction t;
// store new maps: queue for disk and put in the osdmap cache
epoch_t start = MAX(osdmap->get_epoch() + 1, first);
if (last <= osdmap->get_epoch()) {
dout(10) << " no new maps here, dropping" << dendl;
- delete _t;
m->put();
return;
}
write_superblock(t);
store->queue_transaction(
service.meta_osr.get(),
- _t,
- new C_OnMapApply(&service, _t, pinned_maps, osdmap->get_epoch()),
+ std::move(t),
+ new C_OnMapApply(&service, pinned_maps, osdmap->get_epoch()),
0, 0);
service.publish_superblock(superblock);
!superblock.compat_features.incompat.contains(CEPH_OSD_FEATURE_INCOMPAT_SHARDS)) {
dout(0) << __func__ << " enabling on-disk ERASURE CODES compat feature" << dendl;
superblock.compat_features.incompat.insert(CEPH_OSD_FEATURE_INCOMPAT_SHARDS);
- ObjectStore::Transaction *t = new ObjectStore::Transaction;
- write_superblock(*t);
- int err = store->queue_transaction_and_cleanup(service.meta_osr.get(), t);
+ ObjectStore::Transaction t;
+ write_superblock(t);
+ int err = store->queue_transaction(service.meta_osr.get(), std::move(t), NULL);
assert(err == 0);
}
}
if (!ctx.created_pgs.empty()) {
ctx.on_applied->add(new C_OpenPGs(ctx.created_pgs, store));
}
- ctx.on_applied->add(new ObjectStore::C_DeleteTransaction(ctx.transaction));
int tr = store->queue_transaction(
pg->osr.get(),
- ctx.transaction, ctx.on_applied, ctx.on_safe, NULL,
+ std::move(*ctx.transaction), ctx.on_applied, ctx.on_safe, NULL,
TrackedOpRef(), handle);
+ delete (ctx.transaction);
assert(tr == 0);
ctx.transaction = new ObjectStore::Transaction;
ctx.on_applied = new C_Contexts(cct);
if (!ctx.created_pgs.empty()) {
ctx.on_applied->add(new C_OpenPGs(ctx.created_pgs, store));
}
- ctx.on_applied->add(new ObjectStore::C_DeleteTransaction(ctx.transaction));
int tr = store->queue_transaction(
pg->osr.get(),
- ctx.transaction, ctx.on_applied, ctx.on_safe, NULL, TrackedOpRef(),
+ std::move(*ctx.transaction), ctx.on_applied, ctx.on_safe, NULL, TrackedOpRef(),
handle);
+ delete (ctx.transaction);
assert(tr == 0);
}
}
}
} else {
// primary is instructing us to trim
- ObjectStore::Transaction *t = new ObjectStore::Transaction;
+ ObjectStore::Transaction t;
PG::PGLogEntryHandler handler;
pg->pg_log.trim(&handler, m->trim_to, pg->info);
- handler.apply(pg, t);
+ handler.apply(pg, &t);
pg->dirty_info = true;
- pg->write_if_dirty(*t);
+ pg->write_if_dirty(t);
int tr = store->queue_transaction(
- pg->osr.get(), t,
- new ObjectStore::C_DeleteTransaction(t));
+ pg->osr.get(), std::move(t), NULL);
assert(tr == 0);
}
pg->unlock();
void OSD::_remove_pg(PG *pg)
{
- ObjectStore::Transaction *rmt = new ObjectStore::Transaction;
+ ObjectStore::Transaction rmt ;
// on_removal, which calls remove_watchers_and_notifies, and the erasure from
// the pg_map must be done together without unlocking the pg lock,
// to avoid racing with watcher cleanup in ms_handle_reset
// and handle_notify_timeout
- pg->on_removal(rmt);
+ pg->on_removal(&rmt);
service.cancel_pending_splits_for_parent(pg->info.pgid);
store->queue_transaction(
- pg->osr.get(), rmt,
- new ObjectStore::C_DeleteTransactionHolder<
- SequencerRef>(rmt, pg->osr),
+ pg->osr.get(), std::move(rmt), NULL,
new ContainerContext<
SequencerRef>(pg->osr));
}
if (dirty_info) {
- ObjectStore::Transaction *t = new ObjectStore::Transaction;
- write_if_dirty(*t);
- int tr = osd->store->queue_transaction_and_cleanup(osr.get(), t);
+ ObjectStore::Transaction t;
+ write_if_dirty(t);
+ int tr = osd->store->queue_transaction(osr.get(), std::move(t), NULL);
assert(tr == 0);
}
ceph::shared_ptr<ObjectStore::Sequencer> osr(
new ObjectStore::Sequencer("upgrade"));
- int r = store->apply_transaction(osr.get(), t);
+ int r = store->apply_transaction(osr.get(), std::move(t));
if (r != 0) {
derr << __func__ << ": apply_transaction returned "
<< cpp_strerror(r) << dendl;
const vector<ghobject_t> &rollback_obs,
ThreadPool::TPHandle &handle)
{
- ObjectStore::Transaction *t = NULL;
+ ObjectStore::Transaction t;
eversion_t trimmed_to = last_rollback_info_trimmed_to_applied;
for (vector<ghobject_t>::const_iterator i = rollback_obs.begin();
i != rollback_obs.end();
<< *i << " generation < trimmed_to "
<< trimmed_to
<< "...repaired";
- if (!t)
- t = new ObjectStore::Transaction;
- t->remove(coll, *i);
+ t.remove(coll, *i);
}
}
- if (t) {
+ if (!t.empty()) {
derr << __func__ << ": queueing trans to clean up obsolete rollback objs"
<< dendl;
- osd->store->queue_transaction_and_cleanup(osr.get(), t);
+ osd->store->queue_transaction(osr.get(), std::move(t), NULL);
}
}
<< "...repaired";
}
snap_mapper.add_oid(hoid, oi_snaps, &_t);
- r = osd->store->apply_transaction(osr.get(), t);
+ r = osd->store->apply_transaction(osr.get(), std::move(t));
if (r != 0) {
derr << __func__ << ": apply_transaction got " << cpp_strerror(r)
<< dendl;
reg_next_scrub();
{
- ObjectStore::Transaction *t = new ObjectStore::Transaction;
+ ObjectStore::Transaction t;
dirty_info = true;
- write_if_dirty(*t);
- int tr = osd->store->queue_transaction_and_cleanup(osr.get(), t);
+ write_if_dirty(t);
+ int tr = osd->store->queue_transaction(osr.get(), std::move(t), NULL);
assert(tr == 0);
}
virtual void send_message(int to_osd, Message *m) = 0;
virtual void queue_transaction(
- ObjectStore::Transaction *t,
+ ObjectStore::Transaction&& t,
OpRequestRef op = OpRequestRef()
) = 0;
virtual void queue_transactions(
- list<ObjectStore::Transaction*>& tls,
+ vector<ObjectStore::Transaction>& tls,
OpRequestRef op = OpRequestRef()
) = 0;
virtual epoch_t get_epoch() const = 0;
coll_t coll;
set<hobject_t, hobject_t::BitwiseComparator> temp_added;
set<hobject_t, hobject_t::BitwiseComparator> temp_cleared;
- ObjectStore::Transaction *t;
+ mutable ObjectStore::Transaction t;
uint64_t written;
const coll_t &get_coll_ct(const hobject_t &hoid) {
if (hoid.is_temp()) {
}
public:
RPGTransaction(coll_t coll, bool use_tbl)
- : coll(coll), t(new ObjectStore::Transaction), written(0) {
- t->set_use_tbl(use_tbl);
+ : coll(coll), written(0) {
+ t.set_use_tbl(use_tbl);
}
/// Yields ownership of contained transaction
- ObjectStore::Transaction *get_transaction() {
- ObjectStore::Transaction *_t = t;
- t = 0;
- return _t;
+ ObjectStore::Transaction&& get_transaction() {
+ return std::move(t);
}
const set<hobject_t, hobject_t::BitwiseComparator> &get_temp_added() {
return temp_added;
uint32_t fadvise_flags
) {
written += len;
- t->write(get_coll_ct(hoid), ghobject_t(hoid), off, len, bl, fadvise_flags);
+ t.write(get_coll_ct(hoid), ghobject_t(hoid), off, len, bl, fadvise_flags);
}
void remove(
const hobject_t &hoid
) {
- t->remove(get_coll_rm(hoid), ghobject_t(hoid));
+ t.remove(get_coll_rm(hoid), ghobject_t(hoid));
}
void stash(
const hobject_t &hoid,
version_t former_version) {
- t->collection_move_rename(
+ t.collection_move_rename(
coll, ghobject_t(hoid), coll,
ghobject_t(hoid, former_version, shard_id_t::NO_SHARD));
}
const hobject_t &hoid,
map<string, bufferlist> &attrs
) {
- t->setattrs(get_coll(hoid), ghobject_t(hoid), attrs);
+ t.setattrs(get_coll(hoid), ghobject_t(hoid), attrs);
}
void setattr(
const hobject_t &hoid,
const string &attrname,
bufferlist &bl
) {
- t->setattr(get_coll(hoid), ghobject_t(hoid), attrname, bl);
+ t.setattr(get_coll(hoid), ghobject_t(hoid), attrname, bl);
}
void rmattr(
const hobject_t &hoid,
const string &attrname
) {
- t->rmattr(get_coll(hoid), ghobject_t(hoid), attrname);
+ t.rmattr(get_coll(hoid), ghobject_t(hoid), attrname);
}
void omap_setkeys(
const hobject_t &hoid,
) {
for (map<string, bufferlist>::iterator p = keys.begin(); p != keys.end(); ++p)
written += p->first.length() + p->second.length();
- return t->omap_setkeys(get_coll(hoid), ghobject_t(hoid), keys);
+ return t.omap_setkeys(get_coll(hoid), ghobject_t(hoid), keys);
}
void omap_setkeys(
const hobject_t &hoid,
bufferlist &keys_bl
) {
written += keys_bl.length();
- return t->omap_setkeys(get_coll(hoid), ghobject_t(hoid), keys_bl);
+ return t.omap_setkeys(get_coll(hoid), ghobject_t(hoid), keys_bl);
}
void omap_rmkeys(
const hobject_t &hoid,
set<string> &keys
) {
- t->omap_rmkeys(get_coll(hoid), ghobject_t(hoid), keys);
+ t.omap_rmkeys(get_coll(hoid), ghobject_t(hoid), keys);
}
void omap_rmkeys(
const hobject_t &hoid,
bufferlist &keys_bl
) {
- t->omap_rmkeys(get_coll(hoid), ghobject_t(hoid), keys_bl);
+ t.omap_rmkeys(get_coll(hoid), ghobject_t(hoid), keys_bl);
}
void omap_clear(
const hobject_t &hoid
) {
- t->omap_clear(get_coll(hoid), ghobject_t(hoid));
+ t.omap_clear(get_coll(hoid), ghobject_t(hoid));
}
void omap_setheader(
const hobject_t &hoid,
bufferlist &header
) {
written += header.length();
- t->omap_setheader(get_coll(hoid), ghobject_t(hoid), header);
+ t.omap_setheader(get_coll(hoid), ghobject_t(hoid), header);
}
void clone_range(
const hobject_t &from,
uint64_t tooff
) {
assert(get_coll(from) == get_coll_ct(to) && get_coll(from) == coll);
- t->clone_range(coll, ghobject_t(from), ghobject_t(to), fromoff, len, tooff);
+ t.clone_range(coll, ghobject_t(from), ghobject_t(to), fromoff, len, tooff);
}
void clone(
const hobject_t &from,
const hobject_t &to
) {
assert(get_coll(from) == get_coll_ct(to) && get_coll(from) == coll);
- t->clone(coll, ghobject_t(from), ghobject_t(to));
+ t.clone(coll, ghobject_t(from), ghobject_t(to));
}
void rename(
const hobject_t &from,
const hobject_t &to
) {
- t->collection_move_rename(
+ t.collection_move_rename(
get_coll_rm(from),
ghobject_t(from),
get_coll_ct(to),
void touch(
const hobject_t &hoid
) {
- t->touch(get_coll_ct(hoid), ghobject_t(hoid));
+ t.touch(get_coll_ct(hoid), ghobject_t(hoid));
}
void truncate(
const hobject_t &hoid,
uint64_t off
) {
- t->truncate(get_coll(hoid), ghobject_t(hoid), off);
+ t.truncate(get_coll(hoid), ghobject_t(hoid), off);
}
void zero(
const hobject_t &hoid,
uint64_t off,
uint64_t len
) {
- t->zero(get_coll(hoid), ghobject_t(hoid), off, len);
+ t.zero(get_coll(hoid), ghobject_t(hoid), off, len);
}
void set_alloc_hint(
uint64_t expected_object_size,
uint64_t expected_write_size
) {
- t->set_alloc_hint(get_coll(hoid), ghobject_t(hoid), expected_object_size,
+ t.set_alloc_hint(get_coll(hoid), ghobject_t(hoid), expected_object_size,
expected_write_size);
}
assert(to_append);
written += to_append->written;
to_append->written = 0;
- t->append(*(to_append->t));
+ t.append((to_append->t));
for (set<hobject_t, hobject_t::BitwiseComparator>::iterator i = to_append->temp_added.begin();
i != to_append->temp_added.end();
++i) {
}
}
void nop() {
- t->nop();
+ t.nop();
}
bool empty() const {
- return t->empty();
+ return t.empty();
}
uint64_t get_bytes_written() const {
return written;
}
- ~RPGTransaction() { delete t; }
+ ~RPGTransaction() { }
};
PGBackend::PGTransaction *ReplicatedBackend::get_transaction()
{
RPGTransaction *t = dynamic_cast<RPGTransaction*>(_t);
assert(t);
- ObjectStore::Transaction *op_t = t->get_transaction();
+ ObjectStore::Transaction op_t = t->get_transaction();
assert(t->get_temp_added().size() <= 1);
assert(t->get_temp_cleared().size() <= 1);
log_entries,
hset_history,
&op,
- op_t);
+ &op_t);
if (!(t->get_temp_added().empty())) {
add_temp_objs(t->get_temp_added());
trim_to,
trim_rollback_to,
true,
- op_t);
+ &op_t);
- op_t->register_on_applied_sync(on_local_applied_sync);
- op_t->register_on_applied(
+ op_t.register_on_applied_sync(on_local_applied_sync);
+ op_t.register_on_applied(
parent->bless_context(
new C_OSD_OnOpApplied(this, &op)));
- op_t->register_on_applied(
- new ObjectStore::C_DeleteTransaction(op_t));
- op_t->register_on_commit(
+ op_t.register_on_commit(
parent->bless_context(
new C_OSD_OnOpCommit(this, &op)));
-
- list<ObjectStore::Transaction*> tls;
- tls.push_back(op_t);
+ vector<ObjectStore::Transaction> tls;
+ tls.push_back(std::move(op_t));
parent->queue_transactions(tls, op.op);
delete t;
}
pg_shard_t from = m->from;
vector<PushReplyOp> replies;
- ObjectStore::Transaction *t = new ObjectStore::Transaction;
+ ObjectStore::Transaction t;
for (vector<PushOp>::iterator i = m->pushes.begin();
i != m->pushes.end();
++i) {
replies.push_back(PushReplyOp());
- handle_push(from, *i, &(replies.back()), t);
+ handle_push(from, *i, &(replies.back()), &t);
}
MOSDPGPushReply *reply = new MOSDPGPushReply;
reply->replies.swap(replies);
reply->compute_cost(cct);
- t->register_on_complete(
+ t.register_on_complete(
new PG_SendMessageOnConn(
get_parent(), reply, m->get_connection()));
- t->register_on_applied(
- new ObjectStore::C_DeleteTransaction(t));
- get_parent()->queue_transaction(t);
+ get_parent()->queue_transaction(std::move(t));
}
struct C_ReplicatedBackend_OnPullComplete : GenContext<ThreadPool::TPHandle&> {
pg_shard_t from = m->from;
vector<PullOp> replies(1);
- ObjectStore::Transaction *t = new ObjectStore::Transaction;
+ ObjectStore::Transaction t;
list<hobject_t> to_continue;
for (vector<PushOp>::iterator i = m->pushes.begin();
i != m->pushes.end();
++i) {
- bool more = handle_pull_response(from, *i, &(replies.back()), &to_continue, t);
+ bool more = handle_pull_response(from, *i, &(replies.back()), &to_continue, &t);
if (more)
replies.push_back(PullOp());
}
this,
m->get_priority());
c->to_continue.swap(to_continue);
- t->register_on_complete(
+ t.register_on_complete(
new PG_RecoveryQueueAsync(
get_parent(),
get_parent()->bless_gencontext(c)));
reply->pulls.swap(replies);
reply->compute_cost(cct);
- t->register_on_complete(
+ t.register_on_complete(
new PG_SendMessageOnConn(
get_parent(), reply, m->get_connection()));
}
- t->register_on_applied(
- new ObjectStore::C_DeleteTransaction(t));
- get_parent()->queue_transaction(t);
+ get_parent()->queue_transaction(std::move(t));
}
void ReplicatedBackend::do_pull(OpRequestRef op)
rm->localt.register_on_applied(
parent->bless_context(
new C_OSD_RepModifyApply(this, rm)));
- list<ObjectStore::Transaction*> tls;
- tls.push_back(&(rm->localt));
- tls.push_back(&(rm->opt));
+ vector<ObjectStore::Transaction> tls;
+ tls.reserve(2);
+ tls.push_back(std::move(rm->localt));
+ tls.push_back(std::move(rm->opt));
parent->queue_transactions(tls, op);
// op is cleaned up by oncommit/onapply when both are executed
}
pop.recovery_info = m->recovery_info;
pop.before_progress = m->current_progress;
pop.after_progress = m->recovery_progress;
- ObjectStore::Transaction *t = new ObjectStore::Transaction;
+ ObjectStore::Transaction t;
if (is_primary()) {
PullOp resp;
list<hobject_t> to_continue;
bool more = handle_pull_response(
m->from, pop, &resp,
- &to_continue, t);
+ &to_continue, &t);
if (more) {
send_pull_legacy(
m->get_priority(),
this,
op->get_req()->get_priority());
c->to_continue.swap(to_continue);
- t->register_on_complete(
+ t.register_on_complete(
new PG_RecoveryQueueAsync(
get_parent(),
get_parent()->bless_gencontext(c)));
get_osdmap()->get_epoch(), CEPH_OSD_FLAG_ACK);
reply->set_priority(m->get_priority());
assert(entity_name_t::TYPE_OSD == m->get_connection()->peer_type);
- handle_push(m->from, pop, &resp, t);
- t->register_on_complete(new PG_SendMessageOnConn(
+ handle_push(m->from, pop, &resp, &t);
+ t.register_on_complete(new PG_SendMessageOnConn(
get_parent(), reply, m->get_connection()));
}
- t->register_on_applied(
- new ObjectStore::C_DeleteTransaction(t));
- get_parent()->queue_transaction(t);
+ get_parent()->queue_transaction(std::move(t));
return;
}
info.stats = m->stats;
}
- ObjectStore::Transaction *t = new ObjectStore::Transaction;
+ ObjectStore::Transaction t;
dirty_info = true;
- write_if_dirty(*t);
- int tr = osd->store->queue_transaction_and_cleanup(osr.get(), t);
+ write_if_dirty(t);
+ int tr = osd->store->queue_transaction(osr.get(), std::move(t), NULL);
assert(tr == 0);
}
break;
op->mark_started();
- ObjectStore::Transaction *t = new ObjectStore::Transaction;
- remove_snap_mapped_object(*t, m->poid);
- int r = osd->store->queue_transaction_and_cleanup(osr.get(), t);
+ ObjectStore::Transaction t;
+ remove_snap_mapped_object(t, m->poid);
+ int r = osd->store->queue_transaction(osr.get(), std::move(t), NULL);
assert(r == 0);
}
pg_log.get_log().print(*_dout);
*_dout << dendl;
- ObjectStore::Transaction *t = new ObjectStore::Transaction;
+ ObjectStore::Transaction t;
C_PG_MarkUnfoundLost *c = new C_PG_MarkUnfoundLost(this);
utime_t mtime = ceph_clock_now(cct);
switch (what) {
case pg_log_entry_t::LOST_MARK:
- obc = mark_object_lost(t, oid, m->second.need, mtime, pg_log_entry_t::LOST_MARK);
+ obc = mark_object_lost(&t, oid, m->second.need, mtime, pg_log_entry_t::LOST_MARK);
pg_log.missing_got(m++);
assert(0 == "actually, not implemented yet!");
// we need to be careful about how this is handled on the replica!
pg_log.add(e);
dout(10) << e << dendl;
- t->remove(
+ t.remove(
coll,
ghobject_t(oid, ghobject_t::NO_GEN, pg_whoami.shard));
pg_log.missing_add_event(e);
}
dirty_info = true;
- write_if_dirty(*t);
-
- t->register_on_complete(new ObjectStore::C_DeleteTransaction(t));
+ write_if_dirty(t);
- osd->store->queue_transaction(osr.get(), t, c, NULL, new C_OSD_OndiskWriteUnlockList(&c->obcs));
+
+ osd->store->queue_transaction(osr.get(), std::move(t), c, NULL,
+ new C_OSD_OndiskWriteUnlockList(&c->obcs));
// Send out the PG log to all replicas
// So that they know what is lost
obc->ondisk_write_lock();
obc->obs.oi.version = latest->version;
- ObjectStore::Transaction *t = new ObjectStore::Transaction;
- t->register_on_applied(new ObjectStore::C_DeleteTransaction(t));
+ ObjectStore::Transaction t;
bufferlist b2;
obc->obs.oi.encode(b2);
assert(!pool.info.require_rollback());
- t->setattr(coll, ghobject_t(soid), OI_ATTR, b2);
+ t.setattr(coll, ghobject_t(soid), OI_ATTR, b2);
recover_got(soid, latest->version);
missing_loc.add_location(soid, pg_whoami);
++active_pushes;
- osd->store->queue_transaction(osr.get(), t,
+ osd->store->queue_transaction(osr.get(), std::move(t),
new C_OSD_AppliedRecoveredObject(this, obc),
new C_OSD_CommittedPushedObject(
this,
dout(10) << "purged_snaps now " << pg->info.purged_snaps << ", snap_trimq now "
<< pg->snap_trimq << dendl;
- ObjectStore::Transaction *t = new ObjectStore::Transaction;
+ ObjectStore::Transaction t;
pg->dirty_big_info = true;
- pg->write_if_dirty(*t);
- int tr = pg->osd->store->queue_transaction_and_cleanup(pg->osr.get(), t);
+ pg->write_if_dirty(t);
+ int tr = pg->osd->store->queue_transaction(pg->osr.get(), std::move(t), NULL);
assert(tr == 0);
context<SnapTrimmer>().need_share_pg_info = true;
void send_message(int to_osd, Message *m) {
osd->send_message_osd_cluster(to_osd, m, get_osdmap()->get_epoch());
}
- void queue_transaction(ObjectStore::Transaction *t, OpRequestRef op) {
- osd->store->queue_transaction(osr.get(), t, 0, 0, 0, op);
+ void queue_transaction(ObjectStore::Transaction&& t, OpRequestRef op) {
+ osd->store->queue_transaction(osr.get(), std::move(t), 0, 0, 0, op);
}
- void queue_transactions(list<ObjectStore::Transaction*>& tls, OpRequestRef op) {
- osd->store->queue_transactions(osr.get(), tls, 0, 0, 0, op);
+ void queue_transactions(vector<ObjectStore::Transaction>& tls, OpRequestRef op) {
+ osd->store->queue_transactions(osr.get(), tls, 0, 0, 0, op, NULL);
}
epoch_t get_epoch() const {
return get_osdmap()->get_epoch();
std::cout << "collection " << pgid << std::endl;
ObjectStore::Transaction t;
t.create_collection(coll_t(pgid), 0);
- fs.apply_transaction(&osr, t);
+ fs.apply_transaction(&osr, std::move(t));
}
{
ObjectStore::Transaction t;
t.create_collection(coll_t(), 0);
- fs.apply_transaction(&osr, t);
+ fs.apply_transaction(&osr, std::move(t));
}
vector<ceph::shared_ptr<Bencher> > benchers(
#include "global/global_init.h"
#include "os/ObjectStore.h"
-struct C_DeleteTransWrapper : public Context {
- Context *c;
- ObjectStore::Transaction *t;
- C_DeleteTransWrapper(
- ObjectStore::Transaction *t,
- Context *c) : c(c), t(t) {}
- void finish(int r) {
- c->complete(r);
- delete t;
- }
-};
TestFileStoreBackend::TestFileStoreBackend(
ObjectStore *os, bool write_infos)
os->queue_transaction(
osr,
- t,
- new C_DeleteTransWrapper(t, on_applied),
+ std::move(*t),
+ on_applied,
on_commit);
+ delete t;
}
void TestFileStoreBackend::read(
dout(0) << "Give collection: " << cid << " a hint, pg_num is: " << pg_num << ", num_objs is: "
<< num_objs << dendl;
- m_store->apply_transaction(&m_osr, t);
+ m_store->apply_transaction(&m_osr, std::move(t));
}
void DeterministicOpSequence::_do_touch(coll_t coll, hobject_t& obj)
ObjectStore::Transaction t;
note_txn(&t);
t.touch(coll, ghobject_t(obj));
- m_store->apply_transaction(&m_osr, t);
+ m_store->apply_transaction(&m_osr, std::move(t));
}
void DeterministicOpSequence::_do_remove(coll_t coll, hobject_t& obj)
ObjectStore::Transaction t;
note_txn(&t);
t.remove(coll, ghobject_t(obj));
- m_store->apply_transaction(&m_osr, t);
+ m_store->apply_transaction(&m_osr, std::move(t));
}
void DeterministicOpSequence::_do_set_attrs(coll_t coll,
ObjectStore::Transaction t;
note_txn(&t);
t.omap_setkeys(coll, ghobject_t(obj), attrs);
- m_store->apply_transaction(&m_osr, t);
+ m_store->apply_transaction(&m_osr, std::move(t));
}
void DeterministicOpSequence::_do_write(coll_t coll, hobject_t& obj,
ObjectStore::Transaction t;
note_txn(&t);
t.write(coll, ghobject_t(obj), off, len, data);
- m_store->apply_transaction(&m_osr, t);
+ m_store->apply_transaction(&m_osr, std::move(t));
}
void DeterministicOpSequence::_do_clone(coll_t coll, hobject_t& orig_obj,
ObjectStore::Transaction t;
note_txn(&t);
t.clone(coll, ghobject_t(orig_obj), ghobject_t(new_obj));
- m_store->apply_transaction(&m_osr, t);
+ m_store->apply_transaction(&m_osr, std::move(t));
}
void DeterministicOpSequence::_do_clone_range(coll_t coll,
note_txn(&t);
t.clone_range(coll, ghobject_t(orig_obj), ghobject_t(new_obj),
srcoff, srclen, dstoff);
- m_store->apply_transaction(&m_osr, t);
+ m_store->apply_transaction(&m_osr, std::move(t));
}
void DeterministicOpSequence::_do_write_and_clone_range(coll_t coll,
t.write(coll, ghobject_t(orig_obj), srcoff, bl.length(), bl);
t.clone_range(coll, ghobject_t(orig_obj), ghobject_t(new_obj),
srcoff, srclen, dstoff);
- m_store->apply_transaction(&m_osr, t);
+ m_store->apply_transaction(&m_osr, std::move(t));
}
void DeterministicOpSequence::_do_coll_move(coll_t orig_coll, coll_t new_coll,
note_txn(&t);
t.remove(new_coll, ghobject_t(obj));
t.collection_move_rename(orig_coll, ghobject_t(obj), new_coll, ghobject_t(obj));
- m_store->apply_transaction(&m_osr, t);
+ m_store->apply_transaction(&m_osr, std::move(t));
}
class OnApplied : public Context {
FileStoreTracker *tracker;
list<pair<pair<coll_t, string>, uint64_t> > in_flight;
- ObjectStore::Transaction *t;
public:
OnApplied(FileStoreTracker *tracker,
- list<pair<pair<coll_t, string>, uint64_t> > in_flight,
- ObjectStore::Transaction *t)
- : tracker(tracker), in_flight(in_flight), t(t) {}
+ list<pair<pair<coll_t, string>, uint64_t> > in_flight)
+ : tracker(tracker), in_flight(in_flight) {}
void finish(int r) {
for (list<pair<pair<coll_t, string>, uint64_t> >::iterator i =
++i) {
tracker->applied(i->first, i->second);
}
- delete t;
}
};
(**i)(this, &out);
}
store->queue_transaction(
- 0, out.t,
- new OnApplied(this, in_flight, out.t),
+ 0, std::move(*out.t),
+ new OnApplied(this, in_flight),
new OnCommitted(this, in_flight));
+ delete out.t;
}
void FileStoreTracker::write(const pair<coll_t, string> &obj,
dout(5) << "init " << colls << " colls " << objs << " objs" << dendl;
ObjectStore::Sequencer osr(__func__);
- ObjectStore::Transaction *t;
- t = new ObjectStore::Transaction;
+ ObjectStore::Transaction t;
- t->create_collection(coll_t::meta(), 0);
- m_store->apply_transaction(&osr, *t);
+ t.create_collection(coll_t::meta(), 0);
+ m_store->apply_transaction(&osr, std::move(t));
wait_for_ready();
dout(5) << "init create collection " << entry->m_coll.to_str()
<< " meta " << entry->m_meta_obj << dendl;
- t = new ObjectStore::Transaction;
+ ObjectStore::Transaction *t = new ObjectStore::Transaction;
t->create_collection(entry->m_coll, 32);
bufferlist hint;
uint32_t pg_num = colls;
}
baseid += objs;
- m_store->queue_transaction(&(entry->m_osr), t,
- new C_OnFinished(this, t));
+ m_store->queue_transaction(&(entry->m_osr), std::move(*t),
+ new C_OnFinished(this));
+ delete t;
inc_in_flight();
m_collections.insert(make_pair(coll_id, entry));
class C_OnFinished: public Context {
protected:
TestObjectStoreState *m_state;
- ObjectStore::Transaction *m_tx;
public:
- C_OnFinished(TestObjectStoreState *state,
- ObjectStore::Transaction *t) : m_state(state), m_tx(t) { }
+ C_OnFinished(TestObjectStoreState *state) : m_state(state) { }
void finish(int r) {
Mutex::Locker locker(m_state->m_finished_lock);
m_state->dec_in_flight();
m_state->m_finished_cond.Signal();
- delete m_tx;
}
};
};
ObjectStore::Transaction t;
t.create_collection(cid, 0);
t.write(cid, hoid, 0, bl.length(), bl);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
store->umount();
{
ObjectStore::Transaction t;
t.write(cid, hoid2, 0, bl.length(), bl);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
t.remove(cid, hoid2);
t.remove_collection(cid);
cerr << "remove collection" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
store->umount();
{
ObjectStore::Transaction t;
t.create_collection(cid, 0);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
bool exists = store->exists(cid, hoid);
ASSERT_TRUE(!exists);
ObjectStore::Transaction t;
t.remove_collection(cid);
cerr << "remove collection" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
ghobject_t hoid(hobject_t(sobject_t("Object " + stringify(n), CEPH_NOSNAP)));
t.write(cid, hoid, 0, bl.length(), bl);
}
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
// overwrites
ObjectStore::Transaction t;
ghobject_t hoid(hobject_t(sobject_t("Object " + stringify(n), CEPH_NOSNAP)));
t.write(cid, hoid, 1, bl.length(), bl);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
t.remove(cid, hoid);
}
t.remove_collection(cid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
t.create_collection(cid, 0);
t.touch(cid, oid);
t.truncate(cid, oid, 100000);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
t.remove(cid, oid);
t.remove_collection(cid);
cerr << "remove collection" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
t.write(cid, oid, 0, 3, bl);
t.write(cid, oid, 1048576, 3, bl);
t.write(cid, oid, 4194304, 3, bl);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
t.remove(cid, oid);
t.remove_collection(cid);
cerr << "remove collection" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
ObjectStore::Transaction t;
t.create_collection(cid, 0);
cerr << "create collection" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.remove_collection(cid);
cerr << "remove collection" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.create_collection(cid, 0);
cerr << "add collection" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.remove_collection(cid);
cerr << "remove collection" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
ObjectStore::Transaction t;
t.create_collection(cid, 4);
cerr << "create collection" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.remove_collection(cid);
cerr << "remove collection" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.create_collection(cid, 4);
cerr << "add collection" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.remove_collection(cid);
cerr << "remove collection" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
::encode(expected_num_objs, hint);
t.collection_hint(cid, ObjectStore::Transaction::COLL_HINT_EXPECTED_NUM_OBJECTS, hint);
cerr << "collection hint" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.remove_collection(cid);
cerr << "remove collection" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
// Revert the config change so that it does not affect the split/merge tests
ObjectStore::Transaction t;
t.create_collection(cid, 0);
cerr << "Creating collection " << cid << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.touch(cid, hoid);
cerr << "Creating object " << hoid << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
exists = store->exists(cid, hoid);
t.remove(cid, hoid);
t.touch(cid, hoid);
cerr << "Remove then create" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
t.remove(cid, hoid);
t.write(cid, hoid, 0, 5, bl);
cerr << "Remove then create" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
bufferlist in;
exp.append(bl);
t.write(cid, hoid, 5, 5, bl);
cerr << "Append" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
bufferlist in;
exp = bl;
t.write(cid, hoid, 0, 10, bl);
cerr << "Full overwrite" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
bufferlist in;
bl.append("abcde");
t.write(cid, hoid, 3, 5, bl);
cerr << "Partial overwrite" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
bufferlist in, exp;
bl.append("abcde01234012340123401234abcde01234012340123401234abcde01234012340123401234abcde01234012340123401234");
t.write(cid, hoid, 0, bl.length(), bl);
cerr << "larger overwrite" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
bufferlist in;
t.remove(cid, hoid);
t.remove_collection(cid);
cerr << "Cleaning" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
ObjectStore::Transaction t;
t.create_collection(cid, 0);
cerr << "Creating collection " << cid << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
bufferlist bl;
for (int i=0; i<100; ++i) {
ObjectStore::Transaction t;
t.write(cid, a, i*4096, 4096, bl, 0);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
for (int i=0; i<100; ++i) {
ObjectStore::Transaction t;
t.write(cid, b, (rand() % 1024)*4096, 4096, bl, 0);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
t.remove(cid, b);
t.remove_collection(cid);
cerr << "Cleaning" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
ObjectStore::Transaction t;
t.create_collection(cid, 0);
cerr << "Creating collection " << cid << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.touch(cid, a);
t.truncate(cid, a, 3000);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
bl.append(bp);
ObjectStore::Transaction t;
t.write(cid, a, 4096, 4096, bl);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
t.remove(cid, a);
t.remove_collection(cid);
cerr << "Cleaning" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
ObjectStore::Transaction t;
t.create_collection(cid, 0);
cerr << "Creating collection " << cid << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
bufferlist bl;
for (int i=0; i<1000; ++i) {
ObjectStore::Transaction t;
t.write(cid, a, i*len, len, bl, 0);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
t.remove(cid, a);
t.remove_collection(cid);
cerr << "Cleaning" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
ObjectStore::Transaction t;
t.create_collection(cid, 0);
cerr << "Creating collection " << cid << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
bufferlist bl;
for (int i=0; i<10; ++i) {
ObjectStore::Transaction t;
t.write(cid, a, i*4*1048586, 4*1048576, bl, 0);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
// aligned
for (int i=0; i<10; ++i) {
ObjectStore::Transaction t;
t.write(cid, b, (rand() % 256)*4*1048576, 4*1048576, bl, 0);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
// unaligned
for (int i=0; i<10; ++i) {
ObjectStore::Transaction t;
t.write(cid, b, (rand() % (256*4096))*1024, 4*1048576, bl, 0);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
// do some zeros
for (int i=0; i<10; ++i) {
ObjectStore::Transaction t;
t.zero(cid, b, (rand() % (256*4096))*1024, 16*1048576);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
t.remove(cid, b);
t.remove_collection(cid);
cerr << "Cleaning" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
ObjectStore::Transaction t;
t.create_collection(cid, 0);
cerr << "Creating collection " << cid << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
bufferlist bl;
{
ObjectStore::Transaction t;
t.write(cid, a, 0, 524288, bl, 0);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.write(cid, a, 1048576, 524288, bl, 0);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
{
ObjectStore::Transaction t;
t.write(cid, a, 1048576 - 4096, 524288, bl, 0);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
t.remove(cid, a);
t.remove_collection(cid);
cerr << "Cleaning" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.create_collection(cid, 0);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
t.touch(cid, hoid);
t.setattr(cid, hoid, "foo", val);
t.setattr(cid, hoid, "bar", val2);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.remove(cid, hoid);
t.remove_collection(cid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
ObjectStore::Transaction t;
t.create_collection(cid, 0);
cerr << "Creating collection " << cid << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
set<ghobject_t, ghobject_t::BitwiseComparator> all;
t.touch(cid, hoid);
cerr << "Creating object " << hoid << std::endl;
}
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
for (int bitwise=0; bitwise<2; ++bitwise) {
t.remove(cid, *p);
t.remove_collection(cid);
cerr << "Cleaning" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
ObjectStore::Transaction t;
t.create_collection(cid, 0);
cerr << "Creating collection " << cid << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
set<ghobject_t, ghobject_t::BitwiseComparator> all, saw;
t.touch(cid, hoid);
cerr << "Creating object " << hoid << std::endl;
}
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
t.remove(cid, *p);
t.remove_collection(cid);
cerr << "Cleaning" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
ObjectStore::Transaction t;
t.create_collection(cid, 0);
cerr << "Creating collection " << cid << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
ghobject_t hoid(hobject_t(sobject_t("Object 1", CEPH_NOSNAP),
t.write(cid, hoid, 0, small.length(), small);
t.write(cid, hoid, 10, small.length(), small);
cerr << "Creating object and set attr " << hoid << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
ghobject_t hoid2(hobject_t(sobject_t("Object 2", CEPH_NOSNAP),
t.setattr(cid, hoid, "attr1", large);
t.setattr(cid, hoid, "attr2", small);
cerr << "Clone object and rm attr" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
r = store->read(cid, hoid, 10, 5, newdata);
ObjectStore::Transaction t;
t.remove(cid, hoid);
t.remove(cid, hoid2);
- ASSERT_EQ(0u, store->apply_transaction(&osr, t));
+ ASSERT_EQ(0u, store->apply_transaction(&osr, std::move(t)));
}
{
bufferlist final;
al.append(a);
final.append(a);
t.write(cid, hoid, pl.length(), a.length(), al);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
bufferlist rl;
ASSERT_EQ((int)final.length(),
ObjectStore::Transaction t;
t.remove(cid, hoid);
t.remove(cid, hoid2);
- ASSERT_EQ(0u, store->apply_transaction(&osr, t));
+ ASSERT_EQ(0u, store->apply_transaction(&osr, std::move(t)));
}
{
bufferlist final;
al.append(a);
final.append(a);
t.write(cid, hoid, pl.length() + z.length(), a.length(), al);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
bufferlist rl;
ASSERT_EQ((int)final.length(),
ObjectStore::Transaction t;
t.remove(cid, hoid);
t.remove(cid, hoid2);
- ASSERT_EQ(0u, store->apply_transaction(&osr, t));
+ ASSERT_EQ(0u, store->apply_transaction(&osr, std::move(t)));
}
{
bufferlist final;
al.append(a);
final.append(a);
t.write(cid, hoid, 17000, a.length(), al);
- ASSERT_EQ(0u, store->apply_transaction(&osr, t));
+ ASSERT_EQ(0u, store->apply_transaction(&osr, std::move(t)));
bufferlist rl;
ASSERT_EQ((int)final.length(),
store->read(cid, hoid, 0, final.length(), rl));
ObjectStore::Transaction t;
t.remove(cid, hoid);
t.remove(cid, hoid2);
- ASSERT_EQ(0u, store->apply_transaction(&osr, t));
+ ASSERT_EQ(0u, store->apply_transaction(&osr, std::move(t)));
}
{
bufferptr p(1048576);
bufferlist al;
al.append(a);
t.write(cid, hoid, a.length(), a.length(), al);
- ASSERT_EQ(0u, store->apply_transaction(&osr, t));
+ ASSERT_EQ(0u, store->apply_transaction(&osr, std::move(t)));
bufferlist rl;
bufferlist final;
final.substr_of(pl, 0, al.length());
ObjectStore::Transaction t;
t.remove(cid, hoid);
t.remove(cid, hoid2);
- ASSERT_EQ(0u, store->apply_transaction(&osr, t));
+ ASSERT_EQ(0u, store->apply_transaction(&osr, std::move(t)));
}
{
bufferptr p(65536);
bufferlist al;
al.append(a);
t.write(cid, hoid, 32768, a.length(), al);
- ASSERT_EQ(0u, store->apply_transaction(&osr, t));
+ ASSERT_EQ(0u, store->apply_transaction(&osr, std::move(t)));
bufferlist rl;
bufferlist final;
final.substr_of(pl, 0, 32768);
ObjectStore::Transaction t;
t.remove(cid, hoid);
t.remove(cid, hoid2);
- ASSERT_EQ(0u, store->apply_transaction(&osr, t));
+ ASSERT_EQ(0u, store->apply_transaction(&osr, std::move(t)));
}
{
bufferptr p(65536);
bufferlist al;
al.append(a);
t.write(cid, hoid, 33768, a.length(), al);
- ASSERT_EQ(0u, store->apply_transaction(&osr, t));
+ ASSERT_EQ(0u, store->apply_transaction(&osr, std::move(t)));
bufferlist rl;
bufferlist final;
final.substr_of(pl, 0, 33768);
t.remove(cid, hoid2);
t.remove_collection(cid);
cerr << "Cleaning" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
ObjectStore::Transaction t;
t.create_collection(cid, 0);
cerr << "Creating collection " << cid << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
ghobject_t hoid(hobject_t(sobject_t("omap_obj", CEPH_NOSNAP),
t.omap_setkeys(cid, hoid, km);
t.omap_setheader(cid, hoid, header);
cerr << "Creating object and set omap " << hoid << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
// get header, keys
t.remove(cid, hoid);
t.remove_collection(cid);
cerr << "Cleaning" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
ObjectStore::Transaction t;
t.create_collection(cid, 0);
cerr << "Creating collection " << cid << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
ghobject_t hoid(hobject_t(sobject_t("Object 1", CEPH_NOSNAP),
t.omap_setkeys(cid, hoid, km);
t.omap_setheader(cid, hoid, header);
cerr << "Creating object and set omap " << hoid << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
ghobject_t hoid2(hobject_t(sobject_t("Object 2", CEPH_NOSNAP),
ObjectStore::Transaction t;
t.clone(cid, hoid, hoid2);
cerr << "Clone object" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
t.remove(cid, hoid2);
t.remove_collection(cid);
cerr << "Cleaning" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
ObjectStore::Transaction t;
t.create_collection(cid, 0);
cerr << "Creating collection " << cid << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
ghobject_t hoid(hobject_t(sobject_t("Object 1", CEPH_NOSNAP)));
ObjectStore::Transaction t;
t.write(cid, hoid, 10, 5, small);
cerr << "Creating object and write bl " << hoid << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
ghobject_t hoid2(hobject_t(sobject_t("Object 2", CEPH_NOSNAP)));
ObjectStore::Transaction t;
t.clone_range(cid, hoid, hoid2, 10, 5, 0);
cerr << "Clone range object" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
r = store->read(cid, hoid2, 0, 5, newdata);
ASSERT_EQ(r, 5);
t.truncate(cid, hoid, 1024*1024);
t.clone_range(cid, hoid, hoid2, 0, 1024*1024, 0);
cerr << "Clone range object" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
struct stat stat, stat2;
r = store->stat(cid, hoid, &stat);
t.remove(cid, hoid2);
t.remove_collection(cid);
cerr << "Cleaning" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
ObjectStore::Transaction t;
t.create_collection(cid, 0);
cerr << "Creating collection " << cid << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
ghobject_t hoid(hobject_t(sobject_t("aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaObjectaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa 1", CEPH_NOSNAP)));
ObjectStore::Transaction t;
t.touch(cid, hoid);
cerr << "Creating object " << hoid << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
t.remove(cid, hoid);
t.remove_collection(cid);
cerr << "Cleaning" << std::endl;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
{
ObjectStore::Transaction t;
t.create_collection(cid, 0);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
for (int i = 0; i < NUM_OBJS; ++i) {
ghobject_t hoid(hobject_t(sobject_t(string(buf) + base, CEPH_NOSNAP)));
t.touch(cid, hoid);
created.insert(hoid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
++i) {
ObjectStore::Transaction t;
t.remove(cid, *i);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
cerr << "cleaning up" << std::endl;
{
ObjectStore::Transaction t;
t.remove_collection(cid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
int init() {
ObjectStore::Transaction t;
t.create_collection(cid, 0);
- return store->apply_transaction(osr, t);
+ return store->apply_transaction(osr, std::move(t));
}
void shutdown() {
while (1) {
p != objects.end(); ++p) {
t.remove(cid, *p);
}
- store->apply_transaction(osr, t);
+ store->apply_transaction(osr, std::move(t));
}
ObjectStore::Transaction t;
t.remove_collection(cid);
- store->apply_transaction(osr, t);
+ store->apply_transaction(osr, std::move(t));
}
ghobject_t get_uniform_random_object() {
in_flight_objects.insert(new_obj);
if (!contents.count(new_obj))
contents[new_obj] = Object();
- return store->queue_transaction(osr, t, new C_SyntheticOnReadable(this, t, new_obj));
+ int status = store->queue_transaction(osr, std::move(*t), new C_SyntheticOnReadable(this, t, new_obj));
+ delete t;
+ return status;
}
int clone() {
contents[new_obj].data.clear();
contents[new_obj].data.append(contents[old_obj].data.c_str(),
contents[old_obj].data.length());
- return store->queue_transaction(osr, t, new C_SyntheticOnClone(this, t, old_obj, new_obj));
+ int status = store->queue_transaction(osr, std::move(*t), new C_SyntheticOnClone(this, t, old_obj, new_obj));
+ delete t;
+ return status;
}
int setattrs() {
t->setattrs(cid, obj, attrs);
++in_flight;
in_flight_objects.insert(obj);
- return store->queue_transaction(osr, t, new C_SyntheticOnReadable(this, t, obj));
+ int status = store->queue_transaction(osr, std::move(*t), new C_SyntheticOnReadable(this, t, obj));
+ delete t;
+ return status;
}
void getattrs() {
contents[obj].attrs.erase(it->first);
++in_flight;
in_flight_objects.insert(obj);
- return store->queue_transaction(osr, t, new C_SyntheticOnReadable(this, t, obj));
+ int status = store->queue_transaction(osr, std::move(*t), new C_SyntheticOnReadable(this, t, obj));
+ delete t;
+ return status;
}
int write() {
t->write(cid, new_obj, offset, len, bl);
++in_flight;
in_flight_objects.insert(new_obj);
- return store->queue_transaction(osr, t, new C_SyntheticOnReadable(this, t, new_obj));
+ int status = store->queue_transaction(osr, std::move(*t), new C_SyntheticOnReadable(this, t, new_obj));
+ delete t;
+ return status;
}
void read() {
bl.swap(data);
}
- return store->queue_transaction(osr, t, new C_SyntheticOnReadable(this, t, obj));
+ int status = store->queue_transaction(osr, std::move(*t), new C_SyntheticOnReadable(this, t, obj));
+ delete t;
+ return status;
}
void scan() {
available_objects.erase(to_remove);
in_flight_objects.insert(to_remove);
contents.erase(to_remove);
- return store->queue_transaction(osr, t, new C_SyntheticOnReadable(this, t, to_remove));
+ int status = store->queue_transaction(osr, std::move(*t), new C_SyntheticOnReadable(this, t, to_remove));
+ delete t;
+ return status;
}
int zero() {
t->zero(cid, new_obj, offset, len);
++in_flight;
in_flight_objects.insert(new_obj);
- return store->queue_transaction(osr, t, new C_SyntheticOnReadable(this, t, new_obj));
+ int status = store->queue_transaction(osr, std::move(*t), new C_SyntheticOnReadable(this, t, new_obj));
+ delete t;
+ return status;
}
void print_internal_state() {
{
ObjectStore::Transaction t;
t.create_collection(cid, 0);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
string base = "";
{
ObjectStore::Transaction t;
t.touch(cid, hoid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
created.insert(hoid);
++i) {
ObjectStore::Transaction t;
t.remove(cid, *i);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
ObjectStore::Transaction t;
t.remove_collection(cid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.create_collection(cid, 0);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
string base = "aaaaa";
{
ObjectStore::Transaction t;
t.touch(cid, hoid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
created.insert(hoid);
t.touch(cid, hoid1);
t.touch(cid, hoid2);
t.touch(cid, hoid3);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
created.insert(hoid1);
created.insert(hoid2);
created.insert(hoid3);
++i) {
ObjectStore::Transaction t;
t.remove(cid, *i);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
ObjectStore::Transaction t;
t.remove_collection(cid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.create_collection(cid, 0);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
t.omap_clear(cid, hoid);
map<string, bufferlist> start_set;
t.omap_setkeys(cid, hoid, start_set);
- store->apply_transaction(&osr, t);
+ store->apply_transaction(&osr, std::move(t));
}
for (int i = 0; i < 100; i++) {
to_add.insert(pair<string, bufferlist>("key-" + string(buf), bl));
attrs.insert(pair<string, bufferlist>("key-" + string(buf), bl));
t.omap_setkeys(cid, hoid, to_add);
- store->apply_transaction(&osr, t);
+ store->apply_transaction(&osr, std::move(t));
}
int i = 0;
set<string> keys_to_remove;
keys_to_remove.insert(to_remove);
t.omap_rmkeys(cid, hoid, keys_to_remove);
- store->apply_transaction(&osr, t);
+ store->apply_transaction(&osr, std::move(t));
attrs.erase(to_remove);
bl1.append("omap_header");
ObjectStore::Transaction t;
t.omap_setheader(cid, hoid, bl1);
- store->apply_transaction(&osr, t);
-
+ store->apply_transaction(&osr, std::move(t));
+ t = ObjectStore::Transaction();
+
bufferlist bl2;
bl2.append("value");
map<string, bufferlist> to_add;
to_add.insert(pair<string, bufferlist>("key", bl2));
t.omap_setkeys(cid, hoid, to_add);
- store->apply_transaction(&osr, t);
+ store->apply_transaction(&osr, std::move(t));
bufferlist bl3;
map<string, bufferlist> cur_attrs;
t.touch(cid, hoid);
t.omap_setheader(cid, hoid, h);
t.omap_setkeys(cid, hoid, to_set);
- store->apply_transaction(&osr, t);
+ store->apply_transaction(&osr, std::move(t));
}
{
ObjectStore::Transaction t;
t.omap_rmkeyrange(cid, hoid, "3", "7");
- store->apply_transaction(&osr, t);
+ store->apply_transaction(&osr, std::move(t));
}
{
bufferlist hdr;
{
ObjectStore::Transaction t;
t.omap_clear(cid, hoid);
- store->apply_transaction(&osr, t);
+ store->apply_transaction(&osr, std::move(t));
}
{
bufferlist hdr;
ObjectStore::Transaction t;
t.remove(cid, hoid);
t.remove_collection(cid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.create_collection(cid, 0);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
t.omap_clear(cid, hoid);
map<string, bufferlist> start_set;
t.omap_setkeys(cid, hoid, start_set);
- store->apply_transaction(&osr, t);
+ store->apply_transaction(&osr, std::move(t));
}
ObjectMap::ObjectMapIterator iter;
bool correct;
attrs.insert(pair<string, bufferlist>("key-" + string(buf), bl));
ObjectStore::Transaction t;
t.omap_setkeys(cid, hoid, to_add);
- store->apply_transaction(&osr, t);
+ store->apply_transaction(&osr, std::move(t));
}
iter = store->get_omap_iterator(cid, hoid);
ObjectStore::Transaction t;
t.remove(cid, hoid);
t.remove_collection(cid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
ObjectStore::Transaction t;
t.create_collection(cid, 0);
t.touch(cid, hoid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
attrs["attr4"] = big;
t.setattr(cid, hoid, "attr3", big);
attrs["attr3"] = big;
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
ObjectStore::Transaction t;
t.rmattr(cid, hoid, "attr2");
attrs.erase("attr2");
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
ObjectStore::Transaction t;
t.remove(cid, hoid);
t.remove_collection(cid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.create_collection(cid, common_suffix_size);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
i<<common_suffix_size,
52, "")));
}
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.create_collection(tid, common_suffix_size + 1);
t.split_collection(cid, common_suffix_size+1, 1<<common_suffix_size, tid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
t.remove_collection(cid);
t.remove_collection(tid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.create_collection(cid, 0);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
std::cout << "Making objects" << std::endl;
}
o.hobj.set_hash((i << 16) | 0xB1);
t.touch(cid, o);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
std::cout << "Removing half" << std::endl;
o.hobj.pool = -1;
o.hobj.set_hash((i << 16) | 0xA1);
t.remove(cid, o);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
std::cout << "Checking" << std::endl;
t.remove(cid, o);
o.hobj.set_hash((i << 16) | 0xB1);
t.remove(cid, o);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
ObjectStore::Transaction t;
t.remove_collection(cid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
ObjectStore::Transaction t;
t.create_collection(cid, 0);
t.write(cid, srcoid, 0, data.length(), data);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
ASSERT_TRUE(store->exists(cid, srcoid));
t.collection_move_rename(cid, srcoid, cid, dstoid);
t.write(cid, srcoid, 0, data.length(), data);
t.setattr(cid, srcoid, "attr", data);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
ASSERT_TRUE(store->exists(cid, srcoid));
t.remove(cid, dstoid);
t.remove(cid, srcoid);
t.remove_collection(cid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
ObjectStore::Transaction t;
t.create_collection(cid, 0);
t.touch(cid, oid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
ASSERT_TRUE(store->exists(cid, oid));
t.write(cid, temp_oid, 0, data.length(), data);
t.setattr(cid, temp_oid, "attr", attr);
t.omap_setkeys(cid, temp_oid, omap);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
ASSERT_TRUE(store->exists(cid, temp_oid));
ObjectStore::Transaction t;
t.remove(cid, oid);
t.collection_move_rename(cid, temp_oid, cid, oid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
ASSERT_TRUE(store->exists(cid, oid));
ObjectStore::Transaction t;
t.remove(cid, oid);
t.remove_collection(cid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
t.collection_move_rename(cid, oidhead, cid, oid);
t.touch(cid, oidhead);
t.collection_move_rename(cid, oidhead, cid, oid2);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.remove(cid, oid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
ObjectStore::Transaction t;
t.remove(cid, oid2);
t.remove_collection(cid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
ObjectStore::Transaction t;
t.create_collection(cid, 0);
t.touch(cid, hoid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.set_alloc_hint(cid, hoid, 4*1024*1024, 1024*4);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.remove(cid, hoid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.set_alloc_hint(cid, hoid, 4*1024*1024, 1024*4);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
{
ObjectStore::Transaction t;
t.remove_collection(cid);
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
ASSERT_EQ(r, 0);
}
}
ObjectStore::Transaction t;
assert(!store->mount());
t.create_collection(coll, 0);
- store->apply_transaction(&osr, t);
+ store->apply_transaction(&osr, std::move(t));
} else {
assert(!store->mount());
}
break;
}
- c = new C_OnReadable(this, t);
+ c = new C_OnReadable(this);
goto queue_tx;
}
if (destroy_collection) {
do_destroy_collection(t, entry, stat_state);
- c = new C_OnDestroyed(this, t, entry);
+ c = new C_OnDestroyed(this, entry);
if (!m_num_ops)
create_coll = true;
} else {
do_pgmeta_omap_set(t, entry->m_pgid, entry->m_coll, stat_state);
do_append_log(t, entry, stat_state);
- c = new C_OnReadable(this, t);
+ c = new C_OnReadable(this);
}
queue_tx:
c = new C_StatWrapper(stat_state, tmp);
}
- m_store->queue_transaction(&(entry->m_osr), t, c);
+ m_store->queue_transaction(&(entry->m_osr), std::move(*t), c);
+ delete t;
inc_in_flight();
WorkloadGenerator *wrkldgen_state;
public:
- C_OnReadable(WorkloadGenerator *state,
- ObjectStore::Transaction *t)
- :TestObjectStoreState::C_OnFinished(state, t), wrkldgen_state(state) { }
+ C_OnReadable(WorkloadGenerator *state)
+ :TestObjectStoreState::C_OnFinished(state), wrkldgen_state(state) { }
void finish(int r)
{
coll_entry_t *m_entry;
public:
- C_OnDestroyed(WorkloadGenerator *state,
- ObjectStore::Transaction *t, coll_entry_t *entry) :
- C_OnReadable(state, t), m_entry(entry) {}
+ C_OnDestroyed(WorkloadGenerator *state, coll_entry_t *entry) :
+ C_OnReadable(state), m_entry(entry) {}
void finish(int r) {
C_OnReadable::finish(r);
uint64_t offset = starting_offset;
size_t len = cfg.size;
- list<ObjectStore::Transaction*> tls;
+ vector<ObjectStore::Transaction> tls;
std::cout << "Write cycle " << i << std::endl;
while (len) {
auto t = new ObjectStore::Transaction;
t->write(cid, oid, offset, count, data);
- tls.push_back(t);
+ tls.push_back(std::move(*t));
+ delete t;
offset += count;
if (offset > cfg.size)
cond.wait(lock, [&done](){ return done; });
lock.unlock();
- while (!tls.empty()) {
- auto t = tls.front();
- tls.pop_front();
- delete t;
- }
+
}
sequencer.flush();
}
ObjectStore::Sequencer osr(__func__);
ObjectStore::Transaction t;
t.create_collection(cid, 0);
- os->apply_transaction(&osr, t);
+ os->apply_transaction(&osr, std::move(t));
}
// create the objects
ObjectStore::Sequencer osr(__func__);
ObjectStore::Transaction t;
t.touch(cid, oids[i]);
- int r = os->apply_transaction(&osr, t);
+ int r = os->apply_transaction(&osr, std::move(t));
assert(r == 0);
}
} else {
ObjectStore::Sequencer osr(__func__);
ObjectStore::Transaction t;
t.touch(cid, oids.back());
- int r = os->apply_transaction(&osr, t);
+ int r = os->apply_transaction(&osr, std::move(t));
assert(r == 0);
}
ObjectStore::Transaction t;
for (const auto &oid : oids)
t.remove(cid, oid);
- os->apply_transaction(&osr,t);
+ os->apply_transaction(&osr,std::move(t));
os->umount();
return 0;
ASSERT_EQ(0, j.create());
j.make_writeable();
- list<ObjectStore::Transaction*> tls;
+ vector<ObjectStore::Transaction> tls;
bufferlist bl;
bl.append("small");
int orig_len = j.prepare_entry(tls, &bl);
memset(foo, 1, sizeof(foo));
bl.append(foo, sizeof(foo));
}
- list<ObjectStore::Transaction*> tls;
+ vector<ObjectStore::Transaction> tls;
int orig_len = j.prepare_entry(tls, &bl);
j.submit_entry(1, bl, orig_len, new C_SafeCond(&wait_lock, &cond, &done));
wait();
C_GatherBuilder gb(g_ceph_context, new C_SafeCond(&wait_lock, &cond, &done));
- list<ObjectStore::Transaction*> tls;
+ vector<ObjectStore::Transaction> tls;
bufferlist bl;
bl.append("small");
uint64_t seq = 1;
bufferlist first;
first.append("small");
- list<ObjectStore::Transaction*> tls;
+ vector<ObjectStore::Transaction> tls;
int orig_len = j.prepare_entry(tls, &first);
j.submit_entry(1, first, orig_len, gb.new_sub());
g_ceph_context->_conf->set_val("journal_write_header_frequency", "0");
g_ceph_context->_conf->apply_changes(NULL);
- list<ObjectStore::Transaction*> tls;
+ vector<ObjectStore::Transaction> tls;
for (unsigned i = 0 ; i < 3; ++i) {
SCOPED_TRACE(subtests[i].description);
g_ceph_context->_conf->set_val("journal_write_header_frequency", "0");
g_ceph_context->_conf->apply_changes(NULL);
- list<ObjectStore::Transaction*> tls;
+ vector<ObjectStore::Transaction> tls;
for (unsigned i = 0 ; i < 3; ++i) {
SCOPED_TRACE(subtests[i].description);
fsid.generate_random();
memset(foo, 1, sizeof(foo));
uint64_t seq = 1, committed = 0;
- list<ObjectStore::Transaction*> tls;
+ vector<ObjectStore::Transaction> tls;
for (unsigned i=0; i<size_mb*2; i++) {
bl.clear();
g_ceph_context->_conf->set_val("journal_ignore_corruption", "false");
g_ceph_context->_conf->set_val("journal_write_header_frequency", "0");
g_ceph_context->_conf->apply_changes(NULL);
- list<ObjectStore::Transaction*> tls;
+ vector<ObjectStore::Transaction> tls;
for (unsigned i = 0 ; i < 3; ++i) {
SCOPED_TRACE(subtests[i].description);
g_ceph_context->_conf->set_val("journal_write_header_frequency", "1");
g_ceph_context->_conf->apply_changes(NULL);
- list<ObjectStore::Transaction*> tls;
+ vector<ObjectStore::Transaction> tls;
for (unsigned i = 0 ; i < 3; ++i) {
SCOPED_TRACE(subtests[i].description);
fsid.generate_random();
g_ceph_context->_conf->set_val("journal_write_header_frequency", "1");
g_ceph_context->_conf->apply_changes(NULL);
- list<ObjectStore::Transaction*> tls;
+ vector<ObjectStore::Transaction> tls;
for (unsigned i = 0 ; i < 3; ++i) {
SCOPED_TRACE(subtests[i].description);
fsid.generate_random();
g_ceph_context->_conf->set_val("journal_write_header_frequency", "1");
g_ceph_context->_conf->apply_changes(NULL);
- list<ObjectStore::Transaction*> tls;
+ vector<ObjectStore::Transaction> tls;
for (unsigned i = 0 ; i < 3; ++i) {
SCOPED_TRACE(subtests[i].description);
fsid.generate_random();
dout(0) << "starting thread" << dendl;
foo.create("foo");
dout(0) << "starting op" << dendl;
- fs->apply_transaction(&osr, t);
+ fs->apply_transaction(&osr, std::move(t));
}
}
collections[coll] = make_pair(objects, new ObjectStore::Sequencer(coll.to_str()));
}
- store->apply_transaction(&osr, t);
+ store->apply_transaction(&osr, std::move(t));
bufferlist bl;
for (int i = 0; i < attrsize; ++i) {
bl);
}
}
- store->queue_transaction(iter->second.second, t,
+ store->queue_transaction(iter->second.second, std::move(*t),
new OnApplied(&lock, &cond, &in_flight,
t));
+ delete t;
}
{
Mutex::Locker l(lock);
if (r < 0) {
return r;
}
- store->apply_transaction(&osr, rmt);
+ store->apply_transaction(&osr, std::move(rmt));
finish_remove_pgs(store);
return r;
}
ObjectStore::Transaction t;
t.write(coll_t::meta(), inc_oid, 0, bl.length(), bl);
t.truncate(coll_t::meta(), inc_oid, bl.length());
- int ret = store->apply_transaction(&osr, t);
+ int ret = store->apply_transaction(&osr, std::move(t));
if (ret) {
cerr << "Failed to set inc-osdmap (" << inc_oid << "): " << ret << std::endl;
} else {
ObjectStore::Transaction t;
t.write(coll_t::meta(), full_oid, 0, bl.length(), bl);
t.truncate(coll_t::meta(), full_oid, bl.length());
- int ret = store->apply_transaction(&osr, t);
+ int ret = store->apply_transaction(&osr, std::move(t));
if (ret) {
cerr << "Failed to set osdmap (" << full_oid << "): " << ret << std::endl;
} else {
}
}
if (!dry_run)
- store->apply_transaction(&osr, *t);
+ store->apply_transaction(&osr, std::move(*t));
return 0;
}
::encode((char)1, values["_remove"]);
t.omap_setkeys(coll, pgid.make_pgmeta_oid(), values);
- store->apply_transaction(&osr, t);
+ store->apply_transaction(&osr, std::move(t));
}
cout << "Importing pgid " << pgid;
set<string> remove;
remove.insert("_remove");
t.omap_rmkeys(coll, pgid.make_pgmeta_oid(), remove);
- store->apply_transaction(&osr, t);
+ store->apply_transaction(&osr, std::move(t));
}
return 0;
t.remove(coll, ghobj);
- store->apply_transaction(&osr, t);
+ store->apply_transaction(&osr, std::move(t));
return 0;
}
} while(true);
if (!dry_run)
- store->apply_transaction(&osr, *t);
+ store->apply_transaction(&osr, std::move(*t));
return 0;
}
t->setattr(coll, ghobj, key, bl);
- store->apply_transaction(&osr, *t);
+ store->apply_transaction(&osr, std::move(*t));
return 0;
}
t->rmattr(coll, ghobj, key);
- store->apply_transaction(&osr, *t);
+ store->apply_transaction(&osr, std::move(*t));
return 0;
}
t->omap_setkeys(coll, ghobj, attrset);
- store->apply_transaction(&osr, *t);
+ store->apply_transaction(&osr, std::move(*t));
return 0;
}
t->omap_rmkeys(coll, ghobj, keys);
- store->apply_transaction(&osr, *t);
+ store->apply_transaction(&osr, std::move(*t));
return 0;
}
t->omap_setheader(coll, ghobj, hdrbl);
- store->apply_transaction(&osr, *t);
+ store->apply_transaction(&osr, std::move(*t));
return 0;
}
::encode(oi, bl);
ObjectStore::Transaction t;
t.setattr(coll, ghobj, OI_ATTR, bl);
- int r = store->apply_transaction(osr, t);
+ int r = store->apply_transaction(osr, std::move(t));
if (r < 0) {
cerr << "Error getting fixing attr on : " << make_pair(coll, ghobj)
<< ", "
::encode(ss, snapattr);
t.setattr(coll, head, SS_ATTR, snapattr);
}
- r = store->apply_transaction(&osr, t);
+ r = store->apply_transaction(&osr, std::move(t));
if (r < 0) {
cerr << "Error writing object info: " << make_pair(coll, ghobj) << ", "
<< cpp_strerror(r) << std::endl;
::encode(ss, bl);
ObjectStore::Transaction t;
t.setattr(coll, ghobj, SS_ATTR, bl);
- int r = store->apply_transaction(&osr, t);
+ int r = store->apply_transaction(&osr, std::move(t));
if (r < 0) {
cerr << "Error setting snapset on : " << make_pair(coll, ghobj) << ", "
<< cpp_strerror(r) << std::endl;
::encode(snapset, bl);
ObjectStore::Transaction t;
t.setattr(coll, ghobj, SS_ATTR, bl);
- int r = store->apply_transaction(&osr, t);
+ int r = store->apply_transaction(&osr, std::move(t));
if (r < 0) {
cerr << "Error setting snapset on : " << make_pair(coll, ghobj) << ", "
<< cpp_strerror(r) << std::endl;
ret = write_info(*t, map_epoch, info, past_intervals);
if (ret == 0) {
- fs->apply_transaction(osr, *t);
+ fs->apply_transaction(osr, std::move(*t));
cout << "Removal succeeded" << std::endl;
}
} else if (op == "mark-complete") {
ret = write_info(*t, map_epoch, info, past_intervals);
if (ret != 0)
goto out;
- fs->apply_transaction(osr, *t);
+ fs->apply_transaction(osr, std::move(*t));
}
cout << "Marking complete succeeded" << std::endl;
} else {