From: Jason Dillaman Date: Wed, 28 Feb 2018 19:38:06 +0000 (-0500) Subject: librbd: removed legacy journal event tracking hooks from IO path X-Git-Tag: v13.0.2~31^2~6 X-Git-Url: http://git-server-git.apps.pok.os.sepia.ceph.com/?a=commitdiff_plain;h=898223eea6a5fba844f04a7349012a63108c893f;p=ceph.git librbd: removed legacy journal event tracking hooks from IO path The new journal object dispatch layer handles delaying writes to the OSD until the journal event is safe. This avoid the need for two different code paths for cache and non-cache case. Signed-off-by: Jason Dillaman --- diff --git a/src/librbd/Journal.cc b/src/librbd/Journal.cc index 575ceb25511..2243a2fb7f2 100644 --- a/src/librbd/Journal.cc +++ b/src/librbd/Journal.cc @@ -716,7 +716,6 @@ void Journal::flush_commit_position(Context *on_finish) { template uint64_t Journal::append_write_event(uint64_t offset, size_t length, const bufferlist &bl, - const IOObjectRequests &requests, bool flush_entry) { assert(m_max_append_size > journal::AioWriteEvent::get_fixed_size()); uint64_t max_write_data_size = @@ -743,26 +742,24 @@ uint64_t Journal::append_write_event(uint64_t offset, size_t length, bytes_remaining -= event_length; } while (bytes_remaining > 0); - return append_io_events(journal::EVENT_TYPE_AIO_WRITE, bufferlists, requests, - offset, length, flush_entry, 0); + return append_io_events(journal::EVENT_TYPE_AIO_WRITE, bufferlists, offset, + length, flush_entry, 0); } template uint64_t Journal::append_io_event(journal::EventEntry &&event_entry, - const IOObjectRequests &requests, uint64_t offset, size_t length, bool flush_entry, int filter_ret_val) { bufferlist bl; event_entry.timestamp = ceph_clock_now(); encode(event_entry, bl); - return append_io_events(event_entry.get_event_type(), {bl}, requests, offset, - length, flush_entry, filter_ret_val); + return append_io_events(event_entry.get_event_type(), {bl}, offset, length, + flush_entry, filter_ret_val); } template uint64_t Journal::append_io_events(journal::EventType event_type, const Bufferlists &bufferlists, - const IOObjectRequests &requests, uint64_t offset, size_t length, bool flush_entry, int filter_ret_val) { assert(!bufferlists.empty()); @@ -784,13 +781,12 @@ uint64_t Journal::append_io_events(journal::EventType event_type, { Mutex::Locker event_locker(m_event_lock); - m_events[tid] = Event(futures, requests, offset, length, filter_ret_val); + m_events[tid] = Event(futures, offset, length, filter_ret_val); } CephContext *cct = m_image_ctx.cct; ldout(cct, 20) << this << " " << __func__ << ": " << "event=" << event_type << ", " - << "new_reqs=" << requests.size() << ", " << "offset=" << offset << ", " << "length=" << length << ", " << "flush=" << flush_entry << ", tid=" << tid << dendl; @@ -1450,7 +1446,6 @@ void Journal::handle_io_event_safe(int r, uint64_t tid) { << "failed to commit IO event: " << cpp_strerror(r) << dendl; } - IOObjectRequests aio_object_requests; Contexts on_safe_contexts; { Mutex::Locker event_locker(m_event_lock); @@ -1458,7 +1453,6 @@ void Journal::handle_io_event_safe(int r, uint64_t tid) { assert(it != m_events.end()); Event &event = it->second; - aio_object_requests.swap(event.aio_object_requests); on_safe_contexts.swap(event.on_safe_contexts); if (r < 0 || event.committed_io) { @@ -1479,16 +1473,6 @@ void Journal::handle_io_event_safe(int r, uint64_t tid) { ldout(cct, 20) << this << " " << __func__ << ": " << "completing tid=" << tid << dendl; - for (auto it = aio_object_requests.begin(); it != aio_object_requests.end(); - ++it) { - if (r < 0) { - // don't send aio requests if the journal fails -- bubble error up - (*it)->fail(r); - } else { - // send any waiting aio requests now that journal entry is safe - (*it)->send(tid); - } - } // alert the cache about the journal event status for (Contexts::iterator it = on_safe_contexts.begin(); diff --git a/src/librbd/Journal.h b/src/librbd/Journal.h index 088bde522da..609e718bec2 100644 --- a/src/librbd/Journal.h +++ b/src/librbd/Journal.h @@ -37,7 +37,6 @@ namespace librbd { class ImageCtx; -namespace io { struct ObjectDispatchSpec; } namespace journal { template class Replay; } template @@ -91,8 +90,6 @@ public: static const std::string LOCAL_MIRROR_UUID; static const std::string ORPHAN_MIRROR_UUID; - typedef std::list IOObjectRequests; - Journal(ImageCtxT &image_ctx); ~Journal(); @@ -137,10 +134,8 @@ public: uint64_t append_write_event(uint64_t offset, size_t length, const bufferlist &bl, - const IOObjectRequests &requests, bool flush_entry); uint64_t append_io_event(journal::EventEntry &&event_entry, - const IOObjectRequests &requests, uint64_t offset, size_t length, bool flush_entry, int filter_ret_val); void commit_io_event(uint64_t tid, int r); @@ -190,7 +185,6 @@ private: struct Event { Futures futures; - IOObjectRequests aio_object_requests; Contexts on_safe_contexts; ExtentInterval pending_extents; int filter_ret_val = 0; @@ -200,10 +194,9 @@ private: Event() { } - Event(const Futures &_futures, const IOObjectRequests &_requests, - uint64_t offset, size_t length, int filter_ret_val) - : futures(_futures), aio_object_requests(_requests), - filter_ret_val(filter_ret_val) { + Event(const Futures &_futures, uint64_t offset, size_t length, + int filter_ret_val) + : futures(_futures), filter_ret_val(filter_ret_val) { if (length > 0) { pending_extents.insert(offset, length); } @@ -335,7 +328,6 @@ private: uint64_t append_io_events(journal::EventType event_type, const Bufferlists &bufferlists, - const IOObjectRequests &requests, uint64_t offset, size_t length, bool flush_entry, int filter_ret_val); Future wait_event(Mutex &lock, uint64_t tid, Context *on_safe); diff --git a/src/librbd/LibrbdWriteback.cc b/src/librbd/LibrbdWriteback.cc index cccdc6ba6bf..e343c737b53 100644 --- a/src/librbd/LibrbdWriteback.cc +++ b/src/librbd/LibrbdWriteback.cc @@ -149,7 +149,7 @@ namespace librbd { auto req = io::ObjectDispatchSpec::create_read( m_ictx, io::OBJECT_DISPATCH_LAYER_CACHE, oid.name, object_no, off, len, snapid, op_flags, trace, &req_comp->bl, &req_comp->extent_map, req_comp); - req->send(0); + req->send(); } bool LibrbdWriteback::may_copy_on_write(const object_t& oid, uint64_t read_off, uint64_t read_len, snapid_t snapid) @@ -210,7 +210,7 @@ namespace librbd { req->object_dispatch_flags = ( io::OBJECT_DISPATCH_FLAG_FLUSH | io::OBJECT_DISPATCH_FLAG_WILL_RETRY_ON_ERROR); - req->send(0); + req->send(); return ++m_tid; } diff --git a/src/librbd/internal.cc b/src/librbd/internal.cc index cd96a47ce8d..10ad429145a 100644 --- a/src/librbd/internal.cc +++ b/src/librbd/internal.cc @@ -2366,7 +2366,7 @@ bool compare_by_name(const child_info_t& c1, const child_info_t& c2) ictx, io::OBJECT_DISPATCH_LAYER_NONE, q->oid.name, q->objectno, q->offset, q->length, snap_id, 0, {}, &req_comp->read_data, &req_comp->extent_map, req_comp); - req->send(0); + req->send(); } } ictx->perfcounter->inc(l_librbd_readahead); diff --git a/src/librbd/io/AioCompletion.cc b/src/librbd/io/AioCompletion.cc index ef51ad36cc4..00e043518aa 100644 --- a/src/librbd/io/AioCompletion.cc +++ b/src/librbd/io/AioCompletion.cc @@ -80,12 +80,6 @@ void AioCompletion::complete() { break; } - // inform the journal that the op has successfully committed - if (journal_tid != 0) { - assert(ictx->journal != NULL); - ictx->journal->commit_io_event(journal_tid, rval); - } - state = AIO_STATE_CALLBACK; if (complete_cb) { lock.Unlock(); @@ -180,12 +174,6 @@ void AioCompletion::complete_request(ssize_t r) put_unlock(); } -void AioCompletion::associate_journal_event(uint64_t tid) { - Mutex::Locker l(lock); - assert(state == AIO_STATE_PENDING); - journal_tid = tid; -} - bool AioCompletion::is_complete() { tracepoint(librbd, aio_is_complete_enter, this); bool done; diff --git a/src/librbd/io/AioCompletion.h b/src/librbd/io/AioCompletion.h index 7f3abb3947c..6aff99154fa 100644 --- a/src/librbd/io/AioCompletion.h +++ b/src/librbd/io/AioCompletion.h @@ -61,7 +61,6 @@ struct AioCompletion { AsyncOperation async_op; - uint64_t journal_tid; xlist::item m_xlist_item; bool event_notify; @@ -109,8 +108,8 @@ struct AioCompletion { complete_arg(NULL), rbd_comp(NULL), pending_count(0), blockers(1), ref(1), released(false), ictx(NULL), - aio_type(AIO_TYPE_NONE), - journal_tid(0), m_xlist_item(this), event_notify(false) { + aio_type(AIO_TYPE_NONE), m_xlist_item(this), + event_notify(false) { } ~AioCompletion() { @@ -149,8 +148,6 @@ struct AioCompletion { } void complete_request(ssize_t r); - void associate_journal_event(uint64_t tid); - bool is_complete(); ssize_t get_return_value(); diff --git a/src/librbd/io/ImageRequest.cc b/src/librbd/io/ImageRequest.cc index 4cf402dc48c..c7ec7113519 100644 --- a/src/librbd/io/ImageRequest.cc +++ b/src/librbd/io/ImageRequest.cc @@ -213,7 +213,7 @@ void ImageReadRequest::send_request() { &image_ctx, OBJECT_DISPATCH_LAYER_NONE, extent.oid.name, extent.objectno, extent.offset, extent.length, snap_id, m_op_flags, this->m_trace, &req_comp->bl, &req_comp->extent_map, req_comp); - req->send(0); + req->send(); } } @@ -318,7 +318,7 @@ void AbstractImageWriteRequest::send_object_requests( // if journaling, stash the request for later; otherwise send if (request != NULL) { - request->send(0); + request->send(); } } } @@ -347,7 +347,7 @@ uint64_t ImageWriteRequest::append_journal_event(bool synchronous) { buffer_offset += extent.second; tid = image_ctx.journal->append_write_event(extent.first, extent.second, - sub_bl, {}, synchronous); + sub_bl, synchronous); } return tid; @@ -399,8 +399,8 @@ uint64_t ImageDiscardRequest::append_journal_event(bool synchronous) { extent.second, this->m_skip_partial_discard)); tid = image_ctx.journal->append_io_event(std::move(event_entry), - {}, extent.first, - extent.second, synchronous, 0); + extent.first, extent.second, + synchronous, 0); } return tid; @@ -481,8 +481,7 @@ void ImageFlushRequest::send_request() { if (journaling) { // in-flight ops are flushed prior to closing the journal uint64_t journal_tid = image_ctx.journal->append_io_event( - journal::EventEntry(journal::AioFlushEvent()), - ObjectRequests(), 0, 0, false, 0); + journal::EventEntry(journal::AioFlushEvent()), 0, 0, false, 0); ctx = new FunctionContext( [&image_ctx, journal_tid, ctx](int r) { @@ -499,7 +498,7 @@ void ImageFlushRequest::send_request() { &image_ctx, OBJECT_DISPATCH_LAYER_NONE, m_flush_source, this->m_trace, ctx); ctx = new FunctionContext([object_dispatch_spec](int r) { - object_dispatch_spec->send(0); + object_dispatch_spec->send(); }); } @@ -533,8 +532,8 @@ uint64_t ImageWriteSameRequest::append_journal_event(bool synchronous) { extent.second, m_data_bl)); tid = image_ctx.journal->append_io_event(std::move(event_entry), - {}, extent.first, - extent.second, synchronous, 0); + extent.first, extent.second, + synchronous, 0); } return tid; @@ -600,8 +599,8 @@ uint64_t ImageCompareAndWriteRequest::append_journal_event( journal::AioCompareAndWriteEvent(extent.first, extent.second, m_cmp_bl, m_bl)); tid = image_ctx.journal->append_io_event(std::move(event_entry), - {}, extent.first, - extent.second, synchronous, -EILSEQ); + extent.first, extent.second, + synchronous, -EILSEQ); return tid; } diff --git a/src/librbd/io/ObjectDispatchSpec.cc b/src/librbd/io/ObjectDispatchSpec.cc index 14830a5f85c..1d0c24e003b 100644 --- a/src/librbd/io/ObjectDispatchSpec.cc +++ b/src/librbd/io/ObjectDispatchSpec.cc @@ -17,7 +17,7 @@ void ObjectDispatchSpec::C_Dispatcher::complete(int r) { switch (object_dispatch_spec->dispatch_result) { case DISPATCH_RESULT_CONTINUE: - object_dispatch_spec->send(0); + object_dispatch_spec->send(); break; case DISPATCH_RESULT_COMPLETE: finish(r); @@ -33,27 +33,7 @@ void ObjectDispatchSpec::C_Dispatcher::finish(int r) { delete object_dispatch_spec; } -struct ObjectDispatchSpec::SetJournalTid : public boost::static_visitor { - uint64_t journal_tid; - - SetJournalTid(uint64_t journal_tid) : journal_tid(journal_tid) { - } - - template - void operator()(T& t) const { - } - - void operator()(WriteRequestBase& write_request_base) const { - write_request_base.journal_tid = journal_tid; - } -}; - -void ObjectDispatchSpec::send(uint64_t journal_tid) { - // TODO removed in future commit - if (journal_tid != 0) { - boost::apply_visitor(SetJournalTid{journal_tid}, request); - } - +void ObjectDispatchSpec::send() { object_dispatcher->send(this); } diff --git a/src/librbd/io/ObjectDispatchSpec.h b/src/librbd/io/ObjectDispatchSpec.h index eddff49330c..a26d89fe456 100644 --- a/src/librbd/io/ObjectDispatchSpec.h +++ b/src/librbd/io/ObjectDispatchSpec.h @@ -243,15 +243,12 @@ public: parent_trace, on_finish); } - void send(uint64_t journal_tid /* removed in future commit */); + void send(); void fail(int r); private: template friend class ObjectDispatcher; - // TODO removed in future commmit - struct SetJournalTid; - ObjectDispatchSpec(ObjectDispatcherInterface* object_dispatcher, ObjectDispatchLayer object_dispatch_layer, Request&& request, int op_flags, diff --git a/src/librbd/operation/TrimRequest.cc b/src/librbd/operation/TrimRequest.cc index 7fc61406293..99d144e064b 100644 --- a/src/librbd/operation/TrimRequest.cc +++ b/src/librbd/operation/TrimRequest.cc @@ -50,7 +50,7 @@ public: &image_ctx, io::OBJECT_DISPATCH_LAYER_NONE, oid, m_object_no, 0, image_ctx.layout.object_size, m_snapc, io::OBJECT_DISCARD_FLAG_DISABLE_OBJECT_MAP_UPDATE, 0, {}, this); - object_dispatch_spec->send(0); + object_dispatch_spec->send(); return 0; } private: @@ -349,7 +349,7 @@ void TrimRequest::send_clean_boundary() { auto object_dispatch_spec = io::ObjectDispatchSpec::create_discard( &image_ctx, io::OBJECT_DISPATCH_LAYER_NONE, p->oid.name, p->objectno, p->offset, p->length, snapc, 0, 0, {}, req_comp); - object_dispatch_spec->send(0); + object_dispatch_spec->send(); } completion->finish_adding_requests(); } diff --git a/src/test/librbd/io/test_mock_ImageRequest.cc b/src/test/librbd/io/test_mock_ImageRequest.cc index 4112989fd94..4891696c631 100644 --- a/src/test/librbd/io/test_mock_ImageRequest.cc +++ b/src/test/librbd/io/test_mock_ImageRequest.cc @@ -7,7 +7,6 @@ #include "test/librbd/mock/MockJournal.h" #include "test/librbd/mock/cache/MockImageCache.h" #include "librbd/io/ImageRequest.h" -#include "librbd/io/ObjectRequest.h" #include "librbd/io/ObjectDispatchSpec.h" namespace librbd { @@ -16,21 +15,16 @@ namespace { struct MockTestImageCtx; struct MockTestJournal : public MockJournal { - typedef std::list ObjectRequests; - - MOCK_METHOD5(append_write_event, uint64_t(uint64_t, size_t, - const bufferlist &, - const ObjectRequests &, bool)); - MOCK_METHOD6(append_io_event_mock, uint64_t(const journal::EventEntry&, - const ObjectRequests &, + MOCK_METHOD4(append_write_event, uint64_t(uint64_t, size_t, + const bufferlist &, bool)); + MOCK_METHOD5(append_io_event_mock, uint64_t(const journal::EventEntry&, uint64_t, size_t, bool, int)); uint64_t append_io_event(journal::EventEntry &&event_entry, - const ObjectRequests &requests, uint64_t offset, size_t length, bool flush_entry, int filter_ret_val) { // googlemock doesn't support move semantics - return append_io_event_mock(event_entry, requests, offset, length, - flush_entry, filter_ret_val); + return append_io_event_mock(event_entry, offset, length, flush_entry, + filter_ret_val); } MOCK_METHOD2(commit_io_event, void(uint64_t, int)); diff --git a/src/test/librbd/journal/test_Replay.cc b/src/test/librbd/journal/test_Replay.cc index 0f3d944dd25..c2846c34469 100644 --- a/src/test/librbd/journal/test_Replay.cc +++ b/src/test/librbd/journal/test_Replay.cc @@ -46,11 +46,10 @@ public: void inject_into_journal(librbd::ImageCtx *ictx, T event) { C_SaferCond ctx; librbd::journal::EventEntry event_entry(event); - librbd::Journal<>::IOObjectRequests requests; { RWLock::RLocker owner_locker(ictx->owner_lock); - uint64_t tid = ictx->journal->append_io_event(std::move(event_entry), - requests, 0, 0, true, 0); + uint64_t tid = ictx->journal->append_io_event(std::move(event_entry),0, 0, + true, 0); ictx->journal->wait_event(tid, &ctx); } ASSERT_EQ(0, ctx.wait()); diff --git a/src/test/librbd/test_mock_Journal.cc b/src/test/librbd/test_mock_Journal.cc index adfe0734b2e..1f5cd749eac 100644 --- a/src/test/librbd/test_mock_Journal.cc +++ b/src/test/librbd/test_mock_Journal.cc @@ -452,21 +452,16 @@ public: bl.append_zero(length); RWLock::RLocker owner_locker(mock_image_ctx.owner_lock); - return mock_journal.append_write_event(0, length, bl, {}, false); + return mock_journal.append_write_event(0, length, bl, false); } uint64_t when_append_io_event(MockJournalImageCtx &mock_image_ctx, MockJournal &mock_journal, - io::ObjectDispatchSpec *object_request, int filter_ret_val) { RWLock::RLocker owner_locker(mock_image_ctx.owner_lock); - MockJournal::IOObjectRequests object_requests; - if (object_request != nullptr) { - object_requests.push_back(object_request); - } return mock_journal.append_io_event( - journal::EventEntry{journal::AioFlushEvent{}}, object_requests, 0, 0, - false, filter_ret_val); + journal::EventEntry{journal::AioFlushEvent{}}, 0, 0, false, + filter_ret_val); } void save_commit_context(Context *ctx) { @@ -1016,13 +1011,13 @@ TEST_F(TestMockJournal, EventAndIOCommitOrder) { Context *on_journal_safe1; expect_append_journaler(mock_journaler); expect_wait_future(mock_future, &on_journal_safe1); - ASSERT_EQ(1U, when_append_io_event(mock_image_ctx, mock_journal, nullptr, 0)); + ASSERT_EQ(1U, when_append_io_event(mock_image_ctx, mock_journal, 0)); mock_journal.get_work_queue()->drain(); Context *on_journal_safe2; expect_append_journaler(mock_journaler); expect_wait_future(mock_future, &on_journal_safe2); - ASSERT_EQ(2U, when_append_io_event(mock_image_ctx, mock_journal, nullptr, 0)); + ASSERT_EQ(2U, when_append_io_event(mock_image_ctx, mock_journal, 0)); mock_journal.get_work_queue()->drain(); // commit journal event followed by IO event (standard) @@ -1103,21 +1098,17 @@ TEST_F(TestMockJournal, EventCommitError) { close_journal(mock_image_ctx, mock_journal, mock_journaler); }; - C_SaferCond object_request_ctx; - auto object_request = io::ObjectDispatchSpec::create_discard( - &mock_image_ctx, io::OBJECT_DISPATCH_LAYER_NONE, "object0", 0, 0, 0, {}, 0, - 0, {}, &object_request_ctx); - ::journal::MockFuture mock_future; Context *on_journal_safe; expect_append_journaler(mock_journaler); expect_wait_future(mock_future, &on_journal_safe); - ASSERT_EQ(1U, when_append_io_event(mock_image_ctx, mock_journal, - object_request, 0)); + ASSERT_EQ(1U, when_append_io_event(mock_image_ctx, mock_journal, 0)); mock_journal.get_work_queue()->drain(); // commit the event in the journal w/o waiting writeback expect_future_committed(mock_journaler); + C_SaferCond object_request_ctx; + mock_journal.wait_event(1U, &object_request_ctx); on_journal_safe->complete(-EINVAL); ASSERT_EQ(-EINVAL, object_request_ctx.wait()); @@ -1147,17 +1138,11 @@ TEST_F(TestMockJournal, EventCommitErrorWithPendingWriteback) { close_journal(mock_image_ctx, mock_journal, mock_journaler); }; - C_SaferCond object_request_ctx; - auto object_request = io::ObjectDispatchSpec::create_discard( - &mock_image_ctx, io::OBJECT_DISPATCH_LAYER_NONE, "object0", 0, 0, 0, {}, 0, - 0, {}, &object_request_ctx); - ::journal::MockFuture mock_future; Context *on_journal_safe; expect_append_journaler(mock_journaler); expect_wait_future(mock_future, &on_journal_safe); - ASSERT_EQ(1U, when_append_io_event(mock_image_ctx, mock_journal, - object_request, 0)); + ASSERT_EQ(1U, when_append_io_event(mock_image_ctx, mock_journal, 0)); mock_journal.get_work_queue()->drain(); expect_future_is_valid(mock_future); @@ -1166,6 +1151,8 @@ TEST_F(TestMockJournal, EventCommitErrorWithPendingWriteback) { // commit the event in the journal w/ waiting cache writeback expect_future_committed(mock_journaler); + C_SaferCond object_request_ctx; + mock_journal.wait_event(1U, &object_request_ctx); on_journal_safe->complete(-EINVAL); ASSERT_EQ(-EINVAL, object_request_ctx.wait()); @@ -1196,7 +1183,7 @@ TEST_F(TestMockJournal, IOCommitError) { Context *on_journal_safe; expect_append_journaler(mock_journaler); expect_wait_future(mock_future, &on_journal_safe); - ASSERT_EQ(1U, when_append_io_event(mock_image_ctx, mock_journal, nullptr, 0)); + ASSERT_EQ(1U, when_append_io_event(mock_image_ctx, mock_journal, 0)); mock_journal.get_work_queue()->drain(); // failed IO remains uncommitted in journal @@ -1228,8 +1215,7 @@ TEST_F(TestMockJournal, IOCommitErrorFiltered) { Context *on_journal_safe; expect_append_journaler(mock_journaler); expect_wait_future(mock_future, &on_journal_safe); - ASSERT_EQ(1U, when_append_io_event(mock_image_ctx, mock_journal, nullptr, - -EILSEQ)); + ASSERT_EQ(1U, when_append_io_event(mock_image_ctx, mock_journal, -EILSEQ)); mock_journal.get_work_queue()->drain(); // filter failed IO committed in journal