]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
osd: add override in osd subsystem 13439/head
authorliuchang0812 <liuchang0812@gmail.com>
Wed, 15 Feb 2017 13:22:01 +0000 (21:22 +0800)
committerliuchang0812 <liuchang0812@gmail.com>
Wed, 15 Feb 2017 13:22:01 +0000 (21:22 +0800)
Fixes: http://tracker.ceph.com/issues/18922
Signed-off-by: liuchang0812 <liuchang0812@gmail.com>
27 files changed:
src/os/bluestore/BitmapFreelistManager.cc
src/os/bluestore/BlueRocksEnv.cc
src/os/bluestore/BlueStore.cc
src/os/filestore/FileStore.cc
src/os/memstore/MemStore.cc
src/osd/ECBackend.cc
src/osd/OSD.cc
src/osd/OSDMap.cc
src/osd/PG.cc
src/osd/PGBackend.cc
src/osd/PrimaryLogPG.cc
src/osd/ReplicatedBackend.cc
src/osd/Watch.cc
src/osdc/Filer.cc
src/osdc/Journaler.cc
src/osdc/ObjectCacher.cc
src/osdc/Objecter.cc
src/test/objectstore/FileStoreTracker.cc
src/test/objectstore/store_test.cc
src/test/objectstore/test_kv.cc
src/test/os/TestLFNIndex.cc
src/test/osd/TestPGLog.cc
src/test/osd/TestRados.cc
src/test/osd/types.cc
src/test/osdc/FakeWriteback.cc
src/test/osdc/MemWriteback.cc
src/test/osdc/object_cacher_stress.cc

index 7b474750037a7e7eb1279b7d64c3a814aa81738f..a3b3a5ec8a5905e4170337c9b81856e64c248fb9 100644 (file)
@@ -19,14 +19,14 @@ void make_offset_key(uint64_t offset, std::string *key)
 }
 
 struct XorMergeOperator : public KeyValueDB::MergeOperator {
-  virtual void merge_nonexistent(
+  void merge_nonexistent(
     const char *rdata, size_t rlen, std::string *new_value) override {
     *new_value = std::string(rdata, rlen);
   }
-  virtual void merge(
+  void merge(
     const char *ldata, size_t llen,
     const char *rdata, size_t rlen,
-    std::string *new_value) {
+    std::string *new_value) override {
     assert(llen == rlen);
     *new_value = std::string(ldata, llen);
     for (size_t i = 0; i < rlen; ++i) {
@@ -35,7 +35,7 @@ struct XorMergeOperator : public KeyValueDB::MergeOperator {
   }
   // We use each operator name and each prefix to construct the
   // overall RocksDB operator name for consistency check at open time.
-  virtual string name() const {
+  string name() const override {
     return "bitwise_xor";
   }
 };
index d37d725035ad1c62c629f43363fff2fd06430ae8..ff5aaa6f1112c3c4ad31cdde8650ecb7c97f1e80 100644 (file)
@@ -42,7 +42,7 @@ class BlueRocksSequentialFile : public rocksdb::SequentialFile {
   // If an error was encountered, returns a non-OK status.
   //
   // REQUIRES: External synchronization
-  rocksdb::Status Read(size_t n, rocksdb::Slice* result, char* scratch) {
+  rocksdb::Status Read(size_t n, rocksdb::Slice* result, char* scratch) override {
     int r = fs->read(h, &h->buf, h->buf.pos, n, NULL, scratch);
     assert(r >= 0);
     *result = rocksdb::Slice(scratch, r);
@@ -56,7 +56,7 @@ class BlueRocksSequentialFile : public rocksdb::SequentialFile {
   // file, and Skip will return OK.
   //
   // REQUIRES: External synchronization
-  rocksdb::Status Skip(uint64_t n) {
+  rocksdb::Status Skip(uint64_t n) override {
     h->buf.skip(n);
     return rocksdb::Status::OK();
   }
@@ -64,7 +64,7 @@ class BlueRocksSequentialFile : public rocksdb::SequentialFile {
   // Remove any kind of caching of data from the offset to offset+length
   // of this file. If the length is 0, then it refers to the end of file.
   // If the system is not caching the file contents, then this is a noop.
-  rocksdb::Status InvalidateCache(size_t offset, size_t length) {
+  rocksdb::Status InvalidateCache(size_t offset, size_t length) override {
     fs->invalidate_cache(h->file, offset, length);
     return rocksdb::Status::OK();
   }
@@ -90,7 +90,7 @@ class BlueRocksRandomAccessFile : public rocksdb::RandomAccessFile {
   //
   // Safe for concurrent use by multiple threads.
   rocksdb::Status Read(uint64_t offset, size_t n, rocksdb::Slice* result,
-                      char* scratch) const {
+                      char* scratch) const override {
     int r = fs->read_random(h, offset, n, scratch);
     assert(r >= 0);
     *result = rocksdb::Slice(scratch, r);
@@ -99,13 +99,13 @@ class BlueRocksRandomAccessFile : public rocksdb::RandomAccessFile {
 
   // Used by the file_reader_writer to decide if the ReadAhead wrapper
   // should simply forward the call and do not enact buffering or locking.
-  bool ShouldForwardRawRequest() const {
+  bool ShouldForwardRawRequest() const override {
     return false;
   }
 
   // For cases when read-ahead is implemented in the platform dependent
   // layer
-  void EnableReadAhead() {}
+  void EnableReadAhead() override {}
 
   // Tries to get an unique ID for this file that will be the same each time
   // the file is opened (and will stay the same while the file is open).
@@ -122,14 +122,14 @@ class BlueRocksRandomAccessFile : public rocksdb::RandomAccessFile {
   // a single varint.
   //
   // Note: these IDs are only valid for the duration of the process.
-  size_t GetUniqueId(char* id, size_t max_size) const {
+  size_t GetUniqueId(char* id, size_t max_size) const override {
     return snprintf(id, max_size, "%016llx",
                    (unsigned long long)h->file->fnode.ino);
   };
 
   //enum AccessPattern { NORMAL, RANDOM, SEQUENTIAL, WILLNEED, DONTNEED };
 
-  void Hint(AccessPattern pattern) {
+  void Hint(AccessPattern pattern) override {
     if (pattern == RANDOM)
       h->buf.max_prefetch = 4096;
     else if (pattern == SEQUENTIAL)
@@ -139,7 +139,7 @@ class BlueRocksRandomAccessFile : public rocksdb::RandomAccessFile {
   // Remove any kind of caching of data from the offset to offset+length
   // of this file. If the length is 0, then it refers to the end of file.
   // If the system is not caching the file contents, then this is a noop.
-  rocksdb::Status InvalidateCache(size_t offset, size_t length) {
+  rocksdb::Status InvalidateCache(size_t offset, size_t length) override {
     fs->invalidate_cache(h->file, offset, length);
     return rocksdb::Status::OK();
   }
@@ -170,7 +170,7 @@ class BlueRocksWritableFile : public rocksdb::WritableFile {
     return c_DefaultPageSize;
     }*/
 
-  rocksdb::Status Append(const rocksdb::Slice& data) {
+  rocksdb::Status Append(const rocksdb::Slice& data) override {
     h->append(data.data(), data.size());
     return rocksdb::Status::OK();
   }
@@ -179,7 +179,7 @@ class BlueRocksWritableFile : public rocksdb::WritableFile {
   // to simple append as most of the tests are buffered by default
   rocksdb::Status PositionedAppend(
     const rocksdb::Slice& /* data */,
-    uint64_t /* offset */) {
+    uint64_t /* offset */) override {
     return rocksdb::Status::NotSupported();
   }
 
@@ -187,7 +187,7 @@ class BlueRocksWritableFile : public rocksdb::WritableFile {
   // before closing. It is not always possible to keep track of the file
   // size due to whole pages writes. The behavior is undefined if called
   // with other writes to follow.
-  rocksdb::Status Truncate(uint64_t size) {
+  rocksdb::Status Truncate(uint64_t size) override {
     // we mirror the posix env, which does nothing here; instead, it
     // truncates to the final size on close.  whatever!
     return rocksdb::Status::OK();
@@ -195,7 +195,7 @@ class BlueRocksWritableFile : public rocksdb::WritableFile {
     //  return err_to_status(r);
   }
 
-  rocksdb::Status Close() {
+  rocksdb::Status Close() override {
     Flush();
 
     // mimic posix env, here.  shrug.
@@ -211,19 +211,19 @@ class BlueRocksWritableFile : public rocksdb::WritableFile {
     return rocksdb::Status::OK();
   }
 
-  rocksdb::Status Flush() {
+  rocksdb::Status Flush() override {
     fs->flush(h);
     return rocksdb::Status::OK();
   }
 
-  rocksdb::Status Sync() { // sync data
+  rocksdb::Status Sync() override { // sync data
     fs->fsync(h);
     return rocksdb::Status::OK();
   }
 
   // true if Sync() and Fsync() are safe to call concurrently with Append()
   // and Flush().
-  bool IsSyncThreadSafe() const {
+  bool IsSyncThreadSafe() const override {
     return true;
   }
 
@@ -236,12 +236,12 @@ class BlueRocksWritableFile : public rocksdb::WritableFile {
   /*
    * Get the size of valid data in the file.
    */
-  uint64_t GetFileSize() {
+  uint64_t GetFileSize() override {
     return h->file->fnode.size + h->buffer.length();;
   }
 
   // For documentation, refer to RandomAccessFile::GetUniqueId()
-  size_t GetUniqueId(char* id, size_t max_size) const {
+  size_t GetUniqueId(char* id, size_t max_size) const override {
     return snprintf(id, max_size, "%016llx",
                    (unsigned long long)h->file->fnode.ino);
   }
@@ -250,7 +250,7 @@ class BlueRocksWritableFile : public rocksdb::WritableFile {
   // of this file. If the length is 0, then it refers to the end of file.
   // If the system is not caching the file contents, then this is a noop.
   // This call has no effect on dirty pages in the cache.
-  rocksdb::Status InvalidateCache(size_t offset, size_t length) {
+  rocksdb::Status InvalidateCache(size_t offset, size_t length) override {
     fs->invalidate_cache(h->file, offset, length);
     return rocksdb::Status::OK();
   }
@@ -293,7 +293,7 @@ class BlueRocksDirectory : public rocksdb::Directory {
   explicit BlueRocksDirectory(BlueFS *f) : fs(f) {}
 
   // Fsync directory. Can be called concurrently from multiple threads.
-  rocksdb::Status Fsync() {
+  rocksdb::Status Fsync() override {
     // it is sufficient to flush the log.
     fs->sync_metadata();
     return rocksdb::Status::OK();
index ffcddf747c2aa7073250cd51e8b2a9f91d1848e3..5cf832b684f3c5eb506bf3dd4d051ff495c15752 100644 (file)
@@ -494,14 +494,14 @@ static void get_wal_key(uint64_t seq, string *out)
 // merge operators
 
 struct Int64ArrayMergeOperator : public KeyValueDB::MergeOperator {
-  virtual void merge_nonexistent(
+  void merge_nonexistent(
     const char *rdata, size_t rlen, std::string *new_value) override {
     *new_value = std::string(rdata, rlen);
   }
-  virtual void merge(
+  void merge(
     const char *ldata, size_t llen,
     const char *rdata, size_t rlen,
-    std::string *new_value) {
+    std::string *new_value) override {
     assert(llen == rlen);
     assert((rlen % 8) == 0);
     new_value->resize(rlen);
@@ -514,7 +514,7 @@ struct Int64ArrayMergeOperator : public KeyValueDB::MergeOperator {
   }
   // We use each operator name and each prefix to construct the
   // overall RocksDB operator name for consistency check at open time.
-  virtual string name() const {
+  string name() const override {
     return "int64_array";
   }
 };
index 21cc1823d9910796081eeb8644c65d3ce2a89114..6b23f37ec9e84d670ad2df0f4b681734eb6f145d 100644 (file)
@@ -2049,7 +2049,7 @@ struct C_JournaledAhead : public Context {
 
   C_JournaledAhead(FileStore *f, FileStore::OpSequencer *os, FileStore::Op *o, Context *ondisk):
     fs(f), osr(os), o(o), ondisk(ondisk) { }
-  void finish(int r) {
+  void finish(int r) override {
     fs->_journaled_ahead(osr, o, ondisk);
   }
 };
@@ -3807,7 +3807,7 @@ public:
   {
   }
 
-  void finish(int r) {
+  void finish(int r) override {
     BackTrace *bt = new BackTrace(1);
     generic_dout(-1) << "FileStore: sync_entry timed out after "
           << m_commit_timeo << " seconds.\n";
index 4ff957527c59e57a2e0135c5264707c7414c0953..707e9ca605316103fbc7cf937932fed5984a6a0a 100644 (file)
@@ -615,39 +615,39 @@ public:
   OmapIteratorImpl(CollectionRef c, ObjectRef o)
     : c(c), o(o), it(o->omap.begin()) {}
 
-  int seek_to_first() {
+  int seek_to_first() override {
     std::lock_guard<std::mutex>(o->omap_mutex);
     it = o->omap.begin();
     return 0;
   }
-  int upper_bound(const string &after) {
+  int upper_bound(const string &after) override {
     std::lock_guard<std::mutex>(o->omap_mutex);
     it = o->omap.upper_bound(after);
     return 0;
   }
-  int lower_bound(const string &to) {
+  int lower_bound(const string &to) override {
     std::lock_guard<std::mutex>(o->omap_mutex);
     it = o->omap.lower_bound(to);
     return 0;
   }
-  bool valid() {
+  bool valid() override {
     std::lock_guard<std::mutex>(o->omap_mutex);
     return it != o->omap.end();
   }
-  int next(bool validate=true) {
+  int next(bool validate=true) override {
     std::lock_guard<std::mutex>(o->omap_mutex);
     ++it;
     return 0;
   }
-  string key() {
+  string key() override {
     std::lock_guard<std::mutex>(o->omap_mutex);
     return it->first;
   }
-  bufferlist value() {
+  bufferlist value() override {
     std::lock_guard<std::mutex>(o->omap_mutex);
     return it->second;
   }
-  int status() {
+  int status() override {
     return 0;
   }
 };
index ddf238c83e02342c1dd11e8e9e11f0b689824c16..3fe4234ab6ed44e092fbd1d41b21a7738fde2a40 100644 (file)
@@ -234,7 +234,7 @@ struct OnRecoveryReadComplete :
   set<int> want;
   OnRecoveryReadComplete(ECBackend *pg, const hobject_t &hoid)
     : pg(pg), hoid(hoid) {}
-  void finish(pair<RecoveryMessages *, ECBackend::read_result_t &> &in) {
+  void finish(pair<RecoveryMessages *, ECBackend::read_result_t &> &in) override {
     ECBackend::read_result_t &res = in.second;
     if (!(res.r == 0 && res.errors.empty())) {
         pg->_failed_push(hoid, in);
@@ -450,7 +450,7 @@ struct SendPushReplies : public Context {
     map<int, MOSDPGPushReply*> &in) : l(l), epoch(epoch) {
     replies.swap(in);
   }
-  void finish(int) {
+  void finish(int) override {
     for (map<int, MOSDPGPushReply*>::iterator i = replies.begin();
         i != replies.end();
         ++i) {
@@ -799,7 +799,7 @@ struct SubWriteCommitted : public Context {
     eversion_t last_complete)
     : pg(pg), msg(msg), tid(tid),
       version(version), last_complete(last_complete) {}
-  void finish(int) {
+  void finish(int) override {
     if (msg)
       msg->mark_event("sub_op_committed");
     pg->sub_write_committed(tid, version, last_complete);
@@ -842,7 +842,7 @@ struct SubWriteApplied : public Context {
     ceph_tid_t tid,
     eversion_t version)
     : pg(pg), msg(msg), tid(tid), version(version) {}
-  void finish(int) {
+  void finish(int) override {
     if (msg)
       msg->mark_event("sub_op_applied");
     pg->sub_write_applied(tid, version);
@@ -1227,7 +1227,7 @@ struct FinishReadOp : public GenContext<ThreadPool::TPHandle&>  {
   ECBackend *ec;
   ceph_tid_t tid;
   FinishReadOp(ECBackend *ec, ceph_tid_t tid) : ec(ec), tid(tid) {}
-  void finish(ThreadPool::TPHandle &handle) {
+  void finish(ThreadPool::TPHandle &handle) override {
     auto ropiter = ec->tid_to_read_map.find(tid);
     assert(ropiter != ec->tid_to_read_map.end());
     int priority = ropiter->second.priority;
@@ -2138,7 +2138,7 @@ struct CallClientContexts :
     ECBackend::ClientAsyncReadStatus *status,
     const list<boost::tuple<uint64_t, uint64_t, uint32_t> > &to_read)
     : hoid(hoid), ec(ec), status(status), to_read(to_read) {}
-  void finish(pair<RecoveryMessages *, ECBackend::read_result_t &> &in) {
+  void finish(pair<RecoveryMessages *, ECBackend::read_result_t &> &in) override {
     ECBackend::read_result_t &res = in.second;
     extent_map result;
     if (res.r != 0)
index 29fb6f46593e521d5ca06ec91d2b9986fd8fda8a..600d17373561bc7688408deff53ac9e722d0aaf6 100644 (file)
@@ -529,7 +529,7 @@ class AgentTimeoutCB : public Context {
   PGRef pg;
 public:
   explicit AgentTimeoutCB(PGRef _pg) : pg(_pg) {}
-  void finish(int) {
+  void finish(int) override {
     pg->agent_choose_mode_restart();
   }
 };
@@ -1784,7 +1784,7 @@ class OSDSocketHook : public AdminSocketHook {
 public:
   explicit OSDSocketHook(OSD *o) : osd(o) {}
   bool call(std::string command, cmdmap_t& cmdmap, std::string format,
-           bufferlist& out) {
+           bufferlist& out) override {
     stringstream ss;
     bool r = osd->asok_command(command, cmdmap, format, ss);
     out.append(ss);
@@ -4989,7 +4989,7 @@ struct C_OSD_GetVersion : public Context {
   OSD *osd;
   uint64_t oldest, newest;
   explicit C_OSD_GetVersion(OSD *o) : osd(o), oldest(0), newest(0) {}
-  void finish(int r) {
+  void finish(int r) override {
     if (r >= 0)
       osd->_got_mon_epochs(oldest, newest);
   }
@@ -6797,7 +6797,7 @@ struct C_OnMapCommit : public Context {
   MOSDMap *msg;
   C_OnMapCommit(OSD *o, epoch_t f, epoch_t l, MOSDMap *m)
     : osd(o), first(f), last(l), msg(m) {}
-  void finish(int r) {
+  void finish(int r) override {
     osd->_committed_osd_maps(first, last, msg);
   }
 };
@@ -6810,7 +6810,7 @@ struct C_OnMapApply : public Context {
               const list<OSDMapRef> &pinned_maps,
               epoch_t e)
     : service(service), pinned_maps(pinned_maps), e(e) {}
-  void finish(int r) {
+  void finish(int r) override {
     service->clear_map_bl_cache_pins(e);
   }
 };
@@ -7905,7 +7905,7 @@ struct C_OpenPGs : public Context {
   C_OpenPGs(set<PGRef>& p, ObjectStore *s, OSD* o) : store(s), osd(o) {
     pgs.swap(p);
   }
-  void finish(int r) {
+  void finish(int r) override {
     RWLock::RLocker l(osd->pg_map_lock);
     for (auto p : pgs) {
       if (osd->pg_map.count(p->info.pgid)) {
@@ -8703,7 +8703,7 @@ public:
     osd(osd), name(n), con(con), osdmap(osdmap), map_epoch(map_epoch) {
   }
 
-  void finish(ThreadPool::TPHandle& tp) {
+  void finish(ThreadPool::TPHandle& tp) override {
     Session *session = static_cast<Session *>(
         con->get_priv());
     epoch_t last_sent_epoch;
@@ -9172,7 +9172,7 @@ struct C_CompleteSplits : public Context {
   set<boost::intrusive_ptr<PG> > pgs;
   C_CompleteSplits(OSD *osd, const set<boost::intrusive_ptr<PG> > &in)
     : osd(osd), pgs(in) {}
-  void finish(int r) {
+  void finish(int r) override {
     Mutex::Locker l(osd->osd_lock);
     if (osd->is_stopping())
       return;
index 9dcfee1dd01f95dcdaaf82cb6643f0d608744ae1..f43f1eea5743d82f041667f9f350aadd9902c3c4 100644 (file)
@@ -2530,7 +2530,7 @@ public:
   }
 
 protected:
-  virtual void dump_item(const CrushTreeDumper::Item &qi, TextTable *tbl) {
+  void dump_item(const CrushTreeDumper::Item &qi, TextTable *tbl) override {
 
     *tbl << qi.id
         << weightf_t(qi.weight);
@@ -2583,7 +2583,7 @@ public:
   }
 
 protected:
-  virtual void dump_item_fields(const CrushTreeDumper::Item &qi, Formatter *f) {
+  void dump_item_fields(const CrushTreeDumper::Item &qi, Formatter *f) override {
     Parent::dump_item_fields(qi, f);
     if (!qi.is_bucket())
     {
index e0000c30abd10c890c15889ed2bc65fd881ffba3..1fa6da0785876f9183de6dd55ee1936ff384e457 100644 (file)
@@ -1538,7 +1538,7 @@ struct C_PG_ActivateCommitted : public Context {
   epoch_t activation_epoch;
   C_PG_ActivateCommitted(PG *p, epoch_t e, epoch_t ae)
     : pg(p), epoch(e), activation_epoch(ae) {}
-  void finish(int r) {
+  void finish(int r) override {
     pg->_activate_committed(epoch, activation_epoch);
   }
 };
@@ -2053,7 +2053,7 @@ unsigned PG::get_scrub_priority()
 struct C_PG_FinishRecovery : public Context {
   PGRef pg;
   explicit C_PG_FinishRecovery(PG *p) : pg(p) {}
-  void finish(int r) {
+  void finish(int r) override {
     pg->_finish_recovery(this);
   }
 };
index 024657d8e4366d0e0fa3a20b1928400b4f82fe93..edf6f1a16a823bdc90b9d17520378cb97e909a68 100644 (file)
@@ -110,7 +110,7 @@ struct Trimmer : public ObjectModDesc::Visitor {
     PGBackend *pg,
     ObjectStore::Transaction *t)
     : soid(soid), pg(pg), t(t) {}
-  void rmobject(version_t old_version) {
+  void rmobject(version_t old_version) override {
     pg->trim_rollback_object(
       soid,
       old_version,
index 35a8c1997424da343272e31e715ba3f42c6bd5e9..93ae393eb5f0d99e9034fdb88c591f034b665774 100644 (file)
@@ -118,11 +118,11 @@ protected:
    * results.get<1>() is a pointer to a CopyResults object, which you are
    * responsible for deleting.
    */
-  virtual void finish(CopyCallbackResults results_) = 0;
+  void finish(CopyCallbackResults results_) override = 0;
 
 public:
   /// Provide the final size of the copied object to the CopyCallback
-  virtual ~CopyCallback() {}
+  ~CopyCallback() {}
 };
 
 template <typename T>
@@ -133,7 +133,7 @@ class PrimaryLogPG::BlessedGenContext : public GenContext<T> {
 public:
   BlessedGenContext(PrimaryLogPG *pg, GenContext<T> *c, epoch_t e)
     : pg(pg), c(c), e(e) {}
-  void finish(T t) {
+  void finish(T t) override {
     pg->lock();
     if (pg->pg_has_reset_since(e))
       c.reset();
@@ -156,7 +156,7 @@ class PrimaryLogPG::BlessedContext : public Context {
 public:
   BlessedContext(PrimaryLogPG *pg, Context *c, epoch_t e)
     : pg(pg), c(c), e(e) {}
-  void finish(int r) {
+  void finish(int r) override {
     pg->lock();
     if (pg->pg_has_reset_since(e))
       c.reset();
@@ -177,7 +177,7 @@ class PrimaryLogPG::C_PG_ObjectContext : public Context {
   public:
   C_PG_ObjectContext(PrimaryLogPG *p, ObjectContext *o) :
     pg(p), obc(o) {}
-  void finish(int r) {
+  void finish(int r) override {
     pg->object_context_destructor_callback(obc);
   }
 };
@@ -189,7 +189,7 @@ class PrimaryLogPG::C_OSD_OndiskWriteUnlock : public Context {
     ObjectContextRef o,
     ObjectContextRef o2 = ObjectContextRef(),
     ObjectContextRef o3 = ObjectContextRef()) : obc(o), obc2(o2), obc3(o3) {}
-  void finish(int r) {
+  void finish(int r) override {
     obc->ondisk_write_unlock();
     if (obc2)
       obc2->ondisk_write_unlock();
@@ -204,7 +204,7 @@ struct OnReadComplete : public Context {
   OnReadComplete(
     PrimaryLogPG *pg,
     PrimaryLogPG::OpContext *ctx) : pg(pg), opcontext(ctx) {}
-  void finish(int r) {
+  void finish(int r) override {
     if (r < 0)
       opcontext->async_read_result = r;
     opcontext->finish_read(pg);
@@ -218,7 +218,7 @@ class PrimaryLogPG::C_OSD_AppliedRecoveredObject : public Context {
   public:
   C_OSD_AppliedRecoveredObject(PrimaryLogPG *p, ObjectContextRef o) :
     pg(p), obc(o) {}
-  void finish(int r) {
+  void finish(int r) override {
     pg->_applied_recovered_object(obc);
   }
 };
@@ -232,7 +232,7 @@ class PrimaryLogPG::C_OSD_CommittedPushedObject : public Context {
     PrimaryLogPG *p, epoch_t epoch, eversion_t lc) :
     pg(p), epoch(epoch), last_complete(lc) {
   }
-  void finish(int r) {
+  void finish(int r) override {
     pg->_committed_pushed_object(epoch, last_complete);
   }
 };
@@ -242,7 +242,7 @@ class PrimaryLogPG::C_OSD_AppliedRecoveredObjectReplica : public Context {
   public:
   explicit C_OSD_AppliedRecoveredObjectReplica(PrimaryLogPG *p) :
     pg(p) {}
-  void finish(int r) {
+  void finish(int r) override {
     pg->_applied_recovered_object_replica();
   }
 };
@@ -282,7 +282,7 @@ public:
       ctx(ctx_) {}
   ~CopyFromCallback() {}
 
-  virtual void finish(PrimaryLogPG::CopyCallbackResults results_) {
+  void finish(PrimaryLogPG::CopyCallbackResults results_) override {
     results = results_.get<1>();
     int r = results_.get<0>();
     retval = r;
@@ -680,7 +680,7 @@ void PrimaryLogPG::wait_for_blocked_object(const hobject_t& soid, OpRequestRef o
 class PGLSPlainFilter : public PGLSFilter {
   string val;
 public:
-  virtual int init(bufferlist::iterator &params)
+  int init(bufferlist::iterator &params) override
   {
     try {
       ::decode(xattr, params);
@@ -691,9 +691,9 @@ public:
 
     return 0;
   }
-  virtual ~PGLSPlainFilter() {}
-  virtual bool filter(const hobject_t &obj, bufferlist& xattr_data,
-                      bufferlist& outdata);
+  ~PGLSPlainFilter() {}
+  bool filter(const hobject_t &obj, bufferlist& xattr_data,
+                      bufferlist& outdata) override;
 };
 
 class PGLSParentFilter : public PGLSFilter {
@@ -703,7 +703,7 @@ public:
   PGLSParentFilter(CephContext* cct) : cct(cct) {
     xattr = "_parent";
   }
-  virtual int init(bufferlist::iterator &params)
+  int init(bufferlist::iterator &params) override
   {
     try {
       ::decode(parent_ino, params);
@@ -714,9 +714,9 @@ public:
 
     return 0;
   }
-  virtual ~PGLSParentFilter() {}
-  virtual bool filter(const hobject_t &obj, bufferlist& xattr_data,
-                      bufferlist& outdata);
+  ~PGLSParentFilter() {}
+  bool filter(const hobject_t &obj, bufferlist& xattr_data,
+                      bufferlist& outdata) override;
 };
 
 bool PGLSParentFilter::filter(const hobject_t &obj,
@@ -2623,7 +2623,7 @@ struct C_ProxyRead : public Context {
     : pg(p), oid(o), last_peering_reset(lpr),
       tid(0), prdop(prd), start(ceph_clock_now())
   {}
-  void finish(int r) {
+  void finish(int r) override {
     if (prdop->canceled)
       return;
     pg->lock();
@@ -2818,7 +2818,7 @@ struct C_ProxyWrite_Commit : public Context {
     : pg(p), oid(o), last_peering_reset(lpr),
       tid(0), pwop(pw)
   {}
-  void finish(int r) {
+  void finish(int r) override {
     if (pwop->canceled)
       return;
     pg->lock();
@@ -2966,7 +2966,7 @@ public:
       pg(pg_),
       start(ceph_clock_now()) {}
 
-  virtual void finish(PrimaryLogPG::CopyCallbackResults results) {
+  void finish(PrimaryLogPG::CopyCallbackResults results) override {
     PrimaryLogPG::CopyResults *results_data = results.get<1>();
     int r = results.get<0>();
     pg->finish_promote(r, results_data, obc);
@@ -4165,7 +4165,7 @@ struct FillInVerifyExtent : public Context {
                     OSDService *osd, hobject_t soid, __le32 flags) :
     r(r), rval(rv), outdatap(blp), maybe_crc(mc),
     size(size), osd(osd), soid(soid), flags(flags) {}
-  void finish(int len) {
+  void finish(int len) override {
     *rval = len;
     *r = len;
     if (len < 0)
@@ -4192,7 +4192,7 @@ struct ToSparseReadResult : public Context {
   ceph_le64& len;
   ToSparseReadResult(bufferlist& bl, uint64_t offset, ceph_le64& len):
     data_bl(bl), data_offset(offset),len(len) {}
-  void finish(int r) {
+  void finish(int r) override {
     if (r < 0) return;
     len = r;
     bufferlist outdata;
@@ -6965,7 +6965,7 @@ struct C_Copyfrom : public Context {
     : pg(p), oid(o), last_peering_reset(lpr),
       tid(0), cop(c)
   {}
-  void finish(int r) {
+  void finish(int r) override {
     if (r == -ECANCELED)
       return;
     pg->lock();
@@ -6984,7 +6984,7 @@ struct C_CopyFrom_AsyncReadCb : public Context {
   size_t len;
   C_CopyFrom_AsyncReadCb(OSDOp *osd_op, uint64_t features, bool classic) :
     osd_op(osd_op), features(features), classic(classic), len(0) {}
-  void finish(int r) {
+  void finish(int r) override {
     assert(len > 0);
     assert(len <= reply_obj.data.length());
     bufferlist bl;
@@ -7891,7 +7891,7 @@ struct C_Flush : public Context {
     : pg(p), oid(o), last_peering_reset(lpr),
       tid(0), start(ceph_clock_now())
   {}
-  void finish(int r) {
+  void finish(int r) override {
     if (r == -ECANCELED)
       return;
     pg->lock();
@@ -8323,7 +8323,7 @@ class C_OSD_RepopApplied : public Context {
 public:
   C_OSD_RepopApplied(PrimaryLogPG *pg, PrimaryLogPG::RepGather *repop)
   : pg(pg), repop(repop) {}
-  void finish(int) {
+  void finish(int) override {
     pg->repop_all_applied(repop.get());
   }
 };
@@ -8346,7 +8346,7 @@ class C_OSD_RepopCommit : public Context {
 public:
   C_OSD_RepopCommit(PrimaryLogPG *pg, PrimaryLogPG::RepGather *repop)
     : pg(pg), repop(repop) {}
-  void finish(int) {
+  void finish(int) override {
     pg->repop_all_committed(repop.get());
   }
 };
index 51b50e10b6646a04c9b40f6fb609632dd124f8d0..166c857cb3b307cf6be400e8cd258dd8e636c3bd 100644 (file)
@@ -42,7 +42,7 @@ class PG_SendMessageOnConn: public Context {
     PGBackend::Listener *pg,
     Message *reply,
     ConnectionRef conn) : pg(pg), reply(reply), conn(conn) {}
-  void finish(int) {
+  void finish(int) override {
     pg->send_message_osd_cluster(reply, conn.get());
   }
 };
@@ -54,7 +54,7 @@ class PG_RecoveryQueueAsync : public Context {
   PG_RecoveryQueueAsync(
     PGBackend::Listener *pg,
     GenContext<ThreadPool::TPHandle&> *c) : pg(pg), c(c) {}
-  void finish(int) {
+  void finish(int) override {
     pg->schedule_recovery_work(c.release());
   }
 };
@@ -65,7 +65,7 @@ struct ReplicatedBackend::C_OSD_RepModifyApply : public Context {
   RepModifyRef rm;
   C_OSD_RepModifyApply(ReplicatedBackend *pg, RepModifyRef r)
     : pg(pg), rm(r) {}
-  void finish(int r) {
+  void finish(int r) override {
     pg->sub_op_modify_applied(rm);
   }
 };
@@ -75,7 +75,7 @@ struct ReplicatedBackend::C_OSD_RepModifyCommit : public Context {
   RepModifyRef rm;
   C_OSD_RepModifyCommit(ReplicatedBackend *pg, RepModifyRef r)
     : pg(pg), rm(r) {}
-  void finish(int r) {
+  void finish(int r) override {
     pg->sub_op_modify_commit(rm);
   }
 };
@@ -320,7 +320,7 @@ struct AsyncReadCallback : public GenContext<ThreadPool::TPHandle&> {
   int r;
   Context *c;
   AsyncReadCallback(int r, Context *c) : r(r), c(c) {}
-  void finish(ThreadPool::TPHandle&) {
+  void finish(ThreadPool::TPHandle&) override {
     c->complete(r);
     c = NULL;
   }
@@ -366,7 +366,7 @@ class C_OSD_OnOpCommit : public Context {
 public:
   C_OSD_OnOpCommit(ReplicatedBackend *pg, ReplicatedBackend::InProgressOp *op) 
     : pg(pg), op(op) {}
-  void finish(int) {
+  void finish(int) override {
     pg->op_commit(op);
   }
 };
@@ -377,7 +377,7 @@ class C_OSD_OnOpApplied : public Context {
 public:
   C_OSD_OnOpApplied(ReplicatedBackend *pg, ReplicatedBackend::InProgressOp *op) 
     : pg(pg), op(op) {}
-  void finish(int) {
+  void finish(int) override {
     pg->op_applied(op);
   }
 };
@@ -877,7 +877,7 @@ struct C_ReplicatedBackend_OnPullComplete : GenContext<ThreadPool::TPHandle&> {
   C_ReplicatedBackend_OnPullComplete(ReplicatedBackend *bc, int priority)
     : bc(bc), priority(priority) {}
 
-  void finish(ThreadPool::TPHandle &handle) {
+  void finish(ThreadPool::TPHandle &handle) override {
     ReplicatedBackend::RPGHandle *h = bc->_open_recovery_op();
     for (auto &&i: to_continue) {
       if (!bc->start_pushes(i.hoid, i.obc, h)) {
index 284a2447f4c90674a27e7bc4b8c84e3e480e6429..df92bb7712c451e9718877722309fd0b631fbada 100644 (file)
@@ -74,7 +74,7 @@ class NotifyTimeoutCB : public CancelableContext {
   bool canceled; // protected by notif lock
 public:
   explicit NotifyTimeoutCB(NotifyRef notif) : notif(notif), canceled(false) {}
-  void finish(int) {
+  void finish(int) override {
     notif->osd->watch_lock.Unlock();
     notif->lock.Lock();
     if (!canceled)
@@ -83,7 +83,7 @@ public:
       notif->lock.Unlock();
     notif->osd->watch_lock.Lock();
   }
-  void cancel() {
+  void cancel() override {
     assert(notif->lock.is_locked_by_me());
     canceled = true;
   }
@@ -237,11 +237,11 @@ class HandleWatchTimeout : public CancelableContext {
 public:
   bool canceled; // protected by watch->pg->lock
   explicit HandleWatchTimeout(WatchRef watch) : watch(watch), canceled(false) {}
-  void cancel() {
+  void cancel() override {
     canceled = true;
   }
-  void finish(int) { ceph_abort(); /* not used */ }
-  void complete(int) {
+  void finish(int) override { ceph_abort(); /* not used */ }
+  void complete(int) override {
     OSDService *osd(watch->osd);
     ldout(osd->cct, 10) << "HandleWatchTimeout" << dendl;
     boost::intrusive_ptr<PrimaryLogPG> pg(watch->pg);
@@ -261,10 +261,10 @@ class HandleDelayedWatchTimeout : public CancelableContext {
 public:
   bool canceled;
   explicit HandleDelayedWatchTimeout(WatchRef watch) : watch(watch), canceled(false) {}
-  void cancel() {
+  void cancel() override {
     canceled = true;
   }
-  void finish(int) {
+  void finish(int) override {
     OSDService *osd(watch->osd);
     dout(10) << "HandleWatchTimeoutDelayed" << dendl;
     assert(watch->pg->is_locked());
index 5d2884b50892c8fb60adfde8c553703571f2d15d..592593e43521da875ddafc146b8a978381dd2aa7 100644 (file)
@@ -43,7 +43,7 @@ public:
   ceph::real_time mtime;
   C_Probe(Filer *f, Probe *p, object_t o) : filer(f), probe(p), oid(o),
                                            size(0) {}
-  void finish(int r) {
+  void finish(int r) override {
     if (r == -ENOENT) {
       r = 0;
       assert(size == 0);
@@ -338,7 +338,7 @@ struct C_PurgeRange : public Context {
   Filer *filer;
   PurgeRange *pr;
   C_PurgeRange(Filer *f, PurgeRange *p) : filer(f), pr(p) {}
-  void finish(int r) {
+  void finish(int r) override {
     filer->_do_purge_range(pr, 1);
   }
 };
index 8d2b33ac684ebb2c925ded8c850ddf2c57f738cc..61e4d996353dce590a61237264839a94d5993e08 100644 (file)
@@ -33,7 +33,7 @@ class Journaler::C_DelayFlush : public Context {
   Journaler *journaler;
   public:
   C_DelayFlush(Journaler *j) : journaler(j) {}
-  void finish(int r) {
+  void finish(int r) override {
     journaler->_do_delayed_flush();
   }
 };
@@ -112,7 +112,7 @@ class Journaler::C_ReadHead : public Context {
 public:
   bufferlist bl;
   explicit C_ReadHead(Journaler *l) : ls(l) {}
-  void finish(int r) {
+  void finish(int r) override {
     ls->_finish_read_head(r, bl);
   }
 };
@@ -124,7 +124,7 @@ public:
   bufferlist bl;
   C_RereadHead(Journaler *l, Context *onfinish_) : ls (l),
                                                   onfinish(onfinish_) {}
-  void finish(int r) {
+  void finish(int r) override {
     ls->_finish_reread_head(r, bl, onfinish);
   }
 };
@@ -134,7 +134,7 @@ class Journaler::C_ProbeEnd : public Context {
 public:
   uint64_t end;
   explicit C_ProbeEnd(Journaler *l) : ls(l), end(-1) {}
-  void finish(int r) {
+  void finish(int r) override {
     ls->_finish_probe_end(r, end);
   }
 };
@@ -146,7 +146,7 @@ public:
   uint64_t end;
   C_ReProbe(Journaler *l, C_OnFinisher *onfinish_) :
     ls(l), onfinish(onfinish_), end(0) {}
-  void finish(int r) {
+  void finish(int r) override {
     ls->_finish_reprobe(r, end, onfinish);
   }
 };
@@ -380,7 +380,7 @@ class Journaler::C_RereadHeadProbe : public Context
 public:
   C_RereadHeadProbe(Journaler *l, C_OnFinisher *finish) :
     ls(l), final_finish(finish) {}
-  void finish(int r) {
+  void finish(int r) override {
     ls->_finish_reread_head_and_probe(r, final_finish);
   }
 };
@@ -412,7 +412,7 @@ public:
   C_OnFinisher *oncommit;
   C_WriteHead(Journaler *l, Header& h_, C_OnFinisher *c) : ls(l), h(h_),
                                                           oncommit(c) {}
-  void finish(int r) {
+  void finish(int r) override {
     ls->_finish_write_head(r, h, oncommit);
   }
 };
@@ -484,7 +484,7 @@ class Journaler::C_Flush : public Context {
 public:
   C_Flush(Journaler *l, int64_t s, ceph::real_time st)
     : ls(l), start(s), stamp(st) {}
-  void finish(int r) {
+  void finish(int r) override {
     ls->_finish_flush(r, start, stamp);
   }
 };
@@ -749,7 +749,7 @@ struct C_Journaler_Prezero : public Context {
   uint64_t from, len;
   C_Journaler_Prezero(Journaler *j, uint64_t f, uint64_t l)
     : journaler(j), from(f), len(l) {}
-  void finish(int r) {
+  void finish(int r) override {
     journaler->_finish_prezero(r, from, len);
   }
 };
@@ -848,7 +848,7 @@ class Journaler::C_Read : public Context {
 public:
   bufferlist bl;
   C_Read(Journaler *j, uint64_t o, uint64_t l) : ls(j), offset(o), length(l) {}
-  void finish(int r) {
+  void finish(int r) override {
     ls->_finish_read(r, offset, length, bl);
   }
 };
@@ -858,7 +858,7 @@ class Journaler::C_RetryRead : public Context {
 public:
   explicit C_RetryRead(Journaler *l) : ls(l) {}
 
-  void finish(int r) {
+  void finish(int r) override {
     // Should only be called from waitfor_safe i.e. already inside lock
     // (ls->lock is locked
     ls->_prefetch();
@@ -1109,7 +1109,7 @@ class Journaler::C_EraseFinish : public Context {
   C_OnFinisher *completion;
   public:
   C_EraseFinish(Journaler *j, C_OnFinisher *c) : journaler(j), completion(c) {}
-  void finish(int r) {
+  void finish(int r) override {
     journaler->_finish_erase(r, completion);
   }
 };
@@ -1228,7 +1228,7 @@ class Journaler::C_Trim : public Context {
   uint64_t to;
 public:
   C_Trim(Journaler *l, int64_t t) : ls(l), to(t) {}
-  void finish(int r) {
+  void finish(int r) override {
     ls->_finish_trim(r, to);
   }
 };
index 1c670419557639dd6a62a519b76a7fee2b4ff930..9a0b6ee285192ba4fe4ca0780f0e729db47725dc 100644 (file)
@@ -47,7 +47,7 @@ public:
     ob->reads.push_back(&set_item);
   }
 
-  void finish(int r) {
+  void finish(int r) override {
     oc->bh_read_finish(poolid, oid, tid, start, length, bl, r, trust_enoent);
 
     // object destructor clears the list
@@ -68,7 +68,7 @@ class ObjectCacher::C_RetryRead : public Context {
 public:
   C_RetryRead(ObjectCacher *_oc, OSDRead *r, ObjectSet *os, Context *c)
     : oc(_oc), rd(r), oset(os), onfinish(c) {}
-  void finish(int r) {
+  void finish(int r) override {
     if (r < 0) {
       if (onfinish)
         onfinish->complete(r);
@@ -991,7 +991,7 @@ public:
     oc(c), poolid(_poolid), oid(o), tid(0) {
       ranges.swap(_ranges);
     }
-  void finish(int r) {
+  void finish(int r) override {
     oc->bh_write_commit(poolid, oid, ranges, tid, r);
   }
 };
@@ -1708,7 +1708,7 @@ class ObjectCacher::C_WaitForWrite : public Context {
 public:
   C_WaitForWrite(ObjectCacher *oc, uint64_t len, Context *onfinish) :
     m_oc(oc), m_len(len), m_onfinish(onfinish) {}
-  void finish(int r);
+  void finish(int r) override;
 private:
   ObjectCacher *m_oc;
   uint64_t m_len;
index 52da00c1702914ae174a2f98612e24cf6b16977b..52b04b5aa9483208b6e37094fe764922f25d9633 100644 (file)
@@ -159,7 +159,7 @@ class Objecter::RequestStateHook : public AdminSocketHook {
 public:
   explicit RequestStateHook(Objecter *objecter);
   bool call(std::string command, cmdmap_t& cmdmap, std::string format,
-            bufferlist& out);
+            bufferlist& out) override;
 };
 
 /**
@@ -172,14 +172,14 @@ class ObjectOperation::C_TwoContexts : public Context {
 public:
   C_TwoContexts(Context *first, Context *second) :
     first(first), second(second) {}
-  void finish(int r) {
+  void finish(int r) override {
     first->complete(r);
     second->complete(r);
     first = NULL;
     second = NULL;
   }
 
-  virtual ~C_TwoContexts() {
+  ~C_TwoContexts() {
     delete first;
     delete second;
   }
@@ -621,7 +621,7 @@ struct C_DoWatchError : public Context {
     info->get();
     info->_queued_async();
   }
-  void finish(int r) {
+  void finish(int r) override {
     Objecter::unique_lock wl(objecter->rwlock);
     bool canceled = info->canceled;
     wl.unlock();
@@ -880,7 +880,7 @@ struct C_DoWatchNotify : public Context {
     info->_queued_async();
     msg->get();
   }
-  void finish(int r) {
+  void finish(int r) override {
     objecter->_do_watch_notify(info, msg);
   }
 };
@@ -1830,7 +1830,7 @@ struct C_Objecter_GetVersion : public Context {
   Context *fin;
   C_Objecter_GetVersion(Objecter *o, Context *c)
     : objecter(o), oldest(0), newest(0), fin(c) {}
-  void finish(int r) {
+  void finish(int r) override {
     if (r >= 0) {
       objecter->get_latest_version(oldest, newest, fin);
     } else if (r == -EAGAIN) { // try again as instructed
@@ -3820,7 +3820,7 @@ struct C_SelfmanagedSnap : public Context {
   snapid_t *psnapid;
   Context *fin;
   C_SelfmanagedSnap(snapid_t *ps, Context *f) : psnapid(ps), fin(f) {}
-  void finish(int r) {
+  void finish(int r) override {
     if (r == 0) {
       bufferlist::iterator p = bl.begin();
       ::decode(*psnapid, p);
@@ -4995,7 +4995,7 @@ struct C_EnumerateReply : public Context {
     epoch(0), budget(0)
   {}
 
-  void finish(int r) {
+  void finish(int r) override {
     objecter->_enumerate_reply(
       bl, r, end, pool_id, budget, epoch, result, next, on_finish);
   }
index c0a81c483c3d254a03aaf7041f1b87a6fc5fa556..de0e3a7d020cf135482dac3378e9e5ea8770a48e 100644 (file)
@@ -14,7 +14,7 @@ public:
            list<pair<pair<coll_t, string>, uint64_t> > in_flight)
     : tracker(tracker), in_flight(in_flight) {}
 
-  void finish(int r) {
+  void finish(int r) override {
     for (list<pair<pair<coll_t, string>, uint64_t> >::iterator i =
           in_flight.begin();
         i != in_flight.end();
@@ -32,7 +32,7 @@ public:
              list<pair<pair<coll_t, string>, uint64_t> > in_flight)
     : tracker(tracker), in_flight(in_flight) {}
 
-  void finish(int r) {
+  void finish(int r) override {
     for (list<pair<pair<coll_t, string>, uint64_t> >::iterator i =
           in_flight.begin();
         i != in_flight.end();
index da74c1b701aa406e319c2afcf05d648ba7f5d1da..9cdc176885c264fe5db26d3b925133236d748b5d 100644 (file)
@@ -3150,7 +3150,7 @@ public:
   unsigned seq;
   int64_t poolid;
   explicit MixedGenerator(int64_t p) : seq(0), poolid(p) {}
-  ghobject_t create_object(gen_type *gen) {
+  ghobject_t create_object(gen_type *gen) override {
     char buf[100];
     snprintf(buf, sizeof(buf), "OBJ_%u", seq);
     string name(buf);
@@ -3212,7 +3212,7 @@ public:
     C_SyntheticOnReadable(SyntheticWorkloadState *state, ghobject_t hoid)
       : state(state), hoid(hoid) {}
 
-    void finish(int r) {
+    void finish(int r) override {
       Mutex::Locker locker(state->lock);
       EnterExit ee("onreadable finish");
       ASSERT_TRUE(state->in_flight_objects.count(hoid));
@@ -3239,7 +3239,7 @@ public:
                       ghobject_t oid, ghobject_t noid)
       : state(state), oid(oid), noid(noid) {}
 
-    void finish(int r) {
+    void finish(int r) override {
       Mutex::Locker locker(state->lock);
       EnterExit ee("stash finish");
       ASSERT_TRUE(state->in_flight_objects.count(oid));
@@ -3266,7 +3266,7 @@ public:
                        ghobject_t oid, ghobject_t noid)
       : state(state), oid(oid), noid(noid) {}
 
-    void finish(int r) {
+    void finish(int r) override {
       Mutex::Locker locker(state->lock);
       EnterExit ee("clone finish");
       ASSERT_TRUE(state->in_flight_objects.count(oid));
index b233a32b2f68355e19dc7f4cbdf856813f9d2ec1..87bd62b9a4426201330f85c22b62aa3123452a56 100644 (file)
@@ -54,7 +54,7 @@ public:
     db.reset(NULL);
   }
 
-  virtual void SetUp() {
+  void SetUp() override {
     int r = ::mkdir("kv_test_temp_dir", 0777);
     if (r < 0 && errno != EEXIST) {
       r = -errno;
@@ -64,7 +64,7 @@ public:
     }
     init();
   }
-  virtual void TearDown() {
+  void TearDown() override {
     fini();
     rm_r("kv_test_temp_dir");
   }
@@ -191,20 +191,20 @@ TEST_P(KVTest, BenchCommit) {
 }
 
 struct AppendMOP : public KeyValueDB::MergeOperator {
-  virtual void merge_nonexistent(
+  void merge_nonexistent(
     const char *rdata, size_t rlen, std::string *new_value) override {
     *new_value = "?" + std::string(rdata, rlen);
   }
-  virtual void merge(
+  void merge(
     const char *ldata, size_t llen,
     const char *rdata, size_t rlen,
-    std::string *new_value) {
+    std::string *new_value) override {
 
     *new_value = std::string(ldata, llen) + std::string(rdata, rlen);
   }
   // We use each operator name and each prefix to construct the
   // overall RocksDB operator name for consistency check at open time.
-  virtual string name() const {
+  string name() const override {
     return "Append";
   }
 };
index 367bba2dd945c4d781c2660ad77a03361bc95dc8..fee9d78db4072a4eac05d127ee1e07cc0337fc75 100644 (file)
@@ -35,17 +35,17 @@ public:
                   uint32_t index_version)
     : LFNIndex(cct, collection, base_path, index_version) {}
 
-  virtual uint32_t collection_version() {
+  uint32_t collection_version() override {
     return index_version;
   }
 
-  int cleanup() { return 0; }
+  int cleanup() override { return 0; }
 
-  virtual int _split(
+  int _split(
                     uint32_t match,                           
                     uint32_t bits,                            
                     CollectionIndex* dest
-                    ) { return 0; }
+                    ) override { return 0; }
 
   void test_generate_and_parse(const ghobject_t &hoid, const std::string &mangled_expected) {
     const std::string mangled_name = lfn_generate_object_name(hoid);
@@ -56,39 +56,39 @@ public:
   }
 
 protected:
-  virtual int _init() { return 0; }
+  int _init() override { return 0; }
 
-  virtual int _created(
+  int _created(
                       const vector<string> &path,
                       const ghobject_t &hoid,
                       const string &mangled_name 
-                      ) { return 0; }
+                      ) override { return 0; }
 
-  virtual int _remove(
+  int _remove(
                      const vector<string> &path,
                      const ghobject_t &hoid,
                      const string &mangled_name
-                     ) { return 0; }
+                     ) override { return 0; }
 
-  virtual int _lookup(
+  int _lookup(
                      const ghobject_t &hoid,
                      vector<string> *path,
                      string *mangled_name,
                      int *exists                
-                     ) { return 0; }
+                     ) override { return 0; }
 
-  virtual int _collection_list_partial(
+  int _collection_list_partial(
                                       const ghobject_t &start,
                                       const ghobject_t &end,
                                       bool sort_bitwise,
                                       int max_count,
                                       vector<ghobject_t> *ls,
                                       ghobject_t *next
-                                      ) { return 0; }
-  virtual int _pre_hash_collection(
+                                      ) override { return 0; }
+  int _pre_hash_collection(
                                    uint32_t pg_num,
                                    uint64_t expected_num_objs
-                                  ) { return 0; }
+                                  ) override { return 0; }
 
 };
 
@@ -190,13 +190,13 @@ public:
                       CollectionIndex::HOBJECT_WITH_POOL) {
   }
 
-  virtual void SetUp() {
+  void SetUp() override {
     ::chmod("PATH_1", 0700);
     ASSERT_EQ(0, ::system("rm -fr PATH_1"));
     ASSERT_EQ(0, ::mkdir("PATH_1", 0700));
   }
 
-  virtual void TearDown() {
+  void TearDown() override {
     ASSERT_EQ(0, ::system("rm -fr PATH_1"));
   }
 };
index e00449ca860e0b9049159bcbab7de51e63d16787..ed68ac634abfe010f0a9e0268c4c7ce94c877290 100644 (file)
@@ -28,9 +28,9 @@
 class PGLogTest : public ::testing::Test, protected PGLog {
 public:
   PGLogTest() : PGLog(g_ceph_context) {}
-  virtual void SetUp() { }
+  void SetUp() override { }
 
-  virtual void TearDown() {
+  void TearDown() override {
     clear();
   }
 
@@ -157,20 +157,20 @@ public:
     list<pg_log_entry_t> rolledback;
     
     void rollback(
-      const pg_log_entry_t &entry) {
+      const pg_log_entry_t &entry) override {
       rolledback.push_back(entry);
     }
     void rollforward(
-      const pg_log_entry_t &entry) {}
+      const pg_log_entry_t &entry) override {}
     void remove(
-      const hobject_t &hoid) {
+      const hobject_t &hoid) override {
       removed.insert(hoid);
     }
     void try_stash(const hobject_t &, version_t) override {
       // lost/unfound cases are not tested yet
     }
     void trim(
-      const pg_log_entry_t &entry) {}
+      const pg_log_entry_t &entry) override {}
   };
 
   template <typename missing_t>
@@ -276,11 +276,11 @@ struct TestHandler : public PGLog::LogEntryHandler {
   explicit TestHandler(list<hobject_t> &removed) : removed(removed) {}
 
   void rollback(
-    const pg_log_entry_t &entry) {}
+    const pg_log_entry_t &entry) override {}
   void rollforward(
-    const pg_log_entry_t &entry) {}
+    const pg_log_entry_t &entry) override {}
   void remove(
-    const hobject_t &hoid) {
+    const hobject_t &hoid) override {
     removed.push_back(hoid);
   }
   void cant_rollback(const pg_log_entry_t &entry) {}
@@ -288,7 +288,7 @@ struct TestHandler : public PGLog::LogEntryHandler {
     // lost/unfound cases are not tested yet
   }
   void trim(
-    const pg_log_entry_t &entry) {}
+    const pg_log_entry_t &entry) override {}
 };
 
 TEST_F(PGLogTest, rewind_divergent_log) {
index b09d4f2c02d2bc3eac5b6af506b7a4d7f8a8f4ef..4a1bf8ef374f798932e682b843d1889085459138 100644 (file)
@@ -46,7 +46,7 @@ public:
     }
   }
 
-  TestOp *next(RadosTestContext &context)
+  TestOp *next(RadosTestContext &context) override
   {
     TestOp *retval = NULL;
 
index 419c637177bc3540c747c158d5e8e34db22873ab..17ad3857cfd8100ea6cd79204dbc2a8171c81ab0 100644 (file)
@@ -1093,7 +1093,7 @@ protected:
     {
     }
 
-    virtual void *entry() {
+    void *entry() override {
       obc.ondisk_read_lock();
       return NULL;
     }
@@ -1108,7 +1108,7 @@ protected:
     {
     }
 
-    virtual void *entry() {
+    void *entry() override {
       obc.ondisk_write_lock();
       return NULL;
     }
index 5ba05d35f77e4eb0c733c3ae42efb4ee144dd452..47e778ef021c1c4551efee13ecf4fcd2ec4ef7b0 100644 (file)
@@ -31,7 +31,7 @@ public:
          bufferlist *pbl, uint64_t delay_ns=0)
     : m_cct(cct), m_con(c), m_delay(delay_ns * std::chrono::nanoseconds(1)),
       m_lock(lock), m_bl(pbl), m_off(off) {}
-  void finish(int r) {
+  void finish(int r) override {
     std::this_thread::sleep_for(m_delay);
     if (m_bl) {
       buffer::ptr bp(r);
index ddcf3ef109f1cf36ed26a7acd5424d028e617ef7..801d155608202509435700a19e16b7ce61e1e46f 100644 (file)
@@ -37,7 +37,7 @@ public:
     : wb(mwb), m_cct(cct), m_con(c),
       m_delay(delay_ns * std::chrono::nanoseconds(1)),
       m_lock(lock), m_oid(oid), m_off(off), m_len(len), m_bl(pbl) {}
-  void finish(int r) {
+  void finish(int r) override {
     std::this_thread::sleep_for(m_delay);
     m_lock->Lock();
     r = wb->read_object_data(m_oid, m_off, m_len, m_bl);
@@ -65,7 +65,7 @@ public:
     : wb(mwb), m_cct(cct), m_con(c),
       m_delay(delay_ns * std::chrono::nanoseconds(1)),
       m_lock(lock), m_oid(oid), m_off(off), m_len(len), m_bl(bl) {}
-  void finish(int r) {
+  void finish(int r) override {
     std::this_thread::sleep_for(m_delay);
     m_lock->Lock();
     wb->write_object_data(m_oid, m_off, m_len, m_bl);
index 566c218bc3d3b0e28884ae7f4036ff58791161ad..c23c6c26ed33de26c9ff8c371c06c1b39a4f0562 100644 (file)
@@ -44,7 +44,7 @@ class C_Count : public Context {
 public:
   C_Count(op_data *op, atomic_t *outstanding)
     : m_op(op), m_outstanding(outstanding) {}
-  void finish(int r) {
+  void finish(int r) override {
     m_op->done.inc();
     assert(m_outstanding->read() > 0);
     m_outstanding->dec();